xen-devel.lists.xenproject.org archive mirror
 help / color / mirror / Atom feed
* [PATCH v14 0/2] Add VT-d Posted-Interrupts support
@ 2016-02-29  3:00 Feng Wu
  2016-02-29  3:00 ` [PATCH v14 1/2] vmx: VT-d posted-interrupt core logic handling Feng Wu
  2016-02-29  3:00 ` [PATCH v14 2/2] Add a command line parameter for VT-d posted-interrupts Feng Wu
  0 siblings, 2 replies; 53+ messages in thread
From: Feng Wu @ 2016-02-29  3:00 UTC (permalink / raw)
  To: xen-devel; +Cc: Feng Wu

VT-d Posted-Interrupts is an enhancement to CPU side Posted-Interrupt.
With VT-d Posted-Interrupts enabled, external interrupts from
direct-assigned devices can be delivered to guests without VMM
intervention when guest is running in non-root mode.

You can find the VT-d Posted-Interrtups Spec. in the following URL:
http://www.intel.com/content/www/us/en/intelligent-systems/intel-technology/vt-directed-io-spec.html

Feng Wu (2):
  vmx: VT-d posted-interrupt core logic handling
  Add a command line parameter for VT-d posted-interrupts

 docs/misc/xen-command-line.markdown |   9 +-
 xen/arch/x86/hvm/vmx/vmcs.c         |   2 +
 xen/arch/x86/hvm/vmx/vmx.c          | 187 ++++++++++++++++++++++++++++++++++++
 xen/common/schedule.c               |   4 +
 xen/drivers/passthrough/iommu.c     |   3 +
 xen/drivers/passthrough/vtd/iommu.c |  11 +++
 xen/include/asm-arm/domain.h        |   2 +
 xen/include/asm-x86/hvm/hvm.h       |  12 +++
 xen/include/asm-x86/hvm/vmx/vmcs.h  |  71 ++++++++++++++
 xen/include/asm-x86/hvm/vmx/vmx.h   |   5 +
 10 files changed, 305 insertions(+), 1 deletion(-)

-- 
2.1.0


_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* [PATCH v14 1/2] vmx: VT-d posted-interrupt core logic handling
  2016-02-29  3:00 [PATCH v14 0/2] Add VT-d Posted-Interrupts support Feng Wu
@ 2016-02-29  3:00 ` Feng Wu
  2016-02-29 13:33   ` Jan Beulich
                     ` (2 more replies)
  2016-02-29  3:00 ` [PATCH v14 2/2] Add a command line parameter for VT-d posted-interrupts Feng Wu
  1 sibling, 3 replies; 53+ messages in thread
From: Feng Wu @ 2016-02-29  3:00 UTC (permalink / raw)
  To: xen-devel
  Cc: Kevin Tian, Keir Fraser, George Dunlap, Andrew Cooper,
	Dario Faggioli, Jan Beulich, Feng Wu

This is the core logic handling for VT-d posted-interrupts. Basically it
deals with how and when to update posted-interrupts during the following
scenarios:
- vCPU is preempted
- vCPU is slept
- vCPU is blocked

When vCPU is preempted/slept, we update the posted-interrupts during
scheduling by introducing two new architecutral scheduler hooks:
vmx_pi_switch_from() and vmx_pi_switch_to(). When vCPU is blocked, we
introduce a new architectural hook: arch_vcpu_block() to update
posted-interrupts descriptor.

Besides that, before VM-entry, we will make sure the 'NV' filed is set
to 'posted_intr_vector' and the vCPU is not in any blocking lists, which
is needed when vCPU is running in non-root mode. The reason we do this check
is because we change the posted-interrupts descriptor in vcpu_block(),
however, we don't change it back in vcpu_unblock() or when vcpu_block()
directly returns due to event delivery (in fact, we don't need to do it
in the two places, that is why we do it before VM-Entry).

When we handle the lazy context switch for the following two scenarios:
- Preempted by a tasklet, which uses in an idle context.
- the prev vcpu is in offline and no new available vcpus in run queue.
We don't change the 'SN' bit in posted-interrupt descriptor, this
may incur spurious PI notification events, but since PI notification
event is only sent when 'ON' is clear, and once the PI notificatoin
is sent, ON is set by hardware, hence no more notification events
before 'ON' is clear. Besides that, spurious PI notification events are
going to happen from time to time in Xen hypervisor, such as, when
guests trap to Xen and PI notification event happens, there is
nothing Xen actually needs to do about it, the interrupts will be
delivered to guest atht the next time we do a VMENTRY.

CC: Keir Fraser <keir@xen.org>
CC: Jan Beulich <jbeulich@suse.com>
CC: Andrew Cooper <andrew.cooper3@citrix.com>
CC: Kevin Tian <kevin.tian@intel.com>
CC: George Dunlap <george.dunlap@eu.citrix.com>
CC: Dario Faggioli <dario.faggioli@citrix.com>
Suggested-by: Yang Zhang <yang.z.zhang@intel.com>
Suggested-by: Dario Faggioli <dario.faggioli@citrix.com>
Suggested-by: George Dunlap <george.dunlap@citrix.com>
Suggested-by: Jan Beulich <jbeulich@suse.com>
Signed-off-by: Feng Wu <feng.wu@intel.com>
Reviewed-by: George Dunlap <george.dunlap@citrix.com>
---
v14:
- Introduce a local variable in macro arch_vcpu_block()
- Shorten some variable name and remove the related macro accordingly
- Only use has_arch_pdevs() to determine whether needs to call
  vmx_pi_hooks_assign()/vmx_pi_hooks_deassign()
- Use local variable for the last parameter in cmpxchg()
- Check has_hvm_container_vcpu() in arch_vcpu_block()

v13:
- Define the blocking vcpu list and lock in a structure
- Define the two local per-CPU variables in a structure
- Some adjustment to vmx_pi_hooks_assign() and vmx_pi_hooks_deassign()
- Use smp_rmb() instead of barrier(), and put it a little earlier
- Minor changes to macro arch_vcpu_block() to make 'v' evaluated only once.
- Remove the pointless parentheses in the function arguments in macro arch_vcpu_block()
- coding style

v12:
- Move the ASSERT to the locked region in vmx_vcpu_block()
- Add barrier() before using the local variable in vmx_pi_do_resume()
- Split vmx_pi_hooks_reassign() to two functions:
      * vmx_pi_hooks_assign()
      * vmx_pi_hooks_deassign()
- Add more comments about how PI works during vCPU state transition
- coding style

v11:
- Add ASSERT() in vmx_vcpu_block()
- Add some comments in vmx_pi_switch_from()
- Remove some comments which should have been removed when the
  related code was removed during v9 -> v10
- Rename 'vmx_pi_state_to_normal' to 'vmx_pi_do_resume'
- Coding style
- Make arch_vcpu_block() a macro
- Make 'pi_wakeup_vector' static
- Move hook 'vcpu_block' to 'struct hvm_vcpu'
- Initial hook 'vcpu_block' when assigning the first pci device
  and zap it on removal of the last device
- Save pointer to the block list lock instead of the processor
  id in 'struct arch_vmx_struct'
- Implement the following functions as hooks, so we
  can elimilate lots of checkings and spinlocks in scheduling
  related code path, which is good for performance.
        vmx_pi_switch_from
        vmx_pi_switch_to
        vmx_pi_do_resume

v10:
- Check iommu_intpost first
- Remove pointless checking of has_hvm_container_vcpu(v)
- Rename 'vmx_pi_state_change' to 'vmx_pi_state_to_normal'
- Since vcpu_unblock() doesn't acquire 'pi_blocked_vcpu_lock', we
  don't need use another list to save the vCPUs with 'ON' set, just
  directly call vcpu_unblock(v).

v9:
- Remove arch_vcpu_block_cancel() and arch_vcpu_wake_prepare()
- Add vmx_pi_state_change() and call it before VM Entry

v8:
- Remove the lazy context switch handling for PI state transition
- Change PI state in vcpu_block() and do_poll() when the vCPU
  is going to be blocked

v7:
- Merge [PATCH v6 16/18] vmx: Add some scheduler hooks for VT-d posted interrupts
  and "[PATCH v6 14/18] vmx: posted-interrupt handling when vCPU is blocked"
  into this patch, so it is self-contained and more convenient
  for code review.
- Make 'pi_blocked_vcpu' and 'pi_blocked_vcpu_lock' static
- Coding style
- Use per_cpu() instead of this_cpu() in pi_wakeup_interrupt()
- Move ack_APIC_irq() to the beginning of pi_wakeup_interrupt()
- Rename 'pi_ctxt_switch_from' to 'ctxt_switch_prepare'
- Rename 'pi_ctxt_switch_to' to 'ctxt_switch_cancel'
- Use 'has_hvm_container_vcpu' instead of 'is_hvm_vcpu'
- Use 'spin_lock' and 'spin_unlock' when the interrupt has been
  already disabled.
- Rename arch_vcpu_wake_prepare to vmx_vcpu_wake_prepare
- Define vmx_vcpu_wake_prepare in xen/arch/x86/hvm/hvm.c
- Call .pi_ctxt_switch_to() __context_switch() instead of directly
  calling vmx_post_ctx_switch_pi() in vmx_ctxt_switch_to()
- Make .pi_block_cpu unsigned int
- Use list_del() instead of list_del_init()
- Coding style

One remaining item in v7:
Jan has concern about calling vcpu_unblock() in vmx_pre_ctx_switch_pi(),
need Dario or George's input about this.

v6:
- Add two static inline functions for pi context switch
- Fix typos

v5:
- Rename arch_vcpu_wake to arch_vcpu_wake_prepare
- Make arch_vcpu_wake_prepare() inline for ARM
- Merge the ARM dummy hook with together
- Changes to some code comments
- Leave 'pi_ctxt_switch_from' and 'pi_ctxt_switch_to' NULL if
  PI is disabled or the vCPU is not in HVM
- Coding style

v4:
- Newly added

Changlog for "vmx: posted-interrupt handling when vCPU is blocked"
v6:
- Fix some typos
- Ack the interrupt right after the spin_unlock in pi_wakeup_interrupt()

v4:
- Use local variables in pi_wakeup_interrupt()
- Remove vcpu from the blocked list when pi_desc.on==1, this
- avoid kick vcpu multiple times.
- Remove tasklet

v3:
- This patch is generated by merging the following three patches in v2:
   [RFC v2 09/15] Add a new per-vCPU tasklet to wakeup the blocked vCPU
   [RFC v2 10/15] vmx: Define two per-cpu variables
   [RFC v2 11/15] vmx: Add a global wake-up vector for VT-d Posted-Interrupts
- rename 'vcpu_wakeup_tasklet' to 'pi_vcpu_wakeup_tasklet'
- Move the definition of 'pi_vcpu_wakeup_tasklet' to 'struct arch_vmx_struct'
- rename 'vcpu_wakeup_tasklet_handler' to 'pi_vcpu_wakeup_tasklet_handler'
- Make pi_wakeup_interrupt() static
- Rename 'blocked_vcpu_list' to 'pi_blocked_vcpu_list'
- move 'pi_blocked_vcpu_list' to 'struct arch_vmx_struct'
- Rename 'blocked_vcpu' to 'pi_blocked_vcpu'
- Rename 'blocked_vcpu_lock' to 'pi_blocked_vcpu_lock'

 xen/arch/x86/hvm/vmx/vmcs.c         |   2 +
 xen/arch/x86/hvm/vmx/vmx.c          | 187 ++++++++++++++++++++++++++++++++++++
 xen/common/schedule.c               |   4 +
 xen/drivers/passthrough/vtd/iommu.c |  11 +++
 xen/include/asm-arm/domain.h        |   2 +
 xen/include/asm-x86/hvm/hvm.h       |  12 +++
 xen/include/asm-x86/hvm/vmx/vmcs.h  |  71 ++++++++++++++
 xen/include/asm-x86/hvm/vmx/vmx.h   |   5 +
 8 files changed, 294 insertions(+)

diff --git a/xen/arch/x86/hvm/vmx/vmcs.c b/xen/arch/x86/hvm/vmx/vmcs.c
index edd4c8d..2e535de 100644
--- a/xen/arch/x86/hvm/vmx/vmcs.c
+++ b/xen/arch/x86/hvm/vmx/vmcs.c
@@ -676,6 +676,8 @@ int vmx_cpu_up(void)
     if ( cpu_has_vmx_vpid )
         vpid_sync_all();
 
+    vmx_pi_per_cpu_init(cpu);
+
     return 0;
 }
 
diff --git a/xen/arch/x86/hvm/vmx/vmx.c b/xen/arch/x86/hvm/vmx/vmx.c
index 7917fb7..47d40e6 100644
--- a/xen/arch/x86/hvm/vmx/vmx.c
+++ b/xen/arch/x86/hvm/vmx/vmx.c
@@ -83,7 +83,148 @@ static int vmx_msr_write_intercept(unsigned int msr, uint64_t msr_content);
 static void vmx_invlpg_intercept(unsigned long vaddr);
 static int vmx_vmfunc_intercept(struct cpu_user_regs *regs);
 
+struct vmx_pi_blocking_vcpu {
+    struct list_head     list;
+    spinlock_t           lock;
+};
+
+/*
+ * We maintain a per-CPU linked-list of vCPUs, so in PI wakeup
+ * handler we can find which vCPU should be woken up.
+ */
+static DEFINE_PER_CPU(struct vmx_pi_blocking_vcpu, vmx_pi_blocking);
+
 uint8_t __read_mostly posted_intr_vector;
+static uint8_t __read_mostly pi_wakeup_vector;
+
+void vmx_pi_per_cpu_init(unsigned int cpu)
+{
+    INIT_LIST_HEAD(&per_cpu(vmx_pi_blocking, cpu).list);
+    spin_lock_init(&per_cpu(vmx_pi_blocking, cpu).lock);
+}
+
+static void vmx_vcpu_block(struct vcpu *v)
+{
+    unsigned long flags;
+    unsigned int dest;
+    spinlock_t *old_lock;
+    spinlock_t *pi_blocking_list_lock =
+		&per_cpu(vmx_pi_blocking, v->processor).lock;
+    struct pi_desc *pi_desc = &v->arch.hvm_vmx.pi_desc;
+
+    spin_lock_irqsave(pi_blocking_list_lock, flags);
+    old_lock = cmpxchg(&v->arch.hvm_vmx.pi_blocking.lock, NULL,
+                       pi_blocking_list_lock);
+
+    /*
+     * 'v->arch.hvm_vmx.pi_blocking.lock' should be NULL before
+     * being assigned to a new value, since the vCPU is currently
+     * running and it cannot be on any blocking list.
+     */
+    ASSERT(old_lock == NULL);
+
+    list_add_tail(&v->arch.hvm_vmx.pi_blocking.list,
+                  &per_cpu(vmx_pi_blocking, v->processor).list);
+    spin_unlock_irqrestore(pi_blocking_list_lock, flags);
+
+    ASSERT(!pi_test_sn(pi_desc));
+
+    dest = cpu_physical_id(v->processor);
+
+    ASSERT(pi_desc->ndst ==
+           (x2apic_enabled ? dest : MASK_INSR(dest, PI_xAPIC_NDST_MASK)));
+
+    write_atomic(&pi_desc->nv, pi_wakeup_vector);
+}
+
+static void vmx_pi_switch_from(struct vcpu *v)
+{
+    struct pi_desc *pi_desc = &v->arch.hvm_vmx.pi_desc;
+
+    if ( test_bit(_VPF_blocked, &v->pause_flags) )
+        return;
+
+    pi_set_sn(pi_desc);
+}
+
+static void vmx_pi_switch_to(struct vcpu *v)
+{
+    struct pi_desc *pi_desc = &v->arch.hvm_vmx.pi_desc;
+    unsigned int dest = cpu_physical_id(v->processor);
+
+    write_atomic(&pi_desc->ndst,
+                 x2apic_enabled ? dest : MASK_INSR(dest, PI_xAPIC_NDST_MASK));
+
+    pi_clear_sn(pi_desc);
+}
+
+static void vmx_pi_do_resume(struct vcpu *v)
+{
+    unsigned long flags;
+    spinlock_t *pi_blocking_list_lock;
+    struct pi_desc *pi_desc = &v->arch.hvm_vmx.pi_desc;
+
+    ASSERT(!test_bit(_VPF_blocked, &v->pause_flags));
+
+    /*
+     * Set 'NV' field back to posted_intr_vector, so the
+     * Posted-Interrupts can be delivered to the vCPU when
+     * it is running in non-root mode.
+     */
+    write_atomic(&pi_desc->nv, posted_intr_vector);
+
+    /* The vCPU is not on any blocking list. */
+    pi_blocking_list_lock = v->arch.hvm_vmx.pi_blocking.lock;
+
+    /* Prevent the compiler from eliminating the local variable.*/
+    smp_rmb();
+
+    if ( pi_blocking_list_lock == NULL )
+        return;
+
+    spin_lock_irqsave(pi_blocking_list_lock, flags);
+
+    /*
+     * v->arch.hvm_vmx.pi_blocking.lock == NULL here means the vCPU
+     * was removed from the blocking list while we are acquiring the lock.
+     */
+    if ( v->arch.hvm_vmx.pi_blocking.lock != NULL )
+    {
+        ASSERT(v->arch.hvm_vmx.pi_blocking.lock == pi_blocking_list_lock);
+        list_del(&v->arch.hvm_vmx.pi_blocking.list);
+        v->arch.hvm_vmx.pi_blocking.lock = NULL;
+    }
+
+    spin_unlock_irqrestore(pi_blocking_list_lock, flags);
+}
+
+/* This function is called when pcidevs_lock is held */
+void vmx_pi_hooks_assign(struct domain *d)
+{
+    if ( !iommu_intpost || !has_hvm_container_domain(d) )
+        return;
+
+    ASSERT(!d->arch.hvm_domain.vmx.vcpu_block);
+
+    d->arch.hvm_domain.vmx.vcpu_block = vmx_vcpu_block;
+    d->arch.hvm_domain.vmx.pi_switch_from = vmx_pi_switch_from;
+    d->arch.hvm_domain.vmx.pi_switch_to = vmx_pi_switch_to;
+    d->arch.hvm_domain.vmx.pi_do_resume = vmx_pi_do_resume;
+}
+
+/* This function is called when pcidevs_lock is held */
+void vmx_pi_hooks_deassign(struct domain *d)
+{
+    if ( !iommu_intpost || !has_hvm_container_domain(d) )
+        return;
+
+    ASSERT(d->arch.hvm_domain.vmx.vcpu_block);
+
+    d->arch.hvm_domain.vmx.vcpu_block = NULL;
+    d->arch.hvm_domain.vmx.pi_switch_from = NULL;
+    d->arch.hvm_domain.vmx.pi_switch_to = NULL;
+    d->arch.hvm_domain.vmx.pi_do_resume = NULL;
+}
 
 static int vmx_domain_initialise(struct domain *d)
 {
@@ -112,6 +253,8 @@ static int vmx_vcpu_initialise(struct vcpu *v)
 
     spin_lock_init(&v->arch.hvm_vmx.vmcs_lock);
 
+    INIT_LIST_HEAD(&v->arch.hvm_vmx.pi_blocking.list);
+
     v->arch.schedule_tail    = vmx_do_resume;
     v->arch.ctxt_switch_from = vmx_ctxt_switch_from;
     v->arch.ctxt_switch_to   = vmx_ctxt_switch_to;
@@ -740,6 +883,9 @@ static void vmx_ctxt_switch_from(struct vcpu *v)
     vmx_save_guest_msrs(v);
     vmx_restore_host_msrs();
     vmx_save_dr(v);
+
+    if ( v->domain->arch.hvm_domain.vmx.pi_switch_from )
+        v->domain->arch.hvm_domain.vmx.pi_switch_from(v);
 }
 
 static void vmx_ctxt_switch_to(struct vcpu *v)
@@ -752,6 +898,9 @@ static void vmx_ctxt_switch_to(struct vcpu *v)
 
     vmx_restore_guest_msrs(v);
     vmx_restore_dr(v);
+
+    if ( v->domain->arch.hvm_domain.vmx.pi_switch_to )
+        v->domain->arch.hvm_domain.vmx.pi_switch_to(v);
 }
 
 
@@ -2010,6 +2159,38 @@ static struct hvm_function_table __initdata vmx_function_table = {
     .altp2m_vcpu_emulate_vmfunc = vmx_vcpu_emulate_vmfunc,
 };
 
+/* Handle VT-d posted-interrupt when VCPU is blocked. */
+static void pi_wakeup_interrupt(struct cpu_user_regs *regs)
+{
+    struct arch_vmx_struct *vmx, *tmp;
+    spinlock_t *lock = &per_cpu(vmx_pi_blocking, smp_processor_id()).lock;
+    struct list_head *blocked_vcpus =
+		&per_cpu(vmx_pi_blocking, smp_processor_id()).list;
+
+    ack_APIC_irq();
+    this_cpu(irq_count)++;
+
+    spin_lock(lock);
+
+    /*
+     * XXX: The length of the list depends on how many vCPU is current
+     * blocked on this specific pCPU. This may hurt the interrupt latency
+     * if the list grows to too many entries.
+     */
+    list_for_each_entry_safe(vmx, tmp, blocked_vcpus, pi_blocking.list)
+    {
+        if ( pi_test_on(&vmx->pi_desc) )
+        {
+            list_del(&vmx->pi_blocking.list);
+            ASSERT(vmx->pi_blocking.lock == lock);
+            vmx->pi_blocking.lock = NULL;
+            vcpu_unblock(container_of(vmx, struct vcpu, arch.hvm_vmx));
+        }
+    }
+
+    spin_unlock(lock);
+}
+
 /* Handle VT-d posted-interrupt when VCPU is running. */
 static void pi_notification_interrupt(struct cpu_user_regs *regs)
 {
@@ -2096,7 +2277,10 @@ const struct hvm_function_table * __init start_vmx(void)
     if ( cpu_has_vmx_posted_intr_processing )
     {
         if ( iommu_intpost )
+        {
             alloc_direct_apic_vector(&posted_intr_vector, pi_notification_interrupt);
+            alloc_direct_apic_vector(&pi_wakeup_vector, pi_wakeup_interrupt);
+        }
         else
             alloc_direct_apic_vector(&posted_intr_vector, event_check_interrupt);
     }
@@ -3574,6 +3758,9 @@ void vmx_vmenter_helper(const struct cpu_user_regs *regs)
     struct hvm_vcpu_asid *p_asid;
     bool_t need_flush;
 
+    if ( curr->domain->arch.hvm_domain.vmx.pi_do_resume )
+        curr->domain->arch.hvm_domain.vmx.pi_do_resume(curr);
+
     if ( !cpu_has_vmx_vpid )
         goto out;
     if ( nestedhvm_vcpu_in_guestmode(curr) )
diff --git a/xen/common/schedule.c b/xen/common/schedule.c
index d121896..2d87021 100644
--- a/xen/common/schedule.c
+++ b/xen/common/schedule.c
@@ -802,6 +802,8 @@ void vcpu_block(void)
 
     set_bit(_VPF_blocked, &v->pause_flags);
 
+    arch_vcpu_block(v);
+
     /* Check for events /after/ blocking: avoids wakeup waiting race. */
     if ( local_events_need_delivery() )
     {
@@ -839,6 +841,8 @@ static long do_poll(struct sched_poll *sched_poll)
     v->poll_evtchn = -1;
     set_bit(v->vcpu_id, d->poll_mask);
 
+    arch_vcpu_block(v);
+
 #ifndef CONFIG_X86 /* set_bit() implies mb() on x86 */
     /* Check for events /after/ setting flags: avoids wakeup waiting race. */
     smp_mb();
diff --git a/xen/drivers/passthrough/vtd/iommu.c b/xen/drivers/passthrough/vtd/iommu.c
index ec31c6b..8022702 100644
--- a/xen/drivers/passthrough/vtd/iommu.c
+++ b/xen/drivers/passthrough/vtd/iommu.c
@@ -2283,9 +2283,17 @@ static int reassign_device_ownership(
     if ( ret )
         return ret;
 
+    if ( !has_arch_pdevs(target) )
+        vmx_pi_hooks_assign(target);
+
     ret = domain_context_mapping(target, devfn, pdev);
     if ( ret )
+    {
+        if ( !has_arch_pdevs(target) )
+            vmx_pi_hooks_deassign(target);
+
         return ret;
+    }
 
     if ( devfn == pdev->devfn )
     {
@@ -2293,6 +2301,9 @@ static int reassign_device_ownership(
         pdev->domain = target;
     }
 
+    if ( !has_arch_pdevs(source) )
+        vmx_pi_hooks_deassign(source);
+
     return ret;
 }
 
diff --git a/xen/include/asm-arm/domain.h b/xen/include/asm-arm/domain.h
index aa7f283..37afa80 100644
--- a/xen/include/asm-arm/domain.h
+++ b/xen/include/asm-arm/domain.h
@@ -310,6 +310,8 @@ static inline void free_vcpu_guest_context(struct vcpu_guest_context *vgc)
     xfree(vgc);
 }
 
+static inline void arch_vcpu_block(struct vcpu *v) {}
+
 #endif /* __ASM_DOMAIN_H__ */
 
 /*
diff --git a/xen/include/asm-x86/hvm/hvm.h b/xen/include/asm-x86/hvm/hvm.h
index b9d893d..88ec737 100644
--- a/xen/include/asm-x86/hvm/hvm.h
+++ b/xen/include/asm-x86/hvm/hvm.h
@@ -565,6 +565,18 @@ const char *hvm_efer_valid(const struct vcpu *v, uint64_t value,
                            signed int cr0_pg);
 unsigned long hvm_cr4_guest_reserved_bits(const struct vcpu *v, bool_t restore);
 
+/*
+ * This must be defined as a macro instead of an inline function,
+ * because it uses 'struct vcpu' and 'struct domain' which have
+ * not been defined yet.
+ */
+#define arch_vcpu_block(v) ({                                             \
+    struct vcpu *v_ = (v);                                                \
+    if ( has_hvm_container_vcpu(v_) &&                                    \
+         (v_)->domain->arch.hvm_domain.vmx.vcpu_block )                   \
+        (v_)->domain->arch.hvm_domain.vmx.vcpu_block(v_);                 \
+})
+
 #endif /* __ASM_X86_HVM_HVM_H__ */
 
 /*
diff --git a/xen/include/asm-x86/hvm/vmx/vmcs.h b/xen/include/asm-x86/hvm/vmx/vmcs.h
index d1496b8..8247d59 100644
--- a/xen/include/asm-x86/hvm/vmx/vmcs.h
+++ b/xen/include/asm-x86/hvm/vmx/vmcs.h
@@ -77,6 +77,65 @@ struct vmx_domain {
     unsigned long apic_access_mfn;
     /* VMX_DOMAIN_* */
     unsigned int status;
+
+    /*
+     * To handle posted interrupts correctly, we need to set the following
+     * state:
+     *
+     * * The PI notification vector (NV)
+     * * The PI notification destination processor (NDST)
+     * * The PI "suppress notification" bit (SN)
+     * * The vcpu pi "blocked" list
+     *
+     * If a VM is currently running, we want the PI delivered to the guest vcpu
+     * on the proper pcpu (NDST = v->processor, SN clear).
+     *
+     * If the vm is blocked, we want the PI delivered to Xen so that it can
+     * wake it up  (SN clear, NV = pi_wakeup_vector, vcpu on block list).
+     *
+     * If the VM is currently either preempted or offline (i.e., not running
+     * because of some reason other than blocking waiting for an interrupt),
+     * there's nothing Xen can do -- we want the interrupt pending bit set in
+     * the guest, but we don't want to bother Xen with an interrupt (SN clear).
+     *
+     * There's a brief window of time between vmx_intr_assist() and checking
+     * softirqs where if an interrupt comes in it may be lost; so we need Xen
+     * to get an interrupt and raise a softirq so that it will go through the
+     * vmx_intr_assist() path again (SN clear, NV = posted_interrupt).
+     *
+     * The way we implement this now is by looking at what needs to happen on
+     * the following runstate transitions:
+     *
+     * A: runnable -> running
+     *  - SN = 0
+     *  - NDST = v->processor
+     * B: running -> runnable
+     *  - SN = 1
+     * C: running -> blocked
+     *  - NV = pi_wakeup_vector
+     *  - Add vcpu to blocked list
+     * D: blocked -> runnable
+     *  - NV = posted_intr_vector
+     *  - Take vcpu off blocked list
+     *
+     * For transitions A and B, we add hooks into vmx_ctxt_switch_{from,to}
+     * paths.
+     *
+     * For transition C, we add a new arch hook, arch_vcpu_block(), which is
+     * called from vcpu_block() and vcpu_do_poll().
+     *
+     * For transition D, rather than add an extra arch hook on vcpu_wake, we
+     * add a hook on the vmentry path which checks to see if either of the two
+     * actions need to be taken.
+     *
+     * These hooks only need to be called when the domain in question actually
+     * has a physical device assigned to it, so we set and clear the callbacks
+     * as appropriate when device assignment changes.
+     */
+    void (*vcpu_block) (struct vcpu *);
+    void (*pi_switch_from) (struct vcpu *v);
+    void (*pi_switch_to) (struct vcpu *v);
+    void (*pi_do_resume) (struct vcpu *v);
 };
 
 struct pi_desc {
@@ -101,6 +160,11 @@ struct pi_desc {
 
 #define NR_PML_ENTRIES   512
 
+struct pi_blocking_vcpu {
+    struct list_head     list;
+    spinlock_t           *lock;
+};
+
 struct arch_vmx_struct {
     /* Physical address of VMCS. */
     paddr_t              vmcs_pa;
@@ -160,6 +224,13 @@ struct arch_vmx_struct {
     struct page_info     *vmwrite_bitmap;
 
     struct page_info     *pml_pg;
+
+    /*
+     * Before it is blocked, vCPU is added to the per-cpu list.
+     * VT-d engine can send wakeup notification event to the
+     * pCPU and wakeup the related vCPU.
+     */
+    struct pi_blocking_vcpu pi_blocking;
 };
 
 int vmx_create_vmcs(struct vcpu *v);
diff --git a/xen/include/asm-x86/hvm/vmx/vmx.h b/xen/include/asm-x86/hvm/vmx/vmx.h
index 1719965..359b2a9 100644
--- a/xen/include/asm-x86/hvm/vmx/vmx.h
+++ b/xen/include/asm-x86/hvm/vmx/vmx.h
@@ -563,6 +563,11 @@ int alloc_p2m_hap_data(struct p2m_domain *p2m);
 void free_p2m_hap_data(struct p2m_domain *p2m);
 void p2m_init_hap_data(struct p2m_domain *p2m);
 
+void vmx_pi_per_cpu_init(unsigned int cpu);
+
+void vmx_pi_hooks_assign(struct domain *d);
+void vmx_pi_hooks_deassign(struct domain *d);
+
 /* EPT violation qualifications definitions */
 #define _EPT_READ_VIOLATION         0
 #define EPT_READ_VIOLATION          (1UL<<_EPT_READ_VIOLATION)
-- 
2.1.0


_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* [PATCH v14 2/2] Add a command line parameter for VT-d posted-interrupts
  2016-02-29  3:00 [PATCH v14 0/2] Add VT-d Posted-Interrupts support Feng Wu
  2016-02-29  3:00 ` [PATCH v14 1/2] vmx: VT-d posted-interrupt core logic handling Feng Wu
@ 2016-02-29  3:00 ` Feng Wu
  1 sibling, 0 replies; 53+ messages in thread
From: Feng Wu @ 2016-02-29  3:00 UTC (permalink / raw)
  To: xen-devel; +Cc: Feng Wu, Jan Beulich

Enable VT-d Posted-Interrupts and add a command line
parameter for it.

CC: Jan Beulich <jbeulich@suse.com>
Signed-off-by: Feng Wu <feng.wu@intel.com>
Reviewed-by: Kevin Tian <kevin.tian@intel.com>
Acked-by: Jan Beulich <jbeulich@suse.com>
---
 docs/misc/xen-command-line.markdown | 9 ++++++++-
 xen/drivers/passthrough/iommu.c     | 3 +++
 2 files changed, 11 insertions(+), 1 deletion(-)

diff --git a/docs/misc/xen-command-line.markdown b/docs/misc/xen-command-line.markdown
index 467dc8f..ea1d60d 100644
--- a/docs/misc/xen-command-line.markdown
+++ b/docs/misc/xen-command-line.markdown
@@ -868,7 +868,7 @@ debug hypervisor only).
 > Default: `new` unless directed-EOI is supported
 
 ### iommu
-> `= List of [ <boolean> | force | required | intremap | qinval | snoop | sharept | dom0-passthrough | dom0-strict | amd-iommu-perdev-intremap | workaround_bios_bug | igfx | verbose | debug ]`
+> `= List of [ <boolean> | force | required | intremap | intpost | qinval | snoop | sharept | dom0-passthrough | dom0-strict | amd-iommu-perdev-intremap | workaround_bios_bug | igfx | verbose | debug ]`
 
 > Sub-options:
 
@@ -895,6 +895,13 @@ debug hypervisor only).
 >> Control the use of interrupt remapping (DMA remapping will always be enabled
 >> if IOMMU functionality is enabled).
 
+> `intpost`
+
+> Default: `false`
+
+>> Control the use of interrupt posting, which depends on the availability of
+>> interrupt remapping.
+
 > `qinval` (VT-d)
 
 > Default: `true`
diff --git a/xen/drivers/passthrough/iommu.c b/xen/drivers/passthrough/iommu.c
index 0b2abf4..50d74a5 100644
--- a/xen/drivers/passthrough/iommu.c
+++ b/xen/drivers/passthrough/iommu.c
@@ -32,6 +32,7 @@ static void iommu_dump_p2m_table(unsigned char key);
  *   off|no|false|disable       Disable IOMMU (default)
  *   force|required             Don't boot unless IOMMU is enabled
  *   no-intremap                Disable interrupt remapping
+ *   no-intpost                 Disable VT-d Interrupt posting
  *   verbose                    Be more verbose
  *   debug                      Enable debugging messages and checks
  *   workaround_bios_bug        Workaround some bios issue to still enable
@@ -105,6 +106,8 @@ static void __init parse_iommu_param(char *s)
             iommu_qinval = val;
         else if ( !strcmp(s, "intremap") )
             iommu_intremap = val;
+        else if ( !strcmp(s, "intpost") )
+            iommu_intpost = val;
         else if ( !strcmp(s, "debug") )
         {
             iommu_debug = val;
-- 
2.1.0


_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: [PATCH v14 1/2] vmx: VT-d posted-interrupt core logic handling
  2016-02-29  3:00 ` [PATCH v14 1/2] vmx: VT-d posted-interrupt core logic handling Feng Wu
@ 2016-02-29 13:33   ` Jan Beulich
  2016-02-29 13:52     ` Dario Faggioli
  2016-03-01 10:16     ` George Dunlap
  2016-03-01  5:24   ` Tian, Kevin
  2016-03-04 22:00   ` Ideas " Konrad Rzeszutek Wilk
  2 siblings, 2 replies; 53+ messages in thread
From: Jan Beulich @ 2016-02-29 13:33 UTC (permalink / raw)
  To: George Dunlap, Feng Wu
  Cc: Andrew Cooper, Dario Faggioli, Keir Fraser, Kevin Tian, xen-devel

>>> On 29.02.16 at 04:00, <feng.wu@intel.com> wrote:
> This is the core logic handling for VT-d posted-interrupts. Basically it
> deals with how and when to update posted-interrupts during the following
> scenarios:
> - vCPU is preempted
> - vCPU is slept
> - vCPU is blocked
> 
> When vCPU is preempted/slept, we update the posted-interrupts during
> scheduling by introducing two new architecutral scheduler hooks:
> vmx_pi_switch_from() and vmx_pi_switch_to(). When vCPU is blocked, we
> introduce a new architectural hook: arch_vcpu_block() to update
> posted-interrupts descriptor.
> 
> Besides that, before VM-entry, we will make sure the 'NV' filed is set
> to 'posted_intr_vector' and the vCPU is not in any blocking lists, which
> is needed when vCPU is running in non-root mode. The reason we do this check
> is because we change the posted-interrupts descriptor in vcpu_block(),
> however, we don't change it back in vcpu_unblock() or when vcpu_block()
> directly returns due to event delivery (in fact, we don't need to do it
> in the two places, that is why we do it before VM-Entry).
> 
> When we handle the lazy context switch for the following two scenarios:
> - Preempted by a tasklet, which uses in an idle context.
> - the prev vcpu is in offline and no new available vcpus in run queue.
> We don't change the 'SN' bit in posted-interrupt descriptor, this
> may incur spurious PI notification events, but since PI notification
> event is only sent when 'ON' is clear, and once the PI notificatoin
> is sent, ON is set by hardware, hence no more notification events
> before 'ON' is clear. Besides that, spurious PI notification events are
> going to happen from time to time in Xen hypervisor, such as, when
> guests trap to Xen and PI notification event happens, there is
> nothing Xen actually needs to do about it, the interrupts will be
> delivered to guest atht the next time we do a VMENTRY.
> 
> CC: Keir Fraser <keir@xen.org>
> CC: Jan Beulich <jbeulich@suse.com>
> CC: Andrew Cooper <andrew.cooper3@citrix.com>
> CC: Kevin Tian <kevin.tian@intel.com>
> CC: George Dunlap <george.dunlap@eu.citrix.com>
> CC: Dario Faggioli <dario.faggioli@citrix.com>
> Suggested-by: Yang Zhang <yang.z.zhang@intel.com>
> Suggested-by: Dario Faggioli <dario.faggioli@citrix.com>
> Suggested-by: George Dunlap <george.dunlap@citrix.com>
> Suggested-by: Jan Beulich <jbeulich@suse.com>
> Signed-off-by: Feng Wu <feng.wu@intel.com>
> Reviewed-by: George Dunlap <george.dunlap@citrix.com>

With the comments George gave on v13 subsequent to this tag
I'm not sure it was correct to retain it. George?

Reviewed-by: Jan Beulich <jbeulich@suse.com>
albeit in case another version is needed ...

> --- a/xen/include/asm-x86/hvm/hvm.h
> +++ b/xen/include/asm-x86/hvm/hvm.h
> @@ -565,6 +565,18 @@ const char *hvm_efer_valid(const struct vcpu *v, uint64_t value,
>                             signed int cr0_pg);
>  unsigned long hvm_cr4_guest_reserved_bits(const struct vcpu *v, bool_t restore);
>  
> +/*
> + * This must be defined as a macro instead of an inline function,
> + * because it uses 'struct vcpu' and 'struct domain' which have
> + * not been defined yet.
> + */
> +#define arch_vcpu_block(v) ({                                             \
> +    struct vcpu *v_ = (v);                                                \
> +    if ( has_hvm_container_vcpu(v_) &&                                    \
> +         (v_)->domain->arch.hvm_domain.vmx.vcpu_block )                   \
> +        (v_)->domain->arch.hvm_domain.vmx.vcpu_block(v_);                 \
> +})

... please drop the stray parentheses here (I'll try to remember to
do so while committing if this is the version to go in).

Jan


_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: [PATCH v14 1/2] vmx: VT-d posted-interrupt core logic handling
  2016-02-29 13:33   ` Jan Beulich
@ 2016-02-29 13:52     ` Dario Faggioli
  2016-03-01  5:39       ` Wu, Feng
  2016-03-01 10:16     ` George Dunlap
  1 sibling, 1 reply; 53+ messages in thread
From: Dario Faggioli @ 2016-02-29 13:52 UTC (permalink / raw)
  To: Jan Beulich, George Dunlap, Feng Wu
  Cc: Andrew Cooper, Kevin Tian, Keir Fraser, xen-devel


[-- Attachment #1.1: Type: text/plain, Size: 1835 bytes --]

On Mon, 2016-02-29 at 06:33 -0700, Jan Beulich wrote:
> > > > On 29.02.16 at 04:00, <feng.wu@intel.com> wrote:
> > This is the core logic handling for VT-d posted-interrupts.
> > Basically it
> > deals with how and when to update posted-interrupts during the
> > following
> > scenarios:
> > - vCPU is preempted
> > - vCPU is slept
> > - vCPU is blocked
> > 
> [...]
> > CC: Keir Fraser <keir@xen.org>
> > CC: Jan Beulich <jbeulich@suse.com>
> > CC: Andrew Cooper <andrew.cooper3@citrix.com>
> > CC: Kevin Tian <kevin.tian@intel.com>
> > CC: George Dunlap <george.dunlap@eu.citrix.com>
> > CC: Dario Faggioli <dario.faggioli@citrix.com>
> > Suggested-by: Yang Zhang <yang.z.zhang@intel.com>
> > Suggested-by: Dario Faggioli <dario.faggioli@citrix.com>
> > Suggested-by: George Dunlap <george.dunlap@citrix.com>
> > Suggested-by: Jan Beulich <jbeulich@suse.com>
> > Signed-off-by: Feng Wu <feng.wu@intel.com>
> > Reviewed-by: George Dunlap <george.dunlap@citrix.com>
> With the comments George gave on v13 subsequent to this tag
> I'm not sure it was correct to retain it. George?
> 
> Reviewed-by: Jan Beulich <jbeulich@suse.com>
> albeit in case another version is needed ...
> 
And, if I'm still in time (if not, no big deal :-))

Reviewed-by: Dario Faggioli <dario.faggioli@citrix.com>

Also, personally, I don't think that all those Suggested-by add much
useful info, and I'd be fine to see mine one removed (in any case, but
even more so, if the Rev-by: above is applied).

But, again, no big deal.

Thanks and Regards,
Dario
-- 
<<This happens because I choose it to happen!>> (Raistlin Majere)
-----------------------------------------------------------------
Dario Faggioli, Ph.D, http://about.me/dario.faggioli
Senior Software Engineer, Citrix Systems R&D Ltd., Cambridge (UK)


[-- Attachment #1.2: This is a digitally signed message part --]
[-- Type: application/pgp-signature, Size: 181 bytes --]

[-- Attachment #2: Type: text/plain, Size: 126 bytes --]

_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: [PATCH v14 1/2] vmx: VT-d posted-interrupt core logic handling
  2016-02-29  3:00 ` [PATCH v14 1/2] vmx: VT-d posted-interrupt core logic handling Feng Wu
  2016-02-29 13:33   ` Jan Beulich
@ 2016-03-01  5:24   ` Tian, Kevin
  2016-03-01  5:39     ` Wu, Feng
  2016-03-04 22:00   ` Ideas " Konrad Rzeszutek Wilk
  2 siblings, 1 reply; 53+ messages in thread
From: Tian, Kevin @ 2016-03-01  5:24 UTC (permalink / raw)
  To: Wu, Feng, xen-devel
  Cc: George Dunlap, Andrew Cooper, Dario Faggioli, Keir Fraser, Jan Beulich

> From: Wu, Feng
> Sent: Monday, February 29, 2016 11:00 AM
> 
> This is the core logic handling for VT-d posted-interrupts. Basically it
> deals with how and when to update posted-interrupts during the following
> scenarios:
> - vCPU is preempted
> - vCPU is slept
> - vCPU is blocked
> 
> When vCPU is preempted/slept, we update the posted-interrupts during
> scheduling by introducing two new architecutral scheduler hooks:
> vmx_pi_switch_from() and vmx_pi_switch_to(). When vCPU is blocked, we
> introduce a new architectural hook: arch_vcpu_block() to update
> posted-interrupts descriptor.
> 
> Besides that, before VM-entry, we will make sure the 'NV' filed is set
> to 'posted_intr_vector' and the vCPU is not in any blocking lists, which
> is needed when vCPU is running in non-root mode. The reason we do this check
> is because we change the posted-interrupts descriptor in vcpu_block(),
> however, we don't change it back in vcpu_unblock() or when vcpu_block()
> directly returns due to event delivery (in fact, we don't need to do it
> in the two places, that is why we do it before VM-Entry).
> 
> When we handle the lazy context switch for the following two scenarios:
> - Preempted by a tasklet, which uses in an idle context.
> - the prev vcpu is in offline and no new available vcpus in run queue.
> We don't change the 'SN' bit in posted-interrupt descriptor, this
> may incur spurious PI notification events, but since PI notification
> event is only sent when 'ON' is clear, and once the PI notificatoin
> is sent, ON is set by hardware, hence no more notification events
> before 'ON' is clear. Besides that, spurious PI notification events are
> going to happen from time to time in Xen hypervisor, such as, when
> guests trap to Xen and PI notification event happens, there is
> nothing Xen actually needs to do about it, the interrupts will be
> delivered to guest atht the next time we do a VMENTRY.
> 
> CC: Keir Fraser <keir@xen.org>
> CC: Jan Beulich <jbeulich@suse.com>
> CC: Andrew Cooper <andrew.cooper3@citrix.com>
> CC: Kevin Tian <kevin.tian@intel.com>
> CC: George Dunlap <george.dunlap@eu.citrix.com>
> CC: Dario Faggioli <dario.faggioli@citrix.com>
> Suggested-by: Yang Zhang <yang.z.zhang@intel.com>
> Suggested-by: Dario Faggioli <dario.faggioli@citrix.com>
> Suggested-by: George Dunlap <george.dunlap@citrix.com>
> Suggested-by: Jan Beulich <jbeulich@suse.com>
> Signed-off-by: Feng Wu <feng.wu@intel.com>
> Reviewed-by: George Dunlap <george.dunlap@citrix.com>

Acked-by: Kevin Tian <kevin.tian@intel.com>

_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: [PATCH v14 1/2] vmx: VT-d posted-interrupt core logic handling
  2016-02-29 13:52     ` Dario Faggioli
@ 2016-03-01  5:39       ` Wu, Feng
  2016-03-01  9:24         ` Jan Beulich
  0 siblings, 1 reply; 53+ messages in thread
From: Wu, Feng @ 2016-03-01  5:39 UTC (permalink / raw)
  To: Dario Faggioli, Jan Beulich, George Dunlap
  Cc: Andrew Cooper, Tian, Kevin, Keir Fraser, Wu, Feng, xen-devel



> -----Original Message-----
> From: Dario Faggioli [mailto:dario.faggioli@citrix.com]
> Sent: Monday, February 29, 2016 9:52 PM
> To: Jan Beulich <JBeulich@suse.com>; George Dunlap
> <george.dunlap@eu.citrix.com>; Wu, Feng <feng.wu@intel.com>
> Cc: Andrew Cooper <andrew.cooper3@citrix.com>; Tian, Kevin
> <kevin.tian@intel.com>; xen-devel@lists.xen.org; Keir Fraser <keir@xen.org>
> Subject: Re: [Xen-devel] [PATCH v14 1/2] vmx: VT-d posted-interrupt core logic
> handling
> 
> On Mon, 2016-02-29 at 06:33 -0700, Jan Beulich wrote:
> > > > > On 29.02.16 at 04:00, <feng.wu@intel.com> wrote:
> > > This is the core logic handling for VT-d posted-interrupts.
> > > Basically it
> > > deals with how and when to update posted-interrupts during the
> > > following
> > > scenarios:
> > > - vCPU is preempted
> > > - vCPU is slept
> > > - vCPU is blocked
> > >
> > [...]
> > > CC: Keir Fraser <keir@xen.org>
> > > CC: Jan Beulich <jbeulich@suse.com>
> > > CC: Andrew Cooper <andrew.cooper3@citrix.com>
> > > CC: Kevin Tian <kevin.tian@intel.com>
> > > CC: George Dunlap <george.dunlap@eu.citrix.com>
> > > CC: Dario Faggioli <dario.faggioli@citrix.com>
> > > Suggested-by: Yang Zhang <yang.z.zhang@intel.com>
> > > Suggested-by: Dario Faggioli <dario.faggioli@citrix.com>
> > > Suggested-by: George Dunlap <george.dunlap@citrix.com>
> > > Suggested-by: Jan Beulich <jbeulich@suse.com>
> > > Signed-off-by: Feng Wu <feng.wu@intel.com>
> > > Reviewed-by: George Dunlap <george.dunlap@citrix.com>
> > With the comments George gave on v13 subsequent to this tag
> > I'm not sure it was correct to retain it. George?
> >
> > Reviewed-by: Jan Beulich <jbeulich@suse.com>
> > albeit in case another version is needed ...
> >
> And, if I'm still in time (if not, no big deal :-))
> 
> Reviewed-by: Dario Faggioli <dario.faggioli@citrix.com>

Really appreciate your huge efforts on reviewing this series, Jan,
George, and Dario! 

BTW, Jan, considering that Kevin, as the VMX maintainer also gave
his acked-by for this patch, do you think another version is needed?

Thanks,
Feng

> 
> Also, personally, I don't think that all those Suggested-by add much
> useful info, and I'd be fine to see mine one removed (in any case, but
> even more so, if the Rev-by: above is applied).
> 
> But, again, no big deal.
> 
> Thanks and Regards,
> Dario
> --
> <<This happens because I choose it to happen!>> (Raistlin Majere)
> -----------------------------------------------------------------
> Dario Faggioli, Ph.D, http://about.me/dario.faggioli
> Senior Software Engineer, Citrix Systems R&D Ltd., Cambridge (UK)

_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: [PATCH v14 1/2] vmx: VT-d posted-interrupt core logic handling
  2016-03-01  5:24   ` Tian, Kevin
@ 2016-03-01  5:39     ` Wu, Feng
  0 siblings, 0 replies; 53+ messages in thread
From: Wu, Feng @ 2016-03-01  5:39 UTC (permalink / raw)
  To: Tian, Kevin, xen-devel
  Cc: Wu, Feng, George Dunlap, Andrew Cooper, Dario Faggioli,
	Jan Beulich, Keir Fraser



> -----Original Message-----
> From: Tian, Kevin
> Sent: Tuesday, March 1, 2016 1:25 PM
> To: Wu, Feng <feng.wu@intel.com>; xen-devel@lists.xen.org
> Cc: Keir Fraser <keir@xen.org>; Jan Beulich <jbeulich@suse.com>; Andrew
> Cooper <andrew.cooper3@citrix.com>; George Dunlap
> <george.dunlap@eu.citrix.com>; Dario Faggioli <dario.faggioli@citrix.com>
> Subject: RE: [PATCH v14 1/2] vmx: VT-d posted-interrupt core logic handling
> 
> > From: Wu, Feng
> > Sent: Monday, February 29, 2016 11:00 AM
> >
> > This is the core logic handling for VT-d posted-interrupts. Basically it
> > deals with how and when to update posted-interrupts during the following
> > scenarios:
> > - vCPU is preempted
> > - vCPU is slept
> > - vCPU is blocked
> >
> > When vCPU is preempted/slept, we update the posted-interrupts during
> > scheduling by introducing two new architecutral scheduler hooks:
> > vmx_pi_switch_from() and vmx_pi_switch_to(). When vCPU is blocked, we
> > introduce a new architectural hook: arch_vcpu_block() to update
> > posted-interrupts descriptor.
> >
> > Besides that, before VM-entry, we will make sure the 'NV' filed is set
> > to 'posted_intr_vector' and the vCPU is not in any blocking lists, which
> > is needed when vCPU is running in non-root mode. The reason we do this
> check
> > is because we change the posted-interrupts descriptor in vcpu_block(),
> > however, we don't change it back in vcpu_unblock() or when vcpu_block()
> > directly returns due to event delivery (in fact, we don't need to do it
> > in the two places, that is why we do it before VM-Entry).
> >
> > When we handle the lazy context switch for the following two scenarios:
> > - Preempted by a tasklet, which uses in an idle context.
> > - the prev vcpu is in offline and no new available vcpus in run queue.
> > We don't change the 'SN' bit in posted-interrupt descriptor, this
> > may incur spurious PI notification events, but since PI notification
> > event is only sent when 'ON' is clear, and once the PI notificatoin
> > is sent, ON is set by hardware, hence no more notification events
> > before 'ON' is clear. Besides that, spurious PI notification events are
> > going to happen from time to time in Xen hypervisor, such as, when
> > guests trap to Xen and PI notification event happens, there is
> > nothing Xen actually needs to do about it, the interrupts will be
> > delivered to guest atht the next time we do a VMENTRY.
> >
> > CC: Keir Fraser <keir@xen.org>
> > CC: Jan Beulich <jbeulich@suse.com>
> > CC: Andrew Cooper <andrew.cooper3@citrix.com>
> > CC: Kevin Tian <kevin.tian@intel.com>
> > CC: George Dunlap <george.dunlap@eu.citrix.com>
> > CC: Dario Faggioli <dario.faggioli@citrix.com>
> > Suggested-by: Yang Zhang <yang.z.zhang@intel.com>
> > Suggested-by: Dario Faggioli <dario.faggioli@citrix.com>
> > Suggested-by: George Dunlap <george.dunlap@citrix.com>
> > Suggested-by: Jan Beulich <jbeulich@suse.com>
> > Signed-off-by: Feng Wu <feng.wu@intel.com>
> > Reviewed-by: George Dunlap <george.dunlap@citrix.com>
> 
> Acked-by: Kevin Tian <kevin.tian@intel.com>

Thanks a lot, Kevin!

Thanks
Feng

_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: [PATCH v14 1/2] vmx: VT-d posted-interrupt core logic handling
  2016-03-01  5:39       ` Wu, Feng
@ 2016-03-01  9:24         ` Jan Beulich
  0 siblings, 0 replies; 53+ messages in thread
From: Jan Beulich @ 2016-03-01  9:24 UTC (permalink / raw)
  To: George Dunlap, Feng Wu
  Cc: Andrew Cooper, Dario Faggioli, Keir Fraser, Kevin Tian, xen-devel

>>> On 01.03.16 at 06:39, <feng.wu@intel.com> wrote:

> 
>> -----Original Message-----
>> From: Dario Faggioli [mailto:dario.faggioli@citrix.com]
>> Sent: Monday, February 29, 2016 9:52 PM
>> To: Jan Beulich <JBeulich@suse.com>; George Dunlap
>> <george.dunlap@eu.citrix.com>; Wu, Feng <feng.wu@intel.com>
>> Cc: Andrew Cooper <andrew.cooper3@citrix.com>; Tian, Kevin
>> <kevin.tian@intel.com>; xen-devel@lists.xen.org; Keir Fraser <keir@xen.org>
>> Subject: Re: [Xen-devel] [PATCH v14 1/2] vmx: VT-d posted-interrupt core 
> logic
>> handling
>> 
>> On Mon, 2016-02-29 at 06:33 -0700, Jan Beulich wrote:
>> > > > > On 29.02.16 at 04:00, <feng.wu@intel.com> wrote:
>> > > This is the core logic handling for VT-d posted-interrupts.
>> > > Basically it
>> > > deals with how and when to update posted-interrupts during the
>> > > following
>> > > scenarios:
>> > > - vCPU is preempted
>> > > - vCPU is slept
>> > > - vCPU is blocked
>> > >
>> > [...]
>> > > CC: Keir Fraser <keir@xen.org>
>> > > CC: Jan Beulich <jbeulich@suse.com>
>> > > CC: Andrew Cooper <andrew.cooper3@citrix.com>
>> > > CC: Kevin Tian <kevin.tian@intel.com>
>> > > CC: George Dunlap <george.dunlap@eu.citrix.com>
>> > > CC: Dario Faggioli <dario.faggioli@citrix.com>
>> > > Suggested-by: Yang Zhang <yang.z.zhang@intel.com>
>> > > Suggested-by: Dario Faggioli <dario.faggioli@citrix.com>
>> > > Suggested-by: George Dunlap <george.dunlap@citrix.com>
>> > > Suggested-by: Jan Beulich <jbeulich@suse.com>
>> > > Signed-off-by: Feng Wu <feng.wu@intel.com>
>> > > Reviewed-by: George Dunlap <george.dunlap@citrix.com>
>> > With the comments George gave on v13 subsequent to this tag
>> > I'm not sure it was correct to retain it. George?
>> >
>> > Reviewed-by: Jan Beulich <jbeulich@suse.com>
>> > albeit in case another version is needed ...
>> >
>> And, if I'm still in time (if not, no big deal :-))
>> 
>> Reviewed-by: Dario Faggioli <dario.faggioli@citrix.com>
> 
> Really appreciate your huge efforts on reviewing this series, Jan,
> George, and Dario! 
> 
> BTW, Jan, considering that Kevin, as the VMX maintainer also gave
> his acked-by for this patch, do you think another version is needed?

Not unless further review comments make one necessary; namely
pending George's confirmation of the R-b that you've kept in place.

Jan


_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: [PATCH v14 1/2] vmx: VT-d posted-interrupt core logic handling
  2016-02-29 13:33   ` Jan Beulich
  2016-02-29 13:52     ` Dario Faggioli
@ 2016-03-01 10:16     ` George Dunlap
  2016-03-01 13:06       ` Wu, Feng
  1 sibling, 1 reply; 53+ messages in thread
From: George Dunlap @ 2016-03-01 10:16 UTC (permalink / raw)
  To: Jan Beulich, George Dunlap, Feng Wu
  Cc: Andrew Cooper, Dario Faggioli, Keir Fraser, Kevin Tian, xen-devel

On 29/02/16 13:33, Jan Beulich wrote:
>>>> On 29.02.16 at 04:00, <feng.wu@intel.com> wrote:
>> This is the core logic handling for VT-d posted-interrupts. Basically it
>> deals with how and when to update posted-interrupts during the following
>> scenarios:
>> - vCPU is preempted
>> - vCPU is slept
>> - vCPU is blocked
>>
>> When vCPU is preempted/slept, we update the posted-interrupts during
>> scheduling by introducing two new architecutral scheduler hooks:
>> vmx_pi_switch_from() and vmx_pi_switch_to(). When vCPU is blocked, we
>> introduce a new architectural hook: arch_vcpu_block() to update
>> posted-interrupts descriptor.
>>
>> Besides that, before VM-entry, we will make sure the 'NV' filed is set
>> to 'posted_intr_vector' and the vCPU is not in any blocking lists, which
>> is needed when vCPU is running in non-root mode. The reason we do this check
>> is because we change the posted-interrupts descriptor in vcpu_block(),
>> however, we don't change it back in vcpu_unblock() or when vcpu_block()
>> directly returns due to event delivery (in fact, we don't need to do it
>> in the two places, that is why we do it before VM-Entry).
>>
>> When we handle the lazy context switch for the following two scenarios:
>> - Preempted by a tasklet, which uses in an idle context.
>> - the prev vcpu is in offline and no new available vcpus in run queue.
>> We don't change the 'SN' bit in posted-interrupt descriptor, this
>> may incur spurious PI notification events, but since PI notification
>> event is only sent when 'ON' is clear, and once the PI notificatoin
>> is sent, ON is set by hardware, hence no more notification events
>> before 'ON' is clear. Besides that, spurious PI notification events are
>> going to happen from time to time in Xen hypervisor, such as, when
>> guests trap to Xen and PI notification event happens, there is
>> nothing Xen actually needs to do about it, the interrupts will be
>> delivered to guest atht the next time we do a VMENTRY.
>>
>> CC: Keir Fraser <keir@xen.org>
>> CC: Jan Beulich <jbeulich@suse.com>
>> CC: Andrew Cooper <andrew.cooper3@citrix.com>
>> CC: Kevin Tian <kevin.tian@intel.com>
>> CC: George Dunlap <george.dunlap@eu.citrix.com>
>> CC: Dario Faggioli <dario.faggioli@citrix.com>
>> Suggested-by: Yang Zhang <yang.z.zhang@intel.com>
>> Suggested-by: Dario Faggioli <dario.faggioli@citrix.com>
>> Suggested-by: George Dunlap <george.dunlap@citrix.com>
>> Suggested-by: Jan Beulich <jbeulich@suse.com>
>> Signed-off-by: Feng Wu <feng.wu@intel.com>
>> Reviewed-by: George Dunlap <george.dunlap@citrix.com>
> 
> With the comments George gave on v13 subsequent to this tag
> I'm not sure it was correct to retain it. George?
> 
> Reviewed-by: Jan Beulich <jbeulich@suse.com>
> albeit in case another version is needed ...

It probably wasn't correct to retain the Reviewed-by given my
outstanding comments about the macro.  But having looked at it again:

Reviewed-by: George Dunlap <george.dunlap@citrix.com>


_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: [PATCH v14 1/2] vmx: VT-d posted-interrupt core logic handling
  2016-03-01 10:16     ` George Dunlap
@ 2016-03-01 13:06       ` Wu, Feng
  0 siblings, 0 replies; 53+ messages in thread
From: Wu, Feng @ 2016-03-01 13:06 UTC (permalink / raw)
  To: George Dunlap, Jan Beulich, George Dunlap
  Cc: Tian, Kevin, Keir Fraser, Andrew Cooper, Dario Faggioli,
	xen-devel, Wu, Feng



> -----Original Message-----
> From: George Dunlap [mailto:george.dunlap@citrix.com]
> Sent: Tuesday, March 1, 2016 6:17 PM
> To: Jan Beulich <JBeulich@suse.com>; George Dunlap
> <george.dunlap@eu.citrix.com>; Wu, Feng <feng.wu@intel.com>
> Cc: Andrew Cooper <andrew.cooper3@citrix.com>; Dario Faggioli
> <dario.faggioli@citrix.com>; Tian, Kevin <kevin.tian@intel.com>; xen-
> devel@lists.xen.org; Keir Fraser <keir@xen.org>
> Subject: Re: [Xen-devel] [PATCH v14 1/2] vmx: VT-d posted-interrupt core logic
> handling
> 
> > With the comments George gave on v13 subsequent to this tag
> > I'm not sure it was correct to retain it. George?
> >
> > Reviewed-by: Jan Beulich <jbeulich@suse.com>
> > albeit in case another version is needed ...
> 
> It probably wasn't correct to retain the Reviewed-by given my
> outstanding comments about the macro.  But having looked at it again:
> 
> Reviewed-by: George Dunlap <george.dunlap@citrix.com>

Thanks for that, George!

Thanks,
Feng

_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Ideas Re: [PATCH v14 1/2] vmx: VT-d posted-interrupt core logic handling
  2016-02-29  3:00 ` [PATCH v14 1/2] vmx: VT-d posted-interrupt core logic handling Feng Wu
  2016-02-29 13:33   ` Jan Beulich
  2016-03-01  5:24   ` Tian, Kevin
@ 2016-03-04 22:00   ` Konrad Rzeszutek Wilk
  2016-03-07 11:21     ` George Dunlap
  2 siblings, 1 reply; 53+ messages in thread
From: Konrad Rzeszutek Wilk @ 2016-03-04 22:00 UTC (permalink / raw)
  To: Feng Wu
  Cc: Kevin Tian, Keir Fraser, George Dunlap, Andrew Cooper,
	Dario Faggioli, xen-devel, Jan Beulich

> +/* Handle VT-d posted-interrupt when VCPU is blocked. */
> +static void pi_wakeup_interrupt(struct cpu_user_regs *regs)
> +{
> +    struct arch_vmx_struct *vmx, *tmp;
> +    spinlock_t *lock = &per_cpu(vmx_pi_blocking, smp_processor_id()).lock;
> +    struct list_head *blocked_vcpus =
> +		&per_cpu(vmx_pi_blocking, smp_processor_id()).list;
> +
> +    ack_APIC_irq();
> +    this_cpu(irq_count)++;
> +
> +    spin_lock(lock);
> +
> +    /*
> +     * XXX: The length of the list depends on how many vCPU is current
> +     * blocked on this specific pCPU. This may hurt the interrupt latency
> +     * if the list grows to too many entries.
> +     */
> +    list_for_each_entry_safe(vmx, tmp, blocked_vcpus, pi_blocking.list)
> +    {


My recollection of the 'most-horrible' case of this being really bad is when
the scheduler puts the vCPU0 and VCPU1 of the guest on the same pCPU (as an example)
and they round-robin all the time.

<handwaving>
Would it be perhaps possible to have an anti-affinity flag to deter the
scheduler from this? That is whichever struct vcpu has 'anti-affinity' flag
set - the scheduler will try as much as it can _to not_ schedule the 'struct vcpu'
if the previous 'struct vcpu' had this flag as well on this pCPU?

And then try to schedule 'normal' guests.

[I am ignoring the toolstack plumbing for this and so on]

My naive thinking is that while it may result in a lot of a guest vCPU
moving around (as the prev 'struct vcpu' would disallow this new vCPU to run
on a CPU that already has this type of guest) it would 'spread' out the guests
with 'anti-affinity' flag across all the pCPUS.

It would suck for over-subscriptions but <handwaving>.

And maybe this enforcment need not been so strict. Perhaps it can allow
one 'prev' of an 'struct vpu' which has this flag enabled but not more than
two?

</handwaving>

/me goes off to the pub.

_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: Ideas Re: [PATCH v14 1/2] vmx: VT-d posted-interrupt core logic handling
  2016-03-04 22:00   ` Ideas " Konrad Rzeszutek Wilk
@ 2016-03-07 11:21     ` George Dunlap
  2016-03-07 15:53       ` Konrad Rzeszutek Wilk
  0 siblings, 1 reply; 53+ messages in thread
From: George Dunlap @ 2016-03-07 11:21 UTC (permalink / raw)
  To: Konrad Rzeszutek Wilk
  Cc: Kevin Tian, Keir Fraser, Andrew Cooper, Dario Faggioli,
	xen-devel, Jan Beulich, Feng Wu

On Fri, Mar 4, 2016 at 10:00 PM, Konrad Rzeszutek Wilk
<konrad.wilk@oracle.com> wrote:
>> +/* Handle VT-d posted-interrupt when VCPU is blocked. */
>> +static void pi_wakeup_interrupt(struct cpu_user_regs *regs)
>> +{
>> +    struct arch_vmx_struct *vmx, *tmp;
>> +    spinlock_t *lock = &per_cpu(vmx_pi_blocking, smp_processor_id()).lock;
>> +    struct list_head *blocked_vcpus =
>> +             &per_cpu(vmx_pi_blocking, smp_processor_id()).list;
>> +
>> +    ack_APIC_irq();
>> +    this_cpu(irq_count)++;
>> +
>> +    spin_lock(lock);
>> +
>> +    /*
>> +     * XXX: The length of the list depends on how many vCPU is current
>> +     * blocked on this specific pCPU. This may hurt the interrupt latency
>> +     * if the list grows to too many entries.
>> +     */
>> +    list_for_each_entry_safe(vmx, tmp, blocked_vcpus, pi_blocking.list)
>> +    {
>
>
> My recollection of the 'most-horrible' case of this being really bad is when
> the scheduler puts the vCPU0 and VCPU1 of the guest on the same pCPU (as an example)
> and they round-robin all the time.
>
> <handwaving>
> Would it be perhaps possible to have an anti-affinity flag to deter the
> scheduler from this? That is whichever struct vcpu has 'anti-affinity' flag
> set - the scheduler will try as much as it can _to not_ schedule the 'struct vcpu'
> if the previous 'struct vcpu' had this flag as well on this pCPU?

Well having vcpus from the same guest on the same pcpu is problematic
for a number of reasons -- spinlocks first and foremost.  So in
general trying to avoid that would be useful for most guests.

The thing with scheduling is that it's a bit like economics: it seems
simple but it's actually not at all obvious what the emergent behavior
will be from adding a simple rule. :-)

On the whole it seems unlikely that having two vcpus on a single pcpu
is a "stable" situation -- it's likely to be pretty transient, and
thus not have a major impact on performance.

That said, the load balancing code from credit2 *should*, in theory,
make it easier to implement this sort of thing; it has the concept of
a "cost" that it's trying to minimize; so you could in theory add a
"cost" to configurations where vcpus from the same processor share the
same pcpu.  Then it's not a hard-and-fast rule: if you have more vcpus
than pcpus, the scheduler will just deal. :-)

But I think some profiling is in order before anyone does serious work on this.

 -George

_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: Ideas Re: [PATCH v14 1/2] vmx: VT-d posted-interrupt core logic handling
  2016-03-07 11:21     ` George Dunlap
@ 2016-03-07 15:53       ` Konrad Rzeszutek Wilk
  2016-03-07 16:19         ` Dario Faggioli
  2016-03-08 12:02         ` George Dunlap
  0 siblings, 2 replies; 53+ messages in thread
From: Konrad Rzeszutek Wilk @ 2016-03-07 15:53 UTC (permalink / raw)
  To: George Dunlap
  Cc: Kevin Tian, Keir Fraser, Andrew Cooper, Dario Faggioli,
	xen-devel, Jan Beulich, Feng Wu

On Mon, Mar 07, 2016 at 11:21:33AM +0000, George Dunlap wrote:
> On Fri, Mar 4, 2016 at 10:00 PM, Konrad Rzeszutek Wilk
> <konrad.wilk@oracle.com> wrote:
> >> +/* Handle VT-d posted-interrupt when VCPU is blocked. */
> >> +static void pi_wakeup_interrupt(struct cpu_user_regs *regs)
> >> +{
> >> +    struct arch_vmx_struct *vmx, *tmp;
> >> +    spinlock_t *lock = &per_cpu(vmx_pi_blocking, smp_processor_id()).lock;
> >> +    struct list_head *blocked_vcpus =
> >> +             &per_cpu(vmx_pi_blocking, smp_processor_id()).list;
> >> +
> >> +    ack_APIC_irq();
> >> +    this_cpu(irq_count)++;
> >> +
> >> +    spin_lock(lock);
> >> +
> >> +    /*
> >> +     * XXX: The length of the list depends on how many vCPU is current
> >> +     * blocked on this specific pCPU. This may hurt the interrupt latency
> >> +     * if the list grows to too many entries.
> >> +     */
> >> +    list_for_each_entry_safe(vmx, tmp, blocked_vcpus, pi_blocking.list)
> >> +    {
> >
> >
> > My recollection of the 'most-horrible' case of this being really bad is when
> > the scheduler puts the vCPU0 and VCPU1 of the guest on the same pCPU (as an example)
> > and they round-robin all the time.
> >
> > <handwaving>
> > Would it be perhaps possible to have an anti-affinity flag to deter the
> > scheduler from this? That is whichever struct vcpu has 'anti-affinity' flag
> > set - the scheduler will try as much as it can _to not_ schedule the 'struct vcpu'
> > if the previous 'struct vcpu' had this flag as well on this pCPU?
> 
> Well having vcpus from the same guest on the same pcpu is problematic
> for a number of reasons -- spinlocks first and foremost.  So in
> general trying to avoid that would be useful for most guests.

PV ticketlocks in HVM and PV guests make this "manageable".

> 
> The thing with scheduling is that it's a bit like economics: it seems
> simple but it's actually not at all obvious what the emergent behavior
> will be from adding a simple rule. :-)

<nods>
> 
> On the whole it seems unlikely that having two vcpus on a single pcpu
> is a "stable" situation -- it's likely to be pretty transient, and
> thus not have a major impact on performance.

Except that we are concerned with it - in fact we are disabling this
feature because it may happen. How do we make sure it does not happen
all the time? Or at least do some back-off if things do get
in this situation.
> 
> That said, the load balancing code from credit2 *should*, in theory,
> make it easier to implement this sort of thing; it has the concept of
> a "cost" that it's trying to minimize; so you could in theory add a
> "cost" to configurations where vcpus from the same processor share the
> same pcpu.  Then it's not a hard-and-fast rule: if you have more vcpus
> than pcpus, the scheduler will just deal. :-)
> 
> But I think some profiling is in order before anyone does serious work on this.

I appreciate your response being 'profiling' instead of 'Are you
NUTS!?' :-)

> 
>  -George

_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: Ideas Re: [PATCH v14 1/2] vmx: VT-d posted-interrupt core logic handling
  2016-03-07 15:53       ` Konrad Rzeszutek Wilk
@ 2016-03-07 16:19         ` Dario Faggioli
  2016-03-07 20:23           ` Konrad Rzeszutek Wilk
  2016-03-08 12:02         ` George Dunlap
  1 sibling, 1 reply; 53+ messages in thread
From: Dario Faggioli @ 2016-03-07 16:19 UTC (permalink / raw)
  To: Konrad Rzeszutek Wilk, George Dunlap
  Cc: Kevin Tian, Keir Fraser, Andrew Cooper, xen-devel, Jan Beulich, Feng Wu


[-- Attachment #1.1: Type: text/plain, Size: 2088 bytes --]

On Mon, 2016-03-07 at 10:53 -0500, Konrad Rzeszutek Wilk wrote:
> On Mon, Mar 07, 2016 at 11:21:33AM +0000, George Dunlap wrote:
> > 
> > > <handwaving>
> > > Would it be perhaps possible to have an anti-affinity flag to
> > > deter the
> > > scheduler from this? That is whichever struct vcpu has 'anti-
> > > affinity' flag
> > > set - the scheduler will try as much as it can _to not_ schedule
> > > the 'struct vcpu'
> > > if the previous 'struct vcpu' had this flag as well on this pCPU?
>
That can also be seen as step in the direction of (supporting) gang
scheduling, which we've said already it would be something interesting
to look at, although difficult to implement and even more difficult to
figure out whether it is actually a good thing for most workloads.

In any case, I see from where this comes, and am up for thinking about
it, although my fear is that it would complicate the code by quite a
bit, so I agree with George that profiling work is necessary to try to
assess whether it could be really useful (as well as, once
implemented/drafted, whether it is really good and does not cause perf
regressions).

> > On the whole it seems unlikely that having two vcpus on a single
> > pcpu
> > is a "stable" situation -- it's likely to be pretty transient, and
> > thus not have a major impact on performance.
> Except that we are concerned with it - in fact we are disabling this
> feature because it may happen. 
>
I'm sorry, I'm not getting, what feature are you disabling?

> > But I think some profiling is in order before anyone does serious
> > work on this.
> I appreciate your response being 'profiling' instead of 'Are you
> NUTS!?' :-)
> 
That's only because everyone knows you're nuts, there's no need to
state it all the times! :-P :-P

Regards,
Dario
-- 
<<This happens because I choose it to happen!>> (Raistlin Majere)
-----------------------------------------------------------------
Dario Faggioli, Ph.D, http://about.me/dario.faggioli
Senior Software Engineer, Citrix Systems R&D Ltd., Cambridge (UK)


[-- Attachment #1.2: This is a digitally signed message part --]
[-- Type: application/pgp-signature, Size: 181 bytes --]

[-- Attachment #2: Type: text/plain, Size: 126 bytes --]

_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: Ideas Re: [PATCH v14 1/2] vmx: VT-d posted-interrupt core logic handling
  2016-03-07 16:19         ` Dario Faggioli
@ 2016-03-07 20:23           ` Konrad Rzeszutek Wilk
  0 siblings, 0 replies; 53+ messages in thread
From: Konrad Rzeszutek Wilk @ 2016-03-07 20:23 UTC (permalink / raw)
  To: Dario Faggioli
  Cc: Kevin Tian, Keir Fraser, George Dunlap, Andrew Cooper, xen-devel,
	Jan Beulich, Feng Wu

On Mon, Mar 07, 2016 at 05:19:59PM +0100, Dario Faggioli wrote:
> On Mon, 2016-03-07 at 10:53 -0500, Konrad Rzeszutek Wilk wrote:
> > On Mon, Mar 07, 2016 at 11:21:33AM +0000, George Dunlap wrote:
> > > 
> > > > <handwaving>
> > > > Would it be perhaps possible to have an anti-affinity flag to
> > > > deter the
> > > > scheduler from this? That is whichever struct vcpu has 'anti-
> > > > affinity' flag
> > > > set - the scheduler will try as much as it can _to not_ schedule
> > > > the 'struct vcpu'
> > > > if the previous 'struct vcpu' had this flag as well on this pCPU?
> >
> That can also be seen as step in the direction of (supporting) gang
> scheduling, which we've said already it would be something interesting
> to look at, although difficult to implement and even more difficult to
> figure out whether it is actually a good thing for most workloads.
> 
> In any case, I see from where this comes, and am up for thinking about
> it, although my fear is that it would complicate the code by quite a
> bit, so I agree with George that profiling work is necessary to try to
> assess whether it could be really useful (as well as, once
> implemented/drafted, whether it is really good and does not cause perf
> regressions).
> 
> > > On the whole it seems unlikely that having two vcpus on a single
> > > pcpu
> > > is a "stable" situation -- it's likely to be pretty transient, and
> > > thus not have a major impact on performance.
> > Except that we are concerned with it - in fact we are disabling this
> > feature because it may happen. 
> >
> I'm sorry, I'm not getting, what feature are you disabling?

It is already disabled in the code:

 62 /*                                                                              
 63  * In the current implementation of VT-d posted interrupts, in some extreme     
 64  * cases, the per cpu list which saves the blocked vCPU will be very long,      
 65  * and this will affect the interrupt latency, so let this feature off by       
 66  * default until we find a good solution to resolve it.                         
 67  */                                                                             
 68 bool_t __read_mostly iommu_intpost;                     


> 
> > > But I think some profiling is in order before anyone does serious
> > > work on this.
> > I appreciate your response being 'profiling' instead of 'Are you
> > NUTS!?' :-)
> > 
> That's only because everyone knows you're nuts, there's no need to
> state it all the times! :-P :-P

<laughs>

Glad that you have the _right_ expectations of me :) 

_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: Ideas Re: [PATCH v14 1/2] vmx: VT-d posted-interrupt core logic handling
  2016-03-07 15:53       ` Konrad Rzeszutek Wilk
  2016-03-07 16:19         ` Dario Faggioli
@ 2016-03-08 12:02         ` George Dunlap
  2016-03-08 13:10           ` Wu, Feng
  1 sibling, 1 reply; 53+ messages in thread
From: George Dunlap @ 2016-03-08 12:02 UTC (permalink / raw)
  To: Konrad Rzeszutek Wilk, George Dunlap
  Cc: Kevin Tian, Keir Fraser, Andrew Cooper, Dario Faggioli,
	xen-devel, Jan Beulich, Feng Wu

On 07/03/16 15:53, Konrad Rzeszutek Wilk wrote:
> On Mon, Mar 07, 2016 at 11:21:33AM +0000, George Dunlap wrote:
>> On Fri, Mar 4, 2016 at 10:00 PM, Konrad Rzeszutek Wilk
>> <konrad.wilk@oracle.com> wrote:
>>>> +/* Handle VT-d posted-interrupt when VCPU is blocked. */
>>>> +static void pi_wakeup_interrupt(struct cpu_user_regs *regs)
>>>> +{
>>>> +    struct arch_vmx_struct *vmx, *tmp;
>>>> +    spinlock_t *lock = &per_cpu(vmx_pi_blocking, smp_processor_id()).lock;
>>>> +    struct list_head *blocked_vcpus =
>>>> +             &per_cpu(vmx_pi_blocking, smp_processor_id()).list;
>>>> +
>>>> +    ack_APIC_irq();
>>>> +    this_cpu(irq_count)++;
>>>> +
>>>> +    spin_lock(lock);
>>>> +
>>>> +    /*
>>>> +     * XXX: The length of the list depends on how many vCPU is current
>>>> +     * blocked on this specific pCPU. This may hurt the interrupt latency
>>>> +     * if the list grows to too many entries.
>>>> +     */
>>>> +    list_for_each_entry_safe(vmx, tmp, blocked_vcpus, pi_blocking.list)
>>>> +    {
>>>
>>>
>>> My recollection of the 'most-horrible' case of this being really bad is when
>>> the scheduler puts the vCPU0 and VCPU1 of the guest on the same pCPU (as an example)
>>> and they round-robin all the time.
>>>
>>> <handwaving>
>>> Would it be perhaps possible to have an anti-affinity flag to deter the
>>> scheduler from this? That is whichever struct vcpu has 'anti-affinity' flag
>>> set - the scheduler will try as much as it can _to not_ schedule the 'struct vcpu'
>>> if the previous 'struct vcpu' had this flag as well on this pCPU?
>>
>> Well having vcpus from the same guest on the same pcpu is problematic
>> for a number of reasons -- spinlocks first and foremost.  So in
>> general trying to avoid that would be useful for most guests.
> 
> PV ticketlocks in HVM and PV guests make this "manageable".
> 
>>
>> The thing with scheduling is that it's a bit like economics: it seems
>> simple but it's actually not at all obvious what the emergent behavior
>> will be from adding a simple rule. :-)
> 
> <nods>
>>
>> On the whole it seems unlikely that having two vcpus on a single pcpu
>> is a "stable" situation -- it's likely to be pretty transient, and
>> thus not have a major impact on performance.
> 
> Except that we are concerned with it - in fact we are disabling this
> feature because it may happen. How do we make sure it does not happen
> all the time? Or at least do some back-off if things do get
> in this situation.

So it's disabled by default based on a theoretical fear that it *may*
cause performance problems, but without any actual performance problems
having been observed?

It seems like there are a couple of ways we could approach this:

1. Try to optimize the reverse look-up code so that it's not a linear
linked list (getting rid of the theoretical fear)

2. Try to test engineered situations where we expect this to be a
problem, to see how big of a problem it is (proving the theory to be
accurate or inaccurate in this case)

3. Turn the feature on by default as soon as the 4.8 window opens up,
perhaps with some sort of a check that runs when in debug mode that
looks for the condition we're afraid of happening and BUG()s.  If we run
a full development cycle without anyone hitting the bug in testing, then
we just leave the feature on.

Then we'll only look at adding complexity to the scheduler if there's
actually a problem to solve.

 -George


_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: Ideas Re: [PATCH v14 1/2] vmx: VT-d posted-interrupt core logic handling
  2016-03-08 12:02         ` George Dunlap
@ 2016-03-08 13:10           ` Wu, Feng
  2016-03-08 14:42             ` George Dunlap
  0 siblings, 1 reply; 53+ messages in thread
From: Wu, Feng @ 2016-03-08 13:10 UTC (permalink / raw)
  To: George Dunlap, Konrad Rzeszutek Wilk, George Dunlap
  Cc: Tian, Kevin, Keir Fraser, Andrew Cooper, Dario Faggioli,
	xen-devel, Jan Beulich, Wu, Feng



> -----Original Message-----
> From: George Dunlap [mailto:george.dunlap@citrix.com]
> Sent: Tuesday, March 8, 2016 8:02 PM
> To: Konrad Rzeszutek Wilk <konrad.wilk@oracle.com>; George Dunlap
> <George.Dunlap@eu.citrix.com>
> Cc: Wu, Feng <feng.wu@intel.com>; Tian, Kevin <kevin.tian@intel.com>; Keir
> Fraser <keir@xen.org>; Andrew Cooper <andrew.cooper3@citrix.com>; Dario
> Faggioli <dario.faggioli@citrix.com>; xen-devel@lists.xen.org; Jan Beulich
> <jbeulich@suse.com>
> Subject: Re: [Xen-devel] Ideas Re: [PATCH v14 1/2] vmx: VT-d posted-interrupt
> core logic handling
> 
> On 07/03/16 15:53, Konrad Rzeszutek Wilk wrote:
> > On Mon, Mar 07, 2016 at 11:21:33AM +0000, George Dunlap wrote:
> >> On Fri, Mar 4, 2016 at 10:00 PM, Konrad Rzeszutek Wilk
> >> <konrad.wilk@oracle.com> wrote:
> >>>> +/* Handle VT-d posted-interrupt when VCPU is blocked. */
> >>>> +static void pi_wakeup_interrupt(struct cpu_user_regs *regs)
> >>>> +{
> >>>> +    struct arch_vmx_struct *vmx, *tmp;
> >>>> +    spinlock_t *lock = &per_cpu(vmx_pi_blocking,
> smp_processor_id()).lock;
> >>>> +    struct list_head *blocked_vcpus =
> >>>> +             &per_cpu(vmx_pi_blocking, smp_processor_id()).list;
> >>>> +
> >>>> +    ack_APIC_irq();
> >>>> +    this_cpu(irq_count)++;
> >>>> +
> >>>> +    spin_lock(lock);
> >>>> +
> >>>> +    /*
> >>>> +     * XXX: The length of the list depends on how many vCPU is current
> >>>> +     * blocked on this specific pCPU. This may hurt the interrupt latency
> >>>> +     * if the list grows to too many entries.
> >>>> +     */
> >>>> +    list_for_each_entry_safe(vmx, tmp, blocked_vcpus, pi_blocking.list)
> >>>> +    {
> >>>
> >>>
> >>> My recollection of the 'most-horrible' case of this being really bad is when
> >>> the scheduler puts the vCPU0 and VCPU1 of the guest on the same pCPU (as
> an example)
> >>> and they round-robin all the time.
> >>>
> >>> <handwaving>
> >>> Would it be perhaps possible to have an anti-affinity flag to deter the
> >>> scheduler from this? That is whichever struct vcpu has 'anti-affinity' flag
> >>> set - the scheduler will try as much as it can _to not_ schedule the 'struct
> vcpu'
> >>> if the previous 'struct vcpu' had this flag as well on this pCPU?
> >>
> >> Well having vcpus from the same guest on the same pcpu is problematic
> >> for a number of reasons -- spinlocks first and foremost.  So in
> >> general trying to avoid that would be useful for most guests.
> >
> > PV ticketlocks in HVM and PV guests make this "manageable".
> >
> >>
> >> The thing with scheduling is that it's a bit like economics: it seems
> >> simple but it's actually not at all obvious what the emergent behavior
> >> will be from adding a simple rule. :-)
> >
> > <nods>
> >>
> >> On the whole it seems unlikely that having two vcpus on a single pcpu
> >> is a "stable" situation -- it's likely to be pretty transient, and
> >> thus not have a major impact on performance.
> >
> > Except that we are concerned with it - in fact we are disabling this
> > feature because it may happen. How do we make sure it does not happen
> > all the time? Or at least do some back-off if things do get
> > in this situation.
> 
> So it's disabled by default based on a theoretical fear that it *may*
> cause performance problems, but without any actual performance problems
> having been observed?

Yes, according to Jan's comments in previous thread, theoretically, the list
may become very long, so he tend to make this feature default off now.

> 
> It seems like there are a couple of ways we could approach this:
> 
> 1. Try to optimize the reverse look-up code so that it's not a linear
> linked list (getting rid of the theoretical fear)

Good point.

> 
> 2. Try to test engineered situations where we expect this to be a
> problem, to see how big of a problem it is (proving the theory to be
> accurate or inaccurate in this case)

Maybe we can run a SMP guest with all the vcpus pinned to a dedicated
pCPU, we can run some benchmark in the guest with VT-d PI and without
VT-d PI, then see the performance difference between these two sceanrios.

> 
> 3. Turn the feature on by default as soon as the 4.8 window opens up,
> perhaps with some sort of a check that runs when in debug mode that
> looks for the condition we're afraid of happening and BUG()s.  If we run
> a full development cycle without anyone hitting the bug in testing, then
> we just leave the feature on.

Maybe we can pre-define a max acceptable length of the list,  if it really
reach the number, print out a warning or something like that. However,
how to decide the max length is a problem. May need more thinking.

Thanks,
Feng

> 
> Then we'll only look at adding complexity to the scheduler if there's
> actually a problem to solve.
> 
>  -George


_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: Ideas Re: [PATCH v14 1/2] vmx: VT-d posted-interrupt core logic handling
  2016-03-08 13:10           ` Wu, Feng
@ 2016-03-08 14:42             ` George Dunlap
  2016-03-08 15:42               ` Jan Beulich
  0 siblings, 1 reply; 53+ messages in thread
From: George Dunlap @ 2016-03-08 14:42 UTC (permalink / raw)
  To: Wu, Feng
  Cc: Tian, Kevin, Keir Fraser, Andrew Cooper, Dario Faggioli,
	George Dunlap, xen-devel, Jan Beulich

On Tue, Mar 8, 2016 at 1:10 PM, Wu, Feng <feng.wu@intel.com> wrote:
>> -----Original Message-----
>> From: George Dunlap [mailto:george.dunlap@citrix.com]
[snip]
>> It seems like there are a couple of ways we could approach this:
>>
>> 1. Try to optimize the reverse look-up code so that it's not a linear
>> linked list (getting rid of the theoretical fear)
>
> Good point.
>
>>
>> 2. Try to test engineered situations where we expect this to be a
>> problem, to see how big of a problem it is (proving the theory to be
>> accurate or inaccurate in this case)
>
> Maybe we can run a SMP guest with all the vcpus pinned to a dedicated
> pCPU, we can run some benchmark in the guest with VT-d PI and without
> VT-d PI, then see the performance difference between these two sceanrios.

This would give us an idea what the worst-case scenario would be.  But
pinning all vcpus to a single pcpu isn't really a sensible use case we
want to support -- if you have to do something stupid to get a
performance regression, then I as far as I'm concerned it's not a
problem.

Or to put it a different way: If we pin 10 vcpus to a single pcpu and
then pound them all with posted interrupts, and there is *no*
significant performance regression, then that will conclusively prove
that the theoretical performance regression is of no concern, and we
can enable PI by default.

On the other hand, if we pin 10 vcpus to a single pcpu, pound them all
with posted interrupts, and then there *is* a significant performance
regression, then it would still not convince me there is a real
problem to be solved.  There is only actually a problem if the "long
chain of vcpus" can happen in the course of a semi-realistic use-case.

Suppose we had a set of SRIOV NICs with 10-20 virtual functions total,
assigned to 10-20 VMs, and those VMs in a cpupool confined to a single
socket of about 4 cores; and then we do a really network-intensive
benchmark. That's a *bit* far-fetched, but it's something that might
conceivably happen in the real world without any deliberate stupidity.
If there's no significant performance issues in that case, I would
think we can say that posted interrupts are robust enough to be
enabled by default.

>> 3. Turn the feature on by default as soon as the 4.8 window opens up,
>> perhaps with some sort of a check that runs when in debug mode that
>> looks for the condition we're afraid of happening and BUG()s.  If we run
>> a full development cycle without anyone hitting the bug in testing, then
>> we just leave the feature on.
>
> Maybe we can pre-define a max acceptable length of the list,  if it really
> reach the number, print out a warning or something like that. However,
> how to decide the max length is a problem. May need more thinking.

I think we want to measure the amount of time spent in the interrupt
handler (or with interrupts disabled).  It doesn't matter if the list
is 100 items long, if it can be handled in 500us.  On the other hand,
if a list of 4 elements takes 20ms, there's a pretty massive problem.
:-)

I don't have a good idea what an unreasonably large number would be here -- Jan?

 -George

_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: Ideas Re: [PATCH v14 1/2] vmx: VT-d posted-interrupt core logic handling
  2016-03-08 14:42             ` George Dunlap
@ 2016-03-08 15:42               ` Jan Beulich
  2016-03-08 17:05                 ` George Dunlap
  0 siblings, 1 reply; 53+ messages in thread
From: Jan Beulich @ 2016-03-08 15:42 UTC (permalink / raw)
  To: George Dunlap, Feng Wu
  Cc: Kevin Tian, Keir Fraser, Andrew Cooper, Dario Faggioli,
	George Dunlap, xen-devel

>>> On 08.03.16 at 15:42, <George.Dunlap@eu.citrix.com> wrote:
> On Tue, Mar 8, 2016 at 1:10 PM, Wu, Feng <feng.wu@intel.com> wrote:
>>> -----Original Message-----
>>> From: George Dunlap [mailto:george.dunlap@citrix.com]
> [snip]
>>> It seems like there are a couple of ways we could approach this:
>>>
>>> 1. Try to optimize the reverse look-up code so that it's not a linear
>>> linked list (getting rid of the theoretical fear)
>>
>> Good point.
>>
>>>
>>> 2. Try to test engineered situations where we expect this to be a
>>> problem, to see how big of a problem it is (proving the theory to be
>>> accurate or inaccurate in this case)
>>
>> Maybe we can run a SMP guest with all the vcpus pinned to a dedicated
>> pCPU, we can run some benchmark in the guest with VT-d PI and without
>> VT-d PI, then see the performance difference between these two sceanrios.
> 
> This would give us an idea what the worst-case scenario would be.

How would a single VM ever give us an idea about the worst
case? Something getting close to worst case is a ton of single
vCPU guests all temporarily pinned to one and the same pCPU
(could be multi-vCPU ones, but the more vCPU-s the more
artificial this pinning would become) right before they go into
blocked state (i.e. through one of the two callers of
arch_vcpu_block()), the pinning removed while blocked, and
then all getting woken at once.

>  But
> pinning all vcpus to a single pcpu isn't really a sensible use case we
> want to support -- if you have to do something stupid to get a
> performance regression, then I as far as I'm concerned it's not a
> problem.
> 
> Or to put it a different way: If we pin 10 vcpus to a single pcpu and
> then pound them all with posted interrupts, and there is *no*
> significant performance regression, then that will conclusively prove
> that the theoretical performance regression is of no concern, and we
> can enable PI by default.

The point isn't the pinning. The point is what pCPU they're on when
going to sleep. And that could involve quite a few more than just
10 vCPU-s, provided they all sleep long enough.

And the "theoretical performance regression is of no concern" is
also not a proper way of looking at it, I would say: Even if such
a situation would happen extremely rarely, if it can happen at all,
it would still be a security issue.

> On the other hand, if we pin 10 vcpus to a single pcpu, pound them all
> with posted interrupts, and then there *is* a significant performance
> regression, then it would still not convince me there is a real
> problem to be solved.  There is only actually a problem if the "long
> chain of vcpus" can happen in the course of a semi-realistic use-case.
> 
> Suppose we had a set of SRIOV NICs with 10-20 virtual functions total,
> assigned to 10-20 VMs, and those VMs in a cpupool confined to a single
> socket of about 4 cores; and then we do a really network-intensive
> benchmark. That's a *bit* far-fetched, but it's something that might
> conceivably happen in the real world without any deliberate stupidity.
> If there's no significant performance issues in that case, I would
> think we can say that posted interrupts are robust enough to be
> enabled by default.
> 
>>> 3. Turn the feature on by default as soon as the 4.8 window opens up,
>>> perhaps with some sort of a check that runs when in debug mode that
>>> looks for the condition we're afraid of happening and BUG()s.  If we run
>>> a full development cycle without anyone hitting the bug in testing, then
>>> we just leave the feature on.
>>
>> Maybe we can pre-define a max acceptable length of the list,  if it really
>> reach the number, print out a warning or something like that. However,
>> how to decide the max length is a problem. May need more thinking.
> 
> I think we want to measure the amount of time spent in the interrupt
> handler (or with interrupts disabled).  It doesn't matter if the list
> is 100 items long, if it can be handled in 500us.  On the other hand,
> if a list of 4 elements takes 20ms, there's a pretty massive problem.
> :-)

Spending on the order of 500us in an interrupt handler would
already seem pretty long to me, especially when the interrupt
may get raised at a high frequency. Even more so if, when in
that state, _each_ invocation of the interrupt handler would
take that long: With an (imo not unrealistic) interrupt rate of
1kHz we would spend half of the available CPU time in that
handler.

> I don't have a good idea what an unreasonably large number would be here -- 
> Jan?

Neither do I, unfortunately.

Jan

_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: Ideas Re: [PATCH v14 1/2] vmx: VT-d posted-interrupt core logic handling
  2016-03-08 15:42               ` Jan Beulich
@ 2016-03-08 17:05                 ` George Dunlap
  2016-03-08 17:26                   ` Jan Beulich
  2016-03-09  5:22                   ` Ideas Re: [PATCH v14 1/2] " Wu, Feng
  0 siblings, 2 replies; 53+ messages in thread
From: George Dunlap @ 2016-03-08 17:05 UTC (permalink / raw)
  To: Jan Beulich, George Dunlap, Feng Wu
  Cc: Kevin Tian, Keir Fraser, Andrew Cooper, Dario Faggioli, xen-devel

On 08/03/16 15:42, Jan Beulich wrote:
>>>> On 08.03.16 at 15:42, <George.Dunlap@eu.citrix.com> wrote:
>> On Tue, Mar 8, 2016 at 1:10 PM, Wu, Feng <feng.wu@intel.com> wrote:
>>>> -----Original Message-----
>>>> From: George Dunlap [mailto:george.dunlap@citrix.com]
>> [snip]
>>>> It seems like there are a couple of ways we could approach this:
>>>>
>>>> 1. Try to optimize the reverse look-up code so that it's not a linear
>>>> linked list (getting rid of the theoretical fear)
>>>
>>> Good point.
>>>
>>>>
>>>> 2. Try to test engineered situations where we expect this to be a
>>>> problem, to see how big of a problem it is (proving the theory to be
>>>> accurate or inaccurate in this case)
>>>
>>> Maybe we can run a SMP guest with all the vcpus pinned to a dedicated
>>> pCPU, we can run some benchmark in the guest with VT-d PI and without
>>> VT-d PI, then see the performance difference between these two sceanrios.
>>
>> This would give us an idea what the worst-case scenario would be.
> 
> How would a single VM ever give us an idea about the worst
> case? Something getting close to worst case is a ton of single
> vCPU guests all temporarily pinned to one and the same pCPU
> (could be multi-vCPU ones, but the more vCPU-s the more
> artificial this pinning would become) right before they go into
> blocked state (i.e. through one of the two callers of
> arch_vcpu_block()), the pinning removed while blocked, and
> then all getting woken at once.

Why would removing the pinning be important?

And I guess it's actually the case that it doesn't need all VMs to
actually be *receiving* interrupts; it just requires them to be
*capable* of receiving interrupts, for there to be a long chain all
blocked on the same physical cpu.

> 
>>  But
>> pinning all vcpus to a single pcpu isn't really a sensible use case we
>> want to support -- if you have to do something stupid to get a
>> performance regression, then I as far as I'm concerned it's not a
>> problem.
>>
>> Or to put it a different way: If we pin 10 vcpus to a single pcpu and
>> then pound them all with posted interrupts, and there is *no*
>> significant performance regression, then that will conclusively prove
>> that the theoretical performance regression is of no concern, and we
>> can enable PI by default.
> 
> The point isn't the pinning. The point is what pCPU they're on when
> going to sleep. And that could involve quite a few more than just
> 10 vCPU-s, provided they all sleep long enough.
> 
> And the "theoretical performance regression is of no concern" is
> also not a proper way of looking at it, I would say: Even if such
> a situation would happen extremely rarely, if it can happen at all,
> it would still be a security issue.

What I'm trying to get at is -- exactly what situation?  What actually
constitutes a problematic interrupt latency / interrupt processing
workload, how many vcpus must be sleeping on the same pcpu to actually
risk triggering that latency / workload, and how feasible is it that
such a situation would arise in a reasonable scenario?

If 200us is too long, and it only takes 3 sleeping vcpus to get there,
then yes, there is a genuine problem we need to try to address before we
turn it on by default.  If we say that up to 500us is tolerable, and it
takes 100 sleeping vcpus to reach that latency, then this is something I
don't really think we need to worry about.

"I think something bad may happen" is a really difficult to work with.
"I want to make sure that even a high number of blocked cpus won't cause
the interrupt latency to exceed 500us; and I want it to be basically
impossible for the interrupt latency to exceed 5ms under any
circumstances" is a concrete target someone can either demonstrate that
they meet, or aim for when trying to improve the situation.

Feng: It should be pretty easy for you to:
* Implement a modified version of Xen where
 - *All* vcpus get put on the waitqueue
 - Measure how long it took to run the loop in pi_wakeup_interrupt
* Have one VM receiving posted interrupts on a regular basis.
* Slowly increase the number of vcpus blocked on a single cpu (e.g., by
creating more guests), stopping when you either reach 500us or 500
vcpus. :-)

To report the measurements, you could either create a Xen trace record
and use xentrace_format or xenalyze to plot the results; or you could
create some software performance counters for different "buckets" --
less than 100us, 100-200us, 200-300us, 300-400us, 400-500us, and more
than 500us.

Or you could printk the min / average / max every 5000 interrupts or so. :-)

To test, it seems like using a network benchmark with short packet
lengths should be able to trigger large numbers of interrupts; and it
also can let you know if / when there's a performance impact of adding
more vcpus.

Or alternately, you could try to come up with a quicker reverse-lookup
algorithm. :-)

 -George


_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: Ideas Re: [PATCH v14 1/2] vmx: VT-d posted-interrupt core logic handling
  2016-03-08 17:05                 ` George Dunlap
@ 2016-03-08 17:26                   ` Jan Beulich
  2016-03-08 18:38                     ` George Dunlap
  2016-03-09  5:22                   ` Ideas Re: [PATCH v14 1/2] " Wu, Feng
  1 sibling, 1 reply; 53+ messages in thread
From: Jan Beulich @ 2016-03-08 17:26 UTC (permalink / raw)
  To: George Dunlap, George Dunlap, Feng Wu
  Cc: Kevin Tian, Keir Fraser, Andrew Cooper, Dario Faggioli, xen-devel

>>> On 08.03.16 at 18:05, <george.dunlap@citrix.com> wrote:
> On 08/03/16 15:42, Jan Beulich wrote:
>>>>> On 08.03.16 at 15:42, <George.Dunlap@eu.citrix.com> wrote:
>>> On Tue, Mar 8, 2016 at 1:10 PM, Wu, Feng <feng.wu@intel.com> wrote:
>>>>> -----Original Message-----
>>>>> From: George Dunlap [mailto:george.dunlap@citrix.com]
>>>>>
>>>>> 2. Try to test engineered situations where we expect this to be a
>>>>> problem, to see how big of a problem it is (proving the theory to be
>>>>> accurate or inaccurate in this case)
>>>>
>>>> Maybe we can run a SMP guest with all the vcpus pinned to a dedicated
>>>> pCPU, we can run some benchmark in the guest with VT-d PI and without
>>>> VT-d PI, then see the performance difference between these two sceanrios.
>>>
>>> This would give us an idea what the worst-case scenario would be.
>> 
>> How would a single VM ever give us an idea about the worst
>> case? Something getting close to worst case is a ton of single
>> vCPU guests all temporarily pinned to one and the same pCPU
>> (could be multi-vCPU ones, but the more vCPU-s the more
>> artificial this pinning would become) right before they go into
>> blocked state (i.e. through one of the two callers of
>> arch_vcpu_block()), the pinning removed while blocked, and
>> then all getting woken at once.
> 
> Why would removing the pinning be important?

It's not important by itself, other than to avoid all vCPU-s then
waking up on the one pCPU.

> And I guess it's actually the case that it doesn't need all VMs to
> actually be *receiving* interrupts; it just requires them to be
> *capable* of receiving interrupts, for there to be a long chain all
> blocked on the same physical cpu.

Yes.

>>>  But
>>> pinning all vcpus to a single pcpu isn't really a sensible use case we
>>> want to support -- if you have to do something stupid to get a
>>> performance regression, then I as far as I'm concerned it's not a
>>> problem.
>>>
>>> Or to put it a different way: If we pin 10 vcpus to a single pcpu and
>>> then pound them all with posted interrupts, and there is *no*
>>> significant performance regression, then that will conclusively prove
>>> that the theoretical performance regression is of no concern, and we
>>> can enable PI by default.
>> 
>> The point isn't the pinning. The point is what pCPU they're on when
>> going to sleep. And that could involve quite a few more than just
>> 10 vCPU-s, provided they all sleep long enough.
>> 
>> And the "theoretical performance regression is of no concern" is
>> also not a proper way of looking at it, I would say: Even if such
>> a situation would happen extremely rarely, if it can happen at all,
>> it would still be a security issue.
> 
> What I'm trying to get at is -- exactly what situation?  What actually
> constitutes a problematic interrupt latency / interrupt processing
> workload, how many vcpus must be sleeping on the same pcpu to actually
> risk triggering that latency / workload, and how feasible is it that
> such a situation would arise in a reasonable scenario?
> 
> If 200us is too long, and it only takes 3 sleeping vcpus to get there,
> then yes, there is a genuine problem we need to try to address before we
> turn it on by default.  If we say that up to 500us is tolerable, and it
> takes 100 sleeping vcpus to reach that latency, then this is something I
> don't really think we need to worry about.
> 
> "I think something bad may happen" is a really difficult to work with.

I understand that, but coming up with proper numbers here isn't
easy. Fact is - it cannot be excluded that on a system with
hundreds of pCPU-s and thousands or vCPU-s, that all vCPU-s
would at some point pile up on one pCPU's list.

How many would be tolerable on a single list depends upon host
characteristics, so a fixed number won't do anyway. Hence I
think the better approach, instead of improving lookup, is to
distribute vCPU-s evenly across lists. Which in turn would likely
require those lists to no longer be tied to pCPU-s, an aspect I
had already suggested during review. As soon as distribution
would be reasonably even, the security concern would vanish:
Someone placing more vCPU-s on a host than that host can
handle is responsible for the consequences. Quite contrary to
someone placing more vCPU-s on a host than a single pCPU can
reasonably handle in an interrupt handler.

Jan

_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: Ideas Re: [PATCH v14 1/2] vmx: VT-d posted-interrupt core logic handling
  2016-03-08 17:26                   ` Jan Beulich
@ 2016-03-08 18:38                     ` George Dunlap
  2016-03-09  5:06                       ` Wu, Feng
  2016-03-09 13:39                       ` Jan Beulich
  0 siblings, 2 replies; 53+ messages in thread
From: George Dunlap @ 2016-03-08 18:38 UTC (permalink / raw)
  To: Jan Beulich, George Dunlap, Feng Wu
  Cc: Kevin Tian, Keir Fraser, Andrew Cooper, Dario Faggioli, xen-devel

On 08/03/16 17:26, Jan Beulich wrote:
>>>> On 08.03.16 at 18:05, <george.dunlap@citrix.com> wrote:
>> On 08/03/16 15:42, Jan Beulich wrote:
>>>>>> On 08.03.16 at 15:42, <George.Dunlap@eu.citrix.com> wrote:
>>>> On Tue, Mar 8, 2016 at 1:10 PM, Wu, Feng <feng.wu@intel.com> wrote:
>>>>>> -----Original Message-----
>>>>>> From: George Dunlap [mailto:george.dunlap@citrix.com]
>>>>>>
>>>>>> 2. Try to test engineered situations where we expect this to be a
>>>>>> problem, to see how big of a problem it is (proving the theory to be
>>>>>> accurate or inaccurate in this case)
>>>>>
>>>>> Maybe we can run a SMP guest with all the vcpus pinned to a dedicated
>>>>> pCPU, we can run some benchmark in the guest with VT-d PI and without
>>>>> VT-d PI, then see the performance difference between these two sceanrios.
>>>>
>>>> This would give us an idea what the worst-case scenario would be.
>>>
>>> How would a single VM ever give us an idea about the worst
>>> case? Something getting close to worst case is a ton of single
>>> vCPU guests all temporarily pinned to one and the same pCPU
>>> (could be multi-vCPU ones, but the more vCPU-s the more
>>> artificial this pinning would become) right before they go into
>>> blocked state (i.e. through one of the two callers of
>>> arch_vcpu_block()), the pinning removed while blocked, and
>>> then all getting woken at once.
>>
>> Why would removing the pinning be important?
> 
> It's not important by itself, other than to avoid all vCPU-s then
> waking up on the one pCPU.
> 
>> And I guess it's actually the case that it doesn't need all VMs to
>> actually be *receiving* interrupts; it just requires them to be
>> *capable* of receiving interrupts, for there to be a long chain all
>> blocked on the same physical cpu.
> 
> Yes.
> 
>>>>  But
>>>> pinning all vcpus to a single pcpu isn't really a sensible use case we
>>>> want to support -- if you have to do something stupid to get a
>>>> performance regression, then I as far as I'm concerned it's not a
>>>> problem.
>>>>
>>>> Or to put it a different way: If we pin 10 vcpus to a single pcpu and
>>>> then pound them all with posted interrupts, and there is *no*
>>>> significant performance regression, then that will conclusively prove
>>>> that the theoretical performance regression is of no concern, and we
>>>> can enable PI by default.
>>>
>>> The point isn't the pinning. The point is what pCPU they're on when
>>> going to sleep. And that could involve quite a few more than just
>>> 10 vCPU-s, provided they all sleep long enough.
>>>
>>> And the "theoretical performance regression is of no concern" is
>>> also not a proper way of looking at it, I would say: Even if such
>>> a situation would happen extremely rarely, if it can happen at all,
>>> it would still be a security issue.
>>
>> What I'm trying to get at is -- exactly what situation?  What actually
>> constitutes a problematic interrupt latency / interrupt processing
>> workload, how many vcpus must be sleeping on the same pcpu to actually
>> risk triggering that latency / workload, and how feasible is it that
>> such a situation would arise in a reasonable scenario?
>>
>> If 200us is too long, and it only takes 3 sleeping vcpus to get there,
>> then yes, there is a genuine problem we need to try to address before we
>> turn it on by default.  If we say that up to 500us is tolerable, and it
>> takes 100 sleeping vcpus to reach that latency, then this is something I
>> don't really think we need to worry about.
>>
>> "I think something bad may happen" is a really difficult to work with.
> 
> I understand that, but coming up with proper numbers here isn't
> easy. Fact is - it cannot be excluded that on a system with
> hundreds of pCPU-s and thousands or vCPU-s, that all vCPU-s
> would at some point pile up on one pCPU's list.

So it's already the case that when a vcpu is woken, it is inserted into
the runqueue by priority order, both for credit1 and credit2; and this
is an insertion sort, so the amount of time it takes to do the insert is
expected to be the time it takes to traverse half of the list.  This
isn't an exact analog, because in that case it's the number of
*runnable* vcpus, not the number of *blocked* vcpus; but it demonstrates
the point that 1) we already have code that assumes that walking a list
of vcpus per pcpu is a reasonably bounded thing 2) we have years of no
major performance problems reported to back that assumption up.

I guess the slight difference there is that it's already well-understood
that too many *active* vcpus will overload your system and slow things
down; in the case of the pi wake-ups, the problem is that too many
*inactive* vcpus will overload your system and slow things down.

Still -- I have a hard time constructing in my mind a scenario where
huge numbers of idle vcpus for some reason decide to congregate on a
single pcpu.

Suppose we had 1024 pcpus, and 1023 VMs each with 5 vcpus, of which 1
was spinning at 100% and the other 4 were idle.  I'm not seeing a
situation where any of the schedulers put all (1023*4) idle vcpus on a
single pcpu.

For the credit1 scheduler, I'm basically positive that it can't happen
even once, even by chance.  You'd never be able to accrete more than a
dozen vcpus on that one pcpu before they were stolen away.

For the credit2 scheduler, it *might* be possible that if the busy vcpu
on each VM never changes (which itself is pretty unlikely), *and* the
sum of the "load" for all (1023*4) idle vcpus was less than 1 (i.e.,
idle vcpus took less than 0.02% of the cpu time), then you *might*
possibly after a long time end up at a situation where you had all vcpus
on a single pcpu.  But that "accretion" process would take a very long
time; and as soon as any vcpu had a brief "spike" above the "0.02%", a
whole bunch of them get moved somewhere else.

And in any case, are you really going to have 1023 devices so that you
can hand one to each of those 1023 guests?  Because it's only vcpus of
VMs *which have a device assigned* that end up on the block list.

If I may go "meta" for a moment here -- this is exactly what I'm talking
about with "Something bad may happen" being difficult to work with.
Rather than you spelling out exactly the situation which you think may
happen, (which I could then either accept or refute on its merits) *I*
am now spending a lot of time and effort trying to imagine what
situations you may be talking about and then refuting them myself.

If you have concerns, you need to make those concerns concrete, or at
least set clear criteria for how someone could go about addressing your
concerns.  And yes, it is *your* job, as the person doing the objecting
(and even moreso as the x86 maintainer), to make your concerns explicit
and/or set those criteria, and not Feng's job (or even my job) to try to
guess what it is might make you happy.

> How many would be tolerable on a single list depends upon host
> characteristics, so a fixed number won't do anyway. 

Sure, but if we can run through a list of 100 vcpus in 25us on a typical
server, then we can be pretty certain 100 vcpus will never exceed 500us
on basically any server.

On the other hand, if 50 vcpus takes 500us on whatever server Feng uses
for his tests, then yes, we don't really have enough "slack" to be sure
that we won't run into problems at some point.

But at this point we're just pulling numbers out of the air -- when we
have actual data we can make a better judgement about what might or
might not be acceptable.

> Hence I
> think the better approach, instead of improving lookup, is to
> distribute vCPU-s evenly across lists. Which in turn would likely
> require those lists to no longer be tied to pCPU-s, an aspect I
> had already suggested during review. As soon as distribution
> would be reasonably even, the security concern would vanish:
> Someone placing more vCPU-s on a host than that host can
> handle is responsible for the consequences. Quite contrary to
> someone placing more vCPU-s on a host than a single pCPU can
> reasonably handle in an interrupt handler.

I don't really understand your suggestion.  The PI interrupt is
necessarily tied to a specific pcpu; unless we start having multiple PI
interrupts, we only have as many interrupts as we have pcpus, right?
Are you saying that rather than put vcpus on the list of the pcpu it's
running on, we should set the interrupt to that of an arbitrary pcpu
that happens to have room on its list?

 -George

_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: Ideas Re: [PATCH v14 1/2] vmx: VT-d posted-interrupt core logic handling
  2016-03-08 18:38                     ` George Dunlap
@ 2016-03-09  5:06                       ` Wu, Feng
  2016-03-09 13:39                       ` Jan Beulich
  1 sibling, 0 replies; 53+ messages in thread
From: Wu, Feng @ 2016-03-09  5:06 UTC (permalink / raw)
  To: George Dunlap, Jan Beulich, George Dunlap
  Cc: Tian, Kevin, Keir Fraser, Andrew Cooper, Dario Faggioli,
	xen-devel, Wu, Feng



> -----Original Message-----
> From: George Dunlap [mailto:george.dunlap@citrix.com]
> Sent: Wednesday, March 9, 2016 2:39 AM
> To: Jan Beulich <JBeulich@suse.com>; George Dunlap
> <George.Dunlap@eu.citrix.com>; Wu, Feng <feng.wu@intel.com>
> Cc: Andrew Cooper <andrew.cooper3@citrix.com>; Dario Faggioli
> <dario.faggioli@citrix.com>; Tian, Kevin <kevin.tian@intel.com>; xen-
> devel@lists.xen.org; Konrad Rzeszutek Wilk <konrad.wilk@oracle.com>; Keir
> Fraser <keir@xen.org>
> Subject: Re: [Xen-devel] Ideas Re: [PATCH v14 1/2] vmx: VT-d posted-interrupt
> core logic handling
> 
> On 08/03/16 17:26, Jan Beulich wrote:
> >>>> On 08.03.16 at 18:05, <george.dunlap@citrix.com> wrote:
> >> On 08/03/16 15:42, Jan Beulich wrote:
> >>>>>> On 08.03.16 at 15:42, <George.Dunlap@eu.citrix.com> wrote:
> >>>> On Tue, Mar 8, 2016 at 1:10 PM, Wu, Feng <feng.wu@intel.com> wrote:
> >>>>>> -----Original Message-----
> >>>>>> From: George Dunlap [mailto:george.dunlap@citrix.com]
> >>>>>>
> >>>>>> 2. Try to test engineered situations where we expect this to be a
> >>>>>> problem, to see how big of a problem it is (proving the theory to be
> >>>>>> accurate or inaccurate in this case)
> >>>>>
> >>>>> Maybe we can run a SMP guest with all the vcpus pinned to a dedicated
> >>>>> pCPU, we can run some benchmark in the guest with VT-d PI and without
> >>>>> VT-d PI, then see the performance difference between these two
> sceanrios.
> >>>>
> >>>> This would give us an idea what the worst-case scenario would be.
> >>>
> >>> How would a single VM ever give us an idea about the worst
> >>> case? Something getting close to worst case is a ton of single
> >>> vCPU guests all temporarily pinned to one and the same pCPU
> >>> (could be multi-vCPU ones, but the more vCPU-s the more
> >>> artificial this pinning would become) right before they go into
> >>> blocked state (i.e. through one of the two callers of
> >>> arch_vcpu_block()), the pinning removed while blocked, and
> >>> then all getting woken at once.
> >>
> >> Why would removing the pinning be important?
> >
> > It's not important by itself, other than to avoid all vCPU-s then
> > waking up on the one pCPU.
> >
> >> And I guess it's actually the case that it doesn't need all VMs to
> >> actually be *receiving* interrupts; it just requires them to be
> >> *capable* of receiving interrupts, for there to be a long chain all
> >> blocked on the same physical cpu.
> >
> > Yes.
> >
> >>>>  But
> >>>> pinning all vcpus to a single pcpu isn't really a sensible use case we
> >>>> want to support -- if you have to do something stupid to get a
> >>>> performance regression, then I as far as I'm concerned it's not a
> >>>> problem.
> >>>>
> >>>> Or to put it a different way: If we pin 10 vcpus to a single pcpu and
> >>>> then pound them all with posted interrupts, and there is *no*
> >>>> significant performance regression, then that will conclusively prove
> >>>> that the theoretical performance regression is of no concern, and we
> >>>> can enable PI by default.
> >>>
> >>> The point isn't the pinning. The point is what pCPU they're on when
> >>> going to sleep. And that could involve quite a few more than just
> >>> 10 vCPU-s, provided they all sleep long enough.
> >>>
> >>> And the "theoretical performance regression is of no concern" is
> >>> also not a proper way of looking at it, I would say: Even if such
> >>> a situation would happen extremely rarely, if it can happen at all,
> >>> it would still be a security issue.
> >>
> >> What I'm trying to get at is -- exactly what situation?  What actually
> >> constitutes a problematic interrupt latency / interrupt processing
> >> workload, how many vcpus must be sleeping on the same pcpu to actually
> >> risk triggering that latency / workload, and how feasible is it that
> >> such a situation would arise in a reasonable scenario?
> >>
> >> If 200us is too long, and it only takes 3 sleeping vcpus to get there,
> >> then yes, there is a genuine problem we need to try to address before we
> >> turn it on by default.  If we say that up to 500us is tolerable, and it
> >> takes 100 sleeping vcpus to reach that latency, then this is something I
> >> don't really think we need to worry about.
> >>
> >> "I think something bad may happen" is a really difficult to work with.
> >
> > I understand that, but coming up with proper numbers here isn't
> > easy. Fact is - it cannot be excluded that on a system with
> > hundreds of pCPU-s and thousands or vCPU-s, that all vCPU-s
> > would at some point pile up on one pCPU's list.
> 
> So it's already the case that when a vcpu is woken, it is inserted into
> the runqueue by priority order, both for credit1 and credit2; and this
> is an insertion sort, so the amount of time it takes to do the insert is
> expected to be the time it takes to traverse half of the list.  This
> isn't an exact analog, because in that case it's the number of
> *runnable* vcpus, not the number of *blocked* vcpus; but it demonstrates
> the point that 1) we already have code that assumes that walking a list
> of vcpus per pcpu is a reasonably bounded thing 2) we have years of no
> major performance problems reported to back that assumption up.
> 
> I guess the slight difference there is that it's already well-understood
> that too many *active* vcpus will overload your system and slow things
> down; in the case of the pi wake-ups, the problem is that too many
> *inactive* vcpus will overload your system and slow things down.
> 
> Still -- I have a hard time constructing in my mind a scenario where
> huge numbers of idle vcpus for some reason decide to congregate on a
> single pcpu.
> 
> Suppose we had 1024 pcpus, and 1023 VMs each with 5 vcpus, of which 1
> was spinning at 100% and the other 4 were idle.  I'm not seeing a
> situation where any of the schedulers put all (1023*4) idle vcpus on a
> single pcpu.
> 
> For the credit1 scheduler, I'm basically positive that it can't happen
> even once, even by chance.  You'd never be able to accrete more than a
> dozen vcpus on that one pcpu before they were stolen away.
> 
> For the credit2 scheduler, it *might* be possible that if the busy vcpu
> on each VM never changes (which itself is pretty unlikely), *and* the
> sum of the "load" for all (1023*4) idle vcpus was less than 1 (i.e.,
> idle vcpus took less than 0.02% of the cpu time), then you *might*
> possibly after a long time end up at a situation where you had all vcpus
> on a single pcpu.  But that "accretion" process would take a very long
> time; and as soon as any vcpu had a brief "spike" above the "0.02%", a
> whole bunch of them get moved somewhere else.
> 
> And in any case, are you really going to have 1023 devices so that you
> can hand one to each of those 1023 guests?  Because it's only vcpus of
> VMs *which have a device assigned* that end up on the block list.
> 
> If I may go "meta" for a moment here -- this is exactly what I'm talking
> about with "Something bad may happen" being difficult to work with.
> Rather than you spelling out exactly the situation which you think may
> happen, (which I could then either accept or refute on its merits) *I*
> am now spending a lot of time and effort trying to imagine what
> situations you may be talking about and then refuting them myself.
> 
> If you have concerns, you need to make those concerns concrete, or at
> least set clear criteria for how someone could go about addressing your
> concerns.  And yes, it is *your* job, as the person doing the objecting
> (and even moreso as the x86 maintainer), to make your concerns explicit
> and/or set those criteria, and not Feng's job (or even my job) to try to
> guess what it is might make you happy.
> 
> > How many would be tolerable on a single list depends upon host
> > characteristics, so a fixed number won't do anyway.
> 
> Sure, but if we can run through a list of 100 vcpus in 25us on a typical
> server, then we can be pretty certain 100 vcpus will never exceed 500us
> on basically any server.
> 
> On the other hand, if 50 vcpus takes 500us on whatever server Feng uses
> for his tests, then yes, we don't really have enough "slack" to be sure
> that we won't run to problems at some point.
> 
> But at this point we're just pulling numbers out of the air -- when we
> have actual data we can make a better judgement about what might or
> might not be acceptable.
> 
> > Hence I
> > think the better approach, instead of improving lookup, is to
> > distribute vCPU-s evenly across lists. Which in turn would likely
> > require those lists to no longer be tied to pCPU-s, an aspect I
> > had already suggested during review. As soon as distribution
> > would be reasonably even, the security concern would vanish:
> > Someone placing more vCPU-s on a host than that host can
> > handle is responsible for the consequences. Quite contrary to
> > someone placing more vCPU-s on a host than a single pCPU can
> > reasonably handle in an interrupt handler.
> 
> I don't really understand your suggestion.  The PI interrupt is
> necessarily tied to a specific pcpu; unless we start having multiple PI
> interrupts, we only have as many interrupts as we have pcpus, right?
> Are you saying that rather than put vcpus on the list of the pcpu it's
> running on, we should set the interrupt to that of an arbitrary pcpu
> that happens to have room on its list?

I don't think that is a good idea, as George mentioned above, the PI
wakeup notification events (PI interrupts) are bound to a specific
pCPU, that means the 'NDST' filed in the vCPU's PI descriptor is the
specific pCPU, so when the PI interrupts happen, we can find the
right blocking list. If we put the vCPU to another pCPU's (other than
the one indicated by 'NDST' field) blocking list, how should we find
the vCPU to wake up when PI interrupts come in?

Thanks,
Feng

> 
>  -George

_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: Ideas Re: [PATCH v14 1/2] vmx: VT-d posted-interrupt core logic handling
  2016-03-08 17:05                 ` George Dunlap
  2016-03-08 17:26                   ` Jan Beulich
@ 2016-03-09  5:22                   ` Wu, Feng
  2016-03-09 11:25                     ` George Dunlap
  1 sibling, 1 reply; 53+ messages in thread
From: Wu, Feng @ 2016-03-09  5:22 UTC (permalink / raw)
  To: George Dunlap, Jan Beulich, George Dunlap
  Cc: Tian, Kevin, Keir Fraser, Andrew Cooper, Dario Faggioli,
	xen-devel, Wu, Feng



> -----Original Message-----
> From: George Dunlap [mailto:george.dunlap@citrix.com]
> Sent: Wednesday, March 9, 2016 1:06 AM
> To: Jan Beulich <JBeulich@suse.com>; George Dunlap
> <George.Dunlap@eu.citrix.com>; Wu, Feng <feng.wu@intel.com>
> Cc: Andrew Cooper <andrew.cooper3@citrix.com>; Dario Faggioli
> <dario.faggioli@citrix.com>; Tian, Kevin <kevin.tian@intel.com>; xen-
> devel@lists.xen.org; Konrad Rzeszutek Wilk <konrad.wilk@oracle.com>; Keir
> Fraser <keir@xen.org>
> Subject: Re: [Xen-devel] Ideas Re: [PATCH v14 1/2] vmx: VT-d posted-interrupt
> core logic handling
> 
> On 08/03/16 15:42, Jan Beulich wrote:
> >>>> On 08.03.16 at 15:42, <George.Dunlap@eu.citrix.com> wrote:
> >> On Tue, Mar 8, 2016 at 1:10 PM, Wu, Feng <feng.wu@intel.com> wrote:
> >>>> -----Original Message-----
> >>>> From: George Dunlap [mailto:george.dunlap@citrix.com]
> >> [snip]
> >>>> It seems like there are a couple of ways we could approach this:
> >>>>
> >>>> 1. Try to optimize the reverse look-up code so that it's not a linear
> >>>> linked list (getting rid of the theoretical fear)
> >>>
> >>> Good point.
> >>>
> >>>>
> >>>> 2. Try to test engineered situations where we expect this to be a
> >>>> problem, to see how big of a problem it is (proving the theory to be
> >>>> accurate or inaccurate in this case)
> >>>
> >>> Maybe we can run a SMP guest with all the vcpus pinned to a dedicated
> >>> pCPU, we can run some benchmark in the guest with VT-d PI and without
> >>> VT-d PI, then see the performance difference between these two sceanrios.
> >>
> >> This would give us an idea what the worst-case scenario would be.
> >
> > How would a single VM ever give us an idea about the worst
> > case? Something getting close to worst case is a ton of single
> > vCPU guests all temporarily pinned to one and the same pCPU
> > (could be multi-vCPU ones, but the more vCPU-s the more
> > artificial this pinning would become) right before they go into
> > blocked state (i.e. through one of the two callers of
> > arch_vcpu_block()), the pinning removed while blocked, and
> > then all getting woken at once.
> 
> Why would removing the pinning be important?
> 
> And I guess it's actually the case that it doesn't need all VMs to
> actually be *receiving* interrupts; it just requires them to be
> *capable* of receiving interrupts, for there to be a long chain all
> blocked on the same physical cpu.
> 
> >
> >>  But
> >> pinning all vcpus to a single pcpu isn't really a sensible use case we
> >> want to support -- if you have to do something stupid to get a
> >> performance regression, then I as far as I'm concerned it's not a
> >> problem.
> >>
> >> Or to put it a different way: If we pin 10 vcpus to a single pcpu and
> >> then pound them all with posted interrupts, and there is *no*
> >> significant performance regression, then that will conclusively prove
> >> that the theoretical performance regression is of no concern, and we
> >> can enable PI by default.
> >
> > The point isn't the pinning. The point is what pCPU they're on when
> > going to sleep. And that could involve quite a few more than just
> > 10 vCPU-s, provided they all sleep long enough.
> >
> > And the "theoretical performance regression is of no concern" is
> > also not a proper way of looking at it, I would say: Even if such
> > a situation would happen extremely rarely, if it can happen at all,
> > it would still be a security issue.
> 
> What I'm trying to get at is -- exactly what situation?  What actually
> constitutes a problematic interrupt latency / interrupt processing
> workload, how many vcpus must be sleeping on the same pcpu to actually
> risk triggering that latency / workload, and how feasible is it that
> such a situation would arise in a reasonable scenario?
> 
> If 200us is too long, and it only takes 3 sleeping vcpus to get there,
> then yes, there is a genuine problem we need to try to address before we
> turn it on by default.  If we say that up to 500us is tolerable, and it
> takes 100 sleeping vcpus to reach that latency, then this is something I
> don't really think we need to worry about.
> 
> "I think something bad may happen" is a really difficult to work with.
> "I want to make sure that even a high number of blocked cpus won't cause
> the interrupt latency to exceed 500us; and I want it to be basically
> impossible for the interrupt latency to exceed 5ms under any
> circumstances" is a concrete target someone can either demonstrate that
> they meet, or aim for when trying to improve the situation.
> 
> Feng: It should be pretty easy for you to:

George, thanks a lot for you to pointing the possible way to move forward.

> * Implement a modified version of Xen where
>  - *All* vcpus get put on the waitqueue

So this means, all the vcpus are blocked, and hence waiting in the
blocking list, right?

>  - Measure how long it took to run the loop in pi_wakeup_interrupt
> * Have one VM receiving posted interrupts on a regular basis.
> * Slowly increase the number of vcpus blocked on a single cpu (e.g., by
> creating more guests), stopping when you either reach 500us or 500
> vcpus. :-)

This may depends on the environment, I was using a 10G NIC to do the
test, if we increase the number of guests, I need more NICs to get assigned
to the guests, I will see if I can get them.

Thanks,
Feng

> 
> To report the measurements, you could either create a Xen trace record
> and use xentrace_format or xenalyze to plot the results; or you could
> create some software performance counters for different "buckets" --
> less than 100us, 100-200us, 200-300us, 300-400us, 400-500us, and more
> than 500us.
> 
> Or you could printk the min / average / max every 5000 interrupts or so. :-)
> 
> To test, it seems like using a network benchmark with short packet
> lengths should be able to trigger large numbers of interrupts; and it
> also can let you know if / when there's a performance impact of adding
> more vcpus.
> 
> Or alternately, you could try to come up with a quicker reverse-lookup
> algorithm. :-)
> 
>  -George


_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: Ideas Re: [PATCH v14 1/2] vmx: VT-d posted-interrupt core logic handling
  2016-03-09  5:22                   ` Ideas Re: [PATCH v14 1/2] " Wu, Feng
@ 2016-03-09 11:25                     ` George Dunlap
  2016-03-09 12:06                       ` Wu, Feng
  0 siblings, 1 reply; 53+ messages in thread
From: George Dunlap @ 2016-03-09 11:25 UTC (permalink / raw)
  To: Wu, Feng, Jan Beulich, George Dunlap
  Cc: Tian, Kevin, Keir Fraser, Andrew Cooper, Dario Faggioli, xen-devel

On 09/03/16 05:22, Wu, Feng wrote:
> 
> 
>> -----Original Message-----
>> From: George Dunlap [mailto:george.dunlap@citrix.com]
>> Sent: Wednesday, March 9, 2016 1:06 AM
>> To: Jan Beulich <JBeulich@suse.com>; George Dunlap
>> <George.Dunlap@eu.citrix.com>; Wu, Feng <feng.wu@intel.com>
>> Cc: Andrew Cooper <andrew.cooper3@citrix.com>; Dario Faggioli
>> <dario.faggioli@citrix.com>; Tian, Kevin <kevin.tian@intel.com>; xen-
>> devel@lists.xen.org; Konrad Rzeszutek Wilk <konrad.wilk@oracle.com>; Keir
>> Fraser <keir@xen.org>
>> Subject: Re: [Xen-devel] Ideas Re: [PATCH v14 1/2] vmx: VT-d posted-interrupt
>> core logic handling
>>
>> On 08/03/16 15:42, Jan Beulich wrote:
>>>>>> On 08.03.16 at 15:42, <George.Dunlap@eu.citrix.com> wrote:
>>>> On Tue, Mar 8, 2016 at 1:10 PM, Wu, Feng <feng.wu@intel.com> wrote:
>>>>>> -----Original Message-----
>>>>>> From: George Dunlap [mailto:george.dunlap@citrix.com]
>>>> [snip]
>>>>>> It seems like there are a couple of ways we could approach this:
>>>>>>
>>>>>> 1. Try to optimize the reverse look-up code so that it's not a linear
>>>>>> linked list (getting rid of the theoretical fear)
>>>>>
>>>>> Good point.
>>>>>
>>>>>>
>>>>>> 2. Try to test engineered situations where we expect this to be a
>>>>>> problem, to see how big of a problem it is (proving the theory to be
>>>>>> accurate or inaccurate in this case)
>>>>>
>>>>> Maybe we can run a SMP guest with all the vcpus pinned to a dedicated
>>>>> pCPU, we can run some benchmark in the guest with VT-d PI and without
>>>>> VT-d PI, then see the performance difference between these two sceanrios.
>>>>
>>>> This would give us an idea what the worst-case scenario would be.
>>>
>>> How would a single VM ever give us an idea about the worst
>>> case? Something getting close to worst case is a ton of single
>>> vCPU guests all temporarily pinned to one and the same pCPU
>>> (could be multi-vCPU ones, but the more vCPU-s the more
>>> artificial this pinning would become) right before they go into
>>> blocked state (i.e. through one of the two callers of
>>> arch_vcpu_block()), the pinning removed while blocked, and
>>> then all getting woken at once.
>>
>> Why would removing the pinning be important?
>>
>> And I guess it's actually the case that it doesn't need all VMs to
>> actually be *receiving* interrupts; it just requires them to be
>> *capable* of receiving interrupts, for there to be a long chain all
>> blocked on the same physical cpu.
>>
>>>
>>>>  But
>>>> pinning all vcpus to a single pcpu isn't really a sensible use case we
>>>> want to support -- if you have to do something stupid to get a
>>>> performance regression, then I as far as I'm concerned it's not a
>>>> problem.
>>>>
>>>> Or to put it a different way: If we pin 10 vcpus to a single pcpu and
>>>> then pound them all with posted interrupts, and there is *no*
>>>> significant performance regression, then that will conclusively prove
>>>> that the theoretical performance regression is of no concern, and we
>>>> can enable PI by default.
>>>
>>> The point isn't the pinning. The point is what pCPU they're on when
>>> going to sleep. And that could involve quite a few more than just
>>> 10 vCPU-s, provided they all sleep long enough.
>>>
>>> And the "theoretical performance regression is of no concern" is
>>> also not a proper way of looking at it, I would say: Even if such
>>> a situation would happen extremely rarely, if it can happen at all,
>>> it would still be a security issue.
>>
>> What I'm trying to get at is -- exactly what situation?  What actually
>> constitutes a problematic interrupt latency / interrupt processing
>> workload, how many vcpus must be sleeping on the same pcpu to actually
>> risk triggering that latency / workload, and how feasible is it that
>> such a situation would arise in a reasonable scenario?
>>
>> If 200us is too long, and it only takes 3 sleeping vcpus to get there,
>> then yes, there is a genuine problem we need to try to address before we
>> turn it on by default.  If we say that up to 500us is tolerable, and it
>> takes 100 sleeping vcpus to reach that latency, then this is something I
>> don't really think we need to worry about.
>>
>> "I think something bad may happen" is a really difficult to work with.
>> "I want to make sure that even a high number of blocked cpus won't cause
>> the interrupt latency to exceed 500us; and I want it to be basically
>> impossible for the interrupt latency to exceed 5ms under any
>> circumstances" is a concrete target someone can either demonstrate that
>> they meet, or aim for when trying to improve the situation.
>>
>> Feng: It should be pretty easy for you to:
> 
> George, thanks a lot for you to pointing the possible way to move forward.
> 
>> * Implement a modified version of Xen where
>>  - *All* vcpus get put on the waitqueue
> 
> So this means, all the vcpus are blocked, and hence waiting in the
> blocking list, right?

No.

For testing purposes, we need a lot of vcpus on the list, but we only
need one vcpu to actually be woken up to see low long it takes to
traverse the list.

At the moment, a vcpu will only be put on the list if it has the
arch_block callback defined; and it will have the arch_block callback
defined only if the domain it's a part of has a device assigned to it.
But it would be easy enough to make it so that *all* VMs have the
arch_block callback defined; then all vcpus would end up on the
pi_blocked list when they're blocked, even if they don't have a device
assigned.

That way you could have a really long pi_blocked list while only needing
a single device to pass through to the guest.

>>  - Measure how long it took to run the loop in pi_wakeup_interrupt
>> * Have one VM receiving posted interrupts on a regular basis.
>> * Slowly increase the number of vcpus blocked on a single cpu (e.g., by
>> creating more guests), stopping when you either reach 500us or 500
>> vcpus. :-)
> 
> This may depends on the environment, I was using a 10G NIC to do the
> test, if we increase the number of guests, I need more NICs to get assigned
> to the guests, I will see if I can get them.

...which is why I suggested setting the arch_block() callback for all
domains, even those which don't have devices assigned, so that you could
get away with a single passed-through device. :-)

 -George


_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: Ideas Re: [PATCH v14 1/2] vmx: VT-d posted-interrupt core logic handling
  2016-03-09 11:25                     ` George Dunlap
@ 2016-03-09 12:06                       ` Wu, Feng
  0 siblings, 0 replies; 53+ messages in thread
From: Wu, Feng @ 2016-03-09 12:06 UTC (permalink / raw)
  To: George Dunlap, Jan Beulich, George Dunlap
  Cc: Tian, Kevin, Keir Fraser, Andrew Cooper, Dario Faggioli,
	xen-devel, Wu, Feng



> -----Original Message-----
> From: George Dunlap [mailto:george.dunlap@citrix.com]
> Sent: Wednesday, March 9, 2016 7:25 PM
> To: Wu, Feng <feng.wu@intel.com>; Jan Beulich <JBeulich@suse.com>; George
> Dunlap <George.Dunlap@eu.citrix.com>
> Cc: Andrew Cooper <andrew.cooper3@citrix.com>; Dario Faggioli
> <dario.faggioli@citrix.com>; Tian, Kevin <kevin.tian@intel.com>; xen-
> devel@lists.xen.org; Konrad Rzeszutek Wilk <konrad.wilk@oracle.com>; Keir
> Fraser <keir@xen.org>
> Subject: Re: [Xen-devel] Ideas Re: [PATCH v14 1/2] vmx: VT-d posted-interrupt
> core logic handling
> 
> On 09/03/16 05:22, Wu, Feng wrote:
> >
> >
> >> -----Original Message-----
> >> From: George Dunlap [mailto:george.dunlap@citrix.com]
> >> Sent: Wednesday, March 9, 2016 1:06 AM
> >> To: Jan Beulich <JBeulich@suse.com>; George Dunlap
> >> <George.Dunlap@eu.citrix.com>; Wu, Feng <feng.wu@intel.com>
> >> Cc: Andrew Cooper <andrew.cooper3@citrix.com>; Dario Faggioli
> >> <dario.faggioli@citrix.com>; Tian, Kevin <kevin.tian@intel.com>; xen-
> >> devel@lists.xen.org; Konrad Rzeszutek Wilk <konrad.wilk@oracle.com>; Keir
> >> Fraser <keir@xen.org>
> >> Subject: Re: [Xen-devel] Ideas Re: [PATCH v14 1/2] vmx: VT-d posted-
> interrupt
> >> core logic handling
> >>
> >> On 08/03/16 15:42, Jan Beulich wrote:
> >>>>>> On 08.03.16 at 15:42, <George.Dunlap@eu.citrix.com> wrote:
> >>>> On Tue, Mar 8, 2016 at 1:10 PM, Wu, Feng <feng.wu@intel.com> wrote:
> >>>>>> -----Original Message-----
> >>>>>> From: George Dunlap [mailto:george.dunlap@citrix.com]
> >>>> [snip]
> >>>>>> It seems like there are a couple of ways we could approach this:
> >>>>>>
> >>>>>> 1. Try to optimize the reverse look-up code so that it's not a linear
> >>>>>> linked list (getting rid of the theoretical fear)
> >>>>>
> >>>>> Good point.
> >>>>>
> >>>>>>
> >>>>>> 2. Try to test engineered situations where we expect this to be a
> >>>>>> problem, to see how big of a problem it is (proving the theory to be
> >>>>>> accurate or inaccurate in this case)
> >>>>>
> >>>>> Maybe we can run a SMP guest with all the vcpus pinned to a dedicated
> >>>>> pCPU, we can run some benchmark in the guest with VT-d PI and without
> >>>>> VT-d PI, then see the performance difference between these two
> sceanrios.
> >>>>
> >>>> This would give us an idea what the worst-case scenario would be.
> >>>
> >>> How would a single VM ever give us an idea about the worst
> >>> case? Something getting close to worst case is a ton of single
> >>> vCPU guests all temporarily pinned to one and the same pCPU
> >>> (could be multi-vCPU ones, but the more vCPU-s the more
> >>> artificial this pinning would become) right before they go into
> >>> blocked state (i.e. through one of the two callers of
> >>> arch_vcpu_block()), the pinning removed while blocked, and
> >>> then all getting woken at once.
> >>
> >> Why would removing the pinning be important?
> >>
> >> And I guess it's actually the case that it doesn't need all VMs to
> >> actually be *receiving* interrupts; it just requires them to be
> >> *capable* of receiving interrupts, for there to be a long chain all
> >> blocked on the same physical cpu.
> >>
> >>>
> >>>>  But
> >>>> pinning all vcpus to a single pcpu isn't really a sensible use case we
> >>>> want to support -- if you have to do something stupid to get a
> >>>> performance regression, then I as far as I'm concerned it's not a
> >>>> problem.
> >>>>
> >>>> Or to put it a different way: If we pin 10 vcpus to a single pcpu and
> >>>> then pound them all with posted interrupts, and there is *no*
> >>>> significant performance regression, then that will conclusively prove
> >>>> that the theoretical performance regression is of no concern, and we
> >>>> can enable PI by default.
> >>>
> >>> The point isn't the pinning. The point is what pCPU they're on when
> >>> going to sleep. And that could involve quite a few more than just
> >>> 10 vCPU-s, provided they all sleep long enough.
> >>>
> >>> And the "theoretical performance regression is of no concern" is
> >>> also not a proper way of looking at it, I would say: Even if such
> >>> a situation would happen extremely rarely, if it can happen at all,
> >>> it would still be a security issue.
> >>
> >> What I'm trying to get at is -- exactly what situation?  What actually
> >> constitutes a problematic interrupt latency / interrupt processing
> >> workload, how many vcpus must be sleeping on the same pcpu to actually
> >> risk triggering that latency / workload, and how feasible is it that
> >> such a situation would arise in a reasonable scenario?
> >>
> >> If 200us is too long, and it only takes 3 sleeping vcpus to get there,
> >> then yes, there is a genuine problem we need to try to address before we
> >> turn it on by default.  If we say that up to 500us is tolerable, and it
> >> takes 100 sleeping vcpus to reach that latency, then this is something I
> >> don't really think we need to worry about.
> >>
> >> "I think something bad may happen" is a really difficult to work with.
> >> "I want to make sure that even a high number of blocked cpus won't cause
> >> the interrupt latency to exceed 500us; and I want it to be basically
> >> impossible for the interrupt latency to exceed 5ms under any
> >> circumstances" is a concrete target someone can either demonstrate that
> >> they meet, or aim for when trying to improve the situation.
> >>
> >> Feng: It should be pretty easy for you to:
> >
> > George, thanks a lot for you to pointing the possible way to move forward.
> >
> >> * Implement a modified version of Xen where
> >>  - *All* vcpus get put on the waitqueue
> >
> > So this means, all the vcpus are blocked, and hence waiting in the
> > blocking list, right?
> 
> No.
> 
> For testing purposes, we need a lot of vcpus on the list, but we only
> need one vcpu to actually be woken up to see low long it takes to
> traverse the list.
> 
> At the moment, a vcpu will only be put on the list if it has the
> arch_block callback defined; and it will have the arch_block callback
> defined only if the domain it's a part of has a device assigned to it.
> But it would be easy enough to make it so that *all* VMs have the
> arch_block callback defined; then all vcpus would end up on the
> pi_blocked list when they're blocked, even if they don't have a device
> assigned.
> 
> That way you could have a really long pi_blocked list while only needing
> a single device to pass through to the guest.
> 
> >>  - Measure how long it took to run the loop in pi_wakeup_interrupt
> >> * Have one VM receiving posted interrupts on a regular basis.
> >> * Slowly increase the number of vcpus blocked on a single cpu (e.g., by
> >> creating more guests), stopping when you either reach 500us or 500
> >> vcpus. :-)
> >
> > This may depends on the environment, I was using a 10G NIC to do the
> > test, if we increase the number of guests, I need more NICs to get assigned
> > to the guests, I will see if I can get them.
> 
> ...which is why I suggested setting the arch_block() callback for all
> domains, even those which don't have devices assigned, so that you could
> get away with a single passed-through device. :-)

Oh, I've got your point, thanks a lot for the suggestion! Will try to get the
data soon. :)

Thanks,
Feng

> 
>  -George


_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: Ideas Re: [PATCH v14 1/2] vmx: VT-d posted-interrupt core logic handling
  2016-03-08 18:38                     ` George Dunlap
  2016-03-09  5:06                       ` Wu, Feng
@ 2016-03-09 13:39                       ` Jan Beulich
  2016-03-09 16:01                         ` George Dunlap
  2016-03-09 16:23                         ` On setting clear criteria for declaring a feature acceptable (was "vmx: VT-d posted-interrupt core logic handling") George Dunlap
  1 sibling, 2 replies; 53+ messages in thread
From: Jan Beulich @ 2016-03-09 13:39 UTC (permalink / raw)
  To: George Dunlap
  Cc: Kevin Tian, Feng Wu, George Dunlap, Andrew Cooper,
	Dario Faggioli, xen-devel, Keir Fraser

>>> On 08.03.16 at 19:38, <george.dunlap@citrix.com> wrote:
> Still -- I have a hard time constructing in my mind a scenario where
> huge numbers of idle vcpus for some reason decide to congregate on a
> single pcpu.
> 
> Suppose we had 1024 pcpus, and 1023 VMs each with 5 vcpus, of which 1
> was spinning at 100% and the other 4 were idle.  I'm not seeing a
> situation where any of the schedulers put all (1023*4) idle vcpus on a
> single pcpu.

As per my understanding idle vCPU-s don't get migrated at all.
And even if they do, their PI association with a pCPU doesn't
change (because that gets established once an for all at the
time the vCPU blocks).

> For the credit1 scheduler, I'm basically positive that it can't happen
> even once, even by chance.  You'd never be able to accrete more than a
> dozen vcpus on that one pcpu before they were stolen away.

Isn't stealing here happing only for runnable vCPU-s?

> And in any case, are you really going to have 1023 devices so that you
> can hand one to each of those 1023 guests?  Because it's only vcpus of
> VMs *which have a device assigned* that end up on the block list.

Who knows what people put in their (huge) systems, or by what
factor the VF/PF ratio will grow in the next few years?

> If I may go "meta" for a moment here -- this is exactly what I'm talking
> about with "Something bad may happen" being difficult to work with.
> Rather than you spelling out exactly the situation which you think may
> happen, (which I could then either accept or refute on its merits) *I*
> am now spending a lot of time and effort trying to imagine what
> situations you may be talking about and then refuting them myself.

I thought I was precise enough (without going into too much detail),
but looks like I wasn't.

1) vCPU1 blocks on pCPU1 (indefinitely for the purpose here)
2) vCPU2 gets migrated to pCPU1 and blocks (indefinitely ...)
...
n) vCPUn gets migrated to pCPU1 and blocks (indefinitely ...)
n+1) a PI wakeup interrupt arrives on pCPU1

In this consideration it doesn't matter whether the vCPU-s are all
from the same or different VMs. The sole requirement is that they
must satisfy the condition(s) to be put on the blocking list.

> If you have concerns, you need to make those concerns concrete, or at
> least set clear criteria for how someone could go about addressing your
> concerns.  And yes, it is *your* job, as the person doing the objecting
> (and even moreso as the x86 maintainer), to make your concerns explicit
> and/or set those criteria, and not Feng's job (or even my job) to try to
> guess what it is might make you happy.

I'm sorry, George, but no, I don't think this is how things should
work. If for a new feature to be enabled by default it is unclear
whether that puts the system at risk, it's the party suggesting the
default enabling to prove there's no such risk. We just can't allow
code in that sets us up for future security issues. If anything
that's what we should have learned from the various disasters in
the past (XSAVE enabling having been the first and foremost,
which by now I count 4 related XSAs for).

>> How many would be tolerable on a single list depends upon host
>> characteristics, so a fixed number won't do anyway. 
> 
> Sure, but if we can run through a list of 100 vcpus in 25us on a typical
> server, then we can be pretty certain 100 vcpus will never exceed 500us
> on basically any server.
> 
> On the other hand, if 50 vcpus takes 500us on whatever server Feng uses
> for his tests, then yes, we don't really have enough "slack" to be sure
> that we won't run into problems at some point.

I agree with such reasoning, except that we need to scale this
up. Unless (see above) there are reasons why the extraordinary
situation of a majority of all vCPU-s piling up in a single pCPU's
list cannot occur (not even theoretically), the counts to work with
are total vCPU counts that we can reasonably expect could be
placed on a huge system. Which is more likely to be thousands
than hundreds.

>> Hence I
>> think the better approach, instead of improving lookup, is to
>> distribute vCPU-s evenly across lists. Which in turn would likely
>> require those lists to no longer be tied to pCPU-s, an aspect I
>> had already suggested during review. As soon as distribution
>> would be reasonably even, the security concern would vanish:
>> Someone placing more vCPU-s on a host than that host can
>> handle is responsible for the consequences. Quite contrary to
>> someone placing more vCPU-s on a host than a single pCPU can
>> reasonably handle in an interrupt handler.
> 
> I don't really understand your suggestion.  The PI interrupt is
> necessarily tied to a specific pcpu; unless we start having multiple PI
> interrupts, we only have as many interrupts as we have pcpus, right?
> Are you saying that rather than put vcpus on the list of the pcpu it's
> running on, we should set the interrupt to that of an arbitrary pcpu
> that happens to have room on its list?

Ah, right, I think that limitation was named before, yet I've
forgotten about it again. But that only slightly alters the
suggestion: To distribute vCPU-s evenly would then require to
change their placement on the pCPU in the course of entering
blocked state.

Jan

_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: Ideas Re: [PATCH v14 1/2] vmx: VT-d posted-interrupt core logic handling
  2016-03-09 13:39                       ` Jan Beulich
@ 2016-03-09 16:01                         ` George Dunlap
  2016-03-09 16:31                           ` Jan Beulich
  2016-03-09 16:23                         ` On setting clear criteria for declaring a feature acceptable (was "vmx: VT-d posted-interrupt core logic handling") George Dunlap
  1 sibling, 1 reply; 53+ messages in thread
From: George Dunlap @ 2016-03-09 16:01 UTC (permalink / raw)
  To: Jan Beulich
  Cc: Kevin Tian, Feng Wu, George Dunlap, Andrew Cooper,
	Dario Faggioli, xen-devel, Keir Fraser

On 09/03/16 13:39, Jan Beulich wrote:
>>>> On 08.03.16 at 19:38, <george.dunlap@citrix.com> wrote:
>> Still -- I have a hard time constructing in my mind a scenario where
>> huge numbers of idle vcpus for some reason decide to congregate on a
>> single pcpu.
>>
>> Suppose we had 1024 pcpus, and 1023 VMs each with 5 vcpus, of which 1
>> was spinning at 100% and the other 4 were idle.  I'm not seeing a
>> situation where any of the schedulers put all (1023*4) idle vcpus on a
>> single pcpu.
> 
> As per my understanding idle vCPU-s don't get migrated at all.
> And even if they do, their PI association with a pCPU doesn't
> change (because that gets established once an for all at the
> time the vCPU blocks).
> 
>> For the credit1 scheduler, I'm basically positive that it can't happen
>> even once, even by chance.  You'd never be able to accrete more than a
>> dozen vcpus on that one pcpu before they were stolen away.
> 
> Isn't stealing here happing only for runnable vCPU-s?
> 
>> And in any case, are you really going to have 1023 devices so that you
>> can hand one to each of those 1023 guests?  Because it's only vcpus of
>> VMs *which have a device assigned* that end up on the block list.
> 
> Who knows what people put in their (huge) systems, or by what
> factor the VF/PF ratio will grow in the next few years?
> 
>> If I may go "meta" for a moment here -- this is exactly what I'm talking
>> about with "Something bad may happen" being difficult to work with.
>> Rather than you spelling out exactly the situation which you think may
>> happen, (which I could then either accept or refute on its merits) *I*
>> am now spending a lot of time and effort trying to imagine what
>> situations you may be talking about and then refuting them myself.
> 
> I thought I was precise enough (without going into too much detail),
> but looks like I wasn't.
> 
> 1) vCPU1 blocks on pCPU1 (indefinitely for the purpose here)
> 2) vCPU2 gets migrated to pCPU1 and blocks (indefinitely ...)
> ...
> n) vCPUn gets migrated to pCPU1 and blocks (indefinitely ...)
> n+1) a PI wakeup interrupt arrives on pCPU1
> 
> In this consideration it doesn't matter whether the vCPU-s are all
> from the same or different VMs. The sole requirement is that they
> must satisfy the condition(s) to be put on the blocking list.

Right -- so here's one of our differing assumptions.  In my experience
there is no such thing as a truly idle vcpu: they always wake up at
least occasionally (usually a few times a second) for some reason or
other.  (Which is why I talked about the load of each idle vcpu being
less than 0.02%.)  So I was assuming that the vcpu would be stolen
during one of the 0.02% time it was running.

But let's suppose that's not the case -- the chances of something like
you're talking about happening are astronomically small.

So for this to work, you have to have a set of "perversely idle" vcpus,
call it set PI, which do a normal amount of work -- enough to get the
attention of the load balancer -- and then mysteriously block, taking
almost no interrupts at all, for an incredibly long amount of time (on
the order of minutes at least), and then wake up.  The chance of having
a large number of these in itself is pretty minimal.

Then we have the problem of how are we going to get the perversely idle
vcpus onto the same pcpu (call it p)?  Well, somehow all the other cpus
have to be busy, which means we have to have almost exactly the right
number of normally working vcpus (call this set W) to keep all the
*other* pcpus busy.

For a member of PI to be moved to p, p itself has to be idle.  Suppose
we start with a random distribution, and it happens that p only has one
member of PI on  it.  So the load balancer moves some more work there.
If it happens to grab a vcpu from the set W, then the whole thing stops
until W is migrated away, because now p isn't idle -- it's got a fairly
busy vcpu on it.  But of course, if p is busy, then now some other pcpu
is now idle, and so *it* will start attracting members of PI, until it
accidentally grabs a member of W, &c &c.

And of course, only one vcpu will be moved to p at a time.  If all the
vcpus in PI block at the same time, most of them will stay just where
they are. So we have further constraints on PI: Not only do they have to
have this unnatural "Run then block completely" pattern; they must block
in a staged fashion, so that they can be moved one-by-one onto p.

And not only that, the load balancer has to *migrate* them in the right
order.  If it grabs a vcpu that won't block until all the other ones
have already blocked, then p will be busy and the other vcpus in PI will
end up on other pcpus.

So for your nightmare scenario to happen, we must have hundreds of vcpus
which exhibit this strange blocking pattern; they must block in a staged
fashion; the load balancer, when choosing work to move onto p, has to
somehow, 100 times in a row (500 times? 1000 times?), select a running
vcpu that is in PI, instead of a running pcpu which is in W; and not
only that, it has to grab the vcpus in PI *in the order in which they
are going to block*, at least 100 (500 / 1000) times.

This is just incredibly far-fetched.  By the time this happens to
someone they will already have been struck by lightning 50 times and won
the billon dollar Powerball jackpot twice; at that point they won't care.

>>> Hence I
>>> think the better approach, instead of improving lookup, is to
>>> distribute vCPU-s evenly across lists. Which in turn would likely
>>> require those lists to no longer be tied to pCPU-s, an aspect I
>>> had already suggested during review. As soon as distribution
>>> would be reasonably even, the security concern would vanish:
>>> Someone placing more vCPU-s on a host than that host can
>>> handle is responsible for the consequences. Quite contrary to
>>> someone placing more vCPU-s on a host than a single pCPU can
>>> reasonably handle in an interrupt handler.
>>
>> I don't really understand your suggestion.  The PI interrupt is
>> necessarily tied to a specific pcpu; unless we start having multiple PI
>> interrupts, we only have as many interrupts as we have pcpus, right?
>> Are you saying that rather than put vcpus on the list of the pcpu it's
>> running on, we should set the interrupt to that of an arbitrary pcpu
>> that happens to have room on its list?
> 
> Ah, right, I think that limitation was named before, yet I've
> forgotten about it again. But that only slightly alters the
> suggestion: To distribute vCPU-s evenly would then require to
> change their placement on the pCPU in the course of entering
> blocked state.

Right -- well having a mechanism to limit the total number of pi-capable
vcpus assigned to a single pcpu would be something we could consider too
-- once we have an idea what kind of number that might be.

 -George



_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* On setting clear criteria for declaring a feature acceptable (was "vmx: VT-d posted-interrupt core logic handling")
  2016-03-09 13:39                       ` Jan Beulich
  2016-03-09 16:01                         ` George Dunlap
@ 2016-03-09 16:23                         ` George Dunlap
  2016-03-09 16:58                           ` On setting clear criteria for declaring a feature acceptable Jan Beulich
                                             ` (2 more replies)
  1 sibling, 3 replies; 53+ messages in thread
From: George Dunlap @ 2016-03-09 16:23 UTC (permalink / raw)
  To: Jan Beulich
  Cc: Lars Kurth, Kevin Tian, Feng Wu, George Dunlap, Andrew Cooper,
	Dario Faggioli, Ian Jackson, xen-devel, David Vrabel

[Changing the subject and CC'ing more people]

On 09/03/16 13:39, Jan Beulich wrote:
>>>> On 08.03.16 at 19:38, <george.dunlap@citrix.com> wrote:
>> If I may go "meta" for a moment here -- this is exactly what I'm talking
>> about with "Something bad may happen" being difficult to work with.
>> Rather than you spelling out exactly the situation which you think may
>> happen, (which I could then either accept or refute on its merits) *I*
>> am now spending a lot of time and effort trying to imagine what
>> situations you may be talking about and then refuting them myself.
[snip]
> 
>> If you have concerns, you need to make those concerns concrete, or at
>> least set clear criteria for how someone could go about addressing your
>> concerns.  And yes, it is *your* job, as the person doing the objecting
>> (and even moreso as the x86 maintainer), to make your concerns explicit
>> and/or set those criteria, and not Feng's job (or even my job) to try to
>> guess what it is might make you happy.
> 
> I'm sorry, George, but no, I don't think this is how things should
> work. If for a new feature to be enabled by default it is unclear
> whether that puts the system at risk, it's the party suggesting the
> default enabling to prove there's no such risk. 

And it's up to the maintainers to clearly define what kind of "proof"
would be sufficient. I have no objection to saying that Feng (or someone
else who cares about the feature) must do some work to demonstrate that
the feature is in fact safe before it's enabled by default; that's
perfectly reasonable. I have already suggested something that would shed
light on the issue and potentially satisfy me. But it's not at all
reasonable to give them the impossible task of trying to guess what will
satisfy you.

I don't know why this is controversial -- this seems obvious to me.
What do other committers / maintainers think?

> We just can't allow
> code in that sets us up for future security issues. If anything
> that's what we should have learned from the various disasters in
> the past (XSAVE enabling having been the first and foremost,
> which by now I count 4 related XSAs for).

I'm not familiar with the XSAVE feature or its related XSAs, but do you
think simply saying "I'm not sure; prove to me that it's safe" would
have actually helped matters?  Would it have prompted the authors of
that code to actually do some sort of testing / analysis that would have
turned up the subsequent security issues?

It seems to me that saying "I'm not sure, prove it to me", without
further guidance about *how* to prove it, would have ended in one of you
two giving up: either Intel not doing any more work on the feature, or
you eventually giving up and letting it go in anyway, with the same
security bugs it had before.

Again, without knowing much about the XSAVE feature or the XSAs, a
couple of responses which might have led to better outcomes:

* "I'd like to see an analysis of the XSAVE code -- what are all the
possible ways in can be loaded and stored?  How can we be sure that
nothing is leaked? See
marc.info/?i=<1371746007-19073-1-git-send-email-george.dunlap@eu.citrix.com>
for an example of the kind of analysis I'm talking about."

* "I'd like to see a framework that tests a lot of the corner cases to
make sure nothing leaks"

Those are both a fair amount of work, but they're also fairly concrete,
and actually move people towards a helpful conclusion.

 -George


_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: Ideas Re: [PATCH v14 1/2] vmx: VT-d posted-interrupt core logic handling
  2016-03-09 16:01                         ` George Dunlap
@ 2016-03-09 16:31                           ` Jan Beulich
  0 siblings, 0 replies; 53+ messages in thread
From: Jan Beulich @ 2016-03-09 16:31 UTC (permalink / raw)
  To: George Dunlap
  Cc: Kevin Tian, Feng Wu, George Dunlap, Andrew Cooper,
	Dario Faggioli, xen-devel, Keir Fraser

>>> On 09.03.16 at 17:01, <george.dunlap@citrix.com> wrote:
> On 09/03/16 13:39, Jan Beulich wrote:
>>>>> On 08.03.16 at 19:38, <george.dunlap@citrix.com> wrote:
>>> If I may go "meta" for a moment here -- this is exactly what I'm talking
>>> about with "Something bad may happen" being difficult to work with.
>>> Rather than you spelling out exactly the situation which you think may
>>> happen, (which I could then either accept or refute on its merits) *I*
>>> am now spending a lot of time and effort trying to imagine what
>>> situations you may be talking about and then refuting them myself.
>> 
>> I thought I was precise enough (without going into too much detail),
>> but looks like I wasn't.
>> 
>> 1) vCPU1 blocks on pCPU1 (indefinitely for the purpose here)
>> 2) vCPU2 gets migrated to pCPU1 and blocks (indefinitely ...)
>> ...
>> n) vCPUn gets migrated to pCPU1 and blocks (indefinitely ...)
>> n+1) a PI wakeup interrupt arrives on pCPU1
>> 
>> In this consideration it doesn't matter whether the vCPU-s are all
>> from the same or different VMs. The sole requirement is that they
>> must satisfy the condition(s) to be put on the blocking list.
> 
> Right -- so here's one of our differing assumptions.  In my experience
> there is no such thing as a truly idle vcpu: they always wake up at
> least occasionally (usually a few times a second) for some reason or
> other.  (Which is why I talked about the load of each idle vcpu being
> less than 0.02%.)  So I was assuming that the vcpu would be stolen
> during one of the 0.02% time it was running.
> 
> But let's suppose that's not the case -- the chances of something like
> you're talking about happening are astronomically small.
> [...]
> This is just incredibly far-fetched.  By the time this happens to
> someone they will already have been struck by lightning 50 times and won
> the billon dollar Powerball jackpot twice; at that point they won't care.

I agree with all of this, and especially this last paragraph was
really fun to read - except for the (implied) conclusion you want
me to draw. Nevertheless many of the XSAs we issue are about
things that may not arise in practice, unless someone
(maliciously?) arranges for it.

> Right -- well having a mechanism to limit the total number of pi-capable
> vcpus assigned to a single pcpu would be something we could consider too
> -- once we have an idea what kind of number that might be.

As said before, I don't think we need any fixed number here. What
we need is a criteria of how much of an imbalance we're willing to
tolerate. With the assumption that the total load the admin places
on a system is reasonable, a reasonably balanced distribution will
also result in reasonable lookup performance. For example we
could require that the number of vCPU-s on any pCPU list is no
more than double or triple the ratio total-vCPU-s / total-pCPU-s
(with that ratio rounded upwards to an integer, and with the
number on list perhaps always allowed to reach some reasonably
small value - say 16 - even if that exceeds said ratio).

Jan


_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: On setting clear criteria for declaring a feature acceptable
  2016-03-09 16:23                         ` On setting clear criteria for declaring a feature acceptable (was "vmx: VT-d posted-interrupt core logic handling") George Dunlap
@ 2016-03-09 16:58                           ` Jan Beulich
  2016-03-09 18:02                           ` On setting clear criteria for declaring a feature acceptable (was "vmx: VT-d posted-interrupt core logic handling") David Vrabel
  2016-03-10  5:09                           ` Tian, Kevin
  2 siblings, 0 replies; 53+ messages in thread
From: Jan Beulich @ 2016-03-09 16:58 UTC (permalink / raw)
  To: George Dunlap
  Cc: Lars Kurth, Kevin Tian, Feng Wu, George Dunlap, Andrew Cooper,
	Dario Faggioli, Ian Jackson, xen-devel, David Vrabel

>>> On 09.03.16 at 17:23, <george.dunlap@citrix.com> wrote:
> [Changing the subject and CC'ing more people]
> 
> On 09/03/16 13:39, Jan Beulich wrote:
>>>>> On 08.03.16 at 19:38, <george.dunlap@citrix.com> wrote:
>>> If I may go "meta" for a moment here -- this is exactly what I'm talking
>>> about with "Something bad may happen" being difficult to work with.
>>> Rather than you spelling out exactly the situation which you think may
>>> happen, (which I could then either accept or refute on its merits) *I*
>>> am now spending a lot of time and effort trying to imagine what
>>> situations you may be talking about and then refuting them myself.
> [snip]
>> 
>>> If you have concerns, you need to make those concerns concrete, or at
>>> least set clear criteria for how someone could go about addressing your
>>> concerns.  And yes, it is *your* job, as the person doing the objecting
>>> (and even moreso as the x86 maintainer), to make your concerns explicit
>>> and/or set those criteria, and not Feng's job (or even my job) to try to
>>> guess what it is might make you happy.
>> 
>> I'm sorry, George, but no, I don't think this is how things should
>> work. If for a new feature to be enabled by default it is unclear
>> whether that puts the system at risk, it's the party suggesting the
>> default enabling to prove there's no such risk. 
> 
> And it's up to the maintainers to clearly define what kind of "proof"
> would be sufficient. I have no objection to saying that Feng (or someone
> else who cares about the feature) must do some work to demonstrate that
> the feature is in fact safe before it's enabled by default; that's
> perfectly reasonable. I have already suggested something that would shed
> light on the issue and potentially satisfy me. But it's not at all
> reasonable to give them the impossible task of trying to guess what will
> satisfy you.
> 
> I don't know why this is controversial -- this seems obvious to me.

And it is not controversial - as said on the original thread, I
was of the opinion that I had clearly explained which specific
case I want to see taken care of (or to be more precise,
avoided). With that ...

> What do other committers / maintainers think?
> 
>> We just can't allow
>> code in that sets us up for future security issues. If anything
>> that's what we should have learned from the various disasters in
>> the past (XSAVE enabling having been the first and foremost,
>> which by now I count 4 related XSAs for).
> 
> I'm not familiar with the XSAVE feature or its related XSAs, but do you
> think simply saying "I'm not sure; prove to me that it's safe" would

.. this is just an unfair simplification of what I've raised as concerns
so far. That said, ...

> have actually helped matters?  Would it have prompted the authors of
> that code to actually do some sort of testing / analysis that would have
> turned up the subsequent security issues?
> 
> It seems to me that saying "I'm not sure, prove it to me", without
> further guidance about *how* to prove it, would have ended in one of you
> two giving up: either Intel not doing any more work on the feature, or
> you eventually giving up and letting it go in anyway, with the same
> security bugs it had before.

... I agree with you on these points. Just that I don't feel guilty
having acted in this way. Yes, I wasn't anywhere close to precise
in how the questionable state could be reached, but I was (and
continue to be) of the opinion that this doesn't matter, so long as
there are no provisions anywhere in the system that preclude
such a state from being reached (and of that lack of provisions I
am reasonably certain).

And then there's a non-technical aspect to this whole situation:
The set of people introducing features and the set of people
fixing bugs doesn't have a very large overlap. If I could be
certain that it will be those who introduce PI (or any other
feature requiring large or intrusive changes) who are also
going to look into problems with it later on (and in a timely
manner), I might take a more relaxed position. But since history
teaches me that this is not likely going to be the case, my
(submitter dependent) hesitance to accept (even if only
theoretically) risky new features is also some sort of self
defense - not just at the open source project level, but also
from a distro pov.

Jan

_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: On setting clear criteria for declaring a feature acceptable (was "vmx: VT-d posted-interrupt core logic handling")
  2016-03-09 16:23                         ` On setting clear criteria for declaring a feature acceptable (was "vmx: VT-d posted-interrupt core logic handling") George Dunlap
  2016-03-09 16:58                           ` On setting clear criteria for declaring a feature acceptable Jan Beulich
@ 2016-03-09 18:02                           ` David Vrabel
  2016-03-10  1:15                             ` Wu, Feng
  2016-03-10  9:30                             ` George Dunlap
  2016-03-10  5:09                           ` Tian, Kevin
  2 siblings, 2 replies; 53+ messages in thread
From: David Vrabel @ 2016-03-09 18:02 UTC (permalink / raw)
  To: George Dunlap, Jan Beulich
  Cc: Lars Kurth, Kevin Tian, Feng Wu, George Dunlap, Andrew Cooper,
	Dario Faggioli, Ian Jackson, xen-devel

On 09/03/16 16:23, George Dunlap wrote:
> 
> I don't know why this is controversial -- this seems obvious to me.
> What do other committers / maintainers think?

I started on a reply to this but then I went back and read the original
thread...

+    /*
+     * XXX: The length of the list depends on how many vCPU is current
+     * blocked on this specific pCPU. This may hurt the interrupt
+     * latency if the list grows to too many entries.
+     */

Even the original author knows that there's a problem here, so in this
case George, I think you are unfairly criticizing Jan.

David

_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: On setting clear criteria for declaring a feature acceptable (was "vmx: VT-d posted-interrupt core logic handling")
  2016-03-09 18:02                           ` On setting clear criteria for declaring a feature acceptable (was "vmx: VT-d posted-interrupt core logic handling") David Vrabel
@ 2016-03-10  1:15                             ` Wu, Feng
  2016-03-10  9:30                             ` George Dunlap
  1 sibling, 0 replies; 53+ messages in thread
From: Wu, Feng @ 2016-03-10  1:15 UTC (permalink / raw)
  To: David Vrabel, George Dunlap, Jan Beulich
  Cc: Lars Kurth, Tian, Kevin, Wu, Feng, George Dunlap, Andrew Cooper,
	Dario Faggioli, Ian Jackson, xen-devel



> -----Original Message-----
> From: David Vrabel [mailto:david.vrabel@citrix.com]
> Sent: Thursday, March 10, 2016 2:02 AM
> To: George Dunlap <george.dunlap@citrix.com>; Jan Beulich
> <JBeulich@suse.com>
> Cc: Andrew Cooper <andrew.cooper3@citrix.com>; Dario Faggioli
> <dario.faggioli@citrix.com>; George Dunlap <George.Dunlap@eu.citrix.com>;
> Wu, Feng <feng.wu@intel.com>; Tian, Kevin <kevin.tian@intel.com>; xen-
> devel@lists.xen.org; Konrad Rzeszutek Wilk <konrad.wilk@oracle.com>; Ian
> Jackson <Ian.Jackson@eu.citrix.com>; Lars Kurth <lars.kurth@citrix.com>
> Subject: Re: On setting clear criteria for declaring a feature acceptable (was
> "vmx: VT-d posted-interrupt core logic handling")
> 
> On 09/03/16 16:23, George Dunlap wrote:
> >
> > I don't know why this is controversial -- this seems obvious to me.
> > What do other committers / maintainers think?
> 
> I started on a reply to this but then I went back and read the original
> thread...
> 
> +    /*
> +     * XXX: The length of the list depends on how many vCPU is current
> +     * blocked on this specific pCPU. This may hurt the interrupt
> +     * latency if the list grows to too many entries.
> +     */
> 
> Even the original author knows that there's a problem here, so in this
> case George, I think you are unfairly criticizing Jan.

This is the potential issue Jan pointed out, and adding the comments is
according Jan's comments then. But as George pointed out, it is not
very clear to how to reproduce this scenario in real world and what
is the criteria of "the list is too long", so here we are discussing whether
it is reasonable to make this feature default off just because of this
theoretically existing issue, and hence the " criteria for declaring a
feature acceptable ".

Thanks,
Feng

> 
> David

_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: On setting clear criteria for declaring a feature acceptable (was "vmx: VT-d posted-interrupt core logic handling")
  2016-03-09 16:23                         ` On setting clear criteria for declaring a feature acceptable (was "vmx: VT-d posted-interrupt core logic handling") George Dunlap
  2016-03-09 16:58                           ` On setting clear criteria for declaring a feature acceptable Jan Beulich
  2016-03-09 18:02                           ` On setting clear criteria for declaring a feature acceptable (was "vmx: VT-d posted-interrupt core logic handling") David Vrabel
@ 2016-03-10  5:09                           ` Tian, Kevin
  2016-03-10  8:07                             ` vmx: VT-d posted-interrupt core logic handling Jan Beulich
  2 siblings, 1 reply; 53+ messages in thread
From: Tian, Kevin @ 2016-03-10  5:09 UTC (permalink / raw)
  To: George Dunlap, Jan Beulich
  Cc: Lars Kurth, Wu, Feng, George Dunlap, Andrew Cooper,
	Dario Faggioli, Ian Jackson, xen-devel, David Vrabel

> From: George Dunlap [mailto:george.dunlap@citrix.com]
> Sent: Thursday, March 10, 2016 12:23 AM
> 
> [Changing the subject and CC'ing more people]
> 
> On 09/03/16 13:39, Jan Beulich wrote:
> >>>> On 08.03.16 at 19:38, <george.dunlap@citrix.com> wrote:
> >> If I may go "meta" for a moment here -- this is exactly what I'm talking
> >> about with "Something bad may happen" being difficult to work with.
> >> Rather than you spelling out exactly the situation which you think may
> >> happen, (which I could then either accept or refute on its merits) *I*
> >> am now spending a lot of time and effort trying to imagine what
> >> situations you may be talking about and then refuting them myself.
> [snip]
> >
> >> If you have concerns, you need to make those concerns concrete, or at
> >> least set clear criteria for how someone could go about addressing your
> >> concerns.  And yes, it is *your* job, as the person doing the objecting
> >> (and even moreso as the x86 maintainer), to make your concerns explicit
> >> and/or set those criteria, and not Feng's job (or even my job) to try to
> >> guess what it is might make you happy.
> >
> > I'm sorry, George, but no, I don't think this is how things should
> > work. If for a new feature to be enabled by default it is unclear
> > whether that puts the system at risk, it's the party suggesting the
> > default enabling to prove there's no such risk.
> 
> And it's up to the maintainers to clearly define what kind of "proof"
> would be sufficient. I have no objection to saying that Feng (or someone
> else who cares about the feature) must do some work to demonstrate that
> the feature is in fact safe before it's enabled by default; that's
> perfectly reasonable. I have already suggested something that would shed
> light on the issue and potentially satisfy me. But it's not at all
> reasonable to give them the impossible task of trying to guess what will
> satisfy you.
> 
> I don't know why this is controversial -- this seems obvious to me.
> What do other committers / maintainers think?
> 

My 2 cents here.

It's always good to have a clear definition to which extend a performance
issue would become a security risk. I saw 200us/500us used as example
in this thread, however no one can give an accrual criteria. In that case,
how do we call it a problem even when Feng collected some data? Based
on mindset from all maintainers?

I think a good way of looking at this is based on which capability is impacted.
In this specific case the directly impacted metric is the interrupt delivery
latency. However today Xen is not RT-capable. Xen doesn't commit to 
deliver a worst-case 10us interrupt latency. The whole interrupt delivery path 
(from Xen into Guest) has not been optimized yet, then there could be other 
reasons impacting latency too beside the concern on this specific list walk. 
There is no baseline worst-case data w/o PI. There is no final goal to hit. 
There is no test case to measure. 

Then why blocking this feature due to this unmeasurable concern and why
not enabling it and then improving it later when it becomes a measurable 
concern when Xen will commit a clear interrupt latency goal will be committed 
by Xen (at that time people working on that effort will have to identify all kinds 
of problems impacting interrupt latency and then can optimize together)?
People should understand possibly bad interrupt latency in extreme cases
like discussed in this thread (w/ or w/o PI), since Xen doesn't commit anything 
here.

Thanks
Kevin

_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: vmx: VT-d posted-interrupt core logic handling
  2016-03-10  5:09                           ` Tian, Kevin
@ 2016-03-10  8:07                             ` Jan Beulich
  2016-03-10  8:43                               ` Tian, Kevin
  2016-03-10 10:41                               ` George Dunlap
  0 siblings, 2 replies; 53+ messages in thread
From: Jan Beulich @ 2016-03-10  8:07 UTC (permalink / raw)
  To: Kevin Tian
  Cc: Lars Kurth, Feng Wu, George Dunlap, Andrew Cooper,
	Dario Faggioli, Ian Jackson, George Dunlap, xen-devel,
	David Vrabel

>>> On 10.03.16 at 06:09, <kevin.tian@intel.com> wrote:
> It's always good to have a clear definition to which extend a performance
> issue would become a security risk. I saw 200us/500us used as example
> in this thread, however no one can give an accrual criteria. In that case,
> how do we call it a problem even when Feng collected some data? Based
> on mindset from all maintainers?

I think I've already made clear in previous comments that such
measurements won't lead anywhere. What we need is a
guarantee (by way of enforcement in source code) that the
lists can't grow overly large, compared to the total load placed
on the system.

> I think a good way of looking at this is based on which capability is 
> impacted.
> In this specific case the directly impacted metric is the interrupt delivery
> latency. However today Xen is not RT-capable. Xen doesn't commit to 
> deliver a worst-case 10us interrupt latency. The whole interrupt delivery 
> path 
> (from Xen into Guest) has not been optimized yet, then there could be other 
> reasons impacting latency too beside the concern on this specific list walk. 
> There is no baseline worst-case data w/o PI. There is no final goal to hit. 
> There is no test case to measure. 
> 
> Then why blocking this feature due to this unmeasurable concern and why
> not enabling it and then improving it later when it becomes a measurable 
> concern when Xen will commit a clear interrupt latency goal will be 
> committed 
> by Xen (at that time people working on that effort will have to identify all 
> kinds 
> of problems impacting interrupt latency and then can optimize together)?
> People should understand possibly bad interrupt latency in extreme cases
> like discussed in this thread (w/ or w/o PI), since Xen doesn't commit 
> anything 
> here.

I've never made any reference to this being an interrupt latency
issue; I think it was George who somehow implied this from earlier
comments. Interrupt latency, at least generally, isn't a security
concern (generally because of course latency can get so high that
it might become a concern). All my previous remarks regarding the
issue are solely from the common perspective of long running
operations (which we've been dealing with outside of interrupt
context in a variety of cases, as you may recall). Hence the purely
theoretical basis for some sort of measurement would be to
determine how long a worst case list traversal would take. With
"worst case" being derived from the theoretical limits the
hypervisor implementation so far implies: 128 vCPU-s per domain
(a limit which we sooner or later will need to lift, i.e. taking into
consideration a larger value - like the 8k for PV guests - wouldn't
hurt) by 32k domains per host, totaling to 4M possible list entries.
Yes, it is obvious that this limit won't be reachable in practice, but
no, any lower limit can't be guaranteed to be good enough.

But I'm just now noticing this is the wrong thread to have this
discussion in - George specifically branched off the thread with
the new topic to separate the general discussion from the
specific case of the criteria for default enabling VT-d PI. So let's
please move this back to the other sub-thread (and I've
changed to subject back to express this).

Jan


_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: vmx: VT-d posted-interrupt core logic handling
  2016-03-10  8:07                             ` vmx: VT-d posted-interrupt core logic handling Jan Beulich
@ 2016-03-10  8:43                               ` Tian, Kevin
  2016-03-10  9:05                                 ` Jan Beulich
  2016-03-10 10:41                               ` George Dunlap
  1 sibling, 1 reply; 53+ messages in thread
From: Tian, Kevin @ 2016-03-10  8:43 UTC (permalink / raw)
  To: Jan Beulich
  Cc: Lars Kurth, Wu, Feng, George Dunlap, Andrew Cooper,
	Dario Faggioli, Ian Jackson, George Dunlap, xen-devel,
	David Vrabel

> From: Jan Beulich [mailto:JBeulich@suse.com]
> Sent: Thursday, March 10, 2016 4:07 PM
> 
> >>> On 10.03.16 at 06:09, <kevin.tian@intel.com> wrote:
> > It's always good to have a clear definition to which extend a performance
> > issue would become a security risk. I saw 200us/500us used as example
> > in this thread, however no one can give an accrual criteria. In that case,
> > how do we call it a problem even when Feng collected some data? Based
> > on mindset from all maintainers?
> 
> I think I've already made clear in previous comments that such
> measurements won't lead anywhere. What we need is a
> guarantee (by way of enforcement in source code) that the
> lists can't grow overly large, compared to the total load placed
> on the system.

Thanks for clarity here. 

> 
> > I think a good way of looking at this is based on which capability is
> > impacted.
> > In this specific case the directly impacted metric is the interrupt delivery
> > latency. However today Xen is not RT-capable. Xen doesn't commit to
> > deliver a worst-case 10us interrupt latency. The whole interrupt delivery
> > path
> > (from Xen into Guest) has not been optimized yet, then there could be other
> > reasons impacting latency too beside the concern on this specific list walk.
> > There is no baseline worst-case data w/o PI. There is no final goal to hit.
> > There is no test case to measure.
> >
> > Then why blocking this feature due to this unmeasurable concern and why
> > not enabling it and then improving it later when it becomes a measurable
> > concern when Xen will commit a clear interrupt latency goal will be
> > committed
> > by Xen (at that time people working on that effort will have to identify all
> > kinds
> > of problems impacting interrupt latency and then can optimize together)?
> > People should understand possibly bad interrupt latency in extreme cases
> > like discussed in this thread (w/ or w/o PI), since Xen doesn't commit
> > anything
> > here.
> 
> I've never made any reference to this being an interrupt latency
> issue; I think it was George who somehow implied this from earlier
> comments. Interrupt latency, at least generally, isn't a security
> concern (generally because of course latency can get so high that
> it might become a concern). All my previous remarks regarding the
> issue are solely from the common perspective of long running
> operations (which we've been dealing with outside of interrupt
> context in a variety of cases, as you may recall). Hence the purely

Yes, that concern makes sense.

> theoretical basis for some sort of measurement would be to
> determine how long a worst case list traversal would take. With
> "worst case" being derived from the theoretical limits the
> hypervisor implementation so far implies: 128 vCPU-s per domain
> (a limit which we sooner or later will need to lift, i.e. taking into
> consideration a larger value - like the 8k for PV guests - wouldn't
> hurt) by 32k domains per host, totaling to 4M possible list entries.
> Yes, it is obvious that this limit won't be reachable in practice, but
> no, any lower limit can't be guaranteed to be good enough.

Here do you think whether '4M' possible entries are 'overly large'
so we must have some enforcement in code, or still some experiments 
required to verify '4M' does been a problem (since total overhead 
depends on what we do with each entry)? If the latter what's the 
criteria to define it as a problem (e.g. 200us in total)?

There are many linked list usages today in Xen hypervisor, which
have different theoretical maximum possible number. The closest
one to PI might be the usage in tmem (pool->share_list) which is 
page based so could grow 'overly large'. Other examples are 
magnitude lower, e.g. s->ioreq_vcpu_list in ioreq server (which
could be 8K in above example), and d->arch.hvm_domain.msixtbl_list
in MSI-x virtualization (which could be 2^11 per spec). Do we
also want to create some artificial scenarios to examine them 
since based on actual operation K-level entries may also become
a problem? 

Just want to figure out how best we can solve all related linked-list 
usages in current hypervisor. 

> 
> But I'm just now noticing this is the wrong thread to have this
> discussion in - George specifically branched off the thread with
> the new topic to separate the general discussion from the
> specific case of the criteria for default enabling VT-d PI. So let's
> please move this back to the other sub-thread (and I've
> changed to subject back to express this).
> 

Sorry for cross-posting.

Thanks
Kevin

_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: vmx: VT-d posted-interrupt core logic handling
  2016-03-10  8:43                               ` Tian, Kevin
@ 2016-03-10  9:05                                 ` Jan Beulich
  2016-03-10  9:20                                   ` Tian, Kevin
  2016-03-10 10:05                                   ` Tian, Kevin
  0 siblings, 2 replies; 53+ messages in thread
From: Jan Beulich @ 2016-03-10  9:05 UTC (permalink / raw)
  To: Kevin Tian
  Cc: Lars Kurth, Feng Wu, George Dunlap, Andrew Cooper,
	Dario Faggioli, Ian Jackson, GeorgeDunlap, xen-devel,
	David Vrabel

>>> On 10.03.16 at 09:43, <kevin.tian@intel.com> wrote:
>>  From: Jan Beulich [mailto:JBeulich@suse.com]
>> Sent: Thursday, March 10, 2016 4:07 PM
>> 
>> theoretical basis for some sort of measurement would be to
>> determine how long a worst case list traversal would take. With
>> "worst case" being derived from the theoretical limits the
>> hypervisor implementation so far implies: 128 vCPU-s per domain
>> (a limit which we sooner or later will need to lift, i.e. taking into
>> consideration a larger value - like the 8k for PV guests - wouldn't
>> hurt) by 32k domains per host, totaling to 4M possible list entries.
>> Yes, it is obvious that this limit won't be reachable in practice, but
>> no, any lower limit can't be guaranteed to be good enough.
> 
> Here do you think whether '4M' possible entries are 'overly large'
> so we must have some enforcement in code, or still some experiments 
> required to verify '4M' does been a problem (since total overhead 
> depends on what we do with each entry)? If the latter what's the 
> criteria to define it as a problem (e.g. 200us in total)?

Well, 4M makes, even with a single loop iteration just taking 1ns,
4ms already. Anything reaching the order of the minimum
scheduler time slice is potentially problematic. Anything reaching
the order of 1s is known to be actively bad outside of interrupt
context; within interrupt context you need to also consider
interrupt rate of course, so 4ms likely would already open the
potential of a CPU not making any forward progress anymore.

> There are many linked list usages today in Xen hypervisor, which
> have different theoretical maximum possible number. The closest
> one to PI might be the usage in tmem (pool->share_list) which is 
> page based so could grow 'overly large'. Other examples are 
> magnitude lower, e.g. s->ioreq_vcpu_list in ioreq server (which
> could be 8K in above example), and d->arch.hvm_domain.msixtbl_list
> in MSI-x virtualization (which could be 2^11 per spec). Do we
> also want to create some artificial scenarios to examine them 
> since based on actual operation K-level entries may also become
> a problem? 
> 
> Just want to figure out how best we can solve all related linked-list 
> usages in current hypervisor. 

As you say, those are (perhaps with the exception of tmem, which
isn't supported anyway due to XSA-15, and which therefore also
isn't on by default) in the order of a few thousand list elements.
And as mentioned above, different bounds apply for lists traversed
in interrupt context vs such traversed only in "normal" context.

Jan


_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: vmx: VT-d posted-interrupt core logic handling
  2016-03-10  9:05                                 ` Jan Beulich
@ 2016-03-10  9:20                                   ` Tian, Kevin
  2016-03-10 10:05                                   ` Tian, Kevin
  1 sibling, 0 replies; 53+ messages in thread
From: Tian, Kevin @ 2016-03-10  9:20 UTC (permalink / raw)
  To: Jan Beulich
  Cc: Lars Kurth, Wu, Feng, George Dunlap, Andrew Cooper,
	Dario Faggioli, Ian Jackson, GeorgeDunlap, xen-devel,
	David Vrabel

> From: Jan Beulich [mailto:JBeulich@suse.com]
> Sent: Thursday, March 10, 2016 5:06 PM
> 
> 
> > There are many linked list usages today in Xen hypervisor, which
> > have different theoretical maximum possible number. The closest
> > one to PI might be the usage in tmem (pool->share_list) which is
> > page based so could grow 'overly large'. Other examples are
> > magnitude lower, e.g. s->ioreq_vcpu_list in ioreq server (which
> > could be 8K in above example), and d->arch.hvm_domain.msixtbl_list
> > in MSI-x virtualization (which could be 2^11 per spec). Do we
> > also want to create some artificial scenarios to examine them
> > since based on actual operation K-level entries may also become
> > a problem?
> >
> > Just want to figure out how best we can solve all related linked-list
> > usages in current hypervisor.
> 
> As you say, those are (perhaps with the exception of tmem, which
> isn't supported anyway due to XSA-15, and which therefore also
> isn't on by default) in the order of a few thousand list elements.
> And as mentioned above, different bounds apply for lists traversed
> in interrupt context vs such traversed only in "normal" context.
> 

That's a good point. Interrupt context should have more restrictions.

Thanks
Kevin

_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: On setting clear criteria for declaring a feature acceptable (was "vmx: VT-d posted-interrupt core logic handling")
  2016-03-09 18:02                           ` On setting clear criteria for declaring a feature acceptable (was "vmx: VT-d posted-interrupt core logic handling") David Vrabel
  2016-03-10  1:15                             ` Wu, Feng
@ 2016-03-10  9:30                             ` George Dunlap
  1 sibling, 0 replies; 53+ messages in thread
From: George Dunlap @ 2016-03-10  9:30 UTC (permalink / raw)
  To: David Vrabel
  Cc: Lars Kurth, Kevin Tian, Feng Wu, Andrew Cooper, Dario Faggioli,
	Ian Jackson, George Dunlap, xen-devel, Jan Beulich

On Wed, Mar 9, 2016 at 6:02 PM, David Vrabel <david.vrabel@citrix.com> wrote:
> On 09/03/16 16:23, George Dunlap wrote:
>>
>> I don't know why this is controversial -- this seems obvious to me.
>> What do other committers / maintainers think?
>
> I started on a reply to this but then I went back and read the original
> thread...
>
> +    /*
> +     * XXX: The length of the list depends on how many vCPU is current
> +     * blocked on this specific pCPU. This may hurt the interrupt
> +     * latency if the list grows to too many entries.
> +     */
>
> Even the original author knows that there's a problem here, so in this
> case George, I think you are unfairly criticizing Jan.

Yes, as Feng points out, that comment was put there because Jan made
it a prerequisite for acceptance, not because Feng had concrete reason
to believe there was a potential problem.  I don't have any objection
to that in general, *as long as* it's accompanied by actionable
suggestions for evaluating whether it's true and/or fixing it.

 -George

_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: vmx: VT-d posted-interrupt core logic handling
  2016-03-10  9:05                                 ` Jan Beulich
  2016-03-10  9:20                                   ` Tian, Kevin
@ 2016-03-10 10:05                                   ` Tian, Kevin
  2016-03-10 10:18                                     ` Jan Beulich
  2016-03-10 13:36                                     ` Wu, Feng
  1 sibling, 2 replies; 53+ messages in thread
From: Tian, Kevin @ 2016-03-10 10:05 UTC (permalink / raw)
  To: Jan Beulich
  Cc: Lars Kurth, Wu, Feng, George Dunlap, Andrew Cooper,
	Dario Faggioli, Ian Jackson, GeorgeDunlap, xen-devel,
	David Vrabel

> From: Tian, Kevin
> Sent: Thursday, March 10, 2016 5:20 PM
> 
> > From: Jan Beulich [mailto:JBeulich@suse.com]
> > Sent: Thursday, March 10, 2016 5:06 PM
> >
> >
> > > There are many linked list usages today in Xen hypervisor, which
> > > have different theoretical maximum possible number. The closest
> > > one to PI might be the usage in tmem (pool->share_list) which is
> > > page based so could grow 'overly large'. Other examples are
> > > magnitude lower, e.g. s->ioreq_vcpu_list in ioreq server (which
> > > could be 8K in above example), and d->arch.hvm_domain.msixtbl_list
> > > in MSI-x virtualization (which could be 2^11 per spec). Do we
> > > also want to create some artificial scenarios to examine them
> > > since based on actual operation K-level entries may also become
> > > a problem?
> > >
> > > Just want to figure out how best we can solve all related linked-list
> > > usages in current hypervisor.
> >
> > As you say, those are (perhaps with the exception of tmem, which
> > isn't supported anyway due to XSA-15, and which therefore also
> > isn't on by default) in the order of a few thousand list elements.
> > And as mentioned above, different bounds apply for lists traversed
> > in interrupt context vs such traversed only in "normal" context.
> >
> 
> That's a good point. Interrupt context should have more restrictions.

Hi, Jan,

I'm thinking your earlier idea about evenly distributed list:

--
Ah, right, I think that limitation was named before, yet I've
forgotten about it again. But that only slightly alters the
suggestion: To distribute vCPU-s evenly would then require to
change their placement on the pCPU in the course of entering
blocked state.
--

Actually after more thinking, there is no hard requirement that
the vcpu must block on the pcpu which is configured in 'NDST'
of that vcpu's PI descriptor. What really matters, is that the
vcpu is added to the linked list of the very pcpu, then when PI
notification comes we can always find out the vcpu struct from
that pcpu's linked list. Of course one drawback of such placement
is additional IPI incurred in wake up path.

Then one possible optimized policy within vmx_vcpu_block could 
be:

(Say PCPU1 which VCPU1 is currently blocked on)
- As long as the #vcpus in the linked list on PCPU1 is below a 
threshold (say 16), add VCPU1 to the list. NDST set to PCPU1;
Upon PI notification on PCPU1, local linked list is searched to
find VCPU1 and then VCPU1 will be unblocked on PCPU1;

- Otherwise, add VCPU1 to PCPU2 based on a simple distribution 
algorithm (based on vcpu_id/vm_id). VCPU1 still blocks on PCPU1
but NDST set to PCPU2. Upon notification on PCPU2, local linked
list is searched to find VCPU1 and then an IPI is sent to PCPU1 to 
unblock VCPU1;

Feng, do you see any overlook here? :-)

Thanks
Kevin

_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: vmx: VT-d posted-interrupt core logic handling
  2016-03-10 10:05                                   ` Tian, Kevin
@ 2016-03-10 10:18                                     ` Jan Beulich
  2016-03-10 10:35                                       ` David Vrabel
  2016-03-10 11:00                                       ` George Dunlap
  2016-03-10 13:36                                     ` Wu, Feng
  1 sibling, 2 replies; 53+ messages in thread
From: Jan Beulich @ 2016-03-10 10:18 UTC (permalink / raw)
  To: Kevin Tian
  Cc: Lars Kurth, Feng Wu, George Dunlap, Andrew Cooper,
	Dario Faggioli, Ian Jackson, GeorgeDunlap, xen-devel,
	David Vrabel

>>> On 10.03.16 at 11:05, <kevin.tian@intel.com> wrote:
>>  From: Tian, Kevin
>> Sent: Thursday, March 10, 2016 5:20 PM
>> 
>> > From: Jan Beulich [mailto:JBeulich@suse.com]
>> > Sent: Thursday, March 10, 2016 5:06 PM
>> >
>> >
>> > > There are many linked list usages today in Xen hypervisor, which
>> > > have different theoretical maximum possible number. The closest
>> > > one to PI might be the usage in tmem (pool->share_list) which is
>> > > page based so could grow 'overly large'. Other examples are
>> > > magnitude lower, e.g. s->ioreq_vcpu_list in ioreq server (which
>> > > could be 8K in above example), and d->arch.hvm_domain.msixtbl_list
>> > > in MSI-x virtualization (which could be 2^11 per spec). Do we
>> > > also want to create some artificial scenarios to examine them
>> > > since based on actual operation K-level entries may also become
>> > > a problem?
>> > >
>> > > Just want to figure out how best we can solve all related linked-list
>> > > usages in current hypervisor.
>> >
>> > As you say, those are (perhaps with the exception of tmem, which
>> > isn't supported anyway due to XSA-15, and which therefore also
>> > isn't on by default) in the order of a few thousand list elements.
>> > And as mentioned above, different bounds apply for lists traversed
>> > in interrupt context vs such traversed only in "normal" context.
>> >
>> 
>> That's a good point. Interrupt context should have more restrictions.
> 
> Hi, Jan,
> 
> I'm thinking your earlier idea about evenly distributed list:
> 
> --
> Ah, right, I think that limitation was named before, yet I've
> forgotten about it again. But that only slightly alters the
> suggestion: To distribute vCPU-s evenly would then require to
> change their placement on the pCPU in the course of entering
> blocked state.
> --
> 
> Actually after more thinking, there is no hard requirement that
> the vcpu must block on the pcpu which is configured in 'NDST'
> of that vcpu's PI descriptor. What really matters, is that the
> vcpu is added to the linked list of the very pcpu, then when PI
> notification comes we can always find out the vcpu struct from
> that pcpu's linked list. Of course one drawback of such placement
> is additional IPI incurred in wake up path.
> 
> Then one possible optimized policy within vmx_vcpu_block could 
> be:
> 
> (Say PCPU1 which VCPU1 is currently blocked on)
> - As long as the #vcpus in the linked list on PCPU1 is below a 
> threshold (say 16), add VCPU1 to the list. NDST set to PCPU1;
> Upon PI notification on PCPU1, local linked list is searched to
> find VCPU1 and then VCPU1 will be unblocked on PCPU1;
> 
> - Otherwise, add VCPU1 to PCPU2 based on a simple distribution 
> algorithm (based on vcpu_id/vm_id). VCPU1 still blocks on PCPU1
> but NDST set to PCPU2. Upon notification on PCPU2, local linked
> list is searched to find VCPU1 and then an IPI is sent to PCPU1 to 
> unblock VCPU1;

Sounds possible, if the lock handling can be got right. But of
course there can't be any hard limit like 16, at least not alone
(on a systems with extremely many mostly idle vCPU-s we'd
need to allow larger counts - see my earlier explanations in this
regard).

Jan


_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: vmx: VT-d posted-interrupt core logic handling
  2016-03-10 10:18                                     ` Jan Beulich
@ 2016-03-10 10:35                                       ` David Vrabel
  2016-03-10 10:46                                         ` George Dunlap
  2016-03-10 11:00                                       ` George Dunlap
  1 sibling, 1 reply; 53+ messages in thread
From: David Vrabel @ 2016-03-10 10:35 UTC (permalink / raw)
  To: Jan Beulich, Kevin Tian
  Cc: Lars Kurth, Feng Wu, George Dunlap, Andrew Cooper,
	Dario Faggioli, Ian Jackson, GeorgeDunlap, xen-devel

On 10/03/16 10:18, Jan Beulich wrote:
>>>> On 10.03.16 at 11:05, <kevin.tian@intel.com> wrote:
>>>  From: Tian, Kevin
>>> Sent: Thursday, March 10, 2016 5:20 PM
>>>
>>>> From: Jan Beulich [mailto:JBeulich@suse.com]
>>>> Sent: Thursday, March 10, 2016 5:06 PM
>>>>
>>>>
>>>>> There are many linked list usages today in Xen hypervisor, which
>>>>> have different theoretical maximum possible number. The closest
>>>>> one to PI might be the usage in tmem (pool->share_list) which is
>>>>> page based so could grow 'overly large'. Other examples are
>>>>> magnitude lower, e.g. s->ioreq_vcpu_list in ioreq server (which
>>>>> could be 8K in above example), and d->arch.hvm_domain.msixtbl_list
>>>>> in MSI-x virtualization (which could be 2^11 per spec). Do we
>>>>> also want to create some artificial scenarios to examine them
>>>>> since based on actual operation K-level entries may also become
>>>>> a problem?
>>>>>
>>>>> Just want to figure out how best we can solve all related linked-list
>>>>> usages in current hypervisor.
>>>>
>>>> As you say, those are (perhaps with the exception of tmem, which
>>>> isn't supported anyway due to XSA-15, and which therefore also
>>>> isn't on by default) in the order of a few thousand list elements.
>>>> And as mentioned above, different bounds apply for lists traversed
>>>> in interrupt context vs such traversed only in "normal" context.
>>>>
>>>
>>> That's a good point. Interrupt context should have more restrictions.
>>
>> Hi, Jan,
>>
>> I'm thinking your earlier idea about evenly distributed list:
>>
>> --
>> Ah, right, I think that limitation was named before, yet I've
>> forgotten about it again. But that only slightly alters the
>> suggestion: To distribute vCPU-s evenly would then require to
>> change their placement on the pCPU in the course of entering
>> blocked state.
>> --
>>
>> Actually after more thinking, there is no hard requirement that
>> the vcpu must block on the pcpu which is configured in 'NDST'
>> of that vcpu's PI descriptor. What really matters, is that the
>> vcpu is added to the linked list of the very pcpu, then when PI
>> notification comes we can always find out the vcpu struct from
>> that pcpu's linked list. Of course one drawback of such placement
>> is additional IPI incurred in wake up path.
>>
>> Then one possible optimized policy within vmx_vcpu_block could 
>> be:
>>
>> (Say PCPU1 which VCPU1 is currently blocked on)
>> - As long as the #vcpus in the linked list on PCPU1 is below a 
>> threshold (say 16), add VCPU1 to the list. NDST set to PCPU1;
>> Upon PI notification on PCPU1, local linked list is searched to
>> find VCPU1 and then VCPU1 will be unblocked on PCPU1;
>>
>> - Otherwise, add VCPU1 to PCPU2 based on a simple distribution 
>> algorithm (based on vcpu_id/vm_id). VCPU1 still blocks on PCPU1
>> but NDST set to PCPU2. Upon notification on PCPU2, local linked
>> list is searched to find VCPU1 and then an IPI is sent to PCPU1 to 
>> unblock VCPU1;
> 
> Sounds possible, if the lock handling can be got right. But of
> course there can't be any hard limit like 16, at least not alone
> (on a systems with extremely many mostly idle vCPU-s we'd
> need to allow larger counts - see my earlier explanations in this
> regard).

You could also consider only waking the first N VCPUs and just making
the rest runnable.  If you wake more VCPUs than PCPUs at the same time
most of them won't actually be scheduled.

N would be some measure of how many VCPUs could be run immediately (with
N <= number of PCPUs).

David


_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: vmx: VT-d posted-interrupt core logic handling
  2016-03-10  8:07                             ` vmx: VT-d posted-interrupt core logic handling Jan Beulich
  2016-03-10  8:43                               ` Tian, Kevin
@ 2016-03-10 10:41                               ` George Dunlap
  1 sibling, 0 replies; 53+ messages in thread
From: George Dunlap @ 2016-03-10 10:41 UTC (permalink / raw)
  To: Jan Beulich, Kevin Tian
  Cc: Lars Kurth, Feng Wu, George Dunlap, Andrew Cooper,
	Dario Faggioli, Ian Jackson, xen-devel, David Vrabel

On 10/03/16 08:07, Jan Beulich wrote:
>>>> On 10.03.16 at 06:09, <kevin.tian@intel.com> wrote:
>> It's always good to have a clear definition to which extend a performance
>> issue would become a security risk. I saw 200us/500us used as example
>> in this thread, however no one can give an accrual criteria. In that case,
>> how do we call it a problem even when Feng collected some data? Based
>> on mindset from all maintainers?
> 
> I think I've already made clear in previous comments that such
> measurements won't lead anywhere. What we need is a
> guarantee (by way of enforcement in source code) that the
> lists can't grow overly large, compared to the total load placed
> on the system.
> 
>> I think a good way of looking at this is based on which capability is 
>> impacted.
>> In this specific case the directly impacted metric is the interrupt delivery
>> latency. However today Xen is not RT-capable. Xen doesn't commit to 
>> deliver a worst-case 10us interrupt latency. The whole interrupt delivery 
>> path 
>> (from Xen into Guest) has not been optimized yet, then there could be other 
>> reasons impacting latency too beside the concern on this specific list walk. 
>> There is no baseline worst-case data w/o PI. There is no final goal to hit. 
>> There is no test case to measure. 
>>
>> Then why blocking this feature due to this unmeasurable concern and why
>> not enabling it and then improving it later when it becomes a measurable 
>> concern when Xen will commit a clear interrupt latency goal will be 
>> committed 
>> by Xen (at that time people working on that effort will have to identify all 
>> kinds 
>> of problems impacting interrupt latency and then can optimize together)?
>> People should understand possibly bad interrupt latency in extreme cases
>> like discussed in this thread (w/ or w/o PI), since Xen doesn't commit 
>> anything 
>> here.
> 
> I've never made any reference to this being an interrupt latency
> issue; I think it was George who somehow implied this from earlier
> comments. Interrupt latency, at least generally, isn't a security
> concern (generally because of course latency can get so high that
> it might become a concern). All my previous remarks regarding the
> issue are solely from the common perspective of long running
> operations (which we've been dealing with outside of interrupt
> context in a variety of cases, as you may recall). Hence the purely
> theoretical basis for some sort of measurement would be to
> determine how long a worst case list traversal would take. With
> "worst case" being derived from the theoretical limits the
> hypervisor implementation so far implies: 128 vCPU-s per domain
> (a limit which we sooner or later will need to lift, i.e. taking into
> consideration a larger value - like the 8k for PV guests - wouldn't
> hurt) by 32k domains per host, totaling to 4M possible list entries.
> Yes, it is obvious that this limit won't be reachable in practice, but
> no, any lower limit can't be guaranteed to be good enough.

Can I suggest we suspend the discussion of what would or would not be
reasonable and come back to it next week?  I definitely feel myself
digging my heels in here, so it might be good to go away and come back
to the discussion with a bit of distance.

(Potential technical solutions are still game I think.)

 -George

_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: vmx: VT-d posted-interrupt core logic handling
  2016-03-10 10:35                                       ` David Vrabel
@ 2016-03-10 10:46                                         ` George Dunlap
  2016-03-10 11:16                                           ` David Vrabel
  0 siblings, 1 reply; 53+ messages in thread
From: George Dunlap @ 2016-03-10 10:46 UTC (permalink / raw)
  To: David Vrabel, Jan Beulich, Kevin Tian
  Cc: Lars Kurth, Feng Wu, George Dunlap, Andrew Cooper,
	Dario Faggioli, Ian Jackson, xen-devel

On 10/03/16 10:35, David Vrabel wrote:
> On 10/03/16 10:18, Jan Beulich wrote:
>>>>> On 10.03.16 at 11:05, <kevin.tian@intel.com> wrote:
>>>>  From: Tian, Kevin
>>>> Sent: Thursday, March 10, 2016 5:20 PM
>>>>
>>>>> From: Jan Beulich [mailto:JBeulich@suse.com]
>>>>> Sent: Thursday, March 10, 2016 5:06 PM
>>>>>
>>>>>
>>>>>> There are many linked list usages today in Xen hypervisor, which
>>>>>> have different theoretical maximum possible number. The closest
>>>>>> one to PI might be the usage in tmem (pool->share_list) which is
>>>>>> page based so could grow 'overly large'. Other examples are
>>>>>> magnitude lower, e.g. s->ioreq_vcpu_list in ioreq server (which
>>>>>> could be 8K in above example), and d->arch.hvm_domain.msixtbl_list
>>>>>> in MSI-x virtualization (which could be 2^11 per spec). Do we
>>>>>> also want to create some artificial scenarios to examine them
>>>>>> since based on actual operation K-level entries may also become
>>>>>> a problem?
>>>>>>
>>>>>> Just want to figure out how best we can solve all related linked-list
>>>>>> usages in current hypervisor.
>>>>>
>>>>> As you say, those are (perhaps with the exception of tmem, which
>>>>> isn't supported anyway due to XSA-15, and which therefore also
>>>>> isn't on by default) in the order of a few thousand list elements.
>>>>> And as mentioned above, different bounds apply for lists traversed
>>>>> in interrupt context vs such traversed only in "normal" context.
>>>>>
>>>>
>>>> That's a good point. Interrupt context should have more restrictions.
>>>
>>> Hi, Jan,
>>>
>>> I'm thinking your earlier idea about evenly distributed list:
>>>
>>> --
>>> Ah, right, I think that limitation was named before, yet I've
>>> forgotten about it again. But that only slightly alters the
>>> suggestion: To distribute vCPU-s evenly would then require to
>>> change their placement on the pCPU in the course of entering
>>> blocked state.
>>> --
>>>
>>> Actually after more thinking, there is no hard requirement that
>>> the vcpu must block on the pcpu which is configured in 'NDST'
>>> of that vcpu's PI descriptor. What really matters, is that the
>>> vcpu is added to the linked list of the very pcpu, then when PI
>>> notification comes we can always find out the vcpu struct from
>>> that pcpu's linked list. Of course one drawback of such placement
>>> is additional IPI incurred in wake up path.
>>>
>>> Then one possible optimized policy within vmx_vcpu_block could 
>>> be:
>>>
>>> (Say PCPU1 which VCPU1 is currently blocked on)
>>> - As long as the #vcpus in the linked list on PCPU1 is below a 
>>> threshold (say 16), add VCPU1 to the list. NDST set to PCPU1;
>>> Upon PI notification on PCPU1, local linked list is searched to
>>> find VCPU1 and then VCPU1 will be unblocked on PCPU1;
>>>
>>> - Otherwise, add VCPU1 to PCPU2 based on a simple distribution 
>>> algorithm (based on vcpu_id/vm_id). VCPU1 still blocks on PCPU1
>>> but NDST set to PCPU2. Upon notification on PCPU2, local linked
>>> list is searched to find VCPU1 and then an IPI is sent to PCPU1 to 
>>> unblock VCPU1;
>>
>> Sounds possible, if the lock handling can be got right. But of
>> course there can't be any hard limit like 16, at least not alone
>> (on a systems with extremely many mostly idle vCPU-s we'd
>> need to allow larger counts - see my earlier explanations in this
>> regard).
> 
> You could also consider only waking the first N VCPUs and just making
> the rest runnable.  If you wake more VCPUs than PCPUs at the same time
> most of them won't actually be scheduled.

"Waking" a vcpu means "changing from blocked to runnable", so those two
things are the same.  And I can't figure out what you mean instead --
can you elaborate?

Waking up 1000 vcpus is going to take strictly more time than checking
whether there's a PI interrupt pending on 1000 vcpus to see if they need
to be woken up.

 -George

_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: vmx: VT-d posted-interrupt core logic handling
  2016-03-10 10:18                                     ` Jan Beulich
  2016-03-10 10:35                                       ` David Vrabel
@ 2016-03-10 11:00                                       ` George Dunlap
  2016-03-10 11:21                                         ` Dario Faggioli
  1 sibling, 1 reply; 53+ messages in thread
From: George Dunlap @ 2016-03-10 11:00 UTC (permalink / raw)
  To: Jan Beulich, Kevin Tian
  Cc: Lars Kurth, Feng Wu, George Dunlap, Andrew Cooper,
	Dario Faggioli, Ian Jackson, xen-devel, David Vrabel

On 10/03/16 10:18, Jan Beulich wrote:
>>>> On 10.03.16 at 11:05, <kevin.tian@intel.com> wrote:
>>>  From: Tian, Kevin
>>> Sent: Thursday, March 10, 2016 5:20 PM
>>>
>>>> From: Jan Beulich [mailto:JBeulich@suse.com]
>>>> Sent: Thursday, March 10, 2016 5:06 PM
>>>>
>>>>
>>>>> There are many linked list usages today in Xen hypervisor, which
>>>>> have different theoretical maximum possible number. The closest
>>>>> one to PI might be the usage in tmem (pool->share_list) which is
>>>>> page based so could grow 'overly large'. Other examples are
>>>>> magnitude lower, e.g. s->ioreq_vcpu_list in ioreq server (which
>>>>> could be 8K in above example), and d->arch.hvm_domain.msixtbl_list
>>>>> in MSI-x virtualization (which could be 2^11 per spec). Do we
>>>>> also want to create some artificial scenarios to examine them
>>>>> since based on actual operation K-level entries may also become
>>>>> a problem?
>>>>>
>>>>> Just want to figure out how best we can solve all related linked-list
>>>>> usages in current hypervisor.
>>>>
>>>> As you say, those are (perhaps with the exception of tmem, which
>>>> isn't supported anyway due to XSA-15, and which therefore also
>>>> isn't on by default) in the order of a few thousand list elements.
>>>> And as mentioned above, different bounds apply for lists traversed
>>>> in interrupt context vs such traversed only in "normal" context.
>>>>
>>>
>>> That's a good point. Interrupt context should have more restrictions.
>>
>> Hi, Jan,
>>
>> I'm thinking your earlier idea about evenly distributed list:
>>
>> --
>> Ah, right, I think that limitation was named before, yet I've
>> forgotten about it again. But that only slightly alters the
>> suggestion: To distribute vCPU-s evenly would then require to
>> change their placement on the pCPU in the course of entering
>> blocked state.
>> --
>>
>> Actually after more thinking, there is no hard requirement that
>> the vcpu must block on the pcpu which is configured in 'NDST'
>> of that vcpu's PI descriptor. What really matters, is that the
>> vcpu is added to the linked list of the very pcpu, then when PI
>> notification comes we can always find out the vcpu struct from
>> that pcpu's linked list. Of course one drawback of such placement
>> is additional IPI incurred in wake up path.
>>
>> Then one possible optimized policy within vmx_vcpu_block could 
>> be:
>>
>> (Say PCPU1 which VCPU1 is currently blocked on)
>> - As long as the #vcpus in the linked list on PCPU1 is below a 
>> threshold (say 16), add VCPU1 to the list. NDST set to PCPU1;
>> Upon PI notification on PCPU1, local linked list is searched to
>> find VCPU1 and then VCPU1 will be unblocked on PCPU1;
>>
>> - Otherwise, add VCPU1 to PCPU2 based on a simple distribution 
>> algorithm (based on vcpu_id/vm_id). VCPU1 still blocks on PCPU1
>> but NDST set to PCPU2. Upon notification on PCPU2, local linked
>> list is searched to find VCPU1 and then an IPI is sent to PCPU1 to 
>> unblock VCPU1;
> 
> Sounds possible, if the lock handling can be got right. But of
> course there can't be any hard limit like 16, at least not alone
> (on a systems with extremely many mostly idle vCPU-s we'd
> need to allow larger counts - see my earlier explanations in this
> regard).

A lot of the scheduling code uses spin_trylock() to just skip over pcpus
that are busy when doing this sort of load-balancing.  Using a hash to
choose a default and then cycling through pcpus until you find one whose
lock you can grab should be reasonably efficient.

Re "an IPI is sent to PCPU1", all that should be transparent to the PI
code -- it already calls vcpu_unblock(), which will call vcpu_wake(),
which calls the scheduling wake code, which will DTRT.

FWIW I have much less objection to this sort of solution if it were
confined to the PI arch_block() callback, rather than something that
required changes to the schedulers.

 -George

_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: vmx: VT-d posted-interrupt core logic handling
  2016-03-10 10:46                                         ` George Dunlap
@ 2016-03-10 11:16                                           ` David Vrabel
  2016-03-10 11:49                                             ` George Dunlap
  2016-03-10 13:24                                             ` Jan Beulich
  0 siblings, 2 replies; 53+ messages in thread
From: David Vrabel @ 2016-03-10 11:16 UTC (permalink / raw)
  To: George Dunlap, Jan Beulich, Kevin Tian
  Cc: Lars Kurth, Feng Wu, George Dunlap, Andrew Cooper,
	Dario Faggioli, Ian Jackson, xen-devel

On 10/03/16 10:46, George Dunlap wrote:
> On 10/03/16 10:35, David Vrabel wrote:
>> On 10/03/16 10:18, Jan Beulich wrote:
>>>>>> On 10.03.16 at 11:05, <kevin.tian@intel.com> wrote:
>>>>>  From: Tian, Kevin
>>>>> Sent: Thursday, March 10, 2016 5:20 PM
>>>>>
>>>>>> From: Jan Beulich [mailto:JBeulich@suse.com]
>>>>>> Sent: Thursday, March 10, 2016 5:06 PM
>>>>>>
>>>>>>
>>>>>>> There are many linked list usages today in Xen hypervisor, which
>>>>>>> have different theoretical maximum possible number. The closest
>>>>>>> one to PI might be the usage in tmem (pool->share_list) which is
>>>>>>> page based so could grow 'overly large'. Other examples are
>>>>>>> magnitude lower, e.g. s->ioreq_vcpu_list in ioreq server (which
>>>>>>> could be 8K in above example), and d->arch.hvm_domain.msixtbl_list
>>>>>>> in MSI-x virtualization (which could be 2^11 per spec). Do we
>>>>>>> also want to create some artificial scenarios to examine them
>>>>>>> since based on actual operation K-level entries may also become
>>>>>>> a problem?
>>>>>>>
>>>>>>> Just want to figure out how best we can solve all related linked-list
>>>>>>> usages in current hypervisor.
>>>>>>
>>>>>> As you say, those are (perhaps with the exception of tmem, which
>>>>>> isn't supported anyway due to XSA-15, and which therefore also
>>>>>> isn't on by default) in the order of a few thousand list elements.
>>>>>> And as mentioned above, different bounds apply for lists traversed
>>>>>> in interrupt context vs such traversed only in "normal" context.
>>>>>>
>>>>>
>>>>> That's a good point. Interrupt context should have more restrictions.
>>>>
>>>> Hi, Jan,
>>>>
>>>> I'm thinking your earlier idea about evenly distributed list:
>>>>
>>>> --
>>>> Ah, right, I think that limitation was named before, yet I've
>>>> forgotten about it again. But that only slightly alters the
>>>> suggestion: To distribute vCPU-s evenly would then require to
>>>> change their placement on the pCPU in the course of entering
>>>> blocked state.
>>>> --
>>>>
>>>> Actually after more thinking, there is no hard requirement that
>>>> the vcpu must block on the pcpu which is configured in 'NDST'
>>>> of that vcpu's PI descriptor. What really matters, is that the
>>>> vcpu is added to the linked list of the very pcpu, then when PI
>>>> notification comes we can always find out the vcpu struct from
>>>> that pcpu's linked list. Of course one drawback of such placement
>>>> is additional IPI incurred in wake up path.
>>>>
>>>> Then one possible optimized policy within vmx_vcpu_block could 
>>>> be:
>>>>
>>>> (Say PCPU1 which VCPU1 is currently blocked on)
>>>> - As long as the #vcpus in the linked list on PCPU1 is below a 
>>>> threshold (say 16), add VCPU1 to the list. NDST set to PCPU1;
>>>> Upon PI notification on PCPU1, local linked list is searched to
>>>> find VCPU1 and then VCPU1 will be unblocked on PCPU1;
>>>>
>>>> - Otherwise, add VCPU1 to PCPU2 based on a simple distribution 
>>>> algorithm (based on vcpu_id/vm_id). VCPU1 still blocks on PCPU1
>>>> but NDST set to PCPU2. Upon notification on PCPU2, local linked
>>>> list is searched to find VCPU1 and then an IPI is sent to PCPU1 to 
>>>> unblock VCPU1;
>>>
>>> Sounds possible, if the lock handling can be got right. But of
>>> course there can't be any hard limit like 16, at least not alone
>>> (on a systems with extremely many mostly idle vCPU-s we'd
>>> need to allow larger counts - see my earlier explanations in this
>>> regard).
>>
>> You could also consider only waking the first N VCPUs and just making
>> the rest runnable.  If you wake more VCPUs than PCPUs at the same time
>> most of them won't actually be scheduled.
> 
> "Waking" a vcpu means "changing from blocked to runnable", so those two
> things are the same.  And I can't figure out what you mean instead --
> can you elaborate?
> 
> Waking up 1000 vcpus is going to take strictly more time than checking
> whether there's a PI interrupt pending on 1000 vcpus to see if they need
> to be woken up.

Waking means making it runnable /and/ attempt to make it running.

So I mean, for the > N'th VCPU don't call __runq_tickle(), only call
__runq_insert().

David

_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: vmx: VT-d posted-interrupt core logic handling
  2016-03-10 11:00                                       ` George Dunlap
@ 2016-03-10 11:21                                         ` Dario Faggioli
  0 siblings, 0 replies; 53+ messages in thread
From: Dario Faggioli @ 2016-03-10 11:21 UTC (permalink / raw)
  To: George Dunlap, Jan Beulich, Kevin Tian
  Cc: Lars Kurth, Feng Wu, George Dunlap, Andrew Cooper, Ian Jackson,
	xen-devel, David Vrabel


[-- Attachment #1.1: Type: text/plain, Size: 2380 bytes --]

On Thu, 2016-03-10 at 11:00 +0000, George Dunlap wrote:
> > > > > On 10.03.16 at 11:05, <kevin.tian@intel.com> wrote:
> > > >  From: Tian, Kevin
> > > > Sent: Thursday, March 10, 2016 5:20 PM
> > > > 
> > > Actually after more thinking, there is no hard requirement that
> > > the vcpu must block on the pcpu which is configured in 'NDST'
> > > of that vcpu's PI descriptor. What really matters, is that the
> > > vcpu is added to the linked list of the very pcpu, then when PI
> > > notification comes we can always find out the vcpu struct from
> > > that pcpu's linked list. Of course one drawback of such placement
> > > is additional IPI incurred in wake up path.
> > > 
> > > 
> Re "an IPI is sent to PCPU1", all that should be transparent to the
> PI
> code -- it already calls vcpu_unblock(), which will call vcpu_wake(),
> which calls the scheduling wake code, which will DTRT.
> 
Exactly. In fact, whether there will be any IPI involved is under
control of the scheduler, rather than of PI code, even right now.

In fact, no matter in what pCPU's blocked list a vCPU is, it is the
'tickling' logic (for all Credit, Credit2 and RTDS) that really decides
on which pCPU the vCPU should wakeup, and send the IPI, if that is not
the pCPU we're running on.

It can be argued that having a vCPU in the blocked list of the pCPU
where it was running when it blocked could be a good thing, because it
may then be able to restart running there when waking, which may have
positive cache effects, etc.
But that is not at all guaranteed. In fact, it could well be the case
in fairly idle systems, but under high load and/or if hard and soft
affinity are in use (which may well be the case, .e.g, on large NUMA
servers), that isn't necessarily true, and we should not base on such
assumption.

> FWIW I have much less objection to this sort of solution if it were
> confined to the PI arch_block() callback, rather than something that
> required changes to the schedulers.
> 
Same here, and I think this can well be done in such a way... Worth a
shot, IMO.

Regards,
Dario
-- 
<<This happens because I choose it to happen!>> (Raistlin Majere)
-----------------------------------------------------------------
Dario Faggioli, Ph.D, http://about.me/dario.faggioli
Senior Software Engineer, Citrix Systems R&D Ltd., Cambridge (UK)


[-- Attachment #1.2: This is a digitally signed message part --]
[-- Type: application/pgp-signature, Size: 181 bytes --]

[-- Attachment #2: Type: text/plain, Size: 126 bytes --]

_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: vmx: VT-d posted-interrupt core logic handling
  2016-03-10 11:16                                           ` David Vrabel
@ 2016-03-10 11:49                                             ` George Dunlap
  2016-03-10 13:24                                             ` Jan Beulich
  1 sibling, 0 replies; 53+ messages in thread
From: George Dunlap @ 2016-03-10 11:49 UTC (permalink / raw)
  To: David Vrabel, Jan Beulich, Kevin Tian
  Cc: Lars Kurth, Feng Wu, George Dunlap, Andrew Cooper,
	Dario Faggioli, Ian Jackson, xen-devel

On 10/03/16 11:16, David Vrabel wrote:
> On 10/03/16 10:46, George Dunlap wrote:
>> On 10/03/16 10:35, David Vrabel wrote:
>>> On 10/03/16 10:18, Jan Beulich wrote:
>>>>>>> On 10.03.16 at 11:05, <kevin.tian@intel.com> wrote:
>>>>>>  From: Tian, Kevin
>>>>>> Sent: Thursday, March 10, 2016 5:20 PM
>>>>>>
>>>>>>> From: Jan Beulich [mailto:JBeulich@suse.com]
>>>>>>> Sent: Thursday, March 10, 2016 5:06 PM
>>>>>>>
>>>>>>>
>>>>>>>> There are many linked list usages today in Xen hypervisor, which
>>>>>>>> have different theoretical maximum possible number. The closest
>>>>>>>> one to PI might be the usage in tmem (pool->share_list) which is
>>>>>>>> page based so could grow 'overly large'. Other examples are
>>>>>>>> magnitude lower, e.g. s->ioreq_vcpu_list in ioreq server (which
>>>>>>>> could be 8K in above example), and d->arch.hvm_domain.msixtbl_list
>>>>>>>> in MSI-x virtualization (which could be 2^11 per spec). Do we
>>>>>>>> also want to create some artificial scenarios to examine them
>>>>>>>> since based on actual operation K-level entries may also become
>>>>>>>> a problem?
>>>>>>>>
>>>>>>>> Just want to figure out how best we can solve all related linked-list
>>>>>>>> usages in current hypervisor.
>>>>>>>
>>>>>>> As you say, those are (perhaps with the exception of tmem, which
>>>>>>> isn't supported anyway due to XSA-15, and which therefore also
>>>>>>> isn't on by default) in the order of a few thousand list elements.
>>>>>>> And as mentioned above, different bounds apply for lists traversed
>>>>>>> in interrupt context vs such traversed only in "normal" context.
>>>>>>>
>>>>>>
>>>>>> That's a good point. Interrupt context should have more restrictions.
>>>>>
>>>>> Hi, Jan,
>>>>>
>>>>> I'm thinking your earlier idea about evenly distributed list:
>>>>>
>>>>> --
>>>>> Ah, right, I think that limitation was named before, yet I've
>>>>> forgotten about it again. But that only slightly alters the
>>>>> suggestion: To distribute vCPU-s evenly would then require to
>>>>> change their placement on the pCPU in the course of entering
>>>>> blocked state.
>>>>> --
>>>>>
>>>>> Actually after more thinking, there is no hard requirement that
>>>>> the vcpu must block on the pcpu which is configured in 'NDST'
>>>>> of that vcpu's PI descriptor. What really matters, is that the
>>>>> vcpu is added to the linked list of the very pcpu, then when PI
>>>>> notification comes we can always find out the vcpu struct from
>>>>> that pcpu's linked list. Of course one drawback of such placement
>>>>> is additional IPI incurred in wake up path.
>>>>>
>>>>> Then one possible optimized policy within vmx_vcpu_block could 
>>>>> be:
>>>>>
>>>>> (Say PCPU1 which VCPU1 is currently blocked on)
>>>>> - As long as the #vcpus in the linked list on PCPU1 is below a 
>>>>> threshold (say 16), add VCPU1 to the list. NDST set to PCPU1;
>>>>> Upon PI notification on PCPU1, local linked list is searched to
>>>>> find VCPU1 and then VCPU1 will be unblocked on PCPU1;
>>>>>
>>>>> - Otherwise, add VCPU1 to PCPU2 based on a simple distribution 
>>>>> algorithm (based on vcpu_id/vm_id). VCPU1 still blocks on PCPU1
>>>>> but NDST set to PCPU2. Upon notification on PCPU2, local linked
>>>>> list is searched to find VCPU1 and then an IPI is sent to PCPU1 to 
>>>>> unblock VCPU1;
>>>>
>>>> Sounds possible, if the lock handling can be got right. But of
>>>> course there can't be any hard limit like 16, at least not alone
>>>> (on a systems with extremely many mostly idle vCPU-s we'd
>>>> need to allow larger counts - see my earlier explanations in this
>>>> regard).
>>>
>>> You could also consider only waking the first N VCPUs and just making
>>> the rest runnable.  If you wake more VCPUs than PCPUs at the same time
>>> most of them won't actually be scheduled.
>>
>> "Waking" a vcpu means "changing from blocked to runnable", so those two
>> things are the same.  And I can't figure out what you mean instead --
>> can you elaborate?
>>
>> Waking up 1000 vcpus is going to take strictly more time than checking
>> whether there's a PI interrupt pending on 1000 vcpus to see if they need
>> to be woken up.
> 
> Waking means making it runnable /and/ attempt to make it running.
> 
> So I mean, for the > N'th VCPU don't call __runq_tickle(), only call
> __runq_insert().

I'm not sure that would satisfy Jan; inserting 1000 vcpus into the
runqueue (much less inserting 4 million vcpus) is still going to take
quite a while, even without looking for a place to run them.

 -George


_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: vmx: VT-d posted-interrupt core logic handling
  2016-03-10 11:16                                           ` David Vrabel
  2016-03-10 11:49                                             ` George Dunlap
@ 2016-03-10 13:24                                             ` Jan Beulich
  1 sibling, 0 replies; 53+ messages in thread
From: Jan Beulich @ 2016-03-10 13:24 UTC (permalink / raw)
  To: David Vrabel
  Cc: Lars Kurth, Kevin Tian, Feng Wu, GeorgeDunlap, Andrew Cooper,
	Dario Faggioli, Ian Jackson, George Dunlap, xen-devel

>>> On 10.03.16 at 12:16, <david.vrabel@citrix.com> wrote:
> On 10/03/16 10:46, George Dunlap wrote:
>> On 10/03/16 10:35, David Vrabel wrote:
>>> You could also consider only waking the first N VCPUs and just making
>>> the rest runnable.  If you wake more VCPUs than PCPUs at the same time
>>> most of them won't actually be scheduled.
>> 
>> "Waking" a vcpu means "changing from blocked to runnable", so those two
>> things are the same.  And I can't figure out what you mean instead --
>> can you elaborate?
>> 
>> Waking up 1000 vcpus is going to take strictly more time than checking
>> whether there's a PI interrupt pending on 1000 vcpus to see if they need
>> to be woken up.
> 
> Waking means making it runnable /and/ attempt to make it running.
> 
> So I mean, for the > N'th VCPU don't call __runq_tickle(), only call
> __runq_insert().

We expect only some (hopefully small) percentage of the vCPU-s
on the list to actually need unblocking anyway. As George said,
list traversal alone can be an issue here, and we can't stop going
through the list half way. The case where a large portion of the
vCPU-s on the list actually need waking up would be even more
worrying.

Jan


_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: vmx: VT-d posted-interrupt core logic handling
  2016-03-10 10:05                                   ` Tian, Kevin
  2016-03-10 10:18                                     ` Jan Beulich
@ 2016-03-10 13:36                                     ` Wu, Feng
  2016-05-17 13:27                                       ` Konrad Rzeszutek Wilk
  1 sibling, 1 reply; 53+ messages in thread
From: Wu, Feng @ 2016-03-10 13:36 UTC (permalink / raw)
  To: Tian, Kevin, Jan Beulich
  Cc: Lars Kurth, Wu, Feng, George Dunlap, Andrew Cooper,
	Dario Faggioli, Ian Jackson, GeorgeDunlap, xen-devel,
	David Vrabel



> -----Original Message-----
> From: Tian, Kevin
> Sent: Thursday, March 10, 2016 6:06 PM
> To: Jan Beulich <JBeulich@suse.com>
> Cc: Andrew Cooper <andrew.cooper3@citrix.com>; Dario Faggioli
> <dario.faggioli@citrix.com>; David Vrabel <david.vrabel@citrix.com>;
> GeorgeDunlap <george.dunlap@citrix.com>; Lars Kurth <lars.kurth@citrix.com>;
> George Dunlap <George.Dunlap@eu.citrix.com>; Ian Jackson
> <Ian.Jackson@eu.citrix.com>; Wu, Feng <feng.wu@intel.com>; xen-
> devel@lists.xen.org; Konrad Rzeszutek Wilk <konrad.wilk@oracle.com>
> Subject: RE: vmx: VT-d posted-interrupt core logic handling
> 
> 
> Hi, Jan,
> 
> I'm thinking your earlier idea about evenly distributed list:
> 
> --
> Ah, right, I think that limitation was named before, yet I've
> forgotten about it again. But that only slightly alters the
> suggestion: To distribute vCPU-s evenly would then require to
> change their placement on the pCPU in the course of entering
> blocked state.
> --
> 
> Actually after more thinking, there is no hard requirement that
> the vcpu must block on the pcpu which is configured in 'NDST'
> of that vcpu's PI descriptor. What really matters, is that the
> vcpu is added to the linked list of the very pcpu, then when PI
> notification comes we can always find out the vcpu struct from
> that pcpu's linked list. Of course one drawback of such placement
> is additional IPI incurred in wake up path.
> 
> Then one possible optimized policy within vmx_vcpu_block could
> be:
> 
> (Say PCPU1 which VCPU1 is currently blocked on)
> - As long as the #vcpus in the linked list on PCPU1 is below a
> threshold (say 16), add VCPU1 to the list. NDST set to PCPU1;
> Upon PI notification on PCPU1, local linked list is searched to
> find VCPU1 and then VCPU1 will be unblocked on PCPU1;
> 
> - Otherwise, add VCPU1 to PCPU2 based on a simple distribution
> algorithm (based on vcpu_id/vm_id). VCPU1 still blocks on PCPU1
> but NDST set to PCPU2. Upon notification on PCPU2, local linked
> list is searched to find VCPU1 and then an IPI is sent to PCPU1 to
> unblock VCPU1;
> 
> Feng, do you see any overlook here? :-)

Kevin, thanks for the suggestion, it sounds a good idea, I will think
it a bit more and do some trials based on it.

Thanks,
Feng

> 
> Thanks
> Kevin

_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: vmx: VT-d posted-interrupt core logic handling
  2016-03-10 13:36                                     ` Wu, Feng
@ 2016-05-17 13:27                                       ` Konrad Rzeszutek Wilk
  2016-05-19  7:22                                         ` Wu, Feng
  0 siblings, 1 reply; 53+ messages in thread
From: Konrad Rzeszutek Wilk @ 2016-05-17 13:27 UTC (permalink / raw)
  To: Wu, Feng
  Cc: Lars Kurth, Tian, Kevin, George Dunlap, Andrew Cooper,
	Dario Faggioli, Ian Jackson, GeorgeDunlap, xen-devel,
	David Vrabel, Jan Beulich

On Thu, Mar 10, 2016 at 01:36:34PM +0000, Wu, Feng wrote:
> 
> 
> > -----Original Message-----
> > From: Tian, Kevin
> > Sent: Thursday, March 10, 2016 6:06 PM
> > To: Jan Beulich <JBeulich@suse.com>
> > Cc: Andrew Cooper <andrew.cooper3@citrix.com>; Dario Faggioli
> > <dario.faggioli@citrix.com>; David Vrabel <david.vrabel@citrix.com>;
> > GeorgeDunlap <george.dunlap@citrix.com>; Lars Kurth <lars.kurth@citrix.com>;
> > George Dunlap <George.Dunlap@eu.citrix.com>; Ian Jackson
> > <Ian.Jackson@eu.citrix.com>; Wu, Feng <feng.wu@intel.com>; xen-
> > devel@lists.xen.org; Konrad Rzeszutek Wilk <konrad.wilk@oracle.com>
> > Subject: RE: vmx: VT-d posted-interrupt core logic handling
> > 
> > 
> > Hi, Jan,
> > 
> > I'm thinking your earlier idea about evenly distributed list:
> > 
> > --
> > Ah, right, I think that limitation was named before, yet I've
> > forgotten about it again. But that only slightly alters the
> > suggestion: To distribute vCPU-s evenly would then require to
> > change their placement on the pCPU in the course of entering
> > blocked state.
> > --
> > 
> > Actually after more thinking, there is no hard requirement that
> > the vcpu must block on the pcpu which is configured in 'NDST'
> > of that vcpu's PI descriptor. What really matters, is that the
> > vcpu is added to the linked list of the very pcpu, then when PI
> > notification comes we can always find out the vcpu struct from
> > that pcpu's linked list. Of course one drawback of such placement
> > is additional IPI incurred in wake up path.
> > 
> > Then one possible optimized policy within vmx_vcpu_block could
> > be:
> > 
> > (Say PCPU1 which VCPU1 is currently blocked on)
> > - As long as the #vcpus in the linked list on PCPU1 is below a
> > threshold (say 16), add VCPU1 to the list. NDST set to PCPU1;
> > Upon PI notification on PCPU1, local linked list is searched to
> > find VCPU1 and then VCPU1 will be unblocked on PCPU1;
> > 
> > - Otherwise, add VCPU1 to PCPU2 based on a simple distribution
> > algorithm (based on vcpu_id/vm_id). VCPU1 still blocks on PCPU1
> > but NDST set to PCPU2. Upon notification on PCPU2, local linked
> > list is searched to find VCPU1 and then an IPI is sent to PCPU1 to
> > unblock VCPU1;
> > 
> > Feng, do you see any overlook here? :-)
> 
> Kevin, thanks for the suggestion, it sounds a good idea, I will think
> it a bit more and do some trials based on it.

Hey!

I am curious how the trials went? This feature is pretty awesome and
I am wondering what the next step is?

Thanks!

_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

* Re: vmx: VT-d posted-interrupt core logic handling
  2016-05-17 13:27                                       ` Konrad Rzeszutek Wilk
@ 2016-05-19  7:22                                         ` Wu, Feng
  0 siblings, 0 replies; 53+ messages in thread
From: Wu, Feng @ 2016-05-19  7:22 UTC (permalink / raw)
  To: Konrad Rzeszutek Wilk
  Cc: Lars Kurth, Tian, Kevin, Wu, Feng, George Dunlap, Andrew Cooper,
	Dario Faggioli, Ian Jackson, GeorgeDunlap, xen-devel,
	David Vrabel, Jan Beulich



> -----Original Message-----
> From: Konrad Rzeszutek Wilk [mailto:konrad.wilk@oracle.com]
> Sent: Tuesday, May 17, 2016 9:27 PM
> To: Wu, Feng <feng.wu@intel.com>
> Cc: Tian, Kevin <kevin.tian@intel.com>; Jan Beulich <JBeulich@suse.com>;
> Andrew Cooper <andrew.cooper3@citrix.com>; Dario Faggioli
> <dario.faggioli@citrix.com>; David Vrabel <david.vrabel@citrix.com>;
> GeorgeDunlap <george.dunlap@citrix.com>; Lars Kurth <lars.kurth@citrix.com>;
> George Dunlap <George.Dunlap@eu.citrix.com>; Ian Jackson
> <Ian.Jackson@eu.citrix.com>; xen-devel@lists.xen.org
> Subject: Re: vmx: VT-d posted-interrupt core logic handling
> 
> On Thu, Mar 10, 2016 at 01:36:34PM +0000, Wu, Feng wrote:
> >
> >
> > > -----Original Message-----
> > > From: Tian, Kevin
> > > Sent: Thursday, March 10, 2016 6:06 PM
> > > To: Jan Beulich <JBeulich@suse.com>
> > > Cc: Andrew Cooper <andrew.cooper3@citrix.com>; Dario Faggioli
> > > <dario.faggioli@citrix.com>; David Vrabel <david.vrabel@citrix.com>;
> > > GeorgeDunlap <george.dunlap@citrix.com>; Lars Kurth
> <lars.kurth@citrix.com>;
> > > George Dunlap <George.Dunlap@eu.citrix.com>; Ian Jackson
> > > <Ian.Jackson@eu.citrix.com>; Wu, Feng <feng.wu@intel.com>; xen-
> > > devel@lists.xen.org; Konrad Rzeszutek Wilk <konrad.wilk@oracle.com>
> > > Subject: RE: vmx: VT-d posted-interrupt core logic handling
> > >
> > >
> > > Hi, Jan,
> > >
> > > I'm thinking your earlier idea about evenly distributed list:
> > >
> > > --
> > > Ah, right, I think that limitation was named before, yet I've
> > > forgotten about it again. But that only slightly alters the
> > > suggestion: To distribute vCPU-s evenly would then require to
> > > change their placement on the pCPU in the course of entering
> > > blocked state.
> > > --
> > >
> > > Actually after more thinking, there is no hard requirement that
> > > the vcpu must block on the pcpu which is configured in 'NDST'
> > > of that vcpu's PI descriptor. What really matters, is that the
> > > vcpu is added to the linked list of the very pcpu, then when PI
> > > notification comes we can always find out the vcpu struct from
> > > that pcpu's linked list. Of course one drawback of such placement
> > > is additional IPI incurred in wake up path.
> > >
> > > Then one possible optimized policy within vmx_vcpu_block could
> > > be:
> > >
> > > (Say PCPU1 which VCPU1 is currently blocked on)
> > > - As long as the #vcpus in the linked list on PCPU1 is below a
> > > threshold (say 16), add VCPU1 to the list. NDST set to PCPU1;
> > > Upon PI notification on PCPU1, local linked list is searched to
> > > find VCPU1 and then VCPU1 will be unblocked on PCPU1;
> > >
> > > - Otherwise, add VCPU1 to PCPU2 based on a simple distribution
> > > algorithm (based on vcpu_id/vm_id). VCPU1 still blocks on PCPU1
> > > but NDST set to PCPU2. Upon notification on PCPU2, local linked
> > > list is searched to find VCPU1 and then an IPI is sent to PCPU1 to
> > > unblock VCPU1;
> > >
> > > Feng, do you see any overlook here? :-)
> >
> > Kevin, thanks for the suggestion, it sounds a good idea, I will think
> > it a bit more and do some trials based on it.
> 
> Hey!
> 
> I am curious how the trials went? This feature is pretty awesome and
> I am wondering what the next step is?

Good to know that you are interested in this feature. However, I have
been occupied by another things recently, I think I will continue this
work some time later.

Thanks,
Feng

> 
> Thanks!

_______________________________________________
Xen-devel mailing list
Xen-devel@lists.xen.org
http://lists.xen.org/xen-devel

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

end of thread, other threads:[~2016-05-19  7:22 UTC | newest]

Thread overview: 53+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2016-02-29  3:00 [PATCH v14 0/2] Add VT-d Posted-Interrupts support Feng Wu
2016-02-29  3:00 ` [PATCH v14 1/2] vmx: VT-d posted-interrupt core logic handling Feng Wu
2016-02-29 13:33   ` Jan Beulich
2016-02-29 13:52     ` Dario Faggioli
2016-03-01  5:39       ` Wu, Feng
2016-03-01  9:24         ` Jan Beulich
2016-03-01 10:16     ` George Dunlap
2016-03-01 13:06       ` Wu, Feng
2016-03-01  5:24   ` Tian, Kevin
2016-03-01  5:39     ` Wu, Feng
2016-03-04 22:00   ` Ideas " Konrad Rzeszutek Wilk
2016-03-07 11:21     ` George Dunlap
2016-03-07 15:53       ` Konrad Rzeszutek Wilk
2016-03-07 16:19         ` Dario Faggioli
2016-03-07 20:23           ` Konrad Rzeszutek Wilk
2016-03-08 12:02         ` George Dunlap
2016-03-08 13:10           ` Wu, Feng
2016-03-08 14:42             ` George Dunlap
2016-03-08 15:42               ` Jan Beulich
2016-03-08 17:05                 ` George Dunlap
2016-03-08 17:26                   ` Jan Beulich
2016-03-08 18:38                     ` George Dunlap
2016-03-09  5:06                       ` Wu, Feng
2016-03-09 13:39                       ` Jan Beulich
2016-03-09 16:01                         ` George Dunlap
2016-03-09 16:31                           ` Jan Beulich
2016-03-09 16:23                         ` On setting clear criteria for declaring a feature acceptable (was "vmx: VT-d posted-interrupt core logic handling") George Dunlap
2016-03-09 16:58                           ` On setting clear criteria for declaring a feature acceptable Jan Beulich
2016-03-09 18:02                           ` On setting clear criteria for declaring a feature acceptable (was "vmx: VT-d posted-interrupt core logic handling") David Vrabel
2016-03-10  1:15                             ` Wu, Feng
2016-03-10  9:30                             ` George Dunlap
2016-03-10  5:09                           ` Tian, Kevin
2016-03-10  8:07                             ` vmx: VT-d posted-interrupt core logic handling Jan Beulich
2016-03-10  8:43                               ` Tian, Kevin
2016-03-10  9:05                                 ` Jan Beulich
2016-03-10  9:20                                   ` Tian, Kevin
2016-03-10 10:05                                   ` Tian, Kevin
2016-03-10 10:18                                     ` Jan Beulich
2016-03-10 10:35                                       ` David Vrabel
2016-03-10 10:46                                         ` George Dunlap
2016-03-10 11:16                                           ` David Vrabel
2016-03-10 11:49                                             ` George Dunlap
2016-03-10 13:24                                             ` Jan Beulich
2016-03-10 11:00                                       ` George Dunlap
2016-03-10 11:21                                         ` Dario Faggioli
2016-03-10 13:36                                     ` Wu, Feng
2016-05-17 13:27                                       ` Konrad Rzeszutek Wilk
2016-05-19  7:22                                         ` Wu, Feng
2016-03-10 10:41                               ` George Dunlap
2016-03-09  5:22                   ` Ideas Re: [PATCH v14 1/2] " Wu, Feng
2016-03-09 11:25                     ` George Dunlap
2016-03-09 12:06                       ` Wu, Feng
2016-02-29  3:00 ` [PATCH v14 2/2] Add a command line parameter for VT-d posted-interrupts Feng Wu

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