summaryrefslogtreecommitdiff
path: root/src/mongo
diff options
context:
space:
mode:
authorTess Avitabile <tess.avitabile@mongodb.com>2018-09-26 11:26:36 -0400
committerTess Avitabile <tess.avitabile@mongodb.com>2018-10-02 10:58:05 -0400
commitaaa64733625433f31a5e212f4652a58816385388 (patch)
tree538d19b094a41d780b56ca1f7f065ad8cc08acf9 /src/mongo
parente74ff7028c06686611eb4652a80a212bf14e3757 (diff)
downloadmongo-aaa64733625433f31a5e212f4652a58816385388.tar.gz
SERVER-37227 Reintroduce enableMajorityReadConcern:false server parameter
Diffstat (limited to 'src/mongo')
-rw-r--r--src/mongo/db/initialize_operation_session_info.cpp8
-rw-r--r--src/mongo/db/initialize_operation_session_info.h6
-rw-r--r--src/mongo/db/logical_session_id_test.cpp36
-rw-r--r--src/mongo/db/mongod_options.cpp25
-rw-r--r--src/mongo/db/repl/replication_coordinator_external_state_impl.cpp2
-rw-r--r--src/mongo/db/repl/replication_recovery.cpp30
-rw-r--r--src/mongo/db/repl/replication_recovery_test.cpp15
-rw-r--r--src/mongo/db/repl/rollback_test_fixture.h4
-rw-r--r--src/mongo/db/repl/storage_interface.h5
-rw-r--r--src/mongo/db/repl/storage_interface_impl.cpp4
-rw-r--r--src/mongo/db/repl/storage_interface_impl.h2
-rw-r--r--src/mongo/db/repl/storage_interface_mock.h4
-rw-r--r--src/mongo/db/repl/sync_tail.cpp42
-rw-r--r--src/mongo/db/s/session_catalog_migration_destination_test.cpp3
-rw-r--r--src/mongo/db/server_options.h2
-rw-r--r--src/mongo/db/service_entry_point_common.cpp3
-rw-r--r--src/mongo/db/storage/kv/kv_engine.h13
-rw-r--r--src/mongo/db/storage/kv/kv_storage_engine.cpp11
-rw-r--r--src/mongo/db/storage/kv/kv_storage_engine.h4
-rw-r--r--src/mongo/db/storage/storage_engine.h11
-rw-r--r--src/mongo/db/storage/storage_init.cpp2
-rw-r--r--src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.cpp72
-rw-r--r--src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.h26
-rw-r--r--src/mongo/db/storage/wiredtiger/wiredtiger_oplog_manager.cpp16
-rw-r--r--src/mongo/db/storage/wiredtiger/wiredtiger_oplog_manager.h9
-rw-r--r--src/mongo/db/storage/wiredtiger/wiredtiger_record_store.cpp36
-rw-r--r--src/mongo/db/storage/wiredtiger/wiredtiger_snapshot_manager.cpp5
-rw-r--r--src/mongo/db/storage/wiredtiger/wiredtiger_util.cpp5
-rw-r--r--src/mongo/dbtests/SConscript1
-rw-r--r--src/mongo/dbtests/framework_options.cpp14
-rw-r--r--src/mongo/dbtests/repltests.cpp9
-rw-r--r--src/mongo/dbtests/storage_timestamp_tests.cpp6
-rw-r--r--src/mongo/s/commands/strategy.cpp4
-rw-r--r--src/mongo/shell/servers.js19
34 files changed, 309 insertions, 145 deletions
diff --git a/src/mongo/db/initialize_operation_session_info.cpp b/src/mongo/db/initialize_operation_session_info.cpp
index b0e22d31168..4f39c6293ff 100644
--- a/src/mongo/db/initialize_operation_session_info.cpp
+++ b/src/mongo/db/initialize_operation_session_info.cpp
@@ -43,8 +43,7 @@ boost::optional<OperationSessionInfoFromClient> initializeOperationSessionInfo(
const BSONObj& requestBody,
bool requiresAuth,
bool isReplSetMemberOrMongos,
- bool supportsDocLocking,
- bool supportsRecoverToStableTimestamp) {
+ bool supportsDocLocking) {
auto osi = OperationSessionInfoFromClient::parse("OperationSessionInfo"_sd, requestBody);
if (opCtx->getClient()->isInDirectClient()) {
@@ -119,11 +118,6 @@ boost::optional<OperationSessionInfoFromClient> initializeOperationSessionInfo(
uassert(ErrorCodes::InvalidOptions,
"Specifying autocommit=true is not allowed.",
!osi.getAutocommit().value());
-
- uassert(ErrorCodes::IllegalOperation,
- "Multi-document transactions are only allowed on storage engines that support "
- "recover to stable timestamp.",
- supportsRecoverToStableTimestamp);
} else {
uassert(ErrorCodes::InvalidOptions,
"'startTransaction' field requires 'autocommit' field to also be specified",
diff --git a/src/mongo/db/initialize_operation_session_info.h b/src/mongo/db/initialize_operation_session_info.h
index e67bf7531e4..8882c4e7052 100644
--- a/src/mongo/db/initialize_operation_session_info.h
+++ b/src/mongo/db/initialize_operation_session_info.h
@@ -46,9 +46,6 @@ namespace mongo {
* Both isReplSetMemberOrMongos and supportsDocLocking need to be true if the command contains a
* transaction number, otherwise this function will throw.
*
- * supportsRecoverToStableTimestamp needs to be true if the command contains autocommit:false,
- * otherwise this function will throw.
- *
* On success, returns the parsed request information. Returning boost::none implies that the
* proper command or session requirements were not met.
*/
@@ -57,7 +54,6 @@ boost::optional<OperationSessionInfoFromClient> initializeOperationSessionInfo(
const BSONObj& requestBody,
bool requiresAuth,
bool isReplSetMemberOrMongos,
- bool supportsDocLocking,
- bool supportsRecoverToStableTimestamp);
+ bool supportsDocLocking);
} // namespace mongo
diff --git a/src/mongo/db/logical_session_id_test.cpp b/src/mongo/db/logical_session_id_test.cpp
index 128a793b12b..e32e1e88089 100644
--- a/src/mongo/db/logical_session_id_test.cpp
+++ b/src/mongo/db/logical_session_id_test.cpp
@@ -235,7 +235,7 @@ TEST_F(LogicalSessionIdTest, GenWithoutAuthedUser) {
TEST_F(LogicalSessionIdTest, InitializeOperationSessionInfo_NoSessionIdNoTransactionNumber) {
addSimpleUser(UserName("simple", "test"));
- initializeOperationSessionInfo(_opCtx.get(), BSON("TestCmd" << 1), true, true, true, true);
+ initializeOperationSessionInfo(_opCtx.get(), BSON("TestCmd" << 1), true, true, true);
ASSERT(!_opCtx->getLogicalSessionId());
ASSERT(!_opCtx->getTxnNumber());
@@ -251,7 +251,6 @@ TEST_F(LogicalSessionIdTest, InitializeOperationSessionInfo_SessionIdNoTransacti
<< "TestField"),
true,
true,
- true,
true);
ASSERT(_opCtx->getLogicalSessionId());
@@ -268,7 +267,6 @@ TEST_F(LogicalSessionIdTest, InitializeOperationSessionInfo_MissingSessionIdWith
<< "TestField"),
true,
true,
- true,
true),
AssertionException,
ErrorCodes::InvalidOptions);
@@ -285,7 +283,6 @@ TEST_F(LogicalSessionIdTest, InitializeOperationSessionInfo_SessionIdAndTransact
<< "TestField"),
true,
true,
- true,
true);
ASSERT(_opCtx->getLogicalSessionId());
@@ -307,7 +304,6 @@ TEST_F(LogicalSessionIdTest, InitializeOperationSessionInfo_IsReplSetMemberOrMon
<< "TestField"),
true,
false,
- true,
true),
AssertionException,
ErrorCodes::IllegalOperation);
@@ -325,27 +321,6 @@ TEST_F(LogicalSessionIdTest, InitializeOperationSessionInfo_SupportsDocLockingFa
<< "TestField"),
true,
true,
- false,
- true),
- AssertionException,
- ErrorCodes::IllegalOperation);
-}
-
-TEST_F(LogicalSessionIdTest, InitializeOperationSessionInfo_SupportsRecoverToStableTimestampFalse) {
- addSimpleUser(UserName("simple", "test"));
- LogicalSessionFromClient lsid;
- lsid.setId(UUID::gen());
-
- ASSERT_THROWS_CODE(
- initializeOperationSessionInfo(
- _opCtx.get(),
- BSON("TestCmd" << 1 << "lsid" << lsid.toBSON() << "txnNumber" << 100LL << "OtherField"
- << "TestField"
- << "autocommit"
- << false),
- true,
- true,
- true,
false),
AssertionException,
ErrorCodes::IllegalOperation);
@@ -364,7 +339,6 @@ TEST_F(LogicalSessionIdTest, InitializeOperationSessionInfo_IgnoresInfoIfNoCache
<< "TestField"),
true,
true,
- true,
true));
}
@@ -383,10 +357,10 @@ TEST_F(LogicalSessionIdTest, InitializeOperationSessionInfo_SendingInfoFailsInDi
<< "foo");
commandBuilder.appendElements(param);
- ASSERT_THROWS_CODE(initializeOperationSessionInfo(
- _opCtx.get(), commandBuilder.obj(), true, true, true, true),
- AssertionException,
- 50891);
+ ASSERT_THROWS_CODE(
+ initializeOperationSessionInfo(_opCtx.get(), commandBuilder.obj(), true, true, true),
+ AssertionException,
+ 50891);
}
_opCtx->getClient()->setInDirectClient(false);
diff --git a/src/mongo/db/mongod_options.cpp b/src/mongo/db/mongod_options.cpp
index 0376b73cee0..98a1f8b2f68 100644
--- a/src/mongo/db/mongod_options.cpp
+++ b/src/mongo/db/mongod_options.cpp
@@ -301,15 +301,14 @@ Status addMongodOptions(moe::OptionSection* options) {
"specify index prefetching behavior (if secondary) [none|_id_only|all]")
.format("(:?none)|(:?_id_only)|(:?all)", "(none/_id_only/all)");
- // `enableMajorityReadConcern` is always enabled starting in 3.6, regardless of user
- // settings. We're leaving the option in to not break existing deployment scripts. A warning
- // will appear if explicitly set to false.
+ // `enableMajorityReadConcern` is enabled by default starting in 3.6.
rs_options
.addOptionChaining("replication.enableMajorityReadConcern",
"enableMajorityReadConcern",
- moe::Switch,
+ moe::Bool,
"enables majority readConcern")
- .setDefault(moe::Value(true));
+ .setDefault(moe::Value(true))
+ .setImplicit(moe::Value(true));
replication_options.addOptionChaining(
"master", "master", moe::Switch, "Master/slave replication no longer supported");
@@ -873,11 +872,8 @@ Status storeMongodOptions(const moe::Environment& params) {
}
if (params.count("replication.enableMajorityReadConcern")) {
- bool val = params["replication.enableMajorityReadConcern"].as<bool>();
- if (!val) {
- warning() << "enableMajorityReadConcern startup parameter was supplied, but its value "
- "was ignored; majority read concern cannot be disabled.";
- }
+ serverGlobalParams.enableMajorityReadConcern =
+ params["replication.enableMajorityReadConcern"].as<bool>();
}
if (params.count("storage.indexBuildRetry")) {
@@ -934,6 +930,15 @@ Status storeMongodOptions(const moe::Environment& params) {
if (clusterRoleParam == "configsvr") {
serverGlobalParams.clusterRole = ClusterRole::ConfigServer;
+ if (params.count("replication.enableMajorityReadConcern") &&
+ !params["replication.enableMajorityReadConcern"].as<bool>()) {
+ warning()
+ << "Config servers require majority read concern, but it was explicitly "
+ "disabled. The override is being ignored and the process is continuing "
+ "with majority read concern enabled.";
+ }
+ serverGlobalParams.enableMajorityReadConcern = true;
+
// If we haven't explicitly specified a journal option, default journaling to true for
// the config server role
if (!params.count("storage.journal.enabled")) {
diff --git a/src/mongo/db/repl/replication_coordinator_external_state_impl.cpp b/src/mongo/db/repl/replication_coordinator_external_state_impl.cpp
index 4e7d3dd21f7..3af514b94e4 100644
--- a/src/mongo/db/repl/replication_coordinator_external_state_impl.cpp
+++ b/src/mongo/db/repl/replication_coordinator_external_state_impl.cpp
@@ -892,7 +892,7 @@ bool ReplicationCoordinatorExternalStateImpl::isReadCommittedSupportedByStorageE
auto storageEngine = opCtx->getServiceContext()->getStorageEngine();
// This should never be called if the storage engine has not been initialized.
invariant(storageEngine);
- return storageEngine->getSnapshotManager();
+ return storageEngine->supportsReadConcernMajority();
}
bool ReplicationCoordinatorExternalStateImpl::isReadConcernSnapshotSupportedByStorageEngine(
diff --git a/src/mongo/db/repl/replication_recovery.cpp b/src/mongo/db/repl/replication_recovery.cpp
index 16d396b63eb..5b5bfd43a4f 100644
--- a/src/mongo/db/repl/replication_recovery.cpp
+++ b/src/mongo/db/repl/replication_recovery.cpp
@@ -242,13 +242,13 @@ void ReplicationRecoveryImpl::recoverFromOplog(OperationContext* opCtx,
// If we were passed in a stable timestamp, we are in rollback recovery and should recover from
// that stable timestamp. Otherwise, we're recovering at startup. If this storage engine
- // supports recover to stable timestamp, we ask it for the recovery timestamp. If the storage
- // engine returns a timestamp, we recover from that point. However, if the storage engine
- // returns "none", the storage engine does not have a stable checkpoint and we must recover from
- // an unstable checkpoint instead.
- const bool supportsRecoverToStableTimestamp =
- _storageInterface->supportsRecoverToStableTimestamp(opCtx->getServiceContext());
- if (!stableTimestamp && supportsRecoverToStableTimestamp) {
+ // supports recover to stable timestamp or enableMajorityReadConcern=false, we ask it for the
+ // recovery timestamp. If the storage engine returns a timestamp, we recover from that point.
+ // However, if the storage engine returns "none", the storage engine does not have a stable
+ // checkpoint and we must recover from an unstable checkpoint instead.
+ const bool supportsRecoveryTimestamp =
+ _storageInterface->supportsRecoveryTimestamp(opCtx->getServiceContext());
+ if (!stableTimestamp && supportsRecoveryTimestamp) {
stableTimestamp = _storageInterface->getRecoveryTimestamp(opCtx->getServiceContext());
}
@@ -260,7 +260,7 @@ void ReplicationRecoveryImpl::recoverFromOplog(OperationContext* opCtx,
<< appliedThrough.toString());
if (stableTimestamp) {
- invariant(supportsRecoverToStableTimestamp);
+ invariant(supportsRecoveryTimestamp);
_recoverFromStableTimestamp(opCtx, *stableTimestamp, appliedThrough, topOfOplog);
} else {
_recoverFromUnstableCheckpoint(opCtx, appliedThrough, topOfOplog);
@@ -301,6 +301,20 @@ void ReplicationRecoveryImpl::_recoverFromUnstableCheckpoint(OperationContext* o
// application and must apply from the appliedThrough to the top of the oplog.
log() << "Starting recovery oplog application at the appliedThrough: " << appliedThrough
<< ", through the top of the oplog: " << topOfOplog;
+
+ // When `recoverFromOplog` truncates the oplog, that also happens to set the "oldest
+ // timestamp" to the truncation point[1]. `_applyToEndOfOplog` will then perform writes
+ // before the truncation point. Doing so violates the constraint that all updates must be
+ // timestamped newer than the "oldest timestamp". This call will move the "oldest
+ // timestamp" back to the `startPoint`.
+ //
+ // [1] This is arguably incorrect. On rollback for nodes that are not keeping history to
+ // the "majority point", the "oldest timestamp" likely needs to go back in time. The
+ // oplog's `cappedTruncateAfter` method was a convenient location for this logic, which,
+ // unfortunately, conflicts with the usage above.
+ opCtx->getServiceContext()->getStorageEngine()->setOldestTimestamp(
+ appliedThrough.getTimestamp());
+
_applyToEndOfOplog(opCtx, appliedThrough.getTimestamp(), topOfOplog.getTimestamp());
}
diff --git a/src/mongo/db/repl/replication_recovery_test.cpp b/src/mongo/db/repl/replication_recovery_test.cpp
index 1a7cc5f8be8..cd5e072a513 100644
--- a/src/mongo/db/repl/replication_recovery_test.cpp
+++ b/src/mongo/db/repl/replication_recovery_test.cpp
@@ -78,11 +78,22 @@ public:
_supportsRecoverToStableTimestamp = supports;
}
+ bool supportsRecoveryTimestamp(ServiceContext* serviceCtx) const override {
+ stdx::lock_guard<stdx::mutex> lock(_mutex);
+ return _supportsRecoveryTimestamp;
+ }
+
+ void setSupportsRecoveryTimestamp(bool supports) {
+ stdx::lock_guard<stdx::mutex> lock(_mutex);
+ _supportsRecoveryTimestamp = supports;
+ }
+
private:
mutable stdx::mutex _mutex;
Timestamp _initialDataTimestamp = Timestamp::min();
boost::optional<Timestamp> _recoveryTimestamp = boost::none;
bool _supportsRecoverToStableTimestamp = true;
+ bool _supportsRecoveryTimestamp = true;
};
class ReplicationRecoveryTest : public ServiceContextMongoDTest {
@@ -357,9 +368,9 @@ DEATH_TEST_F(ReplicationRecoveryTest,
}
DEATH_TEST_F(ReplicationRecoveryTest,
- RecoveryInvariantsIfStableTimestampAndDoesNotSupportRTT,
+ RecoveryInvariantsIfStableTimestampAndDoesNotSupportRecoveryTimestamp,
"Invariant failure") {
- getStorageInterfaceRecovery()->setSupportsRecoverToStableTimestamp(false);
+ getStorageInterfaceRecovery()->setSupportsRecoveryTimestamp(false);
ReplicationRecoveryImpl recovery(getStorageInterface(), getConsistencyMarkers());
auto opCtx = getOperationContext();
diff --git a/src/mongo/db/repl/rollback_test_fixture.h b/src/mongo/db/repl/rollback_test_fixture.h
index 8d31c63564f..26c568db2c4 100644
--- a/src/mongo/db/repl/rollback_test_fixture.h
+++ b/src/mongo/db/repl/rollback_test_fixture.h
@@ -137,6 +137,10 @@ public:
return true;
}
+ bool supportsRecoveryTimestamp(ServiceContext* serviceCtx) const override {
+ return true;
+ }
+
void setRecoverToTimestampStatus(Status status) {
stdx::lock_guard<stdx::mutex> lock(_mutex);
_recoverToTimestampStatus = status;
diff --git a/src/mongo/db/repl/storage_interface.h b/src/mongo/db/repl/storage_interface.h
index c49a56149dc..32728c0f950 100644
--- a/src/mongo/db/repl/storage_interface.h
+++ b/src/mongo/db/repl/storage_interface.h
@@ -374,6 +374,11 @@ public:
virtual bool supportsRecoverToStableTimestamp(ServiceContext* serviceCtx) const = 0;
/**
+ * Returns whether the storage engine can provide a recovery timestamp.
+ */
+ virtual bool supportsRecoveryTimestamp(ServiceContext* serviceCtx) const = 0;
+
+ /**
* Returns the stable timestamp that the storage engine recovered to on startup. If the
* recovery point was not stable, returns "none".
*/
diff --git a/src/mongo/db/repl/storage_interface_impl.cpp b/src/mongo/db/repl/storage_interface_impl.cpp
index 044ae209c1c..65540f5ac78 100644
--- a/src/mongo/db/repl/storage_interface_impl.cpp
+++ b/src/mongo/db/repl/storage_interface_impl.cpp
@@ -1069,6 +1069,10 @@ bool StorageInterfaceImpl::supportsRecoverToStableTimestamp(ServiceContext* serv
return serviceCtx->getStorageEngine()->supportsRecoverToStableTimestamp();
}
+bool StorageInterfaceImpl::supportsRecoveryTimestamp(ServiceContext* serviceCtx) const {
+ return serviceCtx->getStorageEngine()->supportsRecoveryTimestamp();
+}
+
boost::optional<Timestamp> StorageInterfaceImpl::getRecoveryTimestamp(
ServiceContext* serviceCtx) const {
return serviceCtx->getStorageEngine()->getRecoveryTimestamp();
diff --git a/src/mongo/db/repl/storage_interface_impl.h b/src/mongo/db/repl/storage_interface_impl.h
index 954ae769c52..48de98cf184 100644
--- a/src/mongo/db/repl/storage_interface_impl.h
+++ b/src/mongo/db/repl/storage_interface_impl.h
@@ -166,6 +166,8 @@ public:
bool supportsRecoverToStableTimestamp(ServiceContext* serviceCtx) const override;
+ bool supportsRecoveryTimestamp(ServiceContext* serviceCtx) const override;
+
boost::optional<Timestamp> getRecoveryTimestamp(ServiceContext* serviceCtx) const override;
bool supportsDocLocking(ServiceContext* serviceCtx) const override;
diff --git a/src/mongo/db/repl/storage_interface_mock.h b/src/mongo/db/repl/storage_interface_mock.h
index 9ea177f7e21..5ccc1c4c632 100644
--- a/src/mongo/db/repl/storage_interface_mock.h
+++ b/src/mongo/db/repl/storage_interface_mock.h
@@ -303,6 +303,10 @@ public:
return false;
}
+ bool supportsRecoveryTimestamp(ServiceContext* serviceCtx) const override {
+ return false;
+ }
+
boost::optional<Timestamp> getRecoveryTimestamp(ServiceContext* serviceCtx) const override {
return boost::none;
}
diff --git a/src/mongo/db/repl/sync_tail.cpp b/src/mongo/db/repl/sync_tail.cpp
index 42af9416555..5025b48ff23 100644
--- a/src/mongo/db/repl/sync_tail.cpp
+++ b/src/mongo/db/repl/sync_tail.cpp
@@ -72,6 +72,7 @@
#include "mongo/db/session.h"
#include "mongo/db/session_txn_record_gen.h"
#include "mongo/db/stats/timer_stats.h"
+#include "mongo/db/storage/recovery_unit.h"
#include "mongo/stdx/memory.h"
#include "mongo/util/exit.h"
#include "mongo/util/fail_point_service.h"
@@ -1253,9 +1254,35 @@ StatusWith<OpTime> SyncTail::multiApply(OperationContext* opCtx, MultiApplier::O
// Each node records cumulative batch application stats for itself using this timer.
TimerHolder timer(&applyBatchStats);
+ const bool pinOldestTimestamp = !serverGlobalParams.enableMajorityReadConcern;
+ std::unique_ptr<RecoveryUnit> pinningTransaction;
+ if (pinOldestTimestamp) {
+ // If `enableMajorityReadConcern` is false, storage aggressively trims
+ // history. Documents may not be inserted before the cutoff point. This piece will pin
+ // the "oldest timestamp" until after the batch is fully applied.
+ //
+ // When `enableMajorityReadConcern` is false, storage sets the "oldest timestamp" to
+ // the "get all committed" timestamp. Opening a transaction and setting its timestamp
+ // to first oplog entry's timestamp will prevent the "get all committed" timestamp
+ // from advancing.
+ //
+ // This transaction will be aborted after all writes from the batch of operations are
+ // complete. Aborting the transaction allows the "get all committed" point to be
+ // move forward.
+ pinningTransaction = std::unique_ptr<RecoveryUnit>(
+ opCtx->getServiceContext()->getStorageEngine()->newRecoveryUnit());
+ pinningTransaction->beginUnitOfWork(opCtx);
+ fassert(40677, pinningTransaction->setTimestamp(ops.front().getTimestamp()));
+ }
+
// We must wait for the all work we've dispatched to complete before leaving this block
// because the spawned threads refer to objects on the stack
- ON_BLOCK_EXIT([&] { _writerPool->waitForIdle(); });
+ ON_BLOCK_EXIT([&] {
+ _writerPool->waitForIdle();
+ if (pinOldestTimestamp) {
+ pinningTransaction->abortUnitOfWork();
+ }
+ });
// Write batch of ops into oplog.
if (!_options.skipWritesToOplog) {
@@ -1303,15 +1330,14 @@ StatusWith<OpTime> SyncTail::multiApply(OperationContext* opCtx, MultiApplier::O
}
}
}
-
- // Notify the storage engine that a replication batch has completed.
- // This means that all the writes associated with the oplog entries in the batch are
- // finished and no new writes with timestamps associated with those oplog entries will show
- // up in the future.
- const auto storageEngine = opCtx->getServiceContext()->getStorageEngine();
- storageEngine->replicationBatchIsComplete();
}
+ // Notify the storage engine that a replication batch has completed. This means that all the
+ // writes associated with the oplog entries in the batch are finished and no new writes with
+ // timestamps associated with those oplog entries will show up in the future.
+ const auto storageEngine = opCtx->getServiceContext()->getStorageEngine();
+ storageEngine->replicationBatchIsComplete();
+
// Use this fail point to hold the PBWM lock and prevent the batch from completing.
if (MONGO_FAIL_POINT(pauseBatchApplicationBeforeCompletion)) {
log() << "pauseBatchApplicationBeforeCompletion fail point enabled. Blocking until fail "
diff --git a/src/mongo/db/s/session_catalog_migration_destination_test.cpp b/src/mongo/db/s/session_catalog_migration_destination_test.cpp
index 5299f54142f..7ca1a4a6294 100644
--- a/src/mongo/db/s/session_catalog_migration_destination_test.cpp
+++ b/src/mongo/db/s/session_catalog_migration_destination_test.cpp
@@ -242,8 +242,7 @@ public:
// The ephemeral for test storage engine doesn't support document-level locking, so
// requests with txnNumbers aren't allowed. To get around this, we have to manually set
// up the session state and perform the insert.
- initializeOperationSessionInfo(
- innerOpCtx.get(), insertBuilder.obj(), true, true, true, false);
+ initializeOperationSessionInfo(innerOpCtx.get(), insertBuilder.obj(), true, true, true);
OperationContextSessionMongod sessionTxnState(
innerOpCtx.get(), true, boost::none, boost::none);
diff --git a/src/mongo/db/server_options.h b/src/mongo/db/server_options.h
index 1a3f3a96c3e..34b3389da19 100644
--- a/src/mongo/db/server_options.h
+++ b/src/mongo/db/server_options.h
@@ -248,6 +248,8 @@ struct ServerGlobalParams {
AtomicWord<bool> validateFeaturesAsMaster{true};
std::vector<std::string> disabledSecureAllocatorDomains;
+
+ bool enableMajorityReadConcern = true;
};
extern ServerGlobalParams serverGlobalParams;
diff --git a/src/mongo/db/service_entry_point_common.cpp b/src/mongo/db/service_entry_point_common.cpp
index 55dedecb657..c6e3b7ad7d1 100644
--- a/src/mongo/db/service_entry_point_common.cpp
+++ b/src/mongo/db/service_entry_point_common.cpp
@@ -624,8 +624,7 @@ void execCommandDatabase(OperationContext* opCtx,
request.body,
command->requiresAuth(),
replCoord->getReplicationMode() == repl::ReplicationCoordinator::modeReplSet,
- opCtx->getServiceContext()->getStorageEngine()->supportsDocLocking(),
- opCtx->getServiceContext()->getStorageEngine()->supportsRecoverToStableTimestamp());
+ opCtx->getServiceContext()->getStorageEngine()->supportsDocLocking());
evaluateFailCommandFailPoint(opCtx, command->getName());
diff --git a/src/mongo/db/storage/kv/kv_engine.h b/src/mongo/db/storage/kv/kv_engine.h
index 48e7986c9ea..88ae99e9f82 100644
--- a/src/mongo/db/storage/kv/kv_engine.h
+++ b/src/mongo/db/storage/kv/kv_engine.h
@@ -316,7 +316,7 @@ public:
/**
* See `StorageEngine::setOldestTimestamp`
*/
- virtual void setOldestTimestamp(Timestamp newOldestTimestamp) {}
+ virtual void setOldestTimestamp(Timestamp newOldestTimestamp, bool force) {}
/**
* See `StorageEngine::isCacheUnderPressure()`
@@ -338,6 +338,13 @@ public:
}
/**
+ * See `StorageEngine::supportsRecoveryTimestamp`
+ */
+ virtual bool supportsRecoveryTimestamp() const {
+ return false;
+ }
+
+ /**
* See `StorageEngine::recoverToStableTimestamp`
*/
virtual StatusWith<Timestamp> recoverToStableTimestamp(OperationContext* opCtx) {
@@ -370,6 +377,10 @@ public:
return false;
}
+ virtual bool supportsReadConcernMajority() const {
+ return false;
+ }
+
/**
* See `StorageEngine::replicationBatchIsComplete()`
*/
diff --git a/src/mongo/db/storage/kv/kv_storage_engine.cpp b/src/mongo/db/storage/kv/kv_storage_engine.cpp
index fab3545f636..37788d33008 100644
--- a/src/mongo/db/storage/kv/kv_storage_engine.cpp
+++ b/src/mongo/db/storage/kv/kv_storage_engine.cpp
@@ -644,7 +644,8 @@ void KVStorageEngine::setOldestTimestampFromStable() {
}
void KVStorageEngine::setOldestTimestamp(Timestamp newOldestTimestamp) {
- _engine->setOldestTimestamp(newOldestTimestamp);
+ const bool force = true;
+ _engine->setOldestTimestamp(newOldestTimestamp, force);
}
bool KVStorageEngine::isCacheUnderPressure(OperationContext* opCtx) const {
@@ -659,6 +660,10 @@ bool KVStorageEngine::supportsRecoverToStableTimestamp() const {
return _engine->supportsRecoverToStableTimestamp();
}
+bool KVStorageEngine::supportsRecoveryTimestamp() const {
+ return _engine->supportsRecoveryTimestamp();
+}
+
StatusWith<Timestamp> KVStorageEngine::recoverToStableTimestamp(OperationContext* opCtx) {
invariant(opCtx->lockState()->isW());
@@ -697,6 +702,10 @@ bool KVStorageEngine::supportsReadConcernSnapshot() const {
return _engine->supportsReadConcernSnapshot();
}
+bool KVStorageEngine::supportsReadConcernMajority() const {
+ return _engine->supportsReadConcernMajority();
+}
+
void KVStorageEngine::replicationBatchIsComplete() const {
return _engine->replicationBatchIsComplete();
}
diff --git a/src/mongo/db/storage/kv/kv_storage_engine.h b/src/mongo/db/storage/kv/kv_storage_engine.h
index a25b5438c30..98d8fea43bd 100644
--- a/src/mongo/db/storage/kv/kv_storage_engine.h
+++ b/src/mongo/db/storage/kv/kv_storage_engine.h
@@ -137,6 +137,8 @@ public:
virtual bool supportsRecoverToStableTimestamp() const override;
+ virtual bool supportsRecoveryTimestamp() const override;
+
virtual StatusWith<Timestamp> recoverToStableTimestamp(OperationContext* opCtx) override;
virtual boost::optional<Timestamp> getRecoveryTimestamp() const override;
@@ -147,6 +149,8 @@ public:
bool supportsReadConcernSnapshot() const final;
+ bool supportsReadConcernMajority() const final;
+
virtual void replicationBatchIsComplete() const override;
SnapshotManager* getSnapshotManager() const final;
diff --git a/src/mongo/db/storage/storage_engine.h b/src/mongo/db/storage/storage_engine.h
index 6452044ce30..f236113f6c3 100644
--- a/src/mongo/db/storage/storage_engine.h
+++ b/src/mongo/db/storage/storage_engine.h
@@ -320,12 +320,23 @@ public:
}
/**
+ * Returns whether the storage engine can provide a recovery timestamp.
+ */
+ virtual bool supportsRecoveryTimestamp() const {
+ return false;
+ }
+
+ /**
* Returns true if the storage engine supports the readConcern level "snapshot".
*/
virtual bool supportsReadConcernSnapshot() const {
return false;
}
+ virtual bool supportsReadConcernMajority() const {
+ return false;
+ }
+
/**
* Recovers the storage engine state to the last stable timestamp. "Stable" in this case
* refers to a timestamp that is guaranteed to never be rolled back. The stable timestamp
diff --git a/src/mongo/db/storage/storage_init.cpp b/src/mongo/db/storage/storage_init.cpp
index 6fdc7de4304..782965e206c 100644
--- a/src/mongo/db/storage/storage_init.cpp
+++ b/src/mongo/db/storage/storage_init.cpp
@@ -56,7 +56,7 @@ public:
const BSONElement& configElement) const {
auto engine = opCtx->getClient()->getServiceContext()->getStorageEngine();
return BSON("name" << storageGlobalParams.engine << "supportsCommittedReads"
- << bool(engine->getSnapshotManager())
+ << engine->supportsReadConcernMajority()
<< "supportsSnapshotReadConcern"
<< engine->supportsReadConcernSnapshot()
<< "readOnly"
diff --git a/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.cpp b/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.cpp
index 2bac8fcc5df..2aed5a1feff 100644
--- a/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.cpp
+++ b/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.cpp
@@ -268,17 +268,24 @@ public:
// First, initialDataTimestamp is Timestamp(0, 1) -> Take full checkpoint. This is
// when there is no consistent view of the data (i.e: during initial sync).
//
- // Second, stableTimestamp < initialDataTimestamp: Skip checkpoints. The data on
- // disk is prone to being rolled back. Hold off on checkpoints. Hope that the
- // stable timestamp surpasses the data on disk, allowing storage to persist newer
- // copies to disk.
+ // Second, enableMajorityReadConcern is false. In this case, we are not tracking a
+ // stable timestamp. Take a full checkpoint.
//
- // Third, stableTimestamp >= initialDataTimestamp: Take stable checkpoint. Steady
+ // Third, stableTimestamp < initialDataTimestamp: Skip checkpoints. The data on disk
+ // is prone to being rolled back. Hold off on checkpoints. Hope that the stable
+ // timestamp surpasses the data on disk, allowing storage to persist newer copies to
+ // disk.
+ //
+ // Fourth, stableTimestamp >= initialDataTimestamp: Take stable checkpoint. Steady
// state case.
if (initialDataTimestamp.asULL() <= 1) {
UniqueWiredTigerSession session = _sessionCache->getSession();
WT_SESSION* s = session->getSession();
invariantWTOK(s->checkpoint(s, "use_timestamp=false"));
+ } else if (!serverGlobalParams.enableMajorityReadConcern) {
+ UniqueWiredTigerSession session = _sessionCache->getSession();
+ WT_SESSION* s = session->getSession();
+ invariantWTOK(s->checkpoint(s, "use_timestamp=false"));
} else if (stableTimestamp < initialDataTimestamp) {
LOG_FOR_RECOVERY(2)
<< "Stable timestamp is behind the initial data timestamp, skipping "
@@ -449,7 +456,8 @@ WiredTigerKVEngine::WiredTigerKVEngine(const std::string& canonicalName,
_durable(durable),
_ephemeral(ephemeral),
_inRepairMode(repair),
- _readOnly(readOnly) {
+ _readOnly(readOnly),
+ _keepDataHistory(serverGlobalParams.enableMajorityReadConcern) {
boost::filesystem::path journalPath = path;
journalPath /= "journal";
if (_durable) {
@@ -711,6 +719,10 @@ void WiredTigerKVEngine::cleanShutdown() {
invariantWTOK(_conn->reconfigure(_conn, _fileVersion.getDowngradeString().c_str()));
}
+ if (!serverGlobalParams.enableMajorityReadConcern) {
+ closeConfig += "use_timestamp=false,";
+ }
+
invariantWTOK(_conn->close(_conn, closeConfig.c_str()));
_conn = nullptr;
}
@@ -1362,6 +1374,10 @@ MONGO_FAIL_POINT_DEFINE(WTPreserveSnapshotHistoryIndefinitely);
void WiredTigerKVEngine::setStableTimestamp(Timestamp stableTimestamp,
boost::optional<Timestamp> maximumTruncationTimestamp) {
+ if (!_keepDataHistory) {
+ return;
+ }
+
if (stableTimestamp.isNull()) {
return;
}
@@ -1449,17 +1465,24 @@ void WiredTigerKVEngine::setOldestTimestampFromStable() {
newOldestTimestamp = oplogReadTimestamp;
}
- _setOldestTimestamp(newOldestTimestamp, false);
-}
-
-void WiredTigerKVEngine::setOldestTimestamp(Timestamp newOldestTimestamp) {
- _setOldestTimestamp(newOldestTimestamp, true);
+ setOldestTimestamp(newOldestTimestamp, false);
}
-void WiredTigerKVEngine::_setOldestTimestamp(Timestamp newOldestTimestamp, bool force) {
+void WiredTigerKVEngine::setOldestTimestamp(Timestamp newOldestTimestamp, bool force) {
if (MONGO_FAIL_POINT(WTPreserveSnapshotHistoryIndefinitely)) {
return;
}
+ const auto localSnapshotTimestamp = _sessionCache->snapshotManager().getLocalSnapshot();
+ if (!force && localSnapshotTimestamp && newOldestTimestamp > *localSnapshotTimestamp) {
+ // When force is not set, lag the `oldest timestamp` to the local snapshot timestamp.
+ // Secondary reads are performed at the local snapshot timestamp, so advancing the oldest
+ // timestamp beyond the local snapshot timestamp could cause secondary reads to fail. This
+ // is not a problem when majority read concern is enabled, since the replication system will
+ // not set the stable timestamp ahead of the local snapshot timestamp. However, when
+ // majority read concern is disabled and the oldest timestamp is set by the oplog manager,
+ // the oplog manager can set the oldest timestamp ahead of the local snapshot timestamp.
+ newOldestTimestamp = *localSnapshotTimestamp;
+ }
char oldestTSConfigString["force=true,oldest_timestamp=,commit_timestamp="_sd.size() +
(2 * 8 * 2) /* 2 timestamps of 16 hexadecimal digits each */ +
@@ -1530,6 +1553,13 @@ void WiredTigerKVEngine::setInitialDataTimestamp(Timestamp initialDataTimestamp)
}
bool WiredTigerKVEngine::supportsRecoverToStableTimestamp() const {
+ if (!_keepDataHistory) {
+ return false;
+ }
+ return true;
+}
+
+bool WiredTigerKVEngine::supportsRecoveryTimestamp() const {
return true;
}
@@ -1602,8 +1632,8 @@ Timestamp WiredTigerKVEngine::getAllCommittedTimestamp() const {
}
boost::optional<Timestamp> WiredTigerKVEngine::getRecoveryTimestamp() const {
- if (!supportsRecoverToStableTimestamp()) {
- severe() << "WiredTiger is configured to not support recover to a stable timestamp";
+ if (!supportsRecoveryTimestamp()) {
+ severe() << "WiredTiger is configured to not support providing a recovery timestamp";
fassertFailed(50745);
}
@@ -1642,10 +1672,6 @@ boost::optional<Timestamp> WiredTigerKVEngine::getLastStableRecoveryTimestamp()
}
Timestamp WiredTigerKVEngine::getOplogNeededForRollback() const {
- // TODO: SERVER-36982 intends to allow holding onto minimum history (in front of the stable
- // timestamp). If that results in never calling `StorageEngine::setStableTimestamp`, oplog
- // will never be truncated. This method will need to be updated to accomodate that case, most
- // simply by having this return `Timestamp::max()`.
return Timestamp(_oplogNeededForRollback.load());
}
@@ -1658,6 +1684,10 @@ boost::optional<Timestamp> WiredTigerKVEngine::getOplogNeededForCrashRecovery()
}
Timestamp WiredTigerKVEngine::getPinnedOplog() const {
+ if (!_keepDataHistory) {
+ // We use rollbackViaRefetch and take full checkpoints, so there is no need to pin oplog.
+ return Timestamp::max();
+ }
return getOplogNeededForCrashRecovery().value_or(getOplogNeededForRollback());
}
@@ -1665,12 +1695,16 @@ bool WiredTigerKVEngine::supportsReadConcernSnapshot() const {
return true;
}
+bool WiredTigerKVEngine::supportsReadConcernMajority() const {
+ return _keepDataHistory;
+}
+
void WiredTigerKVEngine::startOplogManager(OperationContext* opCtx,
const std::string& uri,
WiredTigerRecordStore* oplogRecordStore) {
stdx::lock_guard<stdx::mutex> lock(_oplogManagerMutex);
if (_oplogManagerCount == 0)
- _oplogManager->start(opCtx, uri, oplogRecordStore);
+ _oplogManager->start(opCtx, uri, oplogRecordStore, !_keepDataHistory);
_oplogManagerCount++;
}
diff --git a/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.h b/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.h
index 32d8e93d438..6712bd35fc3 100644
--- a/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.h
+++ b/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.h
@@ -195,10 +195,17 @@ public:
virtual void setOldestTimestampFromStable() override;
- virtual void setOldestTimestamp(Timestamp newOldestTimestamp) override;
+ /**
+ * 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).
+ */
+ virtual void setOldestTimestamp(Timestamp newOldestTimestamp, bool force) override;
virtual bool supportsRecoverToStableTimestamp() const override;
+ virtual bool supportsRecoveryTimestamp() const override;
+
virtual StatusWith<Timestamp> recoverToStableTimestamp(OperationContext* opCtx) override;
virtual boost::optional<Timestamp> getRecoveryTimestamp() const override;
@@ -229,6 +236,8 @@ public:
bool isCacheUnderPressure(OperationContext* opCtx) const override;
+ bool supportsReadConcernMajority() const final;
+
// wiredtiger specific
// Calls WT_CONNECTION::reconfigure on the underlying WT_CONNECTION
// held by this class
@@ -384,13 +393,6 @@ private:
*/
bool _canRecoverToStableTimestamp() const;
- /**
- * 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).
- */
- void _setOldestTimestamp(Timestamp newOldestTimestamp, bool force);
-
WT_CONNECTION* _conn;
WiredTigerFileVersion _fileVersion;
WiredTigerEventHandler _eventHandler;
@@ -414,6 +416,14 @@ private:
bool _ephemeral; // whether we are using the in-memory mode of the WT engine
const bool _inRepairMode;
bool _readOnly;
+
+ // If _keepDataHistory is true, then the storage engine keeps all history after the stable
+ // timestamp, and WiredTigerKVEngine is responsible for advancing the oldest timestamp. If
+ // _keepDataHistory is false (i.e. majority reads are disabled), then we only keep history after
+ // the "no holes point", and WiredTigerOplogManager is responsible for advancing the oldest
+ // timestamp.
+ const bool _keepDataHistory = true;
+
std::unique_ptr<WiredTigerJournalFlusher> _journalFlusher; // Depends on _sizeStorer
std::unique_ptr<WiredTigerCheckpointThread> _checkpointThread;
diff --git a/src/mongo/db/storage/wiredtiger/wiredtiger_oplog_manager.cpp b/src/mongo/db/storage/wiredtiger/wiredtiger_oplog_manager.cpp
index 63c5f4f66e1..9359d1210ae 100644
--- a/src/mongo/db/storage/wiredtiger/wiredtiger_oplog_manager.cpp
+++ b/src/mongo/db/storage/wiredtiger/wiredtiger_oplog_manager.cpp
@@ -52,7 +52,8 @@ MONGO_FAIL_POINT_DEFINE(WTPausePrimaryOplogDurabilityLoop);
void WiredTigerOplogManager::start(OperationContext* opCtx,
const std::string& uri,
- WiredTigerRecordStore* oplogRecordStore) {
+ WiredTigerRecordStore* oplogRecordStore,
+ bool updateOldestTimestamp) {
invariant(!_isRunning);
// Prime the oplog read timestamp.
std::unique_ptr<SeekableRecordCursor> reverseOplogCursor =
@@ -80,7 +81,8 @@ void WiredTigerOplogManager::start(OperationContext* opCtx,
_oplogJournalThread = stdx::thread(&WiredTigerOplogManager::_oplogJournalThreadLoop,
this,
WiredTigerRecoveryUnit::get(opCtx)->getSessionCache(),
- oplogRecordStore);
+ oplogRecordStore,
+ updateOldestTimestamp);
_isRunning = true;
_shuttingDown = false;
@@ -161,8 +163,9 @@ void WiredTigerOplogManager::triggerJournalFlush() {
}
}
-void WiredTigerOplogManager::_oplogJournalThreadLoop(
- WiredTigerSessionCache* sessionCache, WiredTigerRecordStore* oplogRecordStore) noexcept {
+void WiredTigerOplogManager::_oplogJournalThreadLoop(WiredTigerSessionCache* sessionCache,
+ WiredTigerRecordStore* oplogRecordStore,
+ const bool updateOldestTimestamp) noexcept {
Client::initThread("WTOplogJournalThread");
// This thread updates the oplog read timestamp, the timestamp used to read from the oplog with
@@ -239,6 +242,11 @@ void WiredTigerOplogManager::_oplogJournalThreadLoop(
}
lk.unlock();
+ if (updateOldestTimestamp) {
+ const bool force = false;
+ sessionCache->getKVEngine()->setOldestTimestamp(Timestamp(newTimestamp), force);
+ }
+
// Wake up any await_data cursors and tell them more data might be visible now.
oplogRecordStore->notifyCappedWaitersIfNeeded();
}
diff --git a/src/mongo/db/storage/wiredtiger/wiredtiger_oplog_manager.h b/src/mongo/db/storage/wiredtiger/wiredtiger_oplog_manager.h
index 819faf80907..7e8f72a775a 100644
--- a/src/mongo/db/storage/wiredtiger/wiredtiger_oplog_manager.h
+++ b/src/mongo/db/storage/wiredtiger/wiredtiger_oplog_manager.h
@@ -52,10 +52,12 @@ public:
~WiredTigerOplogManager() {}
// This method will initialize the oplog read timestamp and start the background thread that
- // refreshes the value.
+ // refreshes the value. If `updateOldestTimestamp` is true, the background thread will also take
+ // responsibility for updating the oldest timestamp.
void start(OperationContext* opCtx,
const std::string& uri,
- WiredTigerRecordStore* oplogRecordStore);
+ WiredTigerRecordStore* oplogRecordStore,
+ bool updateOldestTimestamp);
void halt();
@@ -84,7 +86,8 @@ public:
private:
void _oplogJournalThreadLoop(WiredTigerSessionCache* sessionCache,
- WiredTigerRecordStore* oplogRecordStore) noexcept;
+ WiredTigerRecordStore* oplogRecordStore,
+ const bool updateOldestTimestamp) noexcept;
void _setOplogReadTimestamp(WithLock, uint64_t newTimestamp);
diff --git a/src/mongo/db/storage/wiredtiger/wiredtiger_record_store.cpp b/src/mongo/db/storage/wiredtiger/wiredtiger_record_store.cpp
index b3eb7e9a9f7..04bdd8d3636 100644
--- a/src/mongo/db/storage/wiredtiger/wiredtiger_record_store.cpp
+++ b/src/mongo/db/storage/wiredtiger/wiredtiger_record_store.cpp
@@ -1820,23 +1820,31 @@ void WiredTigerRecordStore::cappedTruncateAfter(OperationContext* opCtx,
Timestamp truncTs(lastKeptId.repr());
LOG(logLevel) << "Rewinding oplog visibility point to " << truncTs << " after truncation.";
+ if (!serverGlobalParams.enableMajorityReadConcern) {
+ // If majority read concern is disabled, we must set the oldest timestamp along with the
+ // commit timestamp. Otherwise, the commit timestamp might be set behind the oldest
+ // timestamp.
+ const bool force = true;
+ _kvEngine->setOldestTimestamp(truncTs, force);
+ } else {
+ char commitTSConfigString["commit_timestamp="_sd.size() +
+ (8 * 2) /* 8 hexadecimal characters */ +
+ 1 /* trailing null */];
+ auto size = std::snprintf(commitTSConfigString,
+ sizeof(commitTSConfigString),
+ "commit_timestamp=%llx",
+ truncTs.asULL());
+ if (size < 0) {
+ int e = errno;
+ error() << "error snprintf " << errnoWithDescription(e);
+ fassertFailedNoTrace(40662);
+ }
- char commitTSConfigString["commit_timestamp="_sd.size() +
- (8 * 2) /* 8 hexadecimal characters */ + 1 /* trailing null */];
- auto size = std::snprintf(commitTSConfigString,
- sizeof(commitTSConfigString),
- "commit_timestamp=%llx",
- truncTs.asULL());
- if (size < 0) {
- int e = errno;
- error() << "error snprintf " << errnoWithDescription(e);
- fassertFailedNoTrace(40662);
+ invariant(static_cast<std::size_t>(size) < sizeof(commitTSConfigString));
+ auto conn = WiredTigerRecoveryUnit::get(opCtx)->getSessionCache()->conn();
+ invariantWTOK(conn->set_timestamp(conn, commitTSConfigString));
}
- invariant(static_cast<std::size_t>(size) < sizeof(commitTSConfigString));
- auto conn = WiredTigerRecoveryUnit::get(opCtx)->getSessionCache()->conn();
- invariantWTOK(conn->set_timestamp(conn, commitTSConfigString));
-
_kvEngine->getOplogManager()->setOplogReadTimestamp(truncTs);
LOG(1) << "truncation new read timestamp: " << truncTs;
}
diff --git a/src/mongo/db/storage/wiredtiger/wiredtiger_snapshot_manager.cpp b/src/mongo/db/storage/wiredtiger/wiredtiger_snapshot_manager.cpp
index 011db3e3566..c7998afc045 100644
--- a/src/mongo/db/storage/wiredtiger/wiredtiger_snapshot_manager.cpp
+++ b/src/mongo/db/storage/wiredtiger/wiredtiger_snapshot_manager.cpp
@@ -33,6 +33,7 @@
#include "mongo/db/storage/wiredtiger/wiredtiger_snapshot_manager.h"
+#include "mongo/db/server_options.h"
#include "mongo/db/storage/wiredtiger/wiredtiger_begin_transaction_block.h"
#include "mongo/db/storage/wiredtiger/wiredtiger_oplog_manager.h"
#include "mongo/db/storage/wiredtiger/wiredtiger_util.h"
@@ -63,6 +64,10 @@ void WiredTigerSnapshotManager::dropAllSnapshots() {
}
boost::optional<Timestamp> WiredTigerSnapshotManager::getMinSnapshotForNextCommittedRead() const {
+ if (!serverGlobalParams.enableMajorityReadConcern) {
+ return boost::none;
+ }
+
stdx::lock_guard<stdx::mutex> lock(_committedSnapshotMutex);
return _committedSnapshot;
}
diff --git a/src/mongo/db/storage/wiredtiger/wiredtiger_util.cpp b/src/mongo/db/storage/wiredtiger/wiredtiger_util.cpp
index 004062a6d04..2188df7da8c 100644
--- a/src/mongo/db/storage/wiredtiger/wiredtiger_util.cpp
+++ b/src/mongo/db/storage/wiredtiger/wiredtiger_util.cpp
@@ -38,6 +38,7 @@
#include "mongo/base/simple_string_data_comparator.h"
#include "mongo/bson/bsonobjbuilder.h"
#include "mongo/db/concurrency/write_conflict_exception.h"
+#include "mongo/db/server_options.h"
#include "mongo/db/server_parameters.h"
#include "mongo/db/snapshot_window_options.h"
#include "mongo/db/storage/wiredtiger/wiredtiger_kv_engine.h"
@@ -531,6 +532,10 @@ bool WiredTigerUtil::useTableLogging(NamespaceString ns, bool replEnabled) {
return true;
}
+ if (!serverGlobalParams.enableMajorityReadConcern) {
+ return true;
+ }
+
// Of the replica set configurations:
if (ns.db() != "local") {
// All replicated collections are not logged.
diff --git a/src/mongo/dbtests/SConscript b/src/mongo/dbtests/SConscript
index 52b834e2e12..61fd4c1b8a6 100644
--- a/src/mongo/dbtests/SConscript
+++ b/src/mongo/dbtests/SConscript
@@ -11,6 +11,7 @@ env.Library(
"framework_options.cpp",
],
LIBDEPS=[
+ '$BUILD_DIR/mongo/db/server_options_core',
'$BUILD_DIR/mongo/db/storage/storage_options',
'$BUILD_DIR/mongo/util/options_parser/options_parser_init',
'$BUILD_DIR/mongo/unittest/unittest',
diff --git a/src/mongo/dbtests/framework_options.cpp b/src/mongo/dbtests/framework_options.cpp
index d9929c5842c..4514ebe5da4 100644
--- a/src/mongo/dbtests/framework_options.cpp
+++ b/src/mongo/dbtests/framework_options.cpp
@@ -95,6 +95,14 @@ Status addTestFrameworkOptions(moe::OptionSection* options) {
options->addOptionChaining(
"perfHist", "perfHist", moe::Unsigned, "number of back runs of perf stats to display");
+ // If set to true, storage engine maintains the data history. Else, it won't maintain the data
+ // history. This setting applies only to 'wiredTiger' storage engine.
+ options
+ ->addOptionChaining("replication.enableMajorityReadConcern",
+ "enableMajorityReadConcern",
+ moe::Bool,
+ "enables majority readConcern")
+ .setDefault(moe::Value(true));
options
->addOptionChaining(
"storage.engine", "storageEngine", moe::String, "what storage engine to use")
@@ -188,6 +196,12 @@ Status storeTestFrameworkOptions(const moe::Environment& params,
storageGlobalParams.engine = params["storage.engine"].as<string>();
+ if (storageGlobalParams.engine == "wiredTiger" &&
+ params.count("replication.enableMajorityReadConcern")) {
+ serverGlobalParams.enableMajorityReadConcern =
+ params["replication.enableMajorityReadConcern"].as<bool>();
+ }
+
if (params.count("suites")) {
frameworkGlobalParams.suites = params["suites"].as<vector<string>>();
}
diff --git a/src/mongo/dbtests/repltests.cpp b/src/mongo/dbtests/repltests.cpp
index a2d6e66b2c7..0a939bd3a44 100644
--- a/src/mongo/dbtests/repltests.cpp
+++ b/src/mongo/dbtests/repltests.cpp
@@ -234,6 +234,15 @@ protected:
}
}
{
+ if (!serverGlobalParams.enableMajorityReadConcern) {
+ if (ops.size() > 0) {
+ if (auto tsElem = ops.front()["ts"]) {
+ _opCtx.getServiceContext()->getStorageEngine()->setOldestTimestamp(
+ tsElem.timestamp());
+ }
+ }
+ }
+
OldClientContext ctx(&_opCtx, ns());
for (vector<BSONObj>::iterator i = ops.begin(); i != ops.end(); ++i) {
if (0) {
diff --git a/src/mongo/dbtests/storage_timestamp_tests.cpp b/src/mongo/dbtests/storage_timestamp_tests.cpp
index bb26239575c..f7abe822a24 100644
--- a/src/mongo/dbtests/storage_timestamp_tests.cpp
+++ b/src/mongo/dbtests/storage_timestamp_tests.cpp
@@ -2432,7 +2432,8 @@ class CreateCollectionWithSystemIndex : public StorageTimestampTest {
public:
void run() {
// Only run on 'wiredTiger'. No other storage engines to-date support timestamp writes.
- if (mongo::storageGlobalParams.engine != "wiredTiger") {
+ if (!(mongo::storageGlobalParams.engine == "wiredTiger" &&
+ mongo::serverGlobalParams.enableMajorityReadConcern)) {
return;
}
@@ -2688,7 +2689,8 @@ public:
void setupTests() {
// Only run on storage engines that support snapshot reads.
auto storageEngine = cc().getServiceContext()->getStorageEngine();
- if (!storageEngine->supportsReadConcernSnapshot()) {
+ if (!storageEngine->supportsReadConcernSnapshot() ||
+ !mongo::serverGlobalParams.enableMajorityReadConcern) {
unittest::log() << "Skipping this test suite because storage engine "
<< storageGlobalParams.engine << " does not support timestamp writes.";
return;
diff --git a/src/mongo/s/commands/strategy.cpp b/src/mongo/s/commands/strategy.cpp
index 861435ae606..99f0d18a807 100644
--- a/src/mongo/s/commands/strategy.cpp
+++ b/src/mongo/s/commands/strategy.cpp
@@ -349,8 +349,8 @@ void runCommand(OperationContext* opCtx,
}
boost::optional<ScopedRouterSession> scopedSession;
- auto osi = initializeOperationSessionInfo(
- opCtx, request.body, command->requiresAuth(), true, true, true);
+ auto osi =
+ initializeOperationSessionInfo(opCtx, request.body, command->requiresAuth(), true, true);
try {
if (osi && osi->getAutocommit()) {
diff --git a/src/mongo/shell/servers.js b/src/mongo/shell/servers.js
index 317dad5755b..5fcb9e05109 100644
--- a/src/mongo/shell/servers.js
+++ b/src/mongo/shell/servers.js
@@ -649,20 +649,6 @@ var MongoRunner, _startMongod, startMongoProgram, runMongoProgram, startMongoPro
opts.auditDestination = jsTestOptions().auditDestination;
}
- if (opts.hasOwnProperty("enableMajorityReadConcern")) {
- // opts.enableMajorityReadConcern, if set, must be an empty string
- if (opts.enableMajorityReadConcern !== "") {
- throw new Error("The enableMajorityReadConcern option must be an empty string if " +
- "it is specified");
- }
- } else if (jsTestOptions().enableMajorityReadConcern !== undefined) {
- if (jsTestOptions().enableMajorityReadConcern !== "") {
- throw new Error("The enableMajorityReadConcern option must be an empty string if " +
- "it is specified");
- }
- opts.enableMajorityReadConcern = "";
- }
-
if (opts.noReplSet)
opts.replSet = null;
if (opts.arbiter)
@@ -1170,6 +1156,11 @@ var MongoRunner, _startMongod, startMongoProgram, runMongoProgram, startMongoPro
if (programName.endsWith('mongod')) {
if (jsTest.options().storageEngine === "wiredTiger" ||
!jsTest.options().storageEngine) {
+ if (!argArrayContains("--enableMajorityReadConcern")) {
+ argArray.push(
+ ...['--enableMajorityReadConcern',
+ jsTest.options().enableMajorityReadConcern.toString()]);
+ }
if (jsTest.options().storageEngineCacheSizeGB &&
!argArrayContains('--wiredTigerCacheSizeGB')) {
argArray.push(...['--wiredTigerCacheSizeGB',