summaryrefslogtreecommitdiff
path: root/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.cpp
diff options
context:
space:
mode:
Diffstat (limited to 'src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.cpp')
-rw-r--r--src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.cpp244
1 files changed, 166 insertions, 78 deletions
diff --git a/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.cpp b/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.cpp
index 5e220754e7e..2d39e7336a3 100644
--- a/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.cpp
+++ b/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.cpp
@@ -83,6 +83,7 @@
#include "mongo/db/storage/wiredtiger/wiredtiger_recovery_unit.h"
#include "mongo/db/storage/wiredtiger/wiredtiger_session_cache.h"
#include "mongo/db/storage/wiredtiger/wiredtiger_size_storer.h"
+#include "mongo/logv2/log.h"
#include "mongo/platform/atomic_word.h"
#include "mongo/util/background.h"
#include "mongo/util/concurrency/idle_thread_block.h"
@@ -197,7 +198,7 @@ public:
virtual void run() {
ThreadClient tc(name(), getGlobalServiceContext());
- LOG(1) << "starting " << name() << " thread";
+ LOGV2_DEBUG(22303, 1, "starting {name} thread", "name"_attr = name());
while (!_shuttingDown.load()) {
{
@@ -210,7 +211,7 @@ public:
_sessionCache->closeExpiredIdleSessions(gWiredTigerSessionCloseIdleTimeSecs.load() *
1000);
}
- LOG(1) << "stopping " << name() << " thread";
+ LOGV2_DEBUG(22304, 1, "stopping {name} thread", "name"_attr = name());
}
void shutdown() {
@@ -245,7 +246,7 @@ public:
virtual void run() {
ThreadClient tc(name(), getGlobalServiceContext());
- LOG(1) << "starting " << name() << " thread";
+ LOGV2_DEBUG(22305, 1, "starting {name} thread", "name"_attr = name());
// Initialize the thread's opCtx.
_uniqueCtx.emplace(tc->makeOperationContext());
@@ -296,7 +297,7 @@ public:
_flushJournalNow = false;
if (_shuttingDown) {
- LOG(1) << "stopping " << name() << " thread";
+ LOGV2_DEBUG(22306, 1, "stopping {name} thread", "name"_attr = name());
_nextSharedPromise->setError(
Status(ErrorCodes::ShutdownInProgress, "The storage catalog is being closed."));
stdx::lock_guard<Latch> lk(_opCtxMutex);
@@ -447,7 +448,7 @@ public:
virtual void run() {
ThreadClient tc(name(), getGlobalServiceContext());
- LOG(1) << "starting " << name() << " thread";
+ LOGV2_DEBUG(22307, 1, "starting {name} thread", "name"_attr = name());
while (!_shuttingDown.load()) {
auto opCtx = tc->makeOperationContext();
@@ -540,16 +541,19 @@ public:
const auto secondsElapsed = durationCount<Seconds>(Date_t::now() - startTime);
if (secondsElapsed >= 30) {
- LOG(1) << "Checkpoint took " << secondsElapsed << " seconds to complete.";
+ LOGV2_DEBUG(22308,
+ 1,
+ "Checkpoint took {secondsElapsed} seconds to complete.",
+ "secondsElapsed"_attr = secondsElapsed);
}
} catch (const WriteConflictException&) {
// Temporary: remove this after WT-3483
- warning() << "Checkpoint encountered a write conflict exception.";
+ LOGV2_WARNING(22346, "Checkpoint encountered a write conflict exception.");
} catch (const AssertionException& exc) {
invariant(ErrorCodes::isShutdownError(exc.code()), exc.what());
}
}
- LOG(1) << "stopping " << name() << " thread";
+ LOGV2_DEBUG(22309, 1, "stopping {name} thread", "name"_attr = name());
}
/**
@@ -575,8 +579,12 @@ public:
invariant(!_hasTriggeredFirstStableCheckpoint);
if (prevStable < initialData && currStable >= initialData) {
_hasTriggeredFirstStableCheckpoint = true;
- log() << "Triggering the first stable checkpoint. Initial Data: " << initialData
- << " PrevStable: " << prevStable << " CurrStable: " << currStable;
+ LOGV2(22310,
+ "Triggering the first stable checkpoint. Initial Data: {initialData} PrevStable: "
+ "{prevStable} CurrStable: {currStable}",
+ "initialData"_attr = initialData,
+ "prevStable"_attr = prevStable,
+ "currStable"_attr = currStable);
stdx::unique_lock<Latch> lock(_mutex);
_condvar.notify_one();
}
@@ -727,8 +735,14 @@ StatusWith<StorageEngine::BackupInformation> getBackupInformationFromBackupCurso
while ((wtRet = dupCursor->next(dupCursor)) == 0) {
uint64_t offset, size, type;
invariantWTOK(dupCursor->get_key(dupCursor, &offset, &size, &type));
- LOG(2) << "Block to copy for incremental backup: filename: " << filePath.string()
- << ", offset: " << offset << ", size: " << size << ", type: " << type;
+ LOGV2_DEBUG(22311,
+ 2,
+ "Block to copy for incremental backup: filename: {filePath_string}, "
+ "offset: {offset}, size: {size}, type: {type}",
+ "filePath_string"_attr = filePath.string(),
+ "offset"_attr = offset,
+ "size"_attr = size,
+ "type"_attr = type);
backupInformation.at(filePath.string()).blocksToCopy.push_back({offset, size});
}
@@ -779,7 +793,10 @@ WiredTigerKVEngine::WiredTigerKVEngine(const std::string& canonicalName,
try {
boost::filesystem::create_directory(journalPath);
} catch (std::exception& e) {
- log() << "error creating journal dir " << journalPath.string() << ' ' << e.what();
+ LOGV2(22312,
+ "error creating journal dir {journalPath_string} {e_what}",
+ "journalPath_string"_attr = journalPath.string(),
+ "e_what"_attr = e.what());
throw;
}
}
@@ -841,8 +858,9 @@ WiredTigerKVEngine::WiredTigerKVEngine(const std::string& canonicalName,
// the normal path without the journal.
if (boost::filesystem::exists(journalPath)) {
string config = ss.str();
- log() << "Detected WT journal files. Running recovery from last checkpoint.";
- log() << "journal to nojournal transition config: " << config;
+ LOGV2(22313, "Detected WT journal files. Running recovery from last checkpoint.");
+ LOGV2(
+ 22314, "journal to nojournal transition config: {config}", "config"_attr = config);
int ret = wiredtiger_open(
path.c_str(), _eventHandler.getWtEventHandler(), config.c_str(), &_conn);
if (ret == EINVAL) {
@@ -856,7 +874,10 @@ WiredTigerKVEngine::WiredTigerKVEngine(const std::string& canonicalName,
try {
boost::filesystem::remove_all(journalPath);
} catch (std::exception& e) {
- error() << "error removing journal dir " << journalPath.string() << ' ' << e.what();
+ LOGV2_ERROR(22355,
+ "error removing journal dir {journalPath_string} {e_what}",
+ "journalPath_string"_attr = journalPath.string(),
+ "e_what"_attr = e.what());
throw;
}
}
@@ -865,7 +886,7 @@ WiredTigerKVEngine::WiredTigerKVEngine(const std::string& canonicalName,
}
string config = ss.str();
- log() << "wiredtiger_open config: " << config;
+ LOGV2(22315, "wiredtiger_open config: {config}", "config"_attr = config);
_openWiredTiger(path, config);
_eventHandler.setStartupSuccessful();
_wtOpenConfig = config;
@@ -907,7 +928,7 @@ WiredTigerKVEngine::WiredTigerKVEngine(const std::string& canonicalName,
_sizeStorerUri = _uri("sizeStorer");
WiredTigerSession session(_conn);
if (!_readOnly && repair && _hasUri(session.getSession(), _sizeStorerUri)) {
- log() << "Repairing size cache";
+ LOGV2(22316, "Repairing size cache");
auto status = _salvageIfNeeded(_sizeStorerUri.c_str());
if (status.code() != ErrorCodes::DataModifiedByRepair)
@@ -998,28 +1019,30 @@ void WiredTigerKVEngine::_openWiredTiger(const std::string& path, const std::str
return;
}
- warning() << "Failed to start up WiredTiger under any compatibility version.";
+ LOGV2_WARNING(22347, "Failed to start up WiredTiger under any compatibility version.");
if (ret == EINVAL) {
fassertFailedNoTrace(28561);
}
if (ret == WT_TRY_SALVAGE) {
- warning() << "WiredTiger metadata corruption detected";
+ LOGV2_WARNING(22348, "WiredTiger metadata corruption detected");
if (!_inRepairMode) {
- severe() << kWTRepairMsg;
+ LOGV2_FATAL(22362, "{kWTRepairMsg}", "kWTRepairMsg"_attr = kWTRepairMsg);
fassertFailedNoTrace(50944);
}
}
- severe() << "Reason: " << wtRCToStatus(ret).reason();
+ LOGV2_FATAL(22363,
+ "Reason: {wtRCToStatus_ret_reason}",
+ "wtRCToStatus_ret_reason"_attr = wtRCToStatus(ret).reason());
if (!_inRepairMode) {
fassertFailedNoTrace(28595);
}
// Always attempt to salvage metadata regardless of error code when in repair mode.
- warning() << "Attempting to salvage WiredTiger metadata";
+ LOGV2_WARNING(22349, "Attempting to salvage WiredTiger metadata");
configStr = wtOpenConfig + ",salvage=true";
ret = wiredtiger_open(path.c_str(), wtEventHandler, configStr.c_str(), &_conn);
if (!ret) {
@@ -1028,12 +1051,15 @@ void WiredTigerKVEngine::_openWiredTiger(const std::string& path, const std::str
return;
}
- severe() << "Failed to salvage WiredTiger metadata: " + wtRCToStatus(ret).reason();
+ LOGV2_FATAL(22364,
+ "{Failed_to_salvage_WiredTiger_metadata_wtRCToStatus_ret_reason}",
+ "Failed_to_salvage_WiredTiger_metadata_wtRCToStatus_ret_reason"_attr =
+ "Failed to salvage WiredTiger metadata: " + wtRCToStatus(ret).reason());
fassertFailedNoTrace(50947);
}
void WiredTigerKVEngine::cleanShutdown() {
- log() << "WiredTigerKVEngine shutting down";
+ LOGV2(22317, "WiredTigerKVEngine shutting down");
if (!_readOnly)
syncSizeInfo(true);
if (!_conn) {
@@ -1042,19 +1068,19 @@ void WiredTigerKVEngine::cleanShutdown() {
// these must be the last things we do before _conn->close();
if (_sessionSweeper) {
- log() << "Shutting down session sweeper thread";
+ LOGV2(22318, "Shutting down session sweeper thread");
_sessionSweeper->shutdown();
- log() << "Finished shutting down session sweeper thread";
+ LOGV2(22319, "Finished shutting down session sweeper thread");
}
if (_journalFlusher) {
- log() << "Shutting down journal flusher thread";
+ LOGV2(22320, "Shutting down journal flusher thread");
_journalFlusher->shutdown();
- log() << "Finished shutting down journal flusher thread";
+ LOGV2(22321, "Finished shutting down journal flusher thread");
}
if (_checkpointThread) {
- log() << "Shutting down checkpoint thread";
+ LOGV2(22322, "Shutting down checkpoint thread");
_checkpointThread->shutdown();
- log() << "Finished shutting down checkpoint thread";
+ LOGV2(22323, "Finished shutting down checkpoint thread");
}
LOG_FOR_RECOVERY(2) << "Shutdown timestamps. StableTimestamp: " << _stableTimestamp.load()
<< " Initial data timestamp: " << _initialDataTimestamp.load();
@@ -1080,12 +1106,15 @@ void WiredTigerKVEngine::cleanShutdown() {
}
if (_fileVersion.shouldDowngrade(_readOnly, _inRepairMode, !_recoveryTimestamp.isNull())) {
- log() << "Downgrading WiredTiger datafiles.";
+ LOGV2(22324, "Downgrading WiredTiger datafiles.");
invariantWTOK(_conn->close(_conn, closeConfig.c_str()));
invariantWTOK(wiredtiger_open(
_path.c_str(), _eventHandler.getWtEventHandler(), _wtOpenConfig.c_str(), &_conn));
- LOG(1) << "Downgrade compatibility configuration: " << _fileVersion.getDowngradeString();
+ LOGV2_DEBUG(22325,
+ 1,
+ "Downgrade compatibility configuration: {fileVersion_getDowngradeString}",
+ "fileVersion_getDowngradeString"_attr = _fileVersion.getDowngradeString());
invariantWTOK(_conn->reconfigure(_conn, _fileVersion.getDowngradeString().c_str()));
}
@@ -1098,9 +1127,11 @@ void WiredTigerKVEngine::cleanShutdown() {
if (stableTimestamp >= initialDataTimestamp) {
invariantWTOK(_conn->close(_conn, closeConfig.c_str()));
} else {
- log() << "Skipping checkpoint during clean shutdown because stableTimestamp ("
- << stableTimestamp << ") is less than the initialDataTimestamp ("
- << initialDataTimestamp << ")";
+ LOGV2(22326,
+ "Skipping checkpoint during clean shutdown because stableTimestamp "
+ "({stableTimestamp}) is less than the initialDataTimestamp ({initialDataTimestamp})",
+ "stableTimestamp"_attr = stableTimestamp,
+ "initialDataTimestamp"_attr = initialDataTimestamp);
quickExit(EXIT_SUCCESS);
}
_conn = nullptr;
@@ -1140,7 +1171,7 @@ Status WiredTigerKVEngine::_salvageIfNeeded(const char* uri) {
int rc = (session->verify)(session, uri, nullptr);
if (rc == 0) {
- log() << "Verify succeeded on uri " << uri << ". Not salvaging.";
+ LOGV2(22327, "Verify succeeded on uri {uri}. Not salvaging.", "uri"_attr = uri);
return Status::OK();
}
@@ -1148,28 +1179,34 @@ Status WiredTigerKVEngine::_salvageIfNeeded(const char* uri) {
// SERVER-16457: verify and salvage are occasionally failing with EBUSY. For now we
// lie and return OK to avoid breaking tests. This block should go away when that ticket
// is resolved.
- error()
- << "Verify on " << uri << " failed with EBUSY. "
- << "This means the collection was being accessed. No repair is necessary unless other "
- "errors are reported.";
+ LOGV2_ERROR(22356,
+ "Verify on {uri} failed with EBUSY. This means the collection was being "
+ "accessed. No repair is necessary unless other "
+ "errors are reported.",
+ "uri"_attr = uri);
return Status::OK();
}
if (rc == ENOENT) {
- warning() << "Data file is missing for " << uri
- << ". Attempting to drop and re-create the collection.";
+ LOGV2_WARNING(
+ 22350,
+ "Data file is missing for {uri}. Attempting to drop and re-create the collection.",
+ "uri"_attr = uri);
return _rebuildIdent(session, uri);
}
- log() << "Verify failed on uri " << uri << ". Running a salvage operation.";
+ LOGV2(22328, "Verify failed on uri {uri}. Running a salvage operation.", "uri"_attr = uri);
auto status = wtRCToStatus(session->salvage(session, uri, nullptr), "Salvage failed:");
if (status.isOK()) {
return {ErrorCodes::DataModifiedByRepair, str::stream() << "Salvaged data for " << uri};
}
- warning() << "Salvage failed for uri " << uri << ": " << status.reason()
- << ". The file will be moved out of the way and a new ident will be created.";
+ LOGV2_WARNING(22351,
+ "Salvage failed for uri {uri}: {status_reason}. The file will be moved out of "
+ "the way and a new ident will be created.",
+ "uri"_attr = uri,
+ "status_reason"_attr = status.reason());
// If the data is unsalvageable, we should completely rebuild the ident.
return _rebuildIdent(session, uri);
@@ -1184,8 +1221,10 @@ Status WiredTigerKVEngine::_rebuildIdent(WT_SESSION* session, const char* uri) {
auto filePath = getDataFilePathForIdent(identName);
if (filePath) {
const boost::filesystem::path corruptFile(filePath->string() + ".corrupt");
- warning() << "Moving data file " << filePath->string() << " to backup as "
- << corruptFile.string();
+ LOGV2_WARNING(22352,
+ "Moving data file {filePath_string} to backup as {corruptFile_string}",
+ "filePath_string"_attr = filePath->string(),
+ "corruptFile_string"_attr = corruptFile.string());
auto status = fsyncRename(filePath.get(), corruptFile);
if (!status.isOK()) {
@@ -1193,34 +1232,37 @@ Status WiredTigerKVEngine::_rebuildIdent(WT_SESSION* session, const char* uri) {
}
}
- warning() << "Rebuilding ident " << identName;
+ LOGV2_WARNING(22353, "Rebuilding ident {identName}", "identName"_attr = identName);
// This is safe to call after moving the file because it only reads from the metadata, and not
// the data file itself.
auto swMetadata = WiredTigerUtil::getMetadataCreate(session, uri);
if (!swMetadata.isOK()) {
- error() << "Failed to get metadata for " << uri;
+ LOGV2_ERROR(22357, "Failed to get metadata for {uri}", "uri"_attr = uri);
return swMetadata.getStatus();
}
int rc = session->drop(session, uri, nullptr);
if (rc != 0) {
- error() << "Failed to drop " << uri;
+ LOGV2_ERROR(22358, "Failed to drop {uri}", "uri"_attr = uri);
return wtRCToStatus(rc);
}
rc = session->create(session, uri, swMetadata.getValue().c_str());
if (rc != 0) {
- error() << "Failed to create " << uri << " with config: " << swMetadata.getValue();
+ LOGV2_ERROR(22359,
+ "Failed to create {uri} with config: {swMetadata_getValue}",
+ "uri"_attr = uri,
+ "swMetadata_getValue"_attr = swMetadata.getValue());
return wtRCToStatus(rc);
}
- log() << "Successfully re-created " << uri << ".";
+ LOGV2(22329, "Successfully re-created {uri}.", "uri"_attr = uri);
return {ErrorCodes::DataModifiedByRepair,
str::stream() << "Re-created empty data file for " << uri};
}
void WiredTigerKVEngine::flushAllFiles(OperationContext* opCtx, bool callerHoldsReadLock) {
- LOG(1) << "WiredTigerKVEngine::flushAllFiles";
+ LOGV2_DEBUG(22330, 1, "WiredTigerKVEngine::flushAllFiles");
if (_ephemeral) {
return;
}
@@ -1285,7 +1327,7 @@ Status WiredTigerKVEngine::disableIncrementalBackup(OperationContext* opCtx) {
int wtRet =
session->open_cursor(session, "backup:", nullptr, "incremental=(force_stop=true)", &cursor);
if (wtRet != 0) {
- error() << "Could not open a backup cursor to disable incremental backups";
+ LOGV2_ERROR(22360, "Could not open a backup cursor to disable incremental backups");
return wtRCToStatus(wtRet);
}
@@ -1448,8 +1490,12 @@ Status WiredTigerKVEngine::createGroupedRecordStore(OperationContext* opCtx,
string uri = _uri(ident);
WT_SESSION* s = session.getSession();
- LOG(2) << "WiredTigerKVEngine::createRecordStore ns: " << ns << " uri: " << uri
- << " config: " << config;
+ LOGV2_DEBUG(22331,
+ 2,
+ "WiredTigerKVEngine::createRecordStore ns: {ns} uri: {uri} config: {config}",
+ "ns"_attr = ns,
+ "uri"_attr = uri,
+ "config"_attr = config);
return wtRCToStatus(s->create(s, uri.c_str(), config.c_str()));
}
@@ -1477,21 +1523,30 @@ Status WiredTigerKVEngine::recoverOrphanedIdent(OperationContext* opCtx,
boost::filesystem::path tmpFile{*identFilePath};
tmpFile += ".tmp";
- log() << "Renaming data file " + identFilePath->string() + " to temporary file " +
- tmpFile.string();
+ LOGV2(22332,
+ "{Renaming_data_file_identFilePath_string_to_temporary_file_tmpFile_string}",
+ "Renaming_data_file_identFilePath_string_to_temporary_file_tmpFile_string"_attr =
+ "Renaming data file " + identFilePath->string() + " to temporary file " +
+ tmpFile.string());
auto status = fsyncRename(identFilePath.get(), tmpFile);
if (!status.isOK()) {
return status;
}
- log() << "Creating new RecordStore for collection " << nss << " with UUID: " << options.uuid;
+ LOGV2(22333,
+ "Creating new RecordStore for collection {nss} with UUID: {options_uuid}",
+ "nss"_attr = nss,
+ "options_uuid"_attr = options.uuid);
status = createGroupedRecordStore(opCtx, nss.ns(), ident, options, KVPrefix::kNotPrefixed);
if (!status.isOK()) {
return status;
}
- log() << "Moving orphaned data file back as " + identFilePath->string();
+ LOGV2(22334,
+ "{Moving_orphaned_data_file_back_as_identFilePath_string}",
+ "Moving_orphaned_data_file_back_as_identFilePath_string"_attr =
+ "Moving orphaned data file back as " + identFilePath->string());
boost::filesystem::remove(*identFilePath, ec);
if (ec) {
@@ -1507,7 +1562,9 @@ Status WiredTigerKVEngine::recoverOrphanedIdent(OperationContext* opCtx,
return status;
}
- log() << "Salvaging ident " + ident;
+ LOGV2(22335,
+ "{Salvaging_ident_ident}",
+ "Salvaging_ident_ident"_attr = "Salvaging ident " + ident);
WiredTigerSession sessionWrapper(_conn);
WT_SESSION* session = sessionWrapper.getSession();
@@ -1517,7 +1574,9 @@ Status WiredTigerKVEngine::recoverOrphanedIdent(OperationContext* opCtx,
return {ErrorCodes::DataModifiedByRepair,
str::stream() << "Salvaged data for ident " << ident};
}
- warning() << "Could not salvage data. Rebuilding ident: " << status.reason();
+ LOGV2_WARNING(22354,
+ "Could not salvage data. Rebuilding ident: {status_reason}",
+ "status_reason"_attr = status.reason());
// If the data is unsalvageable, we should completely rebuild the ident.
return _rebuildIdent(session, _uri(ident).c_str());
@@ -1609,8 +1668,13 @@ Status WiredTigerKVEngine::createGroupedSortedDataInterface(OperationContext* op
std::string config = result.getValue();
- LOG(2) << "WiredTigerKVEngine::createSortedDataInterface ns: " << collection->ns()
- << " ident: " << ident << " config: " << config;
+ LOGV2_DEBUG(22336,
+ 2,
+ "WiredTigerKVEngine::createSortedDataInterface ns: {collection_ns} ident: {ident} "
+ "config: {config}",
+ "collection_ns"_attr = collection->ns(),
+ "ident"_attr = ident,
+ "config"_attr = config);
return wtRCToStatus(WiredTigerIndex::Create(opCtx, _uri(ident), config));
}
@@ -1639,8 +1703,11 @@ std::unique_ptr<RecordStore> WiredTigerKVEngine::makeTemporaryRecordStore(Operat
std::string uri = _uri(ident);
WT_SESSION* session = wtSession.getSession();
- LOG(2) << "WiredTigerKVEngine::createTemporaryRecordStore uri: " << uri
- << " config: " << config;
+ LOGV2_DEBUG(22337,
+ 2,
+ "WiredTigerKVEngine::createTemporaryRecordStore uri: {uri} config: {config}",
+ "uri"_attr = uri,
+ "config"_attr = config);
uassertStatusOK(wtRCToStatus(session->create(session, uri.c_str(), config.c_str())));
WiredTigerRecordStore::Params params;
@@ -1677,7 +1744,7 @@ Status WiredTigerKVEngine::dropIdent(OperationContext* opCtx, RecoveryUnit* ru,
int ret = session.getSession()->drop(
session.getSession(), uri.c_str(), "force,checkpoint_wait=false");
- LOG(1) << "WT drop of " << uri << " res " << ret;
+ LOGV2_DEBUG(22338, 1, "WT drop of {uri} res {ret}", "uri"_attr = uri, "ret"_attr = ret);
if (ret == 0) {
// yay, it worked
@@ -1759,7 +1826,11 @@ void WiredTigerKVEngine::dropSomeQueuedIdents() {
if (tenPercentQueue > 10)
numToDelete = tenPercentQueue;
- LOG(1) << "WT Queue is: " << numInQueue << " attempting to drop: " << numToDelete << " tables";
+ LOGV2_DEBUG(22339,
+ 1,
+ "WT Queue is: {numInQueue} attempting to drop: {numToDelete} tables",
+ "numInQueue"_attr = numInQueue,
+ "numToDelete"_attr = numToDelete);
for (int i = 0; i < numToDelete; i++) {
string uri;
{
@@ -1771,7 +1842,8 @@ void WiredTigerKVEngine::dropSomeQueuedIdents() {
}
int ret = session.getSession()->drop(
session.getSession(), uri.c_str(), "force,checkpoint_wait=false");
- LOG(1) << "WT queued drop of " << uri << " res " << ret;
+ LOGV2_DEBUG(
+ 22340, 1, "WT queued drop of {uri} res {ret}", "uri"_attr = uri, "ret"_attr = ret);
if (ret == EBUSY) {
stdx::lock_guard<Latch> lk(_identToDropMutex);
@@ -1865,11 +1937,14 @@ void WiredTigerKVEngine::_ensureIdentPath(StringData ident) {
boost::filesystem::path subdir = _path;
subdir /= dir.toString();
if (!boost::filesystem::exists(subdir)) {
- LOG(1) << "creating subdirectory: " << dir;
+ LOGV2_DEBUG(22341, 1, "creating subdirectory: {dir}", "dir"_attr = dir);
try {
boost::filesystem::create_directory(subdir);
} catch (const std::exception& e) {
- error() << "error creating path " << subdir.string() << ' ' << e.what();
+ LOGV2_ERROR(22361,
+ "error creating path {subdir_string} {e_what}",
+ "subdir_string"_attr = subdir.string(),
+ "e_what"_attr = e.what());
throw;
}
}
@@ -1970,14 +2045,20 @@ void WiredTigerKVEngine::setOldestTimestamp(Timestamp newOldestTimestamp, bool f
newOldestTimestamp.asULL());
invariantWTOK(_conn->set_timestamp(_conn, oldestTSConfigString.c_str()));
_oldestTimestamp.store(newOldestTimestamp.asULL());
- LOG(2) << "oldest_timestamp and commit_timestamp force set to " << newOldestTimestamp;
+ LOGV2_DEBUG(22342,
+ 2,
+ "oldest_timestamp and commit_timestamp force set to {newOldestTimestamp}",
+ "newOldestTimestamp"_attr = newOldestTimestamp);
} else {
auto oldestTSConfigString = "oldest_timestamp={:x}"_format(newOldestTimestamp.asULL());
invariantWTOK(_conn->set_timestamp(_conn, oldestTSConfigString.c_str()));
// set_timestamp above ignores backwards in time if 'force' is not set.
if (_oldestTimestamp.load() < newOldestTimestamp.asULL())
_oldestTimestamp.store(newOldestTimestamp.asULL());
- LOG(2) << "oldest_timestamp set to " << newOldestTimestamp;
+ LOGV2_DEBUG(22343,
+ 2,
+ "oldest_timestamp set to {newOldestTimestamp}",
+ "newOldestTimestamp"_attr = newOldestTimestamp);
}
}
@@ -2012,7 +2093,10 @@ Timestamp WiredTigerKVEngine::_calculateHistoryLagFromStableTimestamp(Timestamp
}
void WiredTigerKVEngine::setInitialDataTimestamp(Timestamp initialDataTimestamp) {
- LOG(2) << "Setting initial data timestamp. Value: " << initialDataTimestamp;
+ LOGV2_DEBUG(22344,
+ 2,
+ "Setting initial data timestamp. Value: {initialDataTimestamp}",
+ "initialDataTimestamp"_attr = initialDataTimestamp);
_initialDataTimestamp.store(initialDataTimestamp.asULL());
}
@@ -2038,7 +2122,7 @@ bool WiredTigerKVEngine::_canRecoverToStableTimestamp() const {
StatusWith<Timestamp> WiredTigerKVEngine::recoverToStableTimestamp(OperationContext* opCtx) {
if (!supportsRecoverToStableTimestamp()) {
- severe() << "WiredTiger is configured to not support recover to a stable timestamp";
+ LOGV2_FATAL(22365, "WiredTiger is configured to not support recover to a stable timestamp");
fassertFailed(50665);
}
@@ -2111,7 +2195,8 @@ Timestamp WiredTigerKVEngine::getOldestOpenReadTimestamp() const {
boost::optional<Timestamp> WiredTigerKVEngine::getRecoveryTimestamp() const {
if (!supportsRecoveryTimestamp()) {
- severe() << "WiredTiger is configured to not support providing a recovery timestamp";
+ LOGV2_FATAL(22366,
+ "WiredTiger is configured to not support providing a recovery timestamp");
fassertFailed(50745);
}
@@ -2157,7 +2242,10 @@ StatusWith<Timestamp> WiredTigerKVEngine::getOplogNeededForRollback() const {
if (status.isOK()) {
oldestActiveTransactionTimestamp.swap(status.getValue());
} else {
- LOG(1) << "getting oldest active transaction timestamp: " << status.getStatus();
+ LOGV2_DEBUG(22345,
+ 1,
+ "getting oldest active transaction timestamp: {status_getStatus}",
+ "status_getStatus"_attr = status.getStatus());
return status.getStatus();
}
}