summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorLouis Williams <louis.williams@mongodb.com>2021-12-16 17:54:23 +0000
committerEvergreen Agent <no-reply@evergreen.mongodb.com>2021-12-21 22:32:05 +0000
commit78051c291e3d59d167f10ebe7e0770355875896d (patch)
tree069156a5777c3ac8cd1fe60fcf86781fead93499
parent3509ce6cc788062a95af0639f05db6dd1f59c550 (diff)
downloadmongo-78051c291e3d59d167f10ebe7e0770355875896d.tar.gz
SERVER-61754 dbCheck does not hold strong locks during batches
The dbCheck command accepts an optional (default true) parameter, 'snapshotRead', that uses point-in-time reads to check each batch. When true, dbCheck does not block writes. When set to false, dbCheck reverts to blocking writes. (cherry picked from commit da10567348cf7883e2ec40a441e38efc17c23385) (cherry picked from commit 46cc35de8bd699cc7b5c796fc66d6c6c4cf46218)
-rw-r--r--buildscripts/resmokeconfig/suites/concurrency_replication.yml2
-rw-r--r--buildscripts/resmokeconfig/suites/concurrency_sharded_multi_stmt_txn_with_balancer.yml2
-rw-r--r--buildscripts/resmokeconfig/suites/concurrency_sharded_replication.yml2
-rw-r--r--jstests/noPassthrough/dbcheck_no_history_on_secondary.js84
-rw-r--r--jstests/replsets/dbcheck.js21
-rw-r--r--src/mongo/db/commands/dbcheck.cpp101
-rw-r--r--src/mongo/db/concurrency/deferred_writer.cpp26
-rw-r--r--src/mongo/db/concurrency/deferred_writer.h2
-rw-r--r--src/mongo/db/repl/dbcheck.cpp117
-rw-r--r--src/mongo/db/repl/dbcheck.h4
-rw-r--r--src/mongo/db/repl/dbcheck.idl9
11 files changed, 263 insertions, 107 deletions
diff --git a/buildscripts/resmokeconfig/suites/concurrency_replication.yml b/buildscripts/resmokeconfig/suites/concurrency_replication.yml
index 4e3e13aca9d..c2b9e4baa4f 100644
--- a/buildscripts/resmokeconfig/suites/concurrency_replication.yml
+++ b/buildscripts/resmokeconfig/suites/concurrency_replication.yml
@@ -46,6 +46,4 @@ executor:
oplogSize: 1024
set_parameters:
enableTestCommands: 1
- # TODO (SERVER-61754): dbCheck takes strong locks which causes transactions to timeout
- maxTransactionLockRequestTimeoutMillis: 3000
num_nodes: 3
diff --git a/buildscripts/resmokeconfig/suites/concurrency_sharded_multi_stmt_txn_with_balancer.yml b/buildscripts/resmokeconfig/suites/concurrency_sharded_multi_stmt_txn_with_balancer.yml
index 38d15789b4d..f45073c2857 100644
--- a/buildscripts/resmokeconfig/suites/concurrency_sharded_multi_stmt_txn_with_balancer.yml
+++ b/buildscripts/resmokeconfig/suites/concurrency_sharded_multi_stmt_txn_with_balancer.yml
@@ -187,8 +187,6 @@ executor:
mongod_options:
set_parameters:
enableTestCommands: 1
- # TODO (SERVER-61754): dbCheck takes strong locks which causes transactions to timeout
- maxTransactionLockRequestTimeoutMillis: 3000
num_rs_nodes_per_shard: 3
num_shards: 2
num_mongos: 2
diff --git a/buildscripts/resmokeconfig/suites/concurrency_sharded_replication.yml b/buildscripts/resmokeconfig/suites/concurrency_sharded_replication.yml
index d01b29b9f44..dd4e2e55a02 100644
--- a/buildscripts/resmokeconfig/suites/concurrency_sharded_replication.yml
+++ b/buildscripts/resmokeconfig/suites/concurrency_sharded_replication.yml
@@ -150,8 +150,6 @@ executor:
mongod_options:
set_parameters:
enableTestCommands: 1
- # TODO (SERVER-61754): dbCheck takes strong locks which causes transactions to timeout
- maxTransactionLockRequestTimeoutMillis: 3000
num_rs_nodes_per_shard: 3
num_shards: 2
num_mongos: 2
diff --git a/jstests/noPassthrough/dbcheck_no_history_on_secondary.js b/jstests/noPassthrough/dbcheck_no_history_on_secondary.js
new file mode 100644
index 00000000000..95cc2413a19
--- /dev/null
+++ b/jstests/noPassthrough/dbcheck_no_history_on_secondary.js
@@ -0,0 +1,84 @@
+/**
+ * Confirms that a dbCheck batch operation logs an error in the health log of a secondary that does
+ * not have enough available history.
+ *
+ * @tags: [
+ * multiversion_incompatible,
+ * requires_replication,
+ * ]
+ */
+(function() {
+"use strict";
+
+load("jstests/libs/fail_point_util.js");
+
+const replTest = new ReplSetTest({
+ name: "dbcheck_lag",
+ nodes: 2,
+ nodeOptions: {
+ setParameter: {
+ // Set the history window to ensure that dbCheck does not cause the server to crash
+ // even when no history is available.
+ maxTargetSnapshotHistoryWindowInSeconds: 0,
+ }
+ }
+});
+replTest.startSet();
+replTest.initiate();
+
+const primary = replTest.getPrimary();
+const secondary = replTest.getSecondary();
+
+const testDB = primary.getDB('test');
+for (let i = 0; i < 100; i++) {
+ assert.commandWorked(testDB.foo.insert({a: i}));
+}
+
+const sleepMs = 3000;
+const fp = configureFailPoint(primary, 'SleepDbCheckInBatch', {sleepMs: sleepMs});
+
+// Returns immediately and starts a background task.
+assert.commandWorked(testDB.getSiblingDB('test').runCommand({dbCheck: 1}));
+
+// Write some data to advance the durable timestamp while we're waiting for dbCheck to run.
+for (let i = 0; i < 100; i++) {
+ assert.commandWorked(testDB.foo.insert({_id: i}));
+}
+
+fp.off();
+
+// Wait for primary to complete the task and the secondaries to process. Note that we still have to
+// wait for the health log entries to appear because they are logged shortly after processing
+// batches.
+const dbCheckCompleted = (db) => {
+ return db.currentOp().inprog.filter(x => x["desc"] == "dbCheck")[0] === undefined;
+};
+assert.soon(() => dbCheckCompleted(testDB), "dbCheck timed out");
+replTest.awaitReplication();
+
+{
+ // Expect no errors on the primary. Health log write is logged after batch is replicated.
+ const healthlog = primary.getDB('local').system.healthlog;
+ assert.soon(() => healthlog.find().hasNext(), 'expected health log to not be empty');
+
+ const errors = healthlog.find({severity: 'error'});
+ assert(!errors.hasNext(), () => 'expected no errors, found: ' + tojson(errors.next()));
+}
+
+{
+ // Expect an error on the secondary.
+ const healthlog = secondary.getDB('local').system.healthlog;
+ assert.soon(() => healthlog.find().hasNext(), 'expected health log to not be empty');
+
+ const errors = healthlog.find({severity: 'error'});
+ assert(errors.hasNext(), 'expected error, found none');
+
+ const error = errors.next();
+ assert.eq(
+ false, error.data.success, "expected failure, found success. log entry: " + tojson(error));
+ assert(error.data.error.includes("SnapshotTooOld"),
+ "expected to find SnapshotTooOld error. log entry: " + tojson(error));
+}
+
+replTest.stopSet();
+})();
diff --git a/jstests/replsets/dbcheck.js b/jstests/replsets/dbcheck.js
index af3be232c34..66f224f4083 100644
--- a/jstests/replsets/dbcheck.js
+++ b/jstests/replsets/dbcheck.js
@@ -5,6 +5,7 @@
* # We need persistence as we temporarily restart nodes as standalones.
* requires_persistence,
* assumes_against_mongod_not_mongos,
+ * requires_fcv_50,
* ]
*/
@@ -201,6 +202,25 @@ function simpleTestConsistent() {
});
}
+function simpleTestNonSnapshot() {
+ let primary = replSet.getPrimary();
+ clearLog();
+
+ assert.neq(primary, undefined);
+ let db = primary.getDB(dbName);
+ assert.commandWorked(db.runCommand({"dbCheck": multiBatchSimpleCollName, snapshotRead: false}));
+
+ awaitDbCheckCompletion(db, multiBatchSimpleCollName);
+
+ checkLogAllConsistent(primary);
+ checkTotalCounts(primary, db[multiBatchSimpleCollName]);
+
+ forEachSecondary(function(secondary) {
+ checkLogAllConsistent(secondary);
+ checkTotalCounts(secondary, secondary.getDB(dbName)[multiBatchSimpleCollName]);
+ });
+}
+
// Same thing, but now with concurrent updates.
function concurrentTestConsistent() {
let master = replSet.getPrimary();
@@ -229,6 +249,7 @@ function concurrentTestConsistent() {
}
simpleTestConsistent();
+simpleTestNonSnapshot();
concurrentTestConsistent();
// Test the various other parameters.
diff --git a/src/mongo/db/commands/dbcheck.cpp b/src/mongo/db/commands/dbcheck.cpp
index 1cd320d9860..0a6d785b404 100644
--- a/src/mongo/db/commands/dbcheck.cpp
+++ b/src/mongo/db/commands/dbcheck.cpp
@@ -67,6 +67,7 @@ struct DbCheckCollectionInfo {
int64_t maxDocsPerBatch;
int64_t maxBytesPerBatch;
int64_t maxBatchTimeMillis;
+ bool snapshotRead;
};
/**
@@ -104,7 +105,8 @@ std::unique_ptr<DbCheckRun> singleCollectionRun(OperationContext* opCtx,
maxRate,
maxDocsPerBatch,
maxBytesPerBatch,
- maxBatchTimeMillis};
+ maxBatchTimeMillis,
+ invocation.getSnapshotRead()};
auto result = std::make_unique<DbCheckRun>();
result->push_back(info);
return result;
@@ -137,7 +139,8 @@ std::unique_ptr<DbCheckRun> fullDatabaseRun(OperationContext* opCtx,
rate,
maxDocsPerBatch,
maxBytesPerBatch,
- maxBatchTimeMillis};
+ maxBatchTimeMillis,
+ invocation.getSnapshotRead()};
result->push_back(info);
return true;
};
@@ -268,25 +271,37 @@ private:
return;
}
- std::unique_ptr<HealthLogEntry> entry;
-
if (!result.isOK()) {
- auto code = result.getStatus().code();
+ bool retryable = false;
+ std::unique_ptr<HealthLogEntry> entry;
+
+ const auto code = result.getStatus().code();
if (code == ErrorCodes::LockTimeout) {
+ retryable = true;
entry = dbCheckWarningHealthLogEntry(
info.nss,
"retrying dbCheck batch after timeout due to lock unavailability",
OplogEntriesEnum::Batch,
result.getStatus());
- HealthLog::get(Client::getCurrent()->getServiceContext()).log(*entry);
- continue;
- }
- if (code == ErrorCodes::NamespaceNotFound) {
+ } else if (code == ErrorCodes::SnapshotUnavailable) {
+ retryable = true;
+ entry = dbCheckWarningHealthLogEntry(
+ info.nss,
+ "retrying dbCheck batch after conflict with pending catalog operation",
+ OplogEntriesEnum::Batch,
+ result.getStatus());
+ } else if (code == ErrorCodes::NamespaceNotFound) {
entry = dbCheckWarningHealthLogEntry(
info.nss,
"abandoning dbCheck batch because collection no longer exists",
OplogEntriesEnum::Batch,
result.getStatus());
+ } else if (code == ErrorCodes::IndexNotFound) {
+ entry = dbCheckWarningHealthLogEntry(
+ info.nss,
+ "skipping dbCheck on collection because it is missing an _id index",
+ OplogEntriesEnum::Batch,
+ result.getStatus());
} else if (ErrorCodes::isA<ErrorCategory::NotPrimaryError>(code)) {
entry = dbCheckWarningHealthLogEntry(
info.nss,
@@ -300,21 +315,23 @@ private:
result.getStatus());
}
HealthLog::get(Client::getCurrent()->getServiceContext()).log(*entry);
+ if (retryable) {
+ continue;
+ }
return;
- } else {
- auto stats = result.getValue();
- entry = dbCheckBatchEntry(info.nss,
- stats.nDocs,
- stats.nBytes,
- stats.md5,
- stats.md5,
- start,
- stats.lastKey,
- stats.time);
- HealthLog::get(Client::getCurrent()->getServiceContext()).log(*entry);
}
auto stats = result.getValue();
+ auto entry = dbCheckBatchEntry(info.nss,
+ stats.nDocs,
+ stats.nBytes,
+ stats.md5,
+ stats.md5,
+ start,
+ stats.lastKey,
+ stats.readTimestamp,
+ stats.time);
+ HealthLog::get(Client::getCurrent()->getServiceContext()).log(*entry);
start = stats.lastKey;
@@ -351,6 +368,7 @@ private:
BSONKey lastKey;
std::string md5;
repl::OpTime time;
+ boost::optional<Timestamp> readTimestamp;
};
// Set if the job cannot proceed.
@@ -363,6 +381,17 @@ private:
const BSONKey& first,
int64_t batchDocs,
int64_t batchBytes) {
+ auto lockMode = MODE_S;
+ if (info.snapshotRead) {
+ // Each batch will read at the latest no-overlap point, which is the all_durable
+ // timestamp on primaries. We assume that the history window on secondaries is always
+ // longer than the time it takes between starting and replicating a batch on the
+ // primary. Otherwise, the readTimestamp will not be available on a secondary by the
+ // time it processes the oplog entry.
+ lockMode = MODE_IS;
+ opCtx->recoveryUnit()->setTimestampReadSource(RecoveryUnit::ReadSource::kNoOverlap);
+ }
+
BatchStats result;
auto timeoutMs = Milliseconds(gDbCheckCollectionTryLockTimeoutMillis.load());
const auto initialBackoffMs =
@@ -370,14 +399,14 @@ private:
auto backoffMs = initialBackoffMs;
for (int attempt = 1;; attempt++) {
try {
- // Try to acquire collection lock in S mode with increasing timeout and bounded
- // exponential backoff.
+ // Try to acquire collection lock with increasing timeout and bounded exponential
+ // backoff.
auto const lockDeadline = Date_t::now() + timeoutMs;
timeoutMs *= 2;
AutoGetCollection agc(opCtx,
info.nss,
- MODE_S,
+ lockMode,
AutoGetCollection::ViewMode::kViewsForbidden,
lockDeadline);
@@ -391,15 +420,18 @@ private:
CollectionCatalog::get(opCtx).lookupCollectionByNamespace(opCtx, info.nss);
if (!collection) {
const auto msg = "Collection under dbCheck no longer exists";
- auto entry = dbCheckHealthLogEntry(info.nss,
- SeverityEnum::Info,
- "dbCheck failed",
- OplogEntriesEnum::Batch,
- BSON("success" << false << "error" << msg));
- HealthLog::get(opCtx).log(*entry);
return {ErrorCodes::NamespaceNotFound, msg};
}
+ auto readTimestamp = opCtx->recoveryUnit()->getPointInTimeReadTimestamp();
+ auto minVisible = collection->getMinimumVisibleSnapshot();
+ if (readTimestamp && minVisible &&
+ *readTimestamp < *collection->getMinimumVisibleSnapshot()) {
+ return {ErrorCodes::SnapshotUnavailable,
+ str::stream() << "Unable to read from collection " << info.nss
+ << " due to pending catalog changes"};
+ }
+
boost::optional<DbCheckHasher> hasher;
try {
hasher.emplace(opCtx,
@@ -427,9 +459,11 @@ private:
batch.setMd5(md5);
batch.setMinKey(first);
batch.setMaxKey(BSONKey(hasher->lastKey()));
+ batch.setReadTimestamp(readTimestamp);
// Send information on this batch over the oplog.
result.time = _logOp(opCtx, info.nss, collection->uuid(), batch.toBSON());
+ result.readTimestamp = readTimestamp;
result.nDocs = hasher->docsSeen();
result.nBytes = hasher->bytesSeen();
@@ -541,10 +575,11 @@ public:
" maxKey: <last key, inclusive>,\n"
" maxCount: <max number of docs>,\n"
" maxSize: <max size of docs>,\n"
- " maxCountPerSecond: <max rate in docs/sec> } "
- " maxDocsPerBatch: <max number of docs/batch> } "
- " maxBytesPerBatch: <try to keep a batch within max bytes/batch> } "
- " maxBatchTimeMillis: <max time processing a batch in milliseconds> } "
+ " maxCountPerSecond: <max rate in docs/sec>\n"
+ " maxDocsPerBatch: <max number of docs/batch>\n"
+ " maxBytesPerBatch: <try to keep a batch within max bytes/batch>\n"
+ " maxBatchTimeMillis: <max time processing a batch in milliseconds>\n"
+ " readTimestamp: <bool, read at a timestamp without strong locks> }\n"
"to check a collection.\n"
"Invoke with {dbCheck: 1} to check all collections in the database.";
}
diff --git a/src/mongo/db/concurrency/deferred_writer.cpp b/src/mongo/db/concurrency/deferred_writer.cpp
index b0643d47c42..99b791e1913 100644
--- a/src/mongo/db/concurrency/deferred_writer.cpp
+++ b/src/mongo/db/concurrency/deferred_writer.cpp
@@ -48,7 +48,6 @@ auto kLogInterval = stdx::chrono::minutes(1);
void DeferredWriter::_logFailure(const Status& status) {
if (TimePoint::clock::now() - _lastLogged > kLogInterval) {
LOGV2(20516,
- "Unable to write to collection {namespace}: {error}",
"Unable to write to collection",
"namespace"_attr = _nss.toString(),
"error"_attr = status);
@@ -100,14 +99,13 @@ StatusWith<std::unique_ptr<AutoGetCollection>> DeferredWriter::_getCollection(
return std::move(agc);
}
-void DeferredWriter::_worker(InsertStatement stmt) try {
+Status DeferredWriter::_worker(InsertStatement stmt) noexcept try {
auto uniqueOpCtx = Client::getCurrent()->makeOperationContext();
OperationContext* opCtx = uniqueOpCtx.get();
auto result = _getCollection(opCtx);
if (!result.isOK()) {
- _logFailure(result.getStatus());
- return;
+ return result.getStatus();
}
auto agc = std::move(result.getValue());
@@ -128,13 +126,9 @@ void DeferredWriter::_worker(InsertStatement stmt) try {
stdx::lock_guard<Latch> lock(_mutex);
_numBytes -= stmt.doc.objsize();
-
- // If a write to a deferred collection fails, periodically tell the log.
- if (!status.isOK()) {
- _logFailure(status);
- }
+ return status;
} catch (const DBException& e) {
- _logFailure(e.toStatus());
+ return e.toStatus();
}
DeferredWriter::DeferredWriter(NamespaceString nss, CollectionOptions opts, int64_t maxSize)
@@ -155,7 +149,12 @@ void DeferredWriter::startup(std::string workerName) {
options.threadNamePrefix = workerName;
options.minThreads = 0;
options.maxThreads = 1;
- options.onCreateThread = [](const std::string& name) { Client::initThread(name); };
+ options.onCreateThread = [](const std::string& name) {
+ Client::initThread(name);
+
+ stdx::lock_guard<Client> lk(cc());
+ cc().setSystemOperationKillable(lk);
+ };
_pool = std::make_unique<ThreadPool>(options);
_pool->startup();
}
@@ -190,7 +189,10 @@ bool DeferredWriter::insertDocument(BSONObj obj) {
_pool->schedule([this, obj](auto status) {
fassert(40588, status);
- _worker(InsertStatement(obj.getOwned()));
+ auto workerStatus = _worker(InsertStatement(obj.getOwned()));
+ if (!workerStatus.isOK()) {
+ _logFailure(workerStatus);
+ }
});
return true;
}
diff --git a/src/mongo/db/concurrency/deferred_writer.h b/src/mongo/db/concurrency/deferred_writer.h
index 0ac8238fa8d..1f45fbb8d15 100644
--- a/src/mongo/db/concurrency/deferred_writer.h
+++ b/src/mongo/db/concurrency/deferred_writer.h
@@ -136,7 +136,7 @@ private:
/**
* The method that the worker thread will run.
*/
- void _worker(InsertStatement stmt);
+ Status _worker(InsertStatement stmt) noexcept;
/**
* The options for the collection, in case we need to create it.
diff --git a/src/mongo/db/repl/dbcheck.cpp b/src/mongo/db/repl/dbcheck.cpp
index e8a95294866..7986c0bb426 100644
--- a/src/mongo/db/repl/dbcheck.cpp
+++ b/src/mongo/db/repl/dbcheck.cpp
@@ -141,12 +141,14 @@ std::unique_ptr<HealthLogEntry> dbCheckHealthLogEntry(const NamespaceString& nss
std::unique_ptr<HealthLogEntry> dbCheckErrorHealthLogEntry(const NamespaceString& nss,
const std::string& msg,
OplogEntriesEnum operation,
- const Status& err) {
- return dbCheckHealthLogEntry(nss,
- SeverityEnum::Error,
- msg,
- operation,
- BSON("success" << false << "error" << err.toString()));
+ const Status& err,
+ const BSONObj& context) {
+ return dbCheckHealthLogEntry(
+ nss,
+ SeverityEnum::Error,
+ msg,
+ operation,
+ BSON("success" << false << "error" << err.toString() << "context" << context));
}
std::unique_ptr<HealthLogEntry> dbCheckWarningHealthLogEntry(const NamespaceString& nss,
@@ -171,13 +173,22 @@ std::unique_ptr<HealthLogEntry> dbCheckBatchEntry(
const std::string& foundHash,
const BSONKey& minKey,
const BSONKey& maxKey,
+ const boost::optional<Timestamp>& readTimestamp,
const repl::OpTime& optime,
const boost::optional<CollectionOptions>& options) {
auto hashes = expectedFound(expectedHash, foundHash);
- auto data = BSON("success" << true << "count" << count << "bytes" << bytes << "md5"
- << hashes.second << "minKey" << minKey.elem() << "maxKey"
- << maxKey.elem() << "optime" << optime);
+ BSONObjBuilder builder;
+ builder.append("success", true);
+ builder.append("count", count);
+ builder.append("bytes", bytes);
+ builder.append("md5", hashes.second);
+ builder.appendAs(minKey.elem(), "minKey");
+ builder.appendAs(maxKey.elem(), "maxKey");
+ if (readTimestamp) {
+ builder.append("readTimestamp", *readTimestamp);
+ }
+ builder.append("optime", optime.toBSON());
const auto hashesMatch = hashes.first;
const auto severity = [&] {
@@ -196,7 +207,7 @@ std::unique_ptr<HealthLogEntry> dbCheckBatchEntry(
std::string msg =
"dbCheck batch " + (hashesMatch ? std::string("consistent") : std::string("inconsistent"));
- return dbCheckHealthLogEntry(nss, severity, msg, OplogEntriesEnum::Batch, data);
+ return dbCheckHealthLogEntry(nss, severity, msg, OplogEntriesEnum::Batch, builder.obj());
}
DbCheckHasher::DbCheckHasher(OperationContext* opCtx,
@@ -321,62 +332,60 @@ namespace {
Status dbCheckBatchOnSecondary(OperationContext* opCtx,
const repl::OpTime& optime,
const DbCheckOplogBatch& entry) {
- AutoGetCollection coll(opCtx, entry.getNss(), MODE_S);
- const auto& collection = coll.getCollection();
-
- if (!collection) {
- const auto msg = "Collection under dbCheck no longer exists";
- auto logEntry = dbCheckHealthLogEntry(entry.getNss(),
- SeverityEnum::Info,
- "dbCheck failed",
- OplogEntriesEnum::Batch,
- BSON("success" << false << "info" << msg));
- HealthLog::get(opCtx).log(*logEntry);
- return Status::OK();
- }
-
const auto msg = "replication consistency check";
// Set up the hasher,
- Status status = Status::OK();
boost::optional<DbCheckHasher> hasher;
try {
+ auto lockMode = MODE_S;
+ if (entry.getReadTimestamp()) {
+ lockMode = MODE_IS;
+ opCtx->recoveryUnit()->setTimestampReadSource(RecoveryUnit::ReadSource::kProvided,
+ entry.getReadTimestamp());
+ }
+
+ AutoGetCollection coll(opCtx, entry.getNss(), lockMode);
+ const auto& collection = coll.getCollection();
+
+ if (!collection) {
+ const auto msg = "Collection under dbCheck no longer exists";
+ auto logEntry = dbCheckHealthLogEntry(entry.getNss(),
+ SeverityEnum::Info,
+ "dbCheck failed",
+ OplogEntriesEnum::Batch,
+ BSON("success" << false << "info" << msg));
+ HealthLog::get(opCtx).log(*logEntry);
+ return Status::OK();
+ }
+
hasher.emplace(opCtx, collection, entry.getMinKey(), entry.getMaxKey());
+ uassertStatusOK(hasher->hashAll(opCtx));
+
+ std::string expected = entry.getMd5().toString();
+ std::string found = hasher->total();
+
+ auto options =
+ DurableCatalog::get(opCtx)->getCollectionOptions(opCtx, collection->getCatalogId());
+ auto logEntry = dbCheckBatchEntry(entry.getNss(),
+ hasher->docsSeen(),
+ hasher->bytesSeen(),
+ expected,
+ found,
+ entry.getMinKey(),
+ hasher->lastKey(),
+ entry.getReadTimestamp(),
+ optime,
+ options);
+
+ HealthLog::get(opCtx).log(*logEntry);
} catch (const DBException& exception) {
+ // In case of an error, report it to the health log,
auto logEntry = dbCheckErrorHealthLogEntry(
- entry.getNss(), msg, OplogEntriesEnum::Batch, exception.toStatus());
+ entry.getNss(), msg, OplogEntriesEnum::Batch, exception.toStatus(), entry.toBSON());
HealthLog::get(opCtx).log(*logEntry);
return Status::OK();
}
- // run the hasher.
- if (status.isOK()) {
- status = hasher->hashAll(opCtx);
- }
-
- // In case of an error, report it to the health log,
- if (!status.isOK()) {
- auto logEntry =
- dbCheckErrorHealthLogEntry(entry.getNss(), msg, OplogEntriesEnum::Batch, status);
- HealthLog::get(opCtx).log(*logEntry);
- return Status::OK();
- }
-
- std::string expected = entry.getMd5().toString();
- std::string found = hasher->total();
-
- auto logEntry = dbCheckBatchEntry(entry.getNss(),
- hasher->docsSeen(),
- hasher->bytesSeen(),
- expected,
- found,
- entry.getMinKey(),
- hasher->lastKey(),
- optime,
- boost::none);
-
- HealthLog::get(opCtx).log(*logEntry);
-
return Status::OK();
}
diff --git a/src/mongo/db/repl/dbcheck.h b/src/mongo/db/repl/dbcheck.h
index b6ad725e458..d93472d2f0d 100644
--- a/src/mongo/db/repl/dbcheck.h
+++ b/src/mongo/db/repl/dbcheck.h
@@ -62,7 +62,8 @@ std::unique_ptr<HealthLogEntry> dbCheckHealthLogEntry(const NamespaceString& nss
std::unique_ptr<HealthLogEntry> dbCheckErrorHealthLogEntry(const NamespaceString& nss,
const std::string& msg,
OplogEntriesEnum operation,
- const Status& err);
+ const Status& err,
+ const BSONObj& context = BSONObj());
std::unique_ptr<HealthLogEntry> dbCheckWarningHealthLogEntry(const NamespaceString& nss,
const std::string& msg,
@@ -79,6 +80,7 @@ std::unique_ptr<HealthLogEntry> dbCheckBatchEntry(
const std::string& foundHash,
const BSONKey& minKey,
const BSONKey& maxKey,
+ const boost::optional<Timestamp>& timestamp,
const repl::OpTime& optime,
const boost::optional<CollectionOptions>& options = boost::none);
diff --git a/src/mongo/db/repl/dbcheck.idl b/src/mongo/db/repl/dbcheck.idl
index 9a16e3456e0..c1e7fedfacc 100644
--- a/src/mongo/db/repl/dbcheck.idl
+++ b/src/mongo/db/repl/dbcheck.idl
@@ -130,6 +130,9 @@ structs:
validator:
gte: 10
lte: 20000
+ snapshotRead:
+ type: safeBool
+ default: true
DbCheckAllInvocation:
description: "Command object for database-wide form of dbCheck invocation"
@@ -158,6 +161,9 @@ structs:
validator:
gte: 10
lte: 20000
+ snapshotRead:
+ type: safeBool
+ default: true
DbCheckOplogBatch:
description: "Oplog entry for a dbCheck batch"
@@ -180,6 +186,9 @@ structs:
maxRate:
type: safeInt64
optional: true
+ readTimestamp:
+ type: timestamp
+ optional: true
DbCheckOplogCollection:
description: "Oplog entry for dbCheck collection metadata"