All of lore.kernel.org
 help / color / mirror / Atom feed
* [RFC 0/3] Add zero copy feature for tcmu
@ 2022-03-18  9:55 Xiaoguang Wang
  2022-03-18  9:55 ` [RFC 1/3] mm/memory.c: introduce vm_insert_page(s)_mkspecial Xiaoguang Wang
                   ` (3 more replies)
  0 siblings, 4 replies; 19+ messages in thread
From: Xiaoguang Wang @ 2022-03-18  9:55 UTC (permalink / raw)
  To: linux-mm, target-devel, linux-scsi
  Cc: linux-block, xuyu, bostroesser, Xiaoguang Wang

The core idea to implement tcmu zero copy feature is really straight,
which just maps block device io request's sgl pages to tcmu user space
backstore, then we can avoid extra copy overhead between sgl pages and
tcmu internal data area(which really impacts io throughput), please see
https://www.spinics.net/lists/target-devel/msg21121.html for detailed
info.

Initially I use remap_pfn_range or vm_insert_pages to map sgl pages to
user space, but both of them have limits:
1)  Use vm_insert_pages
which is like tcp getsockopt(TCP_ZEROCOPY_RECEIVE), but there're two
restrictions:
  1. anonymous pages can not be mmaped to user spacea.
    ==> vm_insert_pages
    ====> insert_pages
    ======> insert_page_in_batch_locked
    ========> validate_page_before_insert
    In validate_page_before_insert(), it shows that anonymous page can not
    be mapped to use space, we know that if issuing direct io to block
    device, io request's sgl pages mostly comes from anonymous page.
        if (PageAnon(page) || PageSlab(page) || page_has_type(page))
            return -EINVAL;
    I'm not sure why there is such restriction? for safety reasons ?

  2. warn_on triggered in __folio_mark_dirty
    When calling zap_page_range in tcmu user space backstore when io
    completes, there is a warn_on triggered in __folio_mark_dirty:
       if (folio->mapping) {   /* Race with truncate? */
           WARN_ON_ONCE(warn && !folio_test_uptodate(folio));

    I'm not familiar with folio yet, but I think the reason is that when
    issuing a buffered read to tcmu block device, it's page cache mapped
    to user space, backstore write this page and pte will be dirtied. but
    initially it's newly allocated, hence page_update flag not set.
    In zap_pte_range(), there is such codes:
       if (!PageAnon(page)) {
           if (pte_dirty(ptent)) {
               force_flush = 1;
               set_page_dirty(page);
           }
   So this warn_on is reasonable.
   Indeed what I want is just to map io request sgl pages to tcmu user
   space backstore, then backstore can read or write data to mapped area,
   I don't want to care about page or its mapping status, so I choose to
   use remap_pfn_range.

2) Use remap_pfn_range()
  remap_pfn_range works well, but it has somewhat obvious overhead. For a
  512kb io request, it has 128 pages, and usually this 128 page's pfn are
  not consecutive, so in worst cases, for a 512kb io request, I'd need to
  issue 128 calls to remap_pfn_range, it's horrible. And in remap_pfn_range,
  if x86 page attribute table feature is enabled, lookup_memtype called by
  track_pfn_remap() also introduces obvious overhead.

Finally in order to solve these problems, Xu Yu helps to implment a new
helper, which accepts an array of pages as parameter, anonymous pages can
be mapped to user space, pages would be treated as special pte(pte_special
returns true), so vm_normal_page returns NULL, above folio warn_on won't
trigger.

Thanks.

Xiaoguang Wang (2):
  mm: export zap_page_range()
  scsi: target: tcmu: Support zero copy

Xu Yu (1):
  mm/memory.c: introduce vm_insert_page(s)_mkspecial

 drivers/target/target_core_user.c | 257 +++++++++++++++++++++++++++++++++-----
 include/linux/mm.h                |   2 +
 mm/memory.c                       | 183 +++++++++++++++++++++++++++
 3 files changed, 414 insertions(+), 28 deletions(-)

-- 
2.14.4.44.g2045bb6


^ permalink raw reply	[flat|nested] 19+ messages in thread

* [RFC 1/3] mm/memory.c: introduce vm_insert_page(s)_mkspecial
  2022-03-18  9:55 [RFC 0/3] Add zero copy feature for tcmu Xiaoguang Wang
@ 2022-03-18  9:55 ` Xiaoguang Wang
  2022-03-23 16:45   ` Christoph Hellwig
  2022-03-18  9:55 ` [RFC 2/3] mm: export zap_page_range() Xiaoguang Wang
                   ` (2 subsequent siblings)
  3 siblings, 1 reply; 19+ messages in thread
From: Xiaoguang Wang @ 2022-03-18  9:55 UTC (permalink / raw)
  To: linux-mm, target-devel, linux-scsi; +Cc: linux-block, xuyu, bostroesser

From: Xu Yu <xuyu@linux.alibaba.com>

This adds the ability to insert anonymous pages or file pages, used for
direct IO or buffer IO respectively, to a user VM. The intention behind
this is to facilitate mapping pages in IO requests to user space, which
is usually the backend of remote block device.

This integrates the advantage of vm_insert_pages (batching the pmd lock),
and eliminates the overhead of remap_pfn_range (track_pfn_remap), since
the pages to be inserted should always be ram.

NOTE that file page used in buffer IO is either locked (read) or
writeback (sync), while anonymous page used in dio is pinned. Depending
on this premise, such pages can be inserted as special PTE, without
increasing the page refcount and mapcount. On the other hand, such pages
are unlocked, writeback cleared, or unpinned in endio, by when the
special mapping in user space is zapped (and of course, it is the caller's
responsibility).

Signed-off-by: Xu Yu <xuyu@linux.alibaba.com>
---
 include/linux/mm.h |   2 +
 mm/memory.c        | 182 +++++++++++++++++++++++++++++++++++++++++++++++++++++
 2 files changed, 184 insertions(+)

diff --git a/include/linux/mm.h b/include/linux/mm.h
index 213cc569b192..0d660139b29e 100644
--- a/include/linux/mm.h
+++ b/include/linux/mm.h
@@ -2870,6 +2870,8 @@ int remap_pfn_range_notrack(struct vm_area_struct *vma, unsigned long addr,
 int vm_insert_page(struct vm_area_struct *, unsigned long addr, struct page *);
 int vm_insert_pages(struct vm_area_struct *vma, unsigned long addr,
 			struct page **pages, unsigned long *num);
+int vm_insert_pages_mkspecial(struct vm_area_struct *vma, unsigned long addr,
+			struct page **pages, unsigned long *num);
 int vm_map_pages(struct vm_area_struct *vma, struct page **pages,
 				unsigned long num);
 int vm_map_pages_zero(struct vm_area_struct *vma, struct page **pages,
diff --git a/mm/memory.c b/mm/memory.c
index c125c4969913..1f745e4d11c2 100644
--- a/mm/memory.c
+++ b/mm/memory.c
@@ -2510,6 +2510,188 @@ int vm_iomap_memory(struct vm_area_struct *vma, phys_addr_t start, unsigned long
 }
 EXPORT_SYMBOL(vm_iomap_memory);
 
+#ifdef CONFIG_ARCH_HAS_PTE_SPECIAL
+static int insert_page_into_pte_locked_mkspecial(struct mm_struct *mm, pte_t *pte,
+					unsigned long addr, struct page *page, pgprot_t prot)
+{
+	/*
+	 * The page to be inserted should be either anonymous page or file page.
+	 * The anonymous page used in dio should be pinned, while the file page
+	 * used in buffer IO is either locked (read) or writeback (sync).
+	 */
+	if (PageAnon(page)) {
+		int extra = 0;
+
+		if (PageSwapCache(page))
+			extra += 1 + page_has_private(page);
+
+		if ((page_count(page) - extra) <= page_mapcount(page))
+			return -EINVAL;
+	} else if (page_is_file_lru(page)) {
+		if (!PageLocked(page) && !PageWriteback(page))
+			return -EINVAL;
+	} else
+		return -EINVAL;
+
+	flush_dcache_page(page);
+
+	if (!pte_none(*pte))
+		return -EBUSY;
+	set_pte_at(mm, addr, pte, pte_mkspecial(mk_pte(page, prot)));
+	return 0;
+}
+
+static int insert_page_mkspecial(struct vm_area_struct *vma, unsigned long addr,
+				 struct page *page, pgprot_t prot)
+{
+	struct mm_struct *mm = vma->vm_mm;
+	int retval;
+	pte_t *pte;
+	spinlock_t *ptl;
+
+	retval = -ENOMEM;
+	pte = get_locked_pte(mm, addr, &ptl);
+	if (!pte)
+		goto out;
+	retval = insert_page_into_pte_locked_mkspecial(mm, pte, addr, page, prot);
+	pte_unmap_unlock(pte, ptl);
+out:
+	return retval;
+}
+
+/*
+ * vm_insert_page_mkspecial - variant of vm_insert_page, where pte is inserted
+ * with special bit set.
+ *
+ * Different from vm_insert_page(), @page in vm_insert_page_mkspecial() can
+ * either be anonymous page or file page, used for direct IO or buffer IO,
+ * respectively.
+ */
+int vm_insert_page_mkspecial(struct vm_area_struct *vma, unsigned long addr, struct page *page)
+{
+	if (addr < vma->vm_start || addr >= vma->vm_end)
+		return -EFAULT;
+	if (!(vma->vm_flags & VM_MIXEDMAP)) {
+		BUG_ON(mmap_read_trylock(vma->vm_mm));
+		BUG_ON(vma->vm_flags & VM_PFNMAP);
+		vma->vm_flags |= VM_MIXEDMAP;
+	}
+	return insert_page_mkspecial(vma, addr, page, vma->vm_page_prot);
+}
+
+#ifdef pte_index
+/*
+ * insert_pages_mkspecial() amortizes the cost of spinlock operations
+ * when inserting pages in a loop. Arch *must* define pte_index.
+ */
+static int insert_pages_mkspecial(struct vm_area_struct *vma, unsigned long addr,
+				  struct page **pages, unsigned long *num, pgprot_t prot)
+{
+	pmd_t *pmd = NULL;
+	pte_t *start_pte, *pte;
+	spinlock_t *pte_lock;
+	struct mm_struct *const mm = vma->vm_mm;
+	unsigned long curr_page_idx = 0;
+	unsigned long remaining_pages_total = *num;
+	unsigned long pages_to_write_in_pmd;
+	int ret;
+more:
+	ret = -EFAULT;
+	pmd = walk_to_pmd(mm, addr);
+	if (!pmd)
+		goto out;
+
+	pages_to_write_in_pmd = min_t(unsigned long,
+		remaining_pages_total, PTRS_PER_PTE - pte_index(addr));
+
+	/* Allocate the PTE if necessary; takes PMD lock once only. */
+	ret = -ENOMEM;
+	if (pte_alloc(mm, pmd))
+		goto out;
+
+	while (pages_to_write_in_pmd) {
+		int pte_idx = 0;
+		const int batch_size = min_t(int, pages_to_write_in_pmd, 8);
+
+		start_pte = pte_offset_map_lock(mm, pmd, addr, &pte_lock);
+		for (pte = start_pte; pte_idx < batch_size; ++pte, ++pte_idx) {
+			int err = insert_page_into_pte_locked_mkspecial(mm, pte,
+						addr, pages[curr_page_idx], prot);
+			if (unlikely(err)) {
+				pte_unmap_unlock(start_pte, pte_lock);
+				ret = err;
+				remaining_pages_total -= pte_idx;
+				goto out;
+			}
+			addr += PAGE_SIZE;
+			++curr_page_idx;
+		}
+		pte_unmap_unlock(start_pte, pte_lock);
+		pages_to_write_in_pmd -= batch_size;
+		remaining_pages_total -= batch_size;
+	}
+	if (remaining_pages_total)
+		goto more;
+	ret = 0;
+out:
+	*num = remaining_pages_total;
+	return ret;
+}
+#endif  /* pte_index */
+
+/*
+ * vm_insert_pages_mkspecial - variant of vm_insert_pages, where pte is inserted
+ * with special bit set.
+ *
+ * Different from vm_insert_pages(), @pages in vm_insert_pages_mkspecial() can
+ * either be anonymous page or file page, used for direct IO or buffer IO,
+ * respectively.
+ *
+ * The main purpose of vm_insert_pages_mkspecial is to combine the advantages of
+ * vm_insert_pages (batching the pmd lock) and remap_pfn_range_notrack (skipping
+ * track_pfn_insert).
+ */
+int vm_insert_pages_mkspecial(struct vm_area_struct *vma, unsigned long addr,
+			      struct page **pages, unsigned long *num)
+{
+#ifdef pte_index
+	const unsigned long end_addr = addr + (*num * PAGE_SIZE) - 1;
+
+	if (addr < vma->vm_start || end_addr >= vma->vm_end)
+		return -EFAULT;
+	if (!(vma->vm_flags & VM_MIXEDMAP)) {
+		BUG_ON(mmap_read_trylock(vma->vm_mm));
+		BUG_ON(vma->vm_flags & VM_PFNMAP);
+		vma->vm_flags |= VM_MIXEDMAP;
+	}
+	return insert_pages_mkspecial(vma, addr, pages, num, vma->vm_page_prot);
+#else
+	unsigned long idx = 0, pgcount = *num;
+	int err = -EINVAL;
+
+	for (; idx < pgcount; ++idx) {
+		err = vm_insert_page_mkspecial(vma, addr + (PAGE_SIZE * idx), pages[idx]);
+		if (err)
+			break;
+	}
+	*num = pgcount - idx;
+	return err;
+#endif	/* pte_index */
+}
+#else
+int vm_insert_page_mkspecial(struct vm_area_struct *vma, unsigned long addr, struct page *page)
+{
+	return -EINVAL;
+}
+int vm_insert_pages_mkspecial(struct vm_area_struct *vma, unsigned long addr,
+			      struct page **pages, unsigned long *num)
+{
+	return -EINVAL;
+}
+#endif	/* CONFIG_ARCH_HAS_PTE_SPECIAL */
+EXPORT_SYMBOL(vm_insert_page_mkspecial);
+EXPORT_SYMBOL(vm_insert_pages_mkspecial);
+
 static int apply_to_pte_range(struct mm_struct *mm, pmd_t *pmd,
 				     unsigned long addr, unsigned long end,
 				     pte_fn_t fn, void *data, bool create,
-- 
2.14.4.44.g2045bb6


^ permalink raw reply related	[flat|nested] 19+ messages in thread

* [RFC 2/3] mm: export zap_page_range()
  2022-03-18  9:55 [RFC 0/3] Add zero copy feature for tcmu Xiaoguang Wang
  2022-03-18  9:55 ` [RFC 1/3] mm/memory.c: introduce vm_insert_page(s)_mkspecial Xiaoguang Wang
@ 2022-03-18  9:55 ` Xiaoguang Wang
  2022-03-21 12:01   ` David Hildenbrand
                     ` (2 more replies)
  2022-03-18  9:55 ` [RFC 3/3] scsi: target: tcmu: Support zero copy Xiaoguang Wang
  2022-03-22 12:40 ` [RFC 0/3] Add zero copy feature for tcmu Bodo Stroesser
  3 siblings, 3 replies; 19+ messages in thread
From: Xiaoguang Wang @ 2022-03-18  9:55 UTC (permalink / raw)
  To: linux-mm, target-devel, linux-scsi
  Cc: linux-block, xuyu, bostroesser, Xiaoguang Wang

Module target_core_user will use it to implement zero copy feature.

Signed-off-by: Xiaoguang Wang <xiaoguang.wang@linux.alibaba.com>
---
 mm/memory.c | 1 +
 1 file changed, 1 insertion(+)

diff --git a/mm/memory.c b/mm/memory.c
index 1f745e4d11c2..9974d0406dad 100644
--- a/mm/memory.c
+++ b/mm/memory.c
@@ -1664,6 +1664,7 @@ void zap_page_range(struct vm_area_struct *vma, unsigned long start,
 	mmu_notifier_invalidate_range_end(&range);
 	tlb_finish_mmu(&tlb);
 }
+EXPORT_SYMBOL_GPL(zap_page_range);
 
 /**
  * zap_page_range_single - remove user pages in a given range
-- 
2.14.4.44.g2045bb6


^ permalink raw reply related	[flat|nested] 19+ messages in thread

* [RFC 3/3] scsi: target: tcmu: Support zero copy
  2022-03-18  9:55 [RFC 0/3] Add zero copy feature for tcmu Xiaoguang Wang
  2022-03-18  9:55 ` [RFC 1/3] mm/memory.c: introduce vm_insert_page(s)_mkspecial Xiaoguang Wang
  2022-03-18  9:55 ` [RFC 2/3] mm: export zap_page_range() Xiaoguang Wang
@ 2022-03-18  9:55 ` Xiaoguang Wang
  2022-03-22 14:01   ` Bodo Stroesser
  2022-03-22 12:40 ` [RFC 0/3] Add zero copy feature for tcmu Bodo Stroesser
  3 siblings, 1 reply; 19+ messages in thread
From: Xiaoguang Wang @ 2022-03-18  9:55 UTC (permalink / raw)
  To: linux-mm, target-devel, linux-scsi
  Cc: linux-block, xuyu, bostroesser, Xiaoguang Wang

Currently in tcmu, for READ commands, it copies user space backstore's
data buffer to tcmu internal data area, then copies data in data area
to READ commands sgl pages. For WRITE commands, tcmu copies sgl pages
to tcmu internal data area, then copies data in data area to user space
backstore. For both cases, there are obvious copy overhead, which impact
io throughput, especially for large io size.

To mitigate this issue, we implement zero copy feature to tcmu, which
map sgl pages to user space backstore's address space. Currently only
sgl pages's offset and length are both aligned to page size, can this
command go into tcmu zero copy path.

Signed-off-by: Xiaoguang Wang <xiaoguang.wang@linux.alibaba.com>
---
 drivers/target/target_core_user.c | 257 +++++++++++++++++++++++++++++++++-----
 1 file changed, 229 insertions(+), 28 deletions(-)

diff --git a/drivers/target/target_core_user.c b/drivers/target/target_core_user.c
index 7b2a89a67cdb..4314e0c00f8e 100644
--- a/drivers/target/target_core_user.c
+++ b/drivers/target/target_core_user.c
@@ -16,6 +16,8 @@
 #include <linux/xarray.h>
 #include <linux/stringify.h>
 #include <linux/bitops.h>
+#include <linux/sched/mm.h>
+#include <linux/mm.h>
 #include <linux/highmem.h>
 #include <linux/configfs.h>
 #include <linux/mutex.h>
@@ -72,6 +74,7 @@
  */
 #define DATA_PAGES_PER_BLK_DEF 1
 #define DATA_AREA_PAGES_DEF (256 * 1024)
+#define ZC_DATA_AREA_PAGES_DEF (256 * 1024)
 
 #define TCMU_MBS_TO_PAGES(_mbs) ((size_t)_mbs << (20 - PAGE_SHIFT))
 #define TCMU_PAGES_TO_MBS(_pages) (_pages >> (20 - PAGE_SHIFT))
@@ -145,9 +148,20 @@ struct tcmu_dev {
 	struct list_head qfull_queue;
 	struct list_head tmr_queue;
 
+	/* For zero copy handle */
+	int zc_data_area_mb;
+	uint32_t zc_max_blocks;
 	uint32_t dbi_max;
 	uint32_t dbi_thresh;
 	unsigned long *data_bitmap;
+
+	struct mm_struct *vma_vm_mm;
+	struct vm_area_struct *vma;
+
+	uint32_t zc_dbi_max;
+	uint32_t zc_dbi_thresh;
+	unsigned long *zc_data_bitmap;
+
 	struct xarray data_pages;
 	uint32_t data_pages_per_blk;
 	uint32_t data_blk_size;
@@ -177,6 +191,12 @@ struct tcmu_cmd {
 	struct tcmu_dev *tcmu_dev;
 	struct list_head queue_entry;
 
+	/* For zero_copy handle */
+	struct mm_struct *vma_vm_mm;
+	struct vm_area_struct *vma;
+	struct iovec *iov;
+	int iov_cnt;
+
 	uint16_t cmd_id;
 
 	/* Can't use se_cmd when cleaning up expired cmds, because if
@@ -192,6 +212,7 @@ struct tcmu_cmd {
 
 #define TCMU_CMD_BIT_EXPIRED 0
 #define TCMU_CMD_BIT_KEEP_BUF 1
+#define TCMU_CMD_BIT_ZEROCOPY 2
 	unsigned long flags;
 };
 
@@ -495,10 +516,16 @@ static struct genl_family tcmu_genl_family __ro_after_init = {
 static void tcmu_cmd_free_data(struct tcmu_cmd *tcmu_cmd, uint32_t len)
 {
 	struct tcmu_dev *udev = tcmu_cmd->tcmu_dev;
+	unsigned long *data_bitmap;
 	uint32_t i;
 
+	if (test_bit(TCMU_CMD_BIT_ZEROCOPY, &tcmu_cmd->flags))
+		data_bitmap = udev->zc_data_bitmap;
+	else
+		data_bitmap = udev->data_bitmap;
+
 	for (i = 0; i < len; i++)
-		clear_bit(tcmu_cmd->dbi[i], udev->data_bitmap);
+		clear_bit(tcmu_cmd->dbi[i], data_bitmap);
 }
 
 static inline int tcmu_get_empty_block(struct tcmu_dev *udev,
@@ -549,8 +576,30 @@ static inline int tcmu_get_empty_block(struct tcmu_dev *udev,
 	return i == page_cnt ? dbi : -1;
 }
 
+static inline int tcmu_get_zc_empty_block(struct tcmu_dev *udev,
+			struct tcmu_cmd *tcmu_cmd, int prev_dbi,
+			int *iov_cnt)
+{
+	int dbi;
+
+	dbi = find_first_zero_bit(udev->zc_data_bitmap, udev->zc_dbi_thresh);
+	if (dbi == udev->zc_dbi_thresh)
+		return -1;
+
+	if (dbi > udev->zc_dbi_max)
+		udev->zc_dbi_max = dbi;
+
+	set_bit(dbi, udev->zc_data_bitmap);
+	tcmu_cmd_set_dbi(tcmu_cmd, dbi);
+
+	if (dbi != prev_dbi + 1)
+		*iov_cnt += 1;
+	return dbi;
+}
+
 static int tcmu_get_empty_blocks(struct tcmu_dev *udev,
-				 struct tcmu_cmd *tcmu_cmd, int length)
+				 struct tcmu_cmd *tcmu_cmd, int length,
+				 bool zero_copy)
 {
 	/* start value of dbi + 1 must not be a valid dbi */
 	int dbi = -2;
@@ -559,16 +608,111 @@ static int tcmu_get_empty_blocks(struct tcmu_dev *udev,
 
 	for (; length > 0; length -= blk_size) {
 		blk_data_len = min_t(uint32_t, length, blk_size);
-		dbi = tcmu_get_empty_block(udev, tcmu_cmd, dbi, blk_data_len,
-					   &iov_cnt);
+		if (zero_copy) {
+			dbi = tcmu_get_zc_empty_block(udev, tcmu_cmd, dbi,
+						      &iov_cnt);
+		} else {
+			dbi = tcmu_get_empty_block(udev, tcmu_cmd, dbi,
+					blk_data_len, &iov_cnt);
+		}
 		if (dbi < 0)
 			return -1;
 	}
 	return iov_cnt;
 }
 
+#define TCMU_ZEROCOPY_PAGE_BATCH 32
+
+static inline void tcmu_zerocopy_one_seg(struct iovec *iov,
+			struct vm_area_struct *vma,
+			struct sg_page_iter *sgiter)
+{
+	struct page *pages[TCMU_ZEROCOPY_PAGE_BATCH];
+	unsigned int len = iov->iov_len;
+	unsigned long address = (unsigned long)iov->iov_base + vma->vm_start;
+	unsigned long pages_remaining, pg_index = 0;
+	struct page *page;
+
+	while (len > 0) {
+		__sg_page_iter_next(sgiter);
+		page = sg_page_iter_page(sgiter);
+		pages[pg_index++] = page;
+		len -= PAGE_SIZE;
+		if (pg_index == TCMU_ZEROCOPY_PAGE_BATCH || !len) {
+			pages_remaining = pg_index;
+			vm_insert_pages_mkspecial(vma, address, pages, &pages_remaining);
+			address = address + pg_index * PAGE_SIZE;
+			pg_index = 0;
+		}
+	}
+}
+
+static long tcmu_cmd_zerocopy_map(struct tcmu_dev *udev,
+				  struct tcmu_cmd *cmd,
+				  struct iovec *iov,
+				  int iov_cnt)
+{
+	struct se_cmd *se_cmd = cmd->se_cmd;
+	struct scatterlist *data_sg;
+	unsigned int data_nents;
+	struct iovec *tiov;
+	struct sg_page_iter sgiter;
+	struct vm_area_struct *vma = udev->vma;
+	int i, ret = 0;
+
+	mmap_read_lock(udev->vma_vm_mm);
+	data_sg = se_cmd->t_data_sg;
+	data_nents = se_cmd->t_data_nents;
+	__sg_page_iter_start(&sgiter, data_sg, data_nents, 0);
+	tiov = iov;
+	for (i = 0; i < iov_cnt; i++) {
+		tcmu_zerocopy_one_seg(tiov, vma, &sgiter);
+		tiov++;
+	}
+	cmd->iov = iov;
+	cmd->iov_cnt = iov_cnt;
+	cmd->vma_vm_mm = vma->vm_mm;
+	cmd->vma = vma;
+	mmgrab(cmd->vma_vm_mm);
+	mmap_read_unlock(udev->vma_vm_mm);
+	return ret;
+}
+
+static void tcmu_cmd_zerocopy_unmap(struct tcmu_cmd *cmd)
+{
+	struct mm_struct *mm;
+	struct vm_area_struct *vma;
+	struct iovec *iov;
+	unsigned long address;
+	int i;
+
+	mm = cmd->vma_vm_mm;
+	if (!mm)
+		return;
+
+	vma = cmd->vma;
+	iov = cmd->iov;
+	if (mmget_not_zero(mm)) {
+		mmap_read_lock(mm);
+		for (i = 0; i < cmd->iov_cnt; i++) {
+			address = (unsigned long)iov->iov_base + vma->vm_start;
+			zap_page_range(vma, address, iov->iov_len);
+			iov++;
+		}
+		mmap_read_unlock(mm);
+		mmput(mm);
+	}
+
+	cmd->vma_vm_mm = NULL;
+	cmd->vma = NULL;
+	mmdrop(mm);
+}
+
 static inline void tcmu_free_cmd(struct tcmu_cmd *tcmu_cmd)
 {
+	if (test_bit(TCMU_CMD_BIT_ZEROCOPY, &tcmu_cmd->flags))
+		tcmu_cmd_zerocopy_unmap(tcmu_cmd);
+
 	kfree(tcmu_cmd->dbi);
 	kmem_cache_free(tcmu_cmd_cache, tcmu_cmd);
 }
@@ -850,37 +994,57 @@ static bool is_ring_space_avail(struct tcmu_dev *udev, size_t cmd_size)
  * Called with ring lock held.
  */
 static int tcmu_alloc_data_space(struct tcmu_dev *udev, struct tcmu_cmd *cmd,
-				  int *iov_bidi_cnt)
+				  int *iov_bidi_cnt, bool zero_copy)
 {
 	int space, iov_cnt = 0, ret = 0;
 
 	if (!cmd->dbi_cnt)
 		goto wr_iov_cnts;
 
-	/* try to check and get the data blocks as needed */
-	space = spc_bitmap_free(udev->data_bitmap, udev->dbi_thresh);
-	if (space < cmd->dbi_cnt) {
-		unsigned long blocks_left =
-				(udev->max_blocks - udev->dbi_thresh) + space;
+	if (!zero_copy) {
+		/* try to check and get the data blocks as needed */
+		space = spc_bitmap_free(udev->data_bitmap, udev->dbi_thresh);
+		if (space < cmd->dbi_cnt) {
+			unsigned long blocks_left =
+					(udev->max_blocks - udev->dbi_thresh) + space;
+
+			if (blocks_left < cmd->dbi_cnt) {
+				pr_debug("no data space: only %lu available, but ask for %u\n",
+						blocks_left * udev->data_blk_size,
+						cmd->dbi_cnt * udev->data_blk_size);
+				return -1;
+			}
 
-		if (blocks_left < cmd->dbi_cnt) {
-			pr_debug("no data space: only %lu available, but ask for %u\n",
-					blocks_left * udev->data_blk_size,
-					cmd->dbi_cnt * udev->data_blk_size);
-			return -1;
+			udev->dbi_thresh += cmd->dbi_cnt;
+			if (udev->dbi_thresh > udev->max_blocks)
+				udev->dbi_thresh = udev->max_blocks;
 		}
+	} else {
+		/* try to check and get the data blocks as needed */
+		space = spc_bitmap_free(udev->zc_data_bitmap, udev->zc_dbi_thresh);
+		if (space < cmd->dbi_cnt) {
+			unsigned long blocks_left =
+					(udev->zc_max_blocks - udev->zc_dbi_thresh) + space;
+
+			if (blocks_left < cmd->dbi_cnt) {
+				pr_debug("no data space: only %lu available, but ask for %u\n",
+						blocks_left * udev->data_blk_size,
+						cmd->dbi_cnt * udev->data_blk_size);
+				return -1;
+			}
 
-		udev->dbi_thresh += cmd->dbi_cnt;
-		if (udev->dbi_thresh > udev->max_blocks)
-			udev->dbi_thresh = udev->max_blocks;
+			udev->zc_dbi_thresh += cmd->dbi_cnt;
+			if (udev->zc_dbi_thresh > udev->zc_max_blocks)
+				udev->zc_dbi_thresh = udev->zc_max_blocks;
+		}
 	}
 
-	iov_cnt = tcmu_get_empty_blocks(udev, cmd, cmd->se_cmd->data_length);
+	iov_cnt = tcmu_get_empty_blocks(udev, cmd, cmd->se_cmd->data_length, zero_copy);
 	if (iov_cnt < 0)
 		return -1;
 
 	if (cmd->dbi_bidi_cnt) {
-		ret = tcmu_get_empty_blocks(udev, cmd, cmd->data_len_bidi);
+		ret = tcmu_get_empty_blocks(udev, cmd, cmd->data_len_bidi, zero_copy);
 		if (ret < 0)
 			return -1;
 	}
@@ -1021,6 +1185,7 @@ static int queue_cmd_ring(struct tcmu_cmd *tcmu_cmd, sense_reason_t *scsi_err)
 	uint32_t blk_size = udev->data_blk_size;
 	/* size of data buffer needed */
 	size_t data_length = (size_t)tcmu_cmd->dbi_cnt * blk_size;
+	bool zero_copy = false;
 
 	*scsi_err = TCM_NO_SENSE;
 
@@ -1044,7 +1209,22 @@ static int queue_cmd_ring(struct tcmu_cmd *tcmu_cmd, sense_reason_t *scsi_err)
 		return -1;
 	}
 
-	iov_cnt = tcmu_alloc_data_space(udev, tcmu_cmd, &iov_bidi_cnt);
+	if (!(se_cmd->se_cmd_flags & SCF_BIDI) && se_cmd->data_length &&
+	    IS_ALIGNED(se_cmd->data_length, PAGE_SIZE)) {
+		struct scatterlist *data_sg = se_cmd->t_data_sg, *sg;
+		unsigned int data_nents = se_cmd->t_data_nents;
+		int i;
+
+		for_each_sg(data_sg, sg, data_nents, i) {
+			if (!((!sg->offset || IS_ALIGNED(sg->offset, PAGE_SIZE)) &&
+			    IS_ALIGNED(sg->length, PAGE_SIZE)))
+				break;
+		}
+		if (i == data_nents)
+			zero_copy = true;
+	}
+
+	iov_cnt = tcmu_alloc_data_space(udev, tcmu_cmd, &iov_bidi_cnt, zero_copy);
 	if (iov_cnt < 0)
 		goto free_and_queue;
 
@@ -1093,7 +1273,7 @@ static int queue_cmd_ring(struct tcmu_cmd *tcmu_cmd, sense_reason_t *scsi_err)
 	tcmu_cmd_reset_dbi_cur(tcmu_cmd);
 	iov = &entry->req.iov[0];
 
-	if (se_cmd->data_direction == DMA_TO_DEVICE ||
+	if (((se_cmd->data_direction == DMA_TO_DEVICE) && !zero_copy) ||
 	    se_cmd->se_cmd_flags & SCF_BIDI)
 		scatter_data_area(udev, tcmu_cmd, &iov);
 	else
@@ -1111,6 +1291,11 @@ static int queue_cmd_ring(struct tcmu_cmd *tcmu_cmd, sense_reason_t *scsi_err)
 	tcmu_setup_cmd_timer(tcmu_cmd, udev->cmd_time_out, &udev->cmd_timer);
 
 	entry->hdr.cmd_id = tcmu_cmd->cmd_id;
+	if (zero_copy) {
+		iov = &entry->req.iov[0];
+		tcmu_cmd_zerocopy_map(udev, tcmu_cmd, iov, entry->req.iov_cnt);
+		set_bit(TCMU_CMD_BIT_ZEROCOPY, &tcmu_cmd->flags);
+	}
 
 	tcmu_hdr_set_len(&entry->hdr.len_op, command_size);
 
@@ -1366,7 +1551,10 @@ static bool tcmu_handle_completion(struct tcmu_cmd *cmd,
 		else
 			se_cmd->se_cmd_flags |= SCF_TREAT_READ_AS_NORMAL;
 	}
-	if (se_cmd->se_cmd_flags & SCF_BIDI) {
+
+	if (test_bit(TCMU_CMD_BIT_ZEROCOPY, &cmd->flags)) {
+		tcmu_cmd_zerocopy_unmap(cmd);
+	} else if (se_cmd->se_cmd_flags & SCF_BIDI) {
 		/* Get Data-In buffer before clean up */
 		gather_data_area(udev, cmd, true, read_len);
 	} else if (se_cmd->data_direction == DMA_FROM_DEVICE) {
@@ -1520,6 +1708,8 @@ static void tcmu_check_expired_ring_cmd(struct tcmu_cmd *cmd)
 	if (!time_after_eq(jiffies, cmd->deadline))
 		return;
 
+	if (test_bit(TCMU_CMD_BIT_ZEROCOPY, &cmd->flags))
+		tcmu_cmd_zerocopy_unmap(cmd);
 	set_bit(TCMU_CMD_BIT_EXPIRED, &cmd->flags);
 	list_del_init(&cmd->queue_entry);
 	se_cmd = cmd->se_cmd;
@@ -1618,6 +1808,8 @@ static struct se_device *tcmu_alloc_device(struct se_hba *hba, const char *name)
 	udev->data_pages_per_blk = DATA_PAGES_PER_BLK_DEF;
 	udev->max_blocks = DATA_AREA_PAGES_DEF / udev->data_pages_per_blk;
 	udev->data_area_mb = TCMU_PAGES_TO_MBS(DATA_AREA_PAGES_DEF);
+	udev->zc_max_blocks = ZC_DATA_AREA_PAGES_DEF / udev->data_pages_per_blk;
+	udev->zc_data_area_mb = TCMU_PAGES_TO_MBS(ZC_DATA_AREA_PAGES_DEF);
 
 	mutex_init(&udev->cmdr_lock);
 
@@ -1841,6 +2033,9 @@ static void tcmu_vma_open(struct vm_area_struct *vma)
 
 	pr_debug("vma_open\n");
 
+	udev->vma_vm_mm = vma->vm_mm;
+	udev->vma = vma;
+	mmgrab(udev->vma_vm_mm);
 	kref_get(&udev->kref);
 }
 
@@ -1850,6 +2045,10 @@ static void tcmu_vma_close(struct vm_area_struct *vma)
 
 	pr_debug("vma_close\n");
 
+	mmdrop(udev->vma_vm_mm);
+	udev->vma_vm_mm = NULL;
+	udev->vma = NULL;
+
 	/* release ref from tcmu_vma_open */
 	kref_put(&udev->kref, tcmu_dev_kref_release);
 }
@@ -1901,7 +2100,7 @@ static int tcmu_mmap(struct uio_info *info, struct vm_area_struct *vma)
 {
 	struct tcmu_dev *udev = container_of(info, struct tcmu_dev, uio_info);
 
-	vma->vm_flags |= VM_DONTEXPAND | VM_DONTDUMP;
+	vma->vm_flags |= VM_DONTEXPAND | VM_DONTDUMP | VM_MIXEDMAP;
 	vma->vm_ops = &tcmu_vm_ops;
 
 	vma->vm_private_data = udev;
@@ -2172,7 +2371,7 @@ static int tcmu_configure_device(struct se_device *dev)
 	struct tcmu_dev *udev = TCMU_DEV(dev);
 	struct uio_info *info;
 	struct tcmu_mailbox *mb;
-	size_t data_size;
+	size_t data_size, zc_data_size;
 	int ret = 0;
 
 	ret = tcmu_update_uio_info(udev);
@@ -2183,8 +2382,9 @@ static int tcmu_configure_device(struct se_device *dev)
 
 	mutex_lock(&udev->cmdr_lock);
 	udev->data_bitmap = bitmap_zalloc(udev->max_blocks, GFP_KERNEL);
+	udev->zc_data_bitmap = bitmap_zalloc(udev->zc_max_blocks, GFP_KERNEL);
 	mutex_unlock(&udev->cmdr_lock);
-	if (!udev->data_bitmap) {
+	if (!udev->data_bitmap || !udev->zc_data_bitmap) {
 		ret = -ENOMEM;
 		goto err_bitmap_alloc;
 	}
@@ -2201,7 +2401,8 @@ static int tcmu_configure_device(struct se_device *dev)
 	udev->cmdr_size = CMDR_SIZE;
 	udev->data_off = MB_CMDR_SIZE;
 	data_size = TCMU_MBS_TO_PAGES(udev->data_area_mb) << PAGE_SHIFT;
-	udev->mmap_pages = (data_size + MB_CMDR_SIZE) >> PAGE_SHIFT;
+	zc_data_size = TCMU_MBS_TO_PAGES(udev->zc_data_area_mb) << PAGE_SHIFT;
+	udev->mmap_pages = (data_size + zc_data_size + MB_CMDR_SIZE) >> PAGE_SHIFT;
 	udev->data_blk_size = udev->data_pages_per_blk * PAGE_SIZE;
 	udev->dbi_thresh = 0; /* Default in Idle state */
 
@@ -2221,7 +2422,7 @@ static int tcmu_configure_device(struct se_device *dev)
 
 	info->mem[0].name = "tcm-user command & data buffer";
 	info->mem[0].addr = (phys_addr_t)(uintptr_t)udev->mb_addr;
-	info->mem[0].size = data_size + MB_CMDR_SIZE;
+	info->mem[0].size = data_size + zc_data_size + MB_CMDR_SIZE;
 	info->mem[0].memtype = UIO_MEM_NONE;
 
 	info->irqcontrol = tcmu_irqcontrol;
-- 
2.14.4.44.g2045bb6


^ permalink raw reply related	[flat|nested] 19+ messages in thread

* Re: [RFC 2/3] mm: export zap_page_range()
  2022-03-18  9:55 ` [RFC 2/3] mm: export zap_page_range() Xiaoguang Wang
@ 2022-03-21 12:01   ` David Hildenbrand
  2022-03-22 13:02     ` Xiaoguang Wang
  2022-03-23 16:47   ` Christoph Hellwig
  2022-03-24  9:16   ` Ming Lei
  2 siblings, 1 reply; 19+ messages in thread
From: David Hildenbrand @ 2022-03-21 12:01 UTC (permalink / raw)
  To: Xiaoguang Wang, linux-mm, target-devel, linux-scsi
  Cc: linux-block, xuyu, bostroesser

On 18.03.22 10:55, Xiaoguang Wang wrote:
> Module target_core_user will use it to implement zero copy feature.
> 
> Signed-off-by: Xiaoguang Wang <xiaoguang.wang@linux.alibaba.com>
> ---
>  mm/memory.c | 1 +
>  1 file changed, 1 insertion(+)
> 
> diff --git a/mm/memory.c b/mm/memory.c
> index 1f745e4d11c2..9974d0406dad 100644
> --- a/mm/memory.c
> +++ b/mm/memory.c
> @@ -1664,6 +1664,7 @@ void zap_page_range(struct vm_area_struct *vma, unsigned long start,
>  	mmu_notifier_invalidate_range_end(&range);
>  	tlb_finish_mmu(&tlb);
>  }
> +EXPORT_SYMBOL_GPL(zap_page_range);
>  
>  /**
>   * zap_page_range_single - remove user pages in a given range

To which VMAs will you be applying zap_page_range? I assume only to some
special ones where you previously vm_insert_page(s)_mkspecial'ed pages,
not to some otherwise random VMAs, correct?

-- 
Thanks,

David / dhildenb


^ permalink raw reply	[flat|nested] 19+ messages in thread

* Re: [RFC 0/3] Add zero copy feature for tcmu
  2022-03-18  9:55 [RFC 0/3] Add zero copy feature for tcmu Xiaoguang Wang
                   ` (2 preceding siblings ...)
  2022-03-18  9:55 ` [RFC 3/3] scsi: target: tcmu: Support zero copy Xiaoguang Wang
@ 2022-03-22 12:40 ` Bodo Stroesser
  2022-03-22 13:17   ` Xiaoguang Wang
  3 siblings, 1 reply; 19+ messages in thread
From: Bodo Stroesser @ 2022-03-22 12:40 UTC (permalink / raw)
  To: Xiaoguang Wang, linux-mm, target-devel, linux-scsi; +Cc: linux-block, xuyu

On 18.03.22 10:55, Xiaoguang Wang wrote:
> The core idea to implement tcmu zero copy feature is really straight,
> which just maps block device io request's sgl pages to tcmu user space
> backstore, then we can avoid extra copy overhead between sgl pages and
> tcmu internal data area(which really impacts io throughput), please see
> https://www.spinics.net/lists/target-devel/msg21121.html for detailed
> info.
> 

Can you please tell us, how big the performance improvement is and
which configuration you are using for measurenments?

> Initially I use remap_pfn_range or vm_insert_pages to map sgl pages to
> user space, but both of them have limits:
> 1)  Use vm_insert_pages
> which is like tcp getsockopt(TCP_ZEROCOPY_RECEIVE), but there're two
> restrictions:
>    1. anonymous pages can not be mmaped to user spacea.
>      ==> vm_insert_pages
>      ====> insert_pages
>      ======> insert_page_in_batch_locked
>      ========> validate_page_before_insert
>      In validate_page_before_insert(), it shows that anonymous page can not
>      be mapped to use space, we know that if issuing direct io to block
>      device, io request's sgl pages mostly comes from anonymous page.
>          if (PageAnon(page) || PageSlab(page) || page_has_type(page))
>              return -EINVAL;
>      I'm not sure why there is such restriction? for safety reasons ?
> 
>    2. warn_on triggered in __folio_mark_dirty
>      When calling zap_page_range in tcmu user space backstore when io
>      completes, there is a warn_on triggered in __folio_mark_dirty:
>         if (folio->mapping) {   /* Race with truncate? */
>             WARN_ON_ONCE(warn && !folio_test_uptodate(folio));
> 
>      I'm not familiar with folio yet, but I think the reason is that when
>      issuing a buffered read to tcmu block device, it's page cache mapped
>      to user space, backstore write this page and pte will be dirtied. but
>      initially it's newly allocated, hence page_update flag not set.
>      In zap_pte_range(), there is such codes:
>         if (!PageAnon(page)) {
>             if (pte_dirty(ptent)) {
>                 force_flush = 1;
>                 set_page_dirty(page);
>             }
>     So this warn_on is reasonable.
>     Indeed what I want is just to map io request sgl pages to tcmu user
>     space backstore, then backstore can read or write data to mapped area,
>     I don't want to care about page or its mapping status, so I choose to
>     use remap_pfn_range.
> 
> 2) Use remap_pfn_range()
>    remap_pfn_range works well, but it has somewhat obvious overhead. For a
>    512kb io request, it has 128 pages, and usually this 128 page's pfn are
>    not consecutive, so in worst cases, for a 512kb io request, I'd need to
>    issue 128 calls to remap_pfn_range, it's horrible. And in remap_pfn_range,
>    if x86 page attribute table feature is enabled, lookup_memtype called by
>    track_pfn_remap() also introduces obvious overhead.
> 
> Finally in order to solve these problems, Xu Yu helps to implment a new
> helper, which accepts an array of pages as parameter, anonymous pages can
> be mapped to user space, pages would be treated as special pte(pte_special
> returns true), so vm_normal_page returns NULL, above folio warn_on won't
> trigger.
> 
> Thanks.
> 
> Xiaoguang Wang (2):
>    mm: export zap_page_range()
>    scsi: target: tcmu: Support zero copy
> 
> Xu Yu (1):
>    mm/memory.c: introduce vm_insert_page(s)_mkspecial
> 
>   drivers/target/target_core_user.c | 257 +++++++++++++++++++++++++++++++++-----
>   include/linux/mm.h                |   2 +
>   mm/memory.c                       | 183 +++++++++++++++++++++++++++
>   3 files changed, 414 insertions(+), 28 deletions(-)
> 

^ permalink raw reply	[flat|nested] 19+ messages in thread

* Re: [RFC 2/3] mm: export zap_page_range()
  2022-03-21 12:01   ` David Hildenbrand
@ 2022-03-22 13:02     ` Xiaoguang Wang
  2022-03-22 13:08       ` David Hildenbrand
  0 siblings, 1 reply; 19+ messages in thread
From: Xiaoguang Wang @ 2022-03-22 13:02 UTC (permalink / raw)
  To: David Hildenbrand, linux-mm, target-devel, linux-scsi
  Cc: linux-block, xuyu, bostroesser

hi,

> On 18.03.22 10:55, Xiaoguang Wang wrote:
>> Module target_core_user will use it to implement zero copy feature.
>>
>> Signed-off-by: Xiaoguang Wang <xiaoguang.wang@linux.alibaba.com>
>> ---
>>   mm/memory.c | 1 +
>>   1 file changed, 1 insertion(+)
>>
>> diff --git a/mm/memory.c b/mm/memory.c
>> index 1f745e4d11c2..9974d0406dad 100644
>> --- a/mm/memory.c
>> +++ b/mm/memory.c
>> @@ -1664,6 +1664,7 @@ void zap_page_range(struct vm_area_struct *vma, unsigned long start,
>>   	mmu_notifier_invalidate_range_end(&range);
>>   	tlb_finish_mmu(&tlb);
>>   }
>> +EXPORT_SYMBOL_GPL(zap_page_range);
>>   
>>   /**
>>    * zap_page_range_single - remove user pages in a given range
> To which VMAs will you be applying zap_page_range? I assume only to some
> special ones where you previously vm_insert_page(s)_mkspecial'ed pages,
> not to some otherwise random VMAs, correct?
Yes, you're right :)

Regards,
Xiaoguang Wang
>


^ permalink raw reply	[flat|nested] 19+ messages in thread

* Re: [RFC 2/3] mm: export zap_page_range()
  2022-03-22 13:02     ` Xiaoguang Wang
@ 2022-03-22 13:08       ` David Hildenbrand
  2022-03-23 13:59         ` Xiaoguang Wang
  2022-03-23 16:48         ` Christoph Hellwig
  0 siblings, 2 replies; 19+ messages in thread
From: David Hildenbrand @ 2022-03-22 13:08 UTC (permalink / raw)
  To: Xiaoguang Wang, linux-mm, target-devel, linux-scsi
  Cc: linux-block, xuyu, bostroesser

On 22.03.22 14:02, Xiaoguang Wang wrote:
> hi,
> 
>> On 18.03.22 10:55, Xiaoguang Wang wrote:
>>> Module target_core_user will use it to implement zero copy feature.
>>>
>>> Signed-off-by: Xiaoguang Wang <xiaoguang.wang@linux.alibaba.com>
>>> ---
>>>   mm/memory.c | 1 +
>>>   1 file changed, 1 insertion(+)
>>>
>>> diff --git a/mm/memory.c b/mm/memory.c
>>> index 1f745e4d11c2..9974d0406dad 100644
>>> --- a/mm/memory.c
>>> +++ b/mm/memory.c
>>> @@ -1664,6 +1664,7 @@ void zap_page_range(struct vm_area_struct *vma, unsigned long start,
>>>   	mmu_notifier_invalidate_range_end(&range);
>>>   	tlb_finish_mmu(&tlb);
>>>   }
>>> +EXPORT_SYMBOL_GPL(zap_page_range);
>>>   
>>>   /**
>>>    * zap_page_range_single - remove user pages in a given range
>> To which VMAs will you be applying zap_page_range? I assume only to some
>> special ones where you previously vm_insert_page(s)_mkspecial'ed pages,
>> not to some otherwise random VMAs, correct?
> Yes, you're right :)

I'd suggest exposing a dedicated function that performs sanity checks on
the vma (VM_PFNMAP ?) and only zaps within a single VMA.

Essentially zap_page_range_single(), excluding "struct zap_details
*details" and including sanity checks.

Reason is that we don't want anybody to blindly zap_page_range() within
random VMAs from a kernel module.

-- 
Thanks,

David / dhildenb


^ permalink raw reply	[flat|nested] 19+ messages in thread

* Re: [RFC 0/3] Add zero copy feature for tcmu
  2022-03-22 12:40 ` [RFC 0/3] Add zero copy feature for tcmu Bodo Stroesser
@ 2022-03-22 13:17   ` Xiaoguang Wang
  2022-03-22 14:05     ` Bodo Stroesser
  0 siblings, 1 reply; 19+ messages in thread
From: Xiaoguang Wang @ 2022-03-22 13:17 UTC (permalink / raw)
  To: Bodo Stroesser, linux-mm, target-devel, linux-scsi; +Cc: linux-block, xuyu

hi,

> On 18.03.22 10:55, Xiaoguang Wang wrote:
>> The core idea to implement tcmu zero copy feature is really straight,
>> which just maps block device io request's sgl pages to tcmu user space
>> backstore, then we can avoid extra copy overhead between sgl pages and
>> tcmu internal data area(which really impacts io throughput), please see
>> https://www.spinics.net/lists/target-devel/msg21121.html for detailed
>> info.
>>
>
> Can you please tell us, how big the performance improvement is and
> which configuration you are using for measurenments?
Sorry, I should have attached test results here. Initially I tried to use
tcmu user:fbo backstore to evaluate performance improvements, but
it only shows about 10%~15% io throughput improvement. Fio config
is numjobs=1, iodepth=8, bs=256k, which isn't very impressive. The
reason is that user:fbo backstore does buffered reads, it consumes most
of cpu.

Then I test this zero copy feature for our real workload, whose backstore
is a network program visiting distributed file system and it's 
multi-threaded.
For 4 job, 8 depth, 256 kb io size, the write throughput improves from
3.6GB/s to 10GB/s.

Regards,
Xiaoguang Wang

>
>> Initially I use remap_pfn_range or vm_insert_pages to map sgl pages to
>> user space, but both of them have limits:
>> 1)  Use vm_insert_pages
>> which is like tcp getsockopt(TCP_ZEROCOPY_RECEIVE), but there're two
>> restrictions:
>>    1. anonymous pages can not be mmaped to user spacea.
>>      ==> vm_insert_pages
>>      ====> insert_pages
>>      ======> insert_page_in_batch_locked
>>      ========> validate_page_before_insert
>>      In validate_page_before_insert(), it shows that anonymous page 
>> can not
>>      be mapped to use space, we know that if issuing direct io to block
>>      device, io request's sgl pages mostly comes from anonymous page.
>>          if (PageAnon(page) || PageSlab(page) || page_has_type(page))
>>              return -EINVAL;
>>      I'm not sure why there is such restriction? for safety reasons ?
>>
>>    2. warn_on triggered in __folio_mark_dirty
>>      When calling zap_page_range in tcmu user space backstore when io
>>      completes, there is a warn_on triggered in __folio_mark_dirty:
>>         if (folio->mapping) {   /* Race with truncate? */
>>             WARN_ON_ONCE(warn && !folio_test_uptodate(folio));
>>
>>      I'm not familiar with folio yet, but I think the reason is that 
>> when
>>      issuing a buffered read to tcmu block device, it's page cache 
>> mapped
>>      to user space, backstore write this page and pte will be 
>> dirtied. but
>>      initially it's newly allocated, hence page_update flag not set.
>>      In zap_pte_range(), there is such codes:
>>         if (!PageAnon(page)) {
>>             if (pte_dirty(ptent)) {
>>                 force_flush = 1;
>>                 set_page_dirty(page);
>>             }
>>     So this warn_on is reasonable.
>>     Indeed what I want is just to map io request sgl pages to tcmu user
>>     space backstore, then backstore can read or write data to mapped 
>> area,
>>     I don't want to care about page or its mapping status, so I 
>> choose to
>>     use remap_pfn_range.
>>
>> 2) Use remap_pfn_range()
>>    remap_pfn_range works well, but it has somewhat obvious overhead. 
>> For a
>>    512kb io request, it has 128 pages, and usually this 128 page's 
>> pfn are
>>    not consecutive, so in worst cases, for a 512kb io request, I'd 
>> need to
>>    issue 128 calls to remap_pfn_range, it's horrible. And in 
>> remap_pfn_range,
>>    if x86 page attribute table feature is enabled, lookup_memtype 
>> called by
>>    track_pfn_remap() also introduces obvious overhead.
>>
>> Finally in order to solve these problems, Xu Yu helps to implment a new
>> helper, which accepts an array of pages as parameter, anonymous pages 
>> can
>> be mapped to user space, pages would be treated as special 
>> pte(pte_special
>> returns true), so vm_normal_page returns NULL, above folio warn_on won't
>> trigger.
>>
>> Thanks.
>>
>> Xiaoguang Wang (2):
>>    mm: export zap_page_range()
>>    scsi: target: tcmu: Support zero copy
>>
>> Xu Yu (1):
>>    mm/memory.c: introduce vm_insert_page(s)_mkspecial
>>
>>   drivers/target/target_core_user.c | 257 
>> +++++++++++++++++++++++++++++++++-----
>>   include/linux/mm.h                |   2 +
>>   mm/memory.c                       | 183 +++++++++++++++++++++++++++
>>   3 files changed, 414 insertions(+), 28 deletions(-)
>>


^ permalink raw reply	[flat|nested] 19+ messages in thread

* Re: [RFC 3/3] scsi: target: tcmu: Support zero copy
  2022-03-18  9:55 ` [RFC 3/3] scsi: target: tcmu: Support zero copy Xiaoguang Wang
@ 2022-03-22 14:01   ` Bodo Stroesser
  2022-03-23 14:33     ` Xiaoguang Wang
  0 siblings, 1 reply; 19+ messages in thread
From: Bodo Stroesser @ 2022-03-22 14:01 UTC (permalink / raw)
  To: Xiaoguang Wang, linux-mm, target-devel, linux-scsi; +Cc: linux-block, xuyu

On 18.03.22 10:55, Xiaoguang Wang wrote:
> Currently in tcmu, for READ commands, it copies user space backstore's
> data buffer to tcmu internal data area, then copies data in data area
> to READ commands sgl pages. For WRITE commands, tcmu copies sgl pages
> to tcmu internal data area, then copies data in data area to user space
> backstore. For both cases, there are obvious copy overhead, which impact
> io throughput, especially for large io size.
> 
> To mitigate this issue, we implement zero copy feature to tcmu, which
> map sgl pages to user space backstore's address space. Currently only
> sgl pages's offset and length are both aligned to page size, can this
> command go into tcmu zero copy path.

Apart from my further comments below: since this change implies
incompatibilities regarding timeout handling and KEEP_BUF, I think the
new feature should be configurable with in default mode being switched
off.
When userspace switches it on, I think
- tcmu should fix ring timeout to 0 (= off). No change allowed
- tcmu should either disallow KEEP_BUF at all (not set the feature bit
   in mailbox) or should use a per cmd bit in the ring to show, for which
   cmds KEEP_BUF is possible.

> 
> Signed-off-by: Xiaoguang Wang <xiaoguang.wang@linux.alibaba.com>
> ---
>   drivers/target/target_core_user.c | 257 +++++++++++++++++++++++++++++++++-----
>   1 file changed, 229 insertions(+), 28 deletions(-)
> 
> diff --git a/drivers/target/target_core_user.c b/drivers/target/target_core_user.c
> index 7b2a89a67cdb..4314e0c00f8e 100644
> --- a/drivers/target/target_core_user.c
> +++ b/drivers/target/target_core_user.c
> @@ -16,6 +16,8 @@
>   #include <linux/xarray.h>
>   #include <linux/stringify.h>
>   #include <linux/bitops.h>
> +#include <linux/sched/mm.h>
> +#include <linux/mm.h>
>   #include <linux/highmem.h>
>   #include <linux/configfs.h>
>   #include <linux/mutex.h>
> @@ -72,6 +74,7 @@
>    */
>   #define DATA_PAGES_PER_BLK_DEF 1
>   #define DATA_AREA_PAGES_DEF (256 * 1024)
> +#define ZC_DATA_AREA_PAGES_DEF (256 * 1024)
>   
>   #define TCMU_MBS_TO_PAGES(_mbs) ((size_t)_mbs << (20 - PAGE_SHIFT))
>   #define TCMU_PAGES_TO_MBS(_pages) (_pages >> (20 - PAGE_SHIFT))
> @@ -145,9 +148,20 @@ struct tcmu_dev {
>   	struct list_head qfull_queue;
>   	struct list_head tmr_queue;
>   
> +	/* For zero copy handle */
> +	int zc_data_area_mb;
> +	uint32_t zc_max_blocks;
>   	uint32_t dbi_max;
>   	uint32_t dbi_thresh;
>   	unsigned long *data_bitmap;
> +
> +	struct mm_struct *vma_vm_mm;
> +	struct vm_area_struct *vma;
> +
> +	uint32_t zc_dbi_max;
> +	uint32_t zc_dbi_thresh;
> +	unsigned long *zc_data_bitmap;
> +
>   	struct xarray data_pages;
>   	uint32_t data_pages_per_blk;
>   	uint32_t data_blk_size;
> @@ -177,6 +191,12 @@ struct tcmu_cmd {
>   	struct tcmu_dev *tcmu_dev;
>   	struct list_head queue_entry;
>   
> +	/* For zero_copy handle */
> +	struct mm_struct *vma_vm_mm;
> +	struct vm_area_struct *vma;
> +	struct iovec *iov;
> +	int iov_cnt;
> +

I think, you don't need iov and iov_cnt. You can easily use the list of
dbis instead. Also, using the iovs from the ring during command
completion will not work, since the iovs might already be overwritten by
userspace.

>   	uint16_t cmd_id;
>   
>   	/* Can't use se_cmd when cleaning up expired cmds, because if
> @@ -192,6 +212,7 @@ struct tcmu_cmd {
>   
>   #define TCMU_CMD_BIT_EXPIRED 0
>   #define TCMU_CMD_BIT_KEEP_BUF 1
> +#define TCMU_CMD_BIT_ZEROCOPY 2
>   	unsigned long flags;
>   };
>   
> @@ -495,10 +516,16 @@ static struct genl_family tcmu_genl_family __ro_after_init = {
>   static void tcmu_cmd_free_data(struct tcmu_cmd *tcmu_cmd, uint32_t len)
>   {
>   	struct tcmu_dev *udev = tcmu_cmd->tcmu_dev;
> +	unsigned long *data_bitmap;
>   	uint32_t i;
>   
> +	if (test_bit(TCMU_CMD_BIT_ZEROCOPY, &tcmu_cmd->flags))
> +		data_bitmap = udev->zc_data_bitmap;
> +	else
> +		data_bitmap = udev->data_bitmap;
> +
>   	for (i = 0; i < len; i++)
> -		clear_bit(tcmu_cmd->dbi[i], udev->data_bitmap);
> +		clear_bit(tcmu_cmd->dbi[i], data_bitmap);
>   }
>   
>   static inline int tcmu_get_empty_block(struct tcmu_dev *udev,
> @@ -549,8 +576,30 @@ static inline int tcmu_get_empty_block(struct tcmu_dev *udev,
>   	return i == page_cnt ? dbi : -1;
>   }
>   
> +static inline int tcmu_get_zc_empty_block(struct tcmu_dev *udev,
> +			struct tcmu_cmd *tcmu_cmd, int prev_dbi,
> +			int *iov_cnt)
> +{
> +	int dbi;
> +
> +	dbi = find_first_zero_bit(udev->zc_data_bitmap, udev->zc_dbi_thresh);
> +	if (dbi == udev->zc_dbi_thresh)
> +		return -1;
> +
> +	if (dbi > udev->zc_dbi_max)
> +		udev->zc_dbi_max = dbi;
> +
> +	set_bit(dbi, udev->zc_data_bitmap);
> +	tcmu_cmd_set_dbi(tcmu_cmd, dbi);
> +
> +	if (dbi != prev_dbi + 1)
> +		*iov_cnt += 1;
> +	return dbi;
> +}
> +
>   static int tcmu_get_empty_blocks(struct tcmu_dev *udev,
> -				 struct tcmu_cmd *tcmu_cmd, int length)
> +				 struct tcmu_cmd *tcmu_cmd, int length,
> +				 bool zero_copy)
>   {
>   	/* start value of dbi + 1 must not be a valid dbi */
>   	int dbi = -2;
> @@ -559,16 +608,111 @@ static int tcmu_get_empty_blocks(struct tcmu_dev *udev,
>   
>   	for (; length > 0; length -= blk_size) {
>   		blk_data_len = min_t(uint32_t, length, blk_size);
> -		dbi = tcmu_get_empty_block(udev, tcmu_cmd, dbi, blk_data_len,
> -					   &iov_cnt);
> +		if (zero_copy) {
> +			dbi = tcmu_get_zc_empty_block(udev, tcmu_cmd, dbi,
> +						      &iov_cnt);
> +		} else {
> +			dbi = tcmu_get_empty_block(udev, tcmu_cmd, dbi,
> +					blk_data_len, &iov_cnt);
> +		}
>   		if (dbi < 0)
>   			return -1;
>   	}
>   	return iov_cnt;
>   }
>   
> +#define TCMU_ZEROCOPY_PAGE_BATCH 32
> +
> +static inline void tcmu_zerocopy_one_seg(struct iovec *iov,
> +			struct vm_area_struct *vma,
> +			struct sg_page_iter *sgiter)
> +{
> +	struct page *pages[TCMU_ZEROCOPY_PAGE_BATCH];
> +	unsigned int len = iov->iov_len;
> +	unsigned long address = (unsigned long)iov->iov_base + vma->vm_start;
> +	unsigned long pages_remaining, pg_index = 0;
> +	struct page *page;
> +
> +	while (len > 0) {
> +		__sg_page_iter_next(sgiter);
> +		page = sg_page_iter_page(sgiter);
> +		pages[pg_index++] = page;
> +		len -= PAGE_SIZE;
> +		if (pg_index == TCMU_ZEROCOPY_PAGE_BATCH || !len) {
> +			pages_remaining = pg_index;
> +			vm_insert_pages_mkspecial(vma, address, pages, &pages_remaining);
> +			address = address + pg_index * PAGE_SIZE;
> +			pg_index = 0;
> +		}
> +	}
> +}
> +
> +static long tcmu_cmd_zerocopy_map(struct tcmu_dev *udev,
> +				  struct tcmu_cmd *cmd,
> +				  struct iovec *iov,
> +				  int iov_cnt)
> +{
> +	struct se_cmd *se_cmd = cmd->se_cmd;
> +	struct scatterlist *data_sg;
> +	unsigned int data_nents;
> +	struct iovec *tiov;
> +	struct sg_page_iter sgiter;
> +	struct vm_area_struct *vma = udev->vma;
> +	int i, ret = 0;

What happens if the tcmu device currently is not open / mapped?

> +
> +	mmap_read_lock(udev->vma_vm_mm);
> +	data_sg = se_cmd->t_data_sg;
> +	data_nents = se_cmd->t_data_nents;
> +	__sg_page_iter_start(&sgiter, data_sg, data_nents, 0);
> +	tiov = iov;
> +	for (i = 0; i < iov_cnt; i++) {
> +		tcmu_zerocopy_one_seg(tiov, vma, &sgiter);
> +		tiov++;
> +	}
> +	cmd->iov = iov;
> +	cmd->iov_cnt = iov_cnt;

This won't work!

The iov's can already be overwritten when you later want to read
them. So you would have to take a real copy of the iovs.

> +	cmd->vma_vm_mm = vma->vm_mm;
> +	cmd->vma = vma;
> +	mmgrab(cmd->vma_vm_mm);
> +	mmap_read_unlock(udev->vma_vm_mm);
> +	return ret;
> +}
> +
> +static void tcmu_cmd_zerocopy_unmap(struct tcmu_cmd *cmd)
> +{
> +	struct mm_struct *mm;
> +	struct vm_area_struct *vma;
> +	struct iovec *iov;
> +	unsigned long address;
> +	int i;
> +
> +	mm = cmd->vma_vm_mm;
> +	if (!mm)
> +		return;
> +
> +	vma = cmd->vma;
> +	iov = cmd->iov;
> +	if (mmget_not_zero(mm)) {
> +		mmap_read_lock(mm);
> +		for (i = 0; i < cmd->iov_cnt; i++) {
> +			address = (unsigned long)iov->iov_base + vma->vm_start;
> +			zap_page_range(vma, address, iov->iov_len);
> +			iov++;
> +		}
> +		mmap_read_unlock(mm);
> +		mmput(mm);
> +	}
> +
> +	cmd->vma_vm_mm = NULL;
> +	cmd->vma = NULL;
> +	mmdrop(mm);
> +}
> +
>   static inline void tcmu_free_cmd(struct tcmu_cmd *tcmu_cmd)
>   {
> +	if (test_bit(TCMU_CMD_BIT_ZEROCOPY, &tcmu_cmd->flags))
> +		tcmu_cmd_zerocopy_unmap(tcmu_cmd);
> +
>   	kfree(tcmu_cmd->dbi);
>   	kmem_cache_free(tcmu_cmd_cache, tcmu_cmd);
>   }
> @@ -850,37 +994,57 @@ static bool is_ring_space_avail(struct tcmu_dev *udev, size_t cmd_size)
>    * Called with ring lock held.
>    */
>   static int tcmu_alloc_data_space(struct tcmu_dev *udev, struct tcmu_cmd *cmd,
> -				  int *iov_bidi_cnt)
> +				  int *iov_bidi_cnt, bool zero_copy)
>   {
>   	int space, iov_cnt = 0, ret = 0;
>   
>   	if (!cmd->dbi_cnt)
>   		goto wr_iov_cnts;
>   
> -	/* try to check and get the data blocks as needed */
> -	space = spc_bitmap_free(udev->data_bitmap, udev->dbi_thresh);
> -	if (space < cmd->dbi_cnt) {
> -		unsigned long blocks_left =
> -				(udev->max_blocks - udev->dbi_thresh) + space;
> +	if (!zero_copy) {
> +		/* try to check and get the data blocks as needed */
> +		space = spc_bitmap_free(udev->data_bitmap, udev->dbi_thresh);
> +		if (space < cmd->dbi_cnt) {
> +			unsigned long blocks_left =
> +					(udev->max_blocks - udev->dbi_thresh) + space;
> +
> +			if (blocks_left < cmd->dbi_cnt) {
> +				pr_debug("no data space: only %lu available, but ask for %u\n",
> +						blocks_left * udev->data_blk_size,
> +						cmd->dbi_cnt * udev->data_blk_size);
> +				return -1;
> +			}
>   
> -		if (blocks_left < cmd->dbi_cnt) {
> -			pr_debug("no data space: only %lu available, but ask for %u\n",
> -					blocks_left * udev->data_blk_size,
> -					cmd->dbi_cnt * udev->data_blk_size);
> -			return -1;
> +			udev->dbi_thresh += cmd->dbi_cnt;
> +			if (udev->dbi_thresh > udev->max_blocks)
> +				udev->dbi_thresh = udev->max_blocks;
>   		}
> +	} else {
> +		/* try to check and get the data blocks as needed */
> +		space = spc_bitmap_free(udev->zc_data_bitmap, udev->zc_dbi_thresh);
> +		if (space < cmd->dbi_cnt) {
> +			unsigned long blocks_left =
> +					(udev->zc_max_blocks - udev->zc_dbi_thresh) + space;
> +
> +			if (blocks_left < cmd->dbi_cnt) {
> +				pr_debug("no data space: only %lu available, but ask for %u\n",
> +						blocks_left * udev->data_blk_size,
> +						cmd->dbi_cnt * udev->data_blk_size);
> +				return -1;
> +			}
>   
> -		udev->dbi_thresh += cmd->dbi_cnt;
> -		if (udev->dbi_thresh > udev->max_blocks)
> -			udev->dbi_thresh = udev->max_blocks;
> +			udev->zc_dbi_thresh += cmd->dbi_cnt;
> +			if (udev->zc_dbi_thresh > udev->zc_max_blocks)
> +				udev->zc_dbi_thresh = udev->zc_max_blocks;
> +		}
>   	}
>   
> -	iov_cnt = tcmu_get_empty_blocks(udev, cmd, cmd->se_cmd->data_length);
> +	iov_cnt = tcmu_get_empty_blocks(udev, cmd, cmd->se_cmd->data_length, zero_copy);
>   	if (iov_cnt < 0)
>   		return -1;
>   
>   	if (cmd->dbi_bidi_cnt) {
> -		ret = tcmu_get_empty_blocks(udev, cmd, cmd->data_len_bidi);
> +		ret = tcmu_get_empty_blocks(udev, cmd, cmd->data_len_bidi, zero_copy);
>   		if (ret < 0)
>   			return -1;
>   	}
> @@ -1021,6 +1185,7 @@ static int queue_cmd_ring(struct tcmu_cmd *tcmu_cmd, sense_reason_t *scsi_err)
>   	uint32_t blk_size = udev->data_blk_size;
>   	/* size of data buffer needed */
>   	size_t data_length = (size_t)tcmu_cmd->dbi_cnt * blk_size;
> +	bool zero_copy = false;
>   
>   	*scsi_err = TCM_NO_SENSE;
>   
> @@ -1044,7 +1209,22 @@ static int queue_cmd_ring(struct tcmu_cmd *tcmu_cmd, sense_reason_t *scsi_err)
>   		return -1;
>   	}
>   
> -	iov_cnt = tcmu_alloc_data_space(udev, tcmu_cmd, &iov_bidi_cnt);
> +	if (!(se_cmd->se_cmd_flags & SCF_BIDI) && se_cmd->data_length &&
> +	    IS_ALIGNED(se_cmd->data_length, PAGE_SIZE)) {

What is the problem with BIDI here? Why do you skip those cmds?

> +		struct scatterlist *data_sg = se_cmd->t_data_sg, *sg;
> +		unsigned int data_nents = se_cmd->t_data_nents;
> +		int i;
> +
> +		for_each_sg(data_sg, sg, data_nents, i) {
> +			if (!((!sg->offset || IS_ALIGNED(sg->offset, PAGE_SIZE)) &&
> +			    IS_ALIGNED(sg->length, PAGE_SIZE)))
> +				break;
> +		}
> +		if (i == data_nents)
> +			zero_copy = true;
> +	}
> +
> +	iov_cnt = tcmu_alloc_data_space(udev, tcmu_cmd, &iov_bidi_cnt, zero_copy);
>   	if (iov_cnt < 0)
>   		goto free_and_queue;
>   
> @@ -1093,7 +1273,7 @@ static int queue_cmd_ring(struct tcmu_cmd *tcmu_cmd, sense_reason_t *scsi_err)
>   	tcmu_cmd_reset_dbi_cur(tcmu_cmd);
>   	iov = &entry->req.iov[0];
>   
> -	if (se_cmd->data_direction == DMA_TO_DEVICE ||
> +	if (((se_cmd->data_direction == DMA_TO_DEVICE) && !zero_copy) ||
>   	    se_cmd->se_cmd_flags & SCF_BIDI)
>   		scatter_data_area(udev, tcmu_cmd, &iov);
>   	else
> @@ -1111,6 +1291,11 @@ static int queue_cmd_ring(struct tcmu_cmd *tcmu_cmd, sense_reason_t *scsi_err)
>   	tcmu_setup_cmd_timer(tcmu_cmd, udev->cmd_time_out, &udev->cmd_timer);
>   
>   	entry->hdr.cmd_id = tcmu_cmd->cmd_id;
> +	if (zero_copy) {
> +		iov = &entry->req.iov[0];
> +		tcmu_cmd_zerocopy_map(udev, tcmu_cmd, iov, entry->req.iov_cnt);

This call maps the kernel pages from sgl to userspace no matter whether
it is a READ or WRITE type of cmd. That means, if it is READ, then you
might map old kernel data to userspace, right?

> +		set_bit(TCMU_CMD_BIT_ZEROCOPY, &tcmu_cmd->flags);
> +	}
>   
>   	tcmu_hdr_set_len(&entry->hdr.len_op, command_size);
>   
> @@ -1366,7 +1551,10 @@ static bool tcmu_handle_completion(struct tcmu_cmd *cmd,
>   		else
>   			se_cmd->se_cmd_flags |= SCF_TREAT_READ_AS_NORMAL;
>   	}
> -	if (se_cmd->se_cmd_flags & SCF_BIDI) {
> +
> +	if (test_bit(TCMU_CMD_BIT_ZEROCOPY, &cmd->flags)) {
> +		tcmu_cmd_zerocopy_unmap(cmd);
> +	} else if (se_cmd->se_cmd_flags & SCF_BIDI) {
>   		/* Get Data-In buffer before clean up */
>   		gather_data_area(udev, cmd, true, read_len);
>   	} else if (se_cmd->data_direction == DMA_FROM_DEVICE) {
> @@ -1520,6 +1708,8 @@ static void tcmu_check_expired_ring_cmd(struct tcmu_cmd *cmd)
>   	if (!time_after_eq(jiffies, cmd->deadline))
>   		return;
>   
> +	if (test_bit(TCMU_CMD_BIT_ZEROCOPY, &cmd->flags))
> +		tcmu_cmd_zerocopy_unmap(cmd);
>   	set_bit(TCMU_CMD_BIT_EXPIRED, &cmd->flags);
>   	list_del_init(&cmd->queue_entry);
>   	se_cmd = cmd->se_cmd;
> @@ -1618,6 +1808,8 @@ static struct se_device *tcmu_alloc_device(struct se_hba *hba, const char *name)
>   	udev->data_pages_per_blk = DATA_PAGES_PER_BLK_DEF;
>   	udev->max_blocks = DATA_AREA_PAGES_DEF / udev->data_pages_per_blk;
>   	udev->data_area_mb = TCMU_PAGES_TO_MBS(DATA_AREA_PAGES_DEF);
> +	udev->zc_max_blocks = ZC_DATA_AREA_PAGES_DEF / udev->data_pages_per_blk;
> +	udev->zc_data_area_mb = TCMU_PAGES_TO_MBS(ZC_DATA_AREA_PAGES_DEF);
>   
>   	mutex_init(&udev->cmdr_lock);
>   
> @@ -1841,6 +2033,9 @@ static void tcmu_vma_open(struct vm_area_struct *vma)
>   
>   	pr_debug("vma_open\n");
>   
> +	udev->vma_vm_mm = vma->vm_mm;
> +	udev->vma = vma;
> +	mmgrab(udev->vma_vm_mm);

What about the mmap being opened multiple times?
Up to now tcmu allows one open() only. But after open() userspace might
fork with parent and child calling mmap() independently.
AFAIK, tcmu_vma_open will then be called twice, but I don't know whether
it is the same vma in both calls.

>   	kref_get(&udev->kref);
>   }
>   
> @@ -1850,6 +2045,10 @@ static void tcmu_vma_close(struct vm_area_struct *vma)
>   
>   	pr_debug("vma_close\n");
>   
> +	mmdrop(udev->vma_vm_mm);
> +	udev->vma_vm_mm = NULL;
> +	udev->vma = NULL;
> +
>   	/* release ref from tcmu_vma_open */
>   	kref_put(&udev->kref, tcmu_dev_kref_release);
>   }
> @@ -1901,7 +2100,7 @@ static int tcmu_mmap(struct uio_info *info, struct vm_area_struct *vma)
>   {
>   	struct tcmu_dev *udev = container_of(info, struct tcmu_dev, uio_info);
>   
> -	vma->vm_flags |= VM_DONTEXPAND | VM_DONTDUMP;
> +	vma->vm_flags |= VM_DONTEXPAND | VM_DONTDUMP | VM_MIXEDMAP;
>   	vma->vm_ops = &tcmu_vm_ops;
>   
>   	vma->vm_private_data = udev;
> @@ -2172,7 +2371,7 @@ static int tcmu_configure_device(struct se_device *dev)
>   	struct tcmu_dev *udev = TCMU_DEV(dev);
>   	struct uio_info *info;
>   	struct tcmu_mailbox *mb;
> -	size_t data_size;
> +	size_t data_size, zc_data_size;
>   	int ret = 0;
>   
>   	ret = tcmu_update_uio_info(udev);
> @@ -2183,8 +2382,9 @@ static int tcmu_configure_device(struct se_device *dev)
>   
>   	mutex_lock(&udev->cmdr_lock);
>   	udev->data_bitmap = bitmap_zalloc(udev->max_blocks, GFP_KERNEL);
> +	udev->zc_data_bitmap = bitmap_zalloc(udev->zc_max_blocks, GFP_KERNEL);
>   	mutex_unlock(&udev->cmdr_lock);
> -	if (!udev->data_bitmap) {
> +	if (!udev->data_bitmap || !udev->zc_data_bitmap) {
>   		ret = -ENOMEM;
>   		goto err_bitmap_alloc;
>   	}
> @@ -2201,7 +2401,8 @@ static int tcmu_configure_device(struct se_device *dev)
>   	udev->cmdr_size = CMDR_SIZE;
>   	udev->data_off = MB_CMDR_SIZE;
>   	data_size = TCMU_MBS_TO_PAGES(udev->data_area_mb) << PAGE_SHIFT;
> -	udev->mmap_pages = (data_size + MB_CMDR_SIZE) >> PAGE_SHIFT;
> +	zc_data_size = TCMU_MBS_TO_PAGES(udev->zc_data_area_mb) << PAGE_SHIFT;
> +	udev->mmap_pages = (data_size + zc_data_size + MB_CMDR_SIZE) >> PAGE_SHIFT;
>   	udev->data_blk_size = udev->data_pages_per_blk * PAGE_SIZE;
>   	udev->dbi_thresh = 0; /* Default in Idle state */
>   
> @@ -2221,7 +2422,7 @@ static int tcmu_configure_device(struct se_device *dev)
>   
>   	info->mem[0].name = "tcm-user command & data buffer";
>   	info->mem[0].addr = (phys_addr_t)(uintptr_t)udev->mb_addr;
> -	info->mem[0].size = data_size + MB_CMDR_SIZE;
> +	info->mem[0].size = data_size + zc_data_size + MB_CMDR_SIZE;
>   	info->mem[0].memtype = UIO_MEM_NONE;
>   
>   	info->irqcontrol = tcmu_irqcontrol;

^ permalink raw reply	[flat|nested] 19+ messages in thread

* Re: [RFC 0/3] Add zero copy feature for tcmu
  2022-03-22 13:17   ` Xiaoguang Wang
@ 2022-03-22 14:05     ` Bodo Stroesser
  0 siblings, 0 replies; 19+ messages in thread
From: Bodo Stroesser @ 2022-03-22 14:05 UTC (permalink / raw)
  To: Xiaoguang Wang, linux-mm, target-devel, linux-scsi; +Cc: linux-block, xuyu

On 22.03.22 14:17, Xiaoguang Wang wrote:
> hi,
> 
>> On 18.03.22 10:55, Xiaoguang Wang wrote:
>>> The core idea to implement tcmu zero copy feature is really straight,
>>> which just maps block device io request's sgl pages to tcmu user space
>>> backstore, then we can avoid extra copy overhead between sgl pages and
>>> tcmu internal data area(which really impacts io throughput), please see
>>> https://www.spinics.net/lists/target-devel/msg21121.html for detailed
>>> info.
>>>
>>
>> Can you please tell us, how big the performance improvement is and
>> which configuration you are using for measurenments?
> Sorry, I should have attached test results here. Initially I tried to use
> tcmu user:fbo backstore to evaluate performance improvements, but
> it only shows about 10%~15% io throughput improvement. Fio config
> is numjobs=1, iodepth=8, bs=256k, which isn't very impressive. The
> reason is that user:fbo backstore does buffered reads, it consumes most
> of cpu.
> 
> Then I test this zero copy feature for our real workload, whose backstore
> is a network program visiting distributed file system and it's 
> multi-threaded.
> For 4 job, 8 depth, 256 kb io size, the write throughput improves from
> 3.6GB/s to 10GB/s.

Thank you for the info. Sounds promising.

What fabric are you using? iSCSI?
What HW is your target running on?

> 
> Regards,
> Xiaoguang Wang
> 
>>
>>> Initially I use remap_pfn_range or vm_insert_pages to map sgl pages to
>>> user space, but both of them have limits:
>>> 1)  Use vm_insert_pages
>>> which is like tcp getsockopt(TCP_ZEROCOPY_RECEIVE), but there're two
>>> restrictions:
>>>    1. anonymous pages can not be mmaped to user spacea.
>>>      ==> vm_insert_pages
>>>      ====> insert_pages
>>>      ======> insert_page_in_batch_locked
>>>      ========> validate_page_before_insert
>>>      In validate_page_before_insert(), it shows that anonymous page 
>>> can not
>>>      be mapped to use space, we know that if issuing direct io to block
>>>      device, io request's sgl pages mostly comes from anonymous page.
>>>          if (PageAnon(page) || PageSlab(page) || page_has_type(page))
>>>              return -EINVAL;
>>>      I'm not sure why there is such restriction? for safety reasons ?
>>>
>>>    2. warn_on triggered in __folio_mark_dirty
>>>      When calling zap_page_range in tcmu user space backstore when io
>>>      completes, there is a warn_on triggered in __folio_mark_dirty:
>>>         if (folio->mapping) {   /* Race with truncate? */
>>>             WARN_ON_ONCE(warn && !folio_test_uptodate(folio));
>>>
>>>      I'm not familiar with folio yet, but I think the reason is that 
>>> when
>>>      issuing a buffered read to tcmu block device, it's page cache 
>>> mapped
>>>      to user space, backstore write this page and pte will be 
>>> dirtied. but
>>>      initially it's newly allocated, hence page_update flag not set.
>>>      In zap_pte_range(), there is such codes:
>>>         if (!PageAnon(page)) {
>>>             if (pte_dirty(ptent)) {
>>>                 force_flush = 1;
>>>                 set_page_dirty(page);
>>>             }
>>>     So this warn_on is reasonable.
>>>     Indeed what I want is just to map io request sgl pages to tcmu user
>>>     space backstore, then backstore can read or write data to mapped 
>>> area,
>>>     I don't want to care about page or its mapping status, so I 
>>> choose to
>>>     use remap_pfn_range.
>>>
>>> 2) Use remap_pfn_range()
>>>    remap_pfn_range works well, but it has somewhat obvious overhead. 
>>> For a
>>>    512kb io request, it has 128 pages, and usually this 128 page's 
>>> pfn are
>>>    not consecutive, so in worst cases, for a 512kb io request, I'd 
>>> need to
>>>    issue 128 calls to remap_pfn_range, it's horrible. And in 
>>> remap_pfn_range,
>>>    if x86 page attribute table feature is enabled, lookup_memtype 
>>> called by
>>>    track_pfn_remap() also introduces obvious overhead.
>>>
>>> Finally in order to solve these problems, Xu Yu helps to implment a new
>>> helper, which accepts an array of pages as parameter, anonymous pages 
>>> can
>>> be mapped to user space, pages would be treated as special 
>>> pte(pte_special
>>> returns true), so vm_normal_page returns NULL, above folio warn_on won't
>>> trigger.
>>>
>>> Thanks.
>>>
>>> Xiaoguang Wang (2):
>>>    mm: export zap_page_range()
>>>    scsi: target: tcmu: Support zero copy
>>>
>>> Xu Yu (1):
>>>    mm/memory.c: introduce vm_insert_page(s)_mkspecial
>>>
>>>   drivers/target/target_core_user.c | 257 
>>> +++++++++++++++++++++++++++++++++-----
>>>   include/linux/mm.h                |   2 +
>>>   mm/memory.c                       | 183 +++++++++++++++++++++++++++
>>>   3 files changed, 414 insertions(+), 28 deletions(-)
>>>
> 

^ permalink raw reply	[flat|nested] 19+ messages in thread

* Re: [RFC 2/3] mm: export zap_page_range()
  2022-03-22 13:08       ` David Hildenbrand
@ 2022-03-23 13:59         ` Xiaoguang Wang
  2022-03-23 16:48         ` Christoph Hellwig
  1 sibling, 0 replies; 19+ messages in thread
From: Xiaoguang Wang @ 2022-03-23 13:59 UTC (permalink / raw)
  To: David Hildenbrand, linux-mm, target-devel, linux-scsi
  Cc: linux-block, xuyu, bostroesser

hi,

> On 22.03.22 14:02, Xiaoguang Wang wrote:
>> hi,
>>
>>> On 18.03.22 10:55, Xiaoguang Wang wrote:
>>>> Module target_core_user will use it to implement zero copy feature.
>>>>
>>>> Signed-off-by: Xiaoguang Wang <xiaoguang.wang@linux.alibaba.com>
>>>> ---
>>>>    mm/memory.c | 1 +
>>>>    1 file changed, 1 insertion(+)
>>>>
>>>> diff --git a/mm/memory.c b/mm/memory.c
>>>> index 1f745e4d11c2..9974d0406dad 100644
>>>> --- a/mm/memory.c
>>>> +++ b/mm/memory.c
>>>> @@ -1664,6 +1664,7 @@ void zap_page_range(struct vm_area_struct *vma, unsigned long start,
>>>>    	mmu_notifier_invalidate_range_end(&range);
>>>>    	tlb_finish_mmu(&tlb);
>>>>    }
>>>> +EXPORT_SYMBOL_GPL(zap_page_range);
>>>>    
>>>>    /**
>>>>     * zap_page_range_single - remove user pages in a given range
>>> To which VMAs will you be applying zap_page_range? I assume only to some
>>> special ones where you previously vm_insert_page(s)_mkspecial'ed pages,
>>> not to some otherwise random VMAs, correct?
>> Yes, you're right :)
> I'd suggest exposing a dedicated function that performs sanity checks on
> the vma (VM_PFNMAP ?) and only zaps within a single VMA.
>
> Essentially zap_page_range_single(), excluding "struct zap_details
> *details" and including sanity checks.
>
> Reason is that we don't want anybody to blindly zap_page_range() within
> random VMAs from a kernel module.
OK, I see, thanks. Xu Yu and I will try to implement a new helper in 
following new version patches.

Regards,
Xiaoguang Wang
>


^ permalink raw reply	[flat|nested] 19+ messages in thread

* Re: [RFC 3/3] scsi: target: tcmu: Support zero copy
  2022-03-22 14:01   ` Bodo Stroesser
@ 2022-03-23 14:33     ` Xiaoguang Wang
  2022-03-25  9:06       ` Bodo Stroesser
  0 siblings, 1 reply; 19+ messages in thread
From: Xiaoguang Wang @ 2022-03-23 14:33 UTC (permalink / raw)
  To: Bodo Stroesser, linux-mm, target-devel, linux-scsi; +Cc: linux-block, xuyu

hi,

First thanks for your kind comments, it really will make codes better.
> On 18.03.22 10:55, Xiaoguang Wang wrote:
>> Currently in tcmu, for READ commands, it copies user space backstore's
>> data buffer to tcmu internal data area, then copies data in data area
>> to READ commands sgl pages. For WRITE commands, tcmu copies sgl pages
>> to tcmu internal data area, then copies data in data area to user space
>> backstore. For both cases, there are obvious copy overhead, which impact
>> io throughput, especially for large io size.
>>
>> To mitigate this issue, we implement zero copy feature to tcmu, which
>> map sgl pages to user space backstore's address space. Currently only
>> sgl pages's offset and length are both aligned to page size, can this
>> command go into tcmu zero copy path.
>
> Apart from my further comments below: since this change implies
> incompatibilities regarding timeout handling and KEEP_BUF, I think the
> new feature should be configurable with in default mode being switched
> off.
OK, agree.

> When userspace switches it on, I think
> - tcmu should fix ring timeout to 0 (= off). No change allowed
> - tcmu should either disallow KEEP_BUF at all (not set the feature bit
>   in mailbox) or should use a per cmd bit in the ring to show, for which
>   cmds KEEP_BUF is possible.
Sorry, since this is a RFC patch set, so I didn't consider this 
incompatibilities
well, I'll try to figure out a better method in following version.

>
>>
>> Signed-off-by: Xiaoguang Wang <xiaoguang.wang@linux.alibaba.com>
>> ---
>>   drivers/target/target_core_user.c | 257 
>> +++++++++++++++++++++++++++++++++-----
>>   1 file changed, 229 insertions(+), 28 deletions(-)
>>
>> diff --git a/drivers/target/target_core_user.c 
>> b/drivers/target/target_core_user.c
>> index 7b2a89a67cdb..4314e0c00f8e 100644
>> --- a/drivers/target/target_core_user.c
>> +++ b/drivers/target/target_core_user.c
>> @@ -16,6 +16,8 @@
>>   #include <linux/xarray.h>
>>   #include <linux/stringify.h>
>>   #include <linux/bitops.h>
>> +#include <linux/sched/mm.h>
>> +#include <linux/mm.h>
>>   #include <linux/highmem.h>
>>   #include <linux/configfs.h>
>>   #include <linux/mutex.h>
>> @@ -72,6 +74,7 @@
>>    */
>>   #define DATA_PAGES_PER_BLK_DEF 1
>>   #define DATA_AREA_PAGES_DEF (256 * 1024)
>> +#define ZC_DATA_AREA_PAGES_DEF (256 * 1024)
>>     #define TCMU_MBS_TO_PAGES(_mbs) ((size_t)_mbs << (20 - PAGE_SHIFT))
>>   #define TCMU_PAGES_TO_MBS(_pages) (_pages >> (20 - PAGE_SHIFT))
>> @@ -145,9 +148,20 @@ struct tcmu_dev {
>>       struct list_head qfull_queue;
>>       struct list_head tmr_queue;
>>   +    /* For zero copy handle */
>> +    int zc_data_area_mb;
>> +    uint32_t zc_max_blocks;
>>       uint32_t dbi_max;
>>       uint32_t dbi_thresh;
>>       unsigned long *data_bitmap;
>> +
>> +    struct mm_struct *vma_vm_mm;
>> +    struct vm_area_struct *vma;
>> +
>> +    uint32_t zc_dbi_max;
>> +    uint32_t zc_dbi_thresh;
>> +    unsigned long *zc_data_bitmap;
>> +
>>       struct xarray data_pages;
>>       uint32_t data_pages_per_blk;
>>       uint32_t data_blk_size;
>> @@ -177,6 +191,12 @@ struct tcmu_cmd {
>>       struct tcmu_dev *tcmu_dev;
>>       struct list_head queue_entry;
>>   +    /* For zero_copy handle */
>> +    struct mm_struct *vma_vm_mm;
>> +    struct vm_area_struct *vma;
>> +    struct iovec *iov;
>> +    int iov_cnt;
>> +
>
> I think, you don't need iov and iov_cnt. You can easily use the list of
> dbis instead. Also, using the iovs from the ring during command
> completion will not work, since the iovs might already be overwritten by
> userspace.
OK, will fix it.

>
>>       uint16_t cmd_id;
>>         /* Can't use se_cmd when cleaning up expired cmds, because if
>> @@ -192,6 +212,7 @@ struct tcmu_cmd {
>>     #define TCMU_CMD_BIT_EXPIRED 0
>>   #define TCMU_CMD_BIT_KEEP_BUF 1
>> +#define TCMU_CMD_BIT_ZEROCOPY 2
>>       unsigned long flags;
>>   };
>>   @@ -495,10 +516,16 @@ static struct genl_family tcmu_genl_family 
>> __ro_after_init = {
>>   static void tcmu_cmd_free_data(struct tcmu_cmd *tcmu_cmd, uint32_t 
>> len)
>>   {
>>       struct tcmu_dev *udev = tcmu_cmd->tcmu_dev;
>> +    unsigned long *data_bitmap;
>>       uint32_t i;
>>   +    if (test_bit(TCMU_CMD_BIT_ZEROCOPY, &tcmu_cmd->flags))
>> +        data_bitmap = udev->zc_data_bitmap;
>> +    else
>> +        data_bitmap = udev->data_bitmap;
>> +
>>       for (i = 0; i < len; i++)
>> -        clear_bit(tcmu_cmd->dbi[i], udev->data_bitmap);
>> +        clear_bit(tcmu_cmd->dbi[i], data_bitmap);
>>   }
>>     static inline int tcmu_get_empty_block(struct tcmu_dev *udev,
>> @@ -549,8 +576,30 @@ static inline int tcmu_get_empty_block(struct 
>> tcmu_dev *udev,
>>       return i == page_cnt ? dbi : -1;
>>   }
>>   +static inline int tcmu_get_zc_empty_block(struct tcmu_dev *udev,
>> +            struct tcmu_cmd *tcmu_cmd, int prev_dbi,
>> +            int *iov_cnt)
>> +{
>> +    int dbi;
>> +
>> +    dbi = find_first_zero_bit(udev->zc_data_bitmap, 
>> udev->zc_dbi_thresh);
>> +    if (dbi == udev->zc_dbi_thresh)
>> +        return -1;
>> +
>> +    if (dbi > udev->zc_dbi_max)
>> +        udev->zc_dbi_max = dbi;
>> +
>> +    set_bit(dbi, udev->zc_data_bitmap);
>> +    tcmu_cmd_set_dbi(tcmu_cmd, dbi);
>> +
>> +    if (dbi != prev_dbi + 1)
>> +        *iov_cnt += 1;
>> +    return dbi;
>> +}
>> +
>>   static int tcmu_get_empty_blocks(struct tcmu_dev *udev,
>> -                 struct tcmu_cmd *tcmu_cmd, int length)
>> +                 struct tcmu_cmd *tcmu_cmd, int length,
>> +                 bool zero_copy)
>>   {
>>       /* start value of dbi + 1 must not be a valid dbi */
>>       int dbi = -2;
>> @@ -559,16 +608,111 @@ static int tcmu_get_empty_blocks(struct 
>> tcmu_dev *udev,
>>         for (; length > 0; length -= blk_size) {
>>           blk_data_len = min_t(uint32_t, length, blk_size);
>> -        dbi = tcmu_get_empty_block(udev, tcmu_cmd, dbi, blk_data_len,
>> -                       &iov_cnt);
>> +        if (zero_copy) {
>> +            dbi = tcmu_get_zc_empty_block(udev, tcmu_cmd, dbi,
>> +                              &iov_cnt);
>> +        } else {
>> +            dbi = tcmu_get_empty_block(udev, tcmu_cmd, dbi,
>> +                    blk_data_len, &iov_cnt);
>> +        }
>>           if (dbi < 0)
>>               return -1;
>>       }
>>       return iov_cnt;
>>   }
>>   +#define TCMU_ZEROCOPY_PAGE_BATCH 32
>> +
>> +static inline void tcmu_zerocopy_one_seg(struct iovec *iov,
>> +            struct vm_area_struct *vma,
>> +            struct sg_page_iter *sgiter)
>> +{
>> +    struct page *pages[TCMU_ZEROCOPY_PAGE_BATCH];
>> +    unsigned int len = iov->iov_len;
>> +    unsigned long address = (unsigned long)iov->iov_base + 
>> vma->vm_start;
>> +    unsigned long pages_remaining, pg_index = 0;
>> +    struct page *page;
>> +
>> +    while (len > 0) {
>> +        __sg_page_iter_next(sgiter);
>> +        page = sg_page_iter_page(sgiter);
>> +        pages[pg_index++] = page;
>> +        len -= PAGE_SIZE;
>> +        if (pg_index == TCMU_ZEROCOPY_PAGE_BATCH || !len) {
>> +            pages_remaining = pg_index;
>> +            vm_insert_pages_mkspecial(vma, address, pages, 
>> &pages_remaining);
>> +            address = address + pg_index * PAGE_SIZE;
>> +            pg_index = 0;
>> +        }
>> +    }
>> +}
>> +
>> +static long tcmu_cmd_zerocopy_map(struct tcmu_dev *udev,
>> +                  struct tcmu_cmd *cmd,
>> +                  struct iovec *iov,
>> +                  int iov_cnt)
>> +{
>> +    struct se_cmd *se_cmd = cmd->se_cmd;
>> +    struct scatterlist *data_sg;
>> +    unsigned int data_nents;
>> +    struct iovec *tiov;
>> +    struct sg_page_iter sgiter;
>> +    struct vm_area_struct *vma = udev->vma;
>> +    int i, ret = 0;
>
> What happens if the tcmu device currently is not open / mapped?
I'm not sure how it will happen.
But we may check whether udev->vma has a valid value. If yes,
it'll enter tcmu_cmd_zerocopy_map().

>
>> +
>> +    mmap_read_lock(udev->vma_vm_mm);
>> +    data_sg = se_cmd->t_data_sg;
>> +    data_nents = se_cmd->t_data_nents;
>> +    __sg_page_iter_start(&sgiter, data_sg, data_nents, 0);
>> +    tiov = iov;
>> +    for (i = 0; i < iov_cnt; i++) {
>> +        tcmu_zerocopy_one_seg(tiov, vma, &sgiter);
>> +        tiov++;
>> +    }
>> +    cmd->iov = iov;
>> +    cmd->iov_cnt = iov_cnt;
>
> This won't work!
>
> The iov's can already be overwritten when you later want to read
> them. So you would have to take a real copy of the iovs.
OK, will fix it.

>
>> +    cmd->vma_vm_mm = vma->vm_mm;
>> +    cmd->vma = vma;
>> +    mmgrab(cmd->vma_vm_mm);
>> +    mmap_read_unlock(udev->vma_vm_mm);
>> +    return ret;
>> +}
>> +
>> +static void tcmu_cmd_zerocopy_unmap(struct tcmu_cmd *cmd)
>> +{
>> +    struct mm_struct *mm;
>> +    struct vm_area_struct *vma;
>> +    struct iovec *iov;
>> +    unsigned long address;
>> +    int i;
>> +
>> +    mm = cmd->vma_vm_mm;
>> +    if (!mm)
>> +        return;
>> +
>> +    vma = cmd->vma;
>> +    iov = cmd->iov;
>> +    if (mmget_not_zero(mm)) {
>> +        mmap_read_lock(mm);
>> +        for (i = 0; i < cmd->iov_cnt; i++) {
>> +            address = (unsigned long)iov->iov_base + vma->vm_start;
>> +            zap_page_range(vma, address, iov->iov_len);
>> +            iov++;
>> +        }
>> +        mmap_read_unlock(mm);
>> +        mmput(mm);
>> +    }
>> +
>> +    cmd->vma_vm_mm = NULL;
>> +    cmd->vma = NULL;
>> +    mmdrop(mm);
>> +}
>> +
>>   static inline void tcmu_free_cmd(struct tcmu_cmd *tcmu_cmd)
>>   {
>> +    if (test_bit(TCMU_CMD_BIT_ZEROCOPY, &tcmu_cmd->flags))
>> +        tcmu_cmd_zerocopy_unmap(tcmu_cmd);
>> +
>>       kfree(tcmu_cmd->dbi);
>>       kmem_cache_free(tcmu_cmd_cache, tcmu_cmd);
>>   }
>> @@ -850,37 +994,57 @@ static bool is_ring_space_avail(struct tcmu_dev 
>> *udev, size_t cmd_size)
>>    * Called with ring lock held.
>>    */
>>   static int tcmu_alloc_data_space(struct tcmu_dev *udev, struct 
>> tcmu_cmd *cmd,
>> -                  int *iov_bidi_cnt)
>> +                  int *iov_bidi_cnt, bool zero_copy)
>>   {
>>       int space, iov_cnt = 0, ret = 0;
>>         if (!cmd->dbi_cnt)
>>           goto wr_iov_cnts;
>>   -    /* try to check and get the data blocks as needed */
>> -    space = spc_bitmap_free(udev->data_bitmap, udev->dbi_thresh);
>> -    if (space < cmd->dbi_cnt) {
>> -        unsigned long blocks_left =
>> -                (udev->max_blocks - udev->dbi_thresh) + space;
>> +    if (!zero_copy) {
>> +        /* try to check and get the data blocks as needed */
>> +        space = spc_bitmap_free(udev->data_bitmap, udev->dbi_thresh);
>> +        if (space < cmd->dbi_cnt) {
>> +            unsigned long blocks_left =
>> +                    (udev->max_blocks - udev->dbi_thresh) + space;
>> +
>> +            if (blocks_left < cmd->dbi_cnt) {
>> +                pr_debug("no data space: only %lu available, but ask 
>> for %u\n",
>> +                        blocks_left * udev->data_blk_size,
>> +                        cmd->dbi_cnt * udev->data_blk_size);
>> +                return -1;
>> +            }
>>   -        if (blocks_left < cmd->dbi_cnt) {
>> -            pr_debug("no data space: only %lu available, but ask for 
>> %u\n",
>> -                    blocks_left * udev->data_blk_size,
>> -                    cmd->dbi_cnt * udev->data_blk_size);
>> -            return -1;
>> +            udev->dbi_thresh += cmd->dbi_cnt;
>> +            if (udev->dbi_thresh > udev->max_blocks)
>> +                udev->dbi_thresh = udev->max_blocks;
>>           }
>> +    } else {
>> +        /* try to check and get the data blocks as needed */
>> +        space = spc_bitmap_free(udev->zc_data_bitmap, 
>> udev->zc_dbi_thresh);
>> +        if (space < cmd->dbi_cnt) {
>> +            unsigned long blocks_left =
>> +                    (udev->zc_max_blocks - udev->zc_dbi_thresh) + 
>> space;
>> +
>> +            if (blocks_left < cmd->dbi_cnt) {
>> +                pr_debug("no data space: only %lu available, but ask 
>> for %u\n",
>> +                        blocks_left * udev->data_blk_size,
>> +                        cmd->dbi_cnt * udev->data_blk_size);
>> +                return -1;
>> +            }
>>   -        udev->dbi_thresh += cmd->dbi_cnt;
>> -        if (udev->dbi_thresh > udev->max_blocks)
>> -            udev->dbi_thresh = udev->max_blocks;
>> +            udev->zc_dbi_thresh += cmd->dbi_cnt;
>> +            if (udev->zc_dbi_thresh > udev->zc_max_blocks)
>> +                udev->zc_dbi_thresh = udev->zc_max_blocks;
>> +        }
>>       }
>>   -    iov_cnt = tcmu_get_empty_blocks(udev, cmd, 
>> cmd->se_cmd->data_length);
>> +    iov_cnt = tcmu_get_empty_blocks(udev, cmd, 
>> cmd->se_cmd->data_length, zero_copy);
>>       if (iov_cnt < 0)
>>           return -1;
>>         if (cmd->dbi_bidi_cnt) {
>> -        ret = tcmu_get_empty_blocks(udev, cmd, cmd->data_len_bidi);
>> +        ret = tcmu_get_empty_blocks(udev, cmd, cmd->data_len_bidi, 
>> zero_copy);
>>           if (ret < 0)
>>               return -1;
>>       }
>> @@ -1021,6 +1185,7 @@ static int queue_cmd_ring(struct tcmu_cmd 
>> *tcmu_cmd, sense_reason_t *scsi_err)
>>       uint32_t blk_size = udev->data_blk_size;
>>       /* size of data buffer needed */
>>       size_t data_length = (size_t)tcmu_cmd->dbi_cnt * blk_size;
>> +    bool zero_copy = false;
>>         *scsi_err = TCM_NO_SENSE;
>>   @@ -1044,7 +1209,22 @@ static int queue_cmd_ring(struct tcmu_cmd 
>> *tcmu_cmd, sense_reason_t *scsi_err)
>>           return -1;
>>       }
>>   -    iov_cnt = tcmu_alloc_data_space(udev, tcmu_cmd, &iov_bidi_cnt);
>> +    if (!(se_cmd->se_cmd_flags & SCF_BIDI) && se_cmd->data_length &&
>> +        IS_ALIGNED(se_cmd->data_length, PAGE_SIZE)) {
>
> What is the problem with BIDI here? Why do you skip those cmds?
No special reason, just want to be simple in RFC patch set.
I will add this support in next version.

>
>> +        struct scatterlist *data_sg = se_cmd->t_data_sg, *sg;
>> +        unsigned int data_nents = se_cmd->t_data_nents;
>> +        int i;
>> +
>> +        for_each_sg(data_sg, sg, data_nents, i) {
>> +            if (!((!sg->offset || IS_ALIGNED(sg->offset, PAGE_SIZE)) &&
>> +                IS_ALIGNED(sg->length, PAGE_SIZE)))
>> +                break;
>> +        }
>> +        if (i == data_nents)
>> +            zero_copy = true;
>> +    }
>> +
>> +    iov_cnt = tcmu_alloc_data_space(udev, tcmu_cmd, &iov_bidi_cnt, 
>> zero_copy);
>>       if (iov_cnt < 0)
>>           goto free_and_queue;
>>   @@ -1093,7 +1273,7 @@ static int queue_cmd_ring(struct tcmu_cmd 
>> *tcmu_cmd, sense_reason_t *scsi_err)
>>       tcmu_cmd_reset_dbi_cur(tcmu_cmd);
>>       iov = &entry->req.iov[0];
>>   -    if (se_cmd->data_direction == DMA_TO_DEVICE ||
>> +    if (((se_cmd->data_direction == DMA_TO_DEVICE) && !zero_copy) ||
>>           se_cmd->se_cmd_flags & SCF_BIDI)
>>           scatter_data_area(udev, tcmu_cmd, &iov);
>>       else
>> @@ -1111,6 +1291,11 @@ static int queue_cmd_ring(struct tcmu_cmd 
>> *tcmu_cmd, sense_reason_t *scsi_err)
>>       tcmu_setup_cmd_timer(tcmu_cmd, udev->cmd_time_out, 
>> &udev->cmd_timer);
>>         entry->hdr.cmd_id = tcmu_cmd->cmd_id;
>> +    if (zero_copy) {
>> +        iov = &entry->req.iov[0];
>> +        tcmu_cmd_zerocopy_map(udev, tcmu_cmd, iov, entry->req.iov_cnt);
>
> This call maps the kernel pages from sgl to userspace no matter whether
> it is a READ or WRITE type of cmd. That means, if it is READ, then you
> might map old kernel data to userspace, right?
Yes, I'm not sure whether it would be a security issue. Use real hardware
storage will also run into this issue, if this real hardware storage 
isn't credible.

>
>> +        set_bit(TCMU_CMD_BIT_ZEROCOPY, &tcmu_cmd->flags);
>> +    }
>>         tcmu_hdr_set_len(&entry->hdr.len_op, command_size);
>>   @@ -1366,7 +1551,10 @@ static bool tcmu_handle_completion(struct 
>> tcmu_cmd *cmd,
>>           else
>>               se_cmd->se_cmd_flags |= SCF_TREAT_READ_AS_NORMAL;
>>       }
>> -    if (se_cmd->se_cmd_flags & SCF_BIDI) {
>> +
>> +    if (test_bit(TCMU_CMD_BIT_ZEROCOPY, &cmd->flags)) {
>> +        tcmu_cmd_zerocopy_unmap(cmd);
>> +    } else if (se_cmd->se_cmd_flags & SCF_BIDI) {
>>           /* Get Data-In buffer before clean up */
>>           gather_data_area(udev, cmd, true, read_len);
>>       } else if (se_cmd->data_direction == DMA_FROM_DEVICE) {
>> @@ -1520,6 +1708,8 @@ static void tcmu_check_expired_ring_cmd(struct 
>> tcmu_cmd *cmd)
>>       if (!time_after_eq(jiffies, cmd->deadline))
>>           return;
>>   +    if (test_bit(TCMU_CMD_BIT_ZEROCOPY, &cmd->flags))
>> +        tcmu_cmd_zerocopy_unmap(cmd);
>>       set_bit(TCMU_CMD_BIT_EXPIRED, &cmd->flags);
>>       list_del_init(&cmd->queue_entry);
>>       se_cmd = cmd->se_cmd;
>> @@ -1618,6 +1808,8 @@ static struct se_device 
>> *tcmu_alloc_device(struct se_hba *hba, const char *name)
>>       udev->data_pages_per_blk = DATA_PAGES_PER_BLK_DEF;
>>       udev->max_blocks = DATA_AREA_PAGES_DEF / udev->data_pages_per_blk;
>>       udev->data_area_mb = TCMU_PAGES_TO_MBS(DATA_AREA_PAGES_DEF);
>> +    udev->zc_max_blocks = ZC_DATA_AREA_PAGES_DEF / 
>> udev->data_pages_per_blk;
>> +    udev->zc_data_area_mb = TCMU_PAGES_TO_MBS(ZC_DATA_AREA_PAGES_DEF);
>>         mutex_init(&udev->cmdr_lock);
>>   @@ -1841,6 +2033,9 @@ static void tcmu_vma_open(struct 
>> vm_area_struct *vma)
>>         pr_debug("vma_open\n");
>>   +    udev->vma_vm_mm = vma->vm_mm;
>> +    udev->vma = vma;
>> +    mmgrab(udev->vma_vm_mm);
>
> What about the mmap being opened multiple times?
> Up to now tcmu allows one open() only. But after open() userspace might
> fork with parent and child calling mmap() independently.
> AFAIK, tcmu_vma_open will then be called twice, but I don't know whether
> it is the same vma in both calls.
I think parent and child process will have different vma address. I had 
thought
this usage before, it will result in inconsistent or bugs. But In real 
application,
even without zero copy feature, if parent and child process operate the same
comm ring, seems that it will also result inconsistent or bugs.
I'll have a deep look at codes, thanks.


Regards,
Xiaoguang Wang
>
>>       kref_get(&udev->kref);
>>   }
>>   @@ -1850,6 +2045,10 @@ static void tcmu_vma_close(struct 
>> vm_area_struct *vma)
>>         pr_debug("vma_close\n");
>>   +    mmdrop(udev->vma_vm_mm);
>> +    udev->vma_vm_mm = NULL;
>> +    udev->vma = NULL;
>> +
>>       /* release ref from tcmu_vma_open */
>>       kref_put(&udev->kref, tcmu_dev_kref_release);
>>   }
>> @@ -1901,7 +2100,7 @@ static int tcmu_mmap(struct uio_info *info, 
>> struct vm_area_struct *vma)
>>   {
>>       struct tcmu_dev *udev = container_of(info, struct tcmu_dev, 
>> uio_info);
>>   -    vma->vm_flags |= VM_DONTEXPAND | VM_DONTDUMP;
>> +    vma->vm_flags |= VM_DONTEXPAND | VM_DONTDUMP | VM_MIXEDMAP;
>>       vma->vm_ops = &tcmu_vm_ops;
>>         vma->vm_private_data = udev;
>> @@ -2172,7 +2371,7 @@ static int tcmu_configure_device(struct 
>> se_device *dev)
>>       struct tcmu_dev *udev = TCMU_DEV(dev);
>>       struct uio_info *info;
>>       struct tcmu_mailbox *mb;
>> -    size_t data_size;
>> +    size_t data_size, zc_data_size;
>>       int ret = 0;
>>         ret = tcmu_update_uio_info(udev);
>> @@ -2183,8 +2382,9 @@ static int tcmu_configure_device(struct 
>> se_device *dev)
>>         mutex_lock(&udev->cmdr_lock);
>>       udev->data_bitmap = bitmap_zalloc(udev->max_blocks, GFP_KERNEL);
>> +    udev->zc_data_bitmap = bitmap_zalloc(udev->zc_max_blocks, 
>> GFP_KERNEL);
>>       mutex_unlock(&udev->cmdr_lock);
>> -    if (!udev->data_bitmap) {
>> +    if (!udev->data_bitmap || !udev->zc_data_bitmap) {
>>           ret = -ENOMEM;
>>           goto err_bitmap_alloc;
>>       }
>> @@ -2201,7 +2401,8 @@ static int tcmu_configure_device(struct 
>> se_device *dev)
>>       udev->cmdr_size = CMDR_SIZE;
>>       udev->data_off = MB_CMDR_SIZE;
>>       data_size = TCMU_MBS_TO_PAGES(udev->data_area_mb) << PAGE_SHIFT;
>> -    udev->mmap_pages = (data_size + MB_CMDR_SIZE) >> PAGE_SHIFT;
>> +    zc_data_size = TCMU_MBS_TO_PAGES(udev->zc_data_area_mb) << 
>> PAGE_SHIFT;
>> +    udev->mmap_pages = (data_size + zc_data_size + MB_CMDR_SIZE) >> 
>> PAGE_SHIFT;
>>       udev->data_blk_size = udev->data_pages_per_blk * PAGE_SIZE;
>>       udev->dbi_thresh = 0; /* Default in Idle state */
>>   @@ -2221,7 +2422,7 @@ static int tcmu_configure_device(struct 
>> se_device *dev)
>>         info->mem[0].name = "tcm-user command & data buffer";
>>       info->mem[0].addr = (phys_addr_t)(uintptr_t)udev->mb_addr;
>> -    info->mem[0].size = data_size + MB_CMDR_SIZE;
>> +    info->mem[0].size = data_size + zc_data_size + MB_CMDR_SIZE;
>>       info->mem[0].memtype = UIO_MEM_NONE;
>>         info->irqcontrol = tcmu_irqcontrol;


^ permalink raw reply	[flat|nested] 19+ messages in thread

* Re: [RFC 1/3] mm/memory.c: introduce vm_insert_page(s)_mkspecial
  2022-03-18  9:55 ` [RFC 1/3] mm/memory.c: introduce vm_insert_page(s)_mkspecial Xiaoguang Wang
@ 2022-03-23 16:45   ` Christoph Hellwig
  2022-03-24  7:27     ` Xiaoguang Wang
  0 siblings, 1 reply; 19+ messages in thread
From: Christoph Hellwig @ 2022-03-23 16:45 UTC (permalink / raw)
  To: Xiaoguang Wang
  Cc: linux-mm, target-devel, linux-scsi, linux-block, xuyu, bostroesser

I relaly don't think we should add this amount of overhead to the
core VM for a (relatively) irrelevant driver.

^ permalink raw reply	[flat|nested] 19+ messages in thread

* Re: [RFC 2/3] mm: export zap_page_range()
  2022-03-18  9:55 ` [RFC 2/3] mm: export zap_page_range() Xiaoguang Wang
  2022-03-21 12:01   ` David Hildenbrand
@ 2022-03-23 16:47   ` Christoph Hellwig
  2022-03-24  9:16   ` Ming Lei
  2 siblings, 0 replies; 19+ messages in thread
From: Christoph Hellwig @ 2022-03-23 16:47 UTC (permalink / raw)
  To: Xiaoguang Wang
  Cc: linux-mm, target-devel, linux-scsi, linux-block, xuyu, bostroesser

On Fri, Mar 18, 2022 at 05:55:30PM +0800, Xiaoguang Wang wrote:
> Module target_core_user will use it to implement zero copy feature.

NAK, this has no business being exported to modules.

^ permalink raw reply	[flat|nested] 19+ messages in thread

* Re: [RFC 2/3] mm: export zap_page_range()
  2022-03-22 13:08       ` David Hildenbrand
  2022-03-23 13:59         ` Xiaoguang Wang
@ 2022-03-23 16:48         ` Christoph Hellwig
  1 sibling, 0 replies; 19+ messages in thread
From: Christoph Hellwig @ 2022-03-23 16:48 UTC (permalink / raw)
  To: David Hildenbrand
  Cc: Xiaoguang Wang, linux-mm, target-devel, linux-scsi, linux-block,
	xuyu, bostroesser

On Tue, Mar 22, 2022 at 02:08:13PM +0100, David Hildenbrand wrote:
> Reason is that we don't want anybody to blindly zap_page_range() within
> random VMAs from a kernel module.

Not just that, but there is no business for modules doing this at all.
These kinds of VM hooks for random drivers are not a good idea.

^ permalink raw reply	[flat|nested] 19+ messages in thread

* Re: [RFC 1/3] mm/memory.c: introduce vm_insert_page(s)_mkspecial
  2022-03-23 16:45   ` Christoph Hellwig
@ 2022-03-24  7:27     ` Xiaoguang Wang
  0 siblings, 0 replies; 19+ messages in thread
From: Xiaoguang Wang @ 2022-03-24  7:27 UTC (permalink / raw)
  To: Christoph Hellwig
  Cc: linux-mm, target-devel, linux-scsi, linux-block, xuyu, bostroesser

hi,

> I relaly don't think we should add this amount of overhead to the
> core VM for a (relatively) irrelevant driver.
OK, but as what I have described in cover letter, both vm_insert_pages and
remap_pfn_range have performance or usage limits. Do you know any better
method to map block device io request's sgl pages to user space? I think 
block
device backend implemented in use space will benefit from this feature, 
thanks.

Regards,
Xiaoguang Wang

^ permalink raw reply	[flat|nested] 19+ messages in thread

* Re: [RFC 2/3] mm: export zap_page_range()
  2022-03-18  9:55 ` [RFC 2/3] mm: export zap_page_range() Xiaoguang Wang
  2022-03-21 12:01   ` David Hildenbrand
  2022-03-23 16:47   ` Christoph Hellwig
@ 2022-03-24  9:16   ` Ming Lei
  2 siblings, 0 replies; 19+ messages in thread
From: Ming Lei @ 2022-03-24  9:16 UTC (permalink / raw)
  To: Xiaoguang Wang
  Cc: linux-mm, target-devel, linux-scsi, linux-block, xuyu, bostroesser

On Fri, Mar 18, 2022 at 05:55:30PM +0800, Xiaoguang Wang wrote:
> Module target_core_user will use it to implement zero copy feature.
> 
> Signed-off-by: Xiaoguang Wang <xiaoguang.wang@linux.alibaba.com>
> ---
>  mm/memory.c | 1 +
>  1 file changed, 1 insertion(+)
> 
> diff --git a/mm/memory.c b/mm/memory.c
> index 1f745e4d11c2..9974d0406dad 100644
> --- a/mm/memory.c
> +++ b/mm/memory.c
> @@ -1664,6 +1664,7 @@ void zap_page_range(struct vm_area_struct *vma, unsigned long start,
>  	mmu_notifier_invalidate_range_end(&range);
>  	tlb_finish_mmu(&tlb);
>  }
> +EXPORT_SYMBOL_GPL(zap_page_range);

BTW, what is the counter part api of remap_pfn_range() for serving the
unmap? Or does it really need to unmap the vm space for this zero-copy
case?

If it isn't necessary to unmap, maybe remap_pfn_range() is faster than
vm_insert_page(s)_mkspecial + zap_page_range() since zap_page_range()
looks a bit heavy.


Thanks,
Ming


^ permalink raw reply	[flat|nested] 19+ messages in thread

* Re: [RFC 3/3] scsi: target: tcmu: Support zero copy
  2022-03-23 14:33     ` Xiaoguang Wang
@ 2022-03-25  9:06       ` Bodo Stroesser
  0 siblings, 0 replies; 19+ messages in thread
From: Bodo Stroesser @ 2022-03-25  9:06 UTC (permalink / raw)
  To: Xiaoguang Wang, linux-mm, target-devel, linux-scsi; +Cc: linux-block, xuyu



On 23.03.22 15:33, Xiaoguang Wang wrote:

... snip ...

>> What happens if the tcmu device currently is not open / mapped?
> I'm not sure how it will happen.
> But we may check whether udev->vma has a valid value. If yes,
> it'll enter tcmu_cmd_zerocopy_map().
> 

Yeah, but what I also wanted to suggest is, that if userspace maps
the uio device while zero copy cmds already being in the ring,
you have to iterate over those cmds and populate the mapping.

Bodo

^ permalink raw reply	[flat|nested] 19+ messages in thread

end of thread, other threads:[~2022-03-25  9:07 UTC | newest]

Thread overview: 19+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2022-03-18  9:55 [RFC 0/3] Add zero copy feature for tcmu Xiaoguang Wang
2022-03-18  9:55 ` [RFC 1/3] mm/memory.c: introduce vm_insert_page(s)_mkspecial Xiaoguang Wang
2022-03-23 16:45   ` Christoph Hellwig
2022-03-24  7:27     ` Xiaoguang Wang
2022-03-18  9:55 ` [RFC 2/3] mm: export zap_page_range() Xiaoguang Wang
2022-03-21 12:01   ` David Hildenbrand
2022-03-22 13:02     ` Xiaoguang Wang
2022-03-22 13:08       ` David Hildenbrand
2022-03-23 13:59         ` Xiaoguang Wang
2022-03-23 16:48         ` Christoph Hellwig
2022-03-23 16:47   ` Christoph Hellwig
2022-03-24  9:16   ` Ming Lei
2022-03-18  9:55 ` [RFC 3/3] scsi: target: tcmu: Support zero copy Xiaoguang Wang
2022-03-22 14:01   ` Bodo Stroesser
2022-03-23 14:33     ` Xiaoguang Wang
2022-03-25  9:06       ` Bodo Stroesser
2022-03-22 12:40 ` [RFC 0/3] Add zero copy feature for tcmu Bodo Stroesser
2022-03-22 13:17   ` Xiaoguang Wang
2022-03-22 14:05     ` Bodo Stroesser

This is an external index of several public inboxes,
see mirroring instructions on how to clone and mirror
all data and code used by this external index.