linux-kernel.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [PATCH v7 kernel 0/5] Extend virtio-balloon for fast (de)inflating & fast live migration
@ 2017-03-03  5:40 Wei Wang
  2017-03-03  5:40 ` [PATCH v7 kernel 1/5] virtio-balloon: rework deflate to add page to a list Wei Wang
                   ` (4 more replies)
  0 siblings, 5 replies; 30+ messages in thread
From: Wei Wang @ 2017-03-03  5:40 UTC (permalink / raw)
  To: virtio-dev, kvm, qemu-devel, linux-kernel, virtualization, linux-mm
  Cc: Wei Wang, Michael S . Tsirkin, Paolo Bonzini, Cornelia Huck,
	Amit Shah, Dave Hansen, Andrea Arcangeli, David Hildenbrand,
	Liang Li

Take over this work from Liang.

This patch series implements two optimizations: 1) transfer pages in chuncks
between the guest and host; 1) transfer the guest unused pages to the host so
that they can be skipped to migrate in live migration.

Please check patch 0003 for more details about optimization 1).

For an idle guest with 8GB RAM, optimization 2) can help shorten the total live
migration time from 2Sec to about 500ms in 10Gbps network
environment. For a guest with quite a lot of page cache and little
unused pages, it's possible to let the guest drop its page cache before
live migration, this case can benefit from this new feature too.

Cc: Michael S. Tsirkin <mst@redhat.com>
Cc: Paolo Bonzini <pbonzini@redhat.com>
Cc: Cornelia Huck <cornelia.huck@de.ibm.com>
Cc: Amit Shah <amit.shah@redhat.com>
Cc: Dave Hansen <dave.hansen@intel.com>
Cc: Andrea Arcangeli <aarcange@redhat.com>
Cc: David Hildenbrand <david@redhat.com>
Cc: Liang Li <liliang324@gmail.com>
Cc: Wei Wang <wei.w.wang@intel.com>

Liang Li (5):
  virtio-balloon: rework deflate to add page to a list
  virtio-balloon: VIRTIO_BALLOON_F_CHUNK_TRANSFER
  virtio-balloon: implementation of VIRTIO_BALLOON_F_CHUNK_TRANSFER
  virtio-balloon: define flags and head for host request vq
  This patch contains two parts:

 drivers/virtio/virtio_balloon.c     | 510 ++++++++++++++++++++++++++++++++----
 include/linux/mm.h                  |   3 +
 include/uapi/linux/virtio_balloon.h |  34 +++
 mm/page_alloc.c                     | 120 +++++++++
 4 files changed, 620 insertions(+), 47 deletions(-)

-- 
2.7.4

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

* [PATCH v7 kernel 1/5] virtio-balloon: rework deflate to add page to a list
  2017-03-03  5:40 [PATCH v7 kernel 0/5] Extend virtio-balloon for fast (de)inflating & fast live migration Wei Wang
@ 2017-03-03  5:40 ` Wei Wang
  2017-03-03  5:40 ` [PATCH v7 kernel 2/5] virtio-balloon: VIRTIO_BALLOON_F_CHUNK_TRANSFER Wei Wang
                   ` (3 subsequent siblings)
  4 siblings, 0 replies; 30+ messages in thread
From: Wei Wang @ 2017-03-03  5:40 UTC (permalink / raw)
  To: virtio-dev, kvm, qemu-devel, linux-kernel, virtualization, linux-mm
  Cc: Liang Li, Michael S . Tsirkin, Paolo Bonzini, Cornelia Huck,
	Amit Shah, Dave Hansen, Andrea Arcangeli, David Hildenbrand,
	Liang Li, Wei Wang

From: Liang Li <liang.z.li@intel.com>

When doing the inflating/deflating operation, the current virtio-balloon
implementation uses an array to save 256 PFNS, then send these PFNS to
host through virtio and process each PFN one by one. This way is not
efficient when inflating/deflating a large mount of memory because too
many times of the following operations:

    1. Virtio data transmission
    2. Page allocate/free
    3. Address translation(GPA->HVA)
    4. madvise

The over head of these operations will consume a lot of CPU cycles and
will take a long time to complete, it may impact the QoS of the guest as
well as the host. The overhead will be reduced a lot if batch processing
is used. E.g. If there are several pages whose address are physical
contiguous in the guest, these bulk pages can be processed in one
operation.

The main idea for the optimization is to reduce the above operations as
much as possible. And it can be achieved by using a {pfn|length} array
instead of a PFN array. Comparing with PFN array, {pfn|length} array can
present more pages and is fit for batch processing.

This patch saves the deflated pages to a list instead of the PFN array,
which will allow faster notifications using the {pfn|length} down the
road. balloon_pfn_to_page() can be removed because it's useless.

Signed-off-by: Liang Li <liang.z.li@intel.com>
Signed-off-by: Michael S. Tsirkin <mst@redhat.com>
Cc: Paolo Bonzini <pbonzini@redhat.com>
Cc: Cornelia Huck <cornelia.huck@de.ibm.com>
Cc: Amit Shah <amit.shah@redhat.com>
Cc: Dave Hansen <dave.hansen@intel.com>
Cc: Andrea Arcangeli <aarcange@redhat.com>
Cc: David Hildenbrand <david@redhat.com>
Cc: Liang Li <liliang324@gmail.com>
Cc: Wei Wang <wei.w.wang@intel.com>
---
 drivers/virtio/virtio_balloon.c | 22 ++++++++--------------
 1 file changed, 8 insertions(+), 14 deletions(-)

diff --git a/drivers/virtio/virtio_balloon.c b/drivers/virtio/virtio_balloon.c
index 181793f..f59cb4f 100644
--- a/drivers/virtio/virtio_balloon.c
+++ b/drivers/virtio/virtio_balloon.c
@@ -103,12 +103,6 @@ static u32 page_to_balloon_pfn(struct page *page)
 	return pfn * VIRTIO_BALLOON_PAGES_PER_PAGE;
 }
 
-static struct page *balloon_pfn_to_page(u32 pfn)
-{
-	BUG_ON(pfn % VIRTIO_BALLOON_PAGES_PER_PAGE);
-	return pfn_to_page(pfn / VIRTIO_BALLOON_PAGES_PER_PAGE);
-}
-
 static void balloon_ack(struct virtqueue *vq)
 {
 	struct virtio_balloon *vb = vq->vdev->priv;
@@ -181,18 +175,16 @@ static unsigned fill_balloon(struct virtio_balloon *vb, size_t num)
 	return num_allocated_pages;
 }
 
-static void release_pages_balloon(struct virtio_balloon *vb)
+static void release_pages_balloon(struct virtio_balloon *vb,
+				 struct list_head *pages)
 {
-	unsigned int i;
-	struct page *page;
+	struct page *page, *next;
 
-	/* Find pfns pointing at start of each page, get pages and free them. */
-	for (i = 0; i < vb->num_pfns; i += VIRTIO_BALLOON_PAGES_PER_PAGE) {
-		page = balloon_pfn_to_page(virtio32_to_cpu(vb->vdev,
-							   vb->pfns[i]));
+	list_for_each_entry_safe(page, next, pages, lru) {
 		if (!virtio_has_feature(vb->vdev,
 					VIRTIO_BALLOON_F_DEFLATE_ON_OOM))
 			adjust_managed_page_count(page, 1);
+		list_del(&page->lru);
 		put_page(page); /* balloon reference */
 	}
 }
@@ -202,6 +194,7 @@ static unsigned leak_balloon(struct virtio_balloon *vb, size_t num)
 	unsigned num_freed_pages;
 	struct page *page;
 	struct balloon_dev_info *vb_dev_info = &vb->vb_dev_info;
+	LIST_HEAD(pages);
 
 	/* We can only do one array worth at a time. */
 	num = min(num, ARRAY_SIZE(vb->pfns));
@@ -215,6 +208,7 @@ static unsigned leak_balloon(struct virtio_balloon *vb, size_t num)
 		if (!page)
 			break;
 		set_page_pfns(vb, vb->pfns + vb->num_pfns, page);
+		list_add(&page->lru, &pages);
 		vb->num_pages -= VIRTIO_BALLOON_PAGES_PER_PAGE;
 	}
 
@@ -226,7 +220,7 @@ static unsigned leak_balloon(struct virtio_balloon *vb, size_t num)
 	 */
 	if (vb->num_pfns != 0)
 		tell_host(vb, vb->deflate_vq);
-	release_pages_balloon(vb);
+	release_pages_balloon(vb, &pages);
 	mutex_unlock(&vb->balloon_lock);
 	return num_freed_pages;
 }
-- 
2.7.4

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

* [PATCH v7 kernel 2/5] virtio-balloon: VIRTIO_BALLOON_F_CHUNK_TRANSFER
  2017-03-03  5:40 [PATCH v7 kernel 0/5] Extend virtio-balloon for fast (de)inflating & fast live migration Wei Wang
  2017-03-03  5:40 ` [PATCH v7 kernel 1/5] virtio-balloon: rework deflate to add page to a list Wei Wang
@ 2017-03-03  5:40 ` Wei Wang
  2017-03-08  4:01   ` Michael S. Tsirkin
  2017-03-03  5:40 ` [PATCH v7 kernel 3/5] virtio-balloon: implementation of VIRTIO_BALLOON_F_CHUNK_TRANSFER Wei Wang
                   ` (2 subsequent siblings)
  4 siblings, 1 reply; 30+ messages in thread
From: Wei Wang @ 2017-03-03  5:40 UTC (permalink / raw)
  To: virtio-dev, kvm, qemu-devel, linux-kernel, virtualization, linux-mm
  Cc: Liang Li, Wei Wang, Michael S . Tsirkin, Paolo Bonzini,
	Cornelia Huck, Amit Shah, Dave Hansen, Andrea Arcangeli,
	David Hildenbrand, Liang Li

From: Liang Li <liang.z.li@intel.com>

Add a new feature bit, VIRTIO_BALLOON_F_CHUNK_TRANSFER. Please check
the implementation patch commit for details about this feature.

Signed-off-by: Liang Li <liang.z.li@intel.com>
Signed-off-by: Wei Wang <wei.w.wang@intel.com>
Cc: Michael S. Tsirkin <mst@redhat.com>
Cc: Paolo Bonzini <pbonzini@redhat.com>
Cc: Cornelia Huck <cornelia.huck@de.ibm.com>
Cc: Amit Shah <amit.shah@redhat.com>
Cc: Dave Hansen <dave.hansen@intel.com>
Cc: Andrea Arcangeli <aarcange@redhat.com>
Cc: David Hildenbrand <david@redhat.com>
Cc: Liang Li <liliang324@gmail.com>
Cc: Wei Wang <wei.w.wang@intel.com>
---
 include/uapi/linux/virtio_balloon.h | 12 ++++++++++++
 1 file changed, 12 insertions(+)

diff --git a/include/uapi/linux/virtio_balloon.h b/include/uapi/linux/virtio_balloon.h
index 343d7dd..ed627b2 100644
--- a/include/uapi/linux/virtio_balloon.h
+++ b/include/uapi/linux/virtio_balloon.h
@@ -34,10 +34,14 @@
 #define VIRTIO_BALLOON_F_MUST_TELL_HOST	0 /* Tell before reclaiming pages */
 #define VIRTIO_BALLOON_F_STATS_VQ	1 /* Memory Stats virtqueue */
 #define VIRTIO_BALLOON_F_DEFLATE_ON_OOM	2 /* Deflate balloon on OOM */
+#define VIRTIO_BALLOON_F_CHUNK_TRANSFER	3 /* Transfer pages in chunks */
 
 /* Size of a PFN in the balloon interface. */
 #define VIRTIO_BALLOON_PFN_SHIFT 12
 
+/* Shift to get a chunk size */
+#define VIRTIO_BALLOON_CHUNK_SIZE_SHIFT 12
+
 struct virtio_balloon_config {
 	/* Number of pages host wants Guest to give up. */
 	__u32 num_pages;
@@ -82,4 +86,12 @@ struct virtio_balloon_stat {
 	__virtio64 val;
 } __attribute__((packed));
 
+/* Response header structure */
+struct virtio_balloon_resp_hdr {
+	u8 cmd;
+	u8 flag;
+	__le16 id; /* cmd id */
+	__le32 data_len; /* Payload len in bytes */
+};
+
 #endif /* _LINUX_VIRTIO_BALLOON_H */
-- 
2.7.4

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

* [PATCH v7 kernel 3/5] virtio-balloon: implementation of VIRTIO_BALLOON_F_CHUNK_TRANSFER
  2017-03-03  5:40 [PATCH v7 kernel 0/5] Extend virtio-balloon for fast (de)inflating & fast live migration Wei Wang
  2017-03-03  5:40 ` [PATCH v7 kernel 1/5] virtio-balloon: rework deflate to add page to a list Wei Wang
  2017-03-03  5:40 ` [PATCH v7 kernel 2/5] virtio-balloon: VIRTIO_BALLOON_F_CHUNK_TRANSFER Wei Wang
@ 2017-03-03  5:40 ` Wei Wang
  2017-03-08  4:01   ` Michael S. Tsirkin
  2017-03-09 14:14   ` Matthew Wilcox
  2017-03-03  5:40 ` [PATCH v7 kernel 4/5] virtio-balloon: define flags and head for host request vq Wei Wang
  2017-03-03  5:40 ` [PATCH v7 kernel 5/5] This patch contains two parts: Wei Wang
  4 siblings, 2 replies; 30+ messages in thread
From: Wei Wang @ 2017-03-03  5:40 UTC (permalink / raw)
  To: virtio-dev, kvm, qemu-devel, linux-kernel, virtualization, linux-mm
  Cc: Liang Li, Wei Wang, Michael S . Tsirkin, Paolo Bonzini,
	Cornelia Huck, Amit Shah, Dave Hansen, Andrea Arcangeli,
	David Hildenbrand, Liang Li

From: Liang Li <liang.z.li@intel.com>

The implementation of the current virtio-balloon is not very
efficient, because the pages are transferred to the host one by one.
Here is the breakdown of the time in percentage spent on each
step of the balloon inflating process (inflating 7GB of an 8GB
idle guest).

1) allocating pages (6.5%)
2) sending PFNs to host (68.3%)
3) address translation (6.1%)
4) madvise (19%)

It takes about 4126ms for the inflating process to complete.
The above profiling shows that the bottlenecks are stage 2)
and stage 4).

This patch optimizes step 2) by transfering pages to the host in
chunks. A chunk consists of guest physically continuous pages, and
it is offered to the host via a base PFN (i.e. the start PFN of
those physically continuous pages) and the size (i.e. the total
number of the pages). A normal chunk is formated as below:
-----------------------------------------------
|  Base (52 bit)               | Size (12 bit)|
-----------------------------------------------
For large size chunks, an extended chunk format is used:
-----------------------------------------------
|                 Base (64 bit)               |
-----------------------------------------------
-----------------------------------------------
|                 Size (64 bit)               |
-----------------------------------------------

By doing so, step 4) can also be optimized by doing address
translation and madvise() in chunks rather than page by page.

This optimization requires the negotation of a new feature bit,
VIRTIO_BALLOON_F_CHUNK_TRANSFER.

With this new feature, the above ballooning process takes ~590ms
resulting in an improvement of ~85%.

TODO: optimize stage 1) by allocating/freeing a chunk of pages
instead of a single page each time.

Signed-off-by: Liang Li <liang.z.li@intel.com>
Signed-off-by: Wei Wang <wei.w.wang@intel.com>
Suggested-by: Michael S. Tsirkin <mst@redhat.com>
Cc: Michael S. Tsirkin <mst@redhat.com>
Cc: Paolo Bonzini <pbonzini@redhat.com>
Cc: Cornelia Huck <cornelia.huck@de.ibm.com>
Cc: Amit Shah <amit.shah@redhat.com>
Cc: Dave Hansen <dave.hansen@intel.com>
Cc: Andrea Arcangeli <aarcange@redhat.com>
Cc: David Hildenbrand <david@redhat.com>
Cc: Liang Li <liliang324@gmail.com>
Cc: Wei Wang <wei.w.wang@intel.com>
---
 drivers/virtio/virtio_balloon.c | 351 ++++++++++++++++++++++++++++++++++++----
 1 file changed, 323 insertions(+), 28 deletions(-)

diff --git a/drivers/virtio/virtio_balloon.c b/drivers/virtio/virtio_balloon.c
index f59cb4f..4416370 100644
--- a/drivers/virtio/virtio_balloon.c
+++ b/drivers/virtio/virtio_balloon.c
@@ -42,6 +42,10 @@
 #define OOM_VBALLOON_DEFAULT_PAGES 256
 #define VIRTBALLOON_OOM_NOTIFY_PRIORITY 80
 
+#define PAGE_BMAP_SIZE	(8 * PAGE_SIZE)
+#define PFNS_PER_PAGE_BMAP	(PAGE_BMAP_SIZE * BITS_PER_BYTE)
+#define PAGE_BMAP_COUNT_MAX	32
+
 static int oom_pages = OOM_VBALLOON_DEFAULT_PAGES;
 module_param(oom_pages, int, S_IRUSR | S_IWUSR);
 MODULE_PARM_DESC(oom_pages, "pages to free on OOM");
@@ -50,6 +54,16 @@ MODULE_PARM_DESC(oom_pages, "pages to free on OOM");
 static struct vfsmount *balloon_mnt;
 #endif
 
+struct balloon_page_chunk {
+	__le64 base : 52;
+	__le64 size : 12;
+};
+
+struct balloon_page_chunk_ext {
+	__le64 base;
+	__le64 size;
+};
+
 struct virtio_balloon {
 	struct virtio_device *vdev;
 	struct virtqueue *inflate_vq, *deflate_vq, *stats_vq;
@@ -67,6 +81,20 @@ struct virtio_balloon {
 
 	/* Number of balloon pages we've told the Host we're not using. */
 	unsigned int num_pages;
+	/* Pointer to the response header. */
+	struct virtio_balloon_resp_hdr resp_hdr;
+	/* Pointer to the start address of response data. */
+	__le64 *resp_data;
+	/* Size of response data buffer. */
+	unsigned int resp_buf_size;
+	/* Pointer offset of the response data. */
+	unsigned int resp_pos;
+	/* Bitmap used to save the pfns info */
+	unsigned long *page_bitmap[PAGE_BMAP_COUNT_MAX];
+	/* Number of split page bitmaps */
+	unsigned int nr_page_bmap;
+	/* Used to record the processed pfn range */
+	unsigned long min_pfn, max_pfn, start_pfn, end_pfn;
 	/*
 	 * The pages we've told the Host we're not using are enqueued
 	 * at vb_dev_info->pages list.
@@ -110,20 +138,180 @@ static void balloon_ack(struct virtqueue *vq)
 	wake_up(&vb->acked);
 }
 
-static void tell_host(struct virtio_balloon *vb, struct virtqueue *vq)
+static inline void init_bmap_pfn_range(struct virtio_balloon *vb)
 {
-	struct scatterlist sg;
+	vb->min_pfn = ULONG_MAX;
+	vb->max_pfn = 0;
+}
+
+static inline void update_bmap_pfn_range(struct virtio_balloon *vb,
+				 struct page *page)
+{
+	unsigned long balloon_pfn = page_to_balloon_pfn(page);
+
+	vb->min_pfn = min(balloon_pfn, vb->min_pfn);
+	vb->max_pfn = max(balloon_pfn, vb->max_pfn);
+}
+
+static void extend_page_bitmap(struct virtio_balloon *vb,
+				unsigned long nr_pfn)
+{
+	int i, bmap_count;
+	unsigned long bmap_len;
+
+	bmap_len = ALIGN(nr_pfn, BITS_PER_LONG) / BITS_PER_BYTE;
+	bmap_len = ALIGN(bmap_len, PAGE_BMAP_SIZE);
+	bmap_count = min((int)(bmap_len / PAGE_BMAP_SIZE),
+				 PAGE_BMAP_COUNT_MAX);
+
+	for (i = 1; i < bmap_count; i++) {
+		vb->page_bitmap[i] = kmalloc(PAGE_BMAP_SIZE, GFP_KERNEL);
+		if (vb->page_bitmap[i])
+			vb->nr_page_bmap++;
+		else
+			break;
+	}
+}
+
+static void free_extended_page_bitmap(struct virtio_balloon *vb)
+{
+	int i, bmap_count = vb->nr_page_bmap;
+
+	for (i = 1; i < bmap_count; i++) {
+		kfree(vb->page_bitmap[i]);
+		vb->page_bitmap[i] = NULL;
+		vb->nr_page_bmap--;
+	}
+}
+
+static void kfree_page_bitmap(struct virtio_balloon *vb)
+{
+	int i;
+
+	for (i = 0; i < vb->nr_page_bmap; i++)
+		kfree(vb->page_bitmap[i]);
+}
+
+static void clear_page_bitmap(struct virtio_balloon *vb)
+{
+	int i;
+
+	for (i = 0; i < vb->nr_page_bmap; i++)
+		memset(vb->page_bitmap[i], 0, PAGE_BMAP_SIZE);
+}
+
+static void send_resp_data(struct virtio_balloon *vb, struct virtqueue *vq,
+			bool busy_wait)
+{
+	struct scatterlist sg[2];
+	struct virtio_balloon_resp_hdr *hdr = &vb->resp_hdr;
 	unsigned int len;
 
-	sg_init_one(&sg, vb->pfns, sizeof(vb->pfns[0]) * vb->num_pfns);
+	len = hdr->data_len = vb->resp_pos * sizeof(__le64);
+	sg_init_table(sg, 2);
+	sg_set_buf(&sg[0], hdr, sizeof(struct virtio_balloon_resp_hdr));
+	sg_set_buf(&sg[1], vb->resp_data, len);
+
+	if (virtqueue_add_outbuf(vq, sg, 2, vb, GFP_KERNEL) == 0) {
+		virtqueue_kick(vq);
+		if (busy_wait)
+			while (!virtqueue_get_buf(vq, &len)
+				&& !virtqueue_is_broken(vq))
+				cpu_relax();
+		else
+			wait_event(vb->acked, virtqueue_get_buf(vq, &len));
+		vb->resp_pos = 0;
+		free_extended_page_bitmap(vb);
+	}
+}
 
-	/* We should always be able to add one buffer to an empty queue. */
-	virtqueue_add_outbuf(vq, &sg, 1, vb, GFP_KERNEL);
-	virtqueue_kick(vq);
+static void do_set_resp_bitmap(struct virtio_balloon *vb, unsigned long base,
+			       int size)
+{
+	/* Use the extented chunk format if the size is too large */
+	if (size > (1 << VIRTIO_BALLOON_CHUNK_SIZE_SHIFT)) {
+		struct balloon_page_chunk_ext *chunk_ext =
+			(struct balloon_page_chunk_ext *)(vb->resp_data + vb->resp_pos);
+		chunk_ext->base = cpu_to_le64(base);
+		chunk_ext->size = cpu_to_le64(size);
+		vb->resp_pos += sizeof(vb->resp_pos) / sizeof(*chunk_ext);
+	} else {
+		struct balloon_page_chunk *chunk =
+			(struct balloon_page_chunk *)(vb->resp_data + vb->resp_pos);
+		chunk->base = cpu_to_le64(base);
+		chunk->size = cpu_to_le16(size);
+		vb->resp_pos += sizeof(vb->resp_pos) / sizeof(*chunk);
+	}
+}
 
-	/* When host has read buffer, this completes via balloon_ack */
-	wait_event(vb->acked, virtqueue_get_buf(vq, &len));
+static void chunking_pages_from_bmap(struct virtio_balloon *vb,
+				     struct virtqueue *vq,
+				     unsigned long start_pfn, unsigned long *bitmap,
+				     unsigned long len, bool busy_wait)
+{
+	unsigned long pos = 0, end = len * BITS_PER_BYTE;
+
+	while (pos < end) {
+		unsigned long one = find_next_bit(bitmap, end, pos);
+
+		if (one < end) {
+			unsigned long chunk_size, zero;
+
+			zero = find_next_zero_bit(bitmap, end, one + 1);
+			if (zero >= end)
+				chunk_size = end - one;
+			else
+				chunk_size = zero - one;
+			if (chunk_size) {
+				if ((vb->resp_pos + 2) * sizeof(__le64) >
+						vb->resp_buf_size)
+					send_resp_data(vb, vq, busy_wait);
+				do_set_resp_bitmap(vb, start_pfn + one,	chunk_size);
+			}
+			pos = one + chunk_size;
+		} else
+			break;
+	}
+}
 
+static void tell_host(struct virtio_balloon *vb, struct virtqueue *vq)
+{
+	if (virtio_has_feature(vb->vdev, VIRTIO_BALLOON_F_CHUNK_TRANSFER)) {
+		int nr_pfn, nr_used_bmap, i;
+		unsigned long start_pfn, bmap_len;
+
+		start_pfn = vb->start_pfn;
+		nr_pfn = vb->end_pfn - start_pfn + 1;
+		nr_pfn = roundup(nr_pfn, BITS_PER_LONG);
+		nr_used_bmap = nr_pfn / PFNS_PER_PAGE_BMAP;
+		if (nr_pfn % PFNS_PER_PAGE_BMAP)
+			nr_used_bmap++;
+		bmap_len = nr_pfn / BITS_PER_BYTE;
+
+		for (i = 0; i < nr_used_bmap; i++) {
+			unsigned int bmap_size = PAGE_BMAP_SIZE;
+
+			if (i + 1 == nr_used_bmap)
+				bmap_size = bmap_len - PAGE_BMAP_SIZE * i;
+			chunking_pages_from_bmap(vb, vq, start_pfn + i * PFNS_PER_PAGE_BMAP,
+				 vb->page_bitmap[i], bmap_size, false);
+		}
+		if (vb->resp_pos > 0)
+			send_resp_data(vb, vq, false);
+	} else {
+		struct scatterlist sg;
+		unsigned int len;
+
+		sg_init_one(&sg, vb->pfns, sizeof(vb->pfns[0]) * vb->num_pfns);
+
+		/* We should always be able to add one buffer to an
+		 * empty queue
+		 */
+		virtqueue_add_outbuf(vq, &sg, 1, vb, GFP_KERNEL);
+		virtqueue_kick(vq);
+		/* When host has read buffer, this completes via balloon_ack */
+		wait_event(vb->acked, virtqueue_get_buf(vq, &len));
+	}
 }
 
 static void set_page_pfns(struct virtio_balloon *vb,
@@ -138,13 +326,58 @@ static void set_page_pfns(struct virtio_balloon *vb,
 					  page_to_balloon_pfn(page) + i);
 }
 
-static unsigned fill_balloon(struct virtio_balloon *vb, size_t num)
+static void set_page_bitmap(struct virtio_balloon *vb,
+			 struct list_head *pages, struct virtqueue *vq)
+{
+	unsigned long pfn, pfn_limit;
+	struct page *page;
+	bool found;
+
+	vb->min_pfn = rounddown(vb->min_pfn, BITS_PER_LONG);
+	vb->max_pfn = roundup(vb->max_pfn, BITS_PER_LONG);
+	pfn_limit = PFNS_PER_PAGE_BMAP * vb->nr_page_bmap;
+
+	if (vb->nr_page_bmap == 1)
+		extend_page_bitmap(vb, vb->max_pfn - vb->min_pfn + 1);
+	for (pfn = vb->min_pfn; pfn < vb->max_pfn; pfn += pfn_limit) {
+		unsigned long end_pfn;
+
+		clear_page_bitmap(vb);
+		vb->start_pfn = pfn;
+		end_pfn = pfn;
+		found = false;
+		list_for_each_entry(page, pages, lru) {
+			unsigned long bmap_idx, bmap_pos, balloon_pfn;
+
+			balloon_pfn = page_to_balloon_pfn(page);
+			if (balloon_pfn < pfn || balloon_pfn >= pfn + pfn_limit)
+				continue;
+			bmap_idx = (balloon_pfn - pfn) / PFNS_PER_PAGE_BMAP;
+			bmap_pos = (balloon_pfn - pfn) % PFNS_PER_PAGE_BMAP;
+			set_bit(bmap_pos, vb->page_bitmap[bmap_idx]);
+			if (balloon_pfn > end_pfn)
+				end_pfn = balloon_pfn;
+			found = true;
+		}
+		if (found) {
+			vb->end_pfn = end_pfn;
+			tell_host(vb, vq);
+		}
+	}
+}
+
+static unsigned int fill_balloon(struct virtio_balloon *vb, size_t num)
 {
 	struct balloon_dev_info *vb_dev_info = &vb->vb_dev_info;
-	unsigned num_allocated_pages;
+	unsigned int num_allocated_pages;
+	bool chunking = virtio_has_feature(vb->vdev,
+				 VIRTIO_BALLOON_F_CHUNK_TRANSFER);
 
-	/* We can only do one array worth at a time. */
-	num = min(num, ARRAY_SIZE(vb->pfns));
+	if (chunking)
+		init_bmap_pfn_range(vb);
+	else
+		/* We can only do one array worth at a time. */
+		num = min(num, ARRAY_SIZE(vb->pfns));
 
 	mutex_lock(&vb->balloon_lock);
 	for (vb->num_pfns = 0; vb->num_pfns < num;
@@ -159,7 +392,10 @@ static unsigned fill_balloon(struct virtio_balloon *vb, size_t num)
 			msleep(200);
 			break;
 		}
-		set_page_pfns(vb, vb->pfns + vb->num_pfns, page);
+		if (chunking)
+			update_bmap_pfn_range(vb, page);
+		else
+			set_page_pfns(vb, vb->pfns + vb->num_pfns, page);
 		vb->num_pages += VIRTIO_BALLOON_PAGES_PER_PAGE;
 		if (!virtio_has_feature(vb->vdev,
 					VIRTIO_BALLOON_F_DEFLATE_ON_OOM))
@@ -168,8 +404,13 @@ static unsigned fill_balloon(struct virtio_balloon *vb, size_t num)
 
 	num_allocated_pages = vb->num_pfns;
 	/* Did we get any? */
-	if (vb->num_pfns != 0)
-		tell_host(vb, vb->inflate_vq);
+	if (vb->num_pfns != 0) {
+		if (chunking)
+			set_page_bitmap(vb, &vb_dev_info->pages,
+					vb->inflate_vq);
+		else
+			tell_host(vb, vb->inflate_vq);
+	}
 	mutex_unlock(&vb->balloon_lock);
 
 	return num_allocated_pages;
@@ -189,15 +430,20 @@ static void release_pages_balloon(struct virtio_balloon *vb,
 	}
 }
 
-static unsigned leak_balloon(struct virtio_balloon *vb, size_t num)
+static unsigned int leak_balloon(struct virtio_balloon *vb, size_t num)
 {
-	unsigned num_freed_pages;
+	unsigned int num_freed_pages;
 	struct page *page;
 	struct balloon_dev_info *vb_dev_info = &vb->vb_dev_info;
 	LIST_HEAD(pages);
+	bool chunking = virtio_has_feature(vb->vdev,
+			 VIRTIO_BALLOON_F_CHUNK_TRANSFER);
 
-	/* We can only do one array worth at a time. */
-	num = min(num, ARRAY_SIZE(vb->pfns));
+	if (chunking)
+		init_bmap_pfn_range(vb);
+	else
+		/* We can only do one array worth at a time. */
+		num = min(num, ARRAY_SIZE(vb->pfns));
 
 	mutex_lock(&vb->balloon_lock);
 	/* We can't release more pages than taken */
@@ -207,7 +453,10 @@ static unsigned leak_balloon(struct virtio_balloon *vb, size_t num)
 		page = balloon_page_dequeue(vb_dev_info);
 		if (!page)
 			break;
-		set_page_pfns(vb, vb->pfns + vb->num_pfns, page);
+		if (chunking)
+			update_bmap_pfn_range(vb, page);
+		else
+			set_page_pfns(vb, vb->pfns + vb->num_pfns, page);
 		list_add(&page->lru, &pages);
 		vb->num_pages -= VIRTIO_BALLOON_PAGES_PER_PAGE;
 	}
@@ -218,8 +467,12 @@ static unsigned leak_balloon(struct virtio_balloon *vb, size_t num)
 	 * virtio_has_feature(vdev, VIRTIO_BALLOON_F_MUST_TELL_HOST);
 	 * is true, we *have* to do it in this order
 	 */
-	if (vb->num_pfns != 0)
-		tell_host(vb, vb->deflate_vq);
+	if (vb->num_pfns != 0) {
+		if (chunking)
+			set_page_bitmap(vb, &pages, vb->deflate_vq);
+		else
+			tell_host(vb, vb->deflate_vq);
+	}
 	release_pages_balloon(vb, &pages);
 	mutex_unlock(&vb->balloon_lock);
 	return num_freed_pages;
@@ -431,6 +684,18 @@ static int init_vqs(struct virtio_balloon *vb)
 }
 
 #ifdef CONFIG_BALLOON_COMPACTION
+static void tell_host_one_page(struct virtio_balloon *vb,
+	struct virtqueue *vq, struct page *page)
+{
+	__le64 *chunk;
+
+	chunk = vb->resp_data + vb->resp_pos;
+	*chunk = cpu_to_le64((page_to_pfn(page) <<
+				VIRTIO_BALLOON_CHUNK_SIZE_SHIFT) | 1);
+	vb->resp_pos++;
+	send_resp_data(vb, vq, false);
+}
+
 /*
  * virtballoon_migratepage - perform the balloon page migration on behalf of
  *			     a compation thread.     (called under page lock)
@@ -455,6 +720,8 @@ static int virtballoon_migratepage(struct balloon_dev_info *vb_dev_info,
 	struct virtio_balloon *vb = container_of(vb_dev_info,
 			struct virtio_balloon, vb_dev_info);
 	unsigned long flags;
+	bool chunking = virtio_has_feature(vb->vdev,
+				 VIRTIO_BALLOON_F_CHUNK_TRANSFER);
 
 	/*
 	 * In order to avoid lock contention while migrating pages concurrently
@@ -475,15 +742,23 @@ static int virtballoon_migratepage(struct balloon_dev_info *vb_dev_info,
 	vb_dev_info->isolated_pages--;
 	__count_vm_event(BALLOON_MIGRATE);
 	spin_unlock_irqrestore(&vb_dev_info->pages_lock, flags);
-	vb->num_pfns = VIRTIO_BALLOON_PAGES_PER_PAGE;
-	set_page_pfns(vb, vb->pfns, newpage);
-	tell_host(vb, vb->inflate_vq);
+	if (chunking)
+		tell_host_one_page(vb, vb->inflate_vq, newpage);
+	else {
+		vb->num_pfns = VIRTIO_BALLOON_PAGES_PER_PAGE;
+		set_page_pfns(vb, vb->pfns, newpage);
+		tell_host(vb, vb->inflate_vq);
+	}
 
 	/* balloon's page migration 2nd step -- deflate "page" */
 	balloon_page_delete(page);
-	vb->num_pfns = VIRTIO_BALLOON_PAGES_PER_PAGE;
-	set_page_pfns(vb, vb->pfns, page);
-	tell_host(vb, vb->deflate_vq);
+	if (chunking)
+		tell_host_one_page(vb, vb->deflate_vq, page);
+	else {
+		vb->num_pfns = VIRTIO_BALLOON_PAGES_PER_PAGE;
+		set_page_pfns(vb, vb->pfns, page);
+		tell_host(vb, vb->deflate_vq);
+	}
 
 	mutex_unlock(&vb->balloon_lock);
 
@@ -533,6 +808,21 @@ static int virtballoon_probe(struct virtio_device *vdev)
 	spin_lock_init(&vb->stop_update_lock);
 	vb->stop_update = false;
 	vb->num_pages = 0;
+
+	vb->page_bitmap[0] = kmalloc(PAGE_BMAP_SIZE, GFP_KERNEL);
+	if (!vb->page_bitmap[0]) {
+		__virtio_clear_bit(vdev, VIRTIO_BALLOON_F_CHUNK_TRANSFER);
+	} else {
+		vb->nr_page_bmap = 1;
+		vb->resp_data = kmalloc(PAGE_BMAP_SIZE, GFP_KERNEL);
+		if (!vb->resp_data) {
+			__virtio_clear_bit(vdev,
+					VIRTIO_BALLOON_F_CHUNK_TRANSFER);
+			kfree(vb->page_bitmap[0]);
+		}
+	}
+	vb->resp_pos = 0;
+	vb->resp_buf_size = PAGE_BMAP_SIZE;
 	mutex_init(&vb->balloon_lock);
 	init_waitqueue_head(&vb->acked);
 	vb->vdev = vdev;
@@ -578,6 +868,8 @@ static int virtballoon_probe(struct virtio_device *vdev)
 out_del_vqs:
 	vdev->config->del_vqs(vdev);
 out_free_vb:
+	kfree(vb->resp_data);
+	kfree_page_bitmap(vb);
 	kfree(vb);
 out:
 	return err;
@@ -611,6 +903,8 @@ static void virtballoon_remove(struct virtio_device *vdev)
 	remove_common(vb);
 	if (vb->vb_dev_info.inode)
 		iput(vb->vb_dev_info.inode);
+	kfree_page_bitmap(vb);
+	kfree(vb->resp_data);
 	kfree(vb);
 }
 
@@ -649,6 +943,7 @@ static unsigned int features[] = {
 	VIRTIO_BALLOON_F_MUST_TELL_HOST,
 	VIRTIO_BALLOON_F_STATS_VQ,
 	VIRTIO_BALLOON_F_DEFLATE_ON_OOM,
+	VIRTIO_BALLOON_F_CHUNK_TRANSFER,
 };
 
 static struct virtio_driver virtio_balloon_driver = {
-- 
2.7.4

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

* [PATCH v7 kernel 4/5] virtio-balloon: define flags and head for host request vq
  2017-03-03  5:40 [PATCH v7 kernel 0/5] Extend virtio-balloon for fast (de)inflating & fast live migration Wei Wang
                   ` (2 preceding siblings ...)
  2017-03-03  5:40 ` [PATCH v7 kernel 3/5] virtio-balloon: implementation of VIRTIO_BALLOON_F_CHUNK_TRANSFER Wei Wang
@ 2017-03-03  5:40 ` Wei Wang
  2017-03-08  4:02   ` Michael S. Tsirkin
  2017-03-03  5:40 ` [PATCH v7 kernel 5/5] This patch contains two parts: Wei Wang
  4 siblings, 1 reply; 30+ messages in thread
From: Wei Wang @ 2017-03-03  5:40 UTC (permalink / raw)
  To: virtio-dev, kvm, qemu-devel, linux-kernel, virtualization, linux-mm
  Cc: Liang Li, Wei Wang, Andrew Morton, Mel Gorman,
	Michael S . Tsirkin, Paolo Bonzini, Cornelia Huck, Amit Shah,
	Dave Hansen, Andrea Arcangeli, David Hildenbrand, Liang Li

From: Liang Li <liang.z.li@intel.com>

Define the flags and head struct for a new host request virtual
queue. Guest can get requests from host and then responds to
them on this new virtual queue.
Host can make use of this virtqueue to request the guest to do
some operations, e.g. drop page cache, synchronize file system,
etc. The hypervisor can get some of guest's runtime information
through this virtual queue too, e.g. the guest's unused page
information, which can be used for live migration optimization.

Signed-off-by: Liang Li <liang.z.li@intel.com>
Signed-off-by: Wei Wang <wei.w.wang@intel.com>
Cc: Andrew Morton <akpm@linux-foundation.org>
Cc: Mel Gorman <mgorman@techsingularity.net>
Cc: Michael S. Tsirkin <mst@redhat.com>
Cc: Paolo Bonzini <pbonzini@redhat.com>
Cc: Cornelia Huck <cornelia.huck@de.ibm.com>
Cc: Amit Shah <amit.shah@redhat.com>
Cc: Dave Hansen <dave.hansen@intel.com>
Cc: Andrea Arcangeli <aarcange@redhat.com>
Cc: David Hildenbrand <david@redhat.com>
Cc: Liang Li <liliang324@gmail.com>
Cc: Wei Wang <wei.w.wang@intel.com>
---
 include/uapi/linux/virtio_balloon.h | 22 ++++++++++++++++++++++
 1 file changed, 22 insertions(+)

diff --git a/include/uapi/linux/virtio_balloon.h b/include/uapi/linux/virtio_balloon.h
index ed627b2..630b0ef 100644
--- a/include/uapi/linux/virtio_balloon.h
+++ b/include/uapi/linux/virtio_balloon.h
@@ -35,6 +35,7 @@
 #define VIRTIO_BALLOON_F_STATS_VQ	1 /* Memory Stats virtqueue */
 #define VIRTIO_BALLOON_F_DEFLATE_ON_OOM	2 /* Deflate balloon on OOM */
 #define VIRTIO_BALLOON_F_CHUNK_TRANSFER	3 /* Transfer pages in chunks */
+#define VIRTIO_BALLOON_F_HOST_REQ_VQ	4 /* Host request virtqueue */
 
 /* Size of a PFN in the balloon interface. */
 #define VIRTIO_BALLOON_PFN_SHIFT 12
@@ -94,4 +95,25 @@ struct virtio_balloon_resp_hdr {
 	__le32 data_len; /* Payload len in bytes */
 };
 
+enum virtio_balloon_req_id {
+	/* Get unused page information */
+	BALLOON_GET_UNUSED_PAGES,
+};
+
+enum virtio_balloon_flag {
+	/* Have more data for a request */
+	BALLOON_FLAG_CONT,
+	/* No more data for a request */
+	BALLOON_FLAG_DONE,
+};
+
+struct virtio_balloon_req_hdr {
+	/* Used to distinguish different requests */
+	__le16 cmd;
+	/* Reserved */
+	__le16 reserved[3];
+	/* Request parameter */
+	__le64 param;
+};
+
 #endif /* _LINUX_VIRTIO_BALLOON_H */
-- 
2.7.4

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

* [PATCH v7 kernel 5/5] This patch contains two parts:
  2017-03-03  5:40 [PATCH v7 kernel 0/5] Extend virtio-balloon for fast (de)inflating & fast live migration Wei Wang
                   ` (3 preceding siblings ...)
  2017-03-03  5:40 ` [PATCH v7 kernel 4/5] virtio-balloon: define flags and head for host request vq Wei Wang
@ 2017-03-03  5:40 ` Wei Wang
  2017-03-06 13:23   ` David Hildenbrand
  4 siblings, 1 reply; 30+ messages in thread
From: Wei Wang @ 2017-03-03  5:40 UTC (permalink / raw)
  To: virtio-dev, kvm, qemu-devel, linux-kernel, virtualization, linux-mm
  Cc: Liang Li, Wei Wang, Andrew Morton, Mel Gorman,
	Michael S . Tsirkin, Paolo Bonzini, Cornelia Huck, Amit Shah,
	Dave Hansen, Andrea Arcangeli, David Hildenbrand, Liang Li

From: Liang Li <liang.z.li@intel.com>

One is to add a new API to mm go get the unused page information.
The virtio balloon driver will use this new API added to get the
unused page info and send it to hypervisor(QEMU) to speed up live
migration. During sending the bitmap, some the pages may be modified
and are used by the guest, this inaccuracy can be corrected by the
dirty page logging mechanism.

One is to add support the request for vm's unused page information,
QEMU can make use of unused page information and the dirty page
logging mechanism to skip the transportation of some of these unused
pages, this is very helpful to reduce the network traffic and speed
up the live migration process.

Signed-off-by: Liang Li <liang.z.li@intel.com>
Signed-off-by: Wei Wang <wei.w.wang@intel.com>
Cc: Andrew Morton <akpm@linux-foundation.org>
Cc: Mel Gorman <mgorman@techsingularity.net>
Cc: Michael S. Tsirkin <mst@redhat.com>
Cc: Paolo Bonzini <pbonzini@redhat.com>
Cc: Cornelia Huck <cornelia.huck@de.ibm.com>
Cc: Amit Shah <amit.shah@redhat.com>
Cc: Dave Hansen <dave.hansen@intel.com>
Cc: Andrea Arcangeli <aarcange@redhat.com>
Cc: David Hildenbrand <david@redhat.com>
Cc: Liang Li <liliang324@gmail.com>
Cc: Wei Wang <wei.w.wang@intel.com>
---
 drivers/virtio/virtio_balloon.c | 137 ++++++++++++++++++++++++++++++++++++++--
 include/linux/mm.h              |   3 +
 mm/page_alloc.c                 | 120 +++++++++++++++++++++++++++++++++++
 3 files changed, 255 insertions(+), 5 deletions(-)

diff --git a/drivers/virtio/virtio_balloon.c b/drivers/virtio/virtio_balloon.c
index 4416370..9b6cf44f 100644
--- a/drivers/virtio/virtio_balloon.c
+++ b/drivers/virtio/virtio_balloon.c
@@ -66,7 +66,7 @@ struct balloon_page_chunk_ext {
 
 struct virtio_balloon {
 	struct virtio_device *vdev;
-	struct virtqueue *inflate_vq, *deflate_vq, *stats_vq;
+	struct virtqueue *inflate_vq, *deflate_vq, *stats_vq, *host_req_vq;
 
 	/* The balloon servicing is delegated to a freezable workqueue. */
 	struct work_struct update_balloon_stats_work;
@@ -95,6 +95,8 @@ struct virtio_balloon {
 	unsigned int nr_page_bmap;
 	/* Used to record the processed pfn range */
 	unsigned long min_pfn, max_pfn, start_pfn, end_pfn;
+	/* Request header */
+	struct virtio_balloon_req_hdr req_hdr;
 	/*
 	 * The pages we've told the Host we're not using are enqueued
 	 * at vb_dev_info->pages list.
@@ -549,6 +551,80 @@ static void stats_handle_request(struct virtio_balloon *vb)
 	virtqueue_kick(vq);
 }
 
+static void __send_unused_pages(struct virtio_balloon *vb,
+	unsigned long req_id, unsigned int pos, bool done)
+{
+	struct virtio_balloon_resp_hdr *hdr = &vb->resp_hdr;
+	struct virtqueue *vq = vb->host_req_vq;
+
+	vb->resp_pos = pos;
+	hdr->cmd = BALLOON_GET_UNUSED_PAGES;
+	hdr->id = req_id;
+	if (!done)
+		hdr->flag = BALLOON_FLAG_CONT;
+	else
+		hdr->flag = BALLOON_FLAG_DONE;
+
+	if (pos > 0 || done)
+		send_resp_data(vb, vq, true);
+
+}
+
+static void send_unused_pages(struct virtio_balloon *vb,
+				unsigned long req_id)
+{
+	struct scatterlist sg_in;
+	unsigned int pos = 0;
+	struct virtqueue *vq = vb->host_req_vq;
+	int ret, order;
+	struct zone *zone = NULL;
+	bool part_fill = false;
+
+	mutex_lock(&vb->balloon_lock);
+
+	for (order = MAX_ORDER - 1; order >= 0; order--) {
+		ret = mark_unused_pages(&zone, order, vb->resp_data,
+			 vb->resp_buf_size / sizeof(__le64),
+			 &pos, VIRTIO_BALLOON_CHUNK_SIZE_SHIFT, part_fill);
+		if (ret == -ENOSPC) {
+			if (pos == 0) {
+				void *new_resp_data;
+
+				new_resp_data = kmalloc(2 * vb->resp_buf_size,
+							GFP_KERNEL);
+				if (new_resp_data) {
+					kfree(vb->resp_data);
+					vb->resp_data = new_resp_data;
+					vb->resp_buf_size *= 2;
+				} else {
+					part_fill = true;
+					dev_warn(&vb->vdev->dev,
+						 "%s: part fill order: %d\n",
+						 __func__, order);
+				}
+			} else {
+				__send_unused_pages(vb, req_id, pos, false);
+				pos = 0;
+			}
+
+			if (!part_fill) {
+				order++;
+				continue;
+			}
+		} else
+			zone = NULL;
+
+		if (order == 0)
+			__send_unused_pages(vb, req_id, pos, true);
+
+	}
+
+	mutex_unlock(&vb->balloon_lock);
+	sg_init_one(&sg_in, &vb->req_hdr, sizeof(vb->req_hdr));
+	virtqueue_add_inbuf(vq, &sg_in, 1, &vb->req_hdr, GFP_KERNEL);
+	virtqueue_kick(vq);
+}
+
 static void virtballoon_changed(struct virtio_device *vdev)
 {
 	struct virtio_balloon *vb = vdev->priv;
@@ -648,18 +724,51 @@ static void update_balloon_size_func(struct work_struct *work)
 		queue_work(system_freezable_wq, work);
 }
 
+static void handle_host_request(struct virtqueue *vq)
+{
+	struct virtio_balloon *vb = vq->vdev->priv;
+	struct virtio_balloon_req_hdr *ptr_hdr;
+	unsigned int len;
+
+	ptr_hdr = virtqueue_get_buf(vb->host_req_vq, &len);
+	if (!ptr_hdr || len != sizeof(vb->req_hdr))
+		return;
+
+	switch (ptr_hdr->cmd) {
+	case BALLOON_GET_UNUSED_PAGES:
+		send_unused_pages(vb, ptr_hdr->param);
+		break;
+	default:
+		dev_warn(&vb->vdev->dev, "%s: host request %d not supported \n",
+						 __func__, ptr_hdr->cmd);
+	}
+}
+
 static int init_vqs(struct virtio_balloon *vb)
 {
-	struct virtqueue *vqs[3];
-	vq_callback_t *callbacks[] = { balloon_ack, balloon_ack, stats_request };
-	static const char * const names[] = { "inflate", "deflate", "stats" };
+	struct virtqueue *vqs[4];
+	vq_callback_t *callbacks[] = { balloon_ack, balloon_ack,
+				       stats_request, handle_host_request };
+	static const char * const names[] = { "inflate", "deflate",
+					      "stats", "host_request" };
 	int err, nvqs;
 
 	/*
 	 * We expect two virtqueues: inflate and deflate, and
 	 * optionally stat.
 	 */
-	nvqs = virtio_has_feature(vb->vdev, VIRTIO_BALLOON_F_STATS_VQ) ? 3 : 2;
+	if (virtio_has_feature(vb->vdev, VIRTIO_BALLOON_F_HOST_REQ_VQ))
+		nvqs = 4;
+	else if (virtio_has_feature(vb->vdev, VIRTIO_BALLOON_F_STATS_VQ))
+		nvqs = 3;
+	else
+		nvqs = 2;
+
+	if (!virtio_has_feature(vb->vdev, VIRTIO_BALLOON_F_STATS_VQ)) {
+		__virtio_clear_bit(vb->vdev, VIRTIO_BALLOON_F_CHUNK_TRANSFER);
+		__virtio_clear_bit(vb->vdev, VIRTIO_BALLOON_F_HOST_REQ_VQ);
+	}
+
 	err = vb->vdev->config->find_vqs(vb->vdev, nvqs, vqs, callbacks, names);
 	if (err)
 		return err;
@@ -680,6 +789,20 @@ static int init_vqs(struct virtio_balloon *vb)
 			BUG();
 		virtqueue_kick(vb->stats_vq);
 	}
+
+	if (virtio_has_feature(vb->vdev, VIRTIO_BALLOON_F_HOST_REQ_VQ)) {
+		struct scatterlist sg_in;
+
+		vb->host_req_vq = vqs[3];
+		sg_init_one(&sg_in, &vb->req_hdr, sizeof(vb->req_hdr));
+		if (virtqueue_add_inbuf(vb->host_req_vq, &sg_in, 1,
+		    &vb->req_hdr, GFP_KERNEL) < 0)
+			__virtio_clear_bit(vb->vdev,
+					VIRTIO_BALLOON_F_HOST_REQ_VQ);
+		else
+			virtqueue_kick(vb->host_req_vq);
+	}
+
 	return 0;
 }
 
@@ -812,12 +935,15 @@ static int virtballoon_probe(struct virtio_device *vdev)
 	vb->page_bitmap[0] = kmalloc(PAGE_BMAP_SIZE, GFP_KERNEL);
 	if (!vb->page_bitmap[0]) {
 		__virtio_clear_bit(vdev, VIRTIO_BALLOON_F_CHUNK_TRANSFER);
+		__virtio_clear_bit(vdev, VIRTIO_BALLOON_F_HOST_REQ_VQ);
 	} else {
 		vb->nr_page_bmap = 1;
 		vb->resp_data = kmalloc(PAGE_BMAP_SIZE, GFP_KERNEL);
 		if (!vb->resp_data) {
 			__virtio_clear_bit(vdev,
 					VIRTIO_BALLOON_F_CHUNK_TRANSFER);
+			__virtio_clear_bit(vdev,
+					VIRTIO_BALLOON_F_HOST_REQ_VQ);
 			kfree(vb->page_bitmap[0]);
 		}
 	}
@@ -944,6 +1070,7 @@ static unsigned int features[] = {
 	VIRTIO_BALLOON_F_STATS_VQ,
 	VIRTIO_BALLOON_F_DEFLATE_ON_OOM,
 	VIRTIO_BALLOON_F_CHUNK_TRANSFER,
+	VIRTIO_BALLOON_F_HOST_REQ_VQ,
 };
 
 static struct virtio_driver virtio_balloon_driver = {
diff --git a/include/linux/mm.h b/include/linux/mm.h
index b84615b..c9ad89c 100644
--- a/include/linux/mm.h
+++ b/include/linux/mm.h
@@ -1764,6 +1764,9 @@ extern void free_area_init(unsigned long * zones_size);
 extern void free_area_init_node(int nid, unsigned long * zones_size,
 		unsigned long zone_start_pfn, unsigned long *zholes_size);
 extern void free_initmem(void);
+extern int mark_unused_pages(struct zone **start_zone, int order,
+		__le64 *pages, unsigned int size, unsigned int *pos,
+		u8 len_bits, bool part_fill);
 
 /*
  * Free reserved pages within range [PAGE_ALIGN(start), end & PAGE_MASK)
diff --git a/mm/page_alloc.c b/mm/page_alloc.c
index f3e0c69..f0573b1 100644
--- a/mm/page_alloc.c
+++ b/mm/page_alloc.c
@@ -4498,6 +4498,126 @@ void show_free_areas(unsigned int filter)
 	show_swap_cache_info();
 }
 
+static int __mark_unused_pages(struct zone *zone, int order,
+		__le64 *pages, unsigned int size, unsigned int *pos,
+		u8 len_bits, bool part_fill)
+{
+	unsigned long pfn, flags;
+	int t, ret = 0;
+	struct list_head *curr;
+	__le64 *range;
+
+	if (zone_is_empty(zone))
+		return 0;
+
+	spin_lock_irqsave(&zone->lock, flags);
+
+	if (*pos + zone->free_area[order].nr_free > size && !part_fill) {
+		ret = -ENOSPC;
+		goto out;
+	}
+	for (t = 0; t < MIGRATE_TYPES; t++) {
+		list_for_each(curr, &zone->free_area[order].free_list[t]) {
+			pfn = page_to_pfn(list_entry(curr, struct page, lru));
+			range = pages + *pos;
+			if (order < len_bits) {
+				if (*pos + 1 > size) {
+					ret = -ENOSPC;
+					goto out;
+				}
+				*range = cpu_to_le64((pfn << len_bits)
+							| 1 << order);
+				*pos += 1;
+			} else {
+				if (*pos + 2 > size) {
+					ret = -ENOSPC;
+					goto out;
+				}
+				*range = cpu_to_le64((pfn << len_bits) | 0);
+				*(range + 1) = cpu_to_le64(1 << order);
+				*pos += 2;
+			}
+		}
+	}
+
+out:
+	spin_unlock_irqrestore(&zone->lock, flags);
+
+	return ret;
+}
+
+/*
+ * During live migration, page is discardable unless it's content
+ * is needed by the system.
+ * mark_unused_pages provides an API to mark the unused pages, these
+ * unused pages can be discarded if there is no modification since
+ * the request. Some other mechanism, like the dirty page logging
+ * can be used to track the modification.
+ *
+ * This function scans the free page list to mark the unused pages
+ * with the specified order, and set the corresponding range element
+ * in the array 'pages' if unused pages are found for the specified
+ * order.
+ *
+ * @start_zone: zone to start the mark operation.
+ * @order: page order to mark.
+ * @pages: array to save the unused page info.
+ * @size: size of array pages.
+ * @pos: offset in the array to save the page info.
+ * @len_bits: bits for the length field of the range.
+ * @part_fill: indicate if partial fill is used.
+ *
+ * return -EINVAL if parameter is invalid
+ * return -ENOSPC when bitmap can't contain the pages
+ * return 0 when sccess
+ */
+int mark_unused_pages(struct zone **start_zone, int order,
+	__le64 *pages, unsigned int size, unsigned int *pos,
+	u8 len_bits, bool part_fill)
+{
+	struct zone *zone;
+	int ret = 0;
+	bool skip_check = false;
+
+	/* make sure all the parameters are valid */
+	if (pages == NULL || pos == NULL || *pos < 0
+		|| order >= MAX_ORDER || len_bits > 64)
+		return -EINVAL;
+	if (*start_zone != NULL) {
+		bool found = false;
+
+		for_each_populated_zone(zone) {
+			if (zone != *start_zone)
+				continue;
+			found = true;
+			break;
+		}
+		if (!found)
+			return -EINVAL;
+	} else
+		skip_check = true;
+
+	for_each_populated_zone(zone) {
+		/* Start from *start_zone if it's not NULL */
+		if (!skip_check) {
+			if (*start_zone != zone)
+				continue;
+			else
+				skip_check = true;
+		}
+		ret = __mark_unused_pages(zone, order, pages, size,
+					pos, len_bits, part_fill);
+		if (ret < 0) {
+			/* record the failed zone */
+			*start_zone = zone;
+			break;
+		}
+	}
+
+	return ret;
+}
+EXPORT_SYMBOL(mark_unused_pages);
+
 static void zoneref_set_zone(struct zone *zone, struct zoneref *zoneref)
 {
 	zoneref->zone = zone;
-- 
2.7.4

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

* Re: [PATCH v7 kernel 5/5] This patch contains two parts:
  2017-03-03  5:40 ` [PATCH v7 kernel 5/5] This patch contains two parts: Wei Wang
@ 2017-03-06 13:23   ` David Hildenbrand
  2017-03-09  7:04     ` Wei Wang
  0 siblings, 1 reply; 30+ messages in thread
From: David Hildenbrand @ 2017-03-06 13:23 UTC (permalink / raw)
  To: Wei Wang, virtio-dev, kvm, qemu-devel, linux-kernel,
	virtualization, linux-mm
  Cc: Liang Li, Andrew Morton, Mel Gorman, Michael S . Tsirkin,
	Paolo Bonzini, Cornelia Huck, Amit Shah, Dave Hansen,
	Andrea Arcangeli, Liang Li

Am 03.03.2017 um 06:40 schrieb Wei Wang:
> From: Liang Li <liang.z.li@intel.com>

I'd prefer to split this into two parts then and to create proper subjects.

If I remember correctly, the general concept was accepted by most reviewers.

> 
> One is to add a new API to mm go get the unused page information.
> The virtio balloon driver will use this new API added to get the
> unused page info and send it to hypervisor(QEMU) to speed up live
> migration. During sending the bitmap, some the pages may be modified
> and are used by the guest, this inaccuracy can be corrected by the
> dirty page logging mechanism.
> 
> One is to add support the request for vm's unused page information,
> QEMU can make use of unused page information and the dirty page
> logging mechanism to skip the transportation of some of these unused
> pages, this is very helpful to reduce the network traffic and speed
> up the live migration process.

-- 
Thanks,

David

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

* Re: [PATCH v7 kernel 3/5] virtio-balloon: implementation of VIRTIO_BALLOON_F_CHUNK_TRANSFER
  2017-03-03  5:40 ` [PATCH v7 kernel 3/5] virtio-balloon: implementation of VIRTIO_BALLOON_F_CHUNK_TRANSFER Wei Wang
@ 2017-03-08  4:01   ` Michael S. Tsirkin
  2017-03-10 10:02     ` [virtio-dev] " Wei Wang
  2017-03-09 14:14   ` Matthew Wilcox
  1 sibling, 1 reply; 30+ messages in thread
From: Michael S. Tsirkin @ 2017-03-08  4:01 UTC (permalink / raw)
  To: Wei Wang
  Cc: virtio-dev, kvm, qemu-devel, linux-kernel, virtualization,
	linux-mm, Liang Li, Paolo Bonzini, Cornelia Huck, Amit Shah,
	Dave Hansen, Andrea Arcangeli, David Hildenbrand, Liang Li

On Fri, Mar 03, 2017 at 01:40:28PM +0800, Wei Wang wrote:
> From: Liang Li <liang.z.li@intel.com>
> 
> The implementation of the current virtio-balloon is not very
> efficient, because the pages are transferred to the host one by one.
> Here is the breakdown of the time in percentage spent on each
> step of the balloon inflating process (inflating 7GB of an 8GB
> idle guest).
> 
> 1) allocating pages (6.5%)
> 2) sending PFNs to host (68.3%)
> 3) address translation (6.1%)
> 4) madvise (19%)
> 
> It takes about 4126ms for the inflating process to complete.
> The above profiling shows that the bottlenecks are stage 2)
> and stage 4).
> 
> This patch optimizes step 2) by transfering pages to the host in
> chunks. A chunk consists of guest physically continuous pages, and
> it is offered to the host via a base PFN (i.e. the start PFN of
> those physically continuous pages) and the size (i.e. the total
> number of the pages). A normal chunk is formated as below:
> -----------------------------------------------
> |  Base (52 bit)               | Size (12 bit)|
> -----------------------------------------------
> For large size chunks, an extended chunk format is used:
> -----------------------------------------------
> |                 Base (64 bit)               |
> -----------------------------------------------
> -----------------------------------------------
> |                 Size (64 bit)               |
> -----------------------------------------------
> 
> By doing so, step 4) can also be optimized by doing address
> translation and madvise() in chunks rather than page by page.
> 
> This optimization requires the negotation of a new feature bit,
> VIRTIO_BALLOON_F_CHUNK_TRANSFER.
> 
> With this new feature, the above ballooning process takes ~590ms
> resulting in an improvement of ~85%.
> 
> TODO: optimize stage 1) by allocating/freeing a chunk of pages
> instead of a single page each time.
> 
> Signed-off-by: Liang Li <liang.z.li@intel.com>
> Signed-off-by: Wei Wang <wei.w.wang@intel.com>
> Suggested-by: Michael S. Tsirkin <mst@redhat.com>
> Cc: Michael S. Tsirkin <mst@redhat.com>
> Cc: Paolo Bonzini <pbonzini@redhat.com>
> Cc: Cornelia Huck <cornelia.huck@de.ibm.com>
> Cc: Amit Shah <amit.shah@redhat.com>
> Cc: Dave Hansen <dave.hansen@intel.com>
> Cc: Andrea Arcangeli <aarcange@redhat.com>
> Cc: David Hildenbrand <david@redhat.com>
> Cc: Liang Li <liliang324@gmail.com>
> Cc: Wei Wang <wei.w.wang@intel.com>

Does this pass sparse? I see some endian-ness issues here.

> ---
>  drivers/virtio/virtio_balloon.c | 351 ++++++++++++++++++++++++++++++++++++----
>  1 file changed, 323 insertions(+), 28 deletions(-)
> 
> diff --git a/drivers/virtio/virtio_balloon.c b/drivers/virtio/virtio_balloon.c
> index f59cb4f..4416370 100644
> --- a/drivers/virtio/virtio_balloon.c
> +++ b/drivers/virtio/virtio_balloon.c
> @@ -42,6 +42,10 @@
>  #define OOM_VBALLOON_DEFAULT_PAGES 256
>  #define VIRTBALLOON_OOM_NOTIFY_PRIORITY 80
>  
> +#define PAGE_BMAP_SIZE	(8 * PAGE_SIZE)
> +#define PFNS_PER_PAGE_BMAP	(PAGE_BMAP_SIZE * BITS_PER_BYTE)
> +#define PAGE_BMAP_COUNT_MAX	32
> +
>  static int oom_pages = OOM_VBALLOON_DEFAULT_PAGES;
>  module_param(oom_pages, int, S_IRUSR | S_IWUSR);
>  MODULE_PARM_DESC(oom_pages, "pages to free on OOM");
> @@ -50,6 +54,16 @@ MODULE_PARM_DESC(oom_pages, "pages to free on OOM");
>  static struct vfsmount *balloon_mnt;
>  #endif
>  
> +struct balloon_page_chunk {
> +	__le64 base : 52;
> +	__le64 size : 12;
> +};
> +
> +struct balloon_page_chunk_ext {
> +	__le64 base;
> +	__le64 size;
> +};
> +
>  struct virtio_balloon {
>  	struct virtio_device *vdev;
>  	struct virtqueue *inflate_vq, *deflate_vq, *stats_vq;
> @@ -67,6 +81,20 @@ struct virtio_balloon {
>  
>  	/* Number of balloon pages we've told the Host we're not using. */
>  	unsigned int num_pages;
> +	/* Pointer to the response header. */
> +	struct virtio_balloon_resp_hdr resp_hdr;
> +	/* Pointer to the start address of response data. */
> +	__le64 *resp_data;
> +	/* Size of response data buffer. */
> +	unsigned int resp_buf_size;
> +	/* Pointer offset of the response data. */
> +	unsigned int resp_pos;
> +	/* Bitmap used to save the pfns info */
> +	unsigned long *page_bitmap[PAGE_BMAP_COUNT_MAX];
> +	/* Number of split page bitmaps */
> +	unsigned int nr_page_bmap;
> +	/* Used to record the processed pfn range */
> +	unsigned long min_pfn, max_pfn, start_pfn, end_pfn;
>  	/*
>  	 * The pages we've told the Host we're not using are enqueued
>  	 * at vb_dev_info->pages list.
> @@ -110,20 +138,180 @@ static void balloon_ack(struct virtqueue *vq)
>  	wake_up(&vb->acked);
>  }
>  
> -static void tell_host(struct virtio_balloon *vb, struct virtqueue *vq)
> +static inline void init_bmap_pfn_range(struct virtio_balloon *vb)
>  {
> -	struct scatterlist sg;
> +	vb->min_pfn = ULONG_MAX;
> +	vb->max_pfn = 0;
> +}
> +
> +static inline void update_bmap_pfn_range(struct virtio_balloon *vb,
> +				 struct page *page)
> +{
> +	unsigned long balloon_pfn = page_to_balloon_pfn(page);
> +
> +	vb->min_pfn = min(balloon_pfn, vb->min_pfn);
> +	vb->max_pfn = max(balloon_pfn, vb->max_pfn);
> +}
> +
> +static void extend_page_bitmap(struct virtio_balloon *vb,
> +				unsigned long nr_pfn)


what exactly does this do?

> +{
> +	int i, bmap_count;
> +	unsigned long bmap_len;
> +
> +	bmap_len = ALIGN(nr_pfn, BITS_PER_LONG) / BITS_PER_BYTE;
> +	bmap_len = ALIGN(bmap_len, PAGE_BMAP_SIZE);
> +	bmap_count = min((int)(bmap_len / PAGE_BMAP_SIZE),
> +				 PAGE_BMAP_COUNT_MAX);
> +
> +	for (i = 1; i < bmap_count; i++) {
> +		vb->page_bitmap[i] = kmalloc(PAGE_BMAP_SIZE, GFP_KERNEL);
> +		if (vb->page_bitmap[i])
> +			vb->nr_page_bmap++;
> +		else
> +			break;

seems to fail silently ...

> +	}
> +}
> +
> +static void free_extended_page_bitmap(struct virtio_balloon *vb)
> +{
> +	int i, bmap_count = vb->nr_page_bmap;
> +
> +	for (i = 1; i < bmap_count; i++) {
> +		kfree(vb->page_bitmap[i]);
> +		vb->page_bitmap[i] = NULL;
> +		vb->nr_page_bmap--;
> +	}
> +}
> +
> +static void kfree_page_bitmap(struct virtio_balloon *vb)
> +{
> +	int i;
> +
> +	for (i = 0; i < vb->nr_page_bmap; i++)
> +		kfree(vb->page_bitmap[i]);
> +}
> +

A very confusing name. pls don't start with kfree.

> +static void clear_page_bitmap(struct virtio_balloon *vb)
> +{
> +	int i;
> +
> +	for (i = 0; i < vb->nr_page_bmap; i++)
> +		memset(vb->page_bitmap[i], 0, PAGE_BMAP_SIZE);
> +}
> +
> +static void send_resp_data(struct virtio_balloon *vb, struct virtqueue *vq,
> +			bool busy_wait)
> +{
> +	struct scatterlist sg[2];
> +	struct virtio_balloon_resp_hdr *hdr = &vb->resp_hdr;
>  	unsigned int len;
>  
> -	sg_init_one(&sg, vb->pfns, sizeof(vb->pfns[0]) * vb->num_pfns);
> +	len = hdr->data_len = vb->resp_pos * sizeof(__le64);
> +	sg_init_table(sg, 2);
> +	sg_set_buf(&sg[0], hdr, sizeof(struct virtio_balloon_resp_hdr));
> +	sg_set_buf(&sg[1], vb->resp_data, len);
> +
> +	if (virtqueue_add_outbuf(vq, sg, 2, vb, GFP_KERNEL) == 0) {

!virtqueue_add_outbuf

Also how about making this header linear with pfns?
This way you can use 1 s/g and avoid mallocs.

> +		virtqueue_kick(vq);
> +		if (busy_wait)
> +			while (!virtqueue_get_buf(vq, &len)
> +				&& !virtqueue_is_broken(vq))
> +				cpu_relax();
> +		else
> +			wait_event(vb->acked, virtqueue_get_buf(vq, &len));
> +		vb->resp_pos = 0;
> +		free_extended_page_bitmap(vb);
> +	}
> +}
>  
> -	/* We should always be able to add one buffer to an empty queue. */
> -	virtqueue_add_outbuf(vq, &sg, 1, vb, GFP_KERNEL);
> -	virtqueue_kick(vq);
> +static void do_set_resp_bitmap(struct virtio_balloon *vb, unsigned long base,
> +			       int size)
> +{
> +	/* Use the extented chunk format if the size is too large */
> +	if (size > (1 << VIRTIO_BALLOON_CHUNK_SIZE_SHIFT)) {
> +		struct balloon_page_chunk_ext *chunk_ext =
> +			(struct balloon_page_chunk_ext *)(vb->resp_data + vb->resp_pos);
> +		chunk_ext->base = cpu_to_le64(base);
> +		chunk_ext->size = cpu_to_le64(size);
> +		vb->resp_pos += sizeof(vb->resp_pos) / sizeof(*chunk_ext);
> +	} else {
> +		struct balloon_page_chunk *chunk =
> +			(struct balloon_page_chunk *)(vb->resp_data + vb->resp_pos);
> +		chunk->base = cpu_to_le64(base);
> +		chunk->size = cpu_to_le16(size);
> +		vb->resp_pos += sizeof(vb->resp_pos) / sizeof(*chunk);
> +	}
> +}
>  
> -	/* When host has read buffer, this completes via balloon_ack */
> -	wait_event(vb->acked, virtqueue_get_buf(vq, &len));
> +static void chunking_pages_from_bmap(struct virtio_balloon *vb,
> +				     struct virtqueue *vq,
> +				     unsigned long start_pfn, unsigned long *bitmap,
> +				     unsigned long len, bool busy_wait)
> +{
> +	unsigned long pos = 0, end = len * BITS_PER_BYTE;
> +
> +	while (pos < end) {
> +		unsigned long one = find_next_bit(bitmap, end, pos);
> +
> +		if (one < end) {
> +			unsigned long chunk_size, zero;
> +
> +			zero = find_next_zero_bit(bitmap, end, one + 1);
> +			if (zero >= end)
> +				chunk_size = end - one;
> +			else
> +				chunk_size = zero - one;
> +			if (chunk_size) {
> +				if ((vb->resp_pos + 2) * sizeof(__le64) >
> +						vb->resp_buf_size)

why + 2? where does sizeof come from?
Any chance this can be cleaned up using structs sizeof etc?
or at least commented?
How am I supposed to figure out all this convoluted pointer math?

> +					send_resp_data(vb, vq, busy_wait);
> +				do_set_resp_bitmap(vb, start_pfn + one,	chunk_size);
> +			}
> +			pos = one + chunk_size;
> +		} else
> +			break;
> +	}
> +}
>  
> +static void tell_host(struct virtio_balloon *vb, struct virtqueue *vq)
> +{
> +	if (virtio_has_feature(vb->vdev, VIRTIO_BALLOON_F_CHUNK_TRANSFER)) {
> +		int nr_pfn, nr_used_bmap, i;
> +		unsigned long start_pfn, bmap_len;
> +
> +		start_pfn = vb->start_pfn;
> +		nr_pfn = vb->end_pfn - start_pfn + 1;
> +		nr_pfn = roundup(nr_pfn, BITS_PER_LONG);
> +		nr_used_bmap = nr_pfn / PFNS_PER_PAGE_BMAP;

better name here including "page"?
And bmaps instead of bmap?

> +		if (nr_pfn % PFNS_PER_PAGE_BMAP)
> +			nr_used_bmap++;

Pls just use macros like ALIGN or roundup?

> +		bmap_len = nr_pfn / BITS_PER_BYTE;

I guess we need to round up.

> +
> +		for (i = 0; i < nr_used_bmap; i++) {
> +			unsigned int bmap_size = PAGE_BMAP_SIZE;
> +
> +			if (i + 1 == nr_used_bmap)
> +				bmap_size = bmap_len - PAGE_BMAP_SIZE * i;

So last one is treated speacially? why?

> +			chunking_pages_from_bmap(vb, vq, start_pfn + i * PFNS_PER_PAGE_BMAP,
> +				 vb->page_bitmap[i], bmap_size, false);
> +		}
> +		if (vb->resp_pos > 0)
> +			send_resp_data(vb, vq, false);
> +	} else {
> +		struct scatterlist sg;
> +		unsigned int len;
> +
> +		sg_init_one(&sg, vb->pfns, sizeof(vb->pfns[0]) * vb->num_pfns);
> +
> +		/* We should always be able to add one buffer to an
> +		 * empty queue
> +		 */

/*
 * Multi-line comments
 * like this.
 */

> +		virtqueue_add_outbuf(vq, &sg, 1, vb, GFP_KERNEL);
> +		virtqueue_kick(vq);
> +		/* When host has read buffer, this completes via balloon_ack */
> +		wait_event(vb->acked, virtqueue_get_buf(vq, &len));
> +	}
>  }
>  
>  static void set_page_pfns(struct virtio_balloon *vb,
> @@ -138,13 +326,58 @@ static void set_page_pfns(struct virtio_balloon *vb,
>  					  page_to_balloon_pfn(page) + i);
>  }
>  
> -static unsigned fill_balloon(struct virtio_balloon *vb, size_t num)
> +static void set_page_bitmap(struct virtio_balloon *vb,
> +			 struct list_head *pages, struct virtqueue *vq)

align on ( pls.

> +{
> +	unsigned long pfn, pfn_limit;
> +	struct page *page;
> +	bool found;
> +
> +	vb->min_pfn = rounddown(vb->min_pfn, BITS_PER_LONG);
> +	vb->max_pfn = roundup(vb->max_pfn, BITS_PER_LONG);
> +	pfn_limit = PFNS_PER_PAGE_BMAP * vb->nr_page_bmap;
> +
> +	if (vb->nr_page_bmap == 1)
> +		extend_page_bitmap(vb, vb->max_pfn - vb->min_pfn + 1);

how is 1 special?

> +	for (pfn = vb->min_pfn; pfn < vb->max_pfn; pfn += pfn_limit) {
> +		unsigned long end_pfn;
> +
> +		clear_page_bitmap(vb);
> +		vb->start_pfn = pfn;
> +		end_pfn = pfn;
> +		found = false;
> +		list_for_each_entry(page, pages, lru) {
> +			unsigned long bmap_idx, bmap_pos, balloon_pfn;
> +
> +			balloon_pfn = page_to_balloon_pfn(page);
> +			if (balloon_pfn < pfn || balloon_pfn >= pfn + pfn_limit)
> +				continue;
> +			bmap_idx = (balloon_pfn - pfn) / PFNS_PER_PAGE_BMAP;
> +			bmap_pos = (balloon_pfn - pfn) % PFNS_PER_PAGE_BMAP;
> +			set_bit(bmap_pos, vb->page_bitmap[bmap_idx]);
> +			if (balloon_pfn > end_pfn)
> +				end_pfn = balloon_pfn;
> +			found = true;
> +		}
> +		if (found) {
> +			vb->end_pfn = end_pfn;
> +			tell_host(vb, vq);
> +		}
> +	}
> +}
> +
> +static unsigned int fill_balloon(struct virtio_balloon *vb, size_t num)
>  {
>  	struct balloon_dev_info *vb_dev_info = &vb->vb_dev_info;
> -	unsigned num_allocated_pages;
> +	unsigned int num_allocated_pages;
> +	bool chunking = virtio_has_feature(vb->vdev,
> +				 VIRTIO_BALLOON_F_CHUNK_TRANSFER);
>  
> -	/* We can only do one array worth at a time. */
> -	num = min(num, ARRAY_SIZE(vb->pfns));
> +	if (chunking)
> +		init_bmap_pfn_range(vb);
> +	else
> +		/* We can only do one array worth at a time. */
> +		num = min(num, ARRAY_SIZE(vb->pfns));
>  
>  	mutex_lock(&vb->balloon_lock);
>  	for (vb->num_pfns = 0; vb->num_pfns < num;
> @@ -159,7 +392,10 @@ static unsigned fill_balloon(struct virtio_balloon *vb, size_t num)
>  			msleep(200);
>  			break;
>  		}
> -		set_page_pfns(vb, vb->pfns + vb->num_pfns, page);
> +		if (chunking)
> +			update_bmap_pfn_range(vb, page);
> +		else
> +			set_page_pfns(vb, vb->pfns + vb->num_pfns, page);
>  		vb->num_pages += VIRTIO_BALLOON_PAGES_PER_PAGE;
>  		if (!virtio_has_feature(vb->vdev,
>  					VIRTIO_BALLOON_F_DEFLATE_ON_OOM))
> @@ -168,8 +404,13 @@ static unsigned fill_balloon(struct virtio_balloon *vb, size_t num)
>  
>  	num_allocated_pages = vb->num_pfns;
>  	/* Did we get any? */
> -	if (vb->num_pfns != 0)
> -		tell_host(vb, vb->inflate_vq);
> +	if (vb->num_pfns != 0) {
> +		if (chunking)
> +			set_page_bitmap(vb, &vb_dev_info->pages,
> +					vb->inflate_vq);
> +		else
> +			tell_host(vb, vb->inflate_vq);
> +	}
>  	mutex_unlock(&vb->balloon_lock);
>  
>  	return num_allocated_pages;
> @@ -189,15 +430,20 @@ static void release_pages_balloon(struct virtio_balloon *vb,
>  	}
>  }
>  
> -static unsigned leak_balloon(struct virtio_balloon *vb, size_t num)
> +static unsigned int leak_balloon(struct virtio_balloon *vb, size_t num)
>  {
> -	unsigned num_freed_pages;
> +	unsigned int num_freed_pages;
>  	struct page *page;
>  	struct balloon_dev_info *vb_dev_info = &vb->vb_dev_info;
>  	LIST_HEAD(pages);
> +	bool chunking = virtio_has_feature(vb->vdev,
> +			 VIRTIO_BALLOON_F_CHUNK_TRANSFER);

Align on ( pls.

>  
> -	/* We can only do one array worth at a time. */
> -	num = min(num, ARRAY_SIZE(vb->pfns));
> +	if (chunking)
> +		init_bmap_pfn_range(vb);
> +	else
> +		/* We can only do one array worth at a time. */
> +		num = min(num, ARRAY_SIZE(vb->pfns));
>  
>  	mutex_lock(&vb->balloon_lock);
>  	/* We can't release more pages than taken */
> @@ -207,7 +453,10 @@ static unsigned leak_balloon(struct virtio_balloon *vb, size_t num)
>  		page = balloon_page_dequeue(vb_dev_info);
>  		if (!page)
>  			break;
> -		set_page_pfns(vb, vb->pfns + vb->num_pfns, page);
> +		if (chunking)
> +			update_bmap_pfn_range(vb, page);
> +		else
> +			set_page_pfns(vb, vb->pfns + vb->num_pfns, page);
>  		list_add(&page->lru, &pages);
>  		vb->num_pages -= VIRTIO_BALLOON_PAGES_PER_PAGE;
>  	}
> @@ -218,8 +467,12 @@ static unsigned leak_balloon(struct virtio_balloon *vb, size_t num)
>  	 * virtio_has_feature(vdev, VIRTIO_BALLOON_F_MUST_TELL_HOST);
>  	 * is true, we *have* to do it in this order
>  	 */
> -	if (vb->num_pfns != 0)
> -		tell_host(vb, vb->deflate_vq);
> +	if (vb->num_pfns != 0) {
> +		if (chunking)
> +			set_page_bitmap(vb, &pages, vb->deflate_vq);
> +		else
> +			tell_host(vb, vb->deflate_vq);
> +	}
>  	release_pages_balloon(vb, &pages);
>  	mutex_unlock(&vb->balloon_lock);
>  	return num_freed_pages;
> @@ -431,6 +684,18 @@ static int init_vqs(struct virtio_balloon *vb)
>  }
>  
>  #ifdef CONFIG_BALLOON_COMPACTION
> +static void tell_host_one_page(struct virtio_balloon *vb,
> +	struct virtqueue *vq, struct page *page)
> +{
> +	__le64 *chunk;
> +
> +	chunk = vb->resp_data + vb->resp_pos;
> +	*chunk = cpu_to_le64((page_to_pfn(page) <<
> +				VIRTIO_BALLOON_CHUNK_SIZE_SHIFT) | 1);
> +	vb->resp_pos++;
> +	send_resp_data(vb, vq, false);
> +}
> +
>  /*
>   * virtballoon_migratepage - perform the balloon page migration on behalf of
>   *			     a compation thread.     (called under page lock)
> @@ -455,6 +720,8 @@ static int virtballoon_migratepage(struct balloon_dev_info *vb_dev_info,
>  	struct virtio_balloon *vb = container_of(vb_dev_info,
>  			struct virtio_balloon, vb_dev_info);
>  	unsigned long flags;
> +	bool chunking = virtio_has_feature(vb->vdev,
> +				 VIRTIO_BALLOON_F_CHUNK_TRANSFER);
>  
>  	/*
>  	 * In order to avoid lock contention while migrating pages concurrently
> @@ -475,15 +742,23 @@ static int virtballoon_migratepage(struct balloon_dev_info *vb_dev_info,
>  	vb_dev_info->isolated_pages--;
>  	__count_vm_event(BALLOON_MIGRATE);
>  	spin_unlock_irqrestore(&vb_dev_info->pages_lock, flags);
> -	vb->num_pfns = VIRTIO_BALLOON_PAGES_PER_PAGE;
> -	set_page_pfns(vb, vb->pfns, newpage);
> -	tell_host(vb, vb->inflate_vq);
> +	if (chunking)
> +		tell_host_one_page(vb, vb->inflate_vq, newpage);
> +	else {

Ad {} around 1st block too pls.

> +		vb->num_pfns = VIRTIO_BALLOON_PAGES_PER_PAGE;
> +		set_page_pfns(vb, vb->pfns, newpage);
> +		tell_host(vb, vb->inflate_vq);
> +	}
>  
>  	/* balloon's page migration 2nd step -- deflate "page" */
>  	balloon_page_delete(page);
> -	vb->num_pfns = VIRTIO_BALLOON_PAGES_PER_PAGE;
> -	set_page_pfns(vb, vb->pfns, page);
> -	tell_host(vb, vb->deflate_vq);
> +	if (chunking)
> +		tell_host_one_page(vb, vb->deflate_vq, page);


and here

> +	else {
> +		vb->num_pfns = VIRTIO_BALLOON_PAGES_PER_PAGE;
> +		set_page_pfns(vb, vb->pfns, page);
> +		tell_host(vb, vb->deflate_vq);
> +	}
>  
>  	mutex_unlock(&vb->balloon_lock);
>  
> @@ -533,6 +808,21 @@ static int virtballoon_probe(struct virtio_device *vdev)
>  	spin_lock_init(&vb->stop_update_lock);
>  	vb->stop_update = false;
>  	vb->num_pages = 0;
> +
> +	vb->page_bitmap[0] = kmalloc(PAGE_BMAP_SIZE, GFP_KERNEL);
> +	if (!vb->page_bitmap[0]) {
> +		__virtio_clear_bit(vdev, VIRTIO_BALLOON_F_CHUNK_TRANSFER);
> +	} else {
> +		vb->nr_page_bmap = 1;
> +		vb->resp_data = kmalloc(PAGE_BMAP_SIZE, GFP_KERNEL);
> +		if (!vb->resp_data) {
> +			__virtio_clear_bit(vdev,
> +					VIRTIO_BALLOON_F_CHUNK_TRANSFER);

align on ( pls

> +			kfree(vb->page_bitmap[0]);
> +		}
> +	}
> +	vb->resp_pos = 0;
> +	vb->resp_buf_size = PAGE_BMAP_SIZE;
>  	mutex_init(&vb->balloon_lock);
>  	init_waitqueue_head(&vb->acked);
>  	vb->vdev = vdev;
> @@ -578,6 +868,8 @@ static int virtballoon_probe(struct virtio_device *vdev)
>  out_del_vqs:
>  	vdev->config->del_vqs(vdev);
>  out_free_vb:
> +	kfree(vb->resp_data);
> +	kfree_page_bitmap(vb);
>  	kfree(vb);
>  out:
>  	return err;
> @@ -611,6 +903,8 @@ static void virtballoon_remove(struct virtio_device *vdev)
>  	remove_common(vb);
>  	if (vb->vb_dev_info.inode)
>  		iput(vb->vb_dev_info.inode);
> +	kfree_page_bitmap(vb);
> +	kfree(vb->resp_data);
>  	kfree(vb);
>  }
>  
> @@ -649,6 +943,7 @@ static unsigned int features[] = {
>  	VIRTIO_BALLOON_F_MUST_TELL_HOST,
>  	VIRTIO_BALLOON_F_STATS_VQ,
>  	VIRTIO_BALLOON_F_DEFLATE_ON_OOM,
> +	VIRTIO_BALLOON_F_CHUNK_TRANSFER,
>  };
>  
>  static struct virtio_driver virtio_balloon_driver = {
> -- 
> 2.7.4

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

* Re: [PATCH v7 kernel 2/5] virtio-balloon: VIRTIO_BALLOON_F_CHUNK_TRANSFER
  2017-03-03  5:40 ` [PATCH v7 kernel 2/5] virtio-balloon: VIRTIO_BALLOON_F_CHUNK_TRANSFER Wei Wang
@ 2017-03-08  4:01   ` Michael S. Tsirkin
  2017-03-09  7:12     ` Wei Wang
  0 siblings, 1 reply; 30+ messages in thread
From: Michael S. Tsirkin @ 2017-03-08  4:01 UTC (permalink / raw)
  To: Wei Wang
  Cc: virtio-dev, kvm, qemu-devel, linux-kernel, virtualization,
	linux-mm, Liang Li, Paolo Bonzini, Cornelia Huck, Amit Shah,
	Dave Hansen, Andrea Arcangeli, David Hildenbrand, Liang Li

On Fri, Mar 03, 2017 at 01:40:27PM +0800, Wei Wang wrote:
> From: Liang Li <liang.z.li@intel.com>
> 
> Add a new feature bit, VIRTIO_BALLOON_F_CHUNK_TRANSFER. Please check
> the implementation patch commit for details about this feature.


better squash into next patch.

> Signed-off-by: Liang Li <liang.z.li@intel.com>
> Signed-off-by: Wei Wang <wei.w.wang@intel.com>
> Cc: Michael S. Tsirkin <mst@redhat.com>
> Cc: Paolo Bonzini <pbonzini@redhat.com>
> Cc: Cornelia Huck <cornelia.huck@de.ibm.com>
> Cc: Amit Shah <amit.shah@redhat.com>
> Cc: Dave Hansen <dave.hansen@intel.com>
> Cc: Andrea Arcangeli <aarcange@redhat.com>
> Cc: David Hildenbrand <david@redhat.com>
> Cc: Liang Li <liliang324@gmail.com>
> Cc: Wei Wang <wei.w.wang@intel.com>
> ---
>  include/uapi/linux/virtio_balloon.h | 12 ++++++++++++
>  1 file changed, 12 insertions(+)
> 
> diff --git a/include/uapi/linux/virtio_balloon.h b/include/uapi/linux/virtio_balloon.h
> index 343d7dd..ed627b2 100644
> --- a/include/uapi/linux/virtio_balloon.h
> +++ b/include/uapi/linux/virtio_balloon.h
> @@ -34,10 +34,14 @@
>  #define VIRTIO_BALLOON_F_MUST_TELL_HOST	0 /* Tell before reclaiming pages */
>  #define VIRTIO_BALLOON_F_STATS_VQ	1 /* Memory Stats virtqueue */
>  #define VIRTIO_BALLOON_F_DEFLATE_ON_OOM	2 /* Deflate balloon on OOM */
> +#define VIRTIO_BALLOON_F_CHUNK_TRANSFER	3 /* Transfer pages in chunks */
>  
>  /* Size of a PFN in the balloon interface. */
>  #define VIRTIO_BALLOON_PFN_SHIFT 12
>  
> +/* Shift to get a chunk size */
> +#define VIRTIO_BALLOON_CHUNK_SIZE_SHIFT 12
> +
>  struct virtio_balloon_config {
>  	/* Number of pages host wants Guest to give up. */
>  	__u32 num_pages;
> @@ -82,4 +86,12 @@ struct virtio_balloon_stat {
>  	__virtio64 val;
>  } __attribute__((packed));
>  
> +/* Response header structure */
> +struct virtio_balloon_resp_hdr {
> +	u8 cmd;
> +	u8 flag;
> +	__le16 id; /* cmd id */
> +	__le32 data_len; /* Payload len in bytes */
> +};
> +
>  #endif /* _LINUX_VIRTIO_BALLOON_H */
> -- 
> 2.7.4

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

* Re: [PATCH v7 kernel 4/5] virtio-balloon: define flags and head for host request vq
  2017-03-03  5:40 ` [PATCH v7 kernel 4/5] virtio-balloon: define flags and head for host request vq Wei Wang
@ 2017-03-08  4:02   ` Michael S. Tsirkin
  0 siblings, 0 replies; 30+ messages in thread
From: Michael S. Tsirkin @ 2017-03-08  4:02 UTC (permalink / raw)
  To: Wei Wang
  Cc: virtio-dev, kvm, qemu-devel, linux-kernel, virtualization,
	linux-mm, Liang Li, Andrew Morton, Mel Gorman, Paolo Bonzini,
	Cornelia Huck, Amit Shah, Dave Hansen, Andrea Arcangeli,
	David Hildenbrand, Liang Li

On Fri, Mar 03, 2017 at 01:40:29PM +0800, Wei Wang wrote:
> From: Liang Li <liang.z.li@intel.com>
> 
> Define the flags and head struct for a new host request virtual
> queue. Guest can get requests from host and then responds to
> them on this new virtual queue.
> Host can make use of this virtqueue to request the guest to do
> some operations, e.g. drop page cache, synchronize file system,
> etc. The hypervisor can get some of guest's runtime information
> through this virtual queue too, e.g. the guest's unused page
> information, which can be used for live migration optimization.
> 
> Signed-off-by: Liang Li <liang.z.li@intel.com>
> Signed-off-by: Wei Wang <wei.w.wang@intel.com>
> Cc: Andrew Morton <akpm@linux-foundation.org>
> Cc: Mel Gorman <mgorman@techsingularity.net>
> Cc: Michael S. Tsirkin <mst@redhat.com>
> Cc: Paolo Bonzini <pbonzini@redhat.com>
> Cc: Cornelia Huck <cornelia.huck@de.ibm.com>
> Cc: Amit Shah <amit.shah@redhat.com>
> Cc: Dave Hansen <dave.hansen@intel.com>
> Cc: Andrea Arcangeli <aarcange@redhat.com>
> Cc: David Hildenbrand <david@redhat.com>
> Cc: Liang Li <liliang324@gmail.com>
> Cc: Wei Wang <wei.w.wang@intel.com>

I prefer this squashed into next patch makes review easier.


> ---
>  include/uapi/linux/virtio_balloon.h | 22 ++++++++++++++++++++++
>  1 file changed, 22 insertions(+)
> 
> diff --git a/include/uapi/linux/virtio_balloon.h b/include/uapi/linux/virtio_balloon.h
> index ed627b2..630b0ef 100644
> --- a/include/uapi/linux/virtio_balloon.h
> +++ b/include/uapi/linux/virtio_balloon.h
> @@ -35,6 +35,7 @@
>  #define VIRTIO_BALLOON_F_STATS_VQ	1 /* Memory Stats virtqueue */
>  #define VIRTIO_BALLOON_F_DEFLATE_ON_OOM	2 /* Deflate balloon on OOM */
>  #define VIRTIO_BALLOON_F_CHUNK_TRANSFER	3 /* Transfer pages in chunks */
> +#define VIRTIO_BALLOON_F_HOST_REQ_VQ	4 /* Host request virtqueue */
>  
>  /* Size of a PFN in the balloon interface. */
>  #define VIRTIO_BALLOON_PFN_SHIFT 12
> @@ -94,4 +95,25 @@ struct virtio_balloon_resp_hdr {
>  	__le32 data_len; /* Payload len in bytes */
>  };
>  
> +enum virtio_balloon_req_id {
> +	/* Get unused page information */
> +	BALLOON_GET_UNUSED_PAGES,
> +};
> +
> +enum virtio_balloon_flag {
> +	/* Have more data for a request */
> +	BALLOON_FLAG_CONT,
> +	/* No more data for a request */
> +	BALLOON_FLAG_DONE,
> +};
> +
> +struct virtio_balloon_req_hdr {
> +	/* Used to distinguish different requests */
> +	__le16 cmd;
> +	/* Reserved */
> +	__le16 reserved[3];
> +	/* Request parameter */
> +	__le64 param;
> +};
> +
>  #endif /* _LINUX_VIRTIO_BALLOON_H */
> -- 
> 2.7.4

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

* Re: [PATCH v7 kernel 5/5] This patch contains two parts:
  2017-03-06 13:23   ` David Hildenbrand
@ 2017-03-09  7:04     ` Wei Wang
  0 siblings, 0 replies; 30+ messages in thread
From: Wei Wang @ 2017-03-09  7:04 UTC (permalink / raw)
  To: David Hildenbrand
  Cc: virtio-dev, kvm, qemu-devel, linux-kernel, virtualization,
	linux-mm, Andrew Morton, Mel Gorman, Michael S . Tsirkin,
	Paolo Bonzini, Cornelia Huck, Amit Shah, Hansen, Dave,
	Andrea Arcangeli, Liang Li

On 03/06/2017 09:23 PM, David Hildenbrand wrote:Am 03.03.2017 um 06:40 
schrieb Wei Wang:
>> From: Liang Li <liang.z.li@intel.com>
Sorry, I just saw the message due to an email issue.

> I'd prefer to split this into two parts then and to create proper subjects.
Agree, will do.

>
> If I remember correctly, the general concept was accepted by most reviewers.
>

Yes, that's also what I was told.

Best,
Wei

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

* Re: [PATCH v7 kernel 2/5] virtio-balloon: VIRTIO_BALLOON_F_CHUNK_TRANSFER
  2017-03-08  4:01   ` Michael S. Tsirkin
@ 2017-03-09  7:12     ` Wei Wang
  0 siblings, 0 replies; 30+ messages in thread
From: Wei Wang @ 2017-03-09  7:12 UTC (permalink / raw)
  To: Michael S. Tsirkin
  Cc: virtio-dev, kvm, qemu-devel, linux-kernel, virtualization,
	linux-mm, Paolo Bonzini, Cornelia Huck, Amit Shah, Hansen, Dave,
	Andrea Arcangeli, David Hildenbrand, Liang Li

On 03/08/2017 12:01 PM, Michael S. Tsirkin wrote:
> On Fri, Mar 03, 2017 at 01:40:27PM +0800, Wei Wang wrote:
>> From: Liang Li <liang.z.li@intel.com>
>>
>> Add a new feature bit, VIRTIO_BALLOON_F_CHUNK_TRANSFER. Please check
>> the implementation patch commit for details about this feature.
>
> better squash into next patch.

OK, will do.

Best,
Wei

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

* Re: [PATCH v7 kernel 3/5] virtio-balloon: implementation of VIRTIO_BALLOON_F_CHUNK_TRANSFER
  2017-03-03  5:40 ` [PATCH v7 kernel 3/5] virtio-balloon: implementation of VIRTIO_BALLOON_F_CHUNK_TRANSFER Wei Wang
  2017-03-08  4:01   ` Michael S. Tsirkin
@ 2017-03-09 14:14   ` Matthew Wilcox
  2017-03-10 11:37     ` Wei Wang
  1 sibling, 1 reply; 30+ messages in thread
From: Matthew Wilcox @ 2017-03-09 14:14 UTC (permalink / raw)
  To: Wei Wang
  Cc: virtio-dev, kvm, qemu-devel, linux-kernel, virtualization,
	linux-mm, Liang Li, Michael S . Tsirkin, Paolo Bonzini,
	Cornelia Huck, Amit Shah, Dave Hansen, Andrea Arcangeli,
	David Hildenbrand, Liang Li

On Fri, Mar 03, 2017 at 01:40:28PM +0800, Wei Wang wrote:
> From: Liang Li <liang.z.li@intel.com>
> 1) allocating pages (6.5%)
> 2) sending PFNs to host (68.3%)
> 3) address translation (6.1%)
> 4) madvise (19%)
> 
> This patch optimizes step 2) by transfering pages to the host in
> chunks. A chunk consists of guest physically continuous pages, and
> it is offered to the host via a base PFN (i.e. the start PFN of
> those physically continuous pages) and the size (i.e. the total
> number of the pages). A normal chunk is formated as below:
> -----------------------------------------------
> |  Base (52 bit)               | Size (12 bit)|
> -----------------------------------------------
> For large size chunks, an extended chunk format is used:
> -----------------------------------------------
> |                 Base (64 bit)               |
> -----------------------------------------------
> -----------------------------------------------
> |                 Size (64 bit)               |
> -----------------------------------------------

What's the advantage to extended chunks?  IOW, why is the added complexity
of having two chunk formats worth it?  You already reduced the overhead by
a factor of 4096 with normal chunks ... how often are extended chunks used
and how much more efficient are they than having several normal chunks?

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

* Re: [virtio-dev] Re: [PATCH v7 kernel 3/5] virtio-balloon: implementation of VIRTIO_BALLOON_F_CHUNK_TRANSFER
  2017-03-08  4:01   ` Michael S. Tsirkin
@ 2017-03-10 10:02     ` Wei Wang
  2017-03-10 13:26       ` David Hildenbrand
  2017-03-10 15:37       ` Michael S. Tsirkin
  0 siblings, 2 replies; 30+ messages in thread
From: Wei Wang @ 2017-03-10 10:02 UTC (permalink / raw)
  To: Michael S. Tsirkin
  Cc: virtio-dev, kvm, qemu-devel, linux-kernel, virtualization,
	linux-mm, Liang Li, Paolo Bonzini, Cornelia Huck, Amit Shah,
	Hansen, Dave, Andrea Arcangeli, David Hildenbrand, Liang Li

On 03/08/2017 12:01 PM, Michael S. Tsirkin wrote:
> On Fri, Mar 03, 2017 at 01:40:28PM +0800, Wei Wang wrote:
>> From: Liang Li <liang.z.li@intel.com>
>>
>> The implementation of the current virtio-balloon is not very
>> efficient, because the pages are transferred to the host one by one.
>> Here is the breakdown of the time in percentage spent on each
>> step of the balloon inflating process (inflating 7GB of an 8GB
>> idle guest).
>>
>> 1) allocating pages (6.5%)
>> 2) sending PFNs to host (68.3%)
>> 3) address translation (6.1%)
>> 4) madvise (19%)
>>
>> It takes about 4126ms for the inflating process to complete.
>> The above profiling shows that the bottlenecks are stage 2)
>> and stage 4).
>>
>> This patch optimizes step 2) by transfering pages to the host in
>> chunks. A chunk consists of guest physically continuous pages, and
>> it is offered to the host via a base PFN (i.e. the start PFN of
>> those physically continuous pages) and the size (i.e. the total
>> number of the pages). A normal chunk is formated as below:
>> -----------------------------------------------
>> |  Base (52 bit)               | Size (12 bit)|
>> -----------------------------------------------
>> For large size chunks, an extended chunk format is used:
>> -----------------------------------------------
>> |                 Base (64 bit)               |
>> -----------------------------------------------
>> -----------------------------------------------
>> |                 Size (64 bit)               |
>> -----------------------------------------------
>>
>> By doing so, step 4) can also be optimized by doing address
>> translation and madvise() in chunks rather than page by page.
>>
>> This optimization requires the negotation of a new feature bit,
>> VIRTIO_BALLOON_F_CHUNK_TRANSFER.
>>
>> With this new feature, the above ballooning process takes ~590ms
>> resulting in an improvement of ~85%.
>>
>> TODO: optimize stage 1) by allocating/freeing a chunk of pages
>> instead of a single page each time.
>>
>> Signed-off-by: Liang Li <liang.z.li@intel.com>
>> Signed-off-by: Wei Wang <wei.w.wang@intel.com>
>> Suggested-by: Michael S. Tsirkin <mst@redhat.com>
>> Cc: Michael S. Tsirkin <mst@redhat.com>
>> Cc: Paolo Bonzini <pbonzini@redhat.com>
>> Cc: Cornelia Huck <cornelia.huck@de.ibm.com>
>> Cc: Amit Shah <amit.shah@redhat.com>
>> Cc: Dave Hansen <dave.hansen@intel.com>
>> Cc: Andrea Arcangeli <aarcange@redhat.com>
>> Cc: David Hildenbrand <david@redhat.com>
>> Cc: Liang Li <liliang324@gmail.com>
>> Cc: Wei Wang <wei.w.wang@intel.com>
> Does this pass sparse? I see some endian-ness issues here.

"pass sparse"- what does that mean?
I didn't see any complaints from "make" on my machine.

>> ---
>>   drivers/virtio/virtio_balloon.c | 351 ++++++++++++++++++++++++++++++++++++----
>>   1 file changed, 323 insertions(+), 28 deletions(-)
>>
>> diff --git a/drivers/virtio/virtio_balloon.c b/drivers/virtio/virtio_balloon.c
>> index f59cb4f..4416370 100644
>> --- a/drivers/virtio/virtio_balloon.c
>> +++ b/drivers/virtio/virtio_balloon.c
>> @@ -42,6 +42,10 @@
>>   #define OOM_VBALLOON_DEFAULT_PAGES 256
>>   #define VIRTBALLOON_OOM_NOTIFY_PRIORITY 80
>>   
>> +#define PAGE_BMAP_SIZE	(8 * PAGE_SIZE)
>> +#define PFNS_PER_PAGE_BMAP	(PAGE_BMAP_SIZE * BITS_PER_BYTE)
>> +#define PAGE_BMAP_COUNT_MAX	32
>> +
>>   static int oom_pages = OOM_VBALLOON_DEFAULT_PAGES;
>>   module_param(oom_pages, int, S_IRUSR | S_IWUSR);
>>   MODULE_PARM_DESC(oom_pages, "pages to free on OOM");
>> @@ -50,6 +54,16 @@ MODULE_PARM_DESC(oom_pages, "pages to free on OOM");
>>   static struct vfsmount *balloon_mnt;
>>   #endif
>>   
>> +struct balloon_page_chunk {
>> +	__le64 base : 52;
>> +	__le64 size : 12;
>> +};
>> +
>> +struct balloon_page_chunk_ext {
>> +	__le64 base;
>> +	__le64 size;
>> +};
>> +
>>   struct virtio_balloon {
>>   	struct virtio_device *vdev;
>>   	struct virtqueue *inflate_vq, *deflate_vq, *stats_vq;
>> @@ -67,6 +81,20 @@ struct virtio_balloon {
>>   
>>   	/* Number of balloon pages we've told the Host we're not using. */
>>   	unsigned int num_pages;
>> +	/* Pointer to the response header. */
>> +	struct virtio_balloon_resp_hdr resp_hdr;
>> +	/* Pointer to the start address of response data. */
>> +	__le64 *resp_data;
>> +	/* Size of response data buffer. */
>> +	unsigned int resp_buf_size;
>> +	/* Pointer offset of the response data. */
>> +	unsigned int resp_pos;
>> +	/* Bitmap used to save the pfns info */
>> +	unsigned long *page_bitmap[PAGE_BMAP_COUNT_MAX];
>> +	/* Number of split page bitmaps */
>> +	unsigned int nr_page_bmap;
>> +	/* Used to record the processed pfn range */
>> +	unsigned long min_pfn, max_pfn, start_pfn, end_pfn;
>>   	/*
>>   	 * The pages we've told the Host we're not using are enqueued
>>   	 * at vb_dev_info->pages list.
>> @@ -110,20 +138,180 @@ static void balloon_ack(struct virtqueue *vq)
>>   	wake_up(&vb->acked);
>>   }
>>   
>> -static void tell_host(struct virtio_balloon *vb, struct virtqueue *vq)
>> +static inline void init_bmap_pfn_range(struct virtio_balloon *vb)
>>   {
>> -	struct scatterlist sg;
>> +	vb->min_pfn = ULONG_MAX;
>> +	vb->max_pfn = 0;
>> +}
>> +
>> +static inline void update_bmap_pfn_range(struct virtio_balloon *vb,
>> +				 struct page *page)
>> +{
>> +	unsigned long balloon_pfn = page_to_balloon_pfn(page);
>> +
>> +	vb->min_pfn = min(balloon_pfn, vb->min_pfn);
>> +	vb->max_pfn = max(balloon_pfn, vb->max_pfn);
>> +}
>> +
>> +static void extend_page_bitmap(struct virtio_balloon *vb,
>> +				unsigned long nr_pfn)
>
> what exactly does this do?

By default, only one page_bmap is allocated.
When there are too many pfns to fit into one page_bmap,
it tries to allocate more page_bmap using this extend_page_bmap() function.

>
>> +{
>> +	int i, bmap_count;
>> +	unsigned long bmap_len;
>> +
>> +	bmap_len = ALIGN(nr_pfn, BITS_PER_LONG) / BITS_PER_BYTE;
>> +	bmap_len = ALIGN(bmap_len, PAGE_BMAP_SIZE);
>> +	bmap_count = min((int)(bmap_len / PAGE_BMAP_SIZE),
>> +				 PAGE_BMAP_COUNT_MAX);
>> +
>> +	for (i = 1; i < bmap_count; i++) {
>> +		vb->page_bitmap[i] = kmalloc(PAGE_BMAP_SIZE, GFP_KERNEL);
>> +		if (vb->page_bitmap[i])
>> +			vb->nr_page_bmap++;
>> +		else
>> +			break;
> seems to fail silently ...

I think here it is fine whether the page_bmap extension succeeds or 
silently fails.
It doesn't affect the correctness of the implementation - the page_bmap 
is always
used through vb->nr_page_bmap. Maybe the code logic is still a little 
fuzzy. I plan
to re-write most of the implementation to make it clearer.

>> +	}
>> +}
>> +
>> +static void free_extended_page_bitmap(struct virtio_balloon *vb)
>> +{
>> +	int i, bmap_count = vb->nr_page_bmap;
>> +
>> +	for (i = 1; i < bmap_count; i++) {
>> +		kfree(vb->page_bitmap[i]);
>> +		vb->page_bitmap[i] = NULL;
>> +		vb->nr_page_bmap--;
>> +	}
>> +}
>> +
>> +static void kfree_page_bitmap(struct virtio_balloon *vb)
>> +{
>> +	int i;
>> +
>> +	for (i = 0; i < vb->nr_page_bmap; i++)
>> +		kfree(vb->page_bitmap[i]);
>> +}
>> +
> A very confusing name. pls don't start with kfree.

Ok. I'll simply call it free_page_bmap()

>
>> +static void clear_page_bitmap(struct virtio_balloon *vb)
>> +{
>> +	int i;
>> +
>> +	for (i = 0; i < vb->nr_page_bmap; i++)
>> +		memset(vb->page_bitmap[i], 0, PAGE_BMAP_SIZE);
>> +}
>> +
>> +static void send_resp_data(struct virtio_balloon *vb, struct virtqueue *vq,
>> +			bool busy_wait)
>> +{
>> +	struct scatterlist sg[2];
>> +	struct virtio_balloon_resp_hdr *hdr = &vb->resp_hdr;
>>   	unsigned int len;
>>   
>> -	sg_init_one(&sg, vb->pfns, sizeof(vb->pfns[0]) * vb->num_pfns);
>> +	len = hdr->data_len = vb->resp_pos * sizeof(__le64);
>> +	sg_init_table(sg, 2);
>> +	sg_set_buf(&sg[0], hdr, sizeof(struct virtio_balloon_resp_hdr));
>> +	sg_set_buf(&sg[1], vb->resp_data, len);
>> +
>> +	if (virtqueue_add_outbuf(vq, sg, 2, vb, GFP_KERNEL) == 0) {
> !virtqueue_add_outbuf
>
> Also how about making this header linear with pfns?
> This way you can use 1 s/g and avoid mallocs.

Yes, I think it'll be better, thanks.


>> +		virtqueue_kick(vq);
>> +		if (busy_wait)
>> +			while (!virtqueue_get_buf(vq, &len)
>> +				&& !virtqueue_is_broken(vq))
>> +				cpu_relax();
>> +		else
>> +			wait_event(vb->acked, virtqueue_get_buf(vq, &len));
>> +		vb->resp_pos = 0;
>> +		free_extended_page_bitmap(vb);
>> +	}
>> +}
>>   
>> -	/* We should always be able to add one buffer to an empty queue. */
>> -	virtqueue_add_outbuf(vq, &sg, 1, vb, GFP_KERNEL);
>> -	virtqueue_kick(vq);
>> +static void do_set_resp_bitmap(struct virtio_balloon *vb, unsigned long base,
>> +			       int size)
>> +{
>> +	/* Use the extented chunk format if the size is too large */
>> +	if (size > (1 << VIRTIO_BALLOON_CHUNK_SIZE_SHIFT)) {
>> +		struct balloon_page_chunk_ext *chunk_ext =
>> +			(struct balloon_page_chunk_ext *)(vb->resp_data + vb->resp_pos);
>> +		chunk_ext->base = cpu_to_le64(base);
>> +		chunk_ext->size = cpu_to_le64(size);
>> +		vb->resp_pos += sizeof(vb->resp_pos) / sizeof(*chunk_ext);
>> +	} else {
>> +		struct balloon_page_chunk *chunk =
>> +			(struct balloon_page_chunk *)(vb->resp_data + vb->resp_pos);
>> +		chunk->base = cpu_to_le64(base);
>> +		chunk->size = cpu_to_le16(size);
>> +		vb->resp_pos += sizeof(vb->resp_pos) / sizeof(*chunk);
>> +	}
>> +}
>>   
>> -	/* When host has read buffer, this completes via balloon_ack */
>> -	wait_event(vb->acked, virtqueue_get_buf(vq, &len));
>> +static void chunking_pages_from_bmap(struct virtio_balloon *vb,
>> +				     struct virtqueue *vq,
>> +				     unsigned long start_pfn, unsigned long *bitmap,
>> +				     unsigned long len, bool busy_wait)
>> +{
>> +	unsigned long pos = 0, end = len * BITS_PER_BYTE;
>> +
>> +	while (pos < end) {
>> +		unsigned long one = find_next_bit(bitmap, end, pos);
>> +
>> +		if (one < end) {
>> +			unsigned long chunk_size, zero;
>> +
>> +			zero = find_next_zero_bit(bitmap, end, one + 1);
>> +			if (zero >= end)
>> +				chunk_size = end - one;
>> +			else
>> +				chunk_size = zero - one;
>> +			if (chunk_size) {
>> +				if ((vb->resp_pos + 2) * sizeof(__le64) >
>> +						vb->resp_buf_size)
> why + 2? where does sizeof come from?
> Any chance this can be cleaned up using structs sizeof etc?
> or at least commented?
> How am I supposed to figure out all this convoluted pointer math?

Right. I'll take of it in the new version.


>> +					send_resp_data(vb, vq, busy_wait);
>> +				do_set_resp_bitmap(vb, start_pfn + one,	chunk_size);
>> +			}
>> +			pos = one + chunk_size;
>> +		} else
>> +			break;
>> +	}
>> +}
>>   
>> +static void tell_host(struct virtio_balloon *vb, struct virtqueue *vq)
>> +{
>> +	if (virtio_has_feature(vb->vdev, VIRTIO_BALLOON_F_CHUNK_TRANSFER)) {
>> +		int nr_pfn, nr_used_bmap, i;
>> +		unsigned long start_pfn, bmap_len;
>> +
>> +		start_pfn = vb->start_pfn;
>> +		nr_pfn = vb->end_pfn - start_pfn + 1;
>> +		nr_pfn = roundup(nr_pfn, BITS_PER_LONG);
>> +		nr_used_bmap = nr_pfn / PFNS_PER_PAGE_BMAP;
> better name here including "page"?
> And bmaps instead of bmap?
Agree. I'll call it page_bmaps.


>> +		if (nr_pfn % PFNS_PER_PAGE_BMAP)
>> +			nr_used_bmap++;
> Pls just use macros like ALIGN or roundup?
>
>> +		bmap_len = nr_pfn / BITS_PER_BYTE;
> I guess we need to round up.
>
>> +
>> +		for (i = 0; i < nr_used_bmap; i++) {
>> +			unsigned int bmap_size = PAGE_BMAP_SIZE;
>> +
>> +			if (i + 1 == nr_used_bmap)
>> +				bmap_size = bmap_len - PAGE_BMAP_SIZE * i;
> So last one is treated speacially? why?

The last one takes the leftover only, so that chunking_pages_from_bmap()
only searches the necessary part of (rather than the entire) the page_bmap.

>> +			chunking_pages_from_bmap(vb, vq, start_pfn + i * PFNS_PER_PAGE_BMAP,
>> +				 vb->page_bitmap[i], bmap_size, false);
>> +		}
>> +		if (vb->resp_pos > 0)
>> +			send_resp_data(vb, vq, false);
>> +	} else {
>> +		struct scatterlist sg;
>> +		unsigned int len;
>> +
>> +		sg_init_one(&sg, vb->pfns, sizeof(vb->pfns[0]) * vb->num_pfns);
>> +
>> +		/* We should always be able to add one buffer to an
>> +		 * empty queue
>> +		 */
> /*
>   * Multi-line comments
>   * like this.
>   */

OK.

>> +		virtqueue_add_outbuf(vq, &sg, 1, vb, GFP_KERNEL);
>> +		virtqueue_kick(vq);
>> +		/* When host has read buffer, this completes via balloon_ack */
>> +		wait_event(vb->acked, virtqueue_get_buf(vq, &len));
>> +	}
>>   }
>>   
>>   static void set_page_pfns(struct virtio_balloon *vb,
>> @@ -138,13 +326,58 @@ static void set_page_pfns(struct virtio_balloon *vb,
>>   					  page_to_balloon_pfn(page) + i);
>>   }
>>   
>> -static unsigned fill_balloon(struct virtio_balloon *vb, size_t num)
>> +static void set_page_bitmap(struct virtio_balloon *vb,
>> +			 struct list_head *pages, struct virtqueue *vq)
> align on ( pls.
>
>> +{
>> +	unsigned long pfn, pfn_limit;
>> +	struct page *page;
>> +	bool found;
>> +
>> +	vb->min_pfn = rounddown(vb->min_pfn, BITS_PER_LONG);
>> +	vb->max_pfn = roundup(vb->max_pfn, BITS_PER_LONG);
>> +	pfn_limit = PFNS_PER_PAGE_BMAP * vb->nr_page_bmap;
>> +
>> +	if (vb->nr_page_bmap == 1)
>> +		extend_page_bitmap(vb, vb->max_pfn - vb->min_pfn + 1);
> how is 1 special?

Thanks. I think "if (vb->nr_page_bmap == 1)" should be removed.
Whenever the current amount of page_bmap is not enough to hold
the amount of pfns, it should try to entend page_bmap.

Best,
Wei

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

* Re: [PATCH v7 kernel 3/5] virtio-balloon: implementation of VIRTIO_BALLOON_F_CHUNK_TRANSFER
  2017-03-09 14:14   ` Matthew Wilcox
@ 2017-03-10 11:37     ` Wei Wang
  2017-03-10 15:58       ` Michael S. Tsirkin
  0 siblings, 1 reply; 30+ messages in thread
From: Wei Wang @ 2017-03-10 11:37 UTC (permalink / raw)
  To: Matthew Wilcox
  Cc: virtio-dev, kvm, qemu-devel, linux-kernel, virtualization,
	linux-mm, Liang Li, Michael S . Tsirkin, Paolo Bonzini,
	Cornelia Huck, Amit Shah, Dave Hansen, Andrea Arcangeli,
	David Hildenbrand, Liang Li

On 03/09/2017 10:14 PM, Matthew Wilcox wrote:
> On Fri, Mar 03, 2017 at 01:40:28PM +0800, Wei Wang wrote:
>> From: Liang Li <liang.z.li@intel.com>
>> 1) allocating pages (6.5%)
>> 2) sending PFNs to host (68.3%)
>> 3) address translation (6.1%)
>> 4) madvise (19%)
>>
>> This patch optimizes step 2) by transfering pages to the host in
>> chunks. A chunk consists of guest physically continuous pages, and
>> it is offered to the host via a base PFN (i.e. the start PFN of
>> those physically continuous pages) and the size (i.e. the total
>> number of the pages). A normal chunk is formated as below:
>> -----------------------------------------------
>> |  Base (52 bit)               | Size (12 bit)|
>> -----------------------------------------------
>> For large size chunks, an extended chunk format is used:
>> -----------------------------------------------
>> |                 Base (64 bit)               |
>> -----------------------------------------------
>> -----------------------------------------------
>> |                 Size (64 bit)               |
>> -----------------------------------------------
> What's the advantage to extended chunks?  IOW, why is the added complexity
> of having two chunk formats worth it?  You already reduced the overhead by
> a factor of 4096 with normal chunks ... how often are extended chunks used
> and how much more efficient are they than having several normal chunks?
>

Right, chunk_ext may be rarely used, thanks. I will remove chunk_ext if 
there is no objection from others.

Best,
Wei

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

* Re: [virtio-dev] Re: [PATCH v7 kernel 3/5] virtio-balloon: implementation of VIRTIO_BALLOON_F_CHUNK_TRANSFER
  2017-03-10 10:02     ` [virtio-dev] " Wei Wang
@ 2017-03-10 13:26       ` David Hildenbrand
  2017-03-10 15:37       ` Michael S. Tsirkin
  1 sibling, 0 replies; 30+ messages in thread
From: David Hildenbrand @ 2017-03-10 13:26 UTC (permalink / raw)
  To: Wei Wang, Michael S. Tsirkin
  Cc: virtio-dev, kvm, qemu-devel, linux-kernel, virtualization,
	linux-mm, Liang Li, Paolo Bonzini, Cornelia Huck, Amit Shah,
	Hansen, Dave, Andrea Arcangeli, Liang Li

Am 10.03.2017 um 11:02 schrieb Wei Wang:
> On 03/08/2017 12:01 PM, Michael S. Tsirkin wrote:
>> On Fri, Mar 03, 2017 at 01:40:28PM +0800, Wei Wang wrote:
>>> From: Liang Li <liang.z.li@intel.com>
>>>
>>> The implementation of the current virtio-balloon is not very
>>> efficient, because the pages are transferred to the host one by one.
>>> Here is the breakdown of the time in percentage spent on each
>>> step of the balloon inflating process (inflating 7GB of an 8GB
>>> idle guest).
>>>
>>> 1) allocating pages (6.5%)
>>> 2) sending PFNs to host (68.3%)
>>> 3) address translation (6.1%)
>>> 4) madvise (19%)
>>>
>>> It takes about 4126ms for the inflating process to complete.
>>> The above profiling shows that the bottlenecks are stage 2)
>>> and stage 4).
>>>
>>> This patch optimizes step 2) by transfering pages to the host in
>>> chunks. A chunk consists of guest physically continuous pages, and
>>> it is offered to the host via a base PFN (i.e. the start PFN of
>>> those physically continuous pages) and the size (i.e. the total
>>> number of the pages). A normal chunk is formated as below:
>>> -----------------------------------------------
>>> |  Base (52 bit)               | Size (12 bit)|
>>> -----------------------------------------------
>>> For large size chunks, an extended chunk format is used:
>>> -----------------------------------------------
>>> |                 Base (64 bit)               |
>>> -----------------------------------------------
>>> -----------------------------------------------
>>> |                 Size (64 bit)               |
>>> -----------------------------------------------
>>>
>>> By doing so, step 4) can also be optimized by doing address
>>> translation and madvise() in chunks rather than page by page.
>>>
>>> This optimization requires the negotation of a new feature bit,
>>> VIRTIO_BALLOON_F_CHUNK_TRANSFER.
>>>
>>> With this new feature, the above ballooning process takes ~590ms
>>> resulting in an improvement of ~85%.
>>>
>>> TODO: optimize stage 1) by allocating/freeing a chunk of pages
>>> instead of a single page each time.
>>>
>>> Signed-off-by: Liang Li <liang.z.li@intel.com>
>>> Signed-off-by: Wei Wang <wei.w.wang@intel.com>
>>> Suggested-by: Michael S. Tsirkin <mst@redhat.com>
>>> Cc: Michael S. Tsirkin <mst@redhat.com>
>>> Cc: Paolo Bonzini <pbonzini@redhat.com>
>>> Cc: Cornelia Huck <cornelia.huck@de.ibm.com>
>>> Cc: Amit Shah <amit.shah@redhat.com>
>>> Cc: Dave Hansen <dave.hansen@intel.com>
>>> Cc: Andrea Arcangeli <aarcange@redhat.com>
>>> Cc: David Hildenbrand <david@redhat.com>
>>> Cc: Liang Li <liliang324@gmail.com>
>>> Cc: Wei Wang <wei.w.wang@intel.com>
>> Does this pass sparse? I see some endian-ness issues here.
> 
> "pass sparse"- what does that mean?
> I didn't see any complaints from "make" on my machine.

https://kernel.org/doc/html/latest/dev-tools/sparse.html

Static code analysis. You have to run it explicitly.

-- 
Thanks,

David

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

* Re: [virtio-dev] Re: [PATCH v7 kernel 3/5] virtio-balloon: implementation of VIRTIO_BALLOON_F_CHUNK_TRANSFER
  2017-03-10 10:02     ` [virtio-dev] " Wei Wang
  2017-03-10 13:26       ` David Hildenbrand
@ 2017-03-10 15:37       ` Michael S. Tsirkin
  1 sibling, 0 replies; 30+ messages in thread
From: Michael S. Tsirkin @ 2017-03-10 15:37 UTC (permalink / raw)
  To: Wei Wang
  Cc: virtio-dev, kvm, qemu-devel, linux-kernel, virtualization,
	linux-mm, Liang Li, Paolo Bonzini, Cornelia Huck, Amit Shah,
	Hansen, Dave, Andrea Arcangeli, David Hildenbrand, Liang Li

On Fri, Mar 10, 2017 at 06:02:31PM +0800, Wei Wang wrote:
> On 03/08/2017 12:01 PM, Michael S. Tsirkin wrote:
> > On Fri, Mar 03, 2017 at 01:40:28PM +0800, Wei Wang wrote:
> > > From: Liang Li <liang.z.li@intel.com>
> > > 
> > > The implementation of the current virtio-balloon is not very
> > > efficient, because the pages are transferred to the host one by one.
> > > Here is the breakdown of the time in percentage spent on each
> > > step of the balloon inflating process (inflating 7GB of an 8GB
> > > idle guest).
> > > 
> > > 1) allocating pages (6.5%)
> > > 2) sending PFNs to host (68.3%)
> > > 3) address translation (6.1%)
> > > 4) madvise (19%)
> > > 
> > > It takes about 4126ms for the inflating process to complete.
> > > The above profiling shows that the bottlenecks are stage 2)
> > > and stage 4).
> > > 
> > > This patch optimizes step 2) by transfering pages to the host in
> > > chunks. A chunk consists of guest physically continuous pages, and
> > > it is offered to the host via a base PFN (i.e. the start PFN of
> > > those physically continuous pages) and the size (i.e. the total
> > > number of the pages). A normal chunk is formated as below:
> > > -----------------------------------------------
> > > |  Base (52 bit)               | Size (12 bit)|
> > > -----------------------------------------------
> > > For large size chunks, an extended chunk format is used:
> > > -----------------------------------------------
> > > |                 Base (64 bit)               |
> > > -----------------------------------------------
> > > -----------------------------------------------
> > > |                 Size (64 bit)               |
> > > -----------------------------------------------
> > > 
> > > By doing so, step 4) can also be optimized by doing address
> > > translation and madvise() in chunks rather than page by page.
> > > 
> > > This optimization requires the negotation of a new feature bit,
> > > VIRTIO_BALLOON_F_CHUNK_TRANSFER.
> > > 
> > > With this new feature, the above ballooning process takes ~590ms
> > > resulting in an improvement of ~85%.
> > > 
> > > TODO: optimize stage 1) by allocating/freeing a chunk of pages
> > > instead of a single page each time.
> > > 
> > > Signed-off-by: Liang Li <liang.z.li@intel.com>
> > > Signed-off-by: Wei Wang <wei.w.wang@intel.com>
> > > Suggested-by: Michael S. Tsirkin <mst@redhat.com>
> > > Cc: Michael S. Tsirkin <mst@redhat.com>
> > > Cc: Paolo Bonzini <pbonzini@redhat.com>
> > > Cc: Cornelia Huck <cornelia.huck@de.ibm.com>
> > > Cc: Amit Shah <amit.shah@redhat.com>
> > > Cc: Dave Hansen <dave.hansen@intel.com>
> > > Cc: Andrea Arcangeli <aarcange@redhat.com>
> > > Cc: David Hildenbrand <david@redhat.com>
> > > Cc: Liang Li <liliang324@gmail.com>
> > > Cc: Wei Wang <wei.w.wang@intel.com>
> > Does this pass sparse? I see some endian-ness issues here.
> 
> "pass sparse"- what does that mean?
> I didn't see any complaints from "make" on my machine.


Run with make C=1 (or C=2 to check all source).

Generally there's a ton of useful info you will find
if you run make help.

-- 
MST

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

* Re: [PATCH v7 kernel 3/5] virtio-balloon: implementation of VIRTIO_BALLOON_F_CHUNK_TRANSFER
  2017-03-10 11:37     ` Wei Wang
@ 2017-03-10 15:58       ` Michael S. Tsirkin
  2017-03-10 17:11         ` Matthew Wilcox
  0 siblings, 1 reply; 30+ messages in thread
From: Michael S. Tsirkin @ 2017-03-10 15:58 UTC (permalink / raw)
  To: Wei Wang
  Cc: Matthew Wilcox, virtio-dev, kvm, qemu-devel, linux-kernel,
	virtualization, linux-mm, Liang Li, Paolo Bonzini, Cornelia Huck,
	Amit Shah, Dave Hansen, Andrea Arcangeli, David Hildenbrand,
	Liang Li

On Fri, Mar 10, 2017 at 07:37:28PM +0800, Wei Wang wrote:
> On 03/09/2017 10:14 PM, Matthew Wilcox wrote:
> > On Fri, Mar 03, 2017 at 01:40:28PM +0800, Wei Wang wrote:
> > > From: Liang Li <liang.z.li@intel.com>
> > > 1) allocating pages (6.5%)
> > > 2) sending PFNs to host (68.3%)
> > > 3) address translation (6.1%)
> > > 4) madvise (19%)
> > > 
> > > This patch optimizes step 2) by transfering pages to the host in
> > > chunks. A chunk consists of guest physically continuous pages, and
> > > it is offered to the host via a base PFN (i.e. the start PFN of
> > > those physically continuous pages) and the size (i.e. the total
> > > number of the pages). A normal chunk is formated as below:
> > > -----------------------------------------------
> > > |  Base (52 bit)               | Size (12 bit)|
> > > -----------------------------------------------
> > > For large size chunks, an extended chunk format is used:
> > > -----------------------------------------------
> > > |                 Base (64 bit)               |
> > > -----------------------------------------------
> > > -----------------------------------------------
> > > |                 Size (64 bit)               |
> > > -----------------------------------------------
> > What's the advantage to extended chunks?  IOW, why is the added complexity
> > of having two chunk formats worth it?  You already reduced the overhead by
> > a factor of 4096 with normal chunks ... how often are extended chunks used
> > and how much more efficient are they than having several normal chunks?
> > 
> 
> Right, chunk_ext may be rarely used, thanks. I will remove chunk_ext if
> there is no objection from others.
> 
> Best,
> Wei

I don't think we can drop this, this isn't an optimization.


One of the issues of current balloon is the 4k page size
assumption. For example if you free a huge page you
have to split it up and pass 4k chunks to host.
Quite often host can't free these 4k chunks at all (e.g.
when it's using huge tlb fs).
It's even sillier for architectures with base page size >4k.

So as long as we are changing things, let's not hard-code
the 12 shift thing everywhere.


Two things to consider:
- host should pass its base page size to guest
  this can be a separate patch and for now we can fall back on 12 bit if not there

- guest should pass full huge pages to host
  this should be done correctly to avoid breaking up huge pages
  I would say yes let's use a single format but drop the "normal chunk"
  and always use the extended one.
  Also, size is in units of 4k, right? Please document that low 12 bit
  are reserved, they will be handy as e.g. flags.

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

* Re: [PATCH v7 kernel 3/5] virtio-balloon: implementation of VIRTIO_BALLOON_F_CHUNK_TRANSFER
  2017-03-10 15:58       ` Michael S. Tsirkin
@ 2017-03-10 17:11         ` Matthew Wilcox
  2017-03-10 19:10           ` Michael S. Tsirkin
                             ` (2 more replies)
  0 siblings, 3 replies; 30+ messages in thread
From: Matthew Wilcox @ 2017-03-10 17:11 UTC (permalink / raw)
  To: Michael S. Tsirkin
  Cc: Wei Wang, virtio-dev, kvm, qemu-devel, linux-kernel,
	virtualization, linux-mm, Liang Li, Paolo Bonzini, Cornelia Huck,
	Amit Shah, Dave Hansen, Andrea Arcangeli, David Hildenbrand,
	Liang Li

On Fri, Mar 10, 2017 at 05:58:28PM +0200, Michael S. Tsirkin wrote:
> One of the issues of current balloon is the 4k page size
> assumption. For example if you free a huge page you
> have to split it up and pass 4k chunks to host.
> Quite often host can't free these 4k chunks at all (e.g.
> when it's using huge tlb fs).
> It's even sillier for architectures with base page size >4k.

I completely agree with you that we should be able to pass a hugepage
as a single chunk.  Also we shouldn't assume that host and guest have
the same page size.  I think we can come up with a scheme that actually
lets us encode that into a 64-bit word, something like this:

bit 0 clear => bits 1-11 encode a page count, bits 12-63 encode a PFN, page size 4k.
bit 0 set, bit 1 clear => bits 2-12 encode a page count, bits 13-63 encode a PFN, page size 8k
bits 0+1 set, bit 2 clear => bits 3-13 for page count, bits 14-63 for PFN, page size 16k.
bits 0-2 set, bit 3 clear => bits 4-14 for page count, bits 15-63 for PFN, page size 32k
bits 0-3 set, bit 4 clear => bits 5-15 for page count, bits 16-63 for PFN, page size 64k

That means we can always pass 2048 pages (of whatever page size) in a single chunk.  And
we support arbitrary power of two page sizes.  I suggest something like this:

u64 page_to_chunk(struct page *page)
{
	u64 chunk = page_to_pfn(page) << PAGE_SHIFT;
	chunk |= (1UL << compound_order(page)) - 1;
}

(note this is a single page of order N, so we leave the page count bits
set to 0, meaning one page).

> Two things to consider:
> - host should pass its base page size to guest
>   this can be a separate patch and for now we can fall back on 12 bit if not there

With this encoding scheme, I don't think we need to do this?  As long as
it's *at least* 12 bit, then we're fine.

> - guest should pass full huge pages to host
>   this should be done correctly to avoid breaking up huge pages
>   I would say yes let's use a single format but drop the "normal chunk"
>   and always use the extended one.
>   Also, size is in units of 4k, right? Please document that low 12 bit
>   are reserved, they will be handy as e.g. flags.

What per-chunk flags are you thinking would be useful?

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

* Re: [PATCH v7 kernel 3/5] virtio-balloon: implementation of VIRTIO_BALLOON_F_CHUNK_TRANSFER
  2017-03-10 17:11         ` Matthew Wilcox
@ 2017-03-10 19:10           ` Michael S. Tsirkin
  2017-03-10 21:18             ` Matthew Wilcox
  2017-03-10 19:35           ` Michael S. Tsirkin
  2017-03-11 11:59           ` Wei Wang
  2 siblings, 1 reply; 30+ messages in thread
From: Michael S. Tsirkin @ 2017-03-10 19:10 UTC (permalink / raw)
  To: Matthew Wilcox
  Cc: Wei Wang, virtio-dev, kvm, qemu-devel, linux-kernel,
	virtualization, linux-mm, Liang Li, Paolo Bonzini, Cornelia Huck,
	Amit Shah, Dave Hansen, Andrea Arcangeli, David Hildenbrand,
	Liang Li

On Fri, Mar 10, 2017 at 09:11:44AM -0800, Matthew Wilcox wrote:
> On Fri, Mar 10, 2017 at 05:58:28PM +0200, Michael S. Tsirkin wrote:
> > One of the issues of current balloon is the 4k page size
> > assumption. For example if you free a huge page you
> > have to split it up and pass 4k chunks to host.
> > Quite often host can't free these 4k chunks at all (e.g.
> > when it's using huge tlb fs).
> > It's even sillier for architectures with base page size >4k.
> 
> I completely agree with you that we should be able to pass a hugepage
> as a single chunk.  Also we shouldn't assume that host and guest have
> the same page size.  I think we can come up with a scheme that actually
> lets us encode that into a 64-bit word, something like this:
> 
> bit 0 clear => bits 1-11 encode a page count, bits 12-63 encode a PFN, page size 4k.
> bit 0 set, bit 1 clear => bits 2-12 encode a page count, bits 13-63 encode a PFN, page size 8k
> bits 0+1 set, bit 2 clear => bits 3-13 for page count, bits 14-63 for PFN, page size 16k.
> bits 0-2 set, bit 3 clear => bits 4-14 for page count, bits 15-63 for PFN, page size 32k
> bits 0-3 set, bit 4 clear => bits 5-15 for page count, bits 16-63 for PFN, page size 64k

huge page sizes go up to gigabytes.

> That means we can always pass 2048 pages (of whatever page size) in a single chunk.  And
> we support arbitrary power of two page sizes.  I suggest something like this:
> 
> u64 page_to_chunk(struct page *page)
> {
> 	u64 chunk = page_to_pfn(page) << PAGE_SHIFT;
> 	chunk |= (1UL << compound_order(page)) - 1;
> }
> 
> (note this is a single page of order N, so we leave the page count bits
> set to 0, meaning one page).
> 
> > Two things to consider:
> > - host should pass its base page size to guest
> >   this can be a separate patch and for now we can fall back on 12 bit if not there
> 
> With this encoding scheme, I don't think we need to do this?  As long as
> it's *at least* 12 bit, then we're fine.
> 
> > - guest should pass full huge pages to host
> >   this should be done correctly to avoid breaking up huge pages
> >   I would say yes let's use a single format but drop the "normal chunk"
> >   and always use the extended one.
> >   Also, size is in units of 4k, right? Please document that low 12 bit
> >   are reserved, they will be handy as e.g. flags.
> 
> What per-chunk flags are you thinking would be useful?

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

* Re: [PATCH v7 kernel 3/5] virtio-balloon: implementation of VIRTIO_BALLOON_F_CHUNK_TRANSFER
  2017-03-10 17:11         ` Matthew Wilcox
  2017-03-10 19:10           ` Michael S. Tsirkin
@ 2017-03-10 19:35           ` Michael S. Tsirkin
  2017-03-10 21:25             ` Matthew Wilcox
  2017-03-11 11:59           ` Wei Wang
  2 siblings, 1 reply; 30+ messages in thread
From: Michael S. Tsirkin @ 2017-03-10 19:35 UTC (permalink / raw)
  To: Matthew Wilcox
  Cc: Wei Wang, virtio-dev, kvm, qemu-devel, linux-kernel,
	virtualization, linux-mm, Liang Li, Paolo Bonzini, Cornelia Huck,
	Amit Shah, Dave Hansen, Andrea Arcangeli, David Hildenbrand,
	Liang Li

On Fri, Mar 10, 2017 at 09:11:44AM -0800, Matthew Wilcox wrote:
> On Fri, Mar 10, 2017 at 05:58:28PM +0200, Michael S. Tsirkin wrote:
> > One of the issues of current balloon is the 4k page size
> > assumption. For example if you free a huge page you
> > have to split it up and pass 4k chunks to host.
> > Quite often host can't free these 4k chunks at all (e.g.
> > when it's using huge tlb fs).
> > It's even sillier for architectures with base page size >4k.
> 
> I completely agree with you that we should be able to pass a hugepage
> as a single chunk.  Also we shouldn't assume that host and guest have
> the same page size.  I think we can come up with a scheme that actually
> lets us encode that into a 64-bit word, something like this:
> 
> bit 0 clear => bits 1-11 encode a page count, bits 12-63 encode a PFN, page size 4k.
> bit 0 set, bit 1 clear => bits 2-12 encode a page count, bits 13-63 encode a PFN, page size 8k
> bits 0+1 set, bit 2 clear => bits 3-13 for page count, bits 14-63 for PFN, page size 16k.
> bits 0-2 set, bit 3 clear => bits 4-14 for page count, bits 15-63 for PFN, page size 32k
> bits 0-3 set, bit 4 clear => bits 5-15 for page count, bits 16-63 for PFN, page size 64k
> That means we can always pass 2048 pages (of whatever page size) in a single chunk.  And
> we support arbitrary power of two page sizes.  I suggest something like this:
> 
> u64 page_to_chunk(struct page *page)
> {
> 	u64 chunk = page_to_pfn(page) << PAGE_SHIFT;
> 	chunk |= (1UL << compound_order(page)) - 1;
> }

You need to fill in the size, do you not?

> 
> (note this is a single page of order N, so we leave the page count bits
> set to 0, meaning one page).
> 
> > Two things to consider:
> > - host should pass its base page size to guest
> >   this can be a separate patch and for now we can fall back on 12 bit if not there
> 
> With this encoding scheme, I don't think we need to do this?  As long as
> it's *at least* 12 bit, then we're fine.

I think we will still need something like this down the road.  The point
is that not all hosts are able to use 4k pages in a balloon.
So it's pointless for guest to pass 4k pages to such a host,
and we need host to tell guest the page size it needs.

However that's a separate feature that can wait until
another day.

> > - guest should pass full huge pages to host
> >   this should be done correctly to avoid breaking up huge pages
> >   I would say yes let's use a single format but drop the "normal chunk"
> >   and always use the extended one.
> >   Also, size is in units of 4k, right? Please document that low 12 bit
> >   are reserved, they will be handy as e.g. flags.
> 
> What per-chunk flags are you thinking would be useful?

Not entirely sure but I think would have been prudent to leave some free
if possible. Your encoding seems to use them all up, so be it.


-- 
MST

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

* Re: [PATCH v7 kernel 3/5] virtio-balloon: implementation of VIRTIO_BALLOON_F_CHUNK_TRANSFER
  2017-03-10 19:10           ` Michael S. Tsirkin
@ 2017-03-10 21:18             ` Matthew Wilcox
  0 siblings, 0 replies; 30+ messages in thread
From: Matthew Wilcox @ 2017-03-10 21:18 UTC (permalink / raw)
  To: Michael S. Tsirkin
  Cc: Wei Wang, virtio-dev, kvm, qemu-devel, linux-kernel,
	virtualization, linux-mm, Liang Li, Paolo Bonzini, Cornelia Huck,
	Amit Shah, Dave Hansen, Andrea Arcangeli, David Hildenbrand,
	Liang Li

On Fri, Mar 10, 2017 at 09:10:53PM +0200, Michael S. Tsirkin wrote:
> > I completely agree with you that we should be able to pass a hugepage
> > as a single chunk.  Also we shouldn't assume that host and guest have
> > the same page size.  I think we can come up with a scheme that actually
> > lets us encode that into a 64-bit word, something like this:
> > 
> > bit 0 clear => bits 1-11 encode a page count, bits 12-63 encode a PFN, page size 4k.
> > bit 0 set, bit 1 clear => bits 2-12 encode a page count, bits 13-63 encode a PFN, page size 8k
> > bits 0+1 set, bit 2 clear => bits 3-13 for page count, bits 14-63 for PFN, page size 16k.
> > bits 0-2 set, bit 3 clear => bits 4-14 for page count, bits 15-63 for PFN, page size 32k
> > bits 0-3 set, bit 4 clear => bits 5-15 for page count, bits 16-63 for PFN, page size 64k
> 
> huge page sizes go up to gigabytes.

There was supposed to be a '...' there.  For a 16GB hugepage (largest
size I know of today), that'd be:

bits 0-21 set, 22 clear, 23-33 page count, 34-63 PFN, page size 16G

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

* Re: [PATCH v7 kernel 3/5] virtio-balloon: implementation of VIRTIO_BALLOON_F_CHUNK_TRANSFER
  2017-03-10 19:35           ` Michael S. Tsirkin
@ 2017-03-10 21:25             ` Matthew Wilcox
  2017-03-12  0:05               ` Michael S. Tsirkin
  0 siblings, 1 reply; 30+ messages in thread
From: Matthew Wilcox @ 2017-03-10 21:25 UTC (permalink / raw)
  To: Michael S. Tsirkin
  Cc: Wei Wang, virtio-dev, kvm, qemu-devel, linux-kernel,
	virtualization, linux-mm, Liang Li, Paolo Bonzini, Cornelia Huck,
	Amit Shah, Dave Hansen, Andrea Arcangeli, David Hildenbrand,
	Liang Li

On Fri, Mar 10, 2017 at 09:35:21PM +0200, Michael S. Tsirkin wrote:
> > bit 0 clear => bits 1-11 encode a page count, bits 12-63 encode a PFN, page size 4k.
> > bit 0 set, bit 1 clear => bits 2-12 encode a page count, bits 13-63 encode a PFN, page size 8k
> > bits 0+1 set, bit 2 clear => bits 3-13 for page count, bits 14-63 for PFN, page size 16k.
> > bits 0-2 set, bit 3 clear => bits 4-14 for page count, bits 15-63 for PFN, page size 32k
> > bits 0-3 set, bit 4 clear => bits 5-15 for page count, bits 16-63 for PFN, page size 64k
> > That means we can always pass 2048 pages (of whatever page size) in a single chunk.  And
> > we support arbitrary power of two page sizes.  I suggest something like this:
> > 
> > u64 page_to_chunk(struct page *page)
> > {
> > 	u64 chunk = page_to_pfn(page) << PAGE_SHIFT;
> > 	chunk |= (1UL << compound_order(page)) - 1;
> > }
> 
> You need to fill in the size, do you not?

I think I did ... (1UL << compound_order(page)) - 1 sets the bottom
N bits.  Bit N+1 will already be clear.  What am I missing?

> > > - host should pass its base page size to guest
> > >   this can be a separate patch and for now we can fall back on 12 bit if not there
> > 
> > With this encoding scheme, I don't think we need to do this?  As long as
> > it's *at least* 12 bit, then we're fine.
> 
> I think we will still need something like this down the road.  The point
> is that not all hosts are able to use 4k pages in a balloon.
> So it's pointless for guest to pass 4k pages to such a host,
> and we need host to tell guest the page size it needs.
> 
> However that's a separate feature that can wait until
> another day.

Ah, the TRIM/DISCARD debate all over again ... should the guest batch
up or should the host do that work ... probably easier to account it in
the guest.  Might be better to frame it as 'balloon chunk size' rather than
host page size as it might have nothing to do with the host page size.

> > What per-chunk flags are you thinking would be useful?
> 
> Not entirely sure but I think would have been prudent to leave some free
> if possible. Your encoding seems to use them all up, so be it.

We don't necessarily have to support 2048 pages in a single chunk.
If it's worth reserving some bits, we can do that at the expense of
reducing the maximum number of pages per chunk.

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

* Re: [PATCH v7 kernel 3/5] virtio-balloon: implementation of VIRTIO_BALLOON_F_CHUNK_TRANSFER
  2017-03-10 17:11         ` Matthew Wilcox
  2017-03-10 19:10           ` Michael S. Tsirkin
  2017-03-10 19:35           ` Michael S. Tsirkin
@ 2017-03-11 11:59           ` Wei Wang
  2017-03-11 14:09             ` Matthew Wilcox
  2017-03-12  0:07             ` Michael S. Tsirkin
  2 siblings, 2 replies; 30+ messages in thread
From: Wei Wang @ 2017-03-11 11:59 UTC (permalink / raw)
  To: Matthew Wilcox
  Cc: Michael S. Tsirkin, virtio-dev, kvm, qemu-devel, linux-kernel,
	virtualization, linux-mm, Liang Li, Paolo Bonzini, Cornelia Huck,
	Amit Shah, Dave Hansen, Andrea Arcangeli, David Hildenbrand,
	Liang Li

On 03/11/2017 01:11 AM, Matthew Wilcox wrote:
> On Fri, Mar 10, 2017 at 05:58:28PM +0200, Michael S. Tsirkin wrote:
>> One of the issues of current balloon is the 4k page size
>> assumption. For example if you free a huge page you
>> have to split it up and pass 4k chunks to host.
>> Quite often host can't free these 4k chunks at all (e.g.
>> when it's using huge tlb fs).
>> It's even sillier for architectures with base page size >4k.
> I completely agree with you that we should be able to pass a hugepage
> as a single chunk.  Also we shouldn't assume that host and guest have
> the same page size.  I think we can come up with a scheme that actually
> lets us encode that into a 64-bit word, something like this:
>
> bit 0 clear => bits 1-11 encode a page count, bits 12-63 encode a PFN, page size 4k.
> bit 0 set, bit 1 clear => bits 2-12 encode a page count, bits 13-63 encode a PFN, page size 8k
> bits 0+1 set, bit 2 clear => bits 3-13 for page count, bits 14-63 for PFN, page size 16k.
> bits 0-2 set, bit 3 clear => bits 4-14 for page count, bits 15-63 for PFN, page size 32k
> bits 0-3 set, bit 4 clear => bits 5-15 for page count, bits 16-63 for PFN, page size 64k
>
> That means we can always pass 2048 pages (of whatever page size) in a single chunk.  And
> we support arbitrary power of two page sizes.  I suggest something like this:
>
> u64 page_to_chunk(struct page *page)
> {
> 	u64 chunk = page_to_pfn(page) << PAGE_SHIFT;
> 	chunk |= (1UL << compound_order(page)) - 1;
> }
>
> (note this is a single page of order N, so we leave the page count bits
> set to 0, meaning one page).
>

I'm thinking what if the guest needs to transfer these much physically 
continuous
memory to host: 1GB+2MB+64KB+32KB+16KB+4KB.
Is it going to use Six 64-bit chunks? Would it be simpler if we just
use the 128-bit chunk format (we can drop the previous normal 64-bit 
format)?

Best,
Wei

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

* Re: [PATCH v7 kernel 3/5] virtio-balloon: implementation of VIRTIO_BALLOON_F_CHUNK_TRANSFER
  2017-03-11 11:59           ` Wei Wang
@ 2017-03-11 14:09             ` Matthew Wilcox
  2017-03-12  1:59               ` Wang, Wei W
  2017-03-12  0:07             ` Michael S. Tsirkin
  1 sibling, 1 reply; 30+ messages in thread
From: Matthew Wilcox @ 2017-03-11 14:09 UTC (permalink / raw)
  To: Wei Wang
  Cc: Michael S. Tsirkin, virtio-dev, kvm, qemu-devel, linux-kernel,
	virtualization, linux-mm, Liang Li, Paolo Bonzini, Cornelia Huck,
	Amit Shah, Dave Hansen, Andrea Arcangeli, David Hildenbrand,
	Liang Li

On Sat, Mar 11, 2017 at 07:59:31PM +0800, Wei Wang wrote:
> I'm thinking what if the guest needs to transfer these much physically
> continuous
> memory to host: 1GB+2MB+64KB+32KB+16KB+4KB.
> Is it going to use Six 64-bit chunks? Would it be simpler if we just
> use the 128-bit chunk format (we can drop the previous normal 64-bit
> format)?

Is that a likely thing for the guest to need to do though?  Freeing a
1GB page is much more liikely, IMO.

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

* Re: [PATCH v7 kernel 3/5] virtio-balloon: implementation of VIRTIO_BALLOON_F_CHUNK_TRANSFER
  2017-03-10 21:25             ` Matthew Wilcox
@ 2017-03-12  0:05               ` Michael S. Tsirkin
  0 siblings, 0 replies; 30+ messages in thread
From: Michael S. Tsirkin @ 2017-03-12  0:05 UTC (permalink / raw)
  To: Matthew Wilcox
  Cc: Wei Wang, virtio-dev, kvm, qemu-devel, linux-kernel,
	virtualization, linux-mm, Liang Li, Paolo Bonzini, Cornelia Huck,
	Amit Shah, Dave Hansen, Andrea Arcangeli, David Hildenbrand,
	Liang Li

On Fri, Mar 10, 2017 at 01:25:41PM -0800, Matthew Wilcox wrote:
> On Fri, Mar 10, 2017 at 09:35:21PM +0200, Michael S. Tsirkin wrote:
> > > bit 0 clear => bits 1-11 encode a page count, bits 12-63 encode a PFN, page size 4k.
> > > bit 0 set, bit 1 clear => bits 2-12 encode a page count, bits 13-63 encode a PFN, page size 8k
> > > bits 0+1 set, bit 2 clear => bits 3-13 for page count, bits 14-63 for PFN, page size 16k.
> > > bits 0-2 set, bit 3 clear => bits 4-14 for page count, bits 15-63 for PFN, page size 32k
> > > bits 0-3 set, bit 4 clear => bits 5-15 for page count, bits 16-63 for PFN, page size 64k
> > > That means we can always pass 2048 pages (of whatever page size) in a single chunk.  And
> > > we support arbitrary power of two page sizes.  I suggest something like this:
> > > 
> > > u64 page_to_chunk(struct page *page)
> > > {
> > > 	u64 chunk = page_to_pfn(page) << PAGE_SHIFT;
> > > 	chunk |= (1UL << compound_order(page)) - 1;
> > > }
> > 
> > You need to fill in the size, do you not?
> 
> I think I did ... (1UL << compound_order(page)) - 1 sets the bottom
> N bits.  Bit N+1 will already be clear.  What am I missing?

This sets the order but not the number of pages.
For that you would do something like

	chunk |= size << compound_order(page)

right?

> > > > - host should pass its base page size to guest
> > > >   this can be a separate patch and for now we can fall back on 12 bit if not there
> > > 
> > > With this encoding scheme, I don't think we need to do this?  As long as
> > > it's *at least* 12 bit, then we're fine.
> > 
> > I think we will still need something like this down the road.  The point
> > is that not all hosts are able to use 4k pages in a balloon.
> > So it's pointless for guest to pass 4k pages to such a host,
> > and we need host to tell guest the page size it needs.
> > 
> > However that's a separate feature that can wait until
> > another day.
> 
> Ah, the TRIM/DISCARD debate all over again ... should the guest batch
> up or should the host do that work ... probably easier to account it in
> the guest.  Might be better to frame it as 'balloon chunk size' rather than
> host page size as it might have nothing to do with the host page size.

Exactly.

> > > What per-chunk flags are you thinking would be useful?
> > 
> > Not entirely sure but I think would have been prudent to leave some free
> > if possible. Your encoding seems to use them all up, so be it.
> 
> We don't necessarily have to support 2048 pages in a single chunk.
> If it's worth reserving some bits, we can do that at the expense of
> reducing the maximum number of pages per chunk.

Well we can always change things with a feature bit ..
I'll leave this up to you and Wei.

-- 
MST

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

* Re: [PATCH v7 kernel 3/5] virtio-balloon: implementation of VIRTIO_BALLOON_F_CHUNK_TRANSFER
  2017-03-11 11:59           ` Wei Wang
  2017-03-11 14:09             ` Matthew Wilcox
@ 2017-03-12  0:07             ` Michael S. Tsirkin
  1 sibling, 0 replies; 30+ messages in thread
From: Michael S. Tsirkin @ 2017-03-12  0:07 UTC (permalink / raw)
  To: Wei Wang
  Cc: Matthew Wilcox, virtio-dev, kvm, qemu-devel, linux-kernel,
	virtualization, linux-mm, Liang Li, Paolo Bonzini, Cornelia Huck,
	Amit Shah, Dave Hansen, Andrea Arcangeli, David Hildenbrand,
	Liang Li

On Sat, Mar 11, 2017 at 07:59:31PM +0800, Wei Wang wrote:
> On 03/11/2017 01:11 AM, Matthew Wilcox wrote:
> > On Fri, Mar 10, 2017 at 05:58:28PM +0200, Michael S. Tsirkin wrote:
> > > One of the issues of current balloon is the 4k page size
> > > assumption. For example if you free a huge page you
> > > have to split it up and pass 4k chunks to host.
> > > Quite often host can't free these 4k chunks at all (e.g.
> > > when it's using huge tlb fs).
> > > It's even sillier for architectures with base page size >4k.
> > I completely agree with you that we should be able to pass a hugepage
> > as a single chunk.  Also we shouldn't assume that host and guest have
> > the same page size.  I think we can come up with a scheme that actually
> > lets us encode that into a 64-bit word, something like this:
> > 
> > bit 0 clear => bits 1-11 encode a page count, bits 12-63 encode a PFN, page size 4k.
> > bit 0 set, bit 1 clear => bits 2-12 encode a page count, bits 13-63 encode a PFN, page size 8k
> > bits 0+1 set, bit 2 clear => bits 3-13 for page count, bits 14-63 for PFN, page size 16k.
> > bits 0-2 set, bit 3 clear => bits 4-14 for page count, bits 15-63 for PFN, page size 32k
> > bits 0-3 set, bit 4 clear => bits 5-15 for page count, bits 16-63 for PFN, page size 64k
> > 
> > That means we can always pass 2048 pages (of whatever page size) in a single chunk.  And
> > we support arbitrary power of two page sizes.  I suggest something like this:
> > 
> > u64 page_to_chunk(struct page *page)
> > {
> > 	u64 chunk = page_to_pfn(page) << PAGE_SHIFT;
> > 	chunk |= (1UL << compound_order(page)) - 1;
> > }
> > 
> > (note this is a single page of order N, so we leave the page count bits
> > set to 0, meaning one page).
> > 
> 
> I'm thinking what if the guest needs to transfer these much physically
> continuous
> memory to host: 1GB+2MB+64KB+32KB+16KB+4KB.
> Is it going to use Six 64-bit chunks? Would it be simpler if we just
> use the 128-bit chunk format (we can drop the previous normal 64-bit
> format)?
> 
> Best,
> Wei

I think I prefer that as a more straightforward approach, but
I can live with either approach.


-- 
MST

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

* RE: [PATCH v7 kernel 3/5] virtio-balloon: implementation of VIRTIO_BALLOON_F_CHUNK_TRANSFER
  2017-03-11 14:09             ` Matthew Wilcox
@ 2017-03-12  1:59               ` Wang, Wei W
  2017-03-12  4:04                 ` Michael S. Tsirkin
  0 siblings, 1 reply; 30+ messages in thread
From: Wang, Wei W @ 2017-03-12  1:59 UTC (permalink / raw)
  To: Matthew Wilcox
  Cc: Michael S. Tsirkin, virtio-dev, kvm, qemu-devel, linux-kernel,
	virtualization, linux-mm, Paolo Bonzini, Cornelia Huck,
	Amit Shah, Hansen, Dave, Andrea Arcangeli, David Hildenbrand,
	Liang Li

On 03/11/2017 10:10 PM, Matthew Wilcox wrote:
> On Sat, Mar 11, 2017 at 07:59:31PM +0800, Wei Wang wrote:
> > I'm thinking what if the guest needs to transfer these much physically
> > continuous memory to host: 1GB+2MB+64KB+32KB+16KB+4KB.
> > Is it going to use Six 64-bit chunks? Would it be simpler if we just
> > use the 128-bit chunk format (we can drop the previous normal 64-bit
> > format)?
> 
> Is that a likely thing for the guest to need to do though?  Freeing a 1GB page is
> much more liikely, IMO.

Yes, I think it's very possible. The host can ask for any number of pages (e.g. 1.5GB) that the guest can afford.  Also, the ballooned 1.5G memory is not guaranteed to be continuous in any pattern like 1GB+512MB. That's why we need to use a bitmap to draw the whole picture first, and then seek for continuous bits to chunk.

Best,
Wei

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

* Re: [PATCH v7 kernel 3/5] virtio-balloon: implementation of VIRTIO_BALLOON_F_CHUNK_TRANSFER
  2017-03-12  1:59               ` Wang, Wei W
@ 2017-03-12  4:04                 ` Michael S. Tsirkin
  2017-03-13 12:41                   ` Wang, Wei W
  0 siblings, 1 reply; 30+ messages in thread
From: Michael S. Tsirkin @ 2017-03-12  4:04 UTC (permalink / raw)
  To: Wang, Wei W
  Cc: Matthew Wilcox, virtio-dev, kvm, qemu-devel, linux-kernel,
	virtualization, linux-mm, Paolo Bonzini, Cornelia Huck,
	Amit Shah, Hansen, Dave, Andrea Arcangeli, David Hildenbrand,
	Liang Li

On Sun, Mar 12, 2017 at 01:59:54AM +0000, Wang, Wei W wrote:
> On 03/11/2017 10:10 PM, Matthew Wilcox wrote:
> > On Sat, Mar 11, 2017 at 07:59:31PM +0800, Wei Wang wrote:
> > > I'm thinking what if the guest needs to transfer these much physically
> > > continuous memory to host: 1GB+2MB+64KB+32KB+16KB+4KB.
> > > Is it going to use Six 64-bit chunks? Would it be simpler if we just
> > > use the 128-bit chunk format (we can drop the previous normal 64-bit
> > > format)?
> > 
> > Is that a likely thing for the guest to need to do though?  Freeing a 1GB page is
> > much more liikely, IMO.
> 
> Yes, I think it's very possible. The host can ask for any number of pages (e.g. 1.5GB) that the guest can afford.  Also, the ballooned 1.5G memory is not guaranteed to be continuous in any pattern like 1GB+512MB. That's why we need to use a bitmap to draw the whole picture first, and then seek for continuous bits to chunk.
> 
> Best,
> Wei

While I like the clever format that Matthew came up with, I'm also
inclined to say let's keep things simple.
the simplest thing seems to be to use the ext format all the time.
Except let's reserve the low 12 bits in both address and size,
since they are already 0, we might be able to use them for flags down the road.

-- 
MST

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

* RE: [PATCH v7 kernel 3/5] virtio-balloon: implementation of VIRTIO_BALLOON_F_CHUNK_TRANSFER
  2017-03-12  4:04                 ` Michael S. Tsirkin
@ 2017-03-13 12:41                   ` Wang, Wei W
  0 siblings, 0 replies; 30+ messages in thread
From: Wang, Wei W @ 2017-03-13 12:41 UTC (permalink / raw)
  To: Michael S. Tsirkin
  Cc: Matthew Wilcox, virtio-dev, kvm, qemu-devel, linux-kernel,
	virtualization, linux-mm, Paolo Bonzini, Cornelia Huck,
	Amit Shah, Hansen, Dave, Andrea Arcangeli, David Hildenbrand,
	Liang Li

On Sunday, March 12, 2017 12:04 PM, Michael S. Tsirkin wrote:
> On Sun, Mar 12, 2017 at 01:59:54AM +0000, Wang, Wei W wrote:
> > On 03/11/2017 10:10 PM, Matthew Wilcox wrote:
> > > On Sat, Mar 11, 2017 at 07:59:31PM +0800, Wei Wang wrote:
> > > > I'm thinking what if the guest needs to transfer these much
> > > > physically continuous memory to host: 1GB+2MB+64KB+32KB+16KB+4KB.
> > > > Is it going to use Six 64-bit chunks? Would it be simpler if we
> > > > just use the 128-bit chunk format (we can drop the previous normal
> > > > 64-bit format)?
> > >
> > > Is that a likely thing for the guest to need to do though?  Freeing
> > > a 1GB page is much more liikely, IMO.
> >
> > Yes, I think it's very possible. The host can ask for any number of pages (e.g.
> 1.5GB) that the guest can afford.  Also, the ballooned 1.5G memory is not
> guaranteed to be continuous in any pattern like 1GB+512MB. That's why we
> need to use a bitmap to draw the whole picture first, and then seek for
> continuous bits to chunk.
> >
> > Best,
> > Wei
> 
> While I like the clever format that Matthew came up with, I'm also inclined to
> say let's keep things simple.
> the simplest thing seems to be to use the ext format all the time.
> Except let's reserve the low 12 bits in both address and size, since they are
> already 0, we might be able to use them for flags down the road.

Thanks for reminding us about the hugepage story. I'll use the ext format in the implementation if no further objections from others.

Best,
Wei

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

end of thread, other threads:[~2017-03-13 12:41 UTC | newest]

Thread overview: 30+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2017-03-03  5:40 [PATCH v7 kernel 0/5] Extend virtio-balloon for fast (de)inflating & fast live migration Wei Wang
2017-03-03  5:40 ` [PATCH v7 kernel 1/5] virtio-balloon: rework deflate to add page to a list Wei Wang
2017-03-03  5:40 ` [PATCH v7 kernel 2/5] virtio-balloon: VIRTIO_BALLOON_F_CHUNK_TRANSFER Wei Wang
2017-03-08  4:01   ` Michael S. Tsirkin
2017-03-09  7:12     ` Wei Wang
2017-03-03  5:40 ` [PATCH v7 kernel 3/5] virtio-balloon: implementation of VIRTIO_BALLOON_F_CHUNK_TRANSFER Wei Wang
2017-03-08  4:01   ` Michael S. Tsirkin
2017-03-10 10:02     ` [virtio-dev] " Wei Wang
2017-03-10 13:26       ` David Hildenbrand
2017-03-10 15:37       ` Michael S. Tsirkin
2017-03-09 14:14   ` Matthew Wilcox
2017-03-10 11:37     ` Wei Wang
2017-03-10 15:58       ` Michael S. Tsirkin
2017-03-10 17:11         ` Matthew Wilcox
2017-03-10 19:10           ` Michael S. Tsirkin
2017-03-10 21:18             ` Matthew Wilcox
2017-03-10 19:35           ` Michael S. Tsirkin
2017-03-10 21:25             ` Matthew Wilcox
2017-03-12  0:05               ` Michael S. Tsirkin
2017-03-11 11:59           ` Wei Wang
2017-03-11 14:09             ` Matthew Wilcox
2017-03-12  1:59               ` Wang, Wei W
2017-03-12  4:04                 ` Michael S. Tsirkin
2017-03-13 12:41                   ` Wang, Wei W
2017-03-12  0:07             ` Michael S. Tsirkin
2017-03-03  5:40 ` [PATCH v7 kernel 4/5] virtio-balloon: define flags and head for host request vq Wei Wang
2017-03-08  4:02   ` Michael S. Tsirkin
2017-03-03  5:40 ` [PATCH v7 kernel 5/5] This patch contains two parts: Wei Wang
2017-03-06 13:23   ` David Hildenbrand
2017-03-09  7:04     ` Wei Wang

This is a public inbox, see mirroring instructions
for how to clone and mirror all data and code used for this inbox;
as well as URLs for NNTP newsgroup(s).