2019-03-06 07:19:04

by Jason Wang

[permalink] [raw]
Subject: [RFC PATCH V2 0/5] vhost: accelerate metadata access through vmap()

This series tries to access virtqueue metadata through kernel virtual
address instead of copy_user() friends since they had too much
overheads like checks, spec barriers or even hardware feature
toggling. This is done through setup kernel address through vmap() and
resigter MMU notifier for invalidation.

Test shows about 24% improvement on TX PPS. TCP_STREAM doesn't see
obvious improvement.

Thanks

Changes from V4:
- use invalidate_range() instead of invalidate_range_start()
- track dirty pages
Changes from V3:
- don't try to use vmap for file backed pages
- rebase to master
Changes from V2:
- fix buggy range overlapping check
- tear down MMU notifier during vhost ioctl to make sure invalidation
request can read metadata userspace address and vq size without
holding vq mutex.
Changes from V1:
- instead of pinning pages, use MMU notifier to invalidate vmaps and
remap duing metadata prefetch
- fix build warning on MIPS

Jason Wang (5):
vhost: generalize adding used elem
vhost: fine grain userspace memory accessors
vhost: rename vq_iotlb_prefetch() to vq_meta_prefetch()
vhost: introduce helpers to get the size of metadata area
vhost: access vq metadata through kernel virtual address

drivers/vhost/net.c | 6 +-
drivers/vhost/vhost.c | 434 ++++++++++++++++++++++++++++++++++++++++++++------
drivers/vhost/vhost.h | 18 ++-
3 files changed, 407 insertions(+), 51 deletions(-)

--
1.8.3.1



2019-03-06 07:19:25

by Jason Wang

[permalink] [raw]
Subject: [RFC PATCH V2 2/5] vhost: fine grain userspace memory accessors

This is used to hide the metadata address from virtqueue helpers. This
will allow to implement a vmap based fast accessing to metadata.

Signed-off-by: Jason Wang <[email protected]>
---
drivers/vhost/vhost.c | 94 +++++++++++++++++++++++++++++++++++++++++----------
1 file changed, 77 insertions(+), 17 deletions(-)

diff --git a/drivers/vhost/vhost.c b/drivers/vhost/vhost.c
index 400aa78..29709e7 100644
--- a/drivers/vhost/vhost.c
+++ b/drivers/vhost/vhost.c
@@ -869,6 +869,34 @@ static inline void __user *__vhost_get_user(struct vhost_virtqueue *vq,
ret; \
})

+static inline int vhost_put_avail_event(struct vhost_virtqueue *vq)
+{
+ return vhost_put_user(vq, cpu_to_vhost16(vq, vq->avail_idx),
+ vhost_avail_event(vq));
+}
+
+static inline int vhost_put_used(struct vhost_virtqueue *vq,
+ struct vring_used_elem *head, int idx,
+ int count)
+{
+ return vhost_copy_to_user(vq, vq->used->ring + idx, head,
+ count * sizeof(*head));
+}
+
+static inline int vhost_put_used_flags(struct vhost_virtqueue *vq)
+
+{
+ return vhost_put_user(vq, cpu_to_vhost16(vq, vq->used_flags),
+ &vq->used->flags);
+}
+
+static inline int vhost_put_used_idx(struct vhost_virtqueue *vq)
+
+{
+ return vhost_put_user(vq, cpu_to_vhost16(vq, vq->last_used_idx),
+ &vq->used->idx);
+}
+
#define vhost_get_user(vq, x, ptr, type) \
({ \
int ret; \
@@ -907,6 +935,43 @@ static void vhost_dev_unlock_vqs(struct vhost_dev *d)
mutex_unlock(&d->vqs[i]->mutex);
}

+static inline int vhost_get_avail_idx(struct vhost_virtqueue *vq,
+ __virtio16 *idx)
+{
+ return vhost_get_avail(vq, *idx, &vq->avail->idx);
+}
+
+static inline int vhost_get_avail_head(struct vhost_virtqueue *vq,
+ __virtio16 *head, int idx)
+{
+ return vhost_get_avail(vq, *head,
+ &vq->avail->ring[idx & (vq->num - 1)]);
+}
+
+static inline int vhost_get_avail_flags(struct vhost_virtqueue *vq,
+ __virtio16 *flags)
+{
+ return vhost_get_avail(vq, *flags, &vq->avail->flags);
+}
+
+static inline int vhost_get_used_event(struct vhost_virtqueue *vq,
+ __virtio16 *event)
+{
+ return vhost_get_avail(vq, *event, vhost_used_event(vq));
+}
+
+static inline int vhost_get_used_idx(struct vhost_virtqueue *vq,
+ __virtio16 *idx)
+{
+ return vhost_get_used(vq, *idx, &vq->used->idx);
+}
+
+static inline int vhost_get_desc(struct vhost_virtqueue *vq,
+ struct vring_desc *desc, int idx)
+{
+ return vhost_copy_from_user(vq, desc, vq->desc + idx, sizeof(*desc));
+}
+
static int vhost_new_umem_range(struct vhost_umem *umem,
u64 start, u64 size, u64 end,
u64 userspace_addr, int perm)
@@ -1840,8 +1905,7 @@ int vhost_log_write(struct vhost_virtqueue *vq, struct vhost_log *log,
static int vhost_update_used_flags(struct vhost_virtqueue *vq)
{
void __user *used;
- if (vhost_put_user(vq, cpu_to_vhost16(vq, vq->used_flags),
- &vq->used->flags) < 0)
+ if (vhost_put_used_flags(vq))
return -EFAULT;
if (unlikely(vq->log_used)) {
/* Make sure the flag is seen before log. */
@@ -1858,8 +1922,7 @@ static int vhost_update_used_flags(struct vhost_virtqueue *vq)

static int vhost_update_avail_event(struct vhost_virtqueue *vq, u16 avail_event)
{
- if (vhost_put_user(vq, cpu_to_vhost16(vq, vq->avail_idx),
- vhost_avail_event(vq)))
+ if (vhost_put_avail_event(vq))
return -EFAULT;
if (unlikely(vq->log_used)) {
void __user *used;
@@ -1895,7 +1958,7 @@ int vhost_vq_init_access(struct vhost_virtqueue *vq)
r = -EFAULT;
goto err;
}
- r = vhost_get_used(vq, last_used_idx, &vq->used->idx);
+ r = vhost_get_used_idx(vq, &last_used_idx);
if (r) {
vq_err(vq, "Can't access used idx at %p\n",
&vq->used->idx);
@@ -2094,7 +2157,7 @@ int vhost_get_vq_desc(struct vhost_virtqueue *vq,
last_avail_idx = vq->last_avail_idx;

if (vq->avail_idx == vq->last_avail_idx) {
- if (unlikely(vhost_get_avail(vq, avail_idx, &vq->avail->idx))) {
+ if (unlikely(vhost_get_avail_idx(vq, &avail_idx))) {
vq_err(vq, "Failed to access avail idx at %p\n",
&vq->avail->idx);
return -EFAULT;
@@ -2121,8 +2184,7 @@ int vhost_get_vq_desc(struct vhost_virtqueue *vq,

/* Grab the next descriptor number they're advertising, and increment
* the index we've seen. */
- if (unlikely(vhost_get_avail(vq, ring_head,
- &vq->avail->ring[last_avail_idx & (vq->num - 1)]))) {
+ if (unlikely(vhost_get_avail_head(vq, &ring_head, last_avail_idx))) {
vq_err(vq, "Failed to read head: idx %d address %p\n",
last_avail_idx,
&vq->avail->ring[last_avail_idx % vq->num]);
@@ -2157,8 +2219,7 @@ int vhost_get_vq_desc(struct vhost_virtqueue *vq,
i, vq->num, head);
return -EINVAL;
}
- ret = vhost_copy_from_user(vq, &desc, vq->desc + i,
- sizeof desc);
+ ret = vhost_get_desc(vq, &desc, i);
if (unlikely(ret)) {
vq_err(vq, "Failed to get descriptor: idx %d addr %p\n",
i, vq->desc + i);
@@ -2251,7 +2312,7 @@ static int __vhost_add_used_n(struct vhost_virtqueue *vq,

start = vq->last_used_idx & (vq->num - 1);
used = vq->used->ring + start;
- if (vhost_copy_to_user(vq, used, heads, count * sizeof *used)) {
+ if (vhost_put_used(vq, heads, start, count)) {
vq_err(vq, "Failed to write used");
return -EFAULT;
}
@@ -2293,8 +2354,7 @@ int vhost_add_used_n(struct vhost_virtqueue *vq, struct vring_used_elem *heads,

/* Make sure buffer is written before we update index. */
smp_wmb();
- if (vhost_put_user(vq, cpu_to_vhost16(vq, vq->last_used_idx),
- &vq->used->idx)) {
+ if (vhost_put_used_idx(vq)) {
vq_err(vq, "Failed to increment used idx");
return -EFAULT;
}
@@ -2327,7 +2387,7 @@ static bool vhost_notify(struct vhost_dev *dev, struct vhost_virtqueue *vq)

if (!vhost_has_feature(vq, VIRTIO_RING_F_EVENT_IDX)) {
__virtio16 flags;
- if (vhost_get_avail(vq, flags, &vq->avail->flags)) {
+ if (vhost_get_avail_flags(vq, &flags)) {
vq_err(vq, "Failed to get flags");
return true;
}
@@ -2341,7 +2401,7 @@ static bool vhost_notify(struct vhost_dev *dev, struct vhost_virtqueue *vq)
if (unlikely(!v))
return true;

- if (vhost_get_avail(vq, event, vhost_used_event(vq))) {
+ if (vhost_get_used_event(vq, &event)) {
vq_err(vq, "Failed to get used event idx");
return true;
}
@@ -2386,7 +2446,7 @@ bool vhost_vq_avail_empty(struct vhost_dev *dev, struct vhost_virtqueue *vq)
if (vq->avail_idx != vq->last_avail_idx)
return false;

- r = vhost_get_avail(vq, avail_idx, &vq->avail->idx);
+ r = vhost_get_avail_idx(vq, &avail_idx);
if (unlikely(r))
return false;
vq->avail_idx = vhost16_to_cpu(vq, avail_idx);
@@ -2422,7 +2482,7 @@ bool vhost_enable_notify(struct vhost_dev *dev, struct vhost_virtqueue *vq)
/* They could have slipped one in as we were doing that: make
* sure it's written, then check again. */
smp_mb();
- r = vhost_get_avail(vq, avail_idx, &vq->avail->idx);
+ r = vhost_get_avail_idx(vq, &avail_idx);
if (r) {
vq_err(vq, "Failed to check avail idx at %p: %d\n",
&vq->avail->idx, r);
--
1.8.3.1


2019-03-06 07:19:30

by Jason Wang

[permalink] [raw]
Subject: [RFC PATCH V2 3/5] vhost: rename vq_iotlb_prefetch() to vq_meta_prefetch()

Rename the function to be more accurate since it actually tries to
prefetch vq metadata address in IOTLB. And this will be used by
following patch to prefetch metadata virtual addresses.

Signed-off-by: Jason Wang <[email protected]>
---
drivers/vhost/net.c | 4 ++--
drivers/vhost/vhost.c | 4 ++--
drivers/vhost/vhost.h | 2 +-
3 files changed, 5 insertions(+), 5 deletions(-)

diff --git a/drivers/vhost/net.c b/drivers/vhost/net.c
index df51a35..bf55f99 100644
--- a/drivers/vhost/net.c
+++ b/drivers/vhost/net.c
@@ -971,7 +971,7 @@ static void handle_tx(struct vhost_net *net)
if (!sock)
goto out;

- if (!vq_iotlb_prefetch(vq))
+ if (!vq_meta_prefetch(vq))
goto out;

vhost_disable_notify(&net->dev, vq);
@@ -1140,7 +1140,7 @@ static void handle_rx(struct vhost_net *net)
if (!sock)
goto out;

- if (!vq_iotlb_prefetch(vq))
+ if (!vq_meta_prefetch(vq))
goto out;

vhost_disable_notify(&net->dev, vq);
diff --git a/drivers/vhost/vhost.c b/drivers/vhost/vhost.c
index 29709e7..2025543 100644
--- a/drivers/vhost/vhost.c
+++ b/drivers/vhost/vhost.c
@@ -1309,7 +1309,7 @@ static bool iotlb_access_ok(struct vhost_virtqueue *vq,
return true;
}

-int vq_iotlb_prefetch(struct vhost_virtqueue *vq)
+int vq_meta_prefetch(struct vhost_virtqueue *vq)
{
size_t s = vhost_has_feature(vq, VIRTIO_RING_F_EVENT_IDX) ? 2 : 0;
unsigned int num = vq->num;
@@ -1328,7 +1328,7 @@ int vq_iotlb_prefetch(struct vhost_virtqueue *vq)
num * sizeof(*vq->used->ring) + s,
VHOST_ADDR_USED);
}
-EXPORT_SYMBOL_GPL(vq_iotlb_prefetch);
+EXPORT_SYMBOL_GPL(vq_meta_prefetch);

/* Can we log writes? */
/* Caller should have device mutex but not vq mutex */
diff --git a/drivers/vhost/vhost.h b/drivers/vhost/vhost.h
index 9490e7d..7a7fc00 100644
--- a/drivers/vhost/vhost.h
+++ b/drivers/vhost/vhost.h
@@ -209,7 +209,7 @@ void vhost_add_used_and_signal_n(struct vhost_dev *, struct vhost_virtqueue *,
int vhost_log_write(struct vhost_virtqueue *vq, struct vhost_log *log,
unsigned int log_num, u64 len,
struct iovec *iov, int count);
-int vq_iotlb_prefetch(struct vhost_virtqueue *vq);
+int vq_meta_prefetch(struct vhost_virtqueue *vq);

struct vhost_msg_node *vhost_new_msg(struct vhost_virtqueue *vq, int type);
void vhost_enqueue_msg(struct vhost_dev *dev,
--
1.8.3.1


2019-03-06 07:19:43

by Jason Wang

[permalink] [raw]
Subject: [RFC PATCH V2 5/5] vhost: access vq metadata through kernel virtual address

It was noticed that the copy_user() friends that was used to access
virtqueue metdata tends to be very expensive for dataplane
implementation like vhost since it involves lots of software checks,
speculation barrier, hardware feature toggling (e.g SMAP). The
extra cost will be more obvious when transferring small packets since
the time spent on metadata accessing become more significant.

This patch tries to eliminate those overheads by accessing them
through kernel virtual address by vmap(). To make the pages can be
migrated, instead of pinning them through GUP, we use MMU notifiers to
invalidate vmaps and re-establish vmaps during each round of metadata
prefetching if necessary. It looks to me .invalidate_range() is
sufficient for catching this since we don't need extra TLB flush. For
devices that doesn't use metadata prefetching, the memory accessors
fallback to normal copy_user() implementation gracefully. The
invalidation was synchronized with datapath through vq mutex, and in
order to avoid hold vq mutex during range checking, MMU notifier was
teared down when trying to modify vq metadata.

Dirty page checking is done by calling set_page_dirty_locked()
explicitly for the page that used ring stay after each round of
processing.

Note that this was only done when device IOTLB is not enabled. We
could use similar method to optimize it in the future.

Tests shows at most about 22% improvement on TX PPS when using
virtio-user + vhost_net + xdp1 + TAP on 2.6GHz Broadwell:

SMAP on | SMAP off
Before: 5.0Mpps | 6.6Mpps
After: 6.1Mpps | 7.4Mpps

Cc: <[email protected]>
Signed-off-by: Jason Wang <[email protected]>
---
drivers/vhost/net.c | 2 +
drivers/vhost/vhost.c | 281 +++++++++++++++++++++++++++++++++++++++++++++++++-
drivers/vhost/vhost.h | 16 +++
3 files changed, 297 insertions(+), 2 deletions(-)

diff --git a/drivers/vhost/net.c b/drivers/vhost/net.c
index bf55f99..c276371 100644
--- a/drivers/vhost/net.c
+++ b/drivers/vhost/net.c
@@ -982,6 +982,7 @@ static void handle_tx(struct vhost_net *net)
else
handle_tx_copy(net, sock);

+ vq_meta_prefetch_done(vq);
out:
mutex_unlock(&vq->mutex);
}
@@ -1250,6 +1251,7 @@ static void handle_rx(struct vhost_net *net)
vhost_net_enable_vq(net, vq);
out:
vhost_net_signal_used(nvq);
+ vq_meta_prefetch_done(vq);
mutex_unlock(&vq->mutex);
}

diff --git a/drivers/vhost/vhost.c b/drivers/vhost/vhost.c
index 1015464..36ccf7c 100644
--- a/drivers/vhost/vhost.c
+++ b/drivers/vhost/vhost.c
@@ -434,6 +434,74 @@ static size_t vhost_get_desc_size(struct vhost_virtqueue *vq, int num)
return sizeof(*vq->desc) * num;
}

+static void vhost_uninit_vmap(struct vhost_vmap *map)
+{
+ if (map->addr) {
+ vunmap(map->unmap_addr);
+ kfree(map->pages);
+ map->pages = NULL;
+ map->npages = 0;
+ }
+
+ map->addr = NULL;
+ map->unmap_addr = NULL;
+}
+
+static void vhost_invalidate_vmap(struct vhost_virtqueue *vq,
+ struct vhost_vmap *map,
+ unsigned long ustart,
+ size_t size,
+ unsigned long start,
+ unsigned long end)
+{
+ if (end < ustart || start > ustart - 1 + size)
+ return;
+
+ dump_stack();
+ mutex_lock(&vq->mutex);
+ vhost_uninit_vmap(map);
+ mutex_unlock(&vq->mutex);
+}
+
+
+static void vhost_invalidate(struct vhost_dev *dev,
+ unsigned long start, unsigned long end)
+{
+ int i;
+
+ for (i = 0; i < dev->nvqs; i++) {
+ struct vhost_virtqueue *vq = dev->vqs[i];
+
+ vhost_invalidate_vmap(vq, &vq->avail_ring,
+ (unsigned long)vq->avail,
+ vhost_get_avail_size(vq, vq->num),
+ start, end);
+ vhost_invalidate_vmap(vq, &vq->desc_ring,
+ (unsigned long)vq->desc,
+ vhost_get_desc_size(vq, vq->num),
+ start, end);
+ vhost_invalidate_vmap(vq, &vq->used_ring,
+ (unsigned long)vq->used,
+ vhost_get_used_size(vq, vq->num),
+ start, end);
+ }
+}
+
+
+static void vhost_invalidate_range(struct mmu_notifier *mn,
+ struct mm_struct *mm,
+ unsigned long start, unsigned long end)
+{
+ struct vhost_dev *dev = container_of(mn, struct vhost_dev,
+ mmu_notifier);
+
+ vhost_invalidate(dev, start, end);
+}
+
+static const struct mmu_notifier_ops vhost_mmu_notifier_ops = {
+ .invalidate_range = vhost_invalidate_range,
+};
+
void vhost_dev_init(struct vhost_dev *dev,
struct vhost_virtqueue **vqs, int nvqs, int iov_limit)
{
@@ -449,6 +517,7 @@ void vhost_dev_init(struct vhost_dev *dev,
dev->mm = NULL;
dev->worker = NULL;
dev->iov_limit = iov_limit;
+ dev->mmu_notifier.ops = &vhost_mmu_notifier_ops;
init_llist_head(&dev->work_list);
init_waitqueue_head(&dev->wait);
INIT_LIST_HEAD(&dev->read_list);
@@ -462,6 +531,9 @@ void vhost_dev_init(struct vhost_dev *dev,
vq->indirect = NULL;
vq->heads = NULL;
vq->dev = dev;
+ vq->avail_ring.addr = NULL;
+ vq->used_ring.addr = NULL;
+ vq->desc_ring.addr = NULL;
mutex_init(&vq->mutex);
vhost_vq_reset(dev, vq);
if (vq->handle_kick)
@@ -542,7 +614,13 @@ long vhost_dev_set_owner(struct vhost_dev *dev)
if (err)
goto err_cgroup;

+ err = mmu_notifier_register(&dev->mmu_notifier, dev->mm);
+ if (err)
+ goto err_mmu_notifier;
+
return 0;
+err_mmu_notifier:
+ vhost_dev_free_iovecs(dev);
err_cgroup:
kthread_stop(worker);
dev->worker = NULL;
@@ -633,6 +711,81 @@ static void vhost_clear_msg(struct vhost_dev *dev)
spin_unlock(&dev->iotlb_lock);
}

+static int vhost_init_vmap(struct vhost_dev *dev,
+ struct vhost_vmap *map, unsigned long uaddr,
+ size_t size, int write)
+{
+ struct page **pages;
+ int npages = DIV_ROUND_UP(size, PAGE_SIZE);
+ int npinned;
+ void *vaddr;
+ int err = -EFAULT;
+
+ err = -ENOMEM;
+ pages = kmalloc_array(npages, sizeof(struct page *), GFP_KERNEL);
+ if (!pages)
+ goto err_uaddr;
+
+ err = EFAULT;
+ npinned = get_user_pages_fast(uaddr, npages, write, pages);
+ if (npinned != npages)
+ goto err_gup;
+
+ vaddr = vmap(pages, npages, VM_MAP, PAGE_KERNEL);
+ if (!vaddr)
+ goto err_gup;
+
+ map->addr = vaddr + (uaddr & (PAGE_SIZE - 1));
+ map->unmap_addr = vaddr;
+ map->npages = npages;
+ map->pages = pages;
+
+err_gup:
+ /* Don't pin pages, mmu notifier will notify us about page
+ * migration.
+ */
+ if (npinned > 0)
+ release_pages(pages, npinned);
+err_uaddr:
+ return err;
+}
+
+static void vhost_uninit_vq_vmaps(struct vhost_virtqueue *vq)
+{
+ vhost_uninit_vmap(&vq->avail_ring);
+ vhost_uninit_vmap(&vq->desc_ring);
+ vhost_uninit_vmap(&vq->used_ring);
+}
+
+static int vhost_setup_avail_vmap(struct vhost_virtqueue *vq,
+ unsigned long avail)
+{
+ return vhost_init_vmap(vq->dev, &vq->avail_ring, avail,
+ vhost_get_avail_size(vq, vq->num), false);
+}
+
+static int vhost_setup_desc_vmap(struct vhost_virtqueue *vq,
+ unsigned long desc)
+{
+ return vhost_init_vmap(vq->dev, &vq->desc_ring, desc,
+ vhost_get_desc_size(vq, vq->num), false);
+}
+
+static int vhost_setup_used_vmap(struct vhost_virtqueue *vq,
+ unsigned long used)
+{
+ return vhost_init_vmap(vq->dev, &vq->used_ring, used,
+ vhost_get_used_size(vq, vq->num), true);
+}
+
+static void vhost_set_vmap_dirty(struct vhost_vmap *used)
+{
+ int i;
+
+ for (i = 0; i < used->npages; i++)
+ set_page_dirty_lock(used->pages[i]);
+}
+
void vhost_dev_cleanup(struct vhost_dev *dev)
{
int i;
@@ -662,8 +815,12 @@ void vhost_dev_cleanup(struct vhost_dev *dev)
kthread_stop(dev->worker);
dev->worker = NULL;
}
- if (dev->mm)
+ for (i = 0; i < dev->nvqs; i++)
+ vhost_uninit_vq_vmaps(dev->vqs[i]);
+ if (dev->mm) {
+ mmu_notifier_unregister(&dev->mmu_notifier, dev->mm);
mmput(dev->mm);
+ }
dev->mm = NULL;
}
EXPORT_SYMBOL_GPL(vhost_dev_cleanup);
@@ -892,6 +1049,16 @@ static inline void __user *__vhost_get_user(struct vhost_virtqueue *vq,

static inline int vhost_put_avail_event(struct vhost_virtqueue *vq)
{
+ if (!vq->iotlb) {
+ struct vring_used *used = vq->used_ring.addr;
+
+ if (likely(used)) {
+ *((__virtio16 *)&used->ring[vq->num]) =
+ cpu_to_vhost16(vq, vq->avail_idx);
+ return 0;
+ }
+ }
+
return vhost_put_user(vq, cpu_to_vhost16(vq, vq->avail_idx),
vhost_avail_event(vq));
}
@@ -900,6 +1067,16 @@ static inline int vhost_put_used(struct vhost_virtqueue *vq,
struct vring_used_elem *head, int idx,
int count)
{
+ if (!vq->iotlb) {
+ struct vring_used *used = vq->used_ring.addr;
+
+ if (likely(used)) {
+ memcpy(used->ring + idx, head,
+ count * sizeof(*head));
+ return 0;
+ }
+ }
+
return vhost_copy_to_user(vq, vq->used->ring + idx, head,
count * sizeof(*head));
}
@@ -907,6 +1084,15 @@ static inline int vhost_put_used(struct vhost_virtqueue *vq,
static inline int vhost_put_used_flags(struct vhost_virtqueue *vq)

{
+ if (!vq->iotlb) {
+ struct vring_used *used = vq->used_ring.addr;
+
+ if (likely(used)) {
+ used->flags = cpu_to_vhost16(vq, vq->used_flags);
+ return 0;
+ }
+ }
+
return vhost_put_user(vq, cpu_to_vhost16(vq, vq->used_flags),
&vq->used->flags);
}
@@ -914,6 +1100,15 @@ static inline int vhost_put_used_flags(struct vhost_virtqueue *vq)
static inline int vhost_put_used_idx(struct vhost_virtqueue *vq)

{
+ if (!vq->iotlb) {
+ struct vring_used *used = vq->used_ring.addr;
+
+ if (likely(used)) {
+ used->idx = cpu_to_vhost16(vq, vq->last_used_idx);
+ return 0;
+ }
+ }
+
return vhost_put_user(vq, cpu_to_vhost16(vq, vq->last_used_idx),
&vq->used->idx);
}
@@ -959,12 +1154,30 @@ static void vhost_dev_unlock_vqs(struct vhost_dev *d)
static inline int vhost_get_avail_idx(struct vhost_virtqueue *vq,
__virtio16 *idx)
{
+ if (!vq->iotlb) {
+ struct vring_avail *avail = vq->avail_ring.addr;
+
+ if (likely(avail)) {
+ *idx = avail->idx;
+ return 0;
+ }
+ }
+
return vhost_get_avail(vq, *idx, &vq->avail->idx);
}

static inline int vhost_get_avail_head(struct vhost_virtqueue *vq,
__virtio16 *head, int idx)
{
+ if (!vq->iotlb) {
+ struct vring_avail *avail = vq->avail_ring.addr;
+
+ if (likely(avail)) {
+ *head = avail->ring[idx & (vq->num - 1)];
+ return 0;
+ }
+ }
+
return vhost_get_avail(vq, *head,
&vq->avail->ring[idx & (vq->num - 1)]);
}
@@ -972,24 +1185,60 @@ static inline int vhost_get_avail_head(struct vhost_virtqueue *vq,
static inline int vhost_get_avail_flags(struct vhost_virtqueue *vq,
__virtio16 *flags)
{
+ if (!vq->iotlb) {
+ struct vring_avail *avail = vq->avail_ring.addr;
+
+ if (likely(avail)) {
+ *flags = avail->flags;
+ return 0;
+ }
+ }
+
return vhost_get_avail(vq, *flags, &vq->avail->flags);
}

static inline int vhost_get_used_event(struct vhost_virtqueue *vq,
__virtio16 *event)
{
+ if (!vq->iotlb) {
+ struct vring_avail *avail = vq->avail_ring.addr;
+
+ if (likely(avail)) {
+ *event = (__virtio16)avail->ring[vq->num];
+ return 0;
+ }
+ }
+
return vhost_get_avail(vq, *event, vhost_used_event(vq));
}

static inline int vhost_get_used_idx(struct vhost_virtqueue *vq,
__virtio16 *idx)
{
+ if (!vq->iotlb) {
+ struct vring_used *used = vq->used_ring.addr;
+
+ if (likely(used)) {
+ *idx = used->idx;
+ return 0;
+ }
+ }
+
return vhost_get_used(vq, *idx, &vq->used->idx);
}

static inline int vhost_get_desc(struct vhost_virtqueue *vq,
struct vring_desc *desc, int idx)
{
+ if (!vq->iotlb) {
+ struct vring_desc *d = vq->desc_ring.addr;
+
+ if (likely(d)) {
+ *desc = *(d + idx);
+ return 0;
+ }
+ }
+
return vhost_copy_from_user(vq, desc, vq->desc + idx, sizeof(*desc));
}

@@ -1330,8 +1579,16 @@ int vq_meta_prefetch(struct vhost_virtqueue *vq)
{
unsigned int num = vq->num;

- if (!vq->iotlb)
+ if (!vq->iotlb) {
+ if (unlikely(!vq->avail_ring.addr))
+ vhost_setup_avail_vmap(vq, (unsigned long)vq->avail);
+ if (unlikely(!vq->desc_ring.addr))
+ vhost_setup_desc_vmap(vq, (unsigned long)vq->desc);
+ if (unlikely(!vq->used_ring.addr))
+ vhost_setup_used_vmap(vq, (unsigned long)vq->used);
+
return 1;
+ }

return iotlb_access_ok(vq, VHOST_ACCESS_RO, (u64)(uintptr_t)vq->desc,
vhost_get_desc_size(vq, num), VHOST_ADDR_DESC) &&
@@ -1343,6 +1600,15 @@ int vq_meta_prefetch(struct vhost_virtqueue *vq)
}
EXPORT_SYMBOL_GPL(vq_meta_prefetch);

+void vq_meta_prefetch_done(struct vhost_virtqueue *vq)
+{
+ if (vq->iotlb)
+ return;
+ if (likely(vq->used_ring.addr))
+ vhost_set_vmap_dirty(&vq->used_ring);
+}
+EXPORT_SYMBOL_GPL(vq_meta_prefetch_done);
+
/* Can we log writes? */
/* Caller should have device mutex but not vq mutex */
bool vhost_log_access_ok(struct vhost_dev *dev)
@@ -1483,6 +1749,13 @@ long vhost_vring_ioctl(struct vhost_dev *d, unsigned int ioctl, void __user *arg

mutex_lock(&vq->mutex);

+ /* Unregister MMU notifer to allow invalidation callback
+ * can access vq->avail, vq->desc , vq->used and vq->num
+ * without holding vq->mutex.
+ */
+ if (d->mm)
+ mmu_notifier_unregister(&d->mmu_notifier, d->mm);
+
switch (ioctl) {
case VHOST_SET_VRING_NUM:
/* Resizing ring with an active backend?
@@ -1499,6 +1772,7 @@ long vhost_vring_ioctl(struct vhost_dev *d, unsigned int ioctl, void __user *arg
r = -EINVAL;
break;
}
+ vhost_uninit_vq_vmaps(vq);
vq->num = s.num;
break;
case VHOST_SET_VRING_BASE:
@@ -1581,6 +1855,7 @@ long vhost_vring_ioctl(struct vhost_dev *d, unsigned int ioctl, void __user *arg
vq->avail = (void __user *)(unsigned long)a.avail_user_addr;
vq->log_addr = a.log_guest_addr;
vq->used = (void __user *)(unsigned long)a.used_user_addr;
+ vhost_uninit_vq_vmaps(vq);
break;
case VHOST_SET_VRING_KICK:
if (copy_from_user(&f, argp, sizeof f)) {
@@ -1656,6 +1931,8 @@ long vhost_vring_ioctl(struct vhost_dev *d, unsigned int ioctl, void __user *arg
if (pollstart && vq->handle_kick)
r = vhost_poll_start(&vq->poll, vq->kick);

+ if (d->mm)
+ mmu_notifier_register(&d->mmu_notifier, d->mm);
mutex_unlock(&vq->mutex);

if (pollstop && vq->handle_kick)
diff --git a/drivers/vhost/vhost.h b/drivers/vhost/vhost.h
index 7a7fc00..146076e 100644
--- a/drivers/vhost/vhost.h
+++ b/drivers/vhost/vhost.h
@@ -12,6 +12,8 @@
#include <linux/virtio_config.h>
#include <linux/virtio_ring.h>
#include <linux/atomic.h>
+#include <linux/pagemap.h>
+#include <linux/mmu_notifier.h>

struct vhost_work;
typedef void (*vhost_work_fn_t)(struct vhost_work *work);
@@ -80,6 +82,13 @@ enum vhost_uaddr_type {
VHOST_NUM_ADDRS = 3,
};

+struct vhost_vmap {
+ void *addr;
+ void *unmap_addr;
+ int npages;
+ struct page **pages;
+};
+
/* The virtqueue structure describes a queue attached to a device. */
struct vhost_virtqueue {
struct vhost_dev *dev;
@@ -90,6 +99,11 @@ struct vhost_virtqueue {
struct vring_desc __user *desc;
struct vring_avail __user *avail;
struct vring_used __user *used;
+
+ struct vhost_vmap avail_ring;
+ struct vhost_vmap desc_ring;
+ struct vhost_vmap used_ring;
+
const struct vhost_umem_node *meta_iotlb[VHOST_NUM_ADDRS];
struct file *kick;
struct eventfd_ctx *call_ctx;
@@ -158,6 +172,7 @@ struct vhost_msg_node {

struct vhost_dev {
struct mm_struct *mm;
+ struct mmu_notifier mmu_notifier;
struct mutex mutex;
struct vhost_virtqueue **vqs;
int nvqs;
@@ -210,6 +225,7 @@ int vhost_log_write(struct vhost_virtqueue *vq, struct vhost_log *log,
unsigned int log_num, u64 len,
struct iovec *iov, int count);
int vq_meta_prefetch(struct vhost_virtqueue *vq);
+void vq_meta_prefetch_done(struct vhost_virtqueue *vq);

struct vhost_msg_node *vhost_new_msg(struct vhost_virtqueue *vq, int type);
void vhost_enqueue_msg(struct vhost_dev *dev,
--
1.8.3.1


2019-03-06 07:20:20

by Jason Wang

[permalink] [raw]
Subject: [RFC PATCH V2 1/5] vhost: generalize adding used elem

Use one generic vhost_copy_to_user() instead of two dedicated
accessor. This will simplify the conversion to fine grain
accessors. About 2% improvement of PPS were seen during vitio-user
txonly test.

Signed-off-by: Jason Wang <[email protected]>
---
drivers/vhost/vhost.c | 11 +----------
1 file changed, 1 insertion(+), 10 deletions(-)

diff --git a/drivers/vhost/vhost.c b/drivers/vhost/vhost.c
index a2e5dc7..400aa78 100644
--- a/drivers/vhost/vhost.c
+++ b/drivers/vhost/vhost.c
@@ -2251,16 +2251,7 @@ static int __vhost_add_used_n(struct vhost_virtqueue *vq,

start = vq->last_used_idx & (vq->num - 1);
used = vq->used->ring + start;
- if (count == 1) {
- if (vhost_put_user(vq, heads[0].id, &used->id)) {
- vq_err(vq, "Failed to write used id");
- return -EFAULT;
- }
- if (vhost_put_user(vq, heads[0].len, &used->len)) {
- vq_err(vq, "Failed to write used len");
- return -EFAULT;
- }
- } else if (vhost_copy_to_user(vq, used, heads, count * sizeof *used)) {
+ if (vhost_copy_to_user(vq, used, heads, count * sizeof *used)) {
vq_err(vq, "Failed to write used");
return -EFAULT;
}
--
1.8.3.1


2019-03-06 07:32:41

by Jason Wang

[permalink] [raw]
Subject: [RFC PATCH V2 4/5] vhost: introduce helpers to get the size of metadata area

Signed-off-by: Jason Wang <[email protected]>
---
drivers/vhost/vhost.c | 46 ++++++++++++++++++++++++++++------------------
1 file changed, 28 insertions(+), 18 deletions(-)

diff --git a/drivers/vhost/vhost.c b/drivers/vhost/vhost.c
index 2025543..1015464 100644
--- a/drivers/vhost/vhost.c
+++ b/drivers/vhost/vhost.c
@@ -413,6 +413,27 @@ static void vhost_dev_free_iovecs(struct vhost_dev *dev)
vhost_vq_free_iovecs(dev->vqs[i]);
}

+static size_t vhost_get_avail_size(struct vhost_virtqueue *vq, int num)
+{
+ size_t event = vhost_has_feature(vq, VIRTIO_RING_F_EVENT_IDX) ? 2 : 0;
+
+ return sizeof(*vq->avail) +
+ sizeof(*vq->avail->ring) * num + event;
+}
+
+static size_t vhost_get_used_size(struct vhost_virtqueue *vq, int num)
+{
+ size_t event = vhost_has_feature(vq, VIRTIO_RING_F_EVENT_IDX) ? 2 : 0;
+
+ return sizeof(*vq->used) +
+ sizeof(*vq->used->ring) * num + event;
+}
+
+static size_t vhost_get_desc_size(struct vhost_virtqueue *vq, int num)
+{
+ return sizeof(*vq->desc) * num;
+}
+
void vhost_dev_init(struct vhost_dev *dev,
struct vhost_virtqueue **vqs, int nvqs, int iov_limit)
{
@@ -1253,13 +1274,9 @@ static bool vq_access_ok(struct vhost_virtqueue *vq, unsigned int num,
struct vring_used __user *used)

{
- size_t s = vhost_has_feature(vq, VIRTIO_RING_F_EVENT_IDX) ? 2 : 0;
-
- return access_ok(desc, num * sizeof *desc) &&
- access_ok(avail,
- sizeof *avail + num * sizeof *avail->ring + s) &&
- access_ok(used,
- sizeof *used + num * sizeof *used->ring + s);
+ return access_ok(desc, vhost_get_desc_size(vq, num)) &&
+ access_ok(avail, vhost_get_avail_size(vq, num)) &&
+ access_ok(used, vhost_get_used_size(vq, num));
}

static void vhost_vq_meta_update(struct vhost_virtqueue *vq,
@@ -1311,22 +1328,18 @@ static bool iotlb_access_ok(struct vhost_virtqueue *vq,

int vq_meta_prefetch(struct vhost_virtqueue *vq)
{
- size_t s = vhost_has_feature(vq, VIRTIO_RING_F_EVENT_IDX) ? 2 : 0;
unsigned int num = vq->num;

if (!vq->iotlb)
return 1;

return iotlb_access_ok(vq, VHOST_ACCESS_RO, (u64)(uintptr_t)vq->desc,
- num * sizeof(*vq->desc), VHOST_ADDR_DESC) &&
+ vhost_get_desc_size(vq, num), VHOST_ADDR_DESC) &&
iotlb_access_ok(vq, VHOST_ACCESS_RO, (u64)(uintptr_t)vq->avail,
- sizeof *vq->avail +
- num * sizeof(*vq->avail->ring) + s,
+ vhost_get_avail_size(vq, num),
VHOST_ADDR_AVAIL) &&
iotlb_access_ok(vq, VHOST_ACCESS_WO, (u64)(uintptr_t)vq->used,
- sizeof *vq->used +
- num * sizeof(*vq->used->ring) + s,
- VHOST_ADDR_USED);
+ vhost_get_used_size(vq, num), VHOST_ADDR_USED);
}
EXPORT_SYMBOL_GPL(vq_meta_prefetch);

@@ -1343,13 +1356,10 @@ bool vhost_log_access_ok(struct vhost_dev *dev)
static bool vq_log_access_ok(struct vhost_virtqueue *vq,
void __user *log_base)
{
- size_t s = vhost_has_feature(vq, VIRTIO_RING_F_EVENT_IDX) ? 2 : 0;
-
return vq_memory_access_ok(log_base, vq->umem,
vhost_has_feature(vq, VHOST_F_LOG_ALL)) &&
(!vq->log_used || log_access_ok(log_base, vq->log_addr,
- sizeof *vq->used +
- vq->num * sizeof *vq->used->ring + s));
+ vhost_get_used_size(vq, vq->num)));
}

/* Can we start vq? */
--
1.8.3.1


2019-03-06 11:24:26

by Christophe de Dinechin

[permalink] [raw]
Subject: Re: [RFC PATCH V2 4/5] vhost: introduce helpers to get the size of metadata area



> On 6 Mar 2019, at 08:18, Jason Wang <[email protected]> wrote:
>
> Signed-off-by: Jason Wang <[email protected]>
> ---
> drivers/vhost/vhost.c | 46 ++++++++++++++++++++++++++++------------------
> 1 file changed, 28 insertions(+), 18 deletions(-)
>
> diff --git a/drivers/vhost/vhost.c b/drivers/vhost/vhost.c
> index 2025543..1015464 100644
> --- a/drivers/vhost/vhost.c
> +++ b/drivers/vhost/vhost.c
> @@ -413,6 +413,27 @@ static void vhost_dev_free_iovecs(struct vhost_dev *dev)
> vhost_vq_free_iovecs(dev->vqs[i]);
> }
>
> +static size_t vhost_get_avail_size(struct vhost_virtqueue *vq, int num)

Nit: Any reason not to make `num` unsigned or size_t?

> +{
> + size_t event = vhost_has_feature(vq, VIRTIO_RING_F_EVENT_IDX) ? 2 : 0;
> +
> + return sizeof(*vq->avail) +
> + sizeof(*vq->avail->ring) * num + event;
> +}
> +
> +static size_t vhost_get_used_size(struct vhost_virtqueue *vq, int num)
> +{
> + size_t event = vhost_has_feature(vq, VIRTIO_RING_F_EVENT_IDX) ? 2 : 0;
> +
> + return sizeof(*vq->used) +
> + sizeof(*vq->used->ring) * num + event;
> +}
> +
> +static size_t vhost_get_desc_size(struct vhost_virtqueue *vq, int num)
> +{
> + return sizeof(*vq->desc) * num;
> +}
> +
> void vhost_dev_init(struct vhost_dev *dev,
> struct vhost_virtqueue **vqs, int nvqs, int iov_limit)
> {
> @@ -1253,13 +1274,9 @@ static bool vq_access_ok(struct vhost_virtqueue *vq, unsigned int num,
> struct vring_used __user *used)
>
> {
> - size_t s = vhost_has_feature(vq, VIRTIO_RING_F_EVENT_IDX) ? 2 : 0;
> -
> - return access_ok(desc, num * sizeof *desc) &&
> - access_ok(avail,
> - sizeof *avail + num * sizeof *avail->ring + s) &&
> - access_ok(used,
> - sizeof *used + num * sizeof *used->ring + s);
> + return access_ok(desc, vhost_get_desc_size(vq, num)) &&
> + access_ok(avail, vhost_get_avail_size(vq, num)) &&
> + access_ok(used, vhost_get_used_size(vq, num));
> }
>
> static void vhost_vq_meta_update(struct vhost_virtqueue *vq,
> @@ -1311,22 +1328,18 @@ static bool iotlb_access_ok(struct vhost_virtqueue *vq,
>
> int vq_meta_prefetch(struct vhost_virtqueue *vq)
> {
> - size_t s = vhost_has_feature(vq, VIRTIO_RING_F_EVENT_IDX) ? 2 : 0;
> unsigned int num = vq->num;
>
> if (!vq->iotlb)
> return 1;
>
> return iotlb_access_ok(vq, VHOST_ACCESS_RO, (u64)(uintptr_t)vq->desc,
> - num * sizeof(*vq->desc), VHOST_ADDR_DESC) &&
> + vhost_get_desc_size(vq, num), VHOST_ADDR_DESC) &&
> iotlb_access_ok(vq, VHOST_ACCESS_RO, (u64)(uintptr_t)vq->avail,
> - sizeof *vq->avail +
> - num * sizeof(*vq->avail->ring) + s,
> + vhost_get_avail_size(vq, num),
> VHOST_ADDR_AVAIL) &&
> iotlb_access_ok(vq, VHOST_ACCESS_WO, (u64)(uintptr_t)vq->used,
> - sizeof *vq->used +
> - num * sizeof(*vq->used->ring) + s,
> - VHOST_ADDR_USED);
> + vhost_get_used_size(vq, num), VHOST_ADDR_USED);
> }
> EXPORT_SYMBOL_GPL(vq_meta_prefetch);
>
> @@ -1343,13 +1356,10 @@ bool vhost_log_access_ok(struct vhost_dev *dev)
> static bool vq_log_access_ok(struct vhost_virtqueue *vq,
> void __user *log_base)
> {
> - size_t s = vhost_has_feature(vq, VIRTIO_RING_F_EVENT_IDX) ? 2 : 0;
> -
> return vq_memory_access_ok(log_base, vq->umem,
> vhost_has_feature(vq, VHOST_F_LOG_ALL)) &&
> (!vq->log_used || log_access_ok(log_base, vq->log_addr,
> - sizeof *vq->used +
> - vq->num * sizeof *vq->used->ring + s));
> + vhost_get_used_size(vq, vq->num)));
> }
>
> /* Can we start vq? */
> --
> 1.8.3.1
>


2019-03-06 12:08:55

by Christophe de Dinechin

[permalink] [raw]
Subject: Re: [RFC PATCH V2 2/5] vhost: fine grain userspace memory accessors



> On 6 Mar 2019, at 08:18, Jason Wang <[email protected]> wrote:
>
> This is used to hide the metadata address from virtqueue helpers. This
> will allow to implement a vmap based fast accessing to metadata.
>
> Signed-off-by: Jason Wang <[email protected]>
> ---
> drivers/vhost/vhost.c | 94 +++++++++++++++++++++++++++++++++++++++++----------
> 1 file changed, 77 insertions(+), 17 deletions(-)
>
> diff --git a/drivers/vhost/vhost.c b/drivers/vhost/vhost.c
> index 400aa78..29709e7 100644
> --- a/drivers/vhost/vhost.c
> +++ b/drivers/vhost/vhost.c
> @@ -869,6 +869,34 @@ static inline void __user *__vhost_get_user(struct vhost_virtqueue *vq,
> ret; \
> })
>
> +static inline int vhost_put_avail_event(struct vhost_virtqueue *vq)
> +{
> + return vhost_put_user(vq, cpu_to_vhost16(vq, vq->avail_idx),
> + vhost_avail_event(vq));
> +}
> +
> +static inline int vhost_put_used(struct vhost_virtqueue *vq,
> + struct vring_used_elem *head, int idx,
> + int count)
> +{
> + return vhost_copy_to_user(vq, vq->used->ring + idx, head,
> + count * sizeof(*head));
> +}
> +
> +static inline int vhost_put_used_flags(struct vhost_virtqueue *vq)
> +
> +{
> + return vhost_put_user(vq, cpu_to_vhost16(vq, vq->used_flags),
> + &vq->used->flags);
> +}
> +
> +static inline int vhost_put_used_idx(struct vhost_virtqueue *vq)
> +
> +{
> + return vhost_put_user(vq, cpu_to_vhost16(vq, vq->last_used_idx),
> + &vq->used->idx);
> +}
> +
> #define vhost_get_user(vq, x, ptr, type) \
> ({ \
> int ret; \
> @@ -907,6 +935,43 @@ static void vhost_dev_unlock_vqs(struct vhost_dev *d)
> mutex_unlock(&d->vqs[i]->mutex);
> }
>
> +static inline int vhost_get_avail_idx(struct vhost_virtqueue *vq,
> + __virtio16 *idx)
> +{
> + return vhost_get_avail(vq, *idx, &vq->avail->idx);
> +}
> +
> +static inline int vhost_get_avail_head(struct vhost_virtqueue *vq,
> + __virtio16 *head, int idx)
> +{
> + return vhost_get_avail(vq, *head,
> + &vq->avail->ring[idx & (vq->num - 1)]);
> +}
> +
> +static inline int vhost_get_avail_flags(struct vhost_virtqueue *vq,
> + __virtio16 *flags)
> +{
> + return vhost_get_avail(vq, *flags, &vq->avail->flags);
> +}
> +
> +static inline int vhost_get_used_event(struct vhost_virtqueue *vq,
> + __virtio16 *event)
> +{
> + return vhost_get_avail(vq, *event, vhost_used_event(vq));
> +}
> +
> +static inline int vhost_get_used_idx(struct vhost_virtqueue *vq,
> + __virtio16 *idx)
> +{
> + return vhost_get_used(vq, *idx, &vq->used->idx);
> +}
> +
> +static inline int vhost_get_desc(struct vhost_virtqueue *vq,
> + struct vring_desc *desc, int idx)
> +{
> + return vhost_copy_from_user(vq, desc, vq->desc + idx, sizeof(*desc));
> +}
> +
> static int vhost_new_umem_range(struct vhost_umem *umem,
> u64 start, u64 size, u64 end,
> u64 userspace_addr, int perm)
> @@ -1840,8 +1905,7 @@ int vhost_log_write(struct vhost_virtqueue *vq, struct vhost_log *log,
> static int vhost_update_used_flags(struct vhost_virtqueue *vq)
> {
> void __user *used;
> - if (vhost_put_user(vq, cpu_to_vhost16(vq, vq->used_flags),
> - &vq->used->flags) < 0)
> + if (vhost_put_used_flags(vq))
> return -EFAULT;
> if (unlikely(vq->log_used)) {
> /* Make sure the flag is seen before log. */
> @@ -1858,8 +1922,7 @@ static int vhost_update_used_flags(struct vhost_virtqueue *vq)
>
> static int vhost_update_avail_event(struct vhost_virtqueue *vq, u16 avail_event)
> {
> - if (vhost_put_user(vq, cpu_to_vhost16(vq, vq->avail_idx),
> - vhost_avail_event(vq)))
> + if (vhost_put_avail_event(vq))
> return -EFAULT;
> if (unlikely(vq->log_used)) {
> void __user *used;
> @@ -1895,7 +1958,7 @@ int vhost_vq_init_access(struct vhost_virtqueue *vq)
> r = -EFAULT;
> goto err;
> }
> - r = vhost_get_used(vq, last_used_idx, &vq->used->idx);
> + r = vhost_get_used_idx(vq, &last_used_idx);
> if (r) {
> vq_err(vq, "Can't access used idx at %p\n",
> &vq->used->idx);

From the error case, it looks like you are not entirely encapsulating
knowledge of what the accessor uses, i.e. it’s not:

vq_err(vq, "Can't access used idx at %p\n",
&last_user_idx);

Maybe move error message within accessor?

> @@ -2094,7 +2157,7 @@ int vhost_get_vq_desc(struct vhost_virtqueue *vq,
> last_avail_idx = vq->last_avail_idx;
>
> if (vq->avail_idx == vq->last_avail_idx) {
> - if (unlikely(vhost_get_avail(vq, avail_idx, &vq->avail->idx))) {
> + if (unlikely(vhost_get_avail_idx(vq, &avail_idx))) {
> vq_err(vq, "Failed to access avail idx at %p\n",
> &vq->avail->idx);
> return -EFAULT;

Same here.

> @@ -2121,8 +2184,7 @@ int vhost_get_vq_desc(struct vhost_virtqueue *vq,
>
> /* Grab the next descriptor number they're advertising, and increment
> * the index we've seen. */
> - if (unlikely(vhost_get_avail(vq, ring_head,
> - &vq->avail->ring[last_avail_idx & (vq->num - 1)]))) {
> + if (unlikely(vhost_get_avail_head(vq, &ring_head, last_avail_idx))) {
> vq_err(vq, "Failed to read head: idx %d address %p\n",
> last_avail_idx,
> &vq->avail->ring[last_avail_idx % vq->num]);
> @@ -2157,8 +2219,7 @@ int vhost_get_vq_desc(struct vhost_virtqueue *vq,
> i, vq->num, head);
> return -EINVAL;
> }
> - ret = vhost_copy_from_user(vq, &desc, vq->desc + i,
> - sizeof desc);
> + ret = vhost_get_desc(vq, &desc, i);
> if (unlikely(ret)) {
> vq_err(vq, "Failed to get descriptor: idx %d addr %p\n",
> i, vq->desc + i);
> @@ -2251,7 +2312,7 @@ static int __vhost_add_used_n(struct vhost_virtqueue *vq,
>
> start = vq->last_used_idx & (vq->num - 1);
> used = vq->used->ring + start;
> - if (vhost_copy_to_user(vq, used, heads, count * sizeof *used)) {
> + if (vhost_put_used(vq, heads, start, count)) {
> vq_err(vq, "Failed to write used");
> return -EFAULT;
> }
> @@ -2293,8 +2354,7 @@ int vhost_add_used_n(struct vhost_virtqueue *vq, struct vring_used_elem *heads,
>
> /* Make sure buffer is written before we update index. */
> smp_wmb();
> - if (vhost_put_user(vq, cpu_to_vhost16(vq, vq->last_used_idx),
> - &vq->used->idx)) {
> + if (vhost_put_used_idx(vq)) {
> vq_err(vq, "Failed to increment used idx");
> return -EFAULT;
> }
> @@ -2327,7 +2387,7 @@ static bool vhost_notify(struct vhost_dev *dev, struct vhost_virtqueue *vq)
>
> if (!vhost_has_feature(vq, VIRTIO_RING_F_EVENT_IDX)) {
> __virtio16 flags;
> - if (vhost_get_avail(vq, flags, &vq->avail->flags)) {
> + if (vhost_get_avail_flags(vq, &flags)) {
> vq_err(vq, "Failed to get flags");
> return true;
> }
> @@ -2341,7 +2401,7 @@ static bool vhost_notify(struct vhost_dev *dev, struct vhost_virtqueue *vq)
> if (unlikely(!v))
> return true;
>
> - if (vhost_get_avail(vq, event, vhost_used_event(vq))) {
> + if (vhost_get_used_event(vq, &event)) {
> vq_err(vq, "Failed to get used event idx");
> return true;
> }
> @@ -2386,7 +2446,7 @@ bool vhost_vq_avail_empty(struct vhost_dev *dev, struct vhost_virtqueue *vq)
> if (vq->avail_idx != vq->last_avail_idx)
> return false;
>
> - r = vhost_get_avail(vq, avail_idx, &vq->avail->idx);
> + r = vhost_get_avail_idx(vq, &avail_idx);
> if (unlikely(r))
> return false;
> vq->avail_idx = vhost16_to_cpu(vq, avail_idx);
> @@ -2422,7 +2482,7 @@ bool vhost_enable_notify(struct vhost_dev *dev, struct vhost_virtqueue *vq)
> /* They could have slipped one in as we were doing that: make
> * sure it's written, then check again. */
> smp_mb();
> - r = vhost_get_avail(vq, avail_idx, &vq->avail->idx);
> + r = vhost_get_avail_idx(vq, &avail_idx);
> if (r) {
> vq_err(vq, "Failed to check avail idx at %p: %d\n",
> &vq->avail->idx, r);
> --
> 1.8.3.1
>


2019-03-06 19:14:38

by Michael S. Tsirkin

[permalink] [raw]
Subject: Re: [RFC PATCH V2 5/5] vhost: access vq metadata through kernel virtual address

On Wed, Mar 06, 2019 at 02:18:12AM -0500, Jason Wang wrote:
> It was noticed that the copy_user() friends that was used to access
> virtqueue metdata tends to be very expensive for dataplane
> implementation like vhost since it involves lots of software checks,
> speculation barrier, hardware feature toggling (e.g SMAP). The
> extra cost will be more obvious when transferring small packets since
> the time spent on metadata accessing become more significant.
>
> This patch tries to eliminate those overheads by accessing them
> through kernel virtual address by vmap(). To make the pages can be
> migrated, instead of pinning them through GUP, we use MMU notifiers to
> invalidate vmaps and re-establish vmaps during each round of metadata
> prefetching if necessary. It looks to me .invalidate_range() is
> sufficient for catching this since we don't need extra TLB flush. For
> devices that doesn't use metadata prefetching, the memory accessors
> fallback to normal copy_user() implementation gracefully. The
> invalidation was synchronized with datapath through vq mutex, and in
> order to avoid hold vq mutex during range checking, MMU notifier was
> teared down when trying to modify vq metadata.
>
> Dirty page checking is done by calling set_page_dirty_locked()
> explicitly for the page that used ring stay after each round of
> processing.
>
> Note that this was only done when device IOTLB is not enabled. We
> could use similar method to optimize it in the future.
>
> Tests shows at most about 22% improvement on TX PPS when using
> virtio-user + vhost_net + xdp1 + TAP on 2.6GHz Broadwell:
>
> SMAP on | SMAP off
> Before: 5.0Mpps | 6.6Mpps
> After: 6.1Mpps | 7.4Mpps
>
> Cc: <[email protected]>
> Signed-off-by: Jason Wang <[email protected]>
> ---
> drivers/vhost/net.c | 2 +
> drivers/vhost/vhost.c | 281 +++++++++++++++++++++++++++++++++++++++++++++++++-
> drivers/vhost/vhost.h | 16 +++
> 3 files changed, 297 insertions(+), 2 deletions(-)
>
> diff --git a/drivers/vhost/net.c b/drivers/vhost/net.c
> index bf55f99..c276371 100644
> --- a/drivers/vhost/net.c
> +++ b/drivers/vhost/net.c
> @@ -982,6 +982,7 @@ static void handle_tx(struct vhost_net *net)
> else
> handle_tx_copy(net, sock);
>
> + vq_meta_prefetch_done(vq);
> out:
> mutex_unlock(&vq->mutex);
> }
> @@ -1250,6 +1251,7 @@ static void handle_rx(struct vhost_net *net)
> vhost_net_enable_vq(net, vq);
> out:
> vhost_net_signal_used(nvq);
> + vq_meta_prefetch_done(vq);
> mutex_unlock(&vq->mutex);
> }
>
> diff --git a/drivers/vhost/vhost.c b/drivers/vhost/vhost.c
> index 1015464..36ccf7c 100644
> --- a/drivers/vhost/vhost.c
> +++ b/drivers/vhost/vhost.c
> @@ -434,6 +434,74 @@ static size_t vhost_get_desc_size(struct vhost_virtqueue *vq, int num)
> return sizeof(*vq->desc) * num;
> }
>
> +static void vhost_uninit_vmap(struct vhost_vmap *map)
> +{
> + if (map->addr) {
> + vunmap(map->unmap_addr);
> + kfree(map->pages);
> + map->pages = NULL;
> + map->npages = 0;
> + }
> +
> + map->addr = NULL;
> + map->unmap_addr = NULL;
> +}
> +
> +static void vhost_invalidate_vmap(struct vhost_virtqueue *vq,
> + struct vhost_vmap *map,
> + unsigned long ustart,
> + size_t size,
> + unsigned long start,
> + unsigned long end)
> +{
> + if (end < ustart || start > ustart - 1 + size)
> + return;
> +
> + dump_stack();
> + mutex_lock(&vq->mutex);
> + vhost_uninit_vmap(map);
> + mutex_unlock(&vq->mutex);
> +}
> +
> +
> +static void vhost_invalidate(struct vhost_dev *dev,
> + unsigned long start, unsigned long end)
> +{
> + int i;
> +
> + for (i = 0; i < dev->nvqs; i++) {
> + struct vhost_virtqueue *vq = dev->vqs[i];
> +
> + vhost_invalidate_vmap(vq, &vq->avail_ring,
> + (unsigned long)vq->avail,
> + vhost_get_avail_size(vq, vq->num),
> + start, end);
> + vhost_invalidate_vmap(vq, &vq->desc_ring,
> + (unsigned long)vq->desc,
> + vhost_get_desc_size(vq, vq->num),
> + start, end);
> + vhost_invalidate_vmap(vq, &vq->used_ring,
> + (unsigned long)vq->used,
> + vhost_get_used_size(vq, vq->num),
> + start, end);
> + }
> +}
> +
> +
> +static void vhost_invalidate_range(struct mmu_notifier *mn,
> + struct mm_struct *mm,
> + unsigned long start, unsigned long end)
> +{
> + struct vhost_dev *dev = container_of(mn, struct vhost_dev,
> + mmu_notifier);
> +
> + vhost_invalidate(dev, start, end);
> +}
> +
> +static const struct mmu_notifier_ops vhost_mmu_notifier_ops = {
> + .invalidate_range = vhost_invalidate_range,
> +};
> +
> void vhost_dev_init(struct vhost_dev *dev,
> struct vhost_virtqueue **vqs, int nvqs, int iov_limit)
> {


Note that
.invalidate_range seems to be called after page lock has
been dropped.

Looking at page dirty below:




> @@ -449,6 +517,7 @@ void vhost_dev_init(struct vhost_dev *dev,
> dev->mm = NULL;
> dev->worker = NULL;
> dev->iov_limit = iov_limit;
> + dev->mmu_notifier.ops = &vhost_mmu_notifier_ops;
> init_llist_head(&dev->work_list);
> init_waitqueue_head(&dev->wait);
> INIT_LIST_HEAD(&dev->read_list);
> @@ -462,6 +531,9 @@ void vhost_dev_init(struct vhost_dev *dev,
> vq->indirect = NULL;
> vq->heads = NULL;
> vq->dev = dev;
> + vq->avail_ring.addr = NULL;
> + vq->used_ring.addr = NULL;
> + vq->desc_ring.addr = NULL;
> mutex_init(&vq->mutex);
> vhost_vq_reset(dev, vq);
> if (vq->handle_kick)
> @@ -542,7 +614,13 @@ long vhost_dev_set_owner(struct vhost_dev *dev)
> if (err)
> goto err_cgroup;
>
> + err = mmu_notifier_register(&dev->mmu_notifier, dev->mm);
> + if (err)
> + goto err_mmu_notifier;
> +
> return 0;
> +err_mmu_notifier:
> + vhost_dev_free_iovecs(dev);
> err_cgroup:
> kthread_stop(worker);
> dev->worker = NULL;
> @@ -633,6 +711,81 @@ static void vhost_clear_msg(struct vhost_dev *dev)
> spin_unlock(&dev->iotlb_lock);
> }
>
> +static int vhost_init_vmap(struct vhost_dev *dev,
> + struct vhost_vmap *map, unsigned long uaddr,
> + size_t size, int write)
> +{
> + struct page **pages;
> + int npages = DIV_ROUND_UP(size, PAGE_SIZE);
> + int npinned;
> + void *vaddr;
> + int err = -EFAULT;
> +
> + err = -ENOMEM;
> + pages = kmalloc_array(npages, sizeof(struct page *), GFP_KERNEL);
> + if (!pages)
> + goto err_uaddr;
> +
> + err = EFAULT;
> + npinned = get_user_pages_fast(uaddr, npages, write, pages);
> + if (npinned != npages)
> + goto err_gup;
> +
> + vaddr = vmap(pages, npages, VM_MAP, PAGE_KERNEL);
> + if (!vaddr)
> + goto err_gup;
> +
> + map->addr = vaddr + (uaddr & (PAGE_SIZE - 1));
> + map->unmap_addr = vaddr;
> + map->npages = npages;
> + map->pages = pages;
> +
> +err_gup:
> + /* Don't pin pages, mmu notifier will notify us about page
> + * migration.
> + */
> + if (npinned > 0)
> + release_pages(pages, npinned);
> +err_uaddr:
> + return err;
> +}
> +
> +static void vhost_uninit_vq_vmaps(struct vhost_virtqueue *vq)
> +{
> + vhost_uninit_vmap(&vq->avail_ring);
> + vhost_uninit_vmap(&vq->desc_ring);
> + vhost_uninit_vmap(&vq->used_ring);
> +}
> +
> +static int vhost_setup_avail_vmap(struct vhost_virtqueue *vq,
> + unsigned long avail)
> +{
> + return vhost_init_vmap(vq->dev, &vq->avail_ring, avail,
> + vhost_get_avail_size(vq, vq->num), false);
> +}
> +
> +static int vhost_setup_desc_vmap(struct vhost_virtqueue *vq,
> + unsigned long desc)
> +{
> + return vhost_init_vmap(vq->dev, &vq->desc_ring, desc,
> + vhost_get_desc_size(vq, vq->num), false);
> +}
> +
> +static int vhost_setup_used_vmap(struct vhost_virtqueue *vq,
> + unsigned long used)
> +{
> + return vhost_init_vmap(vq->dev, &vq->used_ring, used,
> + vhost_get_used_size(vq, vq->num), true);
> +}
> +
> +static void vhost_set_vmap_dirty(struct vhost_vmap *used)
> +{
> + int i;
> +
> + for (i = 0; i < used->npages; i++)
> + set_page_dirty_lock(used->pages[i]);


This seems to rely on page lock to mark page dirty.

Could it happen that page writeback will check the
page, find it clean, and then you mark it dirty and then
invalidate callback is called?


> +}
> +
> void vhost_dev_cleanup(struct vhost_dev *dev)
> {
> int i;
> @@ -662,8 +815,12 @@ void vhost_dev_cleanup(struct vhost_dev *dev)
> kthread_stop(dev->worker);
> dev->worker = NULL;
> }
> - if (dev->mm)
> + for (i = 0; i < dev->nvqs; i++)
> + vhost_uninit_vq_vmaps(dev->vqs[i]);
> + if (dev->mm) {
> + mmu_notifier_unregister(&dev->mmu_notifier, dev->mm);
> mmput(dev->mm);
> + }
> dev->mm = NULL;
> }
> EXPORT_SYMBOL_GPL(vhost_dev_cleanup);
> @@ -892,6 +1049,16 @@ static inline void __user *__vhost_get_user(struct vhost_virtqueue *vq,
>
> static inline int vhost_put_avail_event(struct vhost_virtqueue *vq)
> {
> + if (!vq->iotlb) {
> + struct vring_used *used = vq->used_ring.addr;
> +
> + if (likely(used)) {
> + *((__virtio16 *)&used->ring[vq->num]) =
> + cpu_to_vhost16(vq, vq->avail_idx);
> + return 0;
> + }
> + }
> +
> return vhost_put_user(vq, cpu_to_vhost16(vq, vq->avail_idx),
> vhost_avail_event(vq));
> }
> @@ -900,6 +1067,16 @@ static inline int vhost_put_used(struct vhost_virtqueue *vq,
> struct vring_used_elem *head, int idx,
> int count)
> {
> + if (!vq->iotlb) {
> + struct vring_used *used = vq->used_ring.addr;
> +
> + if (likely(used)) {
> + memcpy(used->ring + idx, head,
> + count * sizeof(*head));
> + return 0;
> + }
> + }
> +
> return vhost_copy_to_user(vq, vq->used->ring + idx, head,
> count * sizeof(*head));
> }
> @@ -907,6 +1084,15 @@ static inline int vhost_put_used(struct vhost_virtqueue *vq,
> static inline int vhost_put_used_flags(struct vhost_virtqueue *vq)
>
> {
> + if (!vq->iotlb) {
> + struct vring_used *used = vq->used_ring.addr;
> +
> + if (likely(used)) {
> + used->flags = cpu_to_vhost16(vq, vq->used_flags);
> + return 0;
> + }
> + }
> +
> return vhost_put_user(vq, cpu_to_vhost16(vq, vq->used_flags),
> &vq->used->flags);
> }
> @@ -914,6 +1100,15 @@ static inline int vhost_put_used_flags(struct vhost_virtqueue *vq)
> static inline int vhost_put_used_idx(struct vhost_virtqueue *vq)
>
> {
> + if (!vq->iotlb) {
> + struct vring_used *used = vq->used_ring.addr;
> +
> + if (likely(used)) {
> + used->idx = cpu_to_vhost16(vq, vq->last_used_idx);
> + return 0;
> + }
> + }
> +
> return vhost_put_user(vq, cpu_to_vhost16(vq, vq->last_used_idx),
> &vq->used->idx);
> }
> @@ -959,12 +1154,30 @@ static void vhost_dev_unlock_vqs(struct vhost_dev *d)
> static inline int vhost_get_avail_idx(struct vhost_virtqueue *vq,
> __virtio16 *idx)
> {
> + if (!vq->iotlb) {
> + struct vring_avail *avail = vq->avail_ring.addr;
> +
> + if (likely(avail)) {
> + *idx = avail->idx;
> + return 0;
> + }
> + }
> +
> return vhost_get_avail(vq, *idx, &vq->avail->idx);
> }
>
> static inline int vhost_get_avail_head(struct vhost_virtqueue *vq,
> __virtio16 *head, int idx)
> {
> + if (!vq->iotlb) {
> + struct vring_avail *avail = vq->avail_ring.addr;
> +
> + if (likely(avail)) {
> + *head = avail->ring[idx & (vq->num - 1)];
> + return 0;
> + }
> + }
> +
> return vhost_get_avail(vq, *head,
> &vq->avail->ring[idx & (vq->num - 1)]);
> }
> @@ -972,24 +1185,60 @@ static inline int vhost_get_avail_head(struct vhost_virtqueue *vq,
> static inline int vhost_get_avail_flags(struct vhost_virtqueue *vq,
> __virtio16 *flags)
> {
> + if (!vq->iotlb) {
> + struct vring_avail *avail = vq->avail_ring.addr;
> +
> + if (likely(avail)) {
> + *flags = avail->flags;
> + return 0;
> + }
> + }
> +
> return vhost_get_avail(vq, *flags, &vq->avail->flags);
> }
>
> static inline int vhost_get_used_event(struct vhost_virtqueue *vq,
> __virtio16 *event)
> {
> + if (!vq->iotlb) {
> + struct vring_avail *avail = vq->avail_ring.addr;
> +
> + if (likely(avail)) {
> + *event = (__virtio16)avail->ring[vq->num];
> + return 0;
> + }
> + }
> +
> return vhost_get_avail(vq, *event, vhost_used_event(vq));
> }
>
> static inline int vhost_get_used_idx(struct vhost_virtqueue *vq,
> __virtio16 *idx)
> {
> + if (!vq->iotlb) {
> + struct vring_used *used = vq->used_ring.addr;
> +
> + if (likely(used)) {
> + *idx = used->idx;
> + return 0;
> + }
> + }
> +
> return vhost_get_used(vq, *idx, &vq->used->idx);
> }
>
> static inline int vhost_get_desc(struct vhost_virtqueue *vq,
> struct vring_desc *desc, int idx)
> {
> + if (!vq->iotlb) {
> + struct vring_desc *d = vq->desc_ring.addr;
> +
> + if (likely(d)) {
> + *desc = *(d + idx);
> + return 0;
> + }
> + }
> +
> return vhost_copy_from_user(vq, desc, vq->desc + idx, sizeof(*desc));
> }
>
> @@ -1330,8 +1579,16 @@ int vq_meta_prefetch(struct vhost_virtqueue *vq)
> {
> unsigned int num = vq->num;
>
> - if (!vq->iotlb)
> + if (!vq->iotlb) {
> + if (unlikely(!vq->avail_ring.addr))
> + vhost_setup_avail_vmap(vq, (unsigned long)vq->avail);
> + if (unlikely(!vq->desc_ring.addr))
> + vhost_setup_desc_vmap(vq, (unsigned long)vq->desc);
> + if (unlikely(!vq->used_ring.addr))
> + vhost_setup_used_vmap(vq, (unsigned long)vq->used);
> +
> return 1;
> + }
>
> return iotlb_access_ok(vq, VHOST_ACCESS_RO, (u64)(uintptr_t)vq->desc,
> vhost_get_desc_size(vq, num), VHOST_ADDR_DESC) &&
> @@ -1343,6 +1600,15 @@ int vq_meta_prefetch(struct vhost_virtqueue *vq)
> }
> EXPORT_SYMBOL_GPL(vq_meta_prefetch);
>
> +void vq_meta_prefetch_done(struct vhost_virtqueue *vq)
> +{
> + if (vq->iotlb)
> + return;
> + if (likely(vq->used_ring.addr))
> + vhost_set_vmap_dirty(&vq->used_ring);
> +}
> +EXPORT_SYMBOL_GPL(vq_meta_prefetch_done);
> +
> /* Can we log writes? */
> /* Caller should have device mutex but not vq mutex */
> bool vhost_log_access_ok(struct vhost_dev *dev)
> @@ -1483,6 +1749,13 @@ long vhost_vring_ioctl(struct vhost_dev *d, unsigned int ioctl, void __user *arg
>
> mutex_lock(&vq->mutex);
>
> + /* Unregister MMU notifer to allow invalidation callback
> + * can access vq->avail, vq->desc , vq->used and vq->num
> + * without holding vq->mutex.
> + */
> + if (d->mm)
> + mmu_notifier_unregister(&d->mmu_notifier, d->mm);
> +
> switch (ioctl) {
> case VHOST_SET_VRING_NUM:
> /* Resizing ring with an active backend?
> @@ -1499,6 +1772,7 @@ long vhost_vring_ioctl(struct vhost_dev *d, unsigned int ioctl, void __user *arg
> r = -EINVAL;
> break;
> }
> + vhost_uninit_vq_vmaps(vq);
> vq->num = s.num;
> break;
> case VHOST_SET_VRING_BASE:
> @@ -1581,6 +1855,7 @@ long vhost_vring_ioctl(struct vhost_dev *d, unsigned int ioctl, void __user *arg
> vq->avail = (void __user *)(unsigned long)a.avail_user_addr;
> vq->log_addr = a.log_guest_addr;
> vq->used = (void __user *)(unsigned long)a.used_user_addr;
> + vhost_uninit_vq_vmaps(vq);
> break;
> case VHOST_SET_VRING_KICK:
> if (copy_from_user(&f, argp, sizeof f)) {
> @@ -1656,6 +1931,8 @@ long vhost_vring_ioctl(struct vhost_dev *d, unsigned int ioctl, void __user *arg
> if (pollstart && vq->handle_kick)
> r = vhost_poll_start(&vq->poll, vq->kick);
>
> + if (d->mm)
> + mmu_notifier_register(&d->mmu_notifier, d->mm);
> mutex_unlock(&vq->mutex);
>
> if (pollstop && vq->handle_kick)
> diff --git a/drivers/vhost/vhost.h b/drivers/vhost/vhost.h
> index 7a7fc00..146076e 100644
> --- a/drivers/vhost/vhost.h
> +++ b/drivers/vhost/vhost.h
> @@ -12,6 +12,8 @@
> #include <linux/virtio_config.h>
> #include <linux/virtio_ring.h>
> #include <linux/atomic.h>
> +#include <linux/pagemap.h>
> +#include <linux/mmu_notifier.h>
>
> struct vhost_work;
> typedef void (*vhost_work_fn_t)(struct vhost_work *work);
> @@ -80,6 +82,13 @@ enum vhost_uaddr_type {
> VHOST_NUM_ADDRS = 3,
> };
>
> +struct vhost_vmap {
> + void *addr;
> + void *unmap_addr;
> + int npages;
> + struct page **pages;
> +};
> +
> /* The virtqueue structure describes a queue attached to a device. */
> struct vhost_virtqueue {
> struct vhost_dev *dev;
> @@ -90,6 +99,11 @@ struct vhost_virtqueue {
> struct vring_desc __user *desc;
> struct vring_avail __user *avail;
> struct vring_used __user *used;
> +
> + struct vhost_vmap avail_ring;
> + struct vhost_vmap desc_ring;
> + struct vhost_vmap used_ring;
> +
> const struct vhost_umem_node *meta_iotlb[VHOST_NUM_ADDRS];
> struct file *kick;
> struct eventfd_ctx *call_ctx;
> @@ -158,6 +172,7 @@ struct vhost_msg_node {
>
> struct vhost_dev {
> struct mm_struct *mm;
> + struct mmu_notifier mmu_notifier;
> struct mutex mutex;
> struct vhost_virtqueue **vqs;
> int nvqs;
> @@ -210,6 +225,7 @@ int vhost_log_write(struct vhost_virtqueue *vq, struct vhost_log *log,
> unsigned int log_num, u64 len,
> struct iovec *iov, int count);
> int vq_meta_prefetch(struct vhost_virtqueue *vq);
> +void vq_meta_prefetch_done(struct vhost_virtqueue *vq);
>
> struct vhost_msg_node *vhost_new_msg(struct vhost_virtqueue *vq, int type);
> void vhost_enqueue_msg(struct vhost_dev *dev,
> --
> 1.8.3.1

2019-03-06 20:54:19

by Souptick Joarder

[permalink] [raw]
Subject: Re: [RFC PATCH V2 4/5] vhost: introduce helpers to get the size of metadata area

On Wed, Mar 6, 2019 at 12:48 PM Jason Wang <[email protected]> wrote:
>
> Signed-off-by: Jason Wang <[email protected]>

Is the change log left with any particular reason ?
> ---
> drivers/vhost/vhost.c | 46 ++++++++++++++++++++++++++++------------------
> 1 file changed, 28 insertions(+), 18 deletions(-)
>
> diff --git a/drivers/vhost/vhost.c b/drivers/vhost/vhost.c
> index 2025543..1015464 100644
> --- a/drivers/vhost/vhost.c
> +++ b/drivers/vhost/vhost.c
> @@ -413,6 +413,27 @@ static void vhost_dev_free_iovecs(struct vhost_dev *dev)
> vhost_vq_free_iovecs(dev->vqs[i]);
> }
>
> +static size_t vhost_get_avail_size(struct vhost_virtqueue *vq, int num)
> +{
> + size_t event = vhost_has_feature(vq, VIRTIO_RING_F_EVENT_IDX) ? 2 : 0;
> +
> + return sizeof(*vq->avail) +
> + sizeof(*vq->avail->ring) * num + event;
> +}
> +
> +static size_t vhost_get_used_size(struct vhost_virtqueue *vq, int num)
> +{
> + size_t event = vhost_has_feature(vq, VIRTIO_RING_F_EVENT_IDX) ? 2 : 0;
> +
> + return sizeof(*vq->used) +
> + sizeof(*vq->used->ring) * num + event;
> +}
> +
> +static size_t vhost_get_desc_size(struct vhost_virtqueue *vq, int num)
> +{
> + return sizeof(*vq->desc) * num;
> +}
> +
> void vhost_dev_init(struct vhost_dev *dev,
> struct vhost_virtqueue **vqs, int nvqs, int iov_limit)
> {
> @@ -1253,13 +1274,9 @@ static bool vq_access_ok(struct vhost_virtqueue *vq, unsigned int num,
> struct vring_used __user *used)
>
> {
> - size_t s = vhost_has_feature(vq, VIRTIO_RING_F_EVENT_IDX) ? 2 : 0;
> -
> - return access_ok(desc, num * sizeof *desc) &&
> - access_ok(avail,
> - sizeof *avail + num * sizeof *avail->ring + s) &&
> - access_ok(used,
> - sizeof *used + num * sizeof *used->ring + s);
> + return access_ok(desc, vhost_get_desc_size(vq, num)) &&
> + access_ok(avail, vhost_get_avail_size(vq, num)) &&
> + access_ok(used, vhost_get_used_size(vq, num));
> }
>
> static void vhost_vq_meta_update(struct vhost_virtqueue *vq,
> @@ -1311,22 +1328,18 @@ static bool iotlb_access_ok(struct vhost_virtqueue *vq,
>
> int vq_meta_prefetch(struct vhost_virtqueue *vq)
> {
> - size_t s = vhost_has_feature(vq, VIRTIO_RING_F_EVENT_IDX) ? 2 : 0;
> unsigned int num = vq->num;
>
> if (!vq->iotlb)
> return 1;
>
> return iotlb_access_ok(vq, VHOST_ACCESS_RO, (u64)(uintptr_t)vq->desc,
> - num * sizeof(*vq->desc), VHOST_ADDR_DESC) &&
> + vhost_get_desc_size(vq, num), VHOST_ADDR_DESC) &&
> iotlb_access_ok(vq, VHOST_ACCESS_RO, (u64)(uintptr_t)vq->avail,
> - sizeof *vq->avail +
> - num * sizeof(*vq->avail->ring) + s,
> + vhost_get_avail_size(vq, num),
> VHOST_ADDR_AVAIL) &&
> iotlb_access_ok(vq, VHOST_ACCESS_WO, (u64)(uintptr_t)vq->used,
> - sizeof *vq->used +
> - num * sizeof(*vq->used->ring) + s,
> - VHOST_ADDR_USED);
> + vhost_get_used_size(vq, num), VHOST_ADDR_USED);
> }
> EXPORT_SYMBOL_GPL(vq_meta_prefetch);
>
> @@ -1343,13 +1356,10 @@ bool vhost_log_access_ok(struct vhost_dev *dev)
> static bool vq_log_access_ok(struct vhost_virtqueue *vq,
> void __user *log_base)
> {
> - size_t s = vhost_has_feature(vq, VIRTIO_RING_F_EVENT_IDX) ? 2 : 0;
> -
> return vq_memory_access_ok(log_base, vq->umem,
> vhost_has_feature(vq, VHOST_F_LOG_ALL)) &&
> (!vq->log_used || log_access_ok(log_base, vq->log_addr,
> - sizeof *vq->used +
> - vq->num * sizeof *vq->used->ring + s));
> + vhost_get_used_size(vq, vq->num)));
> }
>
> /* Can we start vq? */
> --
> 1.8.3.1
>

2019-03-07 02:39:18

by Jason Wang

[permalink] [raw]
Subject: Re: [RFC PATCH V2 2/5] vhost: fine grain userspace memory accessors


On 2019/3/6 下午6:45, Christophe de Dinechin wrote:
>
>> On 6 Mar 2019, at 08:18, Jason Wang <[email protected]> wrote:
>>
>> This is used to hide the metadata address from virtqueue helpers. This
>> will allow to implement a vmap based fast accessing to metadata.
>>
>> Signed-off-by: Jason Wang <[email protected]>
>> ---
>> drivers/vhost/vhost.c | 94 +++++++++++++++++++++++++++++++++++++++++----------
>> 1 file changed, 77 insertions(+), 17 deletions(-)
>>
>> diff --git a/drivers/vhost/vhost.c b/drivers/vhost/vhost.c
>> index 400aa78..29709e7 100644
>> --- a/drivers/vhost/vhost.c
>> +++ b/drivers/vhost/vhost.c
>> @@ -869,6 +869,34 @@ static inline void __user *__vhost_get_user(struct vhost_virtqueue *vq,
>> ret; \
>> })
>>
>> +static inline int vhost_put_avail_event(struct vhost_virtqueue *vq)
>> +{
>> + return vhost_put_user(vq, cpu_to_vhost16(vq, vq->avail_idx),
>> + vhost_avail_event(vq));
>> +}
>> +
>> +static inline int vhost_put_used(struct vhost_virtqueue *vq,
>> + struct vring_used_elem *head, int idx,
>> + int count)
>> +{
>> + return vhost_copy_to_user(vq, vq->used->ring + idx, head,
>> + count * sizeof(*head));
>> +}
>> +
>> +static inline int vhost_put_used_flags(struct vhost_virtqueue *vq)
>> +
>> +{
>> + return vhost_put_user(vq, cpu_to_vhost16(vq, vq->used_flags),
>> + &vq->used->flags);
>> +}
>> +
>> +static inline int vhost_put_used_idx(struct vhost_virtqueue *vq)
>> +
>> +{
>> + return vhost_put_user(vq, cpu_to_vhost16(vq, vq->last_used_idx),
>> + &vq->used->idx);
>> +}
>> +
>> #define vhost_get_user(vq, x, ptr, type) \
>> ({ \
>> int ret; \
>> @@ -907,6 +935,43 @@ static void vhost_dev_unlock_vqs(struct vhost_dev *d)
>> mutex_unlock(&d->vqs[i]->mutex);
>> }
>>
>> +static inline int vhost_get_avail_idx(struct vhost_virtqueue *vq,
>> + __virtio16 *idx)
>> +{
>> + return vhost_get_avail(vq, *idx, &vq->avail->idx);
>> +}
>> +
>> +static inline int vhost_get_avail_head(struct vhost_virtqueue *vq,
>> + __virtio16 *head, int idx)
>> +{
>> + return vhost_get_avail(vq, *head,
>> + &vq->avail->ring[idx & (vq->num - 1)]);
>> +}
>> +
>> +static inline int vhost_get_avail_flags(struct vhost_virtqueue *vq,
>> + __virtio16 *flags)
>> +{
>> + return vhost_get_avail(vq, *flags, &vq->avail->flags);
>> +}
>> +
>> +static inline int vhost_get_used_event(struct vhost_virtqueue *vq,
>> + __virtio16 *event)
>> +{
>> + return vhost_get_avail(vq, *event, vhost_used_event(vq));
>> +}
>> +
>> +static inline int vhost_get_used_idx(struct vhost_virtqueue *vq,
>> + __virtio16 *idx)
>> +{
>> + return vhost_get_used(vq, *idx, &vq->used->idx);
>> +}
>> +
>> +static inline int vhost_get_desc(struct vhost_virtqueue *vq,
>> + struct vring_desc *desc, int idx)
>> +{
>> + return vhost_copy_from_user(vq, desc, vq->desc + idx, sizeof(*desc));
>> +}
>> +
>> static int vhost_new_umem_range(struct vhost_umem *umem,
>> u64 start, u64 size, u64 end,
>> u64 userspace_addr, int perm)
>> @@ -1840,8 +1905,7 @@ int vhost_log_write(struct vhost_virtqueue *vq, struct vhost_log *log,
>> static int vhost_update_used_flags(struct vhost_virtqueue *vq)
>> {
>> void __user *used;
>> - if (vhost_put_user(vq, cpu_to_vhost16(vq, vq->used_flags),
>> - &vq->used->flags) < 0)
>> + if (vhost_put_used_flags(vq))
>> return -EFAULT;
>> if (unlikely(vq->log_used)) {
>> /* Make sure the flag is seen before log. */
>> @@ -1858,8 +1922,7 @@ static int vhost_update_used_flags(struct vhost_virtqueue *vq)
>>
>> static int vhost_update_avail_event(struct vhost_virtqueue *vq, u16 avail_event)
>> {
>> - if (vhost_put_user(vq, cpu_to_vhost16(vq, vq->avail_idx),
>> - vhost_avail_event(vq)))
>> + if (vhost_put_avail_event(vq))
>> return -EFAULT;
>> if (unlikely(vq->log_used)) {
>> void __user *used;
>> @@ -1895,7 +1958,7 @@ int vhost_vq_init_access(struct vhost_virtqueue *vq)
>> r = -EFAULT;
>> goto err;
>> }
>> - r = vhost_get_used(vq, last_used_idx, &vq->used->idx);
>> + r = vhost_get_used_idx(vq, &last_used_idx);
>> if (r) {
>> vq_err(vq, "Can't access used idx at %p\n",
>> &vq->used->idx);
> From the error case, it looks like you are not entirely encapsulating
> knowledge of what the accessor uses, i.e. it’s not:
>
> vq_err(vq, "Can't access used idx at %p\n",
> &last_user_idx);
>
> Maybe move error message within accessor?


Good catch. Will fix but I still prefer to keep the place of vq_err().
Moving error message (if needed) could be done in the future.

Thanks


>
>> @@ -2094,7 +2157,7 @@ int vhost_get_vq_desc(struct vhost_virtqueue *vq,
>> last_avail_idx = vq->last_avail_idx;
>>
>> if (vq->avail_idx == vq->last_avail_idx) {
>> - if (unlikely(vhost_get_avail(vq, avail_idx, &vq->avail->idx))) {
>> + if (unlikely(vhost_get_avail_idx(vq, &avail_idx))) {
>> vq_err(vq, "Failed to access avail idx at %p\n",
>> &vq->avail->idx);
>> return -EFAULT;
> Same here.
>
>> @@ -2121,8 +2184,7 @@ int vhost_get_vq_desc(struct vhost_virtqueue *vq,
>>
>> /* Grab the next descriptor number they're advertising, and increment
>> * the index we've seen. */
>> - if (unlikely(vhost_get_avail(vq, ring_head,
>> - &vq->avail->ring[last_avail_idx & (vq->num - 1)]))) {
>> + if (unlikely(vhost_get_avail_head(vq, &ring_head, last_avail_idx))) {
>> vq_err(vq, "Failed to read head: idx %d address %p\n",
>> last_avail_idx,
>> &vq->avail->ring[last_avail_idx % vq->num]);
>> @@ -2157,8 +2219,7 @@ int vhost_get_vq_desc(struct vhost_virtqueue *vq,
>> i, vq->num, head);
>> return -EINVAL;
>> }
>> - ret = vhost_copy_from_user(vq, &desc, vq->desc + i,
>> - sizeof desc);
>> + ret = vhost_get_desc(vq, &desc, i);
>> if (unlikely(ret)) {
>> vq_err(vq, "Failed to get descriptor: idx %d addr %p\n",
>> i, vq->desc + i);
>> @@ -2251,7 +2312,7 @@ static int __vhost_add_used_n(struct vhost_virtqueue *vq,
>>
>> start = vq->last_used_idx & (vq->num - 1);
>> used = vq->used->ring + start;
>> - if (vhost_copy_to_user(vq, used, heads, count * sizeof *used)) {
>> + if (vhost_put_used(vq, heads, start, count)) {
>> vq_err(vq, "Failed to write used");
>> return -EFAULT;
>> }
>> @@ -2293,8 +2354,7 @@ int vhost_add_used_n(struct vhost_virtqueue *vq, struct vring_used_elem *heads,
>>
>> /* Make sure buffer is written before we update index. */
>> smp_wmb();
>> - if (vhost_put_user(vq, cpu_to_vhost16(vq, vq->last_used_idx),
>> - &vq->used->idx)) {
>> + if (vhost_put_used_idx(vq)) {
>> vq_err(vq, "Failed to increment used idx");
>> return -EFAULT;
>> }
>> @@ -2327,7 +2387,7 @@ static bool vhost_notify(struct vhost_dev *dev, struct vhost_virtqueue *vq)
>>
>> if (!vhost_has_feature(vq, VIRTIO_RING_F_EVENT_IDX)) {
>> __virtio16 flags;
>> - if (vhost_get_avail(vq, flags, &vq->avail->flags)) {
>> + if (vhost_get_avail_flags(vq, &flags)) {
>> vq_err(vq, "Failed to get flags");
>> return true;
>> }
>> @@ -2341,7 +2401,7 @@ static bool vhost_notify(struct vhost_dev *dev, struct vhost_virtqueue *vq)
>> if (unlikely(!v))
>> return true;
>>
>> - if (vhost_get_avail(vq, event, vhost_used_event(vq))) {
>> + if (vhost_get_used_event(vq, &event)) {
>> vq_err(vq, "Failed to get used event idx");
>> return true;
>> }
>> @@ -2386,7 +2446,7 @@ bool vhost_vq_avail_empty(struct vhost_dev *dev, struct vhost_virtqueue *vq)
>> if (vq->avail_idx != vq->last_avail_idx)
>> return false;
>>
>> - r = vhost_get_avail(vq, avail_idx, &vq->avail->idx);
>> + r = vhost_get_avail_idx(vq, &avail_idx);
>> if (unlikely(r))
>> return false;
>> vq->avail_idx = vhost16_to_cpu(vq, avail_idx);
>> @@ -2422,7 +2482,7 @@ bool vhost_enable_notify(struct vhost_dev *dev, struct vhost_virtqueue *vq)
>> /* They could have slipped one in as we were doing that: make
>> * sure it's written, then check again. */
>> smp_mb();
>> - r = vhost_get_avail(vq, avail_idx, &vq->avail->idx);
>> + r = vhost_get_avail_idx(vq, &avail_idx);
>> if (r) {
>> vq_err(vq, "Failed to check avail idx at %p: %d\n",
>> &vq->avail->idx, r);
>> --
>> 1.8.3.1
>>

2019-03-07 02:41:11

by Jason Wang

[permalink] [raw]
Subject: Re: [RFC PATCH V2 4/5] vhost: introduce helpers to get the size of metadata area


On 2019/3/6 下午6:56, Christophe de Dinechin wrote:
>> On 6 Mar 2019, at 08:18, Jason Wang<[email protected]> wrote:
>>
>> Signed-off-by: Jason Wang<[email protected]>
>> ---
>> drivers/vhost/vhost.c | 46 ++++++++++++++++++++++++++++------------------
>> 1 file changed, 28 insertions(+), 18 deletions(-)
>>
>> diff --git a/drivers/vhost/vhost.c b/drivers/vhost/vhost.c
>> index 2025543..1015464 100644
>> --- a/drivers/vhost/vhost.c
>> +++ b/drivers/vhost/vhost.c
>> @@ -413,6 +413,27 @@ static void vhost_dev_free_iovecs(struct vhost_dev *dev)
>> vhost_vq_free_iovecs(dev->vqs[i]);
>> }
>>
>> +static size_t vhost_get_avail_size(struct vhost_virtqueue *vq, int num)
> Nit: Any reason not to make `num` unsigned or size_t?
>

Let me use unsigned int to match the definition of vq->num.

Thanks



2019-03-07 02:43:42

by Jason Wang

[permalink] [raw]
Subject: Re: [RFC PATCH V2 4/5] vhost: introduce helpers to get the size of metadata area


On 2019/3/7 上午2:43, Souptick Joarder wrote:
> On Wed, Mar 6, 2019 at 12:48 PM Jason Wang <[email protected]> wrote:
>> Signed-off-by: Jason Wang <[email protected]>
> Is the change log left with any particular reason ?


Nope, will add the log.

Thanks


>> ---
>> drivers/vhost/vhost.c | 46 ++++++++++++++++++++++++++++------------------
>> 1 file changed, 28 insertions(+), 18 deletions(-)
>>
>> diff --git a/drivers/vhost/vhost.c b/drivers/vhost/vhost.c
>> index 2025543..1015464 100644
>> --- a/drivers/vhost/vhost.c
>> +++ b/drivers/vhost/vhost.c
>> @@ -413,6 +413,27 @@ static void vhost_dev_free_iovecs(struct vhost_dev *dev)
>> vhost_vq_free_iovecs(dev->vqs[i]);
>> }
>>
>> +static size_t vhost_get_avail_size(struct vhost_virtqueue *vq, int num)
>> +{
>> + size_t event = vhost_has_feature(vq, VIRTIO_RING_F_EVENT_IDX) ? 2 : 0;
>> +
>> + return sizeof(*vq->avail) +
>> + sizeof(*vq->avail->ring) * num + event;
>> +}
>> +
>> +static size_t vhost_get_used_size(struct vhost_virtqueue *vq, int num)
>> +{
>> + size_t event = vhost_has_feature(vq, VIRTIO_RING_F_EVENT_IDX) ? 2 : 0;
>> +
>> + return sizeof(*vq->used) +
>> + sizeof(*vq->used->ring) * num + event;
>> +}
>> +
>> +static size_t vhost_get_desc_size(struct vhost_virtqueue *vq, int num)
>> +{
>> + return sizeof(*vq->desc) * num;
>> +}
>> +
>> void vhost_dev_init(struct vhost_dev *dev,
>> struct vhost_virtqueue **vqs, int nvqs, int iov_limit)
>> {
>> @@ -1253,13 +1274,9 @@ static bool vq_access_ok(struct vhost_virtqueue *vq, unsigned int num,
>> struct vring_used __user *used)
>>
>> {
>> - size_t s = vhost_has_feature(vq, VIRTIO_RING_F_EVENT_IDX) ? 2 : 0;
>> -
>> - return access_ok(desc, num * sizeof *desc) &&
>> - access_ok(avail,
>> - sizeof *avail + num * sizeof *avail->ring + s) &&
>> - access_ok(used,
>> - sizeof *used + num * sizeof *used->ring + s);
>> + return access_ok(desc, vhost_get_desc_size(vq, num)) &&
>> + access_ok(avail, vhost_get_avail_size(vq, num)) &&
>> + access_ok(used, vhost_get_used_size(vq, num));
>> }
>>
>> static void vhost_vq_meta_update(struct vhost_virtqueue *vq,
>> @@ -1311,22 +1328,18 @@ static bool iotlb_access_ok(struct vhost_virtqueue *vq,
>>
>> int vq_meta_prefetch(struct vhost_virtqueue *vq)
>> {
>> - size_t s = vhost_has_feature(vq, VIRTIO_RING_F_EVENT_IDX) ? 2 : 0;
>> unsigned int num = vq->num;
>>
>> if (!vq->iotlb)
>> return 1;
>>
>> return iotlb_access_ok(vq, VHOST_ACCESS_RO, (u64)(uintptr_t)vq->desc,
>> - num * sizeof(*vq->desc), VHOST_ADDR_DESC) &&
>> + vhost_get_desc_size(vq, num), VHOST_ADDR_DESC) &&
>> iotlb_access_ok(vq, VHOST_ACCESS_RO, (u64)(uintptr_t)vq->avail,
>> - sizeof *vq->avail +
>> - num * sizeof(*vq->avail->ring) + s,
>> + vhost_get_avail_size(vq, num),
>> VHOST_ADDR_AVAIL) &&
>> iotlb_access_ok(vq, VHOST_ACCESS_WO, (u64)(uintptr_t)vq->used,
>> - sizeof *vq->used +
>> - num * sizeof(*vq->used->ring) + s,
>> - VHOST_ADDR_USED);
>> + vhost_get_used_size(vq, num), VHOST_ADDR_USED);
>> }
>> EXPORT_SYMBOL_GPL(vq_meta_prefetch);
>>
>> @@ -1343,13 +1356,10 @@ bool vhost_log_access_ok(struct vhost_dev *dev)
>> static bool vq_log_access_ok(struct vhost_virtqueue *vq,
>> void __user *log_base)
>> {
>> - size_t s = vhost_has_feature(vq, VIRTIO_RING_F_EVENT_IDX) ? 2 : 0;
>> -
>> return vq_memory_access_ok(log_base, vq->umem,
>> vhost_has_feature(vq, VHOST_F_LOG_ALL)) &&
>> (!vq->log_used || log_access_ok(log_base, vq->log_addr,
>> - sizeof *vq->used +
>> - vq->num * sizeof *vq->used->ring + s));
>> + vhost_get_used_size(vq, vq->num)));
>> }
>>
>> /* Can we start vq? */
>> --
>> 1.8.3.1
>>

2019-03-07 02:46:46

by Jason Wang

[permalink] [raw]
Subject: Re: [RFC PATCH V2 5/5] vhost: access vq metadata through kernel virtual address


On 2019/3/7 上午12:31, Michael S. Tsirkin wrote:
>> +static void vhost_set_vmap_dirty(struct vhost_vmap *used)
>> +{
>> + int i;
>> +
>> + for (i = 0; i < used->npages; i++)
>> + set_page_dirty_lock(used->pages[i]);
> This seems to rely on page lock to mark page dirty.
>
> Could it happen that page writeback will check the
> page, find it clean, and then you mark it dirty and then
> invalidate callback is called?
>
>

Yes. But does this break anything? The page is still there, we just
remove a kernel mapping to it.

Thanks


2019-03-07 15:35:30

by Michael S. Tsirkin

[permalink] [raw]
Subject: Re: [RFC PATCH V2 5/5] vhost: access vq metadata through kernel virtual address

On Thu, Mar 07, 2019 at 10:45:57AM +0800, Jason Wang wrote:
>
> On 2019/3/7 上午12:31, Michael S. Tsirkin wrote:
> > > +static void vhost_set_vmap_dirty(struct vhost_vmap *used)
> > > +{
> > > + int i;
> > > +
> > > + for (i = 0; i < used->npages; i++)
> > > + set_page_dirty_lock(used->pages[i]);
> > This seems to rely on page lock to mark page dirty.
> >
> > Could it happen that page writeback will check the
> > page, find it clean, and then you mark it dirty and then
> > invalidate callback is called?
> >
> >
>
> Yes. But does this break anything?
> The page is still there, we just remove a
> kernel mapping to it.
>
> Thanks

Yes it's the same problem as e.g. RDMA:
we've just marked the page as dirty without having buffers.
Eventually writeback will find it and filesystem will complain...
So if the pages are backed by a non-RAM-based filesystem, it’s all just broken.

one can hope that RDMA guys will fix it in some way eventually.
For now, maybe add a flag in e.g. VMA that says that there's no
writeback so it's safe to mark page dirty at any point?





--
MST

2019-03-07 15:48:05

by Michael S. Tsirkin

[permalink] [raw]
Subject: Re: [RFC PATCH V2 5/5] vhost: access vq metadata through kernel virtual address

On Wed, Mar 06, 2019 at 02:18:12AM -0500, Jason Wang wrote:
> +static const struct mmu_notifier_ops vhost_mmu_notifier_ops = {
> + .invalidate_range = vhost_invalidate_range,
> +};
> +
> void vhost_dev_init(struct vhost_dev *dev,
> struct vhost_virtqueue **vqs, int nvqs, int iov_limit)
> {

I also wonder here: when page is write protected then
it does not look like .invalidate_range is invoked.

E.g. mm/ksm.c calls

mmu_notifier_invalidate_range_start and
mmu_notifier_invalidate_range_end but not mmu_notifier_invalidate_range.

Similarly, rmap in page_mkclean_one will not call
mmu_notifier_invalidate_range.

If I'm right vhost won't get notified when page is write-protected since you
didn't install start/end notifiers. Note that end notifier can be called
with page locked, so it's not as straight-forward as just adding a call.
Writing into a write-protected page isn't a good idea.

Note that documentation says:
it is fine to delay the mmu_notifier_invalidate_range
call to mmu_notifier_invalidate_range_end() outside the page table lock.
implying it's called just later.

--
MST

2019-03-07 17:57:39

by Michael S. Tsirkin

[permalink] [raw]
Subject: Re: [RFC PATCH V2 5/5] vhost: access vq metadata through kernel virtual address

On Thu, Mar 07, 2019 at 10:47:22AM -0500, Michael S. Tsirkin wrote:
> On Wed, Mar 06, 2019 at 02:18:12AM -0500, Jason Wang wrote:
> > +static const struct mmu_notifier_ops vhost_mmu_notifier_ops = {
> > + .invalidate_range = vhost_invalidate_range,
> > +};
> > +
> > void vhost_dev_init(struct vhost_dev *dev,
> > struct vhost_virtqueue **vqs, int nvqs, int iov_limit)
> > {
>
> I also wonder here: when page is write protected then
> it does not look like .invalidate_range is invoked.
>
> E.g. mm/ksm.c calls
>
> mmu_notifier_invalidate_range_start and
> mmu_notifier_invalidate_range_end but not mmu_notifier_invalidate_range.
>
> Similarly, rmap in page_mkclean_one will not call
> mmu_notifier_invalidate_range.
>
> If I'm right vhost won't get notified when page is write-protected since you
> didn't install start/end notifiers. Note that end notifier can be called
> with page locked, so it's not as straight-forward as just adding a call.
> Writing into a write-protected page isn't a good idea.
>
> Note that documentation says:
> it is fine to delay the mmu_notifier_invalidate_range
> call to mmu_notifier_invalidate_range_end() outside the page table lock.
> implying it's called just later.

OK I missed the fact that _end actually calls
mmu_notifier_invalidate_range internally. So that part is fine but the
fact that you are trying to take page lock under VQ mutex and take same
mutex within notifier probably means it's broken for ksm and rmap at
least since these call invalidate with lock taken.

And generally, Andrea told me offline one can not take mutex under
the notifier callback. I CC'd Andrea for why.

That's a separate issue from set_page_dirty when memory is file backed.

It's because of all these issues that I preferred just accessing
userspace memory and handling faults. Unfortunately there does not
appear to exist an API that whitelists a specific driver along the lines
of "I checked this code for speculative info leaks, don't add barriers
on data path please".


> --
> MST

2019-03-07 19:10:27

by Jerome Glisse

[permalink] [raw]
Subject: Re: [RFC PATCH V2 5/5] vhost: access vq metadata through kernel virtual address

On Thu, Mar 07, 2019 at 10:34:39AM -0500, Michael S. Tsirkin wrote:
> On Thu, Mar 07, 2019 at 10:45:57AM +0800, Jason Wang wrote:
> >
> > On 2019/3/7 上午12:31, Michael S. Tsirkin wrote:
> > > > +static void vhost_set_vmap_dirty(struct vhost_vmap *used)
> > > > +{
> > > > + int i;
> > > > +
> > > > + for (i = 0; i < used->npages; i++)
> > > > + set_page_dirty_lock(used->pages[i]);
> > > This seems to rely on page lock to mark page dirty.
> > >
> > > Could it happen that page writeback will check the
> > > page, find it clean, and then you mark it dirty and then
> > > invalidate callback is called?
> > >
> > >
> >
> > Yes. But does this break anything?
> > The page is still there, we just remove a
> > kernel mapping to it.
> >
> > Thanks
>
> Yes it's the same problem as e.g. RDMA:
> we've just marked the page as dirty without having buffers.
> Eventually writeback will find it and filesystem will complain...
> So if the pages are backed by a non-RAM-based filesystem, it’s all just broken.
>
> one can hope that RDMA guys will fix it in some way eventually.
> For now, maybe add a flag in e.g. VMA that says that there's no
> writeback so it's safe to mark page dirty at any point?

I thought this patch was only for anonymous memory ie not file back ?
If so then set dirty is mostly useless it would only be use for swap
but for this you can use an unlock version to set the page dirty.

Cheers,
Jérôme

2019-03-07 19:17:09

by Andrea Arcangeli

[permalink] [raw]
Subject: Re: [RFC PATCH V2 5/5] vhost: access vq metadata through kernel virtual address

On Thu, Mar 07, 2019 at 12:56:45PM -0500, Michael S. Tsirkin wrote:
> On Thu, Mar 07, 2019 at 10:47:22AM -0500, Michael S. Tsirkin wrote:
> > On Wed, Mar 06, 2019 at 02:18:12AM -0500, Jason Wang wrote:
> > > +static const struct mmu_notifier_ops vhost_mmu_notifier_ops = {
> > > + .invalidate_range = vhost_invalidate_range,
> > > +};
> > > +
> > > void vhost_dev_init(struct vhost_dev *dev,
> > > struct vhost_virtqueue **vqs, int nvqs, int iov_limit)
> > > {
> >
> > I also wonder here: when page is write protected then
> > it does not look like .invalidate_range is invoked.
> >
> > E.g. mm/ksm.c calls
> >
> > mmu_notifier_invalidate_range_start and
> > mmu_notifier_invalidate_range_end but not mmu_notifier_invalidate_range.
> >
> > Similarly, rmap in page_mkclean_one will not call
> > mmu_notifier_invalidate_range.
> >
> > If I'm right vhost won't get notified when page is write-protected since you
> > didn't install start/end notifiers. Note that end notifier can be called
> > with page locked, so it's not as straight-forward as just adding a call.
> > Writing into a write-protected page isn't a good idea.
> >
> > Note that documentation says:
> > it is fine to delay the mmu_notifier_invalidate_range
> > call to mmu_notifier_invalidate_range_end() outside the page table lock.
> > implying it's called just later.
>
> OK I missed the fact that _end actually calls
> mmu_notifier_invalidate_range internally. So that part is fine but the
> fact that you are trying to take page lock under VQ mutex and take same
> mutex within notifier probably means it's broken for ksm and rmap at
> least since these call invalidate with lock taken.

Yes this lock inversion needs more thoughts.

> And generally, Andrea told me offline one can not take mutex under
> the notifier callback. I CC'd Andrea for why.

Yes, the problem then is the ->invalidate_page is called then under PT
lock so it cannot take mutex, you also cannot take the page_lock, it
can at most take a spinlock or trylock_page.

So it must switch back to the _start/_end methods unless you rewrite
the locking.

The difference with _start/_end, is that ->invalidate_range avoids the
_start callback basically, but to avoid the _start callback safely, it
has to be called in between the ptep_clear_flush and the set_pte_at
whenever the pfn changes like during a COW. So it cannot be coalesced
in a single TLB flush that invalidates all sptes in a range like we
prefer for performance reasons for example in KVM. It also cannot
sleep.

In short ->invalidate_range must be really fast (it shouldn't require
to send IPI to all other CPUs like KVM may require during an
invalidate_range_start) and it must not sleep, in order to prefer it
to _start/_end.

I.e. the invalidate of the secondary MMU that walks the linux
pagetables in hardware (in vhost case with GUP in software) has to
happen while the linux pagetable is zero, otherwise a concurrent
hardware pagetable lookup could re-instantiate a mapping to the old
page in between the set_pte_at and the invalidate_range_end (which
internally calls ->invalidate_range). Jerome documented it nicely in
Documentation/vm/mmu_notifier.rst .

Now you don't really walk the pagetable in hardware in vhost, but if
you use gup_fast after usemm() it's similar.

For vhost the invalidate would be really fast, there are no IPI to
deliver at all, the problem is just the mutex.

> That's a separate issue from set_page_dirty when memory is file backed.

Yes. I don't yet know why the ext4 internal __writepage cannot
re-create the bh if they've been freed by the VM and why such race
where the bh are freed for a pinned VM_SHARED ext4 page doesn't even
exist for transient pins like O_DIRECT (does it work by luck?), but
with mmu notifiers there are no long term pins anyway, so this works
normally and it's like the memory isn't pinned. In any case I think
that's a kernel bug in either __writepage or try_to_free_buffers, so I
would ignore it considering qemu will only use anon memory or tmpfs or
hugetlbfs as backing store for the virtio ring. It wouldn't make sense
for qemu to risk triggering I/O on a VM_SHARED ext4, so we shouldn't
be even exposed to what seems to be an orthogonal kernel bug.

I suppose whatever solution will fix the set_page_dirty_lock on
VM_SHARED ext4 for the other places that don't or can't use mmu
notifiers, will then work for vhost too which uses mmu notifiers and
will be less affected from the start if something.

Reading the lwn link about the discussion about the long term GUP pin
from Jan vs set_page_dirty_lock: I can only agree with the last part
where Jerome correctly pointed out at the end that mellanox RDMA got
it right by avoiding completely long term pins by using mmu notifier
and in general mmu notifier is the standard solution to avoid long
term pins. Nothing should ever take long term GUP pins, if it does it
means software is bad or the hardware lacks features to support on
demand paging. Still I don't get why transient pins like O_DIRECT
where mmu notifier would be prohibitive to use (registering into mmu
notifier cannot be done at high frequency, the locking to do so is
massive) cannot end up into the same ext4 _writepage crash as long
term pins: long term or short term transient is a subjective measure
from VM standpoint, the VM won't know the difference, luck will
instead.

> It's because of all these issues that I preferred just accessing
> userspace memory and handling faults. Unfortunately there does not
> appear to exist an API that whitelists a specific driver along the lines
> of "I checked this code for speculative info leaks, don't add barriers
> on data path please".

Yes that's unfortunate, __uaccess_begin_nospec() is now making
prohibitive to frequently access userland code.

I doubt we can do like access_ok() and only check it once. access_ok
checks the virtual address, and if the virtual address is ok doesn't
wrap around and it points to userland in a safe range, it's always
ok. There's no need to run access_ok again if we keep hitting on the
very same address.

__uaccess_begin_nospec() instead is about runtime stuff that can
change the moment copy-user has completed even before returning to
userland, so there's no easy way to do it just once.

On top of skipping the __uaccess_begin_nospec(), the mmu notifier soft
vhost design will further boost the performance by guaranteeing the
use of gigapages TLBs when available (or 2M TLBs worst case) even if
QEMU runs on smaller pages.

Thanks,
Andrea

2019-03-07 19:18:12

by Jerome Glisse

[permalink] [raw]
Subject: Re: [RFC PATCH V2 5/5] vhost: access vq metadata through kernel virtual address

On Thu, Mar 07, 2019 at 12:56:45PM -0500, Michael S. Tsirkin wrote:
> On Thu, Mar 07, 2019 at 10:47:22AM -0500, Michael S. Tsirkin wrote:
> > On Wed, Mar 06, 2019 at 02:18:12AM -0500, Jason Wang wrote:
> > > +static const struct mmu_notifier_ops vhost_mmu_notifier_ops = {
> > > + .invalidate_range = vhost_invalidate_range,
> > > +};
> > > +
> > > void vhost_dev_init(struct vhost_dev *dev,
> > > struct vhost_virtqueue **vqs, int nvqs, int iov_limit)
> > > {
> >
> > I also wonder here: when page is write protected then
> > it does not look like .invalidate_range is invoked.
> >
> > E.g. mm/ksm.c calls
> >
> > mmu_notifier_invalidate_range_start and
> > mmu_notifier_invalidate_range_end but not mmu_notifier_invalidate_range.
> >
> > Similarly, rmap in page_mkclean_one will not call
> > mmu_notifier_invalidate_range.
> >
> > If I'm right vhost won't get notified when page is write-protected since you
> > didn't install start/end notifiers. Note that end notifier can be called
> > with page locked, so it's not as straight-forward as just adding a call.
> > Writing into a write-protected page isn't a good idea.
> >
> > Note that documentation says:
> > it is fine to delay the mmu_notifier_invalidate_range
> > call to mmu_notifier_invalidate_range_end() outside the page table lock.
> > implying it's called just later.
>
> OK I missed the fact that _end actually calls
> mmu_notifier_invalidate_range internally. So that part is fine but the
> fact that you are trying to take page lock under VQ mutex and take same
> mutex within notifier probably means it's broken for ksm and rmap at
> least since these call invalidate with lock taken.
>
> And generally, Andrea told me offline one can not take mutex under
> the notifier callback. I CC'd Andrea for why.

Correct, you _can not_ take mutex or any sleeping lock from within the
invalidate_range callback as those callback happens under the page table
spinlock. You can however do so under the invalidate_range_start call-
back only if it is a blocking allow callback (there is a flag passdown
with the invalidate_range_start callback if you are not allow to block
then return EBUSY and the invalidation will be aborted).


>
> That's a separate issue from set_page_dirty when memory is file backed.

If you can access file back page then i suggest using set_page_dirty
from within a special version of vunmap() so that when you vunmap you
set the page dirty without taking page lock. It is safe to do so
always from within an mmu notifier callback if you had the page map
with write permission which means that the page had write permission
in the userspace pte too and thus it having dirty pte is expected
and calling set_page_dirty on the page is allowed without any lock.
Locking will happen once the userspace pte are tear down through the
page table lock.

> It's because of all these issues that I preferred just accessing
> userspace memory and handling faults. Unfortunately there does not
> appear to exist an API that whitelists a specific driver along the lines
> of "I checked this code for speculative info leaks, don't add barriers
> on data path please".

Maybe it would be better to explore adding such helper then remapping
page into kernel address space ?

Cheers,
J?r?me

2019-03-07 19:39:28

by Andrea Arcangeli

[permalink] [raw]
Subject: Re: [RFC PATCH V2 5/5] vhost: access vq metadata through kernel virtual address

On Thu, Mar 07, 2019 at 02:09:10PM -0500, Jerome Glisse wrote:
> I thought this patch was only for anonymous memory ie not file back ?

Yes, the other common usages are on hugetlbfs/tmpfs that also don't
need to implement writeback and are obviously safe too.

> If so then set dirty is mostly useless it would only be use for swap
> but for this you can use an unlock version to set the page dirty.

It's not a practical issue but a security issue perhaps: you can
change the KVM userland to run on VM_SHARED ext4 as guest physical
memory, you could do that with the qemu command line that is used to
place it on tmpfs or hugetlbfs for example and some proprietary KVM
userland may do for other reasons. In general it shouldn't be possible
to crash the kernel with this, and it wouldn't be nice to fail if
somebody decides to put VM_SHARED ext4 (we could easily allow vhost
ring only backed by anon or tmpfs or hugetlbfs to solve this of
course).

It sounds like we should at least optimize away the _lock from
set_page_dirty if it's anon/hugetlbfs/tmpfs, would be nice if there
was a clean way to do that.

Now assuming we don't nak the use on ext4 VM_SHARED and we stick to
set_page_dirty_lock for such case: could you recap how that
__writepage ext4 crash was solved if try_to_free_buffers() run on a
pinned GUP page (in our vhost case try_to_unmap would have gotten rid
of the pins through the mmu notifier and the page would have been
freed just fine).

The first two things that come to mind is that we can easily forbid
the try_to_free_buffers() if the page might be pinned by GUP, it has
false positives with the speculative pagecache lookups but it cannot
give false negatives. We use those checks to know when a page is
pinned by GUP, for example, where we cannot merge KSM pages with gup
pins etc... However what if the elevated refcount wasn't there when
try_to_free_buffers run and is there when __remove_mapping runs?

What I mean is that it sounds easy to forbid try_to_free_buffers for
the long term pins, but that still won't prevent the same exact issue
for a transient pin (except the window to trigger it will be much smaller).

I basically don't see how long term GUP pins breaks stuff in ext4
while transient short term GUP pins like O_DIRECT don't. The VM code
isn't able to disambiguate if the pin is short or long term and it
won't even be able to tell the difference between a GUP pin (long or
short term) and a speculative get_page_unless_zero run by the
pagecache speculative pagecache lookup. Even a random speculative
pagecache lookup that runs just before __remove_mapping, can cause
__remove_mapping to fail despite try_to_free_buffers() succeeded
before it (like if there was a transient or long term GUP
pin). speculative lookup that can happen across all page struct at all
times and they will cause page_ref_freeze in __remove_mapping to
fail.

I'm sure I'm missing details on the ext4 __writepage problem and how
set_page_dirty_lock broke stuff with long term GUP pins, so I'm
asking...

Thanks!
Andrea

2019-03-07 20:19:27

by Jerome Glisse

[permalink] [raw]
Subject: Re: [RFC PATCH V2 5/5] vhost: access vq metadata through kernel virtual address

On Thu, Mar 07, 2019 at 02:38:38PM -0500, Andrea Arcangeli wrote:
> On Thu, Mar 07, 2019 at 02:09:10PM -0500, Jerome Glisse wrote:
> > I thought this patch was only for anonymous memory ie not file back ?
>
> Yes, the other common usages are on hugetlbfs/tmpfs that also don't
> need to implement writeback and are obviously safe too.
>
> > If so then set dirty is mostly useless it would only be use for swap
> > but for this you can use an unlock version to set the page dirty.
>
> It's not a practical issue but a security issue perhaps: you can
> change the KVM userland to run on VM_SHARED ext4 as guest physical
> memory, you could do that with the qemu command line that is used to
> place it on tmpfs or hugetlbfs for example and some proprietary KVM
> userland may do for other reasons. In general it shouldn't be possible
> to crash the kernel with this, and it wouldn't be nice to fail if
> somebody decides to put VM_SHARED ext4 (we could easily allow vhost
> ring only backed by anon or tmpfs or hugetlbfs to solve this of
> course).
>
> It sounds like we should at least optimize away the _lock from
> set_page_dirty if it's anon/hugetlbfs/tmpfs, would be nice if there
> was a clean way to do that.
>
> Now assuming we don't nak the use on ext4 VM_SHARED and we stick to
> set_page_dirty_lock for such case: could you recap how that
> __writepage ext4 crash was solved if try_to_free_buffers() run on a
> pinned GUP page (in our vhost case try_to_unmap would have gotten rid
> of the pins through the mmu notifier and the page would have been
> freed just fine).

So for the above the easiest thing is to call set_page_dirty() from
the mmu notifier callback. It is always safe to use the non locking
variant from such callback. Well it is safe only if the page was
map with write permission prior to the callback so here i assume
nothing stupid is going on and that you only vmap page with write
if they have a CPU pte with write and if not then you force a write
page fault.

Basicly from mmu notifier callback you have the same right as zap
pte has.
>

> The first two things that come to mind is that we can easily forbid
> the try_to_free_buffers() if the page might be pinned by GUP, it has
> false positives with the speculative pagecache lookups but it cannot
> give false negatives. We use those checks to know when a page is
> pinned by GUP, for example, where we cannot merge KSM pages with gup
> pins etc... However what if the elevated refcount wasn't there when
> try_to_free_buffers run and is there when __remove_mapping runs?
>
> What I mean is that it sounds easy to forbid try_to_free_buffers for
> the long term pins, but that still won't prevent the same exact issue
> for a transient pin (except the window to trigger it will be much smaller).

I think here you do not want to go down the same path as what is being
plane for GUP. GUP is being fix for "broken" hardware. Myself i am
converting proper hardware to no longer use GUP but rely on mmu notifier.

So i would not do any dance with blocking try_to_free_buffer, just
do everything from mmu notifier callback and you are fine.

>
> I basically don't see how long term GUP pins breaks stuff in ext4
> while transient short term GUP pins like O_DIRECT don't. The VM code
> isn't able to disambiguate if the pin is short or long term and it
> won't even be able to tell the difference between a GUP pin (long or
> short term) and a speculative get_page_unless_zero run by the
> pagecache speculative pagecache lookup. Even a random speculative
> pagecache lookup that runs just before __remove_mapping, can cause
> __remove_mapping to fail despite try_to_free_buffers() succeeded
> before it (like if there was a transient or long term GUP
> pin). speculative lookup that can happen across all page struct at all
> times and they will cause page_ref_freeze in __remove_mapping to
> fail.
>
> I'm sure I'm missing details on the ext4 __writepage problem and how
> set_page_dirty_lock broke stuff with long term GUP pins, so I'm
> asking...

O_DIRECT can suffer from the same issue but the race window for that
is small enough that it is unlikely it ever happened. But for device
driver that GUP page for hours/days/weeks/months ... obviously the
race window is big enough here. It affects many fs (ext4, xfs, ...)
in different ways. I think ext4 is the most obvious because of the
kernel log trace it leaves behind.

Bottom line is for set_page_dirty to be safe you need the following:
lock_page()
page_mkwrite()
set_pte_with_write()
unlock_page()

Now when loosing the write permission on the pte you will first get
a mmu notifier callback so anyone that abide by mmu notifier is fine
as long as they only write to the page if they found a pte with
write as it means the above sequence did happen and page is write-
able until the mmu notifier callback happens.

When you lookup a page into the page cache you still need to call
page_mkwrite() before installing a write-able pte.

Here for this vmap thing all you need is that the original user
pte had the write flag. If you only allow write in the vmap when
the original pte had write and you abide by mmu notifier then it
is ok to call set_page_dirty from the mmu notifier (but not after).

Hence why my suggestion is a special vunmap that call set_page_dirty
on the page from the mmu notifier.

Cheers,
J?r?me

2019-03-07 21:28:04

by Andrea Arcangeli

[permalink] [raw]
Subject: Re: [RFC PATCH V2 5/5] vhost: access vq metadata through kernel virtual address

Hello Jerome,

On Thu, Mar 07, 2019 at 03:17:22PM -0500, Jerome Glisse wrote:
> So for the above the easiest thing is to call set_page_dirty() from
> the mmu notifier callback. It is always safe to use the non locking
> variant from such callback. Well it is safe only if the page was
> map with write permission prior to the callback so here i assume
> nothing stupid is going on and that you only vmap page with write
> if they have a CPU pte with write and if not then you force a write
> page fault.

So if the GUP doesn't set FOLL_WRITE, set_page_dirty simply shouldn't
be called in such case. It only ever makes sense if the pte is
writable.

On a side note, the reason the write bit on the pte enabled avoids the
need of the _lock suffix is because of the stable page writeback
guarantees?

> Basicly from mmu notifier callback you have the same right as zap
> pte has.

Good point.

Related to this I already was wondering why the set_page_dirty is not
done in the invalidate. Reading the patch it looks like the dirty is
marked dirty when the ring wraps around, not in the invalidate, Jeson
can tell if I misread something there.

For transient data passing through the ring, nobody should care if
it's lost. It's not user-journaled anyway so it could hit the disk in
any order. The only reason to flush it to do disk is if there's memory
pressure (to pageout like a swapout) and in such case it's enough to
mark it dirty only in the mmu notifier invalidate like you pointed out
(and only if GUP was called with FOLL_WRITE).

> O_DIRECT can suffer from the same issue but the race window for that
> is small enough that it is unlikely it ever happened. But for device

Ok that clarifies things.

> driver that GUP page for hours/days/weeks/months ... obviously the
> race window is big enough here. It affects many fs (ext4, xfs, ...)
> in different ways. I think ext4 is the most obvious because of the
> kernel log trace it leaves behind.
>
> Bottom line is for set_page_dirty to be safe you need the following:
> lock_page()
> page_mkwrite()
> set_pte_with_write()
> unlock_page()

I also wondered why ext4 writepage doesn't recreate the bh if they got
dropped by the VM and page->private is 0. I mean, page->index and
page->mapping are still there, that's enough info for writepage itself
to take a slow path and calls page_mkwrite to find where to write the
page on disk.

> Now when loosing the write permission on the pte you will first get
> a mmu notifier callback so anyone that abide by mmu notifier is fine
> as long as they only write to the page if they found a pte with
> write as it means the above sequence did happen and page is write-
> able until the mmu notifier callback happens.
>
> When you lookup a page into the page cache you still need to call
> page_mkwrite() before installing a write-able pte.
>
> Here for this vmap thing all you need is that the original user
> pte had the write flag. If you only allow write in the vmap when
> the original pte had write and you abide by mmu notifier then it
> is ok to call set_page_dirty from the mmu notifier (but not after).
>
> Hence why my suggestion is a special vunmap that call set_page_dirty
> on the page from the mmu notifier.

Agreed, that will solve all issues in vhost context with regard to
set_page_dirty, including the case the memory is backed by VM_SHARED ext4.

Thanks!
Andrea

2019-03-08 02:21:45

by Michael S. Tsirkin

[permalink] [raw]
Subject: Re: [RFC PATCH V2 5/5] vhost: access vq metadata through kernel virtual address

On Thu, Mar 07, 2019 at 02:17:20PM -0500, Jerome Glisse wrote:
> > It's because of all these issues that I preferred just accessing
> > userspace memory and handling faults. Unfortunately there does not
> > appear to exist an API that whitelists a specific driver along the lines
> > of "I checked this code for speculative info leaks, don't add barriers
> > on data path please".
>
> Maybe it would be better to explore adding such helper then remapping
> page into kernel address space ?

I explored it a bit (see e.g. thread around: "__get_user slower than
get_user") and I can tell you it's not trivial given the issue is around
security. So in practice it does not seem fair to keep a significant
optimization out of kernel because *maybe* we can do it differently even
better :)

--
MST

2019-03-08 02:56:49

by Jerome Glisse

[permalink] [raw]
Subject: Re: [RFC PATCH V2 5/5] vhost: access vq metadata through kernel virtual address

On Thu, Mar 07, 2019 at 09:21:03PM -0500, Michael S. Tsirkin wrote:
> On Thu, Mar 07, 2019 at 02:17:20PM -0500, Jerome Glisse wrote:
> > > It's because of all these issues that I preferred just accessing
> > > userspace memory and handling faults. Unfortunately there does not
> > > appear to exist an API that whitelists a specific driver along the lines
> > > of "I checked this code for speculative info leaks, don't add barriers
> > > on data path please".
> >
> > Maybe it would be better to explore adding such helper then remapping
> > page into kernel address space ?
>
> I explored it a bit (see e.g. thread around: "__get_user slower than
> get_user") and I can tell you it's not trivial given the issue is around
> security. So in practice it does not seem fair to keep a significant
> optimization out of kernel because *maybe* we can do it differently even
> better :)

Maybe a slightly different approach between this patchset and other
copy user API would work here. What you want really is something like
a temporary mlock on a range of memory so that it is safe for the
kernel to access range of userspace virtual address ie page are
present and with proper permission hence there can be no page fault
while you are accessing thing from kernel context.

So you can have like a range structure and mmu notifier. When you
lock the range you block mmu notifier to allow your code to work on
the userspace VA safely. Once you are done you unlock and let the
mmu notifier go on. It is pretty much exactly this patchset except
that you remove all the kernel vmap code. A nice thing about that
is that you do not need to worry about calling set page dirty it
will already be handle by the userspace VA pte. It also use less
memory than when you have kernel vmap.

This idea might be defeated by security feature where the kernel is
running in its own address space without the userspace address
space present.

Anyway just wanted to put the idea forward.

Cheers,
J?r?me

2019-03-08 03:17:01

by Michael S. Tsirkin

[permalink] [raw]
Subject: Re: [RFC PATCH V2 5/5] vhost: access vq metadata through kernel virtual address

On Thu, Mar 07, 2019 at 09:55:39PM -0500, Jerome Glisse wrote:
> On Thu, Mar 07, 2019 at 09:21:03PM -0500, Michael S. Tsirkin wrote:
> > On Thu, Mar 07, 2019 at 02:17:20PM -0500, Jerome Glisse wrote:
> > > > It's because of all these issues that I preferred just accessing
> > > > userspace memory and handling faults. Unfortunately there does not
> > > > appear to exist an API that whitelists a specific driver along the lines
> > > > of "I checked this code for speculative info leaks, don't add barriers
> > > > on data path please".
> > >
> > > Maybe it would be better to explore adding such helper then remapping
> > > page into kernel address space ?
> >
> > I explored it a bit (see e.g. thread around: "__get_user slower than
> > get_user") and I can tell you it's not trivial given the issue is around
> > security. So in practice it does not seem fair to keep a significant
> > optimization out of kernel because *maybe* we can do it differently even
> > better :)
>
> Maybe a slightly different approach between this patchset and other
> copy user API would work here. What you want really is something like
> a temporary mlock on a range of memory so that it is safe for the
> kernel to access range of userspace virtual address ie page are
> present and with proper permission hence there can be no page fault
> while you are accessing thing from kernel context.
>
> So you can have like a range structure and mmu notifier. When you
> lock the range you block mmu notifier to allow your code to work on
> the userspace VA safely. Once you are done you unlock and let the
> mmu notifier go on. It is pretty much exactly this patchset except
> that you remove all the kernel vmap code. A nice thing about that
> is that you do not need to worry about calling set page dirty it
> will already be handle by the userspace VA pte. It also use less
> memory than when you have kernel vmap.
>
> This idea might be defeated by security feature where the kernel is
> running in its own address space without the userspace address
> space present.

Like smap?

> Anyway just wanted to put the idea forward.
>
> Cheers,
> J?r?me

2019-03-08 03:42:37

by Jerome Glisse

[permalink] [raw]
Subject: Re: [RFC PATCH V2 5/5] vhost: access vq metadata through kernel virtual address

On Thu, Mar 07, 2019 at 10:16:00PM -0500, Michael S. Tsirkin wrote:
> On Thu, Mar 07, 2019 at 09:55:39PM -0500, Jerome Glisse wrote:
> > On Thu, Mar 07, 2019 at 09:21:03PM -0500, Michael S. Tsirkin wrote:
> > > On Thu, Mar 07, 2019 at 02:17:20PM -0500, Jerome Glisse wrote:
> > > > > It's because of all these issues that I preferred just accessing
> > > > > userspace memory and handling faults. Unfortunately there does not
> > > > > appear to exist an API that whitelists a specific driver along the lines
> > > > > of "I checked this code for speculative info leaks, don't add barriers
> > > > > on data path please".
> > > >
> > > > Maybe it would be better to explore adding such helper then remapping
> > > > page into kernel address space ?
> > >
> > > I explored it a bit (see e.g. thread around: "__get_user slower than
> > > get_user") and I can tell you it's not trivial given the issue is around
> > > security. So in practice it does not seem fair to keep a significant
> > > optimization out of kernel because *maybe* we can do it differently even
> > > better :)
> >
> > Maybe a slightly different approach between this patchset and other
> > copy user API would work here. What you want really is something like
> > a temporary mlock on a range of memory so that it is safe for the
> > kernel to access range of userspace virtual address ie page are
> > present and with proper permission hence there can be no page fault
> > while you are accessing thing from kernel context.
> >
> > So you can have like a range structure and mmu notifier. When you
> > lock the range you block mmu notifier to allow your code to work on
> > the userspace VA safely. Once you are done you unlock and let the
> > mmu notifier go on. It is pretty much exactly this patchset except
> > that you remove all the kernel vmap code. A nice thing about that
> > is that you do not need to worry about calling set page dirty it
> > will already be handle by the userspace VA pte. It also use less
> > memory than when you have kernel vmap.
> >
> > This idea might be defeated by security feature where the kernel is
> > running in its own address space without the userspace address
> > space present.
>
> Like smap?

Yes like smap but also other newer changes, with similar effect, since
the spectre drama.

Cheers,
J?r?me

2019-03-08 03:43:59

by Michael S. Tsirkin

[permalink] [raw]
Subject: Re: [RFC PATCH V2 5/5] vhost: access vq metadata through kernel virtual address

On Thu, Mar 07, 2019 at 10:40:53PM -0500, Jerome Glisse wrote:
> On Thu, Mar 07, 2019 at 10:16:00PM -0500, Michael S. Tsirkin wrote:
> > On Thu, Mar 07, 2019 at 09:55:39PM -0500, Jerome Glisse wrote:
> > > On Thu, Mar 07, 2019 at 09:21:03PM -0500, Michael S. Tsirkin wrote:
> > > > On Thu, Mar 07, 2019 at 02:17:20PM -0500, Jerome Glisse wrote:
> > > > > > It's because of all these issues that I preferred just accessing
> > > > > > userspace memory and handling faults. Unfortunately there does not
> > > > > > appear to exist an API that whitelists a specific driver along the lines
> > > > > > of "I checked this code for speculative info leaks, don't add barriers
> > > > > > on data path please".
> > > > >
> > > > > Maybe it would be better to explore adding such helper then remapping
> > > > > page into kernel address space ?
> > > >
> > > > I explored it a bit (see e.g. thread around: "__get_user slower than
> > > > get_user") and I can tell you it's not trivial given the issue is around
> > > > security. So in practice it does not seem fair to keep a significant
> > > > optimization out of kernel because *maybe* we can do it differently even
> > > > better :)
> > >
> > > Maybe a slightly different approach between this patchset and other
> > > copy user API would work here. What you want really is something like
> > > a temporary mlock on a range of memory so that it is safe for the
> > > kernel to access range of userspace virtual address ie page are
> > > present and with proper permission hence there can be no page fault
> > > while you are accessing thing from kernel context.
> > >
> > > So you can have like a range structure and mmu notifier. When you
> > > lock the range you block mmu notifier to allow your code to work on
> > > the userspace VA safely. Once you are done you unlock and let the
> > > mmu notifier go on. It is pretty much exactly this patchset except
> > > that you remove all the kernel vmap code. A nice thing about that
> > > is that you do not need to worry about calling set page dirty it
> > > will already be handle by the userspace VA pte. It also use less
> > > memory than when you have kernel vmap.
> > >
> > > This idea might be defeated by security feature where the kernel is
> > > running in its own address space without the userspace address
> > > space present.
> >
> > Like smap?
>
> Yes like smap but also other newer changes, with similar effect, since
> the spectre drama.
>
> Cheers,
> J?r?me

Sorry do you mean meltdown and kpti?

--
MST

2019-03-08 03:46:28

by Jerome Glisse

[permalink] [raw]
Subject: Re: [RFC PATCH V2 5/5] vhost: access vq metadata through kernel virtual address

On Thu, Mar 07, 2019 at 10:43:12PM -0500, Michael S. Tsirkin wrote:
> On Thu, Mar 07, 2019 at 10:40:53PM -0500, Jerome Glisse wrote:
> > On Thu, Mar 07, 2019 at 10:16:00PM -0500, Michael S. Tsirkin wrote:
> > > On Thu, Mar 07, 2019 at 09:55:39PM -0500, Jerome Glisse wrote:
> > > > On Thu, Mar 07, 2019 at 09:21:03PM -0500, Michael S. Tsirkin wrote:
> > > > > On Thu, Mar 07, 2019 at 02:17:20PM -0500, Jerome Glisse wrote:
> > > > > > > It's because of all these issues that I preferred just accessing
> > > > > > > userspace memory and handling faults. Unfortunately there does not
> > > > > > > appear to exist an API that whitelists a specific driver along the lines
> > > > > > > of "I checked this code for speculative info leaks, don't add barriers
> > > > > > > on data path please".
> > > > > >
> > > > > > Maybe it would be better to explore adding such helper then remapping
> > > > > > page into kernel address space ?
> > > > >
> > > > > I explored it a bit (see e.g. thread around: "__get_user slower than
> > > > > get_user") and I can tell you it's not trivial given the issue is around
> > > > > security. So in practice it does not seem fair to keep a significant
> > > > > optimization out of kernel because *maybe* we can do it differently even
> > > > > better :)
> > > >
> > > > Maybe a slightly different approach between this patchset and other
> > > > copy user API would work here. What you want really is something like
> > > > a temporary mlock on a range of memory so that it is safe for the
> > > > kernel to access range of userspace virtual address ie page are
> > > > present and with proper permission hence there can be no page fault
> > > > while you are accessing thing from kernel context.
> > > >
> > > > So you can have like a range structure and mmu notifier. When you
> > > > lock the range you block mmu notifier to allow your code to work on
> > > > the userspace VA safely. Once you are done you unlock and let the
> > > > mmu notifier go on. It is pretty much exactly this patchset except
> > > > that you remove all the kernel vmap code. A nice thing about that
> > > > is that you do not need to worry about calling set page dirty it
> > > > will already be handle by the userspace VA pte. It also use less
> > > > memory than when you have kernel vmap.
> > > >
> > > > This idea might be defeated by security feature where the kernel is
> > > > running in its own address space without the userspace address
> > > > space present.
> > >
> > > Like smap?
> >
> > Yes like smap but also other newer changes, with similar effect, since
> > the spectre drama.
> >
> > Cheers,
> > J?r?me
>
> Sorry do you mean meltdown and kpti?

Yes all that and similar thing. I do not have the full list in my head.

Cheers,
J?r?me

2019-03-08 08:32:43

by Jason Wang

[permalink] [raw]
Subject: Re: [RFC PATCH V2 5/5] vhost: access vq metadata through kernel virtual address


On 2019/3/7 下午11:34, Michael S. Tsirkin wrote:
> On Thu, Mar 07, 2019 at 10:45:57AM +0800, Jason Wang wrote:
>> On 2019/3/7 上午12:31, Michael S. Tsirkin wrote:
>>>> +static void vhost_set_vmap_dirty(struct vhost_vmap *used)
>>>> +{
>>>> + int i;
>>>> +
>>>> + for (i = 0; i < used->npages; i++)
>>>> + set_page_dirty_lock(used->pages[i]);
>>> This seems to rely on page lock to mark page dirty.
>>>
>>> Could it happen that page writeback will check the
>>> page, find it clean, and then you mark it dirty and then
>>> invalidate callback is called?
>>>
>>>
>> Yes. But does this break anything?
>> The page is still there, we just remove a
>> kernel mapping to it.
>>
>> Thanks
> Yes it's the same problem as e.g. RDMA:
> we've just marked the page as dirty without having buffers.
> Eventually writeback will find it and filesystem will complain...
> So if the pages are backed by a non-RAM-based filesystem, it’s all just broken.


Yes, we can't depend on the pages that might have been invalidated. As
suggested, the only suitable place is the MMU notifier callbacks.

Thanks


> one can hope that RDMA guys will fix it in some way eventually.
> For now, maybe add a flag in e.g. VMA that says that there's no
> writeback so it's safe to mark page dirty at any point?
>
>
>
>
>

2019-03-08 08:51:26

by Jason Wang

[permalink] [raw]
Subject: Re: [RFC PATCH V2 5/5] vhost: access vq metadata through kernel virtual address


On 2019/3/8 上午3:16, Andrea Arcangeli wrote:
> On Thu, Mar 07, 2019 at 12:56:45PM -0500, Michael S. Tsirkin wrote:
>> On Thu, Mar 07, 2019 at 10:47:22AM -0500, Michael S. Tsirkin wrote:
>>> On Wed, Mar 06, 2019 at 02:18:12AM -0500, Jason Wang wrote:
>>>> +static const struct mmu_notifier_ops vhost_mmu_notifier_ops = {
>>>> + .invalidate_range = vhost_invalidate_range,
>>>> +};
>>>> +
>>>> void vhost_dev_init(struct vhost_dev *dev,
>>>> struct vhost_virtqueue **vqs, int nvqs, int iov_limit)
>>>> {
>>> I also wonder here: when page is write protected then
>>> it does not look like .invalidate_range is invoked.
>>>
>>> E.g. mm/ksm.c calls
>>>
>>> mmu_notifier_invalidate_range_start and
>>> mmu_notifier_invalidate_range_end but not mmu_notifier_invalidate_range.
>>>
>>> Similarly, rmap in page_mkclean_one will not call
>>> mmu_notifier_invalidate_range.
>>>
>>> If I'm right vhost won't get notified when page is write-protected since you
>>> didn't install start/end notifiers. Note that end notifier can be called
>>> with page locked, so it's not as straight-forward as just adding a call.
>>> Writing into a write-protected page isn't a good idea.
>>>
>>> Note that documentation says:
>>> it is fine to delay the mmu_notifier_invalidate_range
>>> call to mmu_notifier_invalidate_range_end() outside the page table lock.
>>> implying it's called just later.
>> OK I missed the fact that _end actually calls
>> mmu_notifier_invalidate_range internally. So that part is fine but the
>> fact that you are trying to take page lock under VQ mutex and take same
>> mutex within notifier probably means it's broken for ksm and rmap at
>> least since these call invalidate with lock taken.
> Yes this lock inversion needs more thoughts.
>
>> And generally, Andrea told me offline one can not take mutex under
>> the notifier callback. I CC'd Andrea for why.
> Yes, the problem then is the ->invalidate_page is called then under PT
> lock so it cannot take mutex, you also cannot take the page_lock, it
> can at most take a spinlock or trylock_page.
>
> So it must switch back to the _start/_end methods unless you rewrite
> the locking.
>
> The difference with _start/_end, is that ->invalidate_range avoids the
> _start callback basically, but to avoid the _start callback safely, it
> has to be called in between the ptep_clear_flush and the set_pte_at
> whenever the pfn changes like during a COW. So it cannot be coalesced
> in a single TLB flush that invalidates all sptes in a range like we
> prefer for performance reasons for example in KVM. It also cannot
> sleep.
>
> In short ->invalidate_range must be really fast (it shouldn't require
> to send IPI to all other CPUs like KVM may require during an
> invalidate_range_start) and it must not sleep, in order to prefer it
> to _start/_end.
>
> I.e. the invalidate of the secondary MMU that walks the linux
> pagetables in hardware (in vhost case with GUP in software) has to
> happen while the linux pagetable is zero, otherwise a concurrent
> hardware pagetable lookup could re-instantiate a mapping to the old
> page in between the set_pte_at and the invalidate_range_end (which
> internally calls ->invalidate_range). Jerome documented it nicely in
> Documentation/vm/mmu_notifier.rst .


Right, I've actually gone through this several times but some details
were missed by me obviously.


>
> Now you don't really walk the pagetable in hardware in vhost, but if
> you use gup_fast after usemm() it's similar.
>
> For vhost the invalidate would be really fast, there are no IPI to
> deliver at all, the problem is just the mutex.


Yes. A possible solution is to introduce a valid flag for VA. Vhost may
only try to access kernel VA when it was valid. Invalidate_range_start()
will clear this under the protection of the vq mutex when it can block.
Then invalidate_range_end() then can clear this flag. An issue is
blockable is  always false for range_end().


>
>> That's a separate issue from set_page_dirty when memory is file backed.
> Yes. I don't yet know why the ext4 internal __writepage cannot
> re-create the bh if they've been freed by the VM and why such race
> where the bh are freed for a pinned VM_SHARED ext4 page doesn't even
> exist for transient pins like O_DIRECT (does it work by luck?), but
> with mmu notifiers there are no long term pins anyway, so this works
> normally and it's like the memory isn't pinned. In any case I think
> that's a kernel bug in either __writepage or try_to_free_buffers, so I
> would ignore it considering qemu will only use anon memory or tmpfs or
> hugetlbfs as backing store for the virtio ring. It wouldn't make sense
> for qemu to risk triggering I/O on a VM_SHARED ext4, so we shouldn't
> be even exposed to what seems to be an orthogonal kernel bug.
>
> I suppose whatever solution will fix the set_page_dirty_lock on
> VM_SHARED ext4 for the other places that don't or can't use mmu
> notifiers, will then work for vhost too which uses mmu notifiers and
> will be less affected from the start if something.
>
> Reading the lwn link about the discussion about the long term GUP pin
> from Jan vs set_page_dirty_lock: I can only agree with the last part
> where Jerome correctly pointed out at the end that mellanox RDMA got
> it right by avoiding completely long term pins by using mmu notifier
> and in general mmu notifier is the standard solution to avoid long
> term pins. Nothing should ever take long term GUP pins, if it does it
> means software is bad or the hardware lacks features to support on
> demand paging. Still I don't get why transient pins like O_DIRECT
> where mmu notifier would be prohibitive to use (registering into mmu
> notifier cannot be done at high frequency, the locking to do so is
> massive) cannot end up into the same ext4 _writepage crash as long
> term pins: long term or short term transient is a subjective measure
> from VM standpoint, the VM won't know the difference, luck will
> instead.
>
>> It's because of all these issues that I preferred just accessing
>> userspace memory and handling faults. Unfortunately there does not
>> appear to exist an API that whitelists a specific driver along the lines
>> of "I checked this code for speculative info leaks, don't add barriers
>> on data path please".
> Yes that's unfortunate, __uaccess_begin_nospec() is now making
> prohibitive to frequently access userland code.
>
> I doubt we can do like access_ok() and only check it once. access_ok
> checks the virtual address, and if the virtual address is ok doesn't
> wrap around and it points to userland in a safe range, it's always
> ok. There's no need to run access_ok again if we keep hitting on the
> very same address.
>
> __uaccess_begin_nospec() instead is about runtime stuff that can
> change the moment copy-user has completed even before returning to
> userland, so there's no easy way to do it just once.
>
> On top of skipping the __uaccess_begin_nospec(), the mmu notifier soft
> vhost design will further boost the performance by guaranteeing the
> use of gigapages TLBs when available (or 2M TLBs worst case) even if
> QEMU runs on smaller pages.


Just to make sure I understand here. For boosting through huge TLB, do
you mean we can do that in the future (e.g by mapping more userspace
pages to kenrel) or it can be done by this series (only about three 4K
pages were vmapped per virtqueue)?

Thanks


>
> Thanks,
> Andrea

2019-03-08 08:59:49

by Jason Wang

[permalink] [raw]
Subject: Re: [RFC PATCH V2 5/5] vhost: access vq metadata through kernel virtual address


On 2019/3/8 上午3:17, Jerome Glisse wrote:
> On Thu, Mar 07, 2019 at 12:56:45PM -0500, Michael S. Tsirkin wrote:
>> On Thu, Mar 07, 2019 at 10:47:22AM -0500, Michael S. Tsirkin wrote:
>>> On Wed, Mar 06, 2019 at 02:18:12AM -0500, Jason Wang wrote:
>>>> +static const struct mmu_notifier_ops vhost_mmu_notifier_ops = {
>>>> + .invalidate_range = vhost_invalidate_range,
>>>> +};
>>>> +
>>>> void vhost_dev_init(struct vhost_dev *dev,
>>>> struct vhost_virtqueue **vqs, int nvqs, int iov_limit)
>>>> {
>>> I also wonder here: when page is write protected then
>>> it does not look like .invalidate_range is invoked.
>>>
>>> E.g. mm/ksm.c calls
>>>
>>> mmu_notifier_invalidate_range_start and
>>> mmu_notifier_invalidate_range_end but not mmu_notifier_invalidate_range.
>>>
>>> Similarly, rmap in page_mkclean_one will not call
>>> mmu_notifier_invalidate_range.
>>>
>>> If I'm right vhost won't get notified when page is write-protected since you
>>> didn't install start/end notifiers. Note that end notifier can be called
>>> with page locked, so it's not as straight-forward as just adding a call.
>>> Writing into a write-protected page isn't a good idea.
>>>
>>> Note that documentation says:
>>> it is fine to delay the mmu_notifier_invalidate_range
>>> call to mmu_notifier_invalidate_range_end() outside the page table lock.
>>> implying it's called just later.
>> OK I missed the fact that _end actually calls
>> mmu_notifier_invalidate_range internally. So that part is fine but the
>> fact that you are trying to take page lock under VQ mutex and take same
>> mutex within notifier probably means it's broken for ksm and rmap at
>> least since these call invalidate with lock taken.
>>
>> And generally, Andrea told me offline one can not take mutex under
>> the notifier callback. I CC'd Andrea for why.
> Correct, you _can not_ take mutex or any sleeping lock from within the
> invalidate_range callback as those callback happens under the page table
> spinlock. You can however do so under the invalidate_range_start call-
> back only if it is a blocking allow callback (there is a flag passdown
> with the invalidate_range_start callback if you are not allow to block
> then return EBUSY and the invalidation will be aborted).
>
>
>> That's a separate issue from set_page_dirty when memory is file backed.
> If you can access file back page then i suggest using set_page_dirty
> from within a special version of vunmap() so that when you vunmap you
> set the page dirty without taking page lock. It is safe to do so
> always from within an mmu notifier callback if you had the page map
> with write permission which means that the page had write permission
> in the userspace pte too and thus it having dirty pte is expected
> and calling set_page_dirty on the page is allowed without any lock.
> Locking will happen once the userspace pte are tear down through the
> page table lock.


Can I simply can set_page_dirty() before vunmap() in the mmu notifier
callback, or is there any reason that it must be called within vumap()?

Thanks


>
>> It's because of all these issues that I preferred just accessing
>> userspace memory and handling faults. Unfortunately there does not
>> appear to exist an API that whitelists a specific driver along the lines
>> of "I checked this code for speculative info leaks, don't add barriers
>> on data path please".
> Maybe it would be better to explore adding such helper then remapping
> page into kernel address space ?
>
> Cheers,
> Jérôme

2019-03-08 09:14:47

by Jason Wang

[permalink] [raw]
Subject: Re: [RFC PATCH V2 5/5] vhost: access vq metadata through kernel virtual address


On 2019/3/8 上午5:27, Andrea Arcangeli wrote:
> Hello Jerome,
>
> On Thu, Mar 07, 2019 at 03:17:22PM -0500, Jerome Glisse wrote:
>> So for the above the easiest thing is to call set_page_dirty() from
>> the mmu notifier callback. It is always safe to use the non locking
>> variant from such callback. Well it is safe only if the page was
>> map with write permission prior to the callback so here i assume
>> nothing stupid is going on and that you only vmap page with write
>> if they have a CPU pte with write and if not then you force a write
>> page fault.
> So if the GUP doesn't set FOLL_WRITE, set_page_dirty simply shouldn't
> be called in such case. It only ever makes sense if the pte is
> writable.
>
> On a side note, the reason the write bit on the pte enabled avoids the
> need of the _lock suffix is because of the stable page writeback
> guarantees?
>
>> Basicly from mmu notifier callback you have the same right as zap
>> pte has.
> Good point.
>
> Related to this I already was wondering why the set_page_dirty is not
> done in the invalidate. Reading the patch it looks like the dirty is
> marked dirty when the ring wraps around, not in the invalidate, Jeson
> can tell if I misread something there.


Actually not wrapping around,  the pages for used ring was marked as
dirty after a round of virtqueue processing when we're sure vhost wrote
something there.

Thanks


>
> For transient data passing through the ring, nobody should care if
> it's lost. It's not user-journaled anyway so it could hit the disk in
> any order. The only reason to flush it to do disk is if there's memory
> pressure (to pageout like a swapout) and in such case it's enough to
> mark it dirty only in the mmu notifier invalidate like you pointed out
> (and only if GUP was called with FOLL_WRITE).
>
>> O_DIRECT can suffer from the same issue but the race window for that
>> is small enough that it is unlikely it ever happened. But for device
> Ok that clarifies things.
>
>> driver that GUP page for hours/days/weeks/months ... obviously the
>> race window is big enough here. It affects many fs (ext4, xfs, ...)
>> in different ways. I think ext4 is the most obvious because of the
>> kernel log trace it leaves behind.
>>
>> Bottom line is for set_page_dirty to be safe you need the following:
>> lock_page()
>> page_mkwrite()
>> set_pte_with_write()
>> unlock_page()
> I also wondered why ext4 writepage doesn't recreate the bh if they got
> dropped by the VM and page->private is 0. I mean, page->index and
> page->mapping are still there, that's enough info for writepage itself
> to take a slow path and calls page_mkwrite to find where to write the
> page on disk.
>
>> Now when loosing the write permission on the pte you will first get
>> a mmu notifier callback so anyone that abide by mmu notifier is fine
>> as long as they only write to the page if they found a pte with
>> write as it means the above sequence did happen and page is write-
>> able until the mmu notifier callback happens.
>>
>> When you lookup a page into the page cache you still need to call
>> page_mkwrite() before installing a write-able pte.
>>
>> Here for this vmap thing all you need is that the original user
>> pte had the write flag. If you only allow write in the vmap when
>> the original pte had write and you abide by mmu notifier then it
>> is ok to call set_page_dirty from the mmu notifier (but not after).
>>
>> Hence why my suggestion is a special vunmap that call set_page_dirty
>> on the page from the mmu notifier.
> Agreed, that will solve all issues in vhost context with regard to
> set_page_dirty, including the case the memory is backed by VM_SHARED ext4.
>
> Thanks!
> Andrea

2019-03-08 09:17:38

by Jason Wang

[permalink] [raw]
Subject: Re: [RFC PATCH V2 5/5] vhost: access vq metadata through kernel virtual address


On 2019/3/8 上午11:45, Jerome Glisse wrote:
> On Thu, Mar 07, 2019 at 10:43:12PM -0500, Michael S. Tsirkin wrote:
>> On Thu, Mar 07, 2019 at 10:40:53PM -0500, Jerome Glisse wrote:
>>> On Thu, Mar 07, 2019 at 10:16:00PM -0500, Michael S. Tsirkin wrote:
>>>> On Thu, Mar 07, 2019 at 09:55:39PM -0500, Jerome Glisse wrote:
>>>>> On Thu, Mar 07, 2019 at 09:21:03PM -0500, Michael S. Tsirkin wrote:
>>>>>> On Thu, Mar 07, 2019 at 02:17:20PM -0500, Jerome Glisse wrote:
>>>>>>>> It's because of all these issues that I preferred just accessing
>>>>>>>> userspace memory and handling faults. Unfortunately there does not
>>>>>>>> appear to exist an API that whitelists a specific driver along the lines
>>>>>>>> of "I checked this code for speculative info leaks, don't add barriers
>>>>>>>> on data path please".
>>>>>>> Maybe it would be better to explore adding such helper then remapping
>>>>>>> page into kernel address space ?
>>>>>> I explored it a bit (see e.g. thread around: "__get_user slower than
>>>>>> get_user") and I can tell you it's not trivial given the issue is around
>>>>>> security. So in practice it does not seem fair to keep a significant
>>>>>> optimization out of kernel because *maybe* we can do it differently even
>>>>>> better :)
>>>>> Maybe a slightly different approach between this patchset and other
>>>>> copy user API would work here. What you want really is something like
>>>>> a temporary mlock on a range of memory so that it is safe for the
>>>>> kernel to access range of userspace virtual address ie page are
>>>>> present and with proper permission hence there can be no page fault
>>>>> while you are accessing thing from kernel context.
>>>>>
>>>>> So you can have like a range structure and mmu notifier. When you
>>>>> lock the range you block mmu notifier to allow your code to work on
>>>>> the userspace VA safely. Once you are done you unlock and let the
>>>>> mmu notifier go on. It is pretty much exactly this patchset except
>>>>> that you remove all the kernel vmap code. A nice thing about that
>>>>> is that you do not need to worry about calling set page dirty it
>>>>> will already be handle by the userspace VA pte. It also use less
>>>>> memory than when you have kernel vmap.
>>>>>
>>>>> This idea might be defeated by security feature where the kernel is
>>>>> running in its own address space without the userspace address
>>>>> space present.
>>>> Like smap?
>>> Yes like smap but also other newer changes, with similar effect, since
>>> the spectre drama.
>>>
>>> Cheers,
>>> Jérôme
>> Sorry do you mean meltdown and kpti?
> Yes all that and similar thing. I do not have the full list in my head.
>
> Cheers,
> Jérôme


Yes, address space of kernel its own is the main motivation of using
vmap here.

Thanks


2019-03-08 13:06:47

by Michael S. Tsirkin

[permalink] [raw]
Subject: Re: [RFC PATCH V2 5/5] vhost: access vq metadata through kernel virtual address

On Fri, Mar 08, 2019 at 04:58:44PM +0800, Jason Wang wrote:
>
> On 2019/3/8 上午3:17, Jerome Glisse wrote:
> > On Thu, Mar 07, 2019 at 12:56:45PM -0500, Michael S. Tsirkin wrote:
> > > On Thu, Mar 07, 2019 at 10:47:22AM -0500, Michael S. Tsirkin wrote:
> > > > On Wed, Mar 06, 2019 at 02:18:12AM -0500, Jason Wang wrote:
> > > > > +static const struct mmu_notifier_ops vhost_mmu_notifier_ops = {
> > > > > + .invalidate_range = vhost_invalidate_range,
> > > > > +};
> > > > > +
> > > > > void vhost_dev_init(struct vhost_dev *dev,
> > > > > struct vhost_virtqueue **vqs, int nvqs, int iov_limit)
> > > > > {
> > > > I also wonder here: when page is write protected then
> > > > it does not look like .invalidate_range is invoked.
> > > >
> > > > E.g. mm/ksm.c calls
> > > >
> > > > mmu_notifier_invalidate_range_start and
> > > > mmu_notifier_invalidate_range_end but not mmu_notifier_invalidate_range.
> > > >
> > > > Similarly, rmap in page_mkclean_one will not call
> > > > mmu_notifier_invalidate_range.
> > > >
> > > > If I'm right vhost won't get notified when page is write-protected since you
> > > > didn't install start/end notifiers. Note that end notifier can be called
> > > > with page locked, so it's not as straight-forward as just adding a call.
> > > > Writing into a write-protected page isn't a good idea.
> > > >
> > > > Note that documentation says:
> > > > it is fine to delay the mmu_notifier_invalidate_range
> > > > call to mmu_notifier_invalidate_range_end() outside the page table lock.
> > > > implying it's called just later.
> > > OK I missed the fact that _end actually calls
> > > mmu_notifier_invalidate_range internally. So that part is fine but the
> > > fact that you are trying to take page lock under VQ mutex and take same
> > > mutex within notifier probably means it's broken for ksm and rmap at
> > > least since these call invalidate with lock taken.
> > >
> > > And generally, Andrea told me offline one can not take mutex under
> > > the notifier callback. I CC'd Andrea for why.
> > Correct, you _can not_ take mutex or any sleeping lock from within the
> > invalidate_range callback as those callback happens under the page table
> > spinlock. You can however do so under the invalidate_range_start call-
> > back only if it is a blocking allow callback (there is a flag passdown
> > with the invalidate_range_start callback if you are not allow to block
> > then return EBUSY and the invalidation will be aborted).
> >
> >
> > > That's a separate issue from set_page_dirty when memory is file backed.
> > If you can access file back page then i suggest using set_page_dirty
> > from within a special version of vunmap() so that when you vunmap you
> > set the page dirty without taking page lock. It is safe to do so
> > always from within an mmu notifier callback if you had the page map
> > with write permission which means that the page had write permission
> > in the userspace pte too and thus it having dirty pte is expected
> > and calling set_page_dirty on the page is allowed without any lock.
> > Locking will happen once the userspace pte are tear down through the
> > page table lock.
>
>
> Can I simply can set_page_dirty() before vunmap() in the mmu notifier
> callback, or is there any reason that it must be called within vumap()?
>
> Thanks


I think this is what Jerome is saying, yes.
Maybe add a patch to mmu notifier doc file, documenting this?


>
> >
> > > It's because of all these issues that I preferred just accessing
> > > userspace memory and handling faults. Unfortunately there does not
> > > appear to exist an API that whitelists a specific driver along the lines
> > > of "I checked this code for speculative info leaks, don't add barriers
> > > on data path please".
> > Maybe it would be better to explore adding such helper then remapping
> > page into kernel address space ?
> >
> > Cheers,
> > Jérôme

2019-03-08 14:14:27

by Christoph Hellwig

[permalink] [raw]
Subject: Re: [RFC PATCH V2 0/5] vhost: accelerate metadata access through vmap()

On Wed, Mar 06, 2019 at 02:18:07AM -0500, Jason Wang wrote:
> This series tries to access virtqueue metadata through kernel virtual
> address instead of copy_user() friends since they had too much
> overheads like checks, spec barriers or even hardware feature
> toggling. This is done through setup kernel address through vmap() and
> resigter MMU notifier for invalidation.
>
> Test shows about 24% improvement on TX PPS. TCP_STREAM doesn't see
> obvious improvement.

How is this going to work for CPUs with virtually tagged caches?

2019-03-08 14:59:03

by Jerome Glisse

[permalink] [raw]
Subject: Re: [RFC PATCH V2 5/5] vhost: access vq metadata through kernel virtual address

On Fri, Mar 08, 2019 at 04:50:36PM +0800, Jason Wang wrote:
>
> On 2019/3/8 上午3:16, Andrea Arcangeli wrote:
> > On Thu, Mar 07, 2019 at 12:56:45PM -0500, Michael S. Tsirkin wrote:
> > > On Thu, Mar 07, 2019 at 10:47:22AM -0500, Michael S. Tsirkin wrote:
> > > > On Wed, Mar 06, 2019 at 02:18:12AM -0500, Jason Wang wrote:
> > > > > +static const struct mmu_notifier_ops vhost_mmu_notifier_ops = {
> > > > > + .invalidate_range = vhost_invalidate_range,
> > > > > +};
> > > > > +
> > > > > void vhost_dev_init(struct vhost_dev *dev,
> > > > > struct vhost_virtqueue **vqs, int nvqs, int iov_limit)
> > > > > {
> > > > I also wonder here: when page is write protected then
> > > > it does not look like .invalidate_range is invoked.
> > > >
> > > > E.g. mm/ksm.c calls
> > > >
> > > > mmu_notifier_invalidate_range_start and
> > > > mmu_notifier_invalidate_range_end but not mmu_notifier_invalidate_range.
> > > >
> > > > Similarly, rmap in page_mkclean_one will not call
> > > > mmu_notifier_invalidate_range.
> > > >
> > > > If I'm right vhost won't get notified when page is write-protected since you
> > > > didn't install start/end notifiers. Note that end notifier can be called
> > > > with page locked, so it's not as straight-forward as just adding a call.
> > > > Writing into a write-protected page isn't a good idea.
> > > >
> > > > Note that documentation says:
> > > > it is fine to delay the mmu_notifier_invalidate_range
> > > > call to mmu_notifier_invalidate_range_end() outside the page table lock.
> > > > implying it's called just later.
> > > OK I missed the fact that _end actually calls
> > > mmu_notifier_invalidate_range internally. So that part is fine but the
> > > fact that you are trying to take page lock under VQ mutex and take same
> > > mutex within notifier probably means it's broken for ksm and rmap at
> > > least since these call invalidate with lock taken.
> > Yes this lock inversion needs more thoughts.
> >
> > > And generally, Andrea told me offline one can not take mutex under
> > > the notifier callback. I CC'd Andrea for why.
> > Yes, the problem then is the ->invalidate_page is called then under PT
> > lock so it cannot take mutex, you also cannot take the page_lock, it
> > can at most take a spinlock or trylock_page.
> >
> > So it must switch back to the _start/_end methods unless you rewrite
> > the locking.
> >
> > The difference with _start/_end, is that ->invalidate_range avoids the
> > _start callback basically, but to avoid the _start callback safely, it
> > has to be called in between the ptep_clear_flush and the set_pte_at
> > whenever the pfn changes like during a COW. So it cannot be coalesced
> > in a single TLB flush that invalidates all sptes in a range like we
> > prefer for performance reasons for example in KVM. It also cannot
> > sleep.
> >
> > In short ->invalidate_range must be really fast (it shouldn't require
> > to send IPI to all other CPUs like KVM may require during an
> > invalidate_range_start) and it must not sleep, in order to prefer it
> > to _start/_end.
> >
> > I.e. the invalidate of the secondary MMU that walks the linux
> > pagetables in hardware (in vhost case with GUP in software) has to
> > happen while the linux pagetable is zero, otherwise a concurrent
> > hardware pagetable lookup could re-instantiate a mapping to the old
> > page in between the set_pte_at and the invalidate_range_end (which
> > internally calls ->invalidate_range). Jerome documented it nicely in
> > Documentation/vm/mmu_notifier.rst .
>
>
> Right, I've actually gone through this several times but some details were
> missed by me obviously.
>
>
> >
> > Now you don't really walk the pagetable in hardware in vhost, but if
> > you use gup_fast after usemm() it's similar.
> >
> > For vhost the invalidate would be really fast, there are no IPI to
> > deliver at all, the problem is just the mutex.
>
>
> Yes. A possible solution is to introduce a valid flag for VA. Vhost may only
> try to access kernel VA when it was valid. Invalidate_range_start() will
> clear this under the protection of the vq mutex when it can block. Then
> invalidate_range_end() then can clear this flag. An issue is blockable is 
> always false for range_end().
>

Note that there can be multiple asynchronous concurrent invalidate_range
callbacks. So a flag does not work but a counter of number of active
invalidation would. See how KSM is doing it for instance in kvm_main.c

The pattern for this kind of thing is:
my_invalidate_range_start(start,end) {
...
if (mystruct_overlap(mystruct, start, end)) {
mystruct_lock();
mystruct->invalidate_count++;
...
mystruct_unlock();
}
}

my_invalidate_range_end(start,end) {
...
if (mystruct_overlap(mystruct, start, end)) {
mystruct_lock();
mystruct->invalidate_count--;
...
mystruct_unlock();
}
}

my_access_va(mystruct) {
again:
wait_on(!mystruct->invalidate_count)
mystruct_lock();
if (mystruct->invalidate_count) {
mystruct_unlock();
goto again;
}
GUP();
...
mystruct_unlock();
}

Cheers,
Jérôme

2019-03-08 15:03:26

by Jerome Glisse

[permalink] [raw]
Subject: Re: [RFC PATCH V2 5/5] vhost: access vq metadata through kernel virtual address

On Fri, Mar 08, 2019 at 07:56:04AM -0500, Michael S. Tsirkin wrote:
> On Fri, Mar 08, 2019 at 04:58:44PM +0800, Jason Wang wrote:
> >
> > On 2019/3/8 上午3:17, Jerome Glisse wrote:
> > > On Thu, Mar 07, 2019 at 12:56:45PM -0500, Michael S. Tsirkin wrote:
> > > > On Thu, Mar 07, 2019 at 10:47:22AM -0500, Michael S. Tsirkin wrote:
> > > > > On Wed, Mar 06, 2019 at 02:18:12AM -0500, Jason Wang wrote:
> > > > > > +static const struct mmu_notifier_ops vhost_mmu_notifier_ops = {
> > > > > > + .invalidate_range = vhost_invalidate_range,
> > > > > > +};
> > > > > > +
> > > > > > void vhost_dev_init(struct vhost_dev *dev,
> > > > > > struct vhost_virtqueue **vqs, int nvqs, int iov_limit)
> > > > > > {
> > > > > I also wonder here: when page is write protected then
> > > > > it does not look like .invalidate_range is invoked.
> > > > >
> > > > > E.g. mm/ksm.c calls
> > > > >
> > > > > mmu_notifier_invalidate_range_start and
> > > > > mmu_notifier_invalidate_range_end but not mmu_notifier_invalidate_range.
> > > > >
> > > > > Similarly, rmap in page_mkclean_one will not call
> > > > > mmu_notifier_invalidate_range.
> > > > >
> > > > > If I'm right vhost won't get notified when page is write-protected since you
> > > > > didn't install start/end notifiers. Note that end notifier can be called
> > > > > with page locked, so it's not as straight-forward as just adding a call.
> > > > > Writing into a write-protected page isn't a good idea.
> > > > >
> > > > > Note that documentation says:
> > > > > it is fine to delay the mmu_notifier_invalidate_range
> > > > > call to mmu_notifier_invalidate_range_end() outside the page table lock.
> > > > > implying it's called just later.
> > > > OK I missed the fact that _end actually calls
> > > > mmu_notifier_invalidate_range internally. So that part is fine but the
> > > > fact that you are trying to take page lock under VQ mutex and take same
> > > > mutex within notifier probably means it's broken for ksm and rmap at
> > > > least since these call invalidate with lock taken.
> > > >
> > > > And generally, Andrea told me offline one can not take mutex under
> > > > the notifier callback. I CC'd Andrea for why.
> > > Correct, you _can not_ take mutex or any sleeping lock from within the
> > > invalidate_range callback as those callback happens under the page table
> > > spinlock. You can however do so under the invalidate_range_start call-
> > > back only if it is a blocking allow callback (there is a flag passdown
> > > with the invalidate_range_start callback if you are not allow to block
> > > then return EBUSY and the invalidation will be aborted).
> > >
> > >
> > > > That's a separate issue from set_page_dirty when memory is file backed.
> > > If you can access file back page then i suggest using set_page_dirty
> > > from within a special version of vunmap() so that when you vunmap you
> > > set the page dirty without taking page lock. It is safe to do so
> > > always from within an mmu notifier callback if you had the page map
> > > with write permission which means that the page had write permission
> > > in the userspace pte too and thus it having dirty pte is expected
> > > and calling set_page_dirty on the page is allowed without any lock.
> > > Locking will happen once the userspace pte are tear down through the
> > > page table lock.
> >
> >
> > Can I simply can set_page_dirty() before vunmap() in the mmu notifier
> > callback, or is there any reason that it must be called within vumap()?
> >
> > Thanks
>
>
> I think this is what Jerome is saying, yes.
> Maybe add a patch to mmu notifier doc file, documenting this?
>

Better to do in vunmap as you can look at kernel vmap pte to see if
the dirty bit is set and only call set_page_dirty in that case. But
yes you can do it outside vunmap in which case you have to call dirty
for all pages unless you have some other way to know if a page was
written to or not.

Note that if you also need to do that when you tear down the vunmap
through the regular path but with an exclusion from mmu notifier.
So if mmu notifier is running then you can skip the set_page_dirty
if none are running and you hold the lock then you can safely call
set_page_dirty.

Cheers,
Jérôme

2019-03-08 19:12:43

by Andrea Arcangeli

[permalink] [raw]
Subject: Re: [RFC PATCH V2 5/5] vhost: access vq metadata through kernel virtual address

On Fri, Mar 08, 2019 at 05:13:26PM +0800, Jason Wang wrote:
> Actually not wrapping around,? the pages for used ring was marked as
> dirty after a round of virtqueue processing when we're sure vhost wrote
> something there.

Thanks for the clarification. So we need to convert it to
set_page_dirty and move it to the mmu notifier invalidate but in those
cases where gup_fast was called with write=1 (1 out of 3).

If using ->invalidate_range the page pin also must be removed
immediately after get_user_pages returns (not ok to hold the pin in
vmap until ->invalidate_range is called) to avoid false positive gup
pin checks in things like KSM, or the pin must be released in
invalidate_range_start (which is called before the pin checks).

Here's why:

/*
* Check that no O_DIRECT or similar I/O is in progress on the
* page
*/
if (page_mapcount(page) + 1 + swapped != page_count(page)) {
set_pte_at(mm, pvmw.address, pvmw.pte, entry);
goto out_unlock;
}
[..]
set_pte_at_notify(mm, pvmw.address, pvmw.pte, entry);
^^^^^^^ too late release the pin here, the
above already failed

->invalidate_range cannot be used with mutex anyway so you need to go
back with invalidate_range_start/end anyway, just the pin must be
released in _start at the latest in such case.

My prefer is generally to call gup_fast() followed by immediate
put_page() because I always want to drop FOLL_GET from gup_fast as a
whole to avoid 2 useless atomic ops per gup_fast.

I'll write more about vmap in answer to the other email.

Thanks,
Andrea

2019-03-08 19:14:31

by Andrea Arcangeli

[permalink] [raw]
Subject: Re: [RFC PATCH V2 5/5] vhost: access vq metadata through kernel virtual address

On Fri, Mar 08, 2019 at 04:58:44PM +0800, Jason Wang wrote:
> Can I simply can set_page_dirty() before vunmap() in the mmu notifier
> callback, or is there any reason that it must be called within vumap()?

I also don't see any problem in doing it before vunmap. As far as the
mmu notifier and set_page_dirty is concerned vunmap is just
put_page. It's just slower and potentially unnecessary.

2019-03-08 19:51:03

by Andrea Arcangeli

[permalink] [raw]
Subject: Re: [RFC PATCH V2 5/5] vhost: access vq metadata through kernel virtual address

Hello Jeson,

On Fri, Mar 08, 2019 at 04:50:36PM +0800, Jason Wang wrote:
> Just to make sure I understand here. For boosting through huge TLB, do
> you mean we can do that in the future (e.g by mapping more userspace
> pages to kenrel) or it can be done by this series (only about three 4K
> pages were vmapped per virtqueue)?

When I answered about the advantages of mmu notifier and I mentioned
guaranteed 2m/gigapages where available, I overlooked the detail you
were using vmap instead of kmap. So with vmap you're actually doing
the opposite, it slows down the access because it will always use a 4k
TLB even if QEMU runs on THP or gigapages hugetlbfs.

If there's just one page (or a few pages) in each vmap there's no need
of vmap, the linearity vmap provides doesn't pay off in such
case.

So likely there's further room for improvement here that you can
achieve in the current series by just dropping vmap/vunmap.

You can just use kmap (or kmap_atomic if you're in preemptible
section, should work from bh/irq).

In short the mmu notifier to invalidate only sets a "struct page *
userringpage" pointer to NULL without calls to vunmap.

In all cases immediately after gup_fast returns you can always call
put_page immediately (which explains why I'd like an option to drop
FOLL_GET from gup_fast to speed it up).

Then you can check the sequence_counter and inc/dec counter increased
by _start/_end. That will tell you if the page you got and you called
put_page to immediately unpin it or even to free it, cannot go away
under you until the invalidate is called.

If sequence counters and counter tells that gup_fast raced with anyt
mmu notifier invalidate you can just repeat gup_fast. Otherwise you're
done, the page cannot go away under you, the host virtual to host
physical mapping cannot change either. And the page is not pinned
either. So you can just set the "struct page * userringpage = page"
where "page" was the one setup by gup_fast.

When later the invalidate runs, you can just call set_page_dirty if
gup_fast was called with "write = 1" and then you clear the pointer
"userringpage = NULL".

When you need to read/write to the memory
kmap/kmap_atomic(userringpage) should work.

In short because there's no hardware involvement here, the established
mapping is just the pointer to the page, there is no need of setting
up any pagetables or to do any TLB flushes (except on 32bit archs if
the page is above the direct mapping but it never happens on 64bit
archs).

Thanks,
Andrea

2019-03-08 20:06:54

by Jerome Glisse

[permalink] [raw]
Subject: Re: [RFC PATCH V2 5/5] vhost: access vq metadata through kernel virtual address

On Fri, Mar 08, 2019 at 02:48:45PM -0500, Andrea Arcangeli wrote:
> Hello Jeson,
>
> On Fri, Mar 08, 2019 at 04:50:36PM +0800, Jason Wang wrote:
> > Just to make sure I understand here. For boosting through huge TLB, do
> > you mean we can do that in the future (e.g by mapping more userspace
> > pages to kenrel) or it can be done by this series (only about three 4K
> > pages were vmapped per virtqueue)?
>
> When I answered about the advantages of mmu notifier and I mentioned
> guaranteed 2m/gigapages where available, I overlooked the detail you
> were using vmap instead of kmap. So with vmap you're actually doing
> the opposite, it slows down the access because it will always use a 4k
> TLB even if QEMU runs on THP or gigapages hugetlbfs.
>
> If there's just one page (or a few pages) in each vmap there's no need
> of vmap, the linearity vmap provides doesn't pay off in such
> case.
>
> So likely there's further room for improvement here that you can
> achieve in the current series by just dropping vmap/vunmap.
>
> You can just use kmap (or kmap_atomic if you're in preemptible
> section, should work from bh/irq).
>
> In short the mmu notifier to invalidate only sets a "struct page *
> userringpage" pointer to NULL without calls to vunmap.
>
> In all cases immediately after gup_fast returns you can always call
> put_page immediately (which explains why I'd like an option to drop
> FOLL_GET from gup_fast to speed it up).

By the way this is on my todo list, i want to merge HMM page snapshoting
with gup code which means mostly allowing to gup_fast without taking a
reference on the page (so without FOLL_GET). I hope to get to that some-
time before summer.

>
> Then you can check the sequence_counter and inc/dec counter increased
> by _start/_end. That will tell you if the page you got and you called
> put_page to immediately unpin it or even to free it, cannot go away
> under you until the invalidate is called.
>
> If sequence counters and counter tells that gup_fast raced with anyt
> mmu notifier invalidate you can just repeat gup_fast. Otherwise you're
> done, the page cannot go away under you, the host virtual to host
> physical mapping cannot change either. And the page is not pinned
> either. So you can just set the "struct page * userringpage = page"
> where "page" was the one setup by gup_fast.
>
> When later the invalidate runs, you can just call set_page_dirty if
> gup_fast was called with "write = 1" and then you clear the pointer
> "userringpage = NULL".
>
> When you need to read/write to the memory
> kmap/kmap_atomic(userringpage) should work.
>
> In short because there's no hardware involvement here, the established
> mapping is just the pointer to the page, there is no need of setting
> up any pagetables or to do any TLB flushes (except on 32bit archs if
> the page is above the direct mapping but it never happens on 64bit
> archs).

Agree. The vmap is probably overkill if you only have a handfull of
them kmap will be faster.

Cheers,
J?r?me

2019-03-11 07:14:10

by Jason Wang

[permalink] [raw]
Subject: Re: [RFC PATCH V2 0/5] vhost: accelerate metadata access through vmap()


On 2019/3/8 下午10:12, Christoph Hellwig wrote:
> On Wed, Mar 06, 2019 at 02:18:07AM -0500, Jason Wang wrote:
>> This series tries to access virtqueue metadata through kernel virtual
>> address instead of copy_user() friends since they had too much
>> overheads like checks, spec barriers or even hardware feature
>> toggling. This is done through setup kernel address through vmap() and
>> resigter MMU notifier for invalidation.
>>
>> Test shows about 24% improvement on TX PPS. TCP_STREAM doesn't see
>> obvious improvement.
> How is this going to work for CPUs with virtually tagged caches?


Anything different that you worry? I can have a test but do you know any
archs that use virtual tag cache?

Thanks


2019-03-11 07:19:45

by Jason Wang

[permalink] [raw]
Subject: Re: [RFC PATCH V2 5/5] vhost: access vq metadata through kernel virtual address


On 2019/3/8 下午10:58, Jerome Glisse wrote:
> On Fri, Mar 08, 2019 at 04:50:36PM +0800, Jason Wang wrote:
>> On 2019/3/8 上午3:16, Andrea Arcangeli wrote:
>>> On Thu, Mar 07, 2019 at 12:56:45PM -0500, Michael S. Tsirkin wrote:
>>>> On Thu, Mar 07, 2019 at 10:47:22AM -0500, Michael S. Tsirkin wrote:
>>>>> On Wed, Mar 06, 2019 at 02:18:12AM -0500, Jason Wang wrote:
>>>>>> +static const struct mmu_notifier_ops vhost_mmu_notifier_ops = {
>>>>>> + .invalidate_range = vhost_invalidate_range,
>>>>>> +};
>>>>>> +
>>>>>> void vhost_dev_init(struct vhost_dev *dev,
>>>>>> struct vhost_virtqueue **vqs, int nvqs, int iov_limit)
>>>>>> {
>>>>> I also wonder here: when page is write protected then
>>>>> it does not look like .invalidate_range is invoked.
>>>>>
>>>>> E.g. mm/ksm.c calls
>>>>>
>>>>> mmu_notifier_invalidate_range_start and
>>>>> mmu_notifier_invalidate_range_end but not mmu_notifier_invalidate_range.
>>>>>
>>>>> Similarly, rmap in page_mkclean_one will not call
>>>>> mmu_notifier_invalidate_range.
>>>>>
>>>>> If I'm right vhost won't get notified when page is write-protected since you
>>>>> didn't install start/end notifiers. Note that end notifier can be called
>>>>> with page locked, so it's not as straight-forward as just adding a call.
>>>>> Writing into a write-protected page isn't a good idea.
>>>>>
>>>>> Note that documentation says:
>>>>> it is fine to delay the mmu_notifier_invalidate_range
>>>>> call to mmu_notifier_invalidate_range_end() outside the page table lock.
>>>>> implying it's called just later.
>>>> OK I missed the fact that _end actually calls
>>>> mmu_notifier_invalidate_range internally. So that part is fine but the
>>>> fact that you are trying to take page lock under VQ mutex and take same
>>>> mutex within notifier probably means it's broken for ksm and rmap at
>>>> least since these call invalidate with lock taken.
>>> Yes this lock inversion needs more thoughts.
>>>
>>>> And generally, Andrea told me offline one can not take mutex under
>>>> the notifier callback. I CC'd Andrea for why.
>>> Yes, the problem then is the ->invalidate_page is called then under PT
>>> lock so it cannot take mutex, you also cannot take the page_lock, it
>>> can at most take a spinlock or trylock_page.
>>>
>>> So it must switch back to the _start/_end methods unless you rewrite
>>> the locking.
>>>
>>> The difference with _start/_end, is that ->invalidate_range avoids the
>>> _start callback basically, but to avoid the _start callback safely, it
>>> has to be called in between the ptep_clear_flush and the set_pte_at
>>> whenever the pfn changes like during a COW. So it cannot be coalesced
>>> in a single TLB flush that invalidates all sptes in a range like we
>>> prefer for performance reasons for example in KVM. It also cannot
>>> sleep.
>>>
>>> In short ->invalidate_range must be really fast (it shouldn't require
>>> to send IPI to all other CPUs like KVM may require during an
>>> invalidate_range_start) and it must not sleep, in order to prefer it
>>> to _start/_end.
>>>
>>> I.e. the invalidate of the secondary MMU that walks the linux
>>> pagetables in hardware (in vhost case with GUP in software) has to
>>> happen while the linux pagetable is zero, otherwise a concurrent
>>> hardware pagetable lookup could re-instantiate a mapping to the old
>>> page in between the set_pte_at and the invalidate_range_end (which
>>> internally calls ->invalidate_range). Jerome documented it nicely in
>>> Documentation/vm/mmu_notifier.rst .
>>
>> Right, I've actually gone through this several times but some details were
>> missed by me obviously.
>>
>>
>>> Now you don't really walk the pagetable in hardware in vhost, but if
>>> you use gup_fast after usemm() it's similar.
>>>
>>> For vhost the invalidate would be really fast, there are no IPI to
>>> deliver at all, the problem is just the mutex.
>>
>> Yes. A possible solution is to introduce a valid flag for VA. Vhost may only
>> try to access kernel VA when it was valid. Invalidate_range_start() will
>> clear this under the protection of the vq mutex when it can block. Then
>> invalidate_range_end() then can clear this flag. An issue is blockable is
>> always false for range_end().
>>
> Note that there can be multiple asynchronous concurrent invalidate_range
> callbacks. So a flag does not work but a counter of number of active
> invalidation would. See how KSM is doing it for instance in kvm_main.c
>
> The pattern for this kind of thing is:
> my_invalidate_range_start(start,end) {
> ...
> if (mystruct_overlap(mystruct, start, end)) {
> mystruct_lock();
> mystruct->invalidate_count++;
> ...
> mystruct_unlock();
> }
> }
>
> my_invalidate_range_end(start,end) {
> ...
> if (mystruct_overlap(mystruct, start, end)) {
> mystruct_lock();
> mystruct->invalidate_count--;
> ...
> mystruct_unlock();
> }
> }
>
> my_access_va(mystruct) {
> again:
> wait_on(!mystruct->invalidate_count)
> mystruct_lock();
> if (mystruct->invalidate_count) {
> mystruct_unlock();
> goto again;
> }
> GUP();
> ...
> mystruct_unlock();
> }
>
> Cheers,
> Jérôme


Yes, this should work.

Thanks


2019-03-11 07:23:59

by Jason Wang

[permalink] [raw]
Subject: Re: [RFC PATCH V2 5/5] vhost: access vq metadata through kernel virtual address


On 2019/3/9 上午3:11, Andrea Arcangeli wrote:
> On Fri, Mar 08, 2019 at 05:13:26PM +0800, Jason Wang wrote:
>> Actually not wrapping around,  the pages for used ring was marked as
>> dirty after a round of virtqueue processing when we're sure vhost wrote
>> something there.
> Thanks for the clarification. So we need to convert it to
> set_page_dirty and move it to the mmu notifier invalidate but in those
> cases where gup_fast was called with write=1 (1 out of 3).
>
> If using ->invalidate_range the page pin also must be removed
> immediately after get_user_pages returns (not ok to hold the pin in
> vmap until ->invalidate_range is called) to avoid false positive gup
> pin checks in things like KSM, or the pin must be released in
> invalidate_range_start (which is called before the pin checks).
>
> Here's why:
>
> /*
> * Check that no O_DIRECT or similar I/O is in progress on the
> * page
> */
> if (page_mapcount(page) + 1 + swapped != page_count(page)) {
> set_pte_at(mm, pvmw.address, pvmw.pte, entry);
> goto out_unlock;
> }
> [..]
> set_pte_at_notify(mm, pvmw.address, pvmw.pte, entry);
> ^^^^^^^ too late release the pin here, the
> above already failed
>
> ->invalidate_range cannot be used with mutex anyway so you need to go
> back with invalidate_range_start/end anyway, just the pin must be
> released in _start at the latest in such case.


Yes.


>
> My prefer is generally to call gup_fast() followed by immediate
> put_page() because I always want to drop FOLL_GET from gup_fast as a
> whole to avoid 2 useless atomic ops per gup_fast.


Ok, will do this (if I still plan to use vmap() in next version).


>
> I'll write more about vmap in answer to the other email.
>
> Thanks,
> Andrea


Thanks


2019-03-11 07:41:25

by Jason Wang

[permalink] [raw]
Subject: Re: [RFC PATCH V2 5/5] vhost: access vq metadata through kernel virtual address


On 2019/3/9 上午3:48, Andrea Arcangeli wrote:
> Hello Jeson,
>
> On Fri, Mar 08, 2019 at 04:50:36PM +0800, Jason Wang wrote:
>> Just to make sure I understand here. For boosting through huge TLB, do
>> you mean we can do that in the future (e.g by mapping more userspace
>> pages to kenrel) or it can be done by this series (only about three 4K
>> pages were vmapped per virtqueue)?
> When I answered about the advantages of mmu notifier and I mentioned
> guaranteed 2m/gigapages where available, I overlooked the detail you
> were using vmap instead of kmap. So with vmap you're actually doing
> the opposite, it slows down the access because it will always use a 4k
> TLB even if QEMU runs on THP or gigapages hugetlbfs.
>
> If there's just one page (or a few pages) in each vmap there's no need
> of vmap, the linearity vmap provides doesn't pay off in such
> case.
>
> So likely there's further room for improvement here that you can
> achieve in the current series by just dropping vmap/vunmap.
>
> You can just use kmap (or kmap_atomic if you're in preemptible
> section, should work from bh/irq).
>
> In short the mmu notifier to invalidate only sets a "struct page *
> userringpage" pointer to NULL without calls to vunmap.
>
> In all cases immediately after gup_fast returns you can always call
> put_page immediately (which explains why I'd like an option to drop
> FOLL_GET from gup_fast to speed it up).
>
> Then you can check the sequence_counter and inc/dec counter increased
> by _start/_end. That will tell you if the page you got and you called
> put_page to immediately unpin it or even to free it, cannot go away
> under you until the invalidate is called.
>
> If sequence counters and counter tells that gup_fast raced with anyt
> mmu notifier invalidate you can just repeat gup_fast. Otherwise you're
> done, the page cannot go away under you, the host virtual to host
> physical mapping cannot change either. And the page is not pinned
> either. So you can just set the "struct page * userringpage = page"
> where "page" was the one setup by gup_fast.
>
> When later the invalidate runs, you can just call set_page_dirty if
> gup_fast was called with "write = 1" and then you clear the pointer
> "userringpage = NULL".
>
> When you need to read/write to the memory
> kmap/kmap_atomic(userringpage) should work.


Yes, I've considered kmap() from the start. The reason I don't do that
is large virtqueue may need more than one page so VA might not be
contiguous. But this is probably not a big issue which just need more
tricks in the vhost memory accessors.


>
> In short because there's no hardware involvement here, the established
> mapping is just the pointer to the page, there is no need of setting
> up any pagetables or to do any TLB flushes (except on 32bit archs if
> the page is above the direct mapping but it never happens on 64bit
> archs).


I see, I believe we don't care much about the performance of 32bit archs
(or we can just fallback to copy_to_user() friends). Using direct
mapping (I guess kernel will always try hugepage for that?) should be
better and we can even use it for the data transfer not only for the
metadata.

Thanks


>
> Thanks,
> Andrea

2019-03-11 12:50:22

by Michael S. Tsirkin

[permalink] [raw]
Subject: Re: [RFC PATCH V2 5/5] vhost: access vq metadata through kernel virtual address

On Mon, Mar 11, 2019 at 03:40:31PM +0800, Jason Wang wrote:
>
> On 2019/3/9 上午3:48, Andrea Arcangeli wrote:
> > Hello Jeson,
> >
> > On Fri, Mar 08, 2019 at 04:50:36PM +0800, Jason Wang wrote:
> > > Just to make sure I understand here. For boosting through huge TLB, do
> > > you mean we can do that in the future (e.g by mapping more userspace
> > > pages to kenrel) or it can be done by this series (only about three 4K
> > > pages were vmapped per virtqueue)?
> > When I answered about the advantages of mmu notifier and I mentioned
> > guaranteed 2m/gigapages where available, I overlooked the detail you
> > were using vmap instead of kmap. So with vmap you're actually doing
> > the opposite, it slows down the access because it will always use a 4k
> > TLB even if QEMU runs on THP or gigapages hugetlbfs.
> >
> > If there's just one page (or a few pages) in each vmap there's no need
> > of vmap, the linearity vmap provides doesn't pay off in such
> > case.
> >
> > So likely there's further room for improvement here that you can
> > achieve in the current series by just dropping vmap/vunmap.
> >
> > You can just use kmap (or kmap_atomic if you're in preemptible
> > section, should work from bh/irq).
> >
> > In short the mmu notifier to invalidate only sets a "struct page *
> > userringpage" pointer to NULL without calls to vunmap.
> >
> > In all cases immediately after gup_fast returns you can always call
> > put_page immediately (which explains why I'd like an option to drop
> > FOLL_GET from gup_fast to speed it up).
> >
> > Then you can check the sequence_counter and inc/dec counter increased
> > by _start/_end. That will tell you if the page you got and you called
> > put_page to immediately unpin it or even to free it, cannot go away
> > under you until the invalidate is called.
> >
> > If sequence counters and counter tells that gup_fast raced with anyt
> > mmu notifier invalidate you can just repeat gup_fast. Otherwise you're
> > done, the page cannot go away under you, the host virtual to host
> > physical mapping cannot change either. And the page is not pinned
> > either. So you can just set the "struct page * userringpage = page"
> > where "page" was the one setup by gup_fast.
> >
> > When later the invalidate runs, you can just call set_page_dirty if
> > gup_fast was called with "write = 1" and then you clear the pointer
> > "userringpage = NULL".
> >
> > When you need to read/write to the memory
> > kmap/kmap_atomic(userringpage) should work.
>
>
> Yes, I've considered kmap() from the start. The reason I don't do that is
> large virtqueue may need more than one page so VA might not be contiguous.
> But this is probably not a big issue which just need more tricks in the
> vhost memory accessors.
>
>
> >
> > In short because there's no hardware involvement here, the established
> > mapping is just the pointer to the page, there is no need of setting
> > up any pagetables or to do any TLB flushes (except on 32bit archs if
> > the page is above the direct mapping but it never happens on 64bit
> > archs).
>
>
> I see, I believe we don't care much about the performance of 32bit archs (or
> we can just fallback to copy_to_user() friends).

Using copyXuser is better I guess.

> Using direct mapping (I
> guess kernel will always try hugepage for that?) should be better and we can
> even use it for the data transfer not only for the metadata.
>
> Thanks

We can't really. The big issue is get user pages. Doing that on data
path will be slower than copyXuser. Or maybe it won't with the
amount of mitigations spread around. Go ahead and try.


>
> >
> > Thanks,
> > Andrea

2019-03-11 13:45:11

by Andrea Arcangeli

[permalink] [raw]
Subject: Re: [RFC PATCH V2 5/5] vhost: access vq metadata through kernel virtual address

On Mon, Mar 11, 2019 at 08:48:37AM -0400, Michael S. Tsirkin wrote:
> Using copyXuser is better I guess.

It certainly would be faster there, but I don't think it's needed if
that would be the only use case left that justifies supporting two
different models. On small 32bit systems with little RAM kmap won't
perform measurably different on 32bit or 64bit systems. If the 32bit
host has a lot of ram it all gets slow anyway at accessing RAM above
the direct mapping, if compared to 64bit host kernels, it's not just
an issue for vhost + mmu notifier + kmap and the best way to optimize
things is to run 64bit host kernels.

Like Christoph pointed out, the main use case for retaining the
copy-user model would be CPUs with virtually indexed not physically
tagged data caches (they'll still suffer from the spectre-v1 fix,
although I exclude they have to suffer the SMAP
slowdown/feature). Those may require some additional flushing than the
current copy-user model requires.

As a rule of thumb any arch where copy_user_page doesn't define as
copy_page will require some additional cache flushing after the
kmap. Supposedly with vmap, the vmap layer should have taken care of
that (I didn't verify that yet).

There are some accessories like copy_to_user_page()
copy_from_user_page() that could work and obviously defines to raw
memcpy on x86 (the main cons is they don't provide word granular
access) and at least on sparc they're tailored to ptrace assumptions
so then we'd need to evaluate what happens if this is used outside of
ptrace context. kmap has been used generally either to access whole
pages (i.e. copy_user_page), so ptrace may actually be the only use
case with subpage granularity access.

#define copy_to_user_page(vma, page, vaddr, dst, src, len) \
do { \
flush_cache_page(vma, vaddr, page_to_pfn(page)); \
memcpy(dst, src, len); \
flush_ptrace_access(vma, page, vaddr, src, len, 0); \
} while (0)

So I wouldn't rule out the need for a dual model, until we solve how
to run this stable on non-x86 arches with not physically tagged
caches.

Thanks,
Andrea

2019-03-11 14:00:20

by Michael S. Tsirkin

[permalink] [raw]
Subject: Re: [RFC PATCH V2 0/5] vhost: accelerate metadata access through vmap()

On Mon, Mar 11, 2019 at 03:13:17PM +0800, Jason Wang wrote:
>
> On 2019/3/8 下午10:12, Christoph Hellwig wrote:
> > On Wed, Mar 06, 2019 at 02:18:07AM -0500, Jason Wang wrote:
> > > This series tries to access virtqueue metadata through kernel virtual
> > > address instead of copy_user() friends since they had too much
> > > overheads like checks, spec barriers or even hardware feature
> > > toggling. This is done through setup kernel address through vmap() and
> > > resigter MMU notifier for invalidation.
> > >
> > > Test shows about 24% improvement on TX PPS. TCP_STREAM doesn't see
> > > obvious improvement.
> > How is this going to work for CPUs with virtually tagged caches?
>
>
> Anything different that you worry?

If caches have virtual tags then kernel and userspace view of memory
might not be automatically in sync if they access memory
through different virtual addresses. You need to do things like
flush_cache_page, probably multiple times.

> I can have a test but do you know any
> archs that use virtual tag cache?

sparc I believe.

> Thanks




2019-03-11 14:47:43

by Jan Kara

[permalink] [raw]
Subject: Re: [RFC PATCH V2 5/5] vhost: access vq metadata through kernel virtual address

On Thu 07-03-19 16:27:17, Andrea Arcangeli wrote:
> > driver that GUP page for hours/days/weeks/months ... obviously the
> > race window is big enough here. It affects many fs (ext4, xfs, ...)
> > in different ways. I think ext4 is the most obvious because of the
> > kernel log trace it leaves behind.
> >
> > Bottom line is for set_page_dirty to be safe you need the following:
> > lock_page()
> > page_mkwrite()
> > set_pte_with_write()
> > unlock_page()
>
> I also wondered why ext4 writepage doesn't recreate the bh if they got
> dropped by the VM and page->private is 0. I mean, page->index and
> page->mapping are still there, that's enough info for writepage itself
> to take a slow path and calls page_mkwrite to find where to write the
> page on disk.

There are two problems:

1) What to do with errors that page_mkwrite() can generate (ENOMEM, ENOSPC,
EIO). On page fault you just propagate them to userspace, on set_page_dirty()
you have no chance so you just silently loose data.

2) We need various locks to protect page_mkwrite(), possibly do some IO.
set_page_dirty() is rather uncertain context to acquire locks or do IO...

Honza
--
Jan Kara <[email protected]>
SUSE Labs, CR

2019-03-11 18:14:58

by David Miller

[permalink] [raw]
Subject: Re: [RFC PATCH V2 0/5] vhost: accelerate metadata access through vmap()

From: "Michael S. Tsirkin" <[email protected]>
Date: Mon, 11 Mar 2019 09:59:28 -0400

> On Mon, Mar 11, 2019 at 03:13:17PM +0800, Jason Wang wrote:
>>
>> On 2019/3/8 $B2<8a(B10:12, Christoph Hellwig wrote:
>> > On Wed, Mar 06, 2019 at 02:18:07AM -0500, Jason Wang wrote:
>> > > This series tries to access virtqueue metadata through kernel virtual
>> > > address instead of copy_user() friends since they had too much
>> > > overheads like checks, spec barriers or even hardware feature
>> > > toggling. This is done through setup kernel address through vmap() and
>> > > resigter MMU notifier for invalidation.
>> > >
>> > > Test shows about 24% improvement on TX PPS. TCP_STREAM doesn't see
>> > > obvious improvement.
>> > How is this going to work for CPUs with virtually tagged caches?
>>
>>
>> Anything different that you worry?
>
> If caches have virtual tags then kernel and userspace view of memory
> might not be automatically in sync if they access memory
> through different virtual addresses. You need to do things like
> flush_cache_page, probably multiple times.

"flush_dcache_page()"

2019-03-12 02:54:20

by Jason Wang

[permalink] [raw]
Subject: Re: [RFC PATCH V2 5/5] vhost: access vq metadata through kernel virtual address


On 2019/3/11 下午8:48, Michael S. Tsirkin wrote:
> On Mon, Mar 11, 2019 at 03:40:31PM +0800, Jason Wang wrote:
>> On 2019/3/9 上午3:48, Andrea Arcangeli wrote:
>>> Hello Jeson,
>>>
>>> On Fri, Mar 08, 2019 at 04:50:36PM +0800, Jason Wang wrote:
>>>> Just to make sure I understand here. For boosting through huge TLB, do
>>>> you mean we can do that in the future (e.g by mapping more userspace
>>>> pages to kenrel) or it can be done by this series (only about three 4K
>>>> pages were vmapped per virtqueue)?
>>> When I answered about the advantages of mmu notifier and I mentioned
>>> guaranteed 2m/gigapages where available, I overlooked the detail you
>>> were using vmap instead of kmap. So with vmap you're actually doing
>>> the opposite, it slows down the access because it will always use a 4k
>>> TLB even if QEMU runs on THP or gigapages hugetlbfs.
>>>
>>> If there's just one page (or a few pages) in each vmap there's no need
>>> of vmap, the linearity vmap provides doesn't pay off in such
>>> case.
>>>
>>> So likely there's further room for improvement here that you can
>>> achieve in the current series by just dropping vmap/vunmap.
>>>
>>> You can just use kmap (or kmap_atomic if you're in preemptible
>>> section, should work from bh/irq).
>>>
>>> In short the mmu notifier to invalidate only sets a "struct page *
>>> userringpage" pointer to NULL without calls to vunmap.
>>>
>>> In all cases immediately after gup_fast returns you can always call
>>> put_page immediately (which explains why I'd like an option to drop
>>> FOLL_GET from gup_fast to speed it up).
>>>
>>> Then you can check the sequence_counter and inc/dec counter increased
>>> by _start/_end. That will tell you if the page you got and you called
>>> put_page to immediately unpin it or even to free it, cannot go away
>>> under you until the invalidate is called.
>>>
>>> If sequence counters and counter tells that gup_fast raced with anyt
>>> mmu notifier invalidate you can just repeat gup_fast. Otherwise you're
>>> done, the page cannot go away under you, the host virtual to host
>>> physical mapping cannot change either. And the page is not pinned
>>> either. So you can just set the "struct page * userringpage = page"
>>> where "page" was the one setup by gup_fast.
>>>
>>> When later the invalidate runs, you can just call set_page_dirty if
>>> gup_fast was called with "write = 1" and then you clear the pointer
>>> "userringpage = NULL".
>>>
>>> When you need to read/write to the memory
>>> kmap/kmap_atomic(userringpage) should work.
>> Yes, I've considered kmap() from the start. The reason I don't do that is
>> large virtqueue may need more than one page so VA might not be contiguous.
>> But this is probably not a big issue which just need more tricks in the
>> vhost memory accessors.
>>
>>
>>> In short because there's no hardware involvement here, the established
>>> mapping is just the pointer to the page, there is no need of setting
>>> up any pagetables or to do any TLB flushes (except on 32bit archs if
>>> the page is above the direct mapping but it never happens on 64bit
>>> archs).
>> I see, I believe we don't care much about the performance of 32bit archs (or
>> we can just fallback to copy_to_user() friends).
> Using copyXuser is better I guess.


Ok.


>
>> Using direct mapping (I
>> guess kernel will always try hugepage for that?) should be better and we can
>> even use it for the data transfer not only for the metadata.
>>
>> Thanks
> We can't really. The big issue is get user pages. Doing that on data
> path will be slower than copyXuser.


I meant if we can find a way to avoid doing gup in datapath. E.g vhost
maintain a range tree and add or remove ranges through MMU notifier.
Then in datapath, if we find the range, then use direct mapping
otherwise copy_to_user().

Thanks


> Or maybe it won't with the
> amount of mitigations spread around. Go ahead and try.
>
>

2019-03-12 02:58:23

by Jason Wang

[permalink] [raw]
Subject: Re: [RFC PATCH V2 5/5] vhost: access vq metadata through kernel virtual address


On 2019/3/11 下午9:43, Andrea Arcangeli wrote:
> On Mon, Mar 11, 2019 at 08:48:37AM -0400, Michael S. Tsirkin wrote:
>> Using copyXuser is better I guess.
> It certainly would be faster there, but I don't think it's needed if
> that would be the only use case left that justifies supporting two
> different models. On small 32bit systems with little RAM kmap won't
> perform measurably different on 32bit or 64bit systems. If the 32bit
> host has a lot of ram it all gets slow anyway at accessing RAM above
> the direct mapping, if compared to 64bit host kernels, it's not just
> an issue for vhost + mmu notifier + kmap and the best way to optimize
> things is to run 64bit host kernels.
>
> Like Christoph pointed out, the main use case for retaining the
> copy-user model would be CPUs with virtually indexed not physically
> tagged data caches (they'll still suffer from the spectre-v1 fix,
> although I exclude they have to suffer the SMAP
> slowdown/feature). Those may require some additional flushing than the
> current copy-user model requires.
>
> As a rule of thumb any arch where copy_user_page doesn't define as
> copy_page will require some additional cache flushing after the
> kmap. Supposedly with vmap, the vmap layer should have taken care of
> that (I didn't verify that yet).


vmap_page_range()/free_unmap_vmap_area() will call
fluch_cache_vmap()/flush_cache_vunmap(). So vmap layer should be ok.

Thanks


>
> There are some accessories like copy_to_user_page()
> copy_from_user_page() that could work and obviously defines to raw
> memcpy on x86 (the main cons is they don't provide word granular
> access) and at least on sparc they're tailored to ptrace assumptions
> so then we'd need to evaluate what happens if this is used outside of
> ptrace context. kmap has been used generally either to access whole
> pages (i.e. copy_user_page), so ptrace may actually be the only use
> case with subpage granularity access.
>
> #define copy_to_user_page(vma, page, vaddr, dst, src, len) \
> do { \
> flush_cache_page(vma, vaddr, page_to_pfn(page)); \
> memcpy(dst, src, len); \
> flush_ptrace_access(vma, page, vaddr, src, len, 0); \
> } while (0)
>
> So I wouldn't rule out the need for a dual model, until we solve how
> to run this stable on non-x86 arches with not physically tagged
> caches.
>
> Thanks,
> Andrea

2019-03-12 03:01:16

by Jason Wang

[permalink] [raw]
Subject: Re: [RFC PATCH V2 0/5] vhost: accelerate metadata access through vmap()


On 2019/3/12 上午2:14, David Miller wrote:
> From: "Michael S. Tsirkin" <[email protected]>
> Date: Mon, 11 Mar 2019 09:59:28 -0400
>
>> On Mon, Mar 11, 2019 at 03:13:17PM +0800, Jason Wang wrote:
>>> On 2019/3/8 下午10:12, Christoph Hellwig wrote:
>>>> On Wed, Mar 06, 2019 at 02:18:07AM -0500, Jason Wang wrote:
>>>>> This series tries to access virtqueue metadata through kernel virtual
>>>>> address instead of copy_user() friends since they had too much
>>>>> overheads like checks, spec barriers or even hardware feature
>>>>> toggling. This is done through setup kernel address through vmap() and
>>>>> resigter MMU notifier for invalidation.
>>>>>
>>>>> Test shows about 24% improvement on TX PPS. TCP_STREAM doesn't see
>>>>> obvious improvement.
>>>> How is this going to work for CPUs with virtually tagged caches?
>>>
>>> Anything different that you worry?
>> If caches have virtual tags then kernel and userspace view of memory
>> might not be automatically in sync if they access memory
>> through different virtual addresses. You need to do things like
>> flush_cache_page, probably multiple times.
> "flush_dcache_page()"


I get this. Then I think the current set_bit_to_user() is suspicious, we
probably miss a flush_dcache_page() there:


static int set_bit_to_user(int nr, void __user *addr)
{
        unsigned long log = (unsigned long)addr;
        struct page *page;
        void *base;
        int bit = nr + (log % PAGE_SIZE) * 8;
        int r;

        r = get_user_pages_fast(log, 1, 1, &page);
        if (r < 0)
                return r;
        BUG_ON(r != 1);
        base = kmap_atomic(page);
        set_bit(bit, base);
        kunmap_atomic(base);
        set_page_dirty_lock(page);
        put_page(page);
        return 0;
}

Thanks


2019-03-12 03:51:29

by Michael S. Tsirkin

[permalink] [raw]
Subject: Re: [RFC PATCH V2 5/5] vhost: access vq metadata through kernel virtual address

On Tue, Mar 12, 2019 at 10:52:15AM +0800, Jason Wang wrote:
>
> On 2019/3/11 下午8:48, Michael S. Tsirkin wrote:
> > On Mon, Mar 11, 2019 at 03:40:31PM +0800, Jason Wang wrote:
> > > On 2019/3/9 上午3:48, Andrea Arcangeli wrote:
> > > > Hello Jeson,
> > > >
> > > > On Fri, Mar 08, 2019 at 04:50:36PM +0800, Jason Wang wrote:
> > > > > Just to make sure I understand here. For boosting through huge TLB, do
> > > > > you mean we can do that in the future (e.g by mapping more userspace
> > > > > pages to kenrel) or it can be done by this series (only about three 4K
> > > > > pages were vmapped per virtqueue)?
> > > > When I answered about the advantages of mmu notifier and I mentioned
> > > > guaranteed 2m/gigapages where available, I overlooked the detail you
> > > > were using vmap instead of kmap. So with vmap you're actually doing
> > > > the opposite, it slows down the access because it will always use a 4k
> > > > TLB even if QEMU runs on THP or gigapages hugetlbfs.
> > > >
> > > > If there's just one page (or a few pages) in each vmap there's no need
> > > > of vmap, the linearity vmap provides doesn't pay off in such
> > > > case.
> > > >
> > > > So likely there's further room for improvement here that you can
> > > > achieve in the current series by just dropping vmap/vunmap.
> > > >
> > > > You can just use kmap (or kmap_atomic if you're in preemptible
> > > > section, should work from bh/irq).
> > > >
> > > > In short the mmu notifier to invalidate only sets a "struct page *
> > > > userringpage" pointer to NULL without calls to vunmap.
> > > >
> > > > In all cases immediately after gup_fast returns you can always call
> > > > put_page immediately (which explains why I'd like an option to drop
> > > > FOLL_GET from gup_fast to speed it up).
> > > >
> > > > Then you can check the sequence_counter and inc/dec counter increased
> > > > by _start/_end. That will tell you if the page you got and you called
> > > > put_page to immediately unpin it or even to free it, cannot go away
> > > > under you until the invalidate is called.
> > > >
> > > > If sequence counters and counter tells that gup_fast raced with anyt
> > > > mmu notifier invalidate you can just repeat gup_fast. Otherwise you're
> > > > done, the page cannot go away under you, the host virtual to host
> > > > physical mapping cannot change either. And the page is not pinned
> > > > either. So you can just set the "struct page * userringpage = page"
> > > > where "page" was the one setup by gup_fast.
> > > >
> > > > When later the invalidate runs, you can just call set_page_dirty if
> > > > gup_fast was called with "write = 1" and then you clear the pointer
> > > > "userringpage = NULL".
> > > >
> > > > When you need to read/write to the memory
> > > > kmap/kmap_atomic(userringpage) should work.
> > > Yes, I've considered kmap() from the start. The reason I don't do that is
> > > large virtqueue may need more than one page so VA might not be contiguous.
> > > But this is probably not a big issue which just need more tricks in the
> > > vhost memory accessors.
> > >
> > >
> > > > In short because there's no hardware involvement here, the established
> > > > mapping is just the pointer to the page, there is no need of setting
> > > > up any pagetables or to do any TLB flushes (except on 32bit archs if
> > > > the page is above the direct mapping but it never happens on 64bit
> > > > archs).
> > > I see, I believe we don't care much about the performance of 32bit archs (or
> > > we can just fallback to copy_to_user() friends).
> > Using copyXuser is better I guess.
>
>
> Ok.
>
>
> >
> > > Using direct mapping (I
> > > guess kernel will always try hugepage for that?) should be better and we can
> > > even use it for the data transfer not only for the metadata.
> > >
> > > Thanks
> > We can't really. The big issue is get user pages. Doing that on data
> > path will be slower than copyXuser.
>
>
> I meant if we can find a way to avoid doing gup in datapath. E.g vhost
> maintain a range tree and add or remove ranges through MMU notifier. Then in
> datapath, if we find the range, then use direct mapping otherwise
> copy_to_user().
>
> Thanks

We can try. But I'm not sure there's any reason to think there's any
locality there.

>
> > Or maybe it won't with the
> > amount of mitigations spread around. Go ahead and try.
> >
> >

2019-03-12 03:53:56

by Michael S. Tsirkin

[permalink] [raw]
Subject: Re: [RFC PATCH V2 5/5] vhost: access vq metadata through kernel virtual address

On Tue, Mar 12, 2019 at 10:56:20AM +0800, Jason Wang wrote:
>
> On 2019/3/11 下午9:43, Andrea Arcangeli wrote:
> > On Mon, Mar 11, 2019 at 08:48:37AM -0400, Michael S. Tsirkin wrote:
> > > Using copyXuser is better I guess.
> > It certainly would be faster there, but I don't think it's needed if
> > that would be the only use case left that justifies supporting two
> > different models. On small 32bit systems with little RAM kmap won't
> > perform measurably different on 32bit or 64bit systems. If the 32bit
> > host has a lot of ram it all gets slow anyway at accessing RAM above
> > the direct mapping, if compared to 64bit host kernels, it's not just
> > an issue for vhost + mmu notifier + kmap and the best way to optimize
> > things is to run 64bit host kernels.
> >
> > Like Christoph pointed out, the main use case for retaining the
> > copy-user model would be CPUs with virtually indexed not physically
> > tagged data caches (they'll still suffer from the spectre-v1 fix,
> > although I exclude they have to suffer the SMAP
> > slowdown/feature). Those may require some additional flushing than the
> > current copy-user model requires.
> >
> > As a rule of thumb any arch where copy_user_page doesn't define as
> > copy_page will require some additional cache flushing after the
> > kmap. Supposedly with vmap, the vmap layer should have taken care of
> > that (I didn't verify that yet).
>
>
> vmap_page_range()/free_unmap_vmap_area() will call
> fluch_cache_vmap()/flush_cache_vunmap(). So vmap layer should be ok.
>
> Thanks

You only unmap from mmu notifier though.
You don't do it after any access.

>
> >
> > There are some accessories like copy_to_user_page()
> > copy_from_user_page() that could work and obviously defines to raw
> > memcpy on x86 (the main cons is they don't provide word granular
> > access) and at least on sparc they're tailored to ptrace assumptions
> > so then we'd need to evaluate what happens if this is used outside of
> > ptrace context. kmap has been used generally either to access whole
> > pages (i.e. copy_user_page), so ptrace may actually be the only use
> > case with subpage granularity access.
> >
> > #define copy_to_user_page(vma, page, vaddr, dst, src, len) \
> > do { \
> > flush_cache_page(vma, vaddr, page_to_pfn(page)); \
> > memcpy(dst, src, len); \
> > flush_ptrace_access(vma, page, vaddr, src, len, 0); \
> > } while (0)
> >
> > So I wouldn't rule out the need for a dual model, until we solve how
> > to run this stable on non-x86 arches with not physically tagged
> > caches.
> >
> > Thanks,
> > Andrea

2019-03-12 03:54:23

by Michael S. Tsirkin

[permalink] [raw]
Subject: Re: [RFC PATCH V2 0/5] vhost: accelerate metadata access through vmap()

On Tue, Mar 12, 2019 at 10:59:09AM +0800, Jason Wang wrote:
>
> On 2019/3/12 上午2:14, David Miller wrote:
> > From: "Michael S. Tsirkin" <[email protected]>
> > Date: Mon, 11 Mar 2019 09:59:28 -0400
> >
> > > On Mon, Mar 11, 2019 at 03:13:17PM +0800, Jason Wang wrote:
> > > > On 2019/3/8 下午10:12, Christoph Hellwig wrote:
> > > > > On Wed, Mar 06, 2019 at 02:18:07AM -0500, Jason Wang wrote:
> > > > > > This series tries to access virtqueue metadata through kernel virtual
> > > > > > address instead of copy_user() friends since they had too much
> > > > > > overheads like checks, spec barriers or even hardware feature
> > > > > > toggling. This is done through setup kernel address through vmap() and
> > > > > > resigter MMU notifier for invalidation.
> > > > > >
> > > > > > Test shows about 24% improvement on TX PPS. TCP_STREAM doesn't see
> > > > > > obvious improvement.
> > > > > How is this going to work for CPUs with virtually tagged caches?
> > > >
> > > > Anything different that you worry?
> > > If caches have virtual tags then kernel and userspace view of memory
> > > might not be automatically in sync if they access memory
> > > through different virtual addresses. You need to do things like
> > > flush_cache_page, probably multiple times.
> > "flush_dcache_page()"
>
>
> I get this. Then I think the current set_bit_to_user() is suspicious, we
> probably miss a flush_dcache_page() there:
>
>
> static int set_bit_to_user(int nr, void __user *addr)
> {
>         unsigned long log = (unsigned long)addr;
>         struct page *page;
>         void *base;
>         int bit = nr + (log % PAGE_SIZE) * 8;
>         int r;
>
>         r = get_user_pages_fast(log, 1, 1, &page);
>         if (r < 0)
>                 return r;
>         BUG_ON(r != 1);
>         base = kmap_atomic(page);
>         set_bit(bit, base);
>         kunmap_atomic(base);
>         set_page_dirty_lock(page);
>         put_page(page);
>         return 0;
> }
>
> Thanks

I think you are right. The correct fix though is to re-implement
it using asm and handling pagefault, not gup.
Three atomic ops per bit is way to expensive.

--
MST

2019-03-12 05:15:25

by James Bottomley

[permalink] [raw]
Subject: Re: [RFC PATCH V2 0/5] vhost: accelerate metadata access through vmap()

On Tue, 2019-03-12 at 10:59 +0800, Jason Wang wrote:
> On 2019/3/12 上午2:14, David Miller wrote:
> > From: "Michael S. Tsirkin" <[email protected]>
> > Date: Mon, 11 Mar 2019 09:59:28 -0400
> >
> > > On Mon, Mar 11, 2019 at 03:13:17PM +0800, Jason Wang wrote:
> > > > On 2019/3/8 下午10:12, Christoph Hellwig wrote:
> > > > > On Wed, Mar 06, 2019 at 02:18:07AM -0500, Jason Wang wrote:
> > > > > > This series tries to access virtqueue metadata through
> > > > > > kernel virtual
> > > > > > address instead of copy_user() friends since they had too
> > > > > > much
> > > > > > overheads like checks, spec barriers or even hardware
> > > > > > feature
> > > > > > toggling. This is done through setup kernel address through
> > > > > > vmap() and
> > > > > > resigter MMU notifier for invalidation.
> > > > > >
> > > > > > Test shows about 24% improvement on TX PPS. TCP_STREAM
> > > > > > doesn't see
> > > > > > obvious improvement.
> > > > >
> > > > > How is this going to work for CPUs with virtually tagged
> > > > > caches?
> > > >
> > > > Anything different that you worry?
> > >
> > > If caches have virtual tags then kernel and userspace view of
> > > memory
> > > might not be automatically in sync if they access memory
> > > through different virtual addresses. You need to do things like
> > > flush_cache_page, probably multiple times.
> >
> > "flush_dcache_page()"
>
>
> I get this. Then I think the current set_bit_to_user() is suspicious,
> we
> probably miss a flush_dcache_page() there:
>
>
> static int set_bit_to_user(int nr, void __user *addr)
> {
> unsigned long log = (unsigned long)addr;
> struct page *page;
> void *base;
> int bit = nr + (log % PAGE_SIZE) * 8;
> int r;
>
> r = get_user_pages_fast(log, 1, 1, &page);
> if (r < 0)
> return r;
> BUG_ON(r != 1);
> base = kmap_atomic(page);
> set_bit(bit, base);
> kunmap_atomic(base);

This sequence should be OK. get_user_pages() contains a flush which
clears the cache above the user virtual address, so on kmap, the page
is coherent at the new alias. On parisc at least, kunmap embodies a
flush_dcache_page() which pushes any changes in the cache above the
kernel virtual address back to main memory and makes it coherent again
for the user alias to pick it up.

James


2019-03-12 07:16:28

by Jason Wang

[permalink] [raw]
Subject: Re: [RFC PATCH V2 5/5] vhost: access vq metadata through kernel virtual address


On 2019/3/12 上午11:50, Michael S. Tsirkin wrote:
>>>> Using direct mapping (I
>>>> guess kernel will always try hugepage for that?) should be better and we can
>>>> even use it for the data transfer not only for the metadata.
>>>>
>>>> Thanks
>>> We can't really. The big issue is get user pages. Doing that on data
>>> path will be slower than copyXuser.
>> I meant if we can find a way to avoid doing gup in datapath. E.g vhost
>> maintain a range tree and add or remove ranges through MMU notifier. Then in
>> datapath, if we find the range, then use direct mapping otherwise
>> copy_to_user().
>>
>> Thanks
> We can try. But I'm not sure there's any reason to think there's any
> locality there.
>

Ok, but what kind of locality do you mean here?

Thanks


2019-03-12 07:18:12

by Jason Wang

[permalink] [raw]
Subject: Re: [RFC PATCH V2 0/5] vhost: accelerate metadata access through vmap()


On 2019/3/12 上午11:52, Michael S. Tsirkin wrote:
> On Tue, Mar 12, 2019 at 10:59:09AM +0800, Jason Wang wrote:
>> On 2019/3/12 上午2:14, David Miller wrote:
>>> From: "Michael S. Tsirkin" <[email protected]>
>>> Date: Mon, 11 Mar 2019 09:59:28 -0400
>>>
>>>> On Mon, Mar 11, 2019 at 03:13:17PM +0800, Jason Wang wrote:
>>>>> On 2019/3/8 下午10:12, Christoph Hellwig wrote:
>>>>>> On Wed, Mar 06, 2019 at 02:18:07AM -0500, Jason Wang wrote:
>>>>>>> This series tries to access virtqueue metadata through kernel virtual
>>>>>>> address instead of copy_user() friends since they had too much
>>>>>>> overheads like checks, spec barriers or even hardware feature
>>>>>>> toggling. This is done through setup kernel address through vmap() and
>>>>>>> resigter MMU notifier for invalidation.
>>>>>>>
>>>>>>> Test shows about 24% improvement on TX PPS. TCP_STREAM doesn't see
>>>>>>> obvious improvement.
>>>>>> How is this going to work for CPUs with virtually tagged caches?
>>>>> Anything different that you worry?
>>>> If caches have virtual tags then kernel and userspace view of memory
>>>> might not be automatically in sync if they access memory
>>>> through different virtual addresses. You need to do things like
>>>> flush_cache_page, probably multiple times.
>>> "flush_dcache_page()"
>>
>> I get this. Then I think the current set_bit_to_user() is suspicious, we
>> probably miss a flush_dcache_page() there:
>>
>>
>> static int set_bit_to_user(int nr, void __user *addr)
>> {
>>         unsigned long log = (unsigned long)addr;
>>         struct page *page;
>>         void *base;
>>         int bit = nr + (log % PAGE_SIZE) * 8;
>>         int r;
>>
>>         r = get_user_pages_fast(log, 1, 1, &page);
>>         if (r < 0)
>>                 return r;
>>         BUG_ON(r != 1);
>>         base = kmap_atomic(page);
>>         set_bit(bit, base);
>>         kunmap_atomic(base);
>>         set_page_dirty_lock(page);
>>         put_page(page);
>>         return 0;
>> }
>>
>> Thanks
> I think you are right. The correct fix though is to re-implement
> it using asm and handling pagefault, not gup.


I agree but it needs to introduce new helpers in asm  for all archs
which is not trivial. At least for -stable, we need the flush?


> Three atomic ops per bit is way to expensive.


Yes.

Thanks


2019-03-12 07:53:01

by Jason Wang

[permalink] [raw]
Subject: Re: [RFC PATCH V2 0/5] vhost: accelerate metadata access through vmap()


On 2019/3/12 下午1:14, James Bottomley wrote:
> On Tue, 2019-03-12 at 10:59 +0800, Jason Wang wrote:
>> On 2019/3/12 上午2:14, David Miller wrote:
>>> From: "Michael S. Tsirkin" <[email protected]>
>>> Date: Mon, 11 Mar 2019 09:59:28 -0400
>>>
>>>> On Mon, Mar 11, 2019 at 03:13:17PM +0800, Jason Wang wrote:
>>>>> On 2019/3/8 下午10:12, Christoph Hellwig wrote:
>>>>>> On Wed, Mar 06, 2019 at 02:18:07AM -0500, Jason Wang wrote:
>>>>>>> This series tries to access virtqueue metadata through
>>>>>>> kernel virtual
>>>>>>> address instead of copy_user() friends since they had too
>>>>>>> much
>>>>>>> overheads like checks, spec barriers or even hardware
>>>>>>> feature
>>>>>>> toggling. This is done through setup kernel address through
>>>>>>> vmap() and
>>>>>>> resigter MMU notifier for invalidation.
>>>>>>>
>>>>>>> Test shows about 24% improvement on TX PPS. TCP_STREAM
>>>>>>> doesn't see
>>>>>>> obvious improvement.
>>>>>> How is this going to work for CPUs with virtually tagged
>>>>>> caches?
>>>>> Anything different that you worry?
>>>> If caches have virtual tags then kernel and userspace view of
>>>> memory
>>>> might not be automatically in sync if they access memory
>>>> through different virtual addresses. You need to do things like
>>>> flush_cache_page, probably multiple times.
>>> "flush_dcache_page()"
>>
>> I get this. Then I think the current set_bit_to_user() is suspicious,
>> we
>> probably miss a flush_dcache_page() there:
>>
>>
>> static int set_bit_to_user(int nr, void __user *addr)
>> {
>> unsigned long log = (unsigned long)addr;
>> struct page *page;
>> void *base;
>> int bit = nr + (log % PAGE_SIZE) * 8;
>> int r;
>>
>> r = get_user_pages_fast(log, 1, 1, &page);
>> if (r < 0)
>> return r;
>> BUG_ON(r != 1);
>> base = kmap_atomic(page);
>> set_bit(bit, base);
>> kunmap_atomic(base);
> This sequence should be OK. get_user_pages() contains a flush which
> clears the cache above the user virtual address, so on kmap, the page
> is coherent at the new alias. On parisc at least, kunmap embodies a
> flush_dcache_page() which pushes any changes in the cache above the
> kernel virtual address back to main memory and makes it coherent again
> for the user alias to pick it up.


It would be good if kmap()/kunmap() can do this but looks like we can
not assume this? For example, sparc's flush_dcache_page() doesn't do
flush_dcache_page(). And bio_copy_data_iter() do flush_dcache_page()
after kunmap_atomic().

Thanks


>
> James
>

2019-03-12 07:55:40

by Jason Wang

[permalink] [raw]
Subject: Re: [RFC PATCH V2 0/5] vhost: accelerate metadata access through vmap()


On 2019/3/12 下午3:51, Jason Wang wrote:
>
> On 2019/3/12 下午1:14, James Bottomley wrote:
>> On Tue, 2019-03-12 at 10:59 +0800, Jason Wang wrote:
>>> On 2019/3/12 上午2:14, David Miller wrote:
>>>> From: "Michael S. Tsirkin" <[email protected]>
>>>> Date: Mon, 11 Mar 2019 09:59:28 -0400
>>>>
>>>>> On Mon, Mar 11, 2019 at 03:13:17PM +0800, Jason Wang wrote:
>>>>>> On 2019/3/8 下午10:12, Christoph Hellwig wrote:
>>>>>>> On Wed, Mar 06, 2019 at 02:18:07AM -0500, Jason Wang wrote:
>>>>>>>> This series tries to access virtqueue metadata through
>>>>>>>> kernel virtual
>>>>>>>> address instead of copy_user() friends since they had too
>>>>>>>> much
>>>>>>>> overheads like checks, spec barriers or even hardware
>>>>>>>> feature
>>>>>>>> toggling. This is done through setup kernel address through
>>>>>>>> vmap() and
>>>>>>>> resigter MMU notifier for invalidation.
>>>>>>>>
>>>>>>>> Test shows about 24% improvement on TX PPS. TCP_STREAM
>>>>>>>> doesn't see
>>>>>>>> obvious improvement.
>>>>>>> How is this going to work for CPUs with virtually tagged
>>>>>>> caches?
>>>>>> Anything different that you worry?
>>>>> If caches have virtual tags then kernel and userspace view of
>>>>> memory
>>>>> might not be automatically in sync if they access memory
>>>>> through different virtual addresses. You need to do things like
>>>>> flush_cache_page, probably multiple times.
>>>> "flush_dcache_page()"
>>>
>>> I get this. Then I think the current set_bit_to_user() is suspicious,
>>> we
>>> probably miss a flush_dcache_page() there:
>>>
>>>
>>> static int set_bit_to_user(int nr, void __user *addr)
>>> {
>>>           unsigned long log = (unsigned long)addr;
>>>           struct page *page;
>>>           void *base;
>>>           int bit = nr + (log % PAGE_SIZE) * 8;
>>>           int r;
>>>
>>>           r = get_user_pages_fast(log, 1, 1, &page);
>>>           if (r < 0)
>>>                   return r;
>>>           BUG_ON(r != 1);
>>>           base = kmap_atomic(page);
>>>           set_bit(bit, base);
>>>           kunmap_atomic(base);
>> This sequence should be OK.  get_user_pages() contains a flush which
>> clears the cache above the user virtual address, so on kmap, the page
>> is coherent at the new alias.  On parisc at least, kunmap embodies a
>> flush_dcache_page() which pushes any changes in the cache above the
>> kernel virtual address back to main memory and makes it coherent again
>> for the user alias to pick it up.
>
>
> It would be good if kmap()/kunmap() can do this but looks like we can
> not assume this? For example, sparc's flush_dcache_page()


Sorry, I meant kunmap_atomic().

Thanks


> doesn't do flush_dcache_page(). And bio_copy_data_iter() do
> flush_dcache_page() after kunmap_atomic().
>
> Thanks
>
>
>>
>> James
>>
>

2019-03-12 11:56:01

by Michael S. Tsirkin

[permalink] [raw]
Subject: Re: [RFC PATCH V2 0/5] vhost: accelerate metadata access through vmap()

On Tue, Mar 12, 2019 at 03:17:00PM +0800, Jason Wang wrote:
>
> On 2019/3/12 上午11:52, Michael S. Tsirkin wrote:
> > On Tue, Mar 12, 2019 at 10:59:09AM +0800, Jason Wang wrote:
> > > On 2019/3/12 上午2:14, David Miller wrote:
> > > > From: "Michael S. Tsirkin" <[email protected]>
> > > > Date: Mon, 11 Mar 2019 09:59:28 -0400
> > > >
> > > > > On Mon, Mar 11, 2019 at 03:13:17PM +0800, Jason Wang wrote:
> > > > > > On 2019/3/8 下午10:12, Christoph Hellwig wrote:
> > > > > > > On Wed, Mar 06, 2019 at 02:18:07AM -0500, Jason Wang wrote:
> > > > > > > > This series tries to access virtqueue metadata through kernel virtual
> > > > > > > > address instead of copy_user() friends since they had too much
> > > > > > > > overheads like checks, spec barriers or even hardware feature
> > > > > > > > toggling. This is done through setup kernel address through vmap() and
> > > > > > > > resigter MMU notifier for invalidation.
> > > > > > > >
> > > > > > > > Test shows about 24% improvement on TX PPS. TCP_STREAM doesn't see
> > > > > > > > obvious improvement.
> > > > > > > How is this going to work for CPUs with virtually tagged caches?
> > > > > > Anything different that you worry?
> > > > > If caches have virtual tags then kernel and userspace view of memory
> > > > > might not be automatically in sync if they access memory
> > > > > through different virtual addresses. You need to do things like
> > > > > flush_cache_page, probably multiple times.
> > > > "flush_dcache_page()"
> > >
> > > I get this. Then I think the current set_bit_to_user() is suspicious, we
> > > probably miss a flush_dcache_page() there:
> > >
> > >
> > > static int set_bit_to_user(int nr, void __user *addr)
> > > {
> > >         unsigned long log = (unsigned long)addr;
> > >         struct page *page;
> > >         void *base;
> > >         int bit = nr + (log % PAGE_SIZE) * 8;
> > >         int r;
> > >
> > >         r = get_user_pages_fast(log, 1, 1, &page);
> > >         if (r < 0)
> > >                 return r;
> > >         BUG_ON(r != 1);
> > >         base = kmap_atomic(page);
> > >         set_bit(bit, base);
> > >         kunmap_atomic(base);
> > >         set_page_dirty_lock(page);
> > >         put_page(page);
> > >         return 0;
> > > }
> > >
> > > Thanks
> > I think you are right. The correct fix though is to re-implement
> > it using asm and handling pagefault, not gup.
>
>
> I agree but it needs to introduce new helpers in asm  for all archs which is
> not trivial.

We can have a generic implementation using kmap.

> At least for -stable, we need the flush?
>
>
> > Three atomic ops per bit is way to expensive.
>
>
> Yes.
>
> Thanks

See James's reply - I stand corrected we do kunmap so no need to flush.

--
MST

2019-03-12 15:47:59

by James Bottomley

[permalink] [raw]
Subject: Re: [RFC PATCH V2 0/5] vhost: accelerate metadata access through vmap()

On Tue, 2019-03-12 at 07:54 -0400, Michael S. Tsirkin wrote:
> On Tue, Mar 12, 2019 at 03:17:00PM +0800, Jason Wang wrote:
> >
> > On 2019/3/12 上午11:52, Michael S. Tsirkin wrote:
> > > On Tue, Mar 12, 2019 at 10:59:09AM +0800, Jason Wang wrote:
[...]
> > At least for -stable, we need the flush?
> >
> >
> > > Three atomic ops per bit is way to expensive.
> >
> >
> > Yes.
> >
> > Thanks
>
> See James's reply - I stand corrected we do kunmap so no need to
> flush.

Well, I said that's what we do on Parisc. The cachetlb document
definitely says if you alter the data between kmap and kunmap you are
responsible for the flush. It's just that flush_dcache_page() is a no-
op on x86 so they never remember to add it and since it will crash
parisc if you get it wrong we finally gave up trying to make them.

But that's the point: it is a no-op on your favourite architecture so
it costs you nothing to add it.

James


2019-03-12 20:05:33

by Andrea Arcangeli

[permalink] [raw]
Subject: Re: [RFC PATCH V2 0/5] vhost: accelerate metadata access through vmap()

On Tue, Mar 12, 2019 at 08:46:50AM -0700, James Bottomley wrote:
> On Tue, 2019-03-12 at 07:54 -0400, Michael S. Tsirkin wrote:
> > On Tue, Mar 12, 2019 at 03:17:00PM +0800, Jason Wang wrote:
> > >
> > > On 2019/3/12 上午11:52, Michael S. Tsirkin wrote:
> > > > On Tue, Mar 12, 2019 at 10:59:09AM +0800, Jason Wang wrote:
> [...]
> > > At least for -stable, we need the flush?
> > >
> > >
> > > > Three atomic ops per bit is way to expensive.
> > >
> > >
> > > Yes.
> > >
> > > Thanks
> >
> > See James's reply - I stand corrected we do kunmap so no need to
> > flush.
>
> Well, I said that's what we do on Parisc. The cachetlb document
> definitely says if you alter the data between kmap and kunmap you are
> responsible for the flush. It's just that flush_dcache_page() is a no-
> op on x86 so they never remember to add it and since it will crash
> parisc if you get it wrong we finally gave up trying to make them.
>
> But that's the point: it is a no-op on your favourite architecture so
> it costs you nothing to add it.

Yes, the fact Parisc gave up and is doing it on kunmap is reasonable
approach for Parisc, but it doesn't move the needle as far as vhost
common code is concerned, because other archs don't flush any cache on
kunmap.

So either all other archs give up trying to optimize, or vhost still
has to call flush_dcache_page() after kunmap.

Which means after we fix vhost to add the flush_dcache_page after
kunmap, Parisc will get a double hit (but it also means Parisc was the
only one of those archs needed explicit cache flushes, where vhost
worked correctly so far.. so it kinds of proofs your point of giving
up being the safe choice).

Thanks,
Andrea

2019-03-12 20:55:50

by James Bottomley

[permalink] [raw]
Subject: Re: [RFC PATCH V2 0/5] vhost: accelerate metadata access through vmap()

On Tue, 2019-03-12 at 16:04 -0400, Andrea Arcangeli wrote:
> On Tue, Mar 12, 2019 at 08:46:50AM -0700, James Bottomley wrote:
> > On Tue, 2019-03-12 at 07:54 -0400, Michael S. Tsirkin wrote:
> > > On Tue, Mar 12, 2019 at 03:17:00PM +0800, Jason Wang wrote:
> > > >
> > > > On 2019/3/12 上åˆ11:52, Michael S. Tsirkin wrote:
> > > > > On Tue, Mar 12, 2019 at 10:59:09AM +0800, Jason Wang wrote:
> >
> > [...]
> > > > At least for -stable, we need the flush?
> > > >
> > > >
> > > > > Three atomic ops per bit is way to expensive.
> > > >
> > > >
> > > > Yes.
> > > >
> > > > Thanks
> > >
> > > See James's reply - I stand corrected we do kunmap so no need to
> > > flush.
> >
> > Well, I said that's what we do on Parisc. The cachetlb document
> > definitely says if you alter the data between kmap and kunmap you
> > are responsible for the flush. It's just that flush_dcache_page()
> > is a no-op on x86 so they never remember to add it and since it
> > will crash parisc if you get it wrong we finally gave up trying to
> > make them.
> >
> > But that's the point: it is a no-op on your favourite architecture
> > so it costs you nothing to add it.
>
> Yes, the fact Parisc gave up and is doing it on kunmap is reasonable
> approach for Parisc, but it doesn't move the needle as far as vhost
> common code is concerned, because other archs don't flush any cache
> on kunmap.
>
> So either all other archs give up trying to optimize, or vhost still
> has to call flush_dcache_page() after kunmap.

I've got to say: optimize what? What code do we ever have in the
kernel that kmap's a page and then doesn't do anything with it? You can
guarantee that on kunmap the page is either referenced (needs
invalidating) or updated (needs flushing). The in-kernel use of kmap is
always

kmap
do something with the mapped page
kunmap

In a very short interval. It seems just a simplification to make
kunmap do the flush if needed rather than try to have the users
remember. The thing which makes this really simple is that on most
architectures flush and invalidate is the same operation. If you
really want to optimize you can use the referenced and dirty bits on
the kmapped pte to tell you what operation to do, but if your flush is
your invalidate, you simply assume the data needs flushing on kunmap
without checking anything.

> Which means after we fix vhost to add the flush_dcache_page after
> kunmap, Parisc will get a double hit (but it also means Parisc was
> the only one of those archs needed explicit cache flushes, where
> vhost worked correctly so far.. so it kinds of proofs your point of
> giving up being the safe choice).

What double hit? If there's no cache to flush then cache flush is a
no-op. It's also a highly piplineable no-op because the CPU has the L1
cache within easy reach. The only event when flush takes a large
amount time is if we actually have dirty data to write back to main
memory.

James


2019-03-12 21:12:06

by Andrea Arcangeli

[permalink] [raw]
Subject: Re: [RFC PATCH V2 0/5] vhost: accelerate metadata access through vmap()

On Tue, Mar 12, 2019 at 01:53:37PM -0700, James Bottomley wrote:
> I've got to say: optimize what? What code do we ever have in the
> kernel that kmap's a page and then doesn't do anything with it? You can
> guarantee that on kunmap the page is either referenced (needs
> invalidating) or updated (needs flushing). The in-kernel use of kmap is
> always
>
> kmap
> do something with the mapped page
> kunmap
>
> In a very short interval. It seems just a simplification to make
> kunmap do the flush if needed rather than try to have the users
> remember. The thing which makes this really simple is that on most
> architectures flush and invalidate is the same operation. If you
> really want to optimize you can use the referenced and dirty bits on
> the kmapped pte to tell you what operation to do, but if your flush is
> your invalidate, you simply assume the data needs flushing on kunmap
> without checking anything.

Except other archs like arm64 and sparc do the cache flushing on
copy_to_user_page and copy_user_page, not on kunmap.

#define copy_user_page(to,from,vaddr,pg) __cpu_copy_user_page(to, from, vaddr)
void __cpu_copy_user_page(void *kto, const void *kfrom, unsigned long vaddr)
{
struct page *page = virt_to_page(kto);
copy_page(kto, kfrom);
flush_dcache_page(page);
}
#define copy_user_page(to, from, vaddr, page) \
do { copy_page(to, from); \
sparc_flush_page_to_ram(page); \
} while (0)

And they do nothing on kunmap:

static inline void kunmap(struct page *page)
{
BUG_ON(in_interrupt());
if (!PageHighMem(page))
return;
kunmap_high(page);
}
void kunmap_high(struct page *page)
{
unsigned long vaddr;
unsigned long nr;
unsigned long flags;
int need_wakeup;
unsigned int color = get_pkmap_color(page);
wait_queue_head_t *pkmap_map_wait;

lock_kmap_any(flags);
vaddr = (unsigned long)page_address(page);
BUG_ON(!vaddr);
nr = PKMAP_NR(vaddr);

/*
* A count must never go down to zero
* without a TLB flush!
*/
need_wakeup = 0;
switch (--pkmap_count[nr]) {
case 0:
BUG();
case 1:
/*
* Avoid an unnecessary wake_up() function call.
* The common case is pkmap_count[] == 1, but
* no waiters.
* The tasks queued in the wait-queue are guarded
* by both the lock in the wait-queue-head and by
* the kmap_lock. As the kmap_lock is held here,
* no need for the wait-queue-head's lock. Simply
* test if the queue is empty.
*/
pkmap_map_wait = get_pkmap_wait_queue_head(color);
need_wakeup = waitqueue_active(pkmap_map_wait);
}
unlock_kmap_any(flags);

/* do wake-up, if needed, race-free outside of the spin lock */
if (need_wakeup)
wake_up(pkmap_map_wait);
}
static inline void kunmap(struct page *page)
{
}

because they already did it just above.


> > Which means after we fix vhost to add the flush_dcache_page after
> > kunmap, Parisc will get a double hit (but it also means Parisc was
> > the only one of those archs needed explicit cache flushes, where
> > vhost worked correctly so far.. so it kinds of proofs your point of
> > giving up being the safe choice).
>
> What double hit? If there's no cache to flush then cache flush is a
> no-op. It's also a highly piplineable no-op because the CPU has the L1
> cache within easy reach. The only event when flush takes a large
> amount time is if we actually have dirty data to write back to main
> memory.

The double hit is in parisc copy_to_user_page:

#define copy_to_user_page(vma, page, vaddr, dst, src, len) \
do { \
flush_cache_page(vma, vaddr, page_to_pfn(page)); \
memcpy(dst, src, len); \
flush_kernel_dcache_range_asm((unsigned long)dst, (unsigned long)dst + len); \
} while (0)

That is executed just before kunmap:

static inline void kunmap(struct page *page)
{
flush_kernel_dcache_page_addr(page_address(page));
}

Can't argue about the fact your "safer" kunmap is safer, but we cannot
rely on common code unless we remove some optimization from the common
code abstractions and we make all archs do kunmap like parisc.

Thanks,
Andrea

2019-03-12 21:20:05

by James Bottomley

[permalink] [raw]
Subject: Re: [RFC PATCH V2 0/5] vhost: accelerate metadata access through vmap()

I think we might be talking past each other. Let me try the double
flush first

On Tue, 2019-03-12 at 17:11 -0400, Andrea Arcangeli wrote:
> On Tue, Mar 12, 2019 at 01:53:37PM -0700, James Bottomley wrote:
> > > Which means after we fix vhost to add the flush_dcache_page after
> > > kunmap, Parisc will get a double hit (but it also means Parisc
> > > was
> > > the only one of those archs needed explicit cache flushes, where
> > > vhost worked correctly so far.. so it kinds of proofs your point
> > > of
> > > giving up being the safe choice).
> >
> > What double hit? If there's no cache to flush then cache flush is
> > a no-op. It's also a highly piplineable no-op because the CPU has
> > the L1 cache within easy reach. The only event when flush takes a
> > large amount time is if we actually have dirty data to write back
> > to main memory.
>
> The double hit is in parisc copy_to_user_page:
>
> #define copy_to_user_page(vma, page, vaddr, dst, src, len) \
> do { \
> flush_cache_page(vma, vaddr, page_to_pfn(page)); \
> memcpy(dst, src, len); \
> flush_kernel_dcache_range_asm((unsigned long)dst, (unsigned
> long)dst + len); \
> } while (0)
>
> That is executed just before kunmap:
>
> static inline void kunmap(struct page *page)
> {
> flush_kernel_dcache_page_addr(page_address(page));
> }

I mean in the sequence

flush_dcache_page(page);
flush_dcache_page(page);

The first flush_dcache_page did all the work and the second it a
tightly pipelined no-op. That's what I mean by there not really being
a double hit.

James


2019-03-12 21:54:24

by Andrea Arcangeli

[permalink] [raw]
Subject: Re: [RFC PATCH V2 0/5] vhost: accelerate metadata access through vmap()

On Tue, Mar 12, 2019 at 02:19:15PM -0700, James Bottomley wrote:
> I mean in the sequence
>
> flush_dcache_page(page);
> flush_dcache_page(page);
>
> The first flush_dcache_page did all the work and the second it a
> tightly pipelined no-op. That's what I mean by there not really being
> a double hit.

Ok I wasn't sure it was clear there was a double (profiling) hit on
that function.

void flush_kernel_dcache_page_addr(void *addr)
{
unsigned long flags;

flush_kernel_dcache_page_asm(addr);
purge_tlb_start(flags);
pdtlb_kernel(addr);
purge_tlb_end(flags);
}

#define purge_tlb_start(flags) spin_lock_irqsave(&pa_tlb_lock, flags)
#define purge_tlb_end(flags) spin_unlock_irqrestore(&pa_tlb_lock, flags)

You got a system-wide spinlock in there that won't just go away the
second time. So it's a bit more than a tightly pipelined "noop".

Your logic of adding the flush on kunmap makes sense, all I'm saying
is that it's sacrificing some performance for safety. You asked
"optimized what", I meant to optimize away all the above quoted code
that will end running twice for each vhost set_bit when it should run
just once like in other archs. And it clearly paid off until now
(until now it run just once and it was the only safe one).

Before we can leverage your idea to flush the dcache on kunmap in
common code without having to sacrifice performance in arch code, we'd
need to change all other archs to add the cache flushes on kunmap too,
and then remove the cache flushes from the other places like copy_page
or we'd waste CPU. Then you'd have the best of both words, no double
flush and kunmap would be enough.

Thanks,
Andrea

2019-03-12 22:03:37

by James Bottomley

[permalink] [raw]
Subject: Re: [RFC PATCH V2 0/5] vhost: accelerate metadata access through vmap()

On Tue, 2019-03-12 at 17:53 -0400, Andrea Arcangeli wrote:
> On Tue, Mar 12, 2019 at 02:19:15PM -0700, James Bottomley wrote:
> > I mean in the sequence
> >
> > flush_dcache_page(page);
> > flush_dcache_page(page);
> >
> > The first flush_dcache_page did all the work and the second it a
> > tightly pipelined no-op. That's what I mean by there not really
> > being
> > a double hit.
>
> Ok I wasn't sure it was clear there was a double (profiling) hit on
> that function.
>
> void flush_kernel_dcache_page_addr(void *addr)
> {
> unsigned long flags;
>
> flush_kernel_dcache_page_asm(addr);
> purge_tlb_start(flags);
> pdtlb_kernel(addr);
> purge_tlb_end(flags);
> }
>
> #define purge_tlb_start(flags) spin_lock_irqsave(&pa_tlb_lock,
> flags)
> #define purge_tlb_end(flags) spin_unlock_irqrestore(&pa_tlb_lo
> ck, flags)
>
> You got a system-wide spinlock in there that won't just go away the
> second time. So it's a bit more than a tightly pipelined "noop".

Well, yes, guilty as charged. That particular bit of code is a work
around for an N class system which has an internal cross CPU coherency
bus but helpfully crashes if two different CPUs try to use it at once.
Since the N class was a huge power hog, I thought they'd all been
decommisioned and this was an irrelevant anachronism (or at the very
least runtime patched).

> Your logic of adding the flush on kunmap makes sense, all I'm saying
> is that it's sacrificing some performance for safety. You asked
> "optimized what", I meant to optimize away all the above quoted code
> that will end running twice for each vhost set_bit when it should run
> just once like in other archs. And it clearly paid off until now
> (until now it run just once and it was the only safe one).

I'm sure there must be workarounds elsewhere in the other arch code
otherwise things like this, which appear all over drivers/, wouldn't
work:

drivers/scsi/isci/request.c:1430

kaddr = kmap_atomic(page);
memcpy(kaddr + sg->offset, src_addr, copy_len);
kunmap_atomic(kaddr);

the sequence dirties the kernel virtual address but doesn't flush
before doing kunmap. There are hundreds of other examples which is why
I think adding flush_kernel_dcache_page() is an already lost cause.

> Before we can leverage your idea to flush the dcache on kunmap in
> common code without having to sacrifice performance in arch code,
> we'd need to change all other archs to add the cache flushes on
> kunmap too, and then remove the cache flushes from the other places
> like copy_page or we'd waste CPU. Then you'd have the best of both
> words, no double flush and kunmap would be enough.

Actually copy_user_page() is unused in the main kernel. The big
problem is copy_user_highpage() but that's mostly highly optimised by
the VIPT architectures (in other words you can fiddle with kmap without
impacting it).

James


2019-03-12 22:51:23

by Andrea Arcangeli

[permalink] [raw]
Subject: Re: [RFC PATCH V2 0/5] vhost: accelerate metadata access through vmap()

On Tue, Mar 12, 2019 at 03:02:54PM -0700, James Bottomley wrote:
> I'm sure there must be workarounds elsewhere in the other arch code
> otherwise things like this, which appear all over drivers/, wouldn't
> work:
>
> drivers/scsi/isci/request.c:1430
>
> kaddr = kmap_atomic(page);
> memcpy(kaddr + sg->offset, src_addr, copy_len);
> kunmap_atomic(kaddr);
>

Are you sure "page" is an userland page with an alias address?

sg->page_link = (unsigned long)virt_to_page(addr);

page_link seems to point to kernel memory.

I found an apparent solution like parisc on arm 32bit:

void __kunmap_atomic(void *kvaddr)
{
unsigned long vaddr = (unsigned long) kvaddr & PAGE_MASK;
int idx, type;

if (kvaddr >= (void *)FIXADDR_START) {
type = kmap_atomic_idx();
idx = FIX_KMAP_BEGIN + type + KM_TYPE_NR * smp_processor_id();

if (cache_is_vivt())
__cpuc_flush_dcache_area((void *)vaddr, PAGE_SIZE);

However on arm 64bit kunmap_atomic is not implemented at all and other
32bit implementations don't do it, for example sparc seems to do the
cache flush too if the kernel is built with CONFIG_DEBUG_HIGHMEM
(which makes the flushing conditional to the debug option).

The kunmap_atomic where fixmap is used, is flushing the tlb lazily so
even on 32bit you can't even be sure if there was a tlb flush for each
single page you unmapped, so it's hard to see how the above can work
safe, is "page" would have been an userland page mapped with aliased
CPU cache.

> the sequence dirties the kernel virtual address but doesn't flush
> before doing kunmap. There are hundreds of other examples which is why
> I think adding flush_kernel_dcache_page() is an already lost cause.

In lots of cases kmap is needed to just modify kernel memory not to
modify userland memory (where get/put_user is more commonly used
instead..), there's no cache aliasing in such case.

> Actually copy_user_page() is unused in the main kernel. The big
> problem is copy_user_highpage() but that's mostly highly optimised by
> the VIPT architectures (in other words you can fiddle with kmap without
> impacting it).

copy_user_page is not unused, it's called precisely by
copy_user_highpage, which is why the cache flushes are done inside
copy_user_page.

static inline void copy_user_highpage(struct page *to, struct page *from,
unsigned long vaddr, struct vm_area_struct *vma)
{
char *vfrom, *vto;

vfrom = kmap_atomic(from);
vto = kmap_atomic(to);
copy_user_page(vto, vfrom, vaddr, to);
kunmap_atomic(vto);
kunmap_atomic(vfrom);
}

2019-03-12 22:59:09

by James Bottomley

[permalink] [raw]
Subject: Re: [RFC PATCH V2 0/5] vhost: accelerate metadata access through vmap()

On Tue, 2019-03-12 at 18:50 -0400, Andrea Arcangeli wrote:
> On Tue, Mar 12, 2019 at 03:02:54PM -0700, James Bottomley wrote:
> > I'm sure there must be workarounds elsewhere in the other arch code
> > otherwise things like this, which appear all over drivers/,
> > wouldn't
> > work:
> >
> > drivers/scsi/isci/request.c:1430
> >
> > kaddr = kmap_atomic(page);
> > memcpy(kaddr + sg->offset, src_addr, copy_len);
> > kunmap_atomic(kaddr);
> >
>
> Are you sure "page" is an userland page with an alias address?
>
> sg->page_link = (unsigned long)virt_to_page(addr);

Yes, it's an element of a scatter gather list, which may be either a
kernel page or a user page, but is usually the latter.

> page_link seems to point to kernel memory.
>
> I found an apparent solution like parisc on arm 32bit:
>
> void __kunmap_atomic(void *kvaddr)
> {
> unsigned long vaddr = (unsigned long) kvaddr & PAGE_MASK;
> int idx, type;
>
> if (kvaddr >= (void *)FIXADDR_START) {
> type = kmap_atomic_idx();
> idx = FIX_KMAP_BEGIN + type + KM_TYPE_NR *
> smp_processor_id();
>
> if (cache_is_vivt())
> __cpuc_flush_dcache_area((void *)vaddr,
> PAGE_SIZE);
>
> However on arm 64bit kunmap_atomic is not implemented at all and
> other 32bit implementations don't do it, for example sparc seems to
> do the cache flush too if the kernel is built with
> CONFIG_DEBUG_HIGHMEM (which makes the flushing conditional to the
> debug option).
>
> The kunmap_atomic where fixmap is used, is flushing the tlb lazily so
> even on 32bit you can't even be sure if there was a tlb flush for
> each single page you unmapped, so it's hard to see how the above can
> work safe, is "page" would have been an userland page mapped with
> aliased CPU cache.
>
> > the sequence dirties the kernel virtual address but doesn't flush
> > before doing kunmap. There are hundreds of other examples which is
> > why I think adding flush_kernel_dcache_page() is an already lost
> > cause.
>
> In lots of cases kmap is needed to just modify kernel memory not to
> modify userland memory (where get/put_user is more commonly used
> instead..), there's no cache aliasing in such case.

That's why I picked drivers/ The use case in there is mostly kmap to
put a special value into a scatter gather list entry.

> > Actually copy_user_page() is unused in the main kernel. The big
> > problem is copy_user_highpage() but that's mostly highly optimised
> > by the VIPT architectures (in other words you can fiddle with kmap
> > without impacting it).
>
> copy_user_page is not unused, it's called precisely by
> copy_user_highpage, which is why the cache flushes are done inside
> copy_user_page.
>
> static inline void copy_user_highpage(struct page *to, struct page
> *from,
> unsigned long vaddr, struct vm_area_struct *vma)
> {
> char *vfrom, *vto;
>
> vfrom = kmap_atomic(from);
> vto = kmap_atomic(to);
> copy_user_page(vto, vfrom, vaddr, to);
> kunmap_atomic(vto);
> kunmap_atomic(vfrom);
> }

That's the asm/generic implementation. Most VIPT architectures
override it.

James


2019-03-13 16:06:43

by Christoph Hellwig

[permalink] [raw]
Subject: Re: [RFC PATCH V2 0/5] vhost: accelerate metadata access through vmap()

On Tue, Mar 12, 2019 at 01:53:37PM -0700, James Bottomley wrote:
> I've got to say: optimize what? What code do we ever have in the
> kernel that kmap's a page and then doesn't do anything with it? You can
> guarantee that on kunmap the page is either referenced (needs
> invalidating) or updated (needs flushing). The in-kernel use of kmap is
> always
>
> kmap
> do something with the mapped page
> kunmap
>
> In a very short interval. It seems just a simplification to make
> kunmap do the flush if needed rather than try to have the users
> remember. The thing which makes this really simple is that on most
> architectures flush and invalidate is the same operation. If you
> really want to optimize you can use the referenced and dirty bits on
> the kmapped pte to tell you what operation to do, but if your flush is
> your invalidate, you simply assume the data needs flushing on kunmap
> without checking anything.

I agree that this would be a good way to simplify the API. Now
we'd just need volunteers to implement this for all architectures
that need cache flushing and then remove the explicit flushing in
the callers..

> > Which means after we fix vhost to add the flush_dcache_page after
> > kunmap, Parisc will get a double hit (but it also means Parisc was
> > the only one of those archs needed explicit cache flushes, where
> > vhost worked correctly so far.. so it kinds of proofs your point of
> > giving up being the safe choice).
>
> What double hit? If there's no cache to flush then cache flush is a
> no-op. It's also a highly piplineable no-op because the CPU has the L1
> cache within easy reach. The only event when flush takes a large
> amount time is if we actually have dirty data to write back to main
> memory.

I've heard people complaining that on some microarchitectures even
no-op cache flushes are relatively expensive. Don't ask me why,
but if we can easily avoid double flushes we should do that.

2019-03-13 16:39:14

by James Bottomley

[permalink] [raw]
Subject: Re: [RFC PATCH V2 0/5] vhost: accelerate metadata access through vmap()

On Wed, 2019-03-13 at 09:05 -0700, Christoph Hellwig wrote:
> On Tue, Mar 12, 2019 at 01:53:37PM -0700, James Bottomley wrote:
> > I've got to say: optimize what? What code do we ever have in the
> > kernel that kmap's a page and then doesn't do anything with it? You
> > can
> > guarantee that on kunmap the page is either referenced (needs
> > invalidating) or updated (needs flushing). The in-kernel use of
> > kmap is
> > always
> >
> > kmap
> > do something with the mapped page
> > kunmap
> >
> > In a very short interval. It seems just a simplification to make
> > kunmap do the flush if needed rather than try to have the users
> > remember. The thing which makes this really simple is that on most
> > architectures flush and invalidate is the same operation. If you
> > really want to optimize you can use the referenced and dirty bits
> > on the kmapped pte to tell you what operation to do, but if your
> > flush is your invalidate, you simply assume the data needs flushing
> > on kunmap without checking anything.
>
> I agree that this would be a good way to simplify the API. Now
> we'd just need volunteers to implement this for all architectures
> that need cache flushing and then remove the explicit flushing in
> the callers..

Well, it's already done on parisc ... I can help with this if we agree
it's the best way forward. It's really only architectures that
implement flush_dcache_page that would need modifying.

It may also improve performance because some kmap/use/flush/kunmap
sequences have flush_dcache_page() instead of
flush_kernel_dcache_page() and the former is hugely expensive and
usually unnecessary because GUP already flushed all the user aliases.

In the interests of full disclosure the reason we do it for parisc is
because our later machines have problems even with clean aliases. So
on most VIPT systems, doing kmap/read/kunmap creates a fairly harmless
clean alias. Technically it should be invalidated, because if you
remap the same page to the same colour you get cached stale data, but
in practice the data is expired from the cache long before that
happens, so the problem is almost never seen if the flush is forgotten.
Our problem is on the P9xxx processor: they have a L1/L2 VIPT L3 PIPT
cache. As the L1/L2 caches expire clean data, they place the expiring
contents into L3, but because L3 is PIPT, the stale alias suddenly
becomes the default for any read of they physical page because any
update which dirtied the cache line often gets written to main memory
and placed into the L3 as clean *before* the clean alias in L1/L2 gets
expired, so the older clean alias replaces it.

Our only recourse is to kill all aliases with prejudice before the
kernel loses ownership.

> > > Which means after we fix vhost to add the flush_dcache_page after
> > > kunmap, Parisc will get a double hit (but it also means Parisc
> > > was the only one of those archs needed explicit cache flushes,
> > > where vhost worked correctly so far.. so it kinds of proofs your
> > > point of giving up being the safe choice).
> >
> > What double hit? If there's no cache to flush then cache flush is
> > a no-op. It's also a highly piplineable no-op because the CPU has
> > the L1 cache within easy reach. The only event when flush takes a
> > large amount time is if we actually have dirty data to write back
> > to main memory.
>
> I've heard people complaining that on some microarchitectures even
> no-op cache flushes are relatively expensive. Don't ask me why,
> but if we can easily avoid double flushes we should do that.

It's still not entirely free for us. Our internal cache line is around
32 bytes (some have 16 and some have 64) but that means we need 128
flushes for a page ... we definitely can't pipeline them all. So I
agree duplicate flush elimination would be a small improvement.

James


2019-03-14 10:45:01

by Michael S. Tsirkin

[permalink] [raw]
Subject: Re: [RFC PATCH V2 0/5] vhost: accelerate metadata access through vmap()

On Wed, Mar 13, 2019 at 09:37:08AM -0700, James Bottomley wrote:
> On Wed, 2019-03-13 at 09:05 -0700, Christoph Hellwig wrote:
> > On Tue, Mar 12, 2019 at 01:53:37PM -0700, James Bottomley wrote:
> > > I've got to say: optimize what? What code do we ever have in the
> > > kernel that kmap's a page and then doesn't do anything with it? You
> > > can
> > > guarantee that on kunmap the page is either referenced (needs
> > > invalidating) or updated (needs flushing). The in-kernel use of
> > > kmap is
> > > always
> > >
> > > kmap
> > > do something with the mapped page
> > > kunmap
> > >
> > > In a very short interval. It seems just a simplification to make
> > > kunmap do the flush if needed rather than try to have the users
> > > remember. The thing which makes this really simple is that on most
> > > architectures flush and invalidate is the same operation. If you
> > > really want to optimize you can use the referenced and dirty bits
> > > on the kmapped pte to tell you what operation to do, but if your
> > > flush is your invalidate, you simply assume the data needs flushing
> > > on kunmap without checking anything.
> >
> > I agree that this would be a good way to simplify the API. Now
> > we'd just need volunteers to implement this for all architectures
> > that need cache flushing and then remove the explicit flushing in
> > the callers..
>
> Well, it's already done on parisc ... I can help with this if we agree
> it's the best way forward. It's really only architectures that
> implement flush_dcache_page that would need modifying.
>
> It may also improve performance because some kmap/use/flush/kunmap
> sequences have flush_dcache_page() instead of
> flush_kernel_dcache_page() and the former is hugely expensive and
> usually unnecessary because GUP already flushed all the user aliases.
>
> In the interests of full disclosure the reason we do it for parisc is
> because our later machines have problems even with clean aliases. So
> on most VIPT systems, doing kmap/read/kunmap creates a fairly harmless
> clean alias. Technically it should be invalidated, because if you
> remap the same page to the same colour you get cached stale data, but
> in practice the data is expired from the cache long before that
> happens, so the problem is almost never seen if the flush is forgotten.
> Our problem is on the P9xxx processor: they have a L1/L2 VIPT L3 PIPT
> cache. As the L1/L2 caches expire clean data, they place the expiring
> contents into L3, but because L3 is PIPT, the stale alias suddenly
> becomes the default for any read of they physical page because any
> update which dirtied the cache line often gets written to main memory
> and placed into the L3 as clean *before* the clean alias in L1/L2 gets
> expired, so the older clean alias replaces it.
>
> Our only recourse is to kill all aliases with prejudice before the
> kernel loses ownership.
>
> > > > Which means after we fix vhost to add the flush_dcache_page after
> > > > kunmap, Parisc will get a double hit (but it also means Parisc
> > > > was the only one of those archs needed explicit cache flushes,
> > > > where vhost worked correctly so far.. so it kinds of proofs your
> > > > point of giving up being the safe choice).
> > >
> > > What double hit? If there's no cache to flush then cache flush is
> > > a no-op. It's also a highly piplineable no-op because the CPU has
> > > the L1 cache within easy reach. The only event when flush takes a
> > > large amount time is if we actually have dirty data to write back
> > > to main memory.
> >
> > I've heard people complaining that on some microarchitectures even
> > no-op cache flushes are relatively expensive. Don't ask me why,
> > but if we can easily avoid double flushes we should do that.
>
> It's still not entirely free for us. Our internal cache line is around
> 32 bytes (some have 16 and some have 64) but that means we need 128
> flushes for a page ... we definitely can't pipeline them all. So I
> agree duplicate flush elimination would be a small improvement.
>
> James

I suspect we'll keep the copyXuser path around for 32 bit anyway -
right Jason?
So we can also keep using that on parisc...

--
MST

2019-03-14 13:50:10

by Jason Wang

[permalink] [raw]
Subject: Re: [RFC PATCH V2 0/5] vhost: accelerate metadata access through vmap()


On 2019/3/14 下午6:42, Michael S. Tsirkin wrote:
>>>>> Which means after we fix vhost to add the flush_dcache_page after
>>>>> kunmap, Parisc will get a double hit (but it also means Parisc
>>>>> was the only one of those archs needed explicit cache flushes,
>>>>> where vhost worked correctly so far.. so it kinds of proofs your
>>>>> point of giving up being the safe choice).
>>>> What double hit? If there's no cache to flush then cache flush is
>>>> a no-op. It's also a highly piplineable no-op because the CPU has
>>>> the L1 cache within easy reach. The only event when flush takes a
>>>> large amount time is if we actually have dirty data to write back
>>>> to main memory.
>>> I've heard people complaining that on some microarchitectures even
>>> no-op cache flushes are relatively expensive. Don't ask me why,
>>> but if we can easily avoid double flushes we should do that.
>> It's still not entirely free for us. Our internal cache line is around
>> 32 bytes (some have 16 and some have 64) but that means we need 128
>> flushes for a page ... we definitely can't pipeline them all. So I
>> agree duplicate flush elimination would be a small improvement.
>>
>> James
> I suspect we'll keep the copyXuser path around for 32 bit anyway -
> right Jason?


Yes since we don't want to slow down 32bit.

Thanks


> So we can also keep using that on parisc...
>
> --

2019-03-14 19:35:09

by Andrea Arcangeli

[permalink] [raw]
Subject: Re: [RFC PATCH V2 0/5] vhost: accelerate metadata access through vmap()

Hello Jason,

On Thu, Mar 14, 2019 at 09:49:03PM +0800, Jason Wang wrote:
> Yes since we don't want to slow down 32bit.

If you've a lot of ram there's no justification to stick to a 32bit
kernel, so I don't think there's need to maintain a separate model
just for 32bit. I really wouldn't care about the performance of 32bit
with >700MB of RAM if that would cause any maintenance burden. Let's
focus on the best 64bit implementation that will work equally
optimally on 32bit with <= 700M of RAM.

Talking to Jerome about the set_page_dirty issue, he raised the point
of what happens if two thread calls a mmu notifier invalidate
simultaneously. The first mmu notifier could call set_page_dirty and
then proceed in try_to_free_buffers or page_mkclean and then the
concurrent mmu notifier that arrives second, then must not call
set_page_dirty a second time.

With KVM sptes mappings and vhost mappings you would call
set_page_dirty (if you invoked gup with FOLL_WRITE) only when
effectively tearing down any secondary mapping (you've got pointers in
both cases for the mapping). So there's no way to risk a double
set_page_dirty from concurrent mmu notifier invalidate because the
invalidate takes a lock when it has to teardown the mapping and so
set_page_dirty is only run in the first invalidate method and not in
the second. In the spte case even better, as you wouldn't need to call
it even at teardown time unless the spte is dirty (if shadow mask
allows dirty sptes with EPT2 or NPT or shadow MMU).

If you instead had to invalidate a secondary MMU mapping that isn't
tracked by the driver (again: not vhost nor KVM case), you could have
used the dirty bit of the kernel pagetable to call set_page_dirty and
disambiguate but that's really messy, and it would prevent the use of
gigapages in the direct mapping too and it'd require vmap for 4k
tracking.

To make sure set_page_dirty is run a single time no matter if the
invalidate known when a mapping is tear down, I suggested the below
model:

access = FOLL_WRITE

repeat:
page = gup_fast(access)
put_page(page) /* need a way to drop FOLL_GET from gup_fast instead! */

spin_lock(mmu_notifier_lock);
if (race with invalidate) {
spin_unlock..
goto repeat;
}
if (access == FOLL_WRITE)
set_page_dirty(page)
establish writable mapping in secondary MMU on page
spin_unlock

(replace spin_lock with mutex_lock for vhost of course if you stick to
a mutex and _start/_end instead of non-sleepable ->invalidate_range)

"race with invalidate" is the usual "mmu_notifier_retry" in kvm_host.h
to be implemented for vhost.

We could add a FOLL_DIRTY flag to add to FOLL_TOUCH to move the
set_page_dirty inside GUP forced (currently it's not forced if the
linux pte is already dirty). And we could remove FOLL_GET.

Then if you have the ability to disambiguate which is the first
invalidate that tears down a mapping to any given page (vhost can do
that trivially, it's just a pointer to a page struct to kmap), in the
mmu notifier invalidate just before dropping the spinlock you would
do this check:

def vhost_invalidate_range_start():
[..]
spin_lock(mmu_notifier_lock);
[..]
if (vhost->page_pointer) {
if (access == FOLL_WRITE)
VM_WARN_ON(!PageDirty(vhost->page_pointer));
vhost->page_pointer = NULL;
/* no put_page, already done at gup time */
}
spin_unlock(..

Generally speaking set_page_dirty is safer done after the last
modification to the data of the page. However the way stable page
works, as long as the mmu notifier invalidate didn't run, the PG_dirty
cannot go away.

So this model solves the issue with guaranteeing a single
set_page_dirty is run before page_mkclean or try_to_free_buffers can
run, even for drivers that implement the invalidate as a generic "TLB
flush" across the whole secondary MMU and that cannot disambiguate the
first invalidate from a second invalidate if they're issued
concurrently on the same address by two different CPUs.

So for those drivers that can disambiguate trivially (like KVM and
vhost) we'll just add a debug check in the invalidate to validate the
common code for all mmu notifier users.

This is the solution for RDMA hardware and everything that can support
mmu notifiers too and they can take infinitely long secondary MMU
mappins without interfering with stable pages at all (i.e. long term
pins but without pinning) perfectly safe and transparent to the whole
stable page code.

I think O_DIRECT for stable pages shall be solved taking the page lock
or writeback lock or a new rwsem in the inode that is taken for
writing by page_mkclean and try_to_free_buffers and for reading by
outstanding O_DIRECT in flight I/O, like I suggested probably ages ago
but then we only made GUP take the page pin, which is fine for mmu
notifier actually (except those didn't exist back then). To solve
O_DIRECT we can leverage the 100% guarantee that the pin will be
dropped ASAP and stop page_mkclean and stop or trylock in
try_to_free_buffers in such case.

mm_take_all_locks is major hurdle that prevents usage in O_DIRECT
case, even if we "cache it" if you fork(); write; exit() in a loop
it'll still cause heavy lock overhead. MMU notifier registration isn't
intended to happen in fast and frequent paths like the write()
syscall. Dropping mm_take_all_locks would bring other downsides: a
regular quiescent point can happen in between _start/_end and _start
must be always called first all the mmu notifier retry counters we
rely on would break. One way would be to add srcu_read_lock _before_
you can call mm_has_mm_has_notifiers(mm), then yes we could replace
mm_take_all_locks with synchronize_srcu. It would save a lot of CPU
and a ton of locked operations, but it'd potentially increase the
latency of the registration so the first O_DIRECT write() in a process
could still potentially stall (still better than mm_take_all_locks
which would use a lot more CPU and hurt SMP scalability in threaded
programs). The downside is all VM fast paths would get some overhead
because of srcu_read_lock even when mmu notifier is not registered,
which is what we carefully avoided by taking a larger hit in the
registration with mm_take_all_locks. This is why I don't think mmu
notifier is a good solution to solve O_DIRECT stable pages even in
theory O_DIRECT could use the exact same model as vhost to solve
stable pages.

If we solve O_DIRECT with new per-page locking or a new rwsem inode
lock leveraging the fact we're guaranteed the pin to go away ASAP,
what's left is the case of PCI devices mapping userland memory for
indefinite amount of time that cannot support MMU notifier because of
hardware limitations.

Mentioning virtualization as a case taking long term PIN is incorrect,
that didn't happen since the introduction of MMU notifier.

vfio for device assignment to KVM takes the long term pins, but that's
because the iommus may not support the mmu notifier, mmu notifier
could solve the vfio case too.

PCI devices that pretend to keep a constant mapping on userland
virtual memory and that cannot support MMU notifier because they lack
a secondary MMU, cripple the Linux VM and there's no solution to
that. Even if we solve the stable pages problem, they will still
practically disable all advanced VM features.

I think it would be ok to solve the stable pages in 3 different ways:

1) mmu notifier as above when mmu_notifier_register doesn't need to
run often and the hardware can support it

2) O_DIRECT with new locking stopping page_mkclean/try_to_free_buffers
until I/O completion, leveraging the fact the pin&lock are
guaranteed to be dropped&released ASAP

3) something else for pci devices that cannot support MMU notifier
because of hardware limitations, bounce buffers would be fine as
well

I'm not even sure if in 3) it is worth worrying about being able to
routinely flush to disk the dirty data, but then bounce buffers could
solve that. Altering the page mapped in the pte sounds like a complex
solution when you could copy the physical page just before issuing the
I/O in the writeback code. To find if a GUP pin exists it's enough to
check what KSM does:

/*
* Check that no O_DIRECT or similar I/O is in progress on the
* page
*/
if (page_mapcount(page) + 1 + swapped != page_count(page)) {

That can give false positives (even through random pins coming from
speculative cache lookups), but not false negatives.

Thanks,
Andrea

2019-03-15 04:42:08

by Jason Wang

[permalink] [raw]
Subject: Re: [RFC PATCH V2 0/5] vhost: accelerate metadata access through vmap()


On 2019/3/15 上午3:33, Andrea Arcangeli wrote:
> Hello Jason,
>
> On Thu, Mar 14, 2019 at 09:49:03PM +0800, Jason Wang wrote:
>> Yes since we don't want to slow down 32bit.
> If you've a lot of ram there's no justification to stick to a 32bit
> kernel, so I don't think there's need to maintain a separate model
> just for 32bit. I really wouldn't care about the performance of 32bit
> with >700MB of RAM if that would cause any maintenance burden. Let's
> focus on the best 64bit implementation that will work equally
> optimally on 32bit with <= 700M of RAM.


Yes, but probably there are still some reasons of keeping copy_user()
friends as a fallback. When we have a large virtqueue, the ring may
occupy more than one page. This means the VA might not be contiguous
when using kmap(). Instead of doing tricks in the accessories, maybe
it's or simpler better just fall back to copy_user() in this case. And
we meet the similar issue when software device IOTLB is used for vhost.
And in the following example for gup, we can simply do a fallback when
we race with the invalidation.

Michael also tends to keep the copy_user(), he suggested to use
copy_user() for VIVT archs then there's no need for a explicit
flush_dcache_page(). And he also want a module parameter for falling
back to copy_user() for e.g debugging purpose.


>
> Talking to Jerome about the set_page_dirty issue, he raised the point
> of what happens if two thread calls a mmu notifier invalidate
> simultaneously. The first mmu notifier could call set_page_dirty and
> then proceed in try_to_free_buffers or page_mkclean and then the
> concurrent mmu notifier that arrives second, then must not call
> set_page_dirty a second time.
>
> With KVM sptes mappings and vhost mappings you would call
> set_page_dirty (if you invoked gup with FOLL_WRITE) only when
> effectively tearing down any secondary mapping (you've got pointers in
> both cases for the mapping). So there's no way to risk a double
> set_page_dirty from concurrent mmu notifier invalidate because the
> invalidate takes a lock when it has to teardown the mapping and so
> set_page_dirty is only run in the first invalidate method and not in
> the second. In the spte case even better, as you wouldn't need to call
> it even at teardown time unless the spte is dirty (if shadow mask
> allows dirty sptes with EPT2 or NPT or shadow MMU).


I see, the sounds indeed better.


>
> If you instead had to invalidate a secondary MMU mapping that isn't
> tracked by the driver (again: not vhost nor KVM case), you could have
> used the dirty bit of the kernel pagetable to call set_page_dirty and
> disambiguate but that's really messy, and it would prevent the use of
> gigapages in the direct mapping too and it'd require vmap for 4k
> tracking.
>
> To make sure set_page_dirty is run a single time no matter if the
> invalidate known when a mapping is tear down, I suggested the below
> model:
>
> access = FOLL_WRITE
>
> repeat:
> page = gup_fast(access)
> put_page(page) /* need a way to drop FOLL_GET from gup_fast instead! */
>
> spin_lock(mmu_notifier_lock);
> if (race with invalidate) {
> spin_unlock..
> goto repeat;
> }
> if (access == FOLL_WRITE)
> set_page_dirty(page)
> establish writable mapping in secondary MMU on page
> spin_unlock
>
> (replace spin_lock with mutex_lock for vhost of course if you stick to
> a mutex and _start/_end instead of non-sleepable ->invalidate_range)


Yes, I probably stick to the vq mutex since the invalidation needs to be
synchronized with the whole packet processing routine.


> "race with invalidate" is the usual "mmu_notifier_retry" in kvm_host.h
> to be implemented for vhost.
>
> We could add a FOLL_DIRTY flag to add to FOLL_TOUCH to move the
> set_page_dirty inside GUP forced (currently it's not forced if the
> linux pte is already dirty). And we could remove FOLL_GET.
>
> Then if you have the ability to disambiguate which is the first
> invalidate that tears down a mapping to any given page (vhost can do
> that trivially, it's just a pointer to a page struct to kmap), in the
> mmu notifier invalidate just before dropping the spinlock you would
> do this check:
>
> def vhost_invalidate_range_start():
> [..]
> spin_lock(mmu_notifier_lock);
> [..]
> if (vhost->page_pointer) {
> if (access == FOLL_WRITE)
> VM_WARN_ON(!PageDirty(vhost->page_pointer));
> vhost->page_pointer = NULL;
> /* no put_page, already done at gup time */
> }
> spin_unlock(..
>
> Generally speaking set_page_dirty is safer done after the last
> modification to the data of the page. However the way stable page
> works, as long as the mmu notifier invalidate didn't run, the PG_dirty
> cannot go away.


Ok.


>
> So this model solves the issue with guaranteeing a single
> set_page_dirty is run before page_mkclean or try_to_free_buffers can
> run, even for drivers that implement the invalidate as a generic "TLB
> flush" across the whole secondary MMU and that cannot disambiguate the
> first invalidate from a second invalidate if they're issued
> concurrently on the same address by two different CPUs.
>
> So for those drivers that can disambiguate trivially (like KVM and
> vhost) we'll just add a debug check in the invalidate to validate the
> common code for all mmu notifier users.
>
> This is the solution for RDMA hardware and everything that can support
> mmu notifiers too and they can take infinitely long secondary MMU
> mappins without interfering with stable pages at all (i.e. long term
> pins but without pinning) perfectly safe and transparent to the whole
> stable page code.
>
> I think O_DIRECT for stable pages shall be solved taking the page lock
> or writeback lock or a new rwsem in the inode that is taken for
> writing by page_mkclean and try_to_free_buffers and for reading by
> outstanding O_DIRECT in flight I/O, like I suggested probably ages ago
> but then we only made GUP take the page pin, which is fine for mmu
> notifier actually (except those didn't exist back then). To solve
> O_DIRECT we can leverage the 100% guarantee that the pin will be
> dropped ASAP and stop page_mkclean and stop or trylock in
> try_to_free_buffers in such case.
>
> mm_take_all_locks is major hurdle that prevents usage in O_DIRECT
> case, even if we "cache it" if you fork(); write; exit() in a loop
> it'll still cause heavy lock overhead. MMU notifier registration isn't
> intended to happen in fast and frequent paths like the write()
> syscall. Dropping mm_take_all_locks would bring other downsides: a
> regular quiescent point can happen in between _start/_end and _start
> must be always called first all the mmu notifier retry counters we
> rely on would break. One way would be to add srcu_read_lock _before_
> you can call mm_has_mm_has_notifiers(mm), then yes we could replace
> mm_take_all_locks with synchronize_srcu. It would save a lot of CPU
> and a ton of locked operations, but it'd potentially increase the
> latency of the registration so the first O_DIRECT write() in a process
> could still potentially stall (still better than mm_take_all_locks
> which would use a lot more CPU and hurt SMP scalability in threaded
> programs). The downside is all VM fast paths would get some overhead
> because of srcu_read_lock even when mmu notifier is not registered,
> which is what we carefully avoided by taking a larger hit in the
> registration with mm_take_all_locks. This is why I don't think mmu
> notifier is a good solution to solve O_DIRECT stable pages even in
> theory O_DIRECT could use the exact same model as vhost to solve
> stable pages.
>
> If we solve O_DIRECT with new per-page locking or a new rwsem inode
> lock leveraging the fact we're guaranteed the pin to go away ASAP,
> what's left is the case of PCI devices mapping userland memory for
> indefinite amount of time that cannot support MMU notifier because of
> hardware limitations.


Yes and this is part of the issue we met in vhost TX zerocopy code.
What's more interesting is that a skb could be held of a software layer
e.g qdisc as well. MMU notifier could be used for dealing with such
software holding e.g copy packets into new pages. But this may requires
more thoughts since this may race with networking stack.


>
> Mentioning virtualization as a case taking long term PIN is incorrect,
> that didn't happen since the introduction of MMU notifier.
>
> vfio for device assignment to KVM takes the long term pins, but that's
> because the iommus may not support the mmu notifier, mmu notifier
> could solve the vfio case too.


This might require the support of page faults from IOMMU hardware.


>
> PCI devices that pretend to keep a constant mapping on userland
> virtual memory and that cannot support MMU notifier because they lack
> a secondary MMU, cripple the Linux VM and there's no solution to
> that. Even if we solve the stable pages problem, they will still
> practically disable all advanced VM features.
>
> I think it would be ok to solve the stable pages in 3 different ways:
>
> 1) mmu notifier as above when mmu_notifier_register doesn't need to
> run often and the hardware can support it
>
> 2) O_DIRECT with new locking stopping page_mkclean/try_to_free_buffers
> until I/O completion, leveraging the fact the pin&lock are
> guaranteed to be dropped&released ASAP
>
> 3) something else for pci devices that cannot support MMU notifier
> because of hardware limitations, bounce buffers would be fine as
> well
>
> I'm not even sure if in 3) it is worth worrying about being able to
> routinely flush to disk the dirty data, but then bounce buffers could
> solve that. Altering the page mapped in the pte sounds like a complex
> solution when you could copy the physical page just before issuing the
> I/O in the writeback code. To find if a GUP pin exists it's enough to
> check what KSM does:
>
> /*
> * Check that no O_DIRECT or similar I/O is in progress on the
> * page
> */
> if (page_mapcount(page) + 1 + swapped != page_count(page)) {
>
> That can give false positives (even through random pins coming from
> speculative cache lookups), but not false negatives.
>
> Thanks,
> Andrea


Thanks for the patient and detailed explanation with lots of
backgrounds. It help to understand the whole picture a lot.