Storage Engine API
mongo::WiredTigerKVEngine Class Referencefinal

#include <wiredtiger_kv_engine.h>

Inheritance diagram for mongo::WiredTigerKVEngine:
mongo::KVEngine

Classes

class  WiredTigerCheckpointThread
 
class  WiredTigerJournalFlusher
 

Public Member Functions

 WiredTigerKVEngine (const std::string &canonicalName, const std::string &path, ClockSource *cs, const std::string &extraOpenOptions, size_t cacheSizeGB, bool durable, bool ephemeral, bool repair, bool readOnly)
 
virtual ~WiredTigerKVEngine ()
 
void setRecordStoreExtraOptions (const std::string &options)
 
void setSortedDataInterfaceExtraOptions (const std::string &options)
 
virtual bool supportsDocLocking () const
 This must not change over the lifetime of the engine. More...
 
virtual bool supportsDirectoryPerDB () const
 Returns true if storage engine supports –directoryperdb. More...
 
virtual bool isDurable () const
 
virtual bool isEphemeral () const
 Returns true if the KVEngine is ephemeral – that is, it is NOT persistent and all data is lost after shutdown. More...
 
virtual RecoveryUnitnewRecoveryUnit ()
 
virtual Status createRecordStore (OperationContext *opCtx, StringData ns, StringData ident, const CollectionOptions &options)
 The create and drop methods on KVEngine are not transactional. More...
 
virtual std::unique_ptr< RecordStoregetRecordStore (OperationContext *opCtx, StringData ns, StringData ident, const CollectionOptions &options)
 Having multiple out for the same ns is a rules violation; Calling on a non-created ident is invalid and may crash. More...
 
virtual Status createSortedDataInterface (OperationContext *opCtx, StringData ident, const IndexDescriptor *desc)
 
virtual SortedDataInterfacegetSortedDataInterface (OperationContext *opCtx, StringData ident, const IndexDescriptor *desc)
 
virtual Status createGroupedRecordStore (OperationContext *opCtx, StringData ns, StringData ident, const CollectionOptions &options, KVPrefix prefix)
 Create a RecordStore that MongoDB considers eligible to share space in an underlying table with other RecordStores. More...
 
virtual std::unique_ptr< RecordStoregetGroupedRecordStore (OperationContext *opCtx, StringData ns, StringData ident, const CollectionOptions &options, KVPrefix prefix)
 Get a RecordStore that may share an underlying table with other RecordStores. More...
 
virtual Status createGroupedSortedDataInterface (OperationContext *opCtx, StringData ident, const IndexDescriptor *desc, KVPrefix prefix)
 Create a SortedDataInterface that MongoDB considers eligible to share space in an underlying table with other SortedDataInterfaces. More...
 
virtual SortedDataInterfacegetGroupedSortedDataInterface (OperationContext *opCtx, StringData ident, const IndexDescriptor *desc, KVPrefix prefix)
 Get a SortedDataInterface that may share an underlying table with other SortedDataInterface. More...
 
virtual Status dropIdent (OperationContext *opCtx, StringData ident)
 
virtual void alterIdentMetadata (OperationContext *opCtx, StringData ident, const IndexDescriptor *desc)
 
virtual Status okToRename (OperationContext *opCtx, StringData fromNS, StringData toNS, StringData ident, const RecordStore *originalRecordStore) const
 
virtual int flushAllFiles (OperationContext *opCtx, bool sync)
 
virtual Status beginBackup (OperationContext *opCtx)
 See StorageEngine::beginBackup for details. More...
 
virtual void endBackup (OperationContext *opCtx)
 See StorageEngine::endBackup for details. More...
 
virtual int64_t getIdentSize (OperationContext *opCtx, StringData ident)
 
virtual Status repairIdent (OperationContext *opCtx, StringData ident)
 
virtual bool hasIdent (OperationContext *opCtx, StringData ident) const
 
std::vector< std::string > getAllIdents (OperationContext *opCtx) const
 
virtual void cleanShutdown ()
 This method will be called before there is a clean shutdown. More...
 
SnapshotManagergetSnapshotManager () const final
 Return the SnapshotManager for this KVEngine or NULL if not supported. More...
 
void setJournalListener (JournalListener *jl) final
 Sets a new JournalListener, which is used to alert the rest of the system about journaled write progress. More...
 
virtual void setStableTimestamp (Timestamp stableTimestamp) override
 See StorageEngine::setStableTimestamp More...
 
virtual void setInitialDataTimestamp (Timestamp initialDataTimestamp) override
 See StorageEngine::setInitialDataTimestamp More...
 
virtual void setOldestTimestampFromStable () override
 See StorageEngine::setOldestTimestampFromStable More...
 
virtual void setOldestTimestamp (Timestamp newOldestTimestamp) override
 See StorageEngine::setOldestTimestamp More...
 
virtual bool supportsRecoverToStableTimestamp () const override
 See StorageEngine::supportsRecoverToStableTimestamp More...
 
virtual StatusWith< Timestamp > recoverToStableTimestamp (OperationContext *opCtx) override
 See StorageEngine::recoverToStableTimestamp More...
 
virtual boost::optional< Timestamp > getRecoveryTimestamp () const override
 See StorageEngine::getRecoveryTimestamp More...
 
virtual boost::optional< Timestamp > getLastStableCheckpointTimestamp () const override
 Returns a timestamp value that is at or before the last checkpoint. More...
 
virtual Timestamp getAllCommittedTimestamp () const override
 See StorageEngine::getAllCommittedTimestamp More...
 
bool supportsReadConcernSnapshot () const final
 See StorageEngine::supportsReadConcernSnapshot More...
 
int reconfigure (const char *str)
 
WT_CONNECTION * getConnection ()
 
void dropSomeQueuedIdents ()
 
std::list< WiredTigerCachedCursorfilterCursorsWithQueuedDrops (std::list< WiredTigerCachedCursor > *cache)
 
bool haveDropsQueued () const
 
void syncSizeInfo (bool sync) const
 
void startOplogManager (OperationContext *opCtx, const std::string &uri, WiredTigerRecordStore *oplogRecordStore)
 
void haltOplogManager ()
 
WiredTigerOplogManagergetOplogManager () const
 
void replicationBatchIsComplete () const override
 See StorageEngine::replicationBatchIsComplete() More...
 
bool isCacheUnderPressure (OperationContext *opCtx) const override
 See StorageEngine::isCacheUnderPressure() More...
 
Timestamp getStableTimestamp () const
 These are timestamp access functions for serverStatus to be able to report the actual snapshot window size. More...
 
Timestamp getOldestTimestamp () const
 
- Public Member Functions inherited from mongo::KVEngine
virtual bool supportsDBLocking () const
 This must not change over the lifetime of the engine. More...
 
virtual bool supportsCappedCollections () const
 This must not change over the lifetime of the engine. More...
 
virtual void setCachePressureForTest (int pressure)
 See 'StorageEngine::setCachePressureForTest()'. More...
 
virtual ~KVEngine ()
 The destructor will never be called from mongod, but may be called from tests. More...
 

Static Public Member Functions

static void setInitRsOplogBackgroundThreadCallback (stdx::function< bool(StringData)> cb)
 Sets the implementation for initRsOplogBackgroundThread (allowing tests to skip the background job, for example). More...
 
static bool initRsOplogBackgroundThread (StringData ns)
 Initializes a background job to remove excess documents in the oplog collections. More...
 
static void appendGlobalStats (BSONObjBuilder &b)
 

Static Public Attributes

static const int kDefaultJournalDelayMillis = 100
 

Private Member Functions

Status _salvageIfNeeded (const char *uri)
 
void _checkIdentPath (StringData ident)
 
bool _hasUri (WT_SESSION *session, const std::string &uri) const
 
std::string _uri (StringData ident) const
 
Timestamp _calculateHistoryLagFromStableTimestamp (Timestamp stableTimestamp)
 Uses the 'stableTimestamp', the 'targetSnapshotHistoryWindowInSeconds' setting and the current _oldestTimestamp to calculate what the new oldest_timestamp should be, in order to maintain a window of available snapshots on the storage engine from oldest to stable timestamp. More...
 
void _setOldestTimestamp (Timestamp newOldestTimestamp, bool force)
 Sets the oldest timestamp for which the storage engine must maintain snapshot history through. More...
 

Private Attributes

WT_CONNECTION * _conn
 
WT_EVENT_HANDLER _eventHandler
 
std::unique_ptr< WiredTigerSessionCache_sessionCache
 
ClockSource *const _clockSource
 
stdx::mutex _oplogManagerMutex
 
std::size_t _oplogManagerCount = 0
 
std::unique_ptr< WiredTigerOplogManager_oplogManager
 
std::string _canonicalName
 
std::string _path
 
std::string _wtOpenConfig
 
std::unique_ptr< WiredTigerSizeStorer_sizeStorer
 
std::string _sizeStorerUri
 
ElapsedTracker _sizeStorerSyncTracker
 
bool _durable
 
bool _ephemeral
 
const bool _inRepairMode
 
bool _readOnly
 
std::unique_ptr< WiredTigerJournalFlusher_journalFlusher
 
std::unique_ptr< WiredTigerCheckpointThread_checkpointThread
 
std::string _rsOptions
 
std::string _indexOptions
 
stdx::mutex _dropAllQueuesMutex
 
stdx::mutex _identToDropMutex
 
std::list< std::string > _identToDrop
 
Date_t _previousCheckedDropsQueued
 
std::unique_ptr< WiredTigerSession_backupSession
 
Timestamp _recoveryTimestamp
 
WiredTigerFileVersion _fileVersion
 
stdx::mutex _oldestTimestampMutex
 
stdx::mutex _stableTimestampMutex
 
Timestamp _oldestTimestamp
 
Timestamp _stableTimestamp
 

Constructor & Destructor Documentation

◆ WiredTigerKVEngine()

mongo::WiredTigerKVEngine::WiredTigerKVEngine ( const std::string &  canonicalName,
const std::string &  path,
ClockSource *  cs,
const std::string &  extraOpenOptions,
size_t  cacheSizeGB,
bool  durable,
bool  ephemeral,
bool  repair,
bool  readOnly 
)

◆ ~WiredTigerKVEngine()

mongo::WiredTigerKVEngine::~WiredTigerKVEngine ( )
virtual

Member Function Documentation

◆ _calculateHistoryLagFromStableTimestamp()

Timestamp mongo::WiredTigerKVEngine::_calculateHistoryLagFromStableTimestamp ( Timestamp  stableTimestamp)
private

Uses the 'stableTimestamp', the 'targetSnapshotHistoryWindowInSeconds' setting and the current _oldestTimestamp to calculate what the new oldest_timestamp should be, in order to maintain a window of available snapshots on the storage engine from oldest to stable timestamp.

If the returned Timestamp isNull(), oldest_timestamp should not be moved forward.

◆ _checkIdentPath()

void mongo::WiredTigerKVEngine::_checkIdentPath ( StringData  ident)
private

◆ _hasUri()

bool mongo::WiredTigerKVEngine::_hasUri ( WT_SESSION *  session,
const std::string &  uri 
) const
private

◆ _salvageIfNeeded()

Status mongo::WiredTigerKVEngine::_salvageIfNeeded ( const char *  uri)
private

◆ _setOldestTimestamp()

void mongo::WiredTigerKVEngine::_setOldestTimestamp ( Timestamp  newOldestTimestamp,
bool  force 
)
private

Sets the oldest timestamp for which the storage engine must maintain snapshot history through.

If force is true, oldest will be set to the given input value, unmodified, even if it is backwards in time from the last oldest timestamp (accomodating initial sync).

◆ _uri()

string mongo::WiredTigerKVEngine::_uri ( StringData  ident) const
private

◆ alterIdentMetadata()

void mongo::WiredTigerKVEngine::alterIdentMetadata ( OperationContext *  opCtx,
StringData  ident,
const IndexDescriptor *  desc 
)
virtual

Reimplemented from mongo::KVEngine.

◆ appendGlobalStats()

void mongo::WiredTigerKVEngine::appendGlobalStats ( BSONObjBuilder &  b)
static

◆ beginBackup()

Status mongo::WiredTigerKVEngine::beginBackup ( OperationContext *  opCtx)
virtual

See StorageEngine::beginBackup for details.

Reimplemented from mongo::KVEngine.

◆ cleanShutdown()

void mongo::WiredTigerKVEngine::cleanShutdown ( )
virtual

This method will be called before there is a clean shutdown.

Storage engines should override this method if they have clean-up to do that is different from unclean shutdown. MongoDB will not call into the storage subsystem after calling this function.

There is intentionally no uncleanShutdown().

Implements mongo::KVEngine.

◆ createGroupedRecordStore()

Status mongo::WiredTigerKVEngine::createGroupedRecordStore ( OperationContext *  opCtx,
StringData  ns,
StringData  ident,
const CollectionOptions options,
KVPrefix  prefix 
)
virtual

Create a RecordStore that MongoDB considers eligible to share space in an underlying table with other RecordStores.

'prefix' is guaranteed to be 'KVPrefix::kNotPrefixed' when 'groupCollections' is not enabled.

Parameters
prefixsignals whether the RecordStore may be shared by an underlying table. A prefix of KVPrefix::kNotPrefixed must remain isolated in its own table. Otherwise the storage engine implementation ultimately chooses which RecordStores share a table. Sharing RecordStores belonging to different databases within the same table is forbidden.

Reimplemented from mongo::KVEngine.

◆ createGroupedSortedDataInterface()

Status mongo::WiredTigerKVEngine::createGroupedSortedDataInterface ( OperationContext *  opCtx,
StringData  ident,
const IndexDescriptor *  desc,
KVPrefix  prefix 
)
virtual

Create a SortedDataInterface that MongoDB considers eligible to share space in an underlying table with other SortedDataInterfaces.

'prefix' is guaranteed to be 'KVPrefix::kNotPrefixed' when 'groupCollections' is not enabled.

Parameters
prefixsignals whether the SortedDataInterface (index) may be shared by an underlying table. A prefix of KVPrefix::kNotPrefixed must remain isolated in its own table. Otherwise the storage engine implementation ultimately chooses which indexes share a table. Sharing indexes belonging to different databases is forbidden.

Reimplemented from mongo::KVEngine.

◆ createRecordStore()

virtual Status mongo::WiredTigerKVEngine::createRecordStore ( OperationContext *  opCtx,
StringData  ns,
StringData  ident,
const CollectionOptions options 
)
inlinevirtual

The create and drop methods on KVEngine are not transactional.

Transactional semantics are provided by the KVStorageEngine code that calls these. For example, drop will be called if a create is rolled back. A higher-level drop operation will only propagate to a drop call on the KVEngine once the WUOW commits. Therefore drops will never be rolled back and it is safe to immediately reclaim storage.

Implements mongo::KVEngine.

◆ createSortedDataInterface()

virtual Status mongo::WiredTigerKVEngine::createSortedDataInterface ( OperationContext *  opCtx,
StringData  ident,
const IndexDescriptor *  desc 
)
inlinevirtual

Implements mongo::KVEngine.

◆ dropIdent()

Status mongo::WiredTigerKVEngine::dropIdent ( OperationContext *  opCtx,
StringData  ident 
)
virtual

Implements mongo::KVEngine.

◆ dropSomeQueuedIdents()

void mongo::WiredTigerKVEngine::dropSomeQueuedIdents ( )

◆ endBackup()

void mongo::WiredTigerKVEngine::endBackup ( OperationContext *  opCtx)
virtual

See StorageEngine::endBackup for details.

Reimplemented from mongo::KVEngine.

◆ filterCursorsWithQueuedDrops()

std::list< WiredTigerCachedCursor > mongo::WiredTigerKVEngine::filterCursorsWithQueuedDrops ( std::list< WiredTigerCachedCursor > *  cache)

◆ flushAllFiles()

int mongo::WiredTigerKVEngine::flushAllFiles ( OperationContext *  opCtx,
bool  sync 
)
virtual

Reimplemented from mongo::KVEngine.

◆ getAllCommittedTimestamp()

Timestamp mongo::WiredTigerKVEngine::getAllCommittedTimestamp ( ) const
overridevirtual

◆ getAllIdents()

std::vector< std::string > mongo::WiredTigerKVEngine::getAllIdents ( OperationContext *  opCtx) const
virtual

Implements mongo::KVEngine.

◆ getConnection()

WT_CONNECTION* mongo::WiredTigerKVEngine::getConnection ( )
inline

◆ getGroupedRecordStore()

std::unique_ptr< RecordStore > mongo::WiredTigerKVEngine::getGroupedRecordStore ( OperationContext *  opCtx,
StringData  ns,
StringData  ident,
const CollectionOptions options,
KVPrefix  prefix 
)
virtual

Get a RecordStore that may share an underlying table with other RecordStores.

'prefix' is guaranteed to be 'KVPrefix::kNotPrefixed' when 'groupCollections' is not enabled.

Parameters
prefixdictates the value keys for the RecordStore should be prefixed with to distinguish between RecordStores sharing an underlying table. A value of KVPrefix::kNotPrefixed guarantees the index is the sole resident of the table.

Reimplemented from mongo::KVEngine.

◆ getGroupedSortedDataInterface()

SortedDataInterface * mongo::WiredTigerKVEngine::getGroupedSortedDataInterface ( OperationContext *  opCtx,
StringData  ident,
const IndexDescriptor *  desc,
KVPrefix  prefix 
)
virtual

Get a SortedDataInterface that may share an underlying table with other SortedDataInterface.

'prefix' is guaranteed to be 'KVPrefix::kNotPrefixed' when 'groupCollections' is not enabled.

Parameters
prefixdictates the value keys for the index should be prefixed with to distinguish between indexes sharing an underlying table. A value of KVPrefix::kNotPrefixed guarantees the index is the sole resident of the table.

Reimplemented from mongo::KVEngine.

◆ getIdentSize()

int64_t mongo::WiredTigerKVEngine::getIdentSize ( OperationContext *  opCtx,
StringData  ident 
)
virtual

Implements mongo::KVEngine.

◆ getLastStableCheckpointTimestamp()

boost::optional< Timestamp > mongo::WiredTigerKVEngine::getLastStableCheckpointTimestamp ( ) const
overridevirtual

Returns a timestamp value that is at or before the last checkpoint.

Everything before this value is guaranteed to be persisted on disk and replication recovery will not need to replay documents with an earlier time.

Reimplemented from mongo::KVEngine.

◆ getOldestTimestamp()

Timestamp mongo::WiredTigerKVEngine::getOldestTimestamp ( ) const

◆ getOplogManager()

WiredTigerOplogManager* mongo::WiredTigerKVEngine::getOplogManager ( ) const
inline

◆ getRecordStore()

virtual std::unique_ptr<RecordStore> mongo::WiredTigerKVEngine::getRecordStore ( OperationContext *  opCtx,
StringData  ns,
StringData  ident,
const CollectionOptions options 
)
inlinevirtual

Having multiple out for the same ns is a rules violation; Calling on a non-created ident is invalid and may crash.

Implements mongo::KVEngine.

◆ getRecoveryTimestamp()

boost::optional< Timestamp > mongo::WiredTigerKVEngine::getRecoveryTimestamp ( ) const
overridevirtual

◆ getSnapshotManager()

SnapshotManager* mongo::WiredTigerKVEngine::getSnapshotManager ( ) const
inlinefinalvirtual

Return the SnapshotManager for this KVEngine or NULL if not supported.

Pointer remains owned by the StorageEngine, not the caller.

Reimplemented from mongo::KVEngine.

◆ getSortedDataInterface()

virtual SortedDataInterface* mongo::WiredTigerKVEngine::getSortedDataInterface ( OperationContext *  opCtx,
StringData  ident,
const IndexDescriptor *  desc 
)
inlinevirtual

Implements mongo::KVEngine.

◆ getStableTimestamp()

Timestamp mongo::WiredTigerKVEngine::getStableTimestamp ( ) const

These are timestamp access functions for serverStatus to be able to report the actual snapshot window size.

◆ haltOplogManager()

void mongo::WiredTigerKVEngine::haltOplogManager ( )

◆ hasIdent()

bool mongo::WiredTigerKVEngine::hasIdent ( OperationContext *  opCtx,
StringData  ident 
) const
virtual

Implements mongo::KVEngine.

◆ haveDropsQueued()

bool mongo::WiredTigerKVEngine::haveDropsQueued ( ) const

◆ initRsOplogBackgroundThread()

bool mongo::WiredTigerKVEngine::initRsOplogBackgroundThread ( StringData  ns)
static

Initializes a background job to remove excess documents in the oplog collections.

This applies to the capped collections in the local.oplog.* namespaces (specifically local.oplog.rs for replica sets). Returns true if a background job is running for the namespace.

◆ isCacheUnderPressure()

bool mongo::WiredTigerKVEngine::isCacheUnderPressure ( OperationContext *  opCtx) const
overridevirtual

◆ isDurable()

virtual bool mongo::WiredTigerKVEngine::isDurable ( ) const
inlinevirtual

Implements mongo::KVEngine.

◆ isEphemeral()

virtual bool mongo::WiredTigerKVEngine::isEphemeral ( ) const
inlinevirtual

Returns true if the KVEngine is ephemeral – that is, it is NOT persistent and all data is lost after shutdown.

Otherwise, returns false.

Implements mongo::KVEngine.

◆ newRecoveryUnit()

RecoveryUnit * mongo::WiredTigerKVEngine::newRecoveryUnit ( )
virtual

Implements mongo::KVEngine.

◆ okToRename()

Status mongo::WiredTigerKVEngine::okToRename ( OperationContext *  opCtx,
StringData  fromNS,
StringData  toNS,
StringData  ident,
const RecordStore originalRecordStore 
) const
virtual

Reimplemented from mongo::KVEngine.

◆ reconfigure()

int mongo::WiredTigerKVEngine::reconfigure ( const char *  str)

◆ recoverToStableTimestamp()

StatusWith< Timestamp > mongo::WiredTigerKVEngine::recoverToStableTimestamp ( OperationContext *  opCtx)
overridevirtual

◆ repairIdent()

Status mongo::WiredTigerKVEngine::repairIdent ( OperationContext *  opCtx,
StringData  ident 
)
virtual

Implements mongo::KVEngine.

◆ replicationBatchIsComplete()

void mongo::WiredTigerKVEngine::replicationBatchIsComplete ( ) const
overridevirtual

◆ setInitialDataTimestamp()

void mongo::WiredTigerKVEngine::setInitialDataTimestamp ( Timestamp  initialDataTimestamp)
overridevirtual

◆ setInitRsOplogBackgroundThreadCallback()

void mongo::WiredTigerKVEngine::setInitRsOplogBackgroundThreadCallback ( stdx::function< bool(StringData)>  cb)
static

Sets the implementation for initRsOplogBackgroundThread (allowing tests to skip the background job, for example).

Intended to be called from a MONGO_INITIALIZER and therefore in a single threaded context.

◆ setJournalListener()

void mongo::WiredTigerKVEngine::setJournalListener ( JournalListener jl)
finalvirtual

Sets a new JournalListener, which is used to alert the rest of the system about journaled write progress.

Implements mongo::KVEngine.

◆ setOldestTimestamp()

void mongo::WiredTigerKVEngine::setOldestTimestamp ( Timestamp  newOldestTimestamp)
overridevirtual

◆ setOldestTimestampFromStable()

void mongo::WiredTigerKVEngine::setOldestTimestampFromStable ( )
overridevirtual

◆ setRecordStoreExtraOptions()

void mongo::WiredTigerKVEngine::setRecordStoreExtraOptions ( const std::string &  options)

◆ setSortedDataInterfaceExtraOptions()

void mongo::WiredTigerKVEngine::setSortedDataInterfaceExtraOptions ( const std::string &  options)

◆ setStableTimestamp()

void mongo::WiredTigerKVEngine::setStableTimestamp ( Timestamp  stableTimestamp)
overridevirtual

◆ startOplogManager()

void mongo::WiredTigerKVEngine::startOplogManager ( OperationContext *  opCtx,
const std::string &  uri,
WiredTigerRecordStore oplogRecordStore 
)

◆ supportsDirectoryPerDB()

bool mongo::WiredTigerKVEngine::supportsDirectoryPerDB ( ) const
virtual

Returns true if storage engine supports –directoryperdb.

See: http://docs.mongodb.org/manual/reference/program/mongod/#cmdoption&ndash;directoryperdb

Implements mongo::KVEngine.

◆ supportsDocLocking()

bool mongo::WiredTigerKVEngine::supportsDocLocking ( ) const
virtual

This must not change over the lifetime of the engine.

Implements mongo::KVEngine.

◆ supportsReadConcernSnapshot()

bool mongo::WiredTigerKVEngine::supportsReadConcernSnapshot ( ) const
finalvirtual

◆ supportsRecoverToStableTimestamp()

bool mongo::WiredTigerKVEngine::supportsRecoverToStableTimestamp ( ) const
overridevirtual

◆ syncSizeInfo()

void mongo::WiredTigerKVEngine::syncSizeInfo ( bool  sync) const

Member Data Documentation

◆ _backupSession

std::unique_ptr<WiredTigerSession> mongo::WiredTigerKVEngine::_backupSession
private

◆ _canonicalName

std::string mongo::WiredTigerKVEngine::_canonicalName
private

◆ _checkpointThread

std::unique_ptr<WiredTigerCheckpointThread> mongo::WiredTigerKVEngine::_checkpointThread
private

◆ _clockSource

ClockSource* const mongo::WiredTigerKVEngine::_clockSource
private

◆ _conn

WT_CONNECTION* mongo::WiredTigerKVEngine::_conn
private

◆ _dropAllQueuesMutex

stdx::mutex mongo::WiredTigerKVEngine::_dropAllQueuesMutex
mutableprivate

◆ _durable

bool mongo::WiredTigerKVEngine::_durable
private

◆ _ephemeral

bool mongo::WiredTigerKVEngine::_ephemeral
private

◆ _eventHandler

WT_EVENT_HANDLER mongo::WiredTigerKVEngine::_eventHandler
private

◆ _fileVersion

WiredTigerFileVersion mongo::WiredTigerKVEngine::_fileVersion
private

◆ _identToDrop

std::list<std::string> mongo::WiredTigerKVEngine::_identToDrop
private

◆ _identToDropMutex

stdx::mutex mongo::WiredTigerKVEngine::_identToDropMutex
mutableprivate

◆ _indexOptions

std::string mongo::WiredTigerKVEngine::_indexOptions
private

◆ _inRepairMode

const bool mongo::WiredTigerKVEngine::_inRepairMode
private

◆ _journalFlusher

std::unique_ptr<WiredTigerJournalFlusher> mongo::WiredTigerKVEngine::_journalFlusher
private

◆ _oldestTimestamp

Timestamp mongo::WiredTigerKVEngine::_oldestTimestamp
private

◆ _oldestTimestampMutex

stdx::mutex mongo::WiredTigerKVEngine::_oldestTimestampMutex
mutableprivate

◆ _oplogManager

std::unique_ptr<WiredTigerOplogManager> mongo::WiredTigerKVEngine::_oplogManager
private

◆ _oplogManagerCount

std::size_t mongo::WiredTigerKVEngine::_oplogManagerCount = 0
private

◆ _oplogManagerMutex

stdx::mutex mongo::WiredTigerKVEngine::_oplogManagerMutex
mutableprivate

◆ _path

std::string mongo::WiredTigerKVEngine::_path
private

◆ _previousCheckedDropsQueued

Date_t mongo::WiredTigerKVEngine::_previousCheckedDropsQueued
mutableprivate

◆ _readOnly

bool mongo::WiredTigerKVEngine::_readOnly
private

◆ _recoveryTimestamp

Timestamp mongo::WiredTigerKVEngine::_recoveryTimestamp
private

◆ _rsOptions

std::string mongo::WiredTigerKVEngine::_rsOptions
private

◆ _sessionCache

std::unique_ptr<WiredTigerSessionCache> mongo::WiredTigerKVEngine::_sessionCache
private

◆ _sizeStorer

std::unique_ptr<WiredTigerSizeStorer> mongo::WiredTigerKVEngine::_sizeStorer
private

◆ _sizeStorerSyncTracker

ElapsedTracker mongo::WiredTigerKVEngine::_sizeStorerSyncTracker
mutableprivate

◆ _sizeStorerUri

std::string mongo::WiredTigerKVEngine::_sizeStorerUri
private

◆ _stableTimestamp

Timestamp mongo::WiredTigerKVEngine::_stableTimestamp
private

◆ _stableTimestampMutex

stdx::mutex mongo::WiredTigerKVEngine::_stableTimestampMutex
mutableprivate

◆ _wtOpenConfig

std::string mongo::WiredTigerKVEngine::_wtOpenConfig
private

◆ kDefaultJournalDelayMillis

const int mongo::WiredTigerKVEngine::kDefaultJournalDelayMillis = 100
static

The documentation for this class was generated from the following files: