summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-rw-r--r--src/mongo/bson/timestamp.h6
-rw-r--r--src/mongo/db/catalog/collection.h9
-rw-r--r--src/mongo/db/catalog/collection_impl.h7
-rw-r--r--src/mongo/db/catalog/collection_mock.h4
-rw-r--r--src/mongo/db/catalog/index_catalog_entry.h10
-rw-r--r--src/mongo/db/catalog/index_catalog_entry_impl.h8
-rw-r--r--src/mongo/db/commands/feature_compatibility_version.cpp2
-rw-r--r--src/mongo/db/commands/snapshot_management.cpp7
-rw-r--r--src/mongo/db/read_concern.cpp5
-rw-r--r--src/mongo/db/repl/SConscript2
-rw-r--r--src/mongo/db/repl/initial_syncer.cpp9
-rw-r--r--src/mongo/db/repl/initial_syncer_test.cpp11
-rw-r--r--src/mongo/db/repl/oplog.cpp23
-rw-r--r--src/mongo/db/repl/oplog.h5
-rw-r--r--src/mongo/db/repl/oplog_buffer_collection_test.cpp35
-rw-r--r--src/mongo/db/repl/repl_client_info.h8
-rw-r--r--src/mongo/db/repl/replication_consistency_markers_impl_test.cpp2
-rw-r--r--src/mongo/db/repl/replication_coordinator.h14
-rw-r--r--src/mongo/db/repl/replication_coordinator_external_state.cpp47
-rw-r--r--src/mongo/db/repl/replication_coordinator_external_state.h32
-rw-r--r--src/mongo/db/repl/replication_coordinator_external_state_impl.cpp8
-rw-r--r--src/mongo/db/repl/replication_coordinator_external_state_impl.h2
-rw-r--r--src/mongo/db/repl/replication_coordinator_external_state_mock.cpp5
-rw-r--r--src/mongo/db/repl/replication_coordinator_external_state_mock.h2
-rw-r--r--src/mongo/db/repl/replication_coordinator_impl.cpp61
-rw-r--r--src/mongo/db/repl/replication_coordinator_impl.h19
-rw-r--r--src/mongo/db/repl/replication_coordinator_impl_test.cpp22
-rw-r--r--src/mongo/db/repl/replication_coordinator_mock.cpp12
-rw-r--r--src/mongo/db/repl/replication_coordinator_mock.h8
-rw-r--r--src/mongo/db/repl/replication_process_test.cpp8
-rw-r--r--src/mongo/db/repl/replication_recovery.cpp5
-rw-r--r--src/mongo/db/repl/replication_recovery_test.cpp40
-rw-r--r--src/mongo/db/repl/rollback_impl_test.cpp8
-rw-r--r--src/mongo/db/repl/storage_interface.h8
-rw-r--r--src/mongo/db/repl/storage_interface_impl.cpp7
-rw-r--r--src/mongo/db/repl/storage_interface_impl.h5
-rw-r--r--src/mongo/db/repl/storage_interface_impl_test.cpp224
-rw-r--r--src/mongo/db/repl/storage_interface_mock.cpp9
-rw-r--r--src/mongo/db/repl/storage_interface_mock.h13
-rw-r--r--src/mongo/db/repl/sync_tail.cpp6
-rw-r--r--src/mongo/db/storage/SConscript7
-rw-r--r--src/mongo/db/storage/kv/kv_engine.h6
-rw-r--r--src/mongo/db/storage/kv/kv_engine_test_timestamps.cpp55
-rw-r--r--src/mongo/db/storage/kv/kv_storage_engine.cpp4
-rw-r--r--src/mongo/db/storage/kv/kv_storage_engine.h5
-rw-r--r--src/mongo/db/storage/recovery_unit.h14
-rw-r--r--src/mongo/db/storage/snapshot_manager.h3
-rw-r--r--src/mongo/db/storage/snapshot_name.h97
-rw-r--r--src/mongo/db/storage/storage_engine.h6
-rw-r--r--src/mongo/db/storage/storage_snapshot_name_test.cpp48
-rw-r--r--src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.cpp30
-rw-r--r--src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.h9
-rw-r--r--src/mongo/db/storage/wiredtiger/wiredtiger_oplog_manager.cpp2
-rw-r--r--src/mongo/db/storage/wiredtiger/wiredtiger_record_store.cpp6
-rw-r--r--src/mongo/db/storage/wiredtiger/wiredtiger_recovery_unit.cpp11
-rw-r--r--src/mongo/db/storage/wiredtiger/wiredtiger_recovery_unit.h12
-rw-r--r--src/mongo/db/storage/wiredtiger/wiredtiger_snapshot_manager.cpp20
-rw-r--r--src/mongo/db/storage/wiredtiger/wiredtiger_snapshot_manager.h11
-rw-r--r--src/mongo/dbtests/storage_timestamp_tests.cpp43
59 files changed, 410 insertions, 697 deletions
diff --git a/src/mongo/bson/timestamp.h b/src/mongo/bson/timestamp.h
index 7c20b13d5bb..10fe52d4eca 100644
--- a/src/mongo/bson/timestamp.h
+++ b/src/mongo/bson/timestamp.h
@@ -47,6 +47,12 @@ public:
// Maximum Timestamp value.
static Timestamp max();
+ // Returns the minimum timestamp. Used in the context of selecting and ordering storage engine
+ // snapshots.
+ static Timestamp min() {
+ return Timestamp();
+ }
+
/**
* DEPRECATED Constructor that builds a Timestamp from a Date_t by using the
* high-order 4 bytes of "date" for the "secs" field and the low-order 4 bytes
diff --git a/src/mongo/db/catalog/collection.h b/src/mongo/db/catalog/collection.h
index 5d30401e2b6..a516e759cf3 100644
--- a/src/mongo/db/catalog/collection.h
+++ b/src/mongo/db/catalog/collection.h
@@ -36,6 +36,7 @@
#include "mongo/base/status_with.h"
#include "mongo/base/string_data.h"
#include "mongo/bson/mutable/damage_vector.h"
+#include "mongo/bson/timestamp.h"
#include "mongo/db/catalog/coll_mod.h"
#include "mongo/db/catalog/collection_info_cache.h"
#include "mongo/db/catalog/collection_options.h"
@@ -326,9 +327,9 @@ public:
BSONObjBuilder* details,
int scale) = 0;
- virtual boost::optional<SnapshotName> getMinimumVisibleSnapshot() = 0;
+ virtual boost::optional<Timestamp> getMinimumVisibleSnapshot() = 0;
- virtual void setMinimumVisibleSnapshot(SnapshotName name) = 0;
+ virtual void setMinimumVisibleSnapshot(Timestamp name) = 0;
virtual void notifyCappedWaitersIfNeeded() = 0;
@@ -712,11 +713,11 @@ public:
* If return value is not boost::none, reads with majority read concern using an older snapshot
* must error.
*/
- inline boost::optional<SnapshotName> getMinimumVisibleSnapshot() {
+ inline boost::optional<Timestamp> getMinimumVisibleSnapshot() {
return this->_impl().getMinimumVisibleSnapshot();
}
- inline void setMinimumVisibleSnapshot(const SnapshotName name) {
+ inline void setMinimumVisibleSnapshot(const Timestamp name) {
return this->_impl().setMinimumVisibleSnapshot(name);
}
diff --git a/src/mongo/db/catalog/collection_impl.h b/src/mongo/db/catalog/collection_impl.h
index cba3445042e..b823dc7d2c9 100644
--- a/src/mongo/db/catalog/collection_impl.h
+++ b/src/mongo/db/catalog/collection_impl.h
@@ -28,6 +28,7 @@
#pragma once
+#include "mongo/bson/timestamp.h"
#include "mongo/db/catalog/collection.h"
#include "mongo/db/catalog/collection_catalog_entry.h"
#include "mongo/db/catalog/index_catalog.h"
@@ -341,11 +342,11 @@ public:
* If return value is not boost::none, reads with majority read concern using an older snapshot
* must error.
*/
- boost::optional<SnapshotName> getMinimumVisibleSnapshot() final {
+ boost::optional<Timestamp> getMinimumVisibleSnapshot() final {
return _minVisibleSnapshot;
}
- void setMinimumVisibleSnapshot(SnapshotName name) final {
+ void setMinimumVisibleSnapshot(Timestamp name) final {
_minVisibleSnapshot = name;
}
@@ -466,7 +467,7 @@ private:
const std::shared_ptr<CappedInsertNotifier> _cappedNotifier;
// The earliest snapshot that is allowed to use this collection.
- boost::optional<SnapshotName> _minVisibleSnapshot;
+ boost::optional<Timestamp> _minVisibleSnapshot;
Collection* _this;
diff --git a/src/mongo/db/catalog/collection_mock.h b/src/mongo/db/catalog/collection_mock.h
index 405a3fde968..f548e9489ff 100644
--- a/src/mongo/db/catalog/collection_mock.h
+++ b/src/mongo/db/catalog/collection_mock.h
@@ -265,11 +265,11 @@ public:
std::abort();
}
- boost::optional<SnapshotName> getMinimumVisibleSnapshot() {
+ boost::optional<Timestamp> getMinimumVisibleSnapshot() {
std::abort();
}
- void setMinimumVisibleSnapshot(SnapshotName name) {
+ void setMinimumVisibleSnapshot(Timestamp name) {
std::abort();
}
diff --git a/src/mongo/db/catalog/index_catalog_entry.h b/src/mongo/db/catalog/index_catalog_entry.h
index 9a503d317c0..13edf2b4c21 100644
--- a/src/mongo/db/catalog/index_catalog_entry.h
+++ b/src/mongo/db/catalog/index_catalog_entry.h
@@ -33,10 +33,10 @@
#include "mongo/base/owned_pointer_vector.h"
#include "mongo/bson/ordering.h"
+#include "mongo/bson/timestamp.h"
#include "mongo/db/index/multikey_paths.h"
#include "mongo/db/record_id.h"
#include "mongo/db/storage/kv/kv_prefix.h"
-#include "mongo/db/storage/snapshot_name.h"
#include "mongo/platform/atomic_word.h"
#include "mongo/stdx/functional.h"
#include "mongo/stdx/mutex.h"
@@ -97,9 +97,9 @@ public:
virtual KVPrefix getPrefix() const = 0;
- virtual boost::optional<SnapshotName> getMinimumVisibleSnapshot() = 0;
+ virtual boost::optional<Timestamp> getMinimumVisibleSnapshot() = 0;
- virtual void setMinimumVisibleSnapshot(SnapshotName name) = 0;
+ virtual void setMinimumVisibleSnapshot(Timestamp name) = 0;
};
private:
@@ -232,11 +232,11 @@ public:
* If return value is not boost::none, reads with majority read concern using an older snapshot
* must treat this index as unfinished.
*/
- boost::optional<SnapshotName> getMinimumVisibleSnapshot() {
+ boost::optional<Timestamp> getMinimumVisibleSnapshot() {
return this->_impl().getMinimumVisibleSnapshot();
}
- void setMinimumVisibleSnapshot(const SnapshotName name) {
+ void setMinimumVisibleSnapshot(const Timestamp name) {
return this->_impl().setMinimumVisibleSnapshot(name);
}
diff --git a/src/mongo/db/catalog/index_catalog_entry_impl.h b/src/mongo/db/catalog/index_catalog_entry_impl.h
index 18c8896c476..2a03cd363d6 100644
--- a/src/mongo/db/catalog/index_catalog_entry_impl.h
+++ b/src/mongo/db/catalog/index_catalog_entry_impl.h
@@ -33,12 +33,12 @@
#include "mongo/base/owned_pointer_vector.h"
#include "mongo/bson/ordering.h"
+#include "mongo/bson/timestamp.h"
#include "mongo/db/catalog/index_catalog_entry.h"
#include "mongo/db/index/multikey_paths.h"
#include "mongo/db/matcher/expression.h"
#include "mongo/db/record_id.h"
#include "mongo/db/storage/kv/kv_prefix.h"
-#include "mongo/db/storage/snapshot_name.h"
#include "mongo/platform/atomic_word.h"
#include "mongo/stdx/mutex.h"
@@ -152,11 +152,11 @@ public:
* If return value is not boost::none, reads with majority read concern using an older snapshot
* must treat this index as unfinished.
*/
- boost::optional<SnapshotName> getMinimumVisibleSnapshot() final {
+ boost::optional<Timestamp> getMinimumVisibleSnapshot() final {
return _minVisibleSnapshot;
}
- void setMinimumVisibleSnapshot(SnapshotName name) final {
+ void setMinimumVisibleSnapshot(Timestamp name) final {
_minVisibleSnapshot = name;
}
@@ -226,6 +226,6 @@ private:
const KVPrefix _prefix;
// The earliest snapshot that is allowed to read this index.
- boost::optional<SnapshotName> _minVisibleSnapshot;
+ boost::optional<Timestamp> _minVisibleSnapshot;
};
} // namespace mongo
diff --git a/src/mongo/db/commands/feature_compatibility_version.cpp b/src/mongo/db/commands/feature_compatibility_version.cpp
index 9e31469fe51..b14de817ea2 100644
--- a/src/mongo/db/commands/feature_compatibility_version.cpp
+++ b/src/mongo/db/commands/feature_compatibility_version.cpp
@@ -248,7 +248,7 @@ void FeatureCompatibilityVersion::setIfCleanStartup(OperationContext* opCtx,
<< (storeUpgradeVersion
? FeatureCompatibilityVersionCommandParser::kVersion36
: FeatureCompatibilityVersionCommandParser::kVersion34)),
- SnapshotName()},
+ Timestamp()},
repl::OpTime::kUninitializedTerm)); // No timestamp or term because this write is not
// replicated.
}
diff --git a/src/mongo/db/commands/snapshot_management.cpp b/src/mongo/db/commands/snapshot_management.cpp
index 324871e01c2..5bad41d15eb 100644
--- a/src/mongo/db/commands/snapshot_management.cpp
+++ b/src/mongo/db/commands/snapshot_management.cpp
@@ -31,6 +31,7 @@
#include "mongo/platform/basic.h"
#include "mongo/base/init.h"
+#include "mongo/bson/timestamp.h"
#include "mongo/db/commands.h"
#include "mongo/db/concurrency/d_concurrency.h"
#include "mongo/db/operation_context.h"
@@ -79,7 +80,7 @@ public:
auto status = snapshotManager->prepareForCreateSnapshot(opCtx);
if (status.isOK()) {
const auto name = repl::ReplicationCoordinator::get(opCtx)->reserveSnapshotName(opCtx);
- result.append("name", static_cast<long long>(name.asU64()));
+ result.append("name", static_cast<long long>(name.asULL()));
}
return appendCommandStatus(result, status);
}
@@ -121,8 +122,8 @@ public:
}
Lock::GlobalLock lk(opCtx, MODE_IX, UINT_MAX);
- auto name = SnapshotName(cmdObj.firstElement().Long());
- snapshotManager->setCommittedSnapshot(name, Timestamp(name.asU64()));
+ auto timestamp = Timestamp(cmdObj.firstElement().Long());
+ snapshotManager->setCommittedSnapshot(timestamp);
return true;
}
};
diff --git a/src/mongo/db/read_concern.cpp b/src/mongo/db/read_concern.cpp
index 4a11f36908f..afae793f6e8 100644
--- a/src/mongo/db/read_concern.cpp
+++ b/src/mongo/db/read_concern.cpp
@@ -256,8 +256,7 @@ Status waitForReadConcern(OperationContext* opCtx,
auto pointInTime = readConcernArgs.getArgsPointInTime();
if (pointInTime) {
- fassertStatusOK(
- 39345, opCtx->recoveryUnit()->selectSnapshot(SnapshotName(pointInTime->asTimestamp())));
+ fassertStatusOK(39345, opCtx->recoveryUnit()->selectSnapshot(pointInTime->asTimestamp()));
}
if (!readConcernArgs.isEmpty()) {
@@ -296,7 +295,7 @@ Status waitForReadConcern(OperationContext* opCtx,
// Wait until a snapshot is available.
while (status == ErrorCodes::ReadConcernMajorityNotAvailableYet) {
LOG(debugLevel) << "Snapshot not available yet.";
- replCoord->waitUntilSnapshotCommitted(opCtx, SnapshotName::min());
+ replCoord->waitUntilSnapshotCommitted(opCtx, Timestamp());
status = opCtx->recoveryUnit()->setReadFromMajorityCommittedSnapshot();
}
diff --git a/src/mongo/db/repl/SConscript b/src/mongo/db/repl/SConscript
index f456284ecf9..1621bb65651 100644
--- a/src/mongo/db/repl/SConscript
+++ b/src/mongo/db/repl/SConscript
@@ -958,7 +958,6 @@ env.Library(
source=[
'repl_client_info.cpp',
'replication_coordinator.cpp',
- 'replication_coordinator_external_state.cpp',
],
LIBDEPS=[
'optime',
@@ -1654,4 +1653,3 @@ env.Library(
'$BUILD_DIR/mongo/db/service_context_d_test_fixture',
]
)
-
diff --git a/src/mongo/db/repl/initial_syncer.cpp b/src/mongo/db/repl/initial_syncer.cpp
index 66f01046277..8ad9cc954f0 100644
--- a/src/mongo/db/repl/initial_syncer.cpp
+++ b/src/mongo/db/repl/initial_syncer.cpp
@@ -355,9 +355,8 @@ void InitialSyncer::_setUp_inlock(OperationContext* opCtx, std::uint32_t initial
_replicationProcess->getConsistencyMarkers()->setInitialSyncFlag(opCtx);
auto serviceCtx = opCtx->getServiceContext();
- _storage->setInitialDataTimestamp(serviceCtx,
- SnapshotName(Timestamp::kAllowUnstableCheckpointsSentinel));
- _storage->setStableTimestamp(serviceCtx, SnapshotName::min());
+ _storage->setInitialDataTimestamp(serviceCtx, Timestamp::kAllowUnstableCheckpointsSentinel);
+ _storage->setStableTimestamp(serviceCtx, Timestamp::min());
LOG(1) << "Creating oplogBuffer.";
_oplogBuffer = _dataReplicatorExternalState->makeInitialSyncOplogBuffer(opCtx);
@@ -386,7 +385,7 @@ void InitialSyncer::_tearDown_inlock(OperationContext* opCtx,
_storage->waitForAllEarlierOplogWritesToBeVisible(opCtx);
_storage->setInitialDataTimestamp(opCtx->getServiceContext(),
- SnapshotName(lastApplied.getValue().opTime.getTimestamp()));
+ lastApplied.getValue().opTime.getTimestamp());
_replicationProcess->getConsistencyMarkers()->clearInitialSyncFlag(opCtx);
_opts.setMyLastOptime(lastApplied.getValue().opTime);
log() << "initial sync done; took "
@@ -892,7 +891,7 @@ void InitialSyncer::_lastOplogEntryFetcherCallbackForStopTimestamp(
auto status = _storage->insertDocument(
opCtx.get(),
_opts.localOplogNS,
- TimestampedBSONObj{oplogSeedDoc, SnapshotName(optimeWithHash.opTime.getTimestamp())},
+ TimestampedBSONObj{oplogSeedDoc, optimeWithHash.opTime.getTimestamp()},
optimeWithHash.opTime.getTerm());
if (!status.isOK()) {
stdx::lock_guard<stdx::mutex> lock(_mutex);
diff --git a/src/mongo/db/repl/initial_syncer_test.cpp b/src/mongo/db/repl/initial_syncer_test.cpp
index 9df6094127b..7cb8d53cbde 100644
--- a/src/mongo/db/repl/initial_syncer_test.cpp
+++ b/src/mongo/db/repl/initial_syncer_test.cpp
@@ -727,15 +727,15 @@ TEST_F(InitialSyncerTest, StartupSetsInitialDataTimestampAndStableTimestampOnSuc
// Set initial data timestamp forward first.
auto serviceCtx = opCtx.get()->getServiceContext();
- _storageInterface->setInitialDataTimestamp(serviceCtx, SnapshotName(Timestamp(5, 5)));
- _storageInterface->setStableTimestamp(serviceCtx, SnapshotName(Timestamp(6, 6)));
+ _storageInterface->setInitialDataTimestamp(serviceCtx, Timestamp(5, 5));
+ _storageInterface->setStableTimestamp(serviceCtx, Timestamp(6, 6));
ASSERT_OK(initialSyncer->startup(opCtx.get(), maxAttempts));
ASSERT_TRUE(initialSyncer->isActive());
- ASSERT_EQUALS(SnapshotName(Timestamp::kAllowUnstableCheckpointsSentinel),
+ ASSERT_EQUALS(Timestamp::kAllowUnstableCheckpointsSentinel,
_storageInterface->getInitialDataTimestamp());
- ASSERT_EQUALS(SnapshotName::min(), _storageInterface->getStableTimestamp());
+ ASSERT_EQUALS(Timestamp::min(), _storageInterface->getStableTimestamp());
}
TEST_F(InitialSyncerTest, InitialSyncerReturnsCallbackCanceledIfShutdownImmediatelyAfterStartup) {
@@ -3480,8 +3480,7 @@ void InitialSyncerTest::doSuccessfulInitialSyncWithOneBatch() {
ASSERT_EQUALS(lastOp.getOpTime(), unittest::assertGet(_lastApplied).opTime);
ASSERT_EQUALS(lastOp.getHash(), unittest::assertGet(_lastApplied).value);
- ASSERT_EQUALS(SnapshotName(lastOp.getOpTime().getTimestamp()),
- _storageInterface->getInitialDataTimestamp());
+ ASSERT_EQUALS(lastOp.getOpTime().getTimestamp(), _storageInterface->getInitialDataTimestamp());
}
TEST_F(InitialSyncerTest,
diff --git a/src/mongo/db/repl/oplog.cpp b/src/mongo/db/repl/oplog.cpp
index ce512effa10..a42086f4e59 100644
--- a/src/mongo/db/repl/oplog.cpp
+++ b/src/mongo/db/repl/oplog.cpp
@@ -1159,8 +1159,7 @@ Status applyOperation_inlock(OperationContext* opCtx,
// Note: we don't care about statement ids here since the secondaries don't create
// their own oplog entries.
- insertObjs.emplace_back(
- oElem.Obj(), SnapshotName(tsElem.timestamp()), tElem.Long());
+ insertObjs.emplace_back(oElem.Obj(), tsElem.timestamp(), tElem.Long());
if (!fieldOIt.more()) {
// Make sure arrays are the same length.
uassert(ErrorCodes::OperationFailed,
@@ -1213,11 +1212,11 @@ Status applyOperation_inlock(OperationContext* opCtx,
// case.
bool needToDoUpsert = haveWrappingWriteUnitOfWork;
- SnapshotName timestamp;
+ Timestamp timestamp;
long long term = OpTime::kUninitializedTerm;
if (assignOperationTimestamp) {
if (fieldTs) {
- timestamp = SnapshotName(fieldTs.timestamp());
+ timestamp = fieldTs.timestamp();
}
if (fieldT) {
term = fieldT.Long();
@@ -1231,7 +1230,7 @@ Status applyOperation_inlock(OperationContext* opCtx,
// a user to dictate what timestamps appear in the oplog.
if (assignOperationTimestamp) {
if (fieldTs.ok()) {
- timestamp = SnapshotName(fieldTs.timestamp());
+ timestamp = fieldTs.timestamp();
}
if (fieldT.ok()) {
term = fieldT.Long();
@@ -1273,7 +1272,7 @@ Status applyOperation_inlock(OperationContext* opCtx,
WriteUnitOfWork wuow(opCtx);
// If this is an atomic applyOps (i.e: `haveWrappingWriteUnitOfWork` is true),
// do not timestamp the write.
- if (assignOperationTimestamp && timestamp != SnapshotName::min()) {
+ if (assignOperationTimestamp && timestamp != Timestamp::min()) {
uassertStatusOK(opCtx->recoveryUnit()->setTimestamp(timestamp));
}
@@ -1313,15 +1312,15 @@ Status applyOperation_inlock(OperationContext* opCtx,
UpdateLifecycleImpl updateLifecycle(requestNss);
request.setLifecycle(&updateLifecycle);
- SnapshotName timestamp;
+ Timestamp timestamp;
if (assignOperationTimestamp) {
- timestamp = SnapshotName(fieldTs.timestamp());
+ timestamp = fieldTs.timestamp();
}
const StringData ns = fieldNs.valueStringData();
auto status = writeConflictRetry(opCtx, "applyOps_update", ns, [&] {
WriteUnitOfWork wuow(opCtx);
- if (timestamp != SnapshotName::min()) {
+ if (timestamp != Timestamp::min()) {
uassertStatusOK(opCtx->recoveryUnit()->setTimestamp(timestamp));
}
@@ -1388,15 +1387,15 @@ Status applyOperation_inlock(OperationContext* opCtx,
// but we want to do the delete by just _id so we can take advantage of the IDHACK.
BSONObj deleteCriteria = idField.wrap();
- SnapshotName timestamp;
+ Timestamp timestamp;
if (assignOperationTimestamp) {
- timestamp = SnapshotName(fieldTs.timestamp());
+ timestamp = fieldTs.timestamp();
}
const StringData ns = fieldNs.valueStringData();
writeConflictRetry(opCtx, "applyOps_delete", ns, [&] {
WriteUnitOfWork wuow(opCtx);
- if (timestamp != SnapshotName::min()) {
+ if (timestamp != Timestamp::min()) {
uassertStatusOK(opCtx->recoveryUnit()->setTimestamp(timestamp));
}
diff --git a/src/mongo/db/repl/oplog.h b/src/mongo/db/repl/oplog.h
index 5076fab004e..6f773694945 100644
--- a/src/mongo/db/repl/oplog.h
+++ b/src/mongo/db/repl/oplog.h
@@ -38,7 +38,6 @@
#include "mongo/db/logical_session_id.h"
#include "mongo/db/repl/optime.h"
#include "mongo/db/repl/replication_coordinator.h"
-#include "mongo/db/storage/snapshot_name.h"
#include "mongo/stdx/functional.h"
namespace mongo {
@@ -64,8 +63,8 @@ public:
InsertStatement(StmtId statementId, BSONObj toInsert) : stmtId(statementId), doc(toInsert) {}
InsertStatement(StmtId statementId, BSONObj toInsert, OplogSlot os)
: stmtId(statementId), oplogSlot(os), doc(toInsert) {}
- InsertStatement(BSONObj toInsert, SnapshotName ts, long long term)
- : oplogSlot(repl::OpTime(Timestamp(ts.asU64()), term), 0), doc(toInsert) {}
+ InsertStatement(BSONObj toInsert, Timestamp ts, long long term)
+ : oplogSlot(repl::OpTime(ts, term), 0), doc(toInsert) {}
StmtId stmtId = kUninitializedStmtId;
OplogSlot oplogSlot;
diff --git a/src/mongo/db/repl/oplog_buffer_collection_test.cpp b/src/mongo/db/repl/oplog_buffer_collection_test.cpp
index c40bebd731e..dc8d8d9bbe4 100644
--- a/src/mongo/db/repl/oplog_buffer_collection_test.cpp
+++ b/src/mongo/db/repl/oplog_buffer_collection_test.cpp
@@ -273,7 +273,7 @@ TEST_F(OplogBufferCollectionTest, StartupWithExistingCollectionInitializesCorrec
_opCtx.get(),
nss,
TimestampedBSONObj{std::get<0>(OplogBufferCollection::addIdToDocument(oplog[0], {}, 0)),
- SnapshotName(0)},
+ Timestamp(0)},
OpTime::kUninitializedTerm));
_assertDocumentsInCollectionEquals(_opCtx.get(), nss, oplog);
@@ -366,11 +366,10 @@ DEATH_TEST_F(OplogBufferCollectionTest,
CollectionOptions collOpts;
collOpts.setNoIdIndex();
ASSERT_OK(_storageInterface->createCollection(_opCtx.get(), nss, collOpts));
- ASSERT_OK(
- _storageInterface->insertDocument(_opCtx.get(),
- nss,
- TimestampedBSONObj{makeOplogEntry(1), SnapshotName(0)},
- OpTime::kUninitializedTerm));
+ ASSERT_OK(_storageInterface->insertDocument(_opCtx.get(),
+ nss,
+ TimestampedBSONObj{makeOplogEntry(1), Timestamp(0)},
+ OpTime::kUninitializedTerm));
OplogBufferCollection::Options opts;
opts.dropCollectionAtStartup = false;
@@ -383,11 +382,10 @@ DEATH_TEST_F(OplogBufferCollectionTest,
"Fatal assertion 40405 NoSuchKey: Missing expected field \"ts\"") {
auto nss = makeNamespace(_agent);
ASSERT_OK(_storageInterface->createCollection(_opCtx.get(), nss, CollectionOptions()));
- ASSERT_OK(
- _storageInterface->insertDocument(_opCtx.get(),
- nss,
- TimestampedBSONObj{BSON("_id" << 1), SnapshotName(0)},
- OpTime::kUninitializedTerm));
+ ASSERT_OK(_storageInterface->insertDocument(_opCtx.get(),
+ nss,
+ TimestampedBSONObj{BSON("_id" << 1), Timestamp(0)},
+ OpTime::kUninitializedTerm));
OplogBufferCollection::Options opts;
opts.dropCollectionAtStartup = false;
@@ -403,7 +401,7 @@ DEATH_TEST_F(OplogBufferCollectionTest,
ASSERT_OK(_storageInterface->insertDocument(
_opCtx.get(),
nss,
- TimestampedBSONObj{BSON("_id" << BSON("ts" << Timestamp(1, 1))), SnapshotName(1)},
+ TimestampedBSONObj{BSON("_id" << BSON("ts" << Timestamp(1, 1))), Timestamp(1)},
OpTime::kUninitializedTerm));
OplogBufferCollection::Options opts;
@@ -418,8 +416,7 @@ TEST_F(OplogBufferCollectionTest, PeekWithExistingCollectionReturnsEmptyObjectWh
ASSERT_OK(_storageInterface->insertDocument(
_opCtx.get(),
nss,
- TimestampedBSONObj{BSON("_id" << BSON("ts" << Timestamp(1, 1) << "s" << 0)),
- SnapshotName(1)},
+ TimestampedBSONObj{BSON("_id" << BSON("ts" << Timestamp(1, 1) << "s" << 0)), Timestamp(1)},
OpTime::kUninitializedTerm));
OplogBufferCollection::Options opts;
@@ -441,7 +438,7 @@ TEST_F(OplogBufferCollectionTest,
_opCtx.get(),
nss,
TimestampedBSONObj{std::get<0>(OplogBufferCollection::addIdToDocument(oplog[0], {}, 0)),
- SnapshotName(0)},
+ Timestamp(0)},
OpTime::kUninitializedTerm));
OplogBufferCollection::Options opts;
@@ -545,7 +542,7 @@ TEST_F(OplogBufferCollectionTest, PeekingFromExistingCollectionReturnsDocument)
_opCtx.get(),
nss,
TimestampedBSONObj{std::get<0>(OplogBufferCollection::addIdToDocument(oplog[0], {}, 0)),
- SnapshotName(0)},
+ Timestamp(0)},
OpTime::kUninitializedTerm));
OplogBufferCollection::Options opts;
@@ -689,14 +686,14 @@ TEST_F(OplogBufferCollectionTest,
_opCtx.get(),
nss,
TimestampedBSONObj{std::get<0>(OplogBufferCollection::addIdToDocument(firstDoc, {}, 0)),
- SnapshotName(0)},
+ Timestamp(0)},
OpTime::kUninitializedTerm));
auto secondDoc = makeOplogEntry(2);
ASSERT_OK(_storageInterface->insertDocument(
_opCtx.get(),
nss,
TimestampedBSONObj{std::get<0>(OplogBufferCollection::addIdToDocument(secondDoc, {}, 0)),
- SnapshotName(0)},
+ Timestamp(0)},
OpTime::kUninitializedTerm));
OplogBufferCollection::Options opts;
@@ -816,7 +813,7 @@ TEST_F(OplogBufferCollectionTest, WaitForDataReturnsImmediatelyWhenStartedWithEx
nss,
TimestampedBSONObj{
std::get<0>(OplogBufferCollection::addIdToDocument(makeOplogEntry(1), {}, 0)),
- SnapshotName(0)},
+ Timestamp(0)},
OpTime::kUninitializedTerm));
OplogBufferCollection::Options opts;
diff --git a/src/mongo/db/repl/repl_client_info.h b/src/mongo/db/repl/repl_client_info.h
index 3c3910cf78d..f6380aef75e 100644
--- a/src/mongo/db/repl/repl_client_info.h
+++ b/src/mongo/db/repl/repl_client_info.h
@@ -29,9 +29,9 @@
#pragma once
#include "mongo/bson/oid.h"
+#include "mongo/bson/timestamp.h"
#include "mongo/db/client.h"
#include "mongo/db/repl/optime.h"
-#include "mongo/db/storage/snapshot_name.h"
namespace mongo {
@@ -56,10 +56,10 @@ public:
_lastOp = OpTime();
}
- void setLastSnapshot(SnapshotName name) {
+ void setLastSnapshot(Timestamp name) {
_lastSnapshot = name;
}
- SnapshotName getLastSnapshot() const {
+ Timestamp getLastSnapshot() const {
return _lastSnapshot;
}
@@ -82,7 +82,7 @@ private:
static const long long kUninitializedTerm = -1;
OpTime _lastOp = OpTime();
- SnapshotName _lastSnapshot = SnapshotName::min();
+ Timestamp _lastSnapshot;
OID _remoteId = OID();
};
diff --git a/src/mongo/db/repl/replication_consistency_markers_impl_test.cpp b/src/mongo/db/repl/replication_consistency_markers_impl_test.cpp
index 36b9bca7fc7..ae4257507c1 100644
--- a/src/mongo/db/repl/replication_consistency_markers_impl_test.cpp
+++ b/src/mongo/db/repl/replication_consistency_markers_impl_test.cpp
@@ -360,7 +360,7 @@ TEST_F(ReplicationConsistencyMarkersTest, OplogTruncateAfterPointUpgrade) {
<< minValidTime.getTerm()
<< MinValidDocument::kOldOplogDeleteFromPointFieldName
<< time1),
- SnapshotName(0)},
+ Timestamp(0)},
OpTime::kUninitializedTerm));
consistencyMarkers.initializeMinValidDocument(opCtx);
diff --git a/src/mongo/db/repl/replication_coordinator.h b/src/mongo/db/repl/replication_coordinator.h
index 403792a0f99..ed3dc9fb9f0 100644
--- a/src/mongo/db/repl/replication_coordinator.h
+++ b/src/mongo/db/repl/replication_coordinator.h
@@ -33,6 +33,7 @@
#include "mongo/base/disallow_copying.h"
#include "mongo/base/status.h"
#include "mongo/base/status_with.h"
+#include "mongo/bson/timestamp.h"
#include "mongo/db/repl/member_state.h"
#include "mongo/db/repl/repl_settings.h"
#include "mongo/db/repl/sync_source_selector.h"
@@ -47,7 +48,6 @@ class IndexDescriptor;
class NamespaceString;
class OperationContext;
class ServiceContext;
-class SnapshotName;
class Timestamp;
struct WriteConcernOptions;
@@ -814,15 +814,7 @@ public:
* A null OperationContext can be used in cases where the snapshot to wait for should not be
* adjusted.
*/
- virtual SnapshotName reserveSnapshotName(OperationContext* opCtx) = 0;
-
- /**
- * Creates a new snapshot in the storage engine and registers it for use in the replication
- * coordinator.
- */
- virtual void createSnapshot(OperationContext* opCtx,
- OpTime timeOfSnapshot,
- SnapshotName name) = 0;
+ virtual Timestamp reserveSnapshotName(OperationContext* opCtx) = 0;
/**
* Blocks until either the current committed snapshot is at least as high as 'untilSnapshot',
@@ -830,7 +822,7 @@ public:
* 'opCtx' is used to checkForInterrupt and enforce maxTimeMS.
*/
virtual void waitUntilSnapshotCommitted(OperationContext* opCtx,
- const SnapshotName& untilSnapshot) = 0;
+ const Timestamp& untilSnapshot) = 0;
/**
* Resets all information related to snapshotting.
diff --git a/src/mongo/db/repl/replication_coordinator_external_state.cpp b/src/mongo/db/repl/replication_coordinator_external_state.cpp
deleted file mode 100644
index 637257adcba..00000000000
--- a/src/mongo/db/repl/replication_coordinator_external_state.cpp
+++ /dev/null
@@ -1,47 +0,0 @@
-/**
- * Copyright (C) 2014 MongoDB Inc.
- *
- * This program is free software: you can redistribute it and/or modify
- * it under the terms of the GNU Affero General Public License, version 3,
- * as published by the Free Software Foundation.
- *
- * This program is distributed in the hope that it will be useful,
- * but WITHOUT ANY WARRANTY; without even the implied warranty of
- * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
- * GNU Affero General Public License for more details.
- *
- * You should have received a copy of the GNU Affero General Public License
- * along with this program. If not, see <http://www.gnu.org/licenses/>.
- *
- * As a special exception, the copyright holders give permission to link the
- * code of portions of this program with the OpenSSL library under certain
- * conditions as described in each individual source file and distribute
- * linked combinations including the program with the OpenSSL library. You
- * must comply with the GNU Affero General Public License in all respects for
- * all of the code used other than as permitted herein. If you modify file(s)
- * with this exception, you may extend this exception to your version of the
- * file(s), but you are not obligated to do so. If you do not wish to do so,
- * delete this exception statement from your version. If you delete this
- * exception statement from all source files in the program, then also delete
- * it in the license file.
- */
-
-#include "mongo/platform/basic.h"
-
-#include "mongo/db/repl/replication_coordinator_external_state.h"
-
-namespace mongo {
-namespace repl {
-
-std::string SnapshotInfo::toString() const {
- BSONObjBuilder bob;
- bob.append("optime", opTime.toBSON());
- bob.append("name-id", name.toString());
- return bob.obj().toString();
-}
-
-ReplicationCoordinatorExternalState::ReplicationCoordinatorExternalState() {}
-ReplicationCoordinatorExternalState::~ReplicationCoordinatorExternalState() {}
-
-} // namespace repl
-} // namespace mongo
diff --git a/src/mongo/db/repl/replication_coordinator_external_state.h b/src/mongo/db/repl/replication_coordinator_external_state.h
index 5503e5262a3..ff12be19d49 100644
--- a/src/mongo/db/repl/replication_coordinator_external_state.h
+++ b/src/mongo/db/repl/replication_coordinator_external_state.h
@@ -47,7 +47,6 @@ class OID;
class OldThreadPool;
class OperationContext;
class ServiceContext;
-class SnapshotName;
class Status;
struct HostAndPort;
template <typename T>
@@ -59,31 +58,6 @@ class LastVote;
class ReplSettings;
class ReplicationCoordinator;
-struct SnapshotInfo {
- OpTime opTime;
- SnapshotName name;
-
- bool operator==(const SnapshotInfo& other) const {
- return std::tie(opTime, name) == std::tie(other.opTime, other.name);
- }
- bool operator!=(const SnapshotInfo& other) const {
- return std::tie(opTime, name) != std::tie(other.opTime, other.name);
- }
- bool operator<(const SnapshotInfo& other) const {
- return std::tie(opTime, name) < std::tie(other.opTime, other.name);
- }
- bool operator<=(const SnapshotInfo& other) const {
- return std::tie(opTime, name) <= std::tie(other.opTime, other.name);
- }
- bool operator>(const SnapshotInfo& other) const {
- return std::tie(opTime, name) > std::tie(other.opTime, other.name);
- }
- bool operator>=(const SnapshotInfo& other) const {
- return std::tie(opTime, name) >= std::tie(other.opTime, other.name);
- }
- std::string toString() const;
-};
-
/**
* This class represents the interface the ReplicationCoordinator uses to interact with the
* rest of the system. All functionality of the ReplicationCoordinatorImpl that would introduce
@@ -94,8 +68,8 @@ class ReplicationCoordinatorExternalState {
MONGO_DISALLOW_COPYING(ReplicationCoordinatorExternalState);
public:
- ReplicationCoordinatorExternalState();
- virtual ~ReplicationCoordinatorExternalState();
+ ReplicationCoordinatorExternalState() {}
+ virtual ~ReplicationCoordinatorExternalState() {}
/**
* Starts the journal listener, and snapshot threads
@@ -286,7 +260,7 @@ public:
*
* It is illegal to call with a newCommitPoint that does not name an existing snapshot.
*/
- virtual void updateCommittedSnapshot(SnapshotInfo newCommitPoint) = 0;
+ virtual void updateCommittedSnapshot(const OpTime& newCommitPoint) = 0;
/**
* Returns whether or not the SnapshotThread is active.
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 8c31e36eb8e..962e85280c2 100644
--- a/src/mongo/db/repl/replication_coordinator_external_state_impl.cpp
+++ b/src/mongo/db/repl/replication_coordinator_external_state_impl.cpp
@@ -808,13 +808,13 @@ void ReplicationCoordinatorExternalStateImpl::dropAllSnapshots() {
manager->dropAllSnapshots();
}
-void ReplicationCoordinatorExternalStateImpl::updateCommittedSnapshot(SnapshotInfo newCommitPoint) {
+void ReplicationCoordinatorExternalStateImpl::updateCommittedSnapshot(
+ const OpTime& newCommitPoint) {
auto manager = _service->getGlobalStorageEngine()->getSnapshotManager();
if (manager) {
- manager->setCommittedSnapshot(SnapshotName(newCommitPoint.opTime.getTimestamp()),
- newCommitPoint.opTime.getTimestamp());
+ manager->setCommittedSnapshot(newCommitPoint.getTimestamp());
}
- notifyOplogMetadataWaiters(newCommitPoint.opTime);
+ notifyOplogMetadataWaiters(newCommitPoint);
}
bool ReplicationCoordinatorExternalStateImpl::snapshotsEnabled() const {
diff --git a/src/mongo/db/repl/replication_coordinator_external_state_impl.h b/src/mongo/db/repl/replication_coordinator_external_state_impl.h
index f75b4fa7656..26ad9b9bebb 100644
--- a/src/mongo/db/repl/replication_coordinator_external_state_impl.h
+++ b/src/mongo/db/repl/replication_coordinator_external_state_impl.h
@@ -100,7 +100,7 @@ public:
virtual void stopProducer();
virtual void startProducerIfStopped();
void dropAllSnapshots() final;
- void updateCommittedSnapshot(SnapshotInfo newCommitPoint) final;
+ void updateCommittedSnapshot(const OpTime& newCommitPoint) final;
virtual bool snapshotsEnabled() const;
virtual void notifyOplogMetadataWaiters(const OpTime& committedOpTime);
virtual double getElectionTimeoutOffsetLimitFraction() const;
diff --git a/src/mongo/db/repl/replication_coordinator_external_state_mock.cpp b/src/mongo/db/repl/replication_coordinator_external_state_mock.cpp
index 9af8aec2341..f211bacf67f 100644
--- a/src/mongo/db/repl/replication_coordinator_external_state_mock.cpp
+++ b/src/mongo/db/repl/replication_coordinator_external_state_mock.cpp
@@ -37,7 +37,6 @@
#include "mongo/db/client.h"
#include "mongo/db/jsobj.h"
#include "mongo/db/repl/oplog_buffer_blocking_queue.h"
-#include "mongo/db/storage/snapshot_name.h"
#include "mongo/stdx/memory.h"
#include "mongo/util/log.h"
#include "mongo/util/net/hostandport.h"
@@ -218,8 +217,8 @@ void ReplicationCoordinatorExternalStateMock::startProducerIfStopped() {}
void ReplicationCoordinatorExternalStateMock::dropAllSnapshots() {}
-void ReplicationCoordinatorExternalStateMock::updateCommittedSnapshot(SnapshotInfo newCommitPoint) {
-}
+void ReplicationCoordinatorExternalStateMock::updateCommittedSnapshot(
+ const OpTime& newCommitPoint) {}
bool ReplicationCoordinatorExternalStateMock::snapshotsEnabled() const {
return _areSnapshotsEnabled;
diff --git a/src/mongo/db/repl/replication_coordinator_external_state_mock.h b/src/mongo/db/repl/replication_coordinator_external_state_mock.h
index a05186710e3..59ac2bfce33 100644
--- a/src/mongo/db/repl/replication_coordinator_external_state_mock.h
+++ b/src/mongo/db/repl/replication_coordinator_external_state_mock.h
@@ -88,7 +88,7 @@ public:
virtual void stopProducer();
virtual void startProducerIfStopped();
virtual void dropAllSnapshots();
- virtual void updateCommittedSnapshot(SnapshotInfo newCommitPoint);
+ virtual void updateCommittedSnapshot(const OpTime& newCommitPoint);
virtual bool snapshotsEnabled() const;
virtual void notifyOplogMetadataWaiters(const OpTime& committedOpTime);
virtual double getElectionTimeoutOffsetLimitFraction() const;
diff --git a/src/mongo/db/repl/replication_coordinator_impl.cpp b/src/mongo/db/repl/replication_coordinator_impl.cpp
index f50edc09f3c..93384d16026 100644
--- a/src/mongo/db/repl/replication_coordinator_impl.cpp
+++ b/src/mongo/db/repl/replication_coordinator_impl.cpp
@@ -431,7 +431,7 @@ OpTime ReplicationCoordinatorImpl::getCurrentCommittedSnapshotOpTime() const {
OpTime ReplicationCoordinatorImpl::_getCurrentCommittedSnapshotOpTime_inlock() const {
if (_currentCommittedSnapshot) {
- return _currentCommittedSnapshot->opTime;
+ return _currentCommittedSnapshot.get();
}
return OpTime();
}
@@ -1472,7 +1472,7 @@ Status ReplicationCoordinatorImpl::_setLastOptime_inlock(const UpdatePositionArg
}
bool ReplicationCoordinatorImpl::_doneWaitingForReplication_inlock(
- const OpTime& opTime, SnapshotName minSnapshot, const WriteConcernOptions& writeConcern) {
+ const OpTime& opTime, Timestamp minSnapshot, const WriteConcernOptions& writeConcern) {
// The syncMode cannot be unset.
invariant(writeConcern.syncMode != WriteConcernOptions::SyncMode::UNSET);
Status status = _checkIfWriteConcernCanBeSatisfied_inlock(writeConcern);
@@ -1496,9 +1496,8 @@ bool ReplicationCoordinatorImpl::_doneWaitingForReplication_inlock(
}
// Wait for the "current" snapshot to advance to/past the opTime.
- const auto haveSnapshot =
- (_currentCommittedSnapshot->opTime >= opTime &&
- _currentCommittedSnapshot->opTime.getTimestamp().asULL() >= minSnapshot.asU64());
+ const auto haveSnapshot = (_currentCommittedSnapshot >= opTime &&
+ _currentCommittedSnapshot->getTimestamp() >= minSnapshot);
if (!haveSnapshot) {
LOG(1) << "Required snapshot optime: " << opTime << " is not yet part of the "
<< "current 'committed' snapshot: " << *_currentCommittedSnapshot;
@@ -1526,8 +1525,7 @@ ReplicationCoordinator::StatusAndDuration ReplicationCoordinatorImpl::awaitRepli
Timer timer;
WriteConcernOptions fixedWriteConcern = populateUnsetWriteConcernOptionsSyncMode(writeConcern);
stdx::unique_lock<stdx::mutex> lock(_mutex);
- auto status =
- _awaitReplication_inlock(&lock, opCtx, opTime, SnapshotName::min(), fixedWriteConcern);
+ auto status = _awaitReplication_inlock(&lock, opCtx, opTime, Timestamp(), fixedWriteConcern);
return {std::move(status), duration_cast<Milliseconds>(timer.elapsed())};
}
@@ -1547,7 +1545,7 @@ Status ReplicationCoordinatorImpl::_awaitReplication_inlock(
stdx::unique_lock<stdx::mutex>* lock,
OperationContext* opCtx,
const OpTime& opTime,
- SnapshotName minSnapshot,
+ Timestamp minSnapshot,
const WriteConcernOptions& writeConcern) {
// We should never wait for replication if we are holding any locks, because this can
@@ -1568,7 +1566,7 @@ Status ReplicationCoordinatorImpl::_awaitReplication_inlock(
return Status::OK();
}
- if (opTime.isNull() && minSnapshot == SnapshotName::min()) {
+ if (opTime.isNull() && minSnapshot == Timestamp()) {
// If waiting for the empty optime, always say it's been replicated.
return Status::OK();
}
@@ -2062,8 +2060,8 @@ void ReplicationCoordinatorImpl::fillIsMasterForReplSet(IsMasterResponse* respon
OpTime lastOpTime = _getMyLastAppliedOpTime_inlock();
response->setLastWrite(lastOpTime, lastOpTime.getTimestamp().getSecs());
if (_currentCommittedSnapshot) {
- OpTime majorityOpTime = _currentCommittedSnapshot->opTime;
- response->setLastMajorityWrite(majorityOpTime, majorityOpTime.getTimestamp().getSecs());
+ response->setLastMajorityWrite(_currentCommittedSnapshot.get(),
+ _currentCommittedSnapshot->getTimestamp().getSecs());
}
if (response->isMaster() && !_canAcceptNonLocalWrites) {
@@ -2483,8 +2481,7 @@ Status ReplicationCoordinatorImpl::processReplSetInitiate(OperationContext* opCt
// Sets the initial data timestamp on the storage engine so it can assign a timestamp
// to data on disk. We do this after writing the "initiating set" oplog entry.
- auto initialDataTS = SnapshotName(lastAppliedOpTime.getTimestamp().asULL());
- _storage->setInitialDataTimestamp(getServiceContext(), initialDataTS);
+ _storage->setInitialDataTimestamp(getServiceContext(), lastAppliedOpTime.getTimestamp());
_finishReplSetInitiate(opCtx, newConfig, myIndex.getValue());
@@ -2931,7 +2928,7 @@ ReplicationCoordinatorImpl::_setCurrentRSConfig_inlock(OperationContext* opCtx,
void ReplicationCoordinatorImpl::_wakeReadyWaiters_inlock() {
_replicationWaiterList.signalAndRemoveIf_inlock([this](Waiter* waiter) {
return _doneWaitingForReplication_inlock(
- waiter->opTime, SnapshotName::min(), *waiter->writeConcern);
+ waiter->opTime, Timestamp(), *waiter->writeConcern);
});
}
@@ -3238,11 +3235,9 @@ void ReplicationCoordinatorImpl::_setStableTimestampForStorage_inlock() {
if (!testingSnapshotBehaviorInIsolation) {
// Update committed snapshot and wake up any threads waiting on read concern or
// write concern.
- _updateCommittedSnapshot_inlock(SnapshotInfo{stableOpTime.get(), SnapshotName::min()});
-
+ _updateCommittedSnapshot_inlock(stableOpTime.get());
// Update the stable timestamp for the storage engine.
- auto stableTimestamp = SnapshotName(stableOpTime->getTimestamp());
- _storage->setStableTimestamp(getServiceContext(), stableTimestamp);
+ _storage->setStableTimestamp(getServiceContext(), stableOpTime->getTimestamp());
}
_cleanupStableOpTimeCandidates(&_stableOpTimeCandidates, stableOpTime.get());
}
@@ -3495,21 +3490,20 @@ EventHandle ReplicationCoordinatorImpl::_updateTerm_inlock(
return EventHandle();
}
-SnapshotName ReplicationCoordinatorImpl::reserveSnapshotName(OperationContext* opCtx) {
- SnapshotName reservedName;
+Timestamp ReplicationCoordinatorImpl::reserveSnapshotName(OperationContext* opCtx) {
+ Timestamp reservedName;
if (getReplicationMode() == Mode::modeReplSet) {
invariant(opCtx->lockState()->isLocked());
if (getMemberState().primary()) {
// Use the current optime on the node, for primary nodes.
- reservedName = SnapshotName(
- LogicalClock::get(getServiceContext())->getClusterTime().asTimestamp());
+ reservedName = LogicalClock::get(getServiceContext())->getClusterTime().asTimestamp();
} else {
// Use lastApplied time, for secondary nodes.
- reservedName = SnapshotName(getMyLastAppliedOpTime().getTimestamp());
+ reservedName = getMyLastAppliedOpTime().getTimestamp();
}
} else {
// All snapshots are the same for a standalone node.
- reservedName = SnapshotName(0);
+ reservedName = Timestamp();
}
// This was just in case the snapshot name was different from the lastOp in the client.
ReplClientInfo::forClient(opCtx->getClient()).setLastSnapshot(reservedName);
@@ -3517,11 +3511,11 @@ SnapshotName ReplicationCoordinatorImpl::reserveSnapshotName(OperationContext* o
}
void ReplicationCoordinatorImpl::waitUntilSnapshotCommitted(OperationContext* opCtx,
- const SnapshotName& untilSnapshot) {
+ const Timestamp& untilSnapshot) {
stdx::unique_lock<stdx::mutex> lock(_mutex);
while (!_currentCommittedSnapshot ||
- _currentCommittedSnapshot->opTime.getTimestamp().asULL() < untilSnapshot.asU64()) {
+ _currentCommittedSnapshot->getTimestamp() < untilSnapshot) {
opCtx->waitForConditionOrInterrupt(_currentCommittedSnapshotCond, lock);
}
}
@@ -3530,17 +3524,10 @@ size_t ReplicationCoordinatorImpl::getNumUncommittedSnapshots() {
return _uncommittedSnapshotsSize.load();
}
-void ReplicationCoordinatorImpl::createSnapshot(OperationContext* opCtx,
- OpTime timeOfSnapshot,
- SnapshotName name) {
- // SERVER-31304: Delete this function.
- return;
-}
-
MONGO_FP_DECLARE(disableSnapshotting);
void ReplicationCoordinatorImpl::_updateCommittedSnapshot_inlock(
- SnapshotInfo newCommittedSnapshot) {
+ const OpTime& newCommittedSnapshot) {
if (testingSnapshotBehaviorInIsolation) {
return;
}
@@ -3551,11 +3538,11 @@ void ReplicationCoordinatorImpl::_updateCommittedSnapshot_inlock(
log() << "not updating committed snapshot because we are in rollback";
return;
}
- invariant(!newCommittedSnapshot.opTime.isNull());
- invariant(newCommittedSnapshot.opTime.getTimestamp() <=
+ invariant(!newCommittedSnapshot.isNull());
+ invariant(newCommittedSnapshot.getTimestamp() <=
_topCoord->getLastCommittedOpTime().getTimestamp());
if (_currentCommittedSnapshot) {
- invariant(newCommittedSnapshot.opTime >= _currentCommittedSnapshot->opTime);
+ invariant(newCommittedSnapshot >= _currentCommittedSnapshot);
}
if (MONGO_FAIL_POINT(disableSnapshotting))
return;
diff --git a/src/mongo/db/repl/replication_coordinator_impl.h b/src/mongo/db/repl/replication_coordinator_impl.h
index ff6053ce5a7..129dee88cf1 100644
--- a/src/mongo/db/repl/replication_coordinator_impl.h
+++ b/src/mongo/db/repl/replication_coordinator_impl.h
@@ -45,7 +45,6 @@
#include "mongo/db/repl/sync_source_resolver.h"
#include "mongo/db/repl/topology_coordinator.h"
#include "mongo/db/repl/update_position_args.h"
-#include "mongo/db/storage/snapshot_name.h"
#include "mongo/executor/task_executor.h"
#include "mongo/platform/atomic_word.h"
#include "mongo/platform/random.h"
@@ -305,16 +304,12 @@ public:
virtual Status updateTerm(OperationContext* opCtx, long long term) override;
- virtual SnapshotName reserveSnapshotName(OperationContext* opCtx) override;
-
- virtual void createSnapshot(OperationContext* opCtx,
- OpTime timeOfSnapshot,
- SnapshotName name) override;
+ virtual Timestamp reserveSnapshotName(OperationContext* opCtx) override;
virtual OpTime getCurrentCommittedSnapshotOpTime() const override;
virtual void waitUntilSnapshotCommitted(OperationContext* opCtx,
- const SnapshotName& untilSnapshot) override;
+ const Timestamp& untilSnapshot) override;
virtual void appendDiagnosticBSON(BSONObjBuilder*) override;
@@ -625,7 +620,7 @@ private:
Status _awaitReplication_inlock(stdx::unique_lock<stdx::mutex>* lock,
OperationContext* opCtx,
const OpTime& opTime,
- SnapshotName minSnapshot,
+ Timestamp minSnapshot,
const WriteConcernOptions& writeConcern);
/**
@@ -635,7 +630,7 @@ private:
* minSnapshot.
*/
bool _doneWaitingForReplication_inlock(const OpTime& opTime,
- SnapshotName minSnapshot,
+ Timestamp minSnapshot,
const WriteConcernOptions& writeConcern);
Status _checkIfWriteConcernCanBeSatisfied_inlock(const WriteConcernOptions& writeConcern) const;
@@ -998,7 +993,7 @@ private:
/**
* Blesses a snapshot to be used for new committed reads.
*/
- void _updateCommittedSnapshot_inlock(SnapshotInfo newCommittedSnapshot);
+ void _updateCommittedSnapshot_inlock(const OpTime& newCommittedSnapshot);
/**
* Calculates the 'stable' replication optime given a set of optime candidates and the
@@ -1280,7 +1275,7 @@ private:
// The OpTimes and SnapshotNames for all snapshots newer than the current commit point, kept in
// sorted order. Any time this is changed, you must also update _uncommitedSnapshotsSize.
- std::deque<SnapshotInfo> _uncommittedSnapshots; // (M)
+ std::deque<OpTime> _uncommittedSnapshots; // (M)
// A cache of the size of _uncommittedSnaphots that can be read without any locking.
// May only be written to while holding _mutex.
@@ -1289,7 +1284,7 @@ private:
// The non-null OpTime and SnapshotName of the current snapshot used for committed reads, if
// there is one.
// When engaged, this must be <= _lastCommittedOpTime and < _uncommittedSnapshots.front().
- boost::optional<SnapshotInfo> _currentCommittedSnapshot; // (M)
+ boost::optional<OpTime> _currentCommittedSnapshot; // (M)
// A set of optimes that are used for computing the replication system's current 'stable'
// optime. Every time a node's applied optime is updated, it will be added to this set.
diff --git a/src/mongo/db/repl/replication_coordinator_impl_test.cpp b/src/mongo/db/repl/replication_coordinator_impl_test.cpp
index 5b8ce8441a5..e5d0f2ee2ac 100644
--- a/src/mongo/db/repl/replication_coordinator_impl_test.cpp
+++ b/src/mongo/db/repl/replication_coordinator_impl_test.cpp
@@ -393,8 +393,7 @@ TEST_F(ReplCoordTest, InitiateSucceedsWhenQuorumCheckPasses) {
ASSERT_OK(status);
ASSERT_EQUALS(ReplicationCoordinator::modeReplSet, getReplCoord()->getReplicationMode());
- ASSERT_EQUALS(getStorageInterface()->getInitialDataTimestamp().asU64(),
- SnapshotName(appliedTS.asULL()).asU64());
+ ASSERT_EQUALS(getStorageInterface()->getInitialDataTimestamp(), appliedTS);
}
TEST_F(ReplCoordTest,
@@ -3515,7 +3514,6 @@ TEST_F(ReplCoordTest,
getReplCoord()->setMyLastAppliedOpTime(time);
getReplCoord()->setMyLastDurableOpTime(time);
- getReplCoord()->createSnapshot(opCtx.get(), time, SnapshotName(1));
ASSERT_OK(getReplCoord()->setLastAppliedOptime_forTest(2, 1, time));
@@ -3873,17 +3871,17 @@ TEST_F(StableOpTimeTest, SetMyLastAppliedSetsStableOpTimeForStorage) {
// There should be no stable optime candidates until setMyLastAppliedOpTime is called.
repl->advanceCommitPoint(OpTime({1, 2}, 0));
- ASSERT_EQUALS(SnapshotName::min(), getStorageInterface()->getStableTimestamp());
+ ASSERT_EQUALS(Timestamp::min(), getStorageInterface()->getStableTimestamp());
// Check that the stable timestamp is updated for the storage engine when we set the applied
// optime.
repl->setMyLastAppliedOpTime(OpTime({1, 1}, 0));
- stableTimestamp = Timestamp(getStorageInterface()->getStableTimestamp().asU64());
+ stableTimestamp = getStorageInterface()->getStableTimestamp();
ASSERT_EQUALS(Timestamp(1, 1), stableTimestamp);
// Check that timestamp cleanup occurs.
repl->setMyLastAppliedOpTime(OpTime({1, 2}, 0));
- stableTimestamp = Timestamp(getStorageInterface()->getStableTimestamp().asU64());
+ stableTimestamp = getStorageInterface()->getStableTimestamp();
ASSERT_EQUALS(Timestamp(1, 2), stableTimestamp);
auto opTimeCandidates = repl->getStableOpTimeCandidates_forTest();
@@ -3910,13 +3908,12 @@ TEST_F(StableOpTimeTest, AdvanceCommitPointSetsStableOpTimeForStorage) {
// Set a commit point and check the stable optime.
repl->advanceCommitPoint(OpTime({1, 1}, term));
- stableTimestamp = Timestamp(getStorageInterface()->getStableTimestamp().asU64());
+ stableTimestamp = getStorageInterface()->getStableTimestamp();
ASSERT_EQUALS(Timestamp(1, 1), stableTimestamp);
- // Check that the stable timestamp is updated for the storage engine when we advance the
- // commit point.
+ // Check that the stable timestamp is updated when we advance the commit point.
repl->advanceCommitPoint(OpTime({1, 2}, term));
- stableTimestamp = Timestamp(getStorageInterface()->getStableTimestamp().asU64());
+ stableTimestamp = getStorageInterface()->getStableTimestamp();
ASSERT_EQUALS(Timestamp(1, 2), stableTimestamp);
// Check that timestamp candidate cleanup occurs.
@@ -4144,7 +4141,6 @@ TEST_F(ReplCoordTest, ReadAfterCommittedGreaterOpTime) {
getReplCoord()->setMyLastAppliedOpTime(OpTime(Timestamp(100, 0), 1));
getReplCoord()->setMyLastDurableOpTime(OpTime(Timestamp(100, 0), 1));
- getReplCoord()->createSnapshot(opCtx.get(), OpTime(Timestamp(100, 0), 1), SnapshotName(1));
ASSERT_OK(getReplCoord()->waitUntilOpTimeForRead(
opCtx.get(),
@@ -4168,7 +4164,6 @@ TEST_F(ReplCoordTest, ReadAfterCommittedEqualOpTime) {
OpTime time(Timestamp(100, 0), 1);
getReplCoord()->setMyLastAppliedOpTime(time);
getReplCoord()->setMyLastDurableOpTime(time);
- getReplCoord()->createSnapshot(opCtx.get(), time, SnapshotName(1));
ASSERT_OK(getReplCoord()->waitUntilOpTimeForRead(
opCtx.get(), ReadConcernArgs(time, ReadConcernLevel::kMajorityReadConcern)));
@@ -4195,7 +4190,6 @@ TEST_F(ReplCoordTest, ReadAfterCommittedDeferredGreaterOpTime) {
// Not guaranteed to be scheduled after waitUntil blocks...
getReplCoord()->setMyLastAppliedOpTime(committedOpTime);
getReplCoord()->setMyLastDurableOpTime(committedOpTime);
- getReplCoord()->createSnapshot(nullptr, committedOpTime, SnapshotName(1));
});
ASSERT_OK(getReplCoord()->waitUntilOpTimeForRead(
@@ -4225,7 +4219,6 @@ TEST_F(ReplCoordTest, ReadAfterCommittedDeferredEqualOpTime) {
// Not guaranteed to be scheduled after waitUntil blocks...
getReplCoord()->setMyLastAppliedOpTime(opTimeToWait);
getReplCoord()->setMyLastDurableOpTime(opTimeToWait);
- getReplCoord()->createSnapshot(nullptr, opTimeToWait, SnapshotName(1));
});
ASSERT_OK(getReplCoord()->waitUntilOpTimeForRead(
@@ -5223,7 +5216,6 @@ TEST_F(ReplCoordTest, NewStyleUpdatePositionCmdHasMetadata) {
rpc::ReplSetMetadata syncSourceMetadata(optime.getTerm(), optime, optime, 1, OID(), -1, 1);
getReplCoord()->processReplSetMetadata(syncSourceMetadata);
getReplCoord()->advanceCommitPoint(optime);
- getReplCoord()->createSnapshot(opCtx.get(), optime, SnapshotName(1));
BSONObj cmd = unittest::assertGet(getReplCoord()->prepareReplSetUpdatePositionCommand(
ReplicationCoordinator::ReplSetUpdatePositionCommandStyle::kNewStyle));
diff --git a/src/mongo/db/repl/replication_coordinator_mock.cpp b/src/mongo/db/repl/replication_coordinator_mock.cpp
index 58a3efd664d..79c666ef330 100644
--- a/src/mongo/db/repl/replication_coordinator_mock.cpp
+++ b/src/mongo/db/repl/replication_coordinator_mock.cpp
@@ -31,11 +31,11 @@
#include "mongo/db/repl/replication_coordinator_mock.h"
#include "mongo/base/status.h"
+#include "mongo/bson/timestamp.h"
#include "mongo/db/namespace_string.h"
#include "mongo/db/repl/is_master_response.h"
#include "mongo/db/repl/read_concern_args.h"
#include "mongo/db/repl/sync_source_resolver.h"
-#include "mongo/db/storage/snapshot_name.h"
#include "mongo/db/write_concern_options.h"
#include "mongo/util/assert_util.h"
@@ -462,14 +462,10 @@ Status ReplicationCoordinatorMock::updateTerm(OperationContext* opCtx, long long
return Status::OK();
}
-SnapshotName ReplicationCoordinatorMock::reserveSnapshotName(OperationContext* opCtx) {
- return SnapshotName(_snapshotNameGenerator.addAndFetch(1));
+Timestamp ReplicationCoordinatorMock::reserveSnapshotName(OperationContext* opCtx) {
+ return Timestamp(_snapshotNameGenerator.addAndFetch(1));
}
-void ReplicationCoordinatorMock::createSnapshot(OperationContext* opCtx,
- OpTime timeOfSnapshot,
- SnapshotName name){};
-
void ReplicationCoordinatorMock::dropAllSnapshots() {}
OpTime ReplicationCoordinatorMock::getCurrentCommittedSnapshotOpTime() const {
@@ -477,7 +473,7 @@ OpTime ReplicationCoordinatorMock::getCurrentCommittedSnapshotOpTime() const {
}
void ReplicationCoordinatorMock::waitUntilSnapshotCommitted(OperationContext* opCtx,
- const SnapshotName& untilSnapshot) {}
+ const Timestamp& untilSnapshot) {}
size_t ReplicationCoordinatorMock::getNumUncommittedSnapshots() {
return 0;
diff --git a/src/mongo/db/repl/replication_coordinator_mock.h b/src/mongo/db/repl/replication_coordinator_mock.h
index 12bf54c1f6c..91c24379a71 100644
--- a/src/mongo/db/repl/replication_coordinator_mock.h
+++ b/src/mongo/db/repl/replication_coordinator_mock.h
@@ -250,18 +250,14 @@ public:
virtual Status updateTerm(OperationContext* opCtx, long long term);
- virtual SnapshotName reserveSnapshotName(OperationContext* opCtx);
-
- virtual void createSnapshot(OperationContext* opCtx,
- OpTime timeOfSnapshot,
- SnapshotName name) override;
+ virtual Timestamp reserveSnapshotName(OperationContext* opCtx);
virtual void dropAllSnapshots() override;
virtual OpTime getCurrentCommittedSnapshotOpTime() const override;
virtual void waitUntilSnapshotCommitted(OperationContext* opCtx,
- const SnapshotName& untilSnapshot) override;
+ const Timestamp& untilSnapshot) override;
virtual size_t getNumUncommittedSnapshots() override;
diff --git a/src/mongo/db/repl/replication_process_test.cpp b/src/mongo/db/repl/replication_process_test.cpp
index 99b3d22e287..0203ba5b527 100644
--- a/src/mongo/db/repl/replication_process_test.cpp
+++ b/src/mongo/db/repl/replication_process_test.cpp
@@ -113,7 +113,7 @@ TEST_F(ReplicationProcessTest,
ReplicationProcess::kRollbackProgressNamespace,
TimestampedBSONObj{BSON("_id"
<< "not progress"),
- SnapshotName()},
+ Timestamp()},
OpTime::kUninitializedTerm));
ASSERT_EQUALS(ErrorCodes::NoSuchKey, replicationProcess.getRollbackProgress(opCtx.get()));
}
@@ -123,7 +123,7 @@ TEST_F(ReplicationProcessTest, GetRollbackProgressReturnsBadStatusIfApplyUntilFi
<< "rollbackProgress"
<< "applyUntil"
<< "not op time!"),
- SnapshotName(0)};
+ Timestamp(0)};
auto opCtx = makeOpCtx();
ASSERT_OK(_storageInterface->createCollection(
opCtx.get(), ReplicationProcess::kRollbackProgressNamespace, {}));
@@ -148,7 +148,7 @@ TEST_F(ReplicationProcessTest,
<< "not_timestamp"
<< "t"
<< 1LL)),
- SnapshotName(0)};
+ Timestamp(0)};
auto opCtx = makeOpCtx();
ASSERT_OK(_storageInterface->createCollection(
opCtx.get(), ReplicationProcess::kRollbackProgressNamespace, {}));
@@ -171,7 +171,7 @@ TEST_F(ReplicationProcessTest,
<< "rollbackProgress"
<< "applyUntil"
<< applyUntil),
- SnapshotName(0)};
+ Timestamp(0)};
auto opCtx = makeOpCtx();
ASSERT_OK(_storageInterface->createCollection(
opCtx.get(), ReplicationProcess::kRollbackProgressNamespace, {}));
diff --git a/src/mongo/db/repl/replication_recovery.cpp b/src/mongo/db/repl/replication_recovery.cpp
index 6b416de6e3e..d01ac004a69 100644
--- a/src/mongo/db/repl/replication_recovery.cpp
+++ b/src/mongo/db/repl/replication_recovery.cpp
@@ -127,8 +127,7 @@ void ReplicationRecoveryImpl::recoverFromOplog(OperationContext* opCtx) try {
// If we have a checkpoint timestamp, we set the initial data timestamp now so that
// the operations we apply below can be given the proper timestamps.
- _storageInterface->setInitialDataTimestamp(opCtx->getServiceContext(),
- SnapshotName(checkpointTimestamp));
+ _storageInterface->setInitialDataTimestamp(opCtx->getServiceContext(), checkpointTimestamp);
}
// Oplog is empty. There are no oplog entries to apply, so we exit recovery. If there was a
@@ -151,7 +150,7 @@ void ReplicationRecoveryImpl::recoverFromOplog(OperationContext* opCtx) try {
// of the oplog.
if (checkpointTimestamp.isNull()) {
_storageInterface->setInitialDataTimestamp(opCtx->getServiceContext(),
- SnapshotName(topOfOplog->getTimestamp()));
+ topOfOplog->getTimestamp());
}
} catch (...) {
diff --git a/src/mongo/db/repl/replication_recovery_test.cpp b/src/mongo/db/repl/replication_recovery_test.cpp
index 65468b1be59..37f075ff917 100644
--- a/src/mongo/db/repl/replication_recovery_test.cpp
+++ b/src/mongo/db/repl/replication_recovery_test.cpp
@@ -56,13 +56,13 @@ class StorageInterfaceRecovery : public StorageInterfaceImpl {
public:
using OnSetInitialDataTimestampFn = stdx::function<void()>;
- void setInitialDataTimestamp(ServiceContext* serviceCtx, SnapshotName snapshotName) override {
+ void setInitialDataTimestamp(ServiceContext* serviceCtx, Timestamp snapshotName) override {
stdx::lock_guard<stdx::mutex> lock(_mutex);
_initialDataTimestamp = snapshotName;
_onSetInitialDataTimestampFn();
}
- SnapshotName getInitialDataTimestamp() const {
+ Timestamp getInitialDataTimestamp() const {
stdx::lock_guard<stdx::mutex> lock(_mutex);
return _initialDataTimestamp;
}
@@ -74,7 +74,7 @@ public:
private:
mutable stdx::mutex _mutex;
- SnapshotName _initialDataTimestamp = SnapshotName::min();
+ Timestamp _initialDataTimestamp = Timestamp::min();
OnSetInitialDataTimestampFn _onSetInitialDataTimestampFn = []() {};
};
@@ -142,7 +142,7 @@ TimestampedBSONObj _makeOplogEntry(int t) {
<< "i"
<< "o"
<< _makeInsertDocument(t)),
- SnapshotName(t)};
+ Timestamp(t)};
}
/**
@@ -218,7 +218,7 @@ TEST_F(ReplicationRecoveryTest, RecoveryWithNoOplogSucceeds) {
_assertDocsInOplog(opCtx, {});
_assertDocsInTestCollection(opCtx, {});
- ASSERT_EQ(getStorageInterfaceRecovery()->getInitialDataTimestamp(), SnapshotName::min());
+ ASSERT_EQ(getStorageInterfaceRecovery()->getInitialDataTimestamp(), Timestamp::min());
}
TEST_F(ReplicationRecoveryTest, RecoveryWithEmptyOplogSucceeds) {
@@ -231,7 +231,7 @@ TEST_F(ReplicationRecoveryTest, RecoveryWithEmptyOplogSucceeds) {
_assertDocsInOplog(opCtx, {});
_assertDocsInTestCollection(opCtx, {});
- ASSERT_EQ(getStorageInterfaceRecovery()->getInitialDataTimestamp(), SnapshotName::min());
+ ASSERT_EQ(getStorageInterfaceRecovery()->getInitialDataTimestamp(), Timestamp::min());
}
DEATH_TEST_F(ReplicationRecoveryTest,
@@ -275,8 +275,7 @@ TEST_F(ReplicationRecoveryTest, RecoveryTruncatesOplogAtOplogTruncateAfterPoint)
_assertDocsInTestCollection(opCtx, {});
ASSERT_EQ(getConsistencyMarkers()->getOplogTruncateAfterPoint(opCtx), Timestamp());
ASSERT_EQ(getConsistencyMarkers()->getAppliedThrough(opCtx), OpTime(Timestamp(3, 3), 1));
- ASSERT_EQ(getStorageInterfaceRecovery()->getInitialDataTimestamp(),
- SnapshotName(Timestamp(3, 3)));
+ ASSERT_EQ(getStorageInterfaceRecovery()->getInitialDataTimestamp(), Timestamp(3, 3));
}
TEST_F(ReplicationRecoveryTest, RecoverySkipsEverythingIfInitialSyncFlagIsSet) {
@@ -294,7 +293,7 @@ TEST_F(ReplicationRecoveryTest, RecoverySkipsEverythingIfInitialSyncFlagIsSet) {
_assertDocsInTestCollection(opCtx, {});
ASSERT_EQ(getConsistencyMarkers()->getOplogTruncateAfterPoint(opCtx), Timestamp(4, 4));
ASSERT_EQ(getConsistencyMarkers()->getAppliedThrough(opCtx), OpTime(Timestamp(1, 1), 1));
- ASSERT_EQ(getStorageInterfaceRecovery()->getInitialDataTimestamp(), SnapshotName::min());
+ ASSERT_EQ(getStorageInterfaceRecovery()->getInitialDataTimestamp(), Timestamp::min());
}
TEST_F(ReplicationRecoveryTest, RecoveryAppliesDocumentsWhenAppliedThroughIsBehind) {
@@ -310,8 +309,7 @@ TEST_F(ReplicationRecoveryTest, RecoveryAppliesDocumentsWhenAppliedThroughIsBehi
_assertDocsInTestCollection(opCtx, {4, 5});
ASSERT_EQ(getConsistencyMarkers()->getOplogTruncateAfterPoint(opCtx), Timestamp());
ASSERT_EQ(getConsistencyMarkers()->getAppliedThrough(opCtx), OpTime(Timestamp(5, 5), 1));
- ASSERT_EQ(getStorageInterfaceRecovery()->getInitialDataTimestamp(),
- SnapshotName(Timestamp(5, 5)));
+ ASSERT_EQ(getStorageInterfaceRecovery()->getInitialDataTimestamp(), Timestamp(5, 5));
}
TEST_F(ReplicationRecoveryTest, RecoveryAppliesDocumentsWhenAppliedThroughIsBehindAfterTruncation) {
@@ -328,8 +326,7 @@ TEST_F(ReplicationRecoveryTest, RecoveryAppliesDocumentsWhenAppliedThroughIsBehi
_assertDocsInTestCollection(opCtx, {2, 3});
ASSERT_EQ(getConsistencyMarkers()->getOplogTruncateAfterPoint(opCtx), Timestamp());
ASSERT_EQ(getConsistencyMarkers()->getAppliedThrough(opCtx), OpTime(Timestamp(3, 3), 1));
- ASSERT_EQ(getStorageInterfaceRecovery()->getInitialDataTimestamp(),
- SnapshotName(Timestamp(3, 3)));
+ ASSERT_EQ(getStorageInterfaceRecovery()->getInitialDataTimestamp(), Timestamp(3, 3));
}
TEST_F(ReplicationRecoveryTest, RecoveryAppliesDocumentsWhenCheckpointTimestampIsBehind) {
@@ -345,8 +342,7 @@ TEST_F(ReplicationRecoveryTest, RecoveryAppliesDocumentsWhenCheckpointTimestampI
_assertDocsInTestCollection(opCtx, {4, 5});
ASSERT_EQ(getConsistencyMarkers()->getOplogTruncateAfterPoint(opCtx), Timestamp());
ASSERT_EQ(getConsistencyMarkers()->getAppliedThrough(opCtx), OpTime(Timestamp(5, 5), 1));
- ASSERT_EQ(getStorageInterfaceRecovery()->getInitialDataTimestamp(),
- SnapshotName(Timestamp(3, 3)));
+ ASSERT_EQ(getStorageInterfaceRecovery()->getInitialDataTimestamp(), Timestamp(3, 3));
}
TEST_F(ReplicationRecoveryTest,
@@ -364,8 +360,7 @@ TEST_F(ReplicationRecoveryTest,
_assertDocsInTestCollection(opCtx, {2, 3});
ASSERT_EQ(getConsistencyMarkers()->getOplogTruncateAfterPoint(opCtx), Timestamp());
ASSERT_EQ(getConsistencyMarkers()->getAppliedThrough(opCtx), OpTime(Timestamp(3, 3), 1));
- ASSERT_EQ(getStorageInterfaceRecovery()->getInitialDataTimestamp(),
- SnapshotName(Timestamp(1, 1)));
+ ASSERT_EQ(getStorageInterfaceRecovery()->getInitialDataTimestamp(), Timestamp(1, 1));
}
DEATH_TEST_F(ReplicationRecoveryTest, AppliedThroughBehindOplogFasserts, "Fatal Assertion 40292") {
@@ -419,8 +414,7 @@ TEST_F(ReplicationRecoveryTest, RecoverySetsInitialDataTimestampToCheckpointTime
_assertDocsInTestCollection(opCtx, {5, 6});
ASSERT(getConsistencyMarkers()->getOplogTruncateAfterPoint(opCtx).isNull());
ASSERT_EQ(getConsistencyMarkers()->getAppliedThrough(opCtx), OpTime(Timestamp(6, 6), 6));
- ASSERT_EQ(getStorageInterfaceRecovery()->getInitialDataTimestamp(),
- SnapshotName(Timestamp(4, 4)));
+ ASSERT_EQ(getStorageInterfaceRecovery()->getInitialDataTimestamp(), Timestamp(4, 4));
}
TEST_F(ReplicationRecoveryTest,
@@ -437,8 +431,7 @@ TEST_F(ReplicationRecoveryTest,
ASSERT(getConsistencyMarkers()->getOplogTruncateAfterPoint(opCtx).isNull());
ASSERT(getConsistencyMarkers()->getAppliedThrough(opCtx).isNull());
ASSERT(getConsistencyMarkers()->getCheckpointTimestamp(opCtx).isNull());
- ASSERT_EQ(getStorageInterfaceRecovery()->getInitialDataTimestamp(),
- SnapshotName(Timestamp(5, 5)));
+ ASSERT_EQ(getStorageInterfaceRecovery()->getInitialDataTimestamp(), Timestamp(5, 5));
}
TEST_F(ReplicationRecoveryTest,
@@ -461,8 +454,7 @@ TEST_F(ReplicationRecoveryTest,
ASSERT(getConsistencyMarkers()->getOplogTruncateAfterPoint(opCtx).isNull());
ASSERT_EQ(getConsistencyMarkers()->getAppliedThrough(opCtx), OpTime(Timestamp(6, 6), 6));
ASSERT(getConsistencyMarkers()->getCheckpointTimestamp(opCtx).isNull());
- ASSERT_EQ(getStorageInterfaceRecovery()->getInitialDataTimestamp(),
- SnapshotName(Timestamp(6, 6)));
+ ASSERT_EQ(getStorageInterfaceRecovery()->getInitialDataTimestamp(), Timestamp(6, 6));
}
TEST_F(ReplicationRecoveryTest,
@@ -479,7 +471,7 @@ TEST_F(ReplicationRecoveryTest,
ASSERT(getConsistencyMarkers()->getOplogTruncateAfterPoint(opCtx).isNull());
ASSERT(getConsistencyMarkers()->getAppliedThrough(opCtx).isNull());
ASSERT(getConsistencyMarkers()->getCheckpointTimestamp(opCtx).isNull());
- ASSERT_EQ(getStorageInterfaceRecovery()->getInitialDataTimestamp(), SnapshotName::min());
+ ASSERT_EQ(getStorageInterfaceRecovery()->getInitialDataTimestamp(), Timestamp::min());
}
DEATH_TEST_F(ReplicationRecoveryTest,
diff --git a/src/mongo/db/repl/rollback_impl_test.cpp b/src/mongo/db/repl/rollback_impl_test.cpp
index 4b0b8640a53..8ac13bb02eb 100644
--- a/src/mongo/db/repl/rollback_impl_test.cpp
+++ b/src/mongo/db/repl/rollback_impl_test.cpp
@@ -47,9 +47,9 @@ NamespaceString nss("local.oplog.rs");
class StorageInterfaceRollback : public StorageInterfaceImpl {
public:
- void setStableTimestamp(ServiceContext* serviceCtx, SnapshotName snapshotName) override {
+ void setStableTimestamp(ServiceContext* serviceCtx, Timestamp snapshotName) override {
stdx::lock_guard<stdx::mutex> lock(_mutex);
- _stableTimestamp = Timestamp(snapshotName.asU64());
+ _stableTimestamp = snapshotName;
}
/**
@@ -284,7 +284,7 @@ TEST_F(RollbackImplTest, RollbackCallsRecoverToStableTimestamp) {
auto stableTimestamp = Timestamp(10, 0);
auto currTimestamp = Timestamp(20, 0);
- _storageInterface->setStableTimestamp(nullptr, SnapshotName(stableTimestamp));
+ _storageInterface->setStableTimestamp(nullptr, stableTimestamp);
_storageInterface->setCurrentTimestamp(currTimestamp);
// Check the current timestamp.
@@ -305,7 +305,7 @@ TEST_F(RollbackImplTest, RollbackReturnsBadStatusIfRecoverToStableTimestampFails
auto stableTimestamp = Timestamp(10, 0);
auto currTimestamp = Timestamp(20, 0);
- _storageInterface->setStableTimestamp(nullptr, SnapshotName(stableTimestamp));
+ _storageInterface->setStableTimestamp(nullptr, stableTimestamp);
_storageInterface->setCurrentTimestamp(currTimestamp);
// Make it so that the 'recoverToStableTimestamp' method will fail.
diff --git a/src/mongo/db/repl/storage_interface.h b/src/mongo/db/repl/storage_interface.h
index 29bc435cbc5..d6c50c28b26 100644
--- a/src/mongo/db/repl/storage_interface.h
+++ b/src/mongo/db/repl/storage_interface.h
@@ -36,13 +36,13 @@
#include "mongo/base/disallow_copying.h"
#include "mongo/base/string_data.h"
+#include "mongo/bson/timestamp.h"
#include "mongo/db/catalog/collection.h"
#include "mongo/db/namespace_string.h"
#include "mongo/db/query/index_bounds.h"
#include "mongo/db/repl/collection_bulk_loader.h"
#include "mongo/db/repl/optime.h"
#include "mongo/db/service_context.h"
-#include "mongo/db/storage/snapshot_name.h"
namespace mongo {
@@ -54,7 +54,7 @@ namespace repl {
struct TimestampedBSONObj {
BSONObj obj;
- SnapshotName timestamp;
+ Timestamp timestamp;
};
/**
@@ -318,13 +318,13 @@ public:
* Sets the highest timestamp at which the storage engine is allowed to take a checkpoint.
* This timestamp can never decrease, and thus should be a timestamp that can never roll back.
*/
- virtual void setStableTimestamp(ServiceContext* serviceCtx, SnapshotName snapshotName) = 0;
+ virtual void setStableTimestamp(ServiceContext* serviceCtx, Timestamp snapshotName) = 0;
/**
* Tells the storage engine the timestamp of the data at startup. This is necessary because
* timestamps are not persisted in the storage layer.
*/
- virtual void setInitialDataTimestamp(ServiceContext* serviceCtx, SnapshotName snapshotName) = 0;
+ virtual void setInitialDataTimestamp(ServiceContext* serviceCtx, Timestamp snapshotName) = 0;
/**
* Reverts the state of all database data to the last stable timestamp.
diff --git a/src/mongo/db/repl/storage_interface_impl.cpp b/src/mongo/db/repl/storage_interface_impl.cpp
index 4d3b59455fb..05f542ab3a8 100644
--- a/src/mongo/db/repl/storage_interface_impl.cpp
+++ b/src/mongo/db/repl/storage_interface_impl.cpp
@@ -127,7 +127,7 @@ Status StorageInterfaceImpl::initializeRollbackID(OperationContext* opCtx) {
BSONObjBuilder bob;
rbid.serialize(&bob);
- SnapshotName noTimestamp; // This write is not replicated.
+ Timestamp noTimestamp; // This write is not replicated.
return insertDocument(opCtx,
_rollbackIdNss,
TimestampedBSONObj{bob.done(), noTimestamp},
@@ -949,13 +949,12 @@ Status StorageInterfaceImpl::upgradeUUIDSchemaVersionNonReplicated(OperationCont
return updateUUIDSchemaVersionNonReplicated(opCtx, true);
}
-void StorageInterfaceImpl::setStableTimestamp(ServiceContext* serviceCtx,
- SnapshotName snapshotName) {
+void StorageInterfaceImpl::setStableTimestamp(ServiceContext* serviceCtx, Timestamp snapshotName) {
serviceCtx->getGlobalStorageEngine()->setStableTimestamp(snapshotName);
}
void StorageInterfaceImpl::setInitialDataTimestamp(ServiceContext* serviceCtx,
- SnapshotName snapshotName) {
+ Timestamp snapshotName) {
serviceCtx->getGlobalStorageEngine()->setInitialDataTimestamp(snapshotName);
}
diff --git a/src/mongo/db/repl/storage_interface_impl.h b/src/mongo/db/repl/storage_interface_impl.h
index ee4702530c9..404b42bb3b2 100644
--- a/src/mongo/db/repl/storage_interface_impl.h
+++ b/src/mongo/db/repl/storage_interface_impl.h
@@ -34,6 +34,7 @@
#include "mongo/base/status_with.h"
#include "mongo/base/string_data.h"
#include "mongo/bson/bsonobj.h"
+#include "mongo/bson/timestamp.h"
#include "mongo/db/catalog/index_create.h"
#include "mongo/db/namespace_string.h"
#include "mongo/db/repl/storage_interface.h"
@@ -147,9 +148,9 @@ public:
Status upgradeUUIDSchemaVersionNonReplicated(OperationContext* opCtx) override;
- void setStableTimestamp(ServiceContext* serviceCtx, SnapshotName snapshotName) override;
+ void setStableTimestamp(ServiceContext* serviceCtx, Timestamp snapshotName) override;
- void setInitialDataTimestamp(ServiceContext* serviceCtx, SnapshotName snapshotName) override;
+ void setInitialDataTimestamp(ServiceContext* serviceCtx, Timestamp snapshotName) override;
Status recoverToStableTimestamp(ServiceContext* serviceCtx) override;
diff --git a/src/mongo/db/repl/storage_interface_impl_test.cpp b/src/mongo/db/repl/storage_interface_impl_test.cpp
index a134a16642e..9b3e9fada8b 100644
--- a/src/mongo/db/repl/storage_interface_impl_test.cpp
+++ b/src/mongo/db/repl/storage_interface_impl_test.cpp
@@ -121,7 +121,7 @@ TimestampedBSONObj makeOplogEntry(OpTime opTime) {
bob.append("h", 1LL);
bob.append("op", "c");
bob.append("ns", "test.t");
- return {bob.obj(), SnapshotName(opTime.getTimestamp())};
+ return {bob.obj(), opTime.getTimestamp()};
}
/**
@@ -307,7 +307,7 @@ TEST_F(StorageInterfaceImplTest, IncrementRollbackIDRollsToZeroWhenExceedingMaxI
TimestampedBSONObj maxDoc = {BSON("_id" << StorageInterfaceImpl::kRollbackIdDocumentId
<< StorageInterfaceImpl::kRollbackIdFieldName
<< std::numeric_limits<int>::max()),
- SnapshotName(0)};
+ Timestamp::min()};
ASSERT_OK(storage.insertDocument(opCtx, nss, maxDoc, OpTime::kUninitializedTerm));
_assertRollbackIDDocument(opCtx, std::numeric_limits<int>::max());
@@ -336,7 +336,7 @@ TEST_F(StorageInterfaceImplTest, GetRollbackIDReturnsBadStatusIfDocumentHasBadFi
std::vector<TimestampedBSONObj> badDocs = {
{BSON("_id" << StorageInterfaceImpl::kRollbackIdDocumentId << "bad field" << 3),
- SnapshotName(0)}};
+ Timestamp::min()}};
ASSERT_OK(storage.insertDocuments(opCtx, nss, transformInserts(badDocs)));
ASSERT_EQUALS(ErrorCodes::duplicateCodeForTest(40415),
storage.getRollbackID(opCtx).getStatus());
@@ -353,7 +353,7 @@ TEST_F(StorageInterfaceImplTest, GetRollbackIDReturnsBadStatusIfRollbackIDIsNotI
TimestampedBSONObj{BSON("_id" << StorageInterfaceImpl::kRollbackIdDocumentId
<< StorageInterfaceImpl::kRollbackIdFieldName
<< "bad id"),
- SnapshotName(0)}};
+ Timestamp::min()}};
ASSERT_OK(storage.insertDocuments(opCtx, nss, transformInserts(badDoc)));
ASSERT_EQUALS(ErrorCodes::TypeMismatch, storage.getRollbackID(opCtx).getStatus());
}
@@ -461,7 +461,7 @@ TEST_F(StorageInterfaceImplTest, InsertMissingDocWorksOnExistingCappedCollection
opts.cappedSize = 1024 * 1024;
createCollection(opCtx, nss, opts);
ASSERT_OK(storage.insertDocument(
- opCtx, nss, {BSON("_id" << 1), SnapshotName(1)}, OpTime::kUninitializedTerm));
+ opCtx, nss, {BSON("_id" << 1), Timestamp(1)}, OpTime::kUninitializedTerm));
AutoGetCollectionForReadCommand autoColl(opCtx, nss);
ASSERT_TRUE(autoColl.getCollection());
}
@@ -472,7 +472,7 @@ TEST_F(StorageInterfaceImplTest, InsertMissingDocWorksOnExistingCollection) {
auto nss = makeNamespace(_agent);
createCollection(opCtx, nss);
ASSERT_OK(storage.insertDocument(
- opCtx, nss, {BSON("_id" << 1), SnapshotName(1)}, OpTime::kUninitializedTerm));
+ opCtx, nss, {BSON("_id" << 1), Timestamp(1)}, OpTime::kUninitializedTerm));
AutoGetCollectionForReadCommand autoColl(opCtx, nss);
ASSERT_TRUE(autoColl.getCollection());
}
@@ -482,7 +482,7 @@ TEST_F(StorageInterfaceImplTest, InsertMissingDocFailesIfCollectionIsMissing) {
StorageInterfaceImpl storage;
auto nss = makeNamespace(_agent);
const auto status = storage.insertDocument(
- opCtx, nss, {BSON("_id" << 1), SnapshotName(1)}, OpTime::kUninitializedTerm);
+ opCtx, nss, {BSON("_id" << 1), Timestamp(1)}, OpTime::kUninitializedTerm);
ASSERT_NOT_OK(status);
ASSERT_EQ(status.code(), ErrorCodes::NamespaceNotFound);
}
@@ -653,7 +653,7 @@ TEST_F(StorageInterfaceImplTest, DropCollectionWorksWithExistingWithDataCollecti
auto nss = makeNamespace(_agent);
createCollection(opCtx, nss);
ASSERT_OK(storage.insertDocument(
- opCtx, nss, {BSON("_id" << 1), SnapshotName(1)}, OpTime::kUninitializedTerm));
+ opCtx, nss, {BSON("_id" << 1), Timestamp(1)}, OpTime::kUninitializedTerm));
ASSERT_OK(storage.dropCollection(opCtx, nss));
}
@@ -926,11 +926,11 @@ TEST_F(StorageInterfaceImplTest,
ASSERT_OK(storage.insertDocuments(
opCtx,
nss,
- transformInserts(std::vector<TimestampedBSONObj>{{BSON("_id" << 0), SnapshotName(0)},
- {BSON("_id" << 1), SnapshotName(1)},
- {BSON("_id" << 2), SnapshotName(2)},
- {BSON("_id" << 3), SnapshotName(3)},
- {BSON("_id" << 4), SnapshotName(4)}})));
+ transformInserts(std::vector<TimestampedBSONObj>{{BSON("_id" << 0), Timestamp(0)},
+ {BSON("_id" << 1), Timestamp(1)},
+ {BSON("_id" << 2), Timestamp(2)},
+ {BSON("_id" << 3), Timestamp(3)},
+ {BSON("_id" << 4), Timestamp(4)}})));
// startKey not provided
ASSERT_BSONOBJ_EQ(
@@ -1063,11 +1063,11 @@ TEST_F(StorageInterfaceImplTest,
ASSERT_OK(
storage.insertDocuments(opCtx,
nss,
- {{BSON("_id" << 0), SnapshotName(0), OpTime::kUninitializedTerm},
- {BSON("_id" << 1), SnapshotName(1), OpTime::kUninitializedTerm},
- {BSON("_id" << 2), SnapshotName(2), OpTime::kUninitializedTerm},
- {BSON("_id" << 3), SnapshotName(3), OpTime::kUninitializedTerm},
- {BSON("_id" << 4), SnapshotName(4), OpTime::kUninitializedTerm}}));
+ {{BSON("_id" << 0), Timestamp(0), OpTime::kUninitializedTerm},
+ {BSON("_id" << 1), Timestamp(1), OpTime::kUninitializedTerm},
+ {BSON("_id" << 2), Timestamp(2), OpTime::kUninitializedTerm},
+ {BSON("_id" << 3), Timestamp(3), OpTime::kUninitializedTerm},
+ {BSON("_id" << 4), Timestamp(4), OpTime::kUninitializedTerm}}));
// startKey not provided
ASSERT_BSONOBJ_EQ(
@@ -1179,9 +1179,9 @@ TEST_F(StorageInterfaceImplTest,
ASSERT_OK(
storage.insertDocuments(opCtx,
nss,
- {{BSON("_id" << 1), SnapshotName(1), OpTime::kUninitializedTerm},
- {BSON("_id" << 2), SnapshotName(2), OpTime::kUninitializedTerm},
- {BSON("_id" << 0), SnapshotName(0), OpTime::kUninitializedTerm}}));
+ {{BSON("_id" << 1), Timestamp(1), OpTime::kUninitializedTerm},
+ {BSON("_id" << 2), Timestamp(2), OpTime::kUninitializedTerm},
+ {BSON("_id" << 0), Timestamp(0), OpTime::kUninitializedTerm}}));
ASSERT_BSONOBJ_EQ(
BSON("_id" << 1),
_assetGetFront(storage.findDocuments(opCtx,
@@ -1210,9 +1210,9 @@ TEST_F(StorageInterfaceImplTest,
ASSERT_OK(
storage.insertDocuments(opCtx,
nss,
- {{BSON("_id" << 1), SnapshotName(1), OpTime::kUninitializedTerm},
- {BSON("_id" << 2), SnapshotName(2), OpTime::kUninitializedTerm},
- {BSON("_id" << 0), SnapshotName(0), OpTime::kUninitializedTerm}}));
+ {{BSON("_id" << 1), Timestamp(1), OpTime::kUninitializedTerm},
+ {BSON("_id" << 2), Timestamp(2), OpTime::kUninitializedTerm},
+ {BSON("_id" << 0), Timestamp(0), OpTime::kUninitializedTerm}}));
ASSERT_BSONOBJ_EQ(
BSON("_id" << 0),
_assetGetFront(storage.findDocuments(opCtx,
@@ -1235,9 +1235,9 @@ TEST_F(StorageInterfaceImplTest, FindDocumentsCollScanReturnsNoSuchKeyIfStartKey
ASSERT_OK(
storage.insertDocuments(opCtx,
nss,
- {{BSON("_id" << 1), SnapshotName(1), OpTime::kUninitializedTerm},
- {BSON("_id" << 2), SnapshotName(2), OpTime::kUninitializedTerm},
- {BSON("_id" << 0), SnapshotName(0), OpTime::kUninitializedTerm}}));
+ {{BSON("_id" << 1), Timestamp(1), OpTime::kUninitializedTerm},
+ {BSON("_id" << 2), Timestamp(2), OpTime::kUninitializedTerm},
+ {BSON("_id" << 0), Timestamp(0), OpTime::kUninitializedTerm}}));
ASSERT_EQUALS(ErrorCodes::NoSuchKey,
storage
.findDocuments(opCtx,
@@ -1259,9 +1259,9 @@ TEST_F(StorageInterfaceImplTest,
ASSERT_OK(
storage.insertDocuments(opCtx,
nss,
- {{BSON("_id" << 1), SnapshotName(0), OpTime::kUninitializedTerm},
- {BSON("_id" << 2), SnapshotName(0), OpTime::kUninitializedTerm},
- {BSON("_id" << 0), SnapshotName(0), OpTime::kUninitializedTerm}}));
+ {{BSON("_id" << 1), Timestamp(0), OpTime::kUninitializedTerm},
+ {BSON("_id" << 2), Timestamp(0), OpTime::kUninitializedTerm},
+ {BSON("_id" << 0), Timestamp(0), OpTime::kUninitializedTerm}}));
ASSERT_EQUALS(ErrorCodes::InvalidOptions,
storage
.findDocuments(opCtx,
@@ -1336,14 +1336,14 @@ TEST_F(StorageInterfaceImplTest,
ASSERT_OK(
storage.insertDocuments(opCtx,
nss,
- {{BSON("_id" << 0), SnapshotName(0), OpTime::kUninitializedTerm},
- {BSON("_id" << 1), SnapshotName(1), OpTime::kUninitializedTerm},
- {BSON("_id" << 2), SnapshotName(2), OpTime::kUninitializedTerm},
- {BSON("_id" << 3), SnapshotName(3), OpTime::kUninitializedTerm},
- {BSON("_id" << 4), SnapshotName(4), OpTime::kUninitializedTerm},
- {BSON("_id" << 5), SnapshotName(5), OpTime::kUninitializedTerm},
- {BSON("_id" << 6), SnapshotName(6), OpTime::kUninitializedTerm},
- {BSON("_id" << 7), SnapshotName(7), OpTime::kUninitializedTerm}}));
+ {{BSON("_id" << 0), Timestamp(0), OpTime::kUninitializedTerm},
+ {BSON("_id" << 1), Timestamp(1), OpTime::kUninitializedTerm},
+ {BSON("_id" << 2), Timestamp(2), OpTime::kUninitializedTerm},
+ {BSON("_id" << 3), Timestamp(3), OpTime::kUninitializedTerm},
+ {BSON("_id" << 4), Timestamp(4), OpTime::kUninitializedTerm},
+ {BSON("_id" << 5), Timestamp(5), OpTime::kUninitializedTerm},
+ {BSON("_id" << 6), Timestamp(6), OpTime::kUninitializedTerm},
+ {BSON("_id" << 7), Timestamp(7), OpTime::kUninitializedTerm}}));
// startKey not provided
ASSERT_BSONOBJ_EQ(
@@ -1447,14 +1447,14 @@ TEST_F(StorageInterfaceImplTest,
ASSERT_OK(
storage.insertDocuments(opCtx,
nss,
- {{BSON("_id" << 0), SnapshotName(0), OpTime::kUninitializedTerm},
- {BSON("_id" << 1), SnapshotName(1), OpTime::kUninitializedTerm},
- {BSON("_id" << 2), SnapshotName(2), OpTime::kUninitializedTerm},
- {BSON("_id" << 3), SnapshotName(3), OpTime::kUninitializedTerm},
- {BSON("_id" << 4), SnapshotName(4), OpTime::kUninitializedTerm},
- {BSON("_id" << 5), SnapshotName(5), OpTime::kUninitializedTerm},
- {BSON("_id" << 6), SnapshotName(6), OpTime::kUninitializedTerm},
- {BSON("_id" << 7), SnapshotName(7), OpTime::kUninitializedTerm}}));
+ {{BSON("_id" << 0), Timestamp(0), OpTime::kUninitializedTerm},
+ {BSON("_id" << 1), Timestamp(1), OpTime::kUninitializedTerm},
+ {BSON("_id" << 2), Timestamp(2), OpTime::kUninitializedTerm},
+ {BSON("_id" << 3), Timestamp(3), OpTime::kUninitializedTerm},
+ {BSON("_id" << 4), Timestamp(4), OpTime::kUninitializedTerm},
+ {BSON("_id" << 5), Timestamp(5), OpTime::kUninitializedTerm},
+ {BSON("_id" << 6), Timestamp(6), OpTime::kUninitializedTerm},
+ {BSON("_id" << 7), Timestamp(7), OpTime::kUninitializedTerm}}));
// startKey not provided
ASSERT_BSONOBJ_EQ(
@@ -1557,9 +1557,9 @@ TEST_F(StorageInterfaceImplTest,
ASSERT_OK(
storage.insertDocuments(opCtx,
nss,
- {{BSON("_id" << 1), SnapshotName(0), OpTime::kUninitializedTerm},
- {BSON("_id" << 2), SnapshotName(0), OpTime::kUninitializedTerm},
- {BSON("_id" << 0), SnapshotName(0), OpTime::kUninitializedTerm}}));
+ {{BSON("_id" << 1), Timestamp(0), OpTime::kUninitializedTerm},
+ {BSON("_id" << 2), Timestamp(0), OpTime::kUninitializedTerm},
+ {BSON("_id" << 0), Timestamp(0), OpTime::kUninitializedTerm}}));
ASSERT_BSONOBJ_EQ(
BSON("_id" << 1),
_assetGetFront(storage.deleteDocuments(opCtx,
@@ -1582,9 +1582,9 @@ TEST_F(StorageInterfaceImplTest,
ASSERT_OK(
storage.insertDocuments(opCtx,
nss,
- {{BSON("_id" << 1), SnapshotName(0), OpTime::kUninitializedTerm},
- {BSON("_id" << 2), SnapshotName(0), OpTime::kUninitializedTerm},
- {BSON("_id" << 0), SnapshotName(0), OpTime::kUninitializedTerm}}));
+ {{BSON("_id" << 1), Timestamp(0), OpTime::kUninitializedTerm},
+ {BSON("_id" << 2), Timestamp(0), OpTime::kUninitializedTerm},
+ {BSON("_id" << 0), Timestamp(0), OpTime::kUninitializedTerm}}));
ASSERT_BSONOBJ_EQ(
BSON("_id" << 0),
_assetGetFront(storage.deleteDocuments(opCtx,
@@ -1606,9 +1606,9 @@ TEST_F(StorageInterfaceImplTest, DeleteDocumentsCollScanReturnsNoSuchKeyIfStartK
ASSERT_OK(
storage.insertDocuments(opCtx,
nss,
- {{BSON("_id" << 1), SnapshotName(0), OpTime::kUninitializedTerm},
- {BSON("_id" << 2), SnapshotName(0), OpTime::kUninitializedTerm},
- {BSON("_id" << 0), SnapshotName(0), OpTime::kUninitializedTerm}}));
+ {{BSON("_id" << 1), Timestamp(0), OpTime::kUninitializedTerm},
+ {BSON("_id" << 2), Timestamp(0), OpTime::kUninitializedTerm},
+ {BSON("_id" << 0), Timestamp(0), OpTime::kUninitializedTerm}}));
ASSERT_EQUALS(ErrorCodes::NoSuchKey,
storage
.deleteDocuments(opCtx,
@@ -1630,9 +1630,9 @@ TEST_F(StorageInterfaceImplTest,
ASSERT_OK(
storage.insertDocuments(opCtx,
nss,
- {{BSON("_id" << 1), SnapshotName(0), OpTime::kUninitializedTerm},
- {BSON("_id" << 2), SnapshotName(0), OpTime::kUninitializedTerm},
- {BSON("_id" << 0), SnapshotName(0), OpTime::kUninitializedTerm}}));
+ {{BSON("_id" << 1), Timestamp(0), OpTime::kUninitializedTerm},
+ {BSON("_id" << 2), Timestamp(0), OpTime::kUninitializedTerm},
+ {BSON("_id" << 0), Timestamp(0), OpTime::kUninitializedTerm}}));
ASSERT_EQUALS(ErrorCodes::InvalidOptions,
storage
.deleteDocuments(opCtx,
@@ -1678,8 +1678,8 @@ TEST_F(StorageInterfaceImplTest,
auto doc2 = BSON("_id" << 1 << "x" << 1);
ASSERT_OK(storage.insertDocuments(opCtx,
nss,
- {{doc1, SnapshotName(0), OpTime::kUninitializedTerm},
- {doc2, SnapshotName(0), OpTime::kUninitializedTerm}}));
+ {{doc1, Timestamp(0), OpTime::kUninitializedTerm},
+ {doc2, Timestamp(0), OpTime::kUninitializedTerm}}));
ASSERT_EQUALS(ErrorCodes::TooManyMatchingDocuments,
storage.findSingleton(opCtx, nss).getStatus());
}
@@ -1690,8 +1690,7 @@ TEST_F(StorageInterfaceImplTest, FindSingletonReturnsDocumentWhenSingletonDocume
auto nss = makeNamespace(_agent);
ASSERT_OK(storage.createCollection(opCtx, nss, CollectionOptions()));
auto doc1 = BSON("_id" << 0 << "x" << 0);
- ASSERT_OK(
- storage.insertDocument(opCtx, nss, {doc1, SnapshotName(0)}, OpTime::kUninitializedTerm));
+ ASSERT_OK(storage.insertDocument(opCtx, nss, {doc1, Timestamp(0)}, OpTime::kUninitializedTerm));
ASSERT_BSONOBJ_EQ(doc1, unittest::assertGet(storage.findSingleton(opCtx, nss)));
}
@@ -1731,8 +1730,7 @@ TEST_F(StorageInterfaceImplTest, PutSingletonUpdatesDocumentWhenCollectionIsNotE
auto nss = makeNamespace(_agent);
ASSERT_OK(storage.createCollection(opCtx, nss, CollectionOptions()));
auto doc1 = BSON("_id" << 0 << "x" << 0);
- ASSERT_OK(
- storage.insertDocument(opCtx, nss, {doc1, SnapshotName(0)}, OpTime::kUninitializedTerm));
+ ASSERT_OK(storage.insertDocument(opCtx, nss, {doc1, Timestamp(0)}, OpTime::kUninitializedTerm));
auto update = BSON("$set" << BSON("x" << 1));
ASSERT_OK(storage.putSingleton(opCtx, nss, update));
ASSERT_BSONOBJ_EQ(BSON("_id" << 0 << "x" << 1),
@@ -1749,8 +1747,8 @@ TEST_F(StorageInterfaceImplTest, PutSingletonUpdatesFirstDocumentWhenCollectionI
auto doc2 = BSON("_id" << 1 << "x" << 1);
ASSERT_OK(storage.insertDocuments(opCtx,
nss,
- {{doc1, SnapshotName(0), OpTime::kUninitializedTerm},
- {doc2, SnapshotName(0), OpTime::kUninitializedTerm}}));
+ {{doc1, Timestamp(0), OpTime::kUninitializedTerm},
+ {doc2, Timestamp(0), OpTime::kUninitializedTerm}}));
auto update = BSON("$set" << BSON("x" << 2));
ASSERT_OK(storage.putSingleton(opCtx, nss, update));
_assertDocumentsInCollectionEquals(opCtx, nss, {BSON("_id" << 0 << "x" << 2), doc2});
@@ -1774,7 +1772,7 @@ TEST_F(StorageInterfaceImplTest, UpdateSingletonUpdatesDocumentWhenCollectionIsN
ASSERT_OK(storage.createCollection(opCtx, nss, CollectionOptions()));
auto doc1 = BSON("_id" << 0 << "x" << 0);
ASSERT_OK(
- storage.insertDocument(opCtx, nss, {doc1, SnapshotName(0)}, OpTime::kUninitializedTerm));
+ storage.insertDocument(opCtx, nss, {doc1, Timestamp::min()}, OpTime::kUninitializedTerm));
auto update = BSON("$set" << BSON("x" << 1));
ASSERT_OK(storage.updateSingleton(opCtx, nss, BSON("_id" << 0), update));
ASSERT_BSONOBJ_EQ(BSON("_id" << 0 << "x" << 1),
@@ -1810,8 +1808,8 @@ TEST_F(StorageInterfaceImplTest, FindByIdReturnsNoSuchKeyWhenDocumentIsNotFound)
auto doc3 = BSON("_id" << 2 << "x" << 2);
ASSERT_OK(storage.insertDocuments(opCtx,
nss,
- {{doc1, SnapshotName(0), OpTime::kUninitializedTerm},
- {doc3, SnapshotName(0), OpTime::kUninitializedTerm}}));
+ {{doc1, Timestamp(0), OpTime::kUninitializedTerm},
+ {doc3, Timestamp(0), OpTime::kUninitializedTerm}}));
ASSERT_EQUALS(ErrorCodes::NoSuchKey, storage.findById(opCtx, nss, doc2["_id"]).getStatus());
}
@@ -1825,9 +1823,9 @@ TEST_F(StorageInterfaceImplTest, FindByIdReturnsDocumentWhenDocumentExists) {
auto doc3 = BSON("_id" << 2 << "x" << 2);
ASSERT_OK(storage.insertDocuments(opCtx,
nss,
- {{doc1, SnapshotName(0), OpTime::kUninitializedTerm},
- {doc2, SnapshotName(0), OpTime::kUninitializedTerm},
- {doc3, SnapshotName(0), OpTime::kUninitializedTerm}}));
+ {{doc1, Timestamp(0), OpTime::kUninitializedTerm},
+ {doc2, Timestamp(0), OpTime::kUninitializedTerm},
+ {doc3, Timestamp(0), OpTime::kUninitializedTerm}}));
ASSERT_BSONOBJ_EQ(doc2, unittest::assertGet(storage.findById(opCtx, nss, doc2["_id"])));
}
@@ -1860,8 +1858,8 @@ TEST_F(StorageInterfaceImplTest, DeleteByIdReturnsNoSuchKeyWhenDocumentIsNotFoun
auto doc3 = BSON("_id" << 2 << "x" << 2);
ASSERT_OK(storage.insertDocuments(opCtx,
nss,
- {{doc1, SnapshotName(0), OpTime::kUninitializedTerm},
- {doc3, SnapshotName(0), OpTime::kUninitializedTerm}}));
+ {{doc1, Timestamp(0), OpTime::kUninitializedTerm},
+ {doc3, Timestamp(0), OpTime::kUninitializedTerm}}));
ASSERT_EQUALS(ErrorCodes::NoSuchKey, storage.deleteById(opCtx, nss, doc2["_id"]).getStatus());
_assertDocumentsInCollectionEquals(opCtx, nss, {doc1, doc3});
}
@@ -1876,9 +1874,9 @@ TEST_F(StorageInterfaceImplTest, DeleteByIdReturnsDocumentWhenDocumentExists) {
auto doc3 = BSON("_id" << 2 << "x" << 2);
ASSERT_OK(storage.insertDocuments(opCtx,
nss,
- {{doc1, SnapshotName(0), OpTime::kUninitializedTerm},
- {doc2, SnapshotName(0), OpTime::kUninitializedTerm},
- {doc3, SnapshotName(0), OpTime::kUninitializedTerm}}));
+ {{doc1, Timestamp(0), OpTime::kUninitializedTerm},
+ {doc2, Timestamp(0), OpTime::kUninitializedTerm},
+ {doc3, Timestamp(0), OpTime::kUninitializedTerm}}));
ASSERT_BSONOBJ_EQ(doc2, unittest::assertGet(storage.deleteById(opCtx, nss, doc2["_id"])));
_assertDocumentsInCollectionEquals(opCtx, nss, {doc1, doc3});
}
@@ -1918,9 +1916,9 @@ TEST_F(StorageInterfaceImplTest, UpsertSingleDocumentReplacesExistingDocumentInC
ASSERT_OK(storage.insertDocuments(
opCtx,
nss,
- {{BSON("_id" << 0 << "x" << 0), SnapshotName(0), OpTime::kUninitializedTerm},
- {originalDoc, SnapshotName(0), OpTime::kUninitializedTerm},
- {BSON("_id" << 2 << "x" << 2), SnapshotName(2), OpTime::kUninitializedTerm}}));
+ {{BSON("_id" << 0 << "x" << 0), Timestamp(0), OpTime::kUninitializedTerm},
+ {originalDoc, Timestamp(0), OpTime::kUninitializedTerm},
+ {BSON("_id" << 2 << "x" << 2), Timestamp(2), OpTime::kUninitializedTerm}}));
ASSERT_OK(storage.upsertById(opCtx, nss, originalDoc["_id"], BSON("x" << 100)));
@@ -1940,8 +1938,8 @@ TEST_F(StorageInterfaceImplTest, UpsertSingleDocumentInsertsNewDocumentInCollect
ASSERT_OK(storage.insertDocuments(
opCtx,
nss,
- {{BSON("_id" << 0 << "x" << 0), SnapshotName(0), OpTime::kUninitializedTerm},
- {BSON("_id" << 2 << "x" << 2), SnapshotName(2), OpTime::kUninitializedTerm}}));
+ {{BSON("_id" << 0 << "x" << 0), Timestamp(0), OpTime::kUninitializedTerm},
+ {BSON("_id" << 2 << "x" << 2), Timestamp(2), OpTime::kUninitializedTerm}}));
ASSERT_OK(storage.upsertById(opCtx, nss, BSON("" << 1).firstElement(), BSON("x" << 100)));
@@ -1969,9 +1967,9 @@ TEST_F(StorageInterfaceImplTest,
ASSERT_OK(storage.insertDocuments(
opCtx,
nss,
- {{BSON("_id" << 0 << "x" << 0), SnapshotName(0), OpTime::kUninitializedTerm},
- {originalDoc, SnapshotName(0), OpTime::kUninitializedTerm},
- {BSON("_id" << 2 << "x" << 2), SnapshotName(2), OpTime::kUninitializedTerm}}));
+ {{BSON("_id" << 0 << "x" << 0), Timestamp(0), OpTime::kUninitializedTerm},
+ {originalDoc, Timestamp(0), OpTime::kUninitializedTerm},
+ {BSON("_id" << 2 << "x" << 2), Timestamp(2), OpTime::kUninitializedTerm}}));
ASSERT_OK(storage.upsertById(opCtx, nss, originalDoc["_id"], BSON("x" << 100)));
@@ -2082,7 +2080,7 @@ TEST_F(
auto doc = BSON("_id" << 0 << "x" << 0);
ASSERT_OK(
- storage.insertDocuments(opCtx, nss, {{doc, SnapshotName(0), OpTime::kUninitializedTerm}}));
+ storage.insertDocuments(opCtx, nss, {{doc, Timestamp(0), OpTime::kUninitializedTerm}}));
_assertDocumentsInCollectionEquals(opCtx, nss, {doc});
// This test fixture disables replicated writes by default. We want to re-enable this setting
@@ -2108,7 +2106,7 @@ TEST_F(
auto doc = BSON("_id" << 0 << "x" << 0);
ASSERT_OK(
- storage.insertDocuments(opCtx, nss, {{doc, SnapshotName(0), OpTime::kUninitializedTerm}}));
+ storage.insertDocuments(opCtx, nss, {{doc, Timestamp(0), OpTime::kUninitializedTerm}}));
_assertDocumentsInCollectionEquals(opCtx, nss, {doc});
// This test fixture disables replicated writes by default. We want to re-enable this setting
@@ -2158,8 +2156,8 @@ TEST_F(StorageInterfaceImplTest, DeleteByFilterLeavesCollectionUnchangedIfNoDocu
auto nss = makeNamespace(_agent);
ASSERT_OK(storage.createCollection(opCtx, nss, CollectionOptions()));
- std::vector<TimestampedBSONObj> docs = {{BSON("_id" << 0 << "x" << 0), SnapshotName(0)},
- {BSON("_id" << 2 << "x" << 2), SnapshotName(0)}};
+ std::vector<TimestampedBSONObj> docs = {{BSON("_id" << 0 << "x" << 0), Timestamp(0)},
+ {BSON("_id" << 2 << "x" << 2), Timestamp(0)}};
ASSERT_OK(storage.insertDocuments(opCtx, nss, transformInserts(docs)));
auto filter = BSON("x" << 1);
@@ -2174,10 +2172,10 @@ TEST_F(StorageInterfaceImplTest, DeleteByFilterRemoveDocumentsThatMatchFilter) {
auto nss = makeNamespace(_agent);
ASSERT_OK(storage.createCollection(opCtx, nss, CollectionOptions()));
- std::vector<TimestampedBSONObj> docs = {{BSON("_id" << 0 << "x" << 0), SnapshotName(0)},
- {BSON("_id" << 1 << "x" << 1), SnapshotName(0)},
- {BSON("_id" << 2 << "x" << 2), SnapshotName(0)},
- {BSON("_id" << 3 << "x" << 3), SnapshotName(0)}};
+ std::vector<TimestampedBSONObj> docs = {{BSON("_id" << 0 << "x" << 0), Timestamp(0)},
+ {BSON("_id" << 1 << "x" << 1), Timestamp(0)},
+ {BSON("_id" << 2 << "x" << 2), Timestamp(0)},
+ {BSON("_id" << 3 << "x" << 3), Timestamp(0)}};
ASSERT_OK(storage.insertDocuments(opCtx, nss, transformInserts(docs)));
auto filter = BSON("x" << BSON("$in" << BSON_ARRAY(1 << 2)));
@@ -2194,10 +2192,10 @@ TEST_F(StorageInterfaceImplTest, DeleteByFilterExpandsDottedFieldNamesAsPaths) {
ASSERT_OK(storage.createCollection(opCtx, nss, CollectionOptions()));
std::vector<TimestampedBSONObj> docs = {
- {BSON("_id" << 0 << "x" << BSON("y" << 0)), SnapshotName(0)},
- {BSON("_id" << 1 << "x" << BSON("y" << 1)), SnapshotName(0)},
- {BSON("_id" << 2 << "x" << BSON("y" << 2)), SnapshotName(0)},
- {BSON("_id" << 3 << "x" << BSON("y" << 3)), SnapshotName(0)}};
+ {BSON("_id" << 0 << "x" << BSON("y" << 0)), Timestamp::min()},
+ {BSON("_id" << 1 << "x" << BSON("y" << 1)), Timestamp::min()},
+ {BSON("_id" << 2 << "x" << BSON("y" << 2)), Timestamp::min()},
+ {BSON("_id" << 3 << "x" << BSON("y" << 3)), Timestamp::min()}};
ASSERT_OK(storage.insertDocuments(opCtx, nss, transformInserts(docs)));
auto filter = BSON("x.y" << BSON("$gte" << 1));
@@ -2213,8 +2211,8 @@ TEST_F(StorageInterfaceImplTest, DeleteByFilterUsesIdHackIfFilterContainsIdField
auto nss = makeNamespace(_agent);
ASSERT_OK(storage.createCollection(opCtx, nss, CollectionOptions()));
- std::vector<TimestampedBSONObj> docs = {{BSON("_id" << 0 << "x" << 0), SnapshotName(0)},
- {BSON("_id" << 1 << "x" << 1), SnapshotName(0)}};
+ std::vector<TimestampedBSONObj> docs = {{BSON("_id" << 0 << "x" << 0), Timestamp(0)},
+ {BSON("_id" << 1 << "x" << 1), Timestamp(0)}};
ASSERT_OK(storage.insertDocuments(opCtx, nss, transformInserts(docs)));
auto filter = BSON("_id" << 1);
@@ -2234,10 +2232,10 @@ TEST_F(StorageInterfaceImplTest, DeleteByFilterRemovesDocumentsInIllegalClientSy
StorageInterfaceImpl storage;
ASSERT_OK(storage.createCollection(opCtx, nss, CollectionOptions()));
- std::vector<TimestampedBSONObj> docs = {{BSON("_id" << 0 << "x" << 0), SnapshotName(0)},
- {BSON("_id" << 1 << "x" << 1), SnapshotName(0)},
- {BSON("_id" << 2 << "x" << 2), SnapshotName(0)},
- {BSON("_id" << 3 << "x" << 3), SnapshotName(0)}};
+ std::vector<TimestampedBSONObj> docs = {{BSON("_id" << 0 << "x" << 0), Timestamp(0)},
+ {BSON("_id" << 1 << "x" << 1), Timestamp(0)},
+ {BSON("_id" << 2 << "x" << 2), Timestamp(0)},
+ {BSON("_id" << 3 << "x" << 3), Timestamp(0)}};
ASSERT_OK(storage.insertDocuments(opCtx, nss, transformInserts(docs)));
auto filter = BSON("$or" << BSON_ARRAY(BSON("x" << 0) << BSON("_id" << 2)));
@@ -2271,10 +2269,10 @@ TEST_F(StorageInterfaceImplTest,
<< "def");
ASSERT_OK(storage.insertDocuments(opCtx,
nss,
- {{doc1, SnapshotName(0), OpTime::kUninitializedTerm},
- {doc2, SnapshotName(0), OpTime::kUninitializedTerm},
- {doc3, SnapshotName(0), OpTime::kUninitializedTerm},
- {doc4, SnapshotName(0), OpTime::kUninitializedTerm}}));
+ {{doc1, Timestamp(0), OpTime::kUninitializedTerm},
+ {doc2, Timestamp(0), OpTime::kUninitializedTerm},
+ {doc3, Timestamp(0), OpTime::kUninitializedTerm},
+ {doc4, Timestamp(0), OpTime::kUninitializedTerm}}));
// This filter should remove doc1 and doc2 because the values of the field "x"
// are equivalent to "aBc" under the case-insensive collation.
@@ -2322,9 +2320,9 @@ TEST_F(StorageInterfaceImplTest, GetCollectionCountReturnsCollectionCount) {
ASSERT_OK(
storage.insertDocuments(opCtx,
nss,
- {{BSON("_id" << 1), SnapshotName(0), OpTime::kUninitializedTerm},
- {BSON("_id" << 2), SnapshotName(0), OpTime::kUninitializedTerm},
- {BSON("_id" << 0), SnapshotName(0), OpTime::kUninitializedTerm}}));
+ {{BSON("_id" << 1), Timestamp(0), OpTime::kUninitializedTerm},
+ {BSON("_id" << 2), Timestamp(0), OpTime::kUninitializedTerm},
+ {BSON("_id" << 0), Timestamp(0), OpTime::kUninitializedTerm}}));
auto count = unittest::assertGet(storage.getCollectionCount(opCtx, nss));
ASSERT_EQUALS(3UL, count);
}
@@ -2462,9 +2460,9 @@ TEST_F(StorageInterfaceImplTest, GetCollectionSizeReturnsCollectionSize) {
ASSERT_OK(
storage.insertDocuments(opCtx,
nss,
- {{BSON("_id" << 1), SnapshotName(0), OpTime::kUninitializedTerm},
- {BSON("_id" << 2), SnapshotName(0), OpTime::kUninitializedTerm},
- {BSON("_id" << 0), SnapshotName(0), OpTime::kUninitializedTerm}}));
+ {{BSON("_id" << 1), Timestamp(0), OpTime::kUninitializedTerm},
+ {BSON("_id" << 2), Timestamp(0), OpTime::kUninitializedTerm},
+ {BSON("_id" << 0), Timestamp(0), OpTime::kUninitializedTerm}}));
auto size = unittest::assertGet(storage.getCollectionSize(opCtx, nss));
ASSERT_NOT_EQUALS(0UL, size);
}
diff --git a/src/mongo/db/repl/storage_interface_mock.cpp b/src/mongo/db/repl/storage_interface_mock.cpp
index aee45106f92..01add91bd00 100644
--- a/src/mongo/db/repl/storage_interface_mock.cpp
+++ b/src/mongo/db/repl/storage_interface_mock.cpp
@@ -68,24 +68,23 @@ Status StorageInterfaceMock::incrementRollbackID(OperationContext* opCtx) {
return Status::OK();
}
-void StorageInterfaceMock::setStableTimestamp(ServiceContext* serviceCtx,
- SnapshotName snapshotName) {
+void StorageInterfaceMock::setStableTimestamp(ServiceContext* serviceCtx, Timestamp snapshotName) {
stdx::lock_guard<stdx::mutex> lock(_mutex);
_stableTimestamp = snapshotName;
}
void StorageInterfaceMock::setInitialDataTimestamp(ServiceContext* serviceCtx,
- SnapshotName snapshotName) {
+ Timestamp snapshotName) {
stdx::lock_guard<stdx::mutex> lock(_mutex);
_initialDataTimestamp = snapshotName;
}
-SnapshotName StorageInterfaceMock::getStableTimestamp() const {
+Timestamp StorageInterfaceMock::getStableTimestamp() const {
stdx::lock_guard<stdx::mutex> lock(_mutex);
return _stableTimestamp;
}
-SnapshotName StorageInterfaceMock::getInitialDataTimestamp() const {
+Timestamp StorageInterfaceMock::getInitialDataTimestamp() const {
stdx::lock_guard<stdx::mutex> lock(_mutex);
return _initialDataTimestamp;
}
diff --git a/src/mongo/db/repl/storage_interface_mock.h b/src/mongo/db/repl/storage_interface_mock.h
index bc16ff01bbc..c67f339a691 100644
--- a/src/mongo/db/repl/storage_interface_mock.h
+++ b/src/mongo/db/repl/storage_interface_mock.h
@@ -34,6 +34,7 @@
#include "mongo/base/status_with.h"
#include "mongo/base/string_data.h"
#include "mongo/bson/bsonobj.h"
+#include "mongo/bson/timestamp.h"
#include "mongo/db/namespace_string.h"
#include "mongo/db/repl/storage_interface.h"
#include "mongo/stdx/mutex.h"
@@ -272,13 +273,13 @@ public:
return upgradeUUIDSchemaVersionNonReplicatedFn(opCtx);
}
- void setStableTimestamp(ServiceContext* serviceCtx, SnapshotName snapshotName) override;
+ void setStableTimestamp(ServiceContext* serviceCtx, Timestamp snapshotName) override;
- void setInitialDataTimestamp(ServiceContext* serviceCtx, SnapshotName snapshotName) override;
+ void setInitialDataTimestamp(ServiceContext* serviceCtx, Timestamp snapshotName) override;
- SnapshotName getStableTimestamp() const;
+ Timestamp getStableTimestamp() const;
- SnapshotName getInitialDataTimestamp() const;
+ Timestamp getInitialDataTimestamp() const;
Status recoverToStableTimestamp(ServiceContext* serviceCtx) override {
return Status{ErrorCodes::IllegalOperation, "recoverToStableTimestamp not implemented."};
@@ -363,8 +364,8 @@ private:
mutable stdx::mutex _mutex;
int _rbid;
bool _rbidInitialized = false;
- SnapshotName _stableTimestamp = SnapshotName::min();
- SnapshotName _initialDataTimestamp = SnapshotName::min();
+ Timestamp _stableTimestamp = Timestamp::min();
+ Timestamp _initialDataTimestamp = Timestamp::min();
OptionalCollectionUUID _uuid;
bool _schemaUpgraded;
};
diff --git a/src/mongo/db/repl/sync_tail.cpp b/src/mongo/db/repl/sync_tail.cpp
index 35362355aad..304c6caf9d8 100644
--- a/src/mongo/db/repl/sync_tail.cpp
+++ b/src/mongo/db/repl/sync_tail.cpp
@@ -39,6 +39,7 @@
#include "mongo/base/counter.h"
#include "mongo/bson/bsonelement_comparator.h"
+#include "mongo/bson/timestamp.h"
#include "mongo/db/auth/authorization_session.h"
#include "mongo/db/catalog/collection.h"
#include "mongo/db/catalog/database.h"
@@ -500,9 +501,8 @@ void scheduleWritesToOplog(OperationContext* opCtx,
for (size_t i = begin; i < end; i++) {
// Add as unowned BSON to avoid unnecessary ref-count bumps.
// 'ops' will outlive 'docs' so the BSON lifetime will be guaranteed.
- docs.emplace_back(InsertStatement{ops[i].raw,
- SnapshotName(ops[i].getOpTime().getTimestamp()),
- ops[i].getOpTime().getTerm()});
+ docs.emplace_back(InsertStatement{
+ ops[i].raw, ops[i].getOpTime().getTimestamp(), ops[i].getOpTime().getTerm()});
}
fassertStatusOK(40141,
diff --git a/src/mongo/db/storage/SConscript b/src/mongo/db/storage/SConscript
index 079a1e4ec91..3f1362e3a0c 100644
--- a/src/mongo/db/storage/SConscript
+++ b/src/mongo/db/storage/SConscript
@@ -222,10 +222,3 @@ env.CppUnitTest(
'$BUILD_DIR/mongo/base',
]
)
-
-env.CppUnitTest(
- target='storage_snapshot_name_test',
- source='storage_snapshot_name_test.cpp',
- LIBDEPS=[],
- LIBDEPS_PRIVATE=[],
-)
diff --git a/src/mongo/db/storage/kv/kv_engine.h b/src/mongo/db/storage/kv/kv_engine.h
index 3350971bddc..c0e5bc1b906 100644
--- a/src/mongo/db/storage/kv/kv_engine.h
+++ b/src/mongo/db/storage/kv/kv_engine.h
@@ -36,10 +36,10 @@
#include "mongo/base/status.h"
#include "mongo/base/string_data.h"
+#include "mongo/bson/timestamp.h"
#include "mongo/db/catalog/collection_options.h"
#include "mongo/db/storage/kv/kv_prefix.h"
#include "mongo/db/storage/record_store.h"
-#include "mongo/db/storage/snapshot_name.h"
namespace mongo {
@@ -249,12 +249,12 @@ public:
/**
* See `StorageEngine::setStableTimestamp`
*/
- virtual void setStableTimestamp(SnapshotName stableTimestamp) {}
+ virtual void setStableTimestamp(Timestamp stableTimestamp) {}
/**
* See `StorageEngine::setInitialDataTimestamp`
*/
- virtual void setInitialDataTimestamp(SnapshotName initialDataTimestamp) {}
+ virtual void setInitialDataTimestamp(Timestamp initialDataTimestamp) {}
/**
* See `StorageEngine::supportsRecoverToStableTimestamp`
diff --git a/src/mongo/db/storage/kv/kv_engine_test_timestamps.cpp b/src/mongo/db/storage/kv/kv_engine_test_timestamps.cpp
index a45da7bf28b..7112ce561c8 100644
--- a/src/mongo/db/storage/kv/kv_engine_test_timestamps.cpp
+++ b/src/mongo/db/storage/kv/kv_engine_test_timestamps.cpp
@@ -31,6 +31,7 @@
#include <memory>
#include <string>
+#include "mongo/bson/timestamp.h"
#include "mongo/db/operation_context_noop.h"
#include "mongo/db/service_context_noop.h"
#include "mongo/db/storage/kv/kv_engine.h"
@@ -89,10 +90,11 @@ public:
return Operation(service.makeClient(""), helper->getEngine()->newRecoveryUnit());
}
- SnapshotName incrementTimestamp() {
- auto name = SnapshotName(_counter);
+ // Returns the timestamp before incrementing.
+ Timestamp fetchAndIncrementTimestamp() {
+ auto preImage = _counter;
_counter = Timestamp(_counter.getSecs() + 1, _counter.getInc());
- return name;
+ return preImage;
}
RecordId insertRecord(OperationContext* opCtx, std::string contents = "abcd") {
@@ -112,7 +114,7 @@ public:
void updateRecordAndCommit(RecordId id, std::string contents) {
auto op = makeOperation();
WriteUnitOfWork wuow(op);
- ASSERT_OK(op->recoveryUnit()->setTimestamp(SnapshotName(_counter)));
+ ASSERT_OK(op->recoveryUnit()->setTimestamp(_counter));
ASSERT_OK(
rs->updateRecord(op, id, contents.c_str(), contents.length() + 1, false, nullptr));
wuow.commit();
@@ -121,7 +123,7 @@ public:
void deleteRecordAndCommit(RecordId id) {
auto op = makeOperation();
WriteUnitOfWork wuow(op);
- ASSERT_OK(op->recoveryUnit()->setTimestamp(SnapshotName(_counter)));
+ ASSERT_OK(op->recoveryUnit()->setTimestamp(_counter));
rs->deleteRecord(op, id);
wuow.commit();
}
@@ -208,12 +210,12 @@ TEST_F(SnapshotManagerTests, FailsWithNoCommittedSnapshot) {
ErrorCodes::ReadConcernMajorityNotAvailableYet);
// There is a snapshot but it isn't committed.
- auto name = incrementTimestamp();
+ auto snap = fetchAndIncrementTimestamp();
ASSERT_EQ(ru->setReadFromMajorityCommittedSnapshot(),
ErrorCodes::ReadConcernMajorityNotAvailableYet);
// Now there is a committed snapshot.
- snapshotManager->setCommittedSnapshot(name, Timestamp(name.asU64()));
+ snapshotManager->setCommittedSnapshot(snap);
ASSERT_OK(ru->setReadFromMajorityCommittedSnapshot());
// Not anymore!
@@ -229,8 +231,8 @@ TEST_F(SnapshotManagerTests, FailsAfterDropAllSnapshotsWhileYielded) {
auto op = makeOperation();
// Start an operation using a committed snapshot.
- auto name = incrementTimestamp();
- snapshotManager->setCommittedSnapshot(name, Timestamp(name.asU64()));
+ auto snap = fetchAndIncrementTimestamp();
+ snapshotManager->setCommittedSnapshot(snap);
ASSERT_OK(op->recoveryUnit()->setReadFromMajorityCommittedSnapshot());
ASSERT_EQ(itCountOn(op), 0); // acquires a snapshot.
@@ -248,10 +250,8 @@ TEST_F(SnapshotManagerTests, BasicFunctionality) {
if (!snapshotManager)
return; // This test is only for engines that DO support SnapshotMangers.
- // Snapshot variables are named according to the size of the RecordStore at the time of the
- // snapshot.
- auto snap0 = incrementTimestamp();
- snapshotManager->setCommittedSnapshot(snap0, Timestamp(snap0.asU64()));
+ auto snap0 = fetchAndIncrementTimestamp();
+ snapshotManager->setCommittedSnapshot(snap0);
ASSERT_EQ(itCountCommitted(), 0);
insertRecordAndCommit();
@@ -259,11 +259,11 @@ TEST_F(SnapshotManagerTests, BasicFunctionality) {
ASSERT_EQ(itCountCommitted(), 0);
- auto snap1 = incrementTimestamp();
+ auto snap1 = fetchAndIncrementTimestamp();
insertRecordAndCommit();
insertRecordAndCommit();
- auto snap3 = incrementTimestamp();
+ auto snap3 = fetchAndIncrementTimestamp();
{
auto op = makeOperation();
@@ -273,13 +273,13 @@ TEST_F(SnapshotManagerTests, BasicFunctionality) {
}
insertRecordAndCommit();
- auto snap4 = incrementTimestamp();
+ auto snap4 = fetchAndIncrementTimestamp();
// If these fail, everything is busted.
ASSERT_EQ(itCountCommitted(), 0);
- snapshotManager->setCommittedSnapshot(snap1, Timestamp(snap1.asU64()));
+ snapshotManager->setCommittedSnapshot(snap1);
ASSERT_EQ(itCountCommitted(), 1);
- snapshotManager->setCommittedSnapshot(snap3, Timestamp(snap3.asU64()));
+ snapshotManager->setCommittedSnapshot(snap3);
ASSERT_EQ(itCountCommitted(), 3);
// This op should keep its original snapshot until abandoned.
@@ -288,7 +288,7 @@ TEST_F(SnapshotManagerTests, BasicFunctionality) {
ASSERT_EQ(itCountOn(longOp), 3);
// If this fails, the snapshot contains writes that were rolled back.
- snapshotManager->setCommittedSnapshot(snap4, Timestamp(snap4.asU64()));
+ snapshotManager->setCommittedSnapshot(snap4);
ASSERT_EQ(itCountCommitted(), 4);
// If this fails, longOp changed snapshots at an illegal time.
@@ -303,31 +303,30 @@ TEST_F(SnapshotManagerTests, UpdateAndDelete) {
if (!snapshotManager)
return; // This test is only for engines that DO support SnapshotMangers.
- // Snapshot variables are named according to the state of the record.
- auto snapBeforeInsert = incrementTimestamp();
+ auto snapBeforeInsert = fetchAndIncrementTimestamp();
auto id = insertRecordAndCommit("Dog");
- auto snapDog = incrementTimestamp();
+ auto snapDog = fetchAndIncrementTimestamp();
updateRecordAndCommit(id, "Cat");
- auto snapCat = incrementTimestamp();
+ auto snapCat = fetchAndIncrementTimestamp();
deleteRecordAndCommit(id);
- auto snapAfterDelete = incrementTimestamp();
+ auto snapAfterDelete = fetchAndIncrementTimestamp();
- snapshotManager->setCommittedSnapshot(snapBeforeInsert, Timestamp(snapBeforeInsert.asU64()));
+ snapshotManager->setCommittedSnapshot(snapBeforeInsert);
ASSERT_EQ(itCountCommitted(), 0);
ASSERT(!readRecordCommitted(id));
- snapshotManager->setCommittedSnapshot(snapDog, Timestamp(snapDog.asU64()));
+ snapshotManager->setCommittedSnapshot(snapDog);
ASSERT_EQ(itCountCommitted(), 1);
ASSERT_EQ(readStringCommitted(id), "Dog");
- snapshotManager->setCommittedSnapshot(snapCat, Timestamp(snapCat.asU64()));
+ snapshotManager->setCommittedSnapshot(snapCat);
ASSERT_EQ(itCountCommitted(), 1);
ASSERT_EQ(readStringCommitted(id), "Cat");
- snapshotManager->setCommittedSnapshot(snapAfterDelete, Timestamp(snapAfterDelete.asU64()));
+ snapshotManager->setCommittedSnapshot(snapAfterDelete);
ASSERT_EQ(itCountCommitted(), 0);
ASSERT(!readRecordCommitted(id));
}
diff --git a/src/mongo/db/storage/kv/kv_storage_engine.cpp b/src/mongo/db/storage/kv/kv_storage_engine.cpp
index 55d03fd0b52..63f589fcd7d 100644
--- a/src/mongo/db/storage/kv/kv_storage_engine.cpp
+++ b/src/mongo/db/storage/kv/kv_storage_engine.cpp
@@ -364,11 +364,11 @@ void KVStorageEngine::setJournalListener(JournalListener* jl) {
_engine->setJournalListener(jl);
}
-void KVStorageEngine::setStableTimestamp(SnapshotName stableTimestamp) {
+void KVStorageEngine::setStableTimestamp(Timestamp stableTimestamp) {
_engine->setStableTimestamp(stableTimestamp);
}
-void KVStorageEngine::setInitialDataTimestamp(SnapshotName initialDataTimestamp) {
+void KVStorageEngine::setInitialDataTimestamp(Timestamp initialDataTimestamp) {
_engine->setInitialDataTimestamp(initialDataTimestamp);
}
diff --git a/src/mongo/db/storage/kv/kv_storage_engine.h b/src/mongo/db/storage/kv/kv_storage_engine.h
index 580e1c7d069..5ecf45aaf04 100644
--- a/src/mongo/db/storage/kv/kv_storage_engine.h
+++ b/src/mongo/db/storage/kv/kv_storage_engine.h
@@ -33,6 +33,7 @@
#include "mongo/base/status_with.h"
#include "mongo/base/string_data.h"
+#include "mongo/bson/timestamp.h"
#include "mongo/db/storage/journal_listener.h"
#include "mongo/db/storage/kv/kv_catalog.h"
#include "mongo/db/storage/kv/kv_database_catalog_entry_base.h"
@@ -112,9 +113,9 @@ public:
virtual void cleanShutdown();
- virtual void setStableTimestamp(SnapshotName stableTimestamp) override;
+ virtual void setStableTimestamp(Timestamp stableTimestamp) override;
- virtual void setInitialDataTimestamp(SnapshotName initialDataTimestamp) override;
+ virtual void setInitialDataTimestamp(Timestamp initialDataTimestamp) override;
virtual bool supportsRecoverToStableTimestamp() const override;
diff --git a/src/mongo/db/storage/recovery_unit.h b/src/mongo/db/storage/recovery_unit.h
index 5ac712b4887..72b25fccb63 100644
--- a/src/mongo/db/storage/recovery_unit.h
+++ b/src/mongo/db/storage/recovery_unit.h
@@ -34,8 +34,8 @@
#include "mongo/base/disallow_copying.h"
#include "mongo/base/status.h"
+#include "mongo/bson/timestamp.h"
#include "mongo/db/storage/snapshot.h"
-#include "mongo/db/storage/snapshot_name.h"
namespace mongo {
@@ -120,13 +120,13 @@ public:
}
/**
- * Returns the SnapshotName being used by this recovery unit or boost::none if not reading from
+ * Returns the Timestamp being used by this recovery unit or boost::none if not reading from
* a majority committed snapshot.
*
* It is possible for reads to occur from later snapshots, but they may not occur from earlier
* snapshots.
*/
- virtual boost::optional<SnapshotName> getMajorityCommittedSnapshot() const {
+ virtual boost::optional<Timestamp> getMajorityCommittedSnapshot() const {
dassert(!isReadingFromMajorityCommittedSnapshot());
return {};
}
@@ -136,7 +136,7 @@ public:
*
* It is only valid to compare SnapshotIds generated by a single RecoveryUnit.
*
- * This is unrelated to SnapshotName which must be globally comparable.
+ * This is unrelated to Timestamp which must be globally comparable.
*/
virtual SnapshotId getSnapshotId() const = 0;
@@ -148,14 +148,14 @@ public:
* Writes that occur before any setTimestamp() is called will be assigned the timestamp
* specified in the last setTimestamp() call in the transaction, at commit time.
*/
- virtual Status setTimestamp(SnapshotName timestamp) {
+ virtual Status setTimestamp(Timestamp timestamp) {
return Status::OK();
}
/**
- * Chooses which snapshot to use for read transactions.
+ * Chooses which timestamp to use for read transactions.
*/
- virtual Status selectSnapshot(SnapshotName timestamp) {
+ virtual Status selectSnapshot(Timestamp timestamp) {
return Status(ErrorCodes::CommandNotSupported,
"point-in-time reads are not implemented for this storage engine");
}
diff --git a/src/mongo/db/storage/snapshot_manager.h b/src/mongo/db/storage/snapshot_manager.h
index 4ca7be39a7c..0577236a4ce 100644
--- a/src/mongo/db/storage/snapshot_manager.h
+++ b/src/mongo/db/storage/snapshot_manager.h
@@ -34,7 +34,6 @@
#include "mongo/base/status.h"
#include "mongo/bson/timestamp.h"
#include "mongo/db/operation_context.h"
-#include "mongo/db/storage/snapshot_name.h"
#include "mongo/util/assert_util.h"
namespace mongo {
@@ -70,7 +69,7 @@ public:
* can be done later. In particular, cleaning up of old snapshots should be deferred until
* cleanupUnneededSnapshots is called.
*/
- virtual void setCommittedSnapshot(const SnapshotName& name, Timestamp ts) = 0;
+ virtual void setCommittedSnapshot(const Timestamp& timestamp) = 0;
/**
* Cleans up all snapshots older than the current committed snapshot.
diff --git a/src/mongo/db/storage/snapshot_name.h b/src/mongo/db/storage/snapshot_name.h
deleted file mode 100644
index d9f8618c673..00000000000
--- a/src/mongo/db/storage/snapshot_name.h
+++ /dev/null
@@ -1,97 +0,0 @@
-/**
- * Copyright (C) 2015 MongoDB Inc.
- *
- * This program is free software: you can redistribute it and/or modify
- * it under the terms of the GNU Affero General Public License, version 3,
- * as published by the Free Software Foundation.
- *
- * This program is distributed in the hope that it will be useful,
- * but WITHOUT ANY WARRANTY; without even the implied warranty of
- * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
- * GNU Affero General Public License for more details.
- *
- * You should have received a copy of the GNU Affero General Public License
- * along with this program. If not, see <http://www.gnu.org/licenses/>.
- *
- * As a special exception, the copyright holders give permission to link the
- * code of portions of this program with the OpenSSL library under certain
- * conditions as described in each individual source file and distribute
- * linked combinations including the program with the OpenSSL library. You
- * must comply with the GNU Affero General Public License in all respects for
- * all of the code used other than as permitted herein. If you modify file(s)
- * with this exception, you may extend this exception to your version of the
- * file(s), but you are not obligated to do so. If you do not wish to do so,
- * delete this exception statement from your version. If you delete this
- * exception statement from all source files in the program, then also delete
- * it in the license file.
- */
-
-#pragma once
-
-#include <cstdint>
-#include <limits>
-#include <ostream>
-
-#include "mongo/bson/timestamp.h"
-#include "mongo/util/hex.h"
-
-namespace mongo {
-
-class SnapshotName {
-public:
- explicit SnapshotName(uint64_t value) : _value(value) {}
- explicit SnapshotName(Timestamp timestamp) : _value(timestamp.asULL()) {}
- SnapshotName() : _value(0) {}
-
- /**
- * Returns a SnapshotName guaranteed to compare < all names of actual snapshots.
- */
- static SnapshotName min() {
- return SnapshotName(std::numeric_limits<uint64_t>::min());
- }
-
- /**
- * Returns a SnapshotName guaranteed to compare > all names of actual snapshots.
- */
- static SnapshotName max() {
- return SnapshotName(std::numeric_limits<uint64_t>::max());
- }
-
- /**
- * Returns an unsigned number that compares with the same ordering as the SnapshotName.
- */
- uint64_t asU64() const {
- return _value;
- }
-
- std::string toString() const {
- return integerToHex(_value);
- }
-
- bool operator==(const SnapshotName& rhs) const {
- return _value == rhs._value;
- }
- bool operator!=(const SnapshotName& rhs) const {
- return _value != rhs._value;
- }
- bool operator<(const SnapshotName& rhs) const {
- return _value < rhs._value;
- }
- bool operator<=(const SnapshotName& rhs) const {
- return _value <= rhs._value;
- }
- bool operator>(const SnapshotName& rhs) const {
- return _value > rhs._value;
- }
- bool operator>=(const SnapshotName& rhs) const {
- return _value >= rhs._value;
- }
-
- friend std::ostream& operator<<(std::ostream& out, const SnapshotName& snapshotName) {
- return out << snapshotName.toString();
- }
-
-private:
- uint64_t _value;
-};
-}
diff --git a/src/mongo/db/storage/storage_engine.h b/src/mongo/db/storage/storage_engine.h
index 7af4e45b942..fe6a22dab44 100644
--- a/src/mongo/db/storage/storage_engine.h
+++ b/src/mongo/db/storage/storage_engine.h
@@ -35,7 +35,7 @@
#include "mongo/base/status.h"
#include "mongo/bson/bsonobj.h"
-#include "mongo/db/storage/snapshot_name.h"
+#include "mongo/bson/timestamp.h"
#include "mongo/util/mongoutils/str.h"
namespace mongo {
@@ -320,13 +320,13 @@ public:
* Sets the highest timestamp at which the storage engine is allowed to take a checkpoint.
* This timestamp can never decrease, and thus should be a timestamp that can never roll back.
*/
- virtual void setStableTimestamp(SnapshotName snapshotName) {}
+ virtual void setStableTimestamp(Timestamp timestamp) {}
/**
* Tells the storage engine the timestamp of the data at startup. This is necessary because
* timestamps are not persisted in the storage layer.
*/
- virtual void setInitialDataTimestamp(SnapshotName snapshotName) {}
+ virtual void setInitialDataTimestamp(Timestamp timestamp) {}
/**
* Notifies the storage engine that a replication batch has completed.
diff --git a/src/mongo/db/storage/storage_snapshot_name_test.cpp b/src/mongo/db/storage/storage_snapshot_name_test.cpp
deleted file mode 100644
index b4b7bebb33f..00000000000
--- a/src/mongo/db/storage/storage_snapshot_name_test.cpp
+++ /dev/null
@@ -1,48 +0,0 @@
-/**
- * Copyright (C) 2017 MongoDB Inc.
- *
- * This program is free software: you can redistribute it and/or modify
- * it under the terms of the GNU Affero General Public License, version 3,
- * as published by the Free Software Foundation.
- *
- * This program is distributed in the hope that it will be useful,
- * but WITHOUT ANY WARRANTY; without even the implied warranty of
- * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
- * GNU Affero General Public License for more details.
- *
- * You should have received a copy of the GNU Affero General Public License
- * along with this program. If not, see <http://www.gnu.org/licenses/>.
- *
- * As a special exception, the copyright holders give permission to link the
- * code of portions of this program with the OpenSSL library under certain
- * conditions as described in each individual source file and distribute
- * linked combinations including the program with the OpenSSL library. You
- * must comply with the GNU Affero General Public License in all respects for
- * all of the code used other than as permitted herein. If you modify file(s)
- * with this exception, you may extend this exception to your version of the
- * file(s), but you are not obligated to do so. If you do not wish to do so,
- * delete this exception statement from your version. If you delete this
- * exception statement from all source files in the program, then also delete
- * it in the license file.
- */
-
-#include "mongo/platform/basic.h"
-
-#include <utility>
-#include <vector>
-
-#include "mongo/unittest/unittest.h"
-#include "snapshot_name.h"
-
-namespace mongo {
-TEST(SnapshotNameTest, TestToString) {
- std::vector<std::pair<uint64_t, std::string>> tests = {
- {1234, "4D2"},
- {1501697192, "598214A8"},
- {6449740328135032833, "598214A800000001"},
- {6449740328135037744, "598214A800001330"}};
- for (auto pair : tests) {
- ASSERT_EQUALS(pair.second, SnapshotName(pair.first).toString());
- }
-}
-} // namespace mongo
diff --git a/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.cpp b/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.cpp
index 509b6864744..efee60428e4 100644
--- a/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.cpp
+++ b/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.cpp
@@ -163,8 +163,8 @@ public:
wiredTigerGlobalOptions.checkpointDelaySecs)));
}
- const SnapshotName stableTimestamp(_stableTimestamp.load());
- const SnapshotName initialDataTimestamp(_initialDataTimestamp.load());
+ const Timestamp stableTimestamp(_stableTimestamp.load());
+ const Timestamp initialDataTimestamp(_initialDataTimestamp.load());
const bool keepOldBehavior = true;
try {
@@ -186,7 +186,7 @@ public:
//
// Third, stableTimestamp >= initialDataTimestamp: Take stable
// checkpoint. Steady state case.
- if (initialDataTimestamp.asU64() <= 1) {
+ if (initialDataTimestamp.asULL() <= 1) {
const bool forceCheckpoint = true;
const bool stableCheckpoint = false;
_sessionCache->waitUntilDurable(forceCheckpoint, stableCheckpoint);
@@ -234,12 +234,12 @@ public:
return _stableTimestamp.load() > initialDataTimestamp;
}
- void setStableTimestamp(SnapshotName stableTimestamp) {
- _stableTimestamp.store(stableTimestamp.asU64());
+ void setStableTimestamp(Timestamp stableTimestamp) {
+ _stableTimestamp.store(stableTimestamp.asULL());
}
- void setInitialDataTimestamp(SnapshotName initialDataTimestamp) {
- _initialDataTimestamp.store(initialDataTimestamp.asU64());
+ void setInitialDataTimestamp(Timestamp initialDataTimestamp) {
+ _initialDataTimestamp.store(initialDataTimestamp.asULL());
}
void shutdown() {
@@ -995,7 +995,7 @@ bool WiredTigerKVEngine::initRsOplogBackgroundThread(StringData ns) {
return initRsOplogBackgroundThreadCallback(ns);
}
-void WiredTigerKVEngine::setStableTimestamp(SnapshotName stableTimestamp) {
+void WiredTigerKVEngine::setStableTimestamp(Timestamp stableTimestamp) {
const bool keepOldBehavior = true;
// Communicate to WiredTiger what the "stable timestamp" is. Timestamp-aware checkpoints will
// only persist to disk transactions committed with a timestamp earlier than the "stable
@@ -1025,8 +1025,8 @@ void WiredTigerKVEngine::setStableTimestamp(SnapshotName stableTimestamp) {
_setOldestTimestamp(stableTimestamp);
}
-void WiredTigerKVEngine::_setOldestTimestamp(SnapshotName oldestTimestamp) {
- if (oldestTimestamp == SnapshotName()) {
+void WiredTigerKVEngine::_setOldestTimestamp(Timestamp oldestTimestamp) {
+ if (oldestTimestamp == Timestamp()) {
// No oldestTimestamp to set, yet.
return;
}
@@ -1036,7 +1036,7 @@ void WiredTigerKVEngine::_setOldestTimestamp(SnapshotName oldestTimestamp) {
// No oplog yet, so don't bother setting oldest_timestamp.
return;
}
- if (_oplogManager->getOplogReadTimestamp() < oldestTimestamp.asU64()) {
+ if (_oplogManager->getOplogReadTimestamp() < oldestTimestamp.asULL()) {
// For one node replica sets, the commit point might race ahead of the oplog read
// timestamp.
// For now, we will simply avoid setting the oldestTimestamp in such cases.
@@ -1045,7 +1045,7 @@ void WiredTigerKVEngine::_setOldestTimestamp(SnapshotName oldestTimestamp) {
}
auto timestampToSet = _previousSetOldestTimestamp;
_previousSetOldestTimestamp = oldestTimestamp;
- if (timestampToSet == SnapshotName()) {
+ if (timestampToSet == Timestamp()) {
// Nothing to set yet.
return;
}
@@ -1055,7 +1055,7 @@ void WiredTigerKVEngine::_setOldestTimestamp(SnapshotName oldestTimestamp) {
auto size = std::snprintf(oldestTSConfigString,
sizeof(oldestTSConfigString),
"oldest_timestamp=%llx",
- static_cast<unsigned long long>(timestampToSet.asU64()));
+ timestampToSet.asULL());
if (size < 0) {
int e = errno;
error() << "error snprintf " << errnoWithDescription(e);
@@ -1063,10 +1063,10 @@ void WiredTigerKVEngine::_setOldestTimestamp(SnapshotName oldestTimestamp) {
}
invariant(static_cast<std::size_t>(size) < sizeof(oldestTSConfigString));
invariantWTOK(_conn->set_timestamp(_conn, oldestTSConfigString));
- LOG(2) << "oldest_timestamp set to " << timestampToSet.asU64();
+ LOG(2) << "oldest_timestamp set to " << timestampToSet;
}
-void WiredTigerKVEngine::setInitialDataTimestamp(SnapshotName initialDataTimestamp) {
+void WiredTigerKVEngine::setInitialDataTimestamp(Timestamp initialDataTimestamp) {
if (_checkpointThread) {
_checkpointThread->setInitialDataTimestamp(initialDataTimestamp);
}
diff --git a/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.h b/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.h
index b34d12146f3..5f7ca29e13f 100644
--- a/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.h
+++ b/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.h
@@ -38,6 +38,7 @@
#include <wiredtiger.h>
#include "mongo/bson/ordering.h"
+#include "mongo/bson/timestamp.h"
#include "mongo/db/storage/kv/kv_engine.h"
#include "mongo/db/storage/wiredtiger/wiredtiger_oplog_manager.h"
#include "mongo/db/storage/wiredtiger/wiredtiger_session_cache.h"
@@ -162,9 +163,9 @@ public:
void setJournalListener(JournalListener* jl) final;
- virtual void setStableTimestamp(SnapshotName stableTimestamp) override;
+ virtual void setStableTimestamp(Timestamp stableTimestamp) override;
- virtual void setInitialDataTimestamp(SnapshotName initialDataTimestamp) override;
+ virtual void setInitialDataTimestamp(Timestamp initialDataTimestamp) override;
virtual bool supportsRecoverToStableTimestamp() const override;
@@ -246,8 +247,8 @@ private:
std::string _uri(StringData ident) const;
// Not threadsafe; callers must be serialized.
- void _setOldestTimestamp(SnapshotName oldestTimestamp);
- SnapshotName _previousSetOldestTimestamp;
+ void _setOldestTimestamp(Timestamp oldestTimestamp);
+ Timestamp _previousSetOldestTimestamp;
WT_CONNECTION* _conn;
WT_EVENT_HANDLER _eventHandler;
diff --git a/src/mongo/db/storage/wiredtiger/wiredtiger_oplog_manager.cpp b/src/mongo/db/storage/wiredtiger/wiredtiger_oplog_manager.cpp
index b270fe8bcdc..8b8c130cba0 100644
--- a/src/mongo/db/storage/wiredtiger/wiredtiger_oplog_manager.cpp
+++ b/src/mongo/db/storage/wiredtiger/wiredtiger_oplog_manager.cpp
@@ -199,7 +199,7 @@ void WiredTigerOplogManager::_oplogJournalThreadLoop(WiredTigerSessionCache* ses
// For master/slave masters, set oldest timestamp here so that we clean up old timestamp
// data. SERVER-31802
if (isMasterSlave) {
- sessionCache->getKVEngine()->setStableTimestamp(SnapshotName(newTimestamp));
+ sessionCache->getKVEngine()->setStableTimestamp(Timestamp(newTimestamp));
}
}
}
diff --git a/src/mongo/db/storage/wiredtiger/wiredtiger_record_store.cpp b/src/mongo/db/storage/wiredtiger/wiredtiger_record_store.cpp
index 336b7d9d486..ac302cb4921 100644
--- a/src/mongo/db/storage/wiredtiger/wiredtiger_record_store.cpp
+++ b/src/mongo/db/storage/wiredtiger/wiredtiger_record_store.cpp
@@ -1146,8 +1146,8 @@ Status WiredTigerRecordStore::_insertRecords(OperationContext* opCtx,
ts = timestamps[i];
}
if (!ts.isNull()) {
- LOG(4) << "inserting record with timestamp " << ts.asULL();
- fassertStatusOK(39001, opCtx->recoveryUnit()->setTimestamp(SnapshotName(ts)));
+ LOG(4) << "inserting record with timestamp " << ts;
+ fassertStatusOK(39001, opCtx->recoveryUnit()->setTimestamp(ts));
}
setKey(c, record.id);
WiredTigerItem value(record.data.data(), record.data.size());
@@ -1678,7 +1678,7 @@ Status WiredTigerRecordStore::oplogDiskLocRegister(OperationContext* opCtx,
// This labels the current transaction with a timestamp.
// This is required for oplog visibility to work correctly, as WiredTiger uses the transaction
// list to determine where there are holes in the oplog.
- return opCtx->recoveryUnit()->setTimestamp(SnapshotName(opTime));
+ return opCtx->recoveryUnit()->setTimestamp(opTime);
}
// Cursor Base:
diff --git a/src/mongo/db/storage/wiredtiger/wiredtiger_recovery_unit.cpp b/src/mongo/db/storage/wiredtiger/wiredtiger_recovery_unit.cpp
index 54db183fe36..fa0d2d2b66d 100644
--- a/src/mongo/db/storage/wiredtiger/wiredtiger_recovery_unit.cpp
+++ b/src/mongo/db/storage/wiredtiger/wiredtiger_recovery_unit.cpp
@@ -39,6 +39,7 @@
#include "mongo/db/storage/wiredtiger/wiredtiger_kv_engine.h"
#include "mongo/db/storage/wiredtiger/wiredtiger_session_cache.h"
#include "mongo/db/storage/wiredtiger/wiredtiger_util.h"
+#include "mongo/util/hex.h"
#include "mongo/util/log.h"
namespace mongo {
@@ -237,7 +238,7 @@ Status WiredTigerRecoveryUnit::setReadFromMajorityCommittedSnapshot() {
return Status::OK();
}
-boost::optional<SnapshotName> WiredTigerRecoveryUnit::getMajorityCommittedSnapshot() const {
+boost::optional<Timestamp> WiredTigerRecoveryUnit::getMajorityCommittedSnapshot() const {
if (!_readFromMajorityCommittedSnapshot)
return {};
return _majorityCommittedSnapshot;
@@ -253,7 +254,7 @@ void WiredTigerRecoveryUnit::_txnOpen() {
}
WT_SESSION* session = _session->getSession();
- if (_readAtTimestamp != SnapshotName::min()) {
+ if (_readAtTimestamp != Timestamp::min()) {
uassertStatusOK(_sessionCache->snapshotManager().beginTransactionAtTimestamp(
_readAtTimestamp, session));
} else if (_readFromMajorityCommittedSnapshot) {
@@ -271,7 +272,7 @@ void WiredTigerRecoveryUnit::_txnOpen() {
}
-Status WiredTigerRecoveryUnit::setTimestamp(SnapshotName timestamp) {
+Status WiredTigerRecoveryUnit::setTimestamp(Timestamp timestamp) {
_ensureSession();
LOG(3) << "WT set timestamp of future write operations to " << timestamp;
WT_SESSION* session = _session->getSession();
@@ -280,7 +281,7 @@ Status WiredTigerRecoveryUnit::setTimestamp(SnapshotName timestamp) {
// Starts the WT transaction associated with this session.
getSession();
- const std::string conf = "commit_timestamp=" + timestamp.toString();
+ const std::string conf = "commit_timestamp=" + integerToHex(timestamp.asULL());
auto rc = session->timestamp_transaction(session, conf.c_str());
if (rc == 0) {
_isTimestamped = true;
@@ -288,7 +289,7 @@ Status WiredTigerRecoveryUnit::setTimestamp(SnapshotName timestamp) {
return wtRCToStatus(rc, "timestamp_transaction");
}
-Status WiredTigerRecoveryUnit::selectSnapshot(SnapshotName timestamp) {
+Status WiredTigerRecoveryUnit::selectSnapshot(Timestamp timestamp) {
_readAtTimestamp = timestamp;
return Status::OK();
}
diff --git a/src/mongo/db/storage/wiredtiger/wiredtiger_recovery_unit.h b/src/mongo/db/storage/wiredtiger/wiredtiger_recovery_unit.h
index a1040952dc3..5da0d2987fb 100644
--- a/src/mongo/db/storage/wiredtiger/wiredtiger_recovery_unit.h
+++ b/src/mongo/db/storage/wiredtiger/wiredtiger_recovery_unit.h
@@ -36,10 +36,10 @@
#include <vector>
#include "mongo/base/checked_cast.h"
+#include "mongo/bson/timestamp.h"
#include "mongo/db/operation_context.h"
#include "mongo/db/record_id.h"
#include "mongo/db/storage/recovery_unit.h"
-#include "mongo/db/storage/snapshot_name.h"
#include "mongo/db/storage/wiredtiger/wiredtiger_session_cache.h"
#include "mongo/util/timer.h"
@@ -77,13 +77,13 @@ public:
return _readFromMajorityCommittedSnapshot;
}
- boost::optional<SnapshotName> getMajorityCommittedSnapshot() const override;
+ boost::optional<Timestamp> getMajorityCommittedSnapshot() const override;
SnapshotId getSnapshotId() const override;
- Status setTimestamp(SnapshotName timestamp) override;
+ Status setTimestamp(Timestamp timestamp) override;
- Status selectSnapshot(SnapshotName timestamp) override;
+ Status selectSnapshot(Timestamp timestamp) override;
void* writingPtr(void* data, size_t len) override;
@@ -148,8 +148,8 @@ private:
bool _isTimestamped = false;
uint64_t _mySnapshotId;
bool _readFromMajorityCommittedSnapshot = false;
- SnapshotName _majorityCommittedSnapshot = SnapshotName::min();
- SnapshotName _readAtTimestamp = SnapshotName::min();
+ Timestamp _majorityCommittedSnapshot;
+ Timestamp _readAtTimestamp;
std::unique_ptr<Timer> _timer;
bool _isOplogReader = false;
typedef std::vector<std::unique_ptr<Change>> Changes;
diff --git a/src/mongo/db/storage/wiredtiger/wiredtiger_snapshot_manager.cpp b/src/mongo/db/storage/wiredtiger/wiredtiger_snapshot_manager.cpp
index a666ad6d5cb..a554015102b 100644
--- a/src/mongo/db/storage/wiredtiger/wiredtiger_snapshot_manager.cpp
+++ b/src/mongo/db/storage/wiredtiger/wiredtiger_snapshot_manager.cpp
@@ -50,11 +50,11 @@ Status WiredTigerSnapshotManager::prepareForCreateSnapshot(OperationContext* opC
return Status::OK();
}
-void WiredTigerSnapshotManager::setCommittedSnapshot(const SnapshotName& name, Timestamp ts) {
+void WiredTigerSnapshotManager::setCommittedSnapshot(const Timestamp& timestamp) {
stdx::lock_guard<stdx::mutex> lock(_mutex);
- invariant(!_committedSnapshot || *_committedSnapshot <= name);
- _committedSnapshot = name;
+ invariant(!_committedSnapshot || *_committedSnapshot <= timestamp);
+ _committedSnapshot = timestamp;
}
void WiredTigerSnapshotManager::cleanupUnneededSnapshots() {}
@@ -72,20 +72,17 @@ void WiredTigerSnapshotManager::shutdown() {
_session = nullptr;
}
-boost::optional<SnapshotName> WiredTigerSnapshotManager::getMinSnapshotForNextCommittedRead()
- const {
+boost::optional<Timestamp> WiredTigerSnapshotManager::getMinSnapshotForNextCommittedRead() const {
stdx::lock_guard<stdx::mutex> lock(_mutex);
return _committedSnapshot;
}
-Status WiredTigerSnapshotManager::beginTransactionAtTimestamp(SnapshotName pointInTime,
+Status WiredTigerSnapshotManager::beginTransactionAtTimestamp(Timestamp pointInTime,
WT_SESSION* session) const {
char readTSConfigString[15 /* read_timestamp= */ + (8 * 2) /* 8 hexadecimal characters */ +
1 /* trailing null */];
- auto size = std::snprintf(readTSConfigString,
- sizeof(readTSConfigString),
- "read_timestamp=%llx",
- static_cast<unsigned long long>(pointInTime.asU64()));
+ auto size = std::snprintf(
+ readTSConfigString, sizeof(readTSConfigString), "read_timestamp=%llx", pointInTime.asULL());
if (size < 0) {
int e = errno;
error() << "error snprintf " << errnoWithDescription(e);
@@ -96,13 +93,14 @@ Status WiredTigerSnapshotManager::beginTransactionAtTimestamp(SnapshotName point
return wtRCToStatus(session->begin_transaction(session, readTSConfigString));
}
-SnapshotName WiredTigerSnapshotManager::beginTransactionOnCommittedSnapshot(
+Timestamp WiredTigerSnapshotManager::beginTransactionOnCommittedSnapshot(
WT_SESSION* session) const {
stdx::lock_guard<stdx::mutex> lock(_mutex);
uassert(ErrorCodes::ReadConcernMajorityNotAvailableYet,
"Committed view disappeared while running operation",
_committedSnapshot);
+
auto status = beginTransactionAtTimestamp(_committedSnapshot.get(), session);
fassertStatusOK(30635, status);
return *_committedSnapshot;
diff --git a/src/mongo/db/storage/wiredtiger/wiredtiger_snapshot_manager.h b/src/mongo/db/storage/wiredtiger/wiredtiger_snapshot_manager.h
index 2609b396e24..b3853206bcc 100644
--- a/src/mongo/db/storage/wiredtiger/wiredtiger_snapshot_manager.h
+++ b/src/mongo/db/storage/wiredtiger/wiredtiger_snapshot_manager.h
@@ -33,6 +33,7 @@
#include <wiredtiger.h>
#include "mongo/base/disallow_copying.h"
+#include "mongo/bson/timestamp.h"
#include "mongo/db/storage/snapshot_manager.h"
#include "mongo/db/storage/wiredtiger/wiredtiger_util.h"
#include "mongo/stdx/mutex.h"
@@ -55,7 +56,7 @@ public:
}
Status prepareForCreateSnapshot(OperationContext* opCtx) final;
- void setCommittedSnapshot(const SnapshotName& name, Timestamp ts) final;
+ void setCommittedSnapshot(const Timestamp& timestamp) final;
void cleanupUnneededSnapshots() final;
void dropAllSnapshots() final;
@@ -68,14 +69,14 @@ public:
*/
void shutdown();
- Status beginTransactionAtTimestamp(SnapshotName pointInTime, WT_SESSION* session) const;
+ Status beginTransactionAtTimestamp(Timestamp pointInTime, WT_SESSION* session) const;
/**
* Starts a transaction and returns the SnapshotName used.
*
* Throws if there is currently no committed snapshot.
*/
- SnapshotName beginTransactionOnCommittedSnapshot(WT_SESSION* session) const;
+ Timestamp beginTransactionOnCommittedSnapshot(WT_SESSION* session) const;
/**
* Starts a transaction on the oplog using an appropriate timestamp for oplog visiblity.
@@ -90,11 +91,11 @@ public:
* This should not be used for starting a transaction on this SnapshotName since the named
* snapshot may be deleted by the time you start the transaction.
*/
- boost::optional<SnapshotName> getMinSnapshotForNextCommittedRead() const;
+ boost::optional<Timestamp> getMinSnapshotForNextCommittedRead() const;
private:
mutable stdx::mutex _mutex; // Guards all members.
- boost::optional<SnapshotName> _committedSnapshot;
+ boost::optional<Timestamp> _committedSnapshot;
WT_SESSION* _session;
WT_CONNECTION* _conn;
};
diff --git a/src/mongo/dbtests/storage_timestamp_tests.cpp b/src/mongo/dbtests/storage_timestamp_tests.cpp
index c6ba60584dc..7e15647cf05 100644
--- a/src/mongo/dbtests/storage_timestamp_tests.cpp
+++ b/src/mongo/dbtests/storage_timestamp_tests.cpp
@@ -31,6 +31,7 @@
#include <cstdint>
#include "mongo/bson/simple_bsonobj_comparator.h"
+#include "mongo/bson/timestamp.h"
#include "mongo/db/catalog/collection.h"
#include "mongo/db/catalog/index_catalog.h"
#include "mongo/db/catalog/index_create.h"
@@ -107,7 +108,7 @@ public:
*/
void reset(NamespaceString nss) const {
::mongo::writeConflictRetry(_opCtx, "deleteAll", nss.ns(), [&] {
- invariant(_opCtx->recoveryUnit()->selectSnapshot(SnapshotName::min()).isOK());
+ invariant(_opCtx->recoveryUnit()->selectSnapshot(Timestamp::min()).isOK());
AutoGetCollection collRaii(_opCtx, nss, LockMode::MODE_X);
if (collRaii.getCollection()) {
@@ -248,8 +249,7 @@ public:
for (std::uint32_t idx = 0; idx < docsToInsert; ++idx) {
auto recoveryUnit = _opCtx->recoveryUnit();
recoveryUnit->abandonSnapshot();
- ASSERT_OK(recoveryUnit->selectSnapshot(
- SnapshotName(firstInsertTime.addTicks(idx).asTimestamp())));
+ ASSERT_OK(recoveryUnit->selectSnapshot(firstInsertTime.addTicks(idx).asTimestamp()));
BSONObj result;
ASSERT(Helpers::getLast(_opCtx, nss.ns().c_str(), result)) << " idx is " << idx;
ASSERT_EQ(0, SimpleBSONObjComparator::kInstance.compare(result, BSON("_id" << idx)))
@@ -331,8 +331,7 @@ public:
for (std::uint32_t idx = 0; idx < docsToInsert; ++idx) {
auto recoveryUnit = _opCtx->recoveryUnit();
recoveryUnit->abandonSnapshot();
- ASSERT_OK(recoveryUnit->selectSnapshot(
- SnapshotName(firstInsertTime.addTicks(idx).asTimestamp())));
+ ASSERT_OK(recoveryUnit->selectSnapshot(firstInsertTime.addTicks(idx).asTimestamp()));
BSONObj result;
ASSERT(Helpers::getLast(_opCtx, nss.ns().c_str(), result)) << " idx is " << idx;
ASSERT_EQ(0, SimpleBSONObjComparator::kInstance.compare(result, BSON("_id" << idx)))
@@ -364,11 +363,10 @@ public:
const LogicalTime lastInsertTime = firstInsertTime.addTicks(docsToInsert - 1);
WriteUnitOfWork wunit(_opCtx);
for (std::int32_t num = 0; num < docsToInsert; ++num) {
- insertDocument(
- autoColl.getCollection(),
- InsertStatement(BSON("_id" << num << "a" << num),
- SnapshotName(firstInsertTime.addTicks(num).asTimestamp()),
- 0LL));
+ insertDocument(autoColl.getCollection(),
+ InsertStatement(BSON("_id" << num << "a" << num),
+ firstInsertTime.addTicks(num).asTimestamp(),
+ 0LL));
}
wunit.commit();
ASSERT_EQ(docsToInsert, itCount(autoColl.getCollection()));
@@ -400,8 +398,7 @@ public:
// at each successive tick counts one less document.
auto recoveryUnit = _opCtx->recoveryUnit();
recoveryUnit->abandonSnapshot();
- ASSERT_OK(recoveryUnit->selectSnapshot(
- SnapshotName(lastInsertTime.addTicks(num).asTimestamp())));
+ ASSERT_OK(recoveryUnit->selectSnapshot(lastInsertTime.addTicks(num).asTimestamp()));
ASSERT_EQ(docsToInsert - num, itCount(autoColl.getCollection()));
}
}
@@ -427,9 +424,8 @@ public:
// Insert one document that will go through a series of updates.
const LogicalTime insertTime = _clock->reserveTicks(1);
WriteUnitOfWork wunit(_opCtx);
- insertDocument(
- autoColl.getCollection(),
- InsertStatement(BSON("_id" << 0), SnapshotName(insertTime.asTimestamp()), 0LL));
+ insertDocument(autoColl.getCollection(),
+ InsertStatement(BSON("_id" << 0), insertTime.asTimestamp(), 0LL));
wunit.commit();
ASSERT_EQ(1, itCount(autoColl.getCollection()));
@@ -479,8 +475,7 @@ public:
// the series.
auto recoveryUnit = _opCtx->recoveryUnit();
recoveryUnit->abandonSnapshot();
- ASSERT_OK(recoveryUnit->selectSnapshot(
- SnapshotName(insertTime.addTicks(idx + 1).asTimestamp())));
+ ASSERT_OK(recoveryUnit->selectSnapshot(insertTime.addTicks(idx + 1).asTimestamp()));
auto doc = findOne(autoColl.getCollection());
ASSERT_EQ(0, SimpleBSONObjComparator::kInstance.compare(doc, updates[idx].second))
@@ -546,7 +541,7 @@ public:
// Reading at `insertTime` should show the original document, `{_id: 0, field: 0}`.
auto recoveryUnit = _opCtx->recoveryUnit();
recoveryUnit->abandonSnapshot();
- ASSERT_OK(recoveryUnit->selectSnapshot(SnapshotName(insertTime.asTimestamp())));
+ ASSERT_OK(recoveryUnit->selectSnapshot(insertTime.asTimestamp()));
auto doc = findOne(autoColl.getCollection());
ASSERT_EQ(0,
SimpleBSONObjComparator::kInstance.compare(doc, BSON("_id" << 0 << "field" << 0)))
@@ -555,7 +550,7 @@ public:
// Reading at `insertTime + 1` should show the second insert that got converted to an
// upsert, `{_id: 0}`.
recoveryUnit->abandonSnapshot();
- ASSERT_OK(recoveryUnit->selectSnapshot(SnapshotName(insertTime.addTicks(1).asTimestamp())));
+ ASSERT_OK(recoveryUnit->selectSnapshot(insertTime.addTicks(1).asTimestamp()));
doc = findOne(autoColl.getCollection());
ASSERT_EQ(0, SimpleBSONObjComparator::kInstance.compare(doc, BSON("_id" << 0)))
<< "Doc: " << doc.toString() << " Expected: {_id: 0}";
@@ -604,7 +599,7 @@ public:
// Reading at `preInsertTimestamp` should not find anything.
auto recoveryUnit = _opCtx->recoveryUnit();
recoveryUnit->abandonSnapshot();
- ASSERT_OK(recoveryUnit->selectSnapshot(SnapshotName(preInsertTimestamp.asTimestamp())));
+ ASSERT_OK(recoveryUnit->selectSnapshot(preInsertTimestamp.asTimestamp()));
ASSERT_EQ(0, itCount(autoColl.getCollection()))
<< "Should not observe a write at `preInsertTimestamp`. TS: "
<< preInsertTimestamp.asTimestamp();
@@ -612,8 +607,7 @@ public:
// Reading at `preInsertTimestamp + 1` should observe both inserts.
recoveryUnit = _opCtx->recoveryUnit();
recoveryUnit->abandonSnapshot();
- ASSERT_OK(recoveryUnit->selectSnapshot(
- SnapshotName(preInsertTimestamp.addTicks(1).asTimestamp())));
+ ASSERT_OK(recoveryUnit->selectSnapshot(preInsertTimestamp.addTicks(1).asTimestamp()));
ASSERT_EQ(2, itCount(autoColl.getCollection()))
<< "Should observe both writes at `preInsertTimestamp + 1`. TS: "
<< preInsertTimestamp.addTicks(1).asTimestamp();
@@ -667,15 +661,14 @@ public:
// Reading at `insertTime` should not see any documents.
auto recoveryUnit = _opCtx->recoveryUnit();
recoveryUnit->abandonSnapshot();
- ASSERT_OK(recoveryUnit->selectSnapshot(SnapshotName(preInsertTimestamp.asTimestamp())));
+ ASSERT_OK(recoveryUnit->selectSnapshot(preInsertTimestamp.asTimestamp()));
ASSERT_EQ(0, itCount(autoColl.getCollection()))
<< "Should not find any documents at `preInsertTimestamp`. TS: "
<< preInsertTimestamp.asTimestamp();
// Reading at `preInsertTimestamp + 1` should show the final state of the document.
recoveryUnit->abandonSnapshot();
- ASSERT_OK(recoveryUnit->selectSnapshot(
- SnapshotName(preInsertTimestamp.addTicks(1).asTimestamp())));
+ ASSERT_OK(recoveryUnit->selectSnapshot(preInsertTimestamp.addTicks(1).asTimestamp()));
auto doc = findOne(autoColl.getCollection());
ASSERT_EQ(0, SimpleBSONObjComparator::kInstance.compare(doc, BSON("_id" << 0)))
<< "Doc: " << doc.toString() << " Expected: {_id: 0}";