Start implementing queue state updates using fcmpset loops.

This is in preparation for eliminating the use of the vm_page lock for
protecting queue state operations.

Introduce the vm_page_pqstate_commit_*() functions.  These functions act
as helpers around vm_page_astate_fcmpset() and are specialized for
specific types of operations.  vm_page_pqstate_commit() wraps these
functions.

Convert a number of routines to use these new helpers.  Use
vm_page_release_toq() in vm_page_unwire() and vm_page_release() to
atomically release a wiring reference and release the page into a queue.
This has the side effect that vm_page_unwire() will leave the page in
the active queue if it is already present there.

Convert the page queue scans to use the new helpers.  Simplify
vm_pageout_reinsert_inactive(), which requeues pages that were found to
be busy during an inactive queue scan, to avoid duplicating the work of
vm_pqbatch_process_page().  In particular, if PGA_REQUEUE or
PGA_REQUEUE_HEAD is set, let that be handled during batch processing.

Reviewed by:	jeff
Tested by:	pho
Sponsored by:	Netflix, Intel
Differential Revision:	https://reviews.freebsd.org/D22770
Differential Revision:	https://reviews.freebsd.org/D22771
Differential Revision:	https://reviews.freebsd.org/D22772
Differential Revision:	https://reviews.freebsd.org/D22773
Differential Revision:	https://reviews.freebsd.org/D22776
This commit is contained in:
Mark Johnston 2019-12-28 19:03:32 +00:00
parent 3c01c56b0e
commit f3f38e2580
Notes: svn2git 2020-12-20 02:59:44 +00:00
svn path=/head/; revision=356155
3 changed files with 568 additions and 509 deletions

View file

@ -134,6 +134,11 @@ static int vm_pageproc_waiters;
static SYSCTL_NODE(_vm_stats, OID_AUTO, page, CTLFLAG_RD, 0,
"VM page statistics");
static counter_u64_t pqstate_commit_retries = EARLY_COUNTER;
SYSCTL_COUNTER_U64(_vm_stats_page, OID_AUTO, pqstate_commit_retries,
CTLFLAG_RD, &pqstate_commit_retries,
"Number of failed per-page atomic queue state updates");
static counter_u64_t queue_ops = EARLY_COUNTER;
SYSCTL_COUNTER_U64(_vm_stats_page, OID_AUTO, queue_ops,
CTLFLAG_RD, &queue_ops,
@ -148,6 +153,7 @@ static void
counter_startup(void)
{
pqstate_commit_retries = counter_u64_alloc(M_WAITOK);
queue_ops = counter_u64_alloc(M_WAITOK);
queue_nops = counter_u64_alloc(M_WAITOK);
}
@ -179,7 +185,6 @@ static void vm_page_alloc_check(vm_page_t m);
static bool _vm_page_busy_sleep(vm_object_t obj, vm_page_t m,
const char *wmesg, bool nonshared, bool locked);
static void vm_page_clear_dirty_mask(vm_page_t m, vm_page_bits_t pagebits);
static void vm_page_dequeue_complete(vm_page_t m);
static void vm_page_enqueue(vm_page_t m, uint8_t queue);
static bool vm_page_free_prep(vm_page_t m);
static void vm_page_free_toq(vm_page_t m);
@ -188,9 +193,11 @@ static int vm_page_insert_after(vm_page_t m, vm_object_t object,
vm_pindex_t pindex, vm_page_t mpred);
static void vm_page_insert_radixdone(vm_page_t m, vm_object_t object,
vm_page_t mpred);
static void vm_page_mvqueue(vm_page_t m, uint8_t queue);
static void vm_page_mvqueue(vm_page_t m, const uint8_t queue,
const uint16_t nflag);
static int vm_page_reclaim_run(int req_class, int domain, u_long npages,
vm_page_t m_run, vm_paddr_t high);
static void vm_page_release_toq(vm_page_t m, uint8_t nqueue, bool noreuse);
static int vm_domain_alloc_fail(struct vm_domain *vmd, vm_object_t object,
int req);
static int vm_page_zone_import(void *arg, void **store, int cnt, int domain,
@ -3266,68 +3273,233 @@ vm_waitpfault(struct domainset *dset, int timo)
}
static struct vm_pagequeue *
vm_page_pagequeue(vm_page_t m)
_vm_page_pagequeue(vm_page_t m, uint8_t queue)
{
uint8_t queue;
if ((queue = atomic_load_8(&m->a.queue)) == PQ_NONE)
return (NULL);
return (&vm_pagequeue_domain(m)->vmd_pagequeues[queue]);
}
static inline void
vm_pqbatch_process_page(struct vm_pagequeue *pq, vm_page_t m)
#ifdef INVARIANTS
static struct vm_pagequeue *
vm_page_pagequeue(vm_page_t m)
{
struct vm_domain *vmd;
uint16_t qflags;
CRITICAL_ASSERT(curthread);
vm_pagequeue_assert_locked(pq);
return (_vm_page_pagequeue(m, vm_page_astate_load(m).queue));
}
#endif
/*
* The page daemon is allowed to set m->a.queue = PQ_NONE without
* the page queue lock held. In this case it is about to free the page,
* which must not have any queue state.
*/
qflags = atomic_load_16(&m->a.flags);
KASSERT(pq == vm_page_pagequeue(m) ||
(qflags & PGA_QUEUE_STATE_MASK) == 0,
("page %p doesn't belong to queue %p but has aflags %#x",
m, pq, qflags));
static __always_inline bool
vm_page_pqstate_fcmpset(vm_page_t m, vm_page_astate_t *old, vm_page_astate_t new)
{
vm_page_astate_t tmp;
if ((qflags & PGA_DEQUEUE) != 0) {
if (__predict_true((qflags & PGA_ENQUEUED) != 0))
vm_pagequeue_remove(pq, m);
vm_page_dequeue_complete(m);
counter_u64_add(queue_ops, 1);
} else if ((qflags & (PGA_REQUEUE | PGA_REQUEUE_HEAD)) != 0) {
if ((qflags & PGA_ENQUEUED) != 0)
TAILQ_REMOVE(&pq->pq_pl, m, plinks.q);
else {
vm_pagequeue_cnt_inc(pq);
vm_page_aflag_set(m, PGA_ENQUEUED);
tmp = *old;
do {
if (__predict_true(vm_page_astate_fcmpset(m, old, new)))
return (true);
counter_u64_add(pqstate_commit_retries, 1);
} while (old->_bits == tmp._bits);
return (false);
}
/*
* Give PGA_REQUEUE_HEAD precedence over PGA_REQUEUE.
* In particular, if both flags are set in close succession,
* only PGA_REQUEUE_HEAD will be applied, even if it was set
* first.
* Do the work of committing a queue state update that moves the page out of
* its current queue.
*/
if ((qflags & PGA_REQUEUE_HEAD) != 0) {
KASSERT(m->a.queue == PQ_INACTIVE,
("head enqueue not supported for page %p", m));
vmd = vm_pagequeue_domain(m);
TAILQ_INSERT_BEFORE(&vmd->vmd_inacthead, m, plinks.q);
} else
TAILQ_INSERT_TAIL(&pq->pq_pl, m, plinks.q);
static bool
_vm_page_pqstate_commit_dequeue(struct vm_pagequeue *pq, vm_page_t m,
vm_page_astate_t *old, vm_page_astate_t new)
{
vm_page_t next;
vm_page_aflag_clear(m, qflags & (PGA_REQUEUE |
PGA_REQUEUE_HEAD));
counter_u64_add(queue_ops, 1);
vm_pagequeue_assert_locked(pq);
KASSERT(vm_page_pagequeue(m) == pq,
("%s: queue %p does not match page %p", __func__, pq, m));
KASSERT(old->queue != PQ_NONE && new.queue != old->queue,
("%s: invalid queue indices %d %d",
__func__, old->queue, new.queue));
/*
* Once the queue index of the page changes there is nothing
* synchronizing with further updates to the page's physical
* queue state. Therefore we must speculatively remove the page
* from the queue now and be prepared to roll back if the queue
* state update fails. If the page is not physically enqueued then
* we just update its queue index.
*/
if ((old->flags & PGA_ENQUEUED) != 0) {
new.flags &= ~PGA_ENQUEUED;
next = TAILQ_NEXT(m, plinks.q);
TAILQ_REMOVE(&pq->pq_pl, m, plinks.q);
vm_pagequeue_cnt_dec(pq);
if (!vm_page_pqstate_fcmpset(m, old, new)) {
if (next == NULL)
TAILQ_INSERT_TAIL(&pq->pq_pl, m, plinks.q);
else
TAILQ_INSERT_BEFORE(next, m, plinks.q);
vm_pagequeue_cnt_inc(pq);
return (false);
} else {
return (true);
}
} else {
return (vm_page_pqstate_fcmpset(m, old, new));
}
}
static bool
vm_page_pqstate_commit_dequeue(vm_page_t m, vm_page_astate_t *old,
vm_page_astate_t new)
{
struct vm_pagequeue *pq;
vm_page_astate_t as;
bool ret;
pq = _vm_page_pagequeue(m, old->queue);
/*
* The queue field and PGA_ENQUEUED flag are stable only so long as the
* corresponding page queue lock is held.
*/
vm_pagequeue_lock(pq);
as = vm_page_astate_load(m);
if (__predict_false(as._bits != old->_bits)) {
*old = as;
ret = false;
} else {
ret = _vm_page_pqstate_commit_dequeue(pq, m, old, new);
}
vm_pagequeue_unlock(pq);
return (ret);
}
/*
* Commit a queue state update that enqueues or requeues a page.
*/
static bool
_vm_page_pqstate_commit_requeue(struct vm_pagequeue *pq, vm_page_t m,
vm_page_astate_t *old, vm_page_astate_t new)
{
struct vm_domain *vmd;
vm_pagequeue_assert_locked(pq);
KASSERT(old->queue != PQ_NONE && new.queue == old->queue,
("%s: invalid queue indices %d %d",
__func__, old->queue, new.queue));
new.flags |= PGA_ENQUEUED;
if (!vm_page_pqstate_fcmpset(m, old, new))
return (false);
if ((old->flags & PGA_ENQUEUED) != 0)
TAILQ_REMOVE(&pq->pq_pl, m, plinks.q);
else
vm_pagequeue_cnt_inc(pq);
/*
* Give PGA_REQUEUE_HEAD precedence over PGA_REQUEUE. In particular, if
* both flags are set in close succession, only PGA_REQUEUE_HEAD will be
* applied, even if it was set first.
*/
if ((old->flags & PGA_REQUEUE_HEAD) != 0) {
vmd = vm_pagequeue_domain(m);
KASSERT(pq == &vmd->vmd_pagequeues[PQ_INACTIVE],
("%s: invalid page queue for page %p", __func__, m));
TAILQ_INSERT_BEFORE(&vmd->vmd_inacthead, m, plinks.q);
} else {
TAILQ_INSERT_TAIL(&pq->pq_pl, m, plinks.q);
}
return (true);
}
/*
* Commit a queue state update that encodes a request for a deferred queue
* operation.
*/
static bool
vm_page_pqstate_commit_request(vm_page_t m, vm_page_astate_t *old,
vm_page_astate_t new)
{
KASSERT(old->queue == new.queue || new.queue != PQ_NONE,
("%s: invalid state, queue %d flags %x",
__func__, new.queue, new.flags));
if (old->_bits != new._bits &&
!vm_page_pqstate_fcmpset(m, old, new))
return (false);
vm_page_pqbatch_submit(m, new.queue);
return (true);
}
/*
* A generic queue state update function. This handles more cases than the
* specialized functions above.
*/
bool
vm_page_pqstate_commit(vm_page_t m, vm_page_astate_t *old, vm_page_astate_t new)
{
if (old->_bits == new._bits)
return (true);
if (old->queue != PQ_NONE && new.queue != old->queue) {
if (!vm_page_pqstate_commit_dequeue(m, old, new))
return (false);
if (new.queue != PQ_NONE)
vm_page_pqbatch_submit(m, new.queue);
} else {
if (!vm_page_pqstate_fcmpset(m, old, new))
return (false);
if (new.queue != PQ_NONE &&
((new.flags & ~old->flags) & PGA_QUEUE_OP_MASK) != 0)
vm_page_pqbatch_submit(m, new.queue);
}
return (true);
}
/*
* Apply deferred queue state updates to a page.
*/
static inline void
vm_pqbatch_process_page(struct vm_pagequeue *pq, vm_page_t m, uint8_t queue)
{
vm_page_astate_t new, old;
CRITICAL_ASSERT(curthread);
vm_pagequeue_assert_locked(pq);
KASSERT(queue < PQ_COUNT,
("%s: invalid queue index %d", __func__, queue));
KASSERT(pq == _vm_page_pagequeue(m, queue),
("%s: page %p does not belong to queue %p", __func__, m, pq));
for (old = vm_page_astate_load(m);;) {
if (__predict_false(old.queue != queue ||
(old.flags & PGA_QUEUE_OP_MASK) == 0)) {
counter_u64_add(queue_nops, 1);
break;
}
KASSERT(old.queue != PQ_NONE || (old.flags & PGA_QUEUE_STATE_MASK) == 0,
("%s: page %p has unexpected queue state", __func__, m));
new = old;
if ((old.flags & PGA_DEQUEUE) != 0) {
new.flags &= ~PGA_QUEUE_OP_MASK;
new.queue = PQ_NONE;
if (__predict_true(_vm_page_pqstate_commit_dequeue(pq,
m, &old, new))) {
counter_u64_add(queue_ops, 1);
break;
}
} else {
new.flags &= ~(PGA_REQUEUE | PGA_REQUEUE_HEAD);
if (__predict_true(_vm_page_pqstate_commit_requeue(pq,
m, &old, new))) {
counter_u64_add(queue_ops, 1);
break;
}
}
}
}
@ -3335,15 +3507,10 @@ static void
vm_pqbatch_process(struct vm_pagequeue *pq, struct vm_batchqueue *bq,
uint8_t queue)
{
vm_page_t m;
int i;
for (i = 0; i < bq->bq_cnt; i++) {
m = bq->bq_pa[i];
if (__predict_false(m->a.queue != queue))
continue;
vm_pqbatch_process_page(pq, m);
}
for (i = 0; i < bq->bq_cnt; i++)
vm_pqbatch_process_page(pq, bq->bq_pa[i], queue);
vm_batchqueue_init(bq);
}
@ -3381,21 +3548,7 @@ vm_page_pqbatch_submit(vm_page_t m, uint8_t queue)
critical_enter();
bq = DPCPU_PTR(pqbatch[domain][queue]);
vm_pqbatch_process(pq, bq, queue);
/*
* The page may have been logically dequeued before we acquired the
* page queue lock. In this case, since we either hold the page lock
* or the page is being freed, a different thread cannot be concurrently
* enqueuing the page.
*/
if (__predict_true(m->a.queue == queue))
vm_pqbatch_process_page(pq, m);
else {
KASSERT(m->a.queue == PQ_NONE,
("invalid queue transition for page %p", m));
KASSERT((m->a.flags & PGA_ENQUEUED) == 0,
("page %p is enqueued with invalid queue index", m));
}
vm_pqbatch_process_page(pq, m, queue);
vm_pagequeue_unlock(pq);
critical_exit();
}
@ -3439,21 +3592,6 @@ vm_page_pqbatch_drain(void)
thread_unlock(td);
}
/*
* Complete the logical removal of a page from a page queue. We must be
* careful to synchronize with the page daemon, which may be concurrently
* examining the page with only the page lock held. The page must not be
* in a state where it appears to be logically enqueued.
*/
static void
vm_page_dequeue_complete(vm_page_t m)
{
m->a.queue = PQ_NONE;
atomic_thread_fence_rel();
vm_page_aflag_clear(m, PGA_QUEUE_STATE_MASK);
}
/*
* vm_page_dequeue_deferred: [ internal use only ]
*
@ -3466,109 +3604,45 @@ vm_page_dequeue_complete(vm_page_t m)
void
vm_page_dequeue_deferred(vm_page_t m)
{
uint8_t queue;
vm_page_astate_t new, old;
vm_page_assert_locked(m);
if ((queue = vm_page_queue(m)) == PQ_NONE)
return;
/*
* Set PGA_DEQUEUE if it is not already set to handle a concurrent call
* to vm_page_dequeue_deferred_free(). In particular, avoid modifying
* the page's queue state once vm_page_dequeue_deferred_free() has been
* called. In the event of a race, two batch queue entries for the page
* will be created, but the second will have no effect.
*/
if (vm_page_pqstate_cmpset(m, queue, queue, PGA_DEQUEUE, PGA_DEQUEUE))
vm_page_pqbatch_submit(m, queue);
}
/*
* A variant of vm_page_dequeue_deferred() that does not assert the page
* lock and is only to be called from vm_page_free_prep(). Because the
* page is being freed, we can assume that nothing other than the page
* daemon is scheduling queue operations on this page, so we get for
* free the mutual exclusion that is otherwise provided by the page lock.
* To handle races, the page daemon must take care to atomically check
* for PGA_DEQUEUE when updating queue state.
*/
static void
vm_page_dequeue_deferred_free(vm_page_t m)
{
uint8_t queue;
KASSERT(m->ref_count == 0, ("page %p has references", m));
for (;;) {
if ((m->a.flags & PGA_DEQUEUE) != 0)
return;
atomic_thread_fence_acq();
if ((queue = atomic_load_8(&m->a.queue)) == PQ_NONE)
return;
if (vm_page_pqstate_cmpset(m, queue, queue, PGA_DEQUEUE,
PGA_DEQUEUE)) {
vm_page_pqbatch_submit(m, queue);
old = vm_page_astate_load(m);
do {
if (old.queue == PQ_NONE) {
KASSERT((old.flags & PGA_QUEUE_STATE_MASK) == 0,
("%s: page %p has unexpected queue state",
__func__, m));
break;
}
}
new = old;
new.flags |= PGA_DEQUEUE;
} while (!vm_page_pqstate_commit_request(m, &old, new));
}
/*
* vm_page_dequeue:
*
* Remove the page from whichever page queue it's in, if any.
* The page must either be locked or unallocated. This constraint
* ensures that the queue state of the page will remain consistent
* after this function returns.
* Remove the page from whichever page queue it's in, if any, before
* returning.
*/
void
vm_page_dequeue(vm_page_t m)
{
struct vm_pagequeue *pq, *pq1;
uint16_t aflags;
vm_page_astate_t new, old;
KASSERT(mtx_owned(vm_page_lockptr(m)) || m->ref_count == 0,
("page %p is allocated and unlocked", m));
for (pq = vm_page_pagequeue(m);; pq = pq1) {
if (pq == NULL) {
/*
* A thread may be concurrently executing
* vm_page_dequeue_complete(). Ensure that all queue
* state is cleared before we return.
*/
aflags = atomic_load_16(&m->a.flags);
if ((aflags & PGA_QUEUE_STATE_MASK) == 0)
return;
KASSERT((aflags & PGA_DEQUEUE) != 0,
("page %p has unexpected queue state flags %#x",
m, aflags));
/*
* Busy wait until the thread updating queue state is
* finished. Such a thread must be executing in a
* critical section.
*/
cpu_spinwait();
pq1 = vm_page_pagequeue(m);
continue;
}
vm_pagequeue_lock(pq);
if ((pq1 = vm_page_pagequeue(m)) == pq)
old = vm_page_astate_load(m);
do {
if (old.queue == PQ_NONE) {
KASSERT((old.flags & PGA_QUEUE_STATE_MASK) == 0,
("%s: page %p has unexpected queue state",
__func__, m));
break;
vm_pagequeue_unlock(pq);
}
KASSERT(pq == vm_page_pagequeue(m),
("%s: page %p migrated directly between queues", __func__, m));
KASSERT((m->a.flags & PGA_DEQUEUE) != 0 ||
mtx_owned(vm_page_lockptr(m)),
("%s: queued unlocked page %p", __func__, m));
new = old;
new.flags &= ~PGA_QUEUE_OP_MASK;
new.queue = PQ_NONE;
} while (!vm_page_pqstate_commit_dequeue(m, &old, new));
if ((m->a.flags & PGA_ENQUEUED) != 0)
vm_pagequeue_remove(pq, m);
vm_page_dequeue_complete(m);
vm_pagequeue_unlock(pq);
}
/*
@ -3618,66 +3692,23 @@ vm_page_requeue(vm_page_t m)
* vm_page_swapqueue: [ internal use only ]
*
* Move the page from one queue to another, or to the tail of its
* current queue, in the face of a possible concurrent call to
* vm_page_dequeue_deferred_free().
* current queue, in the face of a possible concurrent free of the
* page.
*/
void
vm_page_swapqueue(vm_page_t m, uint8_t oldq, uint8_t newq)
{
struct vm_pagequeue *pq;
vm_page_t next;
bool queued;
vm_page_astate_t new, old;
KASSERT(oldq < PQ_COUNT && newq < PQ_COUNT && oldq != newq,
("vm_page_swapqueue: invalid queues (%d, %d)", oldq, newq));
vm_page_assert_locked(m);
pq = &vm_pagequeue_domain(m)->vmd_pagequeues[oldq];
vm_pagequeue_lock(pq);
/*
* The physical queue state might change at any point before the page
* queue lock is acquired, so we must verify that we hold the correct
* lock before proceeding.
*/
if (__predict_false(m->a.queue != oldq)) {
vm_pagequeue_unlock(pq);
old = vm_page_astate_load(m);
do {
if (old.queue != oldq || (old.flags & PGA_DEQUEUE) != 0)
return;
}
new = old;
new.flags |= PGA_REQUEUE;
new.queue = newq;
} while (!vm_page_pqstate_commit_dequeue(m, &old, new));
/*
* Once the queue index of the page changes, there is nothing
* synchronizing with further updates to the physical queue state.
* Therefore we must remove the page from the queue now in anticipation
* of a successful commit, and be prepared to roll back.
*/
if (__predict_true((m->a.flags & PGA_ENQUEUED) != 0)) {
next = TAILQ_NEXT(m, plinks.q);
TAILQ_REMOVE(&pq->pq_pl, m, plinks.q);
vm_page_aflag_clear(m, PGA_ENQUEUED);
queued = true;
} else {
queued = false;
}
/*
* Atomically update the queue field and set PGA_REQUEUE while
* ensuring that PGA_DEQUEUE has not been set.
*/
if (__predict_false(!vm_page_pqstate_cmpset(m, oldq, newq, PGA_DEQUEUE,
PGA_REQUEUE))) {
if (queued) {
vm_page_aflag_set(m, PGA_ENQUEUED);
if (next != NULL)
TAILQ_INSERT_BEFORE(next, m, plinks.q);
else
TAILQ_INSERT_TAIL(&pq->pq_pl, m, plinks.q);
}
vm_pagequeue_unlock(pq);
return;
}
vm_pagequeue_cnt_dec(pq);
vm_pagequeue_unlock(pq);
vm_page_pqbatch_submit(m, newq);
}
@ -3766,7 +3797,7 @@ vm_page_free_prep(vm_page_t m)
* dequeue.
*/
if ((m->oflags & VPO_UNMANAGED) == 0)
vm_page_dequeue_deferred_free(m);
vm_page_dequeue_deferred(m);
m->valid = 0;
vm_page_undirty(m);
@ -3903,30 +3934,18 @@ vm_page_wire_mapped(vm_page_t m)
}
/*
* Release one wiring of the specified page, potentially allowing it to be
* paged out.
*
* Only managed pages belonging to an object can be paged out. If the number
* of wirings transitions to zero and the page is eligible for page out, then
* the page is added to the specified paging queue. If the released wiring
* represented the last reference to the page, the page is freed.
*
* A managed page must be locked.
* Release a wiring reference to a managed page. If the page still belongs to
* an object, update its position in the page queues to reflect the reference.
* If the wiring was the last reference to the page, free the page.
*/
void
vm_page_unwire(vm_page_t m, uint8_t queue)
static void
vm_page_unwire_managed(vm_page_t m, uint8_t nqueue, bool noreuse)
{
u_int old;
bool locked;
KASSERT(queue < PQ_COUNT,
("vm_page_unwire: invalid queue %u request for page %p", queue, m));
if ((m->oflags & VPO_UNMANAGED) != 0) {
if (vm_page_unwire_noq(m) && m->ref_count == 0)
vm_page_free(m);
return;
}
KASSERT((m->oflags & VPO_UNMANAGED) == 0,
("%s: page %p is unmanaged", __func__, m));
/*
* Update LRU state before releasing the wiring reference.
@ -3942,10 +3961,7 @@ vm_page_unwire(vm_page_t m, uint8_t queue)
if (!locked && VPRC_WIRE_COUNT(old) == 1) {
vm_page_lock(m);
locked = true;
if (queue == PQ_ACTIVE && vm_page_queue(m) == PQ_ACTIVE)
vm_page_reference(m);
else
vm_page_mvqueue(m, queue);
vm_page_release_toq(m, nqueue, false);
}
} while (!atomic_fcmpset_rel_int(&m->ref_count, &old, old - 1));
@ -3964,6 +3980,33 @@ vm_page_unwire(vm_page_t m, uint8_t queue)
}
}
/*
* Release one wiring of the specified page, potentially allowing it to be
* paged out.
*
* Only managed pages belonging to an object can be paged out. If the number
* of wirings transitions to zero and the page is eligible for page out, then
* the page is added to the specified paging queue. If the released wiring
* represented the last reference to the page, the page is freed.
*
* A managed page must be locked.
*/
void
vm_page_unwire(vm_page_t m, uint8_t nqueue)
{
KASSERT(nqueue < PQ_COUNT,
("vm_page_unwire: invalid queue %u request for page %p",
nqueue, m));
if ((m->oflags & VPO_UNMANAGED) != 0) {
if (vm_page_unwire_noq(m) && m->ref_count == 0)
vm_page_free(m);
return;
}
vm_page_unwire_managed(m, nqueue, false);
}
/*
* Unwire a page without (re-)inserting it into a page queue. It is up
* to the caller to enqueue, requeue, or free the page as appropriate.
@ -3988,10 +4031,9 @@ vm_page_unwire_noq(vm_page_t m)
}
/*
* Ensure that the page is in the specified page queue. If the page is
* Ensure that the page ends up in the specified page queue. If the page is
* active or being moved to the active queue, ensure that its act_count is
* at least ACT_INIT but do not otherwise mess with it. Otherwise, ensure that
* the page is at the tail of its page queue.
* at least ACT_INIT but do not otherwise mess with it.
*
* The page may be wired. The caller should release its wiring reference
* before releasing the page lock, otherwise the page daemon may immediately
@ -4000,24 +4042,31 @@ vm_page_unwire_noq(vm_page_t m)
* A managed page must be locked.
*/
static __always_inline void
vm_page_mvqueue(vm_page_t m, const uint8_t nqueue)
vm_page_mvqueue(vm_page_t m, const uint8_t nqueue, const uint16_t nflag)
{
vm_page_astate_t old, new;
vm_page_assert_locked(m);
KASSERT((m->oflags & VPO_UNMANAGED) == 0,
("vm_page_mvqueue: page %p is unmanaged", m));
("%s: page %p is unmanaged", __func__, m));
KASSERT(m->ref_count > 0,
("%s: page %p does not carry any references", __func__, m));
KASSERT(nflag == PGA_REQUEUE || nflag == PGA_REQUEUE_HEAD,
("%s: invalid flags %x", __func__, nflag));
if (vm_page_queue(m) != nqueue) {
vm_page_dequeue(m);
vm_page_enqueue(m, nqueue);
} else if (nqueue != PQ_ACTIVE) {
vm_page_requeue(m);
old = vm_page_astate_load(m);
do {
new = old;
if (nqueue == PQ_ACTIVE)
new.act_count = max(old.act_count, ACT_INIT);
if (old.queue == nqueue) {
if (nqueue != PQ_ACTIVE)
new.flags |= nflag;
} else {
new.flags |= nflag;
new.queue = nqueue;
}
if (nqueue == PQ_ACTIVE && m->a.act_count < ACT_INIT)
m->a.act_count = ACT_INIT;
} while (!vm_page_pqstate_commit(m, &old, new));
}
/*
@ -4029,7 +4078,7 @@ vm_page_activate(vm_page_t m)
if ((m->oflags & VPO_UNMANAGED) != 0 || vm_page_wired(m))
return;
vm_page_mvqueue(m, PQ_ACTIVE);
vm_page_mvqueue(m, PQ_ACTIVE, PGA_REQUEUE);
}
/*
@ -4042,28 +4091,7 @@ vm_page_deactivate(vm_page_t m)
if ((m->oflags & VPO_UNMANAGED) != 0 || vm_page_wired(m))
return;
vm_page_mvqueue(m, PQ_INACTIVE);
}
/*
* Move the specified page close to the head of the inactive queue,
* bypassing LRU. A marker page is used to maintain FIFO ordering.
* As with regular enqueues, we use a per-CPU batch queue to reduce
* contention on the page queue lock.
*/
static void
_vm_page_deactivate_noreuse(vm_page_t m)
{
vm_page_assert_locked(m);
if (!vm_page_inactive(m)) {
vm_page_dequeue(m);
m->a.queue = PQ_INACTIVE;
}
if ((m->a.flags & PGA_REQUEUE_HEAD) == 0)
vm_page_aflag_set(m, PGA_REQUEUE_HEAD);
vm_page_pqbatch_submit(m, PQ_INACTIVE);
vm_page_mvqueue(m, PQ_INACTIVE, PGA_REQUEUE);
}
void
@ -4073,8 +4101,9 @@ vm_page_deactivate_noreuse(vm_page_t m)
KASSERT(m->object != NULL,
("vm_page_deactivate_noreuse: page %p has no object", m));
if ((m->oflags & VPO_UNMANAGED) == 0 && !vm_page_wired(m))
_vm_page_deactivate_noreuse(m);
if ((m->oflags & VPO_UNMANAGED) != 0 || vm_page_wired(m))
return;
vm_page_mvqueue(m, PQ_INACTIVE, PGA_REQUEUE_HEAD);
}
/*
@ -4086,7 +4115,7 @@ vm_page_launder(vm_page_t m)
if ((m->oflags & VPO_UNMANAGED) != 0 || vm_page_wired(m))
return;
vm_page_mvqueue(m, PQ_LAUNDRY);
vm_page_mvqueue(m, PQ_LAUNDRY, PGA_REQUEUE);
}
/*
@ -4104,9 +4133,14 @@ vm_page_unswappable(vm_page_t m)
vm_page_enqueue(m, PQ_UNSWAPPABLE);
}
/*
* Release a page back to the page queues in preparation for unwiring.
*/
static void
vm_page_release_toq(vm_page_t m, int flags)
vm_page_release_toq(vm_page_t m, uint8_t nqueue, const bool noreuse)
{
vm_page_astate_t old, new;
uint16_t nflag;
vm_page_assert_locked(m);
@ -4120,12 +4154,30 @@ vm_page_release_toq(vm_page_t m, int flags)
* If we were asked to not cache the page, place it near the head of the
* inactive queue so that is reclaimed sooner.
*/
if ((flags & (VPR_TRYFREE | VPR_NOREUSE)) != 0 || m->valid == 0)
_vm_page_deactivate_noreuse(m);
else if (vm_page_active(m))
vm_page_reference(m);
else
vm_page_mvqueue(m, PQ_INACTIVE);
if (noreuse || m->valid == 0) {
nqueue = PQ_INACTIVE;
nflag = PGA_REQUEUE_HEAD;
} else {
nflag = PGA_REQUEUE;
}
old = vm_page_astate_load(m);
do {
new = old;
/*
* If the page is already in the active queue and we are not
* trying to accelerate reclamation, simply mark it as
* referenced and avoid any queue operations.
*/
new.flags &= ~PGA_QUEUE_OP_MASK;
if (nflag != PGA_REQUEUE_HEAD && old.queue == PQ_ACTIVE)
new.flags |= PGA_REFERENCED;
else {
new.flags |= nflag;
new.queue = nqueue;
}
} while (!vm_page_pqstate_commit(m, &old, new));
}
/*
@ -4135,8 +4187,6 @@ void
vm_page_release(vm_page_t m, int flags)
{
vm_object_t object;
u_int old;
bool locked;
KASSERT((m->oflags & VPO_UNMANAGED) == 0,
("vm_page_release: page %p is unmanaged", m));
@ -4157,37 +4207,7 @@ vm_page_release(vm_page_t m, int flags)
VM_OBJECT_WUNLOCK(object);
}
}
/*
* Update LRU state before releasing the wiring reference.
* Use a release store when updating the reference count to
* synchronize with vm_page_free_prep().
*/
old = m->ref_count;
locked = false;
do {
KASSERT(VPRC_WIRE_COUNT(old) > 0,
("vm_page_unwire: wire count underflow for page %p", m));
if (!locked && VPRC_WIRE_COUNT(old) == 1) {
vm_page_lock(m);
locked = true;
vm_page_release_toq(m, flags);
}
} while (!atomic_fcmpset_rel_int(&m->ref_count, &old, old - 1));
/*
* Release the lock only after the wiring is released, to ensure that
* the page daemon does not encounter and dequeue the page while it is
* still wired.
*/
if (locked)
vm_page_unlock(m);
if (VPRC_WIRE_COUNT(old) == 1) {
vm_wire_sub(1);
if (old == 1)
vm_page_free(m);
}
vm_page_unwire_managed(m, PQ_INACTIVE, flags != 0);
}
/* See vm_page_release(). */
@ -4206,7 +4226,7 @@ vm_page_release_locked(vm_page_t m, int flags)
vm_page_free(m);
} else {
vm_page_lock(m);
vm_page_release_toq(m, flags);
vm_page_release_toq(m, PQ_INACTIVE, flags != 0);
vm_page_unlock(m);
}
}

View file

@ -631,6 +631,8 @@ vm_page_t vm_page_lookup (vm_object_t, vm_pindex_t);
vm_page_t vm_page_next(vm_page_t m);
void vm_page_pqbatch_drain(void);
void vm_page_pqbatch_submit(vm_page_t m, uint8_t queue);
bool vm_page_pqstate_commit(vm_page_t m, vm_page_astate_t *old,
vm_page_astate_t new);
vm_page_t vm_page_prev(vm_page_t m);
bool vm_page_ps_test(vm_page_t m, int flags, vm_page_t skip_m);
void vm_page_putfake(vm_page_t m);
@ -901,11 +903,19 @@ vm_page_undirty(vm_page_t m)
m->dirty = 0;
}
static inline uint8_t
_vm_page_queue(vm_page_astate_t as)
{
if ((as.flags & PGA_DEQUEUE) != 0)
return (PQ_NONE);
return (as.queue);
}
/*
* vm_page_queue:
*
* Return the index of the queue containing m. This index is guaranteed
* not to change while the page lock is held.
* Return the index of the queue containing m.
*/
static inline uint8_t
vm_page_queue(vm_page_t m)
@ -913,10 +923,7 @@ vm_page_queue(vm_page_t m)
vm_page_assert_locked(m);
if ((m->a.flags & PGA_DEQUEUE) != 0)
return (PQ_NONE);
atomic_thread_fence_acq();
return (m->a.queue);
return (_vm_page_queue(vm_page_astate_load(m)));
}
static inline bool

View file

@ -718,7 +718,8 @@ vm_pageout_launder(struct vm_domain *vmd, int launder, bool in_shortfall)
struct mtx *mtx;
vm_object_t object;
vm_page_t m, marker;
int act_delta, error, numpagedout, queue, starting_target;
vm_page_astate_t new, old;
int act_delta, error, numpagedout, queue, refs, starting_target;
int vnodes_skipped;
bool pageout_ok;
@ -820,9 +821,8 @@ vm_pageout_launder(struct vm_domain *vmd, int launder, bool in_shortfall)
* wire count is guaranteed not to increase.
*/
if (__predict_false(vm_page_wired(m))) {
vm_page_xunbusy(m);
vm_page_dequeue_deferred(m);
continue;
goto skip_page;
}
/*
@ -832,40 +832,42 @@ vm_pageout_launder(struct vm_domain *vmd, int launder, bool in_shortfall)
if (vm_page_none_valid(m))
goto free_page;
refs = object->ref_count != 0 ? pmap_ts_referenced(m) : 0;
for (old = vm_page_astate_load(m);;) {
/*
* If the page has been referenced and the object is not dead,
* reactivate or requeue the page depending on whether the
* object is mapped.
*
* Test PGA_REFERENCED after calling pmap_ts_referenced() so
* that a reference from a concurrently destroyed mapping is
* observed here and now.
* Check to see if the page has been removed from the
* queue since the first such check. Leave it alone if
* so, discarding any references collected by
* pmap_ts_referenced().
*/
if (object->ref_count != 0)
act_delta = pmap_ts_referenced(m);
else {
KASSERT(!pmap_page_is_mapped(m),
("page %p is mapped", m));
act_delta = 0;
}
if ((m->a.flags & PGA_REFERENCED) != 0) {
vm_page_aflag_clear(m, PGA_REFERENCED);
if (__predict_false(_vm_page_queue(old) == PQ_NONE))
goto skip_page;
new = old;
act_delta = refs;
if ((old.flags & PGA_REFERENCED) != 0) {
new.flags &= ~PGA_REFERENCED;
act_delta++;
}
if (act_delta != 0) {
if (object->ref_count != 0) {
vm_page_xunbusy(m);
VM_CNT_INC(v_reactivated);
vm_page_activate(m);
if (act_delta == 0) {
;
} else if (object->ref_count != 0) {
/*
* Increase the activation count if the page
* was referenced while in the laundry queue.
* This makes it less likely that the page will
* be returned prematurely to the inactive
* queue.
* Increase the activation count if the page was
* referenced while in the laundry queue. This
* makes it less likely that the page will be
* returned prematurely to the laundry queue.
*/
m->a.act_count += act_delta + ACT_ADVANCE;
new.act_count += ACT_ADVANCE +
act_delta;
if (new.act_count > ACT_MAX)
new.act_count = ACT_MAX;
new.flags |= PGA_REQUEUE;
new.queue = PQ_ACTIVE;
if (!vm_page_pqstate_commit(m, &old, new))
continue;
/*
* If this was a background laundering, count
@ -877,12 +879,15 @@ vm_pageout_launder(struct vm_domain *vmd, int launder, bool in_shortfall)
*/
if (!in_shortfall)
launder--;
continue;
VM_CNT_INC(v_reactivated);
goto skip_page;
} else if ((object->flags & OBJ_DEAD) == 0) {
vm_page_xunbusy(m);
vm_page_requeue(m);
new.flags |= PGA_REQUEUE;
if (!vm_page_pqstate_commit(m, &old, new))
continue;
goto skip_page;
}
break;
}
/*
@ -895,9 +900,8 @@ vm_pageout_launder(struct vm_domain *vmd, int launder, bool in_shortfall)
if (object->ref_count != 0) {
vm_page_test_dirty(m);
if (m->dirty == 0 && !vm_page_try_remove_all(m)) {
vm_page_xunbusy(m);
vm_page_dequeue_deferred(m);
continue;
goto skip_page;
}
}
@ -920,9 +924,8 @@ vm_pageout_launder(struct vm_domain *vmd, int launder, bool in_shortfall)
else
pageout_ok = true;
if (!pageout_ok) {
vm_page_xunbusy(m);
vm_page_requeue(m);
continue;
vm_page_launder(m);
goto skip_page;
}
/*
@ -948,9 +951,11 @@ vm_pageout_launder(struct vm_domain *vmd, int launder, bool in_shortfall)
}
mtx = NULL;
object = NULL;
} else
} else {
skip_page:
vm_page_xunbusy(m);
}
}
if (mtx != NULL) {
mtx_unlock(mtx);
mtx = NULL;
@ -1195,8 +1200,10 @@ vm_pageout_scan_active(struct vm_domain *vmd, int page_shortage)
vm_object_t object;
vm_page_t m, marker;
struct vm_pagequeue *pq;
vm_page_astate_t old, new;
long min_scan;
int act_delta, max_scan, scan_tick;
int act_delta, max_scan, ps_delta, refs, scan_tick;
uint8_t nqueue;
marker = &vmd->vmd_markers[PQ_ACTIVE];
pq = &vmd->vmd_pagequeues[PQ_ACTIVE];
@ -1279,6 +1286,14 @@ vm_pageout_scan_active(struct vm_domain *vmd, int page_shortage)
*/
continue;
/* Deferred free of swap space. */
if ((m->a.flags & PGA_SWAP_FREE) != 0 &&
VM_OBJECT_TRYWLOCK(object)) {
if (m->object == object)
vm_pager_page_unswapped(m);
VM_OBJECT_WUNLOCK(object);
}
/*
* Check to see "how much" the page has been used.
*
@ -1298,71 +1313,91 @@ vm_pageout_scan_active(struct vm_domain *vmd, int page_shortage)
* This race delays the detection of a new reference. At
* worst, we will deactivate and reactivate the page.
*/
if (object->ref_count != 0)
act_delta = pmap_ts_referenced(m);
else
act_delta = 0;
if ((m->a.flags & PGA_REFERENCED) != 0) {
vm_page_aflag_clear(m, PGA_REFERENCED);
act_delta++;
}
refs = object->ref_count != 0 ? pmap_ts_referenced(m) : 0;
/* Deferred free of swap space. */
if ((m->a.flags & PGA_SWAP_FREE) != 0 &&
VM_OBJECT_TRYWLOCK(object)) {
if (m->object == object)
vm_pager_page_unswapped(m);
VM_OBJECT_WUNLOCK(object);
}
old = vm_page_astate_load(m);
do {
/*
* Check to see if the page has been removed from the
* queue since the first such check. Leave it alone if
* so, discarding any references collected by
* pmap_ts_referenced().
*/
if (__predict_false(_vm_page_queue(old) == PQ_NONE))
break;
/*
* Advance or decay the act_count based on recent usage.
*/
new = old;
act_delta = refs;
if ((old.flags & PGA_REFERENCED) != 0) {
new.flags &= ~PGA_REFERENCED;
act_delta++;
}
if (act_delta != 0) {
m->a.act_count += ACT_ADVANCE + act_delta;
if (m->a.act_count > ACT_MAX)
m->a.act_count = ACT_MAX;
} else
m->a.act_count -= min(m->a.act_count, ACT_DECLINE);
new.act_count += ACT_ADVANCE + act_delta;
if (new.act_count > ACT_MAX)
new.act_count = ACT_MAX;
} else {
new.act_count -= min(new.act_count,
ACT_DECLINE);
}
if (m->a.act_count == 0) {
if (new.act_count > 0) {
/*
* When not short for inactive pages, let dirty pages go
* through the inactive queue before moving to the
* laundry queues. This gives them some extra time to
* be reactivated, potentially avoiding an expensive
* pageout. However, during a page shortage, the
* inactive queue is necessarily small, and so dirty
* pages would only spend a trivial amount of time in
* the inactive queue. Therefore, we might as well
* place them directly in the laundry queue to reduce
* queuing overhead.
* Adjust the activation count and keep the page
* in the active queue. The count might be left
* unchanged if it is saturated. The page may
* have been moved to a different queue since we
* started the scan, in which case we move it
* back.
*/
if (page_shortage <= 0) {
vm_page_swapqueue(m, PQ_ACTIVE, PQ_INACTIVE);
ps_delta = 0;
if (old.queue != PQ_ACTIVE) {
old.queue = PQ_ACTIVE;
old.flags |= PGA_REQUEUE;
}
} else {
/*
* When not short for inactive pages, let dirty
* pages go through the inactive queue before
* moving to the laundry queue. This gives them
* some extra time to be reactivated,
* potentially avoiding an expensive pageout.
* However, during a page shortage, the inactive
* queue is necessarily small, and so dirty
* pages would only spend a trivial amount of
* time in the inactive queue. Therefore, we
* might as well place them directly in the
* laundry queue to reduce queuing overhead.
*
* Calling vm_page_test_dirty() here would
* require acquisition of the object's write
* lock. However, during a page shortage,
* directing dirty pages into the laundry
* queue is only an optimization and not a
* directing dirty pages into the laundry queue
* is only an optimization and not a
* requirement. Therefore, we simply rely on
* the opportunistic updates to the page's
* dirty field by the pmap.
* the opportunistic updates to the page's dirty
* field by the pmap.
*/
if (m->dirty == 0) {
vm_page_swapqueue(m, PQ_ACTIVE,
PQ_INACTIVE);
page_shortage -=
act_scan_laundry_weight;
if (page_shortage <= 0) {
nqueue = PQ_INACTIVE;
ps_delta = 0;
} else if (m->dirty == 0) {
nqueue = PQ_INACTIVE;
ps_delta = act_scan_laundry_weight;
} else {
vm_page_swapqueue(m, PQ_ACTIVE,
PQ_LAUNDRY);
page_shortage--;
}
nqueue = PQ_LAUNDRY;
ps_delta = 1;
}
new.flags |= PGA_REQUEUE;
new.queue = nqueue;
}
} while (!vm_page_pqstate_commit(m, &old, new));
page_shortage -= ps_delta;
}
if (mtx != NULL) {
mtx_unlock(mtx);
@ -1376,22 +1411,18 @@ vm_pageout_scan_active(struct vm_domain *vmd, int page_shortage)
}
static int
vm_pageout_reinsert_inactive_page(struct scan_state *ss, vm_page_t m)
vm_pageout_reinsert_inactive_page(struct vm_pagequeue *pq, vm_page_t marker,
vm_page_t m)
{
struct vm_domain *vmd;
vm_page_astate_t as;
if (m->a.queue != PQ_INACTIVE || (m->a.flags & PGA_ENQUEUED) != 0)
vm_pagequeue_assert_locked(pq);
as = vm_page_astate_load(m);
if (as.queue != PQ_INACTIVE || (as.flags & PGA_ENQUEUED) != 0)
return (0);
vm_page_aflag_set(m, PGA_ENQUEUED);
if ((m->a.flags & PGA_REQUEUE_HEAD) != 0) {
vmd = vm_pagequeue_domain(m);
TAILQ_INSERT_BEFORE(&vmd->vmd_inacthead, m, plinks.q);
vm_page_aflag_clear(m, PGA_REQUEUE | PGA_REQUEUE_HEAD);
} else if ((m->a.flags & PGA_REQUEUE) != 0) {
TAILQ_INSERT_TAIL(&ss->pq->pq_pl, m, plinks.q);
vm_page_aflag_clear(m, PGA_REQUEUE | PGA_REQUEUE_HEAD);
} else
TAILQ_INSERT_BEFORE(ss->marker, m, plinks.q);
TAILQ_INSERT_BEFORE(marker, m, plinks.q);
return (1);
}
@ -1406,20 +1437,22 @@ vm_pageout_reinsert_inactive(struct scan_state *ss, struct vm_batchqueue *bq,
vm_page_t m)
{
struct vm_pagequeue *pq;
vm_page_t marker;
int delta;
delta = 0;
marker = ss->marker;
pq = ss->pq;
if (m != NULL) {
if (vm_batchqueue_insert(bq, m))
return;
vm_pagequeue_lock(pq);
delta += vm_pageout_reinsert_inactive_page(ss, m);
delta += vm_pageout_reinsert_inactive_page(pq, marker, m);
} else
vm_pagequeue_lock(pq);
while ((m = vm_batchqueue_pop(bq)) != NULL)
delta += vm_pageout_reinsert_inactive_page(ss, m);
delta += vm_pageout_reinsert_inactive_page(pq, marker, m);
vm_pagequeue_cnt_add(pq, delta);
vm_pagequeue_unlock(pq);
vm_batchqueue_init(bq);
@ -1439,7 +1472,8 @@ vm_pageout_scan_inactive(struct vm_domain *vmd, int shortage,
vm_page_t m, marker;
struct vm_pagequeue *pq;
vm_object_t object;
int act_delta, addl_page_shortage, deficit, page_shortage;
vm_page_astate_t old, new;
int act_delta, addl_page_shortage, deficit, page_shortage, refs;
int starting_page_shortage;
/*
@ -1486,19 +1520,10 @@ vm_pageout_scan_inactive(struct vm_domain *vmd, int shortage,
* careful whenever modifying page state. Once the object lock
* has been acquired, we have a stable reference to the page.
*/
if (vm_page_queue(m) != PQ_INACTIVE) {
addl_page_shortage++;
old = vm_page_astate_load(m);
if (old.queue != PQ_INACTIVE ||
(old.flags & PGA_QUEUE_STATE_MASK) != 0)
continue;
}
/*
* The page was re-enqueued after the page queue lock was
* dropped, or a requeue was requested. This page gets a second
* chance.
*/
if ((m->a.flags & (PGA_ENQUEUED | PGA_REQUEUE |
PGA_REQUEUE_HEAD)) != 0)
goto reinsert;
/*
* Wired pages may not be freed. Complete their removal
@ -1563,9 +1588,8 @@ vm_pageout_scan_inactive(struct vm_domain *vmd, int shortage,
* wire count is guaranteed not to increase.
*/
if (__predict_false(vm_page_wired(m))) {
vm_page_xunbusy(m);
vm_page_dequeue_deferred(m);
continue;
goto skip_page;
}
/*
@ -1575,46 +1599,54 @@ vm_pageout_scan_inactive(struct vm_domain *vmd, int shortage,
if (vm_page_none_valid(m))
goto free_page;
refs = object->ref_count != 0 ? pmap_ts_referenced(m) : 0;
for (old = vm_page_astate_load(m);;) {
/*
* If the page has been referenced and the object is not dead,
* reactivate or requeue the page depending on whether the
* object is mapped.
*
* Test PGA_REFERENCED after calling pmap_ts_referenced() so
* that a reference from a concurrently destroyed mapping is
* observed here and now.
* Check to see if the page has been removed from the
* queue since the first such check. Leave it alone if
* so, discarding any references collected by
* pmap_ts_referenced().
*/
if (object->ref_count != 0)
act_delta = pmap_ts_referenced(m);
else {
KASSERT(!pmap_page_is_mapped(m),
("page %p is mapped", m));
act_delta = 0;
}
if ((m->a.flags & PGA_REFERENCED) != 0) {
vm_page_aflag_clear(m, PGA_REFERENCED);
if (__predict_false(_vm_page_queue(old) == PQ_NONE))
goto skip_page;
new = old;
act_delta = refs;
if ((old.flags & PGA_REFERENCED) != 0) {
new.flags &= ~PGA_REFERENCED;
act_delta++;
}
if (act_delta != 0) {
if (object->ref_count != 0) {
vm_page_xunbusy(m);
VM_CNT_INC(v_reactivated);
vm_page_activate(m);
if (act_delta == 0) {
;
} else if (object->ref_count != 0) {
/*
* Increase the activation count if the page
* was referenced while in the inactive queue.
* This makes it less likely that the page will
* be returned prematurely to the inactive
* queue.
* Increase the activation count if the
* page was referenced while in the
* inactive queue. This makes it less
* likely that the page will be returned
* prematurely to the inactive queue.
*/
m->a.act_count += act_delta + ACT_ADVANCE;
new.act_count += ACT_ADVANCE +
act_delta;
if (new.act_count > ACT_MAX)
new.act_count = ACT_MAX;
new.flags |= PGA_REQUEUE;
new.queue = PQ_ACTIVE;
if (!vm_page_pqstate_commit(m, &old, new))
continue;
VM_CNT_INC(v_reactivated);
goto skip_page;
} else if ((object->flags & OBJ_DEAD) == 0) {
vm_page_xunbusy(m);
vm_page_aflag_set(m, PGA_REQUEUE);
goto reinsert;
new.queue = PQ_INACTIVE;
new.flags |= PGA_REQUEUE;
if (!vm_page_pqstate_commit(m, &old, new))
continue;
goto skip_page;
}
break;
}
/*
@ -1627,9 +1659,8 @@ vm_pageout_scan_inactive(struct vm_domain *vmd, int shortage,
if (object->ref_count != 0) {
vm_page_test_dirty(m);
if (m->dirty == 0 && !vm_page_try_remove_all(m)) {
vm_page_xunbusy(m);
vm_page_dequeue_deferred(m);
continue;
goto skip_page;
}
}
@ -1655,9 +1686,10 @@ vm_pageout_scan_inactive(struct vm_domain *vmd, int shortage,
page_shortage--;
continue;
}
vm_page_xunbusy(m);
if ((object->flags & OBJ_DEAD) == 0)
vm_page_launder(m);
skip_page:
vm_page_xunbusy(m);
continue;
reinsert:
vm_pageout_reinsert_inactive(&ss, &rq, m);