summaryrefslogtreecommitdiff
path: root/arch/arm64/kvm/mmu.c
diff options
context:
space:
mode:
Diffstat (limited to 'arch/arm64/kvm/mmu.c')
-rw-r--r--arch/arm64/kvm/mmu.c3171
1 files changed, 1567 insertions, 1604 deletions
diff --git a/arch/arm64/kvm/mmu.c b/arch/arm64/kvm/mmu.c
index 8c0035cab6b6..48d7c372a4cd 100644
--- a/arch/arm64/kvm/mmu.c
+++ b/arch/arm64/kvm/mmu.c
@@ -4,231 +4,289 @@
* Author: Christoffer Dall <c.dall@virtualopensystems.com>
*/
+#include <linux/acpi.h>
#include <linux/mman.h>
#include <linux/kvm_host.h>
#include <linux/io.h>
#include <linux/hugetlb.h>
#include <linux/sched/signal.h>
#include <trace/events/kvm.h>
+#include <asm/acpi.h>
#include <asm/pgalloc.h>
#include <asm/cacheflush.h>
#include <asm/kvm_arm.h>
#include <asm/kvm_mmu.h>
-#include <asm/kvm_ras.h>
+#include <asm/kvm_pgtable.h>
+#include <asm/kvm_pkvm.h>
#include <asm/kvm_asm.h>
#include <asm/kvm_emulate.h>
#include <asm/virt.h>
#include "trace.h"
-static pgd_t *boot_hyp_pgd;
-static pgd_t *hyp_pgd;
-static pgd_t *merged_hyp_pgd;
+static struct kvm_pgtable *hyp_pgtable;
static DEFINE_MUTEX(kvm_hyp_pgd_mutex);
-static unsigned long hyp_idmap_start;
-static unsigned long hyp_idmap_end;
-static phys_addr_t hyp_idmap_vector;
+static unsigned long __ro_after_init hyp_idmap_start;
+static unsigned long __ro_after_init hyp_idmap_end;
+static phys_addr_t __ro_after_init hyp_idmap_vector;
-static unsigned long io_map_base;
+u32 __ro_after_init __hyp_va_bits;
-#define hyp_pgd_order get_order(PTRS_PER_PGD * sizeof(pgd_t))
+static unsigned long __ro_after_init io_map_base;
-#define KVM_S2PTE_FLAG_IS_IOMAP (1UL << 0)
-#define KVM_S2_FLAG_LOGGING_ACTIVE (1UL << 1)
+#define KVM_PGT_FN(fn) (!is_protected_kvm_enabled() ? fn : p ## fn)
-static bool is_iomap(unsigned long flags)
+static phys_addr_t __stage2_range_addr_end(phys_addr_t addr, phys_addr_t end,
+ phys_addr_t size)
{
- return flags & KVM_S2PTE_FLAG_IS_IOMAP;
+ phys_addr_t boundary = ALIGN_DOWN(addr + size, size);
+
+ return (boundary - 1 < end - 1) ? boundary : end;
}
-static bool memslot_is_logging(struct kvm_memory_slot *memslot)
+static phys_addr_t stage2_range_addr_end(phys_addr_t addr, phys_addr_t end)
{
- return memslot->dirty_bitmap && !(memslot->flags & KVM_MEM_READONLY);
+ phys_addr_t size = kvm_granule_size(KVM_PGTABLE_MIN_BLOCK_LEVEL);
+
+ return __stage2_range_addr_end(addr, end, size);
}
-/**
- * kvm_flush_remote_tlbs() - flush all VM TLB entries for v7/8
- * @kvm: pointer to kvm structure.
- *
- * Interface to HYP function to flush all VM TLB entries
+/*
+ * Release kvm_mmu_lock periodically if the memory region is large. Otherwise,
+ * we may see kernel panics with CONFIG_DETECT_HUNG_TASK,
+ * CONFIG_LOCKUP_DETECTOR, CONFIG_LOCKDEP. Additionally, holding the lock too
+ * long will also starve other vCPUs. We have to also make sure that the page
+ * tables are not freed while we released the lock.
*/
-void kvm_flush_remote_tlbs(struct kvm *kvm)
+static int stage2_apply_range(struct kvm_s2_mmu *mmu, phys_addr_t addr,
+ phys_addr_t end,
+ int (*fn)(struct kvm_pgtable *, u64, u64),
+ bool resched)
{
- kvm_call_hyp(__kvm_tlb_flush_vmid, kvm);
-}
+ struct kvm *kvm = kvm_s2_mmu_to_kvm(mmu);
+ int ret;
+ u64 next;
-static void kvm_tlb_flush_vmid_ipa(struct kvm *kvm, phys_addr_t ipa)
-{
- kvm_call_hyp(__kvm_tlb_flush_vmid_ipa, kvm, ipa);
+ do {
+ struct kvm_pgtable *pgt = mmu->pgt;
+ if (!pgt)
+ return -EINVAL;
+
+ next = stage2_range_addr_end(addr, end);
+ ret = fn(pgt, addr, next - addr);
+ if (ret)
+ break;
+
+ if (resched && next != end)
+ cond_resched_rwlock_write(&kvm->mmu_lock);
+ } while (addr = next, addr != end);
+
+ return ret;
}
+#define stage2_apply_range_resched(mmu, addr, end, fn) \
+ stage2_apply_range(mmu, addr, end, fn, true)
+
/*
- * D-Cache management functions. They take the page table entries by
- * value, as they are flushing the cache using the kernel mapping (or
- * kmap on 32bit).
+ * Get the maximum number of page-tables pages needed to split a range
+ * of blocks into PAGE_SIZE PTEs. It assumes the range is already
+ * mapped at level 2, or at level 1 if allowed.
*/
-static void kvm_flush_dcache_pte(pte_t pte)
+static int kvm_mmu_split_nr_page_tables(u64 range)
{
- __kvm_flush_dcache_pte(pte);
+ int n = 0;
+
+ if (KVM_PGTABLE_MIN_BLOCK_LEVEL < 2)
+ n += DIV_ROUND_UP(range, PUD_SIZE);
+ n += DIV_ROUND_UP(range, PMD_SIZE);
+ return n;
}
-static void kvm_flush_dcache_pmd(pmd_t pmd)
+static bool need_split_memcache_topup_or_resched(struct kvm *kvm)
{
- __kvm_flush_dcache_pmd(pmd);
+ struct kvm_mmu_memory_cache *cache;
+ u64 chunk_size, min;
+
+ if (need_resched() || rwlock_needbreak(&kvm->mmu_lock))
+ return true;
+
+ chunk_size = kvm->arch.mmu.split_page_chunk_size;
+ min = kvm_mmu_split_nr_page_tables(chunk_size);
+ cache = &kvm->arch.mmu.split_page_cache;
+ return kvm_mmu_memory_cache_nr_free_objects(cache) < min;
}
-static void kvm_flush_dcache_pud(pud_t pud)
+static int kvm_mmu_split_huge_pages(struct kvm *kvm, phys_addr_t addr,
+ phys_addr_t end)
{
- __kvm_flush_dcache_pud(pud);
+ struct kvm_mmu_memory_cache *cache;
+ struct kvm_pgtable *pgt;
+ int ret, cache_capacity;
+ u64 next, chunk_size;
+
+ lockdep_assert_held_write(&kvm->mmu_lock);
+
+ chunk_size = kvm->arch.mmu.split_page_chunk_size;
+ cache_capacity = kvm_mmu_split_nr_page_tables(chunk_size);
+
+ if (chunk_size == 0)
+ return 0;
+
+ cache = &kvm->arch.mmu.split_page_cache;
+
+ do {
+ if (need_split_memcache_topup_or_resched(kvm)) {
+ write_unlock(&kvm->mmu_lock);
+ cond_resched();
+ /* Eager page splitting is best-effort. */
+ ret = __kvm_mmu_topup_memory_cache(cache,
+ cache_capacity,
+ cache_capacity);
+ write_lock(&kvm->mmu_lock);
+ if (ret)
+ break;
+ }
+
+ pgt = kvm->arch.mmu.pgt;
+ if (!pgt)
+ return -EINVAL;
+
+ next = __stage2_range_addr_end(addr, end, chunk_size);
+ ret = KVM_PGT_FN(kvm_pgtable_stage2_split)(pgt, addr, next - addr, cache);
+ if (ret)
+ break;
+ } while (addr = next, addr != end);
+
+ return ret;
}
-static bool kvm_is_device_pfn(unsigned long pfn)
+static bool memslot_is_logging(struct kvm_memory_slot *memslot)
{
- return !pfn_valid(pfn);
+ return memslot->dirty_bitmap && !(memslot->flags & KVM_MEM_READONLY);
}
/**
- * stage2_dissolve_pmd() - clear and flush huge PMD entry
+ * kvm_arch_flush_remote_tlbs() - flush all VM TLB entries for v7/8
* @kvm: pointer to kvm structure.
- * @addr: IPA
- * @pmd: pmd pointer for IPA
*
- * Function clears a PMD entry, flushes addr 1st and 2nd stage TLBs.
+ * Interface to HYP function to flush all VM TLB entries
*/
-static void stage2_dissolve_pmd(struct kvm *kvm, phys_addr_t addr, pmd_t *pmd)
+int kvm_arch_flush_remote_tlbs(struct kvm *kvm)
{
- if (!pmd_thp_or_huge(*pmd))
- return;
-
- pmd_clear(pmd);
- kvm_tlb_flush_vmid_ipa(kvm, addr);
- put_page(virt_to_page(pmd));
+ if (is_protected_kvm_enabled())
+ kvm_call_hyp_nvhe(__pkvm_tlb_flush_vmid, kvm->arch.pkvm.handle);
+ else
+ kvm_call_hyp(__kvm_tlb_flush_vmid, &kvm->arch.mmu);
+ return 0;
}
-/**
- * stage2_dissolve_pud() - clear and flush huge PUD entry
- * @kvm: pointer to kvm structure.
- * @addr: IPA
- * @pud: pud pointer for IPA
- *
- * Function clears a PUD entry, flushes addr 1st and 2nd stage TLBs.
- */
-static void stage2_dissolve_pud(struct kvm *kvm, phys_addr_t addr, pud_t *pudp)
+int kvm_arch_flush_remote_tlbs_range(struct kvm *kvm,
+ gfn_t gfn, u64 nr_pages)
{
- if (!stage2_pud_huge(kvm, *pudp))
- return;
+ u64 size = nr_pages << PAGE_SHIFT;
+ u64 addr = gfn << PAGE_SHIFT;
- stage2_pud_clear(kvm, pudp);
- kvm_tlb_flush_vmid_ipa(kvm, addr);
- put_page(virt_to_page(pudp));
+ if (is_protected_kvm_enabled())
+ kvm_call_hyp_nvhe(__pkvm_tlb_flush_vmid, kvm->arch.pkvm.handle);
+ else
+ kvm_tlb_flush_vmid_range(&kvm->arch.mmu, addr, size);
+ return 0;
}
-static int mmu_topup_memory_cache(struct kvm_mmu_memory_cache *cache,
- int min, int max)
+static void *stage2_memcache_zalloc_page(void *arg)
{
- void *page;
+ struct kvm_mmu_memory_cache *mc = arg;
+ void *virt;
- BUG_ON(max > KVM_NR_MEM_OBJS);
- if (cache->nobjs >= min)
- return 0;
- while (cache->nobjs < max) {
- page = (void *)__get_free_page(GFP_PGTABLE_USER);
- if (!page)
- return -ENOMEM;
- cache->objects[cache->nobjs++] = page;
- }
- return 0;
+ /* Allocated with __GFP_ZERO, so no need to zero */
+ virt = kvm_mmu_memory_cache_alloc(mc);
+ if (virt)
+ kvm_account_pgtable_pages(virt, 1);
+ return virt;
}
-static void mmu_free_memory_cache(struct kvm_mmu_memory_cache *mc)
+static void *kvm_host_zalloc_pages_exact(size_t size)
{
- while (mc->nobjs)
- free_page((unsigned long)mc->objects[--mc->nobjs]);
+ return alloc_pages_exact(size, GFP_KERNEL_ACCOUNT | __GFP_ZERO);
}
-static void *mmu_memory_cache_alloc(struct kvm_mmu_memory_cache *mc)
+static void *kvm_s2_zalloc_pages_exact(size_t size)
{
- void *p;
+ void *virt = kvm_host_zalloc_pages_exact(size);
- BUG_ON(!mc || !mc->nobjs);
- p = mc->objects[--mc->nobjs];
- return p;
+ if (virt)
+ kvm_account_pgtable_pages(virt, (size >> PAGE_SHIFT));
+ return virt;
}
-static void clear_stage2_pgd_entry(struct kvm *kvm, pgd_t *pgd, phys_addr_t addr)
+static void kvm_s2_free_pages_exact(void *virt, size_t size)
{
- p4d_t *p4d_table __maybe_unused = stage2_p4d_offset(kvm, pgd, 0UL);
- stage2_pgd_clear(kvm, pgd);
- kvm_tlb_flush_vmid_ipa(kvm, addr);
- stage2_p4d_free(kvm, p4d_table);
- put_page(virt_to_page(pgd));
+ kvm_account_pgtable_pages(virt, -(size >> PAGE_SHIFT));
+ free_pages_exact(virt, size);
}
-static void clear_stage2_p4d_entry(struct kvm *kvm, p4d_t *p4d, phys_addr_t addr)
+static struct kvm_pgtable_mm_ops kvm_s2_mm_ops;
+
+static void stage2_free_unlinked_table_rcu_cb(struct rcu_head *head)
{
- pud_t *pud_table __maybe_unused = stage2_pud_offset(kvm, p4d, 0);
- stage2_p4d_clear(kvm, p4d);
- kvm_tlb_flush_vmid_ipa(kvm, addr);
- stage2_pud_free(kvm, pud_table);
- put_page(virt_to_page(p4d));
+ struct page *page = container_of(head, struct page, rcu_head);
+ void *pgtable = page_to_virt(page);
+ s8 level = page_private(page);
+
+ KVM_PGT_FN(kvm_pgtable_stage2_free_unlinked)(&kvm_s2_mm_ops, pgtable, level);
}
-static void clear_stage2_pud_entry(struct kvm *kvm, pud_t *pud, phys_addr_t addr)
+static void stage2_free_unlinked_table(void *addr, s8 level)
{
- pmd_t *pmd_table __maybe_unused = stage2_pmd_offset(kvm, pud, 0);
- VM_BUG_ON(stage2_pud_huge(kvm, *pud));
- stage2_pud_clear(kvm, pud);
- kvm_tlb_flush_vmid_ipa(kvm, addr);
- stage2_pmd_free(kvm, pmd_table);
- put_page(virt_to_page(pud));
+ struct page *page = virt_to_page(addr);
+
+ set_page_private(page, (unsigned long)level);
+ call_rcu(&page->rcu_head, stage2_free_unlinked_table_rcu_cb);
}
-static void clear_stage2_pmd_entry(struct kvm *kvm, pmd_t *pmd, phys_addr_t addr)
+static void kvm_host_get_page(void *addr)
{
- pte_t *pte_table = pte_offset_kernel(pmd, 0);
- VM_BUG_ON(pmd_thp_or_huge(*pmd));
- pmd_clear(pmd);
- kvm_tlb_flush_vmid_ipa(kvm, addr);
- free_page((unsigned long)pte_table);
- put_page(virt_to_page(pmd));
+ get_page(virt_to_page(addr));
}
-static inline void kvm_set_pte(pte_t *ptep, pte_t new_pte)
+static void kvm_host_put_page(void *addr)
{
- WRITE_ONCE(*ptep, new_pte);
- dsb(ishst);
+ put_page(virt_to_page(addr));
}
-static inline void kvm_set_pmd(pmd_t *pmdp, pmd_t new_pmd)
+static void kvm_s2_put_page(void *addr)
{
- WRITE_ONCE(*pmdp, new_pmd);
- dsb(ishst);
+ struct page *p = virt_to_page(addr);
+ /* Dropping last refcount, the page will be freed */
+ if (page_count(p) == 1)
+ kvm_account_pgtable_pages(addr, -1);
+ put_page(p);
}
-static inline void kvm_pmd_populate(pmd_t *pmdp, pte_t *ptep)
+static int kvm_host_page_count(void *addr)
{
- kvm_set_pmd(pmdp, kvm_mk_pmd(ptep));
+ return page_count(virt_to_page(addr));
}
-static inline void kvm_pud_populate(pud_t *pudp, pmd_t *pmdp)
+static phys_addr_t kvm_host_pa(void *addr)
{
- WRITE_ONCE(*pudp, kvm_mk_pud(pmdp));
- dsb(ishst);
+ return __pa(addr);
}
-static inline void kvm_p4d_populate(p4d_t *p4dp, pud_t *pudp)
+static void *kvm_host_va(phys_addr_t phys)
{
- WRITE_ONCE(*p4dp, kvm_mk_p4d(pudp));
- dsb(ishst);
+ return __va(phys);
}
-static inline void kvm_pgd_populate(pgd_t *pgdp, p4d_t *p4dp)
+static void clean_dcache_guest_page(void *va, size_t size)
{
-#ifndef __PAGETABLE_P4D_FOLDED
- WRITE_ONCE(*pgdp, kvm_mk_pgd(p4dp));
- dsb(ishst);
-#endif
+ __clean_dcache_guest_page(va, size);
+}
+
+static void invalidate_icache_guest_page(void *va, size_t size)
+{
+ __invalidate_icache_guest_page(va, size);
}
/*
@@ -248,213 +306,46 @@ static inline void kvm_pgd_populate(pgd_t *pgdp, p4d_t *p4dp)
* end up writing old data to disk.
*
* This is why right after unmapping a page/section and invalidating
- * the corresponding TLBs, we call kvm_flush_dcache_p*() to make sure
- * the IO subsystem will never hit in the cache.
+ * the corresponding TLBs, we flush to make sure the IO subsystem will
+ * never hit in the cache.
*
* This is all avoided on systems that have ARM64_HAS_STAGE2_FWB, as
* we then fully enforce cacheability of RAM, no matter what the guest
* does.
*/
-static void unmap_stage2_ptes(struct kvm *kvm, pmd_t *pmd,
- phys_addr_t addr, phys_addr_t end)
-{
- phys_addr_t start_addr = addr;
- pte_t *pte, *start_pte;
-
- start_pte = pte = pte_offset_kernel(pmd, addr);
- do {
- if (!pte_none(*pte)) {
- pte_t old_pte = *pte;
-
- kvm_set_pte(pte, __pte(0));
- kvm_tlb_flush_vmid_ipa(kvm, addr);
-
- /* No need to invalidate the cache for device mappings */
- if (!kvm_is_device_pfn(pte_pfn(old_pte)))
- kvm_flush_dcache_pte(old_pte);
-
- put_page(virt_to_page(pte));
- }
- } while (pte++, addr += PAGE_SIZE, addr != end);
-
- if (stage2_pte_table_empty(kvm, start_pte))
- clear_stage2_pmd_entry(kvm, pmd, start_addr);
-}
-
-static void unmap_stage2_pmds(struct kvm *kvm, pud_t *pud,
- phys_addr_t addr, phys_addr_t end)
-{
- phys_addr_t next, start_addr = addr;
- pmd_t *pmd, *start_pmd;
-
- start_pmd = pmd = stage2_pmd_offset(kvm, pud, addr);
- do {
- next = stage2_pmd_addr_end(kvm, addr, end);
- if (!pmd_none(*pmd)) {
- if (pmd_thp_or_huge(*pmd)) {
- pmd_t old_pmd = *pmd;
-
- pmd_clear(pmd);
- kvm_tlb_flush_vmid_ipa(kvm, addr);
-
- kvm_flush_dcache_pmd(old_pmd);
-
- put_page(virt_to_page(pmd));
- } else {
- unmap_stage2_ptes(kvm, pmd, addr, next);
- }
- }
- } while (pmd++, addr = next, addr != end);
-
- if (stage2_pmd_table_empty(kvm, start_pmd))
- clear_stage2_pud_entry(kvm, pud, start_addr);
-}
-
-static void unmap_stage2_puds(struct kvm *kvm, p4d_t *p4d,
- phys_addr_t addr, phys_addr_t end)
-{
- phys_addr_t next, start_addr = addr;
- pud_t *pud, *start_pud;
-
- start_pud = pud = stage2_pud_offset(kvm, p4d, addr);
- do {
- next = stage2_pud_addr_end(kvm, addr, end);
- if (!stage2_pud_none(kvm, *pud)) {
- if (stage2_pud_huge(kvm, *pud)) {
- pud_t old_pud = *pud;
-
- stage2_pud_clear(kvm, pud);
- kvm_tlb_flush_vmid_ipa(kvm, addr);
- kvm_flush_dcache_pud(old_pud);
- put_page(virt_to_page(pud));
- } else {
- unmap_stage2_pmds(kvm, pud, addr, next);
- }
- }
- } while (pud++, addr = next, addr != end);
-
- if (stage2_pud_table_empty(kvm, start_pud))
- clear_stage2_p4d_entry(kvm, p4d, start_addr);
-}
-
-static void unmap_stage2_p4ds(struct kvm *kvm, pgd_t *pgd,
- phys_addr_t addr, phys_addr_t end)
-{
- phys_addr_t next, start_addr = addr;
- p4d_t *p4d, *start_p4d;
-
- start_p4d = p4d = stage2_p4d_offset(kvm, pgd, addr);
- do {
- next = stage2_p4d_addr_end(kvm, addr, end);
- if (!stage2_p4d_none(kvm, *p4d))
- unmap_stage2_puds(kvm, p4d, addr, next);
- } while (p4d++, addr = next, addr != end);
-
- if (stage2_p4d_table_empty(kvm, start_p4d))
- clear_stage2_pgd_entry(kvm, pgd, start_addr);
-}
-
/**
- * unmap_stage2_range -- Clear stage2 page table entries to unmap a range
- * @kvm: The VM pointer
+ * __unmap_stage2_range -- Clear stage2 page table entries to unmap a range
+ * @mmu: The KVM stage-2 MMU pointer
* @start: The intermediate physical base address of the range to unmap
* @size: The size of the area to unmap
+ * @may_block: Whether or not we are permitted to block
*
* Clear a range of stage-2 mappings, lowering the various ref-counts. Must
* be called while holding mmu_lock (unless for freeing the stage2 pgd before
* destroying the VM), otherwise another faulting VCPU may come in and mess
* with things behind our backs.
*/
-static void unmap_stage2_range(struct kvm *kvm, phys_addr_t start, u64 size)
+static void __unmap_stage2_range(struct kvm_s2_mmu *mmu, phys_addr_t start, u64 size,
+ bool may_block)
{
- pgd_t *pgd;
- phys_addr_t addr = start, end = start + size;
- phys_addr_t next;
+ struct kvm *kvm = kvm_s2_mmu_to_kvm(mmu);
+ phys_addr_t end = start + size;
- assert_spin_locked(&kvm->mmu_lock);
+ lockdep_assert_held_write(&kvm->mmu_lock);
WARN_ON(size & ~PAGE_MASK);
-
- pgd = kvm->arch.pgd + stage2_pgd_index(kvm, addr);
- do {
- /*
- * Make sure the page table is still active, as another thread
- * could have possibly freed the page table, while we released
- * the lock.
- */
- if (!READ_ONCE(kvm->arch.pgd))
- break;
- next = stage2_pgd_addr_end(kvm, addr, end);
- if (!stage2_pgd_none(kvm, *pgd))
- unmap_stage2_p4ds(kvm, pgd, addr, next);
- /*
- * If the range is too large, release the kvm->mmu_lock
- * to prevent starvation and lockup detector warnings.
- */
- if (next != end)
- cond_resched_lock(&kvm->mmu_lock);
- } while (pgd++, addr = next, addr != end);
-}
-
-static void stage2_flush_ptes(struct kvm *kvm, pmd_t *pmd,
- phys_addr_t addr, phys_addr_t end)
-{
- pte_t *pte;
-
- pte = pte_offset_kernel(pmd, addr);
- do {
- if (!pte_none(*pte) && !kvm_is_device_pfn(pte_pfn(*pte)))
- kvm_flush_dcache_pte(*pte);
- } while (pte++, addr += PAGE_SIZE, addr != end);
-}
-
-static void stage2_flush_pmds(struct kvm *kvm, pud_t *pud,
- phys_addr_t addr, phys_addr_t end)
-{
- pmd_t *pmd;
- phys_addr_t next;
-
- pmd = stage2_pmd_offset(kvm, pud, addr);
- do {
- next = stage2_pmd_addr_end(kvm, addr, end);
- if (!pmd_none(*pmd)) {
- if (pmd_thp_or_huge(*pmd))
- kvm_flush_dcache_pmd(*pmd);
- else
- stage2_flush_ptes(kvm, pmd, addr, next);
- }
- } while (pmd++, addr = next, addr != end);
+ WARN_ON(stage2_apply_range(mmu, start, end, KVM_PGT_FN(kvm_pgtable_stage2_unmap),
+ may_block));
}
-static void stage2_flush_puds(struct kvm *kvm, p4d_t *p4d,
- phys_addr_t addr, phys_addr_t end)
+void kvm_stage2_unmap_range(struct kvm_s2_mmu *mmu, phys_addr_t start,
+ u64 size, bool may_block)
{
- pud_t *pud;
- phys_addr_t next;
-
- pud = stage2_pud_offset(kvm, p4d, addr);
- do {
- next = stage2_pud_addr_end(kvm, addr, end);
- if (!stage2_pud_none(kvm, *pud)) {
- if (stage2_pud_huge(kvm, *pud))
- kvm_flush_dcache_pud(*pud);
- else
- stage2_flush_pmds(kvm, pud, addr, next);
- }
- } while (pud++, addr = next, addr != end);
+ __unmap_stage2_range(mmu, start, size, may_block);
}
-static void stage2_flush_p4ds(struct kvm *kvm, pgd_t *pgd,
- phys_addr_t addr, phys_addr_t end)
+void kvm_stage2_flush_range(struct kvm_s2_mmu *mmu, phys_addr_t addr, phys_addr_t end)
{
- p4d_t *p4d;
- phys_addr_t next;
-
- p4d = stage2_p4d_offset(kvm, pgd, addr);
- do {
- next = stage2_p4d_addr_end(kvm, addr, end);
- if (!stage2_p4d_none(kvm, *p4d))
- stage2_flush_puds(kvm, p4d, addr, next);
- } while (p4d++, addr = next, addr != end);
+ stage2_apply_range_resched(mmu, addr, end, KVM_PGT_FN(kvm_pgtable_stage2_flush));
}
static void stage2_flush_memslot(struct kvm *kvm,
@@ -462,18 +353,8 @@ static void stage2_flush_memslot(struct kvm *kvm,
{
phys_addr_t addr = memslot->base_gfn << PAGE_SHIFT;
phys_addr_t end = addr + PAGE_SIZE * memslot->npages;
- phys_addr_t next;
- pgd_t *pgd;
-
- pgd = kvm->arch.pgd + stage2_pgd_index(kvm, addr);
- do {
- next = stage2_pgd_addr_end(kvm, addr, end);
- if (!stage2_pgd_none(kvm, *pgd))
- stage2_flush_p4ds(kvm, pgd, addr, next);
- if (next != end)
- cond_resched_lock(&kvm->mmu_lock);
- } while (pgd++, addr = next, addr != end);
+ kvm_stage2_flush_range(&kvm->arch.mmu, addr, end);
}
/**
@@ -487,362 +368,213 @@ static void stage2_flush_vm(struct kvm *kvm)
{
struct kvm_memslots *slots;
struct kvm_memory_slot *memslot;
- int idx;
+ int idx, bkt;
idx = srcu_read_lock(&kvm->srcu);
- spin_lock(&kvm->mmu_lock);
+ write_lock(&kvm->mmu_lock);
slots = kvm_memslots(kvm);
- kvm_for_each_memslot(memslot, slots)
+ kvm_for_each_memslot(memslot, bkt, slots)
stage2_flush_memslot(kvm, memslot);
- spin_unlock(&kvm->mmu_lock);
- srcu_read_unlock(&kvm->srcu, idx);
-}
-
-static void clear_hyp_pgd_entry(pgd_t *pgd)
-{
- p4d_t *p4d_table __maybe_unused = p4d_offset(pgd, 0UL);
- pgd_clear(pgd);
- p4d_free(NULL, p4d_table);
- put_page(virt_to_page(pgd));
-}
-
-static void clear_hyp_p4d_entry(p4d_t *p4d)
-{
- pud_t *pud_table __maybe_unused = pud_offset(p4d, 0UL);
- VM_BUG_ON(p4d_huge(*p4d));
- p4d_clear(p4d);
- pud_free(NULL, pud_table);
- put_page(virt_to_page(p4d));
-}
+ kvm_nested_s2_flush(kvm);
-static void clear_hyp_pud_entry(pud_t *pud)
-{
- pmd_t *pmd_table __maybe_unused = pmd_offset(pud, 0);
- VM_BUG_ON(pud_huge(*pud));
- pud_clear(pud);
- pmd_free(NULL, pmd_table);
- put_page(virt_to_page(pud));
+ write_unlock(&kvm->mmu_lock);
+ srcu_read_unlock(&kvm->srcu, idx);
}
-static void clear_hyp_pmd_entry(pmd_t *pmd)
+/**
+ * free_hyp_pgds - free Hyp-mode page tables
+ */
+void __init free_hyp_pgds(void)
{
- pte_t *pte_table = pte_offset_kernel(pmd, 0);
- VM_BUG_ON(pmd_thp_or_huge(*pmd));
- pmd_clear(pmd);
- pte_free_kernel(NULL, pte_table);
- put_page(virt_to_page(pmd));
+ mutex_lock(&kvm_hyp_pgd_mutex);
+ if (hyp_pgtable) {
+ kvm_pgtable_hyp_destroy(hyp_pgtable);
+ kfree(hyp_pgtable);
+ hyp_pgtable = NULL;
+ }
+ mutex_unlock(&kvm_hyp_pgd_mutex);
}
-static void unmap_hyp_ptes(pmd_t *pmd, phys_addr_t addr, phys_addr_t end)
+static bool kvm_host_owns_hyp_mappings(void)
{
- pte_t *pte, *start_pte;
-
- start_pte = pte = pte_offset_kernel(pmd, addr);
- do {
- if (!pte_none(*pte)) {
- kvm_set_pte(pte, __pte(0));
- put_page(virt_to_page(pte));
- }
- } while (pte++, addr += PAGE_SIZE, addr != end);
+ if (is_kernel_in_hyp_mode())
+ return false;
- if (hyp_pte_table_empty(start_pte))
- clear_hyp_pmd_entry(pmd);
-}
+ if (static_branch_likely(&kvm_protected_mode_initialized))
+ return false;
-static void unmap_hyp_pmds(pud_t *pud, phys_addr_t addr, phys_addr_t end)
-{
- phys_addr_t next;
- pmd_t *pmd, *start_pmd;
+ /*
+ * This can happen at boot time when __create_hyp_mappings() is called
+ * after the hyp protection has been enabled, but the static key has
+ * not been flipped yet.
+ */
+ if (!hyp_pgtable && is_protected_kvm_enabled())
+ return false;
- start_pmd = pmd = pmd_offset(pud, addr);
- do {
- next = pmd_addr_end(addr, end);
- /* Hyp doesn't use huge pmds */
- if (!pmd_none(*pmd))
- unmap_hyp_ptes(pmd, addr, next);
- } while (pmd++, addr = next, addr != end);
+ WARN_ON(!hyp_pgtable);
- if (hyp_pmd_table_empty(start_pmd))
- clear_hyp_pud_entry(pud);
+ return true;
}
-static void unmap_hyp_puds(p4d_t *p4d, phys_addr_t addr, phys_addr_t end)
+int __create_hyp_mappings(unsigned long start, unsigned long size,
+ unsigned long phys, enum kvm_pgtable_prot prot)
{
- phys_addr_t next;
- pud_t *pud, *start_pud;
-
- start_pud = pud = pud_offset(p4d, addr);
- do {
- next = pud_addr_end(addr, end);
- /* Hyp doesn't use huge puds */
- if (!pud_none(*pud))
- unmap_hyp_pmds(pud, addr, next);
- } while (pud++, addr = next, addr != end);
-
- if (hyp_pud_table_empty(start_pud))
- clear_hyp_p4d_entry(p4d);
-}
+ int err;
-static void unmap_hyp_p4ds(pgd_t *pgd, phys_addr_t addr, phys_addr_t end)
-{
- phys_addr_t next;
- p4d_t *p4d, *start_p4d;
+ if (WARN_ON(!kvm_host_owns_hyp_mappings()))
+ return -EINVAL;
- start_p4d = p4d = p4d_offset(pgd, addr);
- do {
- next = p4d_addr_end(addr, end);
- /* Hyp doesn't use huge p4ds */
- if (!p4d_none(*p4d))
- unmap_hyp_puds(p4d, addr, next);
- } while (p4d++, addr = next, addr != end);
+ mutex_lock(&kvm_hyp_pgd_mutex);
+ err = kvm_pgtable_hyp_map(hyp_pgtable, start, size, phys, prot);
+ mutex_unlock(&kvm_hyp_pgd_mutex);
- if (hyp_p4d_table_empty(start_p4d))
- clear_hyp_pgd_entry(pgd);
+ return err;
}
-static unsigned int kvm_pgd_index(unsigned long addr, unsigned int ptrs_per_pgd)
+static phys_addr_t kvm_kaddr_to_phys(void *kaddr)
{
- return (addr >> PGDIR_SHIFT) & (ptrs_per_pgd - 1);
+ if (!is_vmalloc_addr(kaddr)) {
+ BUG_ON(!virt_addr_valid(kaddr));
+ return __pa(kaddr);
+ } else {
+ return page_to_phys(vmalloc_to_page(kaddr)) +
+ offset_in_page(kaddr);
+ }
}
-static void __unmap_hyp_range(pgd_t *pgdp, unsigned long ptrs_per_pgd,
- phys_addr_t start, u64 size)
-{
- pgd_t *pgd;
- phys_addr_t addr = start, end = start + size;
- phys_addr_t next;
+struct hyp_shared_pfn {
+ u64 pfn;
+ int count;
+ struct rb_node node;
+};
- /*
- * We don't unmap anything from HYP, except at the hyp tear down.
- * Hence, we don't have to invalidate the TLBs here.
- */
- pgd = pgdp + kvm_pgd_index(addr, ptrs_per_pgd);
- do {
- next = pgd_addr_end(addr, end);
- if (!pgd_none(*pgd))
- unmap_hyp_p4ds(pgd, addr, next);
- } while (pgd++, addr = next, addr != end);
-}
+static DEFINE_MUTEX(hyp_shared_pfns_lock);
+static struct rb_root hyp_shared_pfns = RB_ROOT;
-static void unmap_hyp_range(pgd_t *pgdp, phys_addr_t start, u64 size)
+static struct hyp_shared_pfn *find_shared_pfn(u64 pfn, struct rb_node ***node,
+ struct rb_node **parent)
{
- __unmap_hyp_range(pgdp, PTRS_PER_PGD, start, size);
-}
+ struct hyp_shared_pfn *this;
-static void unmap_hyp_idmap_range(pgd_t *pgdp, phys_addr_t start, u64 size)
-{
- __unmap_hyp_range(pgdp, __kvm_idmap_ptrs_per_pgd(), start, size);
+ *node = &hyp_shared_pfns.rb_node;
+ *parent = NULL;
+ while (**node) {
+ this = container_of(**node, struct hyp_shared_pfn, node);
+ *parent = **node;
+ if (this->pfn < pfn)
+ *node = &((**node)->rb_left);
+ else if (this->pfn > pfn)
+ *node = &((**node)->rb_right);
+ else
+ return this;
+ }
+
+ return NULL;
}
-/**
- * free_hyp_pgds - free Hyp-mode page tables
- *
- * Assumes hyp_pgd is a page table used strictly in Hyp-mode and
- * therefore contains either mappings in the kernel memory area (above
- * PAGE_OFFSET), or device mappings in the idmap range.
- *
- * boot_hyp_pgd should only map the idmap range, and is only used in
- * the extended idmap case.
- */
-void free_hyp_pgds(void)
+static int share_pfn_hyp(u64 pfn)
{
- pgd_t *id_pgd;
-
- mutex_lock(&kvm_hyp_pgd_mutex);
-
- id_pgd = boot_hyp_pgd ? boot_hyp_pgd : hyp_pgd;
-
- if (id_pgd) {
- /* In case we never called hyp_mmu_init() */
- if (!io_map_base)
- io_map_base = hyp_idmap_start;
- unmap_hyp_idmap_range(id_pgd, io_map_base,
- hyp_idmap_start + PAGE_SIZE - io_map_base);
- }
+ struct rb_node **node, *parent;
+ struct hyp_shared_pfn *this;
+ int ret = 0;
- if (boot_hyp_pgd) {
- free_pages((unsigned long)boot_hyp_pgd, hyp_pgd_order);
- boot_hyp_pgd = NULL;
+ mutex_lock(&hyp_shared_pfns_lock);
+ this = find_shared_pfn(pfn, &node, &parent);
+ if (this) {
+ this->count++;
+ goto unlock;
}
- if (hyp_pgd) {
- unmap_hyp_range(hyp_pgd, kern_hyp_va(PAGE_OFFSET),
- (uintptr_t)high_memory - PAGE_OFFSET);
-
- free_pages((unsigned long)hyp_pgd, hyp_pgd_order);
- hyp_pgd = NULL;
- }
- if (merged_hyp_pgd) {
- clear_page(merged_hyp_pgd);
- free_page((unsigned long)merged_hyp_pgd);
- merged_hyp_pgd = NULL;
+ this = kzalloc(sizeof(*this), GFP_KERNEL);
+ if (!this) {
+ ret = -ENOMEM;
+ goto unlock;
}
- mutex_unlock(&kvm_hyp_pgd_mutex);
-}
-
-static void create_hyp_pte_mappings(pmd_t *pmd, unsigned long start,
- unsigned long end, unsigned long pfn,
- pgprot_t prot)
-{
- pte_t *pte;
- unsigned long addr;
+ this->pfn = pfn;
+ this->count = 1;
+ rb_link_node(&this->node, parent, node);
+ rb_insert_color(&this->node, &hyp_shared_pfns);
+ ret = kvm_call_hyp_nvhe(__pkvm_host_share_hyp, pfn, 1);
+unlock:
+ mutex_unlock(&hyp_shared_pfns_lock);
- addr = start;
- do {
- pte = pte_offset_kernel(pmd, addr);
- kvm_set_pte(pte, kvm_pfn_pte(pfn, prot));
- get_page(virt_to_page(pte));
- pfn++;
- } while (addr += PAGE_SIZE, addr != end);
+ return ret;
}
-static int create_hyp_pmd_mappings(pud_t *pud, unsigned long start,
- unsigned long end, unsigned long pfn,
- pgprot_t prot)
+static int unshare_pfn_hyp(u64 pfn)
{
- pmd_t *pmd;
- pte_t *pte;
- unsigned long addr, next;
-
- addr = start;
- do {
- pmd = pmd_offset(pud, addr);
-
- BUG_ON(pmd_sect(*pmd));
+ struct rb_node **node, *parent;
+ struct hyp_shared_pfn *this;
+ int ret = 0;
- if (pmd_none(*pmd)) {
- pte = pte_alloc_one_kernel(NULL);
- if (!pte) {
- kvm_err("Cannot allocate Hyp pte\n");
- return -ENOMEM;
- }
- kvm_pmd_populate(pmd, pte);
- get_page(virt_to_page(pmd));
- }
+ mutex_lock(&hyp_shared_pfns_lock);
+ this = find_shared_pfn(pfn, &node, &parent);
+ if (WARN_ON(!this)) {
+ ret = -ENOENT;
+ goto unlock;
+ }
- next = pmd_addr_end(addr, end);
+ this->count--;
+ if (this->count)
+ goto unlock;
- create_hyp_pte_mappings(pmd, addr, next, pfn, prot);
- pfn += (next - addr) >> PAGE_SHIFT;
- } while (addr = next, addr != end);
+ rb_erase(&this->node, &hyp_shared_pfns);
+ kfree(this);
+ ret = kvm_call_hyp_nvhe(__pkvm_host_unshare_hyp, pfn, 1);
+unlock:
+ mutex_unlock(&hyp_shared_pfns_lock);
- return 0;
+ return ret;
}
-static int create_hyp_pud_mappings(p4d_t *p4d, unsigned long start,
- unsigned long end, unsigned long pfn,
- pgprot_t prot)
+int kvm_share_hyp(void *from, void *to)
{
- pud_t *pud;
- pmd_t *pmd;
- unsigned long addr, next;
+ phys_addr_t start, end, cur;
+ u64 pfn;
int ret;
- addr = start;
- do {
- pud = pud_offset(p4d, addr);
-
- if (pud_none_or_clear_bad(pud)) {
- pmd = pmd_alloc_one(NULL, addr);
- if (!pmd) {
- kvm_err("Cannot allocate Hyp pmd\n");
- return -ENOMEM;
- }
- kvm_pud_populate(pud, pmd);
- get_page(virt_to_page(pud));
- }
-
- next = pud_addr_end(addr, end);
- ret = create_hyp_pmd_mappings(pud, addr, next, pfn, prot);
- if (ret)
- return ret;
- pfn += (next - addr) >> PAGE_SHIFT;
- } while (addr = next, addr != end);
-
- return 0;
-}
-
-static int create_hyp_p4d_mappings(pgd_t *pgd, unsigned long start,
- unsigned long end, unsigned long pfn,
- pgprot_t prot)
-{
- p4d_t *p4d;
- pud_t *pud;
- unsigned long addr, next;
- int ret;
+ if (is_kernel_in_hyp_mode())
+ return 0;
- addr = start;
- do {
- p4d = p4d_offset(pgd, addr);
+ /*
+ * The share hcall maps things in the 'fixed-offset' region of the hyp
+ * VA space, so we can only share physically contiguous data-structures
+ * for now.
+ */
+ if (is_vmalloc_or_module_addr(from) || is_vmalloc_or_module_addr(to))
+ return -EINVAL;
- if (p4d_none(*p4d)) {
- pud = pud_alloc_one(NULL, addr);
- if (!pud) {
- kvm_err("Cannot allocate Hyp pud\n");
- return -ENOMEM;
- }
- kvm_p4d_populate(p4d, pud);
- get_page(virt_to_page(p4d));
- }
+ if (kvm_host_owns_hyp_mappings())
+ return create_hyp_mappings(from, to, PAGE_HYP);
- next = p4d_addr_end(addr, end);
- ret = create_hyp_pud_mappings(p4d, addr, next, pfn, prot);
+ start = ALIGN_DOWN(__pa(from), PAGE_SIZE);
+ end = PAGE_ALIGN(__pa(to));
+ for (cur = start; cur < end; cur += PAGE_SIZE) {
+ pfn = __phys_to_pfn(cur);
+ ret = share_pfn_hyp(pfn);
if (ret)
return ret;
- pfn += (next - addr) >> PAGE_SHIFT;
- } while (addr = next, addr != end);
+ }
return 0;
}
-static int __create_hyp_mappings(pgd_t *pgdp, unsigned long ptrs_per_pgd,
- unsigned long start, unsigned long end,
- unsigned long pfn, pgprot_t prot)
+void kvm_unshare_hyp(void *from, void *to)
{
- pgd_t *pgd;
- p4d_t *p4d;
- unsigned long addr, next;
- int err = 0;
+ phys_addr_t start, end, cur;
+ u64 pfn;
- mutex_lock(&kvm_hyp_pgd_mutex);
- addr = start & PAGE_MASK;
- end = PAGE_ALIGN(end);
- do {
- pgd = pgdp + kvm_pgd_index(addr, ptrs_per_pgd);
-
- if (pgd_none(*pgd)) {
- p4d = p4d_alloc_one(NULL, addr);
- if (!p4d) {
- kvm_err("Cannot allocate Hyp p4d\n");
- err = -ENOMEM;
- goto out;
- }
- kvm_pgd_populate(pgd, p4d);
- get_page(virt_to_page(pgd));
- }
-
- next = pgd_addr_end(addr, end);
- err = create_hyp_p4d_mappings(pgd, addr, next, pfn, prot);
- if (err)
- goto out;
- pfn += (next - addr) >> PAGE_SHIFT;
- } while (addr = next, addr != end);
-out:
- mutex_unlock(&kvm_hyp_pgd_mutex);
- return err;
-}
+ if (is_kernel_in_hyp_mode() || kvm_host_owns_hyp_mappings() || !from)
+ return;
-static phys_addr_t kvm_kaddr_to_phys(void *kaddr)
-{
- if (!is_vmalloc_addr(kaddr)) {
- BUG_ON(!virt_addr_valid(kaddr));
- return __pa(kaddr);
- } else {
- return page_to_phys(vmalloc_to_page(kaddr)) +
- offset_in_page(kaddr);
+ start = ALIGN_DOWN(__pa(from), PAGE_SIZE);
+ end = PAGE_ALIGN(__pa(to));
+ for (cur = start; cur < end; cur += PAGE_SIZE) {
+ pfn = __phys_to_pfn(cur);
+ WARN_ON(unshare_pfn_hyp(pfn));
}
}
@@ -856,7 +588,7 @@ static phys_addr_t kvm_kaddr_to_phys(void *kaddr)
* in Hyp-mode mapping (modulo HYP_PAGE_OFFSET) to the same underlying
* physical pages.
*/
-int create_hyp_mappings(void *from, void *to, pgprot_t prot)
+int create_hyp_mappings(void *from, void *to, enum kvm_pgtable_prot prot)
{
phys_addr_t phys_addr;
unsigned long virt_addr;
@@ -866,6 +598,9 @@ int create_hyp_mappings(void *from, void *to, pgprot_t prot)
if (is_kernel_in_hyp_mode())
return 0;
+ if (!kvm_host_owns_hyp_mappings())
+ return -EPERM;
+
start = start & PAGE_MASK;
end = PAGE_ALIGN(end);
@@ -873,9 +608,7 @@ int create_hyp_mappings(void *from, void *to, pgprot_t prot)
int err;
phys_addr = kvm_kaddr_to_phys(from + virt_addr - start);
- err = __create_hyp_mappings(hyp_pgd, PTRS_PER_PGD,
- virt_addr, virt_addr + PAGE_SIZE,
- __phys_to_pfn(phys_addr),
+ err = __create_hyp_mappings(virt_addr, PAGE_SIZE, phys_addr,
prot);
if (err)
return err;
@@ -884,10 +617,38 @@ int create_hyp_mappings(void *from, void *to, pgprot_t prot)
return 0;
}
-static int __create_hyp_private_mapping(phys_addr_t phys_addr, size_t size,
- unsigned long *haddr, pgprot_t prot)
+static int __hyp_alloc_private_va_range(unsigned long base)
+{
+ lockdep_assert_held(&kvm_hyp_pgd_mutex);
+
+ if (!PAGE_ALIGNED(base))
+ return -EINVAL;
+
+ /*
+ * Verify that BIT(VA_BITS - 1) hasn't been flipped by
+ * allocating the new area, as it would indicate we've
+ * overflowed the idmap/IO address range.
+ */
+ if ((base ^ io_map_base) & BIT(VA_BITS - 1))
+ return -ENOMEM;
+
+ io_map_base = base;
+
+ return 0;
+}
+
+/**
+ * hyp_alloc_private_va_range - Allocates a private VA range.
+ * @size: The size of the VA range to reserve.
+ * @haddr: The hypervisor virtual start address of the allocation.
+ *
+ * The private virtual address (VA) range is allocated below io_map_base
+ * and aligned based on the order of @size.
+ *
+ * Return: 0 on success or negative error code on failure.
+ */
+int hyp_alloc_private_va_range(size_t size, unsigned long *haddr)
{
- pgd_t *pgd = hyp_pgd;
unsigned long base;
int ret = 0;
@@ -895,42 +656,94 @@ static int __create_hyp_private_mapping(phys_addr_t phys_addr, size_t size,
/*
* This assumes that we have enough space below the idmap
- * page to allocate our VAs. If not, the check below will
- * kick. A potential alternative would be to detect that
- * overflow and switch to an allocation above the idmap.
+ * page to allocate our VAs. If not, the check in
+ * __hyp_alloc_private_va_range() will kick. A potential
+ * alternative would be to detect that overflow and switch
+ * to an allocation above the idmap.
*
* The allocated size is always a multiple of PAGE_SIZE.
*/
- size = PAGE_ALIGN(size + offset_in_page(phys_addr));
+ size = PAGE_ALIGN(size);
base = io_map_base - size;
+ ret = __hyp_alloc_private_va_range(base);
+
+ mutex_unlock(&kvm_hyp_pgd_mutex);
+ if (!ret)
+ *haddr = base;
+
+ return ret;
+}
+
+static int __create_hyp_private_mapping(phys_addr_t phys_addr, size_t size,
+ unsigned long *haddr,
+ enum kvm_pgtable_prot prot)
+{
+ unsigned long addr;
+ int ret = 0;
+
+ if (!kvm_host_owns_hyp_mappings()) {
+ addr = kvm_call_hyp_nvhe(__pkvm_create_private_mapping,
+ phys_addr, size, prot);
+ if (IS_ERR_VALUE(addr))
+ return addr;
+ *haddr = addr;
+
+ return 0;
+ }
+
+ size = PAGE_ALIGN(size + offset_in_page(phys_addr));
+ ret = hyp_alloc_private_va_range(size, &addr);
+ if (ret)
+ return ret;
+
+ ret = __create_hyp_mappings(addr, size, phys_addr, prot);
+ if (ret)
+ return ret;
+
+ *haddr = addr + offset_in_page(phys_addr);
+ return ret;
+}
+
+int create_hyp_stack(phys_addr_t phys_addr, unsigned long *haddr)
+{
+ unsigned long base;
+ size_t size;
+ int ret;
+
+ mutex_lock(&kvm_hyp_pgd_mutex);
/*
- * Verify that BIT(VA_BITS - 1) hasn't been flipped by
- * allocating the new area, as it would indicate we've
- * overflowed the idmap/IO address range.
+ * Efficient stack verification using the NVHE_STACK_SHIFT bit implies
+ * an alignment of our allocation on the order of the size.
*/
- if ((base ^ io_map_base) & BIT(VA_BITS - 1))
- ret = -ENOMEM;
- else
- io_map_base = base;
+ size = NVHE_STACK_SIZE * 2;
+ base = ALIGN_DOWN(io_map_base - size, size);
- mutex_unlock(&kvm_hyp_pgd_mutex);
+ ret = __hyp_alloc_private_va_range(base);
- if (ret)
- goto out;
+ mutex_unlock(&kvm_hyp_pgd_mutex);
- if (__kvm_cpu_uses_extended_idmap())
- pgd = boot_hyp_pgd;
+ if (ret) {
+ kvm_err("Cannot allocate hyp stack guard page\n");
+ return ret;
+ }
- ret = __create_hyp_mappings(pgd, __kvm_idmap_ptrs_per_pgd(),
- base, base + size,
- __phys_to_pfn(phys_addr), prot);
+ /*
+ * Since the stack grows downwards, map the stack to the page
+ * at the higher address and leave the lower guard page
+ * unbacked.
+ *
+ * Any valid stack address now has the NVHE_STACK_SHIFT bit as 1
+ * and addresses corresponding to the guard page have the
+ * NVHE_STACK_SHIFT bit as 0 - this is used for overflow detection.
+ */
+ ret = __create_hyp_mappings(base + NVHE_STACK_SIZE, NVHE_STACK_SIZE,
+ phys_addr, PAGE_HYP);
if (ret)
- goto out;
+ kvm_err("Cannot map hyp stack\n");
- *haddr = base + offset_in_page(phys_addr);
+ *haddr = base + size;
-out:
return ret;
}
@@ -948,6 +761,9 @@ int create_hyp_io_mappings(phys_addr_t phys_addr, size_t size,
unsigned long addr;
int ret;
+ if (is_protected_kvm_enabled())
+ return -EPERM;
+
*kaddr = ioremap(phys_addr, size);
if (!*kaddr)
return -ENOMEM;
@@ -995,38 +811,217 @@ int create_hyp_exec_mappings(phys_addr_t phys_addr, size_t size,
return 0;
}
+static struct kvm_pgtable_mm_ops kvm_user_mm_ops = {
+ /* We shouldn't need any other callback to walk the PT */
+ .phys_to_virt = kvm_host_va,
+};
+
+static int get_user_mapping_size(struct kvm *kvm, u64 addr)
+{
+ struct kvm_pgtable pgt = {
+ .pgd = (kvm_pteref_t)kvm->mm->pgd,
+ .ia_bits = vabits_actual,
+ .start_level = (KVM_PGTABLE_LAST_LEVEL -
+ ARM64_HW_PGTABLE_LEVELS(pgt.ia_bits) + 1),
+ .mm_ops = &kvm_user_mm_ops,
+ };
+ unsigned long flags;
+ kvm_pte_t pte = 0; /* Keep GCC quiet... */
+ s8 level = S8_MAX;
+ int ret;
+
+ /*
+ * Disable IRQs so that we hazard against a concurrent
+ * teardown of the userspace page tables (which relies on
+ * IPI-ing threads).
+ */
+ local_irq_save(flags);
+ ret = kvm_pgtable_get_leaf(&pgt, addr, &pte, &level);
+ local_irq_restore(flags);
+
+ if (ret)
+ return ret;
+
+ /*
+ * Not seeing an error, but not updating level? Something went
+ * deeply wrong...
+ */
+ if (WARN_ON(level > KVM_PGTABLE_LAST_LEVEL))
+ return -EFAULT;
+ if (WARN_ON(level < KVM_PGTABLE_FIRST_LEVEL))
+ return -EFAULT;
+
+ /* Oops, the userspace PTs are gone... Replay the fault */
+ if (!kvm_pte_valid(pte))
+ return -EAGAIN;
+
+ return BIT(ARM64_HW_PGTABLE_LEVEL_SHIFT(level));
+}
+
+static struct kvm_pgtable_mm_ops kvm_s2_mm_ops = {
+ .zalloc_page = stage2_memcache_zalloc_page,
+ .zalloc_pages_exact = kvm_s2_zalloc_pages_exact,
+ .free_pages_exact = kvm_s2_free_pages_exact,
+ .free_unlinked_table = stage2_free_unlinked_table,
+ .get_page = kvm_host_get_page,
+ .put_page = kvm_s2_put_page,
+ .page_count = kvm_host_page_count,
+ .phys_to_virt = kvm_host_va,
+ .virt_to_phys = kvm_host_pa,
+ .dcache_clean_inval_poc = clean_dcache_guest_page,
+ .icache_inval_pou = invalidate_icache_guest_page,
+};
+
+static int kvm_init_ipa_range(struct kvm_s2_mmu *mmu, unsigned long type)
+{
+ u32 kvm_ipa_limit = get_kvm_ipa_limit();
+ u64 mmfr0, mmfr1;
+ u32 phys_shift;
+
+ if (type & ~KVM_VM_TYPE_ARM_IPA_SIZE_MASK)
+ return -EINVAL;
+
+ phys_shift = KVM_VM_TYPE_ARM_IPA_SIZE(type);
+ if (is_protected_kvm_enabled()) {
+ phys_shift = kvm_ipa_limit;
+ } else if (phys_shift) {
+ if (phys_shift > kvm_ipa_limit ||
+ phys_shift < ARM64_MIN_PARANGE_BITS)
+ return -EINVAL;
+ } else {
+ phys_shift = KVM_PHYS_SHIFT;
+ if (phys_shift > kvm_ipa_limit) {
+ pr_warn_once("%s using unsupported default IPA limit, upgrade your VMM\n",
+ current->comm);
+ return -EINVAL;
+ }
+ }
+
+ mmfr0 = read_sanitised_ftr_reg(SYS_ID_AA64MMFR0_EL1);
+ mmfr1 = read_sanitised_ftr_reg(SYS_ID_AA64MMFR1_EL1);
+ mmu->vtcr = kvm_get_vtcr(mmfr0, mmfr1, phys_shift);
+
+ return 0;
+}
+
+/*
+ * Assume that @pgt is valid and unlinked from the KVM MMU to free the
+ * page-table without taking the kvm_mmu_lock and without performing any
+ * TLB invalidations.
+ *
+ * Also, the range of addresses can be large enough to cause need_resched
+ * warnings, for instance on CONFIG_PREEMPT_NONE kernels. Hence, invoke
+ * cond_resched() periodically to prevent hogging the CPU for a long time
+ * and schedule something else, if required.
+ */
+static void stage2_destroy_range(struct kvm_pgtable *pgt, phys_addr_t addr,
+ phys_addr_t end)
+{
+ u64 next;
+
+ do {
+ next = stage2_range_addr_end(addr, end);
+ KVM_PGT_FN(kvm_pgtable_stage2_destroy_range)(pgt, addr,
+ next - addr);
+ if (next != end)
+ cond_resched();
+ } while (addr = next, addr != end);
+}
+
+static void kvm_stage2_destroy(struct kvm_pgtable *pgt)
+{
+ unsigned int ia_bits = VTCR_EL2_IPA(pgt->mmu->vtcr);
+
+ stage2_destroy_range(pgt, 0, BIT(ia_bits));
+ KVM_PGT_FN(kvm_pgtable_stage2_destroy_pgd)(pgt);
+}
+
/**
- * kvm_alloc_stage2_pgd - allocate level-1 table for stage-2 translation.
- * @kvm: The KVM struct pointer for the VM.
+ * kvm_init_stage2_mmu - Initialise a S2 MMU structure
+ * @kvm: The pointer to the KVM structure
+ * @mmu: The pointer to the s2 MMU structure
+ * @type: The machine type of the virtual machine
+ *
+ * Allocates only the stage-2 HW PGD level table(s).
+ * Note we don't need locking here as this is only called in two cases:
*
- * Allocates only the stage-2 HW PGD level table(s) of size defined by
- * stage2_pgd_size(kvm).
+ * - when the VM is created, which can't race against anything
*
- * Note we don't need locking here as this is only called when the VM is
- * created, which can only be done once.
+ * - when secondary kvm_s2_mmu structures are initialised for NV
+ * guests, and the caller must hold kvm->lock as this is called on a
+ * per-vcpu basis.
*/
-int kvm_alloc_stage2_pgd(struct kvm *kvm)
+int kvm_init_stage2_mmu(struct kvm *kvm, struct kvm_s2_mmu *mmu, unsigned long type)
{
- phys_addr_t pgd_phys;
- pgd_t *pgd;
+ int cpu, err;
+ struct kvm_pgtable *pgt;
+
+ /*
+ * If we already have our page tables in place, and that the
+ * MMU context is the canonical one, we have a bug somewhere,
+ * as this is only supposed to ever happen once per VM.
+ *
+ * Otherwise, we're building nested page tables, and that's
+ * probably because userspace called KVM_ARM_VCPU_INIT more
+ * than once on the same vcpu. Since that's actually legal,
+ * don't kick a fuss and leave gracefully.
+ */
+ if (mmu->pgt != NULL) {
+ if (kvm_is_nested_s2_mmu(kvm, mmu))
+ return 0;
- if (kvm->arch.pgd != NULL) {
kvm_err("kvm_arch already initialized?\n");
return -EINVAL;
}
- /* Allocate the HW PGD, making sure that each page gets its own refcount */
- pgd = alloc_pages_exact(stage2_pgd_size(kvm), GFP_KERNEL | __GFP_ZERO);
- if (!pgd)
+ err = kvm_init_ipa_range(mmu, type);
+ if (err)
+ return err;
+
+ pgt = kzalloc(sizeof(*pgt), GFP_KERNEL_ACCOUNT);
+ if (!pgt)
return -ENOMEM;
- pgd_phys = virt_to_phys(pgd);
- if (WARN_ON(pgd_phys & ~kvm_vttbr_baddr_mask(kvm)))
- return -EINVAL;
+ mmu->arch = &kvm->arch;
+ err = KVM_PGT_FN(kvm_pgtable_stage2_init)(pgt, mmu, &kvm_s2_mm_ops);
+ if (err)
+ goto out_free_pgtable;
+
+ mmu->pgt = pgt;
+ if (is_protected_kvm_enabled())
+ return 0;
+
+ mmu->last_vcpu_ran = alloc_percpu(typeof(*mmu->last_vcpu_ran));
+ if (!mmu->last_vcpu_ran) {
+ err = -ENOMEM;
+ goto out_destroy_pgtable;
+ }
+
+ for_each_possible_cpu(cpu)
+ *per_cpu_ptr(mmu->last_vcpu_ran, cpu) = -1;
+
+ /* The eager page splitting is disabled by default */
+ mmu->split_page_chunk_size = KVM_ARM_EAGER_SPLIT_CHUNK_SIZE_DEFAULT;
+ mmu->split_page_cache.gfp_zero = __GFP_ZERO;
+
+ mmu->pgd_phys = __pa(pgt->pgd);
+
+ if (kvm_is_nested_s2_mmu(kvm, mmu))
+ kvm_init_nested_s2_mmu(mmu);
- kvm->arch.pgd = pgd;
- kvm->arch.pgd_phys = pgd_phys;
return 0;
+
+out_destroy_pgtable:
+ kvm_stage2_destroy(pgt);
+out_free_pgtable:
+ kfree(pgt);
+ return err;
+}
+
+void kvm_uninit_stage2_mmu(struct kvm *kvm)
+{
+ kvm_free_stage2_pgd(&kvm->arch.mmu);
+ kvm_mmu_free_memory_cache(&kvm->arch.mmu.split_page_cache);
}
static void stage2_unmap_memslot(struct kvm *kvm,
@@ -1050,10 +1045,11 @@ static void stage2_unmap_memslot(struct kvm *kvm,
* +--------------------------------------------+
*/
do {
- struct vm_area_struct *vma = find_vma(current->mm, hva);
+ struct vm_area_struct *vma;
hva_t vm_start, vm_end;
- if (!vma || vma->vm_start >= reg_end)
+ vma = find_vma_intersection(current->mm, hva, reg_end);
+ if (!vma)
break;
/*
@@ -1064,7 +1060,7 @@ static void stage2_unmap_memslot(struct kvm *kvm,
if (!(vma->vm_flags & VM_PFNMAP)) {
gpa_t gpa = addr + (vm_start - memslot->userspace_addr);
- unmap_stage2_range(kvm, gpa, vm_end - vm_start);
+ kvm_stage2_unmap_range(&kvm->arch.mmu, gpa, vm_end - vm_start, true);
}
hva = vm_end;
} while (hva < reg_end);
@@ -1081,378 +1077,91 @@ void stage2_unmap_vm(struct kvm *kvm)
{
struct kvm_memslots *slots;
struct kvm_memory_slot *memslot;
- int idx;
+ int idx, bkt;
idx = srcu_read_lock(&kvm->srcu);
mmap_read_lock(current->mm);
- spin_lock(&kvm->mmu_lock);
+ write_lock(&kvm->mmu_lock);
slots = kvm_memslots(kvm);
- kvm_for_each_memslot(memslot, slots)
+ kvm_for_each_memslot(memslot, bkt, slots)
stage2_unmap_memslot(kvm, memslot);
- spin_unlock(&kvm->mmu_lock);
+ kvm_nested_s2_unmap(kvm, true);
+
+ write_unlock(&kvm->mmu_lock);
mmap_read_unlock(current->mm);
srcu_read_unlock(&kvm->srcu, idx);
}
-/**
- * kvm_free_stage2_pgd - free all stage-2 tables
- * @kvm: The KVM struct pointer for the VM.
- *
- * Walks the level-1 page table pointed to by kvm->arch.pgd and frees all
- * underlying level-2 and level-3 tables before freeing the actual level-1 table
- * and setting the struct pointer to NULL.
- */
-void kvm_free_stage2_pgd(struct kvm *kvm)
-{
- void *pgd = NULL;
-
- spin_lock(&kvm->mmu_lock);
- if (kvm->arch.pgd) {
- unmap_stage2_range(kvm, 0, kvm_phys_size(kvm));
- pgd = READ_ONCE(kvm->arch.pgd);
- kvm->arch.pgd = NULL;
- kvm->arch.pgd_phys = 0;
- }
- spin_unlock(&kvm->mmu_lock);
-
- /* Free the HW pgd, one page at a time */
- if (pgd)
- free_pages_exact(pgd, stage2_pgd_size(kvm));
-}
-
-static p4d_t *stage2_get_p4d(struct kvm *kvm, struct kvm_mmu_memory_cache *cache,
- phys_addr_t addr)
-{
- pgd_t *pgd;
- p4d_t *p4d;
-
- pgd = kvm->arch.pgd + stage2_pgd_index(kvm, addr);
- if (stage2_pgd_none(kvm, *pgd)) {
- if (!cache)
- return NULL;
- p4d = mmu_memory_cache_alloc(cache);
- stage2_pgd_populate(kvm, pgd, p4d);
- get_page(virt_to_page(pgd));
- }
-
- return stage2_p4d_offset(kvm, pgd, addr);
-}
-
-static pud_t *stage2_get_pud(struct kvm *kvm, struct kvm_mmu_memory_cache *cache,
- phys_addr_t addr)
-{
- p4d_t *p4d;
- pud_t *pud;
-
- p4d = stage2_get_p4d(kvm, cache, addr);
- if (stage2_p4d_none(kvm, *p4d)) {
- if (!cache)
- return NULL;
- pud = mmu_memory_cache_alloc(cache);
- stage2_p4d_populate(kvm, p4d, pud);
- get_page(virt_to_page(p4d));
- }
-
- return stage2_pud_offset(kvm, p4d, addr);
-}
-
-static pmd_t *stage2_get_pmd(struct kvm *kvm, struct kvm_mmu_memory_cache *cache,
- phys_addr_t addr)
+void kvm_free_stage2_pgd(struct kvm_s2_mmu *mmu)
{
- pud_t *pud;
- pmd_t *pmd;
+ struct kvm *kvm = kvm_s2_mmu_to_kvm(mmu);
+ struct kvm_pgtable *pgt = NULL;
- pud = stage2_get_pud(kvm, cache, addr);
- if (!pud || stage2_pud_huge(kvm, *pud))
- return NULL;
-
- if (stage2_pud_none(kvm, *pud)) {
- if (!cache)
- return NULL;
- pmd = mmu_memory_cache_alloc(cache);
- stage2_pud_populate(kvm, pud, pmd);
- get_page(virt_to_page(pud));
+ write_lock(&kvm->mmu_lock);
+ pgt = mmu->pgt;
+ if (pgt) {
+ mmu->pgd_phys = 0;
+ mmu->pgt = NULL;
+ free_percpu(mmu->last_vcpu_ran);
}
- return stage2_pmd_offset(kvm, pud, addr);
-}
-
-static int stage2_set_pmd_huge(struct kvm *kvm, struct kvm_mmu_memory_cache
- *cache, phys_addr_t addr, const pmd_t *new_pmd)
-{
- pmd_t *pmd, old_pmd;
+ if (kvm_is_nested_s2_mmu(kvm, mmu))
+ kvm_init_nested_s2_mmu(mmu);
-retry:
- pmd = stage2_get_pmd(kvm, cache, addr);
- VM_BUG_ON(!pmd);
+ write_unlock(&kvm->mmu_lock);
- old_pmd = *pmd;
- /*
- * Multiple vcpus faulting on the same PMD entry, can
- * lead to them sequentially updating the PMD with the
- * same value. Following the break-before-make
- * (pmd_clear() followed by tlb_flush()) process can
- * hinder forward progress due to refaults generated
- * on missing translations.
- *
- * Skip updating the page table if the entry is
- * unchanged.
- */
- if (pmd_val(old_pmd) == pmd_val(*new_pmd))
- return 0;
-
- if (pmd_present(old_pmd)) {
- /*
- * If we already have PTE level mapping for this block,
- * we must unmap it to avoid inconsistent TLB state and
- * leaking the table page. We could end up in this situation
- * if the memory slot was marked for dirty logging and was
- * reverted, leaving PTE level mappings for the pages accessed
- * during the period. So, unmap the PTE level mapping for this
- * block and retry, as we could have released the upper level
- * table in the process.
- *
- * Normal THP split/merge follows mmu_notifier callbacks and do
- * get handled accordingly.
- */
- if (!pmd_thp_or_huge(old_pmd)) {
- unmap_stage2_range(kvm, addr & S2_PMD_MASK, S2_PMD_SIZE);
- goto retry;
- }
- /*
- * Mapping in huge pages should only happen through a
- * fault. If a page is merged into a transparent huge
- * page, the individual subpages of that huge page
- * should be unmapped through MMU notifiers before we
- * get here.
- *
- * Merging of CompoundPages is not supported; they
- * should become splitting first, unmapped, merged,
- * and mapped back in on-demand.
- */
- WARN_ON_ONCE(pmd_pfn(old_pmd) != pmd_pfn(*new_pmd));
- pmd_clear(pmd);
- kvm_tlb_flush_vmid_ipa(kvm, addr);
- } else {
- get_page(virt_to_page(pmd));
+ if (pgt) {
+ kvm_stage2_destroy(pgt);
+ kfree(pgt);
}
-
- kvm_set_pmd(pmd, *new_pmd);
- return 0;
}
-static int stage2_set_pud_huge(struct kvm *kvm, struct kvm_mmu_memory_cache *cache,
- phys_addr_t addr, const pud_t *new_pudp)
+static void hyp_mc_free_fn(void *addr, void *mc)
{
- pud_t *pudp, old_pud;
-
-retry:
- pudp = stage2_get_pud(kvm, cache, addr);
- VM_BUG_ON(!pudp);
-
- old_pud = *pudp;
-
- /*
- * A large number of vcpus faulting on the same stage 2 entry,
- * can lead to a refault due to the stage2_pud_clear()/tlb_flush().
- * Skip updating the page tables if there is no change.
- */
- if (pud_val(old_pud) == pud_val(*new_pudp))
- return 0;
-
- if (stage2_pud_present(kvm, old_pud)) {
- /*
- * If we already have table level mapping for this block, unmap
- * the range for this block and retry.
- */
- if (!stage2_pud_huge(kvm, old_pud)) {
- unmap_stage2_range(kvm, addr & S2_PUD_MASK, S2_PUD_SIZE);
- goto retry;
- }
+ struct kvm_hyp_memcache *memcache = mc;
- WARN_ON_ONCE(kvm_pud_pfn(old_pud) != kvm_pud_pfn(*new_pudp));
- stage2_pud_clear(kvm, pudp);
- kvm_tlb_flush_vmid_ipa(kvm, addr);
- } else {
- get_page(virt_to_page(pudp));
- }
+ if (memcache->flags & HYP_MEMCACHE_ACCOUNT_STAGE2)
+ kvm_account_pgtable_pages(addr, -1);
- kvm_set_pud(pudp, *new_pudp);
- return 0;
+ free_page((unsigned long)addr);
}
-/*
- * stage2_get_leaf_entry - walk the stage2 VM page tables and return
- * true if a valid and present leaf-entry is found. A pointer to the
- * leaf-entry is returned in the appropriate level variable - pudpp,
- * pmdpp, ptepp.
- */
-static bool stage2_get_leaf_entry(struct kvm *kvm, phys_addr_t addr,
- pud_t **pudpp, pmd_t **pmdpp, pte_t **ptepp)
+static void *hyp_mc_alloc_fn(void *mc)
{
- pud_t *pudp;
- pmd_t *pmdp;
- pte_t *ptep;
-
- *pudpp = NULL;
- *pmdpp = NULL;
- *ptepp = NULL;
-
- pudp = stage2_get_pud(kvm, NULL, addr);
- if (!pudp || stage2_pud_none(kvm, *pudp) || !stage2_pud_present(kvm, *pudp))
- return false;
-
- if (stage2_pud_huge(kvm, *pudp)) {
- *pudpp = pudp;
- return true;
- }
-
- pmdp = stage2_pmd_offset(kvm, pudp, addr);
- if (!pmdp || pmd_none(*pmdp) || !pmd_present(*pmdp))
- return false;
-
- if (pmd_thp_or_huge(*pmdp)) {
- *pmdpp = pmdp;
- return true;
- }
+ struct kvm_hyp_memcache *memcache = mc;
+ void *addr;
- ptep = pte_offset_kernel(pmdp, addr);
- if (!ptep || pte_none(*ptep) || !pte_present(*ptep))
- return false;
+ addr = (void *)__get_free_page(GFP_KERNEL_ACCOUNT);
+ if (addr && memcache->flags & HYP_MEMCACHE_ACCOUNT_STAGE2)
+ kvm_account_pgtable_pages(addr, 1);
- *ptepp = ptep;
- return true;
+ return addr;
}
-static bool stage2_is_exec(struct kvm *kvm, phys_addr_t addr)
+void free_hyp_memcache(struct kvm_hyp_memcache *mc)
{
- pud_t *pudp;
- pmd_t *pmdp;
- pte_t *ptep;
- bool found;
-
- found = stage2_get_leaf_entry(kvm, addr, &pudp, &pmdp, &ptep);
- if (!found)
- return false;
+ if (!is_protected_kvm_enabled())
+ return;
- if (pudp)
- return kvm_s2pud_exec(pudp);
- else if (pmdp)
- return kvm_s2pmd_exec(pmdp);
- else
- return kvm_s2pte_exec(ptep);
+ kfree(mc->mapping);
+ __free_hyp_memcache(mc, hyp_mc_free_fn, kvm_host_va, mc);
}
-static int stage2_set_pte(struct kvm *kvm, struct kvm_mmu_memory_cache *cache,
- phys_addr_t addr, const pte_t *new_pte,
- unsigned long flags)
+int topup_hyp_memcache(struct kvm_hyp_memcache *mc, unsigned long min_pages)
{
- pud_t *pud;
- pmd_t *pmd;
- pte_t *pte, old_pte;
- bool iomap = flags & KVM_S2PTE_FLAG_IS_IOMAP;
- bool logging_active = flags & KVM_S2_FLAG_LOGGING_ACTIVE;
-
- VM_BUG_ON(logging_active && !cache);
-
- /* Create stage-2 page table mapping - Levels 0 and 1 */
- pud = stage2_get_pud(kvm, cache, addr);
- if (!pud) {
- /*
- * Ignore calls from kvm_set_spte_hva for unallocated
- * address ranges.
- */
+ if (!is_protected_kvm_enabled())
return 0;
- }
- /*
- * While dirty page logging - dissolve huge PUD, then continue
- * on to allocate page.
- */
- if (logging_active)
- stage2_dissolve_pud(kvm, addr, pud);
-
- if (stage2_pud_none(kvm, *pud)) {
- if (!cache)
- return 0; /* ignore calls from kvm_set_spte_hva */
- pmd = mmu_memory_cache_alloc(cache);
- stage2_pud_populate(kvm, pud, pmd);
- get_page(virt_to_page(pud));
- }
-
- pmd = stage2_pmd_offset(kvm, pud, addr);
- if (!pmd) {
- /*
- * Ignore calls from kvm_set_spte_hva for unallocated
- * address ranges.
- */
- return 0;
- }
-
- /*
- * While dirty page logging - dissolve huge PMD, then continue on to
- * allocate page.
- */
- if (logging_active)
- stage2_dissolve_pmd(kvm, addr, pmd);
-
- /* Create stage-2 page mappings - Level 2 */
- if (pmd_none(*pmd)) {
- if (!cache)
- return 0; /* ignore calls from kvm_set_spte_hva */
- pte = mmu_memory_cache_alloc(cache);
- kvm_pmd_populate(pmd, pte);
- get_page(virt_to_page(pmd));
- }
-
- pte = pte_offset_kernel(pmd, addr);
-
- if (iomap && pte_present(*pte))
- return -EFAULT;
-
- /* Create 2nd stage page table mapping - Level 3 */
- old_pte = *pte;
- if (pte_present(old_pte)) {
- /* Skip page table update if there is no change */
- if (pte_val(old_pte) == pte_val(*new_pte))
- return 0;
-
- kvm_set_pte(pte, __pte(0));
- kvm_tlb_flush_vmid_ipa(kvm, addr);
- } else {
- get_page(virt_to_page(pte));
+ if (!mc->mapping) {
+ mc->mapping = kzalloc(sizeof(struct pkvm_mapping), GFP_KERNEL_ACCOUNT);
+ if (!mc->mapping)
+ return -ENOMEM;
}
- kvm_set_pte(pte, *new_pte);
- return 0;
-}
-
-#ifndef __HAVE_ARCH_PTEP_TEST_AND_CLEAR_YOUNG
-static int stage2_ptep_test_and_clear_young(pte_t *pte)
-{
- if (pte_young(*pte)) {
- *pte = pte_mkold(*pte);
- return 1;
- }
- return 0;
-}
-#else
-static int stage2_ptep_test_and_clear_young(pte_t *pte)
-{
- return __ptep_test_and_clear_young(pte);
-}
-#endif
-
-static int stage2_pmdp_test_and_clear_young(pmd_t *pmd)
-{
- return stage2_ptep_test_and_clear_young((pte_t *)pmd);
-}
-
-static int stage2_pudp_test_and_clear_young(pud_t *pud)
-{
- return stage2_ptep_test_and_clear_young((pte_t *)pud);
+ return __topup_hyp_memcache(mc, min_pages, hyp_mc_alloc_fn,
+ kvm_host_pa, mc);
}
/**
@@ -1462,166 +1171,55 @@ static int stage2_pudp_test_and_clear_young(pud_t *pud)
* @guest_ipa: The IPA at which to insert the mapping
* @pa: The physical address of the device
* @size: The size of the mapping
+ * @writable: Whether or not to create a writable mapping
*/
int kvm_phys_addr_ioremap(struct kvm *kvm, phys_addr_t guest_ipa,
phys_addr_t pa, unsigned long size, bool writable)
{
- phys_addr_t addr, end;
+ phys_addr_t addr;
int ret = 0;
- unsigned long pfn;
- struct kvm_mmu_memory_cache cache = { 0, };
-
- end = (guest_ipa + size + PAGE_SIZE - 1) & PAGE_MASK;
- pfn = __phys_to_pfn(pa);
-
- for (addr = guest_ipa; addr < end; addr += PAGE_SIZE) {
- pte_t pte = kvm_pfn_pte(pfn, PAGE_S2_DEVICE);
-
- if (writable)
- pte = kvm_s2pte_mkwrite(pte);
-
- ret = mmu_topup_memory_cache(&cache,
- kvm_mmu_cache_min_pages(kvm),
- KVM_NR_MEM_OBJS);
+ struct kvm_mmu_memory_cache cache = { .gfp_zero = __GFP_ZERO };
+ struct kvm_s2_mmu *mmu = &kvm->arch.mmu;
+ struct kvm_pgtable *pgt = mmu->pgt;
+ enum kvm_pgtable_prot prot = KVM_PGTABLE_PROT_DEVICE |
+ KVM_PGTABLE_PROT_R |
+ (writable ? KVM_PGTABLE_PROT_W : 0);
+
+ if (is_protected_kvm_enabled())
+ return -EPERM;
+
+ size += offset_in_page(guest_ipa);
+ guest_ipa &= PAGE_MASK;
+
+ for (addr = guest_ipa; addr < guest_ipa + size; addr += PAGE_SIZE) {
+ ret = kvm_mmu_topup_memory_cache(&cache,
+ kvm_mmu_cache_min_pages(mmu));
if (ret)
- goto out;
- spin_lock(&kvm->mmu_lock);
- ret = stage2_set_pte(kvm, &cache, addr, &pte,
- KVM_S2PTE_FLAG_IS_IOMAP);
- spin_unlock(&kvm->mmu_lock);
+ break;
+
+ write_lock(&kvm->mmu_lock);
+ ret = KVM_PGT_FN(kvm_pgtable_stage2_map)(pgt, addr, PAGE_SIZE,
+ pa, prot, &cache, 0);
+ write_unlock(&kvm->mmu_lock);
if (ret)
- goto out;
+ break;
- pfn++;
+ pa += PAGE_SIZE;
}
-out:
- mmu_free_memory_cache(&cache);
+ kvm_mmu_free_memory_cache(&cache);
return ret;
}
/**
- * stage2_wp_ptes - write protect PMD range
- * @pmd: pointer to pmd entry
- * @addr: range start address
- * @end: range end address
- */
-static void stage2_wp_ptes(pmd_t *pmd, phys_addr_t addr, phys_addr_t end)
-{
- pte_t *pte;
-
- pte = pte_offset_kernel(pmd, addr);
- do {
- if (!pte_none(*pte)) {
- if (!kvm_s2pte_readonly(pte))
- kvm_set_s2pte_readonly(pte);
- }
- } while (pte++, addr += PAGE_SIZE, addr != end);
-}
-
-/**
- * stage2_wp_pmds - write protect PUD range
- * kvm: kvm instance for the VM
- * @pud: pointer to pud entry
- * @addr: range start address
- * @end: range end address
- */
-static void stage2_wp_pmds(struct kvm *kvm, pud_t *pud,
- phys_addr_t addr, phys_addr_t end)
-{
- pmd_t *pmd;
- phys_addr_t next;
-
- pmd = stage2_pmd_offset(kvm, pud, addr);
-
- do {
- next = stage2_pmd_addr_end(kvm, addr, end);
- if (!pmd_none(*pmd)) {
- if (pmd_thp_or_huge(*pmd)) {
- if (!kvm_s2pmd_readonly(pmd))
- kvm_set_s2pmd_readonly(pmd);
- } else {
- stage2_wp_ptes(pmd, addr, next);
- }
- }
- } while (pmd++, addr = next, addr != end);
-}
-
-/**
- * stage2_wp_puds - write protect P4D range
- * @pgd: pointer to pgd entry
- * @addr: range start address
- * @end: range end address
- */
-static void stage2_wp_puds(struct kvm *kvm, p4d_t *p4d,
- phys_addr_t addr, phys_addr_t end)
-{
- pud_t *pud;
- phys_addr_t next;
-
- pud = stage2_pud_offset(kvm, p4d, addr);
- do {
- next = stage2_pud_addr_end(kvm, addr, end);
- if (!stage2_pud_none(kvm, *pud)) {
- if (stage2_pud_huge(kvm, *pud)) {
- if (!kvm_s2pud_readonly(pud))
- kvm_set_s2pud_readonly(pud);
- } else {
- stage2_wp_pmds(kvm, pud, addr, next);
- }
- }
- } while (pud++, addr = next, addr != end);
-}
-
-/**
- * stage2_wp_p4ds - write protect PGD range
- * @pgd: pointer to pgd entry
- * @addr: range start address
- * @end: range end address
- */
-static void stage2_wp_p4ds(struct kvm *kvm, pgd_t *pgd,
- phys_addr_t addr, phys_addr_t end)
-{
- p4d_t *p4d;
- phys_addr_t next;
-
- p4d = stage2_p4d_offset(kvm, pgd, addr);
- do {
- next = stage2_p4d_addr_end(kvm, addr, end);
- if (!stage2_p4d_none(kvm, *p4d))
- stage2_wp_puds(kvm, p4d, addr, next);
- } while (p4d++, addr = next, addr != end);
-}
-
-/**
- * stage2_wp_range() - write protect stage2 memory region range
- * @kvm: The KVM pointer
+ * kvm_stage2_wp_range() - write protect stage2 memory region range
+ * @mmu: The KVM stage-2 MMU pointer
* @addr: Start address of range
* @end: End address of range
*/
-static void stage2_wp_range(struct kvm *kvm, phys_addr_t addr, phys_addr_t end)
+void kvm_stage2_wp_range(struct kvm_s2_mmu *mmu, phys_addr_t addr, phys_addr_t end)
{
- pgd_t *pgd;
- phys_addr_t next;
-
- pgd = kvm->arch.pgd + stage2_pgd_index(kvm, addr);
- do {
- /*
- * Release kvm_mmu_lock periodically if the memory region is
- * large. Otherwise, we may see kernel panics with
- * CONFIG_DETECT_HUNG_TASK, CONFIG_LOCKUP_DETECTOR,
- * CONFIG_LOCKDEP. Additionally, holding the lock too long
- * will also starve other vCPUs. We have to also make sure
- * that the page tables are not freed while we released
- * the lock.
- */
- cond_resched_lock(&kvm->mmu_lock);
- if (!READ_ONCE(kvm->arch.pgd))
- break;
- next = stage2_pgd_addr_end(kvm, addr, end);
- if (stage2_pgd_present(kvm, *pgd))
- stage2_wp_p4ds(kvm, pgd, addr, next);
- } while (pgd++, addr = next, addr != end);
+ stage2_apply_range_resched(mmu, addr, end, KVM_PGT_FN(kvm_pgtable_stage2_wrprotect));
}
/**
@@ -1637,7 +1235,7 @@ static void stage2_wp_range(struct kvm *kvm, phys_addr_t addr, phys_addr_t end)
* Acquires kvm_mmu_lock. Called with kvm->slots_lock mutex acquired,
* serializing operations for VM memory regions.
*/
-void kvm_mmu_wp_memory_region(struct kvm *kvm, int slot)
+static void kvm_mmu_wp_memory_region(struct kvm *kvm, int slot)
{
struct kvm_memslots *slots = kvm_memslots(kvm);
struct kvm_memory_slot *memslot = id_to_memslot(slots, slot);
@@ -1649,56 +1247,76 @@ void kvm_mmu_wp_memory_region(struct kvm *kvm, int slot)
start = memslot->base_gfn << PAGE_SHIFT;
end = (memslot->base_gfn + memslot->npages) << PAGE_SHIFT;
- spin_lock(&kvm->mmu_lock);
- stage2_wp_range(kvm, start, end);
- spin_unlock(&kvm->mmu_lock);
- kvm_flush_remote_tlbs(kvm);
+ write_lock(&kvm->mmu_lock);
+ kvm_stage2_wp_range(&kvm->arch.mmu, start, end);
+ kvm_nested_s2_wp(kvm);
+ write_unlock(&kvm->mmu_lock);
+ kvm_flush_remote_tlbs_memslot(kvm, memslot);
}
/**
- * kvm_mmu_write_protect_pt_masked() - write protect dirty pages
+ * kvm_mmu_split_memory_region() - split the stage 2 blocks into PAGE_SIZE
+ * pages for memory slot
* @kvm: The KVM pointer
- * @slot: The memory slot associated with mask
- * @gfn_offset: The gfn offset in memory slot
- * @mask: The mask of dirty pages at offset 'gfn_offset' in this memory
- * slot to be write protected
+ * @slot: The memory slot to split
*
- * Walks bits set in mask write protects the associated pte's. Caller must
- * acquire kvm_mmu_lock.
+ * Acquires kvm->mmu_lock. Called with kvm->slots_lock mutex acquired,
+ * serializing operations for VM memory regions.
*/
-static void kvm_mmu_write_protect_pt_masked(struct kvm *kvm,
- struct kvm_memory_slot *slot,
- gfn_t gfn_offset, unsigned long mask)
+static void kvm_mmu_split_memory_region(struct kvm *kvm, int slot)
{
- phys_addr_t base_gfn = slot->base_gfn + gfn_offset;
- phys_addr_t start = (base_gfn + __ffs(mask)) << PAGE_SHIFT;
- phys_addr_t end = (base_gfn + __fls(mask) + 1) << PAGE_SHIFT;
+ struct kvm_memslots *slots;
+ struct kvm_memory_slot *memslot;
+ phys_addr_t start, end;
- stage2_wp_range(kvm, start, end);
+ lockdep_assert_held(&kvm->slots_lock);
+
+ slots = kvm_memslots(kvm);
+ memslot = id_to_memslot(slots, slot);
+
+ start = memslot->base_gfn << PAGE_SHIFT;
+ end = (memslot->base_gfn + memslot->npages) << PAGE_SHIFT;
+
+ write_lock(&kvm->mmu_lock);
+ kvm_mmu_split_huge_pages(kvm, start, end);
+ write_unlock(&kvm->mmu_lock);
}
/*
- * kvm_arch_mmu_enable_log_dirty_pt_masked - enable dirty logging for selected
- * dirty pages.
+ * kvm_arch_mmu_enable_log_dirty_pt_masked() - enable dirty logging for selected pages.
+ * @kvm: The KVM pointer
+ * @slot: The memory slot associated with mask
+ * @gfn_offset: The gfn offset in memory slot
+ * @mask: The mask of pages at offset 'gfn_offset' in this memory
+ * slot to enable dirty logging on
*
- * It calls kvm_mmu_write_protect_pt_masked to write protect selected pages to
- * enable dirty logging for them.
+ * Writes protect selected pages to enable dirty logging, and then
+ * splits them to PAGE_SIZE. Caller must acquire kvm->mmu_lock.
*/
void kvm_arch_mmu_enable_log_dirty_pt_masked(struct kvm *kvm,
struct kvm_memory_slot *slot,
gfn_t gfn_offset, unsigned long mask)
{
- kvm_mmu_write_protect_pt_masked(kvm, slot, gfn_offset, mask);
-}
+ phys_addr_t base_gfn = slot->base_gfn + gfn_offset;
+ phys_addr_t start = (base_gfn + __ffs(mask)) << PAGE_SHIFT;
+ phys_addr_t end = (base_gfn + __fls(mask) + 1) << PAGE_SHIFT;
-static void clean_dcache_guest_page(kvm_pfn_t pfn, unsigned long size)
-{
- __clean_dcache_guest_page(pfn, size);
-}
+ lockdep_assert_held_write(&kvm->mmu_lock);
-static void invalidate_icache_guest_page(kvm_pfn_t pfn, unsigned long size)
-{
- __invalidate_icache_guest_page(pfn, size);
+ kvm_stage2_wp_range(&kvm->arch.mmu, start, end);
+
+ /*
+ * Eager-splitting is done when manual-protect is set. We
+ * also check for initially-all-set because we can avoid
+ * eager-splitting if initially-all-set is false.
+ * Initially-all-set equal false implies that huge-pages were
+ * already split when enabling dirty logging: no need to do it
+ * again.
+ */
+ if (kvm_dirty_log_manual_protect_and_init_set(kvm))
+ kvm_mmu_split_huge_pages(kvm, start, end);
+
+ kvm_nested_s2_wp(kvm);
}
static void kvm_send_hwpoison_signal(unsigned long address, short lsb)
@@ -1718,6 +1336,10 @@ static bool fault_supports_stage2_huge_mapping(struct kvm_memory_slot *memslot,
if (map_size == PAGE_SIZE)
return true;
+ /* pKVM only supports PMD_SIZE huge-mappings */
+ if (is_protected_kvm_enabled() && map_size != PMD_SIZE)
+ return false;
+
size = memslot->npages * PAGE_SIZE;
gpa_start = memslot->base_gfn << PAGE_SHIFT;
@@ -1775,8 +1397,8 @@ static bool fault_supports_stage2_huge_mapping(struct kvm_memory_slot *memslot,
*
* Returns the size of the mapping.
*/
-static unsigned long
-transparent_hugepage_adjust(struct kvm_memory_slot *memslot,
+static long
+transparent_hugepage_adjust(struct kvm *kvm, struct kvm_memory_slot *memslot,
unsigned long hva, kvm_pfn_t *pfnp,
phys_addr_t *ipap)
{
@@ -1787,30 +1409,17 @@ transparent_hugepage_adjust(struct kvm_memory_slot *memslot,
* sure that the HVA and IPA are sufficiently aligned and that the
* block map is contained within the memslot.
*/
- if (kvm_is_transparent_hugepage(pfn) &&
- fault_supports_stage2_huge_mapping(memslot, hva, PMD_SIZE)) {
- /*
- * The address we faulted on is backed by a transparent huge
- * page. However, because we map the compound huge page and
- * not the individual tail page, we need to transfer the
- * refcount to the head page. We have to be careful that the
- * THP doesn't start to split while we are adjusting the
- * refcounts.
- *
- * We are sure this doesn't happen, because mmu_notifier_retry
- * was successful and we are holding the mmu_lock, so if this
- * THP is trying to split, it will be blocked in the mmu
- * notifier before touching any of the pages, specifically
- * before being able to call __split_huge_page_refcount().
- *
- * We can therefore safely transfer the refcount from PG_tail
- * to PG_head and switch the pfn from a tail page to the head
- * page accordingly.
- */
+ if (fault_supports_stage2_huge_mapping(memslot, hva, PMD_SIZE)) {
+ int sz = get_user_mapping_size(kvm, hva);
+
+ if (sz < 0)
+ return sz;
+
+ if (sz < PMD_SIZE)
+ return PAGE_SIZE;
+
*ipap &= PMD_MASK;
- kvm_release_pfn_clean(pfn);
pfn &= ~(PTRS_PER_PMD - 1);
- kvm_get_pfn(pfn);
*pfnp = pfn;
return PMD_SIZE;
@@ -1820,86 +1429,366 @@ transparent_hugepage_adjust(struct kvm_memory_slot *memslot,
return PAGE_SIZE;
}
+static int get_vma_page_shift(struct vm_area_struct *vma, unsigned long hva)
+{
+ unsigned long pa;
+
+ if (is_vm_hugetlb_page(vma) && !(vma->vm_flags & VM_PFNMAP))
+ return huge_page_shift(hstate_vma(vma));
+
+ if (!(vma->vm_flags & VM_PFNMAP))
+ return PAGE_SHIFT;
+
+ VM_BUG_ON(is_vm_hugetlb_page(vma));
+
+ pa = (vma->vm_pgoff << PAGE_SHIFT) + (hva - vma->vm_start);
+
+#ifndef __PAGETABLE_PMD_FOLDED
+ if ((hva & (PUD_SIZE - 1)) == (pa & (PUD_SIZE - 1)) &&
+ ALIGN_DOWN(hva, PUD_SIZE) >= vma->vm_start &&
+ ALIGN(hva, PUD_SIZE) <= vma->vm_end)
+ return PUD_SHIFT;
+#endif
+
+ if ((hva & (PMD_SIZE - 1)) == (pa & (PMD_SIZE - 1)) &&
+ ALIGN_DOWN(hva, PMD_SIZE) >= vma->vm_start &&
+ ALIGN(hva, PMD_SIZE) <= vma->vm_end)
+ return PMD_SHIFT;
+
+ return PAGE_SHIFT;
+}
+
+/*
+ * The page will be mapped in stage 2 as Normal Cacheable, so the VM will be
+ * able to see the page's tags and therefore they must be initialised first. If
+ * PG_mte_tagged is set, tags have already been initialised.
+ *
+ * Must be called with kvm->mmu_lock held to ensure the memory remains mapped
+ * while the tags are zeroed.
+ */
+static void sanitise_mte_tags(struct kvm *kvm, kvm_pfn_t pfn,
+ unsigned long size)
+{
+ unsigned long i, nr_pages = size >> PAGE_SHIFT;
+ struct page *page = pfn_to_page(pfn);
+ struct folio *folio = page_folio(page);
+
+ if (!kvm_has_mte(kvm))
+ return;
+
+ if (folio_test_hugetlb(folio)) {
+ /* Hugetlb has MTE flags set on head page only */
+ if (folio_try_hugetlb_mte_tagging(folio)) {
+ for (i = 0; i < nr_pages; i++, page++)
+ mte_clear_page_tags(page_address(page));
+ folio_set_hugetlb_mte_tagged(folio);
+ }
+ return;
+ }
+
+ for (i = 0; i < nr_pages; i++, page++) {
+ if (try_page_mte_tagging(page)) {
+ mte_clear_page_tags(page_address(page));
+ set_page_mte_tagged(page);
+ }
+ }
+}
+
+static bool kvm_vma_mte_allowed(struct vm_area_struct *vma)
+{
+ return vma->vm_flags & VM_MTE_ALLOWED;
+}
+
+static bool kvm_vma_is_cacheable(struct vm_area_struct *vma)
+{
+ switch (FIELD_GET(PTE_ATTRINDX_MASK, pgprot_val(vma->vm_page_prot))) {
+ case MT_NORMAL_NC:
+ case MT_DEVICE_nGnRnE:
+ case MT_DEVICE_nGnRE:
+ return false;
+ default:
+ return true;
+ }
+}
+
+static int prepare_mmu_memcache(struct kvm_vcpu *vcpu, bool topup_memcache,
+ void **memcache)
+{
+ int min_pages;
+
+ if (!is_protected_kvm_enabled())
+ *memcache = &vcpu->arch.mmu_page_cache;
+ else
+ *memcache = &vcpu->arch.pkvm_memcache;
+
+ if (!topup_memcache)
+ return 0;
+
+ min_pages = kvm_mmu_cache_min_pages(vcpu->arch.hw_mmu);
+
+ if (!is_protected_kvm_enabled())
+ return kvm_mmu_topup_memory_cache(*memcache, min_pages);
+
+ return topup_hyp_memcache(*memcache, min_pages);
+}
+
+/*
+ * Potentially reduce shadow S2 permissions to match the guest's own S2. For
+ * exec faults, we'd only reach this point if the guest actually allowed it (see
+ * kvm_s2_handle_perm_fault).
+ *
+ * Also encode the level of the original translation in the SW bits of the leaf
+ * entry as a proxy for the span of that translation. This will be retrieved on
+ * TLB invalidation from the guest and used to limit the invalidation scope if a
+ * TTL hint or a range isn't provided.
+ */
+static void adjust_nested_fault_perms(struct kvm_s2_trans *nested,
+ enum kvm_pgtable_prot *prot,
+ bool *writable)
+{
+ *writable &= kvm_s2_trans_writable(nested);
+ if (!kvm_s2_trans_readable(nested))
+ *prot &= ~KVM_PGTABLE_PROT_R;
+
+ *prot |= kvm_encode_nested_level(nested);
+}
+
+static void adjust_nested_exec_perms(struct kvm *kvm,
+ struct kvm_s2_trans *nested,
+ enum kvm_pgtable_prot *prot)
+{
+ if (!kvm_s2_trans_exec_el0(kvm, nested))
+ *prot &= ~KVM_PGTABLE_PROT_UX;
+ if (!kvm_s2_trans_exec_el1(kvm, nested))
+ *prot &= ~KVM_PGTABLE_PROT_PX;
+}
+
+#define KVM_PGTABLE_WALK_MEMABORT_FLAGS (KVM_PGTABLE_WALK_HANDLE_FAULT | KVM_PGTABLE_WALK_SHARED)
+
+static int gmem_abort(struct kvm_vcpu *vcpu, phys_addr_t fault_ipa,
+ struct kvm_s2_trans *nested,
+ struct kvm_memory_slot *memslot, bool is_perm)
+{
+ bool write_fault, exec_fault, writable;
+ enum kvm_pgtable_walk_flags flags = KVM_PGTABLE_WALK_MEMABORT_FLAGS;
+ enum kvm_pgtable_prot prot = KVM_PGTABLE_PROT_R;
+ struct kvm_pgtable *pgt = vcpu->arch.hw_mmu->pgt;
+ unsigned long mmu_seq;
+ struct page *page;
+ struct kvm *kvm = vcpu->kvm;
+ void *memcache;
+ kvm_pfn_t pfn;
+ gfn_t gfn;
+ int ret;
+
+ ret = prepare_mmu_memcache(vcpu, true, &memcache);
+ if (ret)
+ return ret;
+
+ if (nested)
+ gfn = kvm_s2_trans_output(nested) >> PAGE_SHIFT;
+ else
+ gfn = fault_ipa >> PAGE_SHIFT;
+
+ write_fault = kvm_is_write_fault(vcpu);
+ exec_fault = kvm_vcpu_trap_is_exec_fault(vcpu);
+
+ VM_WARN_ON_ONCE(write_fault && exec_fault);
+
+ mmu_seq = kvm->mmu_invalidate_seq;
+ /* Pairs with the smp_wmb() in kvm_mmu_invalidate_end(). */
+ smp_rmb();
+
+ ret = kvm_gmem_get_pfn(kvm, memslot, gfn, &pfn, &page, NULL);
+ if (ret) {
+ kvm_prepare_memory_fault_exit(vcpu, fault_ipa, PAGE_SIZE,
+ write_fault, exec_fault, false);
+ return ret;
+ }
+
+ writable = !(memslot->flags & KVM_MEM_READONLY);
+
+ if (nested)
+ adjust_nested_fault_perms(nested, &prot, &writable);
+
+ if (writable)
+ prot |= KVM_PGTABLE_PROT_W;
+
+ if (exec_fault || cpus_have_final_cap(ARM64_HAS_CACHE_DIC))
+ prot |= KVM_PGTABLE_PROT_X;
+
+ if (nested)
+ adjust_nested_exec_perms(kvm, nested, &prot);
+
+ kvm_fault_lock(kvm);
+ if (mmu_invalidate_retry(kvm, mmu_seq)) {
+ ret = -EAGAIN;
+ goto out_unlock;
+ }
+
+ ret = KVM_PGT_FN(kvm_pgtable_stage2_map)(pgt, fault_ipa, PAGE_SIZE,
+ __pfn_to_phys(pfn), prot,
+ memcache, flags);
+
+out_unlock:
+ kvm_release_faultin_page(kvm, page, !!ret, writable);
+ kvm_fault_unlock(kvm);
+
+ if (writable && !ret)
+ mark_page_dirty_in_slot(kvm, memslot, gfn);
+
+ return ret != -EAGAIN ? ret : 0;
+}
+
static int user_mem_abort(struct kvm_vcpu *vcpu, phys_addr_t fault_ipa,
+ struct kvm_s2_trans *nested,
struct kvm_memory_slot *memslot, unsigned long hva,
- unsigned long fault_status)
+ bool fault_is_perm)
{
- int ret;
- bool write_fault, writable, force_pte = false;
- bool exec_fault, needs_exec;
+ int ret = 0;
+ bool topup_memcache;
+ bool write_fault, writable;
+ bool exec_fault, mte_allowed, is_vma_cacheable;
+ bool s2_force_noncacheable = false, vfio_allow_any_uc = false;
unsigned long mmu_seq;
- gfn_t gfn = fault_ipa >> PAGE_SHIFT;
+ phys_addr_t ipa = fault_ipa;
struct kvm *kvm = vcpu->kvm;
- struct kvm_mmu_memory_cache *memcache = &vcpu->arch.mmu_page_cache;
struct vm_area_struct *vma;
short vma_shift;
+ void *memcache;
+ gfn_t gfn;
kvm_pfn_t pfn;
- pgprot_t mem_type = PAGE_S2;
bool logging_active = memslot_is_logging(memslot);
- unsigned long vma_pagesize, flags = 0;
-
+ bool force_pte = logging_active;
+ long vma_pagesize, fault_granule;
+ enum kvm_pgtable_prot prot = KVM_PGTABLE_PROT_R;
+ struct kvm_pgtable *pgt;
+ struct page *page;
+ vm_flags_t vm_flags;
+ enum kvm_pgtable_walk_flags flags = KVM_PGTABLE_WALK_MEMABORT_FLAGS;
+
+ if (fault_is_perm)
+ fault_granule = kvm_vcpu_trap_get_perm_fault_granule(vcpu);
write_fault = kvm_is_write_fault(vcpu);
- exec_fault = kvm_vcpu_trap_is_iabt(vcpu);
- VM_BUG_ON(write_fault && exec_fault);
+ exec_fault = kvm_vcpu_trap_is_exec_fault(vcpu);
+ VM_WARN_ON_ONCE(write_fault && exec_fault);
- if (fault_status == FSC_PERM && !write_fault && !exec_fault) {
- kvm_err("Unexpected L2 read permission error\n");
- return -EFAULT;
- }
+ /*
+ * Permission faults just need to update the existing leaf entry,
+ * and so normally don't require allocations from the memcache. The
+ * only exception to this is when dirty logging is enabled at runtime
+ * and a write fault needs to collapse a block entry into a table.
+ */
+ topup_memcache = !fault_is_perm || (logging_active && write_fault);
+ ret = prepare_mmu_memcache(vcpu, topup_memcache, &memcache);
+ if (ret)
+ return ret;
- /* Let's check if we will get back a huge page backed by hugetlbfs */
+ /*
+ * Let's check if we will get back a huge page backed by hugetlbfs, or
+ * get block mapping for device MMIO region.
+ */
mmap_read_lock(current->mm);
- vma = find_vma_intersection(current->mm, hva, hva + 1);
+ vma = vma_lookup(current->mm, hva);
if (unlikely(!vma)) {
kvm_err("Failed to find VMA for hva 0x%lx\n", hva);
mmap_read_unlock(current->mm);
return -EFAULT;
}
- if (is_vm_hugetlb_page(vma))
- vma_shift = huge_page_shift(hstate_vma(vma));
- else
+ if (force_pte)
vma_shift = PAGE_SHIFT;
+ else
+ vma_shift = get_vma_page_shift(vma, hva);
- vma_pagesize = 1ULL << vma_shift;
- if (logging_active ||
- (vma->vm_flags & VM_PFNMAP) ||
- !fault_supports_stage2_huge_mapping(memslot, hva, vma_pagesize)) {
+ switch (vma_shift) {
+#ifndef __PAGETABLE_PMD_FOLDED
+ case PUD_SHIFT:
+ if (fault_supports_stage2_huge_mapping(memslot, hva, PUD_SIZE))
+ break;
+ fallthrough;
+#endif
+ case CONT_PMD_SHIFT:
+ vma_shift = PMD_SHIFT;
+ fallthrough;
+ case PMD_SHIFT:
+ if (fault_supports_stage2_huge_mapping(memslot, hva, PMD_SIZE))
+ break;
+ fallthrough;
+ case CONT_PTE_SHIFT:
+ vma_shift = PAGE_SHIFT;
force_pte = true;
- vma_pagesize = PAGE_SIZE;
+ fallthrough;
+ case PAGE_SHIFT:
+ break;
+ default:
+ WARN_ONCE(1, "Unknown vma_shift %d", vma_shift);
+ }
+
+ vma_pagesize = 1UL << vma_shift;
+
+ if (nested) {
+ unsigned long max_map_size;
+
+ max_map_size = force_pte ? PAGE_SIZE : PUD_SIZE;
+
+ ipa = kvm_s2_trans_output(nested);
+
+ /*
+ * If we're about to create a shadow stage 2 entry, then we
+ * can only create a block mapping if the guest stage 2 page
+ * table uses at least as big a mapping.
+ */
+ max_map_size = min(kvm_s2_trans_size(nested), max_map_size);
+
+ /*
+ * Be careful that if the mapping size falls between
+ * two host sizes, take the smallest of the two.
+ */
+ if (max_map_size >= PMD_SIZE && max_map_size < PUD_SIZE)
+ max_map_size = PMD_SIZE;
+ else if (max_map_size >= PAGE_SIZE && max_map_size < PMD_SIZE)
+ max_map_size = PAGE_SIZE;
+
+ force_pte = (max_map_size == PAGE_SIZE);
+ vma_pagesize = min_t(long, vma_pagesize, max_map_size);
}
/*
- * The stage2 has a minimum of 2 level table (For arm64 see
- * kvm_arm_setup_stage2()). Hence, we are guaranteed that we can
- * use PMD_SIZE huge mappings (even when the PMD is folded into PGD).
- * As for PUD huge maps, we must make sure that we have at least
- * 3 levels, i.e, PMD is not folded.
+ * Both the canonical IPA and fault IPA must be hugepage-aligned to
+ * ensure we find the right PFN and lay down the mapping in the right
+ * place.
*/
- if (vma_pagesize == PMD_SIZE ||
- (vma_pagesize == PUD_SIZE && kvm_stage2_has_pmd(kvm)))
- gfn = (fault_ipa & huge_page_mask(hstate_vma(vma))) >> PAGE_SHIFT;
- mmap_read_unlock(current->mm);
+ if (vma_pagesize == PMD_SIZE || vma_pagesize == PUD_SIZE) {
+ fault_ipa &= ~(vma_pagesize - 1);
+ ipa &= ~(vma_pagesize - 1);
+ }
- /* We need minimum second+third level pages */
- ret = mmu_topup_memory_cache(memcache, kvm_mmu_cache_min_pages(kvm),
- KVM_NR_MEM_OBJS);
- if (ret)
- return ret;
+ gfn = ipa >> PAGE_SHIFT;
+ mte_allowed = kvm_vma_mte_allowed(vma);
+
+ vfio_allow_any_uc = vma->vm_flags & VM_ALLOW_ANY_UNCACHED;
+
+ vm_flags = vma->vm_flags;
+
+ is_vma_cacheable = kvm_vma_is_cacheable(vma);
+
+ /* Don't use the VMA after the unlock -- it may have vanished */
+ vma = NULL;
- mmu_seq = vcpu->kvm->mmu_notifier_seq;
/*
- * Ensure the read of mmu_notifier_seq happens before we call
- * gfn_to_pfn_prot (which calls get_user_pages), so that we don't risk
- * the page we just got a reference to gets unmapped before we have a
- * chance to grab the mmu_lock, which ensure that if the page gets
- * unmapped afterwards, the call to kvm_unmap_hva will take it away
- * from us again properly. This smp_rmb() interacts with the smp_wmb()
- * in kvm_mmu_notifier_invalidate_<page|range_end>.
+ * Read mmu_invalidate_seq so that KVM can detect if the results of
+ * vma_lookup() or __kvm_faultin_pfn() become stale prior to
+ * acquiring kvm->mmu_lock.
+ *
+ * Rely on mmap_read_unlock() for an implicit smp_rmb(), which pairs
+ * with the smp_wmb() in kvm_mmu_invalidate_end().
*/
- smp_rmb();
+ mmu_seq = kvm->mmu_invalidate_seq;
+ mmap_read_unlock(current->mm);
- pfn = gfn_to_pfn_prot(kvm, gfn, write_fault, &writable);
+ pfn = __kvm_faultin_pfn(memslot, gfn, write_fault ? FOLL_WRITE : 0,
+ &writable, &page);
if (pfn == KVM_PFN_ERR_HWPOISON) {
kvm_send_hwpoison_signal(hva, vma_shift);
return 0;
@@ -1907,149 +1796,236 @@ static int user_mem_abort(struct kvm_vcpu *vcpu, phys_addr_t fault_ipa,
if (is_error_noslot_pfn(pfn))
return -EFAULT;
- if (kvm_is_device_pfn(pfn)) {
- mem_type = PAGE_S2_DEVICE;
- flags |= KVM_S2PTE_FLAG_IS_IOMAP;
- } else if (logging_active) {
- /*
- * Faults on pages in a memslot with logging enabled
- * should not be mapped with huge pages (it introduces churn
- * and performance degradation), so force a pte mapping.
- */
- flags |= KVM_S2_FLAG_LOGGING_ACTIVE;
-
+ /*
+ * Check if this is non-struct page memory PFN, and cannot support
+ * CMOs. It could potentially be unsafe to access as cacheable.
+ */
+ if (vm_flags & (VM_PFNMAP | VM_MIXEDMAP) && !pfn_is_map_memory(pfn)) {
+ if (is_vma_cacheable) {
+ /*
+ * Whilst the VMA owner expects cacheable mapping to this
+ * PFN, hardware also has to support the FWB and CACHE DIC
+ * features.
+ *
+ * ARM64 KVM relies on kernel VA mapping to the PFN to
+ * perform cache maintenance as the CMO instructions work on
+ * virtual addresses. VM_PFNMAP region are not necessarily
+ * mapped to a KVA and hence the presence of hardware features
+ * S2FWB and CACHE DIC are mandatory to avoid the need for
+ * cache maintenance.
+ */
+ if (!kvm_supports_cacheable_pfnmap())
+ ret = -EFAULT;
+ } else {
+ /*
+ * If the page was identified as device early by looking at
+ * the VMA flags, vma_pagesize is already representing the
+ * largest quantity we can map. If instead it was mapped
+ * via __kvm_faultin_pfn(), vma_pagesize is set to PAGE_SIZE
+ * and must not be upgraded.
+ *
+ * In both cases, we don't let transparent_hugepage_adjust()
+ * change things at the last minute.
+ */
+ s2_force_noncacheable = true;
+ }
+ } else if (logging_active && !write_fault) {
/*
* Only actually map the page as writable if this was a write
* fault.
*/
- if (!write_fault)
- writable = false;
+ writable = false;
}
- if (exec_fault && is_iomap(flags))
- return -ENOEXEC;
+ if (exec_fault && s2_force_noncacheable)
+ ret = -ENOEXEC;
- spin_lock(&kvm->mmu_lock);
- if (mmu_notifier_retry(kvm, mmu_seq))
+ if (ret) {
+ kvm_release_page_unused(page);
+ return ret;
+ }
+
+ if (nested)
+ adjust_nested_fault_perms(nested, &prot, &writable);
+
+ kvm_fault_lock(kvm);
+ pgt = vcpu->arch.hw_mmu->pgt;
+ if (mmu_invalidate_retry(kvm, mmu_seq)) {
+ ret = -EAGAIN;
goto out_unlock;
+ }
/*
* If we are not forced to use page mapping, check if we are
* backed by a THP and thus use block mapping if possible.
*/
- if (vma_pagesize == PAGE_SIZE && !force_pte)
- vma_pagesize = transparent_hugepage_adjust(memslot, hva,
- &pfn, &fault_ipa);
- if (writable)
- kvm_set_pfn_dirty(pfn);
+ if (vma_pagesize == PAGE_SIZE && !(force_pte || s2_force_noncacheable)) {
+ if (fault_is_perm && fault_granule > PAGE_SIZE)
+ vma_pagesize = fault_granule;
+ else
+ vma_pagesize = transparent_hugepage_adjust(kvm, memslot,
+ hva, &pfn,
+ &fault_ipa);
+
+ if (vma_pagesize < 0) {
+ ret = vma_pagesize;
+ goto out_unlock;
+ }
+ }
+
+ if (!fault_is_perm && !s2_force_noncacheable && kvm_has_mte(kvm)) {
+ /* Check the VMM hasn't introduced a new disallowed VMA */
+ if (mte_allowed) {
+ sanitise_mte_tags(kvm, pfn, vma_pagesize);
+ } else {
+ ret = -EFAULT;
+ goto out_unlock;
+ }
+ }
- if (fault_status != FSC_PERM && !is_iomap(flags))
- clean_dcache_guest_page(pfn, vma_pagesize);
+ if (writable)
+ prot |= KVM_PGTABLE_PROT_W;
if (exec_fault)
- invalidate_icache_guest_page(pfn, vma_pagesize);
+ prot |= KVM_PGTABLE_PROT_X;
+
+ if (s2_force_noncacheable) {
+ if (vfio_allow_any_uc)
+ prot |= KVM_PGTABLE_PROT_NORMAL_NC;
+ else
+ prot |= KVM_PGTABLE_PROT_DEVICE;
+ } else if (cpus_have_final_cap(ARM64_HAS_CACHE_DIC)) {
+ prot |= KVM_PGTABLE_PROT_X;
+ }
+
+ if (nested)
+ adjust_nested_exec_perms(kvm, nested, &prot);
/*
- * If we took an execution fault we have made the
- * icache/dcache coherent above and should now let the s2
- * mapping be executable.
- *
- * Write faults (!exec_fault && FSC_PERM) are orthogonal to
- * execute permissions, and we preserve whatever we have.
+ * Under the premise of getting a FSC_PERM fault, we just need to relax
+ * permissions only if vma_pagesize equals fault_granule. Otherwise,
+ * kvm_pgtable_stage2_map() should be called to change block size.
*/
- needs_exec = exec_fault ||
- (fault_status == FSC_PERM && stage2_is_exec(kvm, fault_ipa));
-
- if (vma_pagesize == PUD_SIZE) {
- pud_t new_pud = kvm_pfn_pud(pfn, mem_type);
-
- new_pud = kvm_pud_mkhuge(new_pud);
- if (writable)
- new_pud = kvm_s2pud_mkwrite(new_pud);
-
- if (needs_exec)
- new_pud = kvm_s2pud_mkexec(new_pud);
+ if (fault_is_perm && vma_pagesize == fault_granule) {
+ /*
+ * Drop the SW bits in favour of those stored in the
+ * PTE, which will be preserved.
+ */
+ prot &= ~KVM_NV_GUEST_MAP_SZ;
+ ret = KVM_PGT_FN(kvm_pgtable_stage2_relax_perms)(pgt, fault_ipa, prot, flags);
+ } else {
+ ret = KVM_PGT_FN(kvm_pgtable_stage2_map)(pgt, fault_ipa, vma_pagesize,
+ __pfn_to_phys(pfn), prot,
+ memcache, flags);
+ }
- ret = stage2_set_pud_huge(kvm, memcache, fault_ipa, &new_pud);
- } else if (vma_pagesize == PMD_SIZE) {
- pmd_t new_pmd = kvm_pfn_pmd(pfn, mem_type);
+out_unlock:
+ kvm_release_faultin_page(kvm, page, !!ret, writable);
+ kvm_fault_unlock(kvm);
- new_pmd = kvm_pmd_mkhuge(new_pmd);
+ /* Mark the page dirty only if the fault is handled successfully */
+ if (writable && !ret)
+ mark_page_dirty_in_slot(kvm, memslot, gfn);
- if (writable)
- new_pmd = kvm_s2pmd_mkwrite(new_pmd);
+ return ret != -EAGAIN ? ret : 0;
+}
- if (needs_exec)
- new_pmd = kvm_s2pmd_mkexec(new_pmd);
+/* Resolve the access fault by making the page young again. */
+static void handle_access_fault(struct kvm_vcpu *vcpu, phys_addr_t fault_ipa)
+{
+ enum kvm_pgtable_walk_flags flags = KVM_PGTABLE_WALK_HANDLE_FAULT | KVM_PGTABLE_WALK_SHARED;
+ struct kvm_s2_mmu *mmu;
- ret = stage2_set_pmd_huge(kvm, memcache, fault_ipa, &new_pmd);
- } else {
- pte_t new_pte = kvm_pfn_pte(pfn, mem_type);
+ trace_kvm_access_fault(fault_ipa);
- if (writable) {
- new_pte = kvm_s2pte_mkwrite(new_pte);
- mark_page_dirty(kvm, gfn);
- }
+ read_lock(&vcpu->kvm->mmu_lock);
+ mmu = vcpu->arch.hw_mmu;
+ KVM_PGT_FN(kvm_pgtable_stage2_mkyoung)(mmu->pgt, fault_ipa, flags);
+ read_unlock(&vcpu->kvm->mmu_lock);
+}
- if (needs_exec)
- new_pte = kvm_s2pte_mkexec(new_pte);
+/*
+ * Returns true if the SEA should be handled locally within KVM if the abort
+ * is caused by a kernel memory allocation (e.g. stage-2 table memory).
+ */
+static bool host_owns_sea(struct kvm_vcpu *vcpu, u64 esr)
+{
+ /*
+ * Without FEAT_RAS HCR_EL2.TEA is RES0, meaning any external abort
+ * taken from a guest EL to EL2 is due to a host-imposed access (e.g.
+ * stage-2 PTW).
+ */
+ if (!cpus_have_final_cap(ARM64_HAS_RAS_EXTN))
+ return true;
- ret = stage2_set_pte(kvm, memcache, fault_ipa, &new_pte, flags);
- }
+ /* KVM owns the VNCR when the vCPU isn't in a nested context. */
+ if (is_hyp_ctxt(vcpu) && !kvm_vcpu_trap_is_iabt(vcpu) && (esr & ESR_ELx_VNCR))
+ return true;
-out_unlock:
- spin_unlock(&kvm->mmu_lock);
- kvm_set_pfn_accessed(pfn);
- kvm_release_pfn_clean(pfn);
- return ret;
+ /*
+ * Determining if an external abort during a table walk happened at
+ * stage-2 is only possible with S1PTW is set. Otherwise, since KVM
+ * sets HCR_EL2.TEA, SEAs due to a stage-1 walk (i.e. accessing the
+ * PA of the stage-1 descriptor) can reach here and are reported
+ * with a TTW ESR value.
+ */
+ return (esr_fsc_is_sea_ttw(esr) && (esr & ESR_ELx_S1PTW));
}
-/*
- * Resolve the access fault by making the page young again.
- * Note that because the faulting entry is guaranteed not to be
- * cached in the TLB, we don't need to invalidate anything.
- * Only the HW Access Flag updates are supported for Stage 2 (no DBM),
- * so there is no need for atomic (pte|pmd)_mkyoung operations.
- */
-static void handle_access_fault(struct kvm_vcpu *vcpu, phys_addr_t fault_ipa)
+int kvm_handle_guest_sea(struct kvm_vcpu *vcpu)
{
- pud_t *pud;
- pmd_t *pmd;
- pte_t *pte;
- kvm_pfn_t pfn;
- bool pfn_valid = false;
+ struct kvm *kvm = vcpu->kvm;
+ struct kvm_run *run = vcpu->run;
+ u64 esr = kvm_vcpu_get_esr(vcpu);
+ u64 esr_mask = ESR_ELx_EC_MASK |
+ ESR_ELx_IL |
+ ESR_ELx_FnV |
+ ESR_ELx_EA |
+ ESR_ELx_CM |
+ ESR_ELx_WNR |
+ ESR_ELx_FSC;
+ u64 ipa;
- trace_kvm_access_fault(fault_ipa);
+ /*
+ * Give APEI the opportunity to claim the abort before handling it
+ * within KVM. apei_claim_sea() expects to be called with IRQs enabled.
+ */
+ lockdep_assert_irqs_enabled();
+ if (apei_claim_sea(NULL) == 0)
+ return 1;
- spin_lock(&vcpu->kvm->mmu_lock);
+ if (host_owns_sea(vcpu, esr) ||
+ !test_bit(KVM_ARCH_FLAG_EXIT_SEA, &vcpu->kvm->arch.flags))
+ return kvm_inject_serror(vcpu);
- if (!stage2_get_leaf_entry(vcpu->kvm, fault_ipa, &pud, &pmd, &pte))
- goto out;
+ /* ESR_ELx.SET is RES0 when FEAT_RAS isn't implemented. */
+ if (kvm_has_ras(kvm))
+ esr_mask |= ESR_ELx_SET_MASK;
- if (pud) { /* HugeTLB */
- *pud = kvm_s2pud_mkyoung(*pud);
- pfn = kvm_pud_pfn(*pud);
- pfn_valid = true;
- } else if (pmd) { /* THP, HugeTLB */
- *pmd = pmd_mkyoung(*pmd);
- pfn = pmd_pfn(*pmd);
- pfn_valid = true;
- } else {
- *pte = pte_mkyoung(*pte); /* Just a page... */
- pfn = pte_pfn(*pte);
- pfn_valid = true;
+ /*
+ * Exit to userspace, and provide faulting guest virtual and physical
+ * addresses in case userspace wants to emulate SEA to guest by
+ * writing to FAR_ELx and HPFAR_ELx registers.
+ */
+ memset(&run->arm_sea, 0, sizeof(run->arm_sea));
+ run->exit_reason = KVM_EXIT_ARM_SEA;
+ run->arm_sea.esr = esr & esr_mask;
+
+ if (!(esr & ESR_ELx_FnV))
+ run->arm_sea.gva = kvm_vcpu_get_hfar(vcpu);
+
+ ipa = kvm_vcpu_get_fault_ipa(vcpu);
+ if (ipa != INVALID_GPA) {
+ run->arm_sea.flags |= KVM_EXIT_ARM_SEA_FLAG_GPA_VALID;
+ run->arm_sea.gpa = ipa;
}
-out:
- spin_unlock(&vcpu->kvm->mmu_lock);
- if (pfn_valid)
- kvm_set_pfn_accessed(pfn);
+ return 0;
}
/**
* kvm_handle_guest_abort - handles all 2nd stage aborts
* @vcpu: the VCPU pointer
- * @run: the kvm_run structure
*
* Any abort that gets to the host is almost guaranteed to be caused by a
* missing second stage translation table entry, which can mean that either the
@@ -2058,62 +2034,125 @@ out:
* space. The distinction is based on the IPA causing the fault and whether this
* memory region has been registered as standard RAM by user space.
*/
-int kvm_handle_guest_abort(struct kvm_vcpu *vcpu, struct kvm_run *run)
+int kvm_handle_guest_abort(struct kvm_vcpu *vcpu)
{
- unsigned long fault_status;
- phys_addr_t fault_ipa;
+ struct kvm_s2_trans nested_trans, *nested = NULL;
+ unsigned long esr;
+ phys_addr_t fault_ipa; /* The address we faulted on */
+ phys_addr_t ipa; /* Always the IPA in the L1 guest phys space */
struct kvm_memory_slot *memslot;
unsigned long hva;
bool is_iabt, write_fault, writable;
gfn_t gfn;
int ret, idx;
- fault_status = kvm_vcpu_trap_get_fault_type(vcpu);
+ if (kvm_vcpu_abt_issea(vcpu))
+ return kvm_handle_guest_sea(vcpu);
+
+ esr = kvm_vcpu_get_esr(vcpu);
+
+ /*
+ * The fault IPA should be reliable at this point as we're not dealing
+ * with an SEA.
+ */
+ ipa = fault_ipa = kvm_vcpu_get_fault_ipa(vcpu);
+ if (KVM_BUG_ON(ipa == INVALID_GPA, vcpu->kvm))
+ return -EFAULT;
- fault_ipa = kvm_vcpu_get_fault_ipa(vcpu);
is_iabt = kvm_vcpu_trap_is_iabt(vcpu);
- /* Synchronous External Abort? */
- if (kvm_vcpu_dabt_isextabt(vcpu)) {
- /*
- * For RAS the host kernel may handle this abort.
- * There is no need to pass the error into the guest.
- */
- if (!kvm_handle_guest_sea(fault_ipa, kvm_vcpu_get_hsr(vcpu)))
+ if (esr_fsc_is_translation_fault(esr)) {
+ /* Beyond sanitised PARange (which is the IPA limit) */
+ if (fault_ipa >= BIT_ULL(get_kvm_ipa_limit())) {
+ kvm_inject_size_fault(vcpu);
return 1;
+ }
- if (unlikely(!is_iabt)) {
- kvm_inject_vabt(vcpu);
- return 1;
+ /* Falls between the IPA range and the PARange? */
+ if (fault_ipa >= BIT_ULL(VTCR_EL2_IPA(vcpu->arch.hw_mmu->vtcr))) {
+ fault_ipa |= kvm_vcpu_get_hfar(vcpu) & GENMASK(11, 0);
+
+ return kvm_inject_sea(vcpu, is_iabt, fault_ipa);
}
}
- trace_kvm_guest_fault(*vcpu_pc(vcpu), kvm_vcpu_get_hsr(vcpu),
+ trace_kvm_guest_fault(*vcpu_pc(vcpu), kvm_vcpu_get_esr(vcpu),
kvm_vcpu_get_hfar(vcpu), fault_ipa);
/* Check the stage-2 fault is trans. fault or write fault */
- if (fault_status != FSC_FAULT && fault_status != FSC_PERM &&
- fault_status != FSC_ACCESS) {
+ if (!esr_fsc_is_translation_fault(esr) &&
+ !esr_fsc_is_permission_fault(esr) &&
+ !esr_fsc_is_access_flag_fault(esr)) {
kvm_err("Unsupported FSC: EC=%#x xFSC=%#lx ESR_EL2=%#lx\n",
kvm_vcpu_trap_get_class(vcpu),
(unsigned long)kvm_vcpu_trap_get_fault(vcpu),
- (unsigned long)kvm_vcpu_get_hsr(vcpu));
+ (unsigned long)kvm_vcpu_get_esr(vcpu));
return -EFAULT;
}
idx = srcu_read_lock(&vcpu->kvm->srcu);
- gfn = fault_ipa >> PAGE_SHIFT;
+ /*
+ * We may have faulted on a shadow stage 2 page table if we are
+ * running a nested guest. In this case, we have to resolve the L2
+ * IPA to the L1 IPA first, before knowing what kind of memory should
+ * back the L1 IPA.
+ *
+ * If the shadow stage 2 page table walk faults, then we simply inject
+ * this to the guest and carry on.
+ *
+ * If there are no shadow S2 PTs because S2 is disabled, there is
+ * nothing to walk and we treat it as a 1:1 before going through the
+ * canonical translation.
+ */
+ if (kvm_is_nested_s2_mmu(vcpu->kvm,vcpu->arch.hw_mmu) &&
+ vcpu->arch.hw_mmu->nested_stage2_enabled) {
+ u32 esr;
+
+ ret = kvm_walk_nested_s2(vcpu, fault_ipa, &nested_trans);
+ if (ret == -EAGAIN) {
+ ret = 1;
+ goto out_unlock;
+ }
+
+ if (ret) {
+ esr = kvm_s2_trans_esr(&nested_trans);
+ kvm_inject_s2_fault(vcpu, esr);
+ goto out_unlock;
+ }
+
+ ret = kvm_s2_handle_perm_fault(vcpu, &nested_trans);
+ if (ret) {
+ esr = kvm_s2_trans_esr(&nested_trans);
+ kvm_inject_s2_fault(vcpu, esr);
+ goto out_unlock;
+ }
+
+ ipa = kvm_s2_trans_output(&nested_trans);
+ nested = &nested_trans;
+ }
+
+ gfn = ipa >> PAGE_SHIFT;
memslot = gfn_to_memslot(vcpu->kvm, gfn);
hva = gfn_to_hva_memslot_prot(memslot, gfn, &writable);
write_fault = kvm_is_write_fault(vcpu);
if (kvm_is_error_hva(hva) || (write_fault && !writable)) {
+ /*
+ * The guest has put either its instructions or its page-tables
+ * somewhere it shouldn't have. Userspace won't be able to do
+ * anything about this (there's no syndrome for a start), so
+ * re-inject the abort back into the guest.
+ */
if (is_iabt) {
- /* Prefetch Abort on I/O address */
ret = -ENOEXEC;
goto out;
}
+ if (kvm_vcpu_abt_iss1tw(vcpu)) {
+ ret = kvm_inject_sea_dabt(vcpu, kvm_vcpu_get_hfar(vcpu));
+ goto out_unlock;
+ }
+
/*
* Check for a cache maintenance operation. Since we
* ended-up here, we know it is outside of any memory
@@ -2124,8 +2163,8 @@ int kvm_handle_guest_abort(struct kvm_vcpu *vcpu, struct kvm_run *run)
* So let's assume that the guest is just being
* cautious, and skip the instruction.
*/
- if (kvm_vcpu_dabt_is_cm(vcpu)) {
- kvm_skip_instr(vcpu, kvm_vcpu_trap_il_is32bit(vcpu));
+ if (kvm_is_error_hva(hva) && kvm_vcpu_dabt_is_cm(vcpu)) {
+ kvm_incr_pc(vcpu);
ret = 1;
goto out_unlock;
}
@@ -2136,185 +2175,83 @@ int kvm_handle_guest_abort(struct kvm_vcpu *vcpu, struct kvm_run *run)
* faulting VA. This is always 12 bits, irrespective
* of the page size.
*/
- fault_ipa |= kvm_vcpu_get_hfar(vcpu) & ((1 << 12) - 1);
- ret = io_mem_abort(vcpu, run, fault_ipa);
+ ipa |= kvm_vcpu_get_hfar(vcpu) & GENMASK(11, 0);
+ ret = io_mem_abort(vcpu, ipa);
goto out_unlock;
}
/* Userspace should not be able to register out-of-bounds IPAs */
- VM_BUG_ON(fault_ipa >= kvm_phys_size(vcpu->kvm));
+ VM_BUG_ON(ipa >= kvm_phys_size(vcpu->arch.hw_mmu));
- if (fault_status == FSC_ACCESS) {
+ if (esr_fsc_is_access_flag_fault(esr)) {
handle_access_fault(vcpu, fault_ipa);
ret = 1;
goto out_unlock;
}
- ret = user_mem_abort(vcpu, fault_ipa, memslot, hva, fault_status);
+ VM_WARN_ON_ONCE(kvm_vcpu_trap_is_permission_fault(vcpu) &&
+ !write_fault && !kvm_vcpu_trap_is_exec_fault(vcpu));
+
+ if (kvm_slot_has_gmem(memslot))
+ ret = gmem_abort(vcpu, fault_ipa, nested, memslot,
+ esr_fsc_is_permission_fault(esr));
+ else
+ ret = user_mem_abort(vcpu, fault_ipa, nested, memslot, hva,
+ esr_fsc_is_permission_fault(esr));
if (ret == 0)
ret = 1;
out:
- if (ret == -ENOEXEC) {
- kvm_inject_pabt(vcpu, kvm_vcpu_get_hfar(vcpu));
- ret = 1;
- }
+ if (ret == -ENOEXEC)
+ ret = kvm_inject_sea_iabt(vcpu, kvm_vcpu_get_hfar(vcpu));
out_unlock:
srcu_read_unlock(&vcpu->kvm->srcu, idx);
return ret;
}
-static int handle_hva_to_gpa(struct kvm *kvm,
- unsigned long start,
- unsigned long end,
- int (*handler)(struct kvm *kvm,
- gpa_t gpa, u64 size,
- void *data),
- void *data)
-{
- struct kvm_memslots *slots;
- struct kvm_memory_slot *memslot;
- int ret = 0;
-
- slots = kvm_memslots(kvm);
-
- /* we only care about the pages that the guest sees */
- kvm_for_each_memslot(memslot, slots) {
- unsigned long hva_start, hva_end;
- gfn_t gpa;
-
- hva_start = max(start, memslot->userspace_addr);
- hva_end = min(end, memslot->userspace_addr +
- (memslot->npages << PAGE_SHIFT));
- if (hva_start >= hva_end)
- continue;
-
- gpa = hva_to_gfn_memslot(hva_start, memslot) << PAGE_SHIFT;
- ret |= handler(kvm, gpa, (u64)(hva_end - hva_start), data);
- }
-
- return ret;
-}
-
-static int kvm_unmap_hva_handler(struct kvm *kvm, gpa_t gpa, u64 size, void *data)
+bool kvm_unmap_gfn_range(struct kvm *kvm, struct kvm_gfn_range *range)
{
- unmap_stage2_range(kvm, gpa, size);
- return 0;
-}
-
-int kvm_unmap_hva_range(struct kvm *kvm,
- unsigned long start, unsigned long end)
-{
- if (!kvm->arch.pgd)
- return 0;
-
- trace_kvm_unmap_hva_range(start, end);
- handle_hva_to_gpa(kvm, start, end, &kvm_unmap_hva_handler, NULL);
- return 0;
-}
+ if (!kvm->arch.mmu.pgt)
+ return false;
-static int kvm_set_spte_handler(struct kvm *kvm, gpa_t gpa, u64 size, void *data)
-{
- pte_t *pte = (pte_t *)data;
+ __unmap_stage2_range(&kvm->arch.mmu, range->start << PAGE_SHIFT,
+ (range->end - range->start) << PAGE_SHIFT,
+ range->may_block);
- WARN_ON(size != PAGE_SIZE);
- /*
- * We can always call stage2_set_pte with KVM_S2PTE_FLAG_LOGGING_ACTIVE
- * flag clear because MMU notifiers will have unmapped a huge PMD before
- * calling ->change_pte() (which in turn calls kvm_set_spte_hva()) and
- * therefore stage2_set_pte() never needs to clear out a huge PMD
- * through this calling path.
- */
- stage2_set_pte(kvm, NULL, gpa, pte, 0);
- return 0;
+ kvm_nested_s2_unmap(kvm, range->may_block);
+ return false;
}
-
-int kvm_set_spte_hva(struct kvm *kvm, unsigned long hva, pte_t pte)
+bool kvm_age_gfn(struct kvm *kvm, struct kvm_gfn_range *range)
{
- unsigned long end = hva + PAGE_SIZE;
- kvm_pfn_t pfn = pte_pfn(pte);
- pte_t stage2_pte;
-
- if (!kvm->arch.pgd)
- return 0;
+ u64 size = (range->end - range->start) << PAGE_SHIFT;
- trace_kvm_set_spte_hva(hva);
+ if (!kvm->arch.mmu.pgt)
+ return false;
+ return KVM_PGT_FN(kvm_pgtable_stage2_test_clear_young)(kvm->arch.mmu.pgt,
+ range->start << PAGE_SHIFT,
+ size, true);
/*
- * We've moved a page around, probably through CoW, so let's treat it
- * just like a translation fault and clean the cache to the PoC.
+ * TODO: Handle nested_mmu structures here using the reverse mapping in
+ * a later version of patch series.
*/
- clean_dcache_guest_page(pfn, PAGE_SIZE);
- stage2_pte = kvm_pfn_pte(pfn, PAGE_S2);
- handle_hva_to_gpa(kvm, hva, end, &kvm_set_spte_handler, &stage2_pte);
-
- return 0;
-}
-
-static int kvm_age_hva_handler(struct kvm *kvm, gpa_t gpa, u64 size, void *data)
-{
- pud_t *pud;
- pmd_t *pmd;
- pte_t *pte;
-
- WARN_ON(size != PAGE_SIZE && size != PMD_SIZE && size != PUD_SIZE);
- if (!stage2_get_leaf_entry(kvm, gpa, &pud, &pmd, &pte))
- return 0;
-
- if (pud)
- return stage2_pudp_test_and_clear_young(pud);
- else if (pmd)
- return stage2_pmdp_test_and_clear_young(pmd);
- else
- return stage2_ptep_test_and_clear_young(pte);
-}
-
-static int kvm_test_age_hva_handler(struct kvm *kvm, gpa_t gpa, u64 size, void *data)
-{
- pud_t *pud;
- pmd_t *pmd;
- pte_t *pte;
-
- WARN_ON(size != PAGE_SIZE && size != PMD_SIZE && size != PUD_SIZE);
- if (!stage2_get_leaf_entry(kvm, gpa, &pud, &pmd, &pte))
- return 0;
-
- if (pud)
- return kvm_s2pud_young(*pud);
- else if (pmd)
- return pmd_young(*pmd);
- else
- return pte_young(*pte);
}
-int kvm_age_hva(struct kvm *kvm, unsigned long start, unsigned long end)
+bool kvm_test_age_gfn(struct kvm *kvm, struct kvm_gfn_range *range)
{
- if (!kvm->arch.pgd)
- return 0;
- trace_kvm_age_hva(start, end);
- return handle_hva_to_gpa(kvm, start, end, kvm_age_hva_handler, NULL);
-}
+ u64 size = (range->end - range->start) << PAGE_SHIFT;
-int kvm_test_age_hva(struct kvm *kvm, unsigned long hva)
-{
- if (!kvm->arch.pgd)
- return 0;
- trace_kvm_test_age_hva(hva);
- return handle_hva_to_gpa(kvm, hva, hva + PAGE_SIZE,
- kvm_test_age_hva_handler, NULL);
-}
+ if (!kvm->arch.mmu.pgt)
+ return false;
-void kvm_mmu_free_memory_caches(struct kvm_vcpu *vcpu)
-{
- mmu_free_memory_cache(&vcpu->arch.mmu_page_cache);
+ return KVM_PGT_FN(kvm_pgtable_stage2_test_clear_young)(kvm->arch.mmu.pgt,
+ range->start << PAGE_SHIFT,
+ size, false);
}
phys_addr_t kvm_mmu_get_httbr(void)
{
- if (__kvm_cpu_uses_extended_idmap())
- return virt_to_phys(merged_hyp_pgd);
- else
- return virt_to_phys(hyp_pgd);
+ return __pa(hyp_pgtable->pgd);
}
phys_addr_t kvm_get_idmap_vector(void)
@@ -2322,15 +2259,11 @@ phys_addr_t kvm_get_idmap_vector(void)
return hyp_idmap_vector;
}
-static int kvm_map_idmap_text(pgd_t *pgd)
+static int kvm_map_idmap_text(void)
{
- int err;
-
- /* Create the idmap in the boot page tables */
- err = __create_hyp_mappings(pgd, __kvm_idmap_ptrs_per_pgd(),
- hyp_idmap_start, hyp_idmap_end,
- __phys_to_pfn(hyp_idmap_start),
- PAGE_HYP_EXEC);
+ unsigned long size = hyp_idmap_end - hyp_idmap_start;
+ int err = __create_hyp_mappings(hyp_idmap_start, size, hyp_idmap_start,
+ PAGE_HYP_EXEC);
if (err)
kvm_err("Failed to idmap %lx-%lx\n",
hyp_idmap_start, hyp_idmap_end);
@@ -2338,9 +2271,24 @@ static int kvm_map_idmap_text(pgd_t *pgd)
return err;
}
-int kvm_mmu_init(void)
+static void *kvm_hyp_zalloc_page(void *arg)
+{
+ return (void *)get_zeroed_page(GFP_KERNEL);
+}
+
+static struct kvm_pgtable_mm_ops kvm_hyp_mm_ops = {
+ .zalloc_page = kvm_hyp_zalloc_page,
+ .get_page = kvm_host_get_page,
+ .put_page = kvm_host_put_page,
+ .phys_to_virt = kvm_host_va,
+ .virt_to_phys = kvm_host_pa,
+};
+
+int __init kvm_mmu_init(u32 *hyp_va_bits)
{
int err;
+ u32 idmap_bits;
+ u32 kernel_bits;
hyp_idmap_start = __pa_symbol(__hyp_idmap_text_start);
hyp_idmap_start = ALIGN_DOWN(hyp_idmap_start, PAGE_SIZE);
@@ -2354,6 +2302,25 @@ int kvm_mmu_init(void)
*/
BUG_ON((hyp_idmap_start ^ (hyp_idmap_end - 1)) & PAGE_MASK);
+ /*
+ * The ID map is always configured for 48 bits of translation, which
+ * may be fewer than the number of VA bits used by the regular kernel
+ * stage 1, when VA_BITS=52.
+ *
+ * At EL2, there is only one TTBR register, and we can't switch between
+ * translation tables *and* update TCR_EL2.T0SZ at the same time. Bottom
+ * line: we need to use the extended range with *both* our translation
+ * tables.
+ *
+ * So use the maximum of the idmap VA bits and the regular kernel stage
+ * 1 VA bits to assure that the hypervisor can both ID map its code page
+ * and map any kernel memory.
+ */
+ idmap_bits = IDMAP_VA_BITS;
+ kernel_bits = vabits_actual;
+ *hyp_va_bits = max(idmap_bits, kernel_bits);
+
+ kvm_debug("Using %u-bit virtual addresses at EL2\n", *hyp_va_bits);
kvm_debug("IDMAP page: %lx\n", hyp_idmap_start);
kvm_debug("HYP VA range: %lx:%lx\n",
kern_hyp_va(PAGE_OFFSET),
@@ -2371,77 +2338,84 @@ int kvm_mmu_init(void)
goto out;
}
- hyp_pgd = (pgd_t *)__get_free_pages(GFP_KERNEL | __GFP_ZERO, hyp_pgd_order);
- if (!hyp_pgd) {
- kvm_err("Hyp mode PGD not allocated\n");
+ hyp_pgtable = kzalloc(sizeof(*hyp_pgtable), GFP_KERNEL);
+ if (!hyp_pgtable) {
+ kvm_err("Hyp mode page-table not allocated\n");
err = -ENOMEM;
goto out;
}
- if (__kvm_cpu_uses_extended_idmap()) {
- boot_hyp_pgd = (pgd_t *)__get_free_pages(GFP_KERNEL | __GFP_ZERO,
- hyp_pgd_order);
- if (!boot_hyp_pgd) {
- kvm_err("Hyp boot PGD not allocated\n");
- err = -ENOMEM;
- goto out;
- }
-
- err = kvm_map_idmap_text(boot_hyp_pgd);
- if (err)
- goto out;
+ err = kvm_pgtable_hyp_init(hyp_pgtable, *hyp_va_bits, &kvm_hyp_mm_ops);
+ if (err)
+ goto out_free_pgtable;
- merged_hyp_pgd = (pgd_t *)__get_free_page(GFP_KERNEL | __GFP_ZERO);
- if (!merged_hyp_pgd) {
- kvm_err("Failed to allocate extra HYP pgd\n");
- goto out;
- }
- __kvm_extend_hypmap(boot_hyp_pgd, hyp_pgd, merged_hyp_pgd,
- hyp_idmap_start);
- } else {
- err = kvm_map_idmap_text(hyp_pgd);
- if (err)
- goto out;
- }
+ err = kvm_map_idmap_text();
+ if (err)
+ goto out_destroy_pgtable;
io_map_base = hyp_idmap_start;
+ __hyp_va_bits = *hyp_va_bits;
return 0;
+
+out_destroy_pgtable:
+ kvm_pgtable_hyp_destroy(hyp_pgtable);
+out_free_pgtable:
+ kfree(hyp_pgtable);
+ hyp_pgtable = NULL;
out:
- free_hyp_pgds();
return err;
}
void kvm_arch_commit_memory_region(struct kvm *kvm,
- const struct kvm_userspace_memory_region *mem,
struct kvm_memory_slot *old,
const struct kvm_memory_slot *new,
enum kvm_mr_change change)
{
+ bool log_dirty_pages = new && new->flags & KVM_MEM_LOG_DIRTY_PAGES;
+
/*
* At this point memslot has been committed and there is an
* allocated dirty_bitmap[], dirty pages will be tracked while the
* memory slot is write protected.
*/
- if (change != KVM_MR_DELETE && mem->flags & KVM_MEM_LOG_DIRTY_PAGES) {
+ if (log_dirty_pages) {
+
+ if (change == KVM_MR_DELETE)
+ return;
+
/*
- * If we're with initial-all-set, we don't need to write
- * protect any pages because they're all reported as dirty.
- * Huge pages and normal pages will be write protect gradually.
+ * Huge and normal pages are write-protected and split
+ * on either of these two cases:
+ *
+ * 1. with initial-all-set: gradually with CLEAR ioctls,
*/
- if (!kvm_dirty_log_manual_protect_and_init_set(kvm)) {
- kvm_mmu_wp_memory_region(kvm, mem->slot);
- }
+ if (kvm_dirty_log_manual_protect_and_init_set(kvm))
+ return;
+ /*
+ * or
+ * 2. without initial-all-set: all in one shot when
+ * enabling dirty logging.
+ */
+ kvm_mmu_wp_memory_region(kvm, new->id);
+ kvm_mmu_split_memory_region(kvm, new->id);
+ } else {
+ /*
+ * Free any leftovers from the eager page splitting cache. Do
+ * this when deleting, moving, disabling dirty logging, or
+ * creating the memslot (a nop). Doing it for deletes makes
+ * sure we don't leak memory, and there's no need to keep the
+ * cache around for any of the other cases.
+ */
+ kvm_mmu_free_memory_cache(&kvm->arch.mmu.split_page_cache);
}
}
int kvm_arch_prepare_memory_region(struct kvm *kvm,
- struct kvm_memory_slot *memslot,
- const struct kvm_userspace_memory_region *mem,
+ const struct kvm_memory_slot *old,
+ struct kvm_memory_slot *new,
enum kvm_mr_change change)
{
- hva_t hva = mem->userspace_addr;
- hva_t reg_end = hva + mem->memory_size;
- bool writable = !(mem->flags & KVM_MEM_READONLY);
+ hva_t hva, reg_end;
int ret = 0;
if (change != KVM_MR_CREATE && change != KVM_MR_MOVE &&
@@ -2452,15 +2426,23 @@ int kvm_arch_prepare_memory_region(struct kvm *kvm,
* Prevent userspace from creating a memory region outside of the IPA
* space addressable by the KVM guest IPA space.
*/
- if (memslot->base_gfn + memslot->npages >=
- (kvm_phys_size(kvm) >> PAGE_SHIFT))
+ if ((new->base_gfn + new->npages) > (kvm_phys_size(&kvm->arch.mmu) >> PAGE_SHIFT))
return -EFAULT;
+ /*
+ * Only support guest_memfd backed memslots with mappable memory, since
+ * there aren't any CoCo VMs that support only private memory on arm64.
+ */
+ if (kvm_slot_has_gmem(new) && !kvm_memslot_is_gmem_only(new))
+ return -EINVAL;
+
+ hva = new->userspace_addr;
+ reg_end = hva + (new->npages << PAGE_SHIFT);
+
mmap_read_lock(current->mm);
/*
* A memory region could potentially cover multiple VMAs, and any holes
- * between them, so iterate over all of them to find out if we can map
- * any of them right now.
+ * between them, so iterate over all of them.
*
* +--------------------------------------------+
* +---------------+----------------+ +----------------+
@@ -2470,51 +2452,36 @@ int kvm_arch_prepare_memory_region(struct kvm *kvm,
* +--------------------------------------------+
*/
do {
- struct vm_area_struct *vma = find_vma(current->mm, hva);
- hva_t vm_start, vm_end;
+ struct vm_area_struct *vma;
- if (!vma || vma->vm_start >= reg_end)
+ vma = find_vma_intersection(current->mm, hva, reg_end);
+ if (!vma)
break;
- /*
- * Take the intersection of this VMA with the memory region
- */
- vm_start = max(hva, vma->vm_start);
- vm_end = min(reg_end, vma->vm_end);
+ if (kvm_has_mte(kvm) && !kvm_vma_mte_allowed(vma)) {
+ ret = -EINVAL;
+ break;
+ }
if (vma->vm_flags & VM_PFNMAP) {
- gpa_t gpa = mem->guest_phys_addr +
- (vm_start - mem->userspace_addr);
- phys_addr_t pa;
-
- pa = (phys_addr_t)vma->vm_pgoff << PAGE_SHIFT;
- pa += vm_start - vma->vm_start;
-
/* IO region dirty page logging not allowed */
- if (memslot->flags & KVM_MEM_LOG_DIRTY_PAGES) {
+ if (new->flags & KVM_MEM_LOG_DIRTY_PAGES) {
ret = -EINVAL;
- goto out;
+ break;
}
- ret = kvm_phys_addr_ioremap(kvm, gpa, pa,
- vm_end - vm_start,
- writable);
- if (ret)
+ /*
+ * Cacheable PFNMAP is allowed only if the hardware
+ * supports it.
+ */
+ if (kvm_vma_is_cacheable(vma) && !kvm_supports_cacheable_pfnmap()) {
+ ret = -EINVAL;
break;
+ }
}
- hva = vm_end;
+ hva = min(reg_end, vma->vm_end);
} while (hva < reg_end);
- if (change == KVM_MR_FLAGS_ONLY)
- goto out;
-
- spin_lock(&kvm->mmu_lock);
- if (ret)
- unmap_stage2_range(kvm, mem->guest_phys_addr, mem->memory_size);
- else
- stage2_flush_memslot(kvm, memslot);
- spin_unlock(&kvm->mmu_lock);
-out:
mmap_read_unlock(current->mm);
return ret;
}
@@ -2527,20 +2494,16 @@ void kvm_arch_memslots_updated(struct kvm *kvm, u64 gen)
{
}
-void kvm_arch_flush_shadow_all(struct kvm *kvm)
-{
- kvm_free_stage2_pgd(kvm);
-}
-
void kvm_arch_flush_shadow_memslot(struct kvm *kvm,
struct kvm_memory_slot *slot)
{
gpa_t gpa = slot->base_gfn << PAGE_SHIFT;
phys_addr_t size = slot->npages << PAGE_SHIFT;
- spin_lock(&kvm->mmu_lock);
- unmap_stage2_range(kvm, gpa, size);
- spin_unlock(&kvm->mmu_lock);
+ write_lock(&kvm->mmu_lock);
+ kvm_stage2_unmap_range(&kvm->arch.mmu, gpa, size, true);
+ kvm_nested_s2_unmap(kvm, true);
+ write_unlock(&kvm->mmu_lock);
}
/*