linux-kernel.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [PATCH v2 0/5] KVM: x86: hyperv: PV IPI support for Windows guests
@ 2018-06-28 13:53 Vitaly Kuznetsov
  2018-06-28 13:53 ` [PATCH v2 1/5] KVM: x86: hyperv: optimize 'all cpus' case in kvm_hv_flush_tlb() Vitaly Kuznetsov
                   ` (4 more replies)
  0 siblings, 5 replies; 18+ messages in thread
From: Vitaly Kuznetsov @ 2018-06-28 13:53 UTC (permalink / raw)
  To: kvm
  Cc: x86, Paolo Bonzini, Radim Krčmář,
	Roman Kagan, K. Y. Srinivasan, Haiyang Zhang, Stephen Hemminger,
	Michael Kelley (EOSG),
	Mohammed Gamal, Cathy Avery, Wanpeng Li, linux-kernel

Changes since v1 [Radim Krčmář]:
- Implement VP_IDX -> VCPU_IDX mapping, use it for both PV IPIs and
  TLB flush (PATCHes 1-3 added)
- Minor tweaks.

Using hypercall for sending IPIs is faster because this allows to specify
any number of vCPUs (even > 64 with sparse CPU set), the whole procedure
will take only one VMEXIT.

Same as PV TLB flush, this allows Windows guests having > 64 vCPUs to boot
on KVM when Hyper-V extensions are enabled.

Vitaly Kuznetsov (5):
  KVM: x86: hyperv: optimize 'all cpus' case in kvm_hv_flush_tlb()
  KVM: x86: hyperv: introduce vp_index_to_vcpu_idx mapping
  KVM: x86: hyperv: use vp_idx_to_vcpu_idx() in kvm_hv_flush_tlb()
  x86/hyper-v: rename ipi_arg_{ex,non_ex} structures
  KVM: x86: hyperv: implement PV IPI send hypercalls

 Documentation/virtual/kvm/api.txt  |   8 +
 arch/x86/hyperv/hv_apic.c          |  12 +-
 arch/x86/include/asm/hyperv-tlfs.h |  16 +-
 arch/x86/include/asm/kvm_host.h    |  12 ++
 arch/x86/kvm/hyperv.c              | 375 ++++++++++++++++++++++++++++++-------
 arch/x86/kvm/trace.h               |  42 +++++
 arch/x86/kvm/x86.c                 |   1 +
 include/uapi/linux/kvm.h           |   1 +
 virt/kvm/kvm_main.c                |   6 +-
 9 files changed, 391 insertions(+), 82 deletions(-)

-- 
2.14.4


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

* [PATCH v2 1/5] KVM: x86: hyperv: optimize 'all cpus' case in kvm_hv_flush_tlb()
  2018-06-28 13:53 [PATCH v2 0/5] KVM: x86: hyperv: PV IPI support for Windows guests Vitaly Kuznetsov
@ 2018-06-28 13:53 ` Vitaly Kuznetsov
  2018-06-28 13:53 ` [PATCH v2 2/5] KVM: x86: hyperv: introduce vp_index_to_vcpu_idx mapping Vitaly Kuznetsov
                   ` (3 subsequent siblings)
  4 siblings, 0 replies; 18+ messages in thread
From: Vitaly Kuznetsov @ 2018-06-28 13:53 UTC (permalink / raw)
  To: kvm
  Cc: x86, Paolo Bonzini, Radim Krčmář,
	Roman Kagan, K. Y. Srinivasan, Haiyang Zhang, Stephen Hemminger,
	Michael Kelley (EOSG),
	Mohammed Gamal, Cathy Avery, Wanpeng Li, linux-kernel

We can use 'NULL' to represent 'all cpus' case in
kvm_make_vcpus_request_mask() and avoid building vCPU mask with
all vCPUs.

Suggested-by: Radim Krčmář <rkrcmar@redhat.com>
Signed-off-by: Vitaly Kuznetsov <vkuznets@redhat.com>
---
 arch/x86/kvm/hyperv.c | 42 +++++++++++++++++++++++-------------------
 virt/kvm/kvm_main.c   |  6 ++----
 2 files changed, 25 insertions(+), 23 deletions(-)

diff --git a/arch/x86/kvm/hyperv.c b/arch/x86/kvm/hyperv.c
index af8caf965baa..63a17bbbf0e5 100644
--- a/arch/x86/kvm/hyperv.c
+++ b/arch/x86/kvm/hyperv.c
@@ -1310,35 +1310,39 @@ static u64 kvm_hv_flush_tlb(struct kvm_vcpu *current_vcpu, u64 ingpa,
 
 	cpumask_clear(&hv_current->tlb_lush);
 
+	if (all_cpus) {
+		kvm_make_vcpus_request_mask(kvm,
+				    KVM_REQ_TLB_FLUSH | KVM_REQUEST_NO_WAKEUP,
+				    NULL, &hv_current->tlb_lush);
+		goto ret_success;
+	}
+
 	kvm_for_each_vcpu(i, vcpu, kvm) {
 		struct kvm_vcpu_hv *hv = &vcpu->arch.hyperv;
 		int bank = hv->vp_index / 64, sbank = 0;
 
-		if (!all_cpus) {
-			/* Banks >64 can't be represented */
-			if (bank >= 64)
-				continue;
-
-			/* Non-ex hypercalls can only address first 64 vCPUs */
-			if (!ex && bank)
-				continue;
+		/* Banks >64 can't be represented */
+		if (bank >= 64)
+			continue;
 
-			if (ex) {
-				/*
-				 * Check is the bank of this vCPU is in sparse
-				 * set and get the sparse bank number.
-				 */
-				sbank = get_sparse_bank_no(valid_bank_mask,
-							   bank);
+		/* Non-ex hypercalls can only address first 64 vCPUs */
+		if (!ex && bank)
+			continue;
 
-				if (sbank < 0)
-					continue;
-			}
+		if (ex) {
+			/*
+			 * Check is the bank of this vCPU is in sparse
+			 * set and get the sparse bank number.
+			 */
+			sbank = get_sparse_bank_no(valid_bank_mask, bank);
 
-			if (!(sparse_banks[sbank] & BIT_ULL(hv->vp_index % 64)))
+			if (sbank < 0)
 				continue;
 		}
 
+		if (!(sparse_banks[sbank] & BIT_ULL(hv->vp_index % 64)))
+			continue;
+
 		/*
 		 * vcpu->arch.cr3 may not be up-to-date for running vCPUs so we
 		 * can't analyze it here, flush TLB regardless of the specified
diff --git a/virt/kvm/kvm_main.c b/virt/kvm/kvm_main.c
index ada21f47f22b..153f14e91fb1 100644
--- a/virt/kvm/kvm_main.c
+++ b/virt/kvm/kvm_main.c
@@ -213,7 +213,7 @@ bool kvm_make_vcpus_request_mask(struct kvm *kvm, unsigned int req,
 	me = get_cpu();
 
 	kvm_for_each_vcpu(i, vcpu, kvm) {
-		if (!test_bit(i, vcpu_bitmap))
+		if (vcpu_bitmap && !test_bit(i, vcpu_bitmap))
 			continue;
 
 		kvm_make_request(req, vcpu);
@@ -237,12 +237,10 @@ bool kvm_make_all_cpus_request(struct kvm *kvm, unsigned int req)
 {
 	cpumask_var_t cpus;
 	bool called;
-	static unsigned long vcpu_bitmap[BITS_TO_LONGS(KVM_MAX_VCPUS)]
-		= {[0 ... BITS_TO_LONGS(KVM_MAX_VCPUS)-1] = ULONG_MAX};
 
 	zalloc_cpumask_var(&cpus, GFP_ATOMIC);
 
-	called = kvm_make_vcpus_request_mask(kvm, req, vcpu_bitmap, cpus);
+	called = kvm_make_vcpus_request_mask(kvm, req, NULL, cpus);
 
 	free_cpumask_var(cpus);
 	return called;
-- 
2.14.4


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

* [PATCH v2 2/5] KVM: x86: hyperv: introduce vp_index_to_vcpu_idx mapping
  2018-06-28 13:53 [PATCH v2 0/5] KVM: x86: hyperv: PV IPI support for Windows guests Vitaly Kuznetsov
  2018-06-28 13:53 ` [PATCH v2 1/5] KVM: x86: hyperv: optimize 'all cpus' case in kvm_hv_flush_tlb() Vitaly Kuznetsov
@ 2018-06-28 13:53 ` Vitaly Kuznetsov
  2018-06-28 18:49   ` kbuild test robot
                     ` (3 more replies)
  2018-06-28 13:53 ` [PATCH v2 3/5] KVM: x86: hyperv: use vp_idx_to_vcpu_idx() in kvm_hv_flush_tlb() Vitaly Kuznetsov
                   ` (2 subsequent siblings)
  4 siblings, 4 replies; 18+ messages in thread
From: Vitaly Kuznetsov @ 2018-06-28 13:53 UTC (permalink / raw)
  To: kvm
  Cc: x86, Paolo Bonzini, Radim Krčmář,
	Roman Kagan, K. Y. Srinivasan, Haiyang Zhang, Stephen Hemminger,
	Michael Kelley (EOSG),
	Mohammed Gamal, Cathy Avery, Wanpeng Li, linux-kernel

While it is easy to get VP index from vCPU index the reverse task is hard.
Basically, to solve it we have to walk all vCPUs checking if their VP index
matches. For hypercalls like HvFlushVirtualAddress{List,Space}* and the
upcoming HvSendSyntheticClusterIpi* where a single CPU may be specified in
the whole set this is obviously sub-optimal.

As VP index can be set to anything <= U32_MAX by userspace using plain
[0..MAX_VP_INDEX] array is not a viable option. Use condensed sorted
array with logarithmic search complexity instead. Use RCU to make read
access as fast as possible and maintain atomicity of updates.

Give vp_idx == U32_MAX a special 'remove only' meaning to keep
vp_idx_to_vcpu_idx()/vp_idx_update() interfaces simple.

Signed-off-by: Vitaly Kuznetsov <vkuznets@redhat.com>
---
 arch/x86/include/asm/kvm_host.h |  12 +++
 arch/x86/kvm/hyperv.c           | 190 ++++++++++++++++++++++++++++++++++++----
 2 files changed, 187 insertions(+), 15 deletions(-)

diff --git a/arch/x86/include/asm/kvm_host.h b/arch/x86/include/asm/kvm_host.h
index c13cd28d9d1b..2dfcdcaaa696 100644
--- a/arch/x86/include/asm/kvm_host.h
+++ b/arch/x86/include/asm/kvm_host.h
@@ -747,6 +747,16 @@ struct kvm_apic_map {
 	struct kvm_lapic *phys_map[];
 };
 
+/* Mapping from VP number to vCPU idx */
+struct vp_idx_map {
+	struct rcu_head rcu;
+	int len;
+	struct {
+		u32 vp_idx;
+		u32 vcpu_idx;
+	} vp_idx_elem[];
+};
+
 /* Hyper-V emulation context */
 struct kvm_hv {
 	struct mutex hv_lock;
@@ -765,6 +775,8 @@ struct kvm_hv {
 	u64 hv_reenlightenment_control;
 	u64 hv_tsc_emulation_control;
 	u64 hv_tsc_emulation_status;
+
+	struct vp_idx_map *vp_idx_map;
 };
 
 enum kvm_irqchip_mode {
diff --git a/arch/x86/kvm/hyperv.c b/arch/x86/kvm/hyperv.c
index 63a17bbbf0e5..d676335a5448 100644
--- a/arch/x86/kvm/hyperv.c
+++ b/arch/x86/kvm/hyperv.c
@@ -127,19 +127,161 @@ static int synic_set_sint(struct kvm_vcpu_hv_synic *synic, int sint,
 	return 0;
 }
 
+static void vp_idx_map_free(struct rcu_head *rcu)
+{
+	struct vp_idx_map *map = container_of(rcu, struct vp_idx_map, rcu);
+
+	kfree(map);
+}
+
+static u32 vp_idx_to_vcpu_idx(struct kvm *kvm, u32 vp_idx)
+{
+	struct kvm_hv *hv = &kvm->arch.hyperv;
+	u32 vcpu_idx = U32_MAX, tmp_vp_idx;
+	int l_index = 0, r_index, tmp_index;
+	struct vp_idx_map *map;
+
+	/*
+	 * Make an educated guess: vp_idx is initialized to == vcpu_idx, it
+	 * stays this way unless changed by userspace.
+	 */
+	if (vp_idx < KVM_MAX_VCPUS) {
+		struct kvm_vcpu *vcpu = kvm_get_vcpu(kvm, vp_idx);
+
+		if (vcpu && vcpu_to_hv_vcpu(vcpu)->vp_index == vp_idx)
+			return vp_idx;
+	}
+
+	rcu_read_lock();
+	map = rcu_dereference(hv->vp_idx_map);
+	if (!map)
+		goto unlock;
+
+	r_index = map->len - 1;
+
+	while (r_index >= l_index) {
+		tmp_index = (r_index + l_index)/2;
+		tmp_vp_idx = map->vp_idx_elem[tmp_index].vp_idx;
+
+		if (tmp_vp_idx == vp_idx) {
+			vcpu_idx = map->vp_idx_elem[tmp_index].vcpu_idx;
+			break;
+		} else if (tmp_vp_idx < vp_idx) {
+			l_index = tmp_index + 1;
+		} else /* (tmp_vp_idx > vp_idx) */ {
+			r_index = tmp_index - 1;
+		}
+	};
+
+unlock:
+	rcu_read_unlock();
+
+	return vcpu_idx;
+}
+
+/*
+ * Atomically updates vp_idx map removing old and adding new vp_idx->vcpu_idx
+ * mapping. vp_idx == U32_MAX means only the old mapping should be removed.
+ */
+static int vp_idx_update(struct kvm_hv *hv, u32 vp_idx, u32 vcpu_idx)
+{
+	struct vp_idx_map *new, *old;
+	int i, add = 1, remove = 1, nindex, oindex;
+	u32 vp_idx_old = U32_MAX;
+	bool added = false;
+	int ret;
+
+	mutex_lock(&hv->hv_lock);
+	old = rcu_dereference_protected(hv->vp_idx_map, &hv->hv_lock);
+	if (!old) {
+		ret = -EFAULT;
+		goto unlock_exit;
+	}
+
+	if (vp_idx == U32_MAX)
+		add = 0;
+
+	for (i = 0; i < old->len; i++) {
+		/* Check if we have stale mapping for vcpu_idx */
+		if (old->vp_idx_elem[i].vcpu_idx == vcpu_idx)
+			vp_idx_old = old->vp_idx_elem[i].vp_idx;
+
+		/* Check if we have another mapping for vp_idx */
+		if (old->vp_idx_elem[i].vp_idx == vp_idx) {
+			ret = -EEXIST;
+			goto unlock_exit;
+		}
+	}
+
+	if (vp_idx_old == U32_MAX)
+		remove = 0;
+
+	new = kmalloc(sizeof(*new) + sizeof(new->vp_idx_elem[0]) *
+		      (old->len + add - remove), GFP_KERNEL);
+	if (!new) {
+		ret = -ENOMEM;
+		goto unlock_exit;
+	}
+	new->len = old->len + add - remove;
+
+	for (nindex = 0, oindex = 0; nindex < new->len; nindex++, oindex++) {
+		/* Appending new element to the very end */
+		if (oindex == old->len) {
+			new->vp_idx_elem[nindex].vp_idx = vp_idx;
+			new->vp_idx_elem[nindex].vcpu_idx = vcpu_idx;
+			break;
+		}
+
+		/* Removing old mapping */
+		if (old->vp_idx_elem[oindex].vp_idx == vp_idx_old) {
+			nindex--;
+			continue;
+		}
+
+		/* Inserting new element */
+		if (old->vp_idx_elem[oindex].vp_idx > vp_idx && !added) {
+			added = true;
+			new->vp_idx_elem[nindex].vp_idx = vp_idx;
+			new->vp_idx_elem[nindex].vcpu_idx = vcpu_idx;
+
+			/*
+			 * We will add the old->vp_idx_elem[oindex] element on
+			 * the next iteration.
+			 */
+			oindex--;
+			continue;
+		}
+
+		/* Nothing special, just copy */
+		new->vp_idx_elem[nindex].vp_idx =
+			old->vp_idx_elem[oindex].vp_idx;
+		new->vp_idx_elem[nindex].vcpu_idx =
+			old->vp_idx_elem[oindex].vcpu_idx;
+
+	}
+	rcu_assign_pointer(hv->vp_idx_map, new);
+	ret = 0;
+
+unlock_exit:
+	mutex_unlock(&hv->hv_lock);
+
+	if (!ret)
+		call_rcu(&old->rcu, vp_idx_map_free);
+
+	return ret;
+}
+
 static struct kvm_vcpu *get_vcpu_by_vpidx(struct kvm *kvm, u32 vpidx)
 {
 	struct kvm_vcpu *vcpu = NULL;
-	int i;
+	u32 vcpu_idx;
+
+	vcpu_idx = vp_idx_to_vcpu_idx(kvm, vpidx);
 
-	if (vpidx < KVM_MAX_VCPUS)
-		vcpu = kvm_get_vcpu(kvm, vpidx);
-	if (vcpu && vcpu_to_hv_vcpu(vcpu)->vp_index == vpidx)
-		return vcpu;
-	kvm_for_each_vcpu(i, vcpu, kvm)
-		if (vcpu_to_hv_vcpu(vcpu)->vp_index == vpidx)
-			return vcpu;
-	return NULL;
+	if (vcpu_idx < KVM_MAX_VCPUS)
+		vcpu = kvm_get_vcpu(kvm, vcpu_idx);
+
+	return vcpu;
 }
 
 static struct kvm_vcpu_hv_synic *synic_get(struct kvm *kvm, u32 vpidx)
@@ -686,6 +828,8 @@ void kvm_hv_vcpu_uninit(struct kvm_vcpu *vcpu)
 
 	for (i = 0; i < ARRAY_SIZE(hv_vcpu->stimer); i++)
 		stimer_cleanup(&hv_vcpu->stimer[i]);
+
+	vp_idx_update(&vcpu->kvm->arch.hyperv, U32_MAX, kvm_vcpu_get_idx(vcpu));
 }
 
 static void stimer_prepare_msg(struct kvm_vcpu_hv_stimer *stimer)
@@ -727,8 +871,12 @@ void kvm_hv_vcpu_init(struct kvm_vcpu *vcpu)
 void kvm_hv_vcpu_postcreate(struct kvm_vcpu *vcpu)
 {
 	struct kvm_vcpu_hv *hv_vcpu = vcpu_to_hv_vcpu(vcpu);
+	struct kvm_hv *hv = &vcpu->kvm->arch.hyperv;
+	int idx;
 
-	hv_vcpu->vp_index = kvm_vcpu_get_idx(vcpu);
+	idx = kvm_vcpu_get_idx(vcpu);
+	if (!vp_idx_update(hv, idx, idx))
+		hv_vcpu->vp_index = idx;
 }
 
 int kvm_hv_activate_synic(struct kvm_vcpu *vcpu, bool dont_zero_synic_pages)
@@ -1038,7 +1186,12 @@ static int kvm_hv_set_msr(struct kvm_vcpu *vcpu, u32 msr, u64 data, bool host)
 
 	switch (msr) {
 	case HV_X64_MSR_VP_INDEX:
-		if (!host)
+		if (!host || (u32)data == U32_MAX)
+			return 1;
+		if (hv->vp_index == (u32)data)
+			break;
+		if (vp_idx_update(&vcpu->kvm->arch.hyperv, (u32)data,
+				  kvm_vcpu_get_idx(vcpu)))
 			return 1;
 		hv->vp_index = (u32)data;
 		break;
@@ -1540,18 +1693,25 @@ int kvm_hv_hypercall(struct kvm_vcpu *vcpu)
 
 void kvm_hv_init_vm(struct kvm *kvm)
 {
-	mutex_init(&kvm->arch.hyperv.hv_lock);
-	idr_init(&kvm->arch.hyperv.conn_to_evt);
+	struct kvm_hv *hv = &kvm->arch.hyperv;
+
+	mutex_init(&hv->hv_lock);
+	idr_init(&hv->conn_to_evt);
+
+	hv->vp_idx_map = kzalloc(sizeof(*hv->vp_idx_map), GFP_KERNEL);
 }
 
 void kvm_hv_destroy_vm(struct kvm *kvm)
 {
+	struct kvm_hv *hv = &kvm->arch.hyperv;
 	struct eventfd_ctx *eventfd;
 	int i;
 
-	idr_for_each_entry(&kvm->arch.hyperv.conn_to_evt, eventfd, i)
+	idr_for_each_entry(&hv->conn_to_evt, eventfd, i)
 		eventfd_ctx_put(eventfd);
-	idr_destroy(&kvm->arch.hyperv.conn_to_evt);
+	idr_destroy(&hv->conn_to_evt);
+
+	kfree(hv->vp_idx_map);
 }
 
 static int kvm_hv_eventfd_assign(struct kvm *kvm, u32 conn_id, int fd)
-- 
2.14.4


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

* [PATCH v2 3/5] KVM: x86: hyperv: use vp_idx_to_vcpu_idx() in kvm_hv_flush_tlb()
  2018-06-28 13:53 [PATCH v2 0/5] KVM: x86: hyperv: PV IPI support for Windows guests Vitaly Kuznetsov
  2018-06-28 13:53 ` [PATCH v2 1/5] KVM: x86: hyperv: optimize 'all cpus' case in kvm_hv_flush_tlb() Vitaly Kuznetsov
  2018-06-28 13:53 ` [PATCH v2 2/5] KVM: x86: hyperv: introduce vp_index_to_vcpu_idx mapping Vitaly Kuznetsov
@ 2018-06-28 13:53 ` Vitaly Kuznetsov
  2018-06-28 13:53 ` [PATCH v2 4/5] x86/hyper-v: rename ipi_arg_{ex,non_ex} structures Vitaly Kuznetsov
  2018-06-28 13:53 ` [PATCH v2 5/5] KVM: x86: hyperv: implement PV IPI send hypercalls Vitaly Kuznetsov
  4 siblings, 0 replies; 18+ messages in thread
From: Vitaly Kuznetsov @ 2018-06-28 13:53 UTC (permalink / raw)
  To: kvm
  Cc: x86, Paolo Bonzini, Radim Krčmář,
	Roman Kagan, K. Y. Srinivasan, Haiyang Zhang, Stephen Hemminger,
	Michael Kelley (EOSG),
	Mohammed Gamal, Cathy Avery, Wanpeng Li, linux-kernel

As we now have a fast way to get vCPU index from VP number we can
avoid walking through all vCPUs in kvm_hv_flush_tlb().

Signed-off-by: Vitaly Kuznetsov <vkuznets@redhat.com>
---
 arch/x86/kvm/hyperv.c | 59 ++++++++++++++-------------------------------------
 1 file changed, 16 insertions(+), 43 deletions(-)

diff --git a/arch/x86/kvm/hyperv.c b/arch/x86/kvm/hyperv.c
index d676335a5448..9983304d6664 100644
--- a/arch/x86/kvm/hyperv.c
+++ b/arch/x86/kvm/hyperv.c
@@ -1395,20 +1395,6 @@ int kvm_hv_get_msr_common(struct kvm_vcpu *vcpu, u32 msr, u64 *pdata)
 		return kvm_hv_get_msr(vcpu, msr, pdata);
 }
 
-static __always_inline int get_sparse_bank_no(u64 valid_bank_mask, int bank_no)
-{
-	int i = 0, j;
-
-	if (!(valid_bank_mask & BIT_ULL(bank_no)))
-		return -1;
-
-	for (j = 0; j < bank_no; j++)
-		if (valid_bank_mask & BIT_ULL(j))
-			i++;
-
-	return i;
-}
-
 static u64 kvm_hv_flush_tlb(struct kvm_vcpu *current_vcpu, u64 ingpa,
 			    u16 rep_cnt, bool ex)
 {
@@ -1416,11 +1402,10 @@ static u64 kvm_hv_flush_tlb(struct kvm_vcpu *current_vcpu, u64 ingpa,
 	struct kvm_vcpu_hv *hv_current = &current_vcpu->arch.hyperv;
 	struct hv_tlb_flush_ex flush_ex;
 	struct hv_tlb_flush flush;
-	struct kvm_vcpu *vcpu;
 	unsigned long vcpu_bitmap[BITS_TO_LONGS(KVM_MAX_VCPUS)] = {0};
-	unsigned long valid_bank_mask = 0;
+	unsigned long valid_bank_mask;
 	u64 sparse_banks[64];
-	int sparse_banks_len, i;
+	int sparse_banks_len, bank, i;
 	bool all_cpus;
 
 	if (!ex) {
@@ -1430,6 +1415,7 @@ static u64 kvm_hv_flush_tlb(struct kvm_vcpu *current_vcpu, u64 ingpa,
 		trace_kvm_hv_flush_tlb(flush.processor_mask,
 				       flush.address_space, flush.flags);
 
+		valid_bank_mask = BIT_ULL(0);
 		sparse_banks[0] = flush.processor_mask;
 		all_cpus = flush.flags & HV_FLUSH_ALL_PROCESSORS;
 	} else {
@@ -1470,38 +1456,25 @@ static u64 kvm_hv_flush_tlb(struct kvm_vcpu *current_vcpu, u64 ingpa,
 		goto ret_success;
 	}
 
-	kvm_for_each_vcpu(i, vcpu, kvm) {
-		struct kvm_vcpu_hv *hv = &vcpu->arch.hyperv;
-		int bank = hv->vp_index / 64, sbank = 0;
+	for_each_set_bit(bank, (unsigned long *)&valid_bank_mask,
+			 BITS_PER_LONG) {
 
-		/* Banks >64 can't be represented */
-		if (bank >= 64)
-			continue;
+		for_each_set_bit(i, (unsigned long *)&sparse_banks[bank],
+				 BITS_PER_LONG) {
+			u32 vp_index = bank * 64 + i;
+			u32 vcpu_idx = vp_idx_to_vcpu_idx(kvm, vp_index);
 
-		/* Non-ex hypercalls can only address first 64 vCPUs */
-		if (!ex && bank)
-			continue;
+			/* A non-existent vCPU was specified */
+			if (vcpu_idx >= atomic_read(&kvm->online_vcpus))
+				return HV_STATUS_INVALID_HYPERCALL_INPUT;
 
-		if (ex) {
 			/*
-			 * Check is the bank of this vCPU is in sparse
-			 * set and get the sparse bank number.
+			 * vcpu->arch.cr3 may not be up-to-date for running
+			 * vCPUs so we can't analyze it here, flush TLB
+			 * regardless of the specified address space.
 			 */
-			sbank = get_sparse_bank_no(valid_bank_mask, bank);
-
-			if (sbank < 0)
-				continue;
+			__set_bit(vcpu_idx, vcpu_bitmap);
 		}
-
-		if (!(sparse_banks[sbank] & BIT_ULL(hv->vp_index % 64)))
-			continue;
-
-		/*
-		 * vcpu->arch.cr3 may not be up-to-date for running vCPUs so we
-		 * can't analyze it here, flush TLB regardless of the specified
-		 * address space.
-		 */
-		__set_bit(i, vcpu_bitmap);
 	}
 
 	kvm_make_vcpus_request_mask(kvm,
-- 
2.14.4


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

* [PATCH v2 4/5] x86/hyper-v: rename ipi_arg_{ex,non_ex} structures
  2018-06-28 13:53 [PATCH v2 0/5] KVM: x86: hyperv: PV IPI support for Windows guests Vitaly Kuznetsov
                   ` (2 preceding siblings ...)
  2018-06-28 13:53 ` [PATCH v2 3/5] KVM: x86: hyperv: use vp_idx_to_vcpu_idx() in kvm_hv_flush_tlb() Vitaly Kuznetsov
@ 2018-06-28 13:53 ` Vitaly Kuznetsov
  2018-06-28 13:53 ` [PATCH v2 5/5] KVM: x86: hyperv: implement PV IPI send hypercalls Vitaly Kuznetsov
  4 siblings, 0 replies; 18+ messages in thread
From: Vitaly Kuznetsov @ 2018-06-28 13:53 UTC (permalink / raw)
  To: kvm
  Cc: x86, Paolo Bonzini, Radim Krčmář,
	Roman Kagan, K. Y. Srinivasan, Haiyang Zhang, Stephen Hemminger,
	Michael Kelley (EOSG),
	Mohammed Gamal, Cathy Avery, Wanpeng Li, linux-kernel

These structures are going to be used from KVM code so let's make
their names reflect their Hyper-V origin.

Signed-off-by: Vitaly Kuznetsov <vkuznets@redhat.com>
---
 arch/x86/hyperv/hv_apic.c          | 12 ++++++------
 arch/x86/include/asm/hyperv-tlfs.h | 16 +++++++++-------
 2 files changed, 15 insertions(+), 13 deletions(-)

diff --git a/arch/x86/hyperv/hv_apic.c b/arch/x86/hyperv/hv_apic.c
index f68855499391..cb17168e6263 100644
--- a/arch/x86/hyperv/hv_apic.c
+++ b/arch/x86/hyperv/hv_apic.c
@@ -93,14 +93,14 @@ static void hv_apic_eoi_write(u32 reg, u32 val)
  */
 static bool __send_ipi_mask_ex(const struct cpumask *mask, int vector)
 {
-	struct ipi_arg_ex **arg;
-	struct ipi_arg_ex *ipi_arg;
+	struct hv_send_ipi_ex **arg;
+	struct hv_send_ipi_ex *ipi_arg;
 	unsigned long flags;
 	int nr_bank = 0;
 	int ret = 1;
 
 	local_irq_save(flags);
-	arg = (struct ipi_arg_ex **)this_cpu_ptr(hyperv_pcpu_input_arg);
+	arg = (struct hv_send_ipi_ex **)this_cpu_ptr(hyperv_pcpu_input_arg);
 
 	ipi_arg = *arg;
 	if (unlikely(!ipi_arg))
@@ -128,8 +128,8 @@ static bool __send_ipi_mask_ex(const struct cpumask *mask, int vector)
 static bool __send_ipi_mask(const struct cpumask *mask, int vector)
 {
 	int cur_cpu, vcpu;
-	struct ipi_arg_non_ex **arg;
-	struct ipi_arg_non_ex *ipi_arg;
+	struct hv_send_ipi **arg;
+	struct hv_send_ipi *ipi_arg;
 	int ret = 1;
 	unsigned long flags;
 
@@ -146,7 +146,7 @@ static bool __send_ipi_mask(const struct cpumask *mask, int vector)
 		return __send_ipi_mask_ex(mask, vector);
 
 	local_irq_save(flags);
-	arg = (struct ipi_arg_non_ex **)this_cpu_ptr(hyperv_pcpu_input_arg);
+	arg = (struct hv_send_ipi **)this_cpu_ptr(hyperv_pcpu_input_arg);
 
 	ipi_arg = *arg;
 	if (unlikely(!ipi_arg))
diff --git a/arch/x86/include/asm/hyperv-tlfs.h b/arch/x86/include/asm/hyperv-tlfs.h
index b8c89265baf0..b52c9604b20d 100644
--- a/arch/x86/include/asm/hyperv-tlfs.h
+++ b/arch/x86/include/asm/hyperv-tlfs.h
@@ -723,19 +723,21 @@ struct hv_enlightened_vmcs {
 #define HV_STIMER_AUTOENABLE		(1ULL << 3)
 #define HV_STIMER_SINT(config)		(__u8)(((config) >> 16) & 0x0F)
 
-struct ipi_arg_non_ex {
-	u32 vector;
-	u32 reserved;
-	u64 cpu_mask;
-};
-
 struct hv_vpset {
 	u64 format;
 	u64 valid_bank_mask;
 	u64 bank_contents[];
 };
 
-struct ipi_arg_ex {
+/* HvCallSendSyntheticClusterIpi hypercall */
+struct hv_send_ipi {
+	u32 vector;
+	u32 reserved;
+	u64 cpu_mask;
+};
+
+/* HvCallSendSyntheticClusterIpiEx hypercall */
+struct hv_send_ipi_ex {
 	u32 vector;
 	u32 reserved;
 	struct hv_vpset vp_set;
-- 
2.14.4


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

* [PATCH v2 5/5] KVM: x86: hyperv: implement PV IPI send hypercalls
  2018-06-28 13:53 [PATCH v2 0/5] KVM: x86: hyperv: PV IPI support for Windows guests Vitaly Kuznetsov
                   ` (3 preceding siblings ...)
  2018-06-28 13:53 ` [PATCH v2 4/5] x86/hyper-v: rename ipi_arg_{ex,non_ex} structures Vitaly Kuznetsov
@ 2018-06-28 13:53 ` Vitaly Kuznetsov
  4 siblings, 0 replies; 18+ messages in thread
From: Vitaly Kuznetsov @ 2018-06-28 13:53 UTC (permalink / raw)
  To: kvm
  Cc: x86, Paolo Bonzini, Radim Krčmář,
	Roman Kagan, K. Y. Srinivasan, Haiyang Zhang, Stephen Hemminger,
	Michael Kelley (EOSG),
	Mohammed Gamal, Cathy Avery, Wanpeng Li, linux-kernel

Using hypercall for sending IPIs is faster because this allows to specify
any number of vCPUs (even > 64 with sparse CPU set), the whole procedure
will take only one VMEXIT.

Current Hyper-V TLFS (v5.0b) claims that HvCallSendSyntheticClusterIpi
hypercall can't be 'fast' (passing parameters through registers) but
apparently this is not true, Windows always uses it as 'fast' so we need
to support that.

Signed-off-by: Vitaly Kuznetsov <vkuznets@redhat.com>
---
 Documentation/virtual/kvm/api.txt |   8 +++
 arch/x86/kvm/hyperv.c             | 108 ++++++++++++++++++++++++++++++++++++++
 arch/x86/kvm/trace.h              |  42 +++++++++++++++
 arch/x86/kvm/x86.c                |   1 +
 include/uapi/linux/kvm.h          |   1 +
 5 files changed, 160 insertions(+)

diff --git a/Documentation/virtual/kvm/api.txt b/Documentation/virtual/kvm/api.txt
index 495b7742ab58..1f3a9ea6a59c 100644
--- a/Documentation/virtual/kvm/api.txt
+++ b/Documentation/virtual/kvm/api.txt
@@ -4618,3 +4618,11 @@ This capability indicates that KVM supports paravirtualized Hyper-V TLB Flush
 hypercalls:
 HvFlushVirtualAddressSpace, HvFlushVirtualAddressSpaceEx,
 HvFlushVirtualAddressList, HvFlushVirtualAddressListEx.
+
+8.19 KVM_CAP_HYPERV_SEND_IPI
+
+Architectures: x86
+
+This capability indicates that KVM supports paravirtualized Hyper-V IPI send
+hypercalls:
+HvCallSendSyntheticClusterIpi, HvCallSendSyntheticClusterIpiEx.
diff --git a/arch/x86/kvm/hyperv.c b/arch/x86/kvm/hyperv.c
index 9983304d6664..15b1bc8baa0e 100644
--- a/arch/x86/kvm/hyperv.c
+++ b/arch/x86/kvm/hyperv.c
@@ -1487,6 +1487,100 @@ static u64 kvm_hv_flush_tlb(struct kvm_vcpu *current_vcpu, u64 ingpa,
 		((u64)rep_cnt << HV_HYPERCALL_REP_COMP_OFFSET);
 }
 
+static u64 kvm_hv_send_ipi(struct kvm_vcpu *current_vcpu, u64 ingpa, u64 outgpa,
+			   bool ex, bool fast)
+{
+	struct kvm *kvm = current_vcpu->kvm;
+	struct hv_send_ipi_ex send_ipi_ex;
+	struct hv_send_ipi send_ipi;
+	struct kvm_vcpu *vcpu;
+	unsigned long valid_bank_mask;
+	u64 sparse_banks[64];
+	int sparse_banks_len, bank, i;
+	struct kvm_lapic_irq irq = {.delivery_mode = APIC_DM_FIXED};
+	bool all_cpus;
+
+	if (!ex) {
+		if (!fast) {
+			if (unlikely(kvm_read_guest(kvm, ingpa, &send_ipi,
+						    sizeof(send_ipi))))
+				return HV_STATUS_INVALID_HYPERCALL_INPUT;
+			sparse_banks[0] = send_ipi.cpu_mask;
+			irq.vector = send_ipi.vector;
+		} else {
+			/* 'reserved' part of hv_send_ipi should be 0 */
+			if (unlikely(ingpa >> 32 != 0))
+				return HV_STATUS_INVALID_HYPERCALL_INPUT;
+			sparse_banks[0] = outgpa;
+			irq.vector = (u32)ingpa;
+		}
+		all_cpus = false;
+		valid_bank_mask = BIT_ULL(0);
+
+		trace_kvm_hv_send_ipi(irq.vector, sparse_banks[0]);
+	} else {
+		if (unlikely(kvm_read_guest(kvm, ingpa, &send_ipi_ex,
+					    sizeof(send_ipi_ex))))
+			return HV_STATUS_INVALID_HYPERCALL_INPUT;
+
+		trace_kvm_hv_send_ipi_ex(send_ipi_ex.vector,
+					 send_ipi_ex.vp_set.format,
+					 send_ipi_ex.vp_set.valid_bank_mask);
+
+		irq.vector = send_ipi_ex.vector;
+		valid_bank_mask = send_ipi_ex.vp_set.valid_bank_mask;
+		sparse_banks_len = bitmap_weight(&valid_bank_mask, 64) *
+			sizeof(sparse_banks[0]);
+
+		all_cpus = send_ipi_ex.vp_set.format == HV_GENERIC_SET_ALL;
+
+		if (!sparse_banks_len)
+			goto ret_success;
+
+		if (!all_cpus &&
+		    kvm_read_guest(kvm,
+				   ingpa + offsetof(struct hv_send_ipi_ex,
+						    vp_set.bank_contents),
+				   sparse_banks,
+				   sparse_banks_len))
+			return HV_STATUS_INVALID_HYPERCALL_INPUT;
+	}
+
+	if ((irq.vector < HV_IPI_LOW_VECTOR) ||
+	    (irq.vector > HV_IPI_HIGH_VECTOR))
+		return HV_STATUS_INVALID_HYPERCALL_INPUT;
+
+	if (all_cpus) {
+		kvm_for_each_vcpu(i, vcpu, kvm) {
+			/* We fail only when APIC is disabled */
+			if (!kvm_apic_set_irq(vcpu, &irq, NULL))
+				return HV_STATUS_INVALID_HYPERCALL_INPUT;
+		}
+		goto ret_success;
+	}
+
+	for_each_set_bit(bank, (unsigned long *)&valid_bank_mask,
+			 BITS_PER_LONG) {
+
+		for_each_set_bit(i, (unsigned long *)&sparse_banks[bank],
+				 BITS_PER_LONG) {
+			u32 vp_index = bank * 64 + i;
+
+			/* Unknown vCPU specified */
+			vcpu = get_vcpu_by_vpidx(kvm, vp_index);
+			if (!vcpu)
+				return HV_STATUS_INVALID_HYPERCALL_INPUT;
+
+			/* We fail only when APIC is disabled */
+			if (!kvm_apic_set_irq(vcpu, &irq, NULL))
+				return HV_STATUS_INVALID_HYPERCALL_INPUT;
+		}
+	}
+
+ret_success:
+	return HV_STATUS_SUCCESS;
+}
+
 bool kvm_hv_hypercall_enabled(struct kvm *kvm)
 {
 	return READ_ONCE(kvm->arch.hyperv.hv_hypercall) & HV_X64_MSR_HYPERCALL_ENABLE;
@@ -1656,6 +1750,20 @@ int kvm_hv_hypercall(struct kvm_vcpu *vcpu)
 		}
 		ret = kvm_hv_flush_tlb(vcpu, ingpa, rep_cnt, true);
 		break;
+	case HVCALL_SEND_IPI:
+		if (unlikely(rep)) {
+			ret = HV_STATUS_INVALID_HYPERCALL_INPUT;
+			break;
+		}
+		ret = kvm_hv_send_ipi(vcpu, ingpa, outgpa, false, fast);
+		break;
+	case HVCALL_SEND_IPI_EX:
+		if (unlikely(fast || rep)) {
+			ret = HV_STATUS_INVALID_HYPERCALL_INPUT;
+			break;
+		}
+		ret = kvm_hv_send_ipi(vcpu, ingpa, outgpa, true, false);
+		break;
 	default:
 		ret = HV_STATUS_INVALID_HYPERCALL_CODE;
 		break;
diff --git a/arch/x86/kvm/trace.h b/arch/x86/kvm/trace.h
index 0f997683404f..0659465a745c 100644
--- a/arch/x86/kvm/trace.h
+++ b/arch/x86/kvm/trace.h
@@ -1418,6 +1418,48 @@ TRACE_EVENT(kvm_hv_flush_tlb_ex,
 		  __entry->valid_bank_mask, __entry->format,
 		  __entry->address_space, __entry->flags)
 );
+
+/*
+ * Tracepoints for kvm_hv_send_ipi.
+ */
+TRACE_EVENT(kvm_hv_send_ipi,
+	TP_PROTO(u32 vector, u64 processor_mask),
+	TP_ARGS(vector, processor_mask),
+
+	TP_STRUCT__entry(
+		__field(u32, vector)
+		__field(u64, processor_mask)
+	),
+
+	TP_fast_assign(
+		__entry->vector = vector;
+		__entry->processor_mask = processor_mask;
+	),
+
+	TP_printk("vector %x processor_mask 0x%llx",
+		  __entry->vector, __entry->processor_mask)
+);
+
+TRACE_EVENT(kvm_hv_send_ipi_ex,
+	TP_PROTO(u32 vector, u64 format, u64 valid_bank_mask),
+	TP_ARGS(vector, format, valid_bank_mask),
+
+	TP_STRUCT__entry(
+		__field(u32, vector)
+		__field(u64, format)
+		__field(u64, valid_bank_mask)
+	),
+
+	TP_fast_assign(
+		__entry->vector = vector;
+		__entry->format = format;
+		__entry->valid_bank_mask = valid_bank_mask;
+	),
+
+	TP_printk("vector %x format %llx valid_bank_mask 0x%llx",
+		  __entry->vector, __entry->format,
+		  __entry->valid_bank_mask)
+);
 #endif /* _TRACE_KVM_H */
 
 #undef TRACE_INCLUDE_PATH
diff --git a/arch/x86/kvm/x86.c b/arch/x86/kvm/x86.c
index 0046aa70205a..1884b66de9c2 100644
--- a/arch/x86/kvm/x86.c
+++ b/arch/x86/kvm/x86.c
@@ -2874,6 +2874,7 @@ int kvm_vm_ioctl_check_extension(struct kvm *kvm, long ext)
 	case KVM_CAP_HYPERV_VP_INDEX:
 	case KVM_CAP_HYPERV_EVENTFD:
 	case KVM_CAP_HYPERV_TLBFLUSH:
+	case KVM_CAP_HYPERV_SEND_IPI:
 	case KVM_CAP_PCI_SEGMENT:
 	case KVM_CAP_DEBUGREGS:
 	case KVM_CAP_X86_ROBUST_SINGLESTEP:
diff --git a/include/uapi/linux/kvm.h b/include/uapi/linux/kvm.h
index b6270a3b38e9..adce915f80a5 100644
--- a/include/uapi/linux/kvm.h
+++ b/include/uapi/linux/kvm.h
@@ -949,6 +949,7 @@ struct kvm_ppc_resize_hpt {
 #define KVM_CAP_GET_MSR_FEATURES 153
 #define KVM_CAP_HYPERV_EVENTFD 154
 #define KVM_CAP_HYPERV_TLBFLUSH 155
+#define KVM_CAP_HYPERV_SEND_IPI 156
 
 #ifdef KVM_CAP_IRQ_ROUTING
 
-- 
2.14.4


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

* Re: [PATCH v2 2/5] KVM: x86: hyperv: introduce vp_index_to_vcpu_idx mapping
  2018-06-28 13:53 ` [PATCH v2 2/5] KVM: x86: hyperv: introduce vp_index_to_vcpu_idx mapping Vitaly Kuznetsov
@ 2018-06-28 18:49   ` kbuild test robot
  2018-06-28 19:09   ` [PATCH] KVM: x86: hyperv: fix semicolon.cocci warnings kbuild test robot
                     ` (2 subsequent siblings)
  3 siblings, 0 replies; 18+ messages in thread
From: kbuild test robot @ 2018-06-28 18:49 UTC (permalink / raw)
  To: Vitaly Kuznetsov
  Cc: kbuild-all, kvm, x86, Paolo Bonzini, Radim Krčmář,
	Roman Kagan, K. Y. Srinivasan, Haiyang Zhang, Stephen Hemminger,
	Michael Kelley (EOSG),
	Mohammed Gamal, Cathy Avery, Wanpeng Li, linux-kernel

Hi Vitaly,

Thank you for the patch! Perhaps something to improve:

[auto build test WARNING on kvm/linux-next]
[also build test WARNING on v4.18-rc2 next-20180628]
[if your patch is applied to the wrong git tree, please drop us a note to help improve the system]

url:    https://github.com/0day-ci/linux/commits/Vitaly-Kuznetsov/KVM-x86-hyperv-PV-IPI-support-for-Windows-guests/20180629-011544
base:   https://git.kernel.org/pub/scm/virt/kvm/kvm.git linux-next
reproduce:
        # apt-get install sparse
        make ARCH=x86_64 allmodconfig
        make C=1 CF=-D__CHECK_ENDIAN__


sparse warnings: (new ones prefixed by >>)

>> arch/x86/kvm/hyperv.c:156:15: sparse: incompatible types in comparison expression (different address spaces)
   arch/x86/kvm/hyperv.c:195:15: sparse: incompatible types in comparison expression (different address spaces)

vim +156 arch/x86/kvm/hyperv.c

   136	
   137	static u32 vp_idx_to_vcpu_idx(struct kvm *kvm, u32 vp_idx)
   138	{
   139		struct kvm_hv *hv = &kvm->arch.hyperv;
   140		u32 vcpu_idx = U32_MAX, tmp_vp_idx;
   141		int l_index = 0, r_index, tmp_index;
   142		struct vp_idx_map *map;
   143	
   144		/*
   145		 * Make an educated guess: vp_idx is initialized to == vcpu_idx, it
   146		 * stays this way unless changed by userspace.
   147		 */
   148		if (vp_idx < KVM_MAX_VCPUS) {
   149			struct kvm_vcpu *vcpu = kvm_get_vcpu(kvm, vp_idx);
   150	
   151			if (vcpu && vcpu_to_hv_vcpu(vcpu)->vp_index == vp_idx)
   152				return vp_idx;
   153		}
   154	
   155		rcu_read_lock();
 > 156		map = rcu_dereference(hv->vp_idx_map);
   157		if (!map)
   158			goto unlock;
   159	
   160		r_index = map->len - 1;
   161	
   162		while (r_index >= l_index) {
   163			tmp_index = (r_index + l_index)/2;
   164			tmp_vp_idx = map->vp_idx_elem[tmp_index].vp_idx;
   165	
   166			if (tmp_vp_idx == vp_idx) {
   167				vcpu_idx = map->vp_idx_elem[tmp_index].vcpu_idx;
   168				break;
   169			} else if (tmp_vp_idx < vp_idx) {
   170				l_index = tmp_index + 1;
   171			} else /* (tmp_vp_idx > vp_idx) */ {
   172				r_index = tmp_index - 1;
   173			}
   174		};
   175	
   176	unlock:
   177		rcu_read_unlock();
   178	
   179		return vcpu_idx;
   180	}
   181	

---
0-DAY kernel test infrastructure                Open Source Technology Center
https://lists.01.org/pipermail/kbuild-all                   Intel Corporation

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

* Re: [PATCH v2 2/5] KVM: x86: hyperv: introduce vp_index_to_vcpu_idx mapping
  2018-06-28 13:53 ` [PATCH v2 2/5] KVM: x86: hyperv: introduce vp_index_to_vcpu_idx mapping Vitaly Kuznetsov
  2018-06-28 18:49   ` kbuild test robot
  2018-06-28 19:09   ` [PATCH] KVM: x86: hyperv: fix semicolon.cocci warnings kbuild test robot
@ 2018-06-28 19:09   ` kbuild test robot
  2018-06-29 10:11   ` Roman Kagan
  3 siblings, 0 replies; 18+ messages in thread
From: kbuild test robot @ 2018-06-28 19:09 UTC (permalink / raw)
  To: Vitaly Kuznetsov
  Cc: kbuild-all, kvm, x86, Paolo Bonzini, Radim Krčmář,
	Roman Kagan, K. Y. Srinivasan, Haiyang Zhang, Stephen Hemminger,
	Michael Kelley (EOSG),
	Mohammed Gamal, Cathy Avery, Wanpeng Li, linux-kernel

Hi Vitaly,

Thank you for the patch! Perhaps something to improve:

[auto build test WARNING on kvm/linux-next]
[also build test WARNING on v4.18-rc2 next-20180628]
[if your patch is applied to the wrong git tree, please drop us a note to help improve the system]

url:    https://github.com/0day-ci/linux/commits/Vitaly-Kuznetsov/KVM-x86-hyperv-PV-IPI-support-for-Windows-guests/20180629-011544
base:   https://git.kernel.org/pub/scm/virt/kvm/kvm.git linux-next


coccinelle warnings: (new ones prefixed by >>)

>> arch/x86/kvm/hyperv.c:174:2-3: Unneeded semicolon

Please review and possibly fold the followup patch.

---
0-DAY kernel test infrastructure                Open Source Technology Center
https://lists.01.org/pipermail/kbuild-all                   Intel Corporation

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

* [PATCH] KVM: x86: hyperv: fix semicolon.cocci warnings
  2018-06-28 13:53 ` [PATCH v2 2/5] KVM: x86: hyperv: introduce vp_index_to_vcpu_idx mapping Vitaly Kuznetsov
  2018-06-28 18:49   ` kbuild test robot
@ 2018-06-28 19:09   ` kbuild test robot
  2018-06-28 19:09   ` [PATCH v2 2/5] KVM: x86: hyperv: introduce vp_index_to_vcpu_idx mapping kbuild test robot
  2018-06-29 10:11   ` Roman Kagan
  3 siblings, 0 replies; 18+ messages in thread
From: kbuild test robot @ 2018-06-28 19:09 UTC (permalink / raw)
  To: Vitaly Kuznetsov
  Cc: kbuild-all, kvm, x86, Paolo Bonzini, Radim Krčmář,
	Roman Kagan, K. Y. Srinivasan, Haiyang Zhang, Stephen Hemminger,
	Michael Kelley (EOSG),
	Mohammed Gamal, Cathy Avery, Wanpeng Li, linux-kernel

From: kbuild test robot <fengguang.wu@intel.com>

arch/x86/kvm/hyperv.c:174:2-3: Unneeded semicolon


 Remove unneeded semicolon.

Generated by: scripts/coccinelle/misc/semicolon.cocci

Fixes: 5282e205b1f9 ("KVM: x86: hyperv: introduce vp_index_to_vcpu_idx mapping")
CC: Vitaly Kuznetsov <vkuznets@redhat.com>
Signed-off-by: kbuild test robot <fengguang.wu@intel.com>
---

 hyperv.c |    2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

--- a/arch/x86/kvm/hyperv.c
+++ b/arch/x86/kvm/hyperv.c
@@ -171,7 +171,7 @@ static u32 vp_idx_to_vcpu_idx(struct kvm
 		} else /* (tmp_vp_idx > vp_idx) */ {
 			r_index = tmp_index - 1;
 		}
-	};
+	}
 
 unlock:
 	rcu_read_unlock();

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

* Re: [PATCH v2 2/5] KVM: x86: hyperv: introduce vp_index_to_vcpu_idx mapping
  2018-06-28 13:53 ` [PATCH v2 2/5] KVM: x86: hyperv: introduce vp_index_to_vcpu_idx mapping Vitaly Kuznetsov
                     ` (2 preceding siblings ...)
  2018-06-28 19:09   ` [PATCH v2 2/5] KVM: x86: hyperv: introduce vp_index_to_vcpu_idx mapping kbuild test robot
@ 2018-06-29 10:11   ` Roman Kagan
  2018-06-29 10:26     ` Vitaly Kuznetsov
  3 siblings, 1 reply; 18+ messages in thread
From: Roman Kagan @ 2018-06-29 10:11 UTC (permalink / raw)
  To: Vitaly Kuznetsov
  Cc: kvm, x86, Paolo Bonzini, Radim Krčmář,
	K. Y. Srinivasan, Haiyang Zhang, Stephen Hemminger,
	Michael Kelley (EOSG),
	Mohammed Gamal, Cathy Avery, Wanpeng Li, linux-kernel

On Thu, Jun 28, 2018 at 03:53:10PM +0200, Vitaly Kuznetsov wrote:
> While it is easy to get VP index from vCPU index the reverse task is hard.
> Basically, to solve it we have to walk all vCPUs checking if their VP index
> matches. For hypercalls like HvFlushVirtualAddress{List,Space}* and the
> upcoming HvSendSyntheticClusterIpi* where a single CPU may be specified in
> the whole set this is obviously sub-optimal.
> 
> As VP index can be set to anything <= U32_MAX by userspace using plain
> [0..MAX_VP_INDEX] array is not a viable option. Use condensed sorted
> array with logarithmic search complexity instead. Use RCU to make read
> access as fast as possible and maintain atomicity of updates.

Quoting TLFS 5.0C section 7.8.1:

> Virtual processors are identified by using an index (VP index). The
> maximum number of virtual processors per partition supported by the
> current implementation of the hypervisor can be obtained through CPUID
> leaf 0x40000005. A virtual processor index must be less than the
> maximum number of virtual processors per partition.

so this is a dense index, and VP_INDEX >= KVM_MAX_VCPUS is invalid.  I
think we're better off enforcing this in kvm_hv_set_msr and keep the
translation simple.  If the algorithm in get_vcpu_by_vpidx is not good
enough (and yes it can be made to return NULL early on vpidx >=
KVM_MAX_VCPUS instead of taking the slow path) then a simple index array
of KVM_MAX_VCPUS entries should certainly do.

Roman.

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

* Re: [PATCH v2 2/5] KVM: x86: hyperv: introduce vp_index_to_vcpu_idx mapping
  2018-06-29 10:11   ` Roman Kagan
@ 2018-06-29 10:26     ` Vitaly Kuznetsov
  2018-06-29 11:12       ` Roman Kagan
  0 siblings, 1 reply; 18+ messages in thread
From: Vitaly Kuznetsov @ 2018-06-29 10:26 UTC (permalink / raw)
  To: Roman Kagan
  Cc: kvm, x86, Paolo Bonzini, Radim Krčmář,
	K. Y. Srinivasan, Haiyang Zhang, Stephen Hemminger,
	Michael Kelley (EOSG),
	Mohammed Gamal, Cathy Avery, Wanpeng Li, linux-kernel

Roman Kagan <rkagan@virtuozzo.com> writes:

> On Thu, Jun 28, 2018 at 03:53:10PM +0200, Vitaly Kuznetsov wrote:
>> While it is easy to get VP index from vCPU index the reverse task is hard.
>> Basically, to solve it we have to walk all vCPUs checking if their VP index
>> matches. For hypercalls like HvFlushVirtualAddress{List,Space}* and the
>> upcoming HvSendSyntheticClusterIpi* where a single CPU may be specified in
>> the whole set this is obviously sub-optimal.
>> 
>> As VP index can be set to anything <= U32_MAX by userspace using plain
>> [0..MAX_VP_INDEX] array is not a viable option. Use condensed sorted
>> array with logarithmic search complexity instead. Use RCU to make read
>> access as fast as possible and maintain atomicity of updates.
>
> Quoting TLFS 5.0C section 7.8.1:
>
>> Virtual processors are identified by using an index (VP index). The
>> maximum number of virtual processors per partition supported by the
>> current implementation of the hypervisor can be obtained through CPUID
>> leaf 0x40000005. A virtual processor index must be less than the
>> maximum number of virtual processors per partition.
>
> so this is a dense index, and VP_INDEX >= KVM_MAX_VCPUS is invalid.  I
> think we're better off enforcing this in kvm_hv_set_msr and keep the
> translation simple.  If the algorithm in get_vcpu_by_vpidx is not good
> enough (and yes it can be made to return NULL early on vpidx >=
> KVM_MAX_VCPUS instead of taking the slow path) then a simple index array
> of KVM_MAX_VCPUS entries should certainly do.

Sure, we can use pre-allocated [0..KVM_MAX_VCPUS] array instead and put
limits on what userspace can assign VP_INDEX to. Howver, while thinking
about it I decided to go with the more complex condensed array approach
because the tendency is for KVM_MAX_VCPUS to grow and we will be
pre-allocating more and more memory for no particular reason (so I think
even 'struct kvm_vcpu *vcpus[KVM_MAX_VCPUS]' in 'struct kvm' will need
to be converted to something else eventually). 

Anyway, I'm flexible and if you think we should go this way now I'll do
this in v3. We can re-think this when we later decide to raise
KVM_MAX_VCPUS significantly.

-- 
  Vitaly

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

* Re: [PATCH v2 2/5] KVM: x86: hyperv: introduce vp_index_to_vcpu_idx mapping
  2018-06-29 10:26     ` Vitaly Kuznetsov
@ 2018-06-29 11:12       ` Roman Kagan
  2018-06-29 11:37         ` Vitaly Kuznetsov
  0 siblings, 1 reply; 18+ messages in thread
From: Roman Kagan @ 2018-06-29 11:12 UTC (permalink / raw)
  To: Vitaly Kuznetsov
  Cc: kvm, x86, Paolo Bonzini, Radim Krčmář,
	K. Y. Srinivasan, Haiyang Zhang, Stephen Hemminger,
	Michael Kelley (EOSG),
	Mohammed Gamal, Cathy Avery, Wanpeng Li, linux-kernel

On Fri, Jun 29, 2018 at 12:26:23PM +0200, Vitaly Kuznetsov wrote:
> Roman Kagan <rkagan@virtuozzo.com> writes:
> 
> > On Thu, Jun 28, 2018 at 03:53:10PM +0200, Vitaly Kuznetsov wrote:
> >> While it is easy to get VP index from vCPU index the reverse task is hard.
> >> Basically, to solve it we have to walk all vCPUs checking if their VP index
> >> matches. For hypercalls like HvFlushVirtualAddress{List,Space}* and the
> >> upcoming HvSendSyntheticClusterIpi* where a single CPU may be specified in
> >> the whole set this is obviously sub-optimal.
> >> 
> >> As VP index can be set to anything <= U32_MAX by userspace using plain
> >> [0..MAX_VP_INDEX] array is not a viable option. Use condensed sorted
> >> array with logarithmic search complexity instead. Use RCU to make read
> >> access as fast as possible and maintain atomicity of updates.
> >
> > Quoting TLFS 5.0C section 7.8.1:
> >
> >> Virtual processors are identified by using an index (VP index). The
> >> maximum number of virtual processors per partition supported by the
> >> current implementation of the hypervisor can be obtained through CPUID
> >> leaf 0x40000005. A virtual processor index must be less than the
> >> maximum number of virtual processors per partition.
> >
> > so this is a dense index, and VP_INDEX >= KVM_MAX_VCPUS is invalid.  I
> > think we're better off enforcing this in kvm_hv_set_msr and keep the
> > translation simple.  If the algorithm in get_vcpu_by_vpidx is not good
> > enough (and yes it can be made to return NULL early on vpidx >=
> > KVM_MAX_VCPUS instead of taking the slow path) then a simple index array
> > of KVM_MAX_VCPUS entries should certainly do.
> 
> Sure, we can use pre-allocated [0..KVM_MAX_VCPUS] array instead and put
> limits on what userspace can assign VP_INDEX to. Howver, while thinking
> about it I decided to go with the more complex condensed array approach
> because the tendency is for KVM_MAX_VCPUS to grow and we will be
> pre-allocating more and more memory for no particular reason (so I think
> even 'struct kvm_vcpu *vcpus[KVM_MAX_VCPUS]' in 'struct kvm' will need
> to be converted to something else eventually). 

We're talking of kilobytes here.  I guess this is going to be the least
of the scalability problems.

> Anyway, I'm flexible and if you think we should go this way now I'll do
> this in v3. We can re-think this when we later decide to raise
> KVM_MAX_VCPUS significantly.

Although there's no strict requirement for that I think every sensible
userspace will allocate VP_INDEX linearly resulting in it being equal to
KVM's vcpu index.  So we've yet to see a case where get_vcpu_by_vpidx
doesn't take the fast path.  If it ever starts appearing in the profiles
we may consider optimiziing it but ATM I don't even think introducing
the translation array is justified.

Roman.

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

* Re: [PATCH v2 2/5] KVM: x86: hyperv: introduce vp_index_to_vcpu_idx mapping
  2018-06-29 11:12       ` Roman Kagan
@ 2018-06-29 11:37         ` Vitaly Kuznetsov
  2018-06-29 12:52           ` Roman Kagan
  0 siblings, 1 reply; 18+ messages in thread
From: Vitaly Kuznetsov @ 2018-06-29 11:37 UTC (permalink / raw)
  To: Roman Kagan
  Cc: kvm, x86, Paolo Bonzini, Radim Krčmář,
	K. Y. Srinivasan, Haiyang Zhang, Stephen Hemminger,
	Michael Kelley (EOSG),
	Mohammed Gamal, Cathy Avery, Wanpeng Li, linux-kernel

Roman Kagan <rkagan@virtuozzo.com> writes:

> On Fri, Jun 29, 2018 at 12:26:23PM +0200, Vitaly Kuznetsov wrote:
>> Roman Kagan <rkagan@virtuozzo.com> writes:
>> 
>> > On Thu, Jun 28, 2018 at 03:53:10PM +0200, Vitaly Kuznetsov wrote:
>> >> While it is easy to get VP index from vCPU index the reverse task is hard.
>> >> Basically, to solve it we have to walk all vCPUs checking if their VP index
>> >> matches. For hypercalls like HvFlushVirtualAddress{List,Space}* and the
>> >> upcoming HvSendSyntheticClusterIpi* where a single CPU may be specified in
>> >> the whole set this is obviously sub-optimal.
>> >> 
>> >> As VP index can be set to anything <= U32_MAX by userspace using plain
>> >> [0..MAX_VP_INDEX] array is not a viable option. Use condensed sorted
>> >> array with logarithmic search complexity instead. Use RCU to make read
>> >> access as fast as possible and maintain atomicity of updates.
>> >
>> > Quoting TLFS 5.0C section 7.8.1:
>> >
>> >> Virtual processors are identified by using an index (VP index). The
>> >> maximum number of virtual processors per partition supported by the
>> >> current implementation of the hypervisor can be obtained through CPUID
>> >> leaf 0x40000005. A virtual processor index must be less than the
>> >> maximum number of virtual processors per partition.
>> >
>> > so this is a dense index, and VP_INDEX >= KVM_MAX_VCPUS is invalid.  I
>> > think we're better off enforcing this in kvm_hv_set_msr and keep the
>> > translation simple.  If the algorithm in get_vcpu_by_vpidx is not good
>> > enough (and yes it can be made to return NULL early on vpidx >=
>> > KVM_MAX_VCPUS instead of taking the slow path) then a simple index array
>> > of KVM_MAX_VCPUS entries should certainly do.
>> 
>> Sure, we can use pre-allocated [0..KVM_MAX_VCPUS] array instead and put
>> limits on what userspace can assign VP_INDEX to. Howver, while thinking
>> about it I decided to go with the more complex condensed array approach
>> because the tendency is for KVM_MAX_VCPUS to grow and we will be
>> pre-allocating more and more memory for no particular reason (so I think
>> even 'struct kvm_vcpu *vcpus[KVM_MAX_VCPUS]' in 'struct kvm' will need
>> to be converted to something else eventually). 
>
> We're talking of kilobytes here.  I guess this is going to be the least
> of the scalability problems.

Yes, kilobytes but per-VM.

>
>> Anyway, I'm flexible and if you think we should go this way now I'll do
>> this in v3. We can re-think this when we later decide to raise
>> KVM_MAX_VCPUS significantly.
>
> Although there's no strict requirement for that I think every sensible
> userspace will allocate VP_INDEX linearly resulting in it being equal to
> KVM's vcpu index.  So we've yet to see a case where get_vcpu_by_vpidx
> doesn't take the fast path.  If it ever starts appearing in the profiles
> we may consider optimiziing it but ATM I don't even think introducing
> the translation array is justified.

It was Radim who suggested it in the first place :-)

The problem we're trying to solve here is: with PV TLB flush and IPI we
need to walk through the supplied list of VP_INDEXes and get VCPU
ids. Usually they match. But in case they don't we'll fall back to full
scan for every VP_INDEX in the supplied list. Now let's say we have 128
CPUs. We'll need to perform up to 128 * 128 extra comparisons on every
hypercall. Not good. So instead of using get_vcpu_by_vpidx() I opted for
walking the whole VCPU list and checking if VPU's VP_INDEX is in the
supplied set. This way we end up with 128 comparisons in the example
above (worst case scenarion). However, we lose in simple scenarios like
only 1 VP_INDEX was specified in the set: we'll still need to walk the
whole list. So having the translation array (one way or another) is IMO
justified.

-- 
  Vitaly

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

* Re: [PATCH v2 2/5] KVM: x86: hyperv: introduce vp_index_to_vcpu_idx mapping
  2018-06-29 11:37         ` Vitaly Kuznetsov
@ 2018-06-29 12:52           ` Roman Kagan
  2018-06-29 13:10             ` Vitaly Kuznetsov
  0 siblings, 1 reply; 18+ messages in thread
From: Roman Kagan @ 2018-06-29 12:52 UTC (permalink / raw)
  To: Vitaly Kuznetsov
  Cc: kvm, x86, Paolo Bonzini, Radim Krčmář,
	K. Y. Srinivasan, Haiyang Zhang, Stephen Hemminger,
	Michael Kelley (EOSG),
	Mohammed Gamal, Cathy Avery, Wanpeng Li, linux-kernel

On Fri, Jun 29, 2018 at 01:37:44PM +0200, Vitaly Kuznetsov wrote:
> The problem we're trying to solve here is: with PV TLB flush and IPI we
> need to walk through the supplied list of VP_INDEXes and get VCPU
> ids. Usually they match. But in case they don't [...]

Why wouldn't they *in practice*?  Only if the userspace wanted to be
funny and assigned VP_INDEXes randomly?  I'm not sure we need to
optimize for this case.

Note that the userspace can actually do nasty things with these
VP_INDEXes, like, say, have them non-unique.  We need to be resilent to
it, but don't need to optimize for it.

I think I'd rather have a warning in kvm_hv_set_msr if the VP_INDEX
being assigned is not equal to the vcpu index, and start worrying about
optimization only if this warning starts being triggered by real
hypervisor applications.

Anyway I don't see an urgent need to bloat this patchset with optimizing
this translation; it can be done separately, if needed.

Roman.

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

* Re: [PATCH v2 2/5] KVM: x86: hyperv: introduce vp_index_to_vcpu_idx mapping
  2018-06-29 12:52           ` Roman Kagan
@ 2018-06-29 13:10             ` Vitaly Kuznetsov
  2018-06-29 14:32               ` Roman Kagan
  0 siblings, 1 reply; 18+ messages in thread
From: Vitaly Kuznetsov @ 2018-06-29 13:10 UTC (permalink / raw)
  To: Roman Kagan
  Cc: kvm, x86, Paolo Bonzini, Radim Krčmář,
	K. Y. Srinivasan, Haiyang Zhang, Stephen Hemminger,
	Michael Kelley (EOSG),
	Mohammed Gamal, Cathy Avery, Wanpeng Li, linux-kernel

Roman Kagan <rkagan@virtuozzo.com> writes:

> On Fri, Jun 29, 2018 at 01:37:44PM +0200, Vitaly Kuznetsov wrote:
>> The problem we're trying to solve here is: with PV TLB flush and IPI we
>> need to walk through the supplied list of VP_INDEXes and get VCPU
>> ids. Usually they match. But in case they don't [...]
>
> Why wouldn't they *in practice*?  Only if the userspace wanted to be
> funny and assigned VP_INDEXes randomly?  I'm not sure we need to
> optimize for this case.

Can someone please remind me why we allow userspace to change it in the
first place?

>
> Note that the userspace can actually do nasty things with these
> VP_INDEXes, like, say, have them non-unique.  We need to be resilent to
> it, but don't need to optimize for it.
>
> I think I'd rather have a warning in kvm_hv_set_msr if the VP_INDEX
> being assigned is not equal to the vcpu index, and start worrying about
> optimization only if this warning starts being triggered by real
> hypervisor applications.
>
> Anyway I don't see an urgent need to bloat this patchset with optimizing
> this translation; it can be done separately, if needed.

Sure, I can use get_vcpu_by_vpidx() in the patchset pretending that is's
always fast and if we find a valid case where it is not - we'll just
optimize it later. Sounds sane to me.

-- 
  Vitaly

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

* Re: [PATCH v2 2/5] KVM: x86: hyperv: introduce vp_index_to_vcpu_idx mapping
  2018-06-29 13:10             ` Vitaly Kuznetsov
@ 2018-06-29 14:32               ` Roman Kagan
  2018-06-29 15:25                 ` Vitaly Kuznetsov
  0 siblings, 1 reply; 18+ messages in thread
From: Roman Kagan @ 2018-06-29 14:32 UTC (permalink / raw)
  To: Vitaly Kuznetsov
  Cc: kvm, x86, Paolo Bonzini, Radim Krčmář,
	K. Y. Srinivasan, Haiyang Zhang, Stephen Hemminger,
	Michael Kelley (EOSG),
	Mohammed Gamal, Cathy Avery, Wanpeng Li, linux-kernel

On Fri, Jun 29, 2018 at 03:10:14PM +0200, Vitaly Kuznetsov wrote:
> Roman Kagan <rkagan@virtuozzo.com> writes:
> 
> > On Fri, Jun 29, 2018 at 01:37:44PM +0200, Vitaly Kuznetsov wrote:
> >> The problem we're trying to solve here is: with PV TLB flush and IPI we
> >> need to walk through the supplied list of VP_INDEXes and get VCPU
> >> ids. Usually they match. But in case they don't [...]
> >
> > Why wouldn't they *in practice*?  Only if the userspace wanted to be
> > funny and assigned VP_INDEXes randomly?  I'm not sure we need to
> > optimize for this case.
> 
> Can someone please remind me why we allow userspace to change it in the
> first place?

I can ;)

We used not to, and reported KVM's vcpu index as the VP_INDEX.  However,
later we realized that VP_INDEX needed to be persistent across
migrations and otherwise also known to userspace.  Relying on the kernel
to always initialize its indices in the same order was unacceptable, and
we came up with no better way of synchronizing VP_INDEX between the
userspace and the kernel than to let the former to set it explicitly.

However, this is basically a future-proofing feature; in practice, both
QEMU and KVM initialize their indices in the same order.

Roman.

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

* Re: [PATCH v2 2/5] KVM: x86: hyperv: introduce vp_index_to_vcpu_idx mapping
  2018-06-29 14:32               ` Roman Kagan
@ 2018-06-29 15:25                 ` Vitaly Kuznetsov
  2018-06-29 15:55                   ` Roman Kagan
  0 siblings, 1 reply; 18+ messages in thread
From: Vitaly Kuznetsov @ 2018-06-29 15:25 UTC (permalink / raw)
  To: Roman Kagan
  Cc: kvm, x86, Paolo Bonzini, Radim Krčmář,
	K. Y. Srinivasan, Haiyang Zhang, Stephen Hemminger,
	Michael Kelley (EOSG),
	Mohammed Gamal, Cathy Avery, Wanpeng Li, linux-kernel

Roman Kagan <rkagan@virtuozzo.com> writes:

> On Fri, Jun 29, 2018 at 03:10:14PM +0200, Vitaly Kuznetsov wrote:
>> Roman Kagan <rkagan@virtuozzo.com> writes:
>> 
>> > On Fri, Jun 29, 2018 at 01:37:44PM +0200, Vitaly Kuznetsov wrote:
>> >> The problem we're trying to solve here is: with PV TLB flush and IPI we
>> >> need to walk through the supplied list of VP_INDEXes and get VCPU
>> >> ids. Usually they match. But in case they don't [...]
>> >
>> > Why wouldn't they *in practice*?  Only if the userspace wanted to be
>> > funny and assigned VP_INDEXes randomly?  I'm not sure we need to
>> > optimize for this case.
>> 
>> Can someone please remind me why we allow userspace to change it in the
>> first place?
>
> I can ;)
>
> We used not to, and reported KVM's vcpu index as the VP_INDEX.  However,
> later we realized that VP_INDEX needed to be persistent across
> migrations and otherwise also known to userspace.  Relying on the kernel
> to always initialize its indices in the same order was unacceptable, and
> we came up with no better way of synchronizing VP_INDEX between the
> userspace and the kernel than to let the former to set it explicitly.
>
> However, this is basically a future-proofing feature; in practice, both
> QEMU and KVM initialize their indices in the same order.


Thanks!

But in the theoretical case when these indices start to differ after
migration, users will notice a slowdown which will be hard to explain,
right? Does it justify the need for vp_idx_to_vcpu_idx?

In any case I sent v3 with vp_idx_to_vcpu_idx dropped for now, hope
Radim is OK with us de-coupling these discussions.

-- 
  Vitaly

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

* Re: [PATCH v2 2/5] KVM: x86: hyperv: introduce vp_index_to_vcpu_idx mapping
  2018-06-29 15:25                 ` Vitaly Kuznetsov
@ 2018-06-29 15:55                   ` Roman Kagan
  0 siblings, 0 replies; 18+ messages in thread
From: Roman Kagan @ 2018-06-29 15:55 UTC (permalink / raw)
  To: Vitaly Kuznetsov
  Cc: kvm, x86, Paolo Bonzini, Radim Krčmář,
	K. Y. Srinivasan, Haiyang Zhang, Stephen Hemminger,
	Michael Kelley (EOSG),
	Mohammed Gamal, Cathy Avery, Wanpeng Li, linux-kernel

On Fri, Jun 29, 2018 at 05:25:56PM +0200, Vitaly Kuznetsov wrote:
> Roman Kagan <rkagan@virtuozzo.com> writes:
> 
> > On Fri, Jun 29, 2018 at 03:10:14PM +0200, Vitaly Kuznetsov wrote:
> >> Roman Kagan <rkagan@virtuozzo.com> writes:
> >> 
> >> > On Fri, Jun 29, 2018 at 01:37:44PM +0200, Vitaly Kuznetsov wrote:
> >> >> The problem we're trying to solve here is: with PV TLB flush and IPI we
> >> >> need to walk through the supplied list of VP_INDEXes and get VCPU
> >> >> ids. Usually they match. But in case they don't [...]
> >> >
> >> > Why wouldn't they *in practice*?  Only if the userspace wanted to be
> >> > funny and assigned VP_INDEXes randomly?  I'm not sure we need to
> >> > optimize for this case.
> >> 
> >> Can someone please remind me why we allow userspace to change it in the
> >> first place?
> >
> > I can ;)
> >
> > We used not to, and reported KVM's vcpu index as the VP_INDEX.  However,
> > later we realized that VP_INDEX needed to be persistent across
> > migrations and otherwise also known to userspace.  Relying on the kernel
> > to always initialize its indices in the same order was unacceptable, and
> > we came up with no better way of synchronizing VP_INDEX between the
> > userspace and the kernel than to let the former to set it explicitly.
> >
> > However, this is basically a future-proofing feature; in practice, both
> > QEMU and KVM initialize their indices in the same order.
> 
> 
> Thanks!
> 
> But in the theoretical case when these indices start to differ after
> migration, users will notice a slowdown which will be hard to explain,
> right?

That's exactly why I suggested a warning on VP_INDEX != vcpu index in
kvm_hv_set_msr.

> Does it justify the need for vp_idx_to_vcpu_idx?

I'd personally prefer being pointed at a scenario where this becomes
relevant first.

Roman.

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

end of thread, other threads:[~2018-06-29 15:56 UTC | newest]

Thread overview: 18+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2018-06-28 13:53 [PATCH v2 0/5] KVM: x86: hyperv: PV IPI support for Windows guests Vitaly Kuznetsov
2018-06-28 13:53 ` [PATCH v2 1/5] KVM: x86: hyperv: optimize 'all cpus' case in kvm_hv_flush_tlb() Vitaly Kuznetsov
2018-06-28 13:53 ` [PATCH v2 2/5] KVM: x86: hyperv: introduce vp_index_to_vcpu_idx mapping Vitaly Kuznetsov
2018-06-28 18:49   ` kbuild test robot
2018-06-28 19:09   ` [PATCH] KVM: x86: hyperv: fix semicolon.cocci warnings kbuild test robot
2018-06-28 19:09   ` [PATCH v2 2/5] KVM: x86: hyperv: introduce vp_index_to_vcpu_idx mapping kbuild test robot
2018-06-29 10:11   ` Roman Kagan
2018-06-29 10:26     ` Vitaly Kuznetsov
2018-06-29 11:12       ` Roman Kagan
2018-06-29 11:37         ` Vitaly Kuznetsov
2018-06-29 12:52           ` Roman Kagan
2018-06-29 13:10             ` Vitaly Kuznetsov
2018-06-29 14:32               ` Roman Kagan
2018-06-29 15:25                 ` Vitaly Kuznetsov
2018-06-29 15:55                   ` Roman Kagan
2018-06-28 13:53 ` [PATCH v2 3/5] KVM: x86: hyperv: use vp_idx_to_vcpu_idx() in kvm_hv_flush_tlb() Vitaly Kuznetsov
2018-06-28 13:53 ` [PATCH v2 4/5] x86/hyper-v: rename ipi_arg_{ex,non_ex} structures Vitaly Kuznetsov
2018-06-28 13:53 ` [PATCH v2 5/5] KVM: x86: hyperv: implement PV IPI send hypercalls Vitaly Kuznetsov

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).