SERVER-96647 Move Storage Execution README sections (#29547)

GitOrigin-RevId: 1c38fd52868c064c83703df42ebe3cdc1e9186f5
This commit is contained in:
Louis Williams 2024-11-26 14:39:49 -05:00 committed by MongoDB Bot
parent b93d608d2a
commit 7d21878ae5
11 changed files with 1811 additions and 584 deletions

View File

@ -3,18 +3,45 @@
"language": "en",
"words": [
"absl",
"AGCFR",
"bson",
"bsonobj",
"bufbuilder",
"checkpointing",
"Decorable",
"deprioritized",
"deprioritizing",
"directoryperdb",
"fromjson",
"FTDC",
"fullfillable",
"idents",
"journaled",
"journaling",
"logv",
"multikey",
"MVCC",
"oplog",
"preimages",
"quiesces",
"RAII",
"reprioritization",
"reprioritized",
"resumability",
"RSTL",
"Seekable",
"semifuture",
"sizeStorer",
"stdx",
"subarray",
"subobj",
"subobject",
"tailable"
"tailable",
"transactionally",
"unreplicated",
"untimestamped",
"wiredtiger",
"WUOW"
],
"ignoreRegExpList": [
"\\w*assert(ed)?"

View File

@ -1,14 +1,50 @@
# Global Lock Admission Control
There are 2 separate ticketing mechanisms placed in front of the global lock acquisition. Both aim to limit the number of concurrent operations from overwhelming the system. Before an operation can acquire the global lock, it must acquire a ticket from one, or both, of the ticketing mechanisms. When both ticket mechanisms are necessary, the acquisition order is as follows:
1. [Flow Control][] - Required only for global lock requests in MODE_IX
2. [Execution Control] - Required for all global lock requests
[Execution Control]: #execution-control
[Flow Control]: ../repl/README.md#flow-control
# Execution Control
Execution control limits the number of concurrent storage engine transactions in a single mongod to
reduce contention on storage engine resources.
## Ticket Management
There are 2 separate pools of available tickets: one pool for global lock read requests
(MODE_S/MODE_IS), and one pool of tickets for global lock write requests (MODE_IX).
As of v7.0, the size of each ticket pool is managed dynamically by the server to maximize
throughput. Details of the algorithm can be found in [Throughput Probing](#throughput-probing) This
dynamic management can be disabled by specifying the size of each pool manually via server
parameters `storageEngineConcurrentReadTransactions` (read ticket pool) and
`storageEngineConcurrentWriteTransactions` (write ticket pool). #
Each pool of tickets is maintained in a
[TicketHolder](https://github.com/mongodb/mongo/blob/r6.3.0-rc0/src/mongo/util/concurrency/ticketholder.h#L52).
Tickets distributed from a given TicketHolder will always be returned to the same TicketHolder (a
write ticket will always be returned to the TicketHolder with the write ticket pool).
## Throughput Probing
Execution control limits concurrency with a throughput-probing algorithm, described below.
### Server Parameters
- `throughputProbingInitialConcurrency -> gInitialConcurrency`: initial number of concurrent read and write transactions
- `throughputProbingMinConcurrency -> gMinConcurrency`: minimum concurrent read and write transactions
- `throughputProbingMaxConcurrency -> gMaxConcurrency`: maximum concurrenct read and write transactions
- `throughputProbingReadWriteRatio -> gReadWriteRatio`: ratio of read and write tickets where 0.5 indicates 1:1 ratio
- `throughputProbingConcurrencyMovingAverageWeight -> gConcurrencyMovingAverageWeight`: weight of new concurrency measurement in the exponentially-decaying moving average
- `throughputProbingInitialConcurrency -> gInitialConcurrency`: initial number of concurrent read
and write transactions
- `throughputProbingMinConcurrency -> gMinConcurrency`: minimum concurrent read and write
transactions
- `throughputProbingMaxConcurrency -> gMaxConcurrency`: maximum concurrent read and write
transactions
- `throughputProbingReadWriteRatio -> gReadWriteRatio`: ratio of read and write tickets where 0.5
indicates 1:1 ratio
- `throughputProbingConcurrencyMovingAverageWeight -> gConcurrencyMovingAverageWeight`: weight of
new concurrency measurement in the exponentially-decaying moving average
- `throughputProbingStepMultiple -> gStepMultiple`: step size for throughput probing
### Pseudocode
@ -82,43 +118,157 @@ D --> H(Stable Probe)
E --> H
```
## Admission Priority
Associated with every operation is an admission priority, stored as a part of the
[AdmissionContext](https://github.com/mongodb/mongo/blob/r6.3.0-rc0/src/mongo/util/concurrency/admission_context.h#L40).
By default, operations are 'normal' priority.
In the Flow Control ticketing system, operations of 'immediate' priority bypass ticket acquisition
regardless of ticket availability. Tickets that are not 'immediate' priority must throttle when
there are no tickets available in both Flow Control and Execution Control.
Flow Control is only concerned whether an operation is 'immediate' priority and does not
differentiate between 'normal' and 'low' priorities. The current version of Execution Control relies
on admission priority to administer tickets when the server is under load.
**AdmissionContext::Priority**
- `kExempt` - Reserved for operations critical to availability (e.g replication workers), or
observability (e.g. FTDC), and any operation releasing resources (e.g. committing or aborting
prepared transactions).
- `kNormal` - An operation that should be throttled when the server is under load. If an operation
is throttled, it will not affect availability or observability. Most operations, both user and
internal, should use this priority unless they qualify as 'kLow' or 'kExempt' priority.
- `kLow` - Reserved for background tasks that have no other operations dependent on them. The
operation will be throttled under load and make significantly less progress compared to operations
of higher priorities in the Execution Control.
[See AdmissionContext::Priority for more
details](https://github.com/mongodb/mongo/blob/r7.0.0-rc0/src/mongo/util/concurrency/admission_context.h#L45-L67).
### How to Set Admission Priority
The preferred method for setting an operation's priority is through the RAII type
[ScopedAdmissionPriorityForLock](https://github.com/mongodb/mongo/blob/r7.0.0-rc0/src/mongo/db/concurrency/locker.h#L747).
```
ScopedAdmissionPriorityForLock priority(shard_role_details::getLocker(opCtx), AdmissionContext::Priority::kLow);
```
Since the GlobalLock may be acquired and released multiple times throughout an operation's lifetime,
it's important to limit the scope of reprioritization to prevent unintentional side-effects.
However, if there is a special circumstance where the RAII cannot possibly be used, the priority can
be set directly through
[Locker::setAdmissionPriority()](https://github.com/mongodb/mongo/blob/r7.0.0-rc0/src/mongo/db/concurrency/locker.h#L525).
### Developer Guidelines for Declaring Low Admission Priority
Developers must evaluate the consequences of each low priority operation from falling too far
behind, and should try to implement safeguards to avoid any undesirable behaviors for excessive
delays in low priority operations.
Whenever possible, an operation should dynamically choose when to be deprioritized or
re-prioritized. More specifically, all low-priority candidates should assess the impact of
deprioritizing their operation with respect to the state of the system before taking the GlobalLock
with low priority.
For example, since TTL deletes can be an expensive background task, they should default to low
priority. However, it's important they don't fall too far behind TTL inserts - otherwise, there is a
risk of unbounded collection growth. To remedy this issue, TTL deletes on a collection [are
reprioritized](https://github.com/mongodb/mongo/blob/d1a0e34e1e67d4a2b23104af2512d14290b25e5f/src/mongo/db/ttl.idl#L96)
to normal priority if they can't catch up after n-sub-passes.
Examples of Deprioritized Operations:
- [TTL
deletes](https://github.com/mongodb/mongo/blob/0ceb784512f81f77f0bc55001f83ca77d1aa1d84/src/mongo/db/ttl.cpp#L488)
- [Persisting sampled queries for analyze shard
key](https://github.com/mongodb/mongo/blob/0ef2c68f58ea20c2dde99e5ce3ea10b79e18453d/src/mongo/db/commands/write_commands.cpp#L295)
- [Unbounded Index
Scans](https://github.com/mongodb/mongo/blob/0ef2c68f58ea20c2dde99e5ce3ea10b79e18453d/src/mongo/db/query/planner_access.cpp#L1913)
- [Unbounded Collection
Scans](https://github.com/mongodb/mongo/blob/0ef2c68f58ea20c2dde99e5ce3ea10b79e18453d/src/mongo/db/query/planner_analysis.cpp#L1254)
- Index Builds
[(1)](https://github.com/mongodb/mongo/blob/0ef2c68f58ea20c2dde99e5ce3ea10b79e18453d/src/mongo/db/index_builds_coordinator.cpp#L3064),
[(2)](https://github.com/mongodb/mongo/blob/0ef2c68f58ea20c2dde99e5ce3ea10b79e18453d/src/mongo/db/index_builds_coordinator.cpp#L3105)
# Data-Node Ingress Admission Control
### Quick Overview
Ingress Admission Control is the mechanism placed at the data node ingress layer to help prevent data-bearing nodes from becoming overloaded with operations. This is done through a ticketing system that is intended to queue incoming operations based on a configurable overload prevention policy. Simply put, the queue can admit incoming user operations up to the max number of tickets configured, and any additional operations wait until a ticket is freed up. Ingress Admission Control is not applied to all commands however. It's enforced to most user admitted operations, while high priority and critical internal operations are typically exempt. While the number of tickets is defaulted to [1,000,000][ingressACidl], it is configurable at startup and runtime.
Ingress Admission Control is the mechanism placed at the data node ingress layer to help prevent
data-bearing nodes from becoming overloaded with operations. This is done through a ticketing system
that is intended to queue incoming operations based on a configurable overload prevention policy.
Simply put, the queue can admit incoming user operations up to the max number of tickets configured,
and any additional operations wait until a ticket is freed up. Ingress Admission Control is not
applied to all commands however. It's enforced to most user admitted operations, while high priority
and critical internal operations are typically exempt. While the number of tickets is defaulted to
[1,000,000][ingressACidl], it is configurable at startup and runtime.
## Code Structure and Components
Ingress admission control can be broken down into just a few parts:
- **IngressAdmissionsContext**: A decoration on the operation context, which inherits from `AdmissionContext`. This base class provides metadata and priority, which is used when determining if a command is subject to admission control.
- **IngressAdmissionsContext**: A decoration on the operation context, which inherits from
`AdmissionContext`. This base class provides metadata and priority, which is used when determining
if a command is subject to admission control.
- **ScopedAdmissionPriority**: An RAII-style class that sets the admission priority for an operation.
- **ScopedAdmissionPriority**: An RAII-style class that sets the admission priority for an
operation.
- **IngressAdmissionController**: A decoration on the service context that manages ticket pool size, and admission of operations. To be able to utilize these mechanisms, `IngressAdmissionController` owns a `TicketHolder`, which is capable of acquiring tickets for operations, and resizing the ticket pool.
- **IngressAdmissionController**: A decoration on the service context that manages ticket pool size,
and admission of operations. To be able to utilize these mechanisms, `IngressAdmissionController`
owns a `TicketHolder`, which is capable of acquiring tickets for operations, and resizing the
ticket pool.
## Admission Control Ticket Acquisition
The full scope of Admission Control happens inside of [`ExecCommandDatabase::_initiateCommand()`][initiateCommand] within the ServiceEntryPoint.
The full scope of Admission Control happens inside of
[`ExecCommandDatabase::_initiateCommand()`][initiateCommand] within the ServiceEntryPoint.
To begin, the server parameter [`gIngressAdmissionControlEnabled`][admissionServerParam] is checked to see if admission control is enabled. If true, we continue with admission control evaluation.
To begin, the server parameter [`gIngressAdmissionControlEnabled`][admissionServerParam] is checked
to see if admission control is enabled. If true, we continue with admission control evaluation.
Next we check the main trigger for admission control evaluation, `isSubjectToIngressAdmissionControl()`. Commands will initially be exempt from admission control as the default `isSubjectToIngressAdmissionControl` is set to return false. However, each command invocation can have a different override of `isSubjectToIngressAdmissionControl`, depending on if it should be subject to admission control. Since each operation has their own implementation, there is no one collective that determines if an operation needs to be evaluated, so this is left up to each command's own implementation.
Next we check the main trigger for admission control evaluation,
`isSubjectToIngressAdmissionControl()`. Commands will initially be exempt from admission control as
the default `isSubjectToIngressAdmissionControl` is set to return false. However, each command
invocation can have a different override of `isSubjectToIngressAdmissionControl`, depending on if it
should be subject to admission control. Since each operation has their own implementation, there is
no one collective that determines if an operation needs to be evaluated, so this is left up to each
command's own implementation.
Each operation will attempt to acquire a ticket unless an operation is marked **exempt**, or if the operation is already holding a ticket. Exempt tickets typically are held by high priority and critical internal operations. Meanwhile, re-entrancy API's like `DBDirectClient`, where a parent operation will call into a sub-operation, will cause us to re-enter from the admission layer. It's important that a sub-operation never acquires a new ticket if the parent operation is already holding one, otherwise we risk deadlocking the system. In both of these cases, we bypass admission control and set the priority in the `ScopedAdmissionPriority` object to **Exempt**.
Each operation will attempt to acquire a ticket unless an operation is marked **exempt**, or if the
operation is already holding a ticket. Exempt tickets typically are held by high priority and
critical internal operations. Meanwhile, re-entrancy API's like `DBDirectClient`, where a parent
operation will call into a sub-operation, will cause us to re-enter from the admission layer. It's
important that a sub-operation never acquires a new ticket if the parent operation is already
holding one, otherwise we risk deadlocking the system. In both of these cases, we bypass admission
control and set the priority in the `ScopedAdmissionPriority` object to **Exempt**.
When an operation **is** subject to admission control, we attempt to acquire a ticket. If there are available tickets, we return the ticket immediately and the operation can continue its execution. If there are no available tickets, the operation will be blocked, and has to wait for one to become available.
When an operation **is** subject to admission control, we attempt to acquire a ticket. If there are
available tickets, we return the ticket immediately and the operation can continue its execution. If
there are no available tickets, the operation will be blocked, and has to wait for one to become
available.
If we find and return a ticket, it will be used for the lifetime of the command, and will be released when `ExecCommandDatabase` is finished [executing][ticketRelease] the command.
If we find and return a ticket, it will be used for the lifetime of the command, and will be
released when `ExecCommandDatabase` is finished [executing][ticketRelease] the command.
## How to apply Admission Control to your command
With your new command created, you have a few options for implementing Admission Control. If it is a high priority command or internal command that is critical for system monitoring and health, you likely want to exempt it from admission control. The virtual parent function will do this [by default][subjectVirtualFalse]. It is important to scrutinize the list of exempted operations because it is critical to the systems health that appropriate operations should queue when possible in the instance of overload.
With your new command created, you have a few options for implementing Admission Control. If it is a
high priority command or internal command that is critical for system monitoring and health, you
likely want to exempt it from admission control. The virtual parent function will do this [by
default][subjectVirtualFalse]. It is important to scrutinize the list of exempted operations because
it is critical to the systems health that appropriate operations should queue when possible in the
instance of overload.
If you want to apply admission control, you will need to override `isSubjectToIngressAdmissionControl` [and return true][subjectAdmissionExTrue]. **Most operations are expected to fall under this category**.
If you want to apply admission control, you will need to override
`isSubjectToIngressAdmissionControl` [and return true][subjectAdmissionExTrue]. **Most operations
are expected to fall under this category**.
To apply admission control selectively, override `isSubjectToIngressAdmissionControl` and implement selective logic to determine [when it should be applied][subjectAdmissionFind].
To apply admission control selectively, override `isSubjectToIngressAdmissionControl` and implement
selective logic to determine [when it should be applied][subjectAdmissionFind].
[initiateCommand]: https://github.com/10gen/mongo/blob/a86c7f5de2a5de4d2f49e40e8970754ec6a5ba6c/src/mongo/db/service_entry_point_shard_role.cpp#L1588
[admissionServerParam]: https://github.com/10gen/mongo/blob/291b72ec4a8364208d7633d881cddc98787832b8/src/mongo/db/service_entry_point_shard_role.cpp#L1804

View File

@ -1,50 +1,22 @@
# Execution Internals
# Catalog Internals
The storage execution layer provides an interface for higher level MongoDB components, including
query, replication and sharding, to all storage engines compatible with MongoDB. It maintains a
catalog, in-memory and on-disk, of collections and indexes. It also implements an additional (to
whatever a storage engine implements) concurrency control layer to safely modify the catalog while
sustaining correct and consistent collection and index data formatting.
Execution facilitates reads and writes to the storage engine with various persistence guarantees,
builds indexes, supports replication rollback, manages oplog visibility, repairs data corruption
and inconsistencies, and much more.
The main code highlights are: the storage integration layer found in the [**storage/**][] directory;
the lock manager and lock helpers found in the [**concurrency/**][] directory; the catalog found in
the [**catalog/**][] directory; the index build code found in many directories; the various types of
index implementations found in the [**index/**][] directory; the sorter found in the
[**sorter/**][] directory; and the time-series bucket catalog found in the [**timeseries/**][]
directory.
[**storage/**]: https://github.com/mongodb/mongo/tree/master/src/mongo/db/storage
[**concurrency/**]: https://github.com/mongodb/mongo/tree/master/src/mongo/db/concurrency
[**catalog/**]: https://github.com/mongodb/mongo/tree/master/src/mongo/db/catalog
[**index/**]: https://github.com/mongodb/mongo/tree/master/src/mongo/db/index
[**sorter/**]: https://github.com/mongodb/mongo/tree/master/src/mongo/db/sorter
[**timeseries/**]: https://github.com/mongodb/mongo/tree/master/src/mongo/db/timeseries
For more information on the Storage Engine API, see the [storage/README][].
For more information on time-series collections, see the [timeseries/README][].
[storage/README]: https://github.com/mongodb/mongo/blob/master/src/mongo/db/storage/README.md
[timeseries/README]: https://github.com/mongodb/mongo/blob/master/src/mongo/db/timeseries/README.md
# The Catalog
The catalog is where MongoDB stores information about the collections and indexes for a MongoDB
The catalog is where MongoDB stores information about the collections and indexes for a replica set
node. In some contexts we refer to this as metadata and to operations changing this metadata as
[DDL](#glossary) (Data Definition Language) operations. The catalog is persisted as a table with
BSON documents that each describe properties of a collection and its indexes. An in-memory catalog
caches the most recent catalog information for more efficient access.
caches this data for more efficient access.
The catalog provides a mapping from logical user-level namespaces to durable storage engine entities and provides a concurrency control layer to safely modify collections and indexes metadata for DDL operations.
See the [Storage Engine API](../storage/README.md) for relevant information.
## Durable Catalog
The durable catalog is persisted as a table with the `_mdb_catalog` [ident](#glossary). Each entry
in this table is indexed with a 64-bit `RecordId`, referred to as the catalog ID, and contains a
BSON document that describes the properties of a collection and its indexes. The `DurableCatalog`
class allows read and write access to the durable data.
The durable catalog is persisted as a table with the `_mdb_catalog`
[ident](../storage/README.md#idents). Each entry in this table is indexed with a 64-bit `RecordId`,
referred to as the catalog ID, and contains a BSON document that describes the properties of a
collection and its indexes. The `DurableCatalog` class allows read and write access to the durable
data.
Starting in v5.2, catalog entries for time-series collections have a new flag called
`timeseriesBucketsMayHaveMixedSchemaData` in the `md` field. Time-series collections upgraded from
@ -97,7 +69,7 @@ Example command invocation and output from
> const adminDB = db.getSiblingDB('admin');
> adminDB.aggregate([{$listCatalog: {}}]);
Collectionless $listCatalog: [
Collection-less $listCatalog: [
{
"db" : "test",
"name" : "system.views",
@ -248,7 +220,7 @@ using the default or user-provided
[list_catalog_read_concern.js](https://github.com/mongodb/mongo/blob/532c0679ef4fc8313a9e00a1334ca18e04ff6914/jstests/noPassthrough/list_catalog_read_concern.js#L46)
contains examples of using $listCatalog with a variety of read concern settings.
The tradtional alternative would have involved a `listDatabases` command followed by a series of
The traditional alternative would have involved a `listDatabases` command followed by a series of
`listCollections` and `listIndexes` calls, with the downside of reading the catalog at a different
point in time during each command invocation.
@ -357,9 +329,13 @@ cache of the [durable catalog](#durable-catalog) state. It provides the followin
lookup to support rollback to a point in time.
- Ensures `Collection` objects are in-sync with opened storage snapshots.
### Concurrency Control
See [Concurrency Control](../concurrency/README.md)
### Synchronization
Catalog access is synchronized using [Multiversion concurrency control] where readers operate on
Catalog access is synchronized using multi-version concurrency control where readers operate on
immutable catalog, collection and index instances. Writes use [copy-on-write][] to create newer
versions of the catalog, collection and index instances to be changed, contents are copied from the
previous latest version. Readers holding on to a catalog instance will thus not observe any writes
@ -408,11 +384,11 @@ In addition `Collection` objects have shared ownership of:
A writable `Collection` may only be requested in an active [WriteUnitOfWork](#WriteUnitOfWork). The
new `Collection` instance is installed in the catalog when the storage transaction commits as the
first `onCommit` [Changes](#Changes) that run. This means that it is not allowed to perform any
modification to catalog, collection or index instances in `onCommit` handlers. Such modifications
would break the immutability property of these instances for readers. If the storage transaction
rolls back then the writable `Collection` object is simply discarded and no change is ever made to
the catalog.
first `onCommit` [Changes](../storage/README.md#changes) that run. This means that it is not allowed
to perform any modification to catalog, collection or index instances in `onCommit` handlers. Such
modifications would break the immutability property of these instances for readers. If the storage
transaction rolls back then the writable `Collection` object is simply discarded and no change is
ever made to the catalog.
A writable `Collection` is a clone of the existing `Collection`, members are either deep or
shallowed copied. Notably, a shallow copy is made for the [`IndexCatalog`](#index-catalog).
@ -512,10 +488,11 @@ The `CollectionCatalog` contains a mapping of `Namespace` and `UUID` to the `cat
timestamps back to the oldest timestamp. These are used for efficient lookups into the durable
catalog, and are resilient to create, drop and rename operations.
Operations that use collection locks (in any [lockmode](#lock-modes)) can rely on the catalog
information of the collection not changing. However, when unlocking and then relocking, not only
should operations recheck catalog information to ensure it is still valid, they should also make
sure to abandon the storage snapshot, so it is consistent with the in memory catalog.
Operations that use collection locks (in any [lock mode](../concurrency/README.md#lock-modes)) can
rely on the catalog information of the collection not changing. However, when unlocking and then
re-locking, not only should operations recheck catalog information to ensure it is still valid, they
should also make sure to abandon the storage snapshot, so it is consistent with the in memory
catalog.
## Two-Phase Collection and Index Drop
@ -540,7 +517,7 @@ sufficiently persisted such that the catalog change will not be rolled back and
to the collection or index in-memory state (tracked via the drop token) remains. When no concurrent
readers of the collection or index are left, the drop token will be the only remaining reference to
the in-memory state. The drop timestamp must be both older than the timestamp of the last checkpoint
and the oldest_timestamp. Requiring the drop timestamp to reach the checkpointed time ensures that
and the oldest_timestamp. Requiring the drop timestamp to reach the checkpoint time ensures that
startup recovery and rollback via recovery to a stable timestamp, which both recover to the last
checkpoint, will never be missing collection or index data that should still exist at the checkpoint
time that is less than the drop timestamp. Requiring the drop timestamp to pass (become older) than
@ -564,126 +541,6 @@ _Code spelunking starting points:_
- [_Code that signals the reaper with a
timestamp_](https://github.com/mongodb/mongo/blob/r4.5.0/src/mongo/db/storage/storage_engine_impl.cpp#L932-L949)
# Storage Transactions
Through the pluggable [storage engine API](https://github.com/mongodb/mongo/blob/master/src/mongo/db/storage/README.md), MongoDB executes reads and writes on its storage engine
with [snapshot isolation](#glossary). The structure used to achieve this is the [RecoveryUnit
class](../storage/recovery_unit.h).
## RecoveryUnit
Each pluggable storage engine for MongoDB must implement `RecoveryUnit` as one of the base classes
for the storage engine API. Typically, storage engines satisfy the `RecoveryUnit` requirements with
some form of [snapshot isolation](#glossary) with [transactions](#glossary). Such transactions are
called storage transactions elsewhere in this document, to differentiate them from the higher-level
_multi-document transactions_ accessible to users of MongoDB. The RecoveryUnit controls what
[snapshot](#glossary) a storage engine transaction uses for its reads. In MongoDB, a snapshot is defined by a
_timestamp_. A snapshot consists of all data committed with a timestamp less than or equal to the
snapshot's timestamp. No uncommitted data is visible in a snapshot, and data changes in storage
transactions that commit after a snapshot is created, regardless of their timestamps, are also not
visible. Generally, one uses a `RecoveryUnit` to perform transactional reads and writes by first
configuring the `RecoveryUnit` with the desired
[ReadSource](https://github.com/mongodb/mongo/blob/b2c1fa4f121fdb6cdffa924b802271d68c3367a3/src/mongo/db/storage/recovery_unit.h#L391-L421)
and then performing the reads and writes using operations on `RecordStore` or `SortedDataInterface`,
and finally calling `commit()` on the `WriteUnitOfWork` (if performing writes).
## WriteUnitOfWork
A `WriteUnitOfWork` is the mechanism to control how writes are transactionally performed on the
storage engine. All the writes (and reads) performed within its scope are part of the same storage
transaction. After all writes have been staged, one must call `commit()` in order to atomically
commit the transaction to the storage engine. It is illegal to perform writes outside the scope of
a WriteUnitOfWork since there would be no way to commit them. If the `WriteUnitOfWork` falls out of
scope before `commit()` is called, the storage transaction is rolled back and all the staged writes
are lost. Reads can be performed outside of a `WriteUnitOfWork` block; storage transactions outside
of a `WriteUnitOfWork` are always rolled back, since there are no writes to commit.
## Lazy initialization of storage transactions
Note that storage transactions on WiredTiger are not started at the beginning of a `WriteUnitOfWork`
block. Instead, the transaction is started implicitly with the first read or write operation. To
explicitly start a transaction, one can use `RecoveryUnit::preallocateSnapshot()`.
## Changes
One can register a `Change` on a `RecoveryUnit` while in a `WriteUnitOfWork`. This allows extra
actions to be performed based on whether a `WriteUnitOfWork` commits or rolls back. These actions
will typically update in-memory state to match what was written in the storage transaction, in a
transactional way. Note that `Change`s are not executed until the destruction of the
`WriteUnitOfWork`, which can be long after the storage engine committed. Two-phase locking ensures
that all locks are held while a Change's `commit()` or `rollback()` function runs.
## StorageUnavailableException
`StorageUnavailableException` indicates that a storage transaction rolled back due to
resource contention in the storage engine. This exception is the base of exceptions related to
concurrency (`WriteConflict`) and to those related to cache pressure (`TemporarilyUnavailable` and
`TransactionTooLargeForCache`).
We recommend using the [writeConflictRetry](https://github.com/mongodb/mongo/blob/9381db6748aada1d9a0056cea0e9899301e7f70b/src/mongo/db/concurrency/exception_util.h#L140)
helper which transparently handles all exceptions related to this error category.
### WriteConflictException
Writers may conflict with each other when more than one operation stages an uncommitted write to the
same document concurrently. To force one or more of the writers to retry, the storage engine may
throw a WriteConflictException at any point, up to and including the call to commit(). This is
referred to as optimistic concurrency control because it allows uncontended writes to commit
quickly. Because of this behavior, most WUOWs are enclosed in a writeConflictRetry loop that retries
the write transaction until it succeeds, accompanied by a bounded exponential back-off.
### TemporarilyUnavailableException
When the server parameter `enableTemporarilyUnavailableExceptions` is enabled (on by default), a
TemporarilyUnavailableException may be thrown inside the server to indicate that an operation cannot
complete without blocking and must be retried. The storage engine may throw a
TemporarilyUnavailableException (converted to a TemporarilyUnavailable error for users) when an
operation is excessively rolled-back in the storage engine due to cache pressure or any reason that
would prevent the operation from completing without impacting concurrent operations. The operation
may be at fault for writing too much uncommitted data, or it may be a victim. That information is
not exposed. However, if this error is returned, it is likely that the operation was the cause of
the problem, rather than a victim.
Before 6.0, this type of error was returned as a WriteConflict and retried indefinitely inside a
writeConflictRetry loop. As of 6.0, MongoDB will retry the operation internally at most
`temporarilyUnavailableMaxRetries` times, backing off for `temporarilyUnavailableBackoffBaseMs`
milliseconds, with a linearly-increasing backoff on each attempt. After this point, the error will
escape the handler and be returned to the client.
If an operation receives a TemporarilyUnavailable error internally, a `temporarilyUnavailableErrors`
counter will be displayed in the slow query logs and in FTDC.
Notably, this behavior does not apply to multi-document transactions, which continue to return a
WriteConflict to the client in this scenario without retrying internally.
See
[wtRcToStatus](https://github.com/mongodb/mongo/blob/c799851554dc01493d35b43701416e9c78b3665c/src/mongo/db/storage/wiredtiger/wiredtiger_util.cpp#L178-L183)
where we throw the exception in WiredTiger.
See [TemporarilyUnavailableException](https://github.com/mongodb/mongo/blob/c799851554dc01493d35b43701416e9c78b3665c/src/mongo/db/concurrency/temporarily_unavailable_exception.h#L39-L45).
### TransactionTooLargeForCacheException
A TransactionTooLargeForCacheException may be thrown inside the server to indicate that an operation
was rolled-back and is unlikely to ever complete because the storage engine cache is insufficient,
even in the absence of concurrent operations. This is determined by a simple heuristic wherein,
after a rollback, a threshold on the proportion of total dirty cache bytes the running transaction
can represent and still be considered fullfillable is checked. The threshold can be tuned with the
`transactionTooLargeForCacheThreshold` parameter. Setting this threshold to its maximum value (1.0)
causes the check to be skipped and TransactionTooLargeForCacheException to be disabled.
On replica sets, if an operation succeeds on a primary, it should also succeed on a secondary. It
would be possible to convert to both TemporarilyUnavailableException and WriteConflictException,
as if TransactionTooLargeForCacheException was disabled. But on secondaries the only
difference between the two is the rate at which the operation is retried. Hence,
TransactionTooLargeForCacheException is always converted to a WriteConflictException, which retries
faster, to avoid stalling replication longer than necessary.
Prior to 6.3, or when TransactionTooLargeForCacheException is disabled, multi-document
transactions always return a WriteConflictException, which may result in drivers retrying an
operation indefinitely. For non-multi-document operations, there is a limited number of retries on
TemporarilyUnavailableException, but it might still be beneficial to not retry operations which are
unlikely to complete and are disruptive for concurrent operations.
# Read Operations
External reads via the find, count, distinct, aggregation, and mapReduce cmds do not take collection
@ -699,8 +556,6 @@ or from a call to abandonSnapshot(). Lock-free read operations must re-establish
after a query yield, just as at the start of a read operation.
See
[WiredTigerCursor](https://github.com/mongodb/mongo/blob/r4.4.0-rc13/src/mongo/db/storage/wiredtiger/wiredtiger_cursor.cpp#L48),
[WiredTigerRecoveryUnit::getSession()](https://github.com/mongodb/mongo/blob/r4.4.0-rc13/src/mongo/db/storage/wiredtiger/wiredtiger_recovery_unit.cpp#L303-L305),
[~GlobalLock](https://github.com/mongodb/mongo/blob/r4.4.0-rc13/src/mongo/db/concurrency/d_concurrency.h#L228-L239),
[PlanYieldPolicy::\_yieldAllLocks()](https://github.com/mongodb/mongo/blob/r4.4.0-rc13/src/mongo/db/query/plan_yield_policy.cpp#L182),
[RecoveryUnit::abandonSnapshot()](https://github.com/mongodb/mongo/blob/r4.4.0-rc13/src/mongo/db/storage/recovery_unit.h#L217).
@ -805,34 +660,12 @@ to read at
Operations that write to collections and indexes must take collection locks. Storage engines require
all operations to hold at least a collection IX lock to provide document-level concurrency.
Operations must perform writes in the scope of a WriteUnitOfWork.
## WriteUnitOfWork
All reads and writes in the scope of a WriteUnitOfWork (WUOW) operate on the same storage engine
snapshot, and all writes in the scope of a WUOW are transactional; they are either all committed or
all rolled-back. The WUOW commits writes that took place in its scope by a call to commit(). It
rolls-back writes when it goes out of scope and its destructor is called before a call to commit().
The WriteUnitOfWork has a [`groupOplogEntries` option](https://github.com/mongodb/mongo/blob/fa32d665bd63de7a9d246fa99df5e30840a931de/src/mongo/db/storage/write_unit_of_work.h#L67)
to replicate multiple writes transactionally. This option uses the [`BatchedWriteContext` class](https://github.com/mongodb/mongo/blob/9ab71f9b2fac1e384529fafaf2a819ce61834228/src/mongo/db/batched_write_context.h#L46)
to stage writes and to generate a single applyOps entry at commit, similar to what multi-document
transactions do via the [`TransactionParticipant` class](https://github.com/mongodb/mongo/blob/219990f17695b0ea4695f827a42a18e012b1e9cf/src/mongo/db/transaction/transaction_participant.h#L82).
Unlike a multi-document transaction, the applyOps entry lacks the `lsId` and the `txnNumber`
fields. Callers must ensure that the WriteUnitOfWork does not generate more than 16MB of oplog,
otherwise the operation will fail with `TransactionTooLarge` code.
As of MongoDB 6.0, the `groupOplogEntries` mode is only used by the [BatchedDeleteStage](https://github.com/mongodb/mongo/blob/9676cf4ad8d537518eb1b570fc79bad4f31d8a79/src/mongo/db/exec/batched_delete_stage.h)
for efficient mass-deletes.
See
[WriteUnitOfWork](https://github.com/mongodb/mongo/blob/fa32d665bd63de7a9d246fa99df5e30840a931de/src/mongo/db/storage/write_unit_of_work.h).
Operations must perform writes in the scope of a [WriteUnitOfWork](../storage/README.md#writeunit).
## Collection and Index Writes
Collection write operations (inserts, updates, and deletes) perform storage engine writes to both
the collection's RecordStore and relevant index's SortedDataInterface in the same storage transaction, or
WUOW. This ensures that completed, not-building indexes are always consistent with collection data.
the collection's RecordStore and relevant index's SortedDataInterface in the same storage transaction, or WUOW. This ensures that completed, not-building indexes are always consistent with collection data.
## Vectored Inserts
@ -846,128 +679,6 @@ though all writes took place in a single storage transaction.
See
[insertDocuments](https://github.com/mongodb/mongo/blob/r4.4.0/src/mongo/db/ops/write_ops_exec.cpp#L315)
and
[WiredTigerRecordStore::insertRecords](https://github.com/mongodb/mongo/blob/r4.4.0/src/mongo/db/storage/wiredtiger/wiredtiger_record_store.cpp#L1494).
# Concurrency Control
Theoretically, one could design a database that used only mutexes to maintain database consistency
while supporting multiple simultaneous operations; however, this solution would result in pretty bad
performance and would a be strain on the operating system. Therefore, databases typically use a more
complex method of coordinating operations. This design consists of Resources (lockable entities),
some of which may be organized in a Hierarchy, and Locks (requests for access to a resource). A Lock
Manager is responsible for keeping track of Resources and Locks, and for managing each Resource's
Lock Queue. The Lock Manager identifies Resources with a ResourceId.
## Resource Hierarchy
In MongoDB, Resources are arranged in a hierarchy, in order to provide an ordering to prevent
deadlocks when locking multiple Resources, and also as an implementation of Intent Locking (an
optimization for locking higher level resources). The hierarchy of ResourceTypes is as follows:
1. Global (three - see below)
1. Database (one per database on the server)
1. Collection (one per collection on the server)
Each resource must be locked in order from the top. Therefore, if a Collection resource is desired
to be locked, one must first lock the one Global resource, and then lock the Database resource that
is the parent of the Collection. Finally, the Collection resource is locked.
In addition to these ResourceTypes, there also exists ResourceMutex, which is independent of this
hierarchy. One can use ResourceMutex instead of a regular mutex if one desires the features of the
lock manager, such as fair queuing and the ability to have multiple simultaneous lock holders.
## Lock Modes
The lock manager keeps track of each Resource's _granted locks_ and a queue of _waiting locks_.
Rather than the binary "locked-or-not" modes of a mutex, a MongoDB lock can have one of several
_modes_. Modes have different _compatibilities_ with other locks for the same resource. Locks with
compatible modes can be simultaneously granted to the same resource, while locks with modes that are
incompatible with any currently granted lock on a resource must wait in the waiting queue for that
resource until the conflicting granted locks are unlocked. The different types of modes are:
1. X (exclusive): Used to perform writes and reads on the resource.
2. S (shared): Used to perform only reads on the resource (thus, it is okay to Share with other
compatible locks).
3. IX (intent-exclusive): Used to indicate that an X lock is taken at a level in the hierarchy below
this resource. This lock mode is used to block X or S locks on this resource.
4. IS (intent-shared): Used to indicate that an S lock is taken at a level in the hierarchy below
this resource. This lock mode is used to block X locks on this resource.
## Lock Compatibility Matrix
This matrix answers the question, given a granted lock on a resource with the mode given, is a
requested lock on that same resource with the given mode compatible?
| Requested Mode | | | Granted Mode | | |
| :------------- | :-------: | :-----: | :----------: | :----: | :----: |
| | MODE_NONE | MODE_IS | MODE_IX | MODE_S | MODE_X |
| MODE_IS | Y | Y | Y | Y | N |
| MODE_IX | Y | Y | Y | N | N |
| MODE_S | Y | Y | N | Y | N |
| MODE_X | Y | N | N | N | N |
Typically, locks are granted in the order they are queued, but some LockRequest behaviors can be
specially selected to break this rule. One behavior is _enqueueAtFront_, which allows important lock
acquisitions to cut to the front of the line, in order to expedite them. Currently, all mode X and S
locks for the three Global Resources (Global, MultiDocumentTransactionsBarrier, RSTL) automatically
use this option.
Another behavior is _compatibleFirst_, which allows compatible lock requests to cut ahead of others
waiting in the queue and be granted immediately; note that this mode might starve queued lock
requests indefinitely.
### Replication State Transition Lock (RSTL)
The Replication State Transition Lock is of ResourceType Global, so it must be locked prior to
locking any Database level resource. This lock is used to synchronize replica state transitions
(typically transitions between PRIMARY, SECONDARY, and ROLLBACK states).
More information on the RSTL is contained in the [Replication Architecture Guide](https://github.com/mongodb/mongo/blob/b4db8c01a13fd70997a05857be17548b0adec020/src/mongo/db/repl/README.md#replication-state-transition-lock)
### Global Lock
The resource known as the Global Lock is of ResourceType Global. It is currently used to
synchronize shutdown, so that all operations are finished with the storage engine before closing it.
Certain types of global storage engine operations, such as recoverToStableTimestamp(), also require
this lock to be held in exclusive mode.
### Tenant Lock
A resource of ResourceType Tenant is used when a database belongs to a tenant. It is used to synchronize
change streams enablement and disablement for a tenant operation with other operations associated with the tenant.
Enabling or disabling of change streams (by creating or dropping a change collection) for a tenant takes this lock
in exclusive (X) mode. Acquiring this resource with an intent lock is an indication that the operation is doing reads (IS)
or writes (IX) at the database or lower level.
### Database Lock
Any resource of ResourceType Database protects certain database-wide operations such as database
drop. These operations are being phased out, in the hopes that we can eliminate this ResourceType
completely.
### Collection Lock
Any resource of ResourceType Collection protects certain collection-wide operations, and in some
cases also protects the in-memory catalog structure consistency in the face of concurrent readers
and writers of the catalog. Acquiring this resource with an intent lock is an indication that the
operation is doing explicit reads (IS) or writes (IX) at the document level. There is no Document
ResourceType, as locking at this level is done in the storage engine itself for efficiency reasons.
### Document Level Concurrency Control
Each storage engine is responsible for locking at the document level. The WiredTiger storage engine
uses MVCC [multiversion concurrency control][] along with optimistic locking in order to provide
concurrency guarantees.
## Two-Phase Locking
The lock manager automatically provides _two-phase locking_ for a given storage transaction.
Two-phase locking consists of an Expanding phase where locks are acquired but not released, and a
subsequent Shrinking phase where locks are released but not acquired. By adhering to this protocol,
a transaction will be guaranteed to be serializable with other concurrent transactions. The
WriteUnitOfWork class manages two-phase locking in MongoDB. This results in the somewhat unexpected
behavior of the RAII locking types acquiring locks on resources upon their construction but not
unlocking the lock upon their destruction when going out of scope. Instead, the responsibility of
unlocking the locks is transferred to the WriteUnitOfWork destructor. Note this is only true for
transactions that do writes, and therefore only for code that uses WriteUnitOfWork.
# Indexes
@ -1939,8 +1650,6 @@ A new truncate marker is created when the in-progress marker segment contains mo
complete the segment; and the oldest truncate marker's oplog is deleted when the oplog size exceeds
its cap size setting.
Oplog sampling and marker generation is skipped when using `--restore` or `--magicRestore`.
### Special Timestamps That Will Not Be Truncated
The WiredTiger integration layer's `OplogTruncateMarkers` implementation will stall deletion waiting for
@ -2142,13 +1851,9 @@ A time-series collection is a view of an internal clustered collection named
`system.buckets.<name>`, where `<name>` is the name of the time-series collection. The cluster key
values are ObjectId's.
The TTL monitor will only delete data from a time-series bucket collection when a bucket's minimum
time, \_id, is past the expiration plus the bucket maximum time span (default 1 hour). This
procedure avoids deleting buckets with data that is not older than the expiration time.
For more information on time-series collections, see the [timeseries/README][].
[timeseries/README]: https://github.com/mongodb/mongo/blob/master/src/mongo/db/timeseries/README.md
[timeseries/README]: ../timeseries/README.md
## Capped clustered collections
@ -2160,10 +1865,9 @@ inserted. In order to guarantee capped insert-order semantics the caller should
increasing cluster key values.
Because unlike regular capped collections, clustered collections do not need to preserve insertion
order, they allow non-serialised concurrent writes. In order to avoid missing documents while
order, they allow non-serialized concurrent writes. In order to avoid missing documents while
tailing a clustered collection, the user is required to enforce visibility rules similar to the ['no
holes' point](https://github.com/mongodb/mongo/blob/r5.2.0/src/mongo/db/catalog/README.md#oplog-visibility).
Majority read concern is similarly required.
holes' point](../storage/README.md#oplog-visibility). Majority read concern is similarly required.
## Clustered RecordIds
@ -2201,37 +1905,9 @@ these are treated as ascending. Details on how these values are treated can be f
context of relational databases. DDL operations in the MongoDB context include index and collection
creation or drop, as well as `collMod` operations.
**ident**: An ident is a unique identifier given to a storage engine resource. Collections and
indexes map application-layer names to storage engine idents. In WiredTiger, idents are implemented
as tables. Idents map to files on disk, but with a `.wt` file extension.
Examples:
- collection idents: `collection-<counter>-<random number>`
- index idents: `index-<counter>-<random number>`
Server flags that alter the form of idents (this applies to indexes as well):
- `--directoryperdb`: `<db name>/collection-<counter>-<random number>`
- `--wiredTigerDirectoryForIndexes`: `collection/<counter>-<random number>`
- (both of the above): `<db name>/collection/<counter-<random number>`
**oplog hole**: An uncommitted oplog write that can exist with out-of-order writes when a later
timestamped write happens to commit first. Oplog holes can exist in-memory and persisted on disk.
**oplogReadTimestamp**: The timestamp used for WT forward cursor oplog reads in order to avoid
advancing past oplog holes. Tracks in-memory oplog holes.
**oplogTruncateAfterPoint**: The timestamp after which oplog entries will be truncated during
startup recovery after an unclean shutdown. Tracks persisted oplog holes.
**snapshot**: A snapshot consists of a consistent view of data in the database. In MongoDB, a
snapshot consists of all data committed with a timestamp less than or equal to the snapshot's
timestamp.
**snapshot isolation**: A guarantee that all reads in a transaction see the same consistent snapshot
of the database, and that all writes in a transaction had no conflicts with other concurrent writes,
if the transaction commits.
**snapshot**: A snapshot consists of a consistent view of data in the database. When a snapshot is
opened with a timestamp, snapshot only shows data committed with a timestamp less than or equal
to the snapshot's timestamp.
**storage transaction**: A concept provided by a pluggable storage engine through which changes to
data in the database can be performed. In order to satisfy the MongoDB pluggable storage engine
@ -2244,133 +1920,3 @@ only provides guarantees within one node.
[`BSONElement::compareElements`]: https://github.com/mongodb/mongo/blob/v4.4/src/mongo/bson/bsonelement.cpp#L285
[`Ordering`]: https://github.com/mongodb/mongo/blob/v4.4/src/mongo/bson/ordering.h
[initial sync]: ../repl/README.md#initial-sync
# Appendix
## Collection and Index to Table relationship
Creating a collection (record store) or index requires two WT operations that cannot be made
atomic/transactional. A WT table must be created with
[WT_SESSION::create](https://source.wiredtiger.com/develop/struct_w_t___s_e_s_s_i_o_n.html#a358ca4141d59c345f401c58501276bbb "WiredTiger Docs") and an insert/update must be made in the \_mdb_catalog table (MongoDB's
catalog). MongoDB orders these as such:
1. Create the WT table
1. Update \_mdb_catalog to reference the table
Note that if the process crashes in between those steps, the collection/index creation never
succeeded. Upon a restart, the WT table is dangling and can be safely deleted.
Dropping a collection/index follows the same pattern, but in reverse.
1. Delete the table from the \_mdb_catalog
1. [Drop the WT table](https://source.wiredtiger.com/develop/struct_w_t___s_e_s_s_i_o_n.html#adf785ef53c16d9dcc77e22cc04c87b70 "WiredTiger Docs")
In this case, if a crash happens between these steps and the change to the \_mdb_catalog was made
durable (in modern versions, only possible via a checkpoint; the \_mdb_catalog is not logged), the
WT table is once again dangling on restart. Note that in the absense of a history, this state is
indistinguishable from the creation case, establishing a strong invariant.
## Cherry-picked WT log Details
- The WT log is a write ahead log. Before a [transaction commit](https://source.wiredtiger.com/develop/struct_w_t___s_e_s_s_i_o_n.html#a712226eca5ade5bd123026c624468fa2 "WiredTiger Docs") returns to the application, logged writes
must have their log entry bytes written into WiredTiger's log buffer. Depending on `sync` setting,
those bytes may or may not be on disk.
- MongoDB only chooses to log writes to a subset of WT's tables (e.g: the oplog).
- MongoDB does not `sync` the log on transaction commit. But rather uses the [log
flush](https://source.wiredtiger.com/develop/struct_w_t___s_e_s_s_i_o_n.html#a1843292630960309129dcfe00e1a3817 "WiredTiger Docs") API. This optimization is two-fold. Writes that do not require to be
persisted do not need to wait for durability on disk. Second, this pattern allows for batching
of writes to go to disk for improved throughput.
- WiredTiger's log is similar to MongoDB's oplog in that multiple writers can concurrently copy
their bytes representing a log record into WiredTiger's log buffer similar to how multiple
MongoDB writes can concurrently generate oplog entries.
- MongoDB's optime generator for the oplog is analogous to WT's LSN (log sequence number)
generator. Both are a small critical section to ensure concurrent writes don't get the same
timestamp key/memory address to write an oplog entry value/log bytes into.
- While MongoDB's oplog writes are logical (the key is a timestamp), WT's are obviously more
physical (the key is a memory->disk location). WiredTiger is writing to a memory buffer. Thus before a
transaction commit can go to the log buffer to "request a slot", it must know how many bytes it's
going to write. Compare this to a multi-statement transaction replicating as a single applyOps
versus each statement generating an individual oplog entry for each write that's part of the
transaction.
- MongoDB testing sometimes uses a [WT debugging
option](https://github.com/mongodb/mongo/blob/a7bd84dc5ad15694864526612bceb3877672d8a9/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.cpp#L601 "Github") that will write "no-op" log entries for other operations performed on a
transaction. Such as setting a timestamp or writing to a table that is not configured to be
written to WT's log (e.g: a typical user collection and index).
The most important WT log entry for MongoDB is one that represents an insert into the
oplog.
```
{ "lsn" : [1,57984],
"hdr_flags" : "compressed",
"rec_len" : 384,
"mem_len" : 423,
"type" : "commit",
"txnid" : 118,
"ops": [
{ "optype": "row_put",
"fileid": 14 0xe,
"key": "\u00e8^\u00eat@\u00ff\u00ff\u00df\u00c2",
"key-hex": "e85eea7440ffffdfc2",
"value": "\u009f\u0000\u0000\u0000\u0002op\u0000\u0002\u0000\u0000\u0000i\u0000\u0002ns\u0000\n\u0000\u0000\u0000test.coll\u0000\u0005ui\u0000\u0010\u0000\u0000\u0000\u0004\u0017\u009d\u00b0\u00fc\u00b2,O\u0004\u0084\u00bdY\u00e9%\u001dm\u00ba\u0003o\u00002\u0000\u0000\u0000\u0007_id\u0000^\u00eatA\u00d4\u0098\u00b7\u008bD\u009b\u00b2\u008c\u0002payload\u0000\u000f\u0000\u0000\u0000data and bytes\u0000\u0000\u0011ts\u0000\u0002\u0000\u0000\u0000At\u00ea^\u0012t\u0000\u0001\u0000\u0000\u0000\u0000\u0000\u0000\u0000\twall\u0000\u0085\u001e\u00d6\u00c3r\u0001\u0000\u0000\u0012v\u0000\u0002\u0000\u0000\u0000\u0000\u0000\u0000\u0000\u0000",
"value-bson": {
u'ns': u'test.coll',
u'o': {u'_id': ObjectId('5eea7441d498b78b449bb28c'), u'payload': u'data and bytes'},
u'op': u'i',
u't': 1L,
u'ts': Timestamp(1592423489, 2),
u'ui': UUID('179db0fc-b22c-4f04-84bd-59e9251d6dba'),
u'v': 2L,
u'wall': datetime.datetime(2020, 6, 17, 19, 51, 29, 157000)}
}
]
}
```
- `lsn` is a log sequence number. The WiredTiger log files are named with numbers as a
suffix, e.g: `WiredTigerLog.0000000001`. In this example, the LSN's first value `1` maps to log
file `0000000001`. The second value `57984` is the byte offset in the file.
- `hdr_flags` stands for header flags. Think HTTP headers. MongoDB configures WiredTiger to use
snappy compression on its journal entries. Small journal entries (< 128 bytes?) won't be
compressed.
- `rec_len` is the number of bytes for the record
- `type` is...the type of journal entry. The type will be `commit` for application's committing a
transaction. Other types are typically for internal WT operations. Examples include `file_sync`,
`checkpoint` and `system`.
- `txnid` is WT's transaction id associated with the log record.
- `ops` is a list of operations that are part of the transaction. A transaction that inserts two
documents and removes a third will see three entries. Two `row_put` operations followed by a
`row_remove`.
- `ops.fileid` refers to the WT table that the operation is performed against. The fileid mapping
is held in the `WiredTiger.wt` file (a table within itself). This value is faked for WT's
logging debug mode for tables which MongoDB is not logging.
- `ops.key` and `ops.value` are the binary representations of the inserted document (`value` is omitted
for removal).
- `ops.key-hex` and `ops.value-bson` are specific to the pretty printing tool used.
[copy-on-write]: https://en.wikipedia.org/wiki/Copy-on-write
[Multiversion concurrency control]: https://en.wikipedia.org/wiki/Multiversion_concurrency_control
## Table of MongoDB <-> WiredTiger <-> Log version numbers
| MongoDB | WiredTiger | Log |
| ---------------------- | ---------- | --- |
| 3.0.15 | 2.5.3 | 1 |
| 3.2.20 | 2.9.2 | 1 |
| 3.4.15 | 2.9.2 | 1 |
| 3.6.4 | 3.0.1 | 2 |
| 4.0.16 | 3.1.1 | 3 |
| 4.2.1 | 3.2.2 | 3 |
| 4.2.6 | 3.3.0 | 3 |
| 4.2.6 (blessed by 4.4) | 3.3.0 | 4 |
| 4.4.0 | 10.0.0 | 5 |
| 5.0.0 | 10.0.1 | 5 |
| 4.4.11, 5.0.6 | 10.0.2 | 5 |
| 6.0.0 | 10.0.2 | 5 |
| 6.1.0 | 11.0.1 | 5 |
| 6.2.0 | 11.2.0 | 5 |
| 7.0.0 | 11.2.0 | 5 |
| 7.1.0 | 11.2.0 | 5 |
| 7.2.0 | 11.3.0 | 5 |
| 7.3.0 | 11.3.0 | 5 |
| 8.0.0 | 11.3.0 | 5 |

View File

@ -0,0 +1,119 @@
# Concurrency Control
Theoretically, one could design a database that used only mutexes to maintain database consistency
while supporting multiple simultaneous operations; however, this solution would result in pretty bad
performance and would a be strain on the operating system. Therefore, databases typically use a more
complex method of coordinating operations. This design consists of Resources (lockable entities),
some of which may be organized in a Hierarchy, and Locks (requests for access to a resource). A Lock
Manager is responsible for keeping track of Resources and Locks, and for managing each Resource's
Lock Queue. The Lock Manager identifies Resources with a ResourceId.
## Resource Hierarchy
In MongoDB, Resources are arranged in a hierarchy, in order to provide an ordering to prevent
deadlocks when locking multiple Resources, and also as an implementation of Intent Locking (an
optimization for locking higher level resources). The hierarchy of ResourceTypes is as follows:
1. Global (three - see below)
1. Database (one per database on the server)
1. Collection (one per collection on the server)
Each resource must be locked in order from the top. Therefore, if a Collection resource is desired
to be locked, one must first lock the one Global resource, and then lock the Database resource that
is the parent of the Collection. Finally, the Collection resource is locked.
In addition to these ResourceTypes, there also exists ResourceMutex, which is independent of this
hierarchy. One can use ResourceMutex instead of a regular mutex if one desires the features of the
lock manager, such as fair queuing and the ability to have multiple simultaneous lock holders.
## Lock Modes
The lock manager keeps track of each Resource's _granted locks_ and a queue of _waiting locks_.
Rather than the binary "locked-or-not" modes of a mutex, a MongoDB lock can have one of several
_modes_. Modes have different _compatibilities_ with other locks for the same resource. Locks with
compatible modes can be simultaneously granted to the same resource, while locks with modes that are
incompatible with any currently granted lock on a resource must wait in the waiting queue for that
resource until the conflicting granted locks are unlocked. The different types of modes are:
1. X (exclusive): Used to perform writes and reads on the resource.
2. S (shared): Used to perform only reads on the resource (thus, it is okay to Share with other
compatible locks).
3. IX (intent-exclusive): Used to indicate that an X lock is taken at a level in the hierarchy below
this resource. This lock mode is used to block X or S locks on this resource.
4. IS (intent-shared): Used to indicate that an S lock is taken at a level in the hierarchy below
this resource. This lock mode is used to block X locks on this resource.
## Lock Compatibility Matrix
This matrix answers the question, given a granted lock on a resource with the mode given, is a
requested lock on that same resource with the given mode compatible?
| Requested Mode | | | Granted Mode | | |
| :------------- | :-------: | :-----: | :----------: | :----: | :----: |
| | MODE_NONE | MODE_IS | MODE_IX | MODE_S | MODE_X |
| MODE_IS | Y | Y | Y | Y | N |
| MODE_IX | Y | Y | Y | N | N |
| MODE_S | Y | Y | N | Y | N |
| MODE_X | Y | N | N | N | N |
Typically, locks are granted in the order they are queued, but some LockRequest behaviors can be
specially selected to break this rule. One behavior is _enqueueAtFront_, which allows important lock
acquisitions to cut to the front of the line, in order to expedite them. Currently, all mode X and S
locks for the three Global Resources (Global, MultiDocumentTransactionsBarrier, RSTL) automatically
use this option.
Another behavior is _compatibleFirst_, which allows compatible lock requests to cut ahead of others
waiting in the queue and be granted immediately; note that this mode might starve queued lock
requests indefinitely.
### Replication State Transition Lock (RSTL)
The Replication State Transition Lock is of ResourceType Global, so it must be locked prior to
locking any Database level resource. This lock is used to synchronize replica state transitions
(typically transitions between PRIMARY, SECONDARY, and ROLLBACK states).
More information on the RSTL is contained in the [Replication Architecture Guide](https://github.com/mongodb/mongo/blob/b4db8c01a13fd70997a05857be17548b0adec020/src/mongo/db/repl/README.md#replication-state-transition-lock)
### Global Lock
The resource known as the Global Lock is of ResourceType Global. It is currently used to
synchronize shutdown, so that all operations are finished with the storage engine before closing it.
Certain types of global storage engine operations, such as recoverToStableTimestamp(), also require
this lock to be held in exclusive mode.
### Tenant Lock
A resource of ResourceType Tenant is used when a database belongs to a tenant. It is used to synchronize
change streams enablement and disablement for a tenant operation with other operations associated with the tenant.
Enabling or disabling of change streams (by creating or dropping a change collection) for a tenant takes this lock
in exclusive (X) mode. Acquiring this resource with an intent lock is an indication that the operation is doing reads (IS)
or writes (IX) at the database or lower level.
### Database Lock
Any resource of ResourceType Database protects certain database-wide operations such as database
drop. These operations are being phased out, in the hopes that we can eliminate this ResourceType
completely.
### Collection Lock
Any resource of ResourceType Collection protects certain collection-wide operations, and in some
cases also protects the in-memory catalog structure consistency in the face of concurrent readers
and writers of the catalog. Acquiring this resource with an intent lock is an indication that the
operation is doing explicit reads (IS) or writes (IX) at the document level. There is no Document
ResourceType, as locking at this level is done in the storage engine itself for efficiency reasons.
### Document Level Concurrency Control
Each storage engine is responsible for locking at the document level. The [WiredTiger storage
engine](../storage/wiredtiger/README.md) uses MVCC [multi-version concurrency control][Multiversion concurrency control] along with optimistic locking in order to provide concurrency guarantees.
## Two-Phase Locking
The lock manager automatically provides _two-phase locking_ for a given storage transaction.
Two-phase locking consists of an Expanding phase where locks are acquired but not released, and a
subsequent Shrinking phase where locks are released but not acquired. By adhering to this protocol,
a transaction will be guaranteed to be serializable with other concurrent transactions. The
WriteUnitOfWork class manages two-phase locking in MongoDB. This results in the somewhat unexpected
behavior of the RAII locking types acquiring locks on resources upon their construction but not
unlocking the lock upon their destruction when going out of scope. Instead, the responsibility of
unlocking the locks is transferred to the WriteUnitOfWork destructor. Note this is only true for
transactions that do writes, and therefore only for code that uses WriteUnitOfWork.

View File

@ -0,0 +1,254 @@
# Index Builds
Indexes are built by performing a full scan of collection data. To be considered consistent, an
index must correctly map keys to all documents.
At a high level, omitting details that will be elaborated upon in further sections, index builds
have the following procedure:
- While holding a collection X lock, write a new index entry to the array of indexes included as
part of a durable catalog entry. This entry has a `ready: false` component. See [Durable
Catalog](../catalog/README.md#durable-catalog).
- Downgrade to a collection IX lock.
- Scan all documents on the collection to be indexed
- Generate [KeyString](../storage/key_string/README.md) keys for the indexed fields for each
document
- Periodically yield locks and storage engine snapshots
- Insert the generated keys into the [external sorter](../sorter/README.md)
- Read the sorted keys from the external sorter and [bulk
load](http://source.wiredtiger.com/3.2.1/tune_bulk_load.html) into the storage engine index.
Bulk-loading requires keys to be inserted in sorted order, but builds a B-tree structure that is
more efficiently filled than with random insertion.
- While holding a collection X lock, make a final `ready: true` write to the durable catalog.
## Hybrid Index Builds
Hybrid index builds refer to the default procedure introduced in 4.2 that produces efficient index
data structures without blocking reads or writes for extended periods of time. This is achieved by
performing a full collection scan and bulk-loading keys (described above) while concurrently
intercepting new writes into a temporary storage engine table.
### Temporary Side Table For New Writes
During an index build, new writes (i.e. inserts, updates, and deletes) are applied to the collection
as usual. However, instead of writing directly into the index table as a normal write would, index
keys for documents are generated and intercepted by inserting into a temporary _side-writes_ table.
Writes are intercepted for the duration of the index build, from before the collection scan begins
until the build is completed.
Both inserted and removed keys are recorded in the _side-writes_ table. For example, during an index
build on `{a: 1}`, an update on a document from `{_id: 0, a: 1}` to `{_id: 0, a: 2}` is recorded as
a deletion of the key `1` and an insertion of the key `2`.
Once the collection scan and bulk-load phases of the index build are complete, these intercepted
keys are applied directly to the index in three phases:
- While holding a collection IX lock to allow concurrent reads and writes
- Because writes are still accepted, new keys may appear at the end of the _side-writes_ table.
They will be applied in subsequent steps.
- While holding a collection S lock to block concurrent writes, but not reads
- While holding a collection X lock to block all reads and writes
See
[IndexBuildInterceptor::sideWrite](https://github.com/mongodb/mongo/blob/r4.5.0/src/mongo/db/index/index_build_interceptor.cpp#L403)
and
[IndexBuildInterceptor::drainWritesIntoIndex](https://github.com/mongodb/mongo/blob/r4.5.0/src/mongo/db/index/index_build_interceptor.cpp#L135).
### Temporary Table For Duplicate Key Violations
Unique indexes created with `{unique: true}` enforce a constraint that there are no duplicate keys
in an index. The hybrid index procedure makes it challenging to detect duplicates because keys are
split between the bulk-loaded index and the side-writes table. Additionally, during the lifetime of
an index build, concurrent writes may introduce and resolve duplicate key conflicts on the index.
For those reasons, during an index build we temporarily allow duplicate key violations, and record
any detected violations in a temporary table, the _duplicate key table_. At the conclusion of the
index build, under a collection X lock, [duplicate keys are
re-checked](https://github.com/mongodb/mongo/blob/r4.4.0-rc9/src/mongo/db/index_builds_coordinator.cpp#L2312).
If there are still constraint violations, an error is thrown.
See
[DuplicateKeyTracker](https://github.com/mongodb/mongo/blob/r4.5.0/src/mongo/db/index/duplicate_key_tracker.h#L48).
### Temporary Table For Key Generation Errors
In addition to uniqueness constraints, indexes may have per-key constraints. For example, a compound
index may not be built on documents with parallel arrays. An index build on `{a: 1, b: 1}` will fail
to generate a key for `{a: [1, 2, 3], b: [4, 5, 6]}`.
On a primary under normal circumstances, we could fail an index build immediately after encountering
a key generation error. Since secondaries apply oplog entries [out of
order](../repl/README.md#oplog-entry-application), however, spurious key generation errors may be
encountered on otherwise consistent data. To solve this problem, we can relax key constraints and
suppress key generation errors on secondaries.
With the introduction of simultaneous index builds, an index build may be started on a secondary
node, but complete while it is a primary after a state transition. If we ignored constraints while
in the secondary state, we would not be able to commit the index build and guarantee its consistency
since we may have suppressed valid key generation errors.
To solve this problem, on both primaries and secondaries, the records associated with key generation
errors are skipped and recorded in a temporary table, the _skipped record table_. Like duplicate key
constraints, but only on primaries at the conclusion of the index build, the keys for the [skipped
records are
re-generated](https://github.com/mongodb/mongo/blob/r4.4.0-rc9/src/mongo/db/index_builds_coordinator.cpp#L2294)
and re-inserted under a collection X lock. If there are still constraint violations, an error is
thrown. Secondaries rely on the primary's decision to commit as assurance that skipped records do
not need to be checked.
See
[SkippedRecordTracker](https://github.com/mongodb/mongo/blob/r4.5.0/src/mongo/db/index/skipped_record_tracker.h#L45).
## Replica Set Index Builds
Also referred to as "simultaneous index builds" and "two-phase index builds".
As of 4.4, index builds in a replica set use a two-phase commit protocol. When a primary starts an
index build, it spawns a background thread and replicates a `startIndexBuild` oplog entry. Secondary
nodes will start the index build in the background as soon as they apply that oplog entry. When a
primary is done with its indexing, it will decide to replicate either an `abortIndexBuild` or
`commitIndexBuild` oplog entry.
Simultaneous index builds are resilient to replica set state transitions. The node that starts an
index build does not need to be the same node that decides to commit it.
See [Index Builds in Replicated Environments - MongoDB
Manual](https://docs.mongodb.com/master/core/index-creation/#index-builds-in-replicated-environments).
Server 7.1 introduces the following improvements:
- Index builds abort immediately after detecting errors other than duplicate key violations. Before
7.1, index builds aborted the index build close to completion, potentially long after detection.
- A secondary member can abort a two-phase index build. Before 7.1, a secondary was forced to crash
instead. See the [Voting for Abort](#voting-for-abort) section.
- Index builds are cancelled if there isn't enough storage space available. See the [Disk
Space](#disk-space) section.
### Commit Quorum
The purpose of `commitQuorum` is to ensure secondaries are ready to commit an index build quickly.
This minimizes replication lag on secondaries: secondaries, on receipt of a `commitIndexBuild` oplog
entry, will stall oplog application until the local index build can be committed. `commitQuorum`
delays commit of an index build on the primary node until secondaries are also ready to commit. A
primary will not commit an index build until a minimum number of data-bearing nodes are ready to
commit the index build. Index builds can take anywhere from moments to days to complete, so the
replication lag can be very significant. Note: `commitQuorum` makes no guarantee that indexes on
secondaries are ready for use when the command completes, `writeConcern` must still be used for
that.
A `commitQuorum` option can be provided to the `createIndexes` command and specifies the number of
nodes, including itself, for which a primary must wait to be ready before committing. The
`commitQuorum` option accepts the same range of values as the writeConcern `"w"` option. This can be
an integer specifying the number of nodes, `"majority"`, `"votingMembers"`, or a replica set tag.
The default value is `"votingMembers"`, or all voting data-bearing nodes.
Nodes (both primary and secondary) submit votes to the primary when they have finished scanning all
data on a collection and performed the first drain of side-writes. Voting is implemented by a
`voteCommitIndexBuild` command, and is persisted as a write to the replicated
`config.system.indexBuilds` collection.
While waiting for a commit decision, primaries and secondaries continue receiving and applying new
side writes. When a quorum is reached, the current primary, under a collection X lock, will check
the remaining index constraints. If there are errors, it will replicate an `abortIndexBuild` oplog
entry. If the index build is successful, it will replicate a `commitIndexBuild` oplog entry.
Secondaries that were not included in the commit quorum and receive a `commitIndexBuild` oplog entry
will block replication until their index build is complete.
The `commitQuorum` for a running index build may be changed by the user via the
[`setIndexCommitQuorum`](https://github.com/mongodb/mongo/blob/v6.0/src/mongo/db/commands/set_index_commit_quorum_command.cpp#L55)
server command.
See
[IndexBuildsCoordinator::\_waitForNextIndexBuildActionAndCommit](https://github.com/mongodb/mongo/blob/r4.4.0-rc9/src/mongo/db/index_builds_coordinator_mongod.cpp#L632).
### Voting for Abort
As of 7.1, a secondary can abort a two-phase index build by sending a `voteAbortIndexBuild` signal
to the primary. In contrast, before 7.1 it was forced to crash. Common causes for aborting the index
build are a killOp on the index build or running low on storage space. The primary, upon receiving a
vote to abort the index build from a secondary, will replicate an `abortIndexBuild` oplog entry.
This will cause all secondaries to gracefully abort the index build, even if a specific secondary
had already voted to commit the index build.
Note that once a secondary has voted to commit the index build, it cannot retract the vote. In the
unlikely event that a secondary has voted for commit and for some reason it must abort while waiting
for the primary to replicate a `commitIndexBuild` oplog entry, the secondary is forced to crash.
### Disk Space
As of 7.1, an index build can abort due to a replica set member running low on disk space. This
applies both to primary and secondary nodes. Additionally, on a primary the index build won't start
if the available disk space is low. The minimum amount of disk space is controlled by
[indexBuildMinAvailableDiskSpaceMB](https://github.com/mongodb/mongo/blob/406e69f6f5dee8b698c4e4308de2e9e5cef6c12c/src/mongo/db/storage/two_phase_index_build_knobs.idl#L71)
which defaults to 500MB.
## Resumable Index Builds
On clean shutdown, index builds save their progress in internal idents that will be used for
resuming the index builds when the server starts up. The persisted information includes:
- [Phase of the index
build](https://github.com/mongodb/mongo/blob/0d45dd9d7ba9d3a1557217a998ad31c68a897d47/src/mongo/db/resumable_index_builds.idl#L43)
when it was interrupted for shutdown:
- initialized
- collection scan
- bulk load
- drain writes
- Information relevant to the phase for reconstructing the internal state of the index build at
startup. This may include:
- The internal state of the external sorter.
- Idents for side writes, duplicate keys, and skipped records.
During [startup recovery](../storgae/README.md#startup-recovery), the persisted information is used
to reconstruct the in-memory state for the index build and resume from the phase that we left off
in. If we fail to resume the index build for whatever reason, the index build will restart from the
beginning.
Not all incomplete index builds are resumable upon restart. The current criteria for index build
resumability can be found in
[IndexBuildsCoordinator::isIndexBuildResumable()](https://github.com/mongodb/mongo/blob/0d45dd9d7ba9d3a1557217a998ad31c68a897d47/src/mongo/db/index_builds_coordinator.cpp#L375).
Generally, index builds are resumable under the following conditions:
- Storage engine is configured to be persistent with encryption disabled.
- The index build is running on a voting member of the replica set with the default [commit
quorum](#commit-quorum) `"votingMembers"`.
- Majority read concern is enabled.
The [Recover To A Timestamp (RTT) rollback
algorithm](https://github.com/mongodb/mongo/blob/04b12743cbdcfea11b339e6ad21fc24dec8f6539/src/mongo/db/repl/README.md#rollback)
supports resuming index builds interrupted at any phase. On entering rollback, the resumable index
information is persisted to disk using the same mechanism as shutdown. We resume the index build
using the startup recovery logic that RTT uses to bring the node back to a writable state.
For improved rollback semantics, resumable index builds require a majority read cursor during
collection scan phase. Index builds wait for the majority commit point to advance before starting
the collection scan. The majority wait happens after installing the [side table for intercepting new
writes](#temporary-side-table-for-new-writes).
See
[MultiIndexBlock::\_constructStateObject()](https://github.com/mongodb/mongo/blob/0d45dd9d7ba9d3a1557217a998ad31c68a897d47/src/mongo/db/catalog/multi_index_block.cpp#L900)
for where we persist the relevant information necessary to resume the index build at shutdown and
[StorageEngineImpl::\_handleInternalIdents()](https://github.com/mongodb/mongo/blob/0d45dd9d7ba9d3a1557217a998ad31c68a897d47/src/mongo/db/storage/storage_engine_impl.cpp#L329)
for where we search for and parse the resume information on startup.
## Single-Phase Index Builds
Index builds on empty collections replicate a `createIndexes` oplog entry. This oplog entry was used
before FCV 4.4 for all index builds, but continues to be used in FCV 4.4 only for index builds that
are considered "single-phase" and do not need to run in the background. Unlike two-phase index
builds, the `createIndexes` oplog entry is always applied synchronously on secondaries during batch
application.
See
[createIndexForApplyOps](https://github.com/mongodb/mongo/blob/6ea7d1923619b600ea0f16d7ea6e82369f288fd4/src/mongo/db/repl/oplog.cpp#L176-L183).
## Memory Limits
The maximum amount of memory allowed for an index build is controlled by the
`maxIndexBuildMemoryUsageMegabytes` server parameter. The sorter is passed this value and uses it to
regulate when to write a chunk of sorted data out to disk in a temporary file. The sorter keeps
track of the chunks of data spilled to disk using one Iterator for each spill. The memory needed for
the iterators is taken out of the `maxIndexBuildMemoryUsageMegabytes` and it is a percentage of
`maxIndexBuildMemoryUsageMegabytes` define by the `maxIteratorsMemoryUsagePercentage` server
parameter with minimum value enough to store one iterator and maximum value 1MB.

View File

@ -2125,6 +2125,90 @@ the first stable checkpoint, it can safely recover from the last unstable checkp
appliedThrough value. Otherwise, if this node crashes after the first stable checkpoint is taken,
it can safely recover from a stable checkpoint (with a cleared "appliedThrough").
# Flow Control
The Flow Control mechanism aims to keep replica set majority committed lag less than or equal to a
configured maximum. The default value for this maximum lag is 10 seconds. The Flow Control mechanism
starts throttling writes on the primary once the majority committed replication lag reaches a
threshold percentage of the configured maximum. The mechanism uses a "ticket admission"-based
approach to throttle writes. With this mechanism, in a given period of 1 second, a fixed number of
"flow control tickets" is available. Operations must acquire a flow control ticket in order to
acquire a global IX lock to execute a write. Acquisition attempts that occur after this fixed number
has been granted will stall until the next 1 second period. Certain system operations circumvent the
ticket admission mechanism and are allowed to proceed even when there are no tickets available.
To address the possibility of this Flow Control mechanism causing indefinite stalls in
Primary-Secondary-Arbiter replica sets in which a majority cannot be established, the mechanism only
executes when read concern majority is enabled. Additionally, the mechanism can be disabled by an
admin.
Flow Control is configurable via several server parameters. Additionally, currentOp, serverStatus,
database profiling, and slow op log lines include Flow Control information.
## Flow Control Ticket Admission Mechanism
The ticket admission Flow Control mechanism allows a specified number of global IX lock acquisitions
every second. Most global IX lock acquisitions (except for those that explicitly circumvent Flow
Control) must first acquire a "Flow Control ticket" before acquiring a ticket for the lock. When
there are no more flow control tickets available in a one second period, remaining attempts to
acquire flow control tickets stall until the next period, when the available flow control tickets
are replenished. It should be noted that there is no "pool" of flow control tickets that threads
give and take from; an independent mechanism refreshes the ticket counts every second.
When the Flow Control mechanism refreshes available tickets, it calculates how many tickets it
should allow in order to address the majority committed lag.
The Flow Control mechanism determines how many flow control tickets to replenish every period based
on:
1. The current majority committed replication lag with respect to the configured target maximum
replication lag
1. How many operations the secondary sustaining the commit point has applied in the last period
1. How many IX locks per operation were acquired in the last period
## Configurable constants
Criterion #2 determines a "base" number of flow control tickets to be used in the calculation. When
the current majority committed lag is greater than or equal to a certain configurable threshold
percentage of the target maximum, the Flow Control mechanism scales down this "base" number based on
the discrepancy between the two lag values. For some configurable constant 0 < k < 1, it calculates
the following:
`base * k ^ ((lag - threshold)/threshold) * fudge factor`
The fudge factor is also configurable and should be close to 1. Its purpose is to assign slightly
lower than the "base" number of flow control tickets when the current lag is close to the threshold.
Criterion #3 is then multiplied by the result of the above calculation to translate a count of
operations into a count of lock acquisitions.
When the majority committed lag is less than the threshold percentage of the target maximum, the
number of tickets assigned in the previous period is used as the "base" of the calculation. This
number is added to a configurable constant (the ticket "adder" constant), and the sum is multiplied
by another configurable constant (the ticket "multiplier" constant). This product is the new number
of tickets to be assigned in the next period.
When the Flow Control mechanism is disabled, the ticket refresher mechanism always allows one
billion flow control ticket acquisitions per second. The Flow Control mechanism can be disabled via
a server parameter. Additionally, the mechanism is disabled on nodes that cannot accept writes.
Criteria #2 and #3 are determined using a sampling mechanism that periodically stores the necessary
data as primaries process writes. The sampling mechanism executes regardless of whether Flow Control
is enabled.
## Oscillations
There are known scenarios in which the Flow Control mechanism causes write throughput to
oscillate. There is no known work that can be done to eliminate oscillations entirely for this
mechanism without hindering other aspects of the mechanism. Work was done (see SERVER-39867) to
dampen the oscillations at the expense of throughput.
## Throttling internal operations
The Flow Control mechanism throttles all IX lock acquisitions regardless of whether they are from
client or system operations unless they are part of an operation that is explicitly excluded from
Flow Control. Writes that occur as part of replica set elections in particular are excluded. See
SERVER-39868 for more details.
# Feature Compatibility Version
See the [FCV and Feature Flag README](FCV_AND_FEATURE_FLAG_README.md).

View File

@ -0,0 +1,15 @@
# The External Sorter
The external sorter is a MongoDB component that sorts large volumes of data, spilling in-memory data
to disk in order to bound its memory consumption as needed. It is used to sort documents read from
disk for the purposes of index creation and sorted query results. Index creation must read out all
of a collection's documents, generate index keys, sort them for the new index, and finally write the
sorted index keys out to the new index. A query with sorted results that do not match any index
ordering must read all the documents matching its specifications and then sort the results according
to the ordering specifications before returning the sorted results to the user.
The amount of data that is handled for both of these operations can be too large to keep in memory.
Therefore, the data is iteratively read from the storage engine, sorted, and written out to
temporary files whenever / if the maximum user adjustable memory limit is reached. Then the sorted
blocks of entries are iteratively read back into memory (if needed), following the coalescing phase
of the merge sort algorithm, and streamed to their final destination.

View File

@ -0,0 +1,139 @@
# Operation Resource Consumption Metrics
MongoDB supports collecting per-operation resource consumption metrics. These metrics reflect the
impact operations have on the server. They may be aggregated per-database and queried by an
aggregation pipeline stage `$operationMetrics`.
Per-operation metrics collection may be enabled with the
`profileOperationResourceConsumptionMetrics` server parameter (default off). When the parameter is
enabled, operations collect resource consumption metrics and report them in the slow query logs. If
profiling is enabled, these metrics are also profiled.
Per-database aggregation may be enabled with the `aggregateOperationResourceConsumptionMetrics`
server parameter (default off). When this parameter is enabled, in addition to the behavior
described by the profiling server parameter, operations will accumulate metrics to global in-memory
per-database counters upon completion. Aggregated metrics may be queried by using the
`$operationMetrics` aggregation pipeline stage. This stage returns an iterable, copied snapshot of
all metrics, where each document reports metrics for a single database.
Metrics are not cleared for dropped databases, which introduces the potential to slowly leak memory
over time. Metrics may be cleared globally by supplying the `clearMetrics: true` flag to the
pipeline stage or restarting the process.
## Limitations
Metrics are not collected for all operations. The following limitations apply:
- Only operations from user connections collect metrics. For example, internal connections from
other replica set members do not collect metrics.
- Metrics are only collected for a specific set of commands. Those commands override the function
`Command::collectsResourceConsumptionMetrics()`.
- Metrics for write operations are only collected on primary nodes.
- This includes TTL index deletions.
- All attempted write operations collect metrics. This includes writes that fail or retry internally
due to write conflicts.
- Read operations are attributed to the replication state of a node. Read metrics are broken down
into whether they occurred in the primary or secondary replication states.
- Index builds collect metrics. Because index builds survive replication state transitions, they
only record aggregated metrics if the node is currently primary when the index build completes.
- Metrics are not collected on `mongos` and are not supported or tested in sharded environments.
- Storage engines other than WiredTiger do not implement metrics collection.
- Metrics are not adjusted after replication rollback.
## Document and Index Entry Units
In addition to reporting the number of bytes read to and written from the storage engine, MongoDB
reports certain derived metrics: units read and units written.
Document units and index entry units are metric calculations that attempt to account for the
overhead of performing storage engine work by overstating operations on smaller documents and index
entries. For each observed datum, a document or index entry, a unit is calculated as the following:
```
units = ceil (datum bytes / unit size in bytes)
```
This has the tendency to overstate small datums when the unit size is large. These unit sizes are
tunable with the server parameters `documentUnitSizeBytes` and `indexEntryUnitSizeBytes`.
## Total Write Units
For writes, the code also calculates a special combined document and index unit. The code attempts
to associate index writes with an associated document write, and takes those bytes collectively to
calculate units. For each set of bytes written, a unit is calculated as the following:
```
units = ceil (set bytes / unit size in bytes)
```
To associate index writes with document writes, the algorithm is the following:
Within a storage transaction, if a document write precedes as-yet-unassigned index writes, assign
such index bytes with the preceding document bytes, up until the next document write.
If a document write follows as-yet-unassigned index writes, assign such index bytes with the
following document bytes.
The `totalUnitWriteSizeBytes` server parameter affects the unit calculation size for the above
calculation.
## CPU Time
Operations that collect metrics will also collect the amount of active CPU time spent on the command
thread. This is reported as `cpuNanos` and is provided by the `OperationCPUTimer`.
The CPU time metric is only supported on certain flavors of Linux. It is implemented using
`clock_gettime` and `CLOCK_THREAD_CPUTIME_ID`, which has limitations on certain systems. See the
[man page for clock_gettime()](https://linux.die.net/man/3/clock_gettime).
## Example output
The $operationMetrics stage behaves like any other pipeline cursor, and will have the following
schema, per returned document:
```
{
db: "<dbname>",
// Metrics recorded while the node was PRIMARY. Summed with secondaryMetrics metrics gives total
// metrics in all replication states.
primaryMetrics: {
// The number of document bytes read from the storage engine
docBytesRead: 0,
// The number of document units read from the storage engine
docUnitsRead: 0,
// The number of index entry bytes read from the storage engine
idxEntryBytesRead: 0,
// The number of index entry units read from the storage engine
idxEntryUnitsRead: 0,
// The number of document units returned by query operations
docUnitsReturned: 0,
// These fields are ALWAYS ZERO and only present for backwards compatibility:
cursorSeeks: 0,
keysSorted: 0,
sorterSpills: 0,
},
// Metrics recorded while the node was SECONDARY
secondaryMetrics: {
docBytesRead: 0,
docUnitsRead: 0,
idxEntryBytesRead: 0,
idxEntryUnitsRead: 0,
docUnitsReturned: 0,
// These fields are ALWAYS ZERO and only present for backwards compatibility:
cursorSeeks: 0,
keysSorted: 0,
sorterSpills: 0,
},
// The amount of active CPU time used by all operations
cpuNanos: 0,
// The number of document bytes attempted to be written to or deleted from the storage engine
docBytesWritten: 0,
// The number of document units attempted to be written to or deleted from the storage engine
docUnitsWritten: 0,
// The number of index entry bytes attempted to be written to or deleted from the storage engine
idxEntryBytesWritten: 0,
// The number of index entry units attempted to be written to or deleted from the storage engine
idxEntryUnitsWritten: 0,
// The total number of document plus associated index entry units attempted to be written to
// or deleted from the storage engine
totalUnitsWritten: 0
}
```

View File

@ -1,96 +1,102 @@
# Storage Engine API
The purpose of the Storage Engine API is to allow for pluggable storage engines in MongoDB (refer
to the [Storage FAQ][]). This document gives a brief overview of the API, and provides pointers
to places with more detailed documentation. Where referencing code, links are to the version that
was current at the time when the reference was made. Always compare with the latest version for
changes not yet reflected here. For questions on the API that are not addressed by this material,
use the [mongodb-dev][] Google group. Everybody involved in the Storage Engine API will read your
post.
The purpose of the Storage Engine API is to allow for pluggable storage engines in MongoDB (refer to
the [Storage FAQ][]). This document gives a brief overview of the API, and provides pointers to
places with more detailed documentation.
Third-party storage engines are integrated through self-contained modules that can be dropped into
an existing MongoDB source tree, and will be automatically configured and included. A typical
module would at least have the following files:
an existing MongoDB source tree, and will be automatically configured and included.
src/ Directory with the actual source files
README.md Information specific to the storage engine
SConscript Scons build rules
build.py Module configuration script
For more context and information on how this API is used, see the [Catalog](../catalog/README.md).
See <https://github.com/mongodb-partners/mongo-rocks> for a good example of the structure.
For more context and information on how this API is used, see the
[Execution Architecture Guide](https://github.com/mongodb/mongo/blob/master/src/mongo/db/catalog/README.md).
## Concepts
### Record Stores
## Record Stores
A database contains one or more collections, each with a number of indexes, and a catalog listing
them. All MongoDB collections are implemented with record stores: one for the documents themselves,
and one for each index. By using the KVEngine class, you only have to deal with the abstraction, as
the StorageEngineImpl implements the StorageEngine interface, using record stores for catalogs and
indexes.
them. All MongoDB collections are implemented with a [RecordStore](record_store.h) and indexes are
implemented with a [SortedDataInterface](sorted_data_interface.h). By using the
[KVEngine](kv/kv_engine.h) class, you only have to deal with the abstraction, as the
[StorageEngineImpl](storage_engine_impl.h) implements the [StorageEngine](storage_engine.h)
interface, using record stores for catalogs. See the [Catalog](../catalog/README) for more information.
#### Record Identities
### Record Identities
A RecordId is a unique identifier, assigned by the storage engine, for a specific document or entry
in a record store at a given time. For storage engines based in the KVEngine the record identity is
fixed, but other storage engines may change it when updating a document. Note that changing record
ids can be very expensive, as indexes map to the RecordId. A single document with a large array may
have thousands of index entries, resulting in very expensive updates.
A [RecordId](record_id.h) is a unique identifier, assigned by the storage engine, for a specific
document or entry in a record store at a given time. For storage engines based in the KVEngine, the
record identity is fixed, but other storage engines may change it when updating a document. Note
that changing record ids can be very expensive, as indexes map to the RecordId. A single document
with a large array may have thousands of index entries, resulting in very expensive updates.
#### Cloning and bulk operations
Currently all cloning, [initial sync][] and other operations are done in terms of operating on
individual documents, though there is a BulkBuilder class for more efficiently building indexes.
### Locking and Concurrency
## Locking and Concurrency
MongoDB uses multi-granular intent locking; see the [Concurrency FAQ][]. In all cases, this will
ensure that operations to meta-data, such as creation and deletion of record stores, are serialized
with respect to other accesses.
MongoDB uses [two-phase locking][] (2PL) to guarantee serializability of accesses to resources it
manages. MongoDB will only use intent locks for the most common operations, leaving synchronization
at the record store layer up to the storage engine.
See the [Catalog](../catalog/README) and [Concurrency Control](../concurrency/README.md) for more information.
### Transactions
## Transactions
Each operation creates an OperationContext with a new RecoveryUnit, implemented by the storage
engine, that lives until the operation finishes. Currently, query operations that return a cursor
to the client live as long as that client cursor, with the operation context switching between its
own recovery unit and that of the client cursor. In a few other cases an internal command may use
an extra recovery unit as well. The recovery unit must implement transaction semantics as described
below.
Operations use a [RecoveryUnit](recovery_unit.h), implemented by the storage engine, to provide
transaction semantics as described below.
#### Atomicity
### RecoveryUnit Lifetime
Most operations create a [RecoveryUnit](recovery_unit.h) upon creation of their OperationContext
that exists for the lifetime of the operation. Query operations that return a cursor to the client
hold an idle recovery unit as long as that client cursor, with the operation context switching
between its own recovery unit and that of the client cursor. User multi-statement transactions
switch an active recovery unit onto operation contexts used to perform transaction operations.
In some rare cases, operations may use more than one recovery unit at a time.
### RecoveryUnit
Each pluggable storage engine for MongoDB must implement `RecoveryUnit` as one of the base classes
for the storage engine API. Typically, storage engines satisfy the `RecoveryUnit` requirements with
some form of [snapshot isolation](#glossary) with transactions. Such transactions are called storage
transactions elsewhere in this document, to differentiate them from the higher-level _multi-document
transactions_ accessible to users of MongoDB. The RecoveryUnit manages the lifetime of a storage
engine [snapshot](#glossary). MongoDB does not always explicitly open snapshots, therefore
RecoveryUnits implicitly open snapshots on the first read or write operation.
### Timestamps
In MongoDB, a snapshot can be opened with or without a _timestamp_ when using a
[ReadSource](https://github.com/mongodb/mongo/blob/b2c1fa4f121fdb6cdffa924b802271d68c3367a3/src/mongo/db/storage/recovery_unit.h#L391-L421)
that uses timestamps. The snapshot will return all data committed with a timestamp less than or
equal to the snapshot's timestamp. No uncommitted data is visible in a snapshot, and data changes in
storage transactions that commit after a snapshot is created, regardless of their timestamps, are
also not visible. Generally, one uses a `RecoveryUnit` to perform transactional reads and writes by
first configuring the `RecoveryUnit` with the desired `ReadSource` and then performing the reads and
writes using operations on `RecordStore` or `SortedDataInterface`.
### Atomicity
Writes must only become visible when explicitly committed, and in that case all pending writes
become visible atomically. Writes that are not committed before the unit of work ends must be
rolled back. In addition to writes done directly through the Storage API, such as document updates
and creation of record stores, other custom changes can be registered with the recovery unit.
become visible atomically. Writes that are not committed before the unit of work ends must be rolled
back. In addition to writes done directly through the Storage API, such as document updates and
creation of record stores, other custom changes can be registered with the recovery unit.
#### Consistency
### Consistency
Storage engines must ensure that atomicity and isolation guarantees span all record stores, as
otherwise the guarantee of atomic updates on a document and all its indexes would be violated.
#### Isolation
### Isolation
Storage engines must provide snapshot isolation, either through locking, through multi-version
concurrency control (MVCC) or otherwise. The first read implicitly establishes the snapshot.
Operations can always see all changes they make in the context of a recovery unit, but other
operations cannot until a successful commit.
#### Durability
### Durability
Once a transaction is committed, it is not necessarily durable: if, and only if the server fails,
as result of power loss or otherwise, the database may recover to an earlier point in time.
However, atomicity of transactions must remain preserved. Similarly, in a replica set, a primary
that becomes unavailable may need to roll back to an earlier state when rejoining the replica set,
if its changes were not yet seen by a majority of nodes. The RecoveryUnit implements methods to
allow operations to wait for their committed transactions to become durable.
Once a transaction is committed, it is not necessarily durable: if, and only if the server fails, as
result of power loss or otherwise, the database may recover to an earlier point in time. However,
atomicity of transactions must remain preserved. Similarly, in a replica set, a primary that becomes
unavailable may need to roll back to an earlier state when rejoining the replica set, if its changes
were not yet seen by a majority of nodes. The RecoveryUnit implements methods to allow operations to
wait for their committed transactions to become durable.
A transaction may become visible to other transactions as soon as it commits, and a storage engine
may use a group commit, bundling a number of transactions to achieve durability. Alternatively, a
@ -101,17 +107,17 @@ storage engine may wait for durability at commit time.
Systems with optimistic concurrency control (OCC) or multi-version concurrency control (MVCC) may
find that a transaction conflicts with other transactions, that executing an operation would result
in deadlock or violate other resource constraints. In such cases the storage engine may throw a
WriteConflictException to signal the transient failure. MongoDB will handle the exception, abort
and restart the transaction.
WriteConflictException to signal the transient failure. MongoDB will handle the exception, abort and
restart the transaction.
### Point-in-time snapshot reads
Two functions on the RecoveryUnit help storage engines implement point-in-time reads: setTimestamp()
and selectSnapshot(). setTimestamp() is used by write transactions to label any forthcoming writes
with a timestamp; these timestamps are then used to produce a point-in-time read transaction via a
call to selectSnapshot() at the start of the read. The storage engine must produce the effect of
reading from a snapshot that includes only writes with timestamps at or earlier than the
selectSnapshot timestamp. This means that a point-in-time read may slice across prior write
Two functions on the RecoveryUnit help storage engines implement point-in-time reads:
`setTimestamp/setCommitTimestamp()` and `setTimestampReadSource()`. `setTimestamp()` is used by
write transactions to label any forthcoming writes with a timestamp. Future readers can then use a
`ReadSource` before starting any reads or writes to only see writes. The storage engine must produce
the effect of reading from a snapshot that includes only writes with timestamps at or earlier than
the selectSnapshot timestamp. This means that a point-in-time read may slice across prior write
transactions by hiding only some data from a given write transaction, if that transaction had a
different timestamp set prior to each write it did.
@ -126,11 +132,504 @@ details.
- [SeekableRecordCursor](record_store.h)
- [SortedDataInterface](sorted_data_interface.h)
- [ServerStatusSection](../commands/server_status.h)
- [ServerParameter](../server_parameters.h)
[Concurrency FAQ]: http://docs.mongodb.org/manual/faq/concurrency/
[initial sync]: http://docs.mongodb.org/manual/core/replica-set-sync/#replica-set-initial-sync
[mongodb-dev]: https://groups.google.com/forum/#!forum/mongodb-dev
[replica set]: http://docs.mongodb.org/manual/replication/
[Storage FAQ]: http://docs.mongodb.org/manual/faq/storage
[two-phase locking]: http://en.wikipedia.org/wiki/Two-phase_locking
# WriteUnitOfWork
A `WriteUnitOfWork` is the mechanism to control how writes are transactionally performed on the
storage engine. All the writes (and reads) performed within its scope are part of the same storage
transaction. After all writes have been staged, caller must call `commit()` in order to atomically
commit the transaction to the storage engine. It is illegal to perform writes outside the scope of a
WriteUnitOfWork since there would be no way to commit them. If the `WriteUnitOfWork` falls out of
scope before `commit()` is called, the storage transaction is rolled back and all the staged writes
are lost. Reads can be performed outside of a `WriteUnitOfWork` block; storage transactions outside
of a `WriteUnitOfWork` are always rolled back, since there are no writes to commit.
The WriteUnitOfWork has a [`groupOplogEntries` option](https://github.com/mongodb/mongo/blob/fa32d665bd63de7a9d246fa99df5e30840a931de/src/mongo/db/storage/write_unit_of_work.h#L67)
to replicate multiple writes transactionally. This option uses the [`BatchedWriteContext` class](https://github.com/mongodb/mongo/blob/9ab71f9b2fac1e384529fafaf2a819ce61834228/src/mongo/db/batched_write_context.h#L46)
to stage writes and to generate a single applyOps entry at commit, similar to what multi-document
transactions do via the [`TransactionParticipant` class](https://github.com/mongodb/mongo/blob/219990f17695b0ea4695f827a42a18e012b1e9cf/src/mongo/db/transaction/transaction_participant.h#L82).
Unlike a multi-document transaction, the applyOps entry lacks the `lsId` and the `txnNumber`
fields. Callers must ensure that the WriteUnitOfWork does not generate more than 16MB of oplog,
otherwise the operation will fail with `TransactionTooLarge` code.
As of MongoDB 6.0, the `groupOplogEntries` mode is only used by the [BatchedDeleteStage](https://github.com/mongodb/mongo/blob/9676cf4ad8d537518eb1b570fc79bad4f31d8a79/src/mongo/db/exec/batched_delete_stage.h)
for efficient mass-deletes.
See
[WriteUnitOfWork](https://github.com/mongodb/mongo/blob/fa32d665bd63de7a9d246fa99df5e30840a931de/src/mongo/db/storage/write_unit_of_work.h).
See
## Lazy initialization of storage transactions
Note that storage transactions on WiredTiger are not started at the beginning of a `WriteUnitOfWork`
block. Instead, the transaction is started implicitly with the first read or write operation. To
explicitly start a transaction, one can use `RecoveryUnit::preallocateSnapshot()`.
## Changes
One can register a `Change` on a `RecoveryUnit` while in a `WriteUnitOfWork`. This allows extra
actions to be performed based on whether a `WriteUnitOfWork` commits or rolls back. These actions
will typically update in-memory state to match what was written in the storage transaction, in a
transactional way. Note that `Change`s are not executed until the destruction of the
`WriteUnitOfWork`, which can be long after the storage engine committed. Two-phase locking ensures
that all locks are held while a Change's `commit()` or `rollback()` function runs.
# StorageUnavailableException
`StorageUnavailableException` indicates that a storage transaction rolled back due to resource
contention in the storage engine. This exception is the base of exceptions related to concurrency
(`WriteConflict`) and to those related to cache pressure (`TemporarilyUnavailable` and
`TransactionTooLargeForCache`).
We recommend using the
[writeConflictRetry](https://github.com/mongodb/mongo/blob/9381db6748aada1d9a0056cea0e9899301e7f70b/src/mongo/db/concurrency/exception_util.h#L140)
helper which transparently handles all exceptions related to this error category.
## WriteConflictException
Writers may conflict with each other when more than one operation stages an uncommitted write to the
same document concurrently. To force one or more of the writers to retry, the storage engine may
throw a WriteConflictException at any point, up to and including the call to commit(). This is
referred to as optimistic concurrency control because it allows un-contended writes to commit
quickly. Because of this behavior, most WUOWs are enclosed in a writeConflictRetry loop that retries
the write transaction until it succeeds, accompanied by a bounded exponential back-off.
## TemporarilyUnavailableException
When the server parameter `enableTemporarilyUnavailableExceptions` is enabled (on by default), a
TemporarilyUnavailableException may be thrown inside the server to indicate that an operation cannot
complete without blocking and must be retried. The storage engine may throw a
TemporarilyUnavailableException (converted to a TemporarilyUnavailable error for users) when an
operation is excessively rolled-back in the storage engine due to cache pressure or any reason that
would prevent the operation from completing without impacting concurrent operations. The operation
may be at fault for writing too much uncommitted data, or it may be a victim. That information is
not exposed. However, if this error is returned, it is likely that the operation was the cause of
the problem, rather than a victim.
Before 6.0, this type of error was returned as a WriteConflict and retried indefinitely inside a
writeConflictRetry loop. As of 6.0, MongoDB will retry the operation internally at most
`temporarilyUnavailableMaxRetries` times, backing off for `temporarilyUnavailableBackoffBaseMs`
milliseconds, with a linearly-increasing backoff on each attempt. After this point, the error will
escape the handler and be returned to the client.
If an operation receives a TemporarilyUnavailable error internally, a `temporarilyUnavailableErrors`
counter will be displayed in the slow query logs and in FTDC.
Notably, this behavior does not apply to multi-document transactions, which continue to return a
WriteConflict to the client in this scenario without retrying internally.
See
[TemporarilyUnavailableException](https://github.com/mongodb/mongo/blob/c799851554dc01493d35b43701416e9c78b3665c/src/mongo/db/concurrency/temporarily_unavailable_exception.h#L39-L45).
## TransactionTooLargeForCacheException
A TransactionTooLargeForCacheException may be thrown inside the server to indicate that an operation
was rolled-back and is unlikely to ever complete because the storage engine cache is insufficient,
even in the absence of concurrent operations. This is determined by a simple heuristic wherein,
after a rollback, a threshold on the proportion of total dirty cache bytes the running transaction
can represent and still be considered fullfillable is checked. The threshold can be tuned with the
`transactionTooLargeForCacheThreshold` parameter. Setting this threshold to its maximum value (1.0)
causes the check to be skipped and TransactionTooLargeForCacheException to be disabled.
On replica sets, if an operation succeeds on a primary, it should also succeed on a secondary. It
would be possible to convert to both TemporarilyUnavailableException and WriteConflictException, as
if TransactionTooLargeForCacheException was disabled. But on secondaries the only difference between
the two is the rate at which the operation is retried. Hence, TransactionTooLargeForCacheException
is always converted to a WriteConflictException, which retries faster, to avoid stalling replication
longer than necessary.
Prior to 6.3, or when TransactionTooLargeForCacheException is disabled, multi-document transactions
always return a WriteConflictException, which may result in drivers retrying an operation
indefinitely. For non-multi-document operations, there is a limited number of retries on
TemporarilyUnavailableException, but it might still be beneficial to not retry operations which are
unlikely to complete and are disruptive for concurrent operations.
# Idents
An ident is a unique identifier given to a storage engine resource. Collections and indexes map
application-layer names to storage engine idents. In WiredTiger, idents are implemented as tables
and, each with a `.wt` file extension.
Examples in the WiredTiger storage engine:
- collection idents: `collection-<counter>-<random number>`
- index idents: `index-<counter>-<random number>`
Server flags that alter the form of idents (this applies to indexes as well):
- `--directoryperdb`: `<db name>/collection-<counter>-<random number>`
- `--wiredTigerDirectoryForIndexes`: `collection/<counter>-<random number>`
- (both of the above): `<db name>/collection/<counter-<random number>`
# Startup Recovery
There are three components to startup recovery. The first step, of course, is starting the storage
engine. More detail about WiredTiger's startup recovery procedure can be found
[here](wiredtiger/README.md#startup-recovery).
The other two parts of storage startup recovery bring the [catalog](../catalog/README.md) back into
a consistent state. The catalog typically refers to MongoDB's notion of collections and indexes, but
it's important to note that storage engines such as WiredTiger have their own notion of a catalog.
The first step of recovering the catalog is to bring MongoDB's catalog in line with the storage
engine's. This is called reconciliation. Except for rare cases, every MongoDB collection is a
RecordStore and a list of indexes (aka SortedDataInterface). Every record store and index maps to
their own ident. [The WiredTiger README](wiredtiger/README.md) describes the relationship between
creating/dropping a collection and the underlying creation/deletion of a table which justifies the
following logic. In short, the following logic is necessary because not all storage engines can
create and drop idents transactionally. When reconciling, every ident that is not "pointed to" by a
MongoDB record store or index [gets
dropped](https://github.com/mongodb/mongo/blob/6c9adc9a2d518fa046c7739e043a568f9bee6931/src/mongo/db/storage/storage_engine_impl.cpp#L663-L676 "Github"). A MongoDB record store that points to an ident that doesn't exist is considered [a fatal
error](https://github.com/mongodb/mongo/blob/6c9adc9a2d518fa046c7739e043a568f9bee6931/src/mongo/db/storage/storage_engine_impl.cpp#L679-L693 "Github"). An index that doesn't point to an ident is [ignored and
logged](https://github.com/mongodb/mongo/blob/6c9adc9a2d518fa046c7739e043a568f9bee6931/src/mongo/db/storage/storage_engine_impl.cpp#L734-L746 "Github") because there are certain cases where the catalog entry may reference an index ident which
is no longer present, such as when an unclean shutdown occurs before a checkpoint is taken during
startup recovery.
The second step of recovering the catalog is [reconciling unfinished index
builds](https://github.com/mongodb/mongo/blob/6c9adc9a2d518fa046c7739e043a568f9bee6931/src/mongo/db/storage/storage_engine_impl.cpp#L695-L699 "Github"), that could have different outcomes:
- An [index build with a
UUID](https://github.com/mongodb/mongo/blob/6c9adc9a2d518fa046c7739e043a568f9bee6931/src/mongo/db/storage/storage_engine_impl.cpp#L748-L751 "Github") is an unfinished two-phase build and must be restarted, unless we are [resuming
it](#resumable-index-builds). This resume information is stored in an internal ident written at
(clean) shutdown. If we fail to resume the index build, we will clean up the internal ident and
restart the index build in the background.
- An [unfinished index build on
standalone](https://github.com/mongodb/mongo/blob/6c9adc9a2d518fa046c7739e043a568f9bee6931/src/mongo/db/storage/storage_engine_impl.cpp#L792-L794 "Github") will be discarded (no oplog entry was ever written saying the index exists).
After storage completes its recovery, control is passed to [replication
recovery](../repl/README.md#startup-recovery). While storage recovery is responsible for recovering
the oplog to meet durability guarantees and getting the two catalogs in sync, replication recovery
takes responsibility for getting collection data in sync with the oplog. Replication starts
replaying oplog from the [recovery
timestamp](https://github.com/mongodb/mongo/blob/9d3db5a56a6163d4aefd77997784fed21cb2d50a/src/mongo/db/storage/storage_engine.h#L545C40-L545C51).
See the [WiredTiger README](wiredtiger/README.md#checkpoints) for more details.
# File-System Backups
Backups represent a full copy of the data files at a point-in-time. These copies of the data files
can be used to recover data from a consistent state at an earlier time. This technique is commonly
used after a disaster ensued in the database.
The data storage requirements for backups can be large as they correlate to the size of the
database. This can be alleviated by using a technique called incremental backups. Incremental
backups only back up the data that has changed since the last backup.
MongoDB instances used in production should have a strategy in place for capturing and restoring
backups in the case of data loss events.
[Documentation for further reading.](https://docs.mongodb.com/manual/core/backups/)
# Queryable Backup (Read-Only)
This is a feature provided by Ops Manager in which Ops Manager quickly and securely makes a given
snapshot accessible over a MongoDB connection string.
Queryable backups start-up quickly regardless of the snapshot's total data size. They are uniquely
useful for restoring a small subset of data, such as a document that was accidentally deleted or
reading out a single collection. Queryable backups allow access to the snapshot for read-only
operations.
# Checkpoints
Checkpoints provide recovery points that enable the database to load a consistent snapshot of the
data quickly during startup or after a failure. Checkpoints provide basic operation durability in
favor of fast recovery in the event of a crash.
Write-ahead logging, aka [journaling](#journaling), is used in addition to checkpointing to provide
commit-level durability for all operations since the last checkpoint. On startup, all journaled
writes are re-applied to the data from the last checkpoint. Without journaling, all writes between
checkpoints would be lost.
Storage engines need to [support
checkpoints](https://github.com/mongodb/mongo/blob/r4.5.0/src/mongo/db/storage/storage_engine.h#L267)
for MongoDB to take advantage of this, otherwise MongoDB will act as an ephemeral data store. The
frequency of these checkpoints is determined by the ['storage.syncPeriodSecs' or
'syncdelay'](https://github.com/mongodb/mongo/blob/r4.5.0/src/mongo/db/mongod_options_storage.idl#L86-L93)
options.
# Journaling
MongoDB provides write durability via a persisted change log for replicated writes and persistence
of non-replicated writes. The replicated change log and non-replicated collections in WiredTiger are
journaled, i.e. written out to disk. The user writes themselves, however, on a --replSet server, do
not need to be written out to disk to guarantee persistence on the server.
All replicated server writes have matching change log entries representing the changes done. The
change log is stored in the `local.oplog.rs` namespace, which is set up as a capped collection so
that old unneeded log entries are eventually removed. Replication uses the oplog collection to copy
data from one server to another.
Durability of journaled collections and indexes is done by periodic or triggered journal flushes
that specifically flush only journaled writes to disk. MongoDB can disable journaling, such as in
standalone mode, so that the periodic journal flushes do not occur. Instead,
[Checkpoints](#checkpoints), which flush all writes to disk regardless of journal settings, are
taken whenever durability of a write is requested and journaling is disabled. Syncing only journaled
collection entries to disk is cheaper than syncing all data writes.
Data durability is essential for recovery after server shutdown. Data must be persisted to disk to
survive process restart, either in the form of the journal or as the write itself. Server startup
recovery will open the storage engine at the last-made data checkpoint, and all of the journaled
writes flushed to disk will also be found even if they occurred after the last checkpoint. The
replication layer decides what to apply of the change log (oplog collection) past the checkpoint for
cross replica set data consistency. For example, the storage engine might recover data up to time
T9, but the journaled oplog recovered could go up to T20, say. It is a replication level decision to
apply (or not apply) T10 through T20.
See the [WiredTiger README](wiredtiger/README.md#Cherry-picked-WT-log-Details) for more details on
the implementation.
Code Links:
- [_The JournalFlusher
class_](https://github.com/mongodb/mongo/blob/767494374cf12d76fc74911d1d0fcc2bbce0cd6b/src/mongo/db/storage/control/journal_flusher.h)
- Periodically and upon request flushes the journal to disk.
# Fast Truncation on Internal Collections
Logical deletes aren't always performant enough to keep up with inserts. To solve this, several
internal collections use `CollectionTruncateMarkers` for fast, unreplicated and untimestamped
[truncation](http://source.wiredtiger.com/1.4.2/classwiredtiger_1_1_session.html#a80a9ee8697a61a1ad13d893d67d981bb)
of expired data, in lieu of logical document deletions.
## CollectionTruncateMarkers
CollectionTruncateMarkers are an in-memory tracking mechanism to support ranged truncates on a
collection.
A collection is broken up into a number of truncate markers. Each truncate marker tracks a range in
the collection. Newer entries not captured by a truncate marker are tracked by an in-progress
"partial marker".
```
CollectionTruncateMarkers
_______________________________________
| | ...... | | Partial Marker
|_____________|___________|_____________|__________
Oldest Marker Newest Marker
Min RecordId <------------------------------------------------<--- Max RecordId
Truncate Marker
_______________________________________
| . Last Record's RecordId |
| . Last Record's Wall Time |
| . Bytes in Marker |
| . Number of records in Marker |
|_______________________________________|
^
|
Last Record
Marks the end of the marker's range
Most recent record at the time of marker creation
```
A new truncate marker is created when either:
1. An insert causes the in-progress "partial marker" segment to contain more than the minimum bytes
needed for a truncate marker.
- The record inserted serves as the 'last record' of the newly created marker.
2. Partial marker expiration is supported, and an explicit call is made to transform the "partial
marker" into a complete truncate marker.
- Partial marker expiration is supported for change stream collections and ensures that expired
documents in a partial marker will eventually be truncated - even if writes to the namespace
cease and the partial marker never meets the minimum bytes requirement.
### Requirements & Properties
CollectionTruncateMarkers support collections that meet the following requirements:
- Insert and truncate only. No updates or individual document deletes.
- [Clustered](../catalog/README.md#clustered-collections) with no secondary indexes.
- RecordId's in Timestamp order.
- Deletion of content follows RecordId ordering.
- This is a general property of clustered capped collections.
Collections who use CollectionTruncateMarkers share the following properties:
- Fast counts aren't expected to be accurate.
- Truncates don't track the count and size of documents truncated in exchange for performance
gains.
- Markers are a best effort way to keep track of the size metrics and when to truncate expired
data.
- Collections aren't expected to be consistent between replica set members.
- Truncates are unreplicated, and nodes may truncate ranges at different times.
- No snapshot read concern support (ex:
[SERVER-78296](https://jira.mongodb.org/browse/SERVER-78296)).
- Deleting with untimestamped, unreplicated range truncation means point-in-time reads may see
inconsistent data.
Each collection utilizing CollectionTruncateMarkers must implement its [own
policy](https://github.com/mongodb/mongo/blob/r7.1.0-rc3/src/mongo/db/storage/collection_truncate_markers.h#L277)
to determine when there are excess markers and it is time for truncation.
### In-Memory Initialization
At or shortly after startup, an initial set of CollectionTruncateMarkers are created for each
collection. The collection is either scanned or sampled to generate initial markers. Initial
truncate markers are best effort, and may hold incorrect estimates about the number of documents and
bytes within each marker. Eventually, once the initial truncate markers expire, per truncate marker
metrics will converge closer to the correct values.
### Collections that use CollectionTruncateMarkers
- The oplog - `OplogTruncateMarkers`.
- [Change stream pre images collections](#pre-images-collection-truncation) -
`PreImagesTruncateMarkersPerNsUUID`
Read about the WiredTiger implementation of Oplog Truncation Markers [here](wiredtiger/README.md#oplog-truncation).
### Change Stream Collection Truncation
Change stream collection that uses CollectionTruncateMarkers
- pre-images: `<tenantId>_config.system.preimages` in serverless, `config.system.preimages` in
dedicated environments.
The change stream pre-images collections has a periodic remover thread
([ChangeStreamExpiredPreImagesRemover](https://github.com/10gen/mongo/blob/r7.1.0-rc3/src/mongo/db/pipeline/change_stream_expired_pre_image_remover.cpp#L71).
The remover thread:
1. Creates the tenant's initial CollectionTruncateMarkers for the tenant if they do not yet exist
- Lazy initialization of the initial truncate markers is imperative so writes aren't blocked on
startup
2. Iterates through each truncate marker. If a marker is expired, issues a truncate of all records
older than the marker's last record, and removes the marker from the set.
#### Cleanup After Unclean Shutdown
After an unclean shutdown, all expired pre-images are truncated at startup. WiredTiger truncate
cannot guarantee a consistent view of previously truncated data on unreplicated, untimestamped
ranges after a crash. Unlike the oplog, the change stream collections aren't logged, don't persist
any special timestamps, and it's possible that previously truncated documents can resurface after
shutdown.
#### Pre Images Collection Truncation
Each tenant has 1 pre-images collection. Each pre-images collection contains pre-images across all
the tenant's pre-image enabled collections.
A pre-images collection is clustered by
[ChangeStreamPreImageId](https://github.com/10gen/mongo/blob/r7.1.0-rc3/src/mongo/db/pipeline/change_stream_preimage.idl#L69),
which implicitly orders pre-images first by their `'nsUUID'` (the UUID of the collection the
pre-image is from), their `'ts'` (the timestamp associated with the pre-images oplog entry), and
then by their `'applyOpsIndex'` (the index into the applyOps oplog entry which generated the
pre-image, 0 if the pre-image isn't from an applyOps oplog entry).
There is a set of CollectionTruncateMarkers for each 'nsUUD' within a tenant's pre-images
collection, `PreImagesTruncateMarkersPerNsUUID`.
In a serverless environment, each tenant has a set 'expireAfterSeconds' parameter. An entry is
expired if the 'wall time' associated with the pre-image is more than 'expireAfterSeconds' older
than the node's current wall time.
In a dedicated environment, a pre-image is expired if either (1) 'expireAfterSeconds' is set and the
pre-image is expired by it or (2) it's 'ts' is less than or equal to the oldest oplog entry
timestamp.
For each tenant, `ChangeStreamExpiredPreImagesRemover` iterates over each set of
`PreImagesTruncateMarkersPerNsUUID`, and issues a ranged truncate from the truncate marker's last
record to the the minimum RecordId for the nsUUID when there is an expired truncate marker.
### Code spelunking starting points:
- [The CollectionTruncateMarkers
class](https://github.com/mongodb/mongo/blob/r7.1.0-rc3/src/mongo/db/storage/collection_truncate_markers.h#L78)
- The main api for CollectionTruncateMarkers.
- [The OplogTruncateMarkers
class](https://github.com/10gen/mongo/blob/r7.1.0-rc3/src/mongo/db/storage/wiredtiger/wiredtiger_record_store_oplog_truncate_markers.h)
- Oplog specific truncate markers.
- [The PreImagesTruncateMarkersPerNsUUID
class](https://github.com/10gen/mongo/blob/r7.1.0-rc3/src/mongo/db/change_stream_pre_images_truncate_markers_per_nsUUID.h#L62)
- Truncate markers for a given nsUUID captured within a pre-images collection.
- [The PreImagesTruncateManager
class](https://github.com/10gen/mongo/blob/r7.1.0-rc3/src/mongo/db/change_stream_pre_images_truncate_manager.h#L70)
- Manages pre image truncate markers for each tenant.
# Oplog Collection
The `local.oplog.rs` collection maintains a log of all writes done on a server that should be
replicated by other members of its replica set. All replicated writes have corresponding oplog
entries; non-replicated collection writes do not have corresponding oplog entries. On a primary, an
oplog entry is written in the same storage transaction as the write it logs; a secondary writes the
oplog entry and then applies the write reflected therein in separate transactions. The oplog
collection is only created for servers started with the `--replSet` setting. The oplog collection is
a capped collection and therefore self-deleting per the default oplog size. The oplog can be resized
by the user via the `replSetResizeOplog` server command.
A write's persistence is guaranteed when its oplog entry reaches disk. The log is periodically
synced to disk, i.e. [journaled](#journaling). The log can also be immediately synced to disk by an
explicit request to fulfill the durability requirements of a particular write. For example:
replication may need to guarantee a write survives server restart before proceeding, for
correctness; or a user may specify a `j:true` write concern to request the same durability. The data
write itself is not written out to disk until the next periodic [checkpoint](#checkpoints) is taken.
The default log syncing frequency is much higher than the checkpoint default frequency because
syncing the log to disk is cheaper than syncing everything to disk.
The oplog is read by secondaries that then apply the writes therein to themselves. Secondaries can
'fall off the oplog' if replication is too slow and the oplog capped max size is too small: the sync
source may delete oplog entries that a secondary still needs to read. The oplog is also used on
startup recovery to play writes forward from a checkpoint; and it is manipulated -- undone or
reapplied -- for replication rollback.
## Oplog Visibility
MongoDB supports concurrent writes. This means that there are out-of-order commits and 'oplog holes'
can momentarily exist when one write with a later timestamp commits before a concurrent write with
an earlier timestamp. Timestamps are assigned prior to storage transaction commit. Out-of-order
writes are supported because otherwise writes must be serialized, which would harm performance.
Oplog holes must be tracked so that oplog read cursors do not miss data when reading in timestamp
order. Unlike typical collections, the key for a document in the oplog is the timestamp itself.
Because collection cursors return data in key order, cursors on the oplog will return documents in
timestamp order. Oplog readers therefore fetch a timestamp guaranteed not to have holes behind it
and use that timestamp to open a storage engine transaction that does not return entries with later
timestamps. The following is a demonstrative example of what this oplog visibility rule prevents:
Suppose there are two concurrent writers **A** and **B**. **Writer A** opens a storage transaction
first and is assigned a commit timestamp of **T5**; then **Writer** **B** opens a transaction and
acquires a commit timestamp **T6**. The writers are using different threads so **Writer B** happens
to commit first. The oplog now has a 'hole' for timestamp **T5**. A reader opening a read
transaction at this time could now see up to the **T6** write but miss the **T5** write that has not
committed yet: the cursor would see T1, T2, T3, T4, T6. This would be a serious replica set data
consistency problem if secondary replica set members querying the oplog of their sync source could
unknowingly read past these holes and miss the data therein.
| Op | Action | Result |
| -------- | ------------------ | -------------------------------------------- |
| Writer A | open transaction | assigned commit timestamp T5 |
| Writer B | open transaction | assigned commit timestamp T6 |
| Writer B | commit transaction | T1,T2,T3,T4,T6 are visible to new readers |
| Reader X | open transaction | gets a snapshot of T1-T4 and T6 |
| Writer A | commit transaction | T1,T2,T3,T4,T5,T6 are visible to new readers |
| Reader X | close transaction | returns T1,T2,T3,T4,T6, missing T5 |
The in-memory 'no holes' point of the oplog is tracked in order to avoid data inconsistency across
replica set members. The 'oplogReadTimestamp' tracks the in-memory no holes point and is continually
advanced as new oplog writes occur and holes disappear. Forward cursor oplog readers without a
specified timestamp set at which to read (secondary callers) will automatically use the
`oplogReadTimestamp` to avoid missing entries due to oplog holes. This is essential for secondary
replica set members querying the oplog of their sync source so they do not miss any oplog entries:
subsequent `getMores` will fetch entries as they become visible without any holes behind them.
Backward cursor oplog readers bypass the oplog visibility rules to see the latest oplog entries,
disregarding any oplog holes.
# Glossary
**oplog hole**: An uncommitted oplog write that can exist with out-of-order writes when a later
timestamped write happens to commit first. Oplog holes can exist in-memory and persisted on disk.
**oplogReadTimestamp**: The timestamp used for WT forward cursor oplog reads in order to avoid
advancing past oplog holes. Tracks in-memory oplog holes.
**snapshot**: A snapshot consists of a consistent view of data in the database. When a snapshot is
opened with a timestamp, snapshot only shows data committed with a timestamp less than or equal
to the snapshot's timestamp.
**snapshot isolation**: A guarantee that all reads in a transaction see the same consistent snapshot
of the database. Reads with snapshot isolation are repeatable, only see committed data, and never
return newly-committed data. The storage engine must provide concurrency control to resolve
concurrent writes to the same data.

View File

@ -0,0 +1,344 @@
# WiredTiger Storage Engine Integration
## Collection and Index to Table relationship
Creating a collection (record store) or index requires two WT operations that cannot be made
atomic/transactional. A WT table must be created with
[WT_SESSION::create](https://source.wiredtiger.com/develop/struct_w_t___s_e_s_s_i_o_n.html#a358ca4141d59c345f401c58501276bbb "WiredTiger Docs") and an insert/update must be made in the \_mdb_catalog table (MongoDB's
catalog). MongoDB orders these as such:
1. Create the WT table
1. Update \_mdb_catalog to reference the table
Note that if the process crashes in between those steps, the collection/index creation never
succeeded. Upon a restart, the WT table is dangling and can be safely deleted.
Dropping a collection/index follows the same pattern, but in reverse.
1. Delete the table from the \_mdb_catalog
1. [Drop the WT table](https://source.wiredtiger.com/develop/struct_w_t___s_e_s_s_i_o_n.html#adf785ef53c16d9dcc77e22cc04c87b70 "WiredTiger Docs")
In this case, if a crash happens between these steps and the change to the \_mdb_catalog was made
durable (in modern versions, only possible via a checkpoint; the \_mdb_catalog is not logged), the
WT table is once again dangling on restart. Note that in the absence of a history, this state is
indistinguishable from the creation case, establishing a strong invariant.
# Checkpoints
The WiredTiger storage engine [supports
checkpoints](https://github.com/mongodb/mongo/blob/r4.5.0/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.cpp#L443-L647)
, which are a read-only, static view of one or more data sources. When WiredTiger takes a
checkpoint, it writes all of the data in a snapshot to the disk in a consistent way across all of
the data files.
To avoid taking unnecessary checkpoints on an idle server, WiredTiger will only take checkpoints for
the following scenarios:
- When the [stable timestamp](../repl/README.md#replication-timestamp-glossary) is greater than or
equal to the [initial data timestamp](../repl/README.md#replication-timestamp-glossary), we take a
stable checkpoint, which is a durable view of the data at a particular timestamp. This is for
steady-state replication.
- The [initial data timestamp](../repl/README.md#replication-timestamp-glossary) is not set, so we
must take a full checkpoint. This is when there is no consistent view of the data, such as during
initial sync.
Not only does checkpointing provide us with durability for the database, but it also enables us to
take [backups of the data](#file-system-backups).
When WiredTiger takes a checkpoint, it uses the
[`stable_timestamp`](https://github.com/mongodb/mongo/blob/87de9a0cb1/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.cpp#L2011 "Github") (effectively a `read_timestamp`) for what data should be persisted in the checkpoint.
Every "data write" (collection/index contents, \_mdb_catalog contents) corresponding to an oplog
entry with a timestamp <= the `stable_timestamp` will be included in this checkpoint. None of the
data writes later than the `stable_timestamp` are included in the checkpoint. When the checkpoint is
completed, the `stable_timestamp` is known as the checkpoint's
[`checkpoint_timestamp`](https://github.com/mongodb/mongo/blob/834a3c49d9ea9bfe2361650475158fc0dbb374cd/src/third_party/wiredtiger/src/meta/meta_ckpt.c#L921 "Github"). When WiredTiger starts up on a checkpoint, that checkpoint's timestamp is known as the
[`recovery_timestamp`](https://github.com/mongodb/mongo/blob/87de9a0cb1/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.cpp#L684 "Github"). The recovery timestamp is used for Replication startup recovery.
# Journaling
WiredTiger journals any collection or index with `log=(enabled=true)` specified at creation. Such
collection and index tables are specially logged / journaled to disk when requested. The MongoDB
change log stored in the oplog collection is journaled, along with most non-replicated `local`
database collections, when the server instance is started with `--replSet`. In standalone mode,
however, MongoDB does not create the `local.oplog.rs` collection and all collections are journaled.
Code links:
- [_Code that ultimately calls flush journal on
WiredTiger_](https://github.com/mongodb/mongo/blob/767494374cf12d76fc74911d1d0fcc2bbce0cd6b/src/mongo/db/storage/wiredtiger/wiredtiger_session_cache.cpp#L241-L362)
- Skips flushing if ephemeral mode engine; may do a journal flush or take a checkpoint depending
on server settings.
- [_Control of whether journaling is
enabled_](https://github.com/mongodb/mongo/blob/r4.5.0/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.h#L451)
- 'durable' confusingly means journaling is enabled.
- [_Whether WT journals a
collection_](https://github.com/mongodb/mongo/blob/r4.5.0/src/mongo/db/storage/wiredtiger/wiredtiger_util.cpp#L560-L580)
# Startup Recovery
During startup, WiredTiger will replay the write-ahead log (journal) entries, if any, from a crash.
In WiredTiger, the write-ahead log also contains entries that are specific to WiredTiger, most of
its entries are to re-insert items into MongoDB's oplog collection.
## Rollback to Stable
Rollback-to-stable is an operation that retains only modifications that are considered stable. In other words, we are rolling back to the latest checkpoint.
The Replication and Storage Engine integration layers kill all user operations and all internal
threads that could access the storage engine. This is necessary because
`WT_CONNECTION::rollback_to_stable` requires all open cursors to be closed or reset, otherwise
`EBUSY` will be returned. In the server we retry on `EBUSY` until the system quiesces.
Once the system is quiesced, Replication and Storage Engine integration layers prevent new
operations from starting. The in-memory representation of the catalog is cleared and the drop
pending state is cleared in the ident reaper as drops may be rolled back. At this point
`WT_CONNECTION::rollback_to_stable` is called. Once we return from this function, the reverse order
of operations is performed. Such as rebuilding the in-memory representation of the catalog, internal
threads are restarted, and two-phase index builds are resumed.
See [here](https://source.wiredtiger.com/develop/arch-rts.html) for WiredTiger's architecture guide
on rollback-to-stable.
See [here](../repl/README.md#rollback-recover-to-a-timestamp-rtt) for more information on what
happens in the replication layer during rollback-to-stable.
## Repair
Data corruption has a variety of causes, but can usually be attributed to misconfigured or
unreliable I/O subsystems that do not make data durable when called upon, often in the event of
power outages.
MongoDB provides a command-line `--repair` utility that attempts to recover as much data as possible
from an installation that fails to start up due to data corruption.
### Types of Corruption
MongoDB repair attempts to address the following forms of corruption:
- Corrupt WiredTiger data files
- Includes all collections, `_mdb_catalog`, and `sizeStorer`
- Missing WiredTiger data files
- Includes all collections, `_mdb_catalog`, and `sizeStorer`
- Index inconsistencies
- Validate [repair mode](#repair-mode) attempts to fix index inconsistencies to avoid a full index
rebuild.
- Indexes are rebuilt on collections after they have been salvaged or if they fail validation and
validate repair mode is unable to fix all errors.
- Un-salvageable collection data files
- Corrupt metadata
- `WiredTiger.wt`, `WiredTiger.turtle`, and WT journal files
- “Orphaned” data files
- Collection files missing from the `WiredTiger.wt` metadata
- Collection files missing from the `_mdb_catalog` table
- We cannot support restoring orphaned files that are missing from both metadata sources
- Missing `featureCompatibilityVersion` document
### Repair Procedure
1. Initialize the WiredTigerKVEngine. If a call to `wiredtiger_open` returns the `WT_TRY_SALVAGE`
error code, this indicates there is some form of corruption in the WiredTiger metadata. Attempt
to [salvage the
metadata](https://github.com/mongodb/mongo/blob/r4.5.0/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.cpp#L1046-L1071)
by using the WiredTiger `salvage=true` configuration option.
2. Initialize the StorageEngine and [salvage the `_mdb_catalog` table, if
needed](https://github.com/mongodb/mongo/blob/r4.5.0/src/mongo/db/storage/storage_engine_impl.cpp#L95).
3. Recover orphaned collections.
- If an [ident](#glossary) is known to WiredTiger but is not present in the `_mdb_catalog`,
[create a new
collection](https://github.com/mongodb/mongo/blob/r4.5.0/src/mongo/db/storage/storage_engine_impl.cpp#L145-L189)
with the prefix `local.orphan.<ident-name>` that references this ident.
- If an ident is present in the `_mdb_catalog` but not known to WiredTiger, [attempt to recover
the
ident](https://github.com/mongodb/mongo/blob/r4.5.0/src/mongo/db/storage/storage_engine_impl.cpp#L197-L229).
This [procedure for orphan
recovery](https://github.com/mongodb/mongo/blob/r4.5.0/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.cpp#L1525-L1605)
is a less reliable and more invasive. It involves moving the corrupt data file to a temporary
file, creates a new table with the same name, replaces the original data file over the new one,
and
[salvages](https://github.com/mongodb/mongo/blob/r4.5.0/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.cpp#L1525-L1605)
the table in attempt to reconstruct the table.
4. [Verify collection data
files](https://github.com/mongodb/mongo/blob/r4.5.0/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.cpp#L1195-L1226),
and salvage if necessary.
- If call to WiredTiger
[verify()](https://source.wiredtiger.com/develop/struct_w_t___s_e_s_s_i_o_n.html#a0334da4c85fe8af4197c9a7de27467d3)
fails, call
[salvage()](https://source.wiredtiger.com/develop/struct_w_t___s_e_s_s_i_o_n.html#ab3399430e474f7005bd5ea20e6ec7a8e),
which recovers as much data from a WT data file as possible.
- If a salvage is unsuccessful, rename the data file with a `.corrupt` suffix.
- If a data file is missing or a salvage was unsuccessful, [drop the original table from the
metadata, and create a new, empty
table](https://github.com/mongodb/mongo/blob/r4.5.0/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.cpp#L1262-L1274)
under the original name. This allows MongoDB to continue to start up despite present
corruption.
- After any salvage operation, [all indexes are
rebuilt](https://github.com/mongodb/mongo/blob/r4.5.0/src/mongo/db/repair_database.cpp#L134-L149)
for that collection.
5. Validate collection and index consistency
- [Collection validation](#collection-validation) checks for consistency between the collection
and indexes. Validate repair mode attempts to fix any inconsistencies it finds.
6. Rebuild indexes
- If a collection's data has been salvaged or any index inconsistencies are not repairable by
validate repair mode, [all indexes are
rebuilt](https://github.com/mongodb/mongo/blob/4406491b2b137984c2583db98068b7d18ea32171/src/mongo/db/repair.cpp#L273-L275).
- While a unique index is being rebuilt, if any documents are found to have duplicate keys, then
those documents are inserted into a lost and found collection with the format
`local.lost_and_found.<collection UUID>`.
7. [Invalidate the replica set
configuration](https://github.com/mongodb/mongo/blob/r4.5.0/src/mongo/db/repair_database_and_check_version.cpp#L460-L485)
if data has been or could have been modified. This [prevents a repaired node from
joining](https://github.com/mongodb/mongo/blob/r4.5.0/src/mongo/db/repl/replication_coordinator_impl.cpp#L486-L494)
and threatening the consistency of its replica set.
Additionally:
- When repair starts, it creates a temporary file, `_repair_incomplete` that is only removed when
repair completes. The server [will not start up
normally](https://github.com/mongodb/mongo/blob/r4.5.0/src/mongo/db/storage/storage_engine_init.cpp#L82-L86)
as long as this file is present.
- Repair [will restore a
missing](https://github.com/mongodb/mongo/blob/r4.5.0/src/mongo/db/repair_database_and_check_version.cpp#L434)
`featureCompatibilityVersion` document in the `admin.system.version` to the lower FCV version
available.
# Oplog Truncation
The oplog collection can be truncated both at the front end (most recent entries) and the back end
(the oldest entries). The capped setting on the oplog collection causes the oldest oplog entries to
be deleted when new writes increase the collection size past the cap. MongoDB using the WiredTiger
storage engine with `--replSet` handles oplog collection deletion specially via
OplogTruncateMarkers, an oplog specific implementation of the
[CollectionTruncateMarkers](#collectionTruncateMarkers) mechanism, ignoring the generic capped
collection deletion mechanism. The front of the oplog may be truncated back to a particular
timestamp during replication startup recovery or replication rollback.
A new truncate marker is created when the in-progress marker segment contains more than the minimum
bytes needed to complete the segment; and the oldest truncate marker's oplog is deleted when the
oplog size exceeds its cap size setting.
Oplog sampling and marker generation is skipped when using `--restore` or `--magicRestore`.
## Special Timestamps That Will Not Be Truncated
The WiredTiger integration layer's `OplogTruncateMarkers` implementation will stall deletion waiting
for certain significant tracked timestamps to move forward past entries in the oldest truncate
marker. This is done for correctness. Backup pins truncation in order to maintain a consistent view
of the oplog; and startup recovery after an unclean shutdown and rollback both require oplog history
back to certain timestamps.
## Min Oplog Retention
WiredTiger `OplogTruncateMarkers` obey an `oplogMinRetentionHours` configurable setting. When
`oplogMinRetentionHours` is active, the WT `OplogTruncateMarkers` will only truncate the oplog if a
truncate marker (a sequential range of oplog) is not within the minimum time range required to
remain.
# Error Handling
See
[wtRcToStatus](https://github.com/mongodb/mongo/blob/c799851554dc01493d35b43701416e9c78b3665c/src/mongo/db/storage/wiredtiger/wiredtiger_util.cpp#L178-L183)
where we handle errors from WiredTiger and convert to MongoDB errors.
# Cherry-picked WT log Details
- The WT log is a write ahead log. Before a [transaction commit](https://source.wiredtiger.com/develop/struct_w_t___s_e_s_s_i_o_n.html#a712226eca5ade5bd123026c624468fa2 "WiredTiger Docs") returns to the application, logged writes
must have their log entry bytes written into WiredTiger's log buffer. Depending on `sync` setting,
those bytes may or may not be on disk.
- MongoDB only chooses to log writes to a subset of WT's tables (e.g: the oplog).
- MongoDB does not `sync` the log on transaction commit. But rather uses the [log
flush](https://source.wiredtiger.com/develop/struct_w_t___s_e_s_s_i_o_n.html#a1843292630960309129dcfe00e1a3817 "WiredTiger Docs") API. This optimization is two-fold. Writes that do not require to be
persisted do not need to wait for durability on disk. Second, this pattern allows for batching
of writes to go to disk for improved throughput.
- WiredTiger's log is similar to MongoDB's oplog in that multiple writers can concurrently copy
their bytes representing a log record into WiredTiger's log buffer similar to how multiple
MongoDB writes can concurrently generate oplog entries.
- MongoDB's optime generator for the oplog is analogous to WT's LSN (log sequence number)
generator. Both are a small critical section to ensure concurrent writes don't get the same
timestamp key/memory address to write an oplog entry value/log bytes into.
- While MongoDB's oplog writes are logical (the key is a timestamp), WT's are obviously more
physical (the key is a memory->disk location). WiredTiger is writing to a memory buffer. Thus before a
transaction commit can go to the log buffer to "request a slot", it must know how many bytes it's
going to write. Compare this to a multi-statement transaction replicating as a single applyOps
versus each statement generating an individual oplog entry for each write that's part of the
transaction.
- MongoDB testing sometimes uses a [WT debugging
option](https://github.com/mongodb/mongo/blob/a7bd84dc5ad15694864526612bceb3877672d8a9/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.cpp#L601 "Github") that will write "no-op" log entries for other operations performed on a
transaction. Such as setting a timestamp or writing to a table that is not configured to be
written to WT's log (e.g: a typical user collection and index).
The most important WT log entry for MongoDB is one that represents an insert into the
oplog.
```
{ "lsn" : [1,57984],
"hdr_flags" : "compressed",
"rec_len" : 384,
"mem_len" : 423,
"type" : "commit",
"txnid" : 118,
"ops": [
{ "optype": "row_put",
"fileid": 14 0xe,
"key": "\u00e8^\u00eat@\u00ff\u00ff\u00df\u00c2",
"key-hex": "e85eea7440ffffdfc2",
"value": "\u009f\u0000\u0000\u0000\u0002op\u0000\u0002\u0000\u0000\u0000i\u0000\u0002ns\u0000\n\u0000\u0000\u0000test.coll\u0000\u0005ui\u0000\u0010\u0000\u0000\u0000\u0004\u0017\u009d\u00b0\u00fc\u00b2,O\u0004\u0084\u00bdY\u00e9%\u001dm\u00ba\u0003o\u00002\u0000\u0000\u0000\u0007_id\u0000^\u00eatA\u00d4\u0098\u00b7\u008bD\u009b\u00b2\u008c\u0002payload\u0000\u000f\u0000\u0000\u0000data and bytes\u0000\u0000\u0011ts\u0000\u0002\u0000\u0000\u0000At\u00ea^\u0012t\u0000\u0001\u0000\u0000\u0000\u0000\u0000\u0000\u0000\twall\u0000\u0085\u001e\u00d6\u00c3r\u0001\u0000\u0000\u0012v\u0000\u0002\u0000\u0000\u0000\u0000\u0000\u0000\u0000\u0000",
"value-bson": {
u'ns': u'test.coll',
u'o': {u'_id': ObjectId('5eea7441d498b78b449bb28c'), u'payload': u'data and bytes'},
u'op': u'i',
u't': 1L,
u'ts': Timestamp(1592423489, 2),
u'ui': UUID('179db0fc-b22c-4f04-84bd-59e9251d6dba'),
u'v': 2L,
u'wall': datetime.datetime(2020, 6, 17, 19, 51, 29, 157000)}
}
]
}
```
- `lsn` is a log sequence number. The WiredTiger log files are named with numbers as a
suffix, e.g: `WiredTigerLog.0000000001`. In this example, the LSN's first value `1` maps to log
file `0000000001`. The second value `57984` is the byte offset in the file.
- `hdr_flags` stands for header flags. Think HTTP headers. MongoDB configures WiredTiger to use
snappy compression on its journal entries. Small journal entries (< 128 bytes?) won't be
compressed.
- `rec_len` is the number of bytes for the record
- `type` is...the type of journal entry. The type will be `commit` for application's committing a
transaction. Other types are typically for internal WT operations. Examples include `file_sync`,
`checkpoint` and `system`.
- `txnid` is WT's transaction id associated with the log record.
- `ops` is a list of operations that are part of the transaction. A transaction that inserts two
documents and removes a third will see three entries. Two `row_put` operations followed by a
`row_remove`.
- `ops.fileid` refers to the WT table that the operation is performed against. The fileid mapping
is held in the `WiredTiger.wt` file (a table within itself). This value is faked for WT's
logging debug mode for tables which MongoDB is not logging.
- `ops.key` and `ops.value` are the binary representations of the inserted document (`value` is omitted
for removal).
- `ops.key-hex` and `ops.value-bson` are specific to the pretty printing tool used.
[copy-on-write]: https://en.wikipedia.org/wiki/Copy-on-write
# Table of MongoDB <-> WiredTiger <-> Log version numbers
| MongoDB | WiredTiger | Log |
| ---------------------- | ---------- | --- |
| 3.0.15 | 2.5.3 | 1 |
| 3.2.20 | 2.9.2 | 1 |
| 3.4.15 | 2.9.2 | 1 |
| 3.6.4 | 3.0.1 | 2 |
| 4.0.16 | 3.1.1 | 3 |
| 4.2.1 | 3.2.2 | 3 |
| 4.2.6 | 3.3.0 | 3 |
| 4.2.6 (blessed by 4.4) | 3.3.0 | 4 |
| 4.4.0 | 10.0.0 | 5 |
| 5.0.0 | 10.0.1 | 5 |
| 4.4.11, 5.0.6 | 10.0.2 | 5 |
| 6.0.0 | 10.0.2 | 5 |
| 6.1.0 | 11.0.1 | 5 |
| 6.2.0 | 11.2.0 | 5 |
| 7.0.0 | 11.2.0 | 5 |
| 7.1.0 | 11.2.0 | 5 |
| 7.2.0 | 11.3.0 | 5 |
| 7.3.0 | 11.3.0 | 5 |
| 8.0.0 | 11.3.0 | 5 |

View File

@ -0,0 +1,50 @@
# The TTLMonitor
The TTLMonitor runs as a background job on each mongod. On a mongod primary, the TTLMonitor is
responsible for removing documents expired on [TTL
Indexes](https://www.mongodb.com/docs/manual/core/index-ttl/) across the mongod instance. It
continuously runs in a loop that sleeps for
['ttlMonitorSleepSecs'](https://github.com/mongodb/mongo/blob/d88a892d5b18035bd0f5393a42690e705c2007d7/src/mongo/db/ttl.idl#L39)
and then performs a TTL Pass to remove all expired documents.
The TTLMonitor exhibits different behavior pending on whether batched deletes are enabled. When
enabled (the default), the TTLMonitor batches TTL deletions and also removes expired documents more
fairly among TTL indexes. When disabled, the TTLMonitor falls back to legacy, doc-by-doc deletions
and deletes all expired documents from a single TTL index before moving to the next one. The legacy
behavior can lead to the TTLMonitor getting "stuck" deleting large ranges of documents on a single
TTL index, starving other indexes of deletes at regular intervals.
## Fair TTL Deletion
If
['ttlMonitorBatchDeletes'](https://github.com/mongodb/mongo/blob/d88a892d5b18035bd0f5393a42690e705c2007d7/src/mongo/db/ttl.idl#L48)
is specified, the TTLMonitor will batch deletes and provides fair TTL deletion as follows:
- The TTL pass consists of one or more sub-passes.
- Each sub-pass refreshes its view of TTL indexes in the system. It removes documents on each TTL
index in a round-robin fashion until there are no more expired documents or
['ttlMonitorSubPassTargetSecs'](https://github.com/mongodb/mongo/blob/d88a892d5b18035bd0f5393a42690e705c2007d7/src/mongo/db/ttl.idl#L58)
is reached.
- The delete on each TTL index removes up to
['ttlIndexDeleteTargetDocs'](https://github.com/mongodb/mongo/blob/d88a892d5b18035bd0f5393a42690e705c2007d7/src/mongo/db/ttl.idl#L84)
or runs up to
['ttlIndexDeleteTargetTimeMS'](https://github.com/mongodb/mongo/blob/d88a892d5b18035bd0f5393a42690e705c2007d7/src/mongo/db/ttl.idl#L72),
whichever target is met first. The same TTL index can be queued up to be revisited in the same
sub-pass if there are outstanding deletions.
- A TTL index is not visited any longer in a sub-pass once all documents are deleted.
- If there are outstanding deletions by the end of the sub-pass for any TTL index, a new sub-pass
starts immediately within the same pass.
_Code spelunking starting points:_
- [_The TTLMonitor
Class_](https://github.com/mongodb/mongo/blob/d88a892d5b18035bd0f5393a42690e705c2007d7/src/mongo/db/ttl.h)
- [_The TTLCollectionCache
Class_](https://github.com/mongodb/mongo/blob/d88a892d5b18035bd0f5393a42690e705c2007d7/src/mongo/db/ttl_collection_cache.h)
- [_ttl.idl_](https://github.com/mongodb/mongo/blob/d88a892d5b18035bd0f5393a42690e705c2007d7/src/mongo/db/ttl.idl)
# Timeseries Collections
The TTL monitor will only delete data from a time-series bucket collection when a bucket's minimum
time, \_id, is past the expiration plus the bucket maximum time span (default 1 hour). This
procedure avoids deleting buckets with data that is not older than the expiration time.