All of lore.kernel.org
 help / color / mirror / Atom feed
* [PATCH 0/3 V2] Add AMD SEV and SEV-ES intra host migration support
@ 2021-07-14 16:01 Peter Gonda
  2021-07-14 16:01 ` [PATCH 1/3 V2] KVM, SEV: Refactor out function for unregistering encrypted regions Peter Gonda
                   ` (2 more replies)
  0 siblings, 3 replies; 10+ messages in thread
From: Peter Gonda @ 2021-07-14 16:01 UTC (permalink / raw)
  To: pgonda
  Cc: Paolo Bonzini, Sean Christopherson, David Rientjes,
	Dr . David Alan Gilbert, Brijesh Singh, Vitaly Kuznetsov,
	Wanpeng Li, Jim Mattson, Joerg Roedel, Thomas Gleixner,
	Ingo Molnar, Borislav Petkov, H. Peter Anvin, kvm, linux-kernel

Intra host migration provides a low-cost mechanism for userspace VMM upgrades.
It is an alternative to traditional (i.e., remote) live migration. Whereas
remote migration handles move a guest to a new host, intra host migration only
handles moving a guest to a new userspace VMM within a host.  This can be
used to update, rollback, change flags of the VMM, etc. The lower cost
compared to live migration comes from the fact that the guest's memory does
not need to be copied between processes. A handle to the guest memory
simply gets passed to the new VMM, this could be done via using /dev/shm
with share=on or similar feature.

The guest state can be transferred from an old VMM to a new VMM as follows:
1. Export guest state from KVM to the old user-space VMM via a getter
user-space/kernel API 2. Transfer guest state from old VMM to new VMM via
IPC communication 3. Import guest state into KVM from the new user-space
VMM via a setter user-space/kernel API VMMs by exporting from KVM using
getters, sending that data to the new VMM, then setting it again in KVM.

In the common case for intra host migration, we can rely on the normal ioctls
for passing data from one VMM to the next. SEV, SEV-ES, and other
confidential compute environments make most of this information opaque, and
render KVM ioctls such as "KVM_GET_REGS" irrelevant.  As a result, we need
the ability to pass this opaque metadata from one VMM to the next. The
easiest way to do this is to leave this data in the kernel, and transfer
ownership of the metadata from one KVM VM (or vCPU) to the next. For
example, we need to move the SEV enabled ASID, VMSAs, and GHCB metadata
from one VMM to the next.  In general, we need to be able to hand off any
data that would be unsafe/impossible for the kernel to hand directly to
userspace (and cannot be reproduced using data that can be handed safely to
userspace).

During the intra host send operation the SEV required metadata, the guest's
ASID is loaded into a kvm wide hashmap keyed by a value given by
userspace. This allows the userspace VMM to pass the key to the target
VMM. Then on intra host receive the target VMM can be loaded with the
metadata from the hashmap.

v2:
 * Added marcorr@ reviewed by tag
 * Renamed function introduced in 1/3
 * Edited with seanjc@'s review comments
 ** Cleaned up WARN usage
 ** Userspace makes random token now
 * Edited with brijesh.singh@'s review comments
 ** Checks for different LAUNCH_* states in send function

v1: https://lore.kernel.org/kvm/20210621163118.1040170-1-pgonda@google.com/

Peter Gonda (3):
  KVM, SEV: Refactor out function for unregistering encrypted regions
  KVM, SEV: Add support for SEV intra host migration
  KVM, SEV: Add support for SEV-ES intra host migration

 .../virt/kvm/amd-memory-encryption.rst        |  43 ++
 arch/x86/kvm/svm/sev.c                        | 396 +++++++++++++++++-
 arch/x86/kvm/svm/svm.h                        |   1 +
 include/uapi/linux/kvm.h                      |  12 +
 4 files changed, 433 insertions(+), 19 deletions(-)

base-commit: 7caa04b36f20

Cc: Paolo Bonzini <pbonzini@redhat.com>
Cc: Sean Christopherson <seanjc@google.com>
Cc: David Rientjes <rientjes@google.com>
Cc: Dr. David Alan Gilbert <dgilbert@redhat.com>
Cc: Brijesh Singh <brijesh.singh@amd.com>
Cc: Vitaly Kuznetsov <vkuznets@redhat.com>
Cc: Wanpeng Li <wanpengli@tencent.com>
Cc: Jim Mattson <jmattson@google.com>
Cc: Joerg Roedel <joro@8bytes.org>
Cc: Thomas Gleixner <tglx@linutronix.de>
Cc: Ingo Molnar <mingo@redhat.com>
Cc: Borislav Petkov <bp@alien8.de>
Cc: "H. Peter Anvin" <hpa@zytor.com>
Cc: kvm@vger.kernel.org
Cc: linux-kernel@vger.kernel.org

-- 
2.32.0.93.g670b81a890-goog


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

* [PATCH 1/3 V2] KVM, SEV: Refactor out function for unregistering encrypted regions
  2021-07-14 16:01 [PATCH 0/3 V2] Add AMD SEV and SEV-ES intra host migration support Peter Gonda
@ 2021-07-14 16:01 ` Peter Gonda
  2021-07-14 16:01 ` [PATCH 2/3 V2] KVM, SEV: Add support for SEV intra host migration Peter Gonda
  2021-07-14 16:01 ` [PATCH 3/3 V2] KVM, SEV: Add support for SEV-ES " Peter Gonda
  2 siblings, 0 replies; 10+ messages in thread
From: Peter Gonda @ 2021-07-14 16:01 UTC (permalink / raw)
  To: pgonda
  Cc: Brijesh Singh, Marc Orr, Paolo Bonzini, Sean Christopherson,
	David Rientjes, Dr . David Alan Gilbert, Vitaly Kuznetsov,
	Wanpeng Li, Jim Mattson, Joerg Roedel, Thomas Gleixner,
	Ingo Molnar, Borislav Petkov, H. Peter Anvin, kvm, linux-kernel

Factor out helper function for freeing the encrypted region list.

Signed-off-by: Peter Gonda <pgonda@google.com>
Reviewed-by: Brijesh Singh <brijesh.singh@amd.com>
Reviewed-by: Marc Orr <marcorr@google.com>
Cc: Paolo Bonzini <pbonzini@redhat.com>
Cc: Sean Christopherson <seanjc@google.com>
Cc: David Rientjes <rientjes@google.com>
Cc: Dr. David Alan Gilbert <dgilbert@redhat.com>
Cc: Brijesh Singh <brijesh.singh@amd.com>
Cc: Vitaly Kuznetsov <vkuznets@redhat.com>
Cc: Wanpeng Li <wanpengli@tencent.com>
Cc: Jim Mattson <jmattson@google.com>
Cc: Joerg Roedel <joro@8bytes.org>
Cc: Thomas Gleixner <tglx@linutronix.de>
Cc: Ingo Molnar <mingo@redhat.com>
Cc: Borislav Petkov <bp@alien8.de>
Cc: "H. Peter Anvin" <hpa@zytor.com>
Cc: kvm@vger.kernel.org
Cc: linux-kernel@vger.kernel.org
---
 arch/x86/kvm/svm/sev.c | 26 +++++++++++++++++---------
 1 file changed, 17 insertions(+), 9 deletions(-)

diff --git a/arch/x86/kvm/svm/sev.c b/arch/x86/kvm/svm/sev.c
index 8d36f0c73071..78fea9c4048d 100644
--- a/arch/x86/kvm/svm/sev.c
+++ b/arch/x86/kvm/svm/sev.c
@@ -1775,11 +1775,25 @@ int svm_vm_copy_asid_from(struct kvm *kvm, unsigned int source_fd)
 	return ret;
 }
 
+static void unregister_enc_regions(struct kvm *kvm,
+					    struct list_head *mem_regions)
+{
+	struct enc_region *pos, *q;
+
+	lockdep_assert_held(&kvm->lock);
+
+	if (list_empty(mem_regions))
+		return;
+
+	list_for_each_entry_safe(pos, q, mem_regions, list) {
+		__unregister_enc_region_locked(kvm, pos);
+		cond_resched();
+	}
+}
+
 void sev_vm_destroy(struct kvm *kvm)
 {
 	struct kvm_sev_info *sev = &to_kvm_svm(kvm)->sev_info;
-	struct list_head *head = &sev->regions_list;
-	struct list_head *pos, *q;
 
 	if (!sev_guest(kvm))
 		return;
@@ -1803,13 +1817,7 @@ void sev_vm_destroy(struct kvm *kvm)
 	 * if userspace was terminated before unregistering the memory regions
 	 * then lets unpin all the registered memory.
 	 */
-	if (!list_empty(head)) {
-		list_for_each_safe(pos, q, head) {
-			__unregister_enc_region_locked(kvm,
-				list_entry(pos, struct enc_region, list));
-			cond_resched();
-		}
-	}
+	unregister_enc_regions(kvm, &sev->regions_list);
 
 	mutex_unlock(&kvm->lock);
 
-- 
2.32.0.93.g670b81a890-goog


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

* [PATCH 2/3 V2] KVM, SEV: Add support for SEV intra host migration
  2021-07-14 16:01 [PATCH 0/3 V2] Add AMD SEV and SEV-ES intra host migration support Peter Gonda
  2021-07-14 16:01 ` [PATCH 1/3 V2] KVM, SEV: Refactor out function for unregistering encrypted regions Peter Gonda
@ 2021-07-14 16:01 ` Peter Gonda
  2021-07-14 19:22   ` Brijesh Singh
  2021-07-14 16:01 ` [PATCH 3/3 V2] KVM, SEV: Add support for SEV-ES " Peter Gonda
  2 siblings, 1 reply; 10+ messages in thread
From: Peter Gonda @ 2021-07-14 16:01 UTC (permalink / raw)
  To: pgonda
  Cc: Lars Bull, Marc Orr, Paolo Bonzini, Sean Christopherson,
	David Rientjes, Dr . David Alan Gilbert, Brijesh Singh,
	Vitaly Kuznetsov, Wanpeng Li, Jim Mattson, Joerg Roedel,
	Thomas Gleixner, Ingo Molnar, Borislav Petkov, H. Peter Anvin,
	kvm, linux-kernel

For SEV to work with intra host migration, contents of the SEV info struct
such as the ASID (used to index the encryption key in the AMD SP) and
the list of memory regions need to be transferred to the target VM.
This change adds commands for sending and receiving the sev info.

To avoid exposing this internal state to userspace and prevent other
processes from importing state they shouldn't have access to, the send
returns a token to userspace that is handed off to the target VM. The
target passes in this token to receive the sent state. The token is only
valid for one-time use. Functionality on the source becomes limited
after send has been performed. If the source is destroyed before the
target has received, the token becomes invalid.

The target is expected to be initialized (sev_guest_init), but not
launched state (sev_launch_start) when performing receive. Once the
target has received, it will be in a launched state and will not
need to perform the typical SEV launch commands.

Co-developed-by: Lars Bull <larsbull@google.com>
Signed-off-by: Lars Bull <larsbull@google.com>
Signed-off-by: Peter Gonda <pgonda@google.com>
Reviewed-by: Marc Orr <marcorr@google.com>
Cc: Paolo Bonzini <pbonzini@redhat.com>
Cc: Sean Christopherson <seanjc@google.com>
Cc: David Rientjes <rientjes@google.com>
Cc: Dr. David Alan Gilbert <dgilbert@redhat.com>
Cc: Brijesh Singh <brijesh.singh@amd.com>
Cc: Vitaly Kuznetsov <vkuznets@redhat.com>
Cc: Wanpeng Li <wanpengli@tencent.com>
Cc: Jim Mattson <jmattson@google.com>
Cc: Joerg Roedel <joro@8bytes.org>
Cc: Thomas Gleixner <tglx@linutronix.de>
Cc: Ingo Molnar <mingo@redhat.com>
Cc: Borislav Petkov <bp@alien8.de>
Cc: "H. Peter Anvin" <hpa@zytor.com>
Cc: kvm@vger.kernel.org
Cc: linux-kernel@vger.kernel.org
---
 .../virt/kvm/amd-memory-encryption.rst        |  43 +++
 arch/x86/kvm/svm/sev.c                        | 248 +++++++++++++++++-
 arch/x86/kvm/svm/svm.h                        |   1 +
 include/uapi/linux/kvm.h                      |  12 +
 4 files changed, 294 insertions(+), 10 deletions(-)

diff --git a/Documentation/virt/kvm/amd-memory-encryption.rst b/Documentation/virt/kvm/amd-memory-encryption.rst
index 5c081c8c7164..eac5d6e2ef11 100644
--- a/Documentation/virt/kvm/amd-memory-encryption.rst
+++ b/Documentation/virt/kvm/amd-memory-encryption.rst
@@ -427,6 +427,49 @@ issued by the hypervisor to make the guest ready for execution.
 
 Returns: 0 on success, -negative on error
 
+10. KVM_SEV_INTRA_HOST_SEND
+----------------------------------
+
+The KVM_SEV_INTRA_HOST_SEND command is used to stage the VM's SEV info
+for the purposes of migrating memory to a new local VM while using the same SEV
+key. If the source VM is destroyed before the staged info has been received by
+the target, the info is lost. Once the info has been staged, only commands
+KVM_SEV_DBG_DECRYPT, and KVM_SEV_DBG_ENCRYPT
+can be used by the source.
+
+Parameters (in): struct kvm_sev_intra_host_send
+
+Returns: 0 on success, -negative on error
+
+::
+
+    struct kvm_sev_intra_host_send {
+        __u64 info_token;    /* token referencing the staged info */
+    };
+
+11. KVM_SEV_INTRA_HOST_RECEIVE
+-------------------------------------
+
+The KVM_SEV_INTRA_HOST_RECEIVE command is used to transfer staged SEV
+info to a target VM from some source VM. SEV on the target VM should be active
+when receive is performed, but not yet launched and without any pinned memory.
+The launch commands should be skipped after receive because they should have
+already been performed on the source.
+
+Parameters (in/out): struct kvm_sev_intra_host_receive
+
+Returns: 0 on success, -negative on error
+
+::
+
+    struct kvm_sev_intra_host_receive {
+        __u64 info_token;    /* token referencing the staged info */
+        __u32 handle;        /* guest handle */
+    };
+
+On success, the 'handle' field contains the handle for this SEV guest.
+
+
 References
 ==========
 
diff --git a/arch/x86/kvm/svm/sev.c b/arch/x86/kvm/svm/sev.c
index 78fea9c4048d..03b5e690ca56 100644
--- a/arch/x86/kvm/svm/sev.c
+++ b/arch/x86/kvm/svm/sev.c
@@ -14,6 +14,7 @@
 #include <linux/psp-sev.h>
 #include <linux/pagemap.h>
 #include <linux/swap.h>
+#include <linux/random.h>
 #include <linux/misc_cgroup.h>
 #include <linux/processor.h>
 #include <linux/trace_events.h>
@@ -57,6 +58,8 @@ module_param_named(sev_es, sev_es_enabled, bool, 0444);
 #define sev_es_enabled false
 #endif /* CONFIG_KVM_AMD_SEV */
 
+#define MAX_RAND_RETRY    3
+
 static u8 sev_enc_bit;
 static DECLARE_RWSEM(sev_deactivate_lock);
 static DEFINE_MUTEX(sev_bitmap_lock);
@@ -74,6 +77,22 @@ struct enc_region {
 	unsigned long size;
 };
 
+struct sev_info_migration_node {
+	struct hlist_node hnode;
+	u64 token;
+	bool valid;
+
+	unsigned int asid;
+	unsigned int handle;
+	unsigned long pages_locked;
+	struct list_head regions_list;
+	struct misc_cg *misc_cg;
+};
+
+#define SEV_INFO_MIGRATION_HASH_BITS    7
+static DEFINE_HASHTABLE(sev_info_migration_hash, SEV_INFO_MIGRATION_HASH_BITS);
+static DEFINE_SPINLOCK(sev_info_migration_hash_lock);
+
 /* Called with the sev_bitmap_lock held, or on shutdown  */
 static int sev_flush_asids(int min_asid, int max_asid)
 {
@@ -1104,6 +1123,160 @@ static int sev_get_attestation_report(struct kvm *kvm, struct kvm_sev_cmd *argp)
 	return ret;
 }
 
+static struct sev_info_migration_node *find_migration_info(unsigned long token)
+{
+	struct sev_info_migration_node *entry;
+
+	hash_for_each_possible(sev_info_migration_hash, entry, hnode, token) {
+		if (entry->token == token)
+			return entry;
+	}
+
+	return NULL;
+}
+
+/*
+ * Places @entry into the |sev_info_migration_hash|. Returns 0 if successful
+ * and ownership of @entry is transferred to the hashmap.
+ */
+static int place_migration_node(struct sev_info_migration_node *entry)
+{
+	int ret = -EFAULT;
+
+	spin_lock(&sev_info_migration_hash_lock);
+	if (find_migration_info(entry->token))
+		goto out;
+
+	entry->valid = true;
+
+	hash_add(sev_info_migration_hash, &entry->hnode, entry->token);
+	ret = 0;
+
+out:
+	spin_unlock(&sev_info_migration_hash_lock);
+	return ret;
+}
+
+static int sev_intra_host_send(struct kvm *kvm, struct kvm_sev_cmd *argp)
+{
+	struct kvm_sev_info *sev = &to_kvm_svm(kvm)->sev_info;
+	struct sev_info_migration_node *entry;
+	struct kvm_sev_intra_host_send params;
+	int ret = -EFAULT;
+
+	if (!sev_guest(kvm))
+		return -ENOTTY;
+
+	if (sev->es_active)
+		return -EPERM;
+
+	if (sev->handle == 0)
+		return -EPERM;
+
+	if (sev->info_token != 0)
+		return -EEXIST;
+
+	if (copy_from_user(&params, (void __user *)(uintptr_t)argp->data,
+			   sizeof(params)))
+		return -EFAULT;
+
+	if (params.info_token == 0)
+		return -EINVAL;
+
+	entry = kzalloc(sizeof(*entry), GFP_KERNEL);
+	if (!entry)
+		return -ENOMEM;
+
+	entry->asid = sev->asid;
+	entry->handle = sev->handle;
+	entry->pages_locked = sev->pages_locked;
+	entry->misc_cg = sev->misc_cg;
+	entry->token = params.info_token;
+
+	INIT_LIST_HEAD(&entry->regions_list);
+	list_replace_init(&sev->regions_list, &entry->regions_list);
+
+	if (place_migration_node(entry))
+		goto e_listdel;
+
+	sev->info_token = entry->token;
+
+	return 0;
+
+e_listdel:
+	list_replace_init(&entry->regions_list, &sev->regions_list);
+
+	kfree(entry);
+
+	return ret;
+}
+
+static int sev_intra_host_receive(struct kvm *kvm,
+					struct kvm_sev_cmd *argp)
+{
+	struct kvm_sev_info *sev = &to_kvm_svm(kvm)->sev_info;
+	struct sev_info_migration_node *entry;
+	struct kvm_sev_intra_host_receive params;
+	struct kvm_sev_info old_info;
+	void __user *user_param = (void __user *)(uintptr_t)argp->data;
+
+	if (!sev_guest(kvm))
+		return -ENOTTY;
+
+	if (sev->es_active)
+		return -EPERM;
+
+	if (sev->handle != 0)
+		return -EPERM;
+
+	if (!list_empty(&sev->regions_list))
+		return -EPERM;
+
+	if (copy_from_user(&params, user_param, sizeof(params)))
+		return -EFAULT;
+
+	spin_lock(&sev_info_migration_hash_lock);
+	entry = find_migration_info(params.info_token);
+	if (!entry || !entry->valid)
+		goto err_unlock;
+
+	memcpy(&old_info, sev, sizeof(old_info));
+
+	/*
+	 * The source VM always frees @entry On the target we simply
+	 * mark the token as invalid to notify the source the sev info
+	 * has been moved successfully.
+	 */
+	entry->valid = false;
+	sev->active = true;
+	sev->asid = entry->asid;
+	sev->handle = entry->handle;
+	sev->pages_locked = entry->pages_locked;
+	sev->misc_cg = entry->misc_cg;
+
+	INIT_LIST_HEAD(&sev->regions_list);
+	list_replace_init(&entry->regions_list, &sev->regions_list);
+
+	spin_unlock(&sev_info_migration_hash_lock);
+
+	params.handle = sev->handle;
+
+	if (copy_to_user(user_param, &params, sizeof(params))) {
+		list_replace_init(&sev->regions_list, &entry->regions_list);
+		entry->valid = true;
+		memcpy(sev, &old_info, sizeof(*sev));
+		return -EFAULT;
+	}
+
+	sev_asid_free(&old_info);
+	return 0;
+
+err_unlock:
+	spin_unlock(&sev_info_migration_hash_lock);
+
+	return -EFAULT;
+}
+
 /* Userspace wants to query session length. */
 static int
 __sev_send_start_query_session_length(struct kvm *kvm, struct kvm_sev_cmd *argp,
@@ -1499,6 +1672,19 @@ static int sev_receive_finish(struct kvm *kvm, struct kvm_sev_cmd *argp)
 	return sev_issue_cmd(kvm, SEV_CMD_RECEIVE_FINISH, &data, &argp->error);
 }
 
+static bool is_intra_host_mig_active(struct kvm *kvm)
+{
+	return !!to_kvm_svm(kvm)->sev_info.info_token;
+}
+
+static bool cmd_allowed_during_intra_host_mig(u32 cmd_id)
+{
+	if (cmd_id == KVM_SEV_DBG_ENCRYPT || cmd_id == KVM_SEV_DBG_DECRYPT)
+		return true;
+
+	return false;
+}
+
 int svm_mem_enc_op(struct kvm *kvm, void __user *argp)
 {
 	struct kvm_sev_cmd sev_cmd;
@@ -1521,6 +1707,17 @@ int svm_mem_enc_op(struct kvm *kvm, void __user *argp)
 		goto out;
 	}
 
+	/*
+	 * If this VM has started exporting its SEV contents to another VM,
+	 * it's not allowed to do any more SEV operations that may modify the
+	 * SEV state.
+	 */
+	if (is_intra_host_mig_active(kvm) &&
+	    !cmd_allowed_during_intra_host_mig(sev_cmd.id)) {
+		r = -EPERM;
+		goto out;
+	}
+
 	switch (sev_cmd.id) {
 	case KVM_SEV_ES_INIT:
 		if (!sev_es_enabled) {
@@ -1561,6 +1758,12 @@ int svm_mem_enc_op(struct kvm *kvm, void __user *argp)
 	case KVM_SEV_GET_ATTESTATION_REPORT:
 		r = sev_get_attestation_report(kvm, &sev_cmd);
 		break;
+	case KVM_SEV_INTRA_HOST_SEND:
+		r = sev_intra_host_send(kvm, &sev_cmd);
+		break;
+	case KVM_SEV_INTRA_HOST_RECEIVE:
+		r = sev_intra_host_receive(kvm, &sev_cmd);
+		break;
 	case KVM_SEV_SEND_START:
 		r = sev_send_start(kvm, &sev_cmd);
 		break;
@@ -1794,6 +1997,7 @@ static void unregister_enc_regions(struct kvm *kvm,
 void sev_vm_destroy(struct kvm *kvm)
 {
 	struct kvm_sev_info *sev = &to_kvm_svm(kvm)->sev_info;
+	struct sev_info_migration_node *mig_entry;
 
 	if (!sev_guest(kvm))
 		return;
@@ -1804,25 +2008,49 @@ void sev_vm_destroy(struct kvm *kvm)
 		return;
 	}
 
-	mutex_lock(&kvm->lock);
-
 	/*
-	 * Ensure that all guest tagged cache entries are flushed before
-	 * releasing the pages back to the system for use. CLFLUSH will
-	 * not do this, so issue a WBINVD.
+	 * If userspace has requested that we migrate the SEV info to a new VM,
+	 * then we own and must remove an entry node in the tracking data
+	 * structure. Whether we clean up the data in our SEV info struct and
+	 * entry node depends on whether userspace has done the migration,
+	 * which transfers ownership to a new VM. We can identify that
+	 * migration has occurred by checking if the node is marked invalid.
 	 */
-	wbinvd_on_all_cpus();
+	if (sev->info_token != 0) {
+		spin_lock(&sev_info_migration_hash_lock);
+		mig_entry = find_migration_info(sev->info_token);
+		if (!WARN_ON(!mig_entry))
+			hash_del(&mig_entry->hnode);
+		spin_unlock(&sev_info_migration_hash_lock);
+	} else
+		mig_entry = NULL;
+
+	mutex_lock(&kvm->lock);
 
 	/*
-	 * if userspace was terminated before unregistering the memory regions
-	 * then lets unpin all the registered memory.
+	 * Adding memory regions after a intra host send has started
+	 * is dangerous.
 	 */
+	WARN_ON(sev->info_token && !list_empty(&sev->regions_list));
 	unregister_enc_regions(kvm, &sev->regions_list);
 
+	if (mig_entry)
+		unregister_enc_regions(kvm, &mig_entry->regions_list);
+
 	mutex_unlock(&kvm->lock);
 
-	sev_unbind_asid(kvm, sev->handle);
-	sev_asid_free(sev);
+	/*
+	 * Ensure that all guest tagged cache entries are flushed before
+	 * releasing the pages back to the system for use. CLFLUSH will
+	 * not do this, so issue a WBINVD.
+	 */
+	wbinvd_on_all_cpus();
+	if (!mig_entry || !mig_entry->valid) {
+		sev_unbind_asid(kvm, sev->handle);
+		sev_asid_free(sev);
+	}
+
+	kfree(mig_entry);
 }
 
 void __init sev_set_cpu_caps(void)
diff --git a/arch/x86/kvm/svm/svm.h b/arch/x86/kvm/svm/svm.h
index f89b623bb591..559ce44682a8 100644
--- a/arch/x86/kvm/svm/svm.h
+++ b/arch/x86/kvm/svm/svm.h
@@ -79,6 +79,7 @@ struct kvm_sev_info {
 	u64 ap_jump_table;	/* SEV-ES AP Jump Table address */
 	struct kvm *enc_context_owner; /* Owner of copied encryption context */
 	struct misc_cg *misc_cg; /* For misc cgroup accounting */
+	u64 info_token; /* Token for SEV info intra host migration */
 };
 
 struct kvm_svm {
diff --git a/include/uapi/linux/kvm.h b/include/uapi/linux/kvm.h
index 68c9e6d8bbda..01a42a7134af 100644
--- a/include/uapi/linux/kvm.h
+++ b/include/uapi/linux/kvm.h
@@ -1709,6 +1709,9 @@ enum sev_cmd_id {
 	KVM_SEV_GET_ATTESTATION_REPORT,
 	/* Guest Migration Extension */
 	KVM_SEV_SEND_CANCEL,
+	/* Intra host migration commands */
+	KVM_SEV_INTRA_HOST_SEND,
+	KVM_SEV_INTRA_HOST_RECEIVE,
 
 	KVM_SEV_NR_MAX,
 };
@@ -1806,6 +1809,15 @@ struct kvm_sev_receive_update_data {
 	__u32 trans_len;
 };
 
+struct kvm_sev_intra_host_send {
+	__u64 info_token;
+};
+
+struct kvm_sev_intra_host_receive {
+	__u64 info_token;
+	__u32 handle;
+};
+
 #define KVM_DEV_ASSIGN_ENABLE_IOMMU	(1 << 0)
 #define KVM_DEV_ASSIGN_PCI_2_3		(1 << 1)
 #define KVM_DEV_ASSIGN_MASK_INTX	(1 << 2)
-- 
2.32.0.93.g670b81a890-goog


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

* [PATCH 3/3 V2] KVM, SEV: Add support for SEV-ES intra host migration
  2021-07-14 16:01 [PATCH 0/3 V2] Add AMD SEV and SEV-ES intra host migration support Peter Gonda
  2021-07-14 16:01 ` [PATCH 1/3 V2] KVM, SEV: Refactor out function for unregistering encrypted regions Peter Gonda
  2021-07-14 16:01 ` [PATCH 2/3 V2] KVM, SEV: Add support for SEV intra host migration Peter Gonda
@ 2021-07-14 16:01 ` Peter Gonda
  2021-07-14 19:50   ` Brijesh Singh
  2 siblings, 1 reply; 10+ messages in thread
From: Peter Gonda @ 2021-07-14 16:01 UTC (permalink / raw)
  To: pgonda
  Cc: Marc Orr, Paolo Bonzini, Sean Christopherson, David Rientjes,
	Dr . David Alan Gilbert, Brijesh Singh, Vitaly Kuznetsov,
	Wanpeng Li, Jim Mattson, Joerg Roedel, Thomas Gleixner,
	Ingo Molnar, Borislav Petkov, H. Peter Anvin, kvm, linux-kernel

For SEV-ES to work with intra host migration the VMSAs, GHCB metadata,
and other SEV-ES info needs to be preserved along with the guest's
memory.

Signed-off-by: Peter Gonda <pgonda@google.com>
Reviewed-by: Marc Orr <marcorr@google.com>
Cc: Paolo Bonzini <pbonzini@redhat.com>
Cc: Sean Christopherson <seanjc@google.com>
Cc: David Rientjes <rientjes@google.com>
Cc: Dr. David Alan Gilbert <dgilbert@redhat.com>
Cc: Brijesh Singh <brijesh.singh@amd.com>
Cc: Vitaly Kuznetsov <vkuznets@redhat.com>
Cc: Wanpeng Li <wanpengli@tencent.com>
Cc: Jim Mattson <jmattson@google.com>
Cc: Joerg Roedel <joro@8bytes.org>
Cc: Thomas Gleixner <tglx@linutronix.de>
Cc: Ingo Molnar <mingo@redhat.com>
Cc: Borislav Petkov <bp@alien8.de>
Cc: "H. Peter Anvin" <hpa@zytor.com>
Cc: kvm@vger.kernel.org
Cc: linux-kernel@vger.kernel.org
---
 arch/x86/kvm/svm/sev.c | 142 ++++++++++++++++++++++++++++++++++++++---
 1 file changed, 132 insertions(+), 10 deletions(-)

diff --git a/arch/x86/kvm/svm/sev.c b/arch/x86/kvm/svm/sev.c
index 03b5e690ca56..e292e2cd7c99 100644
--- a/arch/x86/kvm/svm/sev.c
+++ b/arch/x86/kvm/svm/sev.c
@@ -77,6 +77,19 @@ struct enc_region {
 	unsigned long size;
 };
 
+struct vmsa_node {
+	struct list_head list;
+	int vcpu_id;
+	struct vmcb_save_area *vmsa;
+	struct ghcb *ghcb;
+	u64 ghcb_gpa;
+
+	void *ghcb_sa;
+	u64 ghcb_sa_len;
+	bool ghcb_sa_sync;
+	bool ghcb_sa_free;
+};
+
 struct sev_info_migration_node {
 	struct hlist_node hnode;
 	u64 token;
@@ -87,6 +100,11 @@ struct sev_info_migration_node {
 	unsigned long pages_locked;
 	struct list_head regions_list;
 	struct misc_cg *misc_cg;
+
+	/* The following fields are for SEV-ES guests */
+	bool es_enabled;
+	struct list_head vmsa_list;
+	u64 ap_jump_table;
 };
 
 #define SEV_INFO_MIGRATION_HASH_BITS    7
@@ -1157,6 +1175,84 @@ static int place_migration_node(struct sev_info_migration_node *entry)
 	return ret;
 }
 
+static void process_vmsa_list(struct kvm *kvm, struct list_head *vmsa_list)
+{
+	struct vmsa_node *vmsa_node, *q;
+	struct kvm_vcpu *vcpu;
+	struct vcpu_svm *svm;
+
+	lockdep_assert_held(&kvm->lock);
+
+	/*
+	 * Move any stashed VMSAs back to their respective VMCBs and delete
+	 * those nodes.
+	 */
+	list_for_each_entry_safe(vmsa_node, q, vmsa_list, list) {
+		vcpu = kvm_get_vcpu_by_id(kvm, vmsa_node->vcpu_id);
+		if (WARN_ON(!vcpu))
+			continue;
+
+		svm = to_svm(vcpu);
+		svm->vmsa = vmsa_node->vmsa;
+		svm->ghcb = vmsa_node->ghcb;
+		svm->vmcb->control.ghcb_gpa = vmsa_node->ghcb_gpa;
+		svm->vcpu.arch.guest_state_protected = true;
+		svm->vmcb->control.vmsa_pa = __pa(svm->vmsa);
+		svm->ghcb_sa = vmsa_node->ghcb_sa;
+		svm->ghcb_sa_len = vmsa_node->ghcb_sa_len;
+		svm->ghcb_sa_sync = vmsa_node->ghcb_sa_sync;
+		svm->ghcb_sa_free = vmsa_node->ghcb_sa_free;
+
+		list_del(&vmsa_node->list);
+		kfree(vmsa_node);
+	}
+}
+
+static int create_vmsa_list(struct kvm *kvm,
+			    struct sev_info_migration_node *entry)
+{
+	int i;
+	const int num_vcpus = atomic_read(&kvm->online_vcpus);
+	struct vmsa_node *node;
+	struct kvm_vcpu *vcpu;
+	struct vcpu_svm *svm;
+
+	INIT_LIST_HEAD(&entry->vmsa_list);
+	for (i = 0; i < num_vcpus; ++i) {
+		node = kzalloc(sizeof(*node), GFP_KERNEL);
+		if (!node)
+			goto e_freelist;
+
+		vcpu = kvm->vcpus[i];
+		if (!vcpu->arch.guest_state_protected)
+			goto e_freelist;
+
+		node->vcpu_id = vcpu->vcpu_id;
+		svm = to_svm(vcpu);
+		node->vmsa = svm->vmsa;
+		svm->vmsa = NULL;
+		node->ghcb = svm->ghcb;
+		svm->ghcb = NULL;
+		node->ghcb_gpa = svm->vmcb->control.ghcb_gpa;
+		node->ghcb_sa = svm->ghcb_sa;
+		svm->ghcb_sa = NULL;
+		node->ghcb_sa_len = svm->ghcb_sa_len;
+		svm->ghcb_sa_len = 0;
+		node->ghcb_sa_sync = svm->ghcb_sa_sync;
+		svm->ghcb_sa_sync = false;
+		node->ghcb_sa_free = svm->ghcb_sa_free;
+		svm->ghcb_sa_free = false;
+
+		list_add_tail(&node->list, &entry->vmsa_list);
+	}
+
+	return 0;
+
+e_freelist:
+	process_vmsa_list(kvm, &entry->vmsa_list);
+	return -1;
+}
+
 static int sev_intra_host_send(struct kvm *kvm, struct kvm_sev_cmd *argp)
 {
 	struct kvm_sev_info *sev = &to_kvm_svm(kvm)->sev_info;
@@ -1167,9 +1263,6 @@ static int sev_intra_host_send(struct kvm *kvm, struct kvm_sev_cmd *argp)
 	if (!sev_guest(kvm))
 		return -ENOTTY;
 
-	if (sev->es_active)
-		return -EPERM;
-
 	if (sev->handle == 0)
 		return -EPERM;
 
@@ -1196,13 +1289,28 @@ static int sev_intra_host_send(struct kvm *kvm, struct kvm_sev_cmd *argp)
 	INIT_LIST_HEAD(&entry->regions_list);
 	list_replace_init(&sev->regions_list, &entry->regions_list);
 
+	if (sev_es_guest(kvm)) {
+		/*
+		 * If this is an ES guest, we need to move each VMCB's VMSA into a
+		 * list for migration.
+		 */
+		entry->es_enabled = true;
+		entry->ap_jump_table = sev->ap_jump_table;
+		if (create_vmsa_list(kvm, entry))
+			goto e_listdel;
+	}
+
 	if (place_migration_node(entry))
-		goto e_listdel;
+		goto e_vmsadel;
 
 	sev->info_token = entry->token;
 
 	return 0;
 
+e_vmsadel:
+	if (sev_es_guest(kvm))
+		process_vmsa_list(kvm, &entry->vmsa_list);
+
 e_listdel:
 	list_replace_init(&entry->regions_list, &sev->regions_list);
 
@@ -1223,9 +1331,6 @@ static int sev_intra_host_receive(struct kvm *kvm,
 	if (!sev_guest(kvm))
 		return -ENOTTY;
 
-	if (sev->es_active)
-		return -EPERM;
-
 	if (sev->handle != 0)
 		return -EPERM;
 
@@ -1242,6 +1347,13 @@ static int sev_intra_host_receive(struct kvm *kvm,
 
 	memcpy(&old_info, sev, sizeof(old_info));
 
+	if (entry->es_enabled) {
+		process_vmsa_list(kvm, &entry->vmsa_list);
+
+		sev->es_active = true;
+		sev->ap_jump_table = entry->ap_jump_table;
+	}
+
 	/*
 	 * The source VM always frees @entry On the target we simply
 	 * mark the token as invalid to notify the source the sev info
@@ -2034,8 +2146,16 @@ void sev_vm_destroy(struct kvm *kvm)
 	WARN_ON(sev->info_token && !list_empty(&sev->regions_list));
 	unregister_enc_regions(kvm, &sev->regions_list);
 
-	if (mig_entry)
+	if (mig_entry) {
+		/*
+		 * If there are any saved VMSAs, restore them so they can be
+		 * destructed through the normal path.
+		 */
+		if (mig_entry->es_enabled)
+			process_vmsa_list(kvm, &mig_entry->vmsa_list);
+
 		unregister_enc_regions(kvm, &mig_entry->regions_list);
+	}
 
 	mutex_unlock(&kvm->lock);
 
@@ -2222,9 +2342,11 @@ void sev_free_vcpu(struct kvm_vcpu *vcpu)
 
 	svm = to_svm(vcpu);
 
-	if (vcpu->arch.guest_state_protected)
+	if (svm->ghcb && vcpu->arch.guest_state_protected)
 		sev_flush_guest_memory(svm, svm->vmsa, PAGE_SIZE);
-	__free_page(virt_to_page(svm->vmsa));
+
+	if (svm->vmsa)
+		__free_page(virt_to_page(svm->vmsa));
 
 	if (svm->ghcb_sa_free)
 		kfree(svm->ghcb_sa);
-- 
2.32.0.93.g670b81a890-goog


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

* Re: [PATCH 2/3 V2] KVM, SEV: Add support for SEV intra host migration
  2021-07-14 16:01 ` [PATCH 2/3 V2] KVM, SEV: Add support for SEV intra host migration Peter Gonda
@ 2021-07-14 19:22   ` Brijesh Singh
  0 siblings, 0 replies; 10+ messages in thread
From: Brijesh Singh @ 2021-07-14 19:22 UTC (permalink / raw)
  To: Peter Gonda
  Cc: brijesh.singh, Lars Bull, Marc Orr, Paolo Bonzini,
	Sean Christopherson, David Rientjes, Dr . David Alan Gilbert,
	Vitaly Kuznetsov, Wanpeng Li, Jim Mattson, Joerg Roedel,
	Thomas Gleixner, Ingo Molnar, Borislav Petkov, H. Peter Anvin,
	kvm, linux-kernel



On 7/14/21 11:01 AM, Peter Gonda wrote:
> For SEV to work with intra host migration, contents of the SEV info struct
> such as the ASID (used to index the encryption key in the AMD SP) and
> the list of memory regions need to be transferred to the target VM.
> This change adds commands for sending and receiving the sev info.
> 
> To avoid exposing this internal state to userspace and prevent other
> processes from importing state they shouldn't have access to, the send
> returns a token to userspace that is handed off to the target VM. The
> target passes in this token to receive the sent state. The token is only
> valid for one-time use. Functionality on the source becomes limited
> after send has been performed. If the source is destroyed before the
> target has received, the token becomes invalid.
> 
> The target is expected to be initialized (sev_guest_init), but not
> launched state (sev_launch_start) when performing receive. Once the
> target has received, it will be in a launched state and will not
> need to perform the typical SEV launch commands.
> 
> Co-developed-by: Lars Bull <larsbull@google.com>
> Signed-off-by: Lars Bull <larsbull@google.com>
> Signed-off-by: Peter Gonda <pgonda@google.com>
> Reviewed-by: Marc Orr <marcorr@google.com>
> Cc: Paolo Bonzini <pbonzini@redhat.com>
> Cc: Sean Christopherson <seanjc@google.com>
> Cc: David Rientjes <rientjes@google.com>
> Cc: Dr. David Alan Gilbert <dgilbert@redhat.com>
> Cc: Brijesh Singh <brijesh.singh@amd.com>
> Cc: Vitaly Kuznetsov <vkuznets@redhat.com>
> Cc: Wanpeng Li <wanpengli@tencent.com>
> Cc: Jim Mattson <jmattson@google.com>
> Cc: Joerg Roedel <joro@8bytes.org>
> Cc: Thomas Gleixner <tglx@linutronix.de>
> Cc: Ingo Molnar <mingo@redhat.com>
> Cc: Borislav Petkov <bp@alien8.de>
> Cc: "H. Peter Anvin" <hpa@zytor.com>
> Cc: kvm@vger.kernel.org
> Cc: linux-kernel@vger.kernel.org

Reviewed-by: Brijesh Singh <brijesh.singh@amd.com>

thanks

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

* Re: [PATCH 3/3 V2] KVM, SEV: Add support for SEV-ES intra host migration
  2021-07-14 16:01 ` [PATCH 3/3 V2] KVM, SEV: Add support for SEV-ES " Peter Gonda
@ 2021-07-14 19:50   ` Brijesh Singh
  0 siblings, 0 replies; 10+ messages in thread
From: Brijesh Singh @ 2021-07-14 19:50 UTC (permalink / raw)
  To: Peter Gonda
  Cc: brijesh.singh, Marc Orr, Paolo Bonzini, Sean Christopherson,
	David Rientjes, Dr . David Alan Gilbert, Vitaly Kuznetsov,
	Wanpeng Li, Jim Mattson, Joerg Roedel, Thomas Gleixner,
	Ingo Molnar, Borislav Petkov, H. Peter Anvin, kvm, linux-kernel



On 7/14/21 11:01 AM, Peter Gonda wrote:
> For SEV-ES to work with intra host migration the VMSAs, GHCB metadata,
> and other SEV-ES info needs to be preserved along with the guest's
> memory.
> 
> Signed-off-by: Peter Gonda <pgonda@google.com>
> Reviewed-by: Marc Orr <marcorr@google.com>
> Cc: Paolo Bonzini <pbonzini@redhat.com>
> Cc: Sean Christopherson <seanjc@google.com>
> Cc: David Rientjes <rientjes@google.com>
> Cc: Dr. David Alan Gilbert <dgilbert@redhat.com>
> Cc: Brijesh Singh <brijesh.singh@amd.com>
> Cc: Vitaly Kuznetsov <vkuznets@redhat.com>
> Cc: Wanpeng Li <wanpengli@tencent.com>
> Cc: Jim Mattson <jmattson@google.com>
> Cc: Joerg Roedel <joro@8bytes.org>
> Cc: Thomas Gleixner <tglx@linutronix.de>
> Cc: Ingo Molnar <mingo@redhat.com>
> Cc: Borislav Petkov <bp@alien8.de>
> Cc: "H. Peter Anvin" <hpa@zytor.com>
> Cc: kvm@vger.kernel.org
> Cc: linux-kernel@vger.kernel.org
> ---
>   arch/x86/kvm/svm/sev.c | 142 ++++++++++++++++++++++++++++++++++++++---
>   1 file changed, 132 insertions(+), 10 deletions(-)
> 

Reviewed-by: Brijesh Singh <brijesh.singh@amd.com>

thanks

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

* Re: [PATCH 3/3 V2] KVM, SEV: Add support for SEV-ES intra host migration
  2021-07-14 16:01 ` [PATCH 3/3 V2] KVM, SEV: Add support for SEV-ES " Peter Gonda
@ 2021-07-19  7:53 ` Dan Carpenter
  0 siblings, 0 replies; 10+ messages in thread
From: kernel test robot @ 2021-07-14 21:35 UTC (permalink / raw)
  To: kbuild

[-- Attachment #1: Type: text/plain, Size: 4852 bytes --]

CC: kbuild-all(a)lists.01.org
In-Reply-To: <20210714160143.2116583-4-pgonda@google.com>
References: <20210714160143.2116583-4-pgonda@google.com>
TO: Peter Gonda <pgonda@google.com>
TO: pgonda(a)google.com
CC: Marc Orr <marcorr@google.com>
CC: Paolo Bonzini <pbonzini@redhat.com>
CC: Sean Christopherson <seanjc@google.com>
CC: David Rientjes <rientjes@google.com>
CC: "Dr . David Alan Gilbert" <dgilbert@redhat.com>
CC: Brijesh Singh <brijesh.singh@amd.com>
CC: Vitaly Kuznetsov <vkuznets@redhat.com>
CC: Wanpeng Li <wanpengli@tencent.com>
CC: Jim Mattson <jmattson@google.com>

Hi Peter,

Thank you for the patch! Perhaps something to improve:

[auto build test WARNING on 7caa04b36f204a01dac65582b71d26d190a1e022]

url:    https://github.com/0day-ci/linux/commits/Peter-Gonda/Add-AMD-SEV-and-SEV-ES-intra-host-migration-support/20210715-000317
base:   7caa04b36f204a01dac65582b71d26d190a1e022
:::::: branch date: 6 hours ago
:::::: commit date: 6 hours ago
config: x86_64-randconfig-m001-20210714 (attached as .config)
compiler: gcc-10 (Debian 10.2.1-6) 10.2.1 20210110

If you fix the issue, kindly add following tag as appropriate
Reported-by: kernel test robot <lkp@intel.com>
Reported-by: Dan Carpenter <dan.carpenter@oracle.com>

New smatch warnings:
arch/x86/kvm/svm/sev.c:1253 create_vmsa_list() warn: possible memory leak of 'node'

Old smatch warnings:
arch/x86/kvm/svm/sev.c:1607 sev_send_update_data() warn: maybe return -EFAULT instead of the bytes remaining?

vim +/node +1253 arch/x86/kvm/svm/sev.c

bec8b6143b5a9c Peter Gonda 2021-07-14  1210  
bec8b6143b5a9c Peter Gonda 2021-07-14  1211  static int create_vmsa_list(struct kvm *kvm,
bec8b6143b5a9c Peter Gonda 2021-07-14  1212  			    struct sev_info_migration_node *entry)
bec8b6143b5a9c Peter Gonda 2021-07-14  1213  {
bec8b6143b5a9c Peter Gonda 2021-07-14  1214  	int i;
bec8b6143b5a9c Peter Gonda 2021-07-14  1215  	const int num_vcpus = atomic_read(&kvm->online_vcpus);
bec8b6143b5a9c Peter Gonda 2021-07-14  1216  	struct vmsa_node *node;
bec8b6143b5a9c Peter Gonda 2021-07-14  1217  	struct kvm_vcpu *vcpu;
bec8b6143b5a9c Peter Gonda 2021-07-14  1218  	struct vcpu_svm *svm;
bec8b6143b5a9c Peter Gonda 2021-07-14  1219  
bec8b6143b5a9c Peter Gonda 2021-07-14  1220  	INIT_LIST_HEAD(&entry->vmsa_list);
bec8b6143b5a9c Peter Gonda 2021-07-14  1221  	for (i = 0; i < num_vcpus; ++i) {
bec8b6143b5a9c Peter Gonda 2021-07-14  1222  		node = kzalloc(sizeof(*node), GFP_KERNEL);
bec8b6143b5a9c Peter Gonda 2021-07-14  1223  		if (!node)
bec8b6143b5a9c Peter Gonda 2021-07-14  1224  			goto e_freelist;
bec8b6143b5a9c Peter Gonda 2021-07-14  1225  
bec8b6143b5a9c Peter Gonda 2021-07-14  1226  		vcpu = kvm->vcpus[i];
bec8b6143b5a9c Peter Gonda 2021-07-14  1227  		if (!vcpu->arch.guest_state_protected)
bec8b6143b5a9c Peter Gonda 2021-07-14  1228  			goto e_freelist;
bec8b6143b5a9c Peter Gonda 2021-07-14  1229  
bec8b6143b5a9c Peter Gonda 2021-07-14  1230  		node->vcpu_id = vcpu->vcpu_id;
bec8b6143b5a9c Peter Gonda 2021-07-14  1231  		svm = to_svm(vcpu);
bec8b6143b5a9c Peter Gonda 2021-07-14  1232  		node->vmsa = svm->vmsa;
bec8b6143b5a9c Peter Gonda 2021-07-14  1233  		svm->vmsa = NULL;
bec8b6143b5a9c Peter Gonda 2021-07-14  1234  		node->ghcb = svm->ghcb;
bec8b6143b5a9c Peter Gonda 2021-07-14  1235  		svm->ghcb = NULL;
bec8b6143b5a9c Peter Gonda 2021-07-14  1236  		node->ghcb_gpa = svm->vmcb->control.ghcb_gpa;
bec8b6143b5a9c Peter Gonda 2021-07-14  1237  		node->ghcb_sa = svm->ghcb_sa;
bec8b6143b5a9c Peter Gonda 2021-07-14  1238  		svm->ghcb_sa = NULL;
bec8b6143b5a9c Peter Gonda 2021-07-14  1239  		node->ghcb_sa_len = svm->ghcb_sa_len;
bec8b6143b5a9c Peter Gonda 2021-07-14  1240  		svm->ghcb_sa_len = 0;
bec8b6143b5a9c Peter Gonda 2021-07-14  1241  		node->ghcb_sa_sync = svm->ghcb_sa_sync;
bec8b6143b5a9c Peter Gonda 2021-07-14  1242  		svm->ghcb_sa_sync = false;
bec8b6143b5a9c Peter Gonda 2021-07-14  1243  		node->ghcb_sa_free = svm->ghcb_sa_free;
bec8b6143b5a9c Peter Gonda 2021-07-14  1244  		svm->ghcb_sa_free = false;
bec8b6143b5a9c Peter Gonda 2021-07-14  1245  
bec8b6143b5a9c Peter Gonda 2021-07-14  1246  		list_add_tail(&node->list, &entry->vmsa_list);
bec8b6143b5a9c Peter Gonda 2021-07-14  1247  	}
bec8b6143b5a9c Peter Gonda 2021-07-14  1248  
bec8b6143b5a9c Peter Gonda 2021-07-14  1249  	return 0;
bec8b6143b5a9c Peter Gonda 2021-07-14  1250  
bec8b6143b5a9c Peter Gonda 2021-07-14  1251  e_freelist:
bec8b6143b5a9c Peter Gonda 2021-07-14  1252  	process_vmsa_list(kvm, &entry->vmsa_list);
bec8b6143b5a9c Peter Gonda 2021-07-14 @1253  	return -1;
bec8b6143b5a9c Peter Gonda 2021-07-14  1254  }
bec8b6143b5a9c Peter Gonda 2021-07-14  1255  

---
0-DAY CI Kernel Test Service, Intel Corporation
https://lists.01.org/hyperkitty/list/kbuild-all(a)lists.01.org

[-- Attachment #2: config.gz --]
[-- Type: application/gzip, Size: 38240 bytes --]

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

* Re: [PATCH 3/3 V2] KVM, SEV: Add support for SEV-ES intra host migration
@ 2021-07-19  7:53 ` Dan Carpenter
  0 siblings, 0 replies; 10+ messages in thread
From: Dan Carpenter @ 2021-07-19  7:53 UTC (permalink / raw)
  To: kbuild-all

[-- Attachment #1: Type: text/plain, Size: 4001 bytes --]

Hi Peter,

url:    https://github.com/0day-ci/linux/commits/Peter-Gonda/Add-AMD-SEV-and-SEV-ES-intra-host-migration-support/20210715-000317
base:   7caa04b36f204a01dac65582b71d26d190a1e022
config: x86_64-randconfig-m001-20210714 (attached as .config)
compiler: gcc-10 (Debian 10.2.1-6) 10.2.1 20210110

If you fix the issue, kindly add following tag as appropriate
Reported-by: kernel test robot <lkp@intel.com>
Reported-by: Dan Carpenter <dan.carpenter@oracle.com>

New smatch warnings:
arch/x86/kvm/svm/sev.c:1253 create_vmsa_list() warn: possible memory leak of 'node'

Old smatch warnings:
arch/x86/kvm/svm/sev.c:1607 sev_send_update_data() warn: maybe return -EFAULT instead of the bytes remaining?

vim +/node +1253 arch/x86/kvm/svm/sev.c

bec8b6143b5a9c Peter Gonda 2021-07-14  1211  static int create_vmsa_list(struct kvm *kvm,
bec8b6143b5a9c Peter Gonda 2021-07-14  1212  			    struct sev_info_migration_node *entry)
bec8b6143b5a9c Peter Gonda 2021-07-14  1213  {
bec8b6143b5a9c Peter Gonda 2021-07-14  1214  	int i;
bec8b6143b5a9c Peter Gonda 2021-07-14  1215  	const int num_vcpus = atomic_read(&kvm->online_vcpus);
bec8b6143b5a9c Peter Gonda 2021-07-14  1216  	struct vmsa_node *node;
bec8b6143b5a9c Peter Gonda 2021-07-14  1217  	struct kvm_vcpu *vcpu;
bec8b6143b5a9c Peter Gonda 2021-07-14  1218  	struct vcpu_svm *svm;
bec8b6143b5a9c Peter Gonda 2021-07-14  1219  
bec8b6143b5a9c Peter Gonda 2021-07-14  1220  	INIT_LIST_HEAD(&entry->vmsa_list);
bec8b6143b5a9c Peter Gonda 2021-07-14  1221  	for (i = 0; i < num_vcpus; ++i) {
bec8b6143b5a9c Peter Gonda 2021-07-14  1222  		node = kzalloc(sizeof(*node), GFP_KERNEL);
bec8b6143b5a9c Peter Gonda 2021-07-14  1223  		if (!node)
bec8b6143b5a9c Peter Gonda 2021-07-14  1224  			goto e_freelist;
bec8b6143b5a9c Peter Gonda 2021-07-14  1225  
bec8b6143b5a9c Peter Gonda 2021-07-14  1226  		vcpu = kvm->vcpus[i];
bec8b6143b5a9c Peter Gonda 2021-07-14  1227  		if (!vcpu->arch.guest_state_protected)
bec8b6143b5a9c Peter Gonda 2021-07-14  1228  			goto e_freelist;

kfree(node) before the goto.

bec8b6143b5a9c Peter Gonda 2021-07-14  1229  
bec8b6143b5a9c Peter Gonda 2021-07-14  1230  		node->vcpu_id = vcpu->vcpu_id;
bec8b6143b5a9c Peter Gonda 2021-07-14  1231  		svm = to_svm(vcpu);
bec8b6143b5a9c Peter Gonda 2021-07-14  1232  		node->vmsa = svm->vmsa;
bec8b6143b5a9c Peter Gonda 2021-07-14  1233  		svm->vmsa = NULL;
bec8b6143b5a9c Peter Gonda 2021-07-14  1234  		node->ghcb = svm->ghcb;
bec8b6143b5a9c Peter Gonda 2021-07-14  1235  		svm->ghcb = NULL;
bec8b6143b5a9c Peter Gonda 2021-07-14  1236  		node->ghcb_gpa = svm->vmcb->control.ghcb_gpa;
bec8b6143b5a9c Peter Gonda 2021-07-14  1237  		node->ghcb_sa = svm->ghcb_sa;
bec8b6143b5a9c Peter Gonda 2021-07-14  1238  		svm->ghcb_sa = NULL;
bec8b6143b5a9c Peter Gonda 2021-07-14  1239  		node->ghcb_sa_len = svm->ghcb_sa_len;
bec8b6143b5a9c Peter Gonda 2021-07-14  1240  		svm->ghcb_sa_len = 0;
bec8b6143b5a9c Peter Gonda 2021-07-14  1241  		node->ghcb_sa_sync = svm->ghcb_sa_sync;
bec8b6143b5a9c Peter Gonda 2021-07-14  1242  		svm->ghcb_sa_sync = false;
bec8b6143b5a9c Peter Gonda 2021-07-14  1243  		node->ghcb_sa_free = svm->ghcb_sa_free;
bec8b6143b5a9c Peter Gonda 2021-07-14  1244  		svm->ghcb_sa_free = false;
bec8b6143b5a9c Peter Gonda 2021-07-14  1245  
bec8b6143b5a9c Peter Gonda 2021-07-14  1246  		list_add_tail(&node->list, &entry->vmsa_list);
bec8b6143b5a9c Peter Gonda 2021-07-14  1247  	}
bec8b6143b5a9c Peter Gonda 2021-07-14  1248  
bec8b6143b5a9c Peter Gonda 2021-07-14  1249  	return 0;
bec8b6143b5a9c Peter Gonda 2021-07-14  1250  
bec8b6143b5a9c Peter Gonda 2021-07-14  1251  e_freelist:
bec8b6143b5a9c Peter Gonda 2021-07-14  1252  	process_vmsa_list(kvm, &entry->vmsa_list);
bec8b6143b5a9c Peter Gonda 2021-07-14 @1253  	return -1;
bec8b6143b5a9c Peter Gonda 2021-07-14  1254  }

---
0-DAY CI Kernel Test Service, Intel Corporation
https://lists.01.org/hyperkitty/list/kbuild-all(a)lists.01.org

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

* Re: [PATCH 3/3 V2] KVM, SEV: Add support for SEV-ES intra host migration
  2021-07-19  7:53 ` Dan Carpenter
  (?)
@ 2021-07-19 21:14 ` Peter Gonda
  -1 siblings, 0 replies; 10+ messages in thread
From: Peter Gonda @ 2021-07-19 21:14 UTC (permalink / raw)
  To: kbuild-all

[-- Attachment #1: Type: text/plain, Size: 4493 bytes --]

Thanks Dan. I'll fix this in a V3.

On Mon, Jul 19, 2021 at 1:53 AM Dan Carpenter <dan.carpenter@oracle.com> wrote:
>
> Hi Peter,
>
> url:    https://github.com/0day-ci/linux/commits/Peter-Gonda/Add-AMD-SEV-and-SEV-ES-intra-host-migration-support/20210715-000317
> base:   7caa04b36f204a01dac65582b71d26d190a1e022
> config: x86_64-randconfig-m001-20210714 (attached as .config)
> compiler: gcc-10 (Debian 10.2.1-6) 10.2.1 20210110
>
> If you fix the issue, kindly add following tag as appropriate
> Reported-by: kernel test robot <lkp@intel.com>
> Reported-by: Dan Carpenter <dan.carpenter@oracle.com>
>
> New smatch warnings:
> arch/x86/kvm/svm/sev.c:1253 create_vmsa_list() warn: possible memory leak of 'node'
>
> Old smatch warnings:
> arch/x86/kvm/svm/sev.c:1607 sev_send_update_data() warn: maybe return -EFAULT instead of the bytes remaining?
>
> vim +/node +1253 arch/x86/kvm/svm/sev.c
>
> bec8b6143b5a9c Peter Gonda 2021-07-14  1211  static int create_vmsa_list(struct kvm *kvm,
> bec8b6143b5a9c Peter Gonda 2021-07-14  1212                         struct sev_info_migration_node *entry)
> bec8b6143b5a9c Peter Gonda 2021-07-14  1213  {
> bec8b6143b5a9c Peter Gonda 2021-07-14  1214     int i;
> bec8b6143b5a9c Peter Gonda 2021-07-14  1215     const int num_vcpus = atomic_read(&kvm->online_vcpus);
> bec8b6143b5a9c Peter Gonda 2021-07-14  1216     struct vmsa_node *node;
> bec8b6143b5a9c Peter Gonda 2021-07-14  1217     struct kvm_vcpu *vcpu;
> bec8b6143b5a9c Peter Gonda 2021-07-14  1218     struct vcpu_svm *svm;
> bec8b6143b5a9c Peter Gonda 2021-07-14  1219
> bec8b6143b5a9c Peter Gonda 2021-07-14  1220     INIT_LIST_HEAD(&entry->vmsa_list);
> bec8b6143b5a9c Peter Gonda 2021-07-14  1221     for (i = 0; i < num_vcpus; ++i) {
> bec8b6143b5a9c Peter Gonda 2021-07-14  1222             node = kzalloc(sizeof(*node), GFP_KERNEL);
> bec8b6143b5a9c Peter Gonda 2021-07-14  1223             if (!node)
> bec8b6143b5a9c Peter Gonda 2021-07-14  1224                     goto e_freelist;
> bec8b6143b5a9c Peter Gonda 2021-07-14  1225
> bec8b6143b5a9c Peter Gonda 2021-07-14  1226             vcpu = kvm->vcpus[i];
> bec8b6143b5a9c Peter Gonda 2021-07-14  1227             if (!vcpu->arch.guest_state_protected)
> bec8b6143b5a9c Peter Gonda 2021-07-14  1228                     goto e_freelist;
>
> kfree(node) before the goto.
>
> bec8b6143b5a9c Peter Gonda 2021-07-14  1229
> bec8b6143b5a9c Peter Gonda 2021-07-14  1230             node->vcpu_id = vcpu->vcpu_id;
> bec8b6143b5a9c Peter Gonda 2021-07-14  1231             svm = to_svm(vcpu);
> bec8b6143b5a9c Peter Gonda 2021-07-14  1232             node->vmsa = svm->vmsa;
> bec8b6143b5a9c Peter Gonda 2021-07-14  1233             svm->vmsa = NULL;
> bec8b6143b5a9c Peter Gonda 2021-07-14  1234             node->ghcb = svm->ghcb;
> bec8b6143b5a9c Peter Gonda 2021-07-14  1235             svm->ghcb = NULL;
> bec8b6143b5a9c Peter Gonda 2021-07-14  1236             node->ghcb_gpa = svm->vmcb->control.ghcb_gpa;
> bec8b6143b5a9c Peter Gonda 2021-07-14  1237             node->ghcb_sa = svm->ghcb_sa;
> bec8b6143b5a9c Peter Gonda 2021-07-14  1238             svm->ghcb_sa = NULL;
> bec8b6143b5a9c Peter Gonda 2021-07-14  1239             node->ghcb_sa_len = svm->ghcb_sa_len;
> bec8b6143b5a9c Peter Gonda 2021-07-14  1240             svm->ghcb_sa_len = 0;
> bec8b6143b5a9c Peter Gonda 2021-07-14  1241             node->ghcb_sa_sync = svm->ghcb_sa_sync;
> bec8b6143b5a9c Peter Gonda 2021-07-14  1242             svm->ghcb_sa_sync = false;
> bec8b6143b5a9c Peter Gonda 2021-07-14  1243             node->ghcb_sa_free = svm->ghcb_sa_free;
> bec8b6143b5a9c Peter Gonda 2021-07-14  1244             svm->ghcb_sa_free = false;
> bec8b6143b5a9c Peter Gonda 2021-07-14  1245
> bec8b6143b5a9c Peter Gonda 2021-07-14  1246             list_add_tail(&node->list, &entry->vmsa_list);
> bec8b6143b5a9c Peter Gonda 2021-07-14  1247     }
> bec8b6143b5a9c Peter Gonda 2021-07-14  1248
> bec8b6143b5a9c Peter Gonda 2021-07-14  1249     return 0;
> bec8b6143b5a9c Peter Gonda 2021-07-14  1250
> bec8b6143b5a9c Peter Gonda 2021-07-14  1251  e_freelist:
> bec8b6143b5a9c Peter Gonda 2021-07-14  1252     process_vmsa_list(kvm, &entry->vmsa_list);
> bec8b6143b5a9c Peter Gonda 2021-07-14 @1253     return -1;
> bec8b6143b5a9c Peter Gonda 2021-07-14  1254  }
>
> ---
> 0-DAY CI Kernel Test Service, Intel Corporation
> https://lists.01.org/hyperkitty/list/kbuild-all(a)lists.01.org
>

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

* Re: [PATCH 2/3 V2] KVM, SEV: Add support for SEV intra host migration
@ 2021-07-14 18:45 kernel test robot
  0 siblings, 0 replies; 10+ messages in thread
From: kernel test robot @ 2021-07-14 18:45 UTC (permalink / raw)
  To: kbuild

[-- Attachment #1: Type: text/plain, Size: 20822 bytes --]

CC: kbuild-all(a)lists.01.org
In-Reply-To: <20210714160143.2116583-3-pgonda@google.com>
References: <20210714160143.2116583-3-pgonda@google.com>
TO: Peter Gonda <pgonda@google.com>
TO: pgonda(a)google.com
CC: Lars Bull <larsbull@google.com>
CC: Marc Orr <marcorr@google.com>
CC: Paolo Bonzini <pbonzini@redhat.com>
CC: Sean Christopherson <seanjc@google.com>
CC: David Rientjes <rientjes@google.com>
CC: "Dr . David Alan Gilbert" <dgilbert@redhat.com>
CC: Brijesh Singh <brijesh.singh@amd.com>
CC: Vitaly Kuznetsov <vkuznets@redhat.com>
CC: Wanpeng Li <wanpengli@tencent.com>

Hi Peter,

Thank you for the patch! Perhaps something to improve:

[auto build test WARNING on 7caa04b36f204a01dac65582b71d26d190a1e022]

url:    https://github.com/0day-ci/linux/commits/Peter-Gonda/Add-AMD-SEV-and-SEV-ES-intra-host-migration-support/20210715-000317
base:   7caa04b36f204a01dac65582b71d26d190a1e022
:::::: branch date: 3 hours ago
:::::: commit date: 3 hours ago
compiler: gcc-10 (Debian 10.2.1-6) 10.2.1 20210110
reproduce:
cd tools/perf && ./check-headers.sh

If you fix the issue, kindly add following tag as appropriate
Reported-by: kernel test robot <lkp@intel.com>


perfheadercheck warnings: (new ones prefixed by >>)
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h':  283> /* Flags that describe what fields in emulation_failure hold valid data. */
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h':  284> #define KVM_INTERNAL_ERROR_EMULATION_FLAG_INSTRUCTION_BYTES (1ULL << 0)
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version@'include/uapi/linux/kvm.h':  285> 
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h':  389> 		/*
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h':  390> 		 * KVM_INTERNAL_ERROR_EMULATION
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h':  391> 		 *
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h':  392> 		 * "struct emulation_failure" is an overlay of "struct internal"
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h':  393> 		 * that is used for the KVM_INTERNAL_ERROR_EMULATION sub-type of
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h':  394> 		 * KVM_EXIT_INTERNAL_ERROR.  Note, unlike other internal error
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h':  395> 		 * sub-types, this struct is ABI!  It also needs to be backwards
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h':  396> 		 * compatible with "struct internal".  Take special care that
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h':  397> 		 * "ndata" is correct, that new fields are enumerated in "flags",
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h':  398> 		 * and that each flag enumerates fields that are 64-bit aligned
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h':  399> 		 * and sized (so that ndata+internal.data[] is valid/accurate).
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h':  400> 		 */
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h':  401> 		struct {
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h':  402> 			__u32 suberror;
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h':  403> 			__u32 ndata;
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h':  404> 			__u64 flags;
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h':  405> 			__u8  insn_size;
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h':  406> 			__u8  insn_bytes[15];
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h':  407> 		} emulation_failure;
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1108> #define KVM_CAP_HYPERV_ENFORCE_CPUID 199
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1109> #define KVM_CAP_SREGS2 200
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1110> #define KVM_CAP_EXIT_HYPERCALL 201
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1111> #define KVM_CAP_PPC_RPT_INVALIDATE 202
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1112> #define KVM_CAP_BINARY_STATS_FD 203
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1113> #define KVM_CAP_EXIT_ON_EMULATION_FAILURE 204
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1652> #define KVM_GET_SREGS2             _IOR(KVMIO,  0xcc, struct kvm_sregs2)
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1653> #define KVM_SET_SREGS2             _IOW(KVMIO,  0xcd, struct kvm_sregs2)
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1654> 
>> Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1712> 	/* Intra host migration commands */
>> Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1713> 	KVM_SEV_INTRA_HOST_SEND,
>> Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1714> 	KVM_SEV_INTRA_HOST_RECEIVE,
>> Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1812> struct kvm_sev_intra_host_send {
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1813> 	__u64 info_token;
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1814> };
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1815> 
>> Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1816> struct kvm_sev_intra_host_receive {
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1817> 	__u64 info_token;
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1818> 	__u32 handle;
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1819> };
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1820> 
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1944> 
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1945> /**
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1946>  * struct kvm_stats_header - Header of per vm/vcpu binary statistics data.
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1947>  * @flags: Some extra information for header, always 0 for now.
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1948>  * @name_size: The size in bytes of the memory which contains statistics
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1949>  *             name string including trailing '\0'. The memory is allocated
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1950>  *             at the send of statistics descriptor.
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1951>  * @num_desc: The number of statistics the vm or vcpu has.
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1952>  * @id_offset: The offset of the vm/vcpu stats' id string in the file pointed
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1953>  *             by vm/vcpu stats fd.
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1954>  * @desc_offset: The offset of the vm/vcpu stats' descriptor block in the file
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1955>  *               pointd by vm/vcpu stats fd.
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1956>  * @data_offset: The offset of the vm/vcpu stats' data block in the file
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1957>  *               pointed by vm/vcpu stats fd.
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1958>  *
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1959>  * This is the header userspace needs to read from stats fd before any other
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1960>  * readings. It is used by userspace to discover all the information about the
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1961>  * vm/vcpu's binary statistics.
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1962>  * Userspace reads this header from the start of the vm/vcpu's stats fd.
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1963>  */
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1964> struct kvm_stats_header {
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1965> 	__u32 flags;
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1966> 	__u32 name_size;
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1967> 	__u32 num_desc;
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1968> 	__u32 id_offset;
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1969> 	__u32 desc_offset;
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1970> 	__u32 data_offset;
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1971> };
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1972> 
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1973> #define KVM_STATS_TYPE_SHIFT		0
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1974> #define KVM_STATS_TYPE_MASK		(0xF << KVM_STATS_TYPE_SHIFT)
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version@'include/uapi/linux/kvm.h': 1975> #define KVM_STATS_TYPE_CUMULATIVE	(0x0 << KVM_STATS_TYPE_SHIFT)
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version@'include/uapi/linux/kvm.h': 1976> #define KVM_STATS_TYPE_INSTANT		(0x1 << KVM_STATS_TYPE_SHIFT)
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version@'include/uapi/linux/kvm.h': 1977> #define KVM_STATS_TYPE_PEAK		(0x2 << KVM_STATS_TYPE_SHIFT)
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version@'include/uapi/linux/kvm.h': 1978> #define KVM_STATS_TYPE_MAX		KVM_STATS_TYPE_PEAK
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1979> 
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1980> #define KVM_STATS_UNIT_SHIFT		4
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1981> #define KVM_STATS_UNIT_MASK		(0xF << KVM_STATS_UNIT_SHIFT)
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version@'include/uapi/linux/kvm.h': 1982> #define KVM_STATS_UNIT_NONE		(0x0 << KVM_STATS_UNIT_SHIFT)
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version@'include/uapi/linux/kvm.h': 1983> #define KVM_STATS_UNIT_BYTES		(0x1 << KVM_STATS_UNIT_SHIFT)
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version@'include/uapi/linux/kvm.h': 1984> #define KVM_STATS_UNIT_SECONDS		(0x2 << KVM_STATS_UNIT_SHIFT)
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version@'include/uapi/linux/kvm.h': 1985> #define KVM_STATS_UNIT_CYCLES		(0x3 << KVM_STATS_UNIT_SHIFT)
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version@'include/uapi/linux/kvm.h': 1986> #define KVM_STATS_UNIT_MAX		KVM_STATS_UNIT_CYCLES
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1987> 
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1988> #define KVM_STATS_BASE_SHIFT		8
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1989> #define KVM_STATS_BASE_MASK		(0xF << KVM_STATS_BASE_SHIFT)
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version@'include/uapi/linux/kvm.h': 1990> #define KVM_STATS_BASE_POW10		(0x0 << KVM_STATS_BASE_SHIFT)
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version@'include/uapi/linux/kvm.h': 1991> #define KVM_STATS_BASE_POW2		(0x1 << KVM_STATS_BASE_SHIFT)
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version@'include/uapi/linux/kvm.h': 1992> #define KVM_STATS_BASE_MAX		KVM_STATS_BASE_POW2
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1993> 
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1994> /**
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1995>  * struct kvm_stats_desc - Descriptor of a KVM statistics.
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1996>  * @flags: Annotations of the stats, like type, unit, etc.
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1997>  * @exponent: Used together with @flags to determine the unit.
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1998>  * @size: The number of data items for this stats.
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 1999>  *        Every data item is of type __u64.
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 2000>  * @offset: The offset of the stats to the start of stat structure in
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 2001>  *          struture kvm or kvm_vcpu.
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 2002>  * @unused: Unused field for future usage. Always 0 for now.
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 2003>  * @name: The name string for the stats. Its size is indicated by the
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 2004>  *        &kvm_stats_header->name_size.
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 2005>  */
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 2006> struct kvm_stats_desc {
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 2007> 	__u32 flags;
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 2008> 	__s16 exponent;
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 2009> 	__u16 size;
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 2010> 	__u32 offset;
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 2011> 	__u32 unused;
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 2012> 	char name[];
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 2013> };
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 2014> 
   Warning: Kernel ABI header at 'tools/include/uapi/linux/kvm.h' differs from latest version at 'include/uapi/linux/kvm.h': 2015> #define KVM_GET_STATS_FD  _IO(KVMIO,  0xce)

---
0-DAY CI Kernel Test Service, Intel Corporation
https://lists.01.org/hyperkitty/list/kbuild-all(a)lists.01.org

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

end of thread, other threads:[~2021-07-19 21:14 UTC | newest]

Thread overview: 10+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2021-07-14 16:01 [PATCH 0/3 V2] Add AMD SEV and SEV-ES intra host migration support Peter Gonda
2021-07-14 16:01 ` [PATCH 1/3 V2] KVM, SEV: Refactor out function for unregistering encrypted regions Peter Gonda
2021-07-14 16:01 ` [PATCH 2/3 V2] KVM, SEV: Add support for SEV intra host migration Peter Gonda
2021-07-14 19:22   ` Brijesh Singh
2021-07-14 16:01 ` [PATCH 3/3 V2] KVM, SEV: Add support for SEV-ES " Peter Gonda
2021-07-14 19:50   ` Brijesh Singh
2021-07-14 18:45 [PATCH 2/3 V2] KVM, SEV: Add support for SEV " kernel test robot
2021-07-14 21:35 [PATCH 3/3 V2] KVM, SEV: Add support for SEV-ES " kernel test robot
2021-07-19  7:53 ` Dan Carpenter
2021-07-19 21:14 ` Peter Gonda

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