From 9e1d37df7bf5bb4c8312f155bd671214f75ea296 Mon Sep 17 00:00:00 2001 From: Allison Easton Date: Tue, 21 Sep 2021 13:39:28 +0000 Subject: SERVER-52847 Make timestamp required in CollectionType and ShardCollectionType IDL --- jstests/core/views/views_all_commands.js | 2 +- jstests/sharding/check_sharding_index_versioned.js | 2 +- jstests/sharding/version1.js | 11 +- jstests/sharding/version2.js | 2 + .../db/pipeline/dispatch_shard_pipeline_test.cpp | 5 +- .../mongos_process_interface_test.cpp | 2 +- .../standalone_process_interface_test.cpp | 4 +- src/mongo/db/pipeline/sharded_union_test.cpp | 16 +- src/mongo/db/s/active_migrations_registry_test.cpp | 2 +- src/mongo/db/s/balancer/balance_stats_test.cpp | 7 +- src/mongo/db/s/balancer/balancer_policy_test.cpp | 2 +- .../s/balancer/scoped_migration_request_test.cpp | 2 +- src/mongo/db/s/balancer/type_migration_test.cpp | 14 +- .../db/s/collection_metadata_filtering_test.cpp | 4 +- src/mongo/db/s/collection_metadata_test.cpp | 4 +- .../db/s/collection_sharding_runtime_test.cpp | 24 +- .../db/s/config/config_server_test_fixture.cpp | 21 +- src/mongo/db/s/config/config_server_test_fixture.h | 4 +- .../db/s/config/initial_split_policy_test.cpp | 2 +- src/mongo/db/s/config/sharding_catalog_manager.h | 2 +- ...talog_manager_assign_key_range_to_zone_test.cpp | 17 +- .../sharding_catalog_manager_chunk_operations.cpp | 50 +-- ...rding_catalog_manager_clear_jumbo_flag_test.cpp | 18 +- ...rding_catalog_manager_collection_operations.cpp | 32 +- ...r_ensure_chunk_version_is_greater_than_test.cpp | 13 +- .../sharding_catalog_manager_merge_chunks_test.cpp | 16 +- .../sharding_catalog_manager_remove_shard_test.cpp | 14 +- .../sharding_catalog_manager_split_chunk_test.cpp | 29 +- src/mongo/db/s/metadata_manager_test.cpp | 5 +- .../migration_chunk_cloner_source_legacy_test.cpp | 7 +- src/mongo/db/s/migration_util_test.cpp | 32 +- src/mongo/db/s/op_observer_sharding_test.cpp | 8 +- src/mongo/db/s/range_deletion_util_test.cpp | 4 +- .../resharding/resharding_coordinator_service.cpp | 1 - .../resharding_coordinator_service_test.cpp | 89 +++-- .../resharding_data_replication_test.cpp | 4 +- .../resharding_donor_recipient_common_test.h | 4 +- .../s/resharding/resharding_oplog_applier_test.cpp | 8 +- .../resharding_oplog_batch_applier_test.cpp | 4 +- .../resharding_oplog_crud_application_test.cpp | 8 +- ...rding_recipient_service_external_state_test.cpp | 72 ++-- .../resharding_recipient_service_test.cpp | 4 +- .../db/s/resharding_destined_recipient_test.cpp | 26 +- src/mongo/db/s/shard_metadata_util.cpp | 2 +- src/mongo/db/s/shard_metadata_util.h | 2 +- src/mongo/db/s/shard_metadata_util_test.cpp | 7 +- .../db/s/shard_server_catalog_cache_loader.cpp | 2 +- .../s/shard_server_catalog_cache_loader_test.cpp | 12 +- src/mongo/db/s/sharding_ddl_util_test.cpp | 4 +- src/mongo/db/s/type_shard_collection.cpp | 17 +- src/mongo/db/s/type_shard_collection.h | 6 +- src/mongo/db/s/type_shard_collection.idl | 6 +- src/mongo/db/s/type_shard_collection_test.cpp | 16 +- src/mongo/s/catalog/sharding_catalog_client.h | 2 +- .../s/catalog/sharding_catalog_client_impl.cpp | 2 +- src/mongo/s/catalog/sharding_catalog_client_impl.h | 2 +- .../s/catalog/sharding_catalog_client_mock.cpp | 2 +- src/mongo/s/catalog/sharding_catalog_client_mock.h | 2 +- .../s/catalog/sharding_catalog_client_test.cpp | 27 +- .../catalog/sharding_catalog_write_retry_test.cpp | 2 +- src/mongo/s/catalog/type_chunk.cpp | 10 +- src/mongo/s/catalog/type_chunk.h | 4 +- src/mongo/s/catalog/type_chunk_test.cpp | 55 +-- src/mongo/s/catalog/type_collection.cpp | 16 +- src/mongo/s/catalog/type_collection.h | 9 +- src/mongo/s/catalog/type_collection.idl | 6 +- src/mongo/s/catalog/type_collection_test.cpp | 48 +-- src/mongo/s/catalog_cache.cpp | 18 +- src/mongo/s/catalog_cache_loader.cpp | 2 +- src/mongo/s/catalog_cache_loader.h | 4 +- src/mongo/s/catalog_cache_refresh_test.cpp | 101 +++--- src/mongo/s/catalog_cache_test.cpp | 168 +-------- src/mongo/s/catalog_cache_test_fixture.cpp | 13 +- src/mongo/s/catalog_cache_test_fixture.h | 2 + src/mongo/s/chunk_manager.cpp | 63 +--- src/mongo/s/chunk_manager.h | 17 +- src/mongo/s/chunk_manager_query_test.cpp | 4 +- src/mongo/s/chunk_manager_refresh_bm.cpp | 8 +- src/mongo/s/chunk_map_test.cpp | 16 +- src/mongo/s/chunk_test.cpp | 10 +- src/mongo/s/chunk_version.cpp | 64 ++-- src/mongo/s/chunk_version.h | 29 +- src/mongo/s/chunk_version_test.cpp | 68 ++-- src/mongo/s/comparable_chunk_version_test.cpp | 337 ++++++------------ src/mongo/s/query/cluster_exchange_test.cpp | 8 +- src/mongo/s/query/sharded_agg_test_fixture.h | 7 +- .../s/request_types/balance_chunk_request_test.cpp | 14 +- .../s/request_types/balance_chunk_request_type.cpp | 2 + .../commit_chunk_migration_request_test.cpp | 4 +- .../commit_chunk_migration_request_type.cpp | 2 + .../s/request_types/move_chunk_request_test.cpp | 6 +- .../set_shard_version_request_test.cpp | 37 +- src/mongo/s/routing_table_history_test.cpp | 119 ++++--- src/mongo/s/write_ops/batch_write_exec_test.cpp | 375 +++++++++------------ src/mongo/s/write_ops/batch_write_op.cpp | 4 +- .../s/write_ops/batched_command_request_test.cpp | 6 +- src/mongo/s/write_ops/write_op_test.cpp | 26 +- 97 files changed, 1039 insertions(+), 1353 deletions(-) diff --git a/jstests/core/views/views_all_commands.js b/jstests/core/views/views_all_commands.js index f40ed053b54..be52f9b4928 100644 --- a/jstests/core/views/views_all_commands.js +++ b/jstests/core/views/views_all_commands.js @@ -573,7 +573,7 @@ let viewsCommandTests = { max: {x: 0}, keyPattern: {x: 1}, splitKeys: [{x: -2}, {x: -1}], - shardVersion: [Timestamp(1, 2), ObjectId()] + shardVersion: [Timestamp(1, 2), ObjectId(), Timestamp(1, 1)] }, skipSharded: true, expectFailure: true, diff --git a/jstests/sharding/check_sharding_index_versioned.js b/jstests/sharding/check_sharding_index_versioned.js index 57e2ece67e5..2fd0e9bf2d7 100644 --- a/jstests/sharding/check_sharding_index_versioned.js +++ b/jstests/sharding/check_sharding_index_versioned.js @@ -21,7 +21,7 @@ assert.throwsWithCode(() => { st.rs0.getPrimary().getDB(dbName).runCommand({ checkShardingIndex: ns, keyPattern: {x: 1}, - shardVersion: [Timestamp(99, 10101), ObjectId()], + shardVersion: [Timestamp(99, 10101), ObjectId(), Timestamp(1, 1)], }); }, ErrorCodes.StaleConfig); diff --git a/jstests/sharding/version1.js b/jstests/sharding/version1.js index 4f56bbd8136..7ec746537d4 100644 --- a/jstests/sharding/version1.js +++ b/jstests/sharding/version1.js @@ -42,11 +42,13 @@ assert.commandFailed(a.runCommand({ "should have failed because version is config is 1|0"); var epoch = s.getDB('config').collections.findOne({_id: "alleyinsider.foo"}).lastmodEpoch; +var timestamp = s.getDB('config').collections.findOne({_id: "alleyinsider.foo"}).timestamp; assert.commandWorked(a.runCommand({ setShardVersion: "alleyinsider.foo", configdb: s._configDB, version: new Timestamp(1, 0), versionEpoch: epoch, + versionTimestamp: timestamp, authoritative: true, shard: s.shard0.shardName, shardHost: s.s.host @@ -57,21 +59,24 @@ assert.commandFailed(a.runCommand({ setShardVersion: "alleyinsider.foo", configdb: "a", version: new Timestamp(0, 2), - versionEpoch: epoch + versionEpoch: epoch, + versionTimestamp: timestamp })); assert.commandFailed(a.runCommand({ setShardVersion: "alleyinsider.foo", configdb: s._configDB, version: new Timestamp(0, 2), - versionEpoch: epoch + versionEpoch: epoch, + versionTimestamp: timestamp })); assert.commandFailed(a.runCommand({ setShardVersion: "alleyinsider.foo", configdb: s._configDB, version: new Timestamp(0, 1), - versionEpoch: epoch + versionEpoch: epoch, + versionTimestamp: timestamp })); // the only way that setSharVersion passes is if the shard agrees with the version diff --git a/jstests/sharding/version2.js b/jstests/sharding/version2.js index a93615c6a9f..c52169053c5 100644 --- a/jstests/sharding/version2.js +++ b/jstests/sharding/version2.js @@ -19,12 +19,14 @@ var a = s.shard0.getDB("admin"); assert.eq(a.runCommand({"getShardVersion": "alleyinsider.foo", configdb: s._configDB}).global.i, 0); var fooEpoch = s.getDB('config').collections.findOne({_id: "alleyinsider.foo"}).lastmodEpoch; +var fooTimestamp = s.getDB('config').collections.findOne({_id: "alleyinsider.foo"}).timestamp; assert.commandWorked(a.runCommand({ setShardVersion: "alleyinsider.foo", configdb: s._configDB, authoritative: true, version: new Timestamp(1, 0), versionEpoch: fooEpoch, + versionTimestamp: fooTimestamp, shard: s.shard0.shardName, shardHost: s.s.host, })); diff --git a/src/mongo/db/pipeline/dispatch_shard_pipeline_test.cpp b/src/mongo/db/pipeline/dispatch_shard_pipeline_test.cpp index 932645f78d1..8c9528e0e09 100644 --- a/src/mongo/db/pipeline/dispatch_shard_pipeline_test.cpp +++ b/src/mongo/db/pipeline/dispatch_shard_pipeline_test.cpp @@ -219,9 +219,10 @@ TEST_F(DispatchShardPipelineTest, WrappedDispatchDoesRetryOnStaleConfigError) { // Mock the expected config server queries. const OID epoch = OID::gen(); const UUID uuid = UUID::gen(); + const Timestamp timestamp(1); const ShardKeyPattern shardKeyPattern(BSON("_id" << 1)); - ChunkVersion version(1, 0, epoch, boost::none /* timestamp */); + ChunkVersion version(1, 0, epoch, timestamp); ChunkType chunk1( uuid, {shardKeyPattern.getKeyPattern().globalMin(), BSON("_id" << 0)}, version, {"0"}); @@ -233,7 +234,7 @@ TEST_F(DispatchShardPipelineTest, WrappedDispatchDoesRetryOnStaleConfigError) { chunk2.setName(OID::gen()); version.incMinor(); expectCollectionAndChunksAggregation( - kTestAggregateNss, epoch, uuid, shardKeyPattern, {chunk1, chunk2}); + kTestAggregateNss, epoch, timestamp, uuid, shardKeyPattern, {chunk1, chunk2}); // That error should be retried, but only the one on that shard. onCommand([&](const executor::RemoteCommandRequest& request) { diff --git a/src/mongo/db/pipeline/process_interface/mongos_process_interface_test.cpp b/src/mongo/db/pipeline/process_interface/mongos_process_interface_test.cpp index dbf4f571c1c..d57abe6b45f 100644 --- a/src/mongo/db/pipeline/process_interface/mongos_process_interface_test.cpp +++ b/src/mongo/db/pipeline/process_interface/mongos_process_interface_test.cpp @@ -63,7 +63,7 @@ public: TEST_F(MongosProcessInterfaceTest, FailsToEnsureFieldsUniqueIfTargetCollectionVersionIsSpecified) { auto expCtx = getExpCtx(); auto targetCollectionVersion = - boost::make_optional(ChunkVersion(0, 0, OID::gen(), boost::none /* timestamp */)); + boost::make_optional(ChunkVersion(0, 0, OID::gen(), Timestamp())); auto processInterface = makeProcessInterface(); ASSERT_THROWS_CODE(processInterface->ensureFieldsUniqueOrResolveDocumentKey( diff --git a/src/mongo/db/pipeline/process_interface/standalone_process_interface_test.cpp b/src/mongo/db/pipeline/process_interface/standalone_process_interface_test.cpp index 05f5adc6976..dd4a80daab5 100644 --- a/src/mongo/db/pipeline/process_interface/standalone_process_interface_test.cpp +++ b/src/mongo/db/pipeline/process_interface/standalone_process_interface_test.cpp @@ -67,7 +67,7 @@ TEST_F(ProcessInterfaceStandaloneTest, FailsToEnsureFieldsUniqueIfTargetCollectionVersionIsSpecifiedOnMongos) { auto expCtx = getExpCtx(); auto targetCollectionVersion = - boost::make_optional(ChunkVersion(0, 0, OID::gen(), boost::none /* timestamp */)); + boost::make_optional(ChunkVersion(0, 0, OID::gen(), Timestamp())); auto processInterface = makeProcessInterface(); // Test that 'targetCollectionVersion' is not accepted if not from mongos. @@ -90,7 +90,7 @@ TEST_F(ProcessInterfaceStandaloneTest, TEST_F(ProcessInterfaceStandaloneTest, FailsToEnsureFieldsUniqueIfJoinFieldsAreNotSentFromMongos) { auto expCtx = getExpCtx(); auto targetCollectionVersion = - boost::make_optional(ChunkVersion(0, 0, OID::gen(), boost::none /* timestamp */)); + boost::make_optional(ChunkVersion(0, 0, OID::gen(), Timestamp())); auto processInterface = makeProcessInterface(); expCtx->fromMongos = true; diff --git a/src/mongo/db/pipeline/sharded_union_test.cpp b/src/mongo/db/pipeline/sharded_union_test.cpp index 395990e5309..e27be5c5477 100644 --- a/src/mongo/db/pipeline/sharded_union_test.cpp +++ b/src/mongo/db/pipeline/sharded_union_test.cpp @@ -168,9 +168,10 @@ TEST_F(ShardedUnionTest, RetriesSubPipelineOnStaleConfigError) { // Mock the expected config server queries. const OID epoch = OID::gen(); const UUID uuid = UUID::gen(); + const Timestamp timestamp; const ShardKeyPattern shardKeyPattern(BSON("_id" << 1)); - ChunkVersion version(1, 0, epoch, boost::none /* timestamp */); + ChunkVersion version(1, 0, epoch, timestamp); ChunkType chunk1(*cm.getUUID(), {shardKeyPattern.getKeyPattern().globalMin(), BSON("_id" << 0)}, @@ -187,7 +188,7 @@ TEST_F(ShardedUnionTest, RetriesSubPipelineOnStaleConfigError) { version.incMinor(); expectCollectionAndChunksAggregation( - kTestAggregateNss, epoch, uuid, shardKeyPattern, {chunk1, chunk2}); + kTestAggregateNss, epoch, timestamp, uuid, shardKeyPattern, {chunk1, chunk2}); // That error should be retried, but only the one on that shard. onCommand([&](const executor::RemoteCommandRequest& request) { @@ -246,9 +247,10 @@ TEST_F(ShardedUnionTest, CorrectlySplitsSubPipelineIfRefreshedDistributionRequir // created and moved to the first shard. const OID epoch = OID::gen(); const UUID uuid = UUID::gen(); + const Timestamp timestamp; const ShardKeyPattern shardKeyPattern(BSON("_id" << 1)); - ChunkVersion version(1, 0, epoch, boost::none /* timestamp */); + ChunkVersion version(1, 0, epoch, timestamp); ChunkType chunk1(*cm.getUUID(), {shardKeyPattern.getKeyPattern().globalMin(), BSON("_id" << 0)}, @@ -269,7 +271,7 @@ TEST_F(ShardedUnionTest, CorrectlySplitsSubPipelineIfRefreshedDistributionRequir chunk3.setName(OID::gen()); expectCollectionAndChunksAggregation( - kTestAggregateNss, epoch, uuid, shardKeyPattern, {chunk1, chunk2, chunk3}); + kTestAggregateNss, epoch, timestamp, uuid, shardKeyPattern, {chunk1, chunk2, chunk3}); // That error should be retried, this time two shards. onCommand([&](const executor::RemoteCommandRequest& request) { @@ -335,8 +337,9 @@ TEST_F(ShardedUnionTest, AvoidsSplittingSubPipelineIfRefreshedDistributionDoesNo // the same shard. const OID epoch = OID::gen(); const UUID uuid = UUID::gen(); + const Timestamp timestamp; const ShardKeyPattern shardKeyPattern(BSON("_id" << 1)); - ChunkVersion version(1, 0, epoch, boost::none /* timestamp */); + ChunkVersion version(1, 0, epoch, timestamp); ChunkType chunk1( *cm.getUUID(), {shardKeyPattern.getKeyPattern().globalMin(), shardKeyPattern.getKeyPattern().globalMax()}, @@ -344,7 +347,8 @@ TEST_F(ShardedUnionTest, AvoidsSplittingSubPipelineIfRefreshedDistributionDoesNo {shards[0].getName()}); chunk1.setName(OID::gen()); - expectCollectionAndChunksAggregation(kTestAggregateNss, epoch, uuid, shardKeyPattern, {chunk1}); + expectCollectionAndChunksAggregation( + kTestAggregateNss, epoch, timestamp, uuid, shardKeyPattern, {chunk1}); // That error should be retried, this time targetting only one shard. onCommand([&](const executor::RemoteCommandRequest& request) { diff --git a/src/mongo/db/s/active_migrations_registry_test.cpp b/src/mongo/db/s/active_migrations_registry_test.cpp index bfe59908b24..28d0d71f27d 100644 --- a/src/mongo/db/s/active_migrations_registry_test.cpp +++ b/src/mongo/db/s/active_migrations_registry_test.cpp @@ -59,7 +59,7 @@ protected: }; MoveChunkRequest createMoveChunkRequest(const NamespaceString& nss) { - const ChunkVersion chunkVersion(1, 2, OID::gen(), boost::none /* timestamp */); + const ChunkVersion chunkVersion(1, 2, OID::gen(), Timestamp()); BSONObjBuilder builder; MoveChunkRequest::appendAsCommand( diff --git a/src/mongo/db/s/balancer/balance_stats_test.cpp b/src/mongo/db/s/balancer/balance_stats_test.cpp index ed1a430126f..a4c2f2b00bf 100644 --- a/src/mongo/db/s/balancer/balance_stats_test.cpp +++ b/src/mongo/db/s/balancer/balance_stats_test.cpp @@ -58,7 +58,7 @@ public: {}, // collator false, // unique _epoch, - boost::none, // timestamp + _timestamp, // timestamp boost::none, // time series fields boost::none, // resharding fields boost::none, // chunk size bytes @@ -75,9 +75,10 @@ private: const NamespaceString _nss{"foo.bar"}; const UUID _uuid = UUID::gen(); const OID _epoch{OID::gen()}; + const Timestamp _timestamp{Timestamp(1, 1)}; const ShardId _shardPrimary{"dummyShardPrimary"}; - const DatabaseVersion _dbVersion{UUID::gen(), Timestamp()}; - ChunkVersion _nextVersion{1, 0, _epoch, boost::none}; + const DatabaseVersion _dbVersion{UUID::gen(), _timestamp}; + ChunkVersion _nextVersion{1, 0, _epoch, _timestamp}; }; TEST_F(BalanceStatsTest, SingleChunkNoZones) { diff --git a/src/mongo/db/s/balancer/balancer_policy_test.cpp b/src/mongo/db/s/balancer/balancer_policy_test.cpp index 43b340f2906..077e1a731f5 100644 --- a/src/mongo/db/s/balancer/balancer_policy_test.cpp +++ b/src/mongo/db/s/balancer/balancer_policy_test.cpp @@ -77,7 +77,7 @@ std::pair generateCluster( int64_t currentChunk = 0; - ChunkVersion chunkVersion(1, 0, OID::gen(), boost::none /* timestamp */); + ChunkVersion chunkVersion(1, 0, OID::gen(), Timestamp()); const UUID uuid = UUID::gen(); const KeyPattern shardKeyPattern(BSON("x" << 1)); diff --git a/src/mongo/db/s/balancer/scoped_migration_request_test.cpp b/src/mongo/db/s/balancer/scoped_migration_request_test.cpp index db4ddcf77f5..44ad31f3820 100644 --- a/src/mongo/db/s/balancer/scoped_migration_request_test.cpp +++ b/src/mongo/db/s/balancer/scoped_migration_request_test.cpp @@ -102,7 +102,7 @@ ScopedMigrationRequest ScopedMigrationRequestTest::makeScopedMigrationRequest( MigrateInfo ScopedMigrationRequestTest::makeMigrateInfo() { const auto collUuid = UUID::gen(); - const ChunkVersion kChunkVersion{1, 2, OID::gen(), boost::none /* timestamp */}; + const ChunkVersion kChunkVersion{1, 2, OID::gen(), Timestamp()}; BSONObjBuilder chunkBuilder; collUuid.appendToBuilder(&chunkBuilder, ChunkType::collectionUUID.name()); diff --git a/src/mongo/db/s/balancer/type_migration_test.cpp b/src/mongo/db/s/balancer/type_migration_test.cpp index b8af0eaa339..0b013a750b5 100644 --- a/src/mongo/db/s/balancer/type_migration_test.cpp +++ b/src/mongo/db/s/balancer/type_migration_test.cpp @@ -50,7 +50,7 @@ const bool kWaitForDelete{true}; TEST(MigrationTypeTest, ConvertFromMigrationInfo) { const auto collUuid = UUID::gen(); const auto collEpoch = OID::gen(); - const auto collTimestamp = boost::none; + const auto collTimestamp = Timestamp(1, 1); const ChunkVersion version(1, 2, collEpoch, collTimestamp); BSONObjBuilder chunkBuilder; @@ -88,7 +88,7 @@ TEST(MigrationTypeTest, ConvertFromMigrationInfo) { } TEST(MigrationTypeTest, FromAndToBSON) { - const ChunkVersion version(1, 2, OID::gen(), boost::none /* timestamp */); + const ChunkVersion version(1, 2, OID::gen(), Timestamp()); BSONObjBuilder builder; builder.append(MigrationType::ns(), kNs); @@ -108,7 +108,7 @@ TEST(MigrationTypeTest, FromAndToBSON) { } TEST(MigrationTypeTest, MissingRequiredNamespaceField) { - const ChunkVersion version(1, 2, OID::gen(), boost::none /* timestamp */); + const ChunkVersion version(1, 2, OID::gen(), Timestamp()); BSONObjBuilder builder; builder.append(MigrationType::min(), kMin); @@ -125,7 +125,7 @@ TEST(MigrationTypeTest, MissingRequiredNamespaceField) { } TEST(MigrationTypeTest, MissingRequiredMinField) { - const ChunkVersion version(1, 2, OID::gen(), boost::none /* timestamp */); + const ChunkVersion version(1, 2, OID::gen(), Timestamp()); BSONObjBuilder builder; builder.append(MigrationType::ns(), kNs); @@ -142,7 +142,7 @@ TEST(MigrationTypeTest, MissingRequiredMinField) { } TEST(MigrationTypeTest, MissingRequiredMaxField) { - const ChunkVersion version(1, 2, OID::gen(), boost::none /* timestamp */); + const ChunkVersion version(1, 2, OID::gen(), Timestamp()); BSONObjBuilder builder; builder.append(MigrationType::ns(), kNs); @@ -159,7 +159,7 @@ TEST(MigrationTypeTest, MissingRequiredMaxField) { } TEST(MigrationTypeTest, MissingRequiredFromShardField) { - const ChunkVersion version(1, 2, OID::gen(), boost::none /* timestamp */); + const ChunkVersion version(1, 2, OID::gen(), Timestamp()); BSONObjBuilder builder; builder.append(MigrationType::ns(), kNs); @@ -176,7 +176,7 @@ TEST(MigrationTypeTest, MissingRequiredFromShardField) { } TEST(MigrationTypeTest, MissingRequiredToShardField) { - const ChunkVersion version(1, 2, OID::gen(), boost::none /* timestamp */); + const ChunkVersion version(1, 2, OID::gen(), Timestamp()); BSONObjBuilder builder; builder.append(MigrationType::ns(), kNs); diff --git a/src/mongo/db/s/collection_metadata_filtering_test.cpp b/src/mongo/db/s/collection_metadata_filtering_test.cpp index 231808a7b5c..a21b5fd1a28 100644 --- a/src/mongo/db/s/collection_metadata_filtering_test.cpp +++ b/src/mongo/db/s/collection_metadata_filtering_test.cpp @@ -73,13 +73,13 @@ protected: nullptr, false, epoch, - boost::none /* timestamp */, + Timestamp(), timeseriesFields, boost::none, boost::none, true, [&] { - ChunkVersion version(1, 0, epoch, boost::none /* timestamp */); + ChunkVersion version(1, 0, epoch, Timestamp()); ChunkType chunk1(uuid, {shardKeyPattern.getKeyPattern().globalMin(), BSON("_id" << -100)}, diff --git a/src/mongo/db/s/collection_metadata_test.cpp b/src/mongo/db/s/collection_metadata_test.cpp index 7d01b0fa39f..139bb5b40b5 100644 --- a/src/mongo/db/s/collection_metadata_test.cpp +++ b/src/mongo/db/s/collection_metadata_test.cpp @@ -62,7 +62,7 @@ CollectionMetadata makeCollectionMetadataImpl( std::vector allChunks; auto nextMinKey = shardKeyPattern.globalMin(); - ChunkVersion version{1, 0, epoch, boost::none /* timestamp */}; + ChunkVersion version{1, 0, epoch, timestamp}; for (const auto& myNextChunk : thisShardsChunks) { if (SimpleBSONObjComparator::kInstance.evaluate(nextMinKey < myNextChunk.first)) { // Need to add a chunk to the other shard from nextMinKey to myNextChunk.first. @@ -94,7 +94,7 @@ CollectionMetadata makeCollectionMetadataImpl( nullptr, false, epoch, - boost::none /* timestamp */, + timestamp, boost::none /* timeseriesFields */, std::move(reshardingFields), boost::none /* chunkSizeBytes */, diff --git a/src/mongo/db/s/collection_sharding_runtime_test.cpp b/src/mongo/db/s/collection_sharding_runtime_test.cpp index 54bb235b299..d1e5a6ff784 100644 --- a/src/mongo/db/s/collection_sharding_runtime_test.cpp +++ b/src/mongo/db/s/collection_sharding_runtime_test.cpp @@ -56,13 +56,12 @@ protected: static CollectionMetadata makeShardedMetadata(OperationContext* opCtx, UUID uuid = UUID::gen()) { const OID epoch = OID::gen(); + const Timestamp timestamp; auto range = ChunkRange(BSON(kShardKey << MINKEY), BSON(kShardKey << MAXKEY)); - auto chunk = ChunkType(uuid, - std::move(range), - ChunkVersion(1, 0, epoch, boost::none /* timestamp */), - ShardId("other")); + auto chunk = ChunkType( + uuid, std::move(range), ChunkVersion(1, 0, epoch, timestamp), ShardId("other")); ChunkManager cm(ShardId("0"), - DatabaseVersion(UUID::gen(), Timestamp()), + DatabaseVersion(UUID::gen(), timestamp), makeStandaloneRoutingTableHistory( RoutingTableHistory::makeNew(kTestNss, uuid, @@ -70,7 +69,7 @@ protected: nullptr, false, epoch, - boost::none /* timestamp */, + timestamp, boost::none /* timeseriesFields */, boost::none, boost::none /* chunkSizeBytes */, @@ -265,8 +264,7 @@ public: return std::make_unique(kShardList); } - CollectionType createCollection(const OID& epoch, - boost::optional timestamp = boost::none) { + CollectionType createCollection(const OID& epoch, const Timestamp& timestamp) { CollectionType res(kNss, epoch, timestamp, Date_t::now(), kCollUUID); res.setKeyPattern(BSON(kShardKey << 1)); res.setUnique(false); @@ -276,7 +274,7 @@ public: std::vector createChunks(const OID& epoch, const UUID& uuid, - boost::optional timestamp = boost::none) { + const Timestamp& timestamp) { auto range1 = ChunkRange(BSON(kShardKey << MINKEY), BSON(kShardKey << 5)); ChunkType chunk1( uuid, range1, ChunkVersion(1, 0, epoch, timestamp), kShardList[0].getName()); @@ -302,8 +300,8 @@ TEST_F(CollectionShardingRuntimeTestWithMockedLoader, const auto epoch = OID::gen(); const Timestamp timestamp(42); - const auto coll = createCollection(epoch); - const auto chunks = createChunks(epoch, coll.getUuid()); + const auto coll = createCollection(epoch, timestamp); + const auto chunks = createChunks(epoch, coll.getUuid(), timestamp); const auto timestampedColl = createCollection(epoch, timestamp); const auto timestampedChunks = createChunks(epoch, timestampedColl.getUuid(), timestamp); @@ -340,8 +338,8 @@ TEST_F(CollectionShardingRuntimeTestWithMockedLoader, const auto epoch = OID::gen(); const Timestamp timestamp(42); - const auto coll = createCollection(epoch); - const auto chunks = createChunks(epoch, coll.getUuid()); + const auto coll = createCollection(epoch, timestamp); + const auto chunks = createChunks(epoch, coll.getUuid(), timestamp); const auto collVersion = chunks.back().getVersion(); const auto timestampedColl = createCollection(epoch, timestamp); diff --git a/src/mongo/db/s/config/config_server_test_fixture.cpp b/src/mongo/db/s/config/config_server_test_fixture.cpp index f6b6566ef99..3144add6236 100644 --- a/src/mongo/db/s/config/config_server_test_fixture.cpp +++ b/src/mongo/db/s/config/config_server_test_fixture.cpp @@ -342,12 +342,11 @@ void ConfigServerTestFixture::setupCollection(const NamespaceString& nss, } } -StatusWith ConfigServerTestFixture::getChunkDoc( - OperationContext* opCtx, - const UUID& uuid, - const BSONObj& minKey, - const OID& collEpoch, - const boost::optional& collTimestamp) { +StatusWith ConfigServerTestFixture::getChunkDoc(OperationContext* opCtx, + const UUID& uuid, + const BSONObj& minKey, + const OID& collEpoch, + const Timestamp& collTimestamp) { const auto query = BSON(ChunkType::collectionUUID() << uuid << ChunkType::min(minKey)); auto doc = findOneOnConfigCollection(opCtx, ChunkType::ConfigNS, query); @@ -357,11 +356,10 @@ StatusWith ConfigServerTestFixture::getChunkDoc( return ChunkType::fromConfigBSON(doc.getValue(), collEpoch, collTimestamp); } -StatusWith ConfigServerTestFixture::getChunkDoc( - OperationContext* opCtx, - const BSONObj& minKey, - const OID& collEpoch, - const boost::optional& collTimestamp) { +StatusWith ConfigServerTestFixture::getChunkDoc(OperationContext* opCtx, + const BSONObj& minKey, + const OID& collEpoch, + const Timestamp& collTimestamp) { auto doc = findOneOnConfigCollection(opCtx, ChunkType::ConfigNS, BSON(ChunkType::min(minKey))); if (!doc.isOK()) return doc.getStatus(); @@ -378,7 +376,6 @@ StatusWith ConfigServerTestFixture::getCollectionVersion(Operation const CollectionType coll(collectionDoc.getValue()); - invariant(coll.getTimestamp()); auto chunkDoc = findOneOnConfigCollection(opCtx, ChunkType::ConfigNS, diff --git a/src/mongo/db/s/config/config_server_test_fixture.h b/src/mongo/db/s/config/config_server_test_fixture.h index 624cf39ab5d..5e00755a652 100644 --- a/src/mongo/db/s/config/config_server_test_fixture.h +++ b/src/mongo/db/s/config/config_server_test_fixture.h @@ -114,7 +114,7 @@ protected: const UUID& uuid, const BSONObj& minKey, const OID& collEpoch, - const boost::optional& collTimestamp); + const Timestamp& collTimestamp); /** * Retrieves the chunk document from the config server. @@ -125,7 +125,7 @@ protected: StatusWith getChunkDoc(OperationContext* opCtx, const BSONObj& minKey, const OID& collEpoch, - const boost::optional& collTimestamp); + const Timestamp& collTimestamp); /** * Returns the collection version. diff --git a/src/mongo/db/s/config/initial_split_policy_test.cpp b/src/mongo/db/s/config/initial_split_policy_test.cpp index 0555915be5b..ec871090f73 100644 --- a/src/mongo/db/s/config/initial_split_policy_test.cpp +++ b/src/mongo/db/s/config/initial_split_policy_test.cpp @@ -206,7 +206,7 @@ public: std::vector chunks; for (unsigned long i = 0; i < chunkRanges.size(); ++i) { - ChunkVersion version(1, 0, OID::gen(), boost::none /* timestamp */); + ChunkVersion version(1, 0, OID::gen(), Timestamp()); ChunkType chunk(_uuid, chunkRanges[i], version, shardIds[i]); chunk.setHistory({ChunkHistory(timeStamp, shardIds[i])}); chunks.push_back(chunk); diff --git a/src/mongo/db/s/config/sharding_catalog_manager.h b/src/mongo/db/s/config/sharding_catalog_manager.h index 733b0164171..c23f6fc5a54 100644 --- a/src/mongo/db/s/config/sharding_catalog_manager.h +++ b/src/mongo/db/s/config/sharding_catalog_manager.h @@ -576,7 +576,7 @@ private: StatusWith _findChunkOnConfig(OperationContext* opCtx, const UUID& uuid, const OID& epoch, - const boost::optional& timestamp, + const Timestamp& timestamp, const BSONObj& key); /** diff --git a/src/mongo/db/s/config/sharding_catalog_manager_assign_key_range_to_zone_test.cpp b/src/mongo/db/s/config/sharding_catalog_manager_assign_key_range_to_zone_test.cpp index ab0a6d3a182..4c49d7e5943 100644 --- a/src/mongo/db/s/config/sharding_catalog_manager_assign_key_range_to_zone_test.cpp +++ b/src/mongo/db/s/config/sharding_catalog_manager_assign_key_range_to_zone_test.cpp @@ -61,7 +61,8 @@ public: setupShards({shard}); - CollectionType shardedCollection(shardedNS(), OID::gen(), Date_t::now(), UUID::gen()); + CollectionType shardedCollection( + shardedNS(), OID::gen(), Timestamp(), Date_t::now(), UUID::gen()); shardedCollection.setKeyPattern(BSON("x" << 1)); ASSERT_OK(insertToConfigCollection( @@ -259,7 +260,7 @@ TEST_F(AssignKeyRangeToZoneTestFixture, RemoveZoneWithDollarPrefixedShardKeysSho TEST_F(AssignKeyRangeToZoneTestFixture, MinThatIsAShardKeyPrefixShouldConvertToFullShardKey) { NamespaceString ns("compound.shard"); - CollectionType shardedCollection(ns, OID::gen(), Date_t::now(), UUID::gen()); + CollectionType shardedCollection(ns, OID::gen(), Timestamp(), Date_t::now(), UUID::gen()); shardedCollection.setKeyPattern(BSON("x" << 1 << "y" << 1)); ASSERT_OK(insertToConfigCollection( @@ -276,7 +277,7 @@ TEST_F(AssignKeyRangeToZoneTestFixture, MinThatIsAShardKeyPrefixShouldConvertToF TEST_F(AssignKeyRangeToZoneTestFixture, MaxThatIsAShardKeyPrefixShouldConvertToFullShardKey) { NamespaceString ns("compound.shard"); - CollectionType shardedCollection(ns, OID::gen(), Date_t::now(), UUID::gen()); + CollectionType shardedCollection(ns, OID::gen(), Timestamp(), Date_t::now(), UUID::gen()); shardedCollection.setKeyPattern(BSON("x" << 1 << "y" << 1)); ASSERT_OK(insertToConfigCollection( @@ -328,7 +329,7 @@ TEST_F(AssignKeyRangeToZoneTestFixture, MinMaxThatIsNotAShardKeyPrefixShouldFail TEST_F(AssignKeyRangeToZoneTestFixture, MinMaxThatIsAShardKeyPrefixShouldSucceed) { NamespaceString ns("compound.shard"); - CollectionType shardedCollection(ns, OID::gen(), Date_t::now(), UUID::gen()); + CollectionType shardedCollection(ns, OID::gen(), Timestamp(), Date_t::now(), UUID::gen()); shardedCollection.setKeyPattern(BSON("x" << 1 << "y" << 1)); ASSERT_OK(insertToConfigCollection( @@ -371,7 +372,7 @@ TEST_F(AssignKeyRangeToZoneTestFixture, TimeseriesCollMustHaveTimeKeyRangeMinKey const std::string controlTimeField = timeseries::kControlMinFieldNamePrefix.toString() + timeField; const TimeseriesOptions timeseriesOptions(timeField.toString()); - CollectionType shardedCollection(ns, OID::gen(), Date_t::now(), UUID::gen()); + CollectionType shardedCollection(ns, OID::gen(), Timestamp(), Date_t::now(), UUID::gen()); TypeCollectionTimeseriesFields timeseriesFields; timeseriesFields.setTimeseriesOptions(timeseriesOptions); shardedCollection.setTimeseriesFields(timeseriesFields); @@ -522,7 +523,7 @@ TEST_F(AssignKeyRangeWithOneRangeFixture, NewRangeOverlappingInsideExistingShoul */ TEST_F(AssignKeyRangeWithOneRangeFixture, NewRangeOverlappingWithDifferentNSShouldSucceed) { CollectionType shardedCollection( - NamespaceString("other.coll"), OID::gen(), Date_t::now(), UUID::gen()); + NamespaceString("other.coll"), OID::gen(), Timestamp(), Date_t::now(), UUID::gen()); shardedCollection.setKeyPattern(BSON("x" << 1)); ASSERT_OK(insertToConfigCollection( @@ -742,7 +743,7 @@ TEST_F(AssignKeyRangeWithOneRangeFixture, RemoveWithInvalidMaxShardKeyShouldFail TEST_F(AssignKeyRangeWithOneRangeFixture, RemoveWithPartialMinPrefixShouldRemoveRange) { NamespaceString ns("compound.shard"); - CollectionType shardedCollection(ns, OID::gen(), Date_t::now(), UUID::gen()); + CollectionType shardedCollection(ns, OID::gen(), Timestamp(), Date_t::now(), UUID::gen()); shardedCollection.setKeyPattern(BSON("x" << 1 << "y" << 1)); ASSERT_OK(insertToConfigCollection( @@ -765,7 +766,7 @@ TEST_F(AssignKeyRangeWithOneRangeFixture, RemoveWithPartialMinPrefixShouldRemove TEST_F(AssignKeyRangeWithOneRangeFixture, RemoveWithPartialMaxPrefixShouldRemoveRange) { NamespaceString ns("compound.shard"); - CollectionType shardedCollection(ns, OID::gen(), Date_t::now(), UUID::gen()); + CollectionType shardedCollection(ns, OID::gen(), Timestamp(), Date_t::now(), UUID::gen()); shardedCollection.setKeyPattern(BSON("x" << 1 << "y" << 1)); ASSERT_OK(insertToConfigCollection( diff --git a/src/mongo/db/s/config/sharding_catalog_manager_chunk_operations.cpp b/src/mongo/db/s/config/sharding_catalog_manager_chunk_operations.cpp index aa0abe9b814..2ac8d4dac46 100644 --- a/src/mongo/db/s/config/sharding_catalog_manager_chunk_operations.cpp +++ b/src/mongo/db/s/config/sharding_catalog_manager_chunk_operations.cpp @@ -138,18 +138,14 @@ BSONArray buildMergeChunksTransactionPrecond(const std::vector& chunk const ChunkVersion& collVersion) { BSONArrayBuilder preCond; - const bool collHasTimestamp = (bool)collVersion.getTimestamp(); - invariant(collHasTimestamp); for (const auto& chunk : chunksToMerge) { BSONObj query = BSON(ChunkType::min(chunk.getMin()) << ChunkType::max(chunk.getMax()) << ChunkType::collectionUUID() << chunk.getCollectionUUID()); - const auto collectionIdentityMatchCondition = collHasTimestamp - ? BSON(ChunkType::collectionUUID() - << chunk.getCollectionUUID() << ChunkType::shard(chunk.getShard().toString())) - : BSON(ChunkType::epoch(collVersion.epoch()) - << ChunkType::shard(chunk.getShard().toString())); + const auto collectionIdentityMatchCondition = + BSON(ChunkType::collectionUUID() + << chunk.getCollectionUUID() << ChunkType::shard(chunk.getShard().toString())); BSONObjBuilder b; b.append("ns", ChunkType::ConfigNS.ns()); @@ -167,13 +163,12 @@ BSONArray buildMergeChunksTransactionPrecond(const std::vector& chunk StatusWith getCurrentChunk(OperationContext* opCtx, const UUID& uuid, const OID& epoch, - const boost::optional& timestamp, + const Timestamp& timestamp, const ChunkType& requestedChunk) { uassert(4683300, "Config server rejecting commitChunkMigration request that does not have a " "ChunkVersion", - requestedChunk.isVersionSet() && requestedChunk.getVersion().isSet() && - requestedChunk.getVersion().epoch().isSet()); + requestedChunk.isVersionSet() && requestedChunk.getVersion().isSet()); BSONObj chunkQuery = BSON(ChunkType::min() << requestedChunk.getMin() << ChunkType::max() @@ -259,7 +254,7 @@ BSONObj makeCommitChunkTransactionCommand(const NamespaceString& nss, boost::optional getControlChunkForMigrate(OperationContext* opCtx, const UUID& uuid, const OID& epoch, - const boost::optional& timestamp, + const Timestamp& timestamp, const ChunkType& migratedChunk, const ShardId& fromShard) { auto const configShard = Grid::get(opCtx)->shardRegistry()->getConfigShard(); @@ -327,7 +322,6 @@ StatusWith getCollectionVersion(OperationContext* opCtx, const Nam } const CollectionType coll(findCollResponse.getValue().docs[0]); - invariant(coll.getTimestamp()); const auto chunksQuery = BSON(ChunkType::collectionUUID << coll.getUuid()); return getMaxChunkVersionFromQueryResponse( coll, @@ -345,7 +339,6 @@ ChunkVersion getShardVersion(OperationContext* opCtx, const CollectionType& coll, const ShardId& fromShard, const ChunkVersion& collectionVersion) { - invariant(coll.getTimestamp()); const auto chunksQuery = BSON(ChunkType::collectionUUID << coll.getUuid() << ChunkType::shard(fromShard.toString())); @@ -392,7 +385,6 @@ void bumpCollectionMinorVersion(OperationContext* opCtx, const CollectionType coll(findCollResponse.docs[0]); // Find the newest chunk - invariant(coll.getTimestamp()); const auto findChunkResponse = uassertStatusOK(configShard->exhaustiveFindOnConfig( opCtx, ReadPreferenceSetting{ReadPreference::PrimaryOnly}, @@ -457,7 +449,6 @@ std::vector getShardsOwningChunksForCollection(OperationContext* opCtx, ErrorCodes::NamespaceNotFound, "Collection does not exist", !findCollResponse.docs.empty()); const CollectionType coll(findCollResponse.docs[0]); - invariant(coll.getTimestamp()); DistinctCommandRequest distinctCmd(ChunkType::ConfigNS, ChunkType::shard.name()); distinctCmd.setQuery(BSON(ChunkType::collectionUUID << coll.getUuid())); @@ -625,17 +616,15 @@ StatusWith ShardingCatalogManager::commitChunkSplit( BSONObjBuilder b; b.append("ns", ChunkType::ConfigNS.ns()); - invariant(origChunk.getValue().getVersion().getTimestamp()); BSONObj query = BSON(ChunkType::min() << range.getMin() << ChunkType::max() << range.getMax() << ChunkType::collectionUUID << origChunk.getValue().getCollectionUUID()); b.append("q", BSON("query" << query << "orderby" << BSON(ChunkType::lastmod() << -1))); - const auto resultMustMatch = origChunk.getValue().getVersion().getTimestamp() - ? BSON(ChunkType::collectionUUID() - << origChunk.getValue().getCollectionUUID() << ChunkType::shard(shardName)) - : BSON(ChunkType::epoch(requestEpoch) << ChunkType::shard(shardName)); + const auto resultMustMatch = + BSON(ChunkType::collectionUUID() + << origChunk.getValue().getCollectionUUID() << ChunkType::shard(shardName)); b.append("res", resultMustMatch); @@ -750,7 +739,6 @@ StatusWith ShardingCatalogManager::commitChunksMerge( } const auto shardChunksInRangeQuery = [&]() { BSONObjBuilder queryBuilder; - invariant(coll.getTimestamp()); queryBuilder << ChunkType::collectionUUID << coll.getUuid(); queryBuilder << ChunkType::shard(shardId.toString()); queryBuilder << ChunkType::min(BSON("$gte" << chunkRange.getMin())); @@ -921,7 +909,6 @@ StatusWith ShardingCatalogManager::commitChunkMigration( "Collection is undergoing changes and chunks cannot be moved", coll.getAllowMigrations()); - invariant(coll.getTimestamp()); const auto findChunkQuery = BSON(ChunkType::collectionUUID() << coll.getUuid()); auto findResponse = uassertStatusOK( @@ -1106,12 +1093,11 @@ StatusWith ShardingCatalogManager::commitChunkMigration( return response.obj(); } -StatusWith ShardingCatalogManager::_findChunkOnConfig( - OperationContext* opCtx, - const UUID& uuid, - const OID& epoch, - const boost::optional& timestamp, - const BSONObj& key) { +StatusWith ShardingCatalogManager::_findChunkOnConfig(OperationContext* opCtx, + const UUID& uuid, + const OID& epoch, + const Timestamp& timestamp, + const BSONObj& key) { auto const configShard = Grid::get(opCtx)->shardRegistry()->getConfigShard(); const auto query = BSON(ChunkType::collectionUUID << uuid << ChunkType::min(key)); @@ -1169,7 +1155,6 @@ void ShardingCatalogManager::clearJumboFlag(OperationContext* opCtx, !findCollResponse.docs.empty()); const CollectionType coll(findCollResponse.docs[0]); - invariant(coll.getTimestamp()); BSONObj targetChunkQuery = BSON(ChunkType::min(chunk.getMin()) << ChunkType::max(chunk.getMax()) << ChunkType::collectionUUID << coll.getUuid()); @@ -1196,7 +1181,6 @@ void ShardingCatalogManager::clearJumboFlag(OperationContext* opCtx, return; } - invariant(coll.getTimestamp()); const auto allChunksQuery = BSON(ChunkType::collectionUUID << coll.getUuid()); // Must use local read concern because we will perform subsequent writes. @@ -1239,7 +1223,6 @@ void ShardingCatalogManager::clearJumboFlag(OperationContext* opCtx, currentCollectionVersion.getTimestamp()); - invariant(coll.getTimestamp()); BSONObj chunkQuery(BSON(ChunkType::min(chunk.getMin()) << ChunkType::max(chunk.getMax()) << ChunkType::collectionUUID << coll.getUuid())); @@ -1307,7 +1290,6 @@ void ShardingCatalogManager::ensureChunkVersionIsGreaterThan(OperationContext* o dassert(collUuid == coll.getUuid()); } - invariant(coll.getTimestamp()); const auto requestedChunkQuery = BSON(ChunkType::min(minKey) << ChunkType::max(maxKey) << ChunkType::collectionUUID() << collUuid); @@ -1365,8 +1347,7 @@ void ShardingCatalogManager::ensureChunkVersionIsGreaterThan(OperationContext* o // Get the chunk with the current collectionVersion for this epoch. ChunkType highestChunk; { - const auto query = coll.getTimestamp() ? BSON(ChunkType::collectionUUID() << collUuid) - : BSON(ChunkType::epoch(version.epoch())); + const auto query = BSON(ChunkType::collectionUUID() << collUuid); const auto highestChunksVector = uassertStatusOK(configShard->exhaustiveFindOnConfig( opCtx, @@ -1509,7 +1490,6 @@ void ShardingCatalogManager::splitOrMarkJumbo(OperationContext* opCtx, !findCollResponse.docs.empty()); const CollectionType coll(findCollResponse.docs[0]); - invariant(coll.getTimestamp()); const auto chunkQuery = BSON(ChunkType::collectionUUID() << coll.getUuid() << ChunkType::min(chunk.getMin())); diff --git a/src/mongo/db/s/config/sharding_catalog_manager_clear_jumbo_flag_test.cpp b/src/mongo/db/s/config/sharding_catalog_manager_clear_jumbo_flag_test.cpp index 168c53f9b24..ef66e5f5d33 100644 --- a/src/mongo/db/s/config/sharding_catalog_manager_clear_jumbo_flag_test.cpp +++ b/src/mongo/db/s/config/sharding_catalog_manager_clear_jumbo_flag_test.cpp @@ -95,15 +95,14 @@ protected: }; TEST_F(ClearJumboFlagTest, ClearJumboShouldBumpVersion) { - auto test = [&](const NamespaceString& nss, const boost::optional& collTimestamp) { + auto test = [&](const NamespaceString& nss, const Timestamp& collTimestamp) { const auto collUuid = UUID::gen(); const auto collEpoch = OID::gen(); - makeCollection(nss, collUuid, collEpoch, *collTimestamp); + makeCollection(nss, collUuid, collEpoch, collTimestamp); ShardingCatalogManager::get(operationContext()) ->clearJumboFlag(operationContext(), nss, collEpoch, jumboChunk()); - invariant(collTimestamp); auto chunkDoc = uassertStatusOK(getChunkDoc( operationContext(), collUuid, jumboChunk().getMin(), collEpoch, collTimestamp)); ASSERT_FALSE(chunkDoc.getJumbo()); @@ -114,15 +113,14 @@ TEST_F(ClearJumboFlagTest, ClearJumboShouldBumpVersion) { } TEST_F(ClearJumboFlagTest, ClearJumboShouldNotBumpVersionIfChunkNotJumbo) { - auto test = [&](const NamespaceString& nss, const boost::optional& collTimestamp) { + auto test = [&](const NamespaceString& nss, const Timestamp& collTimestamp) { const auto collUuid = UUID::gen(); const auto collEpoch = OID::gen(); - makeCollection(nss, collUuid, collEpoch, *collTimestamp); + makeCollection(nss, collUuid, collEpoch, collTimestamp); ShardingCatalogManager::get(operationContext()) ->clearJumboFlag(operationContext(), nss, collEpoch, nonJumboChunk()); - invariant(collTimestamp); auto chunkDoc = uassertStatusOK(getChunkDoc( operationContext(), collUuid, nonJumboChunk().getMin(), collEpoch, collTimestamp)); ASSERT_FALSE(chunkDoc.getJumbo()); @@ -133,10 +131,10 @@ TEST_F(ClearJumboFlagTest, ClearJumboShouldNotBumpVersionIfChunkNotJumbo) { } TEST_F(ClearJumboFlagTest, AssertsOnEpochMismatch) { - auto test = [&](const NamespaceString& nss, const boost::optional& collTimestamp) { + auto test = [&](const NamespaceString& nss, const Timestamp& collTimestamp) { const auto collUuid = UUID::gen(); const auto collEpoch = OID::gen(); - makeCollection(nss, collUuid, collEpoch, *collTimestamp); + makeCollection(nss, collUuid, collEpoch, collTimestamp); ASSERT_THROWS_CODE(ShardingCatalogManager::get(operationContext()) ->clearJumboFlag(operationContext(), nss, OID::gen(), jumboChunk()), @@ -148,10 +146,10 @@ TEST_F(ClearJumboFlagTest, AssertsOnEpochMismatch) { } TEST_F(ClearJumboFlagTest, AssertsIfChunkCantBeFound) { - auto test = [&](const NamespaceString& nss, const boost::optional& collTimestamp) { + auto test = [&](const NamespaceString& nss, const Timestamp& collTimestamp) { const auto collEpoch = OID::gen(); const auto collUuid = UUID::gen(); - makeCollection(nss, collUuid, collEpoch, *collTimestamp); + makeCollection(nss, collUuid, collEpoch, collTimestamp); ChunkRange imaginaryChunk(BSON("x" << 0), BSON("x" << 10)); ASSERT_THROWS(ShardingCatalogManager::get(operationContext()) diff --git a/src/mongo/db/s/config/sharding_catalog_manager_collection_operations.cpp b/src/mongo/db/s/config/sharding_catalog_manager_collection_operations.cpp index 591917bfad4..aa25021aa4e 100644 --- a/src/mongo/db/s/config/sharding_catalog_manager_collection_operations.cpp +++ b/src/mongo/db/s/config/sharding_catalog_manager_collection_operations.cpp @@ -134,7 +134,6 @@ boost::optional checkCollectionOptions(OperationContext* opCtx, } void triggerFireAndForgetShardRefreshes(OperationContext* opCtx, const CollectionType& coll) { - invariant(coll.getTimestamp()); const auto shardRegistry = Grid::get(opCtx)->shardRegistry(); const auto allShards = uassertStatusOK(Grid::get(opCtx)->catalogClient()->getAllShards( opCtx, repl::ReadConcernLevel::kLocalReadConcern)) @@ -177,7 +176,6 @@ void triggerFireAndForgetShardRefreshes(OperationContext* opCtx, const Collectio // // The chunk updates: // [{$set: { -// lastmodEpoch: , // min: {$arrayToObject: {$concatArrays: [ // {$objectToArray: "$min"}, // {$literal: [{k: , v: MinKey}, ...]}, @@ -225,9 +223,7 @@ void triggerFireAndForgetShardRefreshes(OperationContext* opCtx, const Collectio // }} // }}] std::pair, std::vector> makeChunkAndTagUpdatesForRefine( - const BSONObj& newShardKeyFields, - OID newEpoch, - const boost::optional& newTimestamp) { + const BSONObj& newShardKeyFields) { // Make the $literal objects used in the $set below to add new fields to the boundaries of the // existing chunks and tags that may include "." characters. // @@ -273,12 +269,9 @@ std::pair, std::vector> makeChunkAndTagUpdatesForR << "then" << literalMaxObject << "else" << literalMinObject)))))))); - // The chunk updates change the min and max fields and unset the jumbo field. If the collection - // is in the old (pre-5.0 format, it also sets the new epoch). + // The chunk updates change the min and max fields and unset the jumbo field. std::vector chunkUpdates; - chunkUpdates.emplace_back(BSON("$set" << (newTimestamp ? extendMinAndMaxModifier.getOwned() - : extendMinAndMaxModifier.addFields(BSON( - ChunkType::epoch(newEpoch)))))); + chunkUpdates.emplace_back(BSON("$set" << extendMinAndMaxModifier.getOwned())); chunkUpdates.emplace_back(BSON("$unset" << ChunkType::jumbo())); // The tag updates only change the min and max fields. @@ -319,12 +312,9 @@ void ShardingCatalogManager::refineCollectionShardKey(OperationContext* opCtx, collType.setEpoch(newEpoch); collType.setKeyPattern(newShardKeyPattern.getKeyPattern()); - boost::optional newTimestamp; - if (collType.getTimestamp()) { - auto now = VectorClock::get(opCtx)->getTime(); - newTimestamp = now.clusterTime().asTimestamp(); - collType.setTimestamp(newTimestamp); - } + auto now = VectorClock::get(opCtx)->getTime(); + Timestamp newTimestamp = now.clusterTime().asTimestamp(); + collType.setTimestamp(newTimestamp); auto updateCollectionAndChunksFn = [&](OperationContext* opCtx, TxnNumber txnNumber) { // Update the config.collections entry for the given namespace. @@ -345,13 +335,11 @@ void ShardingCatalogManager::refineCollectionShardKey(OperationContext* opCtx, hangRefineCollectionShardKeyBeforeUpdatingChunks.pauseWhileSet(opCtx); } - auto [chunkUpdates, tagUpdates] = - makeChunkAndTagUpdatesForRefine(newFields, newEpoch, newTimestamp); + auto [chunkUpdates, tagUpdates] = makeChunkAndTagUpdatesForRefine(newFields); - // Update all config.chunks entries for the given namespace by setting (i) their epoch - // to the newly-generated objectid, (ii) their bounds for each new field in the refined - // key to MinKey (except for the global max chunk where the max bounds are set to - // MaxKey), and unsetting (iii) their jumbo field. + // Update all config.chunks entries for the given namespace by setting (i) their bounds for + // each new field in the refined key to MinKey (except for the global max chunk where the + // max bounds are set to MaxKey), and unsetting (ii) their jumbo field. const auto chunksQuery = BSON(ChunkType::collectionUUID << collType.getUuid()); writeToConfigDocumentInTxn( opCtx, diff --git a/src/mongo/db/s/config/sharding_catalog_manager_ensure_chunk_version_is_greater_than_test.cpp b/src/mongo/db/s/config/sharding_catalog_manager_ensure_chunk_version_is_greater_than_test.cpp index 4d511518aca..fa1a1818897 100644 --- a/src/mongo/db/s/config/sharding_catalog_manager_ensure_chunk_version_is_greater_than_test.cpp +++ b/src/mongo/db/s/config/sharding_catalog_manager_ensure_chunk_version_is_greater_than_test.cpp @@ -92,13 +92,12 @@ void assertChunkVersionWasBumpedTo(const ChunkType& chunkTypeBefore, } TEST_F(EnsureChunkVersionIsGreaterThanTest, IfNoCollectionFoundReturnsSuccess) { - const auto requestedChunkType = - generateChunkType(_nss, - _collUuid, - ChunkVersion(10, 2, OID::gen(), boost::none /* timestamp */), - ShardId(_shardName), - BSON("a" << 1), - BSON("a" << 10)); + const auto requestedChunkType = generateChunkType(_nss, + _collUuid, + ChunkVersion(10, 2, OID::gen(), Timestamp()), + ShardId(_shardName), + BSON("a" << 1), + BSON("a" << 10)); ShardingCatalogManager::get(operationContext()) ->ensureChunkVersionIsGreaterThan(operationContext(), diff --git a/src/mongo/db/s/config/sharding_catalog_manager_merge_chunks_test.cpp b/src/mongo/db/s/config/sharding_catalog_manager_merge_chunks_test.cpp index 17b1f907751..2d681f93e09 100644 --- a/src/mongo/db/s/config/sharding_catalog_manager_merge_chunks_test.cpp +++ b/src/mongo/db/s/config/sharding_catalog_manager_merge_chunks_test.cpp @@ -60,7 +60,7 @@ protected: TEST_F(MergeChunkTest, MergeExistingChunksCorrectlyShouldSucceed) { const auto collEpoch = OID::gen(); - const boost::optional collTimestamp(42); + const Timestamp collTimestamp(42); const auto collUuid = UUID::gen(); ChunkType chunk; @@ -141,7 +141,7 @@ TEST_F(MergeChunkTest, MergeExistingChunksCorrectlyShouldSucceed) { TEST_F(MergeChunkTest, MergeSeveralChunksCorrectlyShouldSucceed) { const auto collEpoch = OID::gen(); - const boost::optional collTimestamp(42); + const Timestamp collTimestamp(42); const auto collUuid = UUID::gen(); ChunkType chunk; chunk.setName(OID::gen()); @@ -214,7 +214,7 @@ TEST_F(MergeChunkTest, MergeSeveralChunksCorrectlyShouldSucceed) { TEST_F(MergeChunkTest, NewMergeShouldClaimHighestVersion) { const auto collEpoch = OID::gen(); - const boost::optional collTimestamp(42); + const Timestamp collTimestamp(42); const auto collUuid = UUID::gen(); ChunkType chunk, otherChunk; chunk.setName(OID::gen()); @@ -292,7 +292,7 @@ TEST_F(MergeChunkTest, NewMergeShouldClaimHighestVersion) { TEST_F(MergeChunkTest, MergeLeavesOtherChunksAlone) { const auto collEpoch = OID::gen(); - const boost::optional collTimestamp(42); + const Timestamp collTimestamp(42); const auto collUuid = UUID::gen(); ShardId shardId(_shardName); ChunkType chunk; @@ -370,7 +370,7 @@ TEST_F(MergeChunkTest, MergeLeavesOtherChunksAlone) { TEST_F(MergeChunkTest, NonExistingNamespace) { const auto collEpoch = OID::gen(); const auto collUuidAtRequest = UUID::gen(); - const boost::optional collTimestamp(42); + const Timestamp collTimestamp(42); ChunkType chunk; chunk.setCollectionUUID(UUID::gen()); @@ -408,7 +408,7 @@ TEST_F(MergeChunkTest, NonExistingNamespace) { TEST_F(MergeChunkTest, NonMatchingUUIDsOfChunkAndRequestErrors) { const auto collEpoch = OID::gen(); - const boost::optional collTimestamp(42); + const Timestamp collTimestamp(42); const auto collUuid = UUID::gen(); const auto requestUuid = UUID::gen(); ChunkType chunk; @@ -446,7 +446,7 @@ TEST_F(MergeChunkTest, NonMatchingUUIDsOfChunkAndRequestErrors) { TEST_F(MergeChunkTest, MergeAlreadyHappenedSucceeds) { const auto collEpoch = OID::gen(); - const boost::optional collTimestamp(42); + const Timestamp collTimestamp(42); const auto collUuid = UUID::gen(); // Construct chunk range to be merged @@ -498,7 +498,7 @@ TEST_F(MergeChunkTest, MergeAlreadyHappenedSucceeds) { TEST_F(MergeChunkTest, MergingChunksWithDollarPrefixShouldSucceed) { const auto collEpoch = OID::gen(); - const boost::optional collTimestamp(42); + const Timestamp collTimestamp(42); const auto collUuid = UUID::gen(); ChunkType chunk1; chunk1.setName(OID::gen()); diff --git a/src/mongo/db/s/config/sharding_catalog_manager_remove_shard_test.cpp b/src/mongo/db/s/config/sharding_catalog_manager_remove_shard_test.cpp index 397eca70a61..55af4454e3d 100644 --- a/src/mongo/db/s/config/sharding_catalog_manager_remove_shard_test.cpp +++ b/src/mongo/db/s/config/sharding_catalog_manager_remove_shard_test.cpp @@ -200,17 +200,18 @@ TEST_F(RemoveShardTest, RemoveShardStillDrainingChunksRemaining) { auto epoch = OID::gen(); const auto uuid = UUID::gen(); + const auto timestamp = Timestamp(1); ChunkType chunk1(uuid, ChunkRange(BSON("_id" << 0), BSON("_id" << 20)), - ChunkVersion(1, 1, epoch, boost::none /* timestamp */), + ChunkVersion(1, 1, epoch, timestamp), shard1.getName()); ChunkType chunk2(uuid, ChunkRange(BSON("_id" << 21), BSON("_id" << 50)), - ChunkVersion(1, 2, epoch, boost::none /* timestamp */), + ChunkVersion(1, 2, epoch, timestamp), shard1.getName()); ChunkType chunk3(uuid, ChunkRange(BSON("_id" << 51), BSON("_id" << 1000)), - ChunkVersion(1, 3, epoch, boost::none /* timestamp */), + ChunkVersion(1, 3, epoch, timestamp), shard1.getName()); chunk3.setJumbo(true); @@ -286,17 +287,18 @@ TEST_F(RemoveShardTest, RemoveShardCompletion) { auto epoch = OID::gen(); auto uuid = UUID::gen(); + Timestamp timestamp = Timestamp(1); ChunkType chunk1(uuid, ChunkRange(BSON("_id" << 0), BSON("_id" << 20)), - ChunkVersion(1, 1, epoch, boost::none /* timestamp */), + ChunkVersion(1, 1, epoch, timestamp), shard1.getName()); ChunkType chunk2(uuid, ChunkRange(BSON("_id" << 21), BSON("_id" << 50)), - ChunkVersion(1, 2, epoch, boost::none /* timestamp */), + ChunkVersion(1, 2, epoch, timestamp), shard1.getName()); ChunkType chunk3(uuid, ChunkRange(BSON("_id" << 51), BSON("_id" << 1000)), - ChunkVersion(1, 3, epoch, boost::none /* timestamp */), + ChunkVersion(1, 3, epoch, timestamp), shard1.getName()); std::vector chunks{chunk1, chunk2, chunk3}; diff --git a/src/mongo/db/s/config/sharding_catalog_manager_split_chunk_test.cpp b/src/mongo/db/s/config/sharding_catalog_manager_split_chunk_test.cpp index 525011fd0ee..e7986eb2a27 100644 --- a/src/mongo/db/s/config/sharding_catalog_manager_split_chunk_test.cpp +++ b/src/mongo/db/s/config/sharding_catalog_manager_split_chunk_test.cpp @@ -57,7 +57,7 @@ protected: }; TEST_F(SplitChunkTest, SplitExistingChunkCorrectlyShouldSucceed) { - auto test = [&](const NamespaceString& nss, const boost::optional& collTimestamp) { + auto test = [&](const NamespaceString& nss, const Timestamp& collTimestamp) { const auto collEpoch = OID::gen(); const auto collUuid = UUID::gen(); @@ -100,9 +100,6 @@ TEST_F(SplitChunkTest, SplitExistingChunkCorrectlyShouldSucceed) { ASSERT_EQ(expectedShardVersion, shardVersion); ASSERT_EQ(shardVersion, collVersion); - const auto nssOrUuid = - collTimestamp ? NamespaceStringOrUUID(nss.db().toString(), collUuid) : nss; - // First chunkDoc should have range [chunkMin, chunkSplitPoint] auto chunkDocStatus = getChunkDoc(operationContext(), collUuid, chunkMin, collEpoch, collTimestamp); @@ -141,7 +138,7 @@ TEST_F(SplitChunkTest, SplitExistingChunkCorrectlyShouldSucceed) { } TEST_F(SplitChunkTest, MultipleSplitsOnExistingChunkShouldSucceed) { - auto test = [&](const NamespaceString& nss, const boost::optional& collTimestamp) { + auto test = [&](const NamespaceString& nss, const Timestamp& collTimestamp) { const auto collEpoch = OID::gen(); const auto collUuid = UUID::gen(); @@ -174,9 +171,6 @@ TEST_F(SplitChunkTest, MultipleSplitsOnExistingChunkShouldSucceed) { splitPoints, "shard0000")); - const auto nssOrUuid = - collTimestamp ? NamespaceStringOrUUID(nss.db().toString(), collUuid) : nss; - // First chunkDoc should have range [chunkMin, chunkSplitPoint] auto chunkDocStatus = getChunkDoc(operationContext(), collUuid, chunkMin, collEpoch, collTimestamp); @@ -231,7 +225,7 @@ TEST_F(SplitChunkTest, MultipleSplitsOnExistingChunkShouldSucceed) { } TEST_F(SplitChunkTest, NewSplitShouldClaimHighestVersion) { - auto test = [&](const NamespaceString& nss, const boost::optional& collTimestamp) { + auto test = [&](const NamespaceString& nss, const Timestamp& collTimestamp) { const auto collEpoch = OID::gen(); const auto collUuid = UUID::gen(); @@ -272,9 +266,6 @@ TEST_F(SplitChunkTest, NewSplitShouldClaimHighestVersion) { splitPoints, "shard0000")); - const auto nssOrUuid = - collTimestamp ? NamespaceStringOrUUID(nss.db().toString(), collUuid) : nss; - // First chunkDoc should have range [chunkMin, chunkSplitPoint] auto chunkDocStatus = getChunkDoc(operationContext(), collUuid, chunkMin, collEpoch, collTimestamp); @@ -304,7 +295,7 @@ TEST_F(SplitChunkTest, NewSplitShouldClaimHighestVersion) { } TEST_F(SplitChunkTest, PreConditionFailErrors) { - auto test = [&](const NamespaceString& nss, const boost::optional& collTimestamp) { + auto test = [&](const NamespaceString& nss, const Timestamp& collTimestamp) { const auto collEpoch = OID::gen(); ChunkType chunk; @@ -340,7 +331,7 @@ TEST_F(SplitChunkTest, PreConditionFailErrors) { } TEST_F(SplitChunkTest, NonExisingNamespaceErrors) { - auto test = [&](const NamespaceString& nss, const boost::optional& collTimestamp) { + auto test = [&](const NamespaceString& nss, const Timestamp& collTimestamp) { const auto collEpoch = OID::gen(); ChunkType chunk; @@ -373,7 +364,7 @@ TEST_F(SplitChunkTest, NonExisingNamespaceErrors) { } TEST_F(SplitChunkTest, NonMatchingEpochsOfChunkAndRequestErrors) { - auto test = [&](const NamespaceString& nss, const boost::optional& collTimestamp) { + auto test = [&](const NamespaceString& nss, const Timestamp& collTimestamp) { const auto collEpoch = OID::gen(); ChunkType chunk; @@ -406,7 +397,7 @@ TEST_F(SplitChunkTest, NonMatchingEpochsOfChunkAndRequestErrors) { } TEST_F(SplitChunkTest, SplitPointsOutOfOrderShouldFail) { - auto test = [&](const NamespaceString& nss, const boost::optional& collTimestamp) { + auto test = [&](const NamespaceString& nss, const Timestamp& collTimestamp) { const auto collEpoch = OID::gen(); ChunkType chunk; @@ -440,7 +431,7 @@ TEST_F(SplitChunkTest, SplitPointsOutOfOrderShouldFail) { } TEST_F(SplitChunkTest, SplitPointsOutOfRangeAtMinShouldFail) { - auto test = [&](const NamespaceString& nss, const boost::optional& collTimestamp) { + auto test = [&](const NamespaceString& nss, const Timestamp& collTimestamp) { const auto collEpoch = OID::gen(); ChunkType chunk; @@ -473,7 +464,7 @@ TEST_F(SplitChunkTest, SplitPointsOutOfRangeAtMinShouldFail) { } TEST_F(SplitChunkTest, SplitPointsOutOfRangeAtMaxShouldFail) { - auto test = [&](const NamespaceString& nss, const boost::optional& collTimestamp) { + auto test = [&](const NamespaceString& nss, const Timestamp& collTimestamp) { const auto collEpoch = OID::gen(); ChunkType chunk; @@ -507,7 +498,7 @@ TEST_F(SplitChunkTest, SplitPointsOutOfRangeAtMaxShouldFail) { } TEST_F(SplitChunkTest, SplitPointsWithDollarPrefixShouldFail) { - auto test = [&](const NamespaceString& nss, const boost::optional& collTimestamp) { + auto test = [&](const NamespaceString& nss, const Timestamp& collTimestamp) { const auto collEpoch = OID::gen(); ChunkType chunk; diff --git a/src/mongo/db/s/metadata_manager_test.cpp b/src/mongo/db/s/metadata_manager_test.cpp index 2b2b4c846b7..948821e2f91 100644 --- a/src/mongo/db/s/metadata_manager_test.cpp +++ b/src/mongo/db/s/metadata_manager_test.cpp @@ -85,13 +85,12 @@ protected: nullptr, false, epoch, - boost::none /* timestamp */, + Timestamp(), boost::none /* timeseriesFields */, boost::none, boost::none /* chunkSizeBytes */, true, - {ChunkType{ - uuid, range, ChunkVersion(1, 0, epoch, boost::none /* timestamp */), kOtherShard}}); + {ChunkType{uuid, range, ChunkVersion(1, 0, epoch, Timestamp()), kOtherShard}}); return CollectionMetadata(ChunkManager(kThisShard, DatabaseVersion(UUID::gen(), Timestamp()), diff --git a/src/mongo/db/s/migration_chunk_cloner_source_legacy_test.cpp b/src/mongo/db/s/migration_chunk_cloner_source_legacy_test.cpp index c6591e51e52..73a39a676b2 100644 --- a/src/mongo/db/s/migration_chunk_cloner_source_legacy_test.cpp +++ b/src/mongo/db/s/migration_chunk_cloner_source_legacy_test.cpp @@ -162,6 +162,7 @@ protected: [&] { const OID epoch = OID::gen(); + const Timestamp timestamp(1); auto rt = RoutingTableHistory::makeNew( kNss, @@ -170,14 +171,14 @@ protected: nullptr, false, epoch, - boost::none /* timestamp */, + timestamp, boost::none /* timeseriesFields */, boost::none /* resharding Fields */, boost::none /* chunkSizeBytes */, true, {ChunkType{uuid, ChunkRange{BSON(kShardKey << MINKEY), BSON(kShardKey << MAXKEY)}, - ChunkVersion(1, 0, epoch, boost::none /* timestamp */), + ChunkVersion(1, 0, epoch, timestamp), ShardId("dummyShardId")}}); AutoGetDb autoDb(operationContext(), kNss.db(), MODE_IX); @@ -206,7 +207,7 @@ protected: MoveChunkRequest::appendAsCommand( &cmdBuilder, kNss, - ChunkVersion(1, 0, OID::gen(), boost::none /* timestamp */), + ChunkVersion(1, 0, OID::gen(), Timestamp()), kDonorConnStr.getSetName(), kRecipientConnStr.getSetName(), chunkRange, diff --git a/src/mongo/db/s/migration_util_test.cpp b/src/mongo/db/s/migration_util_test.cpp index ccb1ae3bbec..c640421d435 100644 --- a/src/mongo/db/s/migration_util_test.cpp +++ b/src/mongo/db/s/migration_util_test.cpp @@ -350,7 +350,7 @@ public: const ShardKeyPattern kShardKeyPattern = ShardKeyPattern(BSON("_id" << 1)); const UUID kDefaultUUID = UUID::gen(); const OID kEpoch = OID::gen(); - const Timestamp kDefaultTimestamp = Timestamp(); + const Timestamp kDefaultTimestamp = Timestamp(1); const DatabaseType kDefaultDatabaseType = DatabaseType( kNss.db().toString(), ShardId("0"), true, DatabaseVersion(kDefaultUUID, kDefaultTimestamp)); const std::vector kShardList = {ShardType("0", "Host0:12345"), @@ -440,8 +440,8 @@ public: return mockCatalogClient; } - CollectionType makeCollectionType(UUID uuid, OID epoch) { - CollectionType coll(kNss, epoch, Date_t::now(), uuid); + CollectionType makeCollectionType(UUID uuid, OID epoch, Timestamp timestamp) { + CollectionType coll(kNss, epoch, timestamp, Date_t::now(), uuid); coll.setKeyPattern(kShardKeyPattern.getKeyPattern()); coll.setUnique(true); return coll; @@ -572,11 +572,11 @@ TEST_F(SubmitRangeDeletionTaskTest, SucceedsIfFilteringMetadataUUIDMatchesTaskUU migrationutil::markAsReadyRangeDeletionTaskLocally(opCtx, deletionTask.getId()); // Force a metadata refresh with the task's UUID before the task is submitted. - auto coll = makeCollectionType(collectionUUID, kEpoch); + auto coll = makeCollectionType(collectionUUID, kEpoch, kDefaultTimestamp); _mockCatalogCacheLoader->setDatabaseRefreshReturnValue(kDefaultDatabaseType); _mockCatalogCacheLoader->setCollectionRefreshReturnValue(coll); _mockCatalogCacheLoader->setChunkRefreshReturnValue( - makeChangedChunks(ChunkVersion(1, 0, kEpoch, boost::none /* timestamp */))); + makeChangedChunks(ChunkVersion(1, 0, kEpoch, kDefaultTimestamp))); _mockCatalogClient->setCollections({coll}); forceShardFilteringMetadataRefresh(opCtx, kNss); @@ -600,11 +600,11 @@ TEST_F( migrationutil::markAsReadyRangeDeletionTaskLocally(opCtx, deletionTask.getId()); // Make the refresh triggered by submitting the task return a UUID that matches the task's UUID. - auto coll = makeCollectionType(collectionUUID, kEpoch); + auto coll = makeCollectionType(collectionUUID, kEpoch, kDefaultTimestamp); _mockCatalogCacheLoader->setDatabaseRefreshReturnValue(kDefaultDatabaseType); _mockCatalogCacheLoader->setCollectionRefreshReturnValue(coll); _mockCatalogCacheLoader->setChunkRefreshReturnValue( - makeChangedChunks(ChunkVersion(1, 0, kEpoch, boost::none /* timestamp */))); + makeChangedChunks(ChunkVersion(1, 0, kEpoch, kDefaultTimestamp))); _mockCatalogClient->setCollections({coll}); // The task should have been submitted successfully. @@ -633,10 +633,10 @@ TEST_F(SubmitRangeDeletionTaskTest, migrationutil::markAsReadyRangeDeletionTaskLocally(opCtx, deletionTask.getId()); // Make the refresh triggered by submitting the task return a UUID that matches the task's UUID. - auto matchingColl = makeCollectionType(collectionUUID, kEpoch); + auto matchingColl = makeCollectionType(collectionUUID, kEpoch, kDefaultTimestamp); _mockCatalogCacheLoader->setCollectionRefreshReturnValue(matchingColl); _mockCatalogCacheLoader->setChunkRefreshReturnValue( - makeChangedChunks(ChunkVersion(10, 0, kEpoch, boost::none /* timestamp */))); + makeChangedChunks(ChunkVersion(10, 0, kEpoch, kDefaultTimestamp))); _mockCatalogClient->setCollections({matchingColl}); // The task should have been submitted successfully. @@ -652,11 +652,12 @@ TEST_F(SubmitRangeDeletionTaskTest, // stale when the task is submitted. const auto staleUUID = UUID::gen(); const auto staleEpoch = OID::gen(); - auto staleColl = makeCollectionType(staleUUID, staleEpoch); + const auto staleTimestamp = Timestamp(0); + auto staleColl = makeCollectionType(staleUUID, staleEpoch, staleTimestamp); _mockCatalogCacheLoader->setDatabaseRefreshReturnValue(kDefaultDatabaseType); _mockCatalogCacheLoader->setCollectionRefreshReturnValue(staleColl); _mockCatalogCacheLoader->setChunkRefreshReturnValue( - makeChangedChunks(ChunkVersion(1, 0, staleEpoch, boost::none /* timestamp */))); + makeChangedChunks(ChunkVersion(1, 0, staleEpoch, staleTimestamp))); _mockCatalogClient->setCollections({staleColl}); forceShardFilteringMetadataRefresh(opCtx, kNss); @@ -670,10 +671,10 @@ TEST_F(SubmitRangeDeletionTaskTest, migrationutil::markAsReadyRangeDeletionTaskLocally(opCtx, deletionTask.getId()); // Make the refresh triggered by submitting the task return a UUID that matches the task's UUID. - auto matchingColl = makeCollectionType(collectionUUID, kEpoch); + auto matchingColl = makeCollectionType(collectionUUID, kEpoch, kDefaultTimestamp); _mockCatalogCacheLoader->setCollectionRefreshReturnValue(matchingColl); _mockCatalogCacheLoader->setChunkRefreshReturnValue( - makeChangedChunks(ChunkVersion(10, 0, kEpoch, boost::none /* timestamp */))); + makeChangedChunks(ChunkVersion(10, 0, kEpoch, kDefaultTimestamp))); _mockCatalogClient->setCollections({matchingColl}); // The task should have been submitted successfully. @@ -695,11 +696,12 @@ TEST_F(SubmitRangeDeletionTaskTest, // Make the refresh triggered by submitting the task return an arbitrary UUID. const auto otherEpoch = OID::gen(); - auto otherColl = makeCollectionType(UUID::gen(), otherEpoch); + const auto otherTimestamp = Timestamp(2); + auto otherColl = makeCollectionType(UUID::gen(), otherEpoch, otherTimestamp); _mockCatalogCacheLoader->setDatabaseRefreshReturnValue(kDefaultDatabaseType); _mockCatalogCacheLoader->setCollectionRefreshReturnValue(otherColl); _mockCatalogCacheLoader->setChunkRefreshReturnValue( - makeChangedChunks(ChunkVersion(1, 0, otherEpoch, boost::none /* timestamp */))); + makeChangedChunks(ChunkVersion(1, 0, otherEpoch, otherTimestamp))); _mockCatalogClient->setCollections({otherColl}); // The task should not have been submitted, and the task's entry should have been removed from diff --git a/src/mongo/db/s/op_observer_sharding_test.cpp b/src/mongo/db/s/op_observer_sharding_test.cpp index b0e6de71d6c..2ae932483e2 100644 --- a/src/mongo/db/s/op_observer_sharding_test.cpp +++ b/src/mongo/db/s/op_observer_sharding_test.cpp @@ -65,17 +65,15 @@ protected: const UUID uuid = UUID::gen(); const OID epoch = OID::gen(); auto range = ChunkRange(BSON("key" << MINKEY), BSON("key" << MAXKEY)); - auto chunk = ChunkType(uuid, - std::move(range), - ChunkVersion(1, 0, epoch, boost::none /* timestamp */), - ShardId("other")); + auto chunk = ChunkType( + uuid, std::move(range), ChunkVersion(1, 0, epoch, Timestamp()), ShardId("other")); auto rt = RoutingTableHistory::makeNew(kTestNss, uuid, KeyPattern(keyPattern), nullptr, false, epoch, - boost::none /* timestamp */, + Timestamp(), boost::none /* timeseriesFields */, boost::none, boost::none /* chunkSizeBytes */, diff --git a/src/mongo/db/s/range_deletion_util_test.cpp b/src/mongo/db/s/range_deletion_util_test.cpp index fc5b0d61787..bc87d18e0f8 100644 --- a/src/mongo/db/s/range_deletion_util_test.cpp +++ b/src/mongo/db/s/range_deletion_util_test.cpp @@ -105,14 +105,14 @@ public: nullptr, false, epoch, - boost::none /* timestamp */, + Timestamp(), boost::none /* timeseriesFields */, boost::none, boost::none /* chunkSizeBytes */, true, {ChunkType{uuid, ChunkRange{BSON(kShardKey << MINKEY), BSON(kShardKey << MAXKEY)}, - ChunkVersion(1, 0, epoch, boost::none /* timestamp */), + ChunkVersion(1, 0, epoch, Timestamp()), ShardId("dummyShardId")}}); AutoGetDb autoDb(operationContext(), kNss.db(), MODE_IX); diff --git a/src/mongo/db/s/resharding/resharding_coordinator_service.cpp b/src/mongo/db/s/resharding/resharding_coordinator_service.cpp index ea3d2c37ca6..082699876ee 100644 --- a/src/mongo/db/s/resharding/resharding_coordinator_service.cpp +++ b/src/mongo/db/s/resharding/resharding_coordinator_service.cpp @@ -831,7 +831,6 @@ ReshardingCoordinatorExternalStateImpl::calculateParticipantShardsAndChunks( if (const auto& chunks = coordinatorDoc.getPresetReshardedChunks()) { auto version = calculateChunkVersionForInitialChunks(opCtx); - invariant(version.getTimestamp()); // Use the provided shardIds from presetReshardedChunks to construct the // recipient list. diff --git a/src/mongo/db/s/resharding/resharding_coordinator_service_test.cpp b/src/mongo/db/s/resharding/resharding_coordinator_service_test.cpp index d91d0b19836..d7b736119a0 100644 --- a/src/mongo/db/s/resharding/resharding_coordinator_service_test.cpp +++ b/src/mongo/db/s/resharding/resharding_coordinator_service_test.cpp @@ -87,7 +87,6 @@ class ExternalStateForTest : public ReshardingCoordinatorExternalState { // Use the provided shardIds from presetReshardedChunks to construct the // recipient list. if (const auto& chunks = coordinatorDoc.getPresetReshardedChunks()) { - invariant(version.getTimestamp()); for (const auto& reshardedChunk : *chunks) { initialChunks.emplace_back( coordinatorDoc.getReshardingUUID(), @@ -404,13 +403,14 @@ public: std::vector makeChunks(const UUID& uuid, OID epoch, + const Timestamp& timestamp, const ShardKeyPattern& shardKey, std::vector ids) { auto chunkRanges = _newShardKey.isShardKey(shardKey.toBSON()) ? _newChunkRanges : _oldChunkRanges; // Create two chunks, one on each shard with the given namespace and epoch - ChunkVersion version(1, 0, epoch, boost::none /* timestamp */); + ChunkVersion version(1, 0, epoch, timestamp); ChunkType chunk1(uuid, chunkRanges[0], version, ShardId("shard0000")); chunk1.setName(ids[0]); version.incMinor(); @@ -423,9 +423,10 @@ public: // Returns the chunk for the donor shard. ChunkType makeAndInsertChunksForDonorShard(const UUID& uuid, OID epoch, + const Timestamp& timestamp, const ShardKeyPattern& shardKey, std::vector ids) { - auto chunks = makeChunks(uuid, epoch, shardKey, ids); + auto chunks = makeChunks(uuid, epoch, timestamp, shardKey, ids); // Only the chunk corresponding to shard0000 is stored as a donor in the coordinator state // document constructed. @@ -508,10 +509,12 @@ public: NamespaceString _originalNss = NamespaceString("db.foo"); UUID _originalUUID = UUID::gen(); OID _originalEpoch = OID::gen(); + Timestamp _originalTimestamp = Timestamp(1); NamespaceString _tempNss = NamespaceString("db.system.resharding." + _originalUUID.toString()); UUID _reshardingUUID = UUID::gen(); OID _tempEpoch = OID::gen(); + Timestamp _tempTimestamp = Timestamp(2); ShardKeyPattern _oldShardKey = ShardKeyPattern(BSON("oldShardKey" << 1)); ShardKeyPattern _newShardKey = ShardKeyPattern(BSON("newShardKey" << 1)); @@ -545,11 +548,17 @@ TEST_F(ReshardingCoordinatorServiceTest, ReshardingCoordinatorSuccessfullyTransi auto doc = insertStateAndCatalogEntries(CoordinatorStateEnum::kUnused, _originalEpoch); auto opCtx = operationContext(); - auto donorChunk = makeAndInsertChunksForDonorShard( - _originalUUID, _originalEpoch, _oldShardKey, std::vector{OID::gen(), OID::gen()}); - - auto initialChunks = - makeChunks(_reshardingUUID, _tempEpoch, _newShardKey, std::vector{OID::gen(), OID::gen()}); + auto donorChunk = makeAndInsertChunksForDonorShard(_originalUUID, + _originalEpoch, + _originalTimestamp, + _oldShardKey, + std::vector{OID::gen(), OID::gen()}); + + auto initialChunks = makeChunks(_reshardingUUID, + _tempEpoch, + _tempTimestamp, + _newShardKey, + std::vector{OID::gen(), OID::gen()}); std::vector presetReshardedChunks; for (const auto& chunk : initialChunks) { @@ -603,11 +612,17 @@ TEST_F(ReshardingCoordinatorServiceTest, ReshardingCoordinatorTransitionsTokDone auto doc = insertStateAndCatalogEntries(CoordinatorStateEnum::kUnused, _originalEpoch); auto opCtx = operationContext(); - auto donorChunk = makeAndInsertChunksForDonorShard( - _originalUUID, _originalEpoch, _oldShardKey, std::vector{OID::gen(), OID::gen()}); - - auto initialChunks = - makeChunks(_reshardingUUID, _tempEpoch, _newShardKey, std::vector{OID::gen(), OID::gen()}); + auto donorChunk = makeAndInsertChunksForDonorShard(_originalUUID, + _originalEpoch, + _originalTimestamp, + _oldShardKey, + std::vector{OID::gen(), OID::gen()}); + + auto initialChunks = makeChunks(_reshardingUUID, + _tempEpoch, + _tempTimestamp, + _newShardKey, + std::vector{OID::gen(), OID::gen()}); std::vector presetReshardedChunks; for (const auto& chunk : initialChunks) { @@ -667,11 +682,17 @@ TEST_F(ReshardingCoordinatorServiceTest, StepDownStepUpDuringInitializing) { doc.setRecipientShards({}); doc.setDonorShards({}); - auto donorChunk = makeAndInsertChunksForDonorShard( - _originalUUID, _originalEpoch, _oldShardKey, std::vector{OID::gen(), OID::gen()}); + auto donorChunk = makeAndInsertChunksForDonorShard(_originalUUID, + _originalEpoch, + _originalTimestamp, + _oldShardKey, + std::vector{OID::gen(), OID::gen()}); - auto initialChunks = - makeChunks(_reshardingUUID, _tempEpoch, _newShardKey, std::vector{OID::gen(), OID::gen()}); + auto initialChunks = makeChunks(_reshardingUUID, + _tempEpoch, + _tempTimestamp, + _newShardKey, + std::vector{OID::gen(), OID::gen()}); std::vector presetReshardedChunks; for (const auto& chunk : initialChunks) { @@ -727,11 +748,17 @@ TEST_F(ReshardingCoordinatorServiceTest, StepDownStepUpEachTransition) { auto doc = insertStateAndCatalogEntries(CoordinatorStateEnum::kUnused, _originalEpoch); auto opCtx = operationContext(); - auto donorChunk = makeAndInsertChunksForDonorShard( - _originalUUID, _originalEpoch, _oldShardKey, std::vector{OID::gen(), OID::gen()}); - - auto initialChunks = - makeChunks(_reshardingUUID, _tempEpoch, _newShardKey, std::vector{OID::gen(), OID::gen()}); + auto donorChunk = makeAndInsertChunksForDonorShard(_originalUUID, + _originalEpoch, + _originalTimestamp, + _oldShardKey, + std::vector{OID::gen(), OID::gen()}); + + auto initialChunks = makeChunks(_reshardingUUID, + _tempEpoch, + _tempTimestamp, + _newShardKey, + std::vector{OID::gen(), OID::gen()}); std::vector presetReshardedChunks; for (const auto& chunk : initialChunks) { @@ -841,7 +868,7 @@ TEST_F(ReshardingCoordinatorServiceTest, StepDownStepUpEachTransition) { ChunkType::ConfigNS, BSON(ChunkType::collectionUUID() << doc.getReshardingUUID())); while (chunkCursor->more()) { auto d = uassertStatusOK(ChunkType::fromConfigBSON( - chunkCursor->nextSafe().getOwned(), _originalEpoch, boost::none)); + chunkCursor->nextSafe().getOwned(), _originalEpoch, _originalTimestamp)); foundChunks.push_back(d); } ASSERT_EQUALS(foundChunks.size(), initialChunks.size()); @@ -862,11 +889,17 @@ TEST_F(ReshardingCoordinatorServiceTest, StepDownStepUpEachTransition) { TEST_F(ReshardingCoordinatorServiceTest, ReshardingCoordinatorFailsIfMigrationNotAllowed) { auto doc = insertStateAndCatalogEntries(CoordinatorStateEnum::kUnused, _originalEpoch); auto opCtx = operationContext(); - auto donorChunk = makeAndInsertChunksForDonorShard( - _originalUUID, _originalEpoch, _oldShardKey, std::vector{OID::gen(), OID::gen()}); - - auto initialChunks = - makeChunks(_reshardingUUID, _tempEpoch, _newShardKey, std::vector{OID::gen(), OID::gen()}); + auto donorChunk = makeAndInsertChunksForDonorShard(_originalUUID, + _originalEpoch, + _originalTimestamp, + _oldShardKey, + std::vector{OID::gen(), OID::gen()}); + + auto initialChunks = makeChunks(_reshardingUUID, + _tempEpoch, + _tempTimestamp, + _newShardKey, + std::vector{OID::gen(), OID::gen()}); std::vector presetReshardedChunks; for (const auto& chunk : initialChunks) { diff --git a/src/mongo/db/s/resharding/resharding_data_replication_test.cpp b/src/mongo/db/s/resharding/resharding_data_replication_test.cpp index 6cf1d5005d0..b33c3487be9 100644 --- a/src/mongo/db/s/resharding/resharding_data_replication_test.cpp +++ b/src/mongo/db/s/resharding/resharding_data_replication_test.cpp @@ -76,7 +76,7 @@ public: std::vector chunks = {ChunkType{ _sourceUUID, ChunkRange{BSON(_currentShardKey << MINKEY), BSON(_currentShardKey << MAXKEY)}, - ChunkVersion(100, 0, epoch, boost::none /* timestamp */), + ChunkVersion(100, 0, epoch, Timestamp()), _myDonorId}}; auto rt = RoutingTableHistory::makeNew(_sourceNss, @@ -85,7 +85,7 @@ public: std::move(defaultCollator), false /* unique */, std::move(epoch), - boost::none /* timestamp */, + Timestamp(), boost::none /* timeseriesFields */, boost::none /* reshardingFields */, boost::none /* chunkSizeBytes */, diff --git a/src/mongo/db/s/resharding/resharding_donor_recipient_common_test.h b/src/mongo/db/s/resharding/resharding_donor_recipient_common_test.h index c6202d8a8d7..8ceec20e5bd 100644 --- a/src/mongo/db/s/resharding/resharding_donor_recipient_common_test.h +++ b/src/mongo/db/s/resharding/resharding_donor_recipient_common_test.h @@ -109,7 +109,7 @@ protected: const ShardId& shardThatChunkExistsOn) { auto range = ChunkRange(BSON(shardKey << MINKEY), BSON(shardKey << MAXKEY)); auto chunk = ChunkType( - uuid, std::move(range), ChunkVersion(1, 0, epoch, boost::none), shardThatChunkExistsOn); + uuid, std::move(range), ChunkVersion(1, 0, epoch, timestamp), shardThatChunkExistsOn); ChunkManager cm(kThisShard.getShardId(), DatabaseVersion(uuid, timestamp), makeStandaloneRoutingTableHistory( @@ -119,7 +119,7 @@ protected: nullptr, false, epoch, - boost::none, + timestamp, boost::none /* timeseriesFields */, boost::none, boost::none /* chunkSizeBytes */, diff --git a/src/mongo/db/s/resharding/resharding_oplog_applier_test.cpp b/src/mongo/db/s/resharding/resharding_oplog_applier_test.cpp index 036d84d5698..3c88a00b273 100644 --- a/src/mongo/db/s/resharding/resharding_oplog_applier_test.cpp +++ b/src/mongo/db/s/resharding/resharding_oplog_applier_test.cpp @@ -170,17 +170,17 @@ public: kCrudUUID, ChunkRange{BSON(kOriginalShardKey << MINKEY), BSON(kOriginalShardKey << -std::numeric_limits::infinity())}, - ChunkVersion(1, 0, epoch, boost::none /* timestamp */), + ChunkVersion(1, 0, epoch, Timestamp()), _sourceId.getShardId()}, ChunkType{ kCrudUUID, ChunkRange{BSON(kOriginalShardKey << -std::numeric_limits::infinity()), BSON(kOriginalShardKey << 0)}, - ChunkVersion(1, 0, epoch, boost::none /* timestamp */), + ChunkVersion(1, 0, epoch, Timestamp()), kOtherShardId}, ChunkType{kCrudUUID, ChunkRange{BSON(kOriginalShardKey << 0), BSON(kOriginalShardKey << MAXKEY)}, - ChunkVersion(1, 0, epoch, boost::none /* timestamp */), + ChunkVersion(1, 0, epoch, Timestamp()), _sourceId.getShardId()}}; auto rt = RoutingTableHistory::makeNew(kCrudNs, @@ -189,7 +189,7 @@ public: nullptr, false, epoch, - boost::none /* timestamp */, + Timestamp(), boost::none /* timeseriesFields */, boost::none, boost::none /* chunkSizeBytes */, diff --git a/src/mongo/db/s/resharding/resharding_oplog_batch_applier_test.cpp b/src/mongo/db/s/resharding/resharding_oplog_batch_applier_test.cpp index f89da1f5b2d..c437eb3ec05 100644 --- a/src/mongo/db/s/resharding/resharding_oplog_batch_applier_test.cpp +++ b/src/mongo/db/s/resharding/resharding_oplog_batch_applier_test.cpp @@ -295,7 +295,7 @@ private: std::vector chunks = {ChunkType{ _sourceUUID, ChunkRange{BSON(_currentShardKey << MINKEY), BSON(_currentShardKey << MAXKEY)}, - ChunkVersion(100, 0, epoch, boost::none /* timestamp */), + ChunkVersion(100, 0, epoch, Timestamp()), _myDonorId}}; auto rt = RoutingTableHistory::makeNew(_sourceNss, @@ -304,7 +304,7 @@ private: nullptr /* defaultCollator */, false /* unique */, std::move(epoch), - boost::none /* timestamp */, + Timestamp(), boost::none /* timeseriesFields */, boost::none /* reshardingFields */, boost::none /* chunkSizeBytes */, diff --git a/src/mongo/db/s/resharding/resharding_oplog_crud_application_test.cpp b/src/mongo/db/s/resharding/resharding_oplog_crud_application_test.cpp index 2e81a7be471..3a96cd1672e 100644 --- a/src/mongo/db/s/resharding/resharding_oplog_crud_application_test.cpp +++ b/src/mongo/db/s/resharding/resharding_oplog_crud_application_test.cpp @@ -241,16 +241,16 @@ private: _sourceUUID, ChunkRange{BSON(_currentShardKey << MINKEY), BSON(_currentShardKey << -std::numeric_limits::infinity())}, - ChunkVersion(100, 0, epoch, boost::none /* timestamp */), + ChunkVersion(100, 0, epoch, Timestamp()), _myDonorId}, ChunkType{_sourceUUID, ChunkRange{BSON(_currentShardKey << -std::numeric_limits::infinity()), BSON(_currentShardKey << 0)}, - ChunkVersion(100, 1, epoch, boost::none /* timestamp */), + ChunkVersion(100, 1, epoch, Timestamp()), _otherDonorId}, ChunkType{_sourceUUID, ChunkRange{BSON(_currentShardKey << 0), BSON(_currentShardKey << MAXKEY)}, - ChunkVersion(100, 2, epoch, boost::none /* timestamp */), + ChunkVersion(100, 2, epoch, Timestamp()), _myDonorId}}; auto rt = RoutingTableHistory::makeNew(_sourceNss, @@ -259,7 +259,7 @@ private: nullptr /* defaultCollator */, false /* unique */, std::move(epoch), - boost::none /* timestamp */, + Timestamp(), boost::none /* timeseriesFields */, boost::none /* reshardingFields */, boost::none /* chunkSizeBytes */, diff --git a/src/mongo/db/s/resharding/resharding_recipient_service_external_state_test.cpp b/src/mongo/db/s/resharding/resharding_recipient_service_external_state_test.cpp index ad04e9623dd..4d36d53415e 100644 --- a/src/mongo/db/s/resharding/resharding_recipient_service_external_state_test.cpp +++ b/src/mongo/db/s/resharding/resharding_recipient_service_external_state_test.cpp @@ -58,10 +58,12 @@ public: const ShardKeyPattern kShardKey = ShardKeyPattern(BSON("oldKey" << 1)); const OID kOrigEpoch = OID::gen(); const UUID kOrigUUID = UUID::gen(); + const Timestamp kOrigTimestamp = Timestamp(1); const NamespaceString kOrigNss = NamespaceString("db.foo"); const ShardKeyPattern kReshardingKey = ShardKeyPattern(BSON("newKey" << 1)); const OID kReshardingEpoch = OID::gen(); const UUID kReshardingUUID = UUID::gen(); + const Timestamp kReshardingTimestamp = Timestamp(2); const NamespaceString kReshardingNss = NamespaceString( str::stream() << "db." << NamespaceString::kTemporaryReshardingCollectionPrefix << kOrigUUID); @@ -139,11 +141,12 @@ public: const ShardKeyPattern& skey, UUID uuid, OID epoch, + Timestamp timestamp, const BSONObj& collation = {}) { auto future = scheduleRoutingInfoForcedRefresh(tempNss); expectFindSendBSONObjVector(kConfigHostAndPort, [&]() { - CollectionType coll(tempNss, epoch, Date_t::now(), uuid); + CollectionType coll(tempNss, epoch, timestamp, Date_t::now(), uuid); coll.setKeyPattern(skey.getKeyPattern()); coll.setUnique(false); coll.setDefaultCollation(collation); @@ -161,7 +164,7 @@ public: reshardingFields.setRecipientFields(recipientFields); coll.setReshardingFields(reshardingFields); - ChunkVersion version(1, 0, epoch, boost::none /* timestamp */); + ChunkVersion version(1, 0, epoch, timestamp); ChunkType chunk(uuid, {skey.getKeyPattern().globalMin(), skey.getKeyPattern().globalMax()}, @@ -180,13 +183,14 @@ public: void expectRefreshReturnForOriginalColl(const NamespaceString& origNss, const ShardKeyPattern& skey, UUID uuid, - OID epoch) { + OID epoch, + Timestamp timestamp) { expectFindSendBSONObjVector(kConfigHostAndPort, [&]() { - CollectionType coll(origNss, epoch, Date_t::now(), uuid); + CollectionType coll(origNss, epoch, timestamp, Date_t::now(), uuid); coll.setKeyPattern(skey.getKeyPattern()); coll.setUnique(false); - ChunkVersion version(1, 0, epoch, boost::none /* timestamp */); + ChunkVersion version(1, 0, epoch, timestamp); ChunkType chunk(uuid, {skey.getKeyPattern().globalMin(), skey.getKeyPattern().globalMax()}, @@ -280,8 +284,12 @@ TEST_F(RecipientServiceExternalStateTest, CreateLocalReshardingCollectionBasic) } // Simulate a refresh for the temporary resharding collection. - loadOneChunkMetadataForTemporaryReshardingColl( - kReshardingNss, kOrigNss, kReshardingKey, kReshardingUUID, kReshardingEpoch); + loadOneChunkMetadataForTemporaryReshardingColl(kReshardingNss, + kOrigNss, + kReshardingKey, + kReshardingUUID, + kReshardingEpoch, + kReshardingTimestamp); const std::vector indexes = {BSON("v" << 2 << "key" << BSON("_id" << 1) << "name" << "_id_"), @@ -291,7 +299,8 @@ TEST_F(RecipientServiceExternalStateTest, CreateLocalReshardingCollectionBasic) << "name" << "indexOne")}; auto future = launchAsync([&] { - expectRefreshReturnForOriginalColl(kOrigNss, kShardKey, kOrigUUID, kOrigEpoch); + expectRefreshReturnForOriginalColl( + kOrigNss, kShardKey, kOrigUUID, kOrigEpoch, kOrigTimestamp); expectListCollections( kOrigNss, kOrigUUID, @@ -328,8 +337,12 @@ TEST_F(RecipientServiceExternalStateTest, } // Simulate a refresh for the temporary resharding collection. - loadOneChunkMetadataForTemporaryReshardingColl( - kReshardingNss, kOrigNss, kReshardingKey, kReshardingUUID, kReshardingEpoch); + loadOneChunkMetadataForTemporaryReshardingColl(kReshardingNss, + kOrigNss, + kReshardingKey, + kReshardingUUID, + kReshardingEpoch, + kReshardingTimestamp); const std::vector indexes = {BSON("v" << 2 << "key" << BSON("_id" << 1) << "name" << "_id_"), @@ -339,7 +352,8 @@ TEST_F(RecipientServiceExternalStateTest, << "name" << "indexOne")}; auto future = launchAsync([&] { - expectRefreshReturnForOriginalColl(kOrigNss, kShardKey, kOrigUUID, kOrigEpoch); + expectRefreshReturnForOriginalColl( + kOrigNss, kShardKey, kOrigUUID, kOrigEpoch, kOrigTimestamp); expectStaleDbVersionError(kOrigNss, "listCollections"); expectGetDatabase(kOrigNss, shards[1].getHost()); expectListCollections( @@ -352,7 +366,8 @@ TEST_F(RecipientServiceExternalStateTest, HostAndPort(shards[1].getHost())); expectStaleEpochError(kOrigNss, "listIndexes"); - expectRefreshReturnForOriginalColl(kOrigNss, kShardKey, kOrigUUID, kOrigEpoch); + expectRefreshReturnForOriginalColl( + kOrigNss, kShardKey, kOrigUUID, kOrigEpoch, kOrigTimestamp); expectListIndexes(kOrigNss, kOrigUUID, indexes, HostAndPort(shards[0].getHost())); }); @@ -381,8 +396,12 @@ TEST_F(RecipientServiceExternalStateTest, } // Simulate a refresh for the temporary resharding collection. - loadOneChunkMetadataForTemporaryReshardingColl( - kReshardingNss, kOrigNss, kReshardingKey, kReshardingUUID, kReshardingEpoch); + loadOneChunkMetadataForTemporaryReshardingColl(kReshardingNss, + kOrigNss, + kReshardingKey, + kReshardingUUID, + kReshardingEpoch, + kReshardingTimestamp); const std::vector indexes = {BSON("v" << 2 << "key" << BSON("_id" << 1) << "name" << "_id_"), @@ -407,7 +426,8 @@ TEST_F(RecipientServiceExternalStateTest, } auto future = launchAsync([&] { - expectRefreshReturnForOriginalColl(kOrigNss, kShardKey, kOrigUUID, kOrigEpoch); + expectRefreshReturnForOriginalColl( + kOrigNss, kShardKey, kOrigUUID, kOrigEpoch, kOrigTimestamp); expectListCollections( kOrigNss, kOrigUUID, @@ -444,8 +464,12 @@ TEST_F(RecipientServiceExternalStateTest, } // Simulate a refresh for the temporary resharding collection. - loadOneChunkMetadataForTemporaryReshardingColl( - kReshardingNss, kOrigNss, kReshardingKey, kReshardingUUID, kReshardingEpoch); + loadOneChunkMetadataForTemporaryReshardingColl(kReshardingNss, + kOrigNss, + kReshardingKey, + kReshardingUUID, + kReshardingEpoch, + kReshardingTimestamp); const std::vector indexes = {BSON("v" << 2 << "key" << BSON("_id" << 1) << "name" << "_id_"), @@ -472,7 +496,8 @@ TEST_F(RecipientServiceExternalStateTest, } auto future = launchAsync([&] { - expectRefreshReturnForOriginalColl(kOrigNss, kShardKey, kOrigUUID, kOrigEpoch); + expectRefreshReturnForOriginalColl( + kOrigNss, kShardKey, kOrigUUID, kOrigEpoch, kOrigTimestamp); expectListCollections( kOrigNss, kOrigUUID, @@ -509,8 +534,12 @@ TEST_F(RecipientServiceExternalStateTest, } // Simulate a refresh for the temporary resharding collection. - loadOneChunkMetadataForTemporaryReshardingColl( - kReshardingNss, kOrigNss, kReshardingKey, kReshardingUUID, kReshardingEpoch); + loadOneChunkMetadataForTemporaryReshardingColl(kReshardingNss, + kOrigNss, + kReshardingKey, + kReshardingUUID, + kReshardingEpoch, + kReshardingTimestamp); const std::vector indexes = {BSON("v" << 2 << "key" << BSON("_id" << 1) << "name" << "_id_"), @@ -527,7 +556,8 @@ TEST_F(RecipientServiceExternalStateTest, operationContext(), kReshardingNss, optionsAndIndexes); auto future = launchAsync([&] { - expectRefreshReturnForOriginalColl(kOrigNss, kShardKey, kOrigUUID, kOrigEpoch); + expectRefreshReturnForOriginalColl( + kOrigNss, kShardKey, kOrigUUID, kOrigEpoch, kOrigTimestamp); expectListCollections( kOrigNss, kOrigUUID, diff --git a/src/mongo/db/s/resharding/resharding_recipient_service_test.cpp b/src/mongo/db/s/resharding/resharding_recipient_service_test.cpp index a2ee7a5521b..5ec1423036c 100644 --- a/src/mongo/db/s/resharding/resharding_recipient_service_test.cpp +++ b/src/mongo/db/s/resharding/resharding_recipient_service_test.cpp @@ -80,7 +80,7 @@ public: std::vector chunks = {ChunkType{ _sourceUUID, ChunkRange{BSON(_currentShardKey << MINKEY), BSON(_currentShardKey << MAXKEY)}, - ChunkVersion(100, 0, epoch, boost::none /* timestamp */), + ChunkVersion(100, 0, epoch, Timestamp()), _someDonorId}}; auto rt = RoutingTableHistory::makeNew(_sourceNss, @@ -89,7 +89,7 @@ public: nullptr /* defaultCollator */, false /* unique */, std::move(epoch), - boost::none /* timestamp */, + Timestamp(), boost::none /* timeseriesFields */, boost::none /* reshardingFields */, boost::none /* chunkSizeBytes */, diff --git a/src/mongo/db/s/resharding_destined_recipient_test.cpp b/src/mongo/db/s/resharding_destined_recipient_test.cpp index 6592bb4f35d..676154a6a44 100644 --- a/src/mongo/db/s/resharding_destined_recipient_test.cpp +++ b/src/mongo/db/s/resharding_destined_recipient_test.cpp @@ -161,18 +161,15 @@ public: protected: std::vector createChunks(const OID& epoch, const UUID& uuid, + const Timestamp& timestamp, const std::string& shardKey) { auto range1 = ChunkRange(BSON(shardKey << MINKEY), BSON(shardKey << 5)); - ChunkType chunk1(uuid, - range1, - ChunkVersion(1, 0, epoch, boost::none /* timestamp */), - kShardList[0].getName()); + ChunkType chunk1( + uuid, range1, ChunkVersion(1, 0, epoch, timestamp), kShardList[0].getName()); auto range2 = ChunkRange(BSON(shardKey << 5), BSON(shardKey << MAXKEY)); - ChunkType chunk2(uuid, - range2, - ChunkVersion(1, 0, epoch, boost::none /* timestamp */), - kShardList[1].getName()); + ChunkType chunk2( + uuid, range2, ChunkVersion(1, 0, epoch, timestamp), kShardList[1].getName()); return {chunk1, chunk2}; } @@ -201,7 +198,7 @@ protected: ReshardingEnv env(CollectionCatalog::get(opCtx)->lookupUUIDByNSS(opCtx, kNss).value()); env.destShard = kShardList[1].getName(); - env.version = ChunkVersion(1, 0, OID::gen(), boost::none /* timestamp */); + env.version = ChunkVersion(1, 0, OID::gen(), Timestamp()); env.tempNss = NamespaceString(kNss.db(), fmt::format("{}{}", @@ -219,7 +216,8 @@ protected: {ShardId{kShardList[0].getName()}, ShardId{kShardList[1].getName()}}}); reshardingFields.setState(CoordinatorStateEnum::kPreparingToDonate); - CollectionType coll(kNss, env.version.epoch(), Date_t::now(), UUID::gen()); + CollectionType coll( + kNss, env.version.epoch(), env.version.getTimestamp(), Date_t::now(), UUID::gen()); coll.setKeyPattern(BSON(kShardKey << 1)); coll.setUnique(false); coll.setAllowMigrations(false); @@ -229,10 +227,14 @@ protected: _mockCatalogCacheLoader->setCollectionRefreshValues( kNss, coll, - createChunks(env.version.epoch(), env.sourceUuid, kShardKey), + createChunks( + env.version.epoch(), env.sourceUuid, env.version.getTimestamp(), kShardKey), reshardingFields); _mockCatalogCacheLoader->setCollectionRefreshValues( - env.tempNss, coll, createChunks(env.version.epoch(), env.sourceUuid, "y"), boost::none); + env.tempNss, + coll, + createChunks(env.version.epoch(), env.sourceUuid, env.version.getTimestamp(), "y"), + boost::none); forceDatabaseRefresh(opCtx, kNss.db()); forceShardFilteringMetadataRefresh(opCtx, kNss); diff --git a/src/mongo/db/s/shard_metadata_util.cpp b/src/mongo/db/s/shard_metadata_util.cpp index 4a5adb921ef..6b134bbe591 100644 --- a/src/mongo/db/s/shard_metadata_util.cpp +++ b/src/mongo/db/s/shard_metadata_util.cpp @@ -299,7 +299,7 @@ StatusWith> readShardChunks(OperationContext* opCtx, const BSONObj& sort, boost::optional limit, const OID& epoch, - const boost::optional& timestamp) { + const Timestamp& timestamp) { const auto chunksNss = getShardChunksNss(nss, uuid, supportingLongName); try { diff --git a/src/mongo/db/s/shard_metadata_util.h b/src/mongo/db/s/shard_metadata_util.h index b2b25b4c8c0..ce4c0c0cd4d 100644 --- a/src/mongo/db/s/shard_metadata_util.h +++ b/src/mongo/db/s/shard_metadata_util.h @@ -175,7 +175,7 @@ StatusWith> readShardChunks(OperationContext* opCtx, const BSONObj& sort, boost::optional limit, const OID& epoch, - const boost::optional& timestamp); + const Timestamp& timestamp); /** * Takes a vector of 'chunks' and updates the shard's chunks collection for 'nss' or 'uuid'. Any diff --git a/src/mongo/db/s/shard_metadata_util_test.cpp b/src/mongo/db/s/shard_metadata_util_test.cpp index 814517f17d9..bfd1b812f27 100644 --- a/src/mongo/db/s/shard_metadata_util_test.cpp +++ b/src/mongo/db/s/shard_metadata_util_test.cpp @@ -65,6 +65,7 @@ struct ShardMetadataUtilTest : public ShardServerTestFixture { ShardCollectionType shardCollectionType( BSON(ShardCollectionType::kNssFieldName << kNss.ns() << ShardCollectionType::kEpochFieldName << maxCollVersion.epoch() + << ShardCollectionType::kTimestampFieldName << maxCollVersion.getTimestamp() << ShardCollectionType::kUuidFieldName << uuid << ShardCollectionType::kKeyPatternFieldName << keyPattern.toBSON() << ShardCollectionType::kDefaultCollationFieldName << defaultCollation @@ -162,7 +163,7 @@ struct ShardMetadataUtilTest : public ShardServerTestFixture { } } - ChunkVersion maxCollVersion{0, 0, OID::gen(), boost::none /* timestamp */}; + ChunkVersion maxCollVersion{0, 0, OID::gen(), Timestamp(1, 1)}; const KeyPattern keyPattern{BSON("a" << 1)}; const BSONObj defaultCollation{BSON("locale" << "fr_CA")}; @@ -177,6 +178,7 @@ TEST_F(ShardMetadataUtilTest, UpdateAndReadCollectionsEntry) { ASSERT_EQUALS(updateShardCollectionType.getUuid(), readShardCollectionType.getUuid()); ASSERT_EQUALS(updateShardCollectionType.getNss(), readShardCollectionType.getNss()); ASSERT_EQUALS(updateShardCollectionType.getEpoch(), readShardCollectionType.getEpoch()); + ASSERT_EQUALS(updateShardCollectionType.getTimestamp(), readShardCollectionType.getTimestamp()); ASSERT_BSONOBJ_EQ(updateShardCollectionType.getKeyPattern().toBSON(), readShardCollectionType.getKeyPattern().toBSON()); ASSERT_BSONOBJ_EQ(updateShardCollectionType.getDefaultCollation(), @@ -199,6 +201,7 @@ TEST_F(ShardMetadataUtilTest, PersistedRefreshSignalStartAndFinish) { ASSERT_EQUALS(shardCollectionsEntry.getUuid(), uuid); ASSERT_EQUALS(shardCollectionsEntry.getNss().ns(), kNss.ns()); ASSERT_EQUALS(shardCollectionsEntry.getEpoch(), maxCollVersion.epoch()); + ASSERT_EQUALS(shardCollectionsEntry.getTimestamp(), maxCollVersion.getTimestamp()); ASSERT_BSONOBJ_EQ(shardCollectionsEntry.getKeyPattern().toBSON(), keyPattern.toBSON()); ASSERT_BSONOBJ_EQ(shardCollectionsEntry.getDefaultCollation(), defaultCollation); ASSERT_EQUALS(shardCollectionsEntry.getUnique(), kUnique); @@ -237,7 +240,7 @@ TEST_F(ShardMetadataUtilTest, WriteAndReadChunks) { // read all the chunks QueryAndSort allChunkDiff = createShardChunkDiffQuery( - ChunkVersion(0, 0, maxCollVersion.epoch(), boost::none /* timestamp */)); + ChunkVersion(0, 0, maxCollVersion.epoch(), maxCollVersion.getTimestamp())); std::vector readChunks = assertGet(readShardChunks(operationContext(), kNss, uuid, diff --git a/src/mongo/db/s/shard_server_catalog_cache_loader.cpp b/src/mongo/db/s/shard_server_catalog_cache_loader.cpp index 780138ffc0c..677a502fef3 100644 --- a/src/mongo/db/s/shard_server_catalog_cache_loader.cpp +++ b/src/mongo/db/s/shard_server_catalog_cache_loader.cpp @@ -387,7 +387,7 @@ void forcePrimaryDatabaseRefreshAndWaitForReplication(OperationContext* opCtx, S * Does nothing otherwise. */ void patchUpChangedChunksIfNeeded(bool mustPatchUpMetadataResults, - const boost::optional& timestamp, + const Timestamp& timestamp, std::vector& changedChunks) { if (!mustPatchUpMetadataResults) diff --git a/src/mongo/db/s/shard_server_catalog_cache_loader_test.cpp b/src/mongo/db/s/shard_server_catalog_cache_loader_test.cpp index e9800a39810..80409adc861 100644 --- a/src/mongo/db/s/shard_server_catalog_cache_loader_test.cpp +++ b/src/mongo/db/s/shard_server_catalog_cache_loader_test.cpp @@ -203,7 +203,7 @@ CollectionType ShardServerCatalogCacheLoaderTest::makeCollectionType( std::pair> ShardServerCatalogCacheLoaderTest::setUpChunkLoaderWithFiveChunks() { - ChunkVersion collectionVersion(1, 0, OID::gen(), boost::none /* timestamp */); + ChunkVersion collectionVersion(1, 0, OID::gen(), Timestamp()); CollectionType collectionType = makeCollectionType(collectionVersion); vector chunks = makeFiveChunks(collectionVersion); @@ -371,7 +371,7 @@ TEST_F(ShardServerCatalogCacheLoaderTest, PrimaryLoadFromShardedAndFindNewEpoch) // Then refresh again and find that the collection has been dropped and recreated. - ChunkVersion collVersionWithNewEpoch(1, 0, OID::gen(), boost::none /* timestamp */); + ChunkVersion collVersionWithNewEpoch(1, 0, OID::gen(), Timestamp()); CollectionType collectionTypeWithNewEpoch = makeCollectionType(collVersionWithNewEpoch); vector chunksWithNewEpoch = makeFiveChunks(collVersionWithNewEpoch); _remoteLoaderMock->setCollectionRefreshReturnValue(collectionTypeWithNewEpoch); @@ -398,7 +398,7 @@ TEST_F(ShardServerCatalogCacheLoaderTest, PrimaryLoadFromShardedAndFindMixedChun // Then refresh again and retrieve chunks from the config server that have mixed epoches, like // as if the chunks read yielded around a drop and recreate of the collection. - ChunkVersion collVersionWithNewEpoch(1, 0, OID::gen(), boost::none /* timestamp */); + ChunkVersion collVersionWithNewEpoch(1, 0, OID::gen(), Timestamp()); CollectionType collectionTypeWithNewEpoch = makeCollectionType(collVersionWithNewEpoch); vector chunksWithNewEpoch = makeFiveChunks(collVersionWithNewEpoch); vector mixedChunks; @@ -458,7 +458,7 @@ TEST_F(ShardServerCatalogCacheLoaderTest, PrimaryLoadFromShardedAndFindDbMetadat } TEST_F(ShardServerCatalogCacheLoaderTest, TimeseriesFieldsAreProperlyPropagatedOnSSCCL) { - ChunkVersion collectionVersion(1, 0, OID::gen(), boost::none /* timestamp */); + ChunkVersion collectionVersion(1, 0, OID::gen(), Timestamp()); CollectionType collectionType = makeCollectionType(collectionVersion); vector chunks = makeFiveChunks(collectionVersion); @@ -500,7 +500,7 @@ TEST_F(ShardServerCatalogCacheLoaderTest, TimeseriesFieldsAreProperlyPropagatedO } void ShardServerCatalogCacheLoaderTest::refreshCollectionEpochOnRemoteLoader() { - ChunkVersion collectionVersion(1, 2, OID::gen(), boost::none); + ChunkVersion collectionVersion(1, 2, OID::gen(), Timestamp()); CollectionType collectionType = makeCollectionType(collectionVersion); vector chunks = makeFiveChunks(collectionVersion); _remoteLoaderMock->setCollectionRefreshReturnValue(collectionType); @@ -530,7 +530,7 @@ TEST_F(ShardServerCatalogCacheLoaderTest, CollAndChunkTasksConsistency) { } TEST_F(ShardServerCatalogCacheLoaderTest, SupportingLongNameFieldsAreProperlyPropagatedOnSSCCL) { - ChunkVersion collectionVersion(1, 0, OID::gen(), boost::none /* timestamp */); + ChunkVersion collectionVersion(1, 0, OID::gen(), Timestamp()); CollectionType collectionType = makeCollectionType(collectionVersion); collectionType.setSupportingLongName(SupportingLongNameStatusEnum::kExplicitlyEnabled); diff --git a/src/mongo/db/s/sharding_ddl_util_test.cpp b/src/mongo/db/s/sharding_ddl_util_test.cpp index 9421a389193..e182d0f4431 100644 --- a/src/mongo/db/s/sharding_ddl_util_test.cpp +++ b/src/mongo/db/s/sharding_ddl_util_test.cpp @@ -217,7 +217,7 @@ TEST_F(ShardingDDLUtilTest, ShardedRenamePreconditionsAreMet) { sharding_ddl_util::checkShardedRenamePreconditions(opCtx, kToNss, false /* dropTarget */); // Initialize a chunk - ChunkVersion chunkVersion(1, 1, OID::gen(), boost::none); + ChunkVersion chunkVersion(1, 1, OID::gen(), Timestamp(2, 1)); ChunkType chunk; chunk.setName(OID::gen()); chunk.setCollectionUUID(UUID::gen()); @@ -237,7 +237,7 @@ TEST_F(ShardingDDLUtilTest, ShardedRenamePreconditionsTargetCollectionExists) { auto opCtx = operationContext(); // Initialize a chunk - ChunkVersion chunkVersion(1, 1, OID::gen(), boost::none); + ChunkVersion chunkVersion(1, 1, OID::gen(), Timestamp(2, 1)); ChunkType chunk; chunk.setName(OID::gen()); chunk.setCollectionUUID(UUID::gen()); diff --git a/src/mongo/db/s/type_shard_collection.cpp b/src/mongo/db/s/type_shard_collection.cpp index 707032eaeaf..66ab2759fbb 100644 --- a/src/mongo/db/s/type_shard_collection.cpp +++ b/src/mongo/db/s/type_shard_collection.cpp @@ -33,21 +33,18 @@ namespace mongo { -ShardCollectionType::ShardCollectionType( - NamespaceString nss, OID epoch, UUID uuid, KeyPattern keyPattern, bool unique) - : ShardCollectionTypeBase( - std::move(nss), std::move(epoch), std::move(uuid), std::move(keyPattern), unique) {} - ShardCollectionType::ShardCollectionType(NamespaceString nss, OID epoch, - boost::optional creationTime, + Timestamp creationTime, UUID uuid, KeyPattern keyPattern, bool unique) - : ShardCollectionTypeBase( - std::move(nss), std::move(epoch), std::move(uuid), std::move(keyPattern), unique) { - setTimestamp(std::move(creationTime)); -} + : ShardCollectionTypeBase(std::move(nss), + std::move(epoch), + std::move(creationTime), + std::move(uuid), + std::move(keyPattern), + unique) {} ShardCollectionType::ShardCollectionType(const BSONObj& obj) { ShardCollectionTypeBase::parseProtected(IDLParserErrorContext("ShardCollectionType"), obj); diff --git a/src/mongo/db/s/type_shard_collection.h b/src/mongo/db/s/type_shard_collection.h index f2d8123655e..f6a86b0b388 100644 --- a/src/mongo/db/s/type_shard_collection.h +++ b/src/mongo/db/s/type_shard_collection.h @@ -48,6 +48,7 @@ public: using ShardCollectionTypeBase::kReshardingFieldsFieldName; using ShardCollectionTypeBase::kSupportingLongNameFieldName; using ShardCollectionTypeBase::kTimeseriesFieldsFieldName; + using ShardCollectionTypeBase::kTimestampFieldName; using ShardCollectionTypeBase::kUniqueFieldName; using ShardCollectionTypeBase::kUuidFieldName; @@ -80,12 +81,9 @@ public: using ShardCollectionTypeBase::setUnique; using ShardCollectionTypeBase::setUuid; - ShardCollectionType( - NamespaceString nss, OID epoch, UUID uuid, KeyPattern keyPattern, bool unique); - ShardCollectionType(NamespaceString nss, OID epoch, - boost::optional creationTime, + Timestamp creationTime, UUID uuid, KeyPattern keyPattern, bool unique); diff --git a/src/mongo/db/s/type_shard_collection.idl b/src/mongo/db/s/type_shard_collection.idl index 2bf73f19f7e..fbc5cd6ac8f 100644 --- a/src/mongo/db/s/type_shard_collection.idl +++ b/src/mongo/db/s/type_shard_collection.idl @@ -105,11 +105,7 @@ structs: collection was created or it's shard key last refined. Because timestamps are comparable, we are able to define a total order in time in the collection. This field will replace Epoch, which are not - comparable. - - It is optional for parsing purposes, because in versions of MongoDB - prior to 5.0, this value wasn't being written." - optional: true + comparable." uuid: type: uuid description: "The UUID that will be used to create the local collection on each of diff --git a/src/mongo/db/s/type_shard_collection_test.cpp b/src/mongo/db/s/type_shard_collection_test.cpp index 3a62e513af3..bd5ed844a75 100644 --- a/src/mongo/db/s/type_shard_collection_test.cpp +++ b/src/mongo/db/s/type_shard_collection_test.cpp @@ -48,6 +48,7 @@ TEST(ShardCollectionType, FromBSONEmptyShardKeyFails) { ASSERT_THROWS_CODE( ShardCollectionType(BSON(ShardCollectionType::kNssFieldName << kNss.ns() << ShardCollectionType::kEpochFieldName << OID::gen() + << ShardCollectionType::kTimestampFieldName << Timestamp() << ShardCollectionType::kUuidFieldName << UUID::gen() << ShardCollectionType::kKeyPatternFieldName << BSONObj() << ShardCollectionType::kUniqueFieldName << true)), @@ -57,32 +58,39 @@ TEST(ShardCollectionType, FromBSONEmptyShardKeyFails) { TEST(ShardCollectionType, FromBSONEpochMatchesLastRefreshedCollectionVersionWhenBSONTimestamp) { OID epoch = OID::gen(); + Timestamp timestamp(1, 1); ShardCollectionType shardCollType( BSON(ShardCollectionType::kNssFieldName << kNss.ns() << ShardCollectionType::kEpochFieldName << epoch + << ShardCollectionType::kTimestampFieldName << timestamp << ShardCollectionType::kUuidFieldName << UUID::gen() << ShardCollectionType::kKeyPatternFieldName << kKeyPattern << ShardCollectionType::kUniqueFieldName << true << ShardCollectionType::kLastRefreshedCollectionVersionFieldName << Timestamp())); ASSERT_EQ(epoch, shardCollType.getLastRefreshedCollectionVersion()->epoch()); + ASSERT_EQ(timestamp, shardCollType.getLastRefreshedCollectionVersion()->getTimestamp()); } TEST(ShardCollectionType, FromBSONEpochMatchesLastRefreshedCollectionVersionWhenDate) { OID epoch = OID::gen(); + Timestamp timestamp(1, 1); ShardCollectionType shardCollType( BSON(ShardCollectionType::kNssFieldName << kNss.ns() << ShardCollectionType::kEpochFieldName << epoch << ShardCollectionType::kUuidFieldName << UUID::gen() + << ShardCollectionType::kTimestampFieldName << timestamp << ShardCollectionType::kKeyPatternFieldName << kKeyPattern << ShardCollectionType::kUniqueFieldName << true << ShardCollectionType::kLastRefreshedCollectionVersionFieldName << Date_t())); ASSERT_EQ(epoch, shardCollType.getLastRefreshedCollectionVersion()->epoch()); + ASSERT_EQ(timestamp, shardCollType.getLastRefreshedCollectionVersion()->getTimestamp()); } TEST(ShardCollectionType, ToBSONEmptyDefaultCollationNotIncluded) { - ShardCollectionType shardCollType(kNss, OID::gen(), UUID::gen(), kKeyPattern, true); + ShardCollectionType shardCollType( + kNss, OID::gen(), Timestamp(), UUID::gen(), kKeyPattern, true); BSONObj obj = shardCollType.toBSON(); ASSERT_FALSE(obj.hasField(ShardCollectionType::kDefaultCollationFieldName)); @@ -94,7 +102,8 @@ TEST(ShardCollectionType, ToBSONEmptyDefaultCollationNotIncluded) { } TEST(ShardCollectionType, ReshardingFieldsIncluded) { - ShardCollectionType shardCollType(kNss, OID::gen(), UUID::gen(), kKeyPattern, true); + ShardCollectionType shardCollType( + kNss, OID::gen(), Timestamp(), UUID::gen(), kKeyPattern, true); TypeCollectionReshardingFields reshardingFields; const auto reshardingUUID = UUID::gen(); @@ -110,7 +119,8 @@ TEST(ShardCollectionType, ReshardingFieldsIncluded) { } TEST(ShardCollectionType, AllowMigrationsFieldBackwardsCompatibility) { - ShardCollectionType shardCollType(kNss, OID::gen(), UUID::gen(), kKeyPattern, true); + ShardCollectionType shardCollType( + kNss, OID::gen(), Timestamp(), UUID::gen(), kKeyPattern, true); shardCollType.setAllowMigrations(false); ASSERT_EQ(false, shardCollType.toBSON()[ShardCollectionType::kAllowMigrationsFieldName].Bool()); diff --git a/src/mongo/s/catalog/sharding_catalog_client.h b/src/mongo/s/catalog/sharding_catalog_client.h index 0eb7027eb78..60df2570447 100644 --- a/src/mongo/s/catalog/sharding_catalog_client.h +++ b/src/mongo/s/catalog/sharding_catalog_client.h @@ -184,7 +184,7 @@ public: boost::optional limit, repl::OpTime* opTime, const OID& epoch, - const boost::optional& timestamp, + const Timestamp& timestamp, repl::ReadConcernLevel readConcern, const boost::optional& hint = boost::none) = 0; diff --git a/src/mongo/s/catalog/sharding_catalog_client_impl.cpp b/src/mongo/s/catalog/sharding_catalog_client_impl.cpp index fabe24b7928..80a826bfbc9 100644 --- a/src/mongo/s/catalog/sharding_catalog_client_impl.cpp +++ b/src/mongo/s/catalog/sharding_catalog_client_impl.cpp @@ -593,7 +593,7 @@ StatusWith> ShardingCatalogClientImpl::getChunks( boost::optional limit, OpTime* opTime, const OID& epoch, - const boost::optional& timestamp, + const Timestamp& timestamp, repl::ReadConcernLevel readConcern, const boost::optional& hint) { invariant(serverGlobalParams.clusterRole == ClusterRole::ConfigServer || diff --git a/src/mongo/s/catalog/sharding_catalog_client_impl.h b/src/mongo/s/catalog/sharding_catalog_client_impl.h index b614d3a3839..99c86ef03a1 100644 --- a/src/mongo/s/catalog/sharding_catalog_client_impl.h +++ b/src/mongo/s/catalog/sharding_catalog_client_impl.h @@ -97,7 +97,7 @@ public: boost::optional limit, repl::OpTime* opTime, const OID& epoch, - const boost::optional& timestamp, + const Timestamp& timestamp, repl::ReadConcernLevel readConcern, const boost::optional& hint = boost::none) override; diff --git a/src/mongo/s/catalog/sharding_catalog_client_mock.cpp b/src/mongo/s/catalog/sharding_catalog_client_mock.cpp index e5c24f4bfb6..a4153e80d50 100644 --- a/src/mongo/s/catalog/sharding_catalog_client_mock.cpp +++ b/src/mongo/s/catalog/sharding_catalog_client_mock.cpp @@ -90,7 +90,7 @@ StatusWith> ShardingCatalogClientMock::getChunks( boost::optional limit, repl::OpTime* opTime, const OID& epoch, - const boost::optional& timestamp, + const Timestamp& timestamp, repl::ReadConcernLevel readConcern, const boost::optional& hint) { return {ErrorCodes::InternalError, "Method not implemented"}; diff --git a/src/mongo/s/catalog/sharding_catalog_client_mock.h b/src/mongo/s/catalog/sharding_catalog_client_mock.h index 794744e30ac..fdab949c024 100644 --- a/src/mongo/s/catalog/sharding_catalog_client_mock.h +++ b/src/mongo/s/catalog/sharding_catalog_client_mock.h @@ -72,7 +72,7 @@ public: boost::optional limit, repl::OpTime* opTime, const OID& epoch, - const boost::optional& timestamp, + const Timestamp& timestamp, repl::ReadConcernLevel readConcern, const boost::optional& hint) override; diff --git a/src/mongo/s/catalog/sharding_catalog_client_test.cpp b/src/mongo/s/catalog/sharding_catalog_client_test.cpp index 35a5371bb89..49ec74a361a 100644 --- a/src/mongo/s/catalog/sharding_catalog_client_test.cpp +++ b/src/mongo/s/catalog/sharding_catalog_client_test.cpp @@ -86,7 +86,7 @@ TEST_F(ShardingCatalogClientTest, GetCollectionExisting) { configTargeter()->setFindHostReturnValue(HostAndPort("TestHost1")); CollectionType expectedColl( - NamespaceString("TestDB.TestNS"), OID::gen(), Date_t::now(), UUID::gen()); + NamespaceString("TestDB.TestNS"), OID::gen(), Timestamp(), Date_t::now(), UUID::gen()); expectedColl.setKeyPattern(BSON("KeyName" << 1)); const OpTime newOpTime(Timestamp(7, 6), 5); @@ -355,7 +355,7 @@ TEST_F(ShardingCatalogClientTest, GetChunksForNSWithSortAndLimit) { const auto collUuid = UUID::gen(); const auto collEpoch = OID::gen(); - const auto collTimestamp = boost::none; + const auto collTimestamp = Timestamp(1, 1); ChunkType chunkA; chunkA.setName(OID::gen()); @@ -441,7 +441,7 @@ TEST_F(ShardingCatalogClientTest, GetChunksForUUIDNoSortNoLimit) { const auto collUuid = UUID::gen(); const auto collEpoch = OID::gen(); - const auto collTimestamp = boost::none; + const auto collTimestamp = Timestamp(); ChunkVersion queryChunkVersion({1, 2, collEpoch, collTimestamp}); @@ -490,7 +490,7 @@ TEST_F(ShardingCatalogClientTest, GetChunksForNSInvalidChunk) { configTargeter()->setFindHostReturnValue(HostAndPort("TestHost1")); const auto collUuid = UUID::gen(); - ChunkVersion queryChunkVersion({1, 2, OID::gen(), boost::none /* timestamp */}); + ChunkVersion queryChunkVersion({1, 2, OID::gen(), Timestamp()}); const BSONObj chunksQuery( BSON(ChunkType::collectionUUID() @@ -516,14 +516,14 @@ TEST_F(ShardingCatalogClientTest, GetChunksForNSInvalidChunk) { chunkA.setCollectionUUID(collUuid); chunkA.setMin(BSON("a" << 1)); chunkA.setMax(BSON("a" << 100)); - chunkA.setVersion({1, 2, OID::gen(), boost::none /* timestamp */}); + chunkA.setVersion({1, 2, OID::gen(), Timestamp()}); chunkA.setShard(ShardId("shard0000")); ChunkType chunkB; chunkB.setCollectionUUID(collUuid); chunkB.setMin(BSON("a" << 100)); chunkB.setMax(BSON("a" << 200)); - chunkB.setVersion({3, 4, OID::gen(), boost::none /* timestamp */}); + chunkB.setVersion({3, 4, OID::gen(), Timestamp()}); // Missing shard id return vector{chunkA.toConfigBSON(), chunkB.toConfigBSON()}; @@ -765,13 +765,14 @@ TEST_F(ShardingCatalogClientTest, RunUserManagementWriteCommandNotWritablePrimar TEST_F(ShardingCatalogClientTest, GetCollectionsValidResultsNoDb) { configTargeter()->setFindHostReturnValue(HostAndPort("TestHost1")); - CollectionType coll1(NamespaceString{"test.coll1"}, OID::gen(), network()->now(), UUID::gen()); + CollectionType coll1( + NamespaceString{"test.coll1"}, OID::gen(), Timestamp(), network()->now(), UUID::gen()); coll1.setKeyPattern(KeyPattern{BSON("_id" << 1)}); coll1.setUnique(false); CollectionType coll2( - NamespaceString{"anotherdb.coll1"}, OID::gen(), network()->now(), UUID::gen()); + NamespaceString{"anotherdb.coll1"}, OID::gen(), Timestamp(), network()->now(), UUID::gen()); coll2.setKeyPattern(KeyPattern{BSON("_id" << 1)}); coll2.setUnique(false); @@ -818,11 +819,13 @@ TEST_F(ShardingCatalogClientTest, GetCollectionsValidResultsNoDb) { TEST_F(ShardingCatalogClientTest, GetCollectionsValidResultsWithDb) { configTargeter()->setFindHostReturnValue(HostAndPort("TestHost1")); - CollectionType coll1(NamespaceString{"test.coll1"}, OID::gen(), network()->now(), UUID::gen()); + CollectionType coll1( + NamespaceString{"test.coll1"}, OID::gen(), Timestamp(), network()->now(), UUID::gen()); coll1.setKeyPattern(KeyPattern{BSON("_id" << 1)}); coll1.setUnique(true); - CollectionType coll2(NamespaceString{"test.coll2"}, OID::gen(), network()->now(), UUID::gen()); + CollectionType coll2( + NamespaceString{"test.coll2"}, OID::gen(), Timestamp(), network()->now(), UUID::gen()); coll2.setKeyPattern(KeyPattern{BSON("_id" << 1)}); coll2.setUnique(false); @@ -863,7 +866,7 @@ TEST_F(ShardingCatalogClientTest, GetCollectionsInvalidCollectionType) { }); CollectionType validColl( - NamespaceString{"test.coll1"}, OID::gen(), network()->now(), UUID::gen()); + NamespaceString{"test.coll1"}, OID::gen(), Timestamp(), network()->now(), UUID::gen()); validColl.setKeyPattern(KeyPattern{BSON("_id" << 1)}); validColl.setUnique(true); @@ -1185,7 +1188,7 @@ TEST_F(ShardingCatalogClientTest, ApplyChunkOpsDeprecatedSuccessfulWithCheck) { << "second precondition")); const NamespaceString nss("config.chunks"); const UUID uuid = UUID::gen(); - ChunkVersion lastChunkVersion(0, 0, OID(), boost::none /* timestamp */); + ChunkVersion lastChunkVersion(0, 0, OID(), Timestamp()); auto future = launchAsync([this, updateOps, preCondition, uuid, nss, lastChunkVersion] { auto status = diff --git a/src/mongo/s/catalog/sharding_catalog_write_retry_test.cpp b/src/mongo/s/catalog/sharding_catalog_write_retry_test.cpp index 7ad45617c41..149d7950fb3 100644 --- a/src/mongo/s/catalog/sharding_catalog_write_retry_test.cpp +++ b/src/mongo/s/catalog/sharding_catalog_write_retry_test.cpp @@ -408,7 +408,7 @@ TEST_F(UpdateRetryTest, NotWritablePrimaryOnceSuccessAfterRetry) { configTargeter()->setFindHostReturnValue(host1); CollectionType collection( - NamespaceString("db.coll"), OID::gen(), network()->now(), UUID::gen()); + NamespaceString("db.coll"), OID::gen(), Timestamp(), network()->now(), UUID::gen()); collection.setKeyPattern(KeyPattern(BSON("_id" << 1))); BSONObj objToUpdate = BSON("_id" << 1 << "Value" diff --git a/src/mongo/s/catalog/type_chunk.cpp b/src/mongo/s/catalog/type_chunk.cpp index 1235bfb75f0..85615320d18 100644 --- a/src/mongo/s/catalog/type_chunk.cpp +++ b/src/mongo/s/catalog/type_chunk.cpp @@ -312,7 +312,7 @@ StatusWith ChunkType::parseFromConfigBSONCommand(const BSONObj& sourc StatusWith ChunkType::fromConfigBSON(const BSONObj& source, const OID& epoch, - const boost::optional& timestamp) { + const Timestamp& timestamp) { StatusWith chunkStatus = parseFromConfigBSONCommand(source); if (!chunkStatus.isOK()) { return chunkStatus.getStatus(); @@ -331,10 +331,8 @@ StatusWith ChunkType::fromConfigBSON(const BSONObj& source, } const ChunkVersion& version = chunk.getVersion(); - if (version.epoch() == OID()) { - chunk.setVersion( - ChunkVersion(version.majorVersion(), version.minorVersion(), epoch, timestamp)); - } + chunk.setVersion( + ChunkVersion(version.majorVersion(), version.minorVersion(), epoch, timestamp)); return chunk; } @@ -362,7 +360,7 @@ BSONObj ChunkType::toConfigBSON() const { StatusWith ChunkType::fromShardBSON(const BSONObj& source, const OID& epoch, - const boost::optional& timestamp) { + const Timestamp& timestamp) { ChunkType chunk; { diff --git a/src/mongo/s/catalog/type_chunk.h b/src/mongo/s/catalog/type_chunk.h index 1d678aecc67..925bb7075fe 100644 --- a/src/mongo/s/catalog/type_chunk.h +++ b/src/mongo/s/catalog/type_chunk.h @@ -227,7 +227,7 @@ public: static StatusWith parseFromConfigBSONCommand(const BSONObj& source); static StatusWith fromConfigBSON(const BSONObj& source, const OID& epoch, - const boost::optional& timestamp); + const Timestamp& timestamp); /** * Returns the BSON representation of the entry for the config server's config.chunks @@ -243,7 +243,7 @@ public: */ static StatusWith fromShardBSON(const BSONObj& source, const OID& epoch, - const boost::optional& timestamp); + const Timestamp& timestamp); /** * Returns the BSON representation of the entry for a shard server's config.chunks. diff --git a/src/mongo/s/catalog/type_chunk_test.cpp b/src/mongo/s/catalog/type_chunk_test.cpp index b09047edf92..385417734d3 100644 --- a/src/mongo/s/catalog/type_chunk_test.cpp +++ b/src/mongo/s/catalog/type_chunk_test.cpp @@ -49,7 +49,7 @@ const ShardId kShard("shard0000"); TEST(ChunkType, MissingConfigRequiredFields) { const auto collUuid = UUID::gen(); const auto collEpoch = OID::gen(); - const auto collTimestamp = boost::none; + const auto collTimestamp = Timestamp(); ChunkVersion chunkVersion(1, 2, collEpoch, collTimestamp); @@ -86,44 +86,44 @@ TEST(ChunkType, MissingConfigRequiredFields) { TEST(ChunkType, MissingShardRequiredFields) { const OID epoch = OID::gen(); - ChunkVersion chunkVersion(1, 2, epoch, boost::none /* timestamp */); + const Timestamp timestamp; + ChunkVersion chunkVersion(1, 2, epoch, timestamp); const auto lastmod = Timestamp(chunkVersion.toLong()); BSONObj objModMin = BSON(ChunkType::max(kMax) << ChunkType::shard(kShard.toString()) << "lastmod" << lastmod); - StatusWith chunkRes = - ChunkType::fromShardBSON(objModMin, epoch, boost::none /* timestamp */); + StatusWith chunkRes = ChunkType::fromShardBSON(objModMin, epoch, timestamp); ASSERT_EQUALS(chunkRes.getStatus(), ErrorCodes::NoSuchKey); ASSERT_STRING_CONTAINS(chunkRes.getStatus().reason(), ChunkType::minShardID.name()); BSONObj objModMax = BSON(ChunkType::minShardID(kMin) << ChunkType::shard(kShard.toString()) << "lastmod" << lastmod); - chunkRes = ChunkType::fromShardBSON(objModMax, epoch, boost::none /* timestamp */); + chunkRes = ChunkType::fromShardBSON(objModMax, epoch, timestamp); ASSERT_EQUALS(chunkRes.getStatus(), ErrorCodes::NoSuchKey); ASSERT_STRING_CONTAINS(chunkRes.getStatus().reason(), ChunkType::max.name()); BSONObj objModShard = BSON(ChunkType::minShardID(kMin) << ChunkType::max(kMax) << "lastmod" << lastmod); - chunkRes = ChunkType::fromShardBSON(objModShard, epoch, boost::none /* timestamp */); + chunkRes = ChunkType::fromShardBSON(objModShard, epoch, timestamp); ASSERT_EQUALS(chunkRes.getStatus(), ErrorCodes::NoSuchKey); ASSERT_STRING_CONTAINS(chunkRes.getStatus().reason(), ChunkType::shard.name()); BSONObj objModLastmod = BSON(ChunkType::minShardID(kMin) << ChunkType::max(kMax) << ChunkType::shard(kShard.toString())); - chunkRes = ChunkType::fromShardBSON(objModLastmod, epoch, boost::none /* timestamp */); + chunkRes = ChunkType::fromShardBSON(objModLastmod, epoch, timestamp); ASSERT_EQUALS(chunkRes.getStatus(), ErrorCodes::NoSuchKey); } TEST(ChunkType, ToFromShardBSON) { const OID epoch = OID::gen(); - ChunkVersion chunkVersion(1, 2, epoch, boost::none /* timestamp */); + const Timestamp timestamp; + ChunkVersion chunkVersion(1, 2, epoch, timestamp); auto lastmod = Timestamp(chunkVersion.toLong()); BSONObj obj = BSON(ChunkType::minShardID(kMin) << ChunkType::max(kMax) << ChunkType::shard(kShard.toString()) << "lastmod" << lastmod); - ChunkType shardChunk = - assertGet(ChunkType::fromShardBSON(obj, epoch, boost::none /* timestamp */)); + ChunkType shardChunk = assertGet(ChunkType::fromShardBSON(obj, epoch, timestamp)); ASSERT_BSONOBJ_EQ(obj, shardChunk.toShardBSON()); @@ -136,14 +136,15 @@ TEST(ChunkType, ToFromShardBSON) { TEST(ChunkType, MinAndMaxShardKeysDifferInNumberOfKeys) { const auto collUuid = UUID::gen(); const auto collEpoch = OID::gen(); - const auto collTimestamp = boost::none; + const auto collTimestamp = Timestamp(1); ChunkVersion chunkVersion(1, 2, collEpoch, collTimestamp); BSONObj obj = BSON( ChunkType::name(OID::gen()) << ChunkType::collectionUUID() << collUuid << ChunkType::min(BSON("a" << 10 << "b" << 10)) << ChunkType::max(BSON("a" << 20)) << "lastmod" << Timestamp(chunkVersion.toLong()) - << "lastmodEpoch" << chunkVersion.epoch() << ChunkType::shard("shard0001")); + << "lastmodEpoch" << chunkVersion.epoch() << "lastmodTimestamp" + << chunkVersion.getTimestamp() << ChunkType::shard("shard0001")); StatusWith chunkRes = ChunkType::fromConfigBSON(obj, collEpoch, collTimestamp); ASSERT_OK(chunkRes.getStatus()); ASSERT_FALSE(chunkRes.getValue().validate().isOK()); @@ -152,14 +153,15 @@ TEST(ChunkType, MinAndMaxShardKeysDifferInNumberOfKeys) { TEST(ChunkType, MinAndMaxShardKeysDifferInKeyNames) { const auto collUuid = UUID::gen(); const auto collEpoch = OID::gen(); - const auto collTimestamp = boost::none; + const auto collTimestamp = Timestamp(1); ChunkVersion chunkVersion(1, 2, collEpoch, collTimestamp); BSONObj obj = BSON(ChunkType::name(OID::gen()) << ChunkType::collectionUUID() << collUuid << ChunkType::min(BSON("a" << 10)) << ChunkType::max(BSON("b" << 20)) << "lastmod" << Timestamp(chunkVersion.toLong()) - << "lastmodEpoch" << chunkVersion.epoch() << ChunkType::shard("shard0001")); + << "lastmodEpoch" << chunkVersion.epoch() << "lastmodTimestamp" + << chunkVersion.getTimestamp() << ChunkType::shard("shard0001")); StatusWith chunkRes = ChunkType::fromConfigBSON(obj, collEpoch, collTimestamp); ASSERT_OK(chunkRes.getStatus()); ASSERT_FALSE(chunkRes.getValue().validate().isOK()); @@ -168,7 +170,7 @@ TEST(ChunkType, MinAndMaxShardKeysDifferInKeyNames) { TEST(ChunkType, MinToMaxNotAscending) { const auto collUuid = UUID::gen(); const auto collEpoch = OID::gen(); - const auto collTimestamp = boost::none; + const auto collTimestamp = Timestamp(1); ChunkVersion chunkVersion(1, 2, collEpoch, collTimestamp); BSONObj obj = @@ -183,7 +185,7 @@ TEST(ChunkType, MinToMaxNotAscending) { TEST(ChunkType, ToFromConfigBSON) { const auto collUuid = UUID::gen(); const auto collEpoch = OID::gen(); - const auto collTimestamp = boost::none; + const auto collTimestamp = Timestamp(1); const auto chunkID = OID::gen(); ChunkVersion chunkVersion(1, 2, collEpoch, collTimestamp); @@ -209,7 +211,7 @@ TEST(ChunkType, ToFromConfigBSON) { TEST(ChunkType, BadType) { const auto collEpoch = OID::gen(); - const auto collTimestamp = boost::none; + const auto collTimestamp = Timestamp(1); BSONObj obj = BSON(ChunkType::name() << 0); StatusWith chunkRes = ChunkType::fromConfigBSON(obj, collEpoch, collTimestamp); @@ -219,7 +221,7 @@ TEST(ChunkType, BadType) { TEST(ChunkType, BothNsAndUUID) { const auto collUuid = UUID::gen(); const auto collEpoch = OID::gen(); - const auto collTimestamp = boost::none; + const auto collTimestamp = Timestamp(1); ChunkVersion chunkVersion(1, 2, collEpoch, collTimestamp); @@ -228,23 +230,24 @@ TEST(ChunkType, BothNsAndUUID) { << ChunkType::collectionUUID() << collUuid << ChunkType::collectionUUID() << mongo::UUID::gen() << ChunkType::min(BSON("a" << 10 << "b" << 10)) << ChunkType::max(BSON("a" << 20)) << "lastmod" << Timestamp(chunkVersion.toLong()) - << "lastmodEpoch" << chunkVersion.epoch() << ChunkType::shard("shard0001")); + << "lastmodEpoch" << chunkVersion.epoch() << "lastmodTimestamp" + << chunkVersion.getTimestamp() << ChunkType::shard("shard0001")); StatusWith chunkRes = ChunkType::fromConfigBSON(objModNS, collEpoch, collTimestamp); ASSERT_TRUE(chunkRes.isOK()); } TEST(ChunkType, UUIDPresentAndNsMissing) { const auto collEpoch = OID::gen(); - const auto collTimestamp = boost::none; + const auto collTimestamp = Timestamp(1); ChunkVersion chunkVersion(1, 2, collEpoch, collTimestamp); - BSONObj objModNS = - BSON(ChunkType::name(OID::gen()) - << ChunkType::collectionUUID() << mongo::UUID::gen() - << ChunkType::min(BSON("a" << 10 << "b" << 10)) << ChunkType::max(BSON("a" << 20)) - << "lastmod" << Timestamp(chunkVersion.toLong()) << "lastmodEpoch" - << chunkVersion.epoch() << ChunkType::shard("shard0001")); + BSONObj objModNS = BSON( + ChunkType::name(OID::gen()) + << ChunkType::collectionUUID() << mongo::UUID::gen() + << ChunkType::min(BSON("a" << 10 << "b" << 10)) << ChunkType::max(BSON("a" << 20)) + << "lastmod" << Timestamp(chunkVersion.toLong()) << "lastmodEpoch" << chunkVersion.epoch() + << "lastmodTimestamp" << chunkVersion.getTimestamp() << ChunkType::shard("shard0001")); StatusWith chunkRes = ChunkType::fromConfigBSON(objModNS, collEpoch, collTimestamp); ASSERT_TRUE(chunkRes.isOK()); } diff --git a/src/mongo/s/catalog/type_collection.cpp b/src/mongo/s/catalog/type_collection.cpp index b6e549aa626..816787e1ce4 100644 --- a/src/mongo/s/catalog/type_collection.cpp +++ b/src/mongo/s/catalog/type_collection.cpp @@ -43,23 +43,13 @@ namespace mongo { const NamespaceString CollectionType::ConfigNS("config.collections"); -CollectionType::CollectionType(NamespaceString nss, OID epoch, Date_t updatedAt, UUID uuid) - : CollectionTypeBase(std::move(nss), std::move(updatedAt)) { +CollectionType::CollectionType( + NamespaceString nss, OID epoch, Timestamp creationTime, Date_t updatedAt, UUID uuid) + : CollectionTypeBase(std::move(nss), std::move(updatedAt), std::move(creationTime)) { setEpoch(std::move(epoch)); setUuid(std::move(uuid)); } -CollectionType::CollectionType(NamespaceString nss, - OID epoch, - boost::optional creationTime, - Date_t updatedAt, - UUID uuid) - : CollectionTypeBase(std::move(nss), std::move(updatedAt)) { - setEpoch(std::move(epoch)); - setUuid(std::move(uuid)); - setTimestamp(creationTime); -} - CollectionType::CollectionType(const BSONObj& obj) { CollectionType::parseProtected(IDLParserErrorContext("CollectionType"), obj); uassert(ErrorCodes::BadValue, diff --git a/src/mongo/s/catalog/type_collection.h b/src/mongo/s/catalog/type_collection.h index 79a4896306a..a903da4d224 100644 --- a/src/mongo/s/catalog/type_collection.h +++ b/src/mongo/s/catalog/type_collection.h @@ -113,13 +113,8 @@ public: // Name of the collections collection in the config server. static const NamespaceString ConfigNS; - CollectionType(NamespaceString nss, OID epoch, Date_t updatedAt, UUID uuid); - - CollectionType(NamespaceString nss, - OID epoch, - boost::optional creationTime, - Date_t updatedAt, - UUID uuid); + CollectionType( + NamespaceString nss, OID epoch, Timestamp creationTime, Date_t updatedAt, UUID uuid); explicit CollectionType(const BSONObj& obj); diff --git a/src/mongo/s/catalog/type_collection.idl b/src/mongo/s/catalog/type_collection.idl index 1d258b4811f..10612997ec8 100644 --- a/src/mongo/s/catalog/type_collection.idl +++ b/src/mongo/s/catalog/type_collection.idl @@ -70,11 +70,7 @@ structs: collection was created or it's shard key last refined. Because timestamps are comparable, we are able to define a total order in time in the collection. This field will replace Epoch, which are not - comparable. - - It is optional for parsing purposes, because in versions of MongoDB - prior to 5.0, this value wasn't being written." - optional: true + comparable." uuid: cpp_name: pre50CompatibleUuid type: uuid diff --git a/src/mongo/s/catalog/type_collection_test.cpp b/src/mongo/s/catalog/type_collection_test.cpp index a96da4c40b5..187112ad361 100644 --- a/src/mongo/s/catalog/type_collection_test.cpp +++ b/src/mongo/s/catalog/type_collection_test.cpp @@ -44,8 +44,10 @@ TEST(CollectionType, Empty) { TEST(CollectionType, Basic) { const OID oid = OID::gen(); + const Timestamp timestamp; CollectionType coll(BSON(CollectionType::kNssFieldName << "db.coll" << CollectionType::kEpochFieldName << oid + << CollectionType::kTimestampFieldName << timestamp << CollectionType::kUpdatedAtFieldName << Date_t::fromMillisSinceEpoch(1) << CollectionType::kKeyPatternFieldName << BSON("a" << 1) @@ -56,6 +58,7 @@ TEST(CollectionType, Basic) { ASSERT(coll.getNss() == NamespaceString{"db.coll"}); ASSERT_EQUALS(coll.getEpoch(), oid); + ASSERT_EQUALS(coll.getTimestamp(), timestamp); ASSERT_EQUALS(coll.getUpdatedAt(), Date_t::fromMillisSinceEpoch(1)); ASSERT_BSONOBJ_EQ(coll.getKeyPattern().toBSON(), BSON("a" << 1)); ASSERT_BSONOBJ_EQ(coll.getDefaultCollation(), @@ -68,26 +71,26 @@ TEST(CollectionType, Basic) { TEST(CollectionType, AllFieldsPresent) { const OID oid = OID::gen(); const auto uuid = UUID::gen(); + const Timestamp timestamp; const auto reshardingUuid = UUID::gen(); ReshardingFields reshardingFields; reshardingFields.setReshardingUUID(reshardingUuid); - CollectionType coll(BSON(CollectionType::kNssFieldName - << "db.coll" << CollectionType::kEpochFieldName << oid - << CollectionType::kUpdatedAtFieldName - << Date_t::fromMillisSinceEpoch(1) - << CollectionType::kKeyPatternFieldName << BSON("a" << 1) - << CollectionType::kDefaultCollationFieldName - << BSON("locale" - << "fr_CA") - << CollectionType::kUniqueFieldName << true - << CollectionType::kUuidFieldName << uuid - << CollectionType::kReshardingFieldsFieldName - << reshardingFields.toBSON())); + CollectionType coll(BSON( + CollectionType::kNssFieldName + << "db.coll" << CollectionType::kEpochFieldName << oid + << CollectionType::kTimestampFieldName << timestamp << CollectionType::kUpdatedAtFieldName + << Date_t::fromMillisSinceEpoch(1) << CollectionType::kKeyPatternFieldName << BSON("a" << 1) + << CollectionType::kDefaultCollationFieldName + << BSON("locale" + << "fr_CA") + << CollectionType::kUniqueFieldName << true << CollectionType::kUuidFieldName << uuid + << CollectionType::kReshardingFieldsFieldName << reshardingFields.toBSON())); ASSERT(coll.getNss() == NamespaceString{"db.coll"}); ASSERT_EQUALS(coll.getEpoch(), oid); + ASSERT_EQUALS(coll.getTimestamp(), timestamp); ASSERT_EQUALS(coll.getUpdatedAt(), Date_t::fromMillisSinceEpoch(1)); ASSERT_BSONOBJ_EQ(coll.getKeyPattern().toBSON(), BSON("a" << 1)); ASSERT_BSONOBJ_EQ(coll.getDefaultCollation(), @@ -102,19 +105,22 @@ TEST(CollectionType, AllFieldsPresent) { TEST(CollectionType, MissingDefaultCollationParses) { const OID oid = OID::gen(); - CollectionType coll(BSON(CollectionType::kNssFieldName - << "db.coll" << CollectionType::kEpochFieldName << oid - << CollectionType::kUpdatedAtFieldName - << Date_t::fromMillisSinceEpoch(1) - << CollectionType::kKeyPatternFieldName << BSON("a" << 1) - << CollectionType::kUniqueFieldName << true)); + const Timestamp timestamp; + CollectionType coll(BSON( + CollectionType::kNssFieldName + << "db.coll" << CollectionType::kEpochFieldName << oid + << CollectionType::kTimestampFieldName << timestamp << CollectionType::kUpdatedAtFieldName + << Date_t::fromMillisSinceEpoch(1) << CollectionType::kKeyPatternFieldName << BSON("a" << 1) + << CollectionType::kUniqueFieldName << true)); ASSERT_BSONOBJ_EQ(coll.getDefaultCollation(), BSONObj()); } TEST(CollectionType, DefaultCollationSerializesCorrectly) { const OID oid = OID::gen(); + const Timestamp timestamp; CollectionType coll(BSON(CollectionType::kNssFieldName << "db.coll" << CollectionType::kEpochFieldName << oid + << CollectionType::kTimestampFieldName << timestamp << CollectionType::kUpdatedAtFieldName << Date_t::fromMillisSinceEpoch(1) << CollectionType::kKeyPatternFieldName << BSON("a" << 1) @@ -130,7 +136,7 @@ TEST(CollectionType, DefaultCollationSerializesCorrectly) { TEST(CollectionType, Pre22Format) { CollectionType coll(BSON("_id" - << "db.coll" + << "db.coll" << CollectionType::kTimestampFieldName << Timestamp() << "lastmod" << Date_t::fromMillisSinceEpoch(1) << "dropped" << false << "key" << BSON("a" << 1) << "unique" << false)); @@ -145,7 +151,8 @@ TEST(CollectionType, Pre22Format) { TEST(CollectionType, InvalidNamespace) { ASSERT_THROWS(CollectionType(BSON(CollectionType::kNssFieldName << "foo\\bar.coll" << CollectionType::kEpochFieldName - << OID::gen() << CollectionType::kUpdatedAtFieldName + << OID::gen() << CollectionType::kTimestampFieldName + << Timestamp() << CollectionType::kUpdatedAtFieldName << Date_t::fromMillisSinceEpoch(1) << CollectionType::kKeyPatternFieldName << BSON("a" << 1) << CollectionType::kUniqueFieldName << true)), @@ -155,6 +162,7 @@ TEST(CollectionType, InvalidNamespace) { TEST(CollectionType, BadNamespaceType) { ASSERT_THROWS(CollectionType(BSON(CollectionType::kNssFieldName << 1 << CollectionType::kEpochFieldName << OID::gen() + << CollectionType::kTimestampFieldName << Timestamp() << CollectionType::kUpdatedAtFieldName << Date_t::fromMillisSinceEpoch(1) << CollectionType::kKeyPatternFieldName << BSON("a" << 1) diff --git a/src/mongo/s/catalog_cache.cpp b/src/mongo/s/catalog_cache.cpp index bc5acaa48fc..d786da676f9 100644 --- a/src/mongo/s/catalog_cache.cpp +++ b/src/mongo/s/catalog_cache.cpp @@ -659,20 +659,10 @@ CatalogCache::CollectionCache::LookupResult CatalogCache::CollectionCache::_look // updating. Otherwise, we're making a whole new routing table. if (isIncremental && existingHistory->optRt->getVersion().epoch() == collectionAndChunks.epoch) { - if (existingHistory->optRt->getVersion().getTimestamp().is_initialized() != - collectionAndChunks.creationTime.is_initialized()) { - return existingHistory->optRt - ->makeUpdatedReplacingTimestamp(collectionAndChunks.creationTime) - .makeUpdated(collectionAndChunks.reshardingFields, - maxChunkSize, - collectionAndChunks.allowMigrations, - collectionAndChunks.changedChunks); - } else { - return existingHistory->optRt->makeUpdated(collectionAndChunks.reshardingFields, - maxChunkSize, - collectionAndChunks.allowMigrations, - collectionAndChunks.changedChunks); - } + return existingHistory->optRt->makeUpdated(collectionAndChunks.reshardingFields, + maxChunkSize, + collectionAndChunks.allowMigrations, + collectionAndChunks.changedChunks); } auto defaultCollator = [&]() -> std::unique_ptr { diff --git a/src/mongo/s/catalog_cache_loader.cpp b/src/mongo/s/catalog_cache_loader.cpp index da59faa7d50..6815db8de89 100644 --- a/src/mongo/s/catalog_cache_loader.cpp +++ b/src/mongo/s/catalog_cache_loader.cpp @@ -43,7 +43,7 @@ CatalogCacheLoader::CollectionAndChangedChunks::CollectionAndChangedChunks() = d CatalogCacheLoader::CollectionAndChangedChunks::CollectionAndChangedChunks( OID collEpoch, - boost::optional collCreationTime, + Timestamp collCreationTime, UUID collUuid, const BSONObj& collShardKeyPattern, const BSONObj& collDefaultCollation, diff --git a/src/mongo/s/catalog_cache_loader.h b/src/mongo/s/catalog_cache_loader.h index 5b60d190afa..fa83d9aef52 100644 --- a/src/mongo/s/catalog_cache_loader.h +++ b/src/mongo/s/catalog_cache_loader.h @@ -68,7 +68,7 @@ public: CollectionAndChangedChunks(); CollectionAndChangedChunks( OID collEpoch, - boost::optional collCreationTime, + Timestamp collCreationTime, UUID uuid, const BSONObj& collShardKeyPattern, const BSONObj& collDefaultCollation, @@ -83,7 +83,7 @@ public: // Information about the entire collection OID epoch; - boost::optional creationTime; + Timestamp creationTime; boost::optional uuid; // This value can never be boost::none, // except under the default constructor BSONObj shardKeyPattern; diff --git a/src/mongo/s/catalog_cache_refresh_test.cpp b/src/mongo/s/catalog_cache_refresh_test.cpp index 66d247e4285..a6a75a2c696 100644 --- a/src/mongo/s/catalog_cache_refresh_test.cpp +++ b/src/mongo/s/catalog_cache_refresh_test.cpp @@ -65,19 +65,23 @@ protected: }()); } - void expectGetCollection(OID epoch, const ShardKeyPattern& shardKeyPattern) { + void expectGetCollection(OID epoch, + Timestamp timestamp, + const ShardKeyPattern& shardKeyPattern) { expectFindSendBSONObjVector(kConfigHostAndPort, [&]() { - return std::vector{getDefaultCollectionType(epoch, shardKeyPattern).toBSON()}; + return std::vector{ + getDefaultCollectionType(epoch, timestamp, shardKeyPattern).toBSON()}; }()); } void expectCollectionAndChunksAggregationWithReshardingFields( OID epoch, + Timestamp timestamp, const ShardKeyPattern& shardKeyPattern, UUID reshardingUUID, const std::vector& chunks) { expectFindSendBSONObjVector(kConfigHostAndPort, [&]() { - auto collType = getDefaultCollectionType(epoch, shardKeyPattern); + auto collType = getDefaultCollectionType(epoch, timestamp, shardKeyPattern); TypeCollectionReshardingFields reshardingFields; reshardingFields.setReshardingUUID(reshardingUUID); @@ -92,8 +96,10 @@ protected: }()); } - CollectionType getDefaultCollectionType(OID epoch, const ShardKeyPattern& shardKeyPattern) { - CollectionType collType(kNss, epoch, Date_t::now(), UUID::gen()); + CollectionType getDefaultCollectionType(OID epoch, + Timestamp timestamp, + const ShardKeyPattern& shardKeyPattern) { + CollectionType collType(kNss, epoch, timestamp, Date_t::now(), UUID::gen()); collType.setKeyPattern(shardKeyPattern.toBSON()); collType.setUnique(false); return collType; @@ -102,6 +108,7 @@ protected: TEST_F(CatalogCacheRefreshTest, FullLoad) { const OID epoch = OID::gen(); + const Timestamp timestamp(1); const ShardKeyPattern shardKeyPattern(BSON("_id" << 1)); const UUID reshardingUUID = UUID::gen(); @@ -109,7 +116,7 @@ TEST_F(CatalogCacheRefreshTest, FullLoad) { expectGetDatabase(); - ChunkVersion version(1, 0, epoch, boost::none /* timestamp */); + ChunkVersion version(1, 0, epoch, timestamp); ChunkType chunk1(reshardingUUID, {shardKeyPattern.getKeyPattern().globalMin(), BSON("_id" << -100)}, @@ -134,7 +141,7 @@ TEST_F(CatalogCacheRefreshTest, FullLoad) { version.incMinor(); expectCollectionAndChunksAggregationWithReshardingFields( - epoch, shardKeyPattern, reshardingUUID, {chunk1, chunk2, chunk3, chunk4}); + epoch, timestamp, shardKeyPattern, reshardingUUID, {chunk1, chunk2, chunk3, chunk4}); auto cm = *future.default_timed_get(); ASSERT(cm.isSharded()); @@ -238,6 +245,7 @@ TEST_F(CatalogCacheRefreshTest, CollectionBSONCorrupted) { TEST_F(CatalogCacheRefreshTest, FullLoadNoChunksFound) { const OID epoch = OID::gen(); + const Timestamp timestamp(1); const ShardKeyPattern shardKeyPattern(BSON("_id" << 1)); auto future = scheduleRoutingInfoUnforcedRefresh(kNss); @@ -246,17 +254,17 @@ TEST_F(CatalogCacheRefreshTest, FullLoadNoChunksFound) { // Return no chunks three times, which is how frequently the catalog cache retries expectFindSendBSONObjVector(kConfigHostAndPort, [&] { - const auto coll = getDefaultCollectionType(epoch, shardKeyPattern); + const auto coll = getDefaultCollectionType(epoch, timestamp, shardKeyPattern); return std::vector{coll.toBSON()}; }()); expectFindSendBSONObjVector(kConfigHostAndPort, [&] { - const auto coll = getDefaultCollectionType(epoch, shardKeyPattern); + const auto coll = getDefaultCollectionType(epoch, timestamp, shardKeyPattern); return std::vector{coll.toBSON()}; }()); expectFindSendBSONObjVector(kConfigHostAndPort, [&] { - const auto coll = getDefaultCollectionType(epoch, shardKeyPattern); + const auto coll = getDefaultCollectionType(epoch, timestamp, shardKeyPattern); return std::vector{coll.toBSON()}; }()); @@ -274,6 +282,7 @@ TEST_F(CatalogCacheRefreshTest, IncrementalLoadNoChunksFound) { auto initialRoutingInfo(makeChunkManager(kNss, shardKeyPattern, nullptr, true, {})); const OID epoch = initialRoutingInfo.getVersion().epoch(); + const Timestamp timestamp = initialRoutingInfo.getVersion().getTimestamp(); ASSERT_EQ(1, initialRoutingInfo.numChunks()); @@ -281,17 +290,17 @@ TEST_F(CatalogCacheRefreshTest, IncrementalLoadNoChunksFound) { // Return no chunks three times, which is how frequently the catalog cache retries expectFindSendBSONObjVector(kConfigHostAndPort, [&] { - const auto coll = getDefaultCollectionType(epoch, shardKeyPattern); + const auto coll = getDefaultCollectionType(epoch, timestamp, shardKeyPattern); return std::vector{coll.toBSON()}; }()); expectFindSendBSONObjVector(kConfigHostAndPort, [&] { - const auto coll = getDefaultCollectionType(epoch, shardKeyPattern); + const auto coll = getDefaultCollectionType(epoch, timestamp, shardKeyPattern); return std::vector{coll.toBSON()}; }()); expectFindSendBSONObjVector(kConfigHostAndPort, [&] { - const auto coll = getDefaultCollectionType(epoch, shardKeyPattern); + const auto coll = getDefaultCollectionType(epoch, timestamp, shardKeyPattern); return std::vector{coll.toBSON()}; }()); @@ -306,6 +315,7 @@ TEST_F(CatalogCacheRefreshTest, IncrementalLoadNoChunksFound) { TEST_F(CatalogCacheRefreshTest, ChunksBSONCorrupted) { const OID epoch = OID::gen(); + const Timestamp timestamp(1); const ShardKeyPattern shardKeyPattern(BSON("_id" << 1)); auto future = scheduleRoutingInfoUnforcedRefresh(kNss); @@ -314,11 +324,11 @@ TEST_F(CatalogCacheRefreshTest, ChunksBSONCorrupted) { // Return no chunks three times, which is how frequently the catalog cache retries expectFindSendBSONObjVector(kConfigHostAndPort, [&] { - const auto coll = getDefaultCollectionType(epoch, shardKeyPattern); + const auto coll = getDefaultCollectionType(epoch, timestamp, shardKeyPattern); const auto chunk1 = ChunkType(coll.getUuid(), {shardKeyPattern.getKeyPattern().globalMin(), BSON("_id" << 0)}, - ChunkVersion(1, 0, epoch, boost::none /* timestamp */), + ChunkVersion(1, 0, epoch, Timestamp()), {"0"}); return std::vector{/* collection */ coll.toBSON(), @@ -341,6 +351,7 @@ TEST_F(CatalogCacheRefreshTest, ChunksBSONCorrupted) { TEST_F(CatalogCacheRefreshTest, FullLoadMissingChunkWithLowestVersion) { const OID epoch = OID::gen(); const UUID uuid = UUID::gen(); + const Timestamp timestamp(1, 1); const ShardKeyPattern shardKeyPattern(BSON("_id" << 1)); auto future = scheduleRoutingInfoUnforcedRefresh(kNss); @@ -348,7 +359,7 @@ TEST_F(CatalogCacheRefreshTest, FullLoadMissingChunkWithLowestVersion) { expectGetDatabase(); const auto incompleteChunks = [&]() { - ChunkVersion version(1, 0, epoch, boost::none /* timestamp */); + ChunkVersion version(1, 0, epoch, timestamp); // Chunk from (MinKey, -100) is missing (as if someone is dropping the collection // concurrently) and has the lowest version. @@ -374,11 +385,14 @@ TEST_F(CatalogCacheRefreshTest, FullLoadMissingChunkWithLowestVersion) { // Return incomplete set of chunks three times, which is how frequently the catalog cache // retries - expectCollectionAndChunksAggregation(kNss, epoch, uuid, shardKeyPattern, incompleteChunks); + expectCollectionAndChunksAggregation( + kNss, epoch, timestamp, uuid, shardKeyPattern, incompleteChunks); - expectCollectionAndChunksAggregation(kNss, epoch, uuid, shardKeyPattern, incompleteChunks); + expectCollectionAndChunksAggregation( + kNss, epoch, timestamp, uuid, shardKeyPattern, incompleteChunks); - expectCollectionAndChunksAggregation(kNss, epoch, uuid, shardKeyPattern, incompleteChunks); + expectCollectionAndChunksAggregation( + kNss, epoch, timestamp, uuid, shardKeyPattern, incompleteChunks); try { auto cm = *future.default_timed_get(); @@ -393,6 +407,7 @@ TEST_F(CatalogCacheRefreshTest, FullLoadMissingChunkWithLowestVersion) { TEST_F(CatalogCacheRefreshTest, FullLoadMissingChunkWithHighestVersion) { const OID epoch = OID::gen(); const UUID uuid = UUID::gen(); + const Timestamp timestamp; const ShardKeyPattern shardKeyPattern(BSON("_id" << 1)); auto future = scheduleRoutingInfoUnforcedRefresh(kNss); @@ -400,7 +415,7 @@ TEST_F(CatalogCacheRefreshTest, FullLoadMissingChunkWithHighestVersion) { expectGetDatabase(); const auto incompleteChunks = [&]() { - ChunkVersion version(1, 0, epoch, boost::none /* timestamp */); + ChunkVersion version(1, 0, epoch, timestamp); // Chunk from (MinKey, -100) is missing (as if someone is dropping the collection // concurrently) and has the higest version. @@ -426,11 +441,14 @@ TEST_F(CatalogCacheRefreshTest, FullLoadMissingChunkWithHighestVersion) { // Return incomplete set of chunks three times, which is how frequently the catalog cache // retries - expectCollectionAndChunksAggregation(kNss, epoch, uuid, shardKeyPattern, incompleteChunks); + expectCollectionAndChunksAggregation( + kNss, epoch, timestamp, uuid, shardKeyPattern, incompleteChunks); - expectCollectionAndChunksAggregation(kNss, epoch, uuid, shardKeyPattern, incompleteChunks); + expectCollectionAndChunksAggregation( + kNss, epoch, timestamp, uuid, shardKeyPattern, incompleteChunks); - expectCollectionAndChunksAggregation(kNss, epoch, uuid, shardKeyPattern, incompleteChunks); + expectCollectionAndChunksAggregation( + kNss, epoch, timestamp, uuid, shardKeyPattern, incompleteChunks); try { auto cm = *future.default_timed_get(); @@ -482,13 +500,13 @@ TEST_F(CatalogCacheRefreshTest, IncrementalLoadMissingChunkWithLowestVersion) { // Return incomplete set of chunks three times, which is how frequently the catalog cache // retries expectCollectionAndChunksAggregation( - kNss, epoch, UUID::gen(), shardKeyPattern, incompleteChunks); + kNss, epoch, timestamp, UUID::gen(), shardKeyPattern, incompleteChunks); expectCollectionAndChunksAggregation( - kNss, epoch, UUID::gen(), shardKeyPattern, incompleteChunks); + kNss, epoch, timestamp, UUID::gen(), shardKeyPattern, incompleteChunks); expectCollectionAndChunksAggregation( - kNss, epoch, UUID::gen(), shardKeyPattern, incompleteChunks); + kNss, epoch, timestamp, UUID::gen(), shardKeyPattern, incompleteChunks); try { auto cm = *future.default_timed_get(); @@ -539,13 +557,13 @@ TEST_F(CatalogCacheRefreshTest, IncrementalLoadMissingChunkWithHighestVersion) { // Return incomplete set of chunks three times, which is how frequently the catalog cache // retries expectCollectionAndChunksAggregation( - kNss, epoch, UUID::gen(), shardKeyPattern, incompleteChunks); + kNss, epoch, timestamp, UUID::gen(), shardKeyPattern, incompleteChunks); expectCollectionAndChunksAggregation( - kNss, epoch, UUID::gen(), shardKeyPattern, incompleteChunks); + kNss, epoch, timestamp, UUID::gen(), shardKeyPattern, incompleteChunks); expectCollectionAndChunksAggregation( - kNss, epoch, UUID::gen(), shardKeyPattern, incompleteChunks); + kNss, epoch, timestamp, UUID::gen(), shardKeyPattern, incompleteChunks); try { auto cm = *future.default_timed_get(); @@ -569,6 +587,7 @@ TEST_F(CatalogCacheRefreshTest, ChunkEpochChangeDuringIncrementalLoadRecoveryAft ChunkVersion oldVersion = initialRoutingInfo.getVersion(); const OID newEpoch = OID::gen(); + const Timestamp newTimestamp = Timestamp(2); // On the first attempt, return set of chunks, one of which has different epoch. This simulates // the situation where a collection existed with epoch0, we started a refresh for that @@ -587,7 +606,8 @@ TEST_F(CatalogCacheRefreshTest, ChunkEpochChangeDuringIncrementalLoadRecoveryAft pipeline[2]["$unionWith"]["pipeline"].Array()[1]["$match"]["lastmodEpoch"].Obj(), BSON("$ne" << oldVersion.epoch())); - const auto coll = getDefaultCollectionType(oldVersion.epoch(), shardKeyPattern); + const auto coll = getDefaultCollectionType( + oldVersion.epoch(), oldVersion.getTimestamp(), shardKeyPattern); const auto collBSON = coll.toBSON(); oldVersion.incMajor(); @@ -601,7 +621,7 @@ TEST_F(CatalogCacheRefreshTest, ChunkEpochChangeDuringIncrementalLoadRecoveryAft // recreated collection. ChunkType chunk3(coll.getUuid(), {BSON("_id" << 100), shardKeyPattern.getKeyPattern().globalMax()}, - ChunkVersion(5, 2, newEpoch, boost::none /* timestamp */), + ChunkVersion(5, 2, newEpoch, newTimestamp), {"1"}); chunk3.setName(OID::gen()); @@ -611,7 +631,7 @@ TEST_F(CatalogCacheRefreshTest, ChunkEpochChangeDuringIncrementalLoadRecoveryAft }); // On the second retry attempt, return the correct set of chunks from the recreated collection - ChunkVersion newVersion(5, 0, newEpoch, boost::none /* timestamp */); + ChunkVersion newVersion(5, 0, newEpoch, newTimestamp); onFindCommand([&](const RemoteCommandRequest& request) { const auto opMsg = OpMsgRequest::fromDBAndBody(request.dbname, request.cmdObj); const auto aggRequest = unittest::assertGet( @@ -625,8 +645,9 @@ TEST_F(CatalogCacheRefreshTest, ChunkEpochChangeDuringIncrementalLoadRecoveryAft pipeline[2]["$unionWith"]["pipeline"].Array()[1]["$match"]["lastmodEpoch"].Obj(), BSON("$ne" << oldVersion.epoch())); - const auto coll = getDefaultCollectionType(newEpoch, shardKeyPattern); - const auto collBSON = getDefaultCollectionType(newEpoch, shardKeyPattern).toBSON(); + const auto coll = getDefaultCollectionType(newEpoch, newTimestamp, shardKeyPattern); + const auto collBSON = + getDefaultCollectionType(newEpoch, newTimestamp, shardKeyPattern).toBSON(); ChunkType chunk1(coll.getUuid(), {shardKeyPattern.getKeyPattern().globalMin(), BSON("_id" << 0)}, @@ -672,7 +693,7 @@ TEST_F(CatalogCacheRefreshTest, IncrementalLoadAfterCollectionEpochChange) { auto future = scheduleRoutingInfoIncrementalRefresh(kNss); ChunkVersion oldVersion = initialRoutingInfo.getVersion(); - ChunkVersion newVersion(1, 0, OID::gen(), boost::none /* timestamp */); + ChunkVersion newVersion(1, 0, OID::gen(), Timestamp(2)); const UUID uuid = *initialRoutingInfo.getUUID(); // Return collection with a different epoch and a set of chunks, which represent a split @@ -690,7 +711,8 @@ TEST_F(CatalogCacheRefreshTest, IncrementalLoadAfterCollectionEpochChange) { BSON("$ne" << oldVersion.epoch())); const auto collBSON = - getDefaultCollectionType(newVersion.epoch(), shardKeyPattern).toBSON(); + getDefaultCollectionType(newVersion.epoch(), newVersion.getTimestamp(), shardKeyPattern) + .toBSON(); ChunkType chunk1(uuid, {shardKeyPattern.getKeyPattern().globalMin(), BSON("_id" << 0)}, @@ -748,7 +770,8 @@ TEST_F(CatalogCacheRefreshTest, IncrementalLoadAfterSplit) { pipeline[1]["$unionWith"]["pipeline"].Array()[1]["$match"]["lastmodEpoch"].Obj(), BSON("$eq" << version.epoch())); - const auto coll = getDefaultCollectionType(version.epoch(), shardKeyPattern); + const auto coll = + getDefaultCollectionType(version.epoch(), version.getTimestamp(), shardKeyPattern); const auto collBSON = coll.toBSON(); version.incMajor(); @@ -807,7 +830,7 @@ TEST_F(CatalogCacheRefreshTest, IncrementalLoadAfterMoveWithReshardingFieldsAdde chunk2.setName(OID::gen()); expectCollectionAndChunksAggregationWithReshardingFields( - version.epoch(), shardKeyPattern, reshardingUUID, {chunk1, chunk2}); + version.epoch(), version.getTimestamp(), shardKeyPattern, reshardingUUID, {chunk1, chunk2}); auto cm = *future.default_timed_get(); ASSERT(cm.isSharded()); @@ -848,7 +871,7 @@ TEST_F(CatalogCacheRefreshTest, IncrementalLoadAfterMoveLastChunkWithReshardingF chunk1.setName(OID::gen()); expectCollectionAndChunksAggregation( - kNss, version.epoch(), UUID::gen(), shardKeyPattern, {chunk1}); + kNss, version.epoch(), version.getTimestamp(), UUID::gen(), shardKeyPattern, {chunk1}); auto cm = *future.default_timed_get(); ASSERT(cm.isSharded()); diff --git a/src/mongo/s/catalog_cache_test.cpp b/src/mongo/s/catalog_cache_test.cpp index 0d329330dac..0106cf4b393 100644 --- a/src/mongo/s/catalog_cache_test.cpp +++ b/src/mongo/s/catalog_cache_test.cpp @@ -239,7 +239,7 @@ TEST_F(CatalogCacheTest, OnStaleDatabaseVersionNoVersion) { TEST_F(CatalogCacheTest, OnStaleShardVersionWithSameVersion) { const auto dbVersion = DatabaseVersion(UUID::gen(), Timestamp()); - const auto cachedCollVersion = ChunkVersion(1, 0, OID::gen(), boost::none /* timestamp */); + const auto cachedCollVersion = ChunkVersion(1, 0, OID::gen(), Timestamp()); loadDatabases({DatabaseType(kNss.db().toString(), kShards[0], true, dbVersion)}); loadCollection(cachedCollVersion); @@ -250,7 +250,7 @@ TEST_F(CatalogCacheTest, OnStaleShardVersionWithSameVersion) { TEST_F(CatalogCacheTest, OnStaleShardVersionWithNoVersion) { const auto dbVersion = DatabaseVersion(UUID::gen(), Timestamp()); - const auto cachedCollVersion = ChunkVersion(1, 0, OID::gen(), boost::none /* timestamp */); + const auto cachedCollVersion = ChunkVersion(1, 0, OID::gen(), Timestamp()); loadDatabases({DatabaseType(kNss.db().toString(), kShards[0], true, dbVersion)}); loadCollection(cachedCollVersion); @@ -263,7 +263,7 @@ TEST_F(CatalogCacheTest, OnStaleShardVersionWithNoVersion) { TEST_F(CatalogCacheTest, OnStaleShardVersionWithGraterVersion) { const auto dbVersion = DatabaseVersion(UUID::gen(), Timestamp()); - const auto cachedCollVersion = ChunkVersion(1, 0, OID::gen(), boost::none /* timestamp */); + const auto cachedCollVersion = ChunkVersion(1, 0, OID::gen(), Timestamp()); const auto wantedCollVersion = ChunkVersion(2, 0, cachedCollVersion.epoch(), cachedCollVersion.getTimestamp()); @@ -276,168 +276,6 @@ TEST_F(CatalogCacheTest, OnStaleShardVersionWithGraterVersion) { ASSERT(status == ErrorCodes::InternalError); } -TEST_F(CatalogCacheTest, GetDatabaseWithMetadataFormatChange) { - const auto dbName = "testDB"; - const auto uuid = UUID::gen(); - const DatabaseVersion versionWithoutTimestamp(uuid, Timestamp()); - const DatabaseVersion versionWithTimestamp(uuid, Timestamp(42)); - - auto getDatabaseWithRefreshAndCheckResults = [&](const DatabaseVersion& version) { - _catalogCacheLoader->setDatabaseRefreshReturnValue( - DatabaseType(dbName, kShards[0], true, version)); - const auto cachedDb = - _catalogCache->getDatabaseWithRefresh(operationContext(), dbName).getValue(); - const auto cachedDbVersion = cachedDb.databaseVersion(); - ASSERT_EQ(cachedDbVersion.getTimestamp(), version.getTimestamp()); - }; - - // The CatalogCache is refreshed and it finds a DatabaseType using uuids. - getDatabaseWithRefreshAndCheckResults(versionWithoutTimestamp); - // The CatalogCache is forced to refresh and it finds a metadata format missmatch: we are using - // uuids locally but the loader returns a version with uuid and timestamp. The catalog cache - // returns a new DatabaseType with the new format. - getDatabaseWithRefreshAndCheckResults(versionWithTimestamp); - // The CatalogCache is forced to refresh and it finds a metadata format missmatch: we are using - // uuids and timestamps locally but the loader returns a version with only uuid. The catalog - // cache returns a new DatabaseType with the new format. - getDatabaseWithRefreshAndCheckResults(versionWithoutTimestamp); -} - -TEST_F(CatalogCacheTest, GetCollectionWithMetadataFormatChange) { - const auto dbVersion = DatabaseVersion(UUID::gen(), Timestamp()); - const auto epoch = OID::gen(); - const auto collVersionWithoutTimestamp = ChunkVersion(1, 0, epoch, boost::none /* timestamp */); - const auto collVersionWithTimestamp = ChunkVersion(1, 0, epoch, Timestamp(42)); - - loadDatabases({DatabaseType(kNss.db().toString(), kShards[0], true, dbVersion)}); - - auto getCollectionWithRefreshAndCheckResults = [this](const ChunkVersion& version) { - const auto coll = makeCollectionType(version); - const auto scopedCollProv = scopedCollectionProvider(coll); - const auto scopedChunksProv = scopedChunksProvider(makeChunks(version)); - - const auto swChunkManager = - _catalogCache->getCollectionRoutingInfoWithRefresh(operationContext(), coll.getNss()); - ASSERT_OK(swChunkManager.getStatus()); - - const auto& chunkManager = swChunkManager.getValue(); - const auto collectionVersion = chunkManager.getVersion(); - - ASSERT_EQ(collectionVersion.getTimestamp(), version.getTimestamp()); - chunkManager.forEachChunk([&](const Chunk& chunk) { - ASSERT_EQ(chunk.getLastmod().getTimestamp(), version.getTimestamp()); - return true; - }); - }; - // The CatalogCache is refreshed and it finds a Collection using epochs. - getCollectionWithRefreshAndCheckResults(collVersionWithoutTimestamp); - // The CatalogCache is forced to refresh and it finds a metadata format mismatch: we are using - // epochs locally but the loader returns a version with uuid and timestamp. The catalog cache - // returns a new ChunkManager with the new format. - getCollectionWithRefreshAndCheckResults(collVersionWithTimestamp); - // The CatalogCache is forced to refresh and it finds a metadata format mismatch: we are using - // epochs and timestamps locally but the loader returns a version with just epochs. The catalog - // cache returns a new ChunkManager with the new format. - getCollectionWithRefreshAndCheckResults(collVersionWithoutTimestamp); -} - -TEST_F(CatalogCacheTest, - GetCollectionWithRefreshDuringUpgradeWithMetadataFormatChangeChunksDontMatchCollection) { - const auto dbVersion = DatabaseVersion(UUID::gen(), Timestamp()); - const auto epoch = OID::gen(); - const auto timestamp = Timestamp(42); - - const auto collVersionWithoutTimestamp = ChunkVersion(1, 0, epoch, boost::none /* timestamp */); - const auto collVersionWithTimestamp = ChunkVersion(1, 0, epoch, timestamp); - - loadDatabases({DatabaseType(kNss.db().toString(), kShards[0], true, dbVersion)}); - - const auto coll = makeCollectionType(collVersionWithoutTimestamp); - const auto scopedCollProv = scopedCollectionProvider(coll); - const auto scopedChunksProv = scopedChunksProvider(makeChunks(collVersionWithTimestamp)); - - const auto swChunkManager = - _catalogCache->getCollectionRoutingInfoWithRefresh(operationContext(), coll.getNss()); - ASSERT_OK(swChunkManager.getStatus()); - - const auto& chunkManager = swChunkManager.getValue(); - const auto collectionVersion = chunkManager.getVersion(); - - ASSERT_EQ(collectionVersion.getTimestamp(), boost::none); - - chunkManager.forEachChunk([&](const Chunk& chunk) { - ASSERT_EQ(chunk.getLastmod().getTimestamp(), timestamp); - return true; - }); -} - -TEST_F(CatalogCacheTest, - GetCollectionWithRefreshDuringUpgradeWithMetadataFormatChangeSomeChunksMatchCollection) { - const auto dbVersion = DatabaseVersion(UUID::gen(), Timestamp()); - const auto epoch = OID::gen(); - const auto timestamp = Timestamp(42); - - const auto collVersionWithoutTimestamp = ChunkVersion(1, 0, epoch, boost::none /* timestamp */); - const auto collVersionWithTimestamp = ChunkVersion(1, 1, epoch, timestamp); - - loadDatabases({DatabaseType(kNss.db().toString(), kShards[0], true, dbVersion)}); - - const auto coll = makeCollectionType(collVersionWithoutTimestamp); - const auto scopedCollProv = scopedCollectionProvider(coll); - - ChunkType chunk1(kUUID, - {kShardKeyPattern.getKeyPattern().globalMin(), BSON("_id" << 100)}, - collVersionWithTimestamp, - {"0"}); - chunk1.setName(OID::gen()); - - ChunkType chunk2(kUUID, - {BSON("_id" << 100), kShardKeyPattern.getKeyPattern().globalMax()}, - collVersionWithoutTimestamp, - {"0"}); - chunk2.setName(OID::gen()); - - const auto scopedChunksProv = scopedChunksProvider(std::vector{chunk1, chunk2}); - - const auto swChunkManager = - _catalogCache->getCollectionRoutingInfoWithRefresh(operationContext(), coll.getNss()); - ASSERT_OK(swChunkManager.getStatus()); - - const auto& chunkManager = swChunkManager.getValue(); - const auto collectionVersion = chunkManager.getVersion(); - - ASSERT_EQ(collectionVersion.getTimestamp(), boost::none); -} - -TEST_F(CatalogCacheTest, GetCollectionWithRefreshDuringDowngradeWithMetadataFormatChange) { - const auto dbVersion = DatabaseVersion(UUID::gen(), Timestamp()); - const auto epoch = OID::gen(); - const auto timestamp = Timestamp(42); - - const auto collVersionWithoutTimestamp = ChunkVersion(1, 0, epoch, boost::none /* timestamp */); - const auto collVersionWithTimestamp = ChunkVersion(1, 0, epoch, timestamp); - - loadDatabases({DatabaseType(kNss.db().toString(), kShards[0], true, dbVersion)}); - - const auto coll = makeCollectionType(collVersionWithTimestamp); - const auto scopedCollProv = scopedCollectionProvider(coll); - const auto scopedChunksProv = scopedChunksProvider(makeChunks(collVersionWithoutTimestamp)); - - const auto swChunkManager = - _catalogCache->getCollectionRoutingInfoWithRefresh(operationContext(), coll.getNss()); - ASSERT_OK(swChunkManager.getStatus()); - - const auto& chunkManager = swChunkManager.getValue(); - const auto collectionVersion = chunkManager.getVersion(); - - ASSERT_EQ(collectionVersion.getTimestamp(), timestamp); - - chunkManager.forEachChunk([&](const Chunk& chunk) { - ASSERT_EQ(chunk.getLastmod().getTimestamp(), boost::none); - return true; - }); -} - TEST_F(CatalogCacheTest, TimeseriesFieldsAreProperlyPropagatedOnCC) { const auto dbVersion = DatabaseVersion(UUID::gen(), Timestamp()); const auto epoch = OID::gen(); diff --git a/src/mongo/s/catalog_cache_test_fixture.cpp b/src/mongo/s/catalog_cache_test_fixture.cpp index f19903c3246..3408464d6ae 100644 --- a/src/mongo/s/catalog_cache_test_fixture.cpp +++ b/src/mongo/s/catalog_cache_test_fixture.cpp @@ -140,7 +140,7 @@ ChunkManager CatalogCacheTestFixture::makeChunkManager( const auto uuid = UUID::gen(); const BSONObj collectionBSON = [&]() { - CollectionType coll(nss, version.epoch(), Date_t::now(), uuid); + CollectionType coll(nss, version.epoch(), version.getTimestamp(), Date_t::now(), uuid); coll.setKeyPattern(shardKeyPattern.getKeyPattern()); coll.setUnique(unique); @@ -204,10 +204,11 @@ void CatalogCacheTestFixture::expectGetDatabase(NamespaceString nss, std::string void CatalogCacheTestFixture::expectGetCollection(NamespaceString nss, OID epoch, + Timestamp timestamp, UUID uuid, const ShardKeyPattern& shardKeyPattern) { expectFindSendBSONObjVector(kConfigHostAndPort, [&]() { - CollectionType collType(nss, epoch, Date_t::now(), uuid); + CollectionType collType(nss, epoch, timestamp, Date_t::now(), uuid); collType.setKeyPattern(shardKeyPattern.toBSON()); collType.setUnique(false); return std::vector{collType.toBSON()}; @@ -217,11 +218,12 @@ void CatalogCacheTestFixture::expectGetCollection(NamespaceString nss, void CatalogCacheTestFixture::expectCollectionAndChunksAggregation( NamespaceString nss, OID epoch, + Timestamp timestamp, UUID uuid, const ShardKeyPattern& shardKeyPattern, const std::vector& chunks) { expectFindSendBSONObjVector(kConfigHostAndPort, [&]() { - CollectionType collType(nss, epoch, Date_t::now(), uuid); + CollectionType collType(nss, epoch, timestamp, Date_t::now(), uuid); collType.setKeyPattern(shardKeyPattern.toBSON()); collType.setUnique(false); @@ -254,6 +256,7 @@ ChunkManager CatalogCacheTestFixture::loadRoutingTableWithTwoChunksAndTwoShardsI boost::optional primaryShardId, UUID uuid) { const OID epoch = OID::gen(); + const Timestamp timestamp(1); const ShardKeyPattern shardKeyPattern(shardKey); auto future = scheduleRoutingInfoForcedRefresh(nss); @@ -267,11 +270,11 @@ ChunkManager CatalogCacheTestFixture::loadRoutingTableWithTwoChunksAndTwoShardsI } } expectFindSendBSONObjVector(kConfigHostAndPort, [&]() { - CollectionType collType(nss, epoch, Date_t::now(), uuid); + CollectionType collType(nss, epoch, timestamp, Date_t::now(), uuid); collType.setKeyPattern(shardKeyPattern.toBSON()); collType.setUnique(false); - ChunkVersion version(1, 0, epoch, boost::none /* timestamp */); + ChunkVersion version(1, 0, epoch, timestamp); ChunkType chunk1( uuid, {shardKeyPattern.getKeyPattern().globalMin(), BSON("_id" << 0)}, version, {"0"}); diff --git a/src/mongo/s/catalog_cache_test_fixture.h b/src/mongo/s/catalog_cache_test_fixture.h index a1fed893d2d..ee15166dd83 100644 --- a/src/mongo/s/catalog_cache_test_fixture.h +++ b/src/mongo/s/catalog_cache_test_fixture.h @@ -127,10 +127,12 @@ protected: void expectGetDatabase(NamespaceString nss, std::string primaryShard = "0"); void expectGetCollection(NamespaceString nss, OID epoch, + Timestamp timestamp, UUID uuid, const ShardKeyPattern& shardKeyPattern); void expectCollectionAndChunksAggregation(NamespaceString nss, OID epoch, + Timestamp timestamp, UUID uuid, const ShardKeyPattern& shardKeyPattern, const std::vector& chunks); diff --git a/src/mongo/s/chunk_manager.cpp b/src/mongo/s/chunk_manager.cpp index b883348bcd9..d79a69b7b39 100644 --- a/src/mongo/s/chunk_manager.cpp +++ b/src/mongo/s/chunk_manager.cpp @@ -308,8 +308,7 @@ ChunkMap::_overlappingBounds(const BSONObj& min, const BSONObj& max, bool isMaxI return {itMin, itMax}; } -ShardVersionTargetingInfo::ShardVersionTargetingInfo(const OID& epoch, - const boost::optional& timestamp) +ShardVersionTargetingInfo::ShardVersionTargetingInfo(const OID& epoch, const Timestamp& timestamp) : shardVersion(0, 0, epoch, timestamp) {} RoutingTableHistory::RoutingTableHistory( @@ -771,7 +770,7 @@ RoutingTableHistory RoutingTableHistory::makeNew( std::unique_ptr defaultCollator, bool unique, OID epoch, - const boost::optional& timestamp, + const Timestamp& timestamp, boost::optional timeseriesFields, boost::optional reshardingFields, boost::optional maxChunkSizeBytes, @@ -817,38 +816,6 @@ RoutingTableHistory RoutingTableHistory::makeUpdated( std::move(chunkMap)); } -RoutingTableHistory RoutingTableHistory::makeUpdatedReplacingTimestamp( - const boost::optional& timestamp) const { - invariant(getVersion().getTimestamp().is_initialized() != timestamp.is_initialized()); - - ChunkMap newMap(getVersion().epoch(), timestamp, _chunkMap.size()); - _chunkMap.forEach([&](const std::shared_ptr& chunkInfo) { - const ChunkVersion oldVersion = chunkInfo->getLastmod(); - newMap.appendChunk(std::make_shared(chunkInfo->getRange(), - chunkInfo->getMaxKeyString(), - chunkInfo->getShardId(), - ChunkVersion(oldVersion.majorVersion(), - oldVersion.minorVersion(), - oldVersion.epoch(), - timestamp), - chunkInfo->getHistory(), - chunkInfo->isJumbo(), - chunkInfo->getWritesTracker())); - return true; - }); - - return RoutingTableHistory(_nss, - _uuid, - getShardKeyPattern().getKeyPattern(), - CollatorInterface::cloneCollator(getDefaultCollator()), - _unique, - _timeseriesFields, - _reshardingFields, - _maxChunkSizeBytes, - _allowMigrations, - std::move(newMap)); -} - AtomicWord ComparableChunkVersion::_epochDisambiguatingSequenceNumSource{1ULL}; AtomicWord ComparableChunkVersion::_forcedRefreshSequenceNumSource{1ULL}; @@ -921,29 +888,17 @@ bool ComparableChunkVersion::operator<(const ComparableChunkVersion& other) cons // _epochDisambiguatingSequenceNum to see which // one is more recent. - const boost::optional timestamp = _chunkVersion->getTimestamp(); - const boost::optional otherTimestamp = other._chunkVersion->getTimestamp(); - if (timestamp && otherTimestamp) { - if (_chunkVersion->isSet() && other._chunkVersion->isSet()) { - if (*timestamp == *otherTimestamp) - return _chunkVersion->majorVersion() < other._chunkVersion->majorVersion() || - (_chunkVersion->majorVersion() == other._chunkVersion->majorVersion() && - _chunkVersion->minorVersion() < other._chunkVersion->minorVersion()); - else - return *timestamp < *otherTimestamp; - } else if (!_chunkVersion->isSet() && !other._chunkVersion->isSet()) - return false; // Both sides are the "no chunks on the shard version" - } else if (sameEpoch(other)) { - if (_chunkVersion->isSet() && other._chunkVersion->isSet()) + if (_chunkVersion->getTimestamp() == other._chunkVersion->getTimestamp()) { + if (!_chunkVersion->isSet() && !other._chunkVersion->isSet()) { + return false; + } else if (_chunkVersion->isSet() && other._chunkVersion->isSet()) { return _chunkVersion->majorVersion() < other._chunkVersion->majorVersion() || (_chunkVersion->majorVersion() == other._chunkVersion->majorVersion() && _chunkVersion->minorVersion() < other._chunkVersion->minorVersion()); - else if (!_chunkVersion->isSet() && !other._chunkVersion->isSet()) - return false; // Both sides are the "no chunks on the shard version" + } + } else if (_chunkVersion->isSet() && other._chunkVersion->isSet()) { + return _chunkVersion->getTimestamp() < other._chunkVersion->getTimestamp(); } - - // If the epochs are different, or if they match, but one of the versions is the "no chunks" - // version, use the _epochDisambiguatingSequenceNum to disambiguate return _epochDisambiguatingSequenceNum < other._epochDisambiguatingSequenceNum; } diff --git a/src/mongo/s/chunk_manager.h b/src/mongo/s/chunk_manager.h index 313f46bddc4..eaf226a153d 100644 --- a/src/mongo/s/chunk_manager.h +++ b/src/mongo/s/chunk_manager.h @@ -60,7 +60,7 @@ struct ShardVersionTargetingInfo { // Max chunk version for the shard ChunkVersion shardVersion; - ShardVersionTargetingInfo(const OID& epoch, const boost::optional& timestamp); + ShardVersionTargetingInfo(const OID& epoch, const Timestamp& timestamp); }; // Map from a shard to a struct indicating both the max chunk version on that shard and whether the @@ -77,9 +77,7 @@ class ChunkMap { using ChunkVector = std::vector>; public: - explicit ChunkMap(OID epoch, - const boost::optional& timestamp, - size_t initialCapacity = 0) + explicit ChunkMap(OID epoch, const Timestamp& timestamp, size_t initialCapacity = 0) : _collectionVersion(0, 0, epoch, timestamp), _collTimestamp(timestamp) { _chunkMap.reserve(initialCapacity); } @@ -141,7 +139,7 @@ private: // (non-atomically), it is possible that chunks exist with timestamps, but the corresponding // config.collections entry doesn't. In this case, the chunks timestamp should be ignored when // computing the collection version and we should use _collTimestamp instead. - boost::optional _collTimestamp; + Timestamp _collTimestamp; }; /** @@ -177,7 +175,7 @@ public: std::unique_ptr defaultCollator, bool unique, OID epoch, - const boost::optional& timestamp, + const Timestamp& timestamp, boost::optional timeseriesFields, boost::optional reshardingFields, boost::optional maxChunkSizeBytes, @@ -202,13 +200,6 @@ public: bool allowMigrations, const std::vector& changedChunks) const; - /** - * Constructs a new instance with the same routing table but adding or removing timestamp on all - * chunks - */ - RoutingTableHistory makeUpdatedReplacingTimestamp( - const boost::optional& timestamp) const; - const NamespaceString& nss() const { return _nss; } diff --git a/src/mongo/s/chunk_manager_query_test.cpp b/src/mongo/s/chunk_manager_query_test.cpp index d6db4fc03e2..a6018536321 100644 --- a/src/mongo/s/chunk_manager_query_test.cpp +++ b/src/mongo/s/chunk_manager_query_test.cpp @@ -504,7 +504,7 @@ TEST_F(ChunkManagerQueryTest, SimpleCollationNumbersMultiShard) { TEST_F(ChunkManagerQueryTest, SnapshotQueryWithMoreShardsThanLatestMetadata) { const auto uuid = UUID::gen(); const auto epoch = OID::gen(); - ChunkVersion version(1, 0, epoch, boost::none /* timestamp */); + ChunkVersion version(1, 0, epoch, Timestamp()); ChunkType chunk0(uuid, {BSON("x" << MINKEY), BSON("x" << 0)}, version, ShardId("0")); chunk0.setName(OID::gen()); @@ -519,7 +519,7 @@ TEST_F(ChunkManagerQueryTest, SnapshotQueryWithMoreShardsThanLatestMetadata) { nullptr, false, epoch, - boost::none /* timestamp */, + Timestamp(), boost::none /* timeseriesFields */, boost::none, boost::none /* chunkSizeBytes */, diff --git a/src/mongo/s/chunk_manager_refresh_bm.cpp b/src/mongo/s/chunk_manager_refresh_bm.cpp index efaa2fc475b..f7c8a94095d 100644 --- a/src/mongo/s/chunk_manager_refresh_bm.cpp +++ b/src/mongo/s/chunk_manager_refresh_bm.cpp @@ -76,7 +76,7 @@ CollectionMetadata makeChunkManagerWithShardSelector(int nShards, for (uint32_t i = 0; i < nChunks; ++i) { chunks.emplace_back(collUuid, getRangeForChunk(i, nChunks), - ChunkVersion{i + 1, 0, collEpoch, boost::none /* timestamp */}, + ChunkVersion{i + 1, 0, collEpoch, Timestamp()}, selectShard(i, nShards, nChunks)); } @@ -86,7 +86,7 @@ CollectionMetadata makeChunkManagerWithShardSelector(int nShards, nullptr, true, collEpoch, - boost::none /* timestamp */, + Timestamp(), boost::none /* timeseriesFields */, boost::none, boost::none /* chunkSizeBytes */, @@ -168,7 +168,7 @@ auto BM_FullBuildOfChunkManager(benchmark::State& state, ShardSelectorFn selectS for (uint32_t i = 0; i < nChunks; ++i) { chunks.emplace_back(collUuid, getRangeForChunk(i, nChunks), - ChunkVersion{i + 1, 0, collEpoch, boost::none /* timestamp */}, + ChunkVersion{i + 1, 0, collEpoch, Timestamp()}, selectShard(i, nShards, nChunks)); } @@ -179,7 +179,7 @@ auto BM_FullBuildOfChunkManager(benchmark::State& state, ShardSelectorFn selectS nullptr, true, collEpoch, - boost::none /* timestamp */, + Timestamp(), boost::none /* timeseriesFields */, boost::none, boost::none /* chunkSizeBytes */, diff --git a/src/mongo/s/chunk_map_test.cpp b/src/mongo/s/chunk_map_test.cpp index 8ba4bff57c6..5dfa5c31479 100644 --- a/src/mongo/s/chunk_map_test.cpp +++ b/src/mongo/s/chunk_map_test.cpp @@ -58,7 +58,7 @@ private: TEST_F(ChunkMapTest, TestAddChunk) { const OID epoch = OID::gen(); - ChunkVersion version{1, 0, epoch, boost::none /* timestamp */}; + ChunkVersion version{1, 0, epoch, Timestamp()}; auto chunk = std::make_shared( ChunkType{uuid(), @@ -66,7 +66,7 @@ TEST_F(ChunkMapTest, TestAddChunk) { version, kThisShard}); - ChunkMap chunkMap{epoch, boost::none /* timestamp */}; + ChunkMap chunkMap{epoch, Timestamp()}; auto newChunkMap = chunkMap.createMerged({chunk}); ASSERT_EQ(newChunkMap.size(), 1); @@ -74,8 +74,8 @@ TEST_F(ChunkMapTest, TestAddChunk) { TEST_F(ChunkMapTest, TestEnumerateAllChunks) { const OID epoch = OID::gen(); - ChunkMap chunkMap{epoch, boost::none /* timestamp */}; - ChunkVersion version{1, 0, epoch, boost::none /* timestamp */}; + ChunkMap chunkMap{epoch, Timestamp()}; + ChunkVersion version{1, 0, epoch, Timestamp()}; auto newChunkMap = chunkMap.createMerged( {std::make_shared( @@ -109,8 +109,8 @@ TEST_F(ChunkMapTest, TestEnumerateAllChunks) { TEST_F(ChunkMapTest, TestIntersectingChunk) { const OID epoch = OID::gen(); - ChunkMap chunkMap{epoch, boost::none /* timestamp */}; - ChunkVersion version{1, 0, epoch, boost::none /* timestamp */}; + ChunkMap chunkMap{epoch, Timestamp()}; + ChunkVersion version{1, 0, epoch, Timestamp()}; auto newChunkMap = chunkMap.createMerged( {std::make_shared( @@ -139,8 +139,8 @@ TEST_F(ChunkMapTest, TestIntersectingChunk) { TEST_F(ChunkMapTest, TestEnumerateOverlappingChunks) { const OID epoch = OID::gen(); - ChunkMap chunkMap{epoch, boost::none /* timestamp */}; - ChunkVersion version{1, 0, epoch, boost::none /* timestamp */}; + ChunkMap chunkMap{epoch, Timestamp()}; + ChunkVersion version{1, 0, epoch, Timestamp()}; auto newChunkMap = chunkMap.createMerged( {std::make_shared( diff --git a/src/mongo/s/chunk_test.cpp b/src/mongo/s/chunk_test.cpp index 5ce9358a8f7..251c5950245 100644 --- a/src/mongo/s/chunk_test.cpp +++ b/src/mongo/s/chunk_test.cpp @@ -48,7 +48,7 @@ const KeyPattern kShardKeyPattern(BSON("a" << 1)); TEST(ChunkTest, HasMovedSincePinnedTimestamp) { const OID epoch = OID::gen(); const UUID uuid = UUID::gen(); - ChunkVersion version{1, 0, epoch, boost::none /* timestamp */}; + ChunkVersion version{1, 0, epoch, Timestamp()}; ChunkType chunkType(uuid, ChunkRange{kShardKeyPattern.globalMin(), kShardKeyPattern.globalMax()}, @@ -65,7 +65,7 @@ TEST(ChunkTest, HasMovedSincePinnedTimestamp) { TEST(ChunkTest, HasMovedAndReturnedSincePinnedTimestamp) { const OID epoch = OID::gen(); const UUID uuid = UUID::gen(); - ChunkVersion version{1, 0, epoch, boost::none /* timestamp */}; + ChunkVersion version{1, 0, epoch, Timestamp()}; ChunkType chunkType(uuid, ChunkRange{kShardKeyPattern.globalMin(), kShardKeyPattern.globalMax()}, @@ -83,7 +83,7 @@ TEST(ChunkTest, HasMovedAndReturnedSincePinnedTimestamp) { TEST(ChunkTest, HasNotMovedSincePinnedTimestamp) { const OID epoch = OID::gen(); const UUID uuid = UUID::gen(); - ChunkVersion version{1, 0, epoch, boost::none /* timestamp */}; + ChunkVersion version{1, 0, epoch, Timestamp()}; ChunkType chunkType(uuid, ChunkRange{kShardKeyPattern.globalMin(), kShardKeyPattern.globalMax()}, @@ -101,7 +101,7 @@ TEST(ChunkTest, HasNotMovedSincePinnedTimestamp) { TEST(ChunkTest, HasNoHistoryValidForPinnedTimestamp_OneEntry) { const OID epoch = OID::gen(); const UUID uuid = UUID::gen(); - ChunkVersion version{1, 0, epoch, boost::none /* timestamp */}; + ChunkVersion version{1, 0, epoch, Timestamp()}; ChunkType chunkType(uuid, ChunkRange{kShardKeyPattern.globalMin(), kShardKeyPattern.globalMax()}, @@ -117,7 +117,7 @@ TEST(ChunkTest, HasNoHistoryValidForPinnedTimestamp_OneEntry) { TEST(ChunkTest, HasNoHistoryValidForPinnedTimestamp_MoreThanOneEntry) { const OID epoch = OID::gen(); const UUID uuid = UUID::gen(); - ChunkVersion version{1, 0, epoch, boost::none /* timestamp */}; + ChunkVersion version{1, 0, epoch, Timestamp()}; ChunkType chunkType(uuid, ChunkRange{kShardKeyPattern.globalMin(), kShardKeyPattern.globalMax()}, diff --git a/src/mongo/s/chunk_version.cpp b/src/mongo/s/chunk_version.cpp index d980516905c..3379d433f35 100644 --- a/src/mongo/s/chunk_version.cpp +++ b/src/mongo/s/chunk_version.cpp @@ -58,7 +58,7 @@ StatusWith ChunkVersion::fromBSON(const BSONObj& obj) { ChunkVersion version; - // Expect the timestamp + // Expect the major and minor versions { BSONElement tsPart = it.next(); if (tsPart.type() != bsonTimestamp) @@ -80,24 +80,26 @@ StatusWith ChunkVersion::fromBSON(const BSONObj& obj) { version._epoch = epochPart.OID(); } - // The following code handles the optional fields: canThrowSSVOnIgnored and timestamp. It is a - // bit complex because this function relies on the order of the fields, but since both of them - // are optional they might/might not be present. BSONElement nextElem = it.next(); - // + // TODO SERVER-59105: remove once 6.0 is last-lts. For backward compatibility reasons 5.0 // routers sends canThrowSSVOnIgnored even though it is not used, so we attempt to parse and // ignore it. - if (!nextElem.eoo() && nextElem.type() == BSONType::Bool) { + if (nextElem.type() == BSONType::Bool) { nextElem = it.next(); } - if (!nextElem.eoo()) { - if (nextElem.type() != BSONType::bsonTimestamp) - return {ErrorCodes::TypeMismatch, - str::stream() << "Invalid type " << nextElem.type() - << " for version timestamp part."}; + // Check for timestamp + if (nextElem.type() == bsonTimestamp) { version._timestamp = nextElem.timestamp(); + } else if (nextElem.eoo() && version.is50IgnoredOrUnsharded()) { + // In 5.0 binaries, the timestamp is not present in UNSHARDED and IGNORED versions + version._timestamp = + (version.epoch() == UNSHARDED().epoch()) ? Timestamp() : Timestamp::max(); + } else { + return {ErrorCodes::TypeMismatch, + str::stream() << "Invalid type " << nextElem.type() + << " for version timestamp part."}; } return version; @@ -122,12 +124,14 @@ StatusWith ChunkVersion::parseLegacyWithField(const BSONObj& obj, } } + bool fullVersion = false; // Expect the epoch OID { const auto epochField = field + "Epoch"; auto epochElem = obj[epochField]; if (epochElem.type() == jstOID) { version._epoch = epochElem.OID(); + fullVersion = true; } else if (!epochElem.eoo()) { return {ErrorCodes::TypeMismatch, str::stream() << "Invalid type " << epochElem.type() @@ -135,16 +139,24 @@ StatusWith ChunkVersion::parseLegacyWithField(const BSONObj& obj, } } - // Handle the timestamp if present + // Expect the timestamp { const auto timestampField = field + "Timestamp"; auto timestampElem = obj[timestampField]; - if (timestampElem.type() == bsonTimestamp) { - version._timestamp = timestampElem.timestamp(); - } else if (!timestampElem.eoo()) { - return {ErrorCodes::TypeMismatch, - str::stream() << "Invalid type " << timestampElem.type() - << " for version timestamp part."}; + if (fullVersion) { + if (timestampElem.type() == bsonTimestamp) { + version._timestamp = timestampElem.timestamp(); + } else if (timestampElem.eoo() && version.is50IgnoredOrUnsharded()) { + // In 5.0 binaries, the timestamp is not present in UNSHARDED and IGNORED versions + version._timestamp = + (version.epoch() == UNSHARDED().epoch()) ? Timestamp() : Timestamp::max(); + } else { + return {ErrorCodes::TypeMismatch, + str::stream() << "Invalid type " << timestampElem.type() + << " for version timestamp part."}; + } + } else { + invariant(timestampElem.eoo()); } } @@ -155,26 +167,20 @@ void ChunkVersion::appendWithField(BSONObjBuilder* out, StringData field) const BSONArrayBuilder arr(out->subarrayStart(field)); arr.appendTimestamp(_combined); arr.append(_epoch); - if (_timestamp) { - arr.append(*_timestamp); - } + arr.append(_timestamp); } void ChunkVersion::appendLegacyWithField(BSONObjBuilder* out, StringData field) const { out->appendTimestamp(field, _combined); out->append(field + "Epoch", _epoch); - if (_timestamp) { - out->append(field + "Timestamp", *_timestamp); - } + out->append(field + "Timestamp", _timestamp); } BSONObj ChunkVersion::toBSON() const { BSONArrayBuilder b; b.appendTimestamp(_combined); b.append(_epoch); - if (_timestamp) { - b.append(*_timestamp); - } + b.append(_timestamp); return b.arr(); } @@ -183,8 +189,8 @@ void ChunkVersion::legacyToBSON(StringData field, BSONObjBuilder* out) const { } std::string ChunkVersion::toString() const { - return str::stream() << majorVersion() << "|" << minorVersion() << "||" << _epoch - << (_timestamp ? "||" + _timestamp->toString() : ""); + return str::stream() << majorVersion() << "|" << minorVersion() << "||" << _epoch << "||" + << _timestamp.toString(); } } // namespace mongo diff --git a/src/mongo/s/chunk_version.h b/src/mongo/s/chunk_version.h index 70e83b3078c..9df48127f3e 100644 --- a/src/mongo/s/chunk_version.h +++ b/src/mongo/s/chunk_version.h @@ -54,15 +54,12 @@ public: */ static constexpr StringData kShardVersionField = "shardVersion"_sd; - ChunkVersion(uint32_t major, - uint32_t minor, - const OID& epoch, - boost::optional timestamp) + ChunkVersion(uint32_t major, uint32_t minor, const OID& epoch, Timestamp timestamp) : _combined(static_cast(minor) | (static_cast(major) << 32)), _epoch(epoch), _timestamp(std::move(timestamp)) {} - ChunkVersion() : ChunkVersion(0, 0, OID(), boost::none) {} + ChunkVersion() : ChunkVersion(0, 0, OID(), Timestamp()) {} static StatusWith parseFromCommand(const BSONObj& obj) { return parseWithField(obj, kShardVersionField); @@ -76,8 +73,8 @@ public: static StatusWith parseWithField(const BSONObj& obj, StringData field); /** - * Parses 'obj', which is expected to have two elements: the timestamp and the object id. The - * field names don't matter, so 'obj' can be a BSONArray. + * Parses 'obj', which is expected to have three elements: the major/minor versions, the object + * id, and the timestamp. The field names don't matter, so 'obj' can be a BSONArray. */ static StatusWith fromBSON(const BSONObj& obj); @@ -126,13 +123,23 @@ public: */ static ChunkVersion IGNORED() { ChunkVersion version; - version._epoch.init(Date_t(), true); // ignored OID is zero time, max machineId/inc + version._epoch.init(Date_t(), true); // ignored OID is zero time, max machineId/inc + version._timestamp = Timestamp::max(); // ignored Timestamp is the largest timestamp return version; } static bool isIgnoredVersion(const ChunkVersion& version) { return version.majorVersion() == 0 && version.minorVersion() == 0 && - version.epoch() == IGNORED().epoch(); + version.epoch() == IGNORED().epoch() && + version.getTimestamp() == IGNORED().getTimestamp(); + } + + /** + * Needed for parsing IGNORED and UNSHARDED from 5.0 that didn't include a timestamp. Should be + * removed after 6.0 is last-lts. + */ + bool is50IgnoredOrUnsharded() { + return _combined == 0 && (_epoch == UNSHARDED().epoch() || _epoch == IGNORED().epoch()); } void incMajor() { @@ -176,7 +183,7 @@ public: return _epoch; } - boost::optional getTimestamp() const { + const Timestamp& getTimestamp() const { return _timestamp; } @@ -253,7 +260,7 @@ private: uint64_t _combined; OID _epoch; - boost::optional _timestamp; + Timestamp _timestamp; }; inline std::ostream& operator<<(std::ostream& s, const ChunkVersion& v) { diff --git a/src/mongo/s/chunk_version_test.cpp b/src/mongo/s/chunk_version_test.cpp index a3c2d064ded..4350d0917d5 100644 --- a/src/mongo/s/chunk_version_test.cpp +++ b/src/mongo/s/chunk_version_test.cpp @@ -66,15 +66,12 @@ TEST(ChunkVersionParsing, ToFromBSONLegacyRoundtrip) { } TEST(ChunkVersionParsing, FromBSONMissingTimestamp) { - const OID oid = OID::gen(); - ChunkVersion chunkVersionComplete = assertGet(ChunkVersion::parseWithField( - BSON("testVersionField" << BSON_ARRAY(Timestamp(Seconds(2), 3) << oid)), - "testVersionField")); - - ASSERT(chunkVersionComplete.epoch().isSet()); - ASSERT_EQ(oid, chunkVersionComplete.epoch()); - ASSERT_EQ(2u, chunkVersionComplete.majorVersion()); - ASSERT_EQ(3u, chunkVersionComplete.minorVersion()); + ASSERT_THROWS_CODE( + uassertStatusOK(ChunkVersion::parseWithField( + BSON("testVersionField" << BSON_ARRAY(Timestamp(Seconds(2), 3) << OID::gen())), + "testVersionField")), + AssertionException, + ErrorCodes::TypeMismatch); } TEST(ChunkVersionParsing, FromBSON) { @@ -88,7 +85,7 @@ TEST(ChunkVersionParsing, FromBSON) { ASSERT_EQ(oid, chunkVersionComplete.epoch()); ASSERT_EQ(2u, chunkVersionComplete.majorVersion()); ASSERT_EQ(3u, chunkVersionComplete.minorVersion()); - ASSERT_EQ(timestamp, *chunkVersionComplete.getTimestamp()); + ASSERT_EQ(timestamp, chunkVersionComplete.getTimestamp()); } TEST(ChunkVersionParsing, FromBSONMissingEpoch) { @@ -121,7 +118,7 @@ TEST(ChunkVersionParsing, FromBSONLegacy) { ASSERT_EQ(3u, chunkVersionComplete.minorVersion()); } -TEST(ChunkVersionParsing, FromBSONLegacyEpochIsOptional) { +TEST(ChunkVersionParsing, FromBSONLegacyEpochAndTimestampOptional) { ChunkVersion chunkVersionNoEpoch = assertGet( ChunkVersion::parseLegacyWithField(BSON("lastmod" << Timestamp(Seconds(3), 4)), "lastmod")); @@ -132,43 +129,32 @@ TEST(ChunkVersionParsing, FromBSONLegacyEpochIsOptional) { TEST(ChunkVersionComparison, EqualityOperators) { OID epoch = OID::gen(); + Timestamp timestamp = Timestamp(1); + + ASSERT_EQ(ChunkVersion(3, 1, epoch, Timestamp()), ChunkVersion(3, 1, epoch, Timestamp())); + ASSERT_EQ(ChunkVersion(3, 1, OID(), timestamp), ChunkVersion(3, 1, OID(), timestamp)); - ASSERT_EQ(ChunkVersion(3, 1, epoch, boost::none /* timestamp */), - ChunkVersion(3, 1, epoch, boost::none /* timestamp */)); - ASSERT_EQ(ChunkVersion(3, 1, OID(), boost::none /* timestamp */), - ChunkVersion(3, 1, OID(), boost::none /* timestamp */)); - - ASSERT_NE(ChunkVersion(3, 1, epoch, boost::none /* timestamp */), - ChunkVersion(3, 1, OID(), boost::none /* timestamp */)); - ASSERT_NE(ChunkVersion(3, 1, OID(), boost::none /* timestamp */), - ChunkVersion(3, 1, epoch, boost::none /* timestamp */)); - ASSERT_NE(ChunkVersion(4, 2, epoch, boost::none /* timestamp */), - ChunkVersion(4, 1, epoch, boost::none /* timestamp */)); + ASSERT_NE(ChunkVersion(3, 1, epoch, timestamp), ChunkVersion(3, 1, OID(), Timestamp())); + ASSERT_NE(ChunkVersion(3, 1, OID(), Timestamp()), ChunkVersion(3, 1, epoch, timestamp)); + ASSERT_NE(ChunkVersion(4, 2, epoch, timestamp), ChunkVersion(4, 1, epoch, timestamp)); } TEST(ChunkVersionComparison, OlderThan) { OID epoch = OID::gen(); + Timestamp timestamp(1); - ASSERT(ChunkVersion(3, 1, epoch, boost::none /* timestamp */) - .isOlderThan(ChunkVersion(4, 1, epoch, boost::none /* timestamp */))); - ASSERT(!ChunkVersion(4, 1, epoch, boost::none /* timestamp */) - .isOlderThan(ChunkVersion(3, 1, epoch, boost::none /* timestamp */))); + ASSERT(ChunkVersion(3, 1, epoch, timestamp).isOlderThan(ChunkVersion(4, 1, epoch, timestamp))); + ASSERT(!ChunkVersion(4, 1, epoch, timestamp).isOlderThan(ChunkVersion(3, 1, epoch, timestamp))); - ASSERT(ChunkVersion(3, 1, epoch, boost::none /* timestamp */) - .isOlderThan(ChunkVersion(3, 2, epoch, boost::none /* timestamp */))); - ASSERT(!ChunkVersion(3, 2, epoch, boost::none /* timestamp */) - .isOlderThan(ChunkVersion(3, 1, epoch, boost::none /* timestamp */))); + ASSERT(ChunkVersion(3, 1, epoch, timestamp).isOlderThan(ChunkVersion(3, 2, epoch, timestamp))); + ASSERT(!ChunkVersion(3, 2, epoch, timestamp).isOlderThan(ChunkVersion(3, 1, epoch, timestamp))); - ASSERT(!ChunkVersion(3, 1, epoch, boost::none /* timestamp */) - .isOlderThan(ChunkVersion(4, 1, OID(), boost::none /* timestamp */))); - ASSERT(!ChunkVersion(4, 1, OID(), boost::none /* timestamp */) - .isOlderThan(ChunkVersion(3, 1, epoch, boost::none /* timestamp */))); + ASSERT(!ChunkVersion(3, 1, epoch, timestamp).isOlderThan(ChunkVersion(4, 1, OID(), timestamp))); + ASSERT(!ChunkVersion(4, 1, OID(), timestamp).isOlderThan(ChunkVersion(3, 1, epoch, timestamp))); - ASSERT(ChunkVersion(3, 2, epoch, boost::none /* timestamp */) - .isOlderThan(ChunkVersion(4, 1, epoch, boost::none /* timestamp */))); + ASSERT(ChunkVersion(3, 2, epoch, timestamp).isOlderThan(ChunkVersion(4, 1, epoch, timestamp))); - ASSERT(!ChunkVersion(3, 1, epoch, boost::none /* timestamp */) - .isOlderThan(ChunkVersion(3, 1, epoch, boost::none /* timestamp */))); + ASSERT(!ChunkVersion(3, 1, epoch, timestamp).isOlderThan(ChunkVersion(3, 1, epoch, timestamp))); } TEST(ChunkVersionConstruction, CreateWithLargeValues) { @@ -176,7 +162,7 @@ TEST(ChunkVersionConstruction, CreateWithLargeValues) { const uint32_t majorVersion = 1 << 24; const auto epoch = OID::gen(); - ChunkVersion version(majorVersion, minorVersion, epoch, boost::none /* timestamp */); + ChunkVersion version(majorVersion, minorVersion, epoch, Timestamp()); ASSERT_EQ(majorVersion, version.majorVersion()); ASSERT_EQ(minorVersion, version.minorVersion()); ASSERT_EQ(epoch, version.epoch()); @@ -187,7 +173,7 @@ TEST(ChunkVersionManipulation, ThrowsErrorIfOverflowIsAttemptedForMajorVersion) const uint32_t majorVersion = std::numeric_limits::max(); const auto epoch = OID::gen(); - ChunkVersion version(majorVersion, minorVersion, epoch, boost::none /* timestamp */); + ChunkVersion version(majorVersion, minorVersion, epoch, Timestamp()); ASSERT_EQ(majorVersion, version.majorVersion()); ASSERT_EQ(minorVersion, version.minorVersion()); ASSERT_EQ(epoch, version.epoch()); @@ -200,7 +186,7 @@ TEST(ChunkVersionManipulation, ThrowsErrorIfOverflowIsAttemptedForMinorVersion) const uint32_t majorVersion = 0; const auto epoch = OID::gen(); - ChunkVersion version(majorVersion, minorVersion, epoch, boost::none /* timestamp */); + ChunkVersion version(majorVersion, minorVersion, epoch, Timestamp()); ASSERT_EQ(majorVersion, version.majorVersion()); ASSERT_EQ(minorVersion, version.minorVersion()); ASSERT_EQ(epoch, version.epoch()); diff --git a/src/mongo/s/comparable_chunk_version_test.cpp b/src/mongo/s/comparable_chunk_version_test.cpp index 6a306c019a9..8d38f8656e4 100644 --- a/src/mongo/s/comparable_chunk_version_test.cpp +++ b/src/mongo/s/comparable_chunk_version_test.cpp @@ -36,42 +36,31 @@ namespace mongo { namespace { TEST(ComparableChunkVersionTest, VersionsEqual) { - auto versionsEqual = [](const ChunkVersion& v1, const ChunkVersion& v2) { - const auto version1 = ComparableChunkVersion::makeComparableChunkVersion(v1); - const auto version2 = ComparableChunkVersion::makeComparableChunkVersion(v2); - ASSERT(version1 == version2); - }; - const auto epoch = OID::gen(); - versionsEqual(ChunkVersion(1, 0, epoch, boost::none /* timestamp */), - ChunkVersion(1, 0, epoch, boost::none /* timestamp */)); - versionsEqual(ChunkVersion(1, 0, epoch, Timestamp(1)), ChunkVersion(1, 0, epoch, Timestamp(1))); + const Timestamp timestamp; + const ChunkVersion v1(1, 0, epoch, timestamp); + const ChunkVersion v2(1, 0, epoch, timestamp); + const auto version1 = ComparableChunkVersion::makeComparableChunkVersion(v1); + const auto version2 = ComparableChunkVersion::makeComparableChunkVersion(v2); + ASSERT(version1 == version2); } TEST(ComparableChunkVersionTest, VersionsEqualAfterCopy) { - auto equalAfterCopy = [](const ChunkVersion& v) { - const auto version1 = ComparableChunkVersion::makeComparableChunkVersion(v); - const auto version2 = version1; - ASSERT(version1 == version2); - }; - - equalAfterCopy(ChunkVersion(1, 0, OID::gen(), boost::none /* timestamp */)); - equalAfterCopy(ChunkVersion(1, 0, OID::gen(), Timestamp(1))); + const ChunkVersion chunkVersion(1, 0, OID::gen(), Timestamp()); + const auto version1 = ComparableChunkVersion::makeComparableChunkVersion(chunkVersion); + const auto version2 = version1; + ASSERT(version1 == version2); } -TEST(ComparableChunkVersionTest, CompareDifferentVersionsEpochsOrTimestamps) { - auto compareDifferentVersions = [](const ChunkVersion& v1, const ChunkVersion& v2) { - const auto version1 = ComparableChunkVersion::makeComparableChunkVersion(v1); - const auto version2 = ComparableChunkVersion::makeComparableChunkVersion(v2); - ASSERT(version2 != version1); - ASSERT(version2 > version1); - ASSERT_FALSE(version2 < version1); - }; - compareDifferentVersions(ChunkVersion(2, 0, OID::gen(), boost::none /* timestamp */), - ChunkVersion(1, 0, OID::gen(), boost::none /* timestamp */)); - compareDifferentVersions(ChunkVersion(2, 0, OID::gen(), Timestamp(1)), - ChunkVersion(1, 0, OID::gen(), Timestamp(2))); +TEST(ComparableChunkVersionTest, CompareDifferentTimestamps) { + const ChunkVersion v1(2, 0, OID::gen(), Timestamp(1)); + const ChunkVersion v2(1, 0, OID::gen(), Timestamp(2)); + const auto version1 = ComparableChunkVersion::makeComparableChunkVersion(v1); + const auto version2 = ComparableChunkVersion::makeComparableChunkVersion(v2); + ASSERT(version2 != version1); + ASSERT(version2 > version1); + ASSERT_FALSE(version2 < version1); } TEST(ComparableChunkVersionTest, CompareDifferentVersionsTimestampsIgnoreSequenceNumber) { @@ -84,102 +73,38 @@ TEST(ComparableChunkVersionTest, CompareDifferentVersionsTimestampsIgnoreSequenc ASSERT_FALSE(version1 < version2); } -TEST(ComparableChunkVersionTest, VersionGreaterSameEpochsOrTimestamps) { - auto greaterSameEpochsOrTimestamps = - [](const ChunkVersion& v1, const ChunkVersion& v2, const ChunkVersion& v3) { - const auto version1 = ComparableChunkVersion::makeComparableChunkVersion(v1); - const auto version2 = ComparableChunkVersion::makeComparableChunkVersion(v2); - const auto version3 = ComparableChunkVersion::makeComparableChunkVersion(v3); - ASSERT(version2 != version1); - ASSERT(version2 > version1); - ASSERT_FALSE(version2 < version1); - ASSERT(version3 != version2); - ASSERT(version3 > version2); - ASSERT_FALSE(version3 < version2); - }; - +TEST(ComparableChunkVersionTest, VersionGreaterSameTimestamps) { const auto epoch = OID::gen(); - greaterSameEpochsOrTimestamps(ChunkVersion(1, 0, epoch, boost::none /* timestamp */), - ChunkVersion(1, 1, epoch, boost::none /* timestamp */), - ChunkVersion(2, 0, epoch, boost::none /* timestamp */)); - const Timestamp timestamp(1); - greaterSameEpochsOrTimestamps(ChunkVersion(1, 0, epoch, timestamp), - ChunkVersion(1, 1, epoch, timestamp), - ChunkVersion(2, 0, epoch, timestamp)); -} - -TEST(ComparableChunkVersionTest, VersionLessSameEpochsOrTimestamps) { - auto lessSameEpochsOrTimestamps = - [](const ChunkVersion& v1, const ChunkVersion& v2, const ChunkVersion& v3) { - const auto version1 = ComparableChunkVersion::makeComparableChunkVersion(v1); - const auto version2 = ComparableChunkVersion::makeComparableChunkVersion(v2); - const auto version3 = ComparableChunkVersion::makeComparableChunkVersion(v3); - ASSERT(version1 != version2); - ASSERT(version1 < version2); - ASSERT_FALSE(version1 > version2); - ASSERT(version3 != version2); - ASSERT(version2 < version3); - ASSERT_FALSE(version2 > version3); - }; - + const Timestamp timestamp; + const ChunkVersion v1(1, 0, epoch, timestamp); + const ChunkVersion v2(1, 2, epoch, timestamp); + const ChunkVersion v3(2, 0, epoch, timestamp); + const auto version1 = ComparableChunkVersion::makeComparableChunkVersion(v1); + const auto version2 = ComparableChunkVersion::makeComparableChunkVersion(v2); + const auto version3 = ComparableChunkVersion::makeComparableChunkVersion(v3); + ASSERT(version2 != version1); + ASSERT(version2 > version1); + ASSERT_FALSE(version2 < version1); + ASSERT(version3 != version2); + ASSERT(version3 > version2); + ASSERT_FALSE(version3 < version2); +} + +TEST(ComparableChunkVersionTest, VersionLessSameTimestamps) { const auto epoch = OID::gen(); - lessSameEpochsOrTimestamps(ChunkVersion(1, 0, epoch, boost::none /* timestamp */), - ChunkVersion(1, 1, epoch, boost::none /* timestamp */), - ChunkVersion(2, 0, epoch, boost::none /* timestamp */)); - const Timestamp timestamp(1); - lessSameEpochsOrTimestamps(ChunkVersion(1, 0, epoch, timestamp), - ChunkVersion(1, 1, epoch, timestamp), - ChunkVersion(2, 0, epoch, timestamp)); -} - -TEST(ComparableChunkVersionTest, compareEpochBasedVersionAgainstEpochAndTimestampBasedVersion) { - { - auto equalVersions = [](const ChunkVersion& v1, const ChunkVersion& v2) { - const auto version1 = ComparableChunkVersion::makeComparableChunkVersion(v1); - const auto version2 = ComparableChunkVersion::makeComparableChunkVersion(v2); - ASSERT(version1 == version2); - ASSERT_FALSE(version1 < version2); - ASSERT_FALSE(version1 > version2); - }; - - const auto epoch = OID::gen(); - equalVersions(ChunkVersion(1, 0, epoch, boost::none /* timestamp */), - ChunkVersion(1, 0, epoch, Timestamp(1))); - equalVersions(ChunkVersion(1, 0, epoch, Timestamp(1)), - ChunkVersion(1, 0, epoch, boost::none /* timestamp */)); - } - { - auto diffVersionsMoreRecentByMajor = [](const ChunkVersion& v1, const ChunkVersion& v2) { - const auto version1 = ComparableChunkVersion::makeComparableChunkVersion(v1); - const auto version2 = ComparableChunkVersion::makeComparableChunkVersion(v2); - ASSERT(version1 != version2); - ASSERT(version1 > version2); - }; - - const auto epoch = OID::gen(); - diffVersionsMoreRecentByMajor(ChunkVersion(2, 0, epoch, boost::none /* timestamp */), - ChunkVersion(1, 0, epoch, Timestamp(1))); - diffVersionsMoreRecentByMajor(ChunkVersion(2, 0, epoch, Timestamp(1)), - ChunkVersion(1, 0, epoch, boost::none /* timestamp */)); - } - { - auto diffVersionsMoreRecentByDisambigSeqNum = [](const ChunkVersion& v1, - const ChunkVersion& v2) { - const auto version1 = ComparableChunkVersion::makeComparableChunkVersion(v1); - const auto version2 = ComparableChunkVersion::makeComparableChunkVersion(v2); - ASSERT(version1 != version2); - ASSERT(version1 < version2); - }; - - const auto epoch1 = OID::gen(); - const auto epoch2 = OID::gen(); - diffVersionsMoreRecentByDisambigSeqNum( - ChunkVersion(2, 0, epoch1, boost::none /* timestamp */), - ChunkVersion(1, 0, epoch2, Timestamp(1))); - diffVersionsMoreRecentByDisambigSeqNum( - ChunkVersion(1, 0, epoch1, Timestamp(1)), - ChunkVersion(2, 0, epoch2, boost::none /* timestamp */)); - } + const Timestamp timestamp; + const ChunkVersion v1(1, 0, epoch, timestamp); + const ChunkVersion v2(1, 2, epoch, timestamp); + const ChunkVersion v3(2, 0, epoch, timestamp); + const auto version1 = ComparableChunkVersion::makeComparableChunkVersion(v1); + const auto version2 = ComparableChunkVersion::makeComparableChunkVersion(v2); + const auto version3 = ComparableChunkVersion::makeComparableChunkVersion(v3); + ASSERT(version1 != version2); + ASSERT(version1 < version2); + ASSERT_FALSE(version1 > version2); + ASSERT(version3 != version2); + ASSERT(version2 < version3); + ASSERT_FALSE(version2 > version3); } TEST(ComparableChunkVersionTest, DefaultConstructedVersionsAreEqual) { @@ -190,31 +115,21 @@ TEST(ComparableChunkVersionTest, DefaultConstructedVersionsAreEqual) { } TEST(ComparableChunkVersionTest, DefaultConstructedVersionIsAlwaysLessThanWithChunksVersion) { - auto defaultConstructedIsLessThanWithChunks = [](const ChunkVersion& v) { - const ComparableChunkVersion defaultVersion{}; - const auto withChunksVersion = ComparableChunkVersion::makeComparableChunkVersion(v); - ASSERT(defaultVersion != withChunksVersion); - ASSERT(defaultVersion < withChunksVersion); - ASSERT_FALSE(defaultVersion > withChunksVersion); - }; - - defaultConstructedIsLessThanWithChunks( - ChunkVersion(1, 0, OID::gen(), boost::none /* timestamp */)); - defaultConstructedIsLessThanWithChunks(ChunkVersion(1, 0, OID::gen(), Timestamp(1))); + const ChunkVersion chunkVersion(1, 0, OID::gen(), Timestamp()); + const ComparableChunkVersion defaultVersion{}; + const auto withChunksVersion = ComparableChunkVersion::makeComparableChunkVersion(chunkVersion); + ASSERT(defaultVersion != withChunksVersion); + ASSERT(defaultVersion < withChunksVersion); + ASSERT_FALSE(defaultVersion > withChunksVersion); } TEST(ComparableChunkVersionTest, DefaultConstructedVersionIsAlwaysLessThanNoChunksVersion) { - auto defaultConstructedIsLessThanNoChunks = [](const ChunkVersion& v) { - const ComparableChunkVersion defaultVersion{}; - const auto noChunksVersion = ComparableChunkVersion::makeComparableChunkVersion(v); - ASSERT(defaultVersion != noChunksVersion); - ASSERT(defaultVersion < noChunksVersion); - ASSERT_FALSE(defaultVersion > noChunksVersion); - }; - - defaultConstructedIsLessThanNoChunks( - ChunkVersion(0, 0, OID::gen(), boost::none /* timestamp */)); - defaultConstructedIsLessThanNoChunks(ChunkVersion(0, 0, OID::gen(), Timestamp(1))); + const ChunkVersion chunkVersion(0, 0, OID::gen(), Timestamp()); + const ComparableChunkVersion defaultVersion{}; + const auto noChunksVersion = ComparableChunkVersion::makeComparableChunkVersion(chunkVersion); + ASSERT(defaultVersion != noChunksVersion); + ASSERT(defaultVersion < noChunksVersion); + ASSERT_FALSE(defaultVersion > noChunksVersion); } TEST(ComparableChunkVersionTest, DefaultConstructedVersionIsAlwaysLessThanUnsharded) { @@ -227,108 +142,78 @@ TEST(ComparableChunkVersionTest, DefaultConstructedVersionIsAlwaysLessThanUnshar } TEST(ComparableChunkVersionTest, TwoNoChunksVersionsAreTheSame) { - auto twoNoChunkVersionsAreTheSame = [](const ChunkVersion& v1, const ChunkVersion& v2) { - const auto noChunksVersion1 = ComparableChunkVersion::makeComparableChunkVersion(v1); - const auto noChunksVersion2 = ComparableChunkVersion::makeComparableChunkVersion(v2); - ASSERT(noChunksVersion1 == noChunksVersion2); - ASSERT_FALSE(noChunksVersion1 < noChunksVersion2); - ASSERT_FALSE(noChunksVersion1 > noChunksVersion2); - }; - const auto oid = OID::gen(); - twoNoChunkVersionsAreTheSame(ChunkVersion(0, 0, oid, boost::none /* timestamp */), - ChunkVersion(0, 0, oid, boost::none /* timestamp */)); - twoNoChunkVersionsAreTheSame(ChunkVersion(0, 0, oid, Timestamp(1)), - ChunkVersion(0, 0, oid, Timestamp(1))); + const ChunkVersion v1(0, 0, oid, Timestamp()); + const ChunkVersion v2(0, 0, oid, Timestamp()); + const auto noChunksVersion1 = ComparableChunkVersion::makeComparableChunkVersion(v1); + const auto noChunksVersion2 = ComparableChunkVersion::makeComparableChunkVersion(v2); + ASSERT(noChunksVersion1 == noChunksVersion2); + ASSERT_FALSE(noChunksVersion1 < noChunksVersion2); + ASSERT_FALSE(noChunksVersion1 > noChunksVersion2); } TEST(ComparableChunkVersionTest, NoChunksComparedBySequenceNum) { - auto noChunksComparedBySequenceNum = - [](const ChunkVersion& v1, const ChunkVersion& v2, const ChunkVersion& v3) { - const auto version1 = ComparableChunkVersion::makeComparableChunkVersion(v1); - const auto noChunksVersion2 = ComparableChunkVersion::makeComparableChunkVersion(v2); - const auto version3 = ComparableChunkVersion::makeComparableChunkVersion(v3); - ASSERT(version1 != noChunksVersion2); - ASSERT(noChunksVersion2 > version1); - ASSERT(version3 != noChunksVersion2); - ASSERT(version3 > noChunksVersion2); - }; - const auto oid = OID::gen(); - noChunksComparedBySequenceNum(ChunkVersion(1, 0, oid, boost::none /* timestamp */), - ChunkVersion(0, 0, oid, boost::none /* timestamp */), - ChunkVersion(2, 0, oid, boost::none /* timestamp */)); - noChunksComparedBySequenceNum(ChunkVersion(1, 0, oid, Timestamp(1)), - ChunkVersion(0, 0, oid, Timestamp(1)), - ChunkVersion(2, 0, oid, Timestamp(1))); + const Timestamp timestamp(1); + const ChunkVersion v1(1, 0, oid, timestamp); + const ChunkVersion v2(0, 0, oid, timestamp); + const ChunkVersion v3(2, 0, oid, timestamp); + const auto version1 = ComparableChunkVersion::makeComparableChunkVersion(v1); + const auto noChunksVersion2 = ComparableChunkVersion::makeComparableChunkVersion(v2); + const auto version3 = ComparableChunkVersion::makeComparableChunkVersion(v3); + ASSERT(version1 != noChunksVersion2); + ASSERT(noChunksVersion2 > version1); + ASSERT(version3 != noChunksVersion2); + ASSERT(version3 > noChunksVersion2); } TEST(ComparableChunkVersionTest, NoChunksGreaterThanUnshardedBySequenceNum) { - auto noChunksGreaterThanUnshardedBySequenceNum = [](const ChunkVersion& v) { - const auto unsharded = - ComparableChunkVersion::makeComparableChunkVersion(ChunkVersion::UNSHARDED()); - const auto noChunkSV = ComparableChunkVersion::makeComparableChunkVersion(v); - ASSERT(noChunkSV != unsharded); - ASSERT(noChunkSV > unsharded); - }; - - noChunksGreaterThanUnshardedBySequenceNum( - ChunkVersion(0, 0, OID::gen(), boost::none /* timestamp */)); - noChunksGreaterThanUnshardedBySequenceNum(ChunkVersion(0, 0, OID::gen(), Timestamp(1))); + const ChunkVersion chunkVersion(0, 0, OID::gen(), Timestamp(1)); + const auto unsharded = + ComparableChunkVersion::makeComparableChunkVersion(ChunkVersion::UNSHARDED()); + const auto noChunkSV = ComparableChunkVersion::makeComparableChunkVersion(chunkVersion); + ASSERT(noChunkSV != unsharded); + ASSERT(noChunkSV > unsharded); } TEST(ComparableChunkVersionTest, UnshardedGreaterThanNoChunksBySequenceNum) { - auto unshardedGreaterThanNoChunksBySequenceNum = [](const ChunkVersion& v) { - const auto noChunkSV = ComparableChunkVersion::makeComparableChunkVersion(v); - const auto unsharded = - ComparableChunkVersion::makeComparableChunkVersion(ChunkVersion::UNSHARDED()); - ASSERT(noChunkSV != unsharded); - ASSERT(unsharded > noChunkSV); - }; - - unshardedGreaterThanNoChunksBySequenceNum( - ChunkVersion(0, 0, OID::gen(), boost::none /* timestamp */)); - unshardedGreaterThanNoChunksBySequenceNum(ChunkVersion(0, 0, OID::gen(), Timestamp(1))); + const ChunkVersion chunkVersion(0, 0, OID::gen(), Timestamp(1)); + const auto noChunkSV = ComparableChunkVersion::makeComparableChunkVersion(chunkVersion); + const auto unsharded = + ComparableChunkVersion::makeComparableChunkVersion(ChunkVersion::UNSHARDED()); + ASSERT(noChunkSV != unsharded); + ASSERT(unsharded > noChunkSV); } TEST(ComparableChunkVersionTest, NoChunksGreaterThanDefault) { - auto noChunksGreaterThanDefault = [](const ChunkVersion& v) { - const auto noChunkSV = ComparableChunkVersion::makeComparableChunkVersion(v); - const ComparableChunkVersion defaultVersion{}; - ASSERT(noChunkSV != defaultVersion); - ASSERT(noChunkSV > defaultVersion); - }; - - noChunksGreaterThanDefault(ChunkVersion(0, 0, OID::gen(), boost::none /* timestamp */)); - noChunksGreaterThanDefault(ChunkVersion(0, 0, OID::gen(), Timestamp(1))); + const ChunkVersion chunkVersion(0, 0, OID::gen(), Timestamp(1)); + const auto noChunkSV = ComparableChunkVersion::makeComparableChunkVersion(chunkVersion); + const ComparableChunkVersion defaultVersion{}; + ASSERT(noChunkSV != defaultVersion); + ASSERT(noChunkSV > defaultVersion); } TEST(ComparableChunkVersionTest, CompareForcedRefreshVersionVersusValidChunkVersion) { - auto compareForcedRefreshVersionVersusValidChunkVersion = [](const ChunkVersion& v) { - const ComparableChunkVersion defaultVersionBeforeForce; - const auto versionBeforeForce = ComparableChunkVersion::makeComparableChunkVersion(v); - const auto forcedRefreshVersion = - ComparableChunkVersion::makeComparableChunkVersionForForcedRefresh(); - const auto versionAfterForce = ComparableChunkVersion::makeComparableChunkVersion(v); - const ComparableChunkVersion defaultVersionAfterForce; - - ASSERT(defaultVersionBeforeForce != forcedRefreshVersion); - ASSERT(defaultVersionBeforeForce < forcedRefreshVersion); + const ChunkVersion chunkVersion(100, 0, OID::gen(), Timestamp(1)); + const ComparableChunkVersion defaultVersionBeforeForce; + const auto versionBeforeForce = + ComparableChunkVersion::makeComparableChunkVersion(chunkVersion); + const auto forcedRefreshVersion = + ComparableChunkVersion::makeComparableChunkVersionForForcedRefresh(); + const auto versionAfterForce = ComparableChunkVersion::makeComparableChunkVersion(chunkVersion); + const ComparableChunkVersion defaultVersionAfterForce; - ASSERT(versionBeforeForce != forcedRefreshVersion); - ASSERT(versionBeforeForce < forcedRefreshVersion); + ASSERT(defaultVersionBeforeForce != forcedRefreshVersion); + ASSERT(defaultVersionBeforeForce < forcedRefreshVersion); - ASSERT(versionAfterForce != forcedRefreshVersion); - ASSERT(versionAfterForce > forcedRefreshVersion); + ASSERT(versionBeforeForce != forcedRefreshVersion); + ASSERT(versionBeforeForce < forcedRefreshVersion); - ASSERT(defaultVersionAfterForce != forcedRefreshVersion); - ASSERT(defaultVersionAfterForce < forcedRefreshVersion); - }; + ASSERT(versionAfterForce != forcedRefreshVersion); + ASSERT(versionAfterForce > forcedRefreshVersion); - compareForcedRefreshVersionVersusValidChunkVersion( - ChunkVersion(100, 0, OID::gen(), boost::none /* timestamp */)); - compareForcedRefreshVersionVersusValidChunkVersion( - ChunkVersion(100, 0, OID::gen(), Timestamp(1))); + ASSERT(defaultVersionAfterForce != forcedRefreshVersion); + ASSERT(defaultVersionAfterForce < forcedRefreshVersion); } TEST(ComparableChunkVersionTest, CompareTwoForcedRefreshVersions) { diff --git a/src/mongo/s/query/cluster_exchange_test.cpp b/src/mongo/s/query/cluster_exchange_test.cpp index f3207ad23d0..48548d937f9 100644 --- a/src/mongo/s/query/cluster_exchange_test.cpp +++ b/src/mongo/s/query/cluster_exchange_test.cpp @@ -448,12 +448,15 @@ TEST_F(ClusterExchangeTest, WordCountUseCaseExample) { TEST_F(ClusterExchangeTest, WordCountUseCaseExampleShardedByWord) { setupNShards(2); const OID epoch = OID::gen(); + const Timestamp timestamp = Timestamp(1); ShardKeyPattern shardKey(BSON("word" << 1)); loadRoutingTable(kTestTargetNss, epoch, + timestamp, shardKey, makeChunks(UUID::gen(), epoch, + timestamp, {{ChunkRange{BSON("word" << MINKEY), BSON("word" << "hello")}, @@ -523,13 +526,14 @@ TEST_F(ClusterExchangeTest, WordCountUseCaseExampleShardedByWord) { TEST_F(ClusterExchangeTest, CompoundShardKeyThreeShards) { const OID epoch = OID::gen(); const UUID uuid = UUID::gen(); + const Timestamp timestamp; ShardKeyPattern shardKey(BSON("x" << 1 << "y" << 1)); setupNShards(3); const std::vector xBoundaries = {"a", "g", "m", "r", "u"}; auto chunks = [&]() { std::vector chunks; - ChunkVersion version(1, 0, epoch, boost::none /* timestamp */); + ChunkVersion version(1, 0, epoch, timestamp); chunks.emplace_back(uuid, ChunkRange{BSON("x" << MINKEY << "y" << MINKEY), BSON("x" << xBoundaries[0] << "y" << MINKEY)}, @@ -553,7 +557,7 @@ TEST_F(ClusterExchangeTest, CompoundShardKeyThreeShards) { return chunks; }(); - loadRoutingTable(kTestTargetNss, epoch, shardKey, chunks); + loadRoutingTable(kTestTargetNss, epoch, timestamp, shardKey, chunks); auto mergePipe = Pipeline::create({parseStage("{$group: {" " _id: '$x'," diff --git a/src/mongo/s/query/sharded_agg_test_fixture.h b/src/mongo/s/query/sharded_agg_test_fixture.h index ab4f565a53e..d5c02d84b3c 100644 --- a/src/mongo/s/query/sharded_agg_test_fixture.h +++ b/src/mongo/s/query/sharded_agg_test_fixture.h @@ -78,8 +78,9 @@ public: std::vector makeChunks(const UUID& uuid, const OID epoch, + const Timestamp timestamp, std::vector> chunkInfos) { - ChunkVersion version(1, 0, epoch, boost::none /* timestamp */); + ChunkVersion version(1, 0, epoch, timestamp); std::vector chunks; for (auto&& pair : chunkInfos) { chunks.emplace_back(uuid, pair.first, version, pair.second); @@ -91,13 +92,15 @@ public: void loadRoutingTable(NamespaceString nss, const OID epoch, + const Timestamp timestamp, const ShardKeyPattern& shardKey, const std::vector& chunkDistribution) { auto future = scheduleRoutingInfoUnforcedRefresh(nss); // Mock the expected config server queries. expectGetDatabase(nss); - expectCollectionAndChunksAggregation(nss, epoch, UUID::gen(), shardKey, chunkDistribution); + expectCollectionAndChunksAggregation( + nss, epoch, timestamp, UUID::gen(), shardKey, chunkDistribution); const auto cm = future.default_timed_get(); ASSERT(cm->isSharded()); diff --git a/src/mongo/s/request_types/balance_chunk_request_test.cpp b/src/mongo/s/request_types/balance_chunk_request_test.cpp index 1021d8feef0..236b7fbc1d0 100644 --- a/src/mongo/s/request_types/balance_chunk_request_test.cpp +++ b/src/mongo/s/request_types/balance_chunk_request_test.cpp @@ -43,7 +43,7 @@ namespace { using unittest::assertGet; TEST(BalanceChunkRequest, ParseFromConfigCommandNoSecondaryThrottle) { - const ChunkVersion version(1, 0, OID::gen(), boost::none /* timestamp */); + const ChunkVersion version(1, 0, OID::gen(), Timestamp()); auto request = assertGet(BalanceChunkRequest::parseFromConfigCommand( BSON("_configsvrMoveChunk" << 1 << "ns" @@ -51,7 +51,7 @@ TEST(BalanceChunkRequest, ParseFromConfigCommandNoSecondaryThrottle) { << "min" << BSON("a" << -100LL) << "max" << BSON("a" << 100LL) << "shard" << "TestShard0000" << "lastmod" << Date_t::fromMillisSinceEpoch(version.toLong()) << "lastmodEpoch" - << version.epoch()))); + << version.epoch() << "lastmodTimestamp" << version.getTimestamp()))); const auto& chunk = request.getChunk(); ASSERT_EQ("TestDB.TestColl", request.getNss()->ns()); ASSERT_BSONOBJ_EQ(BSON("a" << -100LL), chunk.getMin()); @@ -66,13 +66,14 @@ TEST(BalanceChunkRequest, ParseFromConfigCommandNoSecondaryThrottle) { TEST(BalanceChunkRequest, ParseFromConfigCommandWithUUID) { const auto uuid = UUID::gen(); - const ChunkVersion version(1, 0, OID::gen(), boost::none /* timestamp */); + const ChunkVersion version(1, 0, OID::gen(), Timestamp()); auto request = assertGet(BalanceChunkRequest::parseFromConfigCommand( BSON("_configsvrMoveChunk" << 1 << "uuid" << uuid << "min" << BSON("a" << -100LL) << "max" << BSON("a" << 100LL) << "shard" << "TestShard0000" << "lastmod" << Date_t::fromMillisSinceEpoch(version.toLong()) - << "lastmodEpoch" << version.epoch()))); + << "lastmodEpoch" << version.epoch() << "lastmodTimestamp" + << version.getTimestamp()))); const auto& chunk = request.getChunk(); ASSERT_EQ(uuid, chunk.getCollectionUUID()); ASSERT_BSONOBJ_EQ(BSON("a" << -100LL), chunk.getMin()); @@ -86,7 +87,7 @@ TEST(BalanceChunkRequest, ParseFromConfigCommandWithUUID) { } TEST(BalanceChunkRequest, ParseFromConfigCommandWithSecondaryThrottle) { - const ChunkVersion version(1, 0, OID::gen(), boost::none /* timestamp */); + const ChunkVersion version(1, 0, OID::gen(), Timestamp()); auto request = assertGet(BalanceChunkRequest::parseFromConfigCommand( BSON("_configsvrMoveChunk" << 1 << "ns" @@ -94,7 +95,8 @@ TEST(BalanceChunkRequest, ParseFromConfigCommandWithSecondaryThrottle) { << "min" << BSON("a" << -100LL) << "max" << BSON("a" << 100LL) << "shard" << "TestShard0000" << "lastmod" << Date_t::fromMillisSinceEpoch(version.toLong()) << "lastmodEpoch" - << version.epoch() << "secondaryThrottle" + << version.epoch() << "lastmodTimestamp" << version.getTimestamp() + << "secondaryThrottle" << BSON("_secondaryThrottle" << true << "writeConcern" << BSON("w" << 2))))); const auto& chunk = request.getChunk(); ASSERT_EQ("TestDB.TestColl", request.getNss()->ns()); diff --git a/src/mongo/s/request_types/balance_chunk_request_type.cpp b/src/mongo/s/request_types/balance_chunk_request_type.cpp index 04643f1601d..36ef5b08dac 100644 --- a/src/mongo/s/request_types/balance_chunk_request_type.cpp +++ b/src/mongo/s/request_types/balance_chunk_request_type.cpp @@ -180,6 +180,7 @@ BSONObj BalanceChunkRequest::serializeToMoveCommandForConfig( cmdBuilder.appendElements(chunk.toConfigBSON()); // ChunkType::toConfigBSON() no longer adds the epoch cmdBuilder.append(ChunkType::lastmod() + "Epoch", chunk.getVersion().epoch()); + cmdBuilder.append(ChunkType::lastmod() + "Timestamp", chunk.getVersion().getTimestamp()); cmdBuilder.append(kToShardId, newShardId.toString()); cmdBuilder.append(kMaxChunkSizeBytes, static_cast(maxChunkSizeBytes)); { @@ -203,6 +204,7 @@ BSONObj BalanceChunkRequest::serializeToRebalanceCommandForConfig(const ChunkTyp cmdBuilder.appendElements(chunk.toConfigBSON()); // ChunkType::toConfigBSON() no longer returns the epoch cmdBuilder.append(ChunkType::lastmod() + "Epoch", chunk.getVersion().epoch()); + cmdBuilder.append(ChunkType::lastmod() + "Timestamp", chunk.getVersion().getTimestamp()); cmdBuilder.append(WriteConcernOptions::kWriteConcernField, kMajorityWriteConcernNoTimeout.toBSON()); diff --git a/src/mongo/s/request_types/commit_chunk_migration_request_test.cpp b/src/mongo/s/request_types/commit_chunk_migration_request_test.cpp index fe8d0de4970..52acc9db8ec 100644 --- a/src/mongo/s/request_types/commit_chunk_migration_request_test.cpp +++ b/src/mongo/s/request_types/commit_chunk_migration_request_test.cpp @@ -59,9 +59,9 @@ TEST(CommitChunkMigrationRequest, WithoutControlChunk) { migratedChunk.setCollectionUUID(UUID::gen()); migratedChunk.setMin(kKey0); migratedChunk.setMax(kKey1); - migratedChunk.setVersion({12, 7, OID::gen(), boost::none /* timestamp */}); + migratedChunk.setVersion({12, 7, OID::gen(), Timestamp()}); - ChunkVersion fromShardCollectionVersion(1, 2, OID::gen(), boost::none /* timestamp */); + ChunkVersion fromShardCollectionVersion(1, 2, OID::gen(), Timestamp()); Timestamp validAfter{1}; diff --git a/src/mongo/s/request_types/commit_chunk_migration_request_type.cpp b/src/mongo/s/request_types/commit_chunk_migration_request_type.cpp index 2d1da8be497..5a95441405d 100644 --- a/src/mongo/s/request_types/commit_chunk_migration_request_type.cpp +++ b/src/mongo/s/request_types/commit_chunk_migration_request_type.cpp @@ -169,6 +169,8 @@ void CommitChunkMigrationRequest::appendAsCommand(BSONObjBuilder* builder, migrateChunk.appendElements(migratedChunk.toConfigBSON()); // ChunkType::toConfigBSON() no longer adds the epoch migrateChunk.append(ChunkType::lastmod() + "Epoch", migratedChunk.getVersion().epoch()); + migrateChunk.append(ChunkType::lastmod() + "Timestamp", + migratedChunk.getVersion().getTimestamp()); } fromShardCollectionVersion.appendWithField(builder, kFromShardCollectionVersion); diff --git a/src/mongo/s/request_types/move_chunk_request_test.cpp b/src/mongo/s/request_types/move_chunk_request_test.cpp index 73a1b95490c..b1036aa6a78 100644 --- a/src/mongo/s/request_types/move_chunk_request_test.cpp +++ b/src/mongo/s/request_types/move_chunk_request_test.cpp @@ -49,7 +49,7 @@ const int kMaxChunkSizeBytes = 1024; const bool kWaitForDelete = true; TEST(MoveChunkRequest, Roundtrip) { - const ChunkVersion chunkVersion(3, 1, OID::gen(), boost::none /* timestamp */); + const ChunkVersion chunkVersion(3, 1, OID::gen(), Timestamp()); BSONObjBuilder builder; MoveChunkRequest::appendAsCommand( @@ -81,7 +81,7 @@ TEST(MoveChunkRequest, Roundtrip) { } TEST(MoveChunkRequest, EqualityOperatorSameValue) { - const ChunkVersion chunkVersion(3, 1, OID::gen(), boost::none /* timestamp */); + const ChunkVersion chunkVersion(3, 1, OID::gen(), Timestamp()); BSONObjBuilder builder; MoveChunkRequest::appendAsCommand( @@ -106,7 +106,7 @@ TEST(MoveChunkRequest, EqualityOperatorSameValue) { } TEST(MoveChunkRequest, EqualityOperatorDifferentValues) { - const ChunkVersion chunkVersion(3, 1, OID::gen(), boost::none /* timestamp */); + const ChunkVersion chunkVersion(3, 1, OID::gen(), Timestamp()); BSONObjBuilder builder1; MoveChunkRequest::appendAsCommand( diff --git a/src/mongo/s/request_types/set_shard_version_request_test.cpp b/src/mongo/s/request_types/set_shard_version_request_test.cpp index 7c7bd819da6..60bcd0b52f2 100644 --- a/src/mongo/s/request_types/set_shard_version_request_test.cpp +++ b/src/mongo/s/request_types/set_shard_version_request_test.cpp @@ -41,13 +41,14 @@ using unittest::assertGet; namespace { TEST(SetShardVersionRequest, ParseFull) { - const ChunkVersion chunkVersion(1, 2, OID::gen(), boost::none /* timestamp */); + const ChunkVersion chunkVersion(1, 2, OID::gen(), Timestamp()); - SetShardVersionRequest request = assertGet( - SetShardVersionRequest::parseFromBSON(BSON("setShardVersion" - << "db.coll" - << "version" << Timestamp(chunkVersion.toLong()) - << "versionEpoch" << chunkVersion.epoch()))); + SetShardVersionRequest request = + assertGet(SetShardVersionRequest::parseFromBSON( + BSON("setShardVersion" + << "db.coll" + << "version" << Timestamp(chunkVersion.toLong()) << "versionEpoch" + << chunkVersion.epoch() << "versionTimestamp" << chunkVersion.getTimestamp()))); ASSERT(!request.shouldForceRefresh()); ASSERT(!request.isAuthoritative()); @@ -58,14 +59,15 @@ TEST(SetShardVersionRequest, ParseFull) { } TEST(SetShardVersionRequest, ParseFullWithAuthoritative) { - const ChunkVersion chunkVersion(1, 2, OID::gen(), boost::none /* timestamp */); + const ChunkVersion chunkVersion(1, 2, OID::gen(), Timestamp()); SetShardVersionRequest request = assertGet(SetShardVersionRequest::parseFromBSON( BSON("setShardVersion" << "db.coll" << "version" << Timestamp(chunkVersion.toLong()) << "versionEpoch" - << chunkVersion.epoch() << "authoritative" << true))); + << chunkVersion.epoch() << "versionTimestamp" << chunkVersion.getTimestamp() + << "authoritative" << true))); ASSERT(!request.shouldForceRefresh()); ASSERT(request.isAuthoritative()); @@ -76,7 +78,7 @@ TEST(SetShardVersionRequest, ParseFullWithAuthoritative) { } TEST(SetShardVersionRequest, ParseFullNoNS) { - const ChunkVersion chunkVersion(1, 2, OID::gen(), boost::none /* timestamp */); + const ChunkVersion chunkVersion(1, 2, OID::gen(), Timestamp()); auto ssvStatus = SetShardVersionRequest::parseFromBSON(BSON("setShardVersion" @@ -88,7 +90,7 @@ TEST(SetShardVersionRequest, ParseFullNoNS) { } TEST(SetShardVersionRequest, ParseFullNSContainsDBOnly) { - const ChunkVersion chunkVersion(1, 2, OID::gen(), boost::none /* timestamp */); + const ChunkVersion chunkVersion(1, 2, OID::gen(), Timestamp()); auto ssvStatus = SetShardVersionRequest::parseFromBSON(BSON("setShardVersion" @@ -100,7 +102,7 @@ TEST(SetShardVersionRequest, ParseFullNSContainsDBOnly) { } TEST(SetShardVersionRequest, ToSSVCommandFull) { - const ChunkVersion chunkVersion(1, 2, OID::gen(), boost::none /* timestamp */); + const ChunkVersion chunkVersion(1, 2, OID::gen(), Timestamp()); SetShardVersionRequest ssv(NamespaceString("db.coll"), chunkVersion, false); @@ -115,11 +117,12 @@ TEST(SetShardVersionRequest, ToSSVCommandFull) { << "forceRefresh" << false << "authoritative" << false << "noConnectionVersioning" << true << "version" << Timestamp(chunkVersion.toLong()) << "versionEpoch" - << chunkVersion.epoch())); + << chunkVersion.epoch() << "versionTimestamp" + << chunkVersion.getTimestamp())); } TEST(SetShardVersionRequest, ToSSVCommandFullAuthoritative) { - const ChunkVersion chunkVersion(1, 2, OID::gen(), boost::none /* timestamp */); + const ChunkVersion chunkVersion(1, 2, OID::gen(), Timestamp()); SetShardVersionRequest ssv(NamespaceString("db.coll"), chunkVersion, true); @@ -134,11 +137,12 @@ TEST(SetShardVersionRequest, ToSSVCommandFullAuthoritative) { << "forceRefresh" << false << "authoritative" << true << "noConnectionVersioning" << true << "version" << Timestamp(chunkVersion.toLong()) << "versionEpoch" - << chunkVersion.epoch())); + << chunkVersion.epoch() << "versionTimestamp" + << chunkVersion.getTimestamp())); } TEST(SetShardVersionRequest, ToSSVCommandFullForceRefresh) { - const ChunkVersion chunkVersion(1, 2, OID::gen(), boost::none /* timestamp */); + const ChunkVersion chunkVersion(1, 2, OID::gen(), Timestamp()); SetShardVersionRequest ssv(NamespaceString("db.coll"), chunkVersion, false, true); @@ -153,7 +157,8 @@ TEST(SetShardVersionRequest, ToSSVCommandFullForceRefresh) { << "forceRefresh" << true << "authoritative" << false << "noConnectionVersioning" << true << "version" << Timestamp(chunkVersion.toLong()) << "versionEpoch" - << chunkVersion.epoch())); + << chunkVersion.epoch() << "versionTimestamp" + << chunkVersion.getTimestamp())); } } // namespace diff --git a/src/mongo/s/routing_table_history_test.cpp b/src/mongo/s/routing_table_history_test.cpp index b8670f17d06..72de94f95f8 100644 --- a/src/mongo/s/routing_table_history_test.cpp +++ b/src/mongo/s/routing_table_history_test.cpp @@ -152,8 +152,8 @@ public: void setUp() override { const UUID uuid = UUID::gen(); const OID epoch = OID::gen(); - const Timestamp kRouting(100, 0); - ChunkVersion version{1, 0, epoch, boost::none /* timestamp */}; + const Timestamp timestamp(1); + ChunkVersion version{1, 0, epoch, timestamp}; auto initChunk = ChunkType{uuid, @@ -167,7 +167,7 @@ public: nullptr, false, epoch, - boost::none /* timestamp */, + timestamp, boost::none /* timeseriesFields */, boost::none, boost::none /* chunkSizeBytes */, @@ -330,7 +330,8 @@ TEST_F(RoutingTableHistoryTestThreeInitialChunks, TEST_F(RoutingTableHistoryTest, TestSplits) { const UUID uuid = UUID::gen(); const OID epoch = OID::gen(); - ChunkVersion version{1, 0, epoch, boost::none /* timestamp */}; + const Timestamp timestamp(1); + ChunkVersion version{1, 0, epoch, timestamp}; auto chunkAll = ChunkType{uuid, @@ -344,7 +345,7 @@ TEST_F(RoutingTableHistoryTest, TestSplits) { nullptr, false, epoch, - boost::none /* timestamp */, + timestamp, boost::none /* timeseriesFields */, boost::none, boost::none /* chunkSizeBytes */, @@ -354,44 +355,45 @@ TEST_F(RoutingTableHistoryTest, TestSplits) { std::vector chunks1 = { ChunkType{uuid, ChunkRange{getShardKeyPattern().globalMin(), BSON("a" << 0)}, - ChunkVersion{2, 1, epoch, boost::none /* timestamp */}, + ChunkVersion{2, 1, epoch, timestamp}, kThisShard}, ChunkType{uuid, ChunkRange{BSON("a" << 0), getShardKeyPattern().globalMax()}, - ChunkVersion{2, 2, epoch, boost::none /* timestamp */}, + ChunkVersion{2, 2, epoch, timestamp}, kThisShard}}; auto rt1 = rt.makeUpdated(boost::none, boost::none, true, chunks1); - auto v1 = ChunkVersion{2, 2, epoch, boost::none /* timestamp */}; + auto v1 = ChunkVersion{2, 2, epoch, timestamp}; ASSERT_EQ(v1, rt1.getVersion(kThisShard)); std::vector chunks2 = { ChunkType{uuid, ChunkRange{BSON("a" << 0), getShardKeyPattern().globalMax()}, - ChunkVersion{2, 2, epoch, boost::none /* timestamp */}, + ChunkVersion{2, 2, epoch, timestamp}, kThisShard}, ChunkType{uuid, ChunkRange{getShardKeyPattern().globalMin(), BSON("a" << -1)}, - ChunkVersion{3, 1, epoch, boost::none /* timestamp */}, + ChunkVersion{3, 1, epoch, timestamp}, kThisShard}, ChunkType{uuid, ChunkRange{BSON("a" << -1), BSON("a" << 0)}, - ChunkVersion{3, 2, epoch, boost::none /* timestamp */}, + ChunkVersion{3, 2, epoch, timestamp}, kThisShard}}; auto rt2 = rt1.makeUpdated(boost::none, boost::none, true, chunks2); - auto v2 = ChunkVersion{3, 2, epoch, boost::none /* timestamp */}; + auto v2 = ChunkVersion{3, 2, epoch, timestamp}; ASSERT_EQ(v2, rt2.getVersion(kThisShard)); } TEST_F(RoutingTableHistoryTest, TestReplaceEmptyChunk) { const UUID uuid = UUID::gen(); const OID epoch = OID::gen(); + const Timestamp timestamp(1); std::vector initialChunks = { ChunkType{uuid, ChunkRange{getShardKeyPattern().globalMin(), getShardKeyPattern().globalMax()}, - ChunkVersion{1, 0, epoch, boost::none /* timestamp */}, + ChunkVersion{1, 0, epoch, timestamp}, kThisShard}}; auto rt = RoutingTableHistory::makeNew(kNss, @@ -400,7 +402,7 @@ TEST_F(RoutingTableHistoryTest, TestReplaceEmptyChunk) { nullptr, false, epoch, - boost::none /* timestamp */, + timestamp, boost::none /* timeseriesFields */, boost::none, boost::none /* chunkSizeBytes */, @@ -411,15 +413,15 @@ TEST_F(RoutingTableHistoryTest, TestReplaceEmptyChunk) { std::vector changedChunks = { ChunkType{uuid, ChunkRange{getShardKeyPattern().globalMin(), BSON("a" << 0)}, - ChunkVersion{2, 1, epoch, boost::none /* timestamp */}, + ChunkVersion{2, 1, epoch, timestamp}, kThisShard}, ChunkType{uuid, ChunkRange{BSON("a" << 0), getShardKeyPattern().globalMax()}, - ChunkVersion{2, 2, epoch, boost::none /* timestamp */}, + ChunkVersion{2, 2, epoch, timestamp}, kThisShard}}; auto rt1 = rt.makeUpdated(boost::none, boost::none, true, changedChunks); - auto v1 = ChunkVersion{2, 2, epoch, boost::none /* timestamp */}; + auto v1 = ChunkVersion{2, 2, epoch, timestamp}; ASSERT_EQ(v1, rt1.getVersion(kThisShard)); ASSERT_EQ(rt1.numChunks(), 2); @@ -440,11 +442,12 @@ TEST_F(RoutingTableHistoryTest, TestReplaceEmptyChunk) { TEST_F(RoutingTableHistoryTest, TestUseLatestVersions) { const UUID uuid = UUID::gen(); const OID epoch = OID::gen(); + const Timestamp timestamp(1); std::vector initialChunks = { ChunkType{uuid, ChunkRange{getShardKeyPattern().globalMin(), getShardKeyPattern().globalMax()}, - ChunkVersion{1, 0, epoch, boost::none /* timestamp */}, + ChunkVersion{1, 0, epoch, timestamp}, kThisShard}}; auto rt = RoutingTableHistory::makeNew(kNss, @@ -453,7 +456,7 @@ TEST_F(RoutingTableHistoryTest, TestUseLatestVersions) { nullptr, false, epoch, - boost::none /* timestamp */, + timestamp, boost::none /* timeseriesFields */, boost::none, boost::none /* chunkSizeBytes */, @@ -464,19 +467,19 @@ TEST_F(RoutingTableHistoryTest, TestUseLatestVersions) { std::vector changedChunks = { ChunkType{uuid, ChunkRange{getShardKeyPattern().globalMin(), getShardKeyPattern().globalMax()}, - ChunkVersion{1, 0, epoch, boost::none /* timestamp */}, + ChunkVersion{1, 0, epoch, timestamp}, kThisShard}, ChunkType{uuid, ChunkRange{getShardKeyPattern().globalMin(), BSON("a" << 0)}, - ChunkVersion{2, 1, epoch, boost::none /* timestamp */}, + ChunkVersion{2, 1, epoch, timestamp}, kThisShard}, ChunkType{uuid, ChunkRange{BSON("a" << 0), getShardKeyPattern().globalMax()}, - ChunkVersion{2, 2, epoch, boost::none /* timestamp */}, + ChunkVersion{2, 2, epoch, timestamp}, kThisShard}}; auto rt1 = rt.makeUpdated(boost::none, boost::none, true, changedChunks); - auto v1 = ChunkVersion{2, 2, epoch, boost::none /* timestamp */}; + auto v1 = ChunkVersion{2, 2, epoch, timestamp}; ASSERT_EQ(v1, rt1.getVersion(kThisShard)); ASSERT_EQ(rt1.numChunks(), 2); } @@ -484,15 +487,16 @@ TEST_F(RoutingTableHistoryTest, TestUseLatestVersions) { TEST_F(RoutingTableHistoryTest, TestOutOfOrderVersion) { const UUID uuid = UUID::gen(); const OID epoch = OID::gen(); + const Timestamp timestamp(1); std::vector initialChunks = { ChunkType{uuid, ChunkRange{getShardKeyPattern().globalMin(), BSON("a" << 0)}, - ChunkVersion{2, 1, epoch, boost::none /* timestamp */}, + ChunkVersion{2, 1, epoch, timestamp}, kThisShard}, ChunkType{uuid, ChunkRange{BSON("a" << 0), getShardKeyPattern().globalMax()}, - ChunkVersion{2, 2, epoch, boost::none /* timestamp */}, + ChunkVersion{2, 2, epoch, timestamp}, kThisShard}}; auto rt = RoutingTableHistory::makeNew(kNss, @@ -501,7 +505,7 @@ TEST_F(RoutingTableHistoryTest, TestOutOfOrderVersion) { nullptr, false, epoch, - boost::none /* timestamp */, + timestamp, boost::none /* timeseriesFields */, boost::none, boost::none /* chunkSizeBytes */, @@ -512,20 +516,20 @@ TEST_F(RoutingTableHistoryTest, TestOutOfOrderVersion) { std::vector changedChunks = { ChunkType{uuid, ChunkRange{BSON("a" << 0), getShardKeyPattern().globalMax()}, - ChunkVersion{3, 0, epoch, boost::none /* timestamp */}, + ChunkVersion{3, 0, epoch, timestamp}, kThisShard}, ChunkType{uuid, ChunkRange{getShardKeyPattern().globalMin(), BSON("a" << 0)}, - ChunkVersion{3, 1, epoch, boost::none /* timestamp */}, + ChunkVersion{3, 1, epoch, timestamp}, kThisShard}}; auto rt1 = rt.makeUpdated(boost::none, boost::none, true, changedChunks); - auto v1 = ChunkVersion{3, 1, epoch, boost::none /* timestamp */}; + auto v1 = ChunkVersion{3, 1, epoch, timestamp}; ASSERT_EQ(v1, rt1.getVersion(kThisShard)); ASSERT_EQ(rt1.numChunks(), 2); auto chunk1 = rt1.findIntersectingChunk(BSON("a" << 0)); - ASSERT_EQ(chunk1->getLastmod(), ChunkVersion(3, 0, epoch, boost::none /* timestamp */)); + ASSERT_EQ(chunk1->getLastmod(), ChunkVersion(3, 0, epoch, timestamp)); ASSERT_EQ(chunk1->getMin().woCompare(BSON("a" << 0)), 0); ASSERT_EQ(chunk1->getMax().woCompare(getShardKeyPattern().globalMax()), 0); } @@ -533,19 +537,20 @@ TEST_F(RoutingTableHistoryTest, TestOutOfOrderVersion) { TEST_F(RoutingTableHistoryTest, TestMergeChunks) { const UUID uuid = UUID::gen(); const OID epoch = OID::gen(); + const Timestamp timestamp(1); std::vector initialChunks = { ChunkType{uuid, ChunkRange{BSON("a" << 0), BSON("a" << 10)}, - ChunkVersion{2, 0, epoch, boost::none /* timestamp */}, + ChunkVersion{2, 0, epoch, timestamp}, kThisShard}, ChunkType{uuid, ChunkRange{getShardKeyPattern().globalMin(), BSON("a" << 0)}, - ChunkVersion{2, 1, epoch, boost::none /* timestamp */}, + ChunkVersion{2, 1, epoch, timestamp}, kThisShard}, ChunkType{uuid, ChunkRange{BSON("a" << 10), getShardKeyPattern().globalMax()}, - ChunkVersion{2, 2, epoch, boost::none /* timestamp */}, + ChunkVersion{2, 2, epoch, timestamp}, kThisShard}}; auto rt = RoutingTableHistory::makeNew(kNss, @@ -554,27 +559,27 @@ TEST_F(RoutingTableHistoryTest, TestMergeChunks) { nullptr, false, epoch, + timestamp, boost::none, - boost::none /* timestamp */, boost::none /* timeseriesFields */, boost::none /* chunkSizeBytes */, true, initialChunks); ASSERT_EQ(rt.numChunks(), 3); - ASSERT_EQ(rt.getVersion(), ChunkVersion(2, 2, epoch, boost::none /* timestamp */)); + ASSERT_EQ(rt.getVersion(), ChunkVersion(2, 2, epoch, timestamp)); std::vector changedChunks = { ChunkType{uuid, ChunkRange{BSON("a" << 10), getShardKeyPattern().globalMax()}, - ChunkVersion{3, 0, epoch, boost::none /* timestamp */}, + ChunkVersion{3, 0, epoch, timestamp}, kThisShard}, ChunkType{uuid, ChunkRange{getShardKeyPattern().globalMin(), BSON("a" << 10)}, - ChunkVersion{3, 1, epoch, boost::none /* timestamp */}, + ChunkVersion{3, 1, epoch, timestamp}, kThisShard}}; auto rt1 = rt.makeUpdated(boost::none, boost::none, true, changedChunks); - auto v1 = ChunkVersion{3, 1, epoch, boost::none /* timestamp */}; + auto v1 = ChunkVersion{3, 1, epoch, timestamp}; ASSERT_EQ(v1, rt1.getVersion(kThisShard)); ASSERT_EQ(rt1.numChunks(), 2); } @@ -582,19 +587,20 @@ TEST_F(RoutingTableHistoryTest, TestMergeChunks) { TEST_F(RoutingTableHistoryTest, TestMergeChunksOrdering) { const UUID uuid = UUID::gen(); const OID epoch = OID::gen(); + const Timestamp timestamp(1); std::vector initialChunks = { ChunkType{uuid, ChunkRange{BSON("a" << -10), getShardKeyPattern().globalMax()}, - ChunkVersion{2, 0, epoch, boost::none /* timestamp */}, + ChunkVersion{2, 0, epoch, timestamp}, kThisShard}, ChunkType{uuid, ChunkRange{getShardKeyPattern().globalMin(), BSON("a" << -500)}, - ChunkVersion{2, 1, epoch, boost::none /* timestamp */}, + ChunkVersion{2, 1, epoch, timestamp}, kThisShard}, ChunkType{uuid, ChunkRange{BSON("a" << -500), BSON("a" << -10)}, - ChunkVersion{2, 2, epoch, boost::none /* timestamp */}, + ChunkVersion{2, 2, epoch, timestamp}, kThisShard}}; auto rt = RoutingTableHistory::makeNew(kNss, @@ -603,32 +609,32 @@ TEST_F(RoutingTableHistoryTest, TestMergeChunksOrdering) { nullptr, false, epoch, - boost::none /* timestamp */, + timestamp, boost::none /* timeseriesFields */, boost::none, boost::none /* chunkSizeBytes */, true, initialChunks); ASSERT_EQ(rt.numChunks(), 3); - ASSERT_EQ(rt.getVersion(), ChunkVersion(2, 2, epoch, boost::none /* timestamp */)); + ASSERT_EQ(rt.getVersion(), ChunkVersion(2, 2, epoch, timestamp)); std::vector changedChunks = { ChunkType{uuid, ChunkRange{BSON("a" << -500), BSON("a" << -10)}, - ChunkVersion{2, 2, epoch, boost::none /* timestamp */}, + ChunkVersion{2, 2, epoch, timestamp}, kThisShard}, ChunkType{uuid, ChunkRange{getShardKeyPattern().globalMin(), BSON("a" << -10)}, - ChunkVersion{3, 1, epoch, boost::none /* timestamp */}, + ChunkVersion{3, 1, epoch, timestamp}, kThisShard}}; auto rt1 = rt.makeUpdated(boost::none, boost::none, true, changedChunks); - auto v1 = ChunkVersion{3, 1, epoch, boost::none /* timestamp */}; + auto v1 = ChunkVersion{3, 1, epoch, timestamp}; ASSERT_EQ(v1, rt1.getVersion(kThisShard)); ASSERT_EQ(rt1.numChunks(), 2); auto chunk1 = rt1.findIntersectingChunk(BSON("a" << -500)); - ASSERT_EQ(chunk1->getLastmod(), ChunkVersion(3, 1, epoch, boost::none /* timestamp */)); + ASSERT_EQ(chunk1->getLastmod(), ChunkVersion(3, 1, epoch, timestamp)); ASSERT_EQ(chunk1->getMin().woCompare(getShardKeyPattern().globalMin()), 0); ASSERT_EQ(chunk1->getMax().woCompare(BSON("a" << -10)), 0); } @@ -636,31 +642,32 @@ TEST_F(RoutingTableHistoryTest, TestMergeChunksOrdering) { TEST_F(RoutingTableHistoryTest, TestFlatten) { const UUID uuid = UUID::gen(); const OID epoch = OID::gen(); + const Timestamp timestamp(1); std::vector initialChunks = { ChunkType{uuid, ChunkRange{getShardKeyPattern().globalMin(), BSON("a" << 10)}, - ChunkVersion{2, 0, epoch, boost::none /* timestamp */}, + ChunkVersion{2, 0, epoch, timestamp}, kThisShard}, ChunkType{uuid, ChunkRange{BSON("a" << 10), BSON("a" << 20)}, - ChunkVersion{2, 1, epoch, boost::none /* timestamp */}, + ChunkVersion{2, 1, epoch, timestamp}, kThisShard}, ChunkType{uuid, ChunkRange{BSON("a" << 20), getShardKeyPattern().globalMax()}, - ChunkVersion{2, 2, epoch, boost::none /* timestamp */}, + ChunkVersion{2, 2, epoch, timestamp}, kThisShard}, ChunkType{uuid, ChunkRange{getShardKeyPattern().globalMin(), getShardKeyPattern().globalMax()}, - ChunkVersion{3, 0, epoch, boost::none /* timestamp */}, + ChunkVersion{3, 0, epoch, timestamp}, kThisShard}, ChunkType{uuid, ChunkRange{getShardKeyPattern().globalMin(), BSON("a" << 10)}, - ChunkVersion{4, 0, epoch, boost::none /* timestamp */}, + ChunkVersion{4, 0, epoch, timestamp}, kThisShard}, ChunkType{uuid, ChunkRange{BSON("a" << 10), getShardKeyPattern().globalMax()}, - ChunkVersion{4, 1, epoch, boost::none /* timestamp */}, + ChunkVersion{4, 1, epoch, timestamp}, kThisShard}, }; @@ -670,17 +677,17 @@ TEST_F(RoutingTableHistoryTest, TestFlatten) { nullptr, false, epoch, - boost::none /* timestamp */, + timestamp, boost::none /* timeseriesFields */, boost::none, boost::none /* chunkSizeBytes */, true, initialChunks); ASSERT_EQ(rt.numChunks(), 2); - ASSERT_EQ(rt.getVersion(), ChunkVersion(4, 1, epoch, boost::none /* timestamp */)); + ASSERT_EQ(rt.getVersion(), ChunkVersion(4, 1, epoch, timestamp)); auto chunk1 = rt.findIntersectingChunk(BSON("a" << 0)); - ASSERT_EQ(chunk1->getLastmod(), ChunkVersion(4, 0, epoch, boost::none /* timestamp */)); + ASSERT_EQ(chunk1->getLastmod(), ChunkVersion(4, 0, epoch, timestamp)); ASSERT_EQ(chunk1->getMin().woCompare(getShardKeyPattern().globalMin()), 0); ASSERT_EQ(chunk1->getMax().woCompare(BSON("a" << 10)), 0); } diff --git a/src/mongo/s/write_ops/batch_write_exec_test.cpp b/src/mongo/s/write_ops/batch_write_exec_test.cpp index 8f09574e608..7ff643a4293 100644 --- a/src/mongo/s/write_ops/batch_write_exec_test.cpp +++ b/src/mongo/s/write_ops/batch_write_exec_test.cpp @@ -85,6 +85,7 @@ BSONObj expectInsertsReturnStaleVersionErrorsBase(const NamespaceString& nss, staleResponse.setN(0); auto epoch = OID::gen(); + Timestamp timestamp(1); // Report a stale version error for each write in the batch. int i = 0; @@ -93,8 +94,8 @@ BSONObj expectInsertsReturnStaleVersionErrorsBase(const NamespaceString& nss, error->setStatus({ErrorCodes::StaleShardVersion, ""}); error->setErrInfo([&] { StaleConfigInfo sci(nss, - ChunkVersion(1, 0, epoch, boost::none /* timestamp */), - ChunkVersion(2, 0, epoch, boost::none /* timestamp */), + ChunkVersion(1, 0, epoch, timestamp), + ChunkVersion(2, 0, epoch, timestamp), ShardId(kShardName1)); BSONObjBuilder builder; sci.serialize(&builder); @@ -280,9 +281,8 @@ public: MockNSTargeter singleShardNSTargeter{ nss, - {MockRange(ShardEndpoint(kShardName1, - ChunkVersion(100, 200, OID::gen(), boost::none /* timestamp */), - boost::none), + {MockRange(ShardEndpoint( + kShardName1, ChunkVersion(100, 200, OID::gen(), Timestamp()), boost::none), BSON("x" << MINKEY), BSON("x" << MAXKEY))}}; }; @@ -344,6 +344,7 @@ TEST_F(BatchWriteExecTest, SingleUpdateTargetsShardWithLet) { updateRequest.setWriteConcern(BSONObj()); const static auto epoch = OID::gen(); + const static Timestamp timestamp(2); class MultiShardTargeter : public MockNSTargeter { public: @@ -352,24 +353,20 @@ TEST_F(BatchWriteExecTest, SingleUpdateTargetsShardWithLet) { std::vector targetUpdate(OperationContext* opCtx, const BatchItemRef& itemRef) const override { return std::vector{ - ShardEndpoint(kShardName2, - ChunkVersion(101, 200, epoch, boost::none /* timestamp */), - boost::none)}; + ShardEndpoint(kShardName2, ChunkVersion(101, 200, epoch, timestamp), boost::none)}; } }; MultiShardTargeter multiShardNSTargeter( nss, - {MockRange(ShardEndpoint(kShardName1, - ChunkVersion(100, 200, epoch, boost::none /* timestamp */), - boost::none), - BSON("x" << MINKEY), - BSON("x" << 0)), - MockRange(ShardEndpoint(kShardName2, - ChunkVersion(101, 200, epoch, boost::none /* timestamp */), - boost::none), - BSON("x" << 0), - BSON("x" << MAXKEY))}); + {MockRange( + ShardEndpoint(kShardName1, ChunkVersion(100, 200, epoch, timestamp), boost::none), + BSON("x" << MINKEY), + BSON("x" << 0)), + MockRange( + ShardEndpoint(kShardName2, ChunkVersion(101, 200, epoch, timestamp), boost::none), + BSON("x" << 0), + BSON("x" << MAXKEY))}); auto future = launchAsync([&] { BatchedCommandResponse response; @@ -441,25 +438,21 @@ TEST_F(BatchWriteExecTest, SingleDeleteTargetsShardWithLet) { protected: std::vector targetDelete(OperationContext* opCtx, const BatchItemRef& itemRef) const override { - return std::vector{ - ShardEndpoint(kShardName2, - ChunkVersion(101, 200, epoch, boost::none /* timestamp */), - boost::none)}; + return std::vector{ShardEndpoint( + kShardName2, ChunkVersion(101, 200, epoch, Timestamp()), boost::none)}; } }; MultiShardTargeter multiShardNSTargeter( nss, - {MockRange(ShardEndpoint(kShardName1, - ChunkVersion(100, 200, epoch, boost::none /* timestamp */), - boost::none), - BSON("x" << MINKEY), - BSON("x" << 0)), - MockRange(ShardEndpoint(kShardName2, - ChunkVersion(101, 200, epoch, boost::none /* timestamp */), - boost::none), - BSON("x" << 0), - BSON("x" << MAXKEY))}); + {MockRange( + ShardEndpoint(kShardName1, ChunkVersion(100, 200, epoch, Timestamp()), boost::none), + BSON("x" << MINKEY), + BSON("x" << 0)), + MockRange( + ShardEndpoint(kShardName2, ChunkVersion(101, 200, epoch, Timestamp()), boost::none), + BSON("x" << 0), + BSON("x" << MAXKEY))}); auto future = launchAsync([&] { BatchedCommandResponse response; @@ -606,9 +599,9 @@ TEST_F(BatchWriteExecTest, MultiOpLargeUnorderedWithStaleShardVersionError) { ASSERT_EQ(kNumDocsToInsert, response.getN()); }); - expectInsertsReturnStaleVersionErrors({docsToInsert.begin(), docsToInsert.begin() + 63791}); - expectInsertsReturnSuccess({docsToInsert.begin(), docsToInsert.begin() + 63791}); - expectInsertsReturnSuccess({docsToInsert.begin() + 63791, docsToInsert.end()}); + expectInsertsReturnStaleVersionErrors({docsToInsert.begin(), docsToInsert.begin() + 60133}); + expectInsertsReturnSuccess({docsToInsert.begin(), docsToInsert.begin() + 60133}); + expectInsertsReturnSuccess({docsToInsert.begin() + 60133, docsToInsert.end()}); future.default_timed_get(); } @@ -629,6 +622,7 @@ TEST_F(BatchWriteExecTest, StaleShardVersionReturnedFromBatchWithSingleMultiWrit request.setWriteConcern(BSONObj()); const static auto epoch = OID::gen(); + const static Timestamp timestamp(2); class MultiShardTargeter : public MockNSTargeter { public: @@ -637,27 +631,21 @@ TEST_F(BatchWriteExecTest, StaleShardVersionReturnedFromBatchWithSingleMultiWrit std::vector targetUpdate(OperationContext* opCtx, const BatchItemRef& itemRef) const override { return std::vector{ - ShardEndpoint(kShardName1, - ChunkVersion(100, 200, epoch, boost::none /* timestamp */), - boost::none), - ShardEndpoint(kShardName2, - ChunkVersion(101, 200, epoch, boost::none /* timestamp */), - boost::none)}; + ShardEndpoint(kShardName1, ChunkVersion(100, 200, epoch, timestamp), boost::none), + ShardEndpoint(kShardName2, ChunkVersion(101, 200, epoch, timestamp), boost::none)}; } }; MultiShardTargeter multiShardNSTargeter( nss, - {MockRange(ShardEndpoint(kShardName1, - ChunkVersion(100, 200, epoch, boost::none /* timestamp */), - boost::none), - BSON("x" << MINKEY), - BSON("x" << 0)), - MockRange(ShardEndpoint(kShardName2, - ChunkVersion(101, 200, epoch, boost::none /* timestamp */), - boost::none), - BSON("x" << 0), - BSON("x" << MAXKEY))}); + {MockRange( + ShardEndpoint(kShardName1, ChunkVersion(100, 200, epoch, timestamp), boost::none), + BSON("x" << MINKEY), + BSON("x" << 0)), + MockRange( + ShardEndpoint(kShardName2, ChunkVersion(101, 200, epoch, timestamp), boost::none), + BSON("x" << 0), + BSON("x" << MAXKEY))}); auto future = launchAsync([&] { BatchedCommandResponse response; @@ -689,12 +677,11 @@ TEST_F(BatchWriteExecTest, StaleShardVersionReturnedFromBatchWithSingleMultiWrit errDetail->setIndex(0); errDetail->setStatus({ErrorCodes::StaleShardVersion, "Stale shard version"}); errDetail->setErrInfo([&] { - Status ssvStatus( - StaleConfigInfo(nss, - ChunkVersion(101, 200, epoch, boost::none /* timestamp */), - ChunkVersion(105, 200, epoch, boost::none /* timestamp */), - ShardId(kShardName2)), - "Stale shard version"); + Status ssvStatus(StaleConfigInfo(nss, + ChunkVersion(101, 200, epoch, timestamp), + ChunkVersion(105, 200, epoch, timestamp), + ShardId(kShardName2)), + "Stale shard version"); BSONObjBuilder builder; ssvStatus.serializeErrorToBSON(&builder); return builder.obj(); @@ -741,6 +728,7 @@ TEST_F(BatchWriteExecTest, request.setWriteConcern(BSONObj()); const static auto epoch = OID::gen(); + const static Timestamp timestamp(2); // This allows the batch to target each write operation // to a specific shard (kShardName2), to perform this test @@ -751,27 +739,21 @@ TEST_F(BatchWriteExecTest, std::vector targetUpdate(OperationContext* opCtx, const BatchItemRef& itemRef) const override { return std::vector{ - ShardEndpoint(kShardName1, - ChunkVersion(100, 200, epoch, boost::none /* timestamp */), - boost::none), - ShardEndpoint(kShardName2, - ChunkVersion(101, 200, epoch, boost::none /* timestamp */), - boost::none)}; + ShardEndpoint(kShardName1, ChunkVersion(100, 200, epoch, timestamp), boost::none), + ShardEndpoint(kShardName2, ChunkVersion(101, 200, epoch, timestamp), boost::none)}; } }; MultiShardTargeter multiShardNSTargeter( nss, - {MockRange(ShardEndpoint(kShardName1, - ChunkVersion(100, 200, epoch, boost::none /* timestamp */), - boost::none), - BSON("sk" << MINKEY), - BSON("sk" << 10)), - MockRange(ShardEndpoint(kShardName2, - ChunkVersion(101, 200, epoch, boost::none /* timestamp */), - boost::none), - BSON("sk" << 10), - BSON("sk" << MAXKEY))}); + {MockRange( + ShardEndpoint(kShardName1, ChunkVersion(100, 200, epoch, timestamp), boost::none), + BSON("sk" << MINKEY), + BSON("sk" << 10)), + MockRange( + ShardEndpoint(kShardName2, ChunkVersion(101, 200, epoch, timestamp), boost::none), + BSON("sk" << 10), + BSON("sk" << MAXKEY))}); auto future = launchAsync([&] { BatchedCommandResponse response; @@ -803,12 +785,11 @@ TEST_F(BatchWriteExecTest, errDetail->setIndex(0); errDetail->setStatus({ErrorCodes::StaleShardVersion, "Stale shard version"}); errDetail->setErrInfo([&] { - Status ssvStatus( - StaleConfigInfo(nss, - ChunkVersion(101, 200, epoch, boost::none /* timestamp */), - ChunkVersion(105, 200, epoch, boost::none /* timestamp */), - ShardId(kShardName2)), - "Stale shard version"); + Status ssvStatus(StaleConfigInfo(nss, + ChunkVersion(101, 200, epoch, timestamp), + ChunkVersion(105, 200, epoch, timestamp), + ShardId(kShardName2)), + "Stale shard version"); BSONObjBuilder builder; ssvStatus.serializeErrorToBSON(&builder); return builder.obj(); @@ -820,12 +801,11 @@ TEST_F(BatchWriteExecTest, errDetail->setIndex(1); errDetail->setStatus({ErrorCodes::StaleShardVersion, "Stale shard version"}); errDetail->setErrInfo([&] { - Status ssvStatus( - StaleConfigInfo(nss, - ChunkVersion(101, 200, epoch, boost::none /* timestamp */), - ChunkVersion(105, 200, epoch, boost::none /* timestamp */), - ShardId(kShardName2)), - "Stale shard version"); + Status ssvStatus(StaleConfigInfo(nss, + ChunkVersion(101, 200, epoch, timestamp), + ChunkVersion(105, 200, epoch, timestamp), + ShardId(kShardName2)), + "Stale shard version"); BSONObjBuilder builder; ssvStatus.serializeErrorToBSON(&builder); return builder.obj(); @@ -871,6 +851,7 @@ TEST_F(BatchWriteExecTest, RetryableErrorReturnedFromMultiWriteWithShard1Firs) { request.setWriteConcern(BSONObj()); const static auto epoch = OID::gen(); + const static Timestamp timestamp(2); // This allows the batch to target each write operation // to a specific shard (kShardName2), to perform this test @@ -881,27 +862,21 @@ TEST_F(BatchWriteExecTest, RetryableErrorReturnedFromMultiWriteWithShard1Firs) { std::vector targetUpdate(OperationContext* opCtx, const BatchItemRef& itemRef) const override { return std::vector{ - ShardEndpoint(kShardName1, - ChunkVersion(100, 200, epoch, boost::none /* timestamp */), - boost::none), - ShardEndpoint(kShardName2, - ChunkVersion(101, 200, epoch, boost::none /* timestamp */), - boost::none)}; + ShardEndpoint(kShardName1, ChunkVersion(100, 200, epoch, timestamp), boost::none), + ShardEndpoint(kShardName2, ChunkVersion(101, 200, epoch, timestamp), boost::none)}; } }; MultiShardTargeter multiShardNSTargeter( nss, - {MockRange(ShardEndpoint(kShardName1, - ChunkVersion(100, 200, epoch, boost::none /* timestamp */), - boost::none), - BSON("sk" << MINKEY), - BSON("sk" << 10)), - MockRange(ShardEndpoint(kShardName2, - ChunkVersion(101, 200, epoch, boost::none /* timestamp */), - boost::none), - BSON("sk" << 10), - BSON("sk" << MAXKEY))}); + {MockRange( + ShardEndpoint(kShardName1, ChunkVersion(100, 200, epoch, timestamp), boost::none), + BSON("sk" << MINKEY), + BSON("sk" << 10)), + MockRange( + ShardEndpoint(kShardName2, ChunkVersion(101, 200, epoch, timestamp), boost::none), + BSON("sk" << 10), + BSON("sk" << MAXKEY))}); auto future = launchAsync([&] { BatchedCommandResponse response; @@ -923,12 +898,11 @@ TEST_F(BatchWriteExecTest, RetryableErrorReturnedFromMultiWriteWithShard1Firs) { errDetail->setIndex(1); errDetail->setStatus({ErrorCodes::StaleShardVersion, "Stale shard version"}); errDetail->setErrInfo([&] { - Status ssvStatus( - StaleConfigInfo(nss, - ChunkVersion(101, 200, epoch, boost::none /* timestamp */), - ChunkVersion(105, 200, epoch, boost::none /* timestamp */), - ShardId(kShardName1)), - "Stale shard version"); + Status ssvStatus(StaleConfigInfo(nss, + ChunkVersion(101, 200, epoch, timestamp), + ChunkVersion(105, 200, epoch, timestamp), + ShardId(kShardName1)), + "Stale shard version"); BSONObjBuilder builder; ssvStatus.serializeErrorToBSON(&builder); return builder.obj(); @@ -950,12 +924,11 @@ TEST_F(BatchWriteExecTest, RetryableErrorReturnedFromMultiWriteWithShard1Firs) { errDetail->setIndex(0); errDetail->setStatus({ErrorCodes::StaleShardVersion, "Stale shard version"}); errDetail->setErrInfo([&] { - Status ssvStatus( - StaleConfigInfo(nss, - ChunkVersion(101, 200, epoch, boost::none /* timestamp */), - ChunkVersion(105, 200, epoch, boost::none /* timestamp */), - ShardId(kShardName2)), - "Stale shard version"); + Status ssvStatus(StaleConfigInfo(nss, + ChunkVersion(101, 200, epoch, timestamp), + ChunkVersion(105, 200, epoch, timestamp), + ShardId(kShardName2)), + "Stale shard version"); BSONObjBuilder builder; ssvStatus.serializeErrorToBSON(&builder); return builder.obj(); @@ -1012,6 +985,7 @@ TEST_F(BatchWriteExecTest, RetryableErrorReturnedFromMultiWriteWithShard1FirstOK request.setWriteConcern(BSONObj()); const static auto epoch = OID::gen(); + const static Timestamp timestamp(2); // This allows the batch to target each write operation // to a specific shard (kShardName2), to perform this test @@ -1022,27 +996,21 @@ TEST_F(BatchWriteExecTest, RetryableErrorReturnedFromMultiWriteWithShard1FirstOK std::vector targetUpdate(OperationContext* opCtx, const BatchItemRef& itemRef) const override { return std::vector{ - ShardEndpoint(kShardName1, - ChunkVersion(100, 200, epoch, boost::none /* timestamp */), - boost::none), - ShardEndpoint(kShardName2, - ChunkVersion(101, 200, epoch, boost::none /* timestamp */), - boost::none)}; + ShardEndpoint(kShardName1, ChunkVersion(100, 200, epoch, timestamp), boost::none), + ShardEndpoint(kShardName2, ChunkVersion(101, 200, epoch, timestamp), boost::none)}; } }; MultiShardTargeter multiShardNSTargeter( nss, - {MockRange(ShardEndpoint(kShardName1, - ChunkVersion(100, 200, epoch, boost::none /* timestamp */), - boost::none), - BSON("sk" << MINKEY), - BSON("sk" << 10)), - MockRange(ShardEndpoint(kShardName2, - ChunkVersion(101, 200, epoch, boost::none /* timestamp */), - boost::none), - BSON("sk" << 10), - BSON("sk" << MAXKEY))}); + {MockRange( + ShardEndpoint(kShardName1, ChunkVersion(100, 200, epoch, timestamp), boost::none), + BSON("sk" << MINKEY), + BSON("sk" << 10)), + MockRange( + ShardEndpoint(kShardName2, ChunkVersion(101, 200, epoch, timestamp), boost::none), + BSON("sk" << 10), + BSON("sk" << MAXKEY))}); auto future = launchAsync([&] { BatchedCommandResponse response; @@ -1064,12 +1032,11 @@ TEST_F(BatchWriteExecTest, RetryableErrorReturnedFromMultiWriteWithShard1FirstOK errDetail->setIndex(1); errDetail->setStatus({ErrorCodes::StaleShardVersion, "Stale shard version"}); errDetail->setErrInfo([&] { - Status ssvStatus( - StaleConfigInfo(nss, - ChunkVersion(101, 200, epoch, boost::none /* timestamp */), - ChunkVersion(105, 200, epoch, boost::none /* timestamp */), - ShardId(kShardName1)), - "Stale shard version"); + Status ssvStatus(StaleConfigInfo(nss, + ChunkVersion(101, 200, epoch, timestamp), + ChunkVersion(105, 200, epoch, timestamp), + ShardId(kShardName1)), + "Stale shard version"); BSONObjBuilder builder; ssvStatus.serializeErrorToBSON(&builder); return builder.obj(); @@ -1091,12 +1058,11 @@ TEST_F(BatchWriteExecTest, RetryableErrorReturnedFromMultiWriteWithShard1FirstOK errDetail->setIndex(1); errDetail->setStatus({ErrorCodes::StaleShardVersion, "Stale shard version"}); errDetail->setErrInfo([&] { - Status ssvStatus( - StaleConfigInfo(nss, - ChunkVersion(101, 200, epoch, boost::none /* timestamp */), - ChunkVersion(105, 200, epoch, boost::none /* timestamp */), - ShardId(kShardName2)), - "Stale shard version"); + Status ssvStatus(StaleConfigInfo(nss, + ChunkVersion(101, 200, epoch, timestamp), + ChunkVersion(105, 200, epoch, timestamp), + ShardId(kShardName2)), + "Stale shard version"); BSONObjBuilder builder; ssvStatus.serializeErrorToBSON(&builder); return builder.obj(); @@ -1149,6 +1115,7 @@ TEST_F(BatchWriteExecTest, RetryableErrorReturnedFromWriteWithShard1SSVShard2OK) request.setWriteConcern(BSONObj()); const static auto epoch = OID::gen(); + const static Timestamp timestamp(2); // This allows the batch to target each write operation to perform this test class MultiShardTargeter : public MockNSTargeter { @@ -1159,17 +1126,13 @@ TEST_F(BatchWriteExecTest, RetryableErrorReturnedFromWriteWithShard1SSVShard2OK) const BatchItemRef& itemRef) const override { if (targetAll) { return std::vector{ - ShardEndpoint(kShardName1, - ChunkVersion(100, 200, epoch, boost::none /* timestamp */), - boost::none), - ShardEndpoint(kShardName2, - ChunkVersion(101, 200, epoch, boost::none /* timestamp */), - boost::none)}; + ShardEndpoint( + kShardName1, ChunkVersion(100, 200, epoch, timestamp), boost::none), + ShardEndpoint( + kShardName2, ChunkVersion(101, 200, epoch, timestamp), boost::none)}; } else { - return std::vector{ - ShardEndpoint(kShardName2, - ChunkVersion(101, 200, epoch, boost::none /* timestamp */), - boost::none)}; + return std::vector{ShardEndpoint( + kShardName2, ChunkVersion(101, 200, epoch, timestamp), boost::none)}; } } @@ -1178,16 +1141,14 @@ TEST_F(BatchWriteExecTest, RetryableErrorReturnedFromWriteWithShard1SSVShard2OK) MultiShardTargeter multiShardNSTargeter( nss, - {MockRange(ShardEndpoint(kShardName1, - ChunkVersion(100, 200, epoch, boost::none /* timestamp */), - boost::none), - BSON("sk" << MINKEY), - BSON("sk" << 10)), - MockRange(ShardEndpoint(kShardName2, - ChunkVersion(101, 200, epoch, boost::none /* timestamp */), - boost::none), - BSON("sk" << 10), - BSON("sk" << MAXKEY))}); + {MockRange( + ShardEndpoint(kShardName1, ChunkVersion(100, 200, epoch, timestamp), boost::none), + BSON("sk" << MINKEY), + BSON("sk" << 10)), + MockRange( + ShardEndpoint(kShardName2, ChunkVersion(101, 200, epoch, timestamp), boost::none), + BSON("sk" << 10), + BSON("sk" << MAXKEY))}); auto future = launchAsync([&] { BatchedCommandResponse response; @@ -1210,12 +1171,11 @@ TEST_F(BatchWriteExecTest, RetryableErrorReturnedFromWriteWithShard1SSVShard2OK) errDetail->setIndex(0); errDetail->setStatus({ErrorCodes::StaleShardVersion, "Stale shard version"}); errDetail->setErrInfo([&] { - Status ssvStatus( - StaleConfigInfo(nss, - ChunkVersion(101, 200, epoch, boost::none /* timestamp */), - ChunkVersion(105, 200, epoch, boost::none /* timestamp */), - ShardId(kShardName1)), - "Migration happened"); + Status ssvStatus(StaleConfigInfo(nss, + ChunkVersion(101, 200, epoch, timestamp), + ChunkVersion(105, 200, epoch, timestamp), + ShardId(kShardName1)), + "Migration happened"); BSONObjBuilder builder; ssvStatus.serializeErrorToBSON(&builder); return builder.obj(); @@ -1731,6 +1691,7 @@ TEST_F(BatchWriteExecTargeterErrorTest, TargetedFailedAndErrorResponse) { request.setWriteConcern(BSONObj()); const static auto epoch = OID::gen(); + const static Timestamp timestamp(2); class MultiShardTargeter : public MockNSTargeter { public: @@ -1739,27 +1700,21 @@ TEST_F(BatchWriteExecTargeterErrorTest, TargetedFailedAndErrorResponse) { std::vector targetUpdate(OperationContext* opCtx, const BatchItemRef& itemRef) const override { return std::vector{ - ShardEndpoint(kShardName1, - ChunkVersion(100, 200, epoch, boost::none /* timestamp */), - boost::none), - ShardEndpoint(kShardName2, - ChunkVersion(101, 200, epoch, boost::none /* timestamp */), - boost::none)}; + ShardEndpoint(kShardName1, ChunkVersion(100, 200, epoch, timestamp), boost::none), + ShardEndpoint(kShardName2, ChunkVersion(101, 200, epoch, timestamp), boost::none)}; } }; MultiShardTargeter multiShardNSTargeter( nss, - {MockRange(ShardEndpoint(kShardName1, - ChunkVersion(100, 200, epoch, boost::none /* timestamp */), - boost::none), - BSON("x" << MINKEY), - BSON("x" << 0)), - MockRange(ShardEndpoint(kShardName2, - ChunkVersion(101, 200, epoch, boost::none /* timestamp */), - boost::none), - BSON("x" << 0), - BSON("x" << MAXKEY))}); + {MockRange( + ShardEndpoint(kShardName1, ChunkVersion(100, 200, epoch, timestamp), boost::none), + BSON("x" << MINKEY), + BSON("x" << 0)), + MockRange( + ShardEndpoint(kShardName2, ChunkVersion(101, 200, epoch, timestamp), boost::none), + BSON("x" << 0), + BSON("x" << MAXKEY))}); auto future = launchAsync([&] { BatchedCommandResponse response; @@ -1879,6 +1834,7 @@ TEST_F(BatchWriteExecTransactionTargeterErrorTest, TargetedFailedAndErrorRespons request.setWriteConcern(BSONObj()); const static auto epoch = OID::gen(); + const static Timestamp timestamp(2); class MultiShardTargeter : public MockNSTargeter { public: @@ -1887,27 +1843,21 @@ TEST_F(BatchWriteExecTransactionTargeterErrorTest, TargetedFailedAndErrorRespons std::vector targetUpdate(OperationContext* opCtx, const BatchItemRef& itemRef) const override { return std::vector{ - ShardEndpoint(kShardName1, - ChunkVersion(100, 200, epoch, boost::none /* timestamp */), - boost::none), - ShardEndpoint(kShardName2, - ChunkVersion(101, 200, epoch, boost::none /* timestamp */), - boost::none)}; + ShardEndpoint(kShardName1, ChunkVersion(100, 200, epoch, timestamp), boost::none), + ShardEndpoint(kShardName2, ChunkVersion(101, 200, epoch, timestamp), boost::none)}; } }; MultiShardTargeter multiShardNSTargeter( nss, - {MockRange(ShardEndpoint(kShardName1, - ChunkVersion(100, 200, epoch, boost::none /* timestamp */), - boost::none), - BSON("x" << MINKEY), - BSON("x" << 0)), - MockRange(ShardEndpoint(kShardName2, - ChunkVersion(101, 200, epoch, boost::none /* timestamp */), - boost::none), - BSON("x" << 0), - BSON("x" << MAXKEY))}); + {MockRange( + ShardEndpoint(kShardName1, ChunkVersion(100, 200, epoch, timestamp), boost::none), + BSON("x" << MINKEY), + BSON("x" << 0)), + MockRange( + ShardEndpoint(kShardName2, ChunkVersion(101, 200, epoch, timestamp), boost::none), + BSON("x" << 0), + BSON("x" << MAXKEY))}); auto future = launchAsync([&] { BatchedCommandResponse response; @@ -2030,6 +1980,7 @@ TEST_F(BatchWriteExecTransactionMultiShardTest, TargetedSucceededAndErrorRespons request.setWriteConcern(BSONObj()); const static auto epoch = OID::gen(); + const static Timestamp timestamp(2); class MultiShardTargeter : public MockNSTargeter { public: @@ -2038,27 +1989,21 @@ TEST_F(BatchWriteExecTransactionMultiShardTest, TargetedSucceededAndErrorRespons std::vector targetUpdate(OperationContext* opCtx, const BatchItemRef& itemRef) const override { return std::vector{ - ShardEndpoint(kShardName1, - ChunkVersion(100, 200, epoch, boost::none /* timestamp */), - boost::none), - ShardEndpoint(kShardName2, - ChunkVersion(101, 200, epoch, boost::none /* timestamp */), - boost::none)}; + ShardEndpoint(kShardName1, ChunkVersion(100, 200, epoch, timestamp), boost::none), + ShardEndpoint(kShardName2, ChunkVersion(101, 200, epoch, timestamp), boost::none)}; } }; MultiShardTargeter multiShardNSTargeter( nss, - {MockRange(ShardEndpoint(kShardName1, - ChunkVersion(100, 200, epoch, boost::none /* timestamp */), - boost::none), - BSON("x" << MINKEY), - BSON("x" << 0)), - MockRange(ShardEndpoint(kShardName2, - ChunkVersion(101, 200, epoch, boost::none /* timestamp */), - boost::none), - BSON("x" << 0), - BSON("x" << MAXKEY))}); + {MockRange( + ShardEndpoint(kShardName1, ChunkVersion(100, 200, epoch, timestamp), boost::none), + BSON("x" << MINKEY), + BSON("x" << 0)), + MockRange( + ShardEndpoint(kShardName2, ChunkVersion(101, 200, epoch, timestamp), boost::none), + BSON("x" << 0), + BSON("x" << MAXKEY))}); auto future = launchAsync([&] { BatchedCommandResponse response; diff --git a/src/mongo/s/write_ops/batch_write_op.cpp b/src/mongo/s/write_ops/batch_write_op.cpp index 20e183e5ca6..db2f1623d26 100644 --- a/src/mongo/s/write_ops/batch_write_op.cpp +++ b/src/mongo/s/write_ops/batch_write_op.cpp @@ -394,10 +394,10 @@ Status BatchWriteOp::targetBatch(const NSTargeter& targeter, // StaleShardVersion and has to return number of errors equivalent to the number of writes // in the batch, the response size will not exceed the max BSON size. // - // The constant of 256 is chosen as an approximation of the size of the BSON representataion + // The constant of 272 is chosen as an approximation of the size of the BSON representataion // of the StaleConfigInfo (which contains the shard id) and the adjacent error message. const int errorResponsePotentialSizeBytes = - ordered ? 0 : write_ops::kWriteCommandBSONArrayPerElementOverheadBytes + 256; + ordered ? 0 : write_ops::kWriteCommandBSONArrayPerElementOverheadBytes + 272; if (wouldMakeBatchesTooBig( writes, std::max(writeSizeBytes, errorResponsePotentialSizeBytes), batchMap)) { diff --git a/src/mongo/s/write_ops/batched_command_request_test.cpp b/src/mongo/s/write_ops/batched_command_request_test.cpp index e06aa4e55fa..9a5e968f10d 100644 --- a/src/mongo/s/write_ops/batched_command_request_test.cpp +++ b/src/mongo/s/write_ops/batched_command_request_test.cpp @@ -60,12 +60,14 @@ TEST(BatchedCommandRequest, InsertWithShardVersion) { BSONArray insertArray = BSON_ARRAY(BSON("a" << 1) << BSON("b" << 1)); const OID epoch = OID::gen(); + const Timestamp majorAndMinor(1, 2); + const Timestamp timestamp(2, 2); BSONObj origInsertRequestObj = BSON("insert" << "test" << "documents" << insertArray << "writeConcern" << BSON("w" << 1) << "ordered" << true << "shardVersion" - << BSON_ARRAY(Timestamp(1, 2) << epoch)); + << BSON_ARRAY(majorAndMinor << epoch << timestamp)); for (auto docSeq : {false, true}) { const auto opMsgRequest(toOpMsg("TestDB", origInsertRequestObj, docSeq)); @@ -73,7 +75,7 @@ TEST(BatchedCommandRequest, InsertWithShardVersion) { ASSERT_EQ("TestDB.test", insertRequest.getInsertRequest().getNamespace().ns()); ASSERT(insertRequest.hasShardVersion()); - ASSERT_EQ(ChunkVersion(1, 2, epoch, boost::none /* timestamp */).toString(), + ASSERT_EQ(ChunkVersion(1, 2, epoch, timestamp).toString(), insertRequest.getShardVersion().toString()); } } diff --git a/src/mongo/s/write_ops/write_op_test.cpp b/src/mongo/s/write_ops/write_op_test.cpp index 4ba35fb5470..02d10be8eef 100644 --- a/src/mongo/s/write_ops/write_op_test.cpp +++ b/src/mongo/s/write_ops/write_op_test.cpp @@ -125,11 +125,11 @@ TEST_F(WriteOpTest, TargetSingle) { // Multi-write targeting test where our query goes to one shard TEST_F(WriteOpTest, TargetMultiOneShard) { ShardEndpoint endpointA( - ShardId("shardA"), ChunkVersion(10, 0, OID(), boost::none /* timestamp */), boost::none); + ShardId("shardA"), ChunkVersion(10, 0, OID(), Timestamp()), boost::none); ShardEndpoint endpointB( - ShardId("shardB"), ChunkVersion(20, 0, OID(), boost::none /* timestamp */), boost::none); + ShardId("shardB"), ChunkVersion(20, 0, OID(), Timestamp()), boost::none); ShardEndpoint endpointC( - ShardId("shardB"), ChunkVersion(20, 0, OID(), boost::none /* timestamp */), boost::none); + ShardId("shardB"), ChunkVersion(20, 0, OID(), Timestamp()), boost::none); BatchedCommandRequest request([&] { write_ops::DeleteCommandRequest deleteOp(kNss); @@ -161,11 +161,11 @@ TEST_F(WriteOpTest, TargetMultiOneShard) { // Multi-write targeting test where our write goes to more than one shard TEST_F(WriteOpTest, TargetMultiAllShards) { ShardEndpoint endpointA( - ShardId("shardA"), ChunkVersion(10, 0, OID(), boost::none /* timestamp */), boost::none); + ShardId("shardA"), ChunkVersion(10, 0, OID(), Timestamp()), boost::none); ShardEndpoint endpointB( - ShardId("shardB"), ChunkVersion(20, 0, OID(), boost::none /* timestamp */), boost::none); + ShardId("shardB"), ChunkVersion(20, 0, OID(), Timestamp()), boost::none); ShardEndpoint endpointC( - ShardId("shardB"), ChunkVersion(20, 0, OID(), boost::none /* timestamp */), boost::none); + ShardId("shardB"), ChunkVersion(20, 0, OID(), Timestamp()), boost::none); BatchedCommandRequest request([&] { write_ops::DeleteCommandRequest deleteOp(kNss); @@ -204,9 +204,9 @@ TEST_F(WriteOpTest, TargetMultiAllShards) { TEST_F(WriteOpTest, TargetMultiAllShardsAndErrorSingleChildOp) { ShardEndpoint endpointA( - ShardId("shardA"), ChunkVersion(10, 0, OID(), boost::none /* timestamp */), boost::none); + ShardId("shardA"), ChunkVersion(10, 0, OID(), Timestamp()), boost::none); ShardEndpoint endpointB( - ShardId("shardB"), ChunkVersion(20, 0, OID(), boost::none /* timestamp */), boost::none); + ShardId("shardB"), ChunkVersion(20, 0, OID(), Timestamp()), boost::none); BatchedCommandRequest request([&] { write_ops::DeleteCommandRequest deleteOp(kNss); @@ -356,11 +356,11 @@ private: TEST_F(WriteOpTransactionTest, TargetMultiDoesNotTargetAllShards) { ShardEndpoint endpointA( - ShardId("shardA"), ChunkVersion(10, 0, OID(), boost::none /* timestamp */), boost::none); + ShardId("shardA"), ChunkVersion(10, 0, OID(), Timestamp()), boost::none); ShardEndpoint endpointB( - ShardId("shardB"), ChunkVersion(20, 0, OID(), boost::none /* timestamp */), boost::none); + ShardId("shardB"), ChunkVersion(20, 0, OID(), Timestamp()), boost::none); ShardEndpoint endpointC( - ShardId("shardB"), ChunkVersion(20, 0, OID(), boost::none /* timestamp */), boost::none); + ShardId("shardB"), ChunkVersion(20, 0, OID(), Timestamp()), boost::none); BatchedCommandRequest request([&] { write_ops::DeleteCommandRequest deleteOp(kNss); @@ -397,9 +397,9 @@ TEST_F(WriteOpTransactionTest, TargetMultiDoesNotTargetAllShards) { TEST_F(WriteOpTransactionTest, TargetMultiAllShardsAndErrorSingleChildOp) { ShardEndpoint endpointA( - ShardId("shardA"), ChunkVersion(10, 0, OID(), boost::none /* timestamp */), boost::none); + ShardId("shardA"), ChunkVersion(10, 0, OID(), Timestamp()), boost::none); ShardEndpoint endpointB( - ShardId("shardB"), ChunkVersion(20, 0, OID(), boost::none /* timestamp */), boost::none); + ShardId("shardB"), ChunkVersion(20, 0, OID(), Timestamp()), boost::none); BatchedCommandRequest request([&] { write_ops::DeleteCommandRequest deleteOp(kNss); -- cgit v1.2.1