summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorKaloian Manassiev <kaloian.manassiev@mongodb.com>2022-06-21 16:21:21 +0000
committerEvergreen Agent <no-reply@evergreen.mongodb.com>2022-06-21 17:53:02 +0000
commitb11292a5e8ee11f3302636f9ba80b33876fbfa94 (patch)
tree9e6b6267dcad44c87eda77c953cccb3cb70b785b
parent22efe2b8f4f211bbd5b89b42d274e55af181789d (diff)
downloadmongo-b11292a5e8ee11f3302636f9ba80b33876fbfa94.tar.gz
SERVER-67400 Introduce a CollectionPlacement component of the collection version
-rw-r--r--src/mongo/db/pipeline/dispatch_shard_pipeline_test.cpp6
-rw-r--r--src/mongo/db/pipeline/process_interface/mongos_process_interface_test.cpp4
-rw-r--r--src/mongo/db/pipeline/resharding_initial_split_policy_test.cpp4
-rw-r--r--src/mongo/db/s/balancer/balancer_defragmentation_policy_test.cpp27
-rw-r--r--src/mongo/db/s/balancer/balancer_policy_test.cpp5
-rw-r--r--src/mongo/db/s/collection_sharding_runtime_test.cpp12
-rw-r--r--src/mongo/db/s/config/sharding_catalog_manager_bump_collection_version_and_change_metadata_test.cpp19
-rw-r--r--src/mongo/db/s/config/sharding_catalog_manager_chunk_operations.cpp59
-rw-r--r--src/mongo/db/s/config/sharding_catalog_manager_commit_chunk_migration_test.cpp64
-rw-r--r--src/mongo/db/s/config/sharding_catalog_manager_remove_shard_test.cpp16
-rw-r--r--src/mongo/db/s/migration_chunk_cloner_source_legacy_test.cpp2
-rw-r--r--src/mongo/db/s/migration_util_test.cpp10
-rw-r--r--src/mongo/db/s/op_observer_sharding_test.cpp8
-rw-r--r--src/mongo/db/s/range_deletion_util_test.cpp4
-rw-r--r--src/mongo/db/s/resharding/resharding_coordinator_service.cpp6
-rw-r--r--src/mongo/db/s/resharding/resharding_coordinator_test.cpp20
-rw-r--r--src/mongo/db/s/resharding/resharding_destined_recipient_test.cpp10
-rw-r--r--src/mongo/db/s/resharding/resharding_oplog_applier_test.cpp10
-rw-r--r--src/mongo/db/s/resharding/resharding_oplog_batch_applier_test.cpp6
-rw-r--r--src/mongo/db/s/resharding/resharding_recipient_service_external_state_test.cpp10
-rw-r--r--src/mongo/db/s/shard_server_catalog_cache_loader.cpp19
-rw-r--r--src/mongo/db/s/type_shard_collection.cpp2
-rw-r--r--src/mongo/s/catalog/sharding_catalog_client_test.cpp18
-rw-r--r--src/mongo/s/chunk_manager.cpp14
-rw-r--r--src/mongo/s/chunk_manager.h13
-rw-r--r--src/mongo/s/chunk_version.cpp6
-rw-r--r--src/mongo/s/chunk_version.h78
-rw-r--r--src/mongo/s/query/cluster_exchange_test.cpp4
-rw-r--r--src/mongo/s/request_types/balance_chunk_request_test.cpp8
-rw-r--r--src/mongo/s/write_ops/batch_write_op_test.cpp6
30 files changed, 202 insertions, 268 deletions
diff --git a/src/mongo/db/pipeline/dispatch_shard_pipeline_test.cpp b/src/mongo/db/pipeline/dispatch_shard_pipeline_test.cpp
index 069a7e2f0b2..effa178fc32 100644
--- a/src/mongo/db/pipeline/dispatch_shard_pipeline_test.cpp
+++ b/src/mongo/db/pipeline/dispatch_shard_pipeline_test.cpp
@@ -175,7 +175,7 @@ TEST_F(DispatchShardPipelineTest, DispatchShardPipelineDoesNotRetryOnStaleConfig
OID epoch{OID::gen()};
Timestamp timestamp{1, 0};
return createErrorCursorResponse({StaleConfigInfo(kTestAggregateNss,
- ChunkVersion(1, 0, epoch, timestamp),
+ ChunkVersion({epoch, timestamp}, {1, 0}),
boost::none,
ShardId{"0"}),
"Mock error: shard version mismatch"});
@@ -218,7 +218,7 @@ TEST_F(DispatchShardPipelineTest, WrappedDispatchDoesRetryOnStaleConfigError) {
// namespace, then mock out a successful response.
onCommand([&](const executor::RemoteCommandRequest& request) {
return createErrorCursorResponse({StaleConfigInfo(kTestAggregateNss,
- ChunkVersion(2, 0, epoch, timestamp),
+ ChunkVersion({epoch, timestamp}, {2, 0}),
boost::none,
ShardId{"0"}),
"Mock error: shard version mismatch"});
@@ -227,7 +227,7 @@ TEST_F(DispatchShardPipelineTest, WrappedDispatchDoesRetryOnStaleConfigError) {
// Mock the expected config server queries.
const ShardKeyPattern shardKeyPattern(BSON("_id" << 1));
- ChunkVersion version(2, 0, epoch, timestamp);
+ ChunkVersion version({epoch, timestamp}, {2, 0});
ChunkType chunk1(
uuid, {shardKeyPattern.getKeyPattern().globalMin(), BSON("_id" << 0)}, version, {"0"});
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 83b35a0c9fc..30de3fbfb1f 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
@@ -27,8 +27,6 @@
* it in the license file.
*/
-#include "mongo/platform/basic.h"
-
#include "mongo/db/pipeline/aggregation_context_fixture.h"
#include "mongo/db/pipeline/process_interface/mongos_process_interface.h"
#include "mongo/unittest/unittest.h"
@@ -63,7 +61,7 @@ public:
TEST_F(MongosProcessInterfaceTest, FailsToEnsureFieldsUniqueIfTargetCollectionVersionIsSpecified) {
auto expCtx = getExpCtx();
auto targetCollectionVersion =
- boost::make_optional(ChunkVersion(0, 0, OID::gen(), Timestamp(1, 1)));
+ boost::make_optional(ChunkVersion({OID::gen(), Timestamp(1, 1)}, {0, 0}));
auto processInterface = makeProcessInterface();
ASSERT_THROWS_CODE(processInterface->ensureFieldsUniqueOrResolveDocumentKey(
diff --git a/src/mongo/db/pipeline/resharding_initial_split_policy_test.cpp b/src/mongo/db/pipeline/resharding_initial_split_policy_test.cpp
index bc17e9d0133..2df79a991d4 100644
--- a/src/mongo/db/pipeline/resharding_initial_split_policy_test.cpp
+++ b/src/mongo/db/pipeline/resharding_initial_split_policy_test.cpp
@@ -27,9 +27,6 @@
* it in the license file.
*/
-
-#include "mongo/platform/basic.h"
-
#include "mongo/db/catalog/collection_catalog.h"
#include "mongo/db/pipeline/document_source_mock.h"
#include "mongo/db/pipeline/sharded_agg_helpers.h"
@@ -39,7 +36,6 @@
#define MONGO_LOGV2_DEFAULT_COMPONENT ::mongo::logv2::LogComponent::kTest
-
namespace mongo {
namespace {
diff --git a/src/mongo/db/s/balancer/balancer_defragmentation_policy_test.cpp b/src/mongo/db/s/balancer/balancer_defragmentation_policy_test.cpp
index c42f7e86cd7..d1f431b4082 100644
--- a/src/mongo/db/s/balancer/balancer_defragmentation_policy_test.cpp
+++ b/src/mongo/db/s/balancer/balancer_defragmentation_policy_test.cpp
@@ -47,7 +47,7 @@ protected:
const ShardId kShardId1 = ShardId("shard1");
const ShardId kShardId2 = ShardId("shard2");
const ShardId kShardId3 = ShardId("shard3");
- const ChunkVersion kCollectionVersion = ChunkVersion(1, 1, OID::gen(), Timestamp(10));
+ const ChunkVersion kCollectionVersion = ChunkVersion({OID::gen(), Timestamp(10)}, {1, 1});
const KeyPattern kShardKeyPattern = KeyPattern(BSON("x" << 1));
const BSONObj kKeyAtMin = BSONObjBuilder().appendMinKey("x").obj();
const BSONObj kKeyAtZero = BSON("x" << 0);
@@ -494,7 +494,8 @@ TEST_F(BalancerDefragmentationPolicyTest, TestPhaseOneAllConsecutive) {
ChunkType chunk(
kUuid,
ChunkRange(minKey, maxKey),
- ChunkVersion(1, i, kCollectionVersion.epoch(), kCollectionVersion.getTimestamp()),
+ ChunkVersion({kCollectionVersion.epoch(), kCollectionVersion.getTimestamp()},
+ {1, uint32_t(i)}),
kShardId0);
chunkList.push_back(chunk);
}
@@ -504,7 +505,8 @@ TEST_F(BalancerDefragmentationPolicyTest, TestPhaseOneAllConsecutive) {
ChunkType chunk(
kUuid,
ChunkRange(minKey, maxKey),
- ChunkVersion(1, i, kCollectionVersion.epoch(), kCollectionVersion.getTimestamp()),
+ ChunkVersion({kCollectionVersion.epoch(), kCollectionVersion.getTimestamp()},
+ {1, uint32_t(i)}),
kShardId1);
chunkList.push_back(chunk);
}
@@ -543,7 +545,8 @@ TEST_F(BalancerDefragmentationPolicyTest, PhaseOneNotConsecutive) {
ChunkType chunk(
kUuid,
ChunkRange(minKey, maxKey),
- ChunkVersion(1, i, kCollectionVersion.epoch(), kCollectionVersion.getTimestamp()),
+ ChunkVersion({kCollectionVersion.epoch(), kCollectionVersion.getTimestamp()},
+ {1, uint32_t(i)}),
chosenShard);
chunkList.push_back(chunk);
}
@@ -620,13 +623,13 @@ TEST_F(BalancerDefragmentationPolicyTest, TestPhaseTwoChunkCanBeMovedAndMergedWi
ChunkType biggestChunk(
kUuid,
ChunkRange(kKeyAtMin, kKeyAtZero),
- ChunkVersion(1, 0, kCollectionVersion.epoch(), kCollectionVersion.getTimestamp()),
+ ChunkVersion({kCollectionVersion.epoch(), kCollectionVersion.getTimestamp()}, {1, 0}),
kShardId0);
biggestChunk.setEstimatedSizeBytes(2048);
ChunkType smallestChunk(
kUuid,
ChunkRange(kKeyAtZero, kKeyAtMax),
- ChunkVersion(1, 1, kCollectionVersion.epoch(), kCollectionVersion.getTimestamp()),
+ ChunkVersion({kCollectionVersion.epoch(), kCollectionVersion.getTimestamp()}, {1, 1}),
kShardId1);
smallestChunk.setEstimatedSizeBytes(1024);
@@ -682,42 +685,42 @@ TEST_F(BalancerDefragmentationPolicyTest,
ChunkType firstChunkOnShard0(
kUuid,
ChunkRange(kKeyAtMin, kKeyAtZero),
- ChunkVersion(1, 0, kCollectionVersion.epoch(), kCollectionVersion.getTimestamp()),
+ ChunkVersion({kCollectionVersion.epoch(), kCollectionVersion.getTimestamp()}, {1, 0}),
kShardId0);
firstChunkOnShard0.setEstimatedSizeBytes(1);
ChunkType firstChunkOnShard1(
kUuid,
ChunkRange(kKeyAtZero, kKeyAtTen),
- ChunkVersion(1, 1, kCollectionVersion.epoch(), kCollectionVersion.getTimestamp()),
+ ChunkVersion({kCollectionVersion.epoch(), kCollectionVersion.getTimestamp()}, {1, 1}),
kShardId1);
firstChunkOnShard1.setEstimatedSizeBytes(1);
ChunkType chunkOnShard2(
kUuid,
ChunkRange(kKeyAtTen, kKeyAtTwenty),
- ChunkVersion(1, 2, kCollectionVersion.epoch(), kCollectionVersion.getTimestamp()),
+ ChunkVersion({kCollectionVersion.epoch(), kCollectionVersion.getTimestamp()}, {1, 2}),
kShardId2);
chunkOnShard2.setEstimatedSizeBytes(1);
ChunkType chunkOnShard3(
kUuid,
ChunkRange(kKeyAtTwenty, kKeyAtThirty),
- ChunkVersion(1, 3, kCollectionVersion.epoch(), kCollectionVersion.getTimestamp()),
+ ChunkVersion({kCollectionVersion.epoch(), kCollectionVersion.getTimestamp()}, {1, 3}),
kShardId3);
chunkOnShard3.setEstimatedSizeBytes(1);
ChunkType secondChunkOnShard0(
kUuid,
ChunkRange(kKeyAtThirty, kKeyAtForty),
- ChunkVersion(1, 4, kCollectionVersion.epoch(), kCollectionVersion.getTimestamp()),
+ ChunkVersion({kCollectionVersion.epoch(), kCollectionVersion.getTimestamp()}, {1, 4}),
kShardId0);
secondChunkOnShard0.setEstimatedSizeBytes(1);
ChunkType secondChunkOnShard1(
kUuid,
ChunkRange(kKeyAtForty, kKeyAtMax),
- ChunkVersion(1, 5, kCollectionVersion.epoch(), kCollectionVersion.getTimestamp()),
+ ChunkVersion({kCollectionVersion.epoch(), kCollectionVersion.getTimestamp()}, {1, 5}),
kShardId1);
secondChunkOnShard1.setEstimatedSizeBytes(1);
diff --git a/src/mongo/db/s/balancer/balancer_policy_test.cpp b/src/mongo/db/s/balancer/balancer_policy_test.cpp
index fb98d610b00..be3532fee56 100644
--- a/src/mongo/db/s/balancer/balancer_policy_test.cpp
+++ b/src/mongo/db/s/balancer/balancer_policy_test.cpp
@@ -27,9 +27,6 @@
* it in the license file.
*/
-
-#include "mongo/platform/basic.h"
-
#include "mongo/db/keypattern.h"
#include "mongo/db/s/balancer/balancer_policy.h"
#include "mongo/platform/random.h"
@@ -79,7 +76,7 @@ std::pair<ShardStatisticsVector, ShardToChunksMap> generateCluster(
int64_t currentChunk = 0;
- ChunkVersion chunkVersion(1, 0, OID::gen(), Timestamp(1, 1));
+ ChunkVersion chunkVersion({OID::gen(), Timestamp(1, 1)}, {1, 0});
const UUID uuid = UUID::gen();
const KeyPattern shardKeyPattern(BSON("x" << 1));
diff --git a/src/mongo/db/s/collection_sharding_runtime_test.cpp b/src/mongo/db/s/collection_sharding_runtime_test.cpp
index dcee5b73ac0..c6985aa5742 100644
--- a/src/mongo/db/s/collection_sharding_runtime_test.cpp
+++ b/src/mongo/db/s/collection_sharding_runtime_test.cpp
@@ -27,8 +27,6 @@
* it in the license file.
*/
-#include "mongo/platform/basic.h"
-
#include "boost/optional/optional_io.hpp"
#include "mongo/db/catalog/create_collection.h"
#include "mongo/db/catalog_raii.h"
@@ -63,7 +61,7 @@ protected:
const Timestamp timestamp(1, 1);
auto range = ChunkRange(BSON(kShardKey << MINKEY), BSON(kShardKey << MAXKEY));
auto chunk = ChunkType(
- uuid, std::move(range), ChunkVersion(1, 0, epoch, timestamp), ShardId("other"));
+ uuid, std::move(range), ChunkVersion({epoch, timestamp}, {1, 0}), ShardId("other"));
ChunkManager cm(ShardId("0"),
DatabaseVersion(UUID::gen(), timestamp),
makeStandaloneRoutingTableHistory(
@@ -218,8 +216,8 @@ TEST_F(CollectionShardingRuntimeTest, ReturnUnshardedMetadataInServerlessMode) {
ScopedSetShardRole scopedSetShardRole2{
opCtx,
NamespaceString::kLogicalSessionsNamespace,
- ChunkVersion(1, 0, OID::gen(), Timestamp(1, 1)), /* shardVersion */
- boost::none /* databaseVersion */
+ ChunkVersion({OID::gen(), Timestamp(1, 1)}, {1, 0}), /* shardVersion */
+ boost::none /* databaseVersion */
};
CollectionShardingRuntime csrLogicalSession(
@@ -324,11 +322,11 @@ public:
const Timestamp& timestamp) {
auto range1 = ChunkRange(BSON(kShardKey << MINKEY), BSON(kShardKey << 5));
ChunkType chunk1(
- uuid, range1, ChunkVersion(1, 0, epoch, timestamp), kShardList[0].getName());
+ uuid, range1, ChunkVersion({epoch, timestamp}, {1, 0}), kShardList[0].getName());
auto range2 = ChunkRange(BSON(kShardKey << 5), BSON(kShardKey << MAXKEY));
ChunkType chunk2(
- uuid, range2, ChunkVersion(1, 1, epoch, timestamp), kShardList[0].getName());
+ uuid, range2, ChunkVersion({epoch, timestamp}, {1, 1}), kShardList[0].getName());
return {chunk1, chunk2};
}
diff --git a/src/mongo/db/s/config/sharding_catalog_manager_bump_collection_version_and_change_metadata_test.cpp b/src/mongo/db/s/config/sharding_catalog_manager_bump_collection_version_and_change_metadata_test.cpp
index a4abd0ff45b..fbb502f933b 100644
--- a/src/mongo/db/s/config/sharding_catalog_manager_bump_collection_version_and_change_metadata_test.cpp
+++ b/src/mongo/db/s/config/sharding_catalog_manager_bump_collection_version_and_change_metadata_test.cpp
@@ -27,8 +27,6 @@
* it in the license file.
*/
-#include "mongo/platform/basic.h"
-
#include "mongo/db/concurrency/exception_util.h"
#include "mongo/db/dbdirectclient.h"
#include "mongo/db/logical_session_cache_noop.h"
@@ -43,7 +41,6 @@
#define MONGO_LOGV2_DEFAULT_COMPONENT ::mongo::logv2::LogComponent::kTest
-
namespace mongo {
namespace {
@@ -121,17 +118,17 @@ TEST_F(ShardingCatalogManagerBumpCollectionVersionAndChangeMetadataTest,
const auto collUUID = UUID::gen();
const auto shard0Chunk0 = generateChunkType(collUUID,
- ChunkVersion(10, 1, collEpoch, collTimestamp),
+ ChunkVersion({collEpoch, collTimestamp}, {10, 1}),
kShard0.getName(),
BSON("a" << 1),
BSON("a" << 10));
const auto shard0Chunk1 = generateChunkType(collUUID,
- ChunkVersion(11, 2, collEpoch, collTimestamp),
+ ChunkVersion({collEpoch, collTimestamp}, {11, 2}),
kShard0.getName(),
BSON("a" << 11),
BSON("a" << 20));
const auto shard1Chunk0 = generateChunkType(collUUID,
- ChunkVersion(8, 1, collEpoch, collTimestamp),
+ ChunkVersion({collEpoch, collTimestamp}, {8, 1}),
kShard1.getName(),
BSON("a" << 21),
BSON("a" << 100));
@@ -157,7 +154,7 @@ TEST_F(ShardingCatalogManagerBumpCollectionVersionAndChangeMetadataTest, NoChunk
const auto collUUID = UUID::gen();
const auto shard0Chunk0 = generateChunkType(collUUID,
- ChunkVersion(10, 1, collEpoch, collTimestamp),
+ ChunkVersion({collEpoch, collTimestamp}, {10, 1}),
kShard0.getName(),
BSON("a" << 1),
BSON("a" << 10));
@@ -182,12 +179,12 @@ TEST_F(ShardingCatalogManagerBumpCollectionVersionAndChangeMetadataTest,
const auto collUUID = UUID::gen();
const auto shard0Chunk0 = generateChunkType(collUUID,
- ChunkVersion(10, 1, collEpoch, collTimestamp),
+ ChunkVersion({collEpoch, collTimestamp}, {10, 1}),
kShard0.getName(),
BSON("a" << 1),
BSON("a" << 10));
const auto shard1Chunk0 = generateChunkType(collUUID,
- ChunkVersion(11, 2, collEpoch, collTimestamp),
+ ChunkVersion({collEpoch, collTimestamp}, {11, 2}),
kShard1.getName(),
BSON("a" << 11),
BSON("a" << 20));
@@ -244,12 +241,12 @@ TEST_F(ShardingCatalogManagerBumpCollectionVersionAndChangeMetadataTest,
const auto collUUID = UUID::gen();
const auto shard0Chunk0 = generateChunkType(collUUID,
- ChunkVersion(10, 1, collEpoch, collTimestamp),
+ ChunkVersion({collEpoch, collTimestamp}, {10, 1}),
kShard0.getName(),
BSON("a" << 1),
BSON("a" << 10));
const auto shard1Chunk0 = generateChunkType(collUUID,
- ChunkVersion(11, 2, collEpoch, collTimestamp),
+ ChunkVersion({collEpoch, collTimestamp}, {11, 2}),
kShard1.getName(),
BSON("a" << 11),
BSON("a" << 20));
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 83daf277078..f461f1ae0a5 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
@@ -27,9 +27,6 @@
* it in the license file.
*/
-
-#include "mongo/platform/basic.h"
-
#include "mongo/db/s/config/sharding_catalog_manager.h"
#include "mongo/base/status_with.h"
@@ -67,7 +64,6 @@
#define MONGO_LOGV2_DEFAULT_COMPONENT ::mongo::logv2::LogComponent::kSharding
-
namespace mongo {
namespace {
@@ -268,7 +264,8 @@ ChunkVersion getShardVersion(OperationContext* opCtx,
if (swDonorShardVersion.getStatus().code() == 50577) {
// The query to find 'nss' chunks belonging to the donor shard didn't return any chunks,
// meaning the last chunk for fromShard was donated. Gracefully handle the error.
- return ChunkVersion(0, 0, collectionVersion.epoch(), collectionVersion.getTimestamp());
+ return ChunkVersion({collectionVersion.epoch(), collectionVersion.getTimestamp()},
+ {0, 0});
} else {
// Bubble up any other error
uassertStatusOK(swDonorShardVersion);
@@ -391,10 +388,9 @@ void ShardingCatalogManager::bumpMajorVersionOneChunkPerShard(
TxnNumber txnNumber,
const std::vector<ShardId>& shardIds) {
auto curCollectionVersion = uassertStatusOK(getCollectionVersion(opCtx, nss));
- ChunkVersion targetChunkVersion(curCollectionVersion.majorVersion() + 1,
- 0,
- curCollectionVersion.epoch(),
- curCollectionVersion.getTimestamp());
+ ChunkVersion targetChunkVersion(
+ {curCollectionVersion.epoch(), curCollectionVersion.getTimestamp()},
+ {curCollectionVersion.majorVersion() + 1, 0});
auto const configShard = Grid::get(opCtx)->shardRegistry()->getConfigShard();
auto findCollResponse = uassertStatusOK(
@@ -1127,10 +1123,9 @@ StatusWith<BSONObj> ShardingCatalogManager::commitChunkMigration(
newMigratedChunk->setMin(migratedChunk.getMin());
newMigratedChunk->setMax(migratedChunk.getMax());
newMigratedChunk->setShard(toShard);
- newMigratedChunk->setVersion(ChunkVersion(currentCollectionVersion.majorVersion() + 1,
- minVersionIncrement++,
- currentCollectionVersion.epoch(),
- currentCollectionVersion.getTimestamp()));
+ newMigratedChunk->setVersion(
+ ChunkVersion({currentCollectionVersion.epoch(), currentCollectionVersion.getTimestamp()},
+ {currentCollectionVersion.majorVersion() + 1, minVersionIncrement++}));
// Copy the complete history.
auto newHistory = currentChunk.getHistory();
@@ -1186,10 +1181,9 @@ StatusWith<BSONObj> ShardingCatalogManager::commitChunkMigration(
ChunkType leftSplitChunk = currentChunk;
leftSplitChunk.setName(OID::gen());
leftSplitChunk.setMax(movedChunkMin);
- leftSplitChunk.setVersion(ChunkVersion(movedChunkVersion.majorVersion(),
- minVersionIncrement++,
- movedChunkVersion.epoch(),
- movedChunkVersion.getTimestamp()));
+ leftSplitChunk.setVersion(
+ ChunkVersion({movedChunkVersion.epoch(), movedChunkVersion.getTimestamp()},
+ {movedChunkVersion.majorVersion(), minVersionIncrement++}));
newSplitChunks->emplace_back(std::move(leftSplitChunk));
}
@@ -1199,10 +1193,9 @@ StatusWith<BSONObj> ShardingCatalogManager::commitChunkMigration(
ChunkType rightSplitChunk = currentChunk;
rightSplitChunk.setName(OID::gen());
rightSplitChunk.setMin(movedChunkMax);
- rightSplitChunk.setVersion(ChunkVersion(movedChunkVersion.majorVersion(),
- minVersionIncrement++,
- movedChunkVersion.epoch(),
- movedChunkVersion.getTimestamp()));
+ rightSplitChunk.setVersion(
+ ChunkVersion({movedChunkVersion.epoch(), movedChunkVersion.getTimestamp()},
+ {movedChunkVersion.majorVersion(), minVersionIncrement++}));
newSplitChunks->emplace_back(std::move(rightSplitChunk));
}
}
@@ -1218,10 +1211,9 @@ StatusWith<BSONObj> ShardingCatalogManager::commitChunkMigration(
newControlChunk = std::make_shared<ChunkType>(origControlChunk);
// Setting control chunk's minor version to 1 on the donor shard.
- newControlChunk->setVersion(ChunkVersion(currentCollectionVersion.majorVersion() + 1,
- minVersionIncrement++,
- currentCollectionVersion.epoch(),
- currentCollectionVersion.getTimestamp()));
+ newControlChunk->setVersion(ChunkVersion(
+ {currentCollectionVersion.epoch(), currentCollectionVersion.getTimestamp()},
+ {currentCollectionVersion.majorVersion() + 1, minVersionIncrement++}));
}
_commitChunkMigrationInTransaction(
@@ -1232,7 +1224,7 @@ StatusWith<BSONObj> ShardingCatalogManager::commitChunkMigration(
// We migrated the last chunk from the donor shard.
newMigratedChunk->getVersion().serializeToBSON(kCollectionVersionField, &response);
const ChunkVersion donorShardVersion(
- 0, 0, currentCollectionVersion.epoch(), currentCollectionVersion.getTimestamp());
+ {currentCollectionVersion.epoch(), currentCollectionVersion.getTimestamp()}, {0, 0});
donorShardVersion.serializeToBSON(ChunkVersion::kShardVersionField, &response);
} else {
newControlChunk->getVersion().serializeToBSON(kCollectionVersionField, &response);
@@ -1349,8 +1341,8 @@ void ShardingCatalogManager::upgradeChunksHistory(OperationContext* opCtx,
}();
// Bump the major version in order to be guaranteed to trigger refresh on every shard
- ChunkVersion newCollectionVersion(
- collVersion.majorVersion() + 1, 0, collVersion.epoch(), collVersion.getTimestamp());
+ ChunkVersion newCollectionVersion({collVersion.epoch(), collVersion.getTimestamp()},
+ {collVersion.majorVersion() + 1, 0});
std::set<ShardId> changedShardIds;
for (const auto& chunk : allChunksVector) {
auto upgradeChunk = uassertStatusOK(
@@ -1491,10 +1483,9 @@ void ShardingCatalogManager::clearJumboFlag(OperationContext* opCtx,
<< chunk.toString() << ").",
currentCollectionVersion.epoch() == collectionEpoch);
- ChunkVersion newVersion(currentCollectionVersion.majorVersion() + 1,
- 0,
- currentCollectionVersion.epoch(),
- currentCollectionVersion.getTimestamp());
+ ChunkVersion newVersion(
+ {currentCollectionVersion.epoch(), currentCollectionVersion.getTimestamp()},
+ {currentCollectionVersion.majorVersion() + 1, 0});
BSONObj chunkQuery(BSON(ChunkType::min(chunk.getMin())
<< ChunkType::max(chunk.getMax()) << ChunkType::collectionUUID
@@ -1653,8 +1644,8 @@ void ShardingCatalogManager::ensureChunkVersionIsGreaterThan(OperationContext* o
// Generate a new version for the chunk by incrementing the collectionVersion's major
// version.
auto newChunk = matchingChunk;
- newChunk.setVersion(ChunkVersion(
- highestChunk.getVersion().majorVersion() + 1, 0, coll.getEpoch(), coll.getTimestamp()));
+ newChunk.setVersion(ChunkVersion({coll.getEpoch(), coll.getTimestamp()},
+ {highestChunk.getVersion().majorVersion() + 1, 0}));
// Update the chunk, if it still exists, to have the bumped version.
earlyReturnBeforeDoingWriteGuard.dismiss();
diff --git a/src/mongo/db/s/config/sharding_catalog_manager_commit_chunk_migration_test.cpp b/src/mongo/db/s/config/sharding_catalog_manager_commit_chunk_migration_test.cpp
index 8452740c17e..fc8a55a9635 100644
--- a/src/mongo/db/s/config/sharding_catalog_manager_commit_chunk_migration_test.cpp
+++ b/src/mongo/db/s/config/sharding_catalog_manager_commit_chunk_migration_test.cpp
@@ -27,8 +27,6 @@
* it in the license file.
*/
-#include "mongo/platform/basic.h"
-
#include "mongo/bson/bsonobj.h"
#include "mongo/bson/bsonobjbuilder.h"
#include "mongo/client/read_preference.h"
@@ -49,7 +47,6 @@
#define MONGO_LOGV2_DEFAULT_COMPONENT ::mongo::logv2::LogComponent::kTest
-
namespace mongo {
namespace {
@@ -104,7 +101,7 @@ TEST_F(CommitChunkMigrate, ChunksUpdatedCorrectly) {
ChunkType migratedChunk, controlChunk;
{
- ChunkVersion origVersion(12, 7, collEpoch, collTimestamp);
+ ChunkVersion origVersion({collEpoch, collTimestamp}, {12, 7});
migratedChunk.setName(OID::gen());
migratedChunk.setCollectionUUID(collUUID);
@@ -141,10 +138,9 @@ TEST_F(CommitChunkMigrate, ChunksUpdatedCorrectly) {
// Verify the versions returned match expected values.
auto mver = ChunkVersion::parse(versions["shardVersion"]);
- ASSERT_EQ(ChunkVersion(migratedChunk.getVersion().majorVersion() + 1,
- 1,
- migratedChunk.getVersion().epoch(),
- migratedChunk.getVersion().getTimestamp()),
+ ASSERT_EQ(ChunkVersion(
+ {migratedChunk.getVersion().epoch(), migratedChunk.getVersion().getTimestamp()},
+ {migratedChunk.getVersion().majorVersion() + 1, 1}),
mver);
// Verify that a collection version is returned
@@ -188,8 +184,8 @@ TEST_F(CommitChunkMigrate, ChunksUpdatedCorrectlyWithoutControlChunk) {
setupShards({shard0, shard1});
- int origMajorVersion = 15;
- auto const origVersion = ChunkVersion(origMajorVersion, 4, collEpoch, collTimestamp);
+ uint32_t origMajorVersion = 15;
+ auto const origVersion = ChunkVersion({collEpoch, collTimestamp}, {origMajorVersion, 4});
ChunkType chunk0;
chunk0.setName(OID::gen());
@@ -223,11 +219,11 @@ TEST_F(CommitChunkMigrate, ChunksUpdatedCorrectlyWithoutControlChunk) {
// Verify the version returned matches expected value.
BSONObj versions = resultBSON.getValue();
auto mver = ChunkVersion::parse(versions["shardVersion"]);
- ASSERT_EQ(ChunkVersion(0, 0, origVersion.epoch(), origVersion.getTimestamp()), mver);
+ ASSERT_EQ(ChunkVersion({origVersion.epoch(), origVersion.getTimestamp()}, {0, 0}), mver);
// Verify that a collection version is returned
auto cver = ChunkVersion::parse(versions["collectionVersion"]);
- ASSERT_EQ(ChunkVersion(origMajorVersion + 1, 0, collEpoch, collTimestamp), cver);
+ ASSERT_EQ(ChunkVersion({collEpoch, collTimestamp}, {origMajorVersion + 1, 0}), cver);
// Verify the chunk ended up in the right shard.
auto chunkDoc0 =
@@ -253,8 +249,8 @@ TEST_F(CommitChunkMigrate, CheckCorrectOpsCommandNoCtlTrimHistory) {
setupShards({shard0, shard1});
- int origMajorVersion = 15;
- auto const origVersion = ChunkVersion(origMajorVersion, 4, collEpoch, collTimestamp);
+ uint32_t origMajorVersion = 15;
+ auto const origVersion = ChunkVersion({collEpoch, collTimestamp}, {origMajorVersion, 4});
ChunkType chunk0;
chunk0.setName(OID::gen());
@@ -289,7 +285,7 @@ TEST_F(CommitChunkMigrate, CheckCorrectOpsCommandNoCtlTrimHistory) {
// Verify the version returned matches expected value.
BSONObj versions = resultBSON.getValue();
auto mver = ChunkVersion::parse(versions["shardVersion"]);
- ASSERT_EQ(ChunkVersion(0, 0, origVersion.epoch(), origVersion.getTimestamp()), mver);
+ ASSERT_EQ(ChunkVersion({origVersion.epoch(), origVersion.getTimestamp()}, {0, 0}), mver);
// Verify the chunk ended up in the right shard.
auto chunkDoc0 =
@@ -314,9 +310,8 @@ TEST_F(CommitChunkMigrate, RejectOutOfOrderHistory) {
setupShards({shard0, shard1});
- int origMajorVersion = 15;
- auto const origVersion =
- ChunkVersion(origMajorVersion, 4, OID::gen(), Timestamp(42) /* timestamp */);
+ uint32_t origMajorVersion = 15;
+ auto const origVersion = ChunkVersion({OID::gen(), Timestamp(42)}, {origMajorVersion, 4});
ChunkType chunk0;
chunk0.setName(OID::gen());
@@ -362,9 +357,8 @@ TEST_F(CommitChunkMigrate, RejectWrongCollectionEpoch0) {
setupShards({shard0, shard1});
- int origMajorVersion = 12;
- auto const origVersion =
- ChunkVersion(origMajorVersion, 7, OID::gen(), Timestamp(42) /* timestamp */);
+ uint32_t origMajorVersion = 12;
+ auto const origVersion = ChunkVersion({OID::gen(), Timestamp(42)}, {origMajorVersion, 7});
ChunkType chunk0;
chunk0.setName(OID::gen());
@@ -418,11 +412,9 @@ TEST_F(CommitChunkMigrate, RejectWrongCollectionEpoch1) {
setupShards({shard0, shard1});
- int origMajorVersion = 12;
- auto const origVersion =
- ChunkVersion(origMajorVersion, 7, OID::gen(), Timestamp(42) /* timestamp */);
- auto const otherVersion =
- ChunkVersion(origMajorVersion, 7, OID::gen(), Timestamp(42) /* timestamp */);
+ uint32_t origMajorVersion = 12;
+ auto const origVersion = ChunkVersion({OID::gen(), Timestamp(42)}, {origMajorVersion, 7});
+ auto const otherVersion = ChunkVersion({OID::gen(), Timestamp(42)}, {origMajorVersion, 7});
ChunkType chunk0;
chunk0.setName(OID::gen());
@@ -479,8 +471,8 @@ TEST_F(CommitChunkMigrate, CommitWithLastChunkOnShardShouldNotAffectOtherChunks)
setupShards({shard0, shard1});
- int origMajorVersion = 12;
- auto const origVersion = ChunkVersion(origMajorVersion, 7, collEpoch, collTimestamp);
+ uint32_t origMajorVersion = 12;
+ auto const origVersion = ChunkVersion({collEpoch, collTimestamp}, {origMajorVersion, 7});
ChunkType chunk0;
chunk0.setName(OID::gen());
@@ -526,7 +518,7 @@ TEST_F(CommitChunkMigrate, CommitWithLastChunkOnShardShouldNotAffectOtherChunks)
// Verify the versions returned match expected values.
BSONObj versions = resultBSON.getValue();
auto mver = ChunkVersion::parse(versions["shardVersion"]);
- ASSERT_EQ(ChunkVersion(0, 0, origVersion.epoch(), origVersion.getTimestamp()), mver);
+ ASSERT_EQ(ChunkVersion({origVersion.epoch(), origVersion.getTimestamp()}, {0, 0}), mver);
// Verify the chunks ended up in the right shards.
auto chunkDoc0 =
@@ -560,7 +552,7 @@ TEST_F(CommitChunkMigrate, RejectMissingChunkVersion) {
setupShards({shard0, shard1});
- ChunkVersion origVersion(12, 7, OID::gen(), Timestamp(42) /* timestamp */);
+ ChunkVersion origVersion({OID::gen(), Timestamp(42)}, {12, 7});
// Create migrate chunk with no chunk version set.
ChunkType migratedChunk;
@@ -610,7 +602,7 @@ TEST_F(CommitChunkMigrate, RejectOlderChunkVersion) {
setupShards({shard0, shard1});
auto epoch = OID::gen();
- ChunkVersion origVersion(12, 7, epoch, Timestamp(42) /* timestamp */);
+ ChunkVersion origVersion({epoch, Timestamp(42)}, {12, 7});
ChunkType migratedChunk;
migratedChunk.setName(OID::gen());
@@ -621,7 +613,7 @@ TEST_F(CommitChunkMigrate, RejectOlderChunkVersion) {
migratedChunk.setMin(BSON("a" << 1));
migratedChunk.setMax(BSON("a" << 10));
- ChunkVersion currentChunkVersion(14, 7, epoch, Timestamp(42) /* timestamp */);
+ ChunkVersion currentChunkVersion({epoch, Timestamp(42)}, {14, 7});
ChunkType currentChunk;
currentChunk.setName(OID::gen());
@@ -662,7 +654,7 @@ TEST_F(CommitChunkMigrate, RejectMismatchedEpoch) {
setupShards({shard0, shard1});
- ChunkVersion origVersion(12, 7, OID::gen(), Timestamp(42) /* timestamp */);
+ ChunkVersion origVersion({OID::gen(), Timestamp(42)}, {12, 7});
ChunkType migratedChunk;
migratedChunk.setName(OID::gen());
@@ -673,7 +665,7 @@ TEST_F(CommitChunkMigrate, RejectMismatchedEpoch) {
migratedChunk.setMin(BSON("a" << 1));
migratedChunk.setMax(BSON("a" << 10));
- ChunkVersion currentChunkVersion(12, 7, OID::gen(), Timestamp(42) /* timestamp */);
+ ChunkVersion currentChunkVersion({OID::gen(), Timestamp(42)}, {12, 7});
ChunkType currentChunk;
currentChunk.setName(OID::gen());
@@ -730,7 +722,7 @@ public:
void setupCollectionWithNChunks(int numberOfChunks) {
invariant(numberOfChunks > 0);
- int currentMajorVersion = 1;
+ uint32_t currentMajorVersion = 1;
int historyTimestampSecond = 100;
std::vector<ChunkHistory> history;
@@ -745,7 +737,7 @@ public:
const auto max = chunksMin.at(i + 1); // Max key of the chunk being created
const auto shardId = _shardIds.at(i % 2); // Shard owning the chunk
ChunkVersion version =
- ChunkVersion(currentMajorVersion++, 0, _collEpoch, _collTimestamp);
+ ChunkVersion({_collEpoch, _collTimestamp}, {currentMajorVersion++, 0});
history.insert(history.begin(),
{ChunkHistory(Timestamp(historyTimestampSecond++, 0), shardId)});
ChunkType chunk = createChunk(_collUUID, min, max, version, shardId, history);
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 32544cacc7b..b54338947b1 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
@@ -27,9 +27,6 @@
* it in the license file.
*/
-
-#include "mongo/platform/basic.h"
-
#include <string>
#include <vector>
@@ -58,7 +55,6 @@
#define MONGO_LOGV2_DEFAULT_COMPONENT ::mongo::logv2::LogComponent::kSharding
-
namespace mongo {
namespace {
@@ -227,15 +223,15 @@ TEST_F(RemoveShardTest, RemoveShardStillDrainingChunksRemaining) {
const auto timestamp = Timestamp(1);
ChunkType chunk1(uuid,
ChunkRange(BSON("_id" << 0), BSON("_id" << 20)),
- ChunkVersion(1, 1, epoch, timestamp),
+ ChunkVersion({epoch, timestamp}, {1, 1}),
shard1.getName());
ChunkType chunk2(uuid,
ChunkRange(BSON("_id" << 21), BSON("_id" << 50)),
- ChunkVersion(1, 2, epoch, timestamp),
+ ChunkVersion({epoch, timestamp}, {1, 2}),
shard1.getName());
ChunkType chunk3(uuid,
ChunkRange(BSON("_id" << 51), BSON("_id" << 1000)),
- ChunkVersion(1, 3, epoch, timestamp),
+ ChunkVersion({epoch, timestamp}, {1, 3}),
shard1.getName());
chunk3.setJumbo(true);
@@ -314,15 +310,15 @@ TEST_F(RemoveShardTest, RemoveShardCompletion) {
Timestamp timestamp = Timestamp(1);
ChunkType chunk1(uuid,
ChunkRange(BSON("_id" << 0), BSON("_id" << 20)),
- ChunkVersion(1, 1, epoch, timestamp),
+ ChunkVersion({epoch, timestamp}, {1, 1}),
shard1.getName());
ChunkType chunk2(uuid,
ChunkRange(BSON("_id" << 21), BSON("_id" << 50)),
- ChunkVersion(1, 2, epoch, timestamp),
+ ChunkVersion({epoch, timestamp}, {1, 2}),
shard1.getName());
ChunkType chunk3(uuid,
ChunkRange(BSON("_id" << 51), BSON("_id" << 1000)),
- ChunkVersion(1, 3, epoch, timestamp),
+ ChunkVersion({epoch, timestamp}, {1, 3}),
shard1.getName());
std::vector<ChunkType> chunks{chunk1, chunk2, chunk3};
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 561bcbb713a..dc1eb4579e5 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
@@ -179,7 +179,7 @@ protected:
true,
{ChunkType{uuid,
ChunkRange{BSON(kShardKey << MINKEY), BSON(kShardKey << MAXKEY)},
- ChunkVersion(1, 0, epoch, timestamp),
+ ChunkVersion({epoch, timestamp}, {1, 0}),
ShardId("dummyShardId")}});
AutoGetDb autoDb(operationContext(), kNss.db(), MODE_IX);
diff --git a/src/mongo/db/s/migration_util_test.cpp b/src/mongo/db/s/migration_util_test.cpp
index 8e6f02043da..90a1e9016a1 100644
--- a/src/mongo/db/s/migration_util_test.cpp
+++ b/src/mongo/db/s/migration_util_test.cpp
@@ -27,8 +27,6 @@
* it in the license file.
*/
-#include "mongo/platform/basic.h"
-
#include "mongo/client/remote_command_targeter_factory_mock.h"
#include "mongo/client/remote_command_targeter_mock.h"
#include "mongo/db/catalog_raii.h"
@@ -591,7 +589,7 @@ TEST_F(SubmitRangeDeletionTaskTest, SucceedsIfFilteringMetadataUUIDMatchesTaskUU
_mockCatalogCacheLoader->setDatabaseRefreshReturnValue(kDefaultDatabaseType);
_mockCatalogCacheLoader->setCollectionRefreshReturnValue(coll);
_mockCatalogCacheLoader->setChunkRefreshReturnValue(
- makeChangedChunks(ChunkVersion(1, 0, kEpoch, kDefaultTimestamp)));
+ makeChangedChunks(ChunkVersion({kEpoch, kDefaultTimestamp}, {1, 0})));
_mockCatalogClient->setCollections({coll});
forceShardFilteringMetadataRefresh(opCtx, kTestNss);
@@ -619,7 +617,7 @@ TEST_F(
_mockCatalogCacheLoader->setDatabaseRefreshReturnValue(kDefaultDatabaseType);
_mockCatalogCacheLoader->setCollectionRefreshReturnValue(coll);
_mockCatalogCacheLoader->setChunkRefreshReturnValue(
- makeChangedChunks(ChunkVersion(1, 0, kEpoch, kDefaultTimestamp)));
+ makeChangedChunks(ChunkVersion({kEpoch, kDefaultTimestamp}, {1, 0})));
_mockCatalogClient->setCollections({coll});
auto metadata = makeShardedMetadata(opCtx, collectionUUID);
@@ -654,7 +652,7 @@ TEST_F(SubmitRangeDeletionTaskTest,
auto matchingColl = makeCollectionType(collectionUUID, kEpoch, kDefaultTimestamp);
_mockCatalogCacheLoader->setCollectionRefreshReturnValue(matchingColl);
_mockCatalogCacheLoader->setChunkRefreshReturnValue(
- makeChangedChunks(ChunkVersion(10, 0, kEpoch, kDefaultTimestamp)));
+ makeChangedChunks(ChunkVersion({kEpoch, kDefaultTimestamp}, {10, 0})));
_mockCatalogClient->setCollections({matchingColl});
auto metadata = makeShardedMetadata(opCtx, collectionUUID);
@@ -684,7 +682,7 @@ TEST_F(SubmitRangeDeletionTaskTest,
_mockCatalogCacheLoader->setDatabaseRefreshReturnValue(kDefaultDatabaseType);
_mockCatalogCacheLoader->setCollectionRefreshReturnValue(otherColl);
_mockCatalogCacheLoader->setChunkRefreshReturnValue(
- makeChangedChunks(ChunkVersion(1, 0, otherEpoch, otherTimestamp)));
+ makeChangedChunks(ChunkVersion({otherEpoch, otherTimestamp}, {1, 0})));
_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 1ef41426f26..2c4859ba782 100644
--- a/src/mongo/db/s/op_observer_sharding_test.cpp
+++ b/src/mongo/db/s/op_observer_sharding_test.cpp
@@ -27,8 +27,6 @@
* it in the license file.
*/
-#include "mongo/platform/basic.h"
-
#include "mongo/db/catalog_raii.h"
#include "mongo/db/op_observer_util.h"
#include "mongo/db/s/collection_sharding_runtime.h"
@@ -59,8 +57,10 @@ 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, Timestamp(1, 1)), ShardId("other"));
+ auto chunk = ChunkType(uuid,
+ std::move(range),
+ ChunkVersion({epoch, Timestamp(1, 1)}, {1, 0}),
+ ShardId("other"));
auto rt = RoutingTableHistory::makeNew(kTestNss,
uuid,
KeyPattern(keyPattern),
diff --git a/src/mongo/db/s/range_deletion_util_test.cpp b/src/mongo/db/s/range_deletion_util_test.cpp
index 6efd33ce9d6..567d50748df 100644
--- a/src/mongo/db/s/range_deletion_util_test.cpp
+++ b/src/mongo/db/s/range_deletion_util_test.cpp
@@ -27,8 +27,6 @@
* it in the license file.
*/
-#include "mongo/platform/basic.h"
-
#include "mongo/db/catalog/create_collection.h"
#include "mongo/db/db_raii.h"
#include "mongo/db/dbdirectclient.h"
@@ -113,7 +111,7 @@ public:
true,
{ChunkType{uuid,
ChunkRange{BSON(kShardKey << MINKEY), BSON(kShardKey << MAXKEY)},
- ChunkVersion(1, 0, epoch, Timestamp(1, 1)),
+ ChunkVersion({epoch, Timestamp(1, 1)}, {1, 0}),
ShardId("dummyShardId")}});
ChunkManager cm(ShardId("dummyShardId"),
DatabaseVersion(UUID::gen(), Timestamp(1, 1)),
diff --git a/src/mongo/db/s/resharding/resharding_coordinator_service.cpp b/src/mongo/db/s/resharding/resharding_coordinator_service.cpp
index 4f905f80f51..26ba0393ccc 100644
--- a/src/mongo/db/s/resharding/resharding_coordinator_service.cpp
+++ b/src/mongo/db/s/resharding/resharding_coordinator_service.cpp
@@ -27,9 +27,6 @@
* it in the license file.
*/
-
-#include "mongo/platform/basic.h"
-
#include "mongo/db/s/resharding/resharding_coordinator_service.h"
#include "mongo/base/string_data.h"
@@ -79,7 +76,6 @@
#define MONGO_LOGV2_DEFAULT_COMPONENT ::mongo::logv2::LogComponent::kResharding
-
namespace mongo {
namespace {
@@ -857,7 +853,7 @@ ChunkVersion ReshardingCoordinatorExternalState::calculateChunkVersionForInitial
OperationContext* opCtx) {
const auto now = VectorClock::get(opCtx)->getTime();
const auto timestamp = now.clusterTime().asTimestamp();
- return ChunkVersion(1, 0, OID::gen(), timestamp);
+ return ChunkVersion({OID::gen(), timestamp}, {1, 0});
}
std::vector<DonorShardEntry> constructDonorShardEntries(const std::set<ShardId>& donorShardIds) {
diff --git a/src/mongo/db/s/resharding/resharding_coordinator_test.cpp b/src/mongo/db/s/resharding/resharding_coordinator_test.cpp
index 4a8a994f144..35ffa75b31a 100644
--- a/src/mongo/db/s/resharding/resharding_coordinator_test.cpp
+++ b/src/mongo/db/s/resharding/resharding_coordinator_test.cpp
@@ -27,10 +27,6 @@
* it in the license file.
*/
-#include "mongo/db/s/resharding/coordinator_document_gen.h"
-
-#include "mongo/platform/basic.h"
-
#include <boost/optional.hpp>
#include "mongo/client/remote_command_targeter_mock.h"
@@ -40,6 +36,7 @@
#include "mongo/db/repl/storage_interface_mock.h"
#include "mongo/db/s/config/config_server_test_fixture.h"
#include "mongo/db/s/config/index_on_config.h"
+#include "mongo/db/s/resharding/coordinator_document_gen.h"
#include "mongo/db/s/resharding/resharding_coordinator_service.h"
#include "mongo/db/s/resharding/resharding_util.h"
#include "mongo/db/s/transaction_coordinator_service.h"
@@ -52,7 +49,6 @@
#define MONGO_LOGV2_DEFAULT_COMPONENT ::mongo::logv2::LogComponent::kTest
-
namespace mongo {
namespace {
@@ -179,7 +175,7 @@ protected:
_newShardKey.isShardKey(shardKey.toBSON()) ? _newChunkRanges : _oldChunkRanges;
// Create two chunks, one on each shard with the given namespace and epoch
- ChunkVersion version(1, 0, epoch, Timestamp(1, 2));
+ ChunkVersion version({epoch, Timestamp(1, 2)}, {1, 0});
ChunkType chunk1(uuid, chunkRanges[0], version, ShardId("shard0000"));
chunk1.setName(ids[0]);
version.incMinor();
@@ -226,7 +222,7 @@ protected:
client.insert(CollectionType::ConfigNS.ns(), originalNssCatalogEntry.toBSON());
auto tempNssCatalogEntry = createTempReshardingCollectionType(
- opCtx, coordinatorDoc, ChunkVersion(1, 1, OID::gen(), Timestamp(1, 2)), BSONObj());
+ opCtx, coordinatorDoc, ChunkVersion({OID::gen(), Timestamp(1, 2)}, {1, 1}), BSONObj());
client.insert(CollectionType::ConfigNS.ns(), tempNssCatalogEntry.toBSON());
return coordinatorDoc;
@@ -518,11 +514,11 @@ protected:
// collection should have been removed.
boost::optional<CollectionType> expectedTempCollType = boost::none;
if (expectedCoordinatorDoc.getState() < CoordinatorStateEnum::kCommitting) {
- expectedTempCollType =
- createTempReshardingCollectionType(opCtx,
- expectedCoordinatorDoc,
- ChunkVersion(1, 1, OID::gen(), Timestamp(1, 2)),
- BSONObj());
+ expectedTempCollType = createTempReshardingCollectionType(
+ opCtx,
+ expectedCoordinatorDoc,
+ ChunkVersion({OID::gen(), Timestamp(1, 2)}, {1, 1}),
+ BSONObj());
// It's necessary to add the userCanceled field because the call into
// createTempReshardingCollectionType assumes that the collection entry is
diff --git a/src/mongo/db/s/resharding/resharding_destined_recipient_test.cpp b/src/mongo/db/s/resharding/resharding_destined_recipient_test.cpp
index d95f0fdc23e..632b387a817 100644
--- a/src/mongo/db/s/resharding/resharding_destined_recipient_test.cpp
+++ b/src/mongo/db/s/resharding/resharding_destined_recipient_test.cpp
@@ -27,9 +27,6 @@
* it in the license file.
*/
-
-#include "mongo/platform/basic.h"
-
#include "mongo/db/catalog/create_collection.h"
#include "mongo/db/catalog_raii.h"
#include "mongo/db/dbdirectclient.h"
@@ -55,7 +52,6 @@
#define MONGO_LOGV2_DEFAULT_COMPONENT ::mongo::logv2::LogComponent::kTest
-
namespace mongo {
namespace {
@@ -164,11 +160,11 @@ protected:
const std::string& shardKey) {
auto range1 = ChunkRange(BSON(shardKey << MINKEY), BSON(shardKey << 5));
ChunkType chunk1(
- uuid, range1, ChunkVersion(1, 0, epoch, timestamp), kShardList[0].getName());
+ uuid, range1, ChunkVersion({epoch, timestamp}, {1, 0}), kShardList[0].getName());
auto range2 = ChunkRange(BSON(shardKey << 5), BSON(shardKey << MAXKEY));
ChunkType chunk2(
- uuid, range2, ChunkVersion(1, 0, epoch, timestamp), kShardList[1].getName());
+ uuid, range2, ChunkVersion({epoch, timestamp}, {1, 0}), kShardList[1].getName());
return {chunk1, chunk2};
}
@@ -199,7 +195,7 @@ protected:
ReshardingEnv env(CollectionCatalog::get(opCtx)->lookupUUIDByNSS(opCtx, kNss).value());
env.destShard = kShardList[1].getName();
- env.version = ChunkVersion(1, 0, OID::gen(), Timestamp(1, 1));
+ env.version = ChunkVersion({OID::gen(), Timestamp(1, 1)}, {1, 0});
env.tempNss =
NamespaceString(kNss.db(),
fmt::format("{}{}",
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 f3171d09254..d2313684ff9 100644
--- a/src/mongo/db/s/resharding/resharding_oplog_applier_test.cpp
+++ b/src/mongo/db/s/resharding/resharding_oplog_applier_test.cpp
@@ -27,9 +27,6 @@
* it in the license file.
*/
-
-#include "mongo/platform/basic.h"
-
#include <fmt/format.h>
#include "mongo/db/cancelable_operation_context.h"
@@ -64,7 +61,6 @@
#define MONGO_LOGV2_DEFAULT_COMPONENT ::mongo::logv2::LogComponent::kTest
-
namespace mongo {
namespace {
@@ -194,17 +190,17 @@ public:
kCrudUUID,
ChunkRange{BSON(kOriginalShardKey << MINKEY),
BSON(kOriginalShardKey << -std::numeric_limits<double>::infinity())},
- ChunkVersion(1, 0, epoch, Timestamp(1, 1)),
+ ChunkVersion({epoch, Timestamp(1, 1)}, {1, 0}),
_sourceId.getShardId()},
ChunkType{
kCrudUUID,
ChunkRange{BSON(kOriginalShardKey << -std::numeric_limits<double>::infinity()),
BSON(kOriginalShardKey << 0)},
- ChunkVersion(1, 0, epoch, Timestamp(1, 1)),
+ ChunkVersion({epoch, Timestamp(1, 1)}, {1, 0}),
kOtherShardId},
ChunkType{kCrudUUID,
ChunkRange{BSON(kOriginalShardKey << 0), BSON(kOriginalShardKey << MAXKEY)},
- ChunkVersion(1, 0, epoch, Timestamp(1, 1)),
+ ChunkVersion({epoch, Timestamp(1, 1)}, {1, 0}),
_sourceId.getShardId()}};
auto rt = RoutingTableHistory::makeNew(kCrudNs,
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 c88884ac624..ca596e65e16 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
@@ -27,9 +27,6 @@
* it in the license file.
*/
-
-#include "mongo/platform/basic.h"
-
#include <boost/optional/optional_io.hpp>
#include <memory>
#include <vector>
@@ -66,7 +63,6 @@
#define MONGO_LOGV2_DEFAULT_COMPONENT ::mongo::logv2::LogComponent::kTest
-
namespace mongo {
namespace {
@@ -318,7 +314,7 @@ private:
std::vector<ChunkType> chunks = {ChunkType{
_sourceUUID,
ChunkRange{BSON(_currentShardKey << MINKEY), BSON(_currentShardKey << MAXKEY)},
- ChunkVersion(100, 0, epoch, Timestamp(1, 1)),
+ ChunkVersion({epoch, Timestamp(1, 1)}, {100, 0}),
_myDonorId}};
auto rt = RoutingTableHistory::makeNew(_sourceNss,
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 c4e193e6897..62776bba466 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
@@ -27,9 +27,6 @@
* it in the license file.
*/
-
-#include "mongo/platform/basic.h"
-
#include "mongo/bson/unordered_fields_bsonobj_comparator.h"
#include "mongo/db/catalog_raii.h"
#include "mongo/db/dbdirectclient.h"
@@ -48,9 +45,6 @@
#include "mongo/s/database_version.h"
#include "mongo/s/stale_exception.h"
-#define MONGO_LOGV2_DEFAULT_COMPONENT ::mongo::logv2::LogComponent::kTest
-
-
namespace mongo {
namespace {
@@ -168,7 +162,7 @@ public:
reshardingFields.setRecipientFields(recipientFields);
coll.setReshardingFields(reshardingFields);
- ChunkVersion version(1, 0, epoch, timestamp);
+ ChunkVersion version({epoch, timestamp}, {1, 0});
ChunkType chunk(uuid,
{skey.getKeyPattern().globalMin(), skey.getKeyPattern().globalMax()},
@@ -193,7 +187,7 @@ public:
CollectionType coll(
origNss, epoch, timestamp, Date_t::now(), uuid, skey.getKeyPattern());
- ChunkVersion version(2, 0, epoch, timestamp);
+ ChunkVersion version({epoch, timestamp}, {2, 0});
ChunkType chunk(uuid,
{skey.getKeyPattern().globalMin(), skey.getKeyPattern().globalMax()},
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 283001c3b3e..93a685475d4 100644
--- a/src/mongo/db/s/shard_server_catalog_cache_loader.cpp
+++ b/src/mongo/db/s/shard_server_catalog_cache_loader.cpp
@@ -230,11 +230,9 @@ CollectionAndChangedChunks getPersistedMetadataSinceVersion(OperationContext* op
// If the epochs are the same we can safely take the timestamp from the shard coll entry.
ChunkVersion startingVersion = version.isSameCollection({shardCollectionEntry.getEpoch(),
shardCollectionEntry.getTimestamp()})
- ? ChunkVersion(version.majorVersion(),
- version.minorVersion(),
- version.epoch(),
- shardCollectionEntry.getTimestamp())
- : ChunkVersion(0, 0, shardCollectionEntry.getEpoch(), shardCollectionEntry.getTimestamp());
+ ? version
+ : ChunkVersion({shardCollectionEntry.getEpoch(), shardCollectionEntry.getTimestamp()},
+ {0, 0});
QueryAndSort diff = createShardChunkDiffQuery(startingVersion);
@@ -1277,16 +1275,7 @@ ShardServerCatalogCacheLoader::CollAndChunkTask::CollAndChunkTask(
if (statusWithCollectionAndChangedChunks.isOK()) {
collectionAndChangedChunks = std::move(statusWithCollectionAndChangedChunks.getValue());
invariant(!collectionAndChangedChunks->changedChunks.empty());
- const auto highestVersion = collectionAndChangedChunks->changedChunks.back().getVersion();
- // Note that due to the way Phase 1 of the FCV upgrade writes timestamps to chunks
- // (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 max query version and we should use the
- // timestamp that comes from config.collections.
- maxQueryVersion = ChunkVersion(highestVersion.majorVersion(),
- highestVersion.minorVersion(),
- highestVersion.epoch(),
- collectionAndChangedChunks->timestamp);
+ maxQueryVersion = collectionAndChangedChunks->changedChunks.back().getVersion();
} else {
invariant(statusWithCollectionAndChangedChunks == ErrorCodes::NamespaceNotFound);
dropped = true;
diff --git a/src/mongo/db/s/type_shard_collection.cpp b/src/mongo/db/s/type_shard_collection.cpp
index 30912e47a59..2628297e0d7 100644
--- a/src/mongo/db/s/type_shard_collection.cpp
+++ b/src/mongo/db/s/type_shard_collection.cpp
@@ -82,7 +82,7 @@ boost::optional<ChunkVersion> ShardCollectionType::getLastRefreshedCollectionVer
return boost::none;
Timestamp majorMinor = *getLastRefreshedCollectionMajorMinorVersion();
- return ChunkVersion(majorMinor.getSecs(), majorMinor.getInc(), getEpoch(), getTimestamp());
+ return ChunkVersion({getEpoch(), getTimestamp()}, {majorMinor.getSecs(), majorMinor.getInc()});
}
} // namespace mongo
diff --git a/src/mongo/s/catalog/sharding_catalog_client_test.cpp b/src/mongo/s/catalog/sharding_catalog_client_test.cpp
index effe9ca0a10..92f2ce78a8d 100644
--- a/src/mongo/s/catalog/sharding_catalog_client_test.cpp
+++ b/src/mongo/s/catalog/sharding_catalog_client_test.cpp
@@ -27,9 +27,6 @@
* it in the license file.
*/
-
-#include "mongo/platform/basic.h"
-
#include <pcrecpp.h>
#include "mongo/bson/json.h"
@@ -60,7 +57,6 @@
#define MONGO_LOGV2_DEFAULT_COMPONENT ::mongo::logv2::LogComponent::kSharding
-
namespace mongo {
namespace {
@@ -375,7 +371,7 @@ TEST_F(ShardingCatalogClientTest, GetChunksForNSWithSortAndLimit) {
chunkA.setCollectionUUID(collUuid);
chunkA.setMin(BSON("a" << 1));
chunkA.setMax(BSON("a" << 100));
- chunkA.setVersion({1, 2, collEpoch, collTimestamp});
+ chunkA.setVersion(ChunkVersion({collEpoch, collTimestamp}, {1, 2}));
chunkA.setShard(ShardId("shard0000"));
ChunkType chunkB;
@@ -383,10 +379,10 @@ TEST_F(ShardingCatalogClientTest, GetChunksForNSWithSortAndLimit) {
chunkB.setCollectionUUID(collUuid);
chunkB.setMin(BSON("a" << 100));
chunkB.setMax(BSON("a" << 200));
- chunkB.setVersion({3, 4, collEpoch, collTimestamp});
+ chunkB.setVersion(ChunkVersion({collEpoch, collTimestamp}, {3, 4}));
chunkB.setShard(ShardId("shard0001"));
- ChunkVersion queryChunkVersion({1, 2, collEpoch, collTimestamp});
+ ChunkVersion queryChunkVersion({collEpoch, collTimestamp}, {1, 2});
const BSONObj chunksQuery(
BSON(ChunkType::collectionUUID()
@@ -458,7 +454,7 @@ TEST_F(ShardingCatalogClientTest, GetChunksForUUIDNoSortNoLimit) {
const auto collEpoch = OID::gen();
const auto collTimestamp = Timestamp(1, 1);
- ChunkVersion queryChunkVersion({1, 2, collEpoch, collTimestamp});
+ ChunkVersion queryChunkVersion({collEpoch, collTimestamp}, {1, 2});
const BSONObj chunksQuery(
BSON(ChunkType::collectionUUID()
@@ -507,7 +503,7 @@ TEST_F(ShardingCatalogClientTest, GetChunksForNSInvalidChunk) {
configTargeter()->setFindHostReturnValue(HostAndPort("TestHost1"));
const auto collUuid = UUID::gen();
- ChunkVersion queryChunkVersion({1, 2, OID::gen(), Timestamp(1, 1)});
+ ChunkVersion queryChunkVersion({OID::gen(), Timestamp(1, 1)}, {1, 2});
const BSONObj chunksQuery(
BSON(ChunkType::collectionUUID()
@@ -533,14 +529,14 @@ TEST_F(ShardingCatalogClientTest, GetChunksForNSInvalidChunk) {
chunkA.setCollectionUUID(collUuid);
chunkA.setMin(BSON("a" << 1));
chunkA.setMax(BSON("a" << 100));
- chunkA.setVersion({1, 2, OID::gen(), Timestamp(1, 1)});
+ chunkA.setVersion(ChunkVersion({OID::gen(), Timestamp(1, 1)}, {1, 2}));
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(), Timestamp(1, 1)});
+ chunkB.setVersion(ChunkVersion({OID::gen(), Timestamp(1, 1)}, {3, 4}));
// Missing shard id
return vector<BSONObj>{chunkA.toConfigBSON(), chunkB.toConfigBSON()};
diff --git a/src/mongo/s/chunk_manager.cpp b/src/mongo/s/chunk_manager.cpp
index 92e54aa83bb..593b475ed46 100644
--- a/src/mongo/s/chunk_manager.cpp
+++ b/src/mongo/s/chunk_manager.cpp
@@ -130,6 +130,11 @@ void validateChunkIsNotOlderThan(const std::shared_ptr<ChunkInfo>& chunk,
} // namespace
+ChunkMap::ChunkMap(OID epoch, const Timestamp& timestamp, size_t initialCapacity)
+ : _collectionVersion({epoch, timestamp}, {0, 0}) {
+ _chunkMap.reserve(initialCapacity);
+}
+
ShardVersionMap ChunkMap::constructShardVersionMap() const {
ShardVersionMap shardVersions;
ChunkVector::const_iterator current = _chunkMap.cbegin();
@@ -212,10 +217,7 @@ void ChunkMap::appendChunk(const std::shared_ptr<ChunkInfo>& chunk) {
appendChunkTo(_chunkMap, chunk);
const auto chunkVersion = chunk->getLastmod();
if (_collectionVersion.isOlderThan(chunkVersion)) {
- _collectionVersion = ChunkVersion(chunkVersion.majorVersion(),
- chunkVersion.minorVersion(),
- chunkVersion.epoch(),
- _collTimestamp);
+ _collectionVersion = chunkVersion;
}
}
@@ -317,7 +319,7 @@ ChunkMap::_overlappingBounds(const BSONObj& min, const BSONObj& max, bool isMaxI
}
ShardVersionTargetingInfo::ShardVersionTargetingInfo(const OID& epoch, const Timestamp& timestamp)
- : shardVersion(0, 0, epoch, timestamp) {}
+ : shardVersion({epoch, timestamp}, {0, 0}) {}
RoutingTableHistory::RoutingTableHistory(
NamespaceString nss,
@@ -748,7 +750,7 @@ ChunkVersion RoutingTableHistory::_getVersion(const ShardId& shardName,
// Shards without explicitly tracked shard versions (meaning they have no chunks) always
// have a version of (0, 0, epoch, timestamp)
const auto collVersion = _chunkMap.getVersion();
- return ChunkVersion(0, 0, collVersion.epoch(), collVersion.getTimestamp());
+ return ChunkVersion({collVersion.epoch(), collVersion.getTimestamp()}, {0, 0});
}
if (throwOnStaleShard && gEnableFinerGrainedCatalogCacheRefresh) {
diff --git a/src/mongo/s/chunk_manager.h b/src/mongo/s/chunk_manager.h
index 00c75957d37..7114caed9e7 100644
--- a/src/mongo/s/chunk_manager.h
+++ b/src/mongo/s/chunk_manager.h
@@ -75,10 +75,7 @@ class ChunkMap {
using ChunkVector = std::vector<std::shared_ptr<ChunkInfo>>;
public:
- explicit ChunkMap(OID epoch, const Timestamp& timestamp, size_t initialCapacity = 0)
- : _collectionVersion(0, 0, epoch, timestamp), _collTimestamp(timestamp) {
- _chunkMap.reserve(initialCapacity);
- }
+ ChunkMap(OID epoch, const Timestamp& timestamp, size_t initialCapacity = 0);
size_t size() const {
return _chunkMap.size();
@@ -130,14 +127,6 @@ private:
// Max version across all chunks
ChunkVersion _collectionVersion;
-
- // Represents the timestamp present in config.collections for this ChunkMap.
- //
- // Note that due to the way Phase 1 of the FCV upgrade writes timestamps to chunks
- // (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.
- Timestamp _collTimestamp;
};
/**
diff --git a/src/mongo/s/chunk_version.cpp b/src/mongo/s/chunk_version.cpp
index 12ed1ee72f7..ca260776a34 100644
--- a/src/mongo/s/chunk_version.cpp
+++ b/src/mongo/s/chunk_version.cpp
@@ -60,10 +60,8 @@ ChunkVersion ChunkVersion::parse(const BSONElement& element) {
auto parsedVersion =
ChunkVersion60Format::parse(IDLParserErrorContext("ChunkVersion"), element.Obj());
auto version = parsedVersion.getVersion();
- return ChunkVersion(version.getSecs(),
- version.getInc(),
- parsedVersion.getEpoch(),
- parsedVersion.getTimestamp());
+ return ChunkVersion({parsedVersion.getEpoch(), parsedVersion.getTimestamp()},
+ {version.getSecs(), version.getInc()});
}
void ChunkVersion::serializeToBSON(StringData field, BSONObjBuilder* builder) const {
diff --git a/src/mongo/s/chunk_version.h b/src/mongo/s/chunk_version.h
index b25bae60691..68f33c6b018 100644
--- a/src/mongo/s/chunk_version.h
+++ b/src/mongo/s/chunk_version.h
@@ -53,12 +53,55 @@ public:
std::string toString() const;
+ // TODO: Do not add any new usages of these methods. Use isSameCollection instead.
+
+ const OID& epoch() const {
+ return _epoch;
+ }
+
+ const Timestamp& getTimestamp() const {
+ return _timestamp;
+ }
+
protected:
OID _epoch;
Timestamp _timestamp;
};
/**
+ * Reflects the placement information for a collection. An object of this class has no meaning on
+ * its own without the Generation component above, that's why most of its methods are protected and
+ * are exposed as semantic checks in ChunkVersion below.
+ */
+class CollectionPlacement {
+public:
+ CollectionPlacement(uint32_t major, uint32_t minor)
+ : _combined(static_cast<uint64_t>(minor) | (static_cast<uint64_t>(major) << 32)) {}
+
+ // TODO: Do not add any new usages of these methods. Use isSamePlacement instead.
+
+ uint32_t majorVersion() const {
+ return _combined >> 32;
+ }
+
+ uint32_t minorVersion() const {
+ return _combined & 0xFFFFFFFF;
+ }
+
+protected:
+ /**
+ * Returns whether two collection placements are compatible with each other (meaning that they
+ * refer to the same distribution of chunks across the cluster).
+ */
+ bool isSamePlacement(const CollectionPlacement& other) const {
+ return majorVersion() == other.majorVersion();
+ }
+
+ // The combined major/minor version, which exists as subordinate to the collection generation
+ uint64_t _combined;
+};
+
+/**
* ChunkVersions consist of a major/minor version scoped to a version epoch
*
* Version configurations (format: major version, epoch):
@@ -68,7 +111,7 @@ protected:
* 3. (n, 0), n > 0 - invalid configuration.
* 4. (n, m), n > 0, m > 0 - normal sharded collection version.
*/
-class ChunkVersion : public CollectionGeneration {
+class ChunkVersion : public CollectionGeneration, public CollectionPlacement {
public:
/**
* The name for the shard version information field, which shard-aware commands should include
@@ -76,11 +119,14 @@ public:
*/
static constexpr StringData kShardVersionField = "shardVersion"_sd;
- ChunkVersion(uint32_t major, uint32_t minor, const OID& epoch, const Timestamp& timestamp)
- : CollectionGeneration(epoch, timestamp),
- _combined(static_cast<uint64_t>(minor) | (static_cast<uint64_t>(major) << 32)) {}
+ ChunkVersion(CollectionGeneration geneneration, CollectionPlacement placement)
+ : CollectionGeneration(geneneration), CollectionPlacement(placement) {}
+
+ ChunkVersion() : ChunkVersion({OID(), Timestamp()}, {0, 0}) {}
- ChunkVersion() : ChunkVersion(0, 0, OID(), Timestamp()) {}
+ // TODO: Do not add any new usages of this constructor. Use the one above instead.
+ ChunkVersion(uint32_t major, uint32_t minor, OID epoch, Timestamp timestamp)
+ : CollectionGeneration(epoch, timestamp), CollectionPlacement(major, minor) {}
/**
* Indicates that the collection is not sharded.
@@ -134,22 +180,6 @@ public:
return _combined > 0;
}
- uint32_t majorVersion() const {
- return _combined >> 32;
- }
-
- uint32_t minorVersion() const {
- return _combined & 0xFFFFFFFF;
- }
-
- const OID& epoch() const {
- return _epoch;
- }
-
- const Timestamp& getTimestamp() const {
- return _timestamp;
- }
-
bool operator==(const ChunkVersion& otherVersion) const {
return otherVersion.getTimestamp() == getTimestamp() && otherVersion._combined == _combined;
}
@@ -160,7 +190,7 @@ public:
// Can we write to this data and not have a problem?
bool isWriteCompatibleWith(const ChunkVersion& other) const {
- return isSameCollection(other) && majorVersion() == other.majorVersion();
+ return isSameCollection(other) && isSamePlacement(other);
}
// Unsharded timestamp cannot be compared with other timestamps
@@ -198,10 +228,6 @@ public:
void serializeToBSON(StringData field, BSONObjBuilder* builder) const;
std::string toString() const;
-
-private:
- // The combined major/minor version, which exists as subordinate to the collection generation
- uint64_t _combined;
};
inline std::ostream& operator<<(std::ostream& s, const ChunkVersion& v) {
diff --git a/src/mongo/s/query/cluster_exchange_test.cpp b/src/mongo/s/query/cluster_exchange_test.cpp
index 8b33c58e843..96815019657 100644
--- a/src/mongo/s/query/cluster_exchange_test.cpp
+++ b/src/mongo/s/query/cluster_exchange_test.cpp
@@ -27,8 +27,6 @@
* it in the license file.
*/
-#include "mongo/platform/basic.h"
-
#include "mongo/client/remote_command_targeter_factory_mock.h"
#include "mongo/client/remote_command_targeter_mock.h"
#include "mongo/db/pipeline/document_source_group.h"
@@ -533,7 +531,7 @@ TEST_F(ClusterExchangeTest, CompoundShardKeyThreeShards) {
const std::vector<std::string> xBoundaries = {"a", "g", "m", "r", "u"};
auto chunks = [&]() {
std::vector<ChunkType> chunks;
- ChunkVersion version(1, 0, epoch, timestamp);
+ ChunkVersion version({epoch, timestamp}, {1, 0});
chunks.emplace_back(uuid,
ChunkRange{BSON("x" << MINKEY << "y" << MINKEY),
BSON("x" << xBoundaries[0] << "y" << MINKEY)},
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 12ac80a5c09..a1475991bd2 100644
--- a/src/mongo/s/request_types/balance_chunk_request_test.cpp
+++ b/src/mongo/s/request_types/balance_chunk_request_test.cpp
@@ -41,7 +41,7 @@ using unittest::assertGet;
TEST(BalanceChunkRequest, RoundTrip) {
UUID uuid{UUID::gen()};
- ChunkVersion version(30, 1, OID::gen(), Timestamp{2, 0});
+ ChunkVersion version({OID::gen(), Timestamp(2, 0)}, {30, 1});
auto obj = BalanceChunkRequest::serializeToRebalanceCommandForConfig(
NamespaceString("DB.Test"),
ChunkRange(BSON("A" << 100), BSON("A" << 200)),
@@ -59,7 +59,7 @@ TEST(BalanceChunkRequest, RoundTrip) {
TEST(BalanceChunkRequest, ParseFromConfigCommandNoSecondaryThrottle) {
const auto uuid{UUID::gen()};
- const ChunkVersion version(1, 0, OID::gen(), Timestamp(1, 1));
+ const ChunkVersion version({OID::gen(), Timestamp(1, 1)}, {1, 0});
auto request = assertGet(BalanceChunkRequest::parseFromConfigCommand(
BSON("_configsvrMoveChunk" << 1 << "ns"
<< "TestDB.TestColl"
@@ -83,7 +83,7 @@ TEST(BalanceChunkRequest, ParseFromConfigCommandNoSecondaryThrottle) {
TEST(BalanceChunkRequest, ParseFromConfigCommandWithUUIDNoSecondaryThrottle) {
const auto uuid = UUID::gen();
- const ChunkVersion version(1, 0, OID::gen(), Timestamp(1, 1));
+ const ChunkVersion version({OID::gen(), Timestamp(1, 1)}, {1, 0});
auto request = assertGet(BalanceChunkRequest::parseFromConfigCommand(
BSON("_configsvrMoveChunk" << 1 << "ns"
<< "TestDB.TestColl"
@@ -107,7 +107,7 @@ TEST(BalanceChunkRequest, ParseFromConfigCommandWithUUIDNoSecondaryThrottle) {
TEST(BalanceChunkRequest, ParseFromConfigCommandWithSecondaryThrottle) {
const auto uuid{UUID::gen()};
- const ChunkVersion version(1, 0, OID::gen(), Timestamp(1, 1));
+ const ChunkVersion version({OID::gen(), Timestamp(1, 1)}, {1, 0});
auto request = assertGet(BalanceChunkRequest::parseFromConfigCommand(
BSON("_configsvrMoveChunk"
<< 1 << "ns"
diff --git a/src/mongo/s/write_ops/batch_write_op_test.cpp b/src/mongo/s/write_ops/batch_write_op_test.cpp
index bfda09f0814..eb07226621f 100644
--- a/src/mongo/s/write_ops/batch_write_op_test.cpp
+++ b/src/mongo/s/write_ops/batch_write_op_test.cpp
@@ -27,8 +27,6 @@
* it in the license file.
*/
-#include "mongo/platform/basic.h"
-
#include "mongo/s/concurrency/locker_mongos_client_observer.h"
#include "mongo/s/mock_ns_targeter.h"
#include "mongo/s/session_catalog_router.h"
@@ -289,8 +287,8 @@ TEST_F(BatchWriteOpTest, SingleStaleError) {
response.addToErrDetails(
write_ops::WriteError(0,
Status{StaleConfigInfo(nss,
- ChunkVersion(101, 200, epoch, timestamp),
- ChunkVersion(105, 200, epoch, timestamp),
+ ChunkVersion({epoch, timestamp}, {101, 200}),
+ ChunkVersion({epoch, timestamp}, {105, 200}),
ShardId("shard")),
"mock stale error"}));