summaryrefslogtreecommitdiff
path: root/src/third_party/wiredtiger/src/evict
diff options
context:
space:
mode:
Diffstat (limited to 'src/third_party/wiredtiger/src/evict')
-rw-r--r--src/third_party/wiredtiger/src/evict/evict_file.c31
-rw-r--r--src/third_party/wiredtiger/src/evict/evict_lru.c218
-rw-r--r--src/third_party/wiredtiger/src/evict/evict_page.c119
3 files changed, 159 insertions, 209 deletions
diff --git a/src/third_party/wiredtiger/src/evict/evict_file.c b/src/third_party/wiredtiger/src/evict/evict_file.c
index 7f916ca4a1e..b8ec59372dc 100644
--- a/src/third_party/wiredtiger/src/evict/evict_file.c
+++ b/src/third_party/wiredtiger/src/evict/evict_file.c
@@ -49,24 +49,20 @@ __wt_evict_file(WT_SESSION_IMPL *session, WT_CACHE_OP syncop)
page = ref->page;
/*
- * Eviction can fail when a page in the evicted page's subtree
- * switches state. For example, if we don't evict a page marked
- * empty, because we expect it to be merged into its parent, it
- * might no longer be empty after it's reconciled, in which case
- * eviction of its parent would fail. We can either walk the
- * tree multiple times (until it's finally empty), or reconcile
- * each page to get it to its final state before considering if
- * it's an eviction target or will be merged into its parent.
+ * Eviction can fail when a page in the evicted page's subtree switches state. For example,
+ * if we don't evict a page marked empty, because we expect it to be merged into its parent,
+ * it might no longer be empty after it's reconciled, in which case eviction of its parent
+ * would fail. We can either walk the tree multiple times (until it's finally empty), or
+ * reconcile each page to get it to its final state before considering if it's an eviction
+ * target or will be merged into its parent.
*
- * Don't limit this test to any particular page type, that tends
- * to introduce bugs when the reconciliation of other page types
- * changes, and there's no advantage to doing so.
+ * Don't limit this test to any particular page type, that tends to introduce bugs when the
+ * reconciliation of other page types changes, and there's no advantage to doing so.
*
- * Eviction can also fail because an update cannot be written.
- * If sessions have disjoint sets of files open, updates in a
- * no-longer-referenced file may not yet be globally visible,
- * and the write will fail with EBUSY. Our caller handles that
- * error, retrying later.
+ * Eviction can also fail because an update cannot be written. If sessions have disjoint
+ * sets of files open, updates in a no-longer-referenced file may not yet be globally
+ * visible, and the write will fail with EBUSY. Our caller handles that error, retrying
+ * later.
*/
if (syncop == WT_SYNC_CLOSE && __wt_page_is_modified(page))
WT_ERR(__wt_reconcile(session, ref, NULL, WT_REC_EVICT | WT_REC_VISIBLE_ALL, NULL));
@@ -85,8 +81,7 @@ __wt_evict_file(WT_SESSION_IMPL *session, WT_CACHE_OP syncop)
/*
* Evict the page.
*
- * Ensure the ref state is restored to the previous
- * value if eviction fails.
+ * Ensure the ref state is restored to the previous value if eviction fails.
*/
WT_ERR(__wt_evict(session, ref, ref->state, WT_EVICT_CALL_CLOSING));
break;
diff --git a/src/third_party/wiredtiger/src/evict/evict_lru.c b/src/third_party/wiredtiger/src/evict/evict_lru.c
index c224a3b7b11..2f9f3220106 100644
--- a/src/third_party/wiredtiger/src/evict/evict_lru.c
+++ b/src/third_party/wiredtiger/src/evict/evict_lru.c
@@ -277,10 +277,12 @@ __wt_evict_thread_run(WT_SESSION_IMPL *session, WT_THREAD *thread)
conn = S2C(session);
cache = conn->cache;
- /*
- * The thread group code calls us repeatedly. So each call is one pass through eviction.
- */
- WT_TRACK_TIME(session);
+/*
+ * The thread group code calls us repeatedly. So each call is one pass through eviction.
+ */
+#ifdef HAVE_DIAGNOSTIC
+ __wt_seconds32(session, &session->op_5043_seconds);
+#endif
if (conn->evict_server_running && __wt_spin_trylock(session, &cache->evict_pass_lock) == 0) {
/*
* Cannot use WT_WITH_PASS_LOCK because this is a try lock. Fix when that is supported. We
@@ -426,15 +428,14 @@ __evict_server(WT_SESSION_IMPL *session, bool *did_work)
return (0);
#endif
/*
- * If we're stuck for 5 minutes in diagnostic mode, or the verbose
- * evict_stuck flag is configured, log the cache and transaction state.
+ * If we're stuck for 5 minutes in diagnostic mode, or the verbose evict_stuck flag is
+ * configured, log the cache and transaction state.
*
* If we're stuck for 5 minutes in diagnostic mode, give up.
*
- * We don't do this check for in-memory workloads because application
- * threads are not blocked by the cache being full. If the cache becomes
- * full of clean pages, we can be servicing reads while the cache
- * appears stuck to eviction.
+ * We don't do this check for in-memory workloads because application threads are not blocked by
+ * the cache being full. If the cache becomes full of clean pages, we can be servicing reads
+ * while the cache appears stuck to eviction.
*/
if (F_ISSET(conn, WT_CONN_IN_MEMORY))
return (0);
@@ -578,8 +579,7 @@ __evict_update_work(WT_SESSION_IMPL *session)
/*
* If we need space in the cache, try to find clean pages to evict.
*
- * Avoid division by zero if the cache size has not yet been set in a
- * shared cache.
+ * Avoid division by zero if the cache size has not yet been set in a shared cache.
*/
bytes_max = conn->cache_size + 1;
bytes_inuse = __wt_cache_bytes_inuse(cache);
@@ -679,14 +679,12 @@ __evict_pass(WT_SESSION_IMPL *session)
++cache->evict_pass_gen;
/*
- * Update the oldest ID: we use it to decide whether pages are
- * candidates for eviction. Without this, if all threads are
- * blocked after a long-running transaction (such as a
+ * Update the oldest ID: we use it to decide whether pages are candidates for eviction.
+ * Without this, if all threads are blocked after a long-running transaction (such as a
* checkpoint) completes, we may never start evicting again.
*
- * Do this every time the eviction server wakes up, regardless
- * of whether the cache is full, to prevent the oldest ID
- * falling too far behind. Don't wait to lock the table: with
+ * Do this every time the eviction server wakes up, regardless of whether the cache is full,
+ * to prevent the oldest ID falling too far behind. Don't wait to lock the table: with
* highly threaded workloads, that creates a bottleneck.
*/
WT_RET(__wt_txn_update_oldest(session, WT_TXN_OLDEST_STRICT));
@@ -702,14 +700,12 @@ __evict_pass(WT_SESSION_IMPL *session)
WT_RET(__evict_lru_walk(session));
/*
- * If the queue has been empty recently, keep queuing more
- * pages to evict. If the rate of queuing pages is high
- * enough, this score will go to zero, in which case the
- * eviction server might as well help out with eviction.
+ * If the queue has been empty recently, keep queuing more pages to evict. If the rate of
+ * queuing pages is high enough, this score will go to zero, in which case the eviction
+ * server might as well help out with eviction.
*
- * Also, if there is a single eviction server thread with no
- * workers, it must service the urgent queue in case all
- * application threads are busy.
+ * Also, if there is a single eviction server thread with no workers, it must service the
+ * urgent queue in case all application threads are busy.
*/
if (!WT_EVICT_HAS_WORKERS(session) &&
(cache->evict_empty_score < WT_EVICT_SCORE_CUTOFF ||
@@ -720,16 +716,13 @@ __evict_pass(WT_SESSION_IMPL *session)
break;
/*
- * If we're making progress, keep going; if we're not making
- * any progress at all, mark the cache "stuck" and go back to
- * sleep, it's not something we can fix.
+ * If we're making progress, keep going; if we're not making any progress at all, mark the
+ * cache "stuck" and go back to sleep, it's not something we can fix.
*
- * We check for progress every 20ms, the idea being that the
- * aggressive score will reach 10 after 200ms if we aren't
- * making progress and eviction will start considering more
- * pages. If there is still no progress after 2s, we will
- * treat the cache as stuck and start rolling back
- * transactions and writing updates to the lookaside table.
+ * We check for progress every 20ms, the idea being that the aggressive score will reach 10
+ * after 200ms if we aren't making progress and eviction will start considering more pages.
+ * If there is still no progress after 2s, we will treat the cache as stuck and start
+ * rolling back transactions and writing updates to the lookaside table.
*/
if (eviction_progress == cache->eviction_progress) {
if (WT_CLOCKDIFF_MS(time_now, time_prev) >= 20 &&
@@ -750,14 +743,11 @@ __evict_pass(WT_SESSION_IMPL *session)
*/
if (loop < 100 || cache->evict_aggressive_score < 100) {
/*
- * Back off if we aren't making progress: walks
- * hold the handle list lock, blocking other
- * operations that can free space in cache,
- * such as LSM discarding handles.
+ * Back off if we aren't making progress: walks hold the handle list lock, blocking
+ * other operations that can free space in cache, such as LSM discarding handles.
*
- * Allow this wait to be interrupted (e.g. if a
- * checkpoint completes): make sure we wait for
- * a non-zero number of microseconds).
+ * Allow this wait to be interrupted (e.g. if a checkpoint completes): make sure we
+ * wait for a non-zero number of microseconds).
*/
WT_STAT_CONN_INCR(session, cache_eviction_server_slept);
__wt_cond_wait(session, cache->evict_cond, WT_THOUSAND, NULL);
@@ -1181,8 +1171,8 @@ __evict_lru_walk(WT_SESSION_IMPL *session)
/*
* Get some more pages to consider for eviction.
*
- * If the walk is interrupted, we still need to sort the queue: the
- * next walk assumes there are no entries beyond WT_EVICT_WALK_BASE.
+ * If the walk is interrupted, we still need to sort the queue: the next walk assumes there are
+ * no entries beyond WT_EVICT_WALK_BASE.
*/
if ((ret = __evict_walk(cache->walk_session, queue)) == EBUSY)
ret = 0;
@@ -1264,15 +1254,12 @@ __evict_lru_walk(WT_SESSION_IMPL *session)
queue->evict_candidates = candidates;
else {
/*
- * Take all of the urgent pages plus a third of
- * ordinary candidates (which could be expressed as
- * WT_EVICT_WALK_INCR / WT_EVICT_WALK_BASE). In the
- * steady state, we want to get as many candidates as
- * the eviction walk adds to the queue.
+ * Take all of the urgent pages plus a third of ordinary candidates (which could be
+ * expressed as WT_EVICT_WALK_INCR / WT_EVICT_WALK_BASE). In the steady state, we want
+ * to get as many candidates as the eviction walk adds to the queue.
*
- * That said, if there is only one entry, which is
- * normal when populating an empty file, don't exclude
- * it.
+ * That said, if there is only one entry, which is normal when populating an empty file,
+ * don't exclude it.
*/
queue->evict_candidates = 1 + candidates + ((entries - candidates) - 1) / 3;
cache->read_gen_oldest = read_gen_oldest;
@@ -1468,11 +1455,9 @@ retry:
/*
* Skip files if we have too many active walks.
*
- * This used to be limited by the configured maximum number of
- * hazard pointers per session. Even though that ceiling has
- * been removed, we need to test eviction with huge numbers of
- * active trees before allowing larger numbers of hazard
- * pointers in the walk session.
+ * This used to be limited by the configured maximum number of hazard pointers per session.
+ * Even though that ceiling has been removed, we need to test eviction with huge numbers of
+ * active trees before allowing larger numbers of hazard pointers in the walk session.
*/
if (btree->evict_ref == NULL && session->nhazard > WT_EVICT_MAX_TREES)
continue;
@@ -1490,16 +1475,14 @@ retry:
dhandle_locked = false;
/*
- * Re-check the "no eviction" flag, used to enforce exclusive
- * access when a handle is being closed.
+ * Re-check the "no eviction" flag, used to enforce exclusive access when a handle is being
+ * closed.
*
- * Only try to acquire the lock and simply continue if we fail;
- * the lock is held while the thread turning off eviction clears
- * the tree's current eviction point, and part of the process is
- * waiting on this thread to acknowledge that action.
+ * Only try to acquire the lock and simply continue if we fail; the lock is held while the
+ * thread turning off eviction clears the tree's current eviction point, and part of the
+ * process is waiting on this thread to acknowledge that action.
*
- * If a handle is being discarded, it will still be marked open,
- * but won't have a root page.
+ * If a handle is being discarded, it will still be marked open, but won't have a root page.
*/
if (btree->evict_disabled == 0 && !__wt_spin_trylock(session, &cache->evict_walk_lock)) {
if (btree->evict_disabled == 0 && btree->root.page != NULL) {
@@ -1888,9 +1871,8 @@ __evict_walk_tree(WT_SESSION_IMPL *session, WT_EVICT_QUEUE *queue, u_int max_ent
/*
* Pages that are empty or from dead trees are fast-tracked.
*
- * Also evict lookaside table pages without further filtering:
- * the cache is under pressure by definition and we want to
- * free space.
+ * Also evict lookaside table pages without further filtering: the cache is under pressure
+ * by definition and we want to free space.
*/
if (__wt_page_is_empty(page) || F_ISSET(session->dhandle, WT_DHANDLE_DEAD) ||
F_ISSET(btree, WT_BTREE_LOOKASIDE))
@@ -1920,15 +1902,12 @@ __evict_walk_tree(WT_SESSION_IMPL *session, WT_EVICT_QUEUE *queue, u_int max_ent
continue;
/*
- * Don't attempt eviction of internal pages with children in
- * cache (indicated by seeing an internal page that is the
- * parent of the last page we saw).
+ * Don't attempt eviction of internal pages with children in cache (indicated by seeing an
+ * internal page that is the parent of the last page we saw).
*
- * Also skip internal page unless we get aggressive, the tree
- * is idle (indicated by the tree being skipped for walks),
- * or we are in eviction debug mode.
- * The goal here is that if trees become completely idle, we
- * eventually push them out of cache completely.
+ * Also skip internal page unless we get aggressive, the tree is idle (indicated by the tree
+ * being skipped for walks), or we are in eviction debug mode. The goal here is that if
+ * trees become completely idle, we eventually push them out of cache completely.
*/
if (!F_ISSET(cache, WT_CACHE_EVICT_DEBUG_MODE) && WT_PAGE_IS_INTERNAL(page)) {
if (page == last_parent)
@@ -1987,18 +1966,15 @@ fast:
/*
* Give up the walk occasionally.
*
- * If we happen to end up on the root page or a page requiring urgent
- * eviction, clear it. We have to track hazard pointers, and the root
- * page complicates that calculation.
+ * If we happen to end up on the root page or a page requiring urgent eviction, clear it. We
+ * have to track hazard pointers, and the root page complicates that calculation.
*
- * Likewise if we found no new candidates during the walk: there is no
- * point keeping a page pinned, since it may be the only candidate in
- * an idle tree.
+ * Likewise if we found no new candidates during the walk: there is no point keeping a page
+ * pinned, since it may be the only candidate in an idle tree.
*
- * If we land on a page requiring forced eviction, or that isn't an
- * ordinary in-memory page (e.g., WT_REF_LIMBO), move until we find an
- * ordinary page: we should not prevent exclusive access to the page
- * until the next walk.
+ * If we land on a page requiring forced eviction, or that isn't an ordinary in-memory page
+ * (e.g., WT_REF_LIMBO), move until we find an ordinary page: we should not prevent exclusive
+ * access to the page until the next walk.
*/
if (ref != NULL) {
if (__wt_ref_is_root(ref) || evict == start || give_up ||
@@ -2064,13 +2040,12 @@ __evict_get_ref(WT_SESSION_IMPL *session, bool is_server, WT_BTREE **btreep, WT_
}
/*
- * The server repopulates whenever the other queue is not full, as long
- * as at least one page has been evicted out of the current queue.
+ * The server repopulates whenever the other queue is not full, as long as at least one page has
+ * been evicted out of the current queue.
*
- * Note that there are pathological cases where there are only enough
- * eviction candidates in the cache to fill one queue. In that case,
- * we will continually evict one page and attempt to refill the queues.
- * Such cases are extremely rare in real applications.
+ * Note that there are pathological cases where there are only enough eviction candidates in the
+ * cache to fill one queue. In that case, we will continually evict one page and attempt to
+ * refill the queues. Such cases are extremely rare in real applications.
*/
if (is_server && (!urgent_ok || __evict_queue_empty(urgent_queue, false)) &&
!__evict_queue_full(cache->evict_current_queue) &&
@@ -2088,9 +2063,8 @@ __evict_get_ref(WT_SESSION_IMPL *session, bool is_server, WT_BTREE **btreep, WT_
/*
* Check if the current queue needs to change.
*
- * The server will only evict half of the pages before looking
- * for more, but should only switch queues if there are no
- * other eviction workers.
+ * The server will only evict half of the pages before looking for more, but should only
+ * switch queues if there are no other eviction workers.
*/
queue = cache->evict_current_queue;
other_queue = cache->evict_other_queue;
@@ -2136,14 +2110,13 @@ __evict_get_ref(WT_SESSION_IMPL *session, bool is_server, WT_BTREE **btreep, WT_
WT_ASSERT(session, evict->btree != NULL);
/*
- * Evicting a dirty page in the server thread could stall
- * during a write and prevent eviction from finding new work.
+ * Evicting a dirty page in the server thread could stall during a write and prevent
+ * eviction from finding new work.
*
- * However, we can't skip entries in the urgent queue or they
- * may never be found again.
+ * However, we can't skip entries in the urgent queue or they may never be found again.
*
- * Don't force application threads to evict dirty pages if they
- * aren't stalled by the amount of dirty data in cache.
+ * Don't force application threads to evict dirty pages if they aren't stalled by the amount
+ * of dirty data in cache.
*/
if (!urgent_ok && (is_server || !F_ISSET(cache, WT_CACHE_EVICT_DIRTY_HARD)) &&
__wt_page_is_modified(evict->ref->page)) {
@@ -2233,13 +2206,11 @@ __evict_page(WT_SESSION_IMPL *session, bool is_server)
}
/*
- * In case something goes wrong, don't pick the same set of pages every
- * time.
+ * In case something goes wrong, don't pick the same set of pages every time.
*
- * We used to bump the page's read generation only if eviction failed,
- * but that isn't safe: at that point, eviction has already unlocked
- * the page and some other thread may have evicted it by the time we
- * look at it.
+ * We used to bump the page's read generation only if eviction failed, but that isn't safe: at
+ * that point, eviction has already unlocked the page and some other thread may have evicted it
+ * by the time we look at it.
*/
__wt_cache_read_gen_bump(session, ref->page);
@@ -2295,31 +2266,32 @@ __wt_cache_eviction_worker(WT_SESSION_IMPL *session, bool busy, bool readonly, d
if (timer)
time_start = __wt_clock(session);
- WT_TRACK_TIME(session);
+#ifdef HAVE_DIAGNOSTIC
+ __wt_seconds32(session, &session->op_5043_seconds);
+#endif
for (initial_progress = cache->eviction_progress;; ret = 0) {
/*
- * A pathological case: if we're the oldest transaction in the
- * system and the eviction server is stuck trying to find space
- * (and we're not in recovery, because those transactions can't
- * be rolled back), abort the transaction to give up all hazard
- * pointers before trying again.
+ * If eviction is stuck, check if this thread is likely causing problems and should be
+ * rolled back. Ignore if in recovery, those transactions can't be rolled back.
*/
- if (__wt_cache_stuck(session) && __wt_txn_am_oldest(session) &&
- !F_ISSET(conn, WT_CONN_RECOVERING)) {
- --cache->evict_aggressive_score;
- WT_STAT_CONN_INCR(session, txn_fail_cache);
- WT_ERR(
- __wt_txn_rollback_required(session, "oldest transaction rolled back for eviction"));
+ if (!F_ISSET(conn, WT_CONN_RECOVERING) && __wt_cache_stuck(session)) {
+ ret = __wt_txn_is_blocking_old(session);
+ if (ret == 0)
+ ret = __wt_txn_is_blocking_pin(session);
+ if (ret == WT_ROLLBACK) {
+ --cache->evict_aggressive_score;
+ WT_STAT_CONN_INCR(session, txn_fail_cache);
+ }
+ WT_ERR(ret);
}
/*
* Check if we have become busy.
*
- * If we're busy (because of the transaction check we just did
- * or because our caller is waiting on a longer-than-usual event
- * such as a page read), and the cache level drops below 100%,
- * limit the work to 5 evictions and return. If that's not the
- * case, we can do more.
+ * If we're busy (because of the transaction check we just did or because our caller is
+ * waiting on a longer-than-usual event such as a page read), and the cache level drops
+ * below 100%, limit the work to 5 evictions and return. If that's not the case, we can do
+ * more.
*/
if (!busy && txn_state->pinned_id != WT_TXN_NONE &&
txn_global->current != txn_global->oldest_id)
diff --git a/src/third_party/wiredtiger/src/evict/evict_page.c b/src/third_party/wiredtiger/src/evict/evict_page.c
index 41ecfb40242..a13526302a2 100644
--- a/src/third_party/wiredtiger/src/evict/evict_page.c
+++ b/src/third_party/wiredtiger/src/evict/evict_page.c
@@ -260,14 +260,12 @@ __evict_delete_ref(WT_SESSION_IMPL *session, WT_REF *ref, uint32_t flags)
ndeleted = __wt_atomic_addv32(&pindex->deleted_entries, 1);
/*
- * If more than 10% of the parent references are deleted, try a
- * reverse split. Don't bother if there is a single deleted
- * reference: the internal page is empty and we have to wait
+ * If more than 10% of the parent references are deleted, try a reverse split. Don't bother
+ * if there is a single deleted reference: the internal page is empty and we have to wait
* for eviction to notice.
*
- * This will consume the deleted ref (and eventually free it).
- * If the reverse split can't get the access it needs because
- * something is busy, be sure that the page still ends up
+ * This will consume the deleted ref (and eventually free it). If the reverse split can't
+ * get the access it needs because something is busy, be sure that the page still ends up
* marked deleted.
*/
if (ndeleted > pindex->entries / 10 && pindex->entries > 1) {
@@ -361,20 +359,19 @@ __evict_page_dirty_update(WT_SESSION_IMPL *session, WT_REF *ref, uint32_t evict_
break;
case WT_PM_REC_MULTIBLOCK: /* Multiple blocks */
/*
- * Either a split where we reconciled a page and it turned into
- * a lot of pages or an in-memory page that got too large, we
- * forcibly evicted it, and there wasn't anything to write.
+ * Either a split where we reconciled a page and it turned into a lot
+ * of pages or an in-memory page that got too large, we forcibly
+ * evicted it, and there wasn't anything to write.
*
- * The latter is a special case of forced eviction. Imagine a
- * thread updating a small set keys on a leaf page. The page
- * is too large or has too many deleted items, so we try and
- * evict it, but after reconciliation there's only a small
- * amount of live data (so it's a single page we can't split),
- * and if there's an older reader somewhere, there's data on
- * the page we can't write (so the page can't be evicted). In
- * that case, we end up here with a single block that we can't
- * write. Take advantage of the fact we have exclusive access
- * to the page and rewrite it in memory.
+ * The latter is a special case of forced eviction. Imagine a thread
+ * updating a small set keys on a leaf page. The page is too large or
+ * has too many deleted items, so we try and evict it, but after
+ * reconciliation there's only a small amount of live data (so it's a
+ * single page we can't split), and if there's an older reader
+ * somewhere, there's data on the page we can't write (so the page
+ * can't be evicted). In that case, we end up here with a single
+ * block that we can't write. Take advantage of the fact we have
+ * exclusive access to the page and rewrite it in memory.
*/
if (mod->mod_multi_entries == 1) {
WT_ASSERT(session, closing == false);
@@ -386,11 +383,11 @@ __evict_page_dirty_update(WT_SESSION_IMPL *session, WT_REF *ref, uint32_t evict_
/*
* Update the parent to reference the replacement page.
*
- * A page evicted with lookaside entries may not have an
- * address, if no updates were visible to reconciliation.
+ * A page evicted with lookaside entries may not have an address, if no
+ * updates were visible to reconciliation.
*
- * Publish: a barrier to ensure the structure fields are set
- * before the state change makes the page available to readers.
+ * Publish: a barrier to ensure the structure fields are set before the
+ * state change makes the page available to readers.
*/
if (mod->mod_replace.addr != NULL) {
WT_RET(__wt_calloc_one(session, &addr));
@@ -486,19 +483,16 @@ __evict_child_check(WT_SESSION_IMPL *session, WT_REF *parent)
break;
case WT_REF_DELETED: /* On-disk, deleted */
/*
- * If the child page was part of a truncate,
- * transaction rollback might switch this page into its
- * previous state at any time, so the delete must be
- * resolved before the parent can be evicted.
+ * If the child page was part of a truncate, transaction rollback might
+ * switch this page into its previous state at any time, so the delete
+ * must be resolved before the parent can be evicted.
*
- * We have the internal page locked, which prevents a
- * search from descending into it. However, a walk
- * from an adjacent leaf page could attempt to hazard
- * couple into a child page and free the page_del
- * structure as we are examining it. Flip the state to
- * locked to make this check safe: if that fails, we
- * have raced with a read and should give up on
- * evicting the parent.
+ * We have the internal page locked, which prevents a search from
+ * descending into it. However, a walk from an adjacent leaf page could
+ * attempt to hazard couple into a child page and free the page_del
+ * structure as we are examining it. Flip the state to locked to make
+ * this check safe: if that fails, we have raced with a read and should
+ * give up on evicting the parent.
*/
if (!__wt_atomic_casv32(&child->state, WT_REF_DELETED, WT_REF_LOCKED))
return (__wt_set_return(session, EBUSY));
@@ -613,32 +607,29 @@ __evict_review(WT_SESSION_IMPL *session, WT_REF *ref, uint32_t evict_flags, bool
/*
* If the page is dirty, reconcile it to decide if we can evict it.
*
- * If we have an exclusive lock (we're discarding the tree), assert
- * there are no updates we cannot read.
+ * If we have an exclusive lock (we're discarding the tree), assert there are no updates we
+ * cannot read.
*
- * Don't set any other flags for internal pages: there are no update
- * lists to be saved and restored, changes can't be written into the
- * lookaside table, nor can we re-create internal pages in memory.
+ * Don't set any other flags for internal pages: there are no update lists to be saved and
+ * restored, changes can't be written into the lookaside table, nor can we re-create internal
+ * pages in memory.
*
* For leaf pages:
*
* In-memory pages are a known configuration.
*
- * Set the update/restore flag, so reconciliation will write blocks it
- * can write and create a list of skipped updates for blocks it cannot
- * write, along with disk images. This is how eviction of active, huge
- * pages works: we take a big page and reconcile it into blocks, some of
- * which we write and discard, the rest of which we re-create as smaller
- * in-memory pages, (restoring the updates that stopped us from writing
- * the block), and inserting the whole mess into the page's parent. Set
- * the flag in all cases because the incremental cost of update/restore
- * in reconciliation is minimal, eviction shouldn't have picked a page
- * where update/restore is necessary, absent some cache pressure. It's
- * possible updates occurred after we selected this page for eviction,
- * but it's unlikely and we don't try and manage that risk.
+ * Set the update/restore flag, so reconciliation will write blocks it can write and create a
+ * list of skipped updates for blocks it cannot write, along with disk images. This is how
+ * eviction of active, huge pages works: we take a big page and reconcile it into blocks, some
+ * of which we write and discard, the rest of which we re-create as smaller in-memory pages,
+ * (restoring the updates that stopped us from writing the block), and inserting the whole mess
+ * into the page's parent. Set the flag in all cases because the incremental cost of
+ * update/restore in reconciliation is minimal, eviction shouldn't have picked a page where
+ * update/restore is necessary, absent some cache pressure. It's possible updates occurred after
+ * we selected this page for eviction, but it's unlikely and we don't try and manage that risk.
*
- * Additionally, if we aren't trying to free space in the cache, scrub
- * the page and keep it in memory.
+ * Additionally, if we aren't trying to free space in the cache, scrub the page and keep it in
+ * memory.
*/
cache = conn->cache;
lookaside_retry = false;
@@ -678,15 +669,8 @@ __evict_review(WT_SESSION_IMPL *session, WT_REF *ref, uint32_t evict_flags, bool
/* Reconcile the page. */
ret = __wt_reconcile(session, ref, NULL, flags, lookaside_retryp);
-
- /*
- * If attempting eviction during a checkpoint, we may successfully reconcile but then find that
- * there are updates on the page too new to evict. Give up evicting in that case: checkpoint
- * will include the reconciled page when it visits the parent.
- */
- if (WT_SESSION_BTREE_SYNC(session) && !__wt_page_is_modified(page) &&
- !__wt_txn_visible_all(session, page->modify->rec_max_txn, page->modify->rec_max_timestamp))
- return (__wt_set_return(session, EBUSY));
+ WT_ASSERT(session, __wt_page_is_modified(page) ||
+ __wt_txn_visible_all(session, page->modify->rec_max_txn, page->modify->rec_max_timestamp));
/*
* If reconciliation fails but reports it might succeed if we use the lookaside table, try again
@@ -704,11 +688,10 @@ __evict_review(WT_SESSION_IMPL *session, WT_REF *ref, uint32_t evict_flags, bool
/*
* Give up on eviction during a checkpoint if the page splits.
*
- * We get here if checkpoint reads a page with lookaside entries: if
- * more of those entries are visible now than when the original
- * eviction happened, the page could split. In most workloads, this is
- * very unlikely. However, since checkpoint is partway through
- * reconciling the parent page, a split can corrupt the checkpoint.
+ * We get here if checkpoint reads a page with lookaside entries: if more of those entries are
+ * visible now than when the original eviction happened, the page could split. In most
+ * workloads, this is very unlikely. However, since checkpoint is partway through reconciling
+ * the parent page, a split can corrupt the checkpoint.
*/
if (WT_SESSION_BTREE_SYNC(session) && page->modify->rec_result == WT_PM_REC_MULTIBLOCK)
return (__wt_set_return(session, EBUSY));