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-20 19:22:04 +0000
commit46cc35de8bd699cc7b5c796fc66d6c6c4cf46218 (patch)
tree16688a0e55d63db3f6e047b0f150ed4b0d6d82f5
parentd2a3a3038de810f7877e5a8c7edb2fd9e366eb1b (diff)
downloadmongo-46cc35de8bd699cc7b5c796fc66d6c6c4cf46218.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)
-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.cpp111
-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.cpp115
-rw-r--r--src/mongo/db/repl/dbcheck.h4
-rw-r--r--src/mongo/db/repl/dbcheck.idl9
11 files changed, 271 insertions, 107 deletions
diff --git a/buildscripts/resmokeconfig/suites/concurrency_replication.yml b/buildscripts/resmokeconfig/suites/concurrency_replication.yml
index ef63c635757..872fc477d27 100644
--- a/buildscripts/resmokeconfig/suites/concurrency_replication.yml
+++ b/buildscripts/resmokeconfig/suites/concurrency_replication.yml
@@ -54,7 +54,5 @@ executor:
set_parameters:
enableTestCommands: 1
roleGraphInvalidationIsFatal: 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 3961bc6387d..bf6ba570156 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
@@ -183,8 +183,6 @@ executor:
enableTestCommands: 1
roleGraphInvalidationIsFatal: 1
receiveChunkWaitForRangeDeleterTimeoutMS: 90000
- # 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 bb2a2303a8b..543b7a090ac 100644
--- a/buildscripts/resmokeconfig/suites/concurrency_sharded_replication.yml
+++ b/buildscripts/resmokeconfig/suites/concurrency_sharded_replication.yml
@@ -145,8 +145,6 @@ executor:
enableTestCommands: 1
roleGraphInvalidationIsFatal: 1
receiveChunkWaitForRangeDeleterTimeoutMS: 90000
- # 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..4ca67e82d31
--- /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.
+ minSnapshotHistoryWindowInSeconds: 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 147c6a736f6..e0c5a5d1610 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,
* ]
*/
@@ -199,6 +200,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 primary = replSet.getPrimary();
@@ -227,6 +247,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 abcae164b8c..85ec9a4bce2 100644
--- a/src/mongo/db/commands/dbcheck.cpp
+++ b/src/mongo/db/commands/dbcheck.cpp
@@ -69,6 +69,7 @@ struct DbCheckCollectionInfo {
int64_t maxSize;
int64_t maxRate;
int64_t maxBatchTimeMillis;
+ bool snapshotRead;
};
/**
@@ -96,8 +97,14 @@ std::unique_ptr<DbCheckRun> singleCollectionRun(OperationContext* opCtx,
const auto maxSize = invocation.getMaxSize();
const auto maxRate = invocation.getMaxCountPerSecond();
const auto maxBatchTimeMillis = invocation.getMaxBatchTimeMillis();
- const auto info =
- DbCheckCollectionInfo{nss, start, end, maxCount, maxSize, maxRate, maxBatchTimeMillis};
+ const auto info = DbCheckCollectionInfo{nss,
+ start,
+ end,
+ maxCount,
+ maxSize,
+ maxRate,
+ maxBatchTimeMillis,
+ invocation.getSnapshotRead()};
auto result = std::make_unique<DbCheckRun>();
result->push_back(info);
return result;
@@ -120,8 +127,14 @@ std::unique_ptr<DbCheckRun> fullDatabaseRun(OperationContext* opCtx,
if (!coll->ns().isReplicated() || coll->isClustered()) {
return true;
}
- DbCheckCollectionInfo info{
- coll->ns(), BSONKey::min(), BSONKey::max(), max, max, rate, maxBatchTimeMillis};
+ DbCheckCollectionInfo info{coll->ns(),
+ BSONKey::min(),
+ BSONKey::max(),
+ max,
+ max,
+ rate,
+ maxBatchTimeMillis,
+ invocation.getSnapshotRead()};
result->push_back(info);
return true;
};
@@ -236,25 +249,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,
@@ -268,21 +293,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;
@@ -316,6 +343,7 @@ private:
BSONKey lastKey;
std::string md5;
repl::OpTime time;
+ boost::optional<Timestamp> readTimestamp;
};
// Set if the job cannot proceed.
@@ -331,6 +359,17 @@ private:
auto uniqueOpCtx = Client::getCurrent()->makeOperationContext();
auto opCtx = uniqueOpCtx.get();
+ 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 =
@@ -338,14 +377,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,
AutoGetCollectionViewMode::kViewsForbidden,
lockDeadline);
@@ -359,15 +398,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(opCtx);
+ 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,
@@ -395,9 +437,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();
@@ -509,8 +553,9 @@ public:
" maxKey: <last key, inclusive>,\n"
" maxCount: <max number of docs>,\n"
" maxSize: <max size of docs>,\n"
- " maxCountPerSecond: <max rate in docs/sec> } "
- " maxBatchTimeMillis: <max time processing a batch in milliseconds> } "
+ " maxCountPerSecond: <max rate in docs/sec>\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 40f5422eb62..7dcfc666573 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().setSystemOperationKillableByStepdown(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 ca6d1695b05..48dec58214a 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,58 @@ 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 logEntry = dbCheckBatchEntry(entry.getNss(),
+ hasher->docsSeen(),
+ hasher->bytesSeen(),
+ expected,
+ found,
+ entry.getMinKey(),
+ hasher->lastKey(),
+ entry.getReadTimestamp(),
+ optime,
+ collection->getCollectionOptions());
+
+ 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,
- collection->getCollectionOptions());
-
- 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 150580e7aa1..73c9bef29f1 100644
--- a/src/mongo/db/repl/dbcheck.h
+++ b/src/mongo/db/repl/dbcheck.h
@@ -63,7 +63,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,
@@ -80,6 +81,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 a262aa862d9..0dc9ca1d8d8 100644
--- a/src/mongo/db/repl/dbcheck.idl
+++ b/src/mongo/db/repl/dbcheck.idl
@@ -118,6 +118,9 @@ structs:
validator:
gte: 10
lte: 20000
+ snapshotRead:
+ type: safeBool
+ default: true
DbCheckAllInvocation:
description: "Command object for database-wide form of dbCheck invocation"
@@ -134,6 +137,9 @@ structs:
validator:
gte: 10
lte: 20000
+ snapshotRead:
+ type: safeBool
+ default: true
DbCheckOplogBatch:
description: "Oplog entry for a dbCheck batch"
@@ -156,6 +162,9 @@ structs:
maxRate:
type: safeInt64
optional: true
+ readTimestamp:
+ type: timestamp
+ optional: true
DbCheckOplogCollection:
description: "Oplog entry for dbCheck collection metadata"