summaryrefslogtreecommitdiff
path: root/src/mongo/db
diff options
context:
space:
mode:
authorHenrik Edin <henrik.edin@mongodb.com>2018-05-18 17:29:14 -0400
committerHenrik Edin <henrik.edin@mongodb.com>2018-09-18 16:34:02 -0400
commit27e0275301eed05bea3d65c766dbe76ee1da9b8a (patch)
treec744bad9984e66cbce25c1f907f3e203db78fad5 /src/mongo/db
parentdf38daf87387969413c66df415601663e63cbb34 (diff)
downloadmongo-27e0275301eed05bea3d65c766dbe76ee1da9b8a.tar.gz
SERVER-32198 Get rid of CollectionShardingState::collectionIsSharded
(cherry picked from commit 80de0da37b00dbeed576b28a842cb172b6714358) SERVER-35773 Remove references to the CatalogCache from MetadataManager (cherry picked from commit 2aa65a86193e5d38934a4f2d6b0a8298e2432485) SERVER-32198 Add support for an optional `vWanted` to StaleConfigInfo (cherry picked from commit 60559a00b81293184922b3418a8e56610edf8dd9) SERVER-36054 Remove ScopedCollectionMetadata's operator bool (cherry picked from commit c9c340ad6e9e1f33cb001a8375c62d6b16138c74) SERVER-36054 Remove more unused methods from CSS/MetadataManager (cherry picked from commit ca04f5bcf9bfa73c9162b3a77225c997c6deec8a) SERVER-36116 Get rid of CollectionShardingState::resetAll (cherry picked from commit db1cc80d13d203b6351f5510f7756cc1c7bfc0ea) SERVER-36054 Get rid of unused methods from CollectionShardingState (cherry picked from commit 884d232473dca72e0872f0e540d4c3108c1e0b3d) SERVER-36164 Decouple ScopedCollectionMetadata from MetadataManager (cherry picked from commit d91262c4a2ed7d94923c3b1c5ff5d208aa981c73) SERVER-29908 Move CollectionShardingState under sharding_api_d (cherry picked from commit e491e284e8066929c8272c96a3128241ab481be8) SERVER-29908 Remove ShardingState::appendInfo Expose the ShardingState properties and move the appendInfo logic to be entirely inside the 'getShardingState' function, which is its only consumer. (cherry picked from commit 24e411d5cd7f64c5b4da25a351529cd1873284b8) SERVER-29908 Move 'updateConfigServerOpTimeFromMetadata' out of ShardingState (cherry picked from commit 7a97557ce5bf74dc2b663762b7a5ffb9c958d580) SERVER-29908 Move all runtime logic out of ShardingState ... and move it into a ShardingInitializationMongoD class, which is responsible for driving the sharding-awareness of the node and setting it onto ShardingState. Also gets rid of the 'sharding' library, so there is no more library dependency cycle. (cherry picked from commit 200c3dc58410d8b3287a2075cc9b2ad085100e83) SERVER-29908 Fold the 'sharding_connection_hook' library into 'sharding_initialization' ... and also remove dependency of MongoS on the replication coordinator (cherry picked from commit fab6864f4edcae7bb304f79e601f1f62cc376a77)
Diffstat (limited to 'src/mongo/db')
-rw-r--r--src/mongo/db/SConscript28
-rw-r--r--src/mongo/db/catalog/SConscript10
-rw-r--r--src/mongo/db/catalog/rename_collection.cpp15
-rw-r--r--src/mongo/db/commands/SConscript2
-rw-r--r--src/mongo/db/commands/create_indexes.cpp4
-rw-r--r--src/mongo/db/commands/feature_compatibility_version.cpp1
-rw-r--r--src/mongo/db/commands/mr.cpp6
-rw-r--r--src/mongo/db/commands/run_aggregate.cpp18
-rw-r--r--src/mongo/db/db.cpp16
-rw-r--r--src/mongo/db/db_raii.cpp9
-rw-r--r--src/mongo/db/exec/shard_filter.cpp3
-rw-r--r--src/mongo/db/exec/shard_filter.h2
-rw-r--r--src/mongo/db/exec/update.cpp7
-rw-r--r--src/mongo/db/op_observer_impl.cpp11
-rw-r--r--src/mongo/db/pipeline/pipeline_d.cpp18
-rw-r--r--src/mongo/db/query/get_executor.cpp2
-rw-r--r--src/mongo/db/read_concern.cpp3
-rw-r--r--src/mongo/db/repl/replication_coordinator_external_state_impl.cpp6
-rw-r--r--src/mongo/db/s/SConscript100
-rw-r--r--src/mongo/db/s/active_migrations_registry.cpp4
-rw-r--r--src/mongo/db/s/add_shard_cmd.idl44
-rw-r--r--src/mongo/db/s/chunk_splitter.cpp4
-rw-r--r--src/mongo/db/s/cleanup_orphaned_cmd.cpp11
-rw-r--r--src/mongo/db/s/collection_metadata.cpp57
-rw-r--r--src/mongo/db/s/collection_metadata.h92
-rw-r--r--src/mongo/db/s/collection_metadata_filtering_test.cpp230
-rw-r--r--src/mongo/db/s/collection_range_deleter.cpp22
-rw-r--r--src/mongo/db/s/collection_range_deleter_test.cpp10
-rw-r--r--src/mongo/db/s/collection_sharding_runtime.cpp162
-rw-r--r--src/mongo/db/s/collection_sharding_runtime.h171
-rw-r--r--src/mongo/db/s/collection_sharding_state.cpp370
-rw-r--r--src/mongo/db/s/collection_sharding_state.h217
-rw-r--r--src/mongo/db/s/collection_sharding_state_factory_embedded.cpp83
-rw-r--r--src/mongo/db/s/collection_sharding_state_factory_shard.cpp91
-rw-r--r--src/mongo/db/s/collection_sharding_state_test.cpp95
-rw-r--r--src/mongo/db/s/config/initial_split_policy.cpp1
-rw-r--r--src/mongo/db/s/config/sharding_catalog_manager_collection_operations.cpp4
-rw-r--r--src/mongo/db/s/config/sharding_catalog_manager_database_operations.cpp1
-rw-r--r--src/mongo/db/s/config/sharding_catalog_manager_shard_operations.cpp17
-rw-r--r--src/mongo/db/s/get_shard_version_command.cpp12
-rw-r--r--src/mongo/db/s/merge_chunks_command.cpp20
-rw-r--r--src/mongo/db/s/metadata_manager.cpp207
-rw-r--r--src/mongo/db/s/metadata_manager.h104
-rw-r--r--src/mongo/db/s/metadata_manager_test.cpp185
-rw-r--r--src/mongo/db/s/migration_chunk_cloner_source_legacy_commands.cpp4
-rw-r--r--src/mongo/db/s/migration_chunk_cloner_source_legacy_test.cpp26
-rw-r--r--src/mongo/db/s/migration_destination_manager.cpp15
-rw-r--r--src/mongo/db/s/migration_destination_manager.h5
-rw-r--r--src/mongo/db/s/migration_source_manager.cpp34
-rw-r--r--src/mongo/db/s/migration_source_manager.h8
-rw-r--r--src/mongo/db/s/move_primary_command.cpp2
-rw-r--r--src/mongo/db/s/scoped_collection_metadata.h (renamed from src/mongo/db/views/view_sharding_check.h)63
-rw-r--r--src/mongo/db/s/set_shard_version_command.cpp18
-rw-r--r--src/mongo/db/s/shard_filtering_metadata_refresh.cpp16
-rw-r--r--src/mongo/db/s/shard_server_catalog_cache_loader.cpp4
-rw-r--r--src/mongo/db/s/shard_server_op_observer.cpp34
-rw-r--r--src/mongo/db/s/shard_server_op_observer.h10
-rw-r--r--src/mongo/db/s/sharding_config_optime_gossip.cpp (renamed from src/mongo/db/s/sharding_egress_metadata_hook_for_mongod.cpp)34
-rw-r--r--src/mongo/db/s/sharding_config_optime_gossip.h (renamed from src/mongo/db/s/sharding_egress_metadata_hook_for_mongod.h)6
-rw-r--r--src/mongo/db/s/sharding_initialization_mongod.cpp289
-rw-r--r--src/mongo/db/s/sharding_initialization_mongod.h84
-rw-r--r--src/mongo/db/s/sharding_initialization_mongod_test.cpp (renamed from src/mongo/db/s/sharding_state_test.cpp)403
-rw-r--r--src/mongo/db/s/sharding_initialization_op_observer_test.cpp148
-rw-r--r--src/mongo/db/s/sharding_state.cpp374
-rw-r--r--src/mongo/db/s/sharding_state.h135
-rw-r--r--src/mongo/db/s/sharding_state_command.cpp18
-rw-r--r--src/mongo/db/s/sharding_state_recovery.cpp7
-rw-r--r--src/mongo/db/s/shardsvr_shard_collection.cpp4
-rw-r--r--src/mongo/db/s/split_chunk.cpp2
-rw-r--r--src/mongo/db/s/split_chunk_test.cpp54
-rw-r--r--src/mongo/db/s/split_vector_test.cpp8
-rw-r--r--src/mongo/db/s/type_shard_identity.cpp163
-rw-r--r--src/mongo/db/s/type_shard_identity.h51
-rw-r--r--src/mongo/db/s/type_shard_identity_test.cpp29
-rw-r--r--src/mongo/db/service_entry_point_common.cpp3
-rw-r--r--src/mongo/db/views/SConscript2
-rw-r--r--src/mongo/db/views/view_sharding_check.cpp79
77 files changed, 2418 insertions, 2189 deletions
diff --git a/src/mongo/db/SConscript b/src/mongo/db/SConscript
index 67cb790697f..5737c5b6a85 100644
--- a/src/mongo/db/SConscript
+++ b/src/mongo/db/SConscript
@@ -127,14 +127,14 @@ env.CppUnitTest(
)
env.CppUnitTest(
- target= 'op_observer_impl_test',
- source= 'op_observer_impl_test.cpp',
+ target='op_observer_impl_test',
+ source='op_observer_impl_test.cpp',
LIBDEPS=[
+ 'auth/authmocks',
'common',
'op_observer_d',
+ 'repl/replmocks',
'service_context_d_test_fixture',
- '$BUILD_DIR/mongo/db/auth/authmocks',
- '$BUILD_DIR/mongo/db/repl/replmocks',
],
)
@@ -611,7 +611,6 @@ env.Library(
'catalog_raii',
'curop',
's/sharding_api_d',
- 's/sharding',
'stats/top',
],
)
@@ -720,7 +719,6 @@ env.Library(
'$BUILD_DIR/mongo/db/auth/auth',
'$BUILD_DIR/mongo/db/concurrency/lock_manager',
'$BUILD_DIR/mongo/db/ops/write_ops_parsers',
- '$BUILD_DIR/mongo/db/s/sharding',
'$BUILD_DIR/mongo/db/storage/storage_engine_common',
'$BUILD_DIR/mongo/db/storage/storage_options',
'$BUILD_DIR/mongo/db/storage/storage_engine_lock_file',
@@ -883,7 +881,12 @@ env.Library(
"repl/repl_coordinator_interface",
"stats/timer_stats",
"storage/storage_options",
- "s/sharding",
+ ],
+ LIBDEPS_PRIVATE=[
+ "$BUILD_DIR/mongo/s/grid",
+ "catalog_raii",
+ "commands/server_status_core",
+ "s/sharding_api_d",
],
)
@@ -995,16 +998,16 @@ env.Library(
'audit',
'background',
'bson/dotted_path_support',
- 'catalog/collection',
'catalog/collection_info_cache',
+ 'catalog/collection',
'catalog/database',
'catalog/document_validation',
- 'catalog/index_catalog',
'catalog/index_catalog_entry',
+ 'catalog/index_catalog',
'commands',
'concurrency/write_conflict_exception',
- 'curop',
'curop_failpoint_helpers',
+ 'curop',
'cursor_server_params',
'db_raii',
'dbdirectclient',
@@ -1019,7 +1022,7 @@ env.Library(
'query/query_common',
'query/query_planner',
'repl/repl_coordinator_interface',
- 's/sharding',
+ 's/sharding_api_d',
'stats/serveronly_stats',
'storage/oplog_hack',
'storage/storage_options',
@@ -1041,8 +1044,6 @@ env.Library(
"$BUILD_DIR/mongo/db/ttl_collection_cache",
"$BUILD_DIR/mongo/executor/network_interface_factory",
"$BUILD_DIR/mongo/s/catalog/sharding_catalog_client_impl",
- "$BUILD_DIR/mongo/s/client/sharding_connection_hook",
- "$BUILD_DIR/mongo/s/coreshard",
"$BUILD_DIR/mongo/scripting/scripting_server",
"$BUILD_DIR/mongo/util/clock_sources",
"$BUILD_DIR/mongo/util/elapsed_tracker",
@@ -1092,7 +1093,6 @@ env.Library(
"repl/topology_coordinator",
"rw_concern_d",
"s/commands_db_s",
- "s/sharding_api_d",
"s/sharding_runtime_d",
"startup_warnings_mongod",
"stats/counters",
diff --git a/src/mongo/db/catalog/SConscript b/src/mongo/db/catalog/SConscript
index f4ea51e27bd..56a26e028ec 100644
--- a/src/mongo/db/catalog/SConscript
+++ b/src/mongo/db/catalog/SConscript
@@ -302,17 +302,17 @@ env.Library(
'rename_collection.cpp',
],
LIBDEPS=[
- 'collection',
- 'collection_options',
- 'database',
- 'index_catalog',
- 'index_create',
'$BUILD_DIR/mongo/base',
'$BUILD_DIR/mongo/db/background',
'$BUILD_DIR/mongo/db/db_raii',
'$BUILD_DIR/mongo/db/query_exec',
'$BUILD_DIR/mongo/db/views/views',
'$BUILD_DIR/mongo/db/write_ops',
+ 'collection_options',
+ 'collection',
+ 'database',
+ 'index_catalog',
+ 'index_create',
],
)
diff --git a/src/mongo/db/catalog/rename_collection.cpp b/src/mongo/db/catalog/rename_collection.cpp
index 29d1fff186e..202f36fec57 100644
--- a/src/mongo/db/catalog/rename_collection.cpp
+++ b/src/mongo/db/catalog/rename_collection.cpp
@@ -165,8 +165,11 @@ Status renameCollectionCommon(OperationContext* opCtx,
}
// Make sure the source collection is not sharded.
- if (CollectionShardingState::get(opCtx, source)->getMetadata(opCtx)) {
- return {ErrorCodes::IllegalOperation, "source namespace cannot be sharded"};
+ {
+ auto const css = CollectionShardingState::get(opCtx, source);
+ if (css->getMetadata(opCtx)->isSharded()) {
+ return {ErrorCodes::IllegalOperation, "source namespace cannot be sharded"};
+ }
}
// Ensure that collection name does not exceed maximum length.
@@ -193,8 +196,12 @@ Status renameCollectionCommon(OperationContext* opCtx,
invariant(source == target);
return Status::OK();
}
- if (CollectionShardingState::get(opCtx, target)->getMetadata(opCtx)) {
- return {ErrorCodes::IllegalOperation, "cannot rename to a sharded collection"};
+
+ {
+ auto const css = CollectionShardingState::get(opCtx, target);
+ if (css->getMetadata(opCtx)->isSharded()) {
+ return {ErrorCodes::IllegalOperation, "cannot rename to a sharded collection"};
+ }
}
if (!options.dropTarget) {
diff --git a/src/mongo/db/commands/SConscript b/src/mongo/db/commands/SConscript
index 2b45f314612..b80cd17f957 100644
--- a/src/mongo/db/commands/SConscript
+++ b/src/mongo/db/commands/SConscript
@@ -312,8 +312,8 @@ env.Library(
'$BUILD_DIR/mongo/db/auth/authprivilege',
'$BUILD_DIR/mongo/db/auth/role_graph',
'$BUILD_DIR/mongo/db/auth/sasl_options',
- '$BUILD_DIR/mongo/db/auth/user',
'$BUILD_DIR/mongo/db/auth/user_document_parser',
+ '$BUILD_DIR/mongo/db/auth/user',
'$BUILD_DIR/mongo/db/background',
'$BUILD_DIR/mongo/db/catalog/catalog_helpers',
'$BUILD_DIR/mongo/db/catalog/catalog_impl',
diff --git a/src/mongo/db/commands/create_indexes.cpp b/src/mongo/db/commands/create_indexes.cpp
index e71d703b488..afceb6c9d98 100644
--- a/src/mongo/db/commands/create_indexes.cpp
+++ b/src/mongo/db/commands/create_indexes.cpp
@@ -406,8 +406,8 @@ private:
const BSONObj& newIdxKey) {
invariant(opCtx->lockState()->isCollectionLockedForMode(nss.ns(), MODE_X));
- auto metadata(CollectionShardingState::get(opCtx, nss)->getMetadata(opCtx));
- if (metadata) {
+ auto metadata = CollectionShardingState::get(opCtx, nss)->getMetadata(opCtx);
+ if (metadata->isSharded()) {
ShardKeyPattern shardKeyPattern(metadata->getKeyPattern());
if (!shardKeyPattern.isUniqueIndexCompatible(newIdxKey)) {
return Status(ErrorCodes::CannotCreateIndex,
diff --git a/src/mongo/db/commands/feature_compatibility_version.cpp b/src/mongo/db/commands/feature_compatibility_version.cpp
index a305333b693..e8da9962861 100644
--- a/src/mongo/db/commands/feature_compatibility_version.cpp
+++ b/src/mongo/db/commands/feature_compatibility_version.cpp
@@ -167,7 +167,6 @@ void FeatureCompatibilityVersion::onInsertOrUpdate(OperationContext* opCtx, cons
(newVersion ==
ServerGlobalParams::FeatureCompatibility::Version::kFullyDowngradedTo36 ||
newVersion == ServerGlobalParams::FeatureCompatibility::Version::kFullyUpgradedTo40)) {
- CollectionShardingState::resetAll(opCtx);
Grid::get(opCtx)->catalogCache()->purgeAllDatabases();
}
diff --git a/src/mongo/db/commands/mr.cpp b/src/mongo/db/commands/mr.cpp
index 63d9bf89373..39d2ef8eb10 100644
--- a/src/mongo/db/commands/mr.cpp
+++ b/src/mongo/db/commands/mr.cpp
@@ -1541,7 +1541,7 @@ public:
o = o.getOwned(); // we will be accessing outside of the lock
// check to see if this is a new object we don't own yet
// because of a chunk migration
- if (collMetadata) {
+ if (collMetadata->isSharded()) {
ShardKeyPattern kp(collMetadata->getKeyPattern());
if (!collMetadata->keyBelongsToMe(kp.extractShardKeyFromDoc(o))) {
continue;
@@ -1798,10 +1798,10 @@ public:
if (auto cm = outRoutingInfoStatus.getValue().cm()) {
// Fetch result from other shards 1 chunk at a time. It would be better to do just
// one big $or query, but then the sorting would not be efficient.
- const string shardName = ShardingState::get(opCtx)->getShardName();
+ const auto shardId = ShardingState::get(opCtx)->shardId();
for (const auto& chunk : cm->chunks()) {
- if (chunk.getShardId() == shardName) {
+ if (chunk.getShardId() == shardId) {
chunks.push_back(chunk);
}
}
diff --git a/src/mongo/db/commands/run_aggregate.cpp b/src/mongo/db/commands/run_aggregate.cpp
index 1d0eb9dd324..f4a97b7cfef 100644
--- a/src/mongo/db/commands/run_aggregate.cpp
+++ b/src/mongo/db/commands/run_aggregate.cpp
@@ -58,12 +58,12 @@
#include "mongo/db/read_concern.h"
#include "mongo/db/repl/oplog.h"
#include "mongo/db/repl/read_concern_args.h"
+#include "mongo/db/s/sharding_state.h"
#include "mongo/db/service_context.h"
#include "mongo/db/session_catalog.h"
#include "mongo/db/storage/storage_options.h"
#include "mongo/db/views/view.h"
#include "mongo/db/views/view_catalog.h"
-#include "mongo/db/views/view_sharding_check.h"
#include "mongo/stdx/memory.h"
#include "mongo/util/log.h"
#include "mongo/util/scopeguard.h"
@@ -411,6 +411,7 @@ Status runAggregate(OperationContext* opCtx,
if (ctx && ctx->getView() && !liteParsedPipeline.startsWithCollStats()) {
invariant(nss != NamespaceString::kRsOplogNamespace);
invariant(!nss.isCollectionlessAggregateNS());
+
// Check that the default collation of 'view' is compatible with the operation's
// collation. The check is skipped if the request did not specify a collation.
if (!request.getCollation().isEmpty()) {
@@ -422,27 +423,28 @@ Status runAggregate(OperationContext* opCtx,
}
}
- ViewShardingCheck::throwResolvedViewIfSharded(opCtx, ctx->getDb(), ctx->getView());
-
- auto resolvedView = ctx->getDb()->getViewCatalog()->resolveView(opCtx, nss);
- if (!resolvedView.isOK()) {
- return resolvedView.getStatus();
- }
+ auto resolvedView =
+ uassertStatusOK(ctx->getDb()->getViewCatalog()->resolveView(opCtx, nss));
+ uassert(std::move(resolvedView),
+ "On sharded systems, resolved views must be executed by mongos",
+ !ShardingState::get(opCtx)->enabled());
// With the view & collation resolved, we can relinquish locks.
ctx.reset();
// Parse the resolved view into a new aggregation request.
- auto newRequest = resolvedView.getValue().asExpandedViewAggregation(request);
+ auto newRequest = resolvedView.asExpandedViewAggregation(request);
auto newCmd = newRequest.serializeToCommandObj().toBson();
auto status = runAggregate(opCtx, origNss, newRequest, newCmd, result);
+
{
// Set the namespace of the curop back to the view namespace so ctx records
// stats on this view namespace on destruction.
stdx::lock_guard<Client> lk(*opCtx->getClient());
curOp->setNS_inlock(nss.ns());
}
+
return status;
}
diff --git a/src/mongo/db/db.cpp b/src/mongo/db/db.cpp
index 9aa5c806427..fbc26e91907 100644
--- a/src/mongo/db/db.cpp
+++ b/src/mongo/db/db.cpp
@@ -112,9 +112,7 @@
#include "mongo/db/s/config/sharding_catalog_manager.h"
#include "mongo/db/s/config_server_op_observer.h"
#include "mongo/db/s/shard_server_op_observer.h"
-#include "mongo/db/s/sharded_connection_info.h"
#include "mongo/db/s/sharding_initialization_mongod.h"
-#include "mongo/db/s/sharding_state.h"
#include "mongo/db/s/sharding_state_recovery.h"
#include "mongo/db/server_options.h"
#include "mongo/db/server_parameters.h"
@@ -526,9 +524,8 @@ ExitCode _initAndListen(int listenPort) {
}
// This function may take the global lock.
- auto shardingInitialized =
- uassertStatusOK(ShardingState::get(startupOpCtx.get())
- ->initializeShardingAwarenessIfNeeded(startupOpCtx.get()));
+ auto shardingInitialized = ShardingInitializationMongoD::get(startupOpCtx.get())
+ ->initializeShardingAwarenessIfNeeded(startupOpCtx.get());
if (shardingInitialized) {
waitForShardRegistryReload(startupOpCtx.get()).transitional_ignore();
}
@@ -554,10 +551,9 @@ ExitCode _initAndListen(int listenPort) {
uassertStatusOK(ShardingStateRecovery::recover(startupOpCtx.get()));
}
} else if (serverGlobalParams.clusterRole == ClusterRole::ConfigServer) {
- uassertStatusOK(
- initializeGlobalShardingStateForMongod(startupOpCtx.get(),
- ConnectionString::forLocal(),
- kDistLockProcessIdForConfigServer));
+ initializeGlobalShardingStateForMongoD(startupOpCtx.get(),
+ ConnectionString::forLocal(),
+ kDistLockProcessIdForConfigServer);
Balancer::create(startupOpCtx->getServiceContext());
@@ -885,7 +881,7 @@ void shutdownTask() {
// is building an index.
repl::ReplicationCoordinator::get(serviceContext)->shutdown(opCtx);
- ShardingState::get(serviceContext)->shutDown(opCtx);
+ ShardingInitializationMongoD::get(serviceContext)->shutDown(opCtx);
// Destroy all stashed transaction resources, in order to release locks.
SessionKiller::Matcher matcherAllSessions(
diff --git a/src/mongo/db/db_raii.cpp b/src/mongo/db/db_raii.cpp
index bf5900f7d89..6b74d20aad2 100644
--- a/src/mongo/db/db_raii.cpp
+++ b/src/mongo/db/db_raii.cpp
@@ -290,8 +290,8 @@ OldClientContext::OldClientContext(
case dbDelete: // path, so no need to check them here as well
break;
default:
- auto css = CollectionShardingState::get(_opCtx, ns);
- css->checkShardVersionOrThrow(_opCtx);
+ CollectionShardingState::get(_opCtx, NamespaceString(ns))
+ ->checkShardVersionOrThrow(_opCtx);
break;
}
}
@@ -327,9 +327,8 @@ OldClientWriteContext::OldClientWriteContext(OperationContext* opCtx, StringData
_autoCreateDb.emplace(opCtx, _nss.db(), MODE_IX);
_collLock.emplace(opCtx->lockState(), _nss.ns(), MODE_IX);
- // TODO (Kal): None of the places which use OldClientWriteContext seem to require versioning, so
- // we should consider defaulting this to false
- const bool doShardVersionCheck = true;
+ const bool doShardVersionCheck = false;
+
_clientContext.emplace(opCtx,
_nss.ns(),
doShardVersionCheck,
diff --git a/src/mongo/db/exec/shard_filter.cpp b/src/mongo/db/exec/shard_filter.cpp
index 84318b02357..8cfde38ea85 100644
--- a/src/mongo/db/exec/shard_filter.cpp
+++ b/src/mongo/db/exec/shard_filter.cpp
@@ -35,7 +35,6 @@
#include "mongo/db/exec/filter.h"
#include "mongo/db/exec/scoped_timer.h"
#include "mongo/db/exec/working_set_common.h"
-#include "mongo/db/s/metadata_manager.h"
#include "mongo/s/shard_key_pattern.h"
#include "mongo/stdx/memory.h"
#include "mongo/util/log.h"
@@ -76,7 +75,7 @@ PlanStage::StageState ShardFilterStage::doWork(WorkingSetID* out) {
// If we're sharded make sure that we don't return data that is not owned by us,
// including pending documents from in-progress migrations and orphaned documents from
// aborted migrations
- if (_metadata) {
+ if (_metadata->isSharded()) {
ShardKeyPattern shardKeyPattern(_metadata->getKeyPattern());
WorkingSetMember* member = _ws->get(*out);
WorkingSetMatchableDocument matchable(member);
diff --git a/src/mongo/db/exec/shard_filter.h b/src/mongo/db/exec/shard_filter.h
index 0a180afd2e6..2c94b1f0a4e 100644
--- a/src/mongo/db/exec/shard_filter.h
+++ b/src/mongo/db/exec/shard_filter.h
@@ -29,7 +29,7 @@
#pragma once
#include "mongo/db/exec/plan_stage.h"
-#include "mongo/db/s/metadata_manager.h"
+#include "mongo/db/s/scoped_collection_metadata.h"
namespace mongo {
diff --git a/src/mongo/db/exec/update.cpp b/src/mongo/db/exec/update.cpp
index 0673af4c73d..a071c17969f 100644
--- a/src/mongo/db/exec/update.cpp
+++ b/src/mongo/db/exec/update.cpp
@@ -44,9 +44,7 @@
#include "mongo/db/ops/update_lifecycle.h"
#include "mongo/db/query/explain.h"
#include "mongo/db/repl/replication_coordinator.h"
-#include "mongo/db/s/collection_metadata.h"
#include "mongo/db/s/collection_sharding_state.h"
-#include "mongo/db/s/metadata_manager.h"
#include "mongo/db/service_context.h"
#include "mongo/db/update/storage_validation.h"
#include "mongo/stdx/memory.h"
@@ -136,7 +134,7 @@ bool shouldRestartUpdateIfNoLongerMatches(const UpdateStageParams& params) {
const std::vector<std::unique_ptr<FieldRef>>* getImmutableFields(OperationContext* opCtx,
const NamespaceString& ns) {
auto metadata = CollectionShardingState::get(opCtx, ns)->getMetadata(opCtx);
- if (metadata) {
+ if (metadata->isSharded()) {
const std::vector<std::unique_ptr<FieldRef>>& fields = metadata->getKeyPatternFields();
// Return shard-keys as immutable for the update system.
return &fields;
@@ -294,7 +292,8 @@ BSONObj UpdateStage::transformAndUpdate(const Snapshotted<BSONObj>& oldObj, Reco
args.uuid = _collection->uuid();
args.stmtId = request->getStmtId();
args.update = logObj;
- args.criteria = css->getMetadata(getOpCtx()).extractDocumentKey(newObj);
+ auto metadata = css->getMetadata(getOpCtx());
+ args.criteria = metadata->extractDocumentKey(newObj);
uassert(16980,
"Multi-update operations require all documents to have an '_id' field",
!request->isMulti() || args.criteria.hasField("_id"_sd));
diff --git a/src/mongo/db/op_observer_impl.cpp b/src/mongo/db/op_observer_impl.cpp
index fb30678f818..bfccb37f1e8 100644
--- a/src/mongo/db/op_observer_impl.cpp
+++ b/src/mongo/db/op_observer_impl.cpp
@@ -46,7 +46,6 @@
#include "mongo/db/operation_context.h"
#include "mongo/db/repl/oplog.h"
#include "mongo/db/repl/replication_coordinator.h"
-#include "mongo/db/s/collection_sharding_state.h"
#include "mongo/db/s/shard_server_op_observer.h"
#include "mongo/db/server_options.h"
#include "mongo/db/session_catalog.h"
@@ -402,9 +401,9 @@ void OpObserverImpl::onInserts(OperationContext* opCtx,
onWriteOpCompleted(opCtx, nss, session, stmtIdsWritten, lastOpTime, lastWriteDate);
}
- auto css = (nss == NamespaceString::kSessionTransactionsTableNamespace || fromMigrate)
+ auto* const css = (nss == NamespaceString::kSessionTransactionsTableNamespace || fromMigrate)
? nullptr
- : CollectionShardingState::get(opCtx, nss);
+ : CollectionShardingRuntime::get(opCtx, nss);
size_t index = 0;
for (auto it = first; it != last; it++, index++) {
@@ -476,7 +475,7 @@ void OpObserverImpl::onUpdate(OperationContext* opCtx, const OplogUpdateEntryArg
if (args.nss != NamespaceString::kSessionTransactionsTableNamespace) {
if (!args.fromMigrate) {
- auto css = CollectionShardingState::get(opCtx, args.nss);
+ auto* const css = CollectionShardingRuntime::get(opCtx, args.nss);
shardObserveUpdateOp(
opCtx, css, args.updatedDoc, opTime.writeOpTime, opTime.prePostImageOpTime);
}
@@ -500,7 +499,7 @@ void OpObserverImpl::aboutToDelete(OperationContext* opCtx,
NamespaceString const& nss,
BSONObj const& doc) {
getDeleteState(opCtx) =
- ShardObserverDeleteState::make(opCtx, CollectionShardingState::get(opCtx, nss), doc);
+ ShardObserverDeleteState::make(opCtx, CollectionShardingRuntime::get(opCtx, nss), doc);
}
void OpObserverImpl::onDelete(OperationContext* opCtx,
@@ -534,7 +533,7 @@ void OpObserverImpl::onDelete(OperationContext* opCtx,
if (nss != NamespaceString::kSessionTransactionsTableNamespace) {
if (!fromMigrate) {
- auto css = CollectionShardingState::get(opCtx, nss);
+ auto* const css = CollectionShardingRuntime::get(opCtx, nss);
shardObserveDeleteOp(
opCtx, css, deleteState, opTime.writeOpTime, opTime.prePostImageOpTime);
}
diff --git a/src/mongo/db/pipeline/pipeline_d.cpp b/src/mongo/db/pipeline/pipeline_d.cpp
index 2061755ae64..1c71728e2a6 100644
--- a/src/mongo/db/pipeline/pipeline_d.cpp
+++ b/src/mongo/db/pipeline/pipeline_d.cpp
@@ -69,9 +69,7 @@
#include "mongo/db/query/get_executor.h"
#include "mongo/db/query/plan_summary_stats.h"
#include "mongo/db/query/query_planner.h"
-#include "mongo/db/s/collection_metadata.h"
#include "mongo/db/s/collection_sharding_state.h"
-#include "mongo/db/s/metadata_manager.h"
#include "mongo/db/s/sharding_state.h"
#include "mongo/db/service_context.h"
#include "mongo/db/session_catalog.h"
@@ -588,10 +586,8 @@ DBClientBase* PipelineD::MongoDInterface::directClient() {
bool PipelineD::MongoDInterface::isSharded(OperationContext* opCtx, const NamespaceString& nss) {
AutoGetCollectionForReadCommand autoColl(opCtx, nss);
- // TODO SERVER-24960: Use CollectionShardingState::collectionIsSharded() to confirm sharding
- // state.
- auto css = CollectionShardingState::get(opCtx, nss);
- return bool(css->getMetadata(opCtx));
+ auto const css = CollectionShardingState::get(opCtx, nss);
+ return css->getMetadata(opCtx)->isSharded();
}
BSONObj PipelineD::MongoDInterface::insert(const boost::intrusive_ptr<ExpressionContext>& expCtx,
@@ -722,13 +718,11 @@ Status PipelineD::MongoDInterface::attachCursorSourceToPipeline(
// collection representing the document source to be not-sharded. We confirm sharding state
// here to avoid taking a collection lock elsewhere for this purpose alone.
// TODO SERVER-27616: This check is incorrect in that we don't acquire a collection cursor
- // until after we release the lock, leaving room for a collection to be sharded inbetween.
- // TODO SERVER-24960: Use CollectionShardingState::collectionIsSharded() to confirm sharding
- // state.
+ // until after we release the lock, leaving room for a collection to be sharded in-between.
auto css = CollectionShardingState::get(expCtx->opCtx, expCtx->ns);
uassert(4567,
str::stream() << "from collection (" << expCtx->ns.ns() << ") cannot be sharded",
- !bool(css->getMetadata(expCtx->opCtx)));
+ !css->getMetadata(expCtx->opCtx)->isSharded());
PipelineD::prepareCursorSource(autoColl->getCollection(), expCtx->ns, nullptr, pipeline);
@@ -741,7 +735,7 @@ Status PipelineD::MongoDInterface::attachCursorSourceToPipeline(
std::string PipelineD::MongoDInterface::getShardName(OperationContext* opCtx) const {
if (ShardingState::get(opCtx)->enabled()) {
- return ShardingState::get(opCtx)->getShardName();
+ return ShardingState::get(opCtx)->shardId().toString();
}
return std::string();
@@ -781,7 +775,7 @@ std::pair<std::vector<FieldPath>, bool> PipelineD::MongoDInterface::collectDocum
// Collection is not sharded or UUID mismatch implies collection has been dropped and recreated
// as sharded.
- if (!scm || !scm->uuidMatches(uuid)) {
+ if (!scm->isSharded() || !scm->uuidMatches(uuid)) {
return {{"_id"}, false};
}
diff --git a/src/mongo/db/query/get_executor.cpp b/src/mongo/db/query/get_executor.cpp
index e735b59cff6..8a04b5501f5 100644
--- a/src/mongo/db/query/get_executor.cpp
+++ b/src/mongo/db/query/get_executor.cpp
@@ -178,7 +178,7 @@ void fillOutPlannerParams(OperationContext* opCtx,
if (plannerParams->options & QueryPlannerParams::INCLUDE_SHARD_FILTER) {
auto collMetadata =
CollectionShardingState::get(opCtx, canonicalQuery->nss())->getMetadata(opCtx);
- if (collMetadata) {
+ if (collMetadata->isSharded()) {
plannerParams->shardKey = collMetadata->getKeyPattern();
} else {
// If there's no metadata don't bother w/the shard filter since we won't know what
diff --git a/src/mongo/db/read_concern.cpp b/src/mongo/db/read_concern.cpp
index b8215af1610..9fa171a7251 100644
--- a/src/mongo/db/read_concern.cpp
+++ b/src/mongo/db/read_concern.cpp
@@ -141,8 +141,7 @@ Status makeNoopWriteIfNeeded(OperationContext* opCtx, LogicalTime clusterTime) {
return Status::OK();
}
- auto myShard =
- Grid::get(opCtx)->shardRegistry()->getShard(opCtx, shardingState->getShardName());
+ auto myShard = Grid::get(opCtx)->shardRegistry()->getShard(opCtx, shardingState->shardId());
if (!myShard.isOK()) {
return myShard.getStatus();
}
diff --git a/src/mongo/db/repl/replication_coordinator_external_state_impl.cpp b/src/mongo/db/repl/replication_coordinator_external_state_impl.cpp
index 543ac0108e8..4528cfa2dd5 100644
--- a/src/mongo/db/repl/replication_coordinator_external_state_impl.cpp
+++ b/src/mongo/db/repl/replication_coordinator_external_state_impl.cpp
@@ -71,7 +71,7 @@
#include "mongo/db/s/balancer/balancer.h"
#include "mongo/db/s/chunk_splitter.h"
#include "mongo/db/s/config/sharding_catalog_manager.h"
-#include "mongo/db/s/sharding_state.h"
+#include "mongo/db/s/sharding_initialization_mongod.h"
#include "mongo/db/s/sharding_state_recovery.h"
#include "mongo/db/server_options.h"
#include "mongo/db/server_parameters.h"
@@ -765,8 +765,8 @@ void ReplicationCoordinatorExternalStateImpl::_shardingOnTransitionToPrimaryHook
const auto configsvrConnStr =
Grid::get(opCtx)->shardRegistry()->getConfigShard()->getConnString();
- auto status = ShardingState::get(opCtx)->updateShardIdentityConfigString(
- opCtx, configsvrConnStr.toString());
+ auto status = ShardingInitializationMongoD::get(opCtx)->updateShardIdentityConfigString(
+ opCtx, configsvrConnStr);
if (!status.isOK()) {
warning() << "error encountered while trying to update config connection string to "
<< configsvrConnStr << causedBy(status);
diff --git a/src/mongo/db/s/SConscript b/src/mongo/db/s/SConscript
index b3e68cd3e20..c92e1502a8e 100644
--- a/src/mongo/db/s/SConscript
+++ b/src/mongo/db/s/SConscript
@@ -12,13 +12,16 @@ env.Library(
target='sharding_api_d',
source=[
'collection_metadata.cpp',
+ 'collection_sharding_state.cpp',
'database_sharding_state.cpp',
'operation_sharding_state.cpp',
'sharded_connection_info.cpp',
'sharding_migration_critical_section.cpp',
+ 'sharding_state.cpp',
],
LIBDEPS=[
'$BUILD_DIR/mongo/base',
+ '$BUILD_DIR/mongo/db/concurrency/lock_manager',
'$BUILD_DIR/mongo/db/range_arithmetic',
'$BUILD_DIR/mongo/s/sharding_routing_table',
],
@@ -31,10 +34,14 @@ env.Library(
'active_move_primaries_registry.cpp',
'chunk_move_write_concern_options.cpp',
'chunk_splitter.cpp',
+ 'collection_range_deleter.cpp',
+ 'collection_sharding_runtime.cpp',
+ 'collection_sharding_state_factory_shard.cpp',
'config_server_op_observer.cpp',
'implicit_create_collection.cpp',
- 'migration_chunk_cloner_source.cpp',
+ 'metadata_manager.cpp',
'migration_chunk_cloner_source_legacy.cpp',
+ 'migration_chunk_cloner_source.cpp',
'migration_destination_manager.cpp',
'migration_source_manager.cpp',
'migration_util.cpp',
@@ -50,12 +57,15 @@ env.Library(
'shard_metadata_util.cpp',
'shard_server_catalog_cache_loader.cpp',
'shard_server_op_observer.cpp',
+ 'sharding_config_optime_gossip.cpp',
+ 'sharding_initialization_mongod.cpp',
'sharding_state_recovery.cpp',
'sharding_statistics.cpp',
'split_chunk.cpp',
'split_vector.cpp',
],
LIBDEPS=[
+ '$BUILD_DIR/mongo/db/catalog/index_create',
'$BUILD_DIR/mongo/db/commands/mongod_fcv',
'$BUILD_DIR/mongo/db/db_raii',
'$BUILD_DIR/mongo/db/dbhelpers',
@@ -63,12 +73,24 @@ env.Library(
'$BUILD_DIR/mongo/db/repl/oplog',
'$BUILD_DIR/mongo/db/repl/repl_coordinator_interface',
'$BUILD_DIR/mongo/db/rw_concern_d',
- 'sharding',
+ '$BUILD_DIR/mongo/s/client/shard_local',
+ '$BUILD_DIR/mongo/s/sharding_initialization',
+ 'migration_types',
'sharding_api_d',
'sharding_catalog_manager',
],
)
+env.Library(
+ target='sharding_runtime_d_embedded',
+ source=[
+ 'collection_sharding_state_factory_embedded.cpp',
+ ],
+ LIBDEPS=[
+ 'sharding_api_d',
+ ],
+)
+
env.CppUnitTest(
target='config_server_op_observer_test',
source=[
@@ -97,6 +119,7 @@ env.Library(
env.Library(
target='type_shard_identity',
source=[
+ env.Idlc('add_shard_cmd.idl')[0],
'type_shard_identity.cpp',
],
LIBDEPS=[
@@ -108,41 +131,6 @@ env.Library(
)
env.Library(
- target='sharding',
- source=[
- 'collection_range_deleter.cpp',
- 'collection_sharding_state.cpp',
- 'metadata_manager.cpp',
- 'sharding_egress_metadata_hook_for_mongod.cpp',
- 'sharding_initialization_mongod.cpp',
- 'sharding_state.cpp',
- ],
- LIBDEPS=[
- '$BUILD_DIR/mongo/base',
- '$BUILD_DIR/mongo/bson/util/bson_extract',
- '$BUILD_DIR/mongo/db/bson/dotted_path_support',
- '$BUILD_DIR/mongo/db/catalog/index_catalog',
- '$BUILD_DIR/mongo/db/catalog/index_create',
- '$BUILD_DIR/mongo/db/catalog_raii',
- '$BUILD_DIR/mongo/db/common',
- '$BUILD_DIR/mongo/s/client/shard_local',
- '$BUILD_DIR/mongo/s/coreshard',
- '$BUILD_DIR/mongo/s/is_mongos',
- '$BUILD_DIR/mongo/s/sharding_initialization',
- '$BUILD_DIR/mongo/s/sharding_task_executor',
- '$BUILD_DIR/mongo/util/elapsed_tracker',
- 'balancer',
- 'migration_types',
- 'sharding_api_d',
- 'type_shard_identity',
- ],
- LIBDEPS_TAGS=[
- # TODO(ADAM, 2017-01-06): See `CYCLE` tags above
- 'illegal_cyclic_or_unresolved_dependencies_whitelisted',
- ],
-)
-
-env.Library(
target='balancer',
source=[
'balancer/balancer_chunk_selection_policy_impl.cpp',
@@ -266,6 +254,7 @@ env.Library(
'$BUILD_DIR/mongo/db/repl/replica_set_messages',
'$BUILD_DIR/mongo/s/commands/cluster_commands_helpers',
'$BUILD_DIR/mongo/s/commands/shared_cluster_commands',
+ '$BUILD_DIR/mongo/s/sharding_initialization',
'balancer',
'sharding_runtime_d',
],
@@ -293,7 +282,7 @@ env.CppUnitTest(
)
env.CppUnitTest(
- target='shard_test',
+ target='shard_server_test',
source=[
'active_migrations_registry_test.cpp',
'active_move_primaries_registry_test.cpp',
@@ -302,14 +291,16 @@ env.CppUnitTest(
'migration_chunk_cloner_source_legacy_test.cpp',
'migration_destination_manager_test.cpp',
'namespace_metadata_change_notifications_test.cpp',
- 'sharding_state_test.cpp',
+ 'shard_metadata_util_test.cpp',
'shard_server_catalog_cache_loader_test.cpp',
+ 'sharding_initialization_mongod_test.cpp',
+ 'sharding_initialization_op_observer_test.cpp',
+ 'split_vector_test.cpp',
],
LIBDEPS=[
'sharding_runtime_d',
'$BUILD_DIR/mongo/db/auth/authmocks',
'$BUILD_DIR/mongo/db/query/query_request',
- '$BUILD_DIR/mongo/db/service_context_d_test_fixture',
'$BUILD_DIR/mongo/s/catalog/dist_lock_manager_mock',
'$BUILD_DIR/mongo/s/catalog/sharding_catalog_client_impl',
'$BUILD_DIR/mongo/s/catalog/sharding_catalog_client_mock',
@@ -318,8 +309,9 @@ env.CppUnitTest(
)
env.CppUnitTest(
- target='collection_sharding_state_test',
+ target='collection_sharding_runtime_test',
source=[
+ 'collection_metadata_filtering_test.cpp',
'collection_metadata_test.cpp',
'collection_range_deleter_test.cpp',
'collection_sharding_state_test.cpp',
@@ -337,31 +329,6 @@ env.CppUnitTest(
)
env.CppUnitTest(
- target='shard_metadata_util_test',
- source=[
- 'shard_metadata_util_test.cpp',
- ],
- LIBDEPS=[
- '$BUILD_DIR/mongo/db/auth/authmocks',
- '$BUILD_DIR/mongo/s/shard_server_test_fixture',
- 'sharding',
- ],
-)
-
-env.CppUnitTest(
- target='split_vector_test',
- source=[
- 'split_vector_test.cpp',
- ],
- LIBDEPS=[
- '$BUILD_DIR/mongo/db/auth/authmocks',
- '$BUILD_DIR/mongo/db/dbdirectclient',
- '$BUILD_DIR/mongo/s/shard_server_test_fixture',
- 'sharding',
- ]
-)
-
-env.CppUnitTest(
target='session_catalog_migration_source_test',
source=[
'session_catalog_migration_source_test.cpp',
@@ -369,7 +336,6 @@ env.CppUnitTest(
LIBDEPS=[
'$BUILD_DIR/mongo/db/auth/authmocks',
'$BUILD_DIR/mongo/db/repl/mock_repl_coord_server_fixture',
- 'sharding',
]
)
diff --git a/src/mongo/db/s/active_migrations_registry.cpp b/src/mongo/db/s/active_migrations_registry.cpp
index 0e64ee167b2..7ce21a357ad 100644
--- a/src/mongo/db/s/active_migrations_registry.cpp
+++ b/src/mongo/db/s/active_migrations_registry.cpp
@@ -31,7 +31,7 @@
#include "mongo/db/s/active_migrations_registry.h"
#include "mongo/db/catalog_raii.h"
-#include "mongo/db/s/collection_sharding_state.h"
+#include "mongo/db/s/collection_sharding_runtime.h"
#include "mongo/db/s/migration_session_id.h"
#include "mongo/db/s/migration_source_manager.h"
#include "mongo/db/service_context.h"
@@ -121,7 +121,7 @@ BSONObj ActiveMigrationsRegistry::getActiveMigrationStatusReport(OperationContex
AutoGetCollection autoColl(opCtx, nss.get(), MODE_IS);
if (auto msm =
- MigrationSourceManager::get(CollectionShardingState::get(opCtx, nss.get()))) {
+ MigrationSourceManager::get(CollectionShardingRuntime::get(opCtx, nss.get()))) {
return msm->getMigrationStatusReport();
}
}
diff --git a/src/mongo/db/s/add_shard_cmd.idl b/src/mongo/db/s/add_shard_cmd.idl
new file mode 100644
index 00000000000..0ac3c249a90
--- /dev/null
+++ b/src/mongo/db/s/add_shard_cmd.idl
@@ -0,0 +1,44 @@
+# Copyright (C) 2018 MongoDB Inc.
+#
+# This program is free software: you can redistribute it and/or modify
+# it under the terms of the GNU Affero General Public License, version 3,
+# as published by the Free Software Foundation.
+#
+# This program is distributed in the hope that it will be useful,
+# but WITHOUT ANY WARRANTY; without even the implied warranty of
+# MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+# GNU Affero General Public License for more details.
+#
+# You should have received a copy of the GNU Affero General Public License
+# along with this program. If not, see <http://www.gnu.org/licenses/>.
+#
+global:
+ cpp_namespace: "mongo"
+ cpp_includes:
+ - "mongo/client/connection_string.h"
+
+imports:
+ - "mongo/idl/basic_types.idl"
+
+types:
+ connectionstring:
+ bson_serialization_type: string
+ description: "A MongoDB ConnectionString"
+ cpp_type: "mongo::ConnectionString"
+ serializer: mongo::ConnectionString::toString
+ deserializer: mongo::ConnectionString::deserialize
+
+structs:
+ ShardIdentity:
+ description: "Contains all the information needed to identify a shard and lookup the shard identity document from storage"
+ fields:
+ shardName:
+ description: "The name of the shard"
+ type: string
+ clusterId:
+ description: "The unique identifier of the cluster"
+ type: objectid
+ configsvrConnectionString:
+ description: "Connection string to the config server"
+ type: connectionstring
+
diff --git a/src/mongo/db/s/chunk_splitter.cpp b/src/mongo/db/s/chunk_splitter.cpp
index 814c24c227a..5fccf6f8c38 100644
--- a/src/mongo/db/s/chunk_splitter.cpp
+++ b/src/mongo/db/s/chunk_splitter.cpp
@@ -293,10 +293,10 @@ void ChunkSplitter::_runAutosplit(const NamespaceString& nss,
// Stop if chunk's range differs from the range we were expecting to split.
if ((0 != chunk.getMin().woCompare(min)) || (0 != chunk.getMax().woCompare(max)) ||
- (chunk.getShardId() != ShardingState::get(opCtx.get())->getShardName())) {
+ (chunk.getShardId() != ShardingState::get(opCtx.get())->shardId())) {
LOG(1) << "Cannot auto-split chunk with range '"
<< redact(ChunkRange(min, max).toString()) << "' for nss '" << nss
- << "' on shard '" << ShardingState::get(opCtx.get())->getShardName()
+ << "' on shard '" << ShardingState::get(opCtx.get())->shardId()
<< "' because since scheduling auto-split the chunk has been changed to '"
<< redact(chunk.toString()) << "'";
return;
diff --git a/src/mongo/db/s/cleanup_orphaned_cmd.cpp b/src/mongo/db/s/cleanup_orphaned_cmd.cpp
index fdea6235019..49043a59c40 100644
--- a/src/mongo/db/s/cleanup_orphaned_cmd.cpp
+++ b/src/mongo/db/s/cleanup_orphaned_cmd.cpp
@@ -44,7 +44,7 @@
#include "mongo/db/namespace_string.h"
#include "mongo/db/range_arithmetic.h"
#include "mongo/db/s/chunk_move_write_concern_options.h"
-#include "mongo/db/s/collection_sharding_state.h"
+#include "mongo/db/s/collection_sharding_runtime.h"
#include "mongo/db/s/shard_filtering_metadata_refresh.h"
#include "mongo/db/s/sharding_state.h"
#include "mongo/db/service_context.h"
@@ -74,13 +74,14 @@ CleanupResult cleanupOrphanedData(OperationContext* opCtx,
std::string* errMsg) {
BSONObj startingFromKey = startingFromKeyConst;
boost::optional<ChunkRange> targetRange;
- CollectionShardingState::CleanupNotification notifn;
+ CollectionShardingRuntime::CleanupNotification notifn;
{
AutoGetCollection autoColl(opCtx, ns, MODE_IX);
- const auto css = CollectionShardingState::get(opCtx, ns);
+ auto* const css = CollectionShardingRuntime::get(opCtx, ns);
+
auto metadata = css->getMetadata(opCtx);
- if (!metadata) {
+ if (!metadata->isSharded()) {
log() << "skipping orphaned data cleanup for " << ns.toString()
<< ", collection is not sharded";
return CleanupResult_Done;
@@ -110,7 +111,7 @@ CleanupResult cleanupOrphanedData(OperationContext* opCtx,
*stoppedAtKey = targetRange->getMax();
- notifn = css->cleanUpRange(*targetRange, CollectionShardingState::kNow);
+ notifn = css->cleanUpRange(*targetRange, CollectionShardingRuntime::kNow);
}
// Sleep waiting for our own deletion. We don't actually care about any others, so there is no
diff --git a/src/mongo/db/s/collection_metadata.cpp b/src/mongo/db/s/collection_metadata.cpp
index f5c5354e05b..7615850f73b 100644
--- a/src/mongo/db/s/collection_metadata.cpp
+++ b/src/mongo/db/s/collection_metadata.cpp
@@ -34,6 +34,7 @@
#include "mongo/bson/simple_bsonobj_comparator.h"
#include "mongo/bson/util/builder.h"
+#include "mongo/db/bson/dotted_path_support.h"
#include "mongo/s/catalog/type_chunk.h"
#include "mongo/stdx/memory.h"
#include "mongo/util/log.h"
@@ -42,13 +43,11 @@
namespace mongo {
CollectionMetadata::CollectionMetadata(std::shared_ptr<ChunkManager> cm, const ShardId& thisShardId)
- : _cm(std::move(cm)), _thisShardId(thisShardId) {
-
- invariant(_cm->getVersion().isSet());
- invariant(_cm->getVersion() >= getShardVersion());
-}
+ : _cm(std::move(cm)), _thisShardId(thisShardId) {}
RangeMap CollectionMetadata::getChunks() const {
+ invariant(isSharded());
+
RangeMap chunksMap(SimpleBSONObjComparator::kInstance.makeBSONObjIndexedMap<BSONObj>());
for (const auto& chunk : _cm->chunks()) {
@@ -62,6 +61,8 @@ RangeMap CollectionMetadata::getChunks() const {
}
bool CollectionMetadata::getNextChunk(const BSONObj& lookupKey, ChunkType* chunk) const {
+ invariant(isSharded());
+
auto foundIt = _cm->getNextChunkOnShard(lookupKey, _thisShardId);
if (foundIt.begin() == foundIt.end())
return false;
@@ -87,6 +88,8 @@ bool CollectionMetadata::getDifferentChunk(const BSONObj& chunkMinKey,
}
Status CollectionMetadata::checkChunkIsValid(const ChunkType& chunk) const {
+ invariant(isSharded());
+
ChunkType existingChunk;
if (!getNextChunk(chunk.getMin(), &existingChunk)) {
@@ -108,13 +111,41 @@ Status CollectionMetadata::checkChunkIsValid(const ChunkType& chunk) const {
return Status::OK();
}
+BSONObj CollectionMetadata::extractDocumentKey(BSONObj const& doc) const {
+ BSONObj key;
+
+ if (isSharded()) {
+ auto const& pattern = getChunkManager()->getShardKeyPattern();
+ key = dotted_path_support::extractElementsBasedOnTemplate(doc, pattern.toBSON());
+ if (pattern.hasId()) {
+ return key;
+ }
+ // else, try to append an _id field from the document.
+ }
+
+ if (auto id = doc["_id"]) {
+ return key.isEmpty() ? id.wrap() : BSONObjBuilder(std::move(key)).append(id).obj();
+ }
+
+ // For legacy documents that lack an _id, use the document itself as its key.
+ return doc;
+}
+
void CollectionMetadata::toBSONBasic(BSONObjBuilder& bb) const {
- _cm->getVersion().appendLegacyWithField(&bb, "collVersion");
- getShardVersion().appendLegacyWithField(&bb, "shardVersion");
- bb.append("keyPattern", _cm->getShardKeyPattern().toBSON());
+ if (isSharded()) {
+ _cm->getVersion().appendLegacyWithField(&bb, "collVersion");
+ getShardVersion().appendLegacyWithField(&bb, "shardVersion");
+ bb.append("keyPattern", _cm->getShardKeyPattern().toBSON());
+ } else {
+ ChunkVersion::UNSHARDED().appendLegacyWithField(&bb, "collVersion");
+ ChunkVersion::UNSHARDED().appendLegacyWithField(&bb, "shardVersion");
+ }
}
void CollectionMetadata::toBSONChunks(BSONArrayBuilder& bb) const {
+ if (!isSharded())
+ return;
+
for (const auto& chunk : _cm->chunks()) {
if (chunk.getShardId() == _thisShardId) {
BSONArrayBuilder chunkBB(bb.subarrayStart());
@@ -126,12 +157,18 @@ void CollectionMetadata::toBSONChunks(BSONArrayBuilder& bb) const {
}
std::string CollectionMetadata::toStringBasic() const {
- return str::stream() << "collection version: " << _cm->getVersion().toString()
- << ", shard version: " << getShardVersion().toString();
+ if (isSharded()) {
+ return str::stream() << "collection version: " << _cm->getVersion().toString()
+ << ", shard version: " << getShardVersion().toString();
+ } else {
+ return "collection version: <unsharded>";
+ }
}
boost::optional<ChunkRange> CollectionMetadata::getNextOrphanRange(
RangeMap const& receivingChunks, BSONObj const& origLookupKey) const {
+ invariant(isSharded());
+
const BSONObj maxKey = getMaxKey();
BSONObj lookupKey = origLookupKey;
diff --git a/src/mongo/db/s/collection_metadata.h b/src/mongo/db/s/collection_metadata.h
index 759c4d6862e..769ee9d4f87 100644
--- a/src/mongo/db/s/collection_metadata.h
+++ b/src/mongo/db/s/collection_metadata.h
@@ -48,6 +48,12 @@ namespace mongo {
class CollectionMetadata {
public:
/**
+ * Instantiates a metadata object, which represents an unsharded collection. This 'isSharded'
+ * for this object will return false and it is illegal to use it for filtering.
+ */
+ CollectionMetadata() = default;
+
+ /**
* The main way to construct CollectionMetadata is through MetadataLoader or clone() methods.
*
* "thisShardId" is the shard identity of this shard for purposes of answering questions like
@@ -56,9 +62,61 @@ public:
CollectionMetadata(std::shared_ptr<ChunkManager> cm, const ShardId& thisShardId);
/**
+ * Returns whether this metadata object represents a sharded collection which requires
+ * filtering.
+ */
+ bool isSharded() const {
+ return bool(_cm);
+ }
+
+ /**
+ * Returns the current shard version for the collection or UNSHARDED if it is not sharded.
+ */
+ ChunkVersion getShardVersion() const {
+ return (isSharded() ? _cm->getVersion(_thisShardId) : ChunkVersion::UNSHARDED());
+ }
+
+ /**
+ * Returns the current collection version or UNSHARDED if it is not sharded.
+ */
+ ChunkVersion getCollVersion() const {
+ return (isSharded() ? _cm->getVersion() : ChunkVersion::UNSHARDED());
+ }
+
+ /**
+ * Returns just the shard key fields, if the collection is sharded, and the _id field, from
+ * `doc`. Does not alter any field values (e.g. by hashing); values are copied verbatim.
+ */
+ BSONObj extractDocumentKey(const BSONObj& doc) const;
+
+ /**
+ * BSON output of the basic metadata information (chunk and shard version).
+ */
+ void toBSONBasic(BSONObjBuilder& bb) const;
+
+ /**
+ * BSON output of the chunks metadata into a BSONArray
+ */
+ void toBSONChunks(BSONArrayBuilder& bb) const;
+
+ /**
+ * String output of the collection and shard versions.
+ */
+ std::string toStringBasic() const;
+
+ /**
+ * Obtains the shard id with which this collection metadata is configured.
+ */
+ const ShardId& shardId() const {
+ invariant(isSharded());
+ return _thisShardId;
+ }
+
+ /**
* Returns true if 'key' contains exactly the same fields as the shard key pattern.
*/
bool isValidKey(const BSONObj& key) const {
+ invariant(isSharded());
return _cm->getShardKeyPattern().isShardKey(key);
}
@@ -67,6 +125,7 @@ public:
* returns false. If key is not a valid shard key, the behaviour is undefined.
*/
bool keyBelongsToMe(const BSONObj& key) const {
+ invariant(isSharded());
return _cm->keyBelongsToShard(key, _thisShardId);
}
@@ -92,6 +151,7 @@ public:
* Returns true if the argument range overlaps any chunk.
*/
bool rangeOverlapsChunk(ChunkRange const& range) const {
+ invariant(isSharded());
return _cm->rangeOverlapsShard(range, _thisShardId);
}
@@ -116,52 +176,38 @@ public:
boost::optional<ChunkRange> getNextOrphanRange(RangeMap const& receiveMap,
BSONObj const& lookupKey) const;
- ChunkVersion getCollVersion() const {
- return _cm->getVersion();
- }
-
- ChunkVersion getShardVersion() const {
- return _cm->getVersion(_thisShardId);
- }
-
+ /**
+ * Returns all the chunks which are contained on this shard.
+ */
RangeMap getChunks() const;
const BSONObj& getKeyPattern() const {
+ invariant(isSharded());
return _cm->getShardKeyPattern().toBSON();
}
const std::vector<std::unique_ptr<FieldRef>>& getKeyPatternFields() const {
+ invariant(isSharded());
return _cm->getShardKeyPattern().getKeyPatternFields();
}
BSONObj getMinKey() const {
+ invariant(isSharded());
return _cm->getShardKeyPattern().getKeyPattern().globalMin();
}
BSONObj getMaxKey() const {
+ invariant(isSharded());
return _cm->getShardKeyPattern().getKeyPattern().globalMax();
}
- /**
- * BSON output of the basic metadata information (chunk and shard version).
- */
- void toBSONBasic(BSONObjBuilder& bb) const;
-
- /**
- * BSON output of the chunks metadata into a BSONArray
- */
- void toBSONChunks(BSONArrayBuilder& bb) const;
-
- /**
- * String output of the collection and shard versions.
- */
- std::string toStringBasic() const;
-
std::shared_ptr<ChunkManager> getChunkManager() const {
+ invariant(isSharded());
return _cm;
}
bool uuidMatches(UUID uuid) const {
+ invariant(isSharded());
return _cm->uuidMatches(uuid);
}
diff --git a/src/mongo/db/s/collection_metadata_filtering_test.cpp b/src/mongo/db/s/collection_metadata_filtering_test.cpp
new file mode 100644
index 00000000000..3193a8e53a8
--- /dev/null
+++ b/src/mongo/db/s/collection_metadata_filtering_test.cpp
@@ -0,0 +1,230 @@
+/**
+ * Copyright (C) 2018 MongoDB Inc.
+ *
+ * This program is free software: you can redistribute it and/or modify
+ * it under the terms of the GNU Affero General Public License, version 3,
+ * as published by the Free Software Foundation.
+ *
+ * This program is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ * GNU Affero General Public License for more details.
+ *
+ * You should have received a copy of the GNU Affero General Public License
+ * along with this program. If not, see <http://www.gnu.org/licenses/>.
+ *
+ * As a special exception, the copyright holders give permission to link the
+ * code of portions of this program with the OpenSSL library under certain
+ * conditions as described in each individual source file and distribute
+ * linked combinations including the program with the OpenSSL library. You
+ * must comply with the GNU Affero General Public License in all respects for
+ * all of the code used other than as permitted herein. If you modify file(s)
+ * with this exception, you may extend this exception to your version of the
+ * file(s), but you are not obligated to do so. If you do not wish to do so,
+ * delete this exception statement from your version. If you delete this
+ * exception statement from all source files in the program, then also delete
+ * it in the license file.
+ */
+
+#include "mongo/platform/basic.h"
+
+#include "mongo/db/catalog_raii.h"
+#include "mongo/db/s/collection_sharding_runtime.h"
+#include "mongo/s/catalog/type_chunk.h"
+#include "mongo/s/shard_server_test_fixture.h"
+
+namespace mongo {
+namespace {
+
+const NamespaceString kNss("TestDB", "TestColl");
+
+class CollectionMetadataFilteringTest : public ShardServerTestFixture {
+protected:
+ void setUp() override {
+ ShardServerTestFixture::setUp();
+ _manager = std::make_shared<MetadataManager>(getServiceContext(), kNss, executor());
+ }
+
+ // Prepares data with a history array populated:
+ // chunk1 - [min, -100)
+ // chunk2 - [100, 0)
+ // chunk3 - [0, 100)
+ // chunk4 - [100, max)
+ // and the history:
+ // time (now,75) shard0(chunk1, chunk3) shard1(chunk2, chunk4)
+ // time (75,25) shard0(chunk2, chunk4) shard1(chunk1, chunk3)
+ // time (25,0) - no history
+ void prepareTestData() {
+ const OID epoch = OID::gen();
+ const ShardKeyPattern shardKeyPattern(BSON("_id" << 1));
+
+ auto rt = RoutingTableHistory::makeNew(
+ kNss, UUID::gen(), shardKeyPattern.getKeyPattern(), nullptr, false, epoch, [&] {
+ ChunkVersion version(1, 0, epoch);
+
+ ChunkType chunk1(kNss,
+ {shardKeyPattern.getKeyPattern().globalMin(), BSON("_id" << -100)},
+ version,
+ {"0"});
+ chunk1.setHistory({ChunkHistory(Timestamp(75, 0), ShardId("0")),
+ ChunkHistory(Timestamp(25, 0), ShardId("1"))});
+ version.incMinor();
+
+ ChunkType chunk2(kNss, {BSON("_id" << -100), BSON("_id" << 0)}, version, {"1"});
+ chunk2.setHistory({ChunkHistory(Timestamp(75, 0), ShardId("1")),
+ ChunkHistory(Timestamp(25, 0), ShardId("0"))});
+ version.incMinor();
+
+ ChunkType chunk3(kNss, {BSON("_id" << 0), BSON("_id" << 100)}, version, {"0"});
+ chunk3.setHistory({ChunkHistory(Timestamp(75, 0), ShardId("0")),
+ ChunkHistory(Timestamp(25, 0), ShardId("1"))});
+ version.incMinor();
+
+ ChunkType chunk4(kNss,
+ {BSON("_id" << 100), shardKeyPattern.getKeyPattern().globalMax()},
+ version,
+ {"1"});
+ chunk4.setHistory({ChunkHistory(Timestamp(75, 0), ShardId("1")),
+ ChunkHistory(Timestamp(25, 0), ShardId("0"))});
+ version.incMinor();
+
+ return std::vector<ChunkType>{chunk1, chunk2, chunk3, chunk4};
+ }());
+
+ auto cm = std::make_shared<ChunkManager>(rt, Timestamp(100, 0));
+ ASSERT_EQ(4, cm->numChunks());
+ {
+ AutoGetCollection autoColl(operationContext(), kNss, MODE_X);
+ auto* const css = CollectionShardingRuntime::get(operationContext(), kNss);
+
+ css->refreshMetadata(operationContext(),
+ std::make_unique<CollectionMetadata>(cm, ShardId("0")));
+ }
+
+ _manager->refreshActiveMetadata(std::make_unique<CollectionMetadata>(cm, ShardId("0")));
+ }
+
+ std::shared_ptr<MetadataManager> _manager;
+};
+
+// Verifies that right set of documents is visible.
+TEST_F(CollectionMetadataFilteringTest, FilterDocumentsPresent) {
+ prepareTestData();
+
+ auto metadata = _manager->getActiveMetadata(_manager, LogicalTime(Timestamp(100, 0)));
+
+ ASSERT_TRUE(metadata->keyBelongsToMe(BSON("_id" << -500)));
+ ASSERT_TRUE(metadata->keyBelongsToMe(BSON("_id" << 50)));
+ ASSERT_FALSE(metadata->keyBelongsToMe(BSON("_id" << -50)));
+ ASSERT_FALSE(metadata->keyBelongsToMe(BSON("_id" << 500)));
+}
+
+// Verifies that a different set of documents is visible for a timestamp in the past.
+TEST_F(CollectionMetadataFilteringTest, FilterDocumentsPast) {
+ prepareTestData();
+
+ auto metadata = _manager->getActiveMetadata(_manager, LogicalTime(Timestamp(50, 0)));
+
+ ASSERT_FALSE(metadata->keyBelongsToMe(BSON("_id" << -500)));
+ ASSERT_FALSE(metadata->keyBelongsToMe(BSON("_id" << 50)));
+ ASSERT_TRUE(metadata->keyBelongsToMe(BSON("_id" << -50)));
+ ASSERT_TRUE(metadata->keyBelongsToMe(BSON("_id" << 500)));
+}
+
+// Verifies that when accessing too far into the past we get the stale error.
+TEST_F(CollectionMetadataFilteringTest, FilterDocumentsStale) {
+ prepareTestData();
+
+ auto metadata = _manager->getActiveMetadata(_manager, LogicalTime(Timestamp(10, 0)));
+
+ ASSERT_THROWS_CODE(metadata->keyBelongsToMe(BSON("_id" << -500)),
+ AssertionException,
+ ErrorCodes::StaleChunkHistory);
+ ASSERT_THROWS_CODE(metadata->keyBelongsToMe(BSON("_id" << 50)),
+ AssertionException,
+ ErrorCodes::StaleChunkHistory);
+ ASSERT_THROWS_CODE(metadata->keyBelongsToMe(BSON("_id" << -50)),
+ AssertionException,
+ ErrorCodes::StaleChunkHistory);
+ ASSERT_THROWS_CODE(metadata->keyBelongsToMe(BSON("_id" << 500)),
+ AssertionException,
+ ErrorCodes::StaleChunkHistory);
+}
+
+// The same test as FilterDocumentsPresent but using "readConcern"
+TEST_F(CollectionMetadataFilteringTest, FilterDocumentsPresentShardingState) {
+ prepareTestData();
+
+ BSONObj readConcern = BSON("readConcern" << BSON("level"
+ << "snapshot"
+ << "atClusterTime"
+ << Timestamp(100, 0)));
+
+ auto&& readConcernArgs = repl::ReadConcernArgs::get(operationContext());
+ ASSERT_OK(readConcernArgs.initialize(readConcern["readConcern"]));
+
+ AutoGetCollection autoColl(operationContext(), kNss, MODE_IS);
+ auto const css = CollectionShardingState::get(operationContext(), kNss);
+ auto metadata = css->getMetadata(operationContext());
+
+ ASSERT_TRUE(metadata->keyBelongsToMe(BSON("_id" << -500)));
+ ASSERT_TRUE(metadata->keyBelongsToMe(BSON("_id" << 50)));
+ ASSERT_FALSE(metadata->keyBelongsToMe(BSON("_id" << -50)));
+ ASSERT_FALSE(metadata->keyBelongsToMe(BSON("_id" << 500)));
+}
+
+// The same test as FilterDocumentsPast but using "readConcern"
+TEST_F(CollectionMetadataFilteringTest, FilterDocumentsPastShardingState) {
+ prepareTestData();
+
+ BSONObj readConcern = BSON("readConcern" << BSON("level"
+ << "snapshot"
+ << "atClusterTime"
+ << Timestamp(50, 0)));
+
+ auto&& readConcernArgs = repl::ReadConcernArgs::get(operationContext());
+ ASSERT_OK(readConcernArgs.initialize(readConcern["readConcern"]));
+
+ AutoGetCollection autoColl(operationContext(), kNss, MODE_IS);
+ auto const css = CollectionShardingState::get(operationContext(), kNss);
+ auto metadata = css->getMetadata(operationContext());
+
+ ASSERT_FALSE(metadata->keyBelongsToMe(BSON("_id" << -500)));
+ ASSERT_FALSE(metadata->keyBelongsToMe(BSON("_id" << 50)));
+ ASSERT_TRUE(metadata->keyBelongsToMe(BSON("_id" << -50)));
+ ASSERT_TRUE(metadata->keyBelongsToMe(BSON("_id" << 500)));
+}
+
+// The same test as FilterDocumentsStale but using "readConcern"
+TEST_F(CollectionMetadataFilteringTest, FilterDocumentsStaleShardingState) {
+ prepareTestData();
+
+ BSONObj readConcern = BSON("readConcern" << BSON("level"
+ << "snapshot"
+ << "atClusterTime"
+ << Timestamp(10, 0)));
+
+ auto&& readConcernArgs = repl::ReadConcernArgs::get(operationContext());
+ ASSERT_OK(readConcernArgs.initialize(readConcern["readConcern"]));
+
+ AutoGetCollection autoColl(operationContext(), kNss, MODE_IS);
+ auto const css = CollectionShardingState::get(operationContext(), kNss);
+ auto metadata = css->getMetadata(operationContext());
+
+ ASSERT_THROWS_CODE(metadata->keyBelongsToMe(BSON("_id" << -500)),
+ AssertionException,
+ ErrorCodes::StaleChunkHistory);
+ ASSERT_THROWS_CODE(metadata->keyBelongsToMe(BSON("_id" << 50)),
+ AssertionException,
+ ErrorCodes::StaleChunkHistory);
+ ASSERT_THROWS_CODE(metadata->keyBelongsToMe(BSON("_id" << -50)),
+ AssertionException,
+ ErrorCodes::StaleChunkHistory);
+ ASSERT_THROWS_CODE(metadata->keyBelongsToMe(BSON("_id" << 500)),
+ AssertionException,
+ ErrorCodes::StaleChunkHistory);
+}
+
+
+} // namespace
+} // namespace mongo
diff --git a/src/mongo/db/s/collection_range_deleter.cpp b/src/mongo/db/s/collection_range_deleter.cpp
index 5e1c358aa01..2bf06914bd5 100644
--- a/src/mongo/db/s/collection_range_deleter.cpp
+++ b/src/mongo/db/s/collection_range_deleter.cpp
@@ -48,7 +48,7 @@
#include "mongo/db/query/query_knobs.h"
#include "mongo/db/query/query_planner.h"
#include "mongo/db/repl/repl_client_info.h"
-#include "mongo/db/s/collection_sharding_state.h"
+#include "mongo/db/s/collection_sharding_runtime.h"
#include "mongo/db/s/sharding_state.h"
#include "mongo/db/server_parameters.h"
#include "mongo/db/service_context.h"
@@ -116,12 +116,14 @@ boost::optional<Date_t> CollectionRangeDeleter::cleanUpNextRange(
AutoGetCollection autoColl(opCtx, nss, MODE_IX);
auto* const collection = autoColl.getCollection();
- auto* const css = CollectionShardingState::get(opCtx, nss);
- auto* const self = forTestOnly ? forTestOnly : &css->_metadataManager->_rangesToClean;
+ auto* const css = CollectionShardingRuntime::get(opCtx, nss);
+ auto& metadataManager = css->_metadataManager;
+ auto* const self = forTestOnly ? forTestOnly : &metadataManager->_rangesToClean;
- auto scopedCollectionMetadata = css->getMetadata(opCtx);
+ const auto scopedCollectionMetadata =
+ metadataManager->getActiveMetadata(metadataManager, boost::none);
- if (!forTestOnly && (!collection || !scopedCollectionMetadata)) {
+ if (!forTestOnly && (!collection || !scopedCollectionMetadata->isSharded())) {
if (!collection) {
LOG(0) << "Abandoning any range deletions left over from dropped " << nss.ns();
} else {
@@ -206,8 +208,8 @@ boost::optional<Date_t> CollectionRangeDeleter::cleanUpNextRange(
}
try {
- const auto keyPattern = scopedCollectionMetadata->getKeyPattern();
- wrote = self->_doDeletion(opCtx, collection, keyPattern, *range, maxToDelete);
+ wrote = self->_doDeletion(
+ opCtx, collection, scopedCollectionMetadata->getKeyPattern(), *range, maxToDelete);
} catch (const DBException& e) {
wrote = e.toStatus();
warning() << e.what();
@@ -245,8 +247,10 @@ boost::optional<Date_t> CollectionRangeDeleter::cleanUpNextRange(
// Don't allow lock interrupts while cleaning up.
UninterruptibleLockGuard noInterrupt(opCtx->lockState());
AutoGetCollection autoColl(opCtx, nss, MODE_IX);
- auto* const css = CollectionShardingState::get(opCtx, nss);
- auto* const self = forTestOnly ? forTestOnly : &css->_metadataManager->_rangesToClean;
+ auto* const css = CollectionShardingRuntime::get(opCtx, nss);
+ auto& metadataManager = css->_metadataManager;
+ auto* const self = forTestOnly ? forTestOnly : &metadataManager->_rangesToClean;
+
stdx::lock_guard<stdx::mutex> scopedLock(css->_metadataManager->_managerLock);
if (!status.isOK()) {
diff --git a/src/mongo/db/s/collection_range_deleter_test.cpp b/src/mongo/db/s/collection_range_deleter_test.cpp
index 2f297f0794c..61768efce87 100644
--- a/src/mongo/db/s/collection_range_deleter_test.cpp
+++ b/src/mongo/db/s/collection_range_deleter_test.cpp
@@ -38,7 +38,7 @@
#include "mongo/db/index/index_descriptor.h"
#include "mongo/db/keypattern.h"
#include "mongo/db/repl/replication_coordinator_mock.h"
-#include "mongo/db/s/collection_sharding_state.h"
+#include "mongo/db/s/collection_sharding_runtime.h"
#include "mongo/db/s/sharding_state.h"
#include "mongo/s/balancer_configuration.h"
#include "mongo/s/chunk_version.h"
@@ -84,7 +84,8 @@ protected:
std::shared_ptr<ChunkManager> cm = std::make_shared<ChunkManager>(rt, Timestamp(100, 0));
AutoGetCollection autoColl(operationContext(), kNss, MODE_IX);
- auto const css = CollectionShardingState::get(operationContext(), kNss);
+ auto* const css = CollectionShardingRuntime::get(operationContext(), kNss);
+
css->refreshMetadata(operationContext(),
stdx::make_unique<CollectionMetadata>(cm, ShardId("thisShard")));
}
@@ -92,8 +93,9 @@ protected:
void tearDown() override {
{
AutoGetCollection autoColl(operationContext(), kNss, MODE_IX);
- auto collectionShardingState = CollectionShardingState::get(operationContext(), kNss);
- collectionShardingState->refreshMetadata(operationContext(), nullptr);
+ auto* const css = CollectionShardingRuntime::get(operationContext(), kNss);
+
+ css->refreshMetadata(operationContext(), nullptr);
}
ShardServerTestFixture::tearDown();
diff --git a/src/mongo/db/s/collection_sharding_runtime.cpp b/src/mongo/db/s/collection_sharding_runtime.cpp
new file mode 100644
index 00000000000..770d4a2af4b
--- /dev/null
+++ b/src/mongo/db/s/collection_sharding_runtime.cpp
@@ -0,0 +1,162 @@
+/**
+ * Copyright (C) 2018 MongoDB Inc.
+ *
+ * This program is free software: you can redistribute it and/or modify
+ * it under the terms of the GNU Affero General Public License, version 3,
+ * as published by the Free Software Foundation.
+ *
+ * This program is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ * GNU Affero General Public License for more details.
+ *
+ * You should have received a copy of the GNU Affero General Public License
+ * along with this program. If not, see <http://www.gnu.org/licenses/>.
+ *
+ * As a special exception, the copyright holders give permission to link the
+ * code of portions of this program with the OpenSSL library under certain
+ * conditions as described in each individual source file and distribute
+ * linked combinations including the program with the OpenSSL library. You
+ * must comply with the GNU Affero General Public License in all respects
+ * for all of the code used other than as permitted herein. If you modify
+ * file(s) with this exception, you may extend this exception to your
+ * version of the file(s), but you are not obligated to do so. If you do not
+ * wish to do so, delete this exception statement from your version. If you
+ * delete this exception statement from all source files in the program,
+ * then also delete it in the license file.
+ */
+
+#define MONGO_LOG_DEFAULT_COMPONENT ::mongo::logger::LogComponent::kSharding
+
+#include "mongo/platform/basic.h"
+
+#include "mongo/db/s/collection_sharding_runtime.h"
+
+#include "mongo/base/checked_cast.h"
+#include "mongo/db/catalog_raii.h"
+#include "mongo/db/server_parameters.h"
+#include "mongo/util/log.h"
+
+namespace mongo {
+namespace {
+
+// How long to wait before starting cleanup of an emigrated chunk range
+MONGO_EXPORT_SERVER_PARAMETER(orphanCleanupDelaySecs, int, 900); // 900s = 15m
+
+} // namespace
+
+CollectionShardingRuntime::CollectionShardingRuntime(ServiceContext* sc,
+ NamespaceString nss,
+ executor::TaskExecutor* rangeDeleterExecutor)
+ : CollectionShardingState(nss),
+ _nss(std::move(nss)),
+ _metadataManager(std::make_shared<MetadataManager>(sc, _nss, rangeDeleterExecutor)) {}
+
+CollectionShardingRuntime* CollectionShardingRuntime::get(OperationContext* opCtx,
+ const NamespaceString& nss) {
+ auto* const css = CollectionShardingState::get(opCtx, nss);
+ return checked_cast<CollectionShardingRuntime*>(css);
+}
+
+void CollectionShardingRuntime::refreshMetadata(OperationContext* opCtx,
+ std::unique_ptr<CollectionMetadata> newMetadata) {
+ invariant(opCtx->lockState()->isCollectionLockedForMode(_nss.ns(), MODE_X));
+
+ _metadataManager->refreshActiveMetadata(std::move(newMetadata));
+}
+
+void CollectionShardingRuntime::markNotShardedAtStepdown() {
+ _metadataManager->refreshActiveMetadata(nullptr);
+}
+
+auto CollectionShardingRuntime::beginReceive(ChunkRange const& range) -> CleanupNotification {
+ return _metadataManager->beginReceive(range);
+}
+
+void CollectionShardingRuntime::forgetReceive(const ChunkRange& range) {
+ _metadataManager->forgetReceive(range);
+}
+
+auto CollectionShardingRuntime::cleanUpRange(ChunkRange const& range, CleanWhen when)
+ -> CleanupNotification {
+ Date_t time = (when == kNow) ? Date_t{} : Date_t::now() +
+ stdx::chrono::seconds{orphanCleanupDelaySecs.load()};
+ return _metadataManager->cleanUpRange(range, time);
+}
+
+Status CollectionShardingRuntime::waitForClean(OperationContext* opCtx,
+ const NamespaceString& nss,
+ OID const& epoch,
+ ChunkRange orphanRange) {
+ while (true) {
+ boost::optional<CleanupNotification> stillScheduled;
+
+ {
+ AutoGetCollection autoColl(opCtx, nss, MODE_IX);
+ auto* const self = CollectionShardingRuntime::get(opCtx, nss);
+
+ {
+ // First, see if collection was dropped, but do it in a separate scope in order to
+ // not hold reference on it, which would make it appear in use
+ auto metadata =
+ self->_metadataManager->getActiveMetadata(self->_metadataManager, boost::none);
+
+ if (!metadata->isSharded() || metadata->getCollVersion().epoch() != epoch) {
+ return {ErrorCodes::ConflictingOperationInProgress,
+ "Collection being migrated was dropped"};
+ }
+ }
+
+ stillScheduled = self->trackOrphanedDataCleanup(orphanRange);
+ if (!stillScheduled) {
+ log() << "Finished deleting " << nss.ns() << " range "
+ << redact(orphanRange.toString());
+ return Status::OK();
+ }
+ }
+
+ log() << "Waiting for deletion of " << nss.ns() << " range " << orphanRange;
+
+ Status result = stillScheduled->waitStatus(opCtx);
+ if (!result.isOK()) {
+ return result.withContext(str::stream() << "Failed to delete orphaned " << nss.ns()
+ << " range "
+ << orphanRange.toString());
+ }
+ }
+
+ MONGO_UNREACHABLE;
+}
+
+auto CollectionShardingRuntime::trackOrphanedDataCleanup(ChunkRange const& range)
+ -> boost::optional<CleanupNotification> {
+ return _metadataManager->trackOrphanedDataCleanup(range);
+}
+
+boost::optional<ChunkRange> CollectionShardingRuntime::getNextOrphanRange(BSONObj const& from) {
+ return _metadataManager->getNextOrphanRange(from);
+}
+
+ScopedCollectionMetadata CollectionShardingRuntime::_getMetadata(OperationContext* opCtx) {
+ auto atClusterTime = repl::ReadConcernArgs::get(opCtx).getArgsAtClusterTime();
+ return _metadataManager->getActiveMetadata(_metadataManager, atClusterTime);
+}
+
+CollectionCriticalSection::CollectionCriticalSection(OperationContext* opCtx, NamespaceString ns)
+ : _nss(std::move(ns)), _opCtx(opCtx) {
+ AutoGetCollection autoColl(_opCtx, _nss, MODE_IX, MODE_X);
+ CollectionShardingState::get(opCtx, _nss)->enterCriticalSectionCatchUpPhase(_opCtx);
+}
+
+CollectionCriticalSection::~CollectionCriticalSection() {
+ UninterruptibleLockGuard noInterrupt(_opCtx->lockState());
+ AutoGetCollection autoColl(_opCtx, _nss, MODE_IX, MODE_X);
+ CollectionShardingState::get(_opCtx, _nss)->exitCriticalSection(_opCtx);
+}
+
+void CollectionCriticalSection::enterCommitPhase() {
+ AutoGetCollection autoColl(_opCtx, _nss, MODE_IX, MODE_X);
+ CollectionShardingState::get(_opCtx, _nss)->enterCriticalSectionCommitPhase(_opCtx);
+}
+
+} // namespace mongo
diff --git a/src/mongo/db/s/collection_sharding_runtime.h b/src/mongo/db/s/collection_sharding_runtime.h
new file mode 100644
index 00000000000..0515fde5ab0
--- /dev/null
+++ b/src/mongo/db/s/collection_sharding_runtime.h
@@ -0,0 +1,171 @@
+/**
+ * Copyright (C) 2018 MongoDB Inc.
+ *
+ * This program is free software: you can redistribute it and/or modify
+ * it under the terms of the GNU Affero General Public License, version 3,
+ * as published by the Free Software Foundation.
+ *
+ * This program is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ * GNU Affero General Public License for more details.
+ *
+ * You should have received a copy of the GNU Affero General Public License
+ * along with this program. If not, see <http://www.gnu.org/licenses/>.
+ *
+ * As a special exception, the copyright holders give permission to link the
+ * code of portions of this program with the OpenSSL library under certain
+ * conditions as described in each individual source file and distribute
+ * linked combinations including the program with the OpenSSL library. You
+ * must comply with the GNU Affero General Public License in all respects
+ * for all of the code used other than as permitted herein. If you modify
+ * file(s) with this exception, you may extend this exception to your
+ * version of the file(s), but you are not obligated to do so. If you do not
+ * wish to do so, delete this exception statement from your version. If you
+ * delete this exception statement from all source files in the program,
+ * then also delete it in the license file.
+ */
+
+#pragma once
+
+#include "mongo/base/disallow_copying.h"
+#include "mongo/db/namespace_string.h"
+#include "mongo/db/s/collection_sharding_state.h"
+#include "mongo/db/s/metadata_manager.h"
+#include "mongo/util/decorable.h"
+
+namespace mongo {
+
+/**
+ * See the comments for CollectionShardingState for more information on how this class fits in the
+ * sharding architecture.
+ */
+class CollectionShardingRuntime final : public CollectionShardingState,
+ public Decorable<CollectionShardingRuntime> {
+ MONGO_DISALLOW_COPYING(CollectionShardingRuntime);
+
+public:
+ CollectionShardingRuntime(ServiceContext* sc,
+ NamespaceString nss,
+ executor::TaskExecutor* rangeDeleterExecutor);
+
+ /**
+ * Obtains the sharding state for the specified collection. If it does not exist, it will be
+ * created and will remain active until the collection is dropped or unsharded.
+ *
+ * Must be called with some lock held on the specific collection being looked up and the
+ * returned pointer should never be stored.
+ */
+ static CollectionShardingRuntime* get(OperationContext* opCtx, const NamespaceString& nss);
+
+ /**
+ * Updates the metadata based on changes received from the config server and also resolves the
+ * pending receives map in case some of these pending receives have completed or have been
+ * abandoned. If newMetadata is null, unshard the collection.
+ *
+ * Must always be called with an exclusive collection lock.
+ */
+ void refreshMetadata(OperationContext* opCtx, std::unique_ptr<CollectionMetadata> newMetadata);
+
+ /**
+ * Marks the collection as not sharded at stepdown time so that no filtering will occur for
+ * slaveOk queries.
+ */
+ void markNotShardedAtStepdown();
+
+ /**
+ * Schedules any documents in `range` for immediate cleanup iff no running queries can depend
+ * on them, and adds the range to the list of pending ranges. Otherwise, returns a notification
+ * that yields bad status immediately. Does not block. Call waitStatus(opCtx) on the result
+ * to wait for the deletion to complete or fail. After that, call waitForClean to ensure no
+ * other deletions are pending for the range.
+ */
+ using CleanupNotification = CollectionRangeDeleter::DeleteNotification;
+ CleanupNotification beginReceive(ChunkRange const& range);
+
+ /*
+ * Removes `range` from the list of pending ranges, and schedules any documents in the range for
+ * immediate cleanup. Does not block.
+ */
+ void forgetReceive(const ChunkRange& range);
+
+ /**
+ * Schedules documents in `range` for cleanup after any running queries that may depend on them
+ * have terminated. Does not block. Fails if range overlaps any current local shard chunk.
+ * Passed kDelayed, an additional delay (configured via server parameter orphanCleanupDelaySecs)
+ * is added to permit (most) dependent queries on secondaries to complete, too.
+ *
+ * Call result.waitStatus(opCtx) to wait for the deletion to complete or fail. If that succeeds,
+ * waitForClean can be called to ensure no other deletions are pending for the range. Call
+ * result.abandon(), instead of waitStatus, to ignore the outcome.
+ */
+ enum CleanWhen { kNow, kDelayed };
+ CleanupNotification cleanUpRange(ChunkRange const& range, CleanWhen when);
+
+ /**
+ * Tracks deletion of any documents within the range, returning when deletion is complete.
+ * Throws if the collection is dropped while it sleeps.
+ */
+ static Status waitForClean(OperationContext* opCtx,
+ const NamespaceString& nss,
+ OID const& epoch,
+ ChunkRange orphanRange);
+
+ /**
+ * Reports whether any range still scheduled for deletion overlaps the argument range. If so,
+ * it returns a notification n such that n->get(opCtx) will wake when the newest overlapping
+ * range's deletion (possibly the one of interest) completes or fails. This should be called
+ * again after each wakeup until it returns boost::none, because there can be more than one
+ * range scheduled for deletion that overlaps its argument.
+ */
+ auto trackOrphanedDataCleanup(ChunkRange const& range) -> boost::optional<CleanupNotification>;
+
+ /**
+ * Returns a range _not_ owned by this shard that starts no lower than the specified
+ * startingFrom key value, if any, or boost::none if there is no such range.
+ */
+ boost::optional<ChunkRange> getNextOrphanRange(BSONObj const& startingFrom);
+
+ /**
+ * BSON output of the pending metadata into a BSONArray
+ */
+ void toBSONPending(BSONArrayBuilder& bb) const {
+ _metadataManager->toBSONPending(bb);
+ }
+
+
+private:
+ friend boost::optional<Date_t> CollectionRangeDeleter::cleanUpNextRange(
+ OperationContext*, NamespaceString const&, OID const&, int, CollectionRangeDeleter*);
+
+ // Namespace this state belongs to.
+ const NamespaceString _nss;
+
+ // Contains all the metadata associated with this collection.
+ std::shared_ptr<MetadataManager> _metadataManager;
+
+ ScopedCollectionMetadata _getMetadata(OperationContext* opCtx) override;
+};
+
+/**
+ * RAII-style class, which obtains a reference to the critical section for the specified collection.
+ */
+class CollectionCriticalSection {
+ MONGO_DISALLOW_COPYING(CollectionCriticalSection);
+
+public:
+ CollectionCriticalSection(OperationContext* opCtx, NamespaceString ns);
+ ~CollectionCriticalSection();
+
+ /**
+ * Enters the commit phase of the critical section and blocks reads.
+ */
+ void enterCommitPhase();
+
+private:
+ NamespaceString _nss;
+
+ OperationContext* _opCtx;
+};
+
+} // namespace mongo
diff --git a/src/mongo/db/s/collection_sharding_state.cpp b/src/mongo/db/s/collection_sharding_state.cpp
index 9626862cd0d..2b16652e3c1 100644
--- a/src/mongo/db/s/collection_sharding_state.cpp
+++ b/src/mongo/db/s/collection_sharding_state.cpp
@@ -32,91 +32,29 @@
#include "mongo/db/s/collection_sharding_state.h"
-#include "mongo/bson/bsonobjbuilder.h"
-#include "mongo/bson/util/bson_extract.h"
-#include "mongo/db/catalog_raii.h"
-#include "mongo/db/client.h"
-#include "mongo/db/operation_context.h"
#include "mongo/db/s/operation_sharding_state.h"
#include "mongo/db/s/sharded_connection_info.h"
-#include "mongo/db/server_parameters.h"
-#include "mongo/db/service_context.h"
-#include "mongo/executor/network_interface_factory.h"
-#include "mongo/executor/network_interface_thread_pool.h"
-#include "mongo/executor/thread_pool_task_executor.h"
#include "mongo/s/stale_exception.h"
#include "mongo/util/log.h"
namespace mongo {
namespace {
-// How long to wait before starting cleanup of an emigrated chunk range
-MONGO_EXPORT_SERVER_PARAMETER(orphanCleanupDelaySecs, int, 900); // 900s = 15m
-
-/**
- * Lazy-instantiated task executor shared by the collection range deleters. Must outlive the
- * CollectionShardingStateMap below.
- */
-class RangeDeleterExecutorHolder {
- MONGO_DISALLOW_COPYING(RangeDeleterExecutorHolder);
-
-public:
- RangeDeleterExecutorHolder() = default;
-
- ~RangeDeleterExecutorHolder() {
- if (_taskExecutor) {
- _taskExecutor->shutdown();
- _taskExecutor->join();
- }
- }
-
- executor::TaskExecutor* getOrCreateExecutor() {
- stdx::lock_guard<stdx::mutex> lg(_mutex);
-
- if (!_taskExecutor) {
- const std::string kExecName("CollectionRangeDeleter-TaskExecutor");
-
- auto net = executor::makeNetworkInterface(kExecName);
- auto pool = stdx::make_unique<executor::NetworkInterfaceThreadPool>(net.get());
- auto taskExecutor = stdx::make_unique<executor::ThreadPoolTaskExecutor>(std::move(pool),
- std::move(net));
- taskExecutor->startup();
-
- _taskExecutor = std::move(taskExecutor);
- }
-
- return _taskExecutor.get();
- }
-
-private:
- stdx::mutex _mutex;
- std::unique_ptr<executor::TaskExecutor> _taskExecutor{nullptr};
-};
-
-const auto getRangeDeleterExecutorHolder =
- ServiceContext::declareDecoration<RangeDeleterExecutorHolder>();
-
-/**
- * This map matches 1:1 with the set of collections in the storage catalog. It is not safe to
- * look-up values from this map without holding some form of collection lock. It is only safe to
- * add/remove values when holding X lock on the respective namespace.
- */
class CollectionShardingStateMap {
MONGO_DISALLOW_COPYING(CollectionShardingStateMap);
public:
- CollectionShardingStateMap() = default;
+ static const ServiceContext::Decoration<boost::optional<CollectionShardingStateMap>> get;
- static const ServiceContext::Decoration<CollectionShardingStateMap> get;
+ CollectionShardingStateMap(std::unique_ptr<CollectionShardingStateFactory> factory)
+ : _factory(std::move(factory)) {}
- CollectionShardingState& getOrCreate(const std::string& ns) {
+ CollectionShardingState& getOrCreate(const NamespaceString& nss) {
stdx::lock_guard<stdx::mutex> lg(_mutex);
- auto it = _collections.find(ns);
+ auto it = _collections.find(nss.ns());
if (it == _collections.end()) {
- auto inserted = _collections.emplace(
- ns,
- std::make_unique<CollectionShardingState>(get.owner(this), NamespaceString(ns)));
+ auto inserted = _collections.try_emplace(nss.ns(), _factory->make(nss));
invariant(inserted.second);
it = std::move(inserted.first);
}
@@ -124,16 +62,6 @@ public:
return *it->second;
}
- void resetAll() {
- stdx::lock_guard<stdx::mutex> lg(_mutex);
- for (auto it = _collections.begin(); it != _collections.end(); ++it) {
- // This is a hack to get around CollectionShardingState::refreshMetadata() requiring
- // the X lock: markNotShardedAtStepdown() doesn't have a lock check. Temporary
- // measure until SERVER-31595 removes the X lock requirement.
- it->second->markNotShardedAtStepdown();
- }
- }
-
void report(OperationContext* opCtx, BSONObjBuilder* builder) {
BSONObjBuilder versionB(builder->subobjStart("versions"));
@@ -142,10 +70,8 @@ public:
for (auto& coll : _collections) {
ScopedCollectionMetadata metadata = coll.second->getMetadata(opCtx);
- if (metadata) {
+ if (metadata->isSharded()) {
versionB.appendTimestamp(coll.first, metadata->getShardVersion().toLong());
- } else {
- versionB.appendTimestamp(coll.first, ChunkVersion::UNSHARDED().toLong());
}
}
}
@@ -154,229 +80,90 @@ public:
}
private:
- mutable stdx::mutex _mutex;
+ using CollectionsMap = StringMap<std::shared_ptr<CollectionShardingState>>;
+
+ std::unique_ptr<CollectionShardingStateFactory> _factory;
- using CollectionsMap =
- stdx::unordered_map<std::string, std::unique_ptr<CollectionShardingState>>;
+ stdx::mutex _mutex;
CollectionsMap _collections;
};
-const ServiceContext::Decoration<CollectionShardingStateMap> CollectionShardingStateMap::get =
- ServiceContext::declareDecoration<CollectionShardingStateMap>();
+const ServiceContext::Decoration<boost::optional<CollectionShardingStateMap>>
+ CollectionShardingStateMap::get =
+ ServiceContext::declareDecoration<boost::optional<CollectionShardingStateMap>>();
} // namespace
-CollectionShardingState::CollectionShardingState(ServiceContext* sc, NamespaceString nss)
- : _nss(std::move(nss)),
- _metadataManager(std::make_shared<MetadataManager>(
- sc, _nss, getRangeDeleterExecutorHolder(sc).getOrCreateExecutor())) {}
+CollectionShardingState::CollectionShardingState(NamespaceString nss) : _nss(std::move(nss)) {}
CollectionShardingState* CollectionShardingState::get(OperationContext* opCtx,
const NamespaceString& nss) {
- return CollectionShardingState::get(opCtx, nss.ns());
-}
-
-CollectionShardingState* CollectionShardingState::get(OperationContext* opCtx,
- const std::string& ns) {
// Collection lock must be held to have a reference to the collection's sharding state
- dassert(opCtx->lockState()->isCollectionLockedForMode(ns, MODE_IS));
-
- auto& collectionsMap = CollectionShardingStateMap::get(opCtx->getServiceContext());
- return &collectionsMap.getOrCreate(ns);
-}
+ dassert(opCtx->lockState()->isCollectionLockedForMode(nss.ns(), MODE_IS));
-void CollectionShardingState::resetAll(OperationContext* opCtx) {
auto& collectionsMap = CollectionShardingStateMap::get(opCtx->getServiceContext());
- collectionsMap.resetAll();
+ return &collectionsMap->getOrCreate(nss);
}
void CollectionShardingState::report(OperationContext* opCtx, BSONObjBuilder* builder) {
auto& collectionsMap = CollectionShardingStateMap::get(opCtx->getServiceContext());
- collectionsMap.report(opCtx, builder);
+ collectionsMap->report(opCtx, builder);
}
ScopedCollectionMetadata CollectionShardingState::getMetadata(OperationContext* opCtx) {
- // TODO: SERVER-34276 - find an alternative to get the atClusterTime.
- auto atClusterTime = repl::ReadConcernArgs::get(opCtx).getArgsAtClusterTime();
- return atClusterTime ? _metadataManager->createMetadataAt(opCtx, atClusterTime.get())
- : _metadataManager->getActiveMetadata(_metadataManager);
-}
-
-void CollectionShardingState::refreshMetadata(OperationContext* opCtx,
- std::unique_ptr<CollectionMetadata> newMetadata) {
- invariant(opCtx->lockState()->isCollectionLockedForMode(_nss.ns(), MODE_X));
-
- _metadataManager->refreshActiveMetadata(std::move(newMetadata));
-}
-
-void CollectionShardingState::markNotShardedAtStepdown() {
- _metadataManager->refreshActiveMetadata(nullptr);
-}
-
-auto CollectionShardingState::beginReceive(ChunkRange const& range) -> CleanupNotification {
- return _metadataManager->beginReceive(range);
-}
-
-void CollectionShardingState::forgetReceive(const ChunkRange& range) {
- _metadataManager->forgetReceive(range);
-}
-
-auto CollectionShardingState::cleanUpRange(ChunkRange const& range, CleanWhen when)
- -> CleanupNotification {
- Date_t time = (when == kNow) ? Date_t{} : Date_t::now() +
- stdx::chrono::seconds{orphanCleanupDelaySecs.load()};
- return _metadataManager->cleanUpRange(range, time);
-}
-
-std::vector<ScopedCollectionMetadata> CollectionShardingState::overlappingMetadata(
- ChunkRange const& range) const {
- return _metadataManager->overlappingMetadata(_metadataManager, range);
-}
-
-void CollectionShardingState::enterCriticalSectionCatchUpPhase(OperationContext* opCtx) {
- invariant(opCtx->lockState()->isCollectionLockedForMode(_nss.ns(), MODE_X));
- _critSec.enterCriticalSectionCatchUpPhase();
-}
-
-void CollectionShardingState::enterCriticalSectionCommitPhase(OperationContext* opCtx) {
- invariant(opCtx->lockState()->isCollectionLockedForMode(_nss.ns(), MODE_X));
- _critSec.enterCriticalSectionCommitPhase();
-}
-
-void CollectionShardingState::exitCriticalSection(OperationContext* opCtx) {
- invariant(opCtx->lockState()->isCollectionLockedForMode(_nss.ns(), MODE_X));
- _critSec.exitCriticalSection();
+ return _getMetadata(opCtx);
}
void CollectionShardingState::checkShardVersionOrThrow(OperationContext* opCtx) {
- std::string errmsg;
- ChunkVersion received;
- ChunkVersion wanted;
- if (!_checkShardVersionOk(opCtx, &errmsg, &received, &wanted)) {
- uasserted(StaleConfigInfo(_nss, received, wanted),
- str::stream() << "shard version not ok: " << errmsg);
- }
-}
-
-bool CollectionShardingState::collectionIsSharded(OperationContext* opCtx) {
- auto metadata = getMetadata(opCtx).getMetadata();
- if (metadata && (metadata->getCollVersion() == ChunkVersion::UNSHARDED())) {
- return false;
- }
-
- // If 'metadata' is null, then the shard doesn't know if this collection is sharded or not. In
- // this scenario we will assume this collection is sharded. We will know sharding state
- // definitively once SERVER-24960 has been fixed.
- return true;
-}
-
-// Call with collection unlocked. Note that the CollectionShardingState object involved might not
-// exist anymore at the time of the call, or indeed anytime outside the AutoGetCollection block, so
-// anything that might alias something in it must be copied first.
-
-Status CollectionShardingState::waitForClean(OperationContext* opCtx,
- const NamespaceString& nss,
- OID const& epoch,
- ChunkRange orphanRange) {
- while (true) {
- boost::optional<CleanupNotification> stillScheduled;
-
- {
- AutoGetCollection autoColl(opCtx, nss, MODE_IX);
- auto css = CollectionShardingState::get(opCtx, nss);
-
- {
- // First, see if collection was dropped, but do it in a separate scope in order to
- // not hold reference on it, which would make it appear in use
- auto metadata = css->_metadataManager->getActiveMetadata(css->_metadataManager);
- if (!metadata || metadata->getCollVersion().epoch() != epoch) {
- return {ErrorCodes::StaleShardVersion, "Collection being migrated was dropped"};
- }
- }
-
- stillScheduled = css->trackOrphanedDataCleanup(orphanRange);
- if (!stillScheduled) {
- log() << "Finished deleting " << nss.ns() << " range "
- << redact(orphanRange.toString());
- return Status::OK();
- }
- }
-
- log() << "Waiting for deletion of " << nss.ns() << " range " << orphanRange;
-
- Status result = stillScheduled->waitStatus(opCtx);
- if (!result.isOK()) {
- return result.withContext(str::stream() << "Failed to delete orphaned " << nss.ns()
- << " range "
- << orphanRange.toString());
- }
- }
-
- MONGO_UNREACHABLE;
-}
-
-auto CollectionShardingState::trackOrphanedDataCleanup(ChunkRange const& range)
- -> boost::optional<CleanupNotification> {
- return _metadataManager->trackOrphanedDataCleanup(range);
-}
-
-boost::optional<ChunkRange> CollectionShardingState::getNextOrphanRange(BSONObj const& from) {
- return _metadataManager->getNextOrphanRange(from);
-}
-
-bool CollectionShardingState::_checkShardVersionOk(OperationContext* opCtx,
- std::string* errmsg,
- ChunkVersion* expectedShardVersion,
- ChunkVersion* actualShardVersion) {
auto& oss = OperationShardingState::get(opCtx);
- // If there is a version attached to the OperationContext, use it as the received version.
- // Otherwise, get the received version from the ShardedConnectionInfo.
- if (oss.hasShardVersion()) {
- *expectedShardVersion = oss.getShardVersion(_nss);
- } else {
- auto const info = ShardedConnectionInfo::get(opCtx->getClient(), false);
- if (!info) {
+ const auto receivedShardVersion = [&] {
+ // If there is a version attached to the OperationContext, use it as the received version,
+ // otherwise get the received version from the ShardedConnectionInfo
+ if (oss.hasShardVersion()) {
+ return oss.getShardVersion(_nss);
+ } else if (auto const info = ShardedConnectionInfo::get(opCtx->getClient(), false)) {
+ auto connectionShardVersion = info->getVersion(_nss.ns());
+
+ // For backwards compatibility with map/reduce, which can access up to 2 sharded
+ // collections in a single call, the lack of version for a namespace on the collection
+ // must be treated as UNSHARDED
+ return connectionShardVersion.value_or(ChunkVersion::UNSHARDED());
+ } else {
// There is no shard version information on either 'opCtx' or 'client'. This means that
// the operation represented by 'opCtx' is unversioned, and the shard version is always
// OK for unversioned operations.
- return true;
+ return ChunkVersion::IGNORED();
}
+ }();
- auto connectionExpectedShardVersion = info->getVersion(_nss.ns());
- if (!connectionExpectedShardVersion) {
- *expectedShardVersion = ChunkVersion::UNSHARDED();
- } else {
- *expectedShardVersion = std::move(*connectionExpectedShardVersion);
- }
+ if (ChunkVersion::isIgnoredVersion(receivedShardVersion)) {
+ return;
}
// An operation with read concern 'available' should never have shardVersion set.
invariant(repl::ReadConcernArgs::get(opCtx).getLevel() !=
repl::ReadConcernLevel::kAvailableReadConcern);
- if (ChunkVersion::isIgnoredVersion(*expectedShardVersion)) {
- return true;
- }
-
// Set this for error messaging purposes before potentially returning false.
auto metadata = getMetadata(opCtx);
- *actualShardVersion = metadata ? metadata->getShardVersion() : ChunkVersion::UNSHARDED();
+ const auto wantedShardVersion =
+ metadata->isSharded() ? metadata->getShardVersion() : ChunkVersion::UNSHARDED();
auto criticalSectionSignal = _critSec.getSignal(opCtx->lockState()->isWriteLocked()
? ShardingMigrationCriticalSection::kWrite
: ShardingMigrationCriticalSection::kRead);
if (criticalSectionSignal) {
- *errmsg = str::stream() << "migration commit in progress for " << _nss.ns();
-
- // Set migration critical section on operation sharding state: operation will wait for
- // the migration to finish before returning failure and retrying.
+ // Set migration critical section on operation sharding state: operation will wait for the
+ // migration to finish before returning failure and retrying.
oss.setMigrationCriticalSectionSignal(criticalSectionSignal);
- return false;
+
+ uasserted(StaleConfigInfo(_nss, receivedShardVersion, wantedShardVersion),
+ str::stream() << "migration commit in progress for " << _nss.ns());
}
- if (expectedShardVersion->isWriteCompatibleWith(*actualShardVersion)) {
- return true;
+ if (receivedShardVersion.isWriteCompatibleWith(wantedShardVersion)) {
+ return;
}
//
@@ -384,52 +171,61 @@ bool CollectionShardingState::_checkShardVersionOk(OperationContext* opCtx,
// The versions themselves are returned in the error, so not needed in messages here
//
- // Check epoch first, to send more meaningful message, since other parameters probably won't
- // match either.
- if (actualShardVersion->epoch() != expectedShardVersion->epoch()) {
- *errmsg = str::stream() << "version epoch mismatch detected for " << _nss.ns() << ", "
- << "the collection may have been dropped and recreated";
- return false;
- }
+ StaleConfigInfo sci(_nss, receivedShardVersion, wantedShardVersion);
- if (!actualShardVersion->isSet() && expectedShardVersion->isSet()) {
- *errmsg = str::stream() << "this shard no longer contains chunks for " << _nss.ns() << ", "
- << "the collection may have been dropped";
- return false;
+ uassert(std::move(sci),
+ str::stream() << "epoch mismatch detected for " << _nss.ns() << ", "
+ << "the collection may have been dropped and recreated",
+ wantedShardVersion.epoch() == receivedShardVersion.epoch());
+
+ if (!wantedShardVersion.isSet() && receivedShardVersion.isSet()) {
+ uasserted(std::move(sci),
+ str::stream() << "this shard no longer contains chunks for " << _nss.ns() << ", "
+ << "the collection may have been dropped");
}
- if (actualShardVersion->isSet() && !expectedShardVersion->isSet()) {
- *errmsg = str::stream() << "this shard contains versioned chunks for " << _nss.ns() << ", "
- << "but no version set in request";
- return false;
+ if (wantedShardVersion.isSet() && !receivedShardVersion.isSet()) {
+ uasserted(std::move(sci),
+ str::stream() << "this shard contains chunks for " << _nss.ns() << ", "
+ << "but the client expects unsharded collection");
}
- if (actualShardVersion->majorVersion() != expectedShardVersion->majorVersion()) {
+ if (wantedShardVersion.majorVersion() != receivedShardVersion.majorVersion()) {
// Could be > or < - wanted is > if this is the source of a migration, wanted < if this is
// the target of a migration
- *errmsg = str::stream() << "version mismatch detected for " << _nss.ns();
- return false;
+ uasserted(std::move(sci), str::stream() << "version mismatch detected for " << _nss.ns());
}
// Those are all the reasons the versions can mismatch
MONGO_UNREACHABLE;
}
-CollectionCriticalSection::CollectionCriticalSection(OperationContext* opCtx, NamespaceString ns)
- : _nss(std::move(ns)), _opCtx(opCtx) {
- AutoGetCollection autoColl(_opCtx, _nss, MODE_IX, MODE_X);
- CollectionShardingState::get(opCtx, _nss)->enterCriticalSectionCatchUpPhase(_opCtx);
+void CollectionShardingState::enterCriticalSectionCatchUpPhase(OperationContext* opCtx) {
+ invariant(opCtx->lockState()->isCollectionLockedForMode(_nss.ns(), MODE_X));
+ _critSec.enterCriticalSectionCatchUpPhase();
+}
+
+void CollectionShardingState::enterCriticalSectionCommitPhase(OperationContext* opCtx) {
+ invariant(opCtx->lockState()->isCollectionLockedForMode(_nss.ns(), MODE_X));
+ _critSec.enterCriticalSectionCommitPhase();
+}
+
+void CollectionShardingState::exitCriticalSection(OperationContext* opCtx) {
+ invariant(opCtx->lockState()->isCollectionLockedForMode(_nss.ns(), MODE_X));
+ _critSec.exitCriticalSection();
}
-CollectionCriticalSection::~CollectionCriticalSection() {
- UninterruptibleLockGuard noInterrupt(_opCtx->lockState());
- AutoGetCollection autoColl(_opCtx, _nss, MODE_IX, MODE_X);
- CollectionShardingState::get(_opCtx, _nss)->exitCriticalSection(_opCtx);
+void CollectionShardingStateFactory::set(ServiceContext* service,
+ std::unique_ptr<CollectionShardingStateFactory> factory) {
+ auto& collectionsMap = CollectionShardingStateMap::get(service);
+ invariant(!collectionsMap);
+ invariant(factory);
+ collectionsMap.emplace(std::move(factory));
}
-void CollectionCriticalSection::enterCommitPhase() {
- AutoGetCollection autoColl(_opCtx, _nss, MODE_IX, MODE_X);
- CollectionShardingState::get(_opCtx, _nss)->enterCriticalSectionCommitPhase(_opCtx);
+void CollectionShardingStateFactory::clear(ServiceContext* service) {
+ auto& collectionsMap = CollectionShardingStateMap::get(service);
+ collectionsMap.reset();
}
} // namespace mongo
diff --git a/src/mongo/db/s/collection_sharding_state.h b/src/mongo/db/s/collection_sharding_state.h
index 5ea37a6230f..daa0c3ca5e8 100644
--- a/src/mongo/db/s/collection_sharding_state.h
+++ b/src/mongo/db/s/collection_sharding_state.h
@@ -28,117 +28,63 @@
#pragma once
-#include <memory>
-#include <string>
-
#include "mongo/base/disallow_copying.h"
-#include "mongo/bson/bsonobj.h"
#include "mongo/db/namespace_string.h"
-#include "mongo/db/s/metadata_manager.h"
+#include "mongo/db/s/scoped_collection_metadata.h"
#include "mongo/db/s/sharding_migration_critical_section.h"
-#include "mongo/util/decorable.h"
namespace mongo {
-class OperationContext;
-
/**
- * Contains all sharding-related runtime state for a given collection. One such object is assigned
- * to each sharded collection known on a mongod instance. A set of these objects is linked off the
- * instance's sharding state.
+ * Each collection on a mongod instance is dynamically assigned two pieces of information for the
+ * duration of its lifetime:
+ * CollectionShardingState - this is a passive data-only state, which represents what is the
+ * shard's knowledge of its the shard version and the set of chunks that it owns.
+ * CollectionShardingRuntime (missing from the embedded mongod) - this is the heavyweight machinery
+ * which implements the sharding protocol functions and is what controls the data-only state.
*
- * Synchronization rules: In order to look-up this object in the instance's sharding map, one must
- * have some lock on the respective collection.
+ * The CollectionShardingStateFactory class below is used in order to allow for the collection
+ * runtime to be instantiated separately from the sharding state.
+ *
+ * Synchronization rule: In order to obtain an instance of this object, the caller must have some
+ * lock on the respective collection.
*/
-class CollectionShardingState : public Decorable<CollectionShardingState> {
+class CollectionShardingState {
MONGO_DISALLOW_COPYING(CollectionShardingState);
public:
- using CleanupNotification = CollectionRangeDeleter::DeleteNotification;
-
- /**
- * Instantiates a new per-collection sharding state as unsharded.
- */
- CollectionShardingState(ServiceContext* sc, NamespaceString nss);
+ virtual ~CollectionShardingState() = default;
/**
* Obtains the sharding state for the specified collection. If it does not exist, it will be
- * created and will remain active until the collection is dropped or unsharded.
+ * created and will remain in memory until the collection is dropped.
*
* Must be called with some lock held on the specific collection being looked up and the
- * returned pointer should never be stored.
+ * returned pointer must not be stored.
*/
static CollectionShardingState* get(OperationContext* opCtx, const NamespaceString& nss);
- static CollectionShardingState* get(OperationContext* opCtx, const std::string& ns);
-
- static void resetAll(OperationContext* opCtx);
- static void report(OperationContext* opCtx, BSONObjBuilder* builder);
-
- /**
- * Returns the chunk metadata for the collection. The metadata it represents lives as long as
- * the object itself, and the collection, exist. After dropping the collection lock, the
- * collection may no longer exist, but it is still safe to destroy the object.
- * The metadata is tied to a specific point in time (atClusterTime) and the time is retrieved
- * from the operation context (opCtx).
- */
- ScopedCollectionMetadata getMetadata(OperationContext* opCtx);
-
- /**
- * BSON output of the pending metadata into a BSONArray
- */
- void toBSONPending(BSONArrayBuilder& bb) const {
- _metadataManager->toBSONPending(bb);
- }
-
- /**
- * Updates the metadata based on changes received from the config server and also resolves the
- * pending receives map in case some of these pending receives have completed or have been
- * abandoned. If newMetadata is null, unshard the collection.
- *
- * Must always be called with an exclusive collection lock.
- */
- void refreshMetadata(OperationContext* opCtx, std::unique_ptr<CollectionMetadata> newMetadata);
-
- /**
- * Marks the collection as not sharded at stepdown time so that no filtering will occur for
- * slaveOk queries.
- */
- void markNotShardedAtStepdown();
/**
- * Schedules any documents in `range` for immediate cleanup iff no running queries can depend
- * on them, and adds the range to the list of pending ranges. Otherwise, returns a notification
- * that yields bad status immediately. Does not block. Call waitStatus(opCtx) on the result
- * to wait for the deletion to complete or fail. After that, call waitForClean to ensure no
- * other deletions are pending for the range.
- */
- auto beginReceive(ChunkRange const& range) -> CleanupNotification;
-
- /*
- * Removes `range` from the list of pending ranges, and schedules any documents in the range for
- * immediate cleanup. Does not block.
+ * Reports all collections which have filtering information associated.
*/
- void forgetReceive(const ChunkRange& range);
+ static void report(OperationContext* opCtx, BSONObjBuilder* builder);
/**
- * Schedules documents in `range` for cleanup after any running queries that may depend on them
- * have terminated. Does not block. Fails if range overlaps any current local shard chunk.
- * Passed kDelayed, an additional delay (configured via server parameter orphanCleanupDelaySecs)
- * is added to permit (most) dependent queries on secondaries to complete, too.
+ * Returns the chunk filtering metadata for the collection. The returned object is safe to
+ * access outside of collection lock.
*
- * Call result.waitStatus(opCtx) to wait for the deletion to complete or fail. If that succeeds,
- * waitForClean can be called to ensure no other deletions are pending for the range. Call
- * result.abandon(), instead of waitStatus, to ignore the outcome.
+ * If the operation context contains an 'atClusterTime' property, the returned filtering
+ * metadata will be tied to a specific point in time. Otherwise it will reference the latest
+ * time available.
*/
- enum CleanWhen { kNow, kDelayed };
- auto cleanUpRange(ChunkRange const& range, CleanWhen) -> CleanupNotification;
+ ScopedCollectionMetadata getMetadata(OperationContext* opCtx);
/**
- * Returns a vector of ScopedCollectionMetadata objects representing metadata instances in use
- * by running queries that overlap the argument range, suitable for identifying and invalidating
- * those queries.
+ * Checks whether the shard version in the operation context is compatible with the shard
+ * version of the collection and if not, throws StaleConfigException populated with the received
+ * and wanted versions.
*/
- std::vector<ScopedCollectionMetadata> overlappingMetadata(ChunkRange const& range) const;
+ void checkShardVersionOrThrow(OperationContext* opCtx);
/**
* Methods to control the collection's critical section. Must be called with the collection X
@@ -148,107 +94,56 @@ public:
void enterCriticalSectionCommitPhase(OperationContext* opCtx);
void exitCriticalSection(OperationContext* opCtx);
+ /**
+ * If the collection is currently in a critical section, returns the critical section signal to
+ * be waited on. Otherwise, returns nullptr.
+ */
auto getCriticalSectionSignal(ShardingMigrationCriticalSection::Operation op) const {
return _critSec.getSignal(op);
}
- /**
- * Checks whether the shard version in the context is compatible with the shard version of the
- * collection locally and if not throws StaleConfigException populated with the expected and
- * actual versions.
- *
- * Because StaleConfigException has special semantics in terms of how a sharded command's
- * response is constructed, this function should be the only means of checking for shard version
- * match.
- */
- void checkShardVersionOrThrow(OperationContext* opCtx);
-
- /**
- * Returns whether this collection is sharded. Valid only if mongoD is primary.
- * TODO SERVER-24960: This method may return a false positive until SERVER-24960 is fixed.
- */
- bool collectionIsSharded(OperationContext* opCtx);
-
- /**
- * Tracks deletion of any documents within the range, returning when deletion is complete.
- * Throws if the collection is dropped while it sleeps.
- */
- static Status waitForClean(OperationContext* opCtx,
- const NamespaceString& nss,
- OID const& epoch,
- ChunkRange orphanRange);
-
- /**
- * Reports whether any range still scheduled for deletion overlaps the argument range. If so,
- * it returns a notification n such that n->get(opCtx) will wake when the newest overlapping
- * range's deletion (possibly the one of interest) completes or fails. This should be called
- * again after each wakeup until it returns boost::none, because there can be more than one
- * range scheduled for deletion that overlaps its argument.
- */
- auto trackOrphanedDataCleanup(ChunkRange const& range) -> boost::optional<CleanupNotification>;
-
- /**
- * Returns a range _not_ owned by this shard that starts no lower than the specified
- * startingFrom key value, if any, or boost::none if there is no such range.
- */
- boost::optional<ChunkRange> getNextOrphanRange(BSONObj const& startingFrom);
+protected:
+ CollectionShardingState(NamespaceString nss);
private:
- /**
- * Checks whether the shard version of the operation matches that of the collection.
- *
- * opCtx - Operation context from which to retrieve the operation's expected version.
- * errmsg (out) - On false return contains an explanatory error message.
- * expectedShardVersion (out) - On false return contains the expected collection version on this
- * shard. Obtained from the operation sharding state.
- * actualShardVersion (out) - On false return contains the actual collection version on this
- * shard. Obtained from the collection sharding state.
- *
- * Returns true if the expected collection version on the shard matches its actual version on
- * the shard and false otherwise. Upon false return, the output parameters will be set.
- */
- bool _checkShardVersionOk(OperationContext* opCtx,
- std::string* errmsg,
- ChunkVersion* expectedShardVersion,
- ChunkVersion* actualShardVersion);
-
// Namespace this state belongs to.
const NamespaceString _nss;
- // Contains all the metadata associated with this collection.
- std::shared_ptr<MetadataManager> _metadataManager;
-
+ // Tracks the migration critical section state for this collection.
ShardingMigrationCriticalSection _critSec;
- // for access to _metadataManager
- friend auto CollectionRangeDeleter::cleanUpNextRange(OperationContext*,
- NamespaceString const&,
- OID const& epoch,
- int maxToDelete,
- CollectionRangeDeleter*)
- -> boost::optional<Date_t>;
+ // Obtains the current metadata for the collection
+ virtual ScopedCollectionMetadata _getMetadata(OperationContext* opCtx) = 0;
};
/**
- * RAII-style class, which obtains a reference to the critical section for the
- * specified collection.
+ * Singleton factory to instantiate CollectionShardingState objects specific to the type of instance
+ * which is running.
*/
-class CollectionCriticalSection {
- MONGO_DISALLOW_COPYING(CollectionCriticalSection);
+class CollectionShardingStateFactory {
+ MONGO_DISALLOW_COPYING(CollectionShardingStateFactory);
public:
- CollectionCriticalSection(OperationContext* opCtx, NamespaceString ns);
- ~CollectionCriticalSection();
+ static void set(ServiceContext* service,
+ std::unique_ptr<CollectionShardingStateFactory> factory);
+ static void clear(ServiceContext* service);
+
+ virtual ~CollectionShardingStateFactory() = default;
/**
- * Enters the commit phase of the critical section and blocks reads.
+ * Called by the CollectionShardingState::get method once per newly cached namespace. It is
+ * invoked under a mutex and must not acquire any locks or do blocking work.
+ *
+ * Implementations must be thread-safe when called from multiple threads.
*/
- void enterCommitPhase();
+ virtual std::unique_ptr<CollectionShardingState> make(const NamespaceString& nss) = 0;
-private:
- NamespaceString _nss;
+protected:
+ CollectionShardingStateFactory(ServiceContext* serviceContext)
+ : _serviceContext(serviceContext) {}
- OperationContext* _opCtx;
+ // The service context which owns this factory
+ ServiceContext* const _serviceContext;
};
} // namespace mongo
diff --git a/src/mongo/db/s/collection_sharding_state_factory_embedded.cpp b/src/mongo/db/s/collection_sharding_state_factory_embedded.cpp
new file mode 100644
index 00000000000..afd03a16e1f
--- /dev/null
+++ b/src/mongo/db/s/collection_sharding_state_factory_embedded.cpp
@@ -0,0 +1,83 @@
+/**
+ * Copyright (C) 2018 MongoDB Inc.
+ *
+ * This program is free software: you can redistribute it and/or modify
+ * it under the terms of the GNU Affero General Public License, version 3,
+ * as published by the Free Software Foundation.
+ *
+ * This program is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ * GNU Affero General Public License for more details.
+ *
+ * You should have received a copy of the GNU Affero General Public License
+ * along with this program. If not, see <http://www.gnu.org/licenses/>.
+ *
+ * As a special exception, the copyright holders give permission to link the
+ * code of portions of this program with the OpenSSL library under certain
+ * conditions as described in each individual source file and distribute
+ * linked combinations including the program with the OpenSSL library. You
+ * must comply with the GNU Affero General Public License in all respects
+ * for all of the code used other than as permitted herein. If you modify
+ * file(s) with this exception, you may extend this exception to your
+ * version of the file(s), but you are not obligated to do so. If you do not
+ * wish to do so, delete this exception statement from your version. If you
+ * delete this exception statement from all source files in the program,
+ * then also delete it in the license file.
+ */
+
+#define MONGO_LOG_DEFAULT_COMPONENT ::mongo::logger::LogComponent::kSharding
+
+#include "mongo/platform/basic.h"
+
+#include "mongo/db/s/collection_sharding_state.h"
+#include "mongo/db/service_context.h"
+
+namespace mongo {
+namespace {
+
+class UnshardedCollection : public ScopedCollectionMetadata::Impl {
+public:
+ UnshardedCollection() = default;
+
+ const CollectionMetadata& get() override {
+ return _metadata;
+ }
+
+private:
+ CollectionMetadata _metadata;
+};
+
+const auto kUnshardedCollection = std::make_shared<UnshardedCollection>();
+
+class CollectionShardingStateFactoryEmbedded final : public CollectionShardingStateFactory {
+public:
+ CollectionShardingStateFactoryEmbedded(ServiceContext* serviceContext)
+ : CollectionShardingStateFactory(serviceContext) {}
+
+ std::unique_ptr<CollectionShardingState> make(const NamespaceString& nss) override {
+ class CollectionShardingStateStandalone final : public CollectionShardingState {
+ public:
+ CollectionShardingStateStandalone(NamespaceString nss) : CollectionShardingState(nss) {}
+
+ private:
+ ScopedCollectionMetadata _getMetadata(OperationContext* opCtx) override {
+ return {kUnshardedCollection};
+ }
+ };
+
+ return std::make_unique<CollectionShardingStateStandalone>(nss);
+ }
+};
+
+} // namespace
+
+ServiceContext::ConstructorActionRegisterer collectionShardingStateFactoryRegisterer{
+ "CollectionShardingStateFactory",
+ [](ServiceContext* service) {
+ CollectionShardingStateFactory::set(
+ service, std::make_unique<CollectionShardingStateFactoryEmbedded>(service));
+ },
+ [](ServiceContext* service) { CollectionShardingStateFactory::clear(service); }};
+
+} // namespace mongo
diff --git a/src/mongo/db/s/collection_sharding_state_factory_shard.cpp b/src/mongo/db/s/collection_sharding_state_factory_shard.cpp
new file mode 100644
index 00000000000..39f2399e9ee
--- /dev/null
+++ b/src/mongo/db/s/collection_sharding_state_factory_shard.cpp
@@ -0,0 +1,91 @@
+/**
+ * Copyright (C) 2018 MongoDB Inc.
+ *
+ * This program is free software: you can redistribute it and/or modify
+ * it under the terms of the GNU Affero General Public License, version 3,
+ * as published by the Free Software Foundation.
+ *
+ * This program is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ * GNU Affero General Public License for more details.
+ *
+ * You should have received a copy of the GNU Affero General Public License
+ * along with this program. If not, see <http://www.gnu.org/licenses/>.
+ *
+ * As a special exception, the copyright holders give permission to link the
+ * code of portions of this program with the OpenSSL library under certain
+ * conditions as described in each individual source file and distribute
+ * linked combinations including the program with the OpenSSL library. You
+ * must comply with the GNU Affero General Public License in all respects
+ * for all of the code used other than as permitted herein. If you modify
+ * file(s) with this exception, you may extend this exception to your
+ * version of the file(s), but you are not obligated to do so. If you do not
+ * wish to do so, delete this exception statement from your version. If you
+ * delete this exception statement from all source files in the program,
+ * then also delete it in the license file.
+ */
+
+#define MONGO_LOG_DEFAULT_COMPONENT ::mongo::logger::LogComponent::kSharding
+
+#include "mongo/platform/basic.h"
+
+#include "mongo/db/s/collection_sharding_runtime.h"
+#include "mongo/db/service_context.h"
+#include "mongo/executor/network_interface_factory.h"
+#include "mongo/executor/network_interface_thread_pool.h"
+#include "mongo/executor/thread_pool_task_executor.h"
+
+namespace mongo {
+namespace {
+
+class CollectionShardingStateFactoryShard final : public CollectionShardingStateFactory {
+public:
+ CollectionShardingStateFactoryShard(ServiceContext* serviceContext)
+ : CollectionShardingStateFactory(serviceContext) {}
+
+ ~CollectionShardingStateFactoryShard() {
+ if (_taskExecutor) {
+ _taskExecutor->shutdown();
+ _taskExecutor->join();
+ }
+ }
+
+ std::unique_ptr<CollectionShardingState> make(const NamespaceString& nss) override {
+ return std::make_unique<CollectionShardingRuntime>(_serviceContext, nss, _getExecutor());
+ }
+
+private:
+ executor::TaskExecutor* _getExecutor() {
+ stdx::lock_guard<stdx::mutex> lg(_mutex);
+ if (!_taskExecutor) {
+ const std::string kExecName("CollectionRangeDeleter-TaskExecutor");
+
+ auto net = executor::makeNetworkInterface(kExecName);
+ auto pool = stdx::make_unique<executor::NetworkInterfaceThreadPool>(net.get());
+ auto taskExecutor = stdx::make_unique<executor::ThreadPoolTaskExecutor>(std::move(pool),
+ std::move(net));
+ taskExecutor->startup();
+
+ _taskExecutor = std::move(taskExecutor);
+ }
+
+ return _taskExecutor.get();
+ }
+
+ // Serializes the instantiation of the task executor
+ stdx::mutex _mutex;
+ std::unique_ptr<executor::TaskExecutor> _taskExecutor{nullptr};
+};
+
+} // namespace
+
+ServiceContext::ConstructorActionRegisterer collectionShardingStateFactoryRegisterer{
+ "CollectionShardingStateFactory",
+ [](ServiceContext* service) {
+ CollectionShardingStateFactory::set(
+ service, std::make_unique<CollectionShardingStateFactoryShard>(service));
+ },
+ [](ServiceContext* service) { CollectionShardingStateFactory::clear(service); }};
+
+} // namespace mongo
diff --git a/src/mongo/db/s/collection_sharding_state_test.cpp b/src/mongo/db/s/collection_sharding_state_test.cpp
index d416840861c..10fb4937c1d 100644
--- a/src/mongo/db/s/collection_sharding_state_test.cpp
+++ b/src/mongo/db/s/collection_sharding_state_test.cpp
@@ -28,11 +28,8 @@
#include "mongo/platform/basic.h"
#include "mongo/db/catalog_raii.h"
-#include "mongo/db/dbdirectclient.h"
-#include "mongo/db/namespace_string.h"
-#include "mongo/db/s/collection_sharding_state.h"
+#include "mongo/db/s/collection_sharding_runtime.h"
#include "mongo/db/s/shard_server_op_observer.h"
-#include "mongo/db/s/sharding_state.h"
#include "mongo/db/s/type_shard_identity.h"
#include "mongo/s/shard_server_test_fixture.h"
@@ -41,88 +38,6 @@ namespace {
const NamespaceString kTestNss("TestDB", "TestColl");
-class CollectionShardingStateTest : public ShardServerTestFixture {
-public:
- void setUp() override {
- ShardServerTestFixture::setUp();
-
- // Note: this assumes that globalInit will always be called on the same thread as the main
- // test thread.
- ShardingState::get(operationContext())
- ->setGlobalInitMethodForTest(
- [this](OperationContext*, const ConnectionString&, StringData) {
- _initCallCount++;
- return Status::OK();
- });
- }
-
- int getInitCallCount() const {
- return _initCallCount;
- }
-
-private:
- int _initCallCount = 0;
-};
-
-TEST_F(CollectionShardingStateTest, GlobalInitGetsCalledAfterWriteCommits) {
- ShardIdentityType shardIdentity;
- shardIdentity.setConfigsvrConnString(
- ConnectionString(ConnectionString::SET, "a:1,b:2", "config"));
- shardIdentity.setShardName("a");
- shardIdentity.setClusterId(OID::gen());
-
- DBDirectClient client(operationContext());
- client.insert("admin.system.version", shardIdentity.toBSON());
- ASSERT_EQ(1, getInitCallCount());
-}
-
-TEST_F(CollectionShardingStateTest, GlobalInitDoesntGetCalledIfWriteAborts) {
- ShardIdentityType shardIdentity;
- shardIdentity.setConfigsvrConnString(
- ConnectionString(ConnectionString::SET, "a:1,b:2", "config"));
- shardIdentity.setShardName("a");
- shardIdentity.setClusterId(OID::gen());
-
- // This part of the test ensures that the collection exists for the AutoGetCollection below to
- // find and also validates that the initializer does not get called for non-sharding documents
- DBDirectClient client(operationContext());
- client.insert("admin.system.version", BSON("_id" << 1));
- ASSERT_EQ(0, getInitCallCount());
-
- {
- AutoGetCollection autoColl(
- operationContext(), NamespaceString("admin.system.version"), MODE_IX);
-
- WriteUnitOfWork wuow(operationContext());
- ASSERT_OK(autoColl.getCollection()->insertDocument(
- operationContext(), shardIdentity.toBSON(), {}, false));
- ASSERT_EQ(0, getInitCallCount());
- }
-
- ASSERT_EQ(0, getInitCallCount());
-}
-
-TEST_F(CollectionShardingStateTest, GlobalInitDoesntGetsCalledIfNSIsNotForShardIdentity) {
- ShardIdentityType shardIdentity;
- shardIdentity.setConfigsvrConnString(
- ConnectionString(ConnectionString::SET, "a:1,b:2", "config"));
- shardIdentity.setShardName("a");
- shardIdentity.setClusterId(OID::gen());
-
- DBDirectClient client(operationContext());
- client.insert("admin.user", shardIdentity.toBSON());
- ASSERT_EQ(0, getInitCallCount());
-}
-
-TEST_F(CollectionShardingStateTest, OnInsertOpThrowWithIncompleteShardIdentityDocument) {
- ShardIdentityType shardIdentity;
- shardIdentity.setShardName("a");
-
- DBDirectClient client(operationContext());
- client.insert("admin.system.version", shardIdentity.toBSON());
- ASSERT(!client.getLastError().empty());
-}
-
/**
* Constructs a CollectionMetadata suitable for refreshing a CollectionShardingState. The only
* salient detail is the argument `keyPattern` which, defining the shard key, selects the fields
@@ -144,7 +59,7 @@ using DeleteStateTest = ShardServerTestFixture;
TEST_F(DeleteStateTest, MakeDeleteStateUnsharded) {
AutoGetCollection autoColl(operationContext(), kTestNss, MODE_IX);
- auto* css = CollectionShardingState::get(operationContext(), kTestNss);
+ auto* const css = CollectionShardingRuntime::get(operationContext(), kTestNss);
auto doc = BSON("key3"
<< "abc"
@@ -166,7 +81,7 @@ TEST_F(DeleteStateTest, MakeDeleteStateUnsharded) {
TEST_F(DeleteStateTest, MakeDeleteStateShardedWithoutIdInShardKey) {
AutoGetCollection autoColl(operationContext(), kTestNss, MODE_IX);
- auto* css = CollectionShardingState::get(operationContext(), kTestNss);
+ auto* const css = CollectionShardingRuntime::get(operationContext(), kTestNss);
// Push a CollectionMetadata with a shard key not including "_id"...
css->refreshMetadata(operationContext(), makeAMetadata(BSON("key" << 1 << "key3" << 1)));
@@ -193,7 +108,7 @@ TEST_F(DeleteStateTest, MakeDeleteStateShardedWithoutIdInShardKey) {
TEST_F(DeleteStateTest, MakeDeleteStateShardedWithIdInShardKey) {
AutoGetCollection autoColl(operationContext(), kTestNss, MODE_IX);
- auto* css = CollectionShardingState::get(operationContext(), kTestNss);
+ auto* const css = CollectionShardingRuntime::get(operationContext(), kTestNss);
// Push a CollectionMetadata with a shard key that does have "_id" in the middle...
css->refreshMetadata(operationContext(),
@@ -219,7 +134,7 @@ TEST_F(DeleteStateTest, MakeDeleteStateShardedWithIdInShardKey) {
TEST_F(DeleteStateTest, MakeDeleteStateShardedWithIdHashInShardKey) {
AutoGetCollection autoColl(operationContext(), kTestNss, MODE_IX);
- auto* css = CollectionShardingState::get(operationContext(), kTestNss);
+ auto* const css = CollectionShardingRuntime::get(operationContext(), kTestNss);
// Push a CollectionMetadata with a shard key "_id", hashed.
auto aMetadata = makeAMetadata(BSON("_id"
diff --git a/src/mongo/db/s/config/initial_split_policy.cpp b/src/mongo/db/s/config/initial_split_policy.cpp
index f7fd7ce6bf5..126703fdd19 100644
--- a/src/mongo/db/s/config/initial_split_policy.cpp
+++ b/src/mongo/db/s/config/initial_split_policy.cpp
@@ -37,7 +37,6 @@
#include "mongo/db/logical_clock.h"
#include "mongo/db/server_options.h"
#include "mongo/s/balancer_configuration.h"
-#include "mongo/s/catalog/sharding_catalog_client_impl.h"
#include "mongo/s/catalog/type_shard.h"
#include "mongo/s/grid.h"
#include "mongo/s/shard_util.h"
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 099b6dfb3ab..8e695029b6d 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
@@ -172,7 +172,7 @@ Status ShardingCatalogManager::dropCollection(OperationContext* opCtx, const Nam
"dropCollection.start",
nss.ns(),
BSONObj(),
- ShardingCatalogClientImpl::kMajorityWriteConcern)
+ ShardingCatalogClient::kMajorityWriteConcern)
.ignore();
auto shardsStatus =
@@ -353,7 +353,7 @@ Status ShardingCatalogManager::dropCollection(OperationContext* opCtx, const Nam
"dropCollection",
nss.ns(),
BSONObj(),
- ShardingCatalogClientImpl::kMajorityWriteConcern)
+ ShardingCatalogClient::kMajorityWriteConcern)
.ignore();
return Status::OK();
diff --git a/src/mongo/db/s/config/sharding_catalog_manager_database_operations.cpp b/src/mongo/db/s/config/sharding_catalog_manager_database_operations.cpp
index a7371789025..b5f7ec90cff 100644
--- a/src/mongo/db/s/config/sharding_catalog_manager_database_operations.cpp
+++ b/src/mongo/db/s/config/sharding_catalog_manager_database_operations.cpp
@@ -38,7 +38,6 @@
#include "mongo/db/repl/repl_client_info.h"
#include "mongo/db/server_options.h"
#include "mongo/db/write_concern.h"
-#include "mongo/s/catalog/sharding_catalog_client_impl.h"
#include "mongo/s/catalog/type_database.h"
#include "mongo/s/catalog_cache.h"
#include "mongo/s/client/shard.h"
diff --git a/src/mongo/db/s/config/sharding_catalog_manager_shard_operations.cpp b/src/mongo/db/s/config/sharding_catalog_manager_shard_operations.cpp
index 153e9f78b71..48c37b655ff 100644
--- a/src/mongo/db/s/config/sharding_catalog_manager_shard_operations.cpp
+++ b/src/mongo/db/s/config/sharding_catalog_manager_shard_operations.cpp
@@ -898,14 +898,15 @@ BSONObj ShardingCatalogManager::createShardIdentityUpsertForAddShard(OperationCo
write_ops::UpdateOpEntry entry;
entry.setQ(BSON("_id"
<< "shardIdentity"
- << ShardIdentityType::shardName(shardName)
- << ShardIdentityType::clusterId(
- ClusterIdentityLoader::get(opCtx)->getClusterId())));
- entry.setU(BSON("$set" << BSON(ShardIdentityType::configsvrConnString(
- repl::ReplicationCoordinator::get(opCtx)
- ->getConfig()
- .getConnectionString()
- .toString()))));
+ << ShardIdentity::kShardNameFieldName
+ << shardName
+ << ShardIdentity::kClusterIdFieldName
+ << ClusterIdentityLoader::get(opCtx)->getClusterId()));
+ entry.setU(BSON("$set" << BSON(ShardIdentity::kConfigsvrConnectionStringFieldName
+ << repl::ReplicationCoordinator::get(opCtx)
+ ->getConfig()
+ .getConnectionString()
+ .toString())));
entry.setUpsert(true);
return entry;
}()});
diff --git a/src/mongo/db/s/get_shard_version_command.cpp b/src/mongo/db/s/get_shard_version_command.cpp
index 4fd8a6d2261..8cf767f7e3b 100644
--- a/src/mongo/db/s/get_shard_version_command.cpp
+++ b/src/mongo/db/s/get_shard_version_command.cpp
@@ -37,7 +37,7 @@
#include "mongo/db/auth/privilege.h"
#include "mongo/db/catalog_raii.h"
#include "mongo/db/commands.h"
-#include "mongo/db/s/collection_sharding_state.h"
+#include "mongo/db/s/collection_sharding_runtime.h"
#include "mongo/db/s/sharded_connection_info.h"
#include "mongo/db/s/sharding_state.h"
#include "mongo/s/grid.h"
@@ -60,7 +60,7 @@ public:
}
AllowedOnSecondary secondaryAllowed(ServiceContext*) const override {
- return AllowedOnSecondary::kNever;
+ return AllowedOnSecondary::kAlways;
}
bool adminOnly() const override {
@@ -107,18 +107,18 @@ public:
}
AutoGetCollection autoColl(opCtx, nss, MODE_IS);
- CollectionShardingState* const css = CollectionShardingState::get(opCtx, nss);
+ auto* const css = CollectionShardingRuntime::get(opCtx, nss);
const auto metadata = css->getMetadata(opCtx);
- if (metadata) {
+ if (metadata->isSharded()) {
result.appendTimestamp("global", metadata->getShardVersion().toLong());
} else {
- result.appendTimestamp("global", ChunkVersion(0, 0, OID()).toLong());
+ result.appendTimestamp("global", ChunkVersion::UNSHARDED().toLong());
}
if (cmdObj["fullMetadata"].trueValue()) {
BSONObjBuilder metadataBuilder(result.subobjStart("metadata"));
- if (metadata) {
+ if (metadata->isSharded()) {
metadata->toBSONBasic(metadataBuilder);
BSONArrayBuilder chunksArr(metadataBuilder.subarrayStart("chunks"));
diff --git a/src/mongo/db/s/merge_chunks_command.cpp b/src/mongo/db/s/merge_chunks_command.cpp
index a1fab846dc3..e145f684bc9 100644
--- a/src/mongo/db/s/merge_chunks_command.cpp
+++ b/src/mongo/db/s/merge_chunks_command.cpp
@@ -67,7 +67,7 @@ bool checkMetadataForSuccess(OperationContext* opCtx,
uassert(ErrorCodes::StaleEpoch,
str::stream() << "Collection " << nss.ns() << " became unsharded",
- metadataAfterMerge);
+ metadataAfterMerge->isSharded());
ChunkType chunk;
if (!metadataAfterMerge->getNextChunk(minKey, &chunk)) {
@@ -110,7 +110,7 @@ Status mergeChunks(OperationContext* opCtx,
return CollectionShardingState::get(opCtx, nss)->getMetadata(opCtx);
}();
- if (!metadata) {
+ if (!metadata->isSharded()) {
std::string errmsg = stream() << "could not merge chunks, collection " << nss.ns()
<< " is not sharded";
@@ -162,7 +162,7 @@ Status mergeChunks(OperationContext* opCtx,
std::string errmsg = stream()
<< "could not merge chunks, collection " << nss.ns() << " range starting at "
<< redact(minKey) << " and ending at " << redact(maxKey) << " does not belong to shard "
- << shardingState->getShardName();
+ << shardingState->shardId();
warning() << errmsg;
return Status(ErrorCodes::IllegalOperation, errmsg);
@@ -178,9 +178,9 @@ Status mergeChunks(OperationContext* opCtx,
bool minKeyInRange = rangeContains(firstDocMin, firstDocMax, minKey);
if (!minKeyInRange) {
- std::string errmsg = stream()
- << "could not merge chunks, collection " << nss.ns() << " range starting at "
- << redact(minKey) << " does not belong to shard " << shardingState->getShardName();
+ std::string errmsg = stream() << "could not merge chunks, collection " << nss.ns()
+ << " range starting at " << redact(minKey)
+ << " does not belong to shard " << shardingState->shardId();
warning() << errmsg;
return Status(ErrorCodes::IllegalOperation, errmsg);
@@ -192,9 +192,9 @@ Status mergeChunks(OperationContext* opCtx,
bool maxKeyInRange = lastDocMin.woCompare(maxKey) < 0 && lastDocMax.woCompare(maxKey) >= 0;
if (!maxKeyInRange) {
- std::string errmsg = stream()
- << "could not merge chunks, collection " << nss.ns() << " range ending at "
- << redact(maxKey) << " does not belong to shard " << shardingState->getShardName();
+ std::string errmsg = stream() << "could not merge chunks, collection " << nss.ns()
+ << " range ending at " << redact(maxKey)
+ << " does not belong to shard " << shardingState->shardId();
warning() << errmsg;
return Status(ErrorCodes::IllegalOperation, errmsg);
@@ -242,7 +242,7 @@ Status mergeChunks(OperationContext* opCtx,
// Run _configsvrCommitChunkMerge.
//
MergeChunkRequest request{nss,
- shardingState->getShardName(),
+ shardingState->shardId().toString(),
shardVersion.epoch(),
chunkBoundaries,
LogicalClock::get(opCtx)->getClusterTime().asTimestamp()};
diff --git a/src/mongo/db/s/metadata_manager.cpp b/src/mongo/db/s/metadata_manager.cpp
index 941772e0dac..eb626fcaee9 100644
--- a/src/mongo/db/s/metadata_manager.cpp
+++ b/src/mongo/db/s/metadata_manager.cpp
@@ -35,12 +35,9 @@
#include "mongo/base/string_data.h"
#include "mongo/bson/simple_bsonobj_comparator.h"
#include "mongo/bson/util/builder.h"
-#include "mongo/db/bson/dotted_path_support.h"
#include "mongo/db/query/internal_plans.h"
#include "mongo/db/range_arithmetic.h"
#include "mongo/db/s/collection_sharding_state.h"
-#include "mongo/db/s/sharding_state.h"
-#include "mongo/s/catalog_cache.h"
#include "mongo/s/grid.h"
#include "mongo/stdx/memory.h"
#include "mongo/util/assert_util.h"
@@ -118,6 +115,20 @@ using CallbackArgs = TaskExecutor::CallbackArgs;
MONGO_FAIL_POINT_DEFINE(suspendRangeDeletion);
+class UnshardedCollection : public ScopedCollectionMetadata::Impl {
+public:
+ UnshardedCollection() = default;
+
+ const CollectionMetadata& get() override {
+ return _metadata;
+ }
+
+private:
+ CollectionMetadata _metadata;
+};
+
+const auto kUnshardedCollection = std::make_shared<UnshardedCollection>();
+
/**
* Deletes ranges, in background, until done, normally using a task executor attached to the
* ShardingState.
@@ -154,6 +165,45 @@ void scheduleCleanup(executor::TaskExecutor* executor,
} // namespace
+class RangePreserver : public ScopedCollectionMetadata::Impl {
+public:
+ // Must be called locked with the MetadataManager's _managerLock
+ RangePreserver(WithLock,
+ std::shared_ptr<MetadataManager> metadataManager,
+ std::shared_ptr<MetadataManager::CollectionMetadataTracker> metadataTracker)
+ : _metadataManager(std::move(metadataManager)),
+ _metadataTracker(std::move(metadataTracker)) {
+ ++_metadataTracker->usageCounter;
+ }
+
+ ~RangePreserver() {
+ stdx::lock_guard<stdx::mutex> managerLock(_metadataManager->_managerLock);
+
+ invariant(_metadataTracker->usageCounter != 0);
+ if (--_metadataTracker->usageCounter == 0) {
+ // MetadataManager doesn't care which usageCounter went to zero. It just retires all
+ // that are older than the oldest metadata still in use by queries (some start out at
+ // zero, some go to zero but can't be expired yet).
+ //
+ // Note that new instances of ScopedCollectionMetadata may get attached to
+ // _metadata.back(), so its usage count can increase from zero, unlike other reference
+ // counts.
+ _metadataManager->_retireExpiredMetadata(managerLock);
+ }
+ }
+
+ const CollectionMetadata& get() override {
+ return _metadataTracker->metadata;
+ }
+
+private:
+ friend ScopedCollectionMetadata MetadataManager::getActiveMetadata(
+ std::shared_ptr<MetadataManager>, const boost::optional<LogicalTime>&);
+
+ std::shared_ptr<MetadataManager> _metadataManager;
+ std::shared_ptr<MetadataManager::CollectionMetadataTracker> _metadataTracker;
+};
+
MetadataManager::MetadataManager(ServiceContext* serviceContext,
NamespaceString nss,
TaskExecutor* executor)
@@ -183,34 +233,37 @@ void MetadataManager::_clearAllCleanups(WithLock, Status status) {
_rangesToClean.clear(status);
}
-ScopedCollectionMetadata MetadataManager::getActiveMetadata(std::shared_ptr<MetadataManager> self) {
+ScopedCollectionMetadata MetadataManager::getActiveMetadata(
+ std::shared_ptr<MetadataManager> self, const boost::optional<LogicalTime>& atClusterTime) {
stdx::lock_guard<stdx::mutex> lg(_managerLock);
- if (!_metadata.empty()) {
- return ScopedCollectionMetadata(lg, std::move(self), _metadata.back());
- }
-
- return ScopedCollectionMetadata();
-}
-ScopedCollectionMetadata MetadataManager::createMetadataAt(OperationContext* opCtx,
- LogicalTime atClusterTime) {
- auto cache = Grid::get(opCtx)->catalogCache();
- if (!cache) {
- return ScopedCollectionMetadata();
+ if (_metadata.empty()) {
+ return {kUnshardedCollection};
}
- auto routingTable = cache->getCollectionRoutingTableHistoryNoRefresh(_nss);
- if (!routingTable) {
- return ScopedCollectionMetadata();
+ auto metadataTracker = _metadata.back();
+ if (!atClusterTime) {
+ return {std::make_shared<RangePreserver>(lg, std::move(self), std::move(metadataTracker))};
}
- auto cm = std::make_shared<ChunkManager>(routingTable, atClusterTime.asTimestamp());
- CollectionMetadata metadata(std::move(cm), ShardingState::get(opCtx)->getShardName());
+ auto chunkManager = metadataTracker->metadata.getChunkManager();
+ auto chunkManagerAtClusterTime = std::make_shared<ChunkManager>(
+ chunkManager->getRoutingHistory(), atClusterTime->asTimestamp());
- auto metadataTracker =
- std::make_shared<MetadataManager::CollectionMetadataTracker>(std::move(metadata));
+ class MetadataAtTimestamp : public ScopedCollectionMetadata::Impl {
+ public:
+ MetadataAtTimestamp(CollectionMetadata metadata) : _metadata(std::move(metadata)) {}
- return ScopedCollectionMetadata(std::move(metadataTracker));
+ const CollectionMetadata& get() override {
+ return _metadata;
+ }
+
+ private:
+ CollectionMetadata _metadata;
+ };
+
+ return {std::make_shared<MetadataAtTimestamp>(
+ CollectionMetadata(chunkManagerAtClusterTime, metadataTracker->metadata.shardId()))};
}
size_t MetadataManager::numberOfMetadataSnapshots() const {
@@ -449,35 +502,6 @@ auto MetadataManager::cleanUpRange(ChunkRange const& range, Date_t whenToDelete)
return orphans.back().notification;
}
-std::vector<ScopedCollectionMetadata> MetadataManager::overlappingMetadata(
- std::shared_ptr<MetadataManager> const& self, ChunkRange const& range) {
- stdx::lock_guard<stdx::mutex> lg(_managerLock);
- invariant(!_metadata.empty());
-
- std::vector<ScopedCollectionMetadata> result;
- result.reserve(_metadata.size());
-
- // Start with the active metadata
- auto it = _metadata.rbegin();
- if ((*it)->metadata.rangeOverlapsChunk(range)) {
- // We ignore the refcount of the active mapping; effectively, we assume it is in use.
- result.push_back(ScopedCollectionMetadata(lg, self, (*it)));
- }
-
- // Continue to snapshots
- ++it;
- for (; it != _metadata.rend(); ++it) {
- auto& tracker = *it;
-
- // We want all the overlapping snapshot mappings still possibly in use by a query.
- if (tracker->usageCounter > 0 && tracker->metadata.rangeOverlapsChunk(range)) {
- result.push_back(ScopedCollectionMetadata(lg, self, tracker));
- }
- }
-
- return result;
-}
-
size_t MetadataManager::numberOfRangesToCleanStillInUse() const {
stdx::lock_guard<stdx::mutex> lg(_managerLock);
size_t count = 0;
@@ -551,83 +575,4 @@ boost::optional<ChunkRange> MetadataManager::getNextOrphanRange(BSONObj const& f
return _metadata.back()->metadata.getNextOrphanRange(_receivingChunks, from);
}
-ScopedCollectionMetadata::ScopedCollectionMetadata() = default;
-
-ScopedCollectionMetadata::ScopedCollectionMetadata(
- WithLock,
- std::shared_ptr<MetadataManager> metadataManager,
- std::shared_ptr<MetadataManager::CollectionMetadataTracker> metadataTracker)
- : _metadataManager(std::move(metadataManager)), _metadataTracker(std::move(metadataTracker)) {
- invariant(_metadataManager);
- invariant(_metadataTracker);
- ++_metadataTracker->usageCounter;
-}
-
-ScopedCollectionMetadata::ScopedCollectionMetadata(
- std::shared_ptr<MetadataManager::CollectionMetadataTracker> metadataTracker)
- : _metadataTracker(std::move(metadataTracker)) {
- invariant(_metadataTracker);
-}
-
-ScopedCollectionMetadata::ScopedCollectionMetadata(ScopedCollectionMetadata&& other) {
- *this = std::move(other);
-}
-
-ScopedCollectionMetadata& ScopedCollectionMetadata::operator=(ScopedCollectionMetadata&& other) {
- if (this != &other) {
- _clear();
-
- _metadataManager = std::move(other._metadataManager);
- _metadataTracker = std::move(other._metadataTracker);
-
- other._metadataManager = nullptr;
- other._metadataTracker = nullptr;
- }
- return *this;
-}
-
-CollectionMetadata* ScopedCollectionMetadata::getMetadata() const {
- return _metadataTracker ? &_metadataTracker->metadata : nullptr;
-}
-
-BSONObj ScopedCollectionMetadata::extractDocumentKey(BSONObj const& doc) const {
- BSONObj key;
- if (*this) { // is sharded
- auto const& pattern = _metadataTracker->metadata.getChunkManager()->getShardKeyPattern();
- key = dotted_path_support::extractElementsBasedOnTemplate(doc, pattern.toBSON());
- if (pattern.hasId()) {
- return key;
- }
- // else, try to append an _id field from the document.
- }
-
- if (auto id = doc["_id"]) {
- return key.isEmpty() ? id.wrap() : BSONObjBuilder(std::move(key)).append(id).obj();
- }
-
- // For legacy documents that lack an _id, use the document itself as its key.
- return doc;
-}
-
-void ScopedCollectionMetadata::_clear() {
- if (!_metadataManager) {
- return;
- }
-
- stdx::lock_guard<stdx::mutex> managerLock(_metadataManager->_managerLock);
- invariant(_metadataTracker->usageCounter != 0);
- if (--_metadataTracker->usageCounter == 0) {
- // MetadataManager doesn't care which usageCounter went to zero. It just retires all that
- // are older than the oldest metadata still in use by queries (some start out at zero, some
- // go to zero but can't be expired yet).
- //
- // Note that new instances of ScopedCollectionMetadata may get attached to _metadata.back(),
- // so its usage count can increase from zero, unlike other reference counts.
- _metadataManager->_retireExpiredMetadata(managerLock);
- }
-
- _metadataManager.reset();
- _metadataTracker.reset();
-}
-
} // namespace mongo
diff --git a/src/mongo/db/s/metadata_manager.h b/src/mongo/db/s/metadata_manager.h
index 9fe3a813dea..280f93074ce 100644
--- a/src/mongo/db/s/metadata_manager.h
+++ b/src/mongo/db/s/metadata_manager.h
@@ -34,8 +34,8 @@
#include "mongo/bson/simple_bsonobj_comparator.h"
#include "mongo/db/namespace_string.h"
#include "mongo/db/range_arithmetic.h"
-#include "mongo/db/s/collection_metadata.h"
#include "mongo/db/s/collection_range_deleter.h"
+#include "mongo/db/s/scoped_collection_metadata.h"
#include "mongo/db/service_context.h"
#include "mongo/executor/task_executor.h"
#include "mongo/s/catalog/type_chunk.h"
@@ -45,7 +45,7 @@
namespace mongo {
-class ScopedCollectionMetadata;
+class RangePreserver;
class MetadataManager {
MONGO_DISALLOW_COPYING(MetadataManager);
@@ -66,13 +66,8 @@ public:
* contains the currently active metadata. When the usageCounter goes to zero, the RAII
* object going out of scope will call _removeMetadata.
*/
- ScopedCollectionMetadata getActiveMetadata(std::shared_ptr<MetadataManager> self);
-
- /**
- * Creates the metadata on demand for a specific point in time. The object is not tracked by
- * the metadata manager.
- */
- ScopedCollectionMetadata createMetadataAt(OperationContext* opCtx, LogicalTime atCusterTime);
+ ScopedCollectionMetadata getActiveMetadata(std::shared_ptr<MetadataManager> self,
+ const boost::optional<LogicalTime>& atClusterTime);
/**
* Returns the number of CollectionMetadata objects being maintained on behalf of running
@@ -118,14 +113,6 @@ public:
CleanupNotification cleanUpRange(ChunkRange const& range, Date_t whenToDelete);
/**
- * Returns a vector of ScopedCollectionMetadata objects representing metadata instances in use
- * by running queries that overlap the argument range, suitable for identifying and invalidating
- * those queries.
- */
- std::vector<ScopedCollectionMetadata> overlappingMetadata(
- std::shared_ptr<MetadataManager> const& itself, ChunkRange const& range);
-
- /**
* Returns the number of ranges scheduled to be cleaned, exclusive of such ranges that might
* still be in use by running queries. Outside of test drivers, the actual number may vary
* after it returns, so this is really only useful for unit tests.
@@ -149,8 +136,8 @@ public:
boost::optional<ChunkRange> getNextOrphanRange(BSONObj const& from) const;
private:
- // Management of the _metadata list is implemented in ScopedCollectionMetadata
- friend class ScopedCollectionMetadata;
+ // Management of the _metadata list is implemented in RangePreserver
+ friend class RangePreserver;
// For access to _rangesToClean and _managerLock under task callback
friend boost::optional<Date_t> CollectionRangeDeleter::cleanUpNextRange(
@@ -256,83 +243,4 @@ private:
CollectionRangeDeleter _rangesToClean;
};
-class ScopedCollectionMetadata {
- MONGO_DISALLOW_COPYING(ScopedCollectionMetadata);
-
-public:
- ~ScopedCollectionMetadata() {
- _clear();
- }
-
- /**
- * Binds *this to the same CollectionMetadata as other, if any.
- */
- ScopedCollectionMetadata(ScopedCollectionMetadata&& other);
- ScopedCollectionMetadata& operator=(ScopedCollectionMetadata&& other);
-
- /**
- * Dereferencing the ScopedCollectionMetadata dereferences the private CollectionMetadata.
- */
- CollectionMetadata* getMetadata() const;
-
- CollectionMetadata* operator->() const {
- return getMetadata();
- }
-
- /**
- * True if the ScopedCollectionMetadata stores a metadata (is not empty) and the collection is
- * sharded.
- */
- operator bool() const {
- return getMetadata() != nullptr;
- }
-
- /**
- * Returns just the shard key fields, if collection is sharded, and the _id field, from `doc`.
- * Does not alter any field values (e.g. by hashing); values are copied verbatim.
- */
- BSONObj extractDocumentKey(BSONObj const& doc) const;
-
-private:
- friend ScopedCollectionMetadata MetadataManager::getActiveMetadata(
- std::shared_ptr<MetadataManager>);
-
- friend ScopedCollectionMetadata MetadataManager::createMetadataAt(OperationContext*,
- LogicalTime);
-
- friend std::vector<ScopedCollectionMetadata> MetadataManager::overlappingMetadata(
- std::shared_ptr<MetadataManager> const&, ChunkRange const&);
-
- /**
- * Creates an empty ScopedCollectionMetadata, which is interpreted as if the collection is
- * unsharded.
- */
- ScopedCollectionMetadata();
-
- /**
- * Increments the usageCounter in the specified CollectionMetadata.
- *
- * Must be called with manager->_managerLock held. Arguments must be non-null.
- */
- ScopedCollectionMetadata(
- WithLock,
- std::shared_ptr<MetadataManager> metadataManager,
- std::shared_ptr<MetadataManager::CollectionMetadataTracker> metadataTracker);
-
- /**
- * Metadata not tracked by the manager - created on demand for a specific point in time.
- */
- ScopedCollectionMetadata(
- std::shared_ptr<MetadataManager::CollectionMetadataTracker> metadataTracker);
-
- /**
- * Disconnect from the CollectionMetadata, possibly triggering GC of unused CollectionMetadata.
- */
- void _clear();
-
- std::shared_ptr<MetadataManager> _metadataManager;
-
- std::shared_ptr<MetadataManager::CollectionMetadataTracker> _metadataTracker;
-};
-
} // namespace mongo
diff --git a/src/mongo/db/s/metadata_manager_test.cpp b/src/mongo/db/s/metadata_manager_test.cpp
index ae3abb31ef7..5eb5991aca2 100644
--- a/src/mongo/db/s/metadata_manager_test.cpp
+++ b/src/mongo/db/s/metadata_manager_test.cpp
@@ -33,10 +33,8 @@
#include "mongo/bson/bsonobjbuilder.h"
#include "mongo/db/catalog_raii.h"
#include "mongo/db/client.h"
-#include "mongo/db/dbdirectclient.h"
#include "mongo/db/jsobj.h"
#include "mongo/db/namespace_string.h"
-#include "mongo/db/repl/replication_coordinator_mock.h"
#include "mongo/db/s/metadata_manager.h"
#include "mongo/db/s/sharding_state.h"
#include "mongo/db/server_options.h"
@@ -68,6 +66,9 @@ protected:
_manager = std::make_shared<MetadataManager>(getServiceContext(), kNss, executor());
}
+ /**
+ * Returns an instance of CollectionMetadata which has no chunks owned by 'thisShard'.
+ */
static std::unique_ptr<CollectionMetadata> makeEmptyMetadata() {
const OID epoch = OID::gen();
@@ -97,59 +98,58 @@ protected:
* chunk version is lower than the maximum one.
*/
static std::unique_ptr<CollectionMetadata> cloneMetadataPlusChunk(
- const CollectionMetadata& metadata, const BSONObj& minKey, const BSONObj& maxKey) {
- invariant(minKey.woCompare(maxKey) < 0);
- invariant(!rangeMapOverlaps(metadata.getChunks(), minKey, maxKey));
+ const ScopedCollectionMetadata& metadata, const ChunkRange range) {
+ const BSONObj& minKey = range.getMin();
+ const BSONObj& maxKey = range.getMax();
- auto cm = metadata.getChunkManager();
+ ASSERT(SimpleBSONObjComparator::kInstance.evaluate(minKey < maxKey))
+ << "minKey == " << minKey << "; maxKey == " << maxKey;
+ ASSERT(!rangeMapOverlaps(metadata->getChunks(), minKey, maxKey));
+
+ auto cm = metadata->getChunkManager();
const auto chunkToSplit = cm->findIntersectingChunkWithSimpleCollation(minKey);
ASSERT(SimpleBSONObjComparator::kInstance.evaluate(maxKey <= chunkToSplit.getMax()))
<< "maxKey == " << maxKey << " and chunkToSplit.getMax() == " << chunkToSplit.getMax();
+
auto v1 = cm->getVersion();
v1.incMajor();
auto v2 = v1;
v2.incMajor();
auto v3 = v2;
v3.incMajor();
- auto rt = cm->getRoutingHistory().makeUpdated(
+
+ auto rt = cm->getRoutingHistory()->makeUpdated(
{ChunkType{kNss, ChunkRange{chunkToSplit.getMin(), minKey}, v1, kOtherShard},
ChunkType{kNss, ChunkRange{minKey, maxKey}, v2, kThisShard},
ChunkType{kNss, ChunkRange{maxKey, chunkToSplit.getMax()}, v3, kOtherShard}});
- cm = std::make_shared<ChunkManager>(rt, boost::none);
- return stdx::make_unique<CollectionMetadata>(cm, kThisShard);
- }
-
- CollectionMetadata* addChunk(std::shared_ptr<MetadataManager>& manager) {
- ScopedCollectionMetadata scopedMetadata1 = manager->getActiveMetadata(manager);
-
- std::unique_ptr<CollectionMetadata> cm2 = cloneMetadataPlusChunk(
- *scopedMetadata1.getMetadata(), BSON("key" << 0), BSON("key" << 20));
- auto cm2Ptr = cm2.get();
-
- manager->refreshActiveMetadata(std::move(cm2));
- return cm2Ptr;
+ return stdx::make_unique<CollectionMetadata>(
+ std::make_shared<ChunkManager>(rt, boost::none), kThisShard);
}
std::shared_ptr<MetadataManager> _manager;
};
-// In the following tests, the ranges-to-clean is not drained by the background deleter thread
-// because the collection involved has no CollectionShardingState, so the task just returns without
-// doing anything.
-
TEST_F(MetadataManagerTest, CleanUpForMigrateIn) {
_manager->refreshActiveMetadata(makeEmptyMetadata());
+ // Sanity checks
+ ASSERT(_manager->getActiveMetadata(_manager, boost::none)->isSharded());
+ ASSERT_EQ(0UL, _manager->getActiveMetadata(_manager, boost::none)->getChunks().size());
+
ChunkRange range1(BSON("key" << 0), BSON("key" << 10));
ChunkRange range2(BSON("key" << 10), BSON("key" << 20));
+
auto notif1 = _manager->beginReceive(range1);
- ASSERT_TRUE(!notif1.ready());
+ ASSERT(!notif1.ready());
+
auto notif2 = _manager->beginReceive(range2);
- ASSERT_TRUE(!notif2.ready());
- ASSERT_EQ(_manager->numberOfRangesToClean(), 2UL);
- ASSERT_EQ(_manager->numberOfRangesToCleanStillInUse(), 0UL);
+ ASSERT(!notif2.ready());
+
+ ASSERT_EQ(2UL, _manager->numberOfRangesToClean());
+ ASSERT_EQ(0UL, _manager->numberOfRangesToCleanStillInUse());
+
notif1.abandon();
notif2.abandon();
}
@@ -170,127 +170,130 @@ TEST_F(MetadataManagerTest, AddRangeNotificationsBlockAndYield) {
}
TEST_F(MetadataManagerTest, NotificationBlocksUntilDeletion) {
- ChunkRange cr1(BSON("key" << 20), BSON("key" << 30));
_manager->refreshActiveMetadata(makeEmptyMetadata());
+
+ ChunkRange cr1(BSON("key" << 20), BSON("key" << 30));
auto optNotif = _manager->trackOrphanedDataCleanup(cr1);
- ASSERT_FALSE(optNotif); // nothing to track yet
+ ASSERT(!optNotif);
+
{
ASSERT_EQ(_manager->numberOfMetadataSnapshots(), 0UL);
ASSERT_EQ(_manager->numberOfRangesToClean(), 0UL);
- auto scm1 = _manager->getActiveMetadata(_manager); // and increment refcount
- ASSERT_TRUE(bool(scm1));
- ASSERT_EQ(0ULL, scm1->getChunks().size());
+ auto scm1 = _manager->getActiveMetadata(_manager, boost::none); // and increment refcount
+
+ const auto addChunk = [this] {
+ _manager->refreshActiveMetadata(
+ cloneMetadataPlusChunk(_manager->getActiveMetadata(_manager, boost::none),
+ {BSON("key" << 0), BSON("key" << 20)}));
+ };
- addChunk(_manager); // push new metadata
- auto scm2 = _manager->getActiveMetadata(_manager); // and increment refcount
+ addChunk(); // push new metadata
+ auto scm2 = _manager->getActiveMetadata(_manager, boost::none); // and increment refcount
ASSERT_EQ(1ULL, scm2->getChunks().size());
- // this is here solely to pacify an invariant in addChunk
+ // Simulate drop and recreate
_manager->refreshActiveMetadata(makeEmptyMetadata());
- addChunk(_manager); // push new metadata
- auto scm3 = _manager->getActiveMetadata(_manager); // and increment refcount
+ addChunk(); // push new metadata
+ auto scm3 = _manager->getActiveMetadata(_manager, boost::none); // and increment refcount
ASSERT_EQ(1ULL, scm3->getChunks().size());
- auto overlaps = _manager->overlappingMetadata(
- _manager, ChunkRange(BSON("key" << 0), BSON("key" << 10)));
- ASSERT_EQ(2ULL, overlaps.size());
- std::vector<ScopedCollectionMetadata> ref;
- ref.push_back(std::move(scm3));
- ref.push_back(std::move(scm2));
- ASSERT(ref == overlaps);
-
ASSERT_EQ(_manager->numberOfMetadataSnapshots(), 3UL);
ASSERT_EQ(_manager->numberOfRangesToClean(), 0UL); // not yet...
optNotif = _manager->cleanUpRange(cr1, Date_t{});
+ ASSERT(optNotif);
ASSERT_EQ(_manager->numberOfMetadataSnapshots(), 3UL);
ASSERT_EQ(_manager->numberOfRangesToClean(), 1UL);
- } // scm1,2,3 destroyed, refcount of each metadata goes to zero
+ }
+
+ // At this point scm1,2,3 above are destroyed and the refcount of each metadata goes to zero
+
ASSERT_EQ(_manager->numberOfMetadataSnapshots(), 0UL);
ASSERT_EQ(_manager->numberOfRangesToClean(), 1UL);
- ASSERT_FALSE(optNotif->ready());
+ ASSERT(!optNotif->ready());
+
auto optNotif2 = _manager->trackOrphanedDataCleanup(cr1); // now tracking it in _rangesToClean
- ASSERT_TRUE(optNotif && !optNotif->ready());
- ASSERT_TRUE(optNotif2 && !optNotif2->ready());
+ ASSERT(optNotif2);
+
+ ASSERT(!optNotif->ready());
+ ASSERT(!optNotif2->ready());
ASSERT(*optNotif == *optNotif2);
+
optNotif->abandon();
optNotif2->abandon();
}
TEST_F(MetadataManagerTest, RefreshAfterSuccessfulMigrationSinglePending) {
_manager->refreshActiveMetadata(makeEmptyMetadata());
- const ChunkRange cr1(BSON("key" << 0), BSON("key" << 10));
- ASSERT_EQ(_manager->getActiveMetadata(_manager)->getChunks().size(), 0UL);
- _manager->refreshActiveMetadata(cloneMetadataPlusChunk(
- *_manager->getActiveMetadata(_manager).getMetadata(), cr1.getMin(), cr1.getMax()));
- ASSERT_EQ(_manager->getActiveMetadata(_manager)->getChunks().size(), 1UL);
-}
+ ChunkRange cr1(BSON("key" << 0), BSON("key" << 10));
+ _manager->refreshActiveMetadata(
+ cloneMetadataPlusChunk(_manager->getActiveMetadata(_manager, boost::none), cr1));
+ ASSERT_EQ(_manager->getActiveMetadata(_manager, boost::none)->getChunks().size(), 1UL);
+}
TEST_F(MetadataManagerTest, RefreshAfterSuccessfulMigrationMultiplePending) {
_manager->refreshActiveMetadata(makeEmptyMetadata());
- const ChunkRange cr1(BSON("key" << 0), BSON("key" << 10));
- const ChunkRange cr2(BSON("key" << 30), BSON("key" << 40));
- ASSERT_EQ(_manager->getActiveMetadata(_manager)->getChunks().size(), 0UL);
+ ChunkRange cr1(BSON("key" << 0), BSON("key" << 10));
+ ChunkRange cr2(BSON("key" << 30), BSON("key" << 40));
{
- _manager->refreshActiveMetadata(cloneMetadataPlusChunk(
- *_manager->getActiveMetadata(_manager).getMetadata(), cr1.getMin(), cr1.getMax()));
+ _manager->refreshActiveMetadata(
+ cloneMetadataPlusChunk(_manager->getActiveMetadata(_manager, boost::none), cr1));
ASSERT_EQ(_manager->numberOfRangesToClean(), 0UL);
- ASSERT_EQ(_manager->getActiveMetadata(_manager)->getChunks().size(), 1UL);
+ ASSERT_EQ(_manager->getActiveMetadata(_manager, boost::none)->getChunks().size(), 1UL);
}
{
- _manager->refreshActiveMetadata(cloneMetadataPlusChunk(
- *_manager->getActiveMetadata(_manager).getMetadata(), cr2.getMin(), cr2.getMax()));
- ASSERT_EQ(_manager->getActiveMetadata(_manager)->getChunks().size(), 2UL);
+ _manager->refreshActiveMetadata(
+ cloneMetadataPlusChunk(_manager->getActiveMetadata(_manager, boost::none), cr2));
+ ASSERT_EQ(_manager->getActiveMetadata(_manager, boost::none)->getChunks().size(), 2UL);
}
}
TEST_F(MetadataManagerTest, RefreshAfterNotYetCompletedMigrationMultiplePending) {
_manager->refreshActiveMetadata(makeEmptyMetadata());
- const ChunkRange cr1(BSON("key" << 0), BSON("key" << 10));
- const ChunkRange cr2(BSON("key" << 30), BSON("key" << 40));
- ASSERT_EQ(_manager->getActiveMetadata(_manager)->getChunks().size(), 0UL);
+ ChunkRange cr1(BSON("key" << 0), BSON("key" << 10));
+ ChunkRange cr2(BSON("key" << 30), BSON("key" << 40));
_manager->refreshActiveMetadata(
- cloneMetadataPlusChunk(*_manager->getActiveMetadata(_manager).getMetadata(),
- BSON("key" << 50),
- BSON("key" << 60)));
- ASSERT_EQ(_manager->getActiveMetadata(_manager)->getChunks().size(), 1UL);
+ cloneMetadataPlusChunk(_manager->getActiveMetadata(_manager, boost::none),
+ {BSON("key" << 50), BSON("key" << 60)}));
+ ASSERT_EQ(_manager->getActiveMetadata(_manager, boost::none)->getChunks().size(), 1UL);
}
TEST_F(MetadataManagerTest, BeginReceiveWithOverlappingRange) {
_manager->refreshActiveMetadata(makeEmptyMetadata());
- const ChunkRange cr1(BSON("key" << 0), BSON("key" << 10));
- const ChunkRange cr2(BSON("key" << 30), BSON("key" << 40));
- const ChunkRange crOverlap(BSON("key" << 5), BSON("key" << 35));
+ ChunkRange cr1(BSON("key" << 0), BSON("key" << 10));
+ ChunkRange cr2(BSON("key" << 30), BSON("key" << 40));
+
+ _manager->refreshActiveMetadata(
+ cloneMetadataPlusChunk(_manager->getActiveMetadata(_manager, boost::none), cr1));
+ _manager->refreshActiveMetadata(
+ cloneMetadataPlusChunk(_manager->getActiveMetadata(_manager, boost::none), cr2));
- ASSERT_EQ(_manager->getActiveMetadata(_manager)->getChunks().size(), 0UL);
+ ChunkRange crOverlap(BSON("key" << 5), BSON("key" << 35));
}
TEST_F(MetadataManagerTest, RefreshMetadataAfterDropAndRecreate) {
_manager->refreshActiveMetadata(makeEmptyMetadata());
-
- {
- auto metadata = _manager->getActiveMetadata(_manager);
- _manager->refreshActiveMetadata(
- cloneMetadataPlusChunk(*metadata.getMetadata(), BSON("key" << 0), BSON("key" << 10)));
- }
+ _manager->refreshActiveMetadata(cloneMetadataPlusChunk(
+ _manager->getActiveMetadata(_manager, boost::none), {BSON("key" << 0), BSON("key" << 10)}));
// Now, pretend that the collection was dropped and recreated
- auto recreateMetadata = makeEmptyMetadata();
+ _manager->refreshActiveMetadata(makeEmptyMetadata());
_manager->refreshActiveMetadata(
- cloneMetadataPlusChunk(*recreateMetadata, BSON("key" << 20), BSON("key" << 30)));
- ASSERT_EQ(_manager->getActiveMetadata(_manager)->getChunks().size(), 1UL);
+ cloneMetadataPlusChunk(_manager->getActiveMetadata(_manager, boost::none),
+ {BSON("key" << 20), BSON("key" << 30)}));
- const auto chunks = _manager->getActiveMetadata(_manager)->getChunks();
+ const auto chunks = _manager->getActiveMetadata(_manager, boost::none)->getChunks();
+ ASSERT_EQ(1UL, chunks.size());
const auto chunkEntry = chunks.begin();
ASSERT_BSONOBJ_EQ(BSON("key" << 20), chunkEntry->first);
ASSERT_BSONOBJ_EQ(BSON("key" << 30), chunkEntry->second);
@@ -300,12 +303,14 @@ TEST_F(MetadataManagerTest, RefreshMetadataAfterDropAndRecreate) {
TEST_F(MetadataManagerTest, RangesToCleanMembership) {
_manager->refreshActiveMetadata(makeEmptyMetadata());
- ASSERT(_manager->numberOfRangesToClean() == 0UL);
+ ChunkRange cr(BSON("key" << 0), BSON("key" << 10));
+
+ ASSERT_EQ(0UL, _manager->numberOfRangesToClean());
- ChunkRange cr1 = ChunkRange(BSON("key" << 0), BSON("key" << 10));
- auto notifn = _manager->cleanUpRange(cr1, Date_t{});
+ auto notifn = _manager->cleanUpRange(cr, Date_t{});
ASSERT(!notifn.ready());
- ASSERT(_manager->numberOfRangesToClean() == 1UL);
+ ASSERT_EQ(1UL, _manager->numberOfRangesToClean());
+
notifn.abandon();
}
diff --git a/src/mongo/db/s/migration_chunk_cloner_source_legacy_commands.cpp b/src/mongo/db/s/migration_chunk_cloner_source_legacy_commands.cpp
index fdd3675dfe3..c74ae918cd1 100644
--- a/src/mongo/db/s/migration_chunk_cloner_source_legacy_commands.cpp
+++ b/src/mongo/db/s/migration_chunk_cloner_source_legacy_commands.cpp
@@ -40,7 +40,7 @@
#include "mongo/db/concurrency/write_conflict_exception.h"
#include "mongo/db/repl/replication_process.h"
#include "mongo/db/s/active_migrations_registry.h"
-#include "mongo/db/s/collection_sharding_state.h"
+#include "mongo/db/s/collection_sharding_runtime.h"
#include "mongo/db/s/migration_chunk_cloner_source_legacy.h"
#include "mongo/db/s/migration_source_manager.h"
#include "mongo/db/write_concern.h"
@@ -71,7 +71,7 @@ public:
str::stream() << "Collection " << nss->ns() << " does not exist",
_autoColl->getCollection());
- if (auto msm = MigrationSourceManager::get(CollectionShardingState::get(opCtx, *nss))) {
+ if (auto msm = MigrationSourceManager::get(CollectionShardingRuntime::get(opCtx, *nss))) {
// It is now safe to access the cloner
_chunkCloner = dynamic_cast<MigrationChunkClonerSourceLegacy*>(msm->getCloner());
invariant(_chunkCloner);
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 e92f24d0468..911792e33bc 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
@@ -117,16 +117,24 @@ protected:
}
/**
+ * Inserts the specified docs in 'kNss' and ensures the insert succeeded.
+ */
+ void insertDocsInShardedCollection(const std::vector<BSONObj>& docs) {
+ if (docs.empty())
+ return;
+
+ client()->insert(kNss.ns(), docs);
+ ASSERT_EQ("", client()->getLastError());
+ }
+
+ /**
* Creates a collection, which contains an index corresponding to kShardKeyPattern and insers
* the specified initial documents.
*/
- void createShardedCollection(std::vector<BSONObj> initialDocs) {
+ void createShardedCollection(const std::vector<BSONObj>& initialDocs) {
ASSERT(_client->createCollection(kNss.ns()));
_client->createIndex(kNss.ns(), kShardKeyPattern);
-
- if (!initialDocs.empty()) {
- _client->insert(kNss.ns(), initialDocs);
- }
+ insertDocsInShardedCollection(initialDocs);
}
/**
@@ -232,13 +240,13 @@ TEST_F(MigrationChunkClonerSourceLegacyTest, CorrectDocumentsFetched) {
}
// Insert some documents in the chunk range to be included for migration
- client()->insert(kNss.ns(), createCollectionDocument(150));
- client()->insert(kNss.ns(), createCollectionDocument(151));
+ insertDocsInShardedCollection({createCollectionDocument(150)});
+ insertDocsInShardedCollection({createCollectionDocument(151)});
// Insert some documents which are outside of the chunk range and should not be included for
// migration
- client()->insert(kNss.ns(), createCollectionDocument(90));
- client()->insert(kNss.ns(), createCollectionDocument(210));
+ insertDocsInShardedCollection({createCollectionDocument(90)});
+ insertDocsInShardedCollection({createCollectionDocument(210)});
// Normally the insert above and the onInsert/onDelete callbacks below will happen under the
// same lock and write unit of work
diff --git a/src/mongo/db/s/migration_destination_manager.cpp b/src/mongo/db/s/migration_destination_manager.cpp
index edd2ea74df9..4f378685efc 100644
--- a/src/mongo/db/s/migration_destination_manager.cpp
+++ b/src/mongo/db/s/migration_destination_manager.cpp
@@ -735,7 +735,7 @@ void MigrationDestinationManager::_migrateDriver(OperationContext* opCtx) {
}
// Wait for any other, overlapping queued deletions to drain
- auto status = CollectionShardingState::waitForClean(opCtx, _nss, _epoch, footprint);
+ auto status = CollectionShardingRuntime::waitForClean(opCtx, _nss, _epoch, footprint);
if (!status.isOK()) {
_setStateFail(redact(status.reason()));
return;
@@ -1101,16 +1101,17 @@ bool MigrationDestinationManager::_flushPendingWrites(OperationContext* opCtx,
return true;
}
-CollectionShardingState::CleanupNotification MigrationDestinationManager::_notePending(
+CollectionShardingRuntime::CleanupNotification MigrationDestinationManager::_notePending(
OperationContext* opCtx, ChunkRange const& range) {
AutoGetCollection autoColl(opCtx, _nss, MODE_IX, MODE_X);
- auto css = CollectionShardingState::get(opCtx, _nss);
+ auto* const css = CollectionShardingRuntime::get(opCtx, _nss);
+
auto metadata = css->getMetadata(opCtx);
// This can currently happen because drops aren't synchronized with in-migrations. The idea for
// checking this here is that in the future we shouldn't have this problem.
- if (!metadata || metadata->getCollVersion().epoch() != _epoch) {
+ if (!metadata->isSharded() || metadata->getCollVersion().epoch() != _epoch) {
return Status{ErrorCodes::StaleShardVersion,
str::stream() << "not noting chunk " << redact(range.toString())
<< " as pending because the epoch of "
@@ -1129,19 +1130,19 @@ CollectionShardingState::CleanupNotification MigrationDestinationManager::_noteP
}
void MigrationDestinationManager::_forgetPending(OperationContext* opCtx, ChunkRange const& range) {
-
if (!_chunkMarkedPending) { // (no lock needed, only the migrate thread looks at this.)
return; // no documents can have been moved in, so there is nothing to clean up.
}
UninterruptibleLockGuard noInterrupt(opCtx->lockState());
AutoGetCollection autoColl(opCtx, _nss, MODE_IX, MODE_X);
- auto css = CollectionShardingState::get(opCtx, _nss);
+ auto* const css = CollectionShardingRuntime::get(opCtx, _nss);
+
auto metadata = css->getMetadata(opCtx);
// This can currently happen because drops aren't synchronized with in-migrations. The idea for
// checking this here is that in the future we shouldn't have this problem.
- if (!metadata || metadata->getCollVersion().epoch() != _epoch) {
+ if (!metadata->isSharded() || metadata->getCollVersion().epoch() != _epoch) {
log() << "no need to forget pending chunk " << redact(range.toString())
<< " because the epoch for " << _nss.ns() << " changed";
return;
diff --git a/src/mongo/db/s/migration_destination_manager.h b/src/mongo/db/s/migration_destination_manager.h
index 228dc1d104e..128ffab34eb 100644
--- a/src/mongo/db/s/migration_destination_manager.h
+++ b/src/mongo/db/s/migration_destination_manager.h
@@ -38,7 +38,7 @@
#include "mongo/client/connection_string.h"
#include "mongo/db/namespace_string.h"
#include "mongo/db/s/active_migrations_registry.h"
-#include "mongo/db/s/collection_sharding_state.h"
+#include "mongo/db/s/collection_sharding_runtime.h"
#include "mongo/db/s/migration_session_id.h"
#include "mongo/db/s/session_catalog_migration_destination.h"
#include "mongo/s/shard_id.h"
@@ -161,7 +161,8 @@ private:
* it schedules deletion of any documents in the range, so that process must be seen to be
* complete before migrating any new documents in.
*/
- CollectionShardingState::CleanupNotification _notePending(OperationContext*, ChunkRange const&);
+ CollectionShardingRuntime::CleanupNotification _notePending(OperationContext*,
+ ChunkRange const&);
/**
* Stops tracking a chunk range between 'min' and 'max' that previously was having data
diff --git a/src/mongo/db/s/migration_source_manager.cpp b/src/mongo/db/s/migration_source_manager.cpp
index a0a5821a35e..a07f6f0f8e1 100644
--- a/src/mongo/db/s/migration_source_manager.cpp
+++ b/src/mongo/db/s/migration_source_manager.cpp
@@ -67,7 +67,7 @@ using namespace shardmetadatautil;
namespace {
-const auto msmForCss = CollectionShardingState::declareDecoration<MigrationSourceManager*>();
+const auto msmForCsr = CollectionShardingRuntime::declareDecoration<MigrationSourceManager*>();
// Wait at most this much time for the recipient to catch up sufficiently so critical section can be
// entered
@@ -111,7 +111,7 @@ void refreshRecipientRoutingTable(OperationContext* opCtx,
}
Status checkCollectionEpochMatches(const ScopedCollectionMetadata& metadata, OID expectedEpoch) {
- if (metadata && metadata->getCollVersion().epoch() == expectedEpoch)
+ if (metadata->isSharded() && metadata->getCollVersion().epoch() == expectedEpoch)
return Status::OK();
return {ErrorCodes::IncompatibleShardingMetadata,
@@ -119,8 +119,8 @@ Status checkCollectionEpochMatches(const ScopedCollectionMetadata& metadata, OID
<< "Expected collection epoch: "
<< expectedEpoch.toString()
<< ", but found: "
- << (metadata ? metadata->getCollVersion().epoch().toString()
- : "unsharded collection.")};
+ << (metadata->isSharded() ? metadata->getCollVersion().epoch().toString()
+ : "unsharded collection.")};
}
} // namespace
@@ -130,8 +130,8 @@ MONGO_FAIL_POINT_DEFINE(failMigrationCommit);
MONGO_FAIL_POINT_DEFINE(hangBeforeLeavingCriticalSection);
MONGO_FAIL_POINT_DEFINE(migrationCommitNetworkError);
-MigrationSourceManager* MigrationSourceManager::get(CollectionShardingState& css) {
- return msmForCss(css);
+MigrationSourceManager* MigrationSourceManager::get(CollectionShardingRuntime& csr) {
+ return msmForCsr(csr);
}
MigrationSourceManager::MigrationSourceManager(OperationContext* opCtx,
@@ -171,7 +171,7 @@ MigrationSourceManager::MigrationSourceManager(OperationContext* opCtx,
auto metadata = CollectionShardingState::get(opCtx, getNss())->getMetadata(opCtx);
uassert(ErrorCodes::IncompatibleShardingMetadata,
str::stream() << "cannot move chunks for an unsharded collection",
- metadata);
+ metadata->isSharded());
return std::make_tuple(std::move(metadata), std::move(collectionUUID));
}();
@@ -241,7 +241,7 @@ Status MigrationSourceManager::startClone(OperationContext* opCtx) {
// Register for notifications from the replication subsystem
UninterruptibleLockGuard noInterrupt(opCtx->lockState());
AutoGetCollection autoColl(opCtx, getNss(), MODE_IX, MODE_X);
- auto css = CollectionShardingState::get(opCtx, getNss());
+ auto* const css = CollectionShardingRuntime::get(opCtx, getNss());
const auto metadata = css->getMetadata(opCtx);
Status status = checkCollectionEpochMatches(metadata, _collectionEpoch);
@@ -255,7 +255,7 @@ Status MigrationSourceManager::startClone(OperationContext* opCtx) {
_cloneDriver = stdx::make_unique<MigrationChunkClonerSourceLegacy>(
_args, metadata->getKeyPattern(), _donorConnStr, _recipientHost);
- invariant(nullptr == std::exchange(msmForCss(css), this));
+ invariant(nullptr == std::exchange(msmForCsr(css), this));
}
Status startCloneStatus = _cloneDriver->startClone(opCtx);
@@ -480,7 +480,7 @@ Status MigrationSourceManager::commitChunkMetadataOnConfig(OperationContext* opC
UninterruptibleLockGuard noInterrupt(opCtx->lockState());
AutoGetCollection autoColl(opCtx, getNss(), MODE_IX, MODE_X);
if (!repl::ReplicationCoordinator::get(opCtx)->canAcceptWritesFor(opCtx, getNss())) {
- CollectionShardingState::get(opCtx, getNss())->refreshMetadata(opCtx, nullptr);
+ CollectionShardingRuntime::get(opCtx, getNss())->refreshMetadata(opCtx, nullptr);
uassertStatusOK(status.withContext(
str::stream() << "Unable to verify migration commit for chunk: "
<< redact(_args.toString())
@@ -516,7 +516,7 @@ Status MigrationSourceManager::commitChunkMetadataOnConfig(OperationContext* opC
UninterruptibleLockGuard noInterrupt(opCtx->lockState());
AutoGetCollection autoColl(opCtx, getNss(), MODE_IX, MODE_X);
- CollectionShardingState::get(opCtx, getNss())->refreshMetadata(opCtx, nullptr);
+ CollectionShardingRuntime::get(opCtx, getNss())->refreshMetadata(opCtx, nullptr);
log() << "Failed to refresh metadata after a "
<< (migrationCommitStatus.isOK() ? "failed commit attempt" : "successful commit")
@@ -539,7 +539,7 @@ Status MigrationSourceManager::commitChunkMetadataOnConfig(OperationContext* opC
return CollectionShardingState::get(opCtx, getNss())->getMetadata(opCtx);
}();
- if (!refreshedMetadata) {
+ if (!refreshedMetadata->isSharded()) {
return {ErrorCodes::NamespaceNotSharded,
str::stream() << "Chunk move failed because collection '" << getNss().ns()
<< "' is no longer sharded. The migration commit error was: "
@@ -599,11 +599,11 @@ Status MigrationSourceManager::commitChunkMetadataOnConfig(OperationContext* opC
const ChunkRange range(_args.getMinKey(), _args.getMaxKey());
auto notification = [&] {
- auto const whenToClean = _args.getWaitForDelete() ? CollectionShardingState::kNow
- : CollectionShardingState::kDelayed;
+ auto const whenToClean = _args.getWaitForDelete() ? CollectionShardingRuntime::kNow
+ : CollectionShardingRuntime::kDelayed;
UninterruptibleLockGuard noInterrupt(opCtx->lockState());
AutoGetCollection autoColl(opCtx, getNss(), MODE_IS);
- return CollectionShardingState::get(opCtx, getNss())->cleanUpRange(range, whenToClean);
+ return CollectionShardingRuntime::get(opCtx, getNss())->cleanUpRange(range, whenToClean);
}();
if (!MONGO_FAIL_POINT(doNotRefreshRecipientAfterCommit)) {
@@ -697,9 +697,9 @@ void MigrationSourceManager::_cleanup(OperationContext* opCtx) {
// Unregister from the collection's sharding state and exit the migration critical section.
UninterruptibleLockGuard noInterrupt(opCtx->lockState());
AutoGetCollection autoColl(opCtx, getNss(), MODE_IX, MODE_X);
- auto css = CollectionShardingState::get(opCtx, getNss());
+ auto* const css = CollectionShardingRuntime::get(opCtx, getNss());
- invariant(this == std::exchange(msmForCss(css), nullptr));
+ invariant(this == std::exchange(msmForCsr(css), nullptr));
_critSec.reset();
return std::move(_cloneDriver);
}();
diff --git a/src/mongo/db/s/migration_source_manager.h b/src/mongo/db/s/migration_source_manager.h
index e4e91d8b0fb..1b4d2596f30 100644
--- a/src/mongo/db/s/migration_source_manager.h
+++ b/src/mongo/db/s/migration_source_manager.h
@@ -31,7 +31,7 @@
#include <boost/optional.hpp>
#include "mongo/base/disallow_copying.h"
-#include "mongo/db/s/collection_sharding_state.h"
+#include "mongo/db/s/collection_sharding_runtime.h"
#include "mongo/db/s/migration_chunk_cloner_source.h"
#include "mongo/s/request_types/move_chunk_request.h"
#include "mongo/util/timer.h"
@@ -70,9 +70,9 @@ class MigrationSourceManager {
MONGO_DISALLOW_COPYING(MigrationSourceManager);
public:
- static MigrationSourceManager* get(CollectionShardingState& css);
- static MigrationSourceManager* get(CollectionShardingState* css) {
- return get(*css);
+ static MigrationSourceManager* get(CollectionShardingRuntime& csr);
+ static MigrationSourceManager* get(CollectionShardingRuntime* csr) {
+ return get(*csr);
}
/**
diff --git a/src/mongo/db/s/move_primary_command.cpp b/src/mongo/db/s/move_primary_command.cpp
index dc0905adc51..e4c0ff0fc34 100644
--- a/src/mongo/db/s/move_primary_command.cpp
+++ b/src/mongo/db/s/move_primary_command.cpp
@@ -151,7 +151,7 @@ private:
static void _runImpl(OperationContext* opCtx,
const ShardMovePrimary movePrimaryRequest,
const StringData dbname) {
- ShardId fromShardId = ShardingState::get(opCtx)->getShardName();
+ ShardId fromShardId = ShardingState::get(opCtx)->shardId();
ShardId toShardId = movePrimaryRequest.getTo().toString();
MovePrimarySourceManager movePrimarySourceManager(
diff --git a/src/mongo/db/views/view_sharding_check.h b/src/mongo/db/s/scoped_collection_metadata.h
index 94c26f28332..5ae13126ba8 100644
--- a/src/mongo/db/views/view_sharding_check.h
+++ b/src/mongo/db/s/scoped_collection_metadata.h
@@ -1,5 +1,5 @@
/**
- * Copyright (C) 2016 MongoDB Inc.
+ * Copyright (C) 2018 MongoDB Inc.
*
* This program is free software: you can redistribute it and/or modify
* it under the terms of the GNU Affero General Public License, version 3,
@@ -28,46 +28,43 @@
#pragma once
-#include "mongo/base/status_with.h"
-#include "mongo/bson/bsonobj.h"
-#include "mongo/bson/bsonobjbuilder.h"
-
-#include <string>
-#include <vector>
+#include "mongo/db/s/collection_metadata.h"
namespace mongo {
-class Database;
-class NamespaceString;
-class OperationContext;
-class ViewDefinition;
-
/**
- * When a read against a view is forwarded from mongoS, it is done so without any awareness as to
- * whether the underlying collection is sharded. If it is found that the underlying collection is
- * sharded(*) we return an error to mongos with the view definition requesting
- * that the resolved read operation be executed there.
- *
- * (*) We have incomplete sharding state on secondaries. If we are a secondary, then we have to
- * assume that the collection backing the view could be sharded.
+ * Acts like a shared pointer and exposes sharding filtering metadata to be used by server
+ * operations. It is allowed to be referenced outside of collection lock, but all implementations
+ * must be able to outlive the object from which they were obtained.
*/
-class ViewShardingCheck {
+class ScopedCollectionMetadata {
public:
- /**
- * If it is determined that a view's underlying collection may be sharded this method throws an
- * exception with code CommandOnShardedViewNotSupportedOnMongod with an attached ResolvedView
- * with the required information to run the view on mongos.
- */
- static void throwResolvedViewIfSharded(OperationContext* opCtx,
- Database* db,
- const ViewDefinition* view);
+ class Impl {
+ public:
+ virtual ~Impl() = default;
+
+ virtual const CollectionMetadata& get() = 0;
+
+ protected:
+ Impl() = default;
+ };
+
+ ScopedCollectionMetadata(std::shared_ptr<Impl> impl) : _impl(std::move(impl)) {}
+
+ const auto& get() const {
+ return _impl->get();
+ }
+
+ const auto* operator-> () const {
+ return &get();
+ }
+
+ const auto& operator*() const {
+ return get();
+ }
private:
- /**
- * Confirms whether 'ns' represents a sharded collection. Only valid if the calling
- * member is primary.
- */
- static bool collectionIsSharded(OperationContext* opCtx, const NamespaceString& nss);
+ std::shared_ptr<Impl> _impl;
};
} // namespace mongo
diff --git a/src/mongo/db/s/set_shard_version_command.cpp b/src/mongo/db/s/set_shard_version_command.cpp
index d7f5b27cbd1..e14feab3f7a 100644
--- a/src/mongo/db/s/set_shard_version_command.cpp
+++ b/src/mongo/db/s/set_shard_version_command.cpp
@@ -160,7 +160,7 @@ public:
// Validate shardName parameter.
const auto shardName = cmdObj["shard"].str();
- const auto storedShardName = shardingState->getShardName();
+ const auto storedShardName = shardingState->shardId().toString();
uassert(ErrorCodes::BadValue,
str::stream() << "received shardName " << shardName
<< " which differs from stored shardName "
@@ -230,10 +230,12 @@ public:
boost::optional<Lock::CollectionLock> collLock;
collLock.emplace(opCtx->lockState(), nss.ns(), MODE_IS);
- auto css = CollectionShardingState::get(opCtx, nss);
- const ChunkVersion collectionShardVersion =
- (css->getMetadata(opCtx) ? css->getMetadata(opCtx)->getShardVersion()
- : ChunkVersion::UNSHARDED());
+ auto const css = CollectionShardingState::get(opCtx, nss);
+ const ChunkVersion collectionShardVersion = [&] {
+ auto metadata = css->getMetadata(opCtx);
+ return metadata->isSharded() ? metadata->getShardVersion()
+ : ChunkVersion::UNSHARDED();
+ }();
if (requestedVersion.isWriteCompatibleWith(collectionShardVersion)) {
// MongoS and MongoD agree on what is the collection's shard version
@@ -347,9 +349,9 @@ public:
AutoGetCollection autoColl(opCtx, nss, MODE_IS);
ChunkVersion currVersion = ChunkVersion::UNSHARDED();
- auto collMetadata = CollectionShardingState::get(opCtx, nss)->getMetadata(opCtx);
- if (collMetadata) {
- currVersion = collMetadata->getShardVersion();
+ auto metadata = CollectionShardingState::get(opCtx, nss)->getMetadata(opCtx);
+ if (metadata->isSharded()) {
+ currVersion = metadata->getShardVersion();
}
if (!status.isOK()) {
diff --git a/src/mongo/db/s/shard_filtering_metadata_refresh.cpp b/src/mongo/db/s/shard_filtering_metadata_refresh.cpp
index 1463ac644f3..4f172126ed1 100644
--- a/src/mongo/db/s/shard_filtering_metadata_refresh.cpp
+++ b/src/mongo/db/s/shard_filtering_metadata_refresh.cpp
@@ -36,7 +36,7 @@
#include "mongo/db/catalog_raii.h"
#include "mongo/db/commands/feature_compatibility_version.h"
#include "mongo/db/operation_context.h"
-#include "mongo/db/s/collection_sharding_state.h"
+#include "mongo/db/s/collection_sharding_runtime.h"
#include "mongo/db/s/database_sharding_state.h"
#include "mongo/db/s/operation_sharding_state.h"
#include "mongo/db/s/sharding_state.h"
@@ -73,7 +73,7 @@ void onShardVersionMismatch(OperationContext* opCtx,
const auto currentShardVersion = [&] {
AutoGetCollection autoColl(opCtx, nss, MODE_IS);
const auto currentMetadata = CollectionShardingState::get(opCtx, nss)->getMetadata(opCtx);
- if (currentMetadata) {
+ if (currentMetadata->isSharded()) {
return currentMetadata->getShardVersion();
}
@@ -148,7 +148,7 @@ ChunkVersion forceShardFilteringMetadataRefresh(OperationContext* opCtx,
// Exclusive collection lock needed since we're now changing the metadata
AutoGetCollection autoColl(opCtx, nss, MODE_IX, MODE_X);
- auto css = CollectionShardingState::get(opCtx, nss);
+ auto* const css = CollectionShardingRuntime::get(opCtx, nss);
css->refreshMetadata(opCtx, nullptr);
return ChunkVersion::UNSHARDED();
@@ -159,7 +159,8 @@ ChunkVersion forceShardFilteringMetadataRefresh(OperationContext* opCtx,
auto metadata = CollectionShardingState::get(opCtx, nss)->getMetadata(opCtx);
// We already have newer version
- if (metadata && metadata->getCollVersion().epoch() == cm->getVersion().epoch() &&
+ if (metadata->isSharded() &&
+ metadata->getCollVersion().epoch() == cm->getVersion().epoch() &&
metadata->getCollVersion() >= cm->getVersion()) {
LOG(1) << "Skipping refresh of metadata for " << nss << " "
<< metadata->getCollVersion() << " with an older " << cm->getVersion();
@@ -170,11 +171,12 @@ ChunkVersion forceShardFilteringMetadataRefresh(OperationContext* opCtx,
// Exclusive collection lock needed since we're now changing the metadata
AutoGetCollection autoColl(opCtx, nss, MODE_IX, MODE_X);
- auto css = CollectionShardingState::get(opCtx, nss);
+ auto* const css = CollectionShardingRuntime::get(opCtx, nss);
+
auto metadata = css->getMetadata(opCtx);
// We already have newer version
- if (metadata && metadata->getCollVersion().epoch() == cm->getVersion().epoch() &&
+ if (metadata->isSharded() && metadata->getCollVersion().epoch() == cm->getVersion().epoch() &&
metadata->getCollVersion() >= cm->getVersion()) {
LOG(1) << "Skipping refresh of metadata for " << nss << " " << metadata->getCollVersion()
<< " with an older " << cm->getVersion();
@@ -182,7 +184,7 @@ ChunkVersion forceShardFilteringMetadataRefresh(OperationContext* opCtx,
}
std::unique_ptr<CollectionMetadata> newCollectionMetadata =
- stdx::make_unique<CollectionMetadata>(cm, shardingState->getShardName());
+ stdx::make_unique<CollectionMetadata>(cm, shardingState->shardId());
css->refreshMetadata(opCtx, std::move(newCollectionMetadata));
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 553380d0502..6e30264fc58 100644
--- a/src/mongo/db/s/shard_server_catalog_cache_loader.cpp
+++ b/src/mongo/db/s/shard_server_catalog_cache_loader.cpp
@@ -277,7 +277,7 @@ void forcePrimaryCollectionRefreshAndWaitForReplication(OperationContext* opCtx,
invariant(shardingState->enabled());
auto selfShard = uassertStatusOK(
- Grid::get(opCtx)->shardRegistry()->getShard(opCtx, shardingState->getShardName()));
+ Grid::get(opCtx)->shardRegistry()->getShard(opCtx, shardingState->shardId()));
auto cmdResponse = uassertStatusOK(selfShard->runCommandWithFixedRetryAttempts(
opCtx,
@@ -302,7 +302,7 @@ void forcePrimaryDatabaseRefreshAndWaitForReplication(OperationContext* opCtx, S
invariant(shardingState->enabled());
auto selfShard = uassertStatusOK(
- Grid::get(opCtx)->shardRegistry()->getShard(opCtx, shardingState->getShardName()));
+ Grid::get(opCtx)->shardRegistry()->getShard(opCtx, shardingState->shardId()));
auto cmdResponse = uassertStatusOK(selfShard->runCommandWithFixedRetryAttempts(
opCtx,
diff --git a/src/mongo/db/s/shard_server_op_observer.cpp b/src/mongo/db/s/shard_server_op_observer.cpp
index 51c9fe954c7..1b08061827f 100644
--- a/src/mongo/db/s/shard_server_op_observer.cpp
+++ b/src/mongo/db/s/shard_server_op_observer.cpp
@@ -34,11 +34,10 @@
#include "mongo/bson/util/bson_extract.h"
#include "mongo/db/catalog_raii.h"
-#include "mongo/db/s/collection_sharding_state.h"
#include "mongo/db/s/database_sharding_state.h"
#include "mongo/db/s/migration_source_manager.h"
#include "mongo/db/s/shard_identity_rollback_notifier.h"
-#include "mongo/db/s/sharding_state.h"
+#include "mongo/db/s/sharding_initialization_mongod.h"
#include "mongo/db/s/type_shard_identity.h"
#include "mongo/s/balancer_configuration.h"
#include "mongo/s/catalog/type_shard_collection.h"
@@ -77,7 +76,7 @@ public:
// This is a hack to get around CollectionShardingState::refreshMetadata() requiring the X
// lock: markNotShardedAtStepdown() doesn't have a lock check. Temporary measure until
// SERVER-31595 removes the X lock requirement.
- CollectionShardingState::get(_opCtx, _nss)->markNotShardedAtStepdown();
+ CollectionShardingRuntime::get(_opCtx, _nss)->markNotShardedAtStepdown();
}
void rollback() override {}
@@ -96,8 +95,12 @@ public:
: _opCtx(opCtx), _shardIdentity(std::move(shardIdentity)) {}
void commit(boost::optional<Timestamp>) override {
- fassertNoTrace(
- 40071, ShardingState::get(_opCtx)->initializeFromShardIdentity(_opCtx, _shardIdentity));
+ try {
+ ShardingInitializationMongoD::get(_opCtx)->initializeFromShardIdentity(_opCtx,
+ _shardIdentity);
+ } catch (const AssertionException& ex) {
+ fassertFailedWithStatus(40071, ex.toStatus());
+ }
}
void rollback() override {}
@@ -211,7 +214,7 @@ void ShardServerOpObserver::onInserts(OperationContext* opCtx,
if (auto idElem = insertedDoc["_id"]) {
if (idElem.str() == ShardIdentityType::IdName) {
auto shardIdentityDoc =
- uassertStatusOK(ShardIdentityType::fromBSON(insertedDoc));
+ uassertStatusOK(ShardIdentityType::fromShardIdentityDocument(insertedDoc));
uassertStatusOK(shardIdentityDoc.validate());
opCtx->recoveryUnit()->registerChange(
new ShardIdentityLogOpHandler(opCtx, std::move(shardIdentityDoc)));
@@ -219,7 +222,7 @@ void ShardServerOpObserver::onInserts(OperationContext* opCtx,
}
}
- if (metadata) {
+ if (metadata->isSharded()) {
incrementChunkOnInsertOrUpdate(
opCtx, *metadata->getChunkManager(), insertedDoc, insertedDoc.objsize());
}
@@ -278,7 +281,7 @@ void ShardServerOpObserver::onUpdate(OperationContext* opCtx, const OplogUpdateE
// This is a hack to get around CollectionShardingState::refreshMetadata() requiring
// the X lock: markNotShardedAtStepdown() doesn't have a lock check. Temporary
// measure until SERVER-31595 removes the X lock requirement.
- CollectionShardingState::get(opCtx, updatedNss)->markNotShardedAtStepdown();
+ CollectionShardingRuntime::get(opCtx, updatedNss)->markNotShardedAtStepdown();
}
}
}
@@ -316,7 +319,7 @@ void ShardServerOpObserver::onUpdate(OperationContext* opCtx, const OplogUpdateE
}
}
- if (metadata) {
+ if (metadata->isSharded()) {
incrementChunkOnInsertOrUpdate(
opCtx, *metadata->getChunkManager(), args.updatedDoc, args.updatedDoc.objsize());
}
@@ -325,7 +328,7 @@ void ShardServerOpObserver::onUpdate(OperationContext* opCtx, const OplogUpdateE
void ShardServerOpObserver::aboutToDelete(OperationContext* opCtx,
NamespaceString const& nss,
BSONObj const& doc) {
- auto css = CollectionShardingState::get(opCtx, nss.ns());
+ auto* const css = CollectionShardingRuntime::get(opCtx, nss);
getDeleteState(opCtx) = ShardObserverDeleteState::make(opCtx, css, doc);
}
@@ -394,7 +397,7 @@ repl::OpTime ShardServerOpObserver::onDropCollection(OperationContext* opCtx,
}
void shardObserveInsertOp(OperationContext* opCtx,
- CollectionShardingState* css,
+ CollectionShardingRuntime* css,
const BSONObj& insertedDoc,
const repl::OpTime& opTime) {
css->checkShardVersionOrThrow(opCtx);
@@ -405,7 +408,7 @@ void shardObserveInsertOp(OperationContext* opCtx,
}
void shardObserveUpdateOp(OperationContext* opCtx,
- CollectionShardingState* css,
+ CollectionShardingRuntime* css,
const BSONObj& updatedDoc,
const repl::OpTime& opTime,
const repl::OpTime& prePostImageOpTime) {
@@ -417,7 +420,7 @@ void shardObserveUpdateOp(OperationContext* opCtx,
}
void shardObserveDeleteOp(OperationContext* opCtx,
- CollectionShardingState* css,
+ CollectionShardingRuntime* css,
const ShardObserverDeleteState& deleteState,
const repl::OpTime& opTime,
const repl::OpTime& preImageOpTime) {
@@ -429,10 +432,11 @@ void shardObserveDeleteOp(OperationContext* opCtx,
}
ShardObserverDeleteState ShardObserverDeleteState::make(OperationContext* opCtx,
- CollectionShardingState* css,
+ CollectionShardingRuntime* css,
const BSONObj& docToDelete) {
auto msm = MigrationSourceManager::get(css);
- return {css->getMetadata(opCtx).extractDocumentKey(docToDelete).getOwned(),
+ auto metadata = css->getMetadata(opCtx);
+ return {metadata->extractDocumentKey(docToDelete).getOwned(),
msm && msm->getCloner()->isDocumentInMigratingChunk(docToDelete)};
}
diff --git a/src/mongo/db/s/shard_server_op_observer.h b/src/mongo/db/s/shard_server_op_observer.h
index 05b87bdb238..ea6ffc30f80 100644
--- a/src/mongo/db/s/shard_server_op_observer.h
+++ b/src/mongo/db/s/shard_server_op_observer.h
@@ -30,7 +30,7 @@
#include "mongo/base/disallow_copying.h"
#include "mongo/db/op_observer.h"
-#include "mongo/db/s/collection_sharding_state.h"
+#include "mongo/db/s/collection_sharding_runtime.h"
namespace mongo {
@@ -152,7 +152,7 @@ public:
*/
struct ShardObserverDeleteState {
static ShardObserverDeleteState make(OperationContext* opCtx,
- CollectionShardingState* css,
+ CollectionShardingRuntime* css,
const BSONObj& docToDelete);
// Contains the fields of the document that are in the collection's shard key, and "_id".
BSONObj documentKey;
@@ -164,16 +164,16 @@ struct ShardObserverDeleteState {
};
void shardObserveInsertOp(OperationContext* opCtx,
- CollectionShardingState* css,
+ CollectionShardingRuntime* css,
const BSONObj& insertedDoc,
const repl::OpTime& opTime);
void shardObserveUpdateOp(OperationContext* opCtx,
- CollectionShardingState* css,
+ CollectionShardingRuntime* css,
const BSONObj& updatedDoc,
const repl::OpTime& opTime,
const repl::OpTime& prePostImageOpTime);
void shardObserveDeleteOp(OperationContext* opCtx,
- CollectionShardingState* css,
+ CollectionShardingRuntime* css,
const ShardObserverDeleteState& deleteState,
const repl::OpTime& opTime,
const repl::OpTime& preImageOpTime);
diff --git a/src/mongo/db/s/sharding_egress_metadata_hook_for_mongod.cpp b/src/mongo/db/s/sharding_config_optime_gossip.cpp
index 660be635020..5b2eab91b51 100644
--- a/src/mongo/db/s/sharding_egress_metadata_hook_for_mongod.cpp
+++ b/src/mongo/db/s/sharding_config_optime_gossip.cpp
@@ -30,11 +30,14 @@
#include "mongo/platform/basic.h"
-#include "mongo/db/s/sharding_egress_metadata_hook_for_mongod.h"
+#include "mongo/db/s/sharding_config_optime_gossip.h"
#include "mongo/base/status.h"
+#include "mongo/db/auth/authorization_session.h"
#include "mongo/db/repl/replication_coordinator.h"
+#include "mongo/db/s/sharding_state.h"
#include "mongo/db/server_options.h"
+#include "mongo/rpc/metadata/config_server_metadata.h"
#include "mongo/s/grid.h"
namespace mongo {
@@ -47,11 +50,10 @@ repl::OpTime ShardingEgressMetadataHookForMongod::_getConfigServerOpTime() {
if (serverGlobalParams.clusterRole == ClusterRole::ConfigServer) {
return repl::ReplicationCoordinator::get(_serviceContext)
->getCurrentCommittedSnapshotOpTime();
- } else {
- // TODO uncomment as part of SERVER-22663
- // invariant(serverGlobalParams.clusterRole == ClusterRole::ShardServer);
- return Grid::get(_serviceContext)->configOpTime();
}
+
+ invariant(serverGlobalParams.clusterRole == ClusterRole::ShardServer);
+ return Grid::get(_serviceContext)->configOpTime();
}
Status ShardingEgressMetadataHookForMongod::_advanceConfigOptimeFromShard(
@@ -59,8 +61,30 @@ Status ShardingEgressMetadataHookForMongod::_advanceConfigOptimeFromShard(
if (serverGlobalParams.clusterRole == ClusterRole::ConfigServer) {
return Status::OK();
}
+
return ShardingEgressMetadataHook::_advanceConfigOptimeFromShard(shardId, metadataObj);
}
+void advanceConfigOptimeFromRequestMetadata(OperationContext* opCtx) {
+ auto const shardingState = ShardingState::get(opCtx);
+
+ if (!shardingState->enabled()) {
+ // Nothing to do if sharding state has not been initialized
+ return;
+ }
+
+ boost::optional<repl::OpTime> opTime = rpc::ConfigServerMetadata::get(opCtx).getOpTime();
+ if (!opTime)
+ return;
+
+ uassert(ErrorCodes::Unauthorized,
+ "Unauthorized to update config opTime",
+ AuthorizationSession::get(opCtx->getClient())
+ ->isAuthorizedForActionsOnResource(ResourcePattern::forClusterResource(),
+ ActionType::internal));
+
+ Grid::get(opCtx)->advanceConfigOpTime(*opTime);
+}
+
} // namespace rpc
} // namespace mongo
diff --git a/src/mongo/db/s/sharding_egress_metadata_hook_for_mongod.h b/src/mongo/db/s/sharding_config_optime_gossip.h
index 58633d1dc17..48a1195038a 100644
--- a/src/mongo/db/s/sharding_egress_metadata_hook_for_mongod.h
+++ b/src/mongo/db/s/sharding_config_optime_gossip.h
@@ -45,5 +45,11 @@ private:
Status _advanceConfigOptimeFromShard(ShardId shardId, const BSONObj& metadataObj) override;
};
+/**
+ * Updates the ShardRegistry's stored notion of the config server optime based on the
+ * ConfigServerMetadata decoration attached to the OperationContext.
+ */
+void advanceConfigOptimeFromRequestMetadata(OperationContext* opCtx);
+
} // namespace rpc
} // namespace mongo
diff --git a/src/mongo/db/s/sharding_initialization_mongod.cpp b/src/mongo/db/s/sharding_initialization_mongod.cpp
index a445626f3f1..7ecf692e761 100644
--- a/src/mongo/db/s/sharding_initialization_mongod.cpp
+++ b/src/mongo/db/s/sharding_initialization_mongod.cpp
@@ -28,26 +28,33 @@
#define MONGO_LOG_DEFAULT_COMPONENT ::mongo::logger::LogComponent::kSharding
-#include "mongo/db/s/sharding_initialization_mongod.h"
-
#include "mongo/platform/basic.h"
+#include "mongo/db/s/sharding_initialization_mongod.h"
+
#include "mongo/base/status.h"
#include "mongo/client/connection_string.h"
#include "mongo/client/global_conn_pool.h"
#include "mongo/client/remote_command_targeter.h"
#include "mongo/client/remote_command_targeter_factory_impl.h"
+#include "mongo/client/replica_set_monitor.h"
+#include "mongo/db/catalog_raii.h"
+#include "mongo/db/concurrency/d_concurrency.h"
+#include "mongo/db/dbhelpers.h"
#include "mongo/db/logical_time_metadata_hook.h"
#include "mongo/db/logical_time_validator.h"
#include "mongo/db/operation_context.h"
+#include "mongo/db/ops/update.h"
+#include "mongo/db/ops/update_lifecycle_impl.h"
+#include "mongo/db/repl/replication_coordinator.h"
+#include "mongo/db/s/chunk_splitter.h"
#include "mongo/db/s/read_only_catalog_cache_loader.h"
#include "mongo/db/s/shard_server_catalog_cache_loader.h"
-#include "mongo/db/s/sharding_egress_metadata_hook_for_mongod.h"
+#include "mongo/db/s/sharding_config_optime_gossip.h"
#include "mongo/db/server_options.h"
-#include "mongo/executor/task_executor.h"
+#include "mongo/executor/task_executor_pool.h"
#include "mongo/rpc/metadata/egress_metadata_hook_list.h"
#include "mongo/s/catalog_cache.h"
-#include "mongo/s/catalog_cache_loader.h"
#include "mongo/s/client/shard_connection.h"
#include "mongo/s/client/shard_factory.h"
#include "mongo/s/client/shard_local.h"
@@ -56,12 +63,13 @@
#include "mongo/s/config_server_catalog_cache_loader.h"
#include "mongo/s/grid.h"
#include "mongo/s/sharding_initialization.h"
-#include "mongo/stdx/memory.h"
#include "mongo/util/log.h"
namespace mongo {
namespace {
+const auto getInstance = ServiceContext::declareDecoration<ShardingInitializationMongoD>();
+
auto makeEgressHooksList(ServiceContext* service) {
auto unshardedHookList = stdx::make_unique<rpc::EgressMetadataHookList>();
unshardedHookList->addHook(stdx::make_unique<rpc::LogicalTimeMetadataHook>(service));
@@ -69,13 +77,259 @@ auto makeEgressHooksList(ServiceContext* service) {
stdx::make_unique<rpc::ShardingEgressMetadataHookForMongod>(service));
return unshardedHookList;
-};
+}
+
+/**
+ * Updates the config server field of the shardIdentity document with the given connection string if
+ * setName is equal to the config server replica set name.
+ *
+ * NOTE: This is intended to be used on a new thread that hasn't called Client::initThread.
+ *
+ * One example use case is for the ReplicaSetMonitor asynchronous callback when it detects changes
+ * to replica set membership.
+ */
+void updateShardIdentityConfigStringCB(const std::string& setName,
+ const std::string& newConnectionString) {
+ auto configsvrConnStr =
+ Grid::get(getGlobalServiceContext())->shardRegistry()->getConfigServerConnectionString();
+ if (configsvrConnStr.getSetName() != setName) {
+ // Ignore all change notification for other sets that are not the config server.
+ return;
+ }
+
+ Client::initThread("updateShardIdentityConfigConnString");
+ auto uniqOpCtx = Client::getCurrent()->makeOperationContext();
+
+ auto status = ShardingInitializationMongoD::updateShardIdentityConfigString(
+ uniqOpCtx.get(), uassertStatusOK(ConnectionString::parse(newConnectionString)));
+ if (!status.isOK() && !ErrorCodes::isNotMasterError(status.code())) {
+ warning() << "Error encountered while trying to update config connection string to "
+ << newConnectionString << causedBy(redact(status));
+ }
+}
+
+void initializeShardingEnvironmentOnShardServer(OperationContext* opCtx,
+ const ShardIdentity& shardIdentity,
+ StringData distLockProcessId) {
+ initializeGlobalShardingStateForMongoD(
+ opCtx, shardIdentity.getConfigsvrConnectionString(), distLockProcessId);
+
+ ReplicaSetMonitor::setSynchronousConfigChangeHook(
+ &ShardRegistry::replicaSetChangeShardRegistryUpdateHook);
+ ReplicaSetMonitor::setAsynchronousConfigChangeHook(&updateShardIdentityConfigStringCB);
+
+ // Determine primary/secondary/standalone state in order to properly initialize sharding
+ // components.
+ auto replCoord = repl::ReplicationCoordinator::get(opCtx);
+ bool isReplSet = replCoord->getReplicationMode() == repl::ReplicationCoordinator::modeReplSet;
+ bool isStandaloneOrPrimary =
+ !isReplSet || (repl::ReplicationCoordinator::get(opCtx)->getMemberState() ==
+ repl::MemberState::RS_PRIMARY);
+
+ CatalogCacheLoader::get(opCtx).initializeReplicaSetRole(isStandaloneOrPrimary);
+ ChunkSplitter::get(opCtx).setReplicaSetMode(isStandaloneOrPrimary);
+
+ LOG(0) << "Finished initializing sharding components for "
+ << (isStandaloneOrPrimary ? "primary" : "secondary") << " node.";
+}
} // namespace
-Status initializeGlobalShardingStateForMongod(OperationContext* opCtx,
- const ConnectionString& configCS,
- StringData distLockProcessId) {
+ShardingInitializationMongoD::ShardingInitializationMongoD()
+ : _initFunc(initializeShardingEnvironmentOnShardServer) {}
+
+ShardingInitializationMongoD::~ShardingInitializationMongoD() = default;
+
+ShardingInitializationMongoD* ShardingInitializationMongoD::get(OperationContext* opCtx) {
+ return get(opCtx->getServiceContext());
+}
+
+ShardingInitializationMongoD* ShardingInitializationMongoD::get(ServiceContext* service) {
+ return &getInstance(service);
+}
+
+void ShardingInitializationMongoD::shutDown(OperationContext* opCtx) {
+ auto const shardingState = ShardingState::get(opCtx);
+ auto const grid = Grid::get(opCtx);
+
+ if (!shardingState->enabled())
+ return;
+
+ grid->getExecutorPool()->shutdownAndJoin();
+ grid->catalogClient()->shutDown(opCtx);
+}
+
+bool ShardingInitializationMongoD::initializeShardingAwarenessIfNeeded(OperationContext* opCtx) {
+ invariant(!opCtx->lockState()->isLocked());
+
+ // In sharded readOnly mode, we ignore the shardIdentity document on disk and instead *require*
+ // a shardIdentity document to be passed through --overrideShardIdentity
+ if (storageGlobalParams.readOnly) {
+ if (serverGlobalParams.clusterRole == ClusterRole::ShardServer) {
+ uassert(ErrorCodes::InvalidOptions,
+ "If started with --shardsvr in queryableBackupMode, a shardIdentity document "
+ "must be provided through --overrideShardIdentity",
+ !serverGlobalParams.overrideShardIdentity.isEmpty());
+
+ auto overrideShardIdentity =
+ uassertStatusOK(ShardIdentityType::fromShardIdentityDocument(
+ serverGlobalParams.overrideShardIdentity));
+
+ {
+ // Global lock is required to call initializeFromShardIdentity
+ Lock::GlobalWrite lk(opCtx);
+ initializeFromShardIdentity(opCtx, overrideShardIdentity);
+ }
+
+ return true;
+ } else {
+ // Error if --overrideShardIdentity is used but *not* started with --shardsvr
+ uassert(ErrorCodes::InvalidOptions,
+ str::stream()
+ << "Not started with --shardsvr, but a shardIdentity document was provided "
+ "through --overrideShardIdentity: "
+ << serverGlobalParams.overrideShardIdentity,
+ serverGlobalParams.overrideShardIdentity.isEmpty());
+ return false;
+ }
+
+ MONGO_UNREACHABLE;
+ }
+
+ // In sharded *non*-readOnly mode, error if --overrideShardIdentity is provided
+ uassert(ErrorCodes::InvalidOptions,
+ str::stream() << "--overrideShardIdentity is only allowed in sharded "
+ "queryableBackupMode. If not in queryableBackupMode, you can edit "
+ "the shardIdentity document by starting the server *without* "
+ "--shardsvr, manually updating the shardIdentity document in the "
+ << NamespaceString::kServerConfigurationNamespace.toString()
+ << " collection, and restarting the server with --shardsvr.",
+ serverGlobalParams.overrideShardIdentity.isEmpty());
+
+ // Use the shardIdentity document on disk if one exists, but it is okay if no shardIdentity
+ // document is available at all (sharding awareness will be initialized when a shardIdentity
+ // document is inserted)
+ BSONObj shardIdentityBSON;
+ const bool foundShardIdentity = [&] {
+ AutoGetCollection autoColl(opCtx, NamespaceString::kServerConfigurationNamespace, MODE_IS);
+ return Helpers::findOne(opCtx,
+ autoColl.getCollection(),
+ BSON("_id" << ShardIdentityType::IdName),
+ shardIdentityBSON);
+ }();
+
+ if (serverGlobalParams.clusterRole == ClusterRole::ShardServer) {
+ if (!foundShardIdentity) {
+ warning() << "Started with --shardsvr, but no shardIdentity document was found on "
+ "disk in "
+ << NamespaceString::kServerConfigurationNamespace
+ << ". This most likely means this server has not yet been added to a "
+ "sharded cluster.";
+ return false;
+ }
+
+ invariant(!shardIdentityBSON.isEmpty());
+
+ auto shardIdentity =
+ uassertStatusOK(ShardIdentityType::fromShardIdentityDocument(shardIdentityBSON));
+
+ {
+ // Global lock is required to call initializeFromShardIdentity
+ Lock::GlobalWrite lk(opCtx);
+ initializeFromShardIdentity(opCtx, shardIdentity);
+ }
+
+ return true;
+ } else {
+ // Warn if a shardIdentity document is found on disk but *not* started with --shardsvr.
+ if (!shardIdentityBSON.isEmpty()) {
+ warning() << "Not started with --shardsvr, but a shardIdentity document was found "
+ "on disk in "
+ << NamespaceString::kServerConfigurationNamespace << ": "
+ << shardIdentityBSON;
+ }
+ return false;
+ }
+}
+
+void ShardingInitializationMongoD::initializeFromShardIdentity(
+ OperationContext* opCtx, const ShardIdentityType& shardIdentity) {
+ invariant(serverGlobalParams.clusterRole == ClusterRole::ShardServer);
+ invariant(opCtx->lockState()->isLocked());
+
+ uassertStatusOKWithContext(
+ shardIdentity.validate(),
+ "Invalid shard identity document found when initializing sharding state");
+
+ log() << "initializing sharding state with: " << shardIdentity;
+
+ const auto& configSvrConnStr = shardIdentity.getConfigsvrConnectionString();
+
+ auto const shardingState = ShardingState::get(opCtx);
+ auto const shardRegistry = Grid::get(opCtx)->shardRegistry();
+
+ stdx::unique_lock<stdx::mutex> ul(_initSynchronizationMutex);
+
+ if (shardingState->enabled()) {
+ uassert(40371, "", shardingState->shardId() == shardIdentity.getShardName());
+ uassert(40372, "", shardingState->clusterId() == shardIdentity.getClusterId());
+
+ auto prevConfigsvrConnStr = shardRegistry->getConfigServerConnectionString();
+ uassert(40373, "", prevConfigsvrConnStr.type() == ConnectionString::SET);
+ uassert(40374, "", prevConfigsvrConnStr.getSetName() == configSvrConnStr.getSetName());
+
+ return;
+ }
+
+ auto initializationStatus = shardingState->initializationStatus();
+ uassert(ErrorCodes::ManualInterventionRequired,
+ str::stream() << "Server's sharding metadata manager failed to initialize and will "
+ "remain in this state until the instance is manually reset"
+ << causedBy(*initializationStatus),
+ !initializationStatus);
+
+ try {
+ _initFunc(opCtx, shardIdentity, generateDistLockProcessId(opCtx));
+ shardingState->setInitialized(shardIdentity.getShardName().toString(),
+ shardIdentity.getClusterId());
+ } catch (const DBException& ex) {
+ shardingState->setInitialized(ex.toStatus());
+ }
+}
+
+Status ShardingInitializationMongoD::updateShardIdentityConfigString(
+ OperationContext* opCtx, const ConnectionString& newConnectionString) {
+ BSONObj updateObj(
+ ShardIdentityType::createConfigServerUpdateObject(newConnectionString.toString()));
+
+ UpdateRequest updateReq(NamespaceString::kServerConfigurationNamespace);
+ updateReq.setQuery(BSON("_id" << ShardIdentityType::IdName));
+ updateReq.setUpdates(updateObj);
+ UpdateLifecycleImpl updateLifecycle(NamespaceString::kServerConfigurationNamespace);
+ updateReq.setLifecycle(&updateLifecycle);
+
+ try {
+ AutoGetOrCreateDb autoDb(
+ opCtx, NamespaceString::kServerConfigurationNamespace.db(), MODE_X);
+
+ auto result = update(opCtx, autoDb.getDb(), updateReq);
+ if (result.numMatched == 0) {
+ warning() << "failed to update config string of shard identity document because "
+ << "it does not exist. This shard could have been removed from the cluster";
+ } else {
+ LOG(2) << "Updated config server connection string in shardIdentity document to"
+ << newConnectionString;
+ }
+ } catch (const DBException& exception) {
+ return exception.toStatus();
+ }
+
+ return Status::OK();
+}
+
+void initializeGlobalShardingStateForMongoD(OperationContext* opCtx,
+ const ConnectionString& configCS,
+ StringData distLockProcessId) {
auto targeterFactory = stdx::make_unique<RemoteCommandTargeterFactoryImpl>();
auto targeterFactoryPtr = targeterFactory.get();
@@ -105,7 +359,8 @@ Status initializeGlobalShardingStateForMongod(OperationContext* opCtx,
auto shardFactory =
stdx::make_unique<ShardFactory>(std::move(buildersMap), std::move(targeterFactory));
- auto service = opCtx->getServiceContext();
+ auto const service = opCtx->getServiceContext();
+
if (serverGlobalParams.clusterRole == ClusterRole::ShardServer) {
if (storageGlobalParams.readOnly) {
CatalogCacheLoader::set(service, stdx::make_unique<ReadOnlyCatalogCacheLoader>());
@@ -126,7 +381,7 @@ Status initializeGlobalShardingStateForMongod(OperationContext* opCtx,
globalConnPool.addHook(new ShardingConnectionHook(false, makeEgressHooksList(service)));
shardConnectionPool.addHook(new ShardingConnectionHook(true, makeEgressHooksList(service)));
- Status initStatus = initializeGlobalShardingState(
+ uassertStatusOK(initializeGlobalShardingState(
opCtx,
configCS,
distLockProcessId,
@@ -135,13 +390,15 @@ Status initializeGlobalShardingStateForMongod(OperationContext* opCtx,
[service] { return makeEgressHooksList(service); },
// We only need one task executor here because sharding task executors aren't used for user
// queries in mongod.
- 1);
+ 1));
- if (initStatus.isOK()) {
- Grid::get(opCtx)->setShardingInitialized();
+ auto const replCoord = repl::ReplicationCoordinator::get(service);
+ if (serverGlobalParams.clusterRole == ClusterRole::ConfigServer &&
+ replCoord->getMemberState().primary()) {
+ LogicalTimeValidator::get(opCtx)->enableKeyGenerator(opCtx, true);
}
- return initStatus;
+ Grid::get(opCtx)->setShardingInitialized();
}
} // namespace mongo
diff --git a/src/mongo/db/s/sharding_initialization_mongod.h b/src/mongo/db/s/sharding_initialization_mongod.h
index 9db0bf4357d..0cb0dbc6307 100644
--- a/src/mongo/db/s/sharding_initialization_mongod.h
+++ b/src/mongo/db/s/sharding_initialization_mongod.h
@@ -28,14 +28,90 @@
#pragma once
+#include "mongo/base/disallow_copying.h"
#include "mongo/base/string_data.h"
+#include "mongo/db/s/sharding_state.h"
+#include "mongo/db/s/type_shard_identity.h"
+#include "mongo/stdx/functional.h"
namespace mongo {
class ConnectionString;
class OperationContext;
class ServiceContext;
-class Status;
+
+/**
+ * This class serves as a bootstrap and shutdown for the sharding subsystem and also controls the
+ * persisted cluster identity. The default ShardingEnvironmentInitFunc instantiates all the sharding
+ * services, attaches them to the same service context to which it itself is attached and puts the
+ * ShardingState in the initialized state.
+ */
+class ShardingInitializationMongoD {
+ MONGO_DISALLOW_COPYING(ShardingInitializationMongoD);
+
+public:
+ using ShardingEnvironmentInitFunc = stdx::function<void(
+ OperationContext* opCtx, const ShardIdentity& shardIdentity, StringData distLockProcessId)>;
+
+ ShardingInitializationMongoD();
+ ~ShardingInitializationMongoD();
+
+ static ShardingInitializationMongoD* get(OperationContext* opCtx);
+ static ShardingInitializationMongoD* get(ServiceContext* service);
+
+ /**
+ * If started with --shardsvr, initializes sharding awareness from the shardIdentity document on
+ * disk, if there is one.
+ *
+ * If started with --shardsvr in queryableBackupMode, initializes sharding awareness from the
+ * shardIdentity document passed through the --overrideShardIdentity startup parameter.
+ *
+ * If it returns true, the '_initFunc' was called, meaning all the core classes for sharding
+ * were initialized, but no networking calls were made yet (with the exception of the duplicate
+ * ShardRegistry reload in ShardRegistry::startup() (see SERVER-26123). Outgoing networking
+ * calls to cluster members can now be made.
+ *
+ * If it returns false, this means the node is not yet sharding aware.
+ *
+ * NOTE: this function briefly takes the global lock to determine primary/secondary state.
+ */
+ bool initializeShardingAwarenessIfNeeded(OperationContext* opCtx);
+
+ /**
+ * Initializes the sharding state of this server from the shard identity document argument and
+ * sets secondary or primary state information on the catalog cache loader.
+ *
+ * NOTE: This must be called under at least Global IX lock in order for the replica set member
+ * state to be stable (primary/secondary).
+ */
+ void initializeFromShardIdentity(OperationContext* opCtx,
+ const ShardIdentityType& shardIdentity);
+
+ void shutDown(OperationContext* service);
+
+ /**
+ * Updates the config server field of the shardIdentity document with the given connection
+ * string.
+ */
+ static Status updateShardIdentityConfigString(OperationContext* opCtx,
+ const ConnectionString& newConnectionString);
+
+ /**
+ * For testing only. Mock the initialization method used by initializeFromConfigConnString and
+ * initializeFromShardIdentity after all checks are performed.
+ */
+ void setGlobalInitMethodForTest(ShardingEnvironmentInitFunc func) {
+ _initFunc = std::move(func);
+ }
+
+private:
+ // This mutex ensures that only one thread at a time executes the sharding
+ // initialization/teardown sequence
+ stdx::mutex _initSynchronizationMutex;
+
+ // Function for initializing the sharding environment components (i.e. everything on the Grid)
+ ShardingEnvironmentInitFunc _initFunc;
+};
/**
* Initialize the sharding components of this server. This can be used on both shard and config
@@ -43,8 +119,8 @@ class Status;
*
* NOTE: This does not initialize ShardingState, which should only be done for shard servers.
*/
-Status initializeGlobalShardingStateForMongod(OperationContext* opCtx,
- const ConnectionString& configCS,
- StringData distLockProcessId);
+void initializeGlobalShardingStateForMongoD(OperationContext* opCtx,
+ const ConnectionString& configCS,
+ StringData distLockProcessId);
} // namespace mongo
diff --git a/src/mongo/db/s/sharding_state_test.cpp b/src/mongo/db/s/sharding_initialization_mongod_test.cpp
index 1b29fe5ceae..7748e7492ac 100644
--- a/src/mongo/db/s/sharding_state_test.cpp
+++ b/src/mongo/db/s/sharding_initialization_mongod_test.cpp
@@ -1,5 +1,5 @@
/**
- * Copyright (C) 2016 MongoDB, Inc.
+ * Copyright (C) 2018 MongoDB, Inc.
*
* This program is free software: you can redistribute it and/or modify
* it under the terms of the GNU Affero General Public License, version 3,
@@ -29,6 +29,7 @@
#include "mongo/platform/basic.h"
#include "mongo/client/remote_command_targeter_mock.h"
+#include "mongo/db/catalog_raii.h"
#include "mongo/db/concurrency/d_concurrency.h"
#include "mongo/db/dbdirectclient.h"
#include "mongo/db/op_observer_impl.h"
@@ -37,24 +38,25 @@
#include "mongo/db/s/config_server_op_observer.h"
#include "mongo/db/s/shard_server_catalog_cache_loader.h"
#include "mongo/db/s/shard_server_op_observer.h"
-#include "mongo/db/s/sharding_state.h"
+#include "mongo/db/s/sharding_initialization_mongod.h"
#include "mongo/db/s/type_shard_identity.h"
#include "mongo/db/server_options.h"
-#include "mongo/db/storage/storage_options.h"
#include "mongo/s/catalog/dist_lock_manager_mock.h"
#include "mongo/s/catalog/sharding_catalog_client_impl.h"
#include "mongo/s/client/shard_registry.h"
#include "mongo/s/config_server_catalog_cache_loader.h"
-#include "mongo/s/sharding_mongod_test_fixture.h"
+#include "mongo/s/shard_server_test_fixture.h"
namespace mongo {
namespace {
-using executor::RemoteCommandRequest;
+const std::string kShardName("TestShard");
-const std::string kShardName("a");
-
-class ShardingStateTest : public ShardingMongodTestFixture {
+/**
+ * This test suite directly invokes the sharding initialization code and validates its behaviour and
+ * proper state transitions.
+ */
+class ShardingInitializationMongoDTest : public ShardingMongodTestFixture {
protected:
// Used to write to set up local collections before exercising server logic.
std::unique_ptr<DBDirectClient> _dbDirectClient;
@@ -70,23 +72,24 @@ protected:
stdx::make_unique<ShardServerCatalogCacheLoader>(
stdx::make_unique<ConfigServerCatalogCacheLoader>()));
- _shardingState.setGlobalInitMethodForTest([&](OperationContext* opCtx,
- const ConnectionString& configConnStr,
- StringData distLockProcessId) {
- auto status = initializeGlobalShardingStateForMongodForTest(configConnStr);
- if (!status.isOK()) {
- return status;
- }
+ ShardingInitializationMongoD::get(getServiceContext())
+ ->setGlobalInitMethodForTest([&](OperationContext* opCtx,
+ const ShardIdentity& shardIdentity,
+ StringData distLockProcessId) {
+ const auto& configConnStr = shardIdentity.getConfigsvrConnectionString();
- // Set the ConnectionString return value on the mock targeter so that later calls to the
- // targeter's getConnString() return the appropriate value
- auto configTargeter =
- RemoteCommandTargeterMock::get(shardRegistry()->getConfigShard()->getTargeter());
- configTargeter->setConnectionStringReturnValue(configConnStr);
- configTargeter->setFindHostReturnValue(configConnStr.getServers()[0]);
+ uassertStatusOK(initializeGlobalShardingStateForMongodForTest(configConnStr));
- return Status::OK();
- });
+ // Set the ConnectionString return value on the mock targeter so that later calls to
+ // the
+ // targeter's getConnString() return the appropriate value
+ auto configTargeter = RemoteCommandTargeterMock::get(
+ shardRegistry()->getConfigShard()->getTargeter());
+ configTargeter->setConnectionStringReturnValue(configConnStr);
+ configTargeter->setFindHostReturnValue(configConnStr.getServers()[0]);
+
+ return Status::OK();
+ });
_dbDirectClient = stdx::make_unique<DBDirectClient>(operationContext());
}
@@ -99,6 +102,7 @@ protected:
serverGlobalParams.overrideShardIdentity = BSONObj();
CatalogCacheLoader::clearForTests(getServiceContext());
+ ShardingState::get(getServiceContext())->clearForTests();
ShardingMongodTestFixture::tearDown();
}
@@ -114,12 +118,13 @@ protected:
return stdx::make_unique<ShardingCatalogClientImpl>(std::move(distLockManager));
}
- ShardingState* shardingState() {
- return &_shardingState;
+ auto* shardingInitialization() {
+ return ShardingInitializationMongoD::get(getServiceContext());
}
-private:
- ShardingState _shardingState;
+ auto* shardingState() {
+ return ShardingState::get(getServiceContext());
+ }
};
/**
@@ -150,267 +155,278 @@ private:
ServiceContext* const _serviceContext;
};
-TEST_F(ShardingStateTest, ValidShardIdentitySucceeds) {
+TEST_F(ShardingInitializationMongoDTest, ValidShardIdentitySucceeds) {
// Must hold a lock to call initializeFromShardIdentity.
Lock::GlobalWrite lk(operationContext());
ShardIdentityType shardIdentity;
- shardIdentity.setConfigsvrConnString(
+ shardIdentity.setConfigsvrConnectionString(
ConnectionString(ConnectionString::SET, "a:1,b:2", "config"));
shardIdentity.setShardName(kShardName);
shardIdentity.setClusterId(OID::gen());
- ASSERT_OK(shardingState()->initializeFromShardIdentity(operationContext(), shardIdentity));
- ASSERT_TRUE(shardingState()->enabled());
- ASSERT_EQ(kShardName, shardingState()->getShardName());
+ shardingInitialization()->initializeFromShardIdentity(operationContext(), shardIdentity);
+ ASSERT_OK(shardingState()->canAcceptShardedCommands());
+ ASSERT(shardingState()->enabled());
+ ASSERT_EQ(kShardName, shardingState()->shardId());
ASSERT_EQ("config/a:1,b:2", shardRegistry()->getConfigServerConnectionString().toString());
}
-TEST_F(ShardingStateTest, InitWhilePreviouslyInErrorStateWillStayInErrorState) {
+TEST_F(ShardingInitializationMongoDTest, InitWhilePreviouslyInErrorStateWillStayInErrorState) {
// Must hold a lock to call initializeFromShardIdentity.
Lock::GlobalWrite lk(operationContext());
ShardIdentityType shardIdentity;
- shardIdentity.setConfigsvrConnString(
+ shardIdentity.setConfigsvrConnectionString(
ConnectionString(ConnectionString::SET, "a:1,b:2", "config"));
shardIdentity.setShardName(kShardName);
shardIdentity.setClusterId(OID::gen());
- shardingState()->setGlobalInitMethodForTest(
- [](OperationContext* opCtx, const ConnectionString& connStr, StringData distLockProcessId) {
- return Status{ErrorCodes::ShutdownInProgress, "shutting down"};
- });
+ shardingInitialization()->setGlobalInitMethodForTest([](
+ OperationContext* opCtx, const ShardIdentity& shardIdentity, StringData distLockProcessId) {
+ uasserted(ErrorCodes::ShutdownInProgress, "Not an actual shutdown");
+ });
- {
- auto status =
- shardingState()->initializeFromShardIdentity(operationContext(), shardIdentity);
- ASSERT_EQ(ErrorCodes::ShutdownInProgress, status);
- }
+ shardingInitialization()->initializeFromShardIdentity(operationContext(), shardIdentity);
// ShardingState is now in error state, attempting to call it again will still result in error.
-
- shardingState()->setGlobalInitMethodForTest(
- [](OperationContext* opCtx, const ConnectionString& connStr, StringData distLockProcessId) {
- return Status::OK();
- });
-
- {
- auto status =
- shardingState()->initializeFromShardIdentity(operationContext(), shardIdentity);
- ASSERT_EQ(ErrorCodes::ManualInterventionRequired, status);
- }
-
- ASSERT_FALSE(shardingState()->enabled());
+ shardingInitialization()->setGlobalInitMethodForTest([](
+ OperationContext* opCtx, const ShardIdentity& shardIdentity, StringData distLockProcessId) {
+ FAIL("Should not be invoked!");
+ });
+
+ ASSERT_THROWS_CODE(
+ shardingInitialization()->initializeFromShardIdentity(operationContext(), shardIdentity),
+ AssertionException,
+ ErrorCodes::ManualInterventionRequired);
+ ASSERT_NOT_OK(shardingState()->canAcceptShardedCommands());
+ ASSERT(!shardingState()->enabled());
}
-TEST_F(ShardingStateTest, InitializeAgainWithMatchingShardIdentitySucceeds) {
+TEST_F(ShardingInitializationMongoDTest, InitializeAgainWithMatchingShardIdentitySucceeds) {
// Must hold a lock to call initializeFromShardIdentity.
Lock::GlobalWrite lk(operationContext());
auto clusterID = OID::gen();
ShardIdentityType shardIdentity;
- shardIdentity.setConfigsvrConnString(
+ shardIdentity.setConfigsvrConnectionString(
ConnectionString(ConnectionString::SET, "a:1,b:2", "config"));
shardIdentity.setShardName(kShardName);
shardIdentity.setClusterId(clusterID);
- ASSERT_OK(shardingState()->initializeFromShardIdentity(operationContext(), shardIdentity));
+ shardingInitialization()->initializeFromShardIdentity(operationContext(), shardIdentity);
ShardIdentityType shardIdentity2;
- shardIdentity2.setConfigsvrConnString(
+ shardIdentity2.setConfigsvrConnectionString(
ConnectionString(ConnectionString::SET, "a:1,b:2", "config"));
shardIdentity2.setShardName(kShardName);
shardIdentity2.setClusterId(clusterID);
- shardingState()->setGlobalInitMethodForTest(
- [](OperationContext* opCtx, const ConnectionString& connStr, StringData distLockProcessId) {
- return Status{ErrorCodes::InternalError, "should not reach here"};
- });
+ shardingInitialization()->setGlobalInitMethodForTest([](
+ OperationContext* opCtx, const ShardIdentity& shardIdentity, StringData distLockProcessId) {
+ FAIL("Should not be invoked!");
+ });
- ASSERT_OK(shardingState()->initializeFromShardIdentity(operationContext(), shardIdentity2));
+ shardingInitialization()->initializeFromShardIdentity(operationContext(), shardIdentity2);
+ ASSERT_OK(shardingState()->canAcceptShardedCommands());
ASSERT_TRUE(shardingState()->enabled());
- ASSERT_EQ(kShardName, shardingState()->getShardName());
+
+ ASSERT_EQ(kShardName, shardingState()->shardId());
ASSERT_EQ("config/a:1,b:2", shardRegistry()->getConfigServerConnectionString().toString());
}
-TEST_F(ShardingStateTest, InitializeAgainWithSameReplSetNameSucceeds) {
+TEST_F(ShardingInitializationMongoDTest, InitializeAgainWithMatchingReplSetNameSucceeds) {
// Must hold a lock to call initializeFromShardIdentity.
Lock::GlobalWrite lk(operationContext());
auto clusterID = OID::gen();
ShardIdentityType shardIdentity;
- shardIdentity.setConfigsvrConnString(
+ shardIdentity.setConfigsvrConnectionString(
ConnectionString(ConnectionString::SET, "a:1,b:2", "config"));
shardIdentity.setShardName(kShardName);
shardIdentity.setClusterId(clusterID);
- ASSERT_OK(shardingState()->initializeFromShardIdentity(operationContext(), shardIdentity));
+ shardingInitialization()->initializeFromShardIdentity(operationContext(), shardIdentity);
ShardIdentityType shardIdentity2;
- shardIdentity2.setConfigsvrConnString(
+ shardIdentity2.setConfigsvrConnectionString(
ConnectionString(ConnectionString::SET, "b:2,c:3", "config"));
shardIdentity2.setShardName(kShardName);
shardIdentity2.setClusterId(clusterID);
- shardingState()->setGlobalInitMethodForTest(
- [](OperationContext* opCtx, const ConnectionString& connStr, StringData distLockProcessId) {
- return Status{ErrorCodes::InternalError, "should not reach here"};
- });
+ shardingInitialization()->setGlobalInitMethodForTest([](
+ OperationContext* opCtx, const ShardIdentity& shardIdentity, StringData distLockProcessId) {
+ FAIL("Should not be invoked!");
+ });
- ASSERT_OK(shardingState()->initializeFromShardIdentity(operationContext(), shardIdentity2));
+ shardingInitialization()->initializeFromShardIdentity(operationContext(), shardIdentity2);
+ ASSERT_OK(shardingState()->canAcceptShardedCommands());
ASSERT_TRUE(shardingState()->enabled());
- ASSERT_EQ(kShardName, shardingState()->getShardName());
+
+ ASSERT_EQ(kShardName, shardingState()->shardId());
ASSERT_EQ("config/a:1,b:2", shardRegistry()->getConfigServerConnectionString().toString());
}
// The tests below check for different combinations of the compatible startup parameters for
// --shardsvr, --overrideShardIdentity, and queryableBackup (readOnly) mode
-// readOnly and --shardsvr
-
-TEST_F(ShardingStateTest,
+/**
+ * readOnly and --shardsvr
+ */
+TEST_F(ShardingInitializationMongoDTest,
InitializeShardingAwarenessIfNeededReadOnlyAndShardServerAndNoOverrideShardIdentity) {
storageGlobalParams.readOnly = true;
- auto swShardingInitialized =
- shardingState()->initializeShardingAwarenessIfNeeded(operationContext());
- ASSERT_EQUALS(ErrorCodes::InvalidOptions, swShardingInitialized.getStatus().code());
+
+ ASSERT_THROWS_CODE(
+ shardingInitialization()->initializeShardingAwarenessIfNeeded(operationContext()),
+ AssertionException,
+ ErrorCodes::InvalidOptions);
}
-TEST_F(ShardingStateTest,
+TEST_F(ShardingInitializationMongoDTest,
InitializeShardingAwarenessIfNeededReadOnlyAndShardServerAndInvalidOverrideShardIdentity) {
storageGlobalParams.readOnly = true;
- serverGlobalParams.overrideShardIdentity = BSON("_id"
- << "shardIdentity"
- << "configsvrConnectionString"
- << "invalid");
- auto swShardingInitialized =
- shardingState()->initializeShardingAwarenessIfNeeded(operationContext());
- ASSERT_EQUALS(ErrorCodes::UnsupportedFormat, swShardingInitialized.getStatus().code());
+ serverGlobalParams.overrideShardIdentity =
+ BSON("_id"
+ << "shardIdentity"
+ << ShardIdentity::kShardNameFieldName
+ << kShardName
+ << ShardIdentity::kClusterIdFieldName
+ << OID::gen()
+ << ShardIdentity::kConfigsvrConnectionStringFieldName
+ << "invalid");
+
+ ASSERT_THROWS_CODE(
+ shardingInitialization()->initializeShardingAwarenessIfNeeded(operationContext()),
+ AssertionException,
+ ErrorCodes::UnsupportedFormat);
}
-TEST_F(ShardingStateTest,
+TEST_F(ShardingInitializationMongoDTest,
InitializeShardingAwarenessIfNeededReadOnlyAndShardServerAndValidOverrideShardIdentity) {
storageGlobalParams.readOnly = true;
serverGlobalParams.clusterRole = ClusterRole::ShardServer;
+ serverGlobalParams.overrideShardIdentity = [] {
+ ShardIdentityType shardIdentity;
+ shardIdentity.setConfigsvrConnectionString(
+ ConnectionString(ConnectionString::SET, "a:1,b:2", "config"));
+ shardIdentity.setShardName(kShardName);
+ shardIdentity.setClusterId(OID::gen());
+ ASSERT_OK(shardIdentity.validate());
+ return shardIdentity.toShardIdentityDocument();
+ }();
- ShardIdentityType shardIdentity;
- shardIdentity.setConfigsvrConnString(
- ConnectionString(ConnectionString::SET, "a:1,b:2", "config"));
- shardIdentity.setShardName(kShardName);
- shardIdentity.setClusterId(OID::gen());
- ASSERT_OK(shardIdentity.validate());
- serverGlobalParams.overrideShardIdentity = shardIdentity.toBSON();
-
- auto swShardingInitialized =
- shardingState()->initializeShardingAwarenessIfNeeded(operationContext());
- ASSERT_OK(swShardingInitialized);
- ASSERT_TRUE(swShardingInitialized.getValue());
+ ASSERT(shardingInitialization()->initializeShardingAwarenessIfNeeded(operationContext()));
}
-// readOnly and not --shardsvr
-
-TEST_F(ShardingStateTest,
+/**
+ * readOnly and not --shardsvr
+ */
+TEST_F(ShardingInitializationMongoDTest,
InitializeShardingAwarenessIfNeededReadOnlyAndNotShardServerAndNoOverrideShardIdentity) {
storageGlobalParams.readOnly = true;
serverGlobalParams.clusterRole = ClusterRole::None;
- auto swShardingInitialized =
- shardingState()->initializeShardingAwarenessIfNeeded(operationContext());
- ASSERT_OK(swShardingInitialized);
- ASSERT_FALSE(swShardingInitialized.getValue());
+ ASSERT(!shardingInitialization()->initializeShardingAwarenessIfNeeded(operationContext()));
}
TEST_F(
- ShardingStateTest,
+ ShardingInitializationMongoDTest,
InitializeShardingAwarenessIfNeededReadOnlyAndNotShardServerAndInvalidOverrideShardIdentity) {
storageGlobalParams.readOnly = true;
serverGlobalParams.clusterRole = ClusterRole::None;
-
serverGlobalParams.overrideShardIdentity = BSON("_id"
<< "shardIdentity"
<< "configsvrConnectionString"
<< "invalid");
- auto swShardingInitialized =
- shardingState()->initializeShardingAwarenessIfNeeded(operationContext());
- ASSERT_EQUALS(ErrorCodes::InvalidOptions, swShardingInitialized.getStatus().code());
+
+ ASSERT_THROWS_CODE(
+ shardingInitialization()->initializeShardingAwarenessIfNeeded(operationContext()),
+ AssertionException,
+ ErrorCodes::InvalidOptions);
}
-TEST_F(ShardingStateTest,
+TEST_F(ShardingInitializationMongoDTest,
InitializeShardingAwarenessIfNeededReadOnlyAndNotShardServerAndValidOverrideShardIdentity) {
storageGlobalParams.readOnly = true;
serverGlobalParams.clusterRole = ClusterRole::None;
+ serverGlobalParams.overrideShardIdentity = [] {
+ ShardIdentityType shardIdentity;
+ shardIdentity.setConfigsvrConnectionString(
+ ConnectionString(ConnectionString::SET, "a:1,b:2", "config"));
+ shardIdentity.setShardName(kShardName);
+ shardIdentity.setClusterId(OID::gen());
+ ASSERT_OK(shardIdentity.validate());
+ return shardIdentity.toShardIdentityDocument();
+ }();
- ShardIdentityType shardIdentity;
- shardIdentity.setConfigsvrConnString(
- ConnectionString(ConnectionString::SET, "a:1,b:2", "config"));
- shardIdentity.setShardName(kShardName);
- shardIdentity.setClusterId(OID::gen());
- ASSERT_OK(shardIdentity.validate());
- serverGlobalParams.overrideShardIdentity = shardIdentity.toBSON();
-
- auto swShardingInitialized =
- shardingState()->initializeShardingAwarenessIfNeeded(operationContext());
- ASSERT_EQUALS(ErrorCodes::InvalidOptions, swShardingInitialized.getStatus().code());
+ ASSERT_THROWS_CODE(
+ shardingInitialization()->initializeShardingAwarenessIfNeeded(operationContext()),
+ AssertionException,
+ ErrorCodes::InvalidOptions);
}
-// not readOnly and --overrideShardIdentity
-
-TEST_F(ShardingStateTest,
+/**
+ * not readOnly and --overrideShardIdentity
+ */
+TEST_F(ShardingInitializationMongoDTest,
InitializeShardingAwarenessIfNeededNotReadOnlyAndInvalidOverrideShardIdentity) {
+ serverGlobalParams.clusterRole = ClusterRole::ShardServer;
serverGlobalParams.overrideShardIdentity = BSON("_id"
<< "shardIdentity"
<< "configsvrConnectionString"
<< "invalid");
- // Should error regardless of cluster role.
-
- serverGlobalParams.clusterRole = ClusterRole::ShardServer;
- auto swShardingInitialized =
- shardingState()->initializeShardingAwarenessIfNeeded(operationContext());
- ASSERT_EQUALS(ErrorCodes::InvalidOptions, swShardingInitialized.getStatus().code());
+ ASSERT_THROWS_CODE(
+ shardingInitialization()->initializeShardingAwarenessIfNeeded(operationContext()),
+ AssertionException,
+ ErrorCodes::InvalidOptions);
+ // Should error regardless of cluster role
serverGlobalParams.clusterRole = ClusterRole::None;
- swShardingInitialized =
- shardingState()->initializeShardingAwarenessIfNeeded(operationContext());
- ASSERT_EQUALS(ErrorCodes::InvalidOptions, swShardingInitialized.getStatus().code());
+ ASSERT_THROWS_CODE(
+ shardingInitialization()->initializeShardingAwarenessIfNeeded(operationContext()),
+ AssertionException,
+ ErrorCodes::InvalidOptions);
}
-TEST_F(ShardingStateTest,
+TEST_F(ShardingInitializationMongoDTest,
InitializeShardingAwarenessIfNeededNotReadOnlyAndValidOverrideShardIdentity) {
- ShardIdentityType shardIdentity;
- shardIdentity.setConfigsvrConnString(
- ConnectionString(ConnectionString::SET, "a:1,b:2", "config"));
- shardIdentity.setShardName(kShardName);
- shardIdentity.setClusterId(OID::gen());
- ASSERT_OK(shardIdentity.validate());
- serverGlobalParams.overrideShardIdentity = shardIdentity.toBSON();
-
- // Should error regardless of cluster role.
-
serverGlobalParams.clusterRole = ClusterRole::ShardServer;
- auto swShardingInitialized =
- shardingState()->initializeShardingAwarenessIfNeeded(operationContext());
- ASSERT_EQUALS(ErrorCodes::InvalidOptions, swShardingInitialized.getStatus().code());
+ serverGlobalParams.overrideShardIdentity = [] {
+ ShardIdentityType shardIdentity;
+ shardIdentity.setConfigsvrConnectionString(
+ ConnectionString(ConnectionString::SET, "a:1,b:2", "config"));
+ shardIdentity.setShardName(kShardName);
+ shardIdentity.setClusterId(OID::gen());
+ ASSERT_OK(shardIdentity.validate());
+ return shardIdentity.toShardIdentityDocument();
+ }();
+
+ ASSERT_THROWS_CODE(
+ shardingInitialization()->initializeShardingAwarenessIfNeeded(operationContext()),
+ AssertionException,
+ ErrorCodes::InvalidOptions);
+ // Should error regardless of cluster role
serverGlobalParams.clusterRole = ClusterRole::None;
- swShardingInitialized =
- shardingState()->initializeShardingAwarenessIfNeeded(operationContext());
- ASSERT_EQUALS(ErrorCodes::InvalidOptions, swShardingInitialized.getStatus().code());
+ ASSERT_THROWS_CODE(
+ shardingInitialization()->initializeShardingAwarenessIfNeeded(operationContext()),
+ AssertionException,
+ ErrorCodes::InvalidOptions);
}
-// not readOnly and --shardsvr
-
-TEST_F(ShardingStateTest,
+/**
+ * not readOnly and --shardsvr
+ */
+TEST_F(ShardingInitializationMongoDTest,
InitializeShardingAwarenessIfNeededNotReadOnlyAndShardServerAndNoShardIdentity) {
- auto swShardingInitialized =
- shardingState()->initializeShardingAwarenessIfNeeded(operationContext());
- ASSERT_OK(swShardingInitialized);
- ASSERT_FALSE(swShardingInitialized.getValue());
+ ASSERT(!shardingInitialization()->initializeShardingAwarenessIfNeeded(operationContext()));
}
-TEST_F(ShardingStateTest,
+TEST_F(ShardingInitializationMongoDTest,
InitializeShardingAwarenessIfNeededNotReadOnlyAndShardServerAndInvalidShardIdentity) {
// Insert the shardIdentity doc to disk while pretending that we are in "standalone" mode,
// otherwise OpObserver for inserts will prevent the insert from occurring because the
@@ -420,19 +436,24 @@ TEST_F(ShardingStateTest,
BSONObj invalidShardIdentity = BSON("_id"
<< "shardIdentity"
- << "configsvrConnectionString"
+ << ShardIdentity::kShardNameFieldName
+ << kShardName
+ << ShardIdentity::kClusterIdFieldName
+ << OID::gen()
+ << ShardIdentity::kConfigsvrConnectionStringFieldName
<< "invalid");
+
_dbDirectClient->insert(NamespaceString::kServerConfigurationNamespace.toString(),
invalidShardIdentity);
}
- auto swShardingInitialized =
- shardingState()->initializeShardingAwarenessIfNeeded(operationContext());
- ASSERT_EQUALS(ErrorCodes::UnsupportedFormat, swShardingInitialized.getStatus().code());
+ ASSERT_THROWS_CODE(
+ shardingInitialization()->initializeShardingAwarenessIfNeeded(operationContext()),
+ AssertionException,
+ ErrorCodes::UnsupportedFormat);
}
-
-TEST_F(ShardingStateTest,
+TEST_F(ShardingInitializationMongoDTest,
InitializeShardingAwarenessIfNeededNotReadOnlyAndShardServerAndValidShardIdentity) {
// Insert the shardIdentity doc to disk while pretending that we are in "standalone" mode,
// otherwise OpObserver for inserts will prevent the insert from occurring because the
@@ -442,37 +463,32 @@ TEST_F(ShardingStateTest,
BSONObj validShardIdentity = [&] {
ShardIdentityType shardIdentity;
- shardIdentity.setConfigsvrConnString(
+ shardIdentity.setConfigsvrConnectionString(
ConnectionString(ConnectionString::SET, "a:1,b:2", "config"));
shardIdentity.setShardName(kShardName);
shardIdentity.setClusterId(OID::gen());
ASSERT_OK(shardIdentity.validate());
- return shardIdentity.toBSON();
+ return shardIdentity.toShardIdentityDocument();
}();
_dbDirectClient->insert(NamespaceString::kServerConfigurationNamespace.toString(),
validShardIdentity);
}
- auto swShardingInitialized =
- shardingState()->initializeShardingAwarenessIfNeeded(operationContext());
- ASSERT_OK(swShardingInitialized);
- ASSERT_TRUE(swShardingInitialized.getValue());
+ ASSERT(shardingInitialization()->initializeShardingAwarenessIfNeeded(operationContext()));
}
-// not readOnly and not --shardsvr
-
-TEST_F(ShardingStateTest,
+/**
+ * not readOnly and not --shardsvr
+ */
+TEST_F(ShardingInitializationMongoDTest,
InitializeShardingAwarenessIfNeededNotReadOnlyAndNotShardServerAndNoShardIdentity) {
ScopedSetStandaloneMode standalone(getServiceContext());
- auto swShardingInitialized =
- shardingState()->initializeShardingAwarenessIfNeeded(operationContext());
- ASSERT_OK(swShardingInitialized);
- ASSERT_FALSE(swShardingInitialized.getValue());
+ ASSERT(!shardingInitialization()->initializeShardingAwarenessIfNeeded(operationContext()));
}
-TEST_F(ShardingStateTest,
+TEST_F(ShardingInitializationMongoDTest,
InitializeShardingAwarenessIfNeededNotReadOnlyAndNotShardServerAndInvalidShardIdentity) {
ScopedSetStandaloneMode standalone(getServiceContext());
@@ -484,34 +500,29 @@ TEST_F(ShardingStateTest,
// The shardIdentity doc on disk, even if invalid, is ignored if the ClusterRole is None. This
// is to allow fixing the shardIdentity doc by starting without --shardsvr.
- auto swShardingInitialized =
- shardingState()->initializeShardingAwarenessIfNeeded(operationContext());
- ASSERT_OK(swShardingInitialized);
- ASSERT_FALSE(swShardingInitialized.getValue());
+ ASSERT(!shardingInitialization()->initializeShardingAwarenessIfNeeded(operationContext()));
}
-TEST_F(ShardingStateTest,
+TEST_F(ShardingInitializationMongoDTest,
InitializeShardingAwarenessIfNeededNotReadOnlyAndNotShardServerAndValidShardIdentity) {
ScopedSetStandaloneMode standalone(getServiceContext());
BSONObj validShardIdentity = [&] {
ShardIdentityType shardIdentity;
- shardIdentity.setConfigsvrConnString(
+ shardIdentity.setConfigsvrConnectionString(
ConnectionString(ConnectionString::SET, "a:1,b:2", "config"));
shardIdentity.setShardName(kShardName);
shardIdentity.setClusterId(OID::gen());
ASSERT_OK(shardIdentity.validate());
- return shardIdentity.toBSON();
+ return shardIdentity.toShardIdentityDocument();
}();
_dbDirectClient->insert(NamespaceString::kServerConfigurationNamespace.toString(),
validShardIdentity);
- // The shardIdentity doc on disk is ignored if ClusterRole is None.
- auto swShardingInitialized =
- shardingState()->initializeShardingAwarenessIfNeeded(operationContext());
- ASSERT_OK(swShardingInitialized);
- ASSERT_FALSE(swShardingInitialized.getValue());
+ // The shardIdentity doc on disk, even if invalid, is ignored if the ClusterRole is None. This
+ // is to allow fixing the shardIdentity doc by starting without --shardsvr.
+ ASSERT(!shardingInitialization()->initializeShardingAwarenessIfNeeded(operationContext()));
}
} // namespace
diff --git a/src/mongo/db/s/sharding_initialization_op_observer_test.cpp b/src/mongo/db/s/sharding_initialization_op_observer_test.cpp
new file mode 100644
index 00000000000..da45e424a28
--- /dev/null
+++ b/src/mongo/db/s/sharding_initialization_op_observer_test.cpp
@@ -0,0 +1,148 @@
+/**
+ * Copyright (C) 2018 MongoDB, Inc.
+ *
+ * This program is free software: you can redistribute it and/or modify
+ * it under the terms of the GNU Affero General Public License, version 3,
+ * as published by the Free Software Foundation.
+ *
+ * This program is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ * GNU Affero General Public License for more details.
+ *
+ * You should have received a copy of the GNU Affero General Public License
+ * along with this program. If not, see <http://www.gnu.org/licenses/>.
+ *
+ * As a special exception, the copyright holders give permission to link the
+ * code of portions of this program with the OpenSSL library under certain
+ * conditions as described in each individual source file and distribute
+ * linked combinations including the program with the OpenSSL library. You
+ * must comply with the GNU Affero General Public License in all respects
+ * for all of the code used other than as permitted herein. If you modify
+ * file(s) with this exception, you may extend this exception to your
+ * version of the file(s), but you are not obligated to do so. If you do not
+ * wish to do so, delete this exception statement from your version. If you
+ * delete this exception statement from all source files in the program,
+ * then also delete it in the license file.
+ */
+
+#include "mongo/platform/basic.h"
+
+#include "mongo/client/remote_command_targeter_mock.h"
+#include "mongo/db/catalog_raii.h"
+#include "mongo/db/concurrency/d_concurrency.h"
+#include "mongo/db/dbdirectclient.h"
+#include "mongo/db/op_observer_impl.h"
+#include "mongo/db/op_observer_registry.h"
+#include "mongo/db/repl/replication_coordinator_mock.h"
+#include "mongo/db/s/config_server_op_observer.h"
+#include "mongo/db/s/shard_server_catalog_cache_loader.h"
+#include "mongo/db/s/shard_server_op_observer.h"
+#include "mongo/db/s/sharding_initialization_mongod.h"
+#include "mongo/db/s/type_shard_identity.h"
+#include "mongo/db/server_options.h"
+#include "mongo/s/catalog/dist_lock_manager_mock.h"
+#include "mongo/s/client/shard_registry.h"
+#include "mongo/s/config_server_catalog_cache_loader.h"
+#include "mongo/s/shard_server_test_fixture.h"
+
+namespace mongo {
+namespace {
+
+const std::string kShardName("TestShard");
+
+/**
+ * This test suite validates that when the default OpObserver chain is set up (which happens to
+ * include the ShardServerOpObserver), writes to the 'admin.system.version' collection (and the
+ * shardIdentity document specifically) will invoke the sharding initialization code.
+ */
+class ShardingInitializationOpObserverTest : public ShardServerTestFixture {
+public:
+ void setUp() override {
+ ShardServerTestFixture::setUp();
+
+ // NOTE: this assumes that globalInit will always be called on the same thread as the main
+ // test thread
+ ShardingInitializationMongoD::get(operationContext())
+ ->setGlobalInitMethodForTest([this](OperationContext* opCtx,
+ const ShardIdentity& shardIdentity,
+ StringData distLockProcessId) {
+ _initCallCount++;
+ return Status::OK();
+ });
+ }
+
+ void tearDown() override {
+ ShardingState::get(getServiceContext())->clearForTests();
+
+ ShardServerTestFixture::tearDown();
+ }
+
+ int getInitCallCount() const {
+ return _initCallCount;
+ }
+
+private:
+ int _initCallCount = 0;
+};
+
+TEST_F(ShardingInitializationOpObserverTest, GlobalInitGetsCalledAfterWriteCommits) {
+ ShardIdentityType shardIdentity;
+ shardIdentity.setConfigsvrConnectionString(
+ ConnectionString(ConnectionString::SET, "a:1,b:2", "config"));
+ shardIdentity.setShardName(kShardName);
+ shardIdentity.setClusterId(OID::gen());
+
+ DBDirectClient client(operationContext());
+ client.insert("admin.system.version", shardIdentity.toShardIdentityDocument());
+ ASSERT_EQ(1, getInitCallCount());
+}
+
+TEST_F(ShardingInitializationOpObserverTest, GlobalInitDoesntGetCalledIfWriteAborts) {
+ ShardIdentityType shardIdentity;
+ shardIdentity.setConfigsvrConnectionString(
+ ConnectionString(ConnectionString::SET, "a:1,b:2", "config"));
+ shardIdentity.setShardName(kShardName);
+ shardIdentity.setClusterId(OID::gen());
+
+ // This part of the test ensures that the collection exists for the AutoGetCollection below to
+ // find and also validates that the initializer does not get called for non-sharding documents
+ DBDirectClient client(operationContext());
+ client.insert("admin.system.version", BSON("_id" << 1));
+ ASSERT_EQ(0, getInitCallCount());
+
+ {
+ AutoGetCollection autoColl(
+ operationContext(), NamespaceString("admin.system.version"), MODE_IX);
+
+ WriteUnitOfWork wuow(operationContext());
+ ASSERT_OK(autoColl.getCollection()->insertDocument(
+ operationContext(), shardIdentity.toShardIdentityDocument(), {}, false));
+ ASSERT_EQ(0, getInitCallCount());
+ }
+
+ ASSERT_EQ(0, getInitCallCount());
+}
+
+TEST_F(ShardingInitializationOpObserverTest, GlobalInitDoesntGetsCalledIfNSIsNotForShardIdentity) {
+ ShardIdentityType shardIdentity;
+ shardIdentity.setConfigsvrConnectionString(
+ ConnectionString(ConnectionString::SET, "a:1,b:2", "config"));
+ shardIdentity.setShardName(kShardName);
+ shardIdentity.setClusterId(OID::gen());
+
+ DBDirectClient client(operationContext());
+ client.insert("admin.user", shardIdentity.toShardIdentityDocument());
+ ASSERT_EQ(0, getInitCallCount());
+}
+
+TEST_F(ShardingInitializationOpObserverTest, OnInsertOpThrowWithIncompleteShardIdentityDocument) {
+ DBDirectClient client(operationContext());
+ client.insert("admin.system.version",
+ BSON("_id" << ShardIdentityType::IdName << ShardIdentity::kShardNameFieldName
+ << kShardName));
+ ASSERT(!client.getLastError().empty());
+}
+
+} // namespace
+} // namespace mongo
diff --git a/src/mongo/db/s/sharding_state.cpp b/src/mongo/db/s/sharding_state.cpp
index f5e22df6838..8541a3b45c6 100644
--- a/src/mongo/db/s/sharding_state.cpp
+++ b/src/mongo/db/s/sharding_state.cpp
@@ -32,77 +32,19 @@
#include "mongo/db/s/sharding_state.h"
-#include "mongo/base/init.h"
-#include "mongo/bson/util/bson_extract.h"
-#include "mongo/client/connection_string.h"
-#include "mongo/client/replica_set_monitor.h"
-#include "mongo/db/auth/authorization_session.h"
-#include "mongo/db/catalog_raii.h"
-#include "mongo/db/client.h"
-#include "mongo/db/dbhelpers.h"
-#include "mongo/db/operation_context.h"
-#include "mongo/db/ops/update.h"
-#include "mongo/db/ops/update_lifecycle_impl.h"
-#include "mongo/db/repl/optime.h"
-#include "mongo/db/repl/replication_coordinator.h"
-#include "mongo/db/s/chunk_splitter.h"
#include "mongo/db/s/operation_sharding_state.h"
#include "mongo/db/s/sharded_connection_info.h"
-#include "mongo/db/s/sharding_initialization_mongod.h"
-#include "mongo/db/s/type_shard_identity.h"
-#include "mongo/executor/task_executor_pool.h"
-#include "mongo/rpc/metadata/config_server_metadata.h"
-#include "mongo/rpc/metadata/metadata_hook.h"
-#include "mongo/s/catalog/sharding_catalog_client.h"
-#include "mongo/s/catalog/type_chunk.h"
-#include "mongo/s/catalog_cache.h"
-#include "mongo/s/chunk_version.h"
-#include "mongo/s/client/shard_registry.h"
-#include "mongo/s/client/sharding_network_connection_hook.h"
-#include "mongo/s/grid.h"
-#include "mongo/s/sharding_initialization.h"
+#include "mongo/db/server_options.h"
#include "mongo/util/log.h"
-#include "mongo/util/mongoutils/str.h"
namespace mongo {
namespace {
const auto getShardingState = ServiceContext::declareDecoration<ShardingState>();
-/**
- * Updates the config server field of the shardIdentity document with the given connection string
- * if setName is equal to the config server replica set name.
- *
- * Note: This is intended to be used on a new thread that hasn't called Client::initThread.
- * One example use case is for the ReplicaSetMonitor asynchronous callback when it detects changes
- * to replica set membership.
- */
-void updateShardIdentityConfigStringCB(const std::string& setName,
- const std::string& newConnectionString) {
- auto configsvrConnStr =
- Grid::get(getGlobalServiceContext())->shardRegistry()->getConfigServerConnectionString();
- if (configsvrConnStr.getSetName() != setName) {
- // Ignore all change notification for other sets that are not the config server.
- return;
- }
-
- Client::initThread("updateShardIdentityConfigConnString");
- auto uniqOpCtx = Client::getCurrent()->makeOperationContext();
-
- auto status = ShardingState::get(uniqOpCtx.get())
- ->updateShardIdentityConfigString(uniqOpCtx.get(), newConnectionString);
- if (!status.isOK() && !ErrorCodes::isNotMasterError(status.code())) {
- warning() << "error encountered while trying to update config connection string to "
- << newConnectionString << causedBy(redact(status));
- }
-}
-
} // namespace
-ShardingState::ShardingState()
- : _initializationState(static_cast<uint32_t>(InitializationState::kNew)),
- _initializationStatus(Status(ErrorCodes::InternalError, "Uninitialized value")),
- _globalInit(&initializeGlobalShardingStateForMongod) {}
+ShardingState::ShardingState() = default;
ShardingState::~ShardingState() = default;
@@ -114,13 +56,38 @@ ShardingState* ShardingState::get(OperationContext* operationContext) {
return ShardingState::get(operationContext->getServiceContext());
}
-bool ShardingState::enabled() const {
- return _getInitializationState() == InitializationState::kInitialized;
+void ShardingState::setInitialized(ShardId shardId, OID clusterId) {
+ stdx::unique_lock<stdx::mutex> ul(_mutex);
+ invariant(_getInitializationState() == InitializationState::kNew);
+
+ _shardId = std::move(shardId);
+ _clusterId = std::move(clusterId);
+ _initializationStatus = Status::OK();
+
+ _initializationState.store(static_cast<uint32_t>(InitializationState::kInitialized));
+}
+
+void ShardingState::setInitialized(Status failedStatus) {
+ invariant(!failedStatus.isOK());
+ log() << "Failed to initialize sharding components" << causedBy(failedStatus);
+
+ stdx::unique_lock<stdx::mutex> ul(_mutex);
+ invariant(_getInitializationState() == InitializationState::kNew);
+
+ _initializationStatus = std::move(failedStatus);
+ _initializationState.store(static_cast<uint32_t>(InitializationState::kError));
}
-void ShardingState::setEnabledForTest(const std::string& shardName) {
- _setInitializationState(InitializationState::kInitialized);
- _shardName = shardName;
+boost::optional<Status> ShardingState::initializationStatus() {
+ stdx::unique_lock<stdx::mutex> ul(_mutex);
+ if (_getInitializationState() == InitializationState::kNew)
+ return boost::none;
+
+ return _initializationStatus;
+}
+
+bool ShardingState::enabled() const {
+ return _getInitializationState() == InitializationState::kInitialized;
}
Status ShardingState::canAcceptShardedCommands() const {
@@ -136,254 +103,16 @@ Status ShardingState::canAcceptShardedCommands() const {
}
}
-std::string ShardingState::getShardName() {
+ShardId ShardingState::shardId() {
invariant(enabled());
stdx::lock_guard<stdx::mutex> lk(_mutex);
- return _shardName;
-}
-
-void ShardingState::shutDown(OperationContext* opCtx) {
- stdx::unique_lock<stdx::mutex> lk(_mutex);
- if (enabled()) {
- Grid::get(opCtx)->getExecutorPool()->shutdownAndJoin();
- Grid::get(opCtx)->catalogClient()->shutDown(opCtx);
- }
-}
-
-Status ShardingState::updateConfigServerOpTimeFromMetadata(OperationContext* opCtx) {
- if (!enabled()) {
- // Nothing to do if sharding state has not been initialized.
- return Status::OK();
- }
-
- boost::optional<repl::OpTime> opTime = rpc::ConfigServerMetadata::get(opCtx).getOpTime();
- if (opTime) {
- if (!AuthorizationSession::get(opCtx->getClient())
- ->isAuthorizedForActionsOnResource(ResourcePattern::forClusterResource(),
- ActionType::internal)) {
- return Status(ErrorCodes::Unauthorized, "Unauthorized to update config opTime");
- }
-
- Grid::get(opCtx)->advanceConfigOpTime(*opTime);
- }
-
- return Status::OK();
-}
-
-void ShardingState::setGlobalInitMethodForTest(GlobalInitFunc func) {
- _globalInit = func;
-}
-
-// NOTE: This method will be called inside a database lock so it should never take any database
-// locks, perform I/O, or any long running operations.
-Status ShardingState::initializeFromShardIdentity(OperationContext* opCtx,
- const ShardIdentityType& shardIdentity) {
- invariant(serverGlobalParams.clusterRole == ClusterRole::ShardServer);
- invariant(opCtx->lockState()->isLocked());
-
- Status validationStatus = shardIdentity.validate();
- if (!validationStatus.isOK()) {
- return validationStatus.withContext(
- "Invalid shard identity document found when initializing sharding state");
- }
-
- log() << "initializing sharding state with: " << shardIdentity;
-
- stdx::unique_lock<stdx::mutex> lk(_mutex);
-
- auto configSvrConnStr = shardIdentity.getConfigsvrConnString();
-
- if (enabled()) {
- invariant(!_shardName.empty());
- fassert(40372, _shardName == shardIdentity.getShardName());
-
- auto prevConfigsvrConnStr =
- Grid::get(opCtx)->shardRegistry()->getConfigServerConnectionString();
- invariant(prevConfigsvrConnStr.type() == ConnectionString::SET);
- fassert(40373, prevConfigsvrConnStr.getSetName() == configSvrConnStr.getSetName());
-
- invariant(_clusterId.isSet());
- fassert(40374, _clusterId == shardIdentity.getClusterId());
-
- return Status::OK();
- }
-
- if (_getInitializationState() == InitializationState::kError) {
- return {ErrorCodes::ManualInterventionRequired,
- str::stream() << "Server's sharding metadata manager failed to initialize and will "
- "remain in this state until the instance is manually reset"
- << causedBy(_initializationStatus)};
- }
-
- try {
- Status status = _globalInit(opCtx, configSvrConnStr, generateDistLockProcessId(opCtx));
- if (status.isOK()) {
- ReplicaSetMonitor::setSynchronousConfigChangeHook(
- &ShardRegistry::replicaSetChangeShardRegistryUpdateHook);
- ReplicaSetMonitor::setAsynchronousConfigChangeHook(&updateShardIdentityConfigStringCB);
-
- // Determine primary/secondary/standalone state in order to properly initialize sharding
- // components.
- auto replCoord = repl::ReplicationCoordinator::get(opCtx);
- bool isReplSet =
- replCoord->getReplicationMode() == repl::ReplicationCoordinator::modeReplSet;
- bool isStandaloneOrPrimary =
- !isReplSet || (repl::ReplicationCoordinator::get(opCtx)->getMemberState() ==
- repl::MemberState::RS_PRIMARY);
-
- CatalogCacheLoader::get(opCtx).initializeReplicaSetRole(isStandaloneOrPrimary);
- ChunkSplitter::get(opCtx).setReplicaSetMode(isStandaloneOrPrimary);
-
- log() << "initialized sharding components for "
- << (isStandaloneOrPrimary ? "primary" : "secondary") << " node.";
- _setInitializationState(InitializationState::kInitialized);
- } else {
- log() << "failed to initialize sharding components" << causedBy(status);
- _initializationStatus = status;
- _setInitializationState(InitializationState::kError);
- }
- _shardName = shardIdentity.getShardName();
- _clusterId = shardIdentity.getClusterId();
-
- return status;
- } catch (const DBException& ex) {
- auto errorStatus = ex.toStatus();
- _initializationStatus = errorStatus;
- _setInitializationState(InitializationState::kError);
- return errorStatus;
- }
-
- MONGO_UNREACHABLE;
-}
-
-ShardingState::InitializationState ShardingState::_getInitializationState() const {
- return static_cast<InitializationState>(_initializationState.load());
+ return _shardId;
}
-void ShardingState::_setInitializationState(InitializationState newState) {
- _initializationState.store(static_cast<uint32_t>(newState));
-}
-
-StatusWith<bool> ShardingState::initializeShardingAwarenessIfNeeded(OperationContext* opCtx) {
- invariant(!opCtx->lockState()->isLocked());
-
- // In sharded readOnly mode, we ignore the shardIdentity document on disk and instead *require*
- // a shardIdentity document to be passed through --overrideShardIdentity.
- if (storageGlobalParams.readOnly) {
- if (serverGlobalParams.clusterRole == ClusterRole::ShardServer) {
- if (serverGlobalParams.overrideShardIdentity.isEmpty()) {
- return {ErrorCodes::InvalidOptions,
- "If started with --shardsvr in queryableBackupMode, a shardIdentity "
- "document must be provided through --overrideShardIdentity"};
- }
- auto swOverrideShardIdentity =
- ShardIdentityType::fromBSON(serverGlobalParams.overrideShardIdentity);
- if (!swOverrideShardIdentity.isOK()) {
- return swOverrideShardIdentity.getStatus();
- }
- {
- // Global lock is required to call initializeFromShardIdenetity().
- Lock::GlobalWrite lk(opCtx);
- auto status =
- initializeFromShardIdentity(opCtx, swOverrideShardIdentity.getValue());
- if (!status.isOK()) {
- return status;
- }
- }
- return true;
- } else {
- // Error if --overrideShardIdentity is used but *not* started with --shardsvr.
- if (!serverGlobalParams.overrideShardIdentity.isEmpty()) {
- return {
- ErrorCodes::InvalidOptions,
- str::stream()
- << "Not started with --shardsvr, but a shardIdentity document was provided "
- "through --overrideShardIdentity: "
- << serverGlobalParams.overrideShardIdentity};
- }
- return false;
- }
- }
- // In sharded *non*-readOnly mode, error if --overrideShardIdentity is provided. Use the
- // shardIdentity document on disk if one exists, but it is okay if no shardIdentity document is
- // provided at all (sharding awareness will be initialized when a shardIdentity document is
- // inserted).
- else {
- if (!serverGlobalParams.overrideShardIdentity.isEmpty()) {
- return {
- ErrorCodes::InvalidOptions,
- str::stream() << "--overrideShardIdentity is only allowed in sharded "
- "queryableBackupMode. If not in queryableBackupMode, you can edit "
- "the shardIdentity document by starting the server *without* "
- "--shardsvr, manually updating the shardIdentity document in the "
- << NamespaceString::kServerConfigurationNamespace.toString()
- << " collection, and restarting the server with --shardsvr."};
- }
-
- // Load the shardIdentity document from disk.
- BSONObj shardIdentityBSON;
- bool foundShardIdentity = false;
- try {
- AutoGetCollection autoColl(
- opCtx, NamespaceString::kServerConfigurationNamespace, MODE_IS);
- foundShardIdentity = Helpers::findOne(opCtx,
- autoColl.getCollection(),
- BSON("_id" << ShardIdentityType::IdName),
- shardIdentityBSON);
- } catch (const DBException& ex) {
- return ex.toStatus();
- }
-
- if (serverGlobalParams.clusterRole == ClusterRole::ShardServer) {
- if (!foundShardIdentity) {
- warning() << "Started with --shardsvr, but no shardIdentity document was found on "
- "disk in "
- << NamespaceString::kServerConfigurationNamespace
- << ". This most likely means this server has not yet been added to a "
- "sharded cluster.";
- return false;
- }
-
- invariant(!shardIdentityBSON.isEmpty());
-
- auto swShardIdentity = ShardIdentityType::fromBSON(shardIdentityBSON);
- if (!swShardIdentity.isOK()) {
- return swShardIdentity.getStatus();
- }
- {
- // Global lock is required to call initializeFromShardIdenetity().
- Lock::GlobalWrite lk(opCtx);
- auto status = initializeFromShardIdentity(opCtx, swShardIdentity.getValue());
- if (!status.isOK()) {
- return status;
- }
- }
- return true;
- } else {
- // Warn if a shardIdentity document is found on disk but *not* started with --shardsvr.
- if (!shardIdentityBSON.isEmpty()) {
- warning() << "Not started with --shardsvr, but a shardIdentity document was found "
- "on disk in "
- << NamespaceString::kServerConfigurationNamespace << ": "
- << shardIdentityBSON;
- }
- return false;
- }
- }
-}
-
-void ShardingState::appendInfo(OperationContext* opCtx, BSONObjBuilder& builder) {
- const bool isEnabled = enabled();
- builder.appendBool("enabled", isEnabled);
- if (!isEnabled)
- return;
-
+OID ShardingState::clusterId() {
+ invariant(enabled());
stdx::lock_guard<stdx::mutex> lk(_mutex);
-
- builder.append("configServer",
- Grid::get(opCtx)->shardRegistry()->getConfigServerConnectionString().toString());
- builder.append("shardName", _shardName);
- builder.append("clusterId", _clusterId);
+ return _clusterId;
}
bool ShardingState::needCollectionMetadata(OperationContext* opCtx, const std::string& ns) {
@@ -398,33 +127,8 @@ bool ShardingState::needCollectionMetadata(OperationContext* opCtx, const std::s
OperationShardingState::get(opCtx).hasShardVersion();
}
-Status ShardingState::updateShardIdentityConfigString(OperationContext* opCtx,
- const std::string& newConnectionString) {
- BSONObj updateObj(ShardIdentityType::createConfigServerUpdateObject(newConnectionString));
-
- UpdateRequest updateReq(NamespaceString::kServerConfigurationNamespace);
- updateReq.setQuery(BSON("_id" << ShardIdentityType::IdName));
- updateReq.setUpdates(updateObj);
- UpdateLifecycleImpl updateLifecycle(NamespaceString::kServerConfigurationNamespace);
- updateReq.setLifecycle(&updateLifecycle);
-
- try {
- AutoGetOrCreateDb autoDb(
- opCtx, NamespaceString::kServerConfigurationNamespace.db(), MODE_X);
-
- auto result = update(opCtx, autoDb.getDb(), updateReq);
- if (result.numMatched == 0) {
- warning() << "failed to update config string of shard identity document because "
- << "it does not exist. This shard could have been removed from the cluster";
- } else {
- LOG(2) << "Updated config server connection string in shardIdentity document to"
- << newConnectionString;
- }
- } catch (const DBException& exception) {
- return exception.toStatus();
- }
-
- return Status::OK();
+void ShardingState::clearForTests() {
+ _initializationState.store(static_cast<uint32_t>(InitializationState::kNew));
}
} // namespace mongo
diff --git a/src/mongo/db/s/sharding_state.h b/src/mongo/db/s/sharding_state.h
index 38e663b9087..efdb45a30ec 100644
--- a/src/mongo/db/s/sharding_state.h
+++ b/src/mongo/db/s/sharding_state.h
@@ -29,56 +29,52 @@
#pragma once
#include <string>
-#include <vector>
#include "mongo/base/disallow_copying.h"
#include "mongo/bson/oid.h"
-#include "mongo/db/s/collection_sharding_state.h"
-#include "mongo/stdx/functional.h"
-#include "mongo/stdx/memory.h"
+#include "mongo/s/shard_id.h"
#include "mongo/stdx/mutex.h"
namespace mongo {
-class BSONObj;
-class BSONObjBuilder;
-class ConnectionString;
class OperationContext;
class ServiceContext;
-class ShardIdentityType;
-class Status;
-
-namespace repl {
-class OpTime;
-} // namespace repl
/**
- * Contains the global sharding state for a running mongod. There is one instance of this object per
- * service context and it is never destroyed for the lifetime of the context.
+ * Contains information about the shardingness of a running mongod. This is a passive class and its
+ * state and concurrency control is driven from outside (specifically ShardingInitializationMongoD,
+ * which should be its only caller).
+ *
+ * There is one instance of this object per service context and once 'setInitialized' is called, it
+ * never gets destroyed or uninitialized.
*/
class ShardingState {
MONGO_DISALLOW_COPYING(ShardingState);
public:
- using GlobalInitFunc =
- stdx::function<Status(OperationContext*, const ConnectionString&, StringData)>;
-
ShardingState();
~ShardingState();
- /**
- * Retrieves the sharding state object associated with the specified service context. This
- * method must only be called if ShardingState decoration has been created on the service
- * context, otherwise it will fassert. In other words, it may only be called on MongoD and
- * tests, which specifically require and instantiate ShardingState.
- *
- * Returns the instance's ShardingState.
- */
static ShardingState* get(ServiceContext* serviceContext);
static ShardingState* get(OperationContext* operationContext);
/**
- * Returns true if ShardingState has been successfully initialized.
+ * Puts the sharding state singleton in the "initialization completed" state with either
+ * successful initialization or an error. This method may only be called once for the lifetime
+ * of the object.
+ */
+ void setInitialized(ShardId shardId, OID clusterId);
+ void setInitialized(Status failedStatus);
+
+ /**
+ * If 'setInitialized' has not been called, returns boost::none. Otherwise, returns the status
+ * with which 'setInitialized' was called. This is used by the initialization sequence to decide
+ * whether to set up the sharding services.
+ */
+ boost::optional<Status> initializationStatus();
+
+ /**
+ * Returns true if 'setInitialized' has been called with shardId and clusterId.
*
* Code that needs to perform extra actions if sharding is initialized, but does not need to
* error if not, should use this. Alternatively, see ShardingState::canAcceptShardedCommands().
@@ -86,12 +82,6 @@ public:
bool enabled() const;
/**
- * Force-sets the initialization state to InitializationState::kInitialized, for testing
- * purposes. Note that this function should ONLY be used for testing purposes.
- */
- void setEnabledForTest(const std::string& shardName);
-
- /**
* Returns Status::OK if the ShardingState is enabled; if not, returns an error describing
* whether the ShardingState is just not yet initialized, or if this shard is not running with
* --shardsvr at all.
@@ -101,62 +91,30 @@ public:
*/
Status canAcceptShardedCommands() const;
- std::string getShardName();
-
/**
- * Initializes the sharding state of this server from the shard identity document argument
- * and sets secondary or primary state information on the catalog cache loader.
- *
- * Note: caller must hold a global/database lock! Needed in order to stably check for
- * replica set state (primary, secondary, standalone).
+ * Returns the shard id to which this node belongs. May only be called if 'enabled()' above
+ * returns true.
*/
- Status initializeFromShardIdentity(OperationContext* opCtx,
- const ShardIdentityType& shardIdentity);
+ ShardId shardId();
/**
- * Shuts down sharding machinery on the shard.
+ * Returns the cluster id of the cluster to which this node belongs. May only be called if
+ * 'enabled()' above returns true.
*/
- void shutDown(OperationContext* opCtx);
+ OID clusterId();
/**
- * Updates the ShardRegistry's stored notion of the config server optime based on the
- * ConfigServerMetadata decoration attached to the OperationContext.
+ * Returns true if this node is a shard and if the currently runnint operation must engage the
+ * sharding subsystem (i.e., perform version checking, orphan filtering, etc).
*/
- Status updateConfigServerOpTimeFromMetadata(OperationContext* opCtx);
-
- void appendInfo(OperationContext* opCtx, BSONObjBuilder& b);
-
bool needCollectionMetadata(OperationContext* opCtx, const std::string& ns);
/**
- * Updates the config server field of the shardIdentity document with the given connection
- * string.
- *
- * Note: this can return NotMaster error.
- */
- Status updateShardIdentityConfigString(OperationContext* opCtx,
- const std::string& newConnectionString);
-
- /**
- * For testing only. Mock the initialization method used by initializeFromConfigConnString and
- * initializeFromShardIdentity after all checks are performed.
+ * For testing only. This is a workaround for the fact that it is not possible to get a clean
+ * ServiceContext in between test executions. Because of this, tests which require that they get
+ * started with a clean (uninitialized) ShardingState must invoke this in their tearDown method.
*/
- void setGlobalInitMethodForTest(GlobalInitFunc func);
-
- /**
- * If started with --shardsvr, initializes sharding awareness from the shardIdentity document
- * on disk, if there is one.
- * If started with --shardsvr in queryableBackupMode, initializes sharding awareness from the
- * shardIdentity document passed through the --overrideShardIdentity startup parameter.
- *
- * If returns true, the ShardingState::_globalInit method was called, meaning all the core
- * classes for sharding were initialized, but no networking calls were made yet (with the
- * exception of the duplicate ShardRegistry reload in ShardRegistry::startup() (see
- * SERVER-26123). Outgoing networking calls to cluster members can now be made.
- *
- * Note: this function briefly takes the global lock to determine primary/secondary state.
- */
- StatusWith<bool> initializeShardingAwarenessIfNeeded(OperationContext* opCtx);
+ void clearForTests();
private:
// Progress of the sharding state initialization
@@ -178,33 +136,24 @@ private:
/**
* Returns the initialization state.
*/
- InitializationState _getInitializationState() const;
-
- /**
- * Updates the initialization state.
- */
- void _setInitializationState(InitializationState newState);
+ InitializationState _getInitializationState() const {
+ return static_cast<InitializationState>(_initializationState.load());
+ }
// Protects state below
stdx::mutex _mutex;
// State of the initialization of the sharding state along with any potential errors
- AtomicUInt32 _initializationState;
-
- // Only valid if _initializationState is kError. Contains the reason for initialization failure.
- Status _initializationStatus;
-
- // Signaled when ::initialize finishes.
- stdx::condition_variable _initializationFinishedCondition;
+ AtomicUInt32 _initializationState{static_cast<uint32_t>(InitializationState::kNew)};
// Sets the shard name for this host (comes through setShardVersion)
- std::string _shardName;
+ ShardId _shardId;
// The id for the cluster this shard belongs to.
OID _clusterId;
- // Function for initializing the external sharding state components not owned here.
- GlobalInitFunc _globalInit;
+ // Only valid if _initializationState is kError. Contains the reason for initialization failure.
+ Status _initializationStatus{ErrorCodes::InternalError, "Uninitialized value"};
};
} // namespace mongo
diff --git a/src/mongo/db/s/sharding_state_command.cpp b/src/mongo/db/s/sharding_state_command.cpp
index f22986f7a71..24abfee411c 100644
--- a/src/mongo/db/s/sharding_state_command.cpp
+++ b/src/mongo/db/s/sharding_state_command.cpp
@@ -34,7 +34,9 @@
#include "mongo/db/auth/action_type.h"
#include "mongo/db/auth/privilege.h"
#include "mongo/db/commands.h"
+#include "mongo/db/s/collection_sharding_state.h"
#include "mongo/db/s/sharding_state.h"
+#include "mongo/s/grid.h"
#include "mongo/util/log.h"
namespace mongo {
@@ -68,8 +70,20 @@ public:
const std::string& dbname,
const BSONObj& cmdObj,
BSONObjBuilder& result) override {
- ShardingState::get(opCtx)->appendInfo(opCtx, result);
- CollectionShardingState::report(opCtx, &result);
+ auto const shardingState = ShardingState::get(opCtx);
+ const bool isEnabled = shardingState->enabled();
+ result.appendBool("enabled", isEnabled);
+
+ if (isEnabled) {
+ auto const shardRegistry = Grid::get(opCtx)->shardRegistry();
+ result.append("configServer",
+ shardRegistry->getConfigServerConnectionString().toString());
+ result.append("shardName", shardingState->shardId());
+ result.append("clusterId", shardingState->clusterId());
+
+ CollectionShardingState::report(opCtx, &result);
+ }
+
return true;
}
diff --git a/src/mongo/db/s/sharding_state_recovery.cpp b/src/mongo/db/s/sharding_state_recovery.cpp
index 3623d448ec7..8b6e9d99f22 100644
--- a/src/mongo/db/s/sharding_state_recovery.cpp
+++ b/src/mongo/db/s/sharding_state_recovery.cpp
@@ -159,10 +159,11 @@ Status modifyRecoveryDocument(OperationContext* opCtx,
// nodes still expect to find them, so we must include them until after 4.0 ships.
//
// TODO SERVER-34166: Stop writing config server connection string and shard name.
+ auto const grid = Grid::get(opCtx);
BSONObj updateObj = RecoveryDocument::createChangeObj(
- Grid::get(opCtx)->shardRegistry()->getConfigServerConnectionString(),
- ShardingState::get(opCtx)->getShardName(),
- Grid::get(opCtx)->configOpTime(),
+ grid->shardRegistry()->getConfigServerConnectionString(),
+ ShardingState::get(opCtx)->shardId().toString(),
+ grid->configOpTime(),
change);
LOG(1) << "Changing sharding recovery document " << redact(updateObj);
diff --git a/src/mongo/db/s/shardsvr_shard_collection.cpp b/src/mongo/db/s/shardsvr_shard_collection.cpp
index f85d6ed9fb2..7c7acd530cc 100644
--- a/src/mongo/db/s/shardsvr_shard_collection.cpp
+++ b/src/mongo/db/s/shardsvr_shard_collection.cpp
@@ -42,7 +42,7 @@
#include "mongo/db/logical_clock.h"
#include "mongo/db/namespace_string.h"
#include "mongo/db/query/collation/collator_factory_interface.h"
-#include "mongo/db/s/collection_sharding_state.h"
+#include "mongo/db/s/collection_sharding_runtime.h"
#include "mongo/db/s/config/initial_split_policy.h"
#include "mongo/db/s/config/sharding_catalog_manager.h"
#include "mongo/db/s/shard_filtering_metadata_refresh.h"
@@ -710,7 +710,7 @@ public:
finalSplitPoints,
tags,
fromMapReduce,
- ShardingState::get(opCtx)->getShardName(),
+ ShardingState::get(opCtx)->shardId(),
numContiguousChunksPerShard);
return true;
diff --git a/src/mongo/db/s/split_chunk.cpp b/src/mongo/db/s/split_chunk.cpp
index 0b7e287a774..12470da163a 100644
--- a/src/mongo/db/s/split_chunk.cpp
+++ b/src/mongo/db/s/split_chunk.cpp
@@ -103,7 +103,7 @@ bool checkMetadataForSuccessfulSplitChunk(OperationContext* opCtx,
uassert(ErrorCodes::StaleEpoch,
str::stream() << "Collection " << nss.ns() << " became unsharded",
- metadataAfterSplit);
+ metadataAfterSplit->isSharded());
auto newChunkBounds(splitKeys);
auto startKey = chunkRange.getMin();
diff --git a/src/mongo/db/s/split_chunk_test.cpp b/src/mongo/db/s/split_chunk_test.cpp
index 45430b5272e..8258e3472fc 100644
--- a/src/mongo/db/s/split_chunk_test.cpp
+++ b/src/mongo/db/s/split_chunk_test.cpp
@@ -31,11 +31,10 @@
#include <boost/optional.hpp>
-#include "mongo/db/s/split_chunk.h"
-
#include "mongo/client/remote_command_targeter_mock.h"
#include "mongo/db/json.h"
-#include "mongo/db/s/sharding_state.h"
+#include "mongo/db/s/sharding_initialization_mongod.h"
+#include "mongo/db/s/split_chunk.h"
#include "mongo/db/server_options.h"
#include "mongo/executor/network_interface_mock.h"
#include "mongo/executor/remote_command_request.h"
@@ -66,6 +65,7 @@ public:
void setUp() override {
ShardServerTestFixture::setUp();
+ ShardingState::get(operationContext())->setInitialized(_shardId, OID::gen());
CatalogCacheLoader::get(getServiceContext()).initializeReplicaSetRole(true);
// Instantiate names.
@@ -210,7 +210,6 @@ void SplitChunkTest::emptyResponse() {
}
TEST_F(SplitChunkTest, HashedKeyPatternNumberLongSplitKeys) {
-
BSONObj keyPatternObj = BSON("foo"
<< "hashed");
_coll.setKeyPattern(BSON("_id"
@@ -222,9 +221,6 @@ TEST_F(SplitChunkTest, HashedKeyPatternNumberLongSplitKeys) {
validSplitKeys.push_back(BSON("foo" << i));
}
- // Force-set the sharding state to enabled with the _shardId, for testing purposes.
- ShardingState::get(operationContext())->setEnabledForTest(_shardId.toString());
-
expectLock();
// Call the splitChunk function asynchronously on a different thread, so that we do not block,
@@ -260,7 +256,6 @@ TEST_F(SplitChunkTest, HashedKeyPatternNumberLongSplitKeys) {
}
TEST_F(SplitChunkTest, HashedKeyPatternIntegerSplitKeys) {
-
BSONObj keyPatternObj = BSON("foo"
<< "hashed");
_coll.setKeyPattern(BSON("_id"
@@ -271,9 +266,6 @@ TEST_F(SplitChunkTest, HashedKeyPatternIntegerSplitKeys) {
std::vector<BSONObj> invalidSplitKeys{
BSON("foo" << -1), BSON("foo" << 0), BSON("foo" << 1), BSON("foo" << 42)};
- // Force-set the sharding state to enabled with the _shardId, for testing purposes.
- ShardingState::get(operationContext())->setEnabledForTest(_shardId.toString());
-
expectLock();
// Call the splitChunk function asynchronously on a different thread, so that we do not block,
@@ -302,7 +294,6 @@ TEST_F(SplitChunkTest, HashedKeyPatternIntegerSplitKeys) {
}
TEST_F(SplitChunkTest, HashedKeyPatternDoubleSplitKeys) {
-
BSONObj keyPatternObj = BSON("foo"
<< "hashed");
_coll.setKeyPattern(BSON("_id"
@@ -313,9 +304,6 @@ TEST_F(SplitChunkTest, HashedKeyPatternDoubleSplitKeys) {
std::vector<BSONObj> invalidSplitKeys{
BSON("foo" << 47.21230129), BSON("foo" << 1.0), BSON("foo" << 0.0), BSON("foo" << -0.001)};
- // Force-set the sharding state to enabled with the _shardId, for testing purposes.
- ShardingState::get(operationContext())->setEnabledForTest(_shardId.toString());
-
expectLock();
// Call the splitChunk function asynchronously on a different thread, so that we do not block,
@@ -344,7 +332,6 @@ TEST_F(SplitChunkTest, HashedKeyPatternDoubleSplitKeys) {
}
TEST_F(SplitChunkTest, HashedKeyPatternStringSplitKeys) {
-
BSONObj keyPatternObj = BSON("foo"
<< "hashed");
_coll.setKeyPattern(BSON("_id"
@@ -361,9 +348,6 @@ TEST_F(SplitChunkTest, HashedKeyPatternStringSplitKeys) {
BSON("foo"
<< "")};
- // Force-set the sharding state to enabled with the _shardId, for testing purposes.
- ShardingState::get(operationContext())->setEnabledForTest(_shardId.toString());
-
expectLock();
// Call the splitChunk function asynchronously on a different thread, so that we do not block,
@@ -392,7 +376,6 @@ TEST_F(SplitChunkTest, HashedKeyPatternStringSplitKeys) {
}
TEST_F(SplitChunkTest, ValidRangeKeyPatternSplitKeys) {
-
BSONObj keyPatternObj = BSON("foo" << 1);
// Build a vector of valid split keys, which contains values that may not necessarily be able
@@ -406,9 +389,6 @@ TEST_F(SplitChunkTest, ValidRangeKeyPatternSplitKeys) {
<< ""),
BSON("foo" << 3.1415926535)};
- // Force-set the sharding state to enabled with the _shardId, for testing purposes.
- ShardingState::get(operationContext())->setEnabledForTest(_shardId.toString());
-
expectLock();
// Call the splitChunk function asynchronously on a different thread, so that we do not block,
@@ -444,7 +424,6 @@ TEST_F(SplitChunkTest, ValidRangeKeyPatternSplitKeys) {
}
TEST_F(SplitChunkTest, SplitChunkWithNoErrors) {
-
BSONObj keyPatternObj = BSON("foo" << 1);
// Build a vector of split keys. Note that we start at {"foo" : 256} and end at {"foo" : 768},
@@ -454,9 +433,6 @@ TEST_F(SplitChunkTest, SplitChunkWithNoErrors) {
splitKeys.push_back(BSON("foo" << i));
}
- // Force-set the sharding state to enabled with the _shardId, for testing purposes.
- ShardingState::get(operationContext())->setEnabledForTest(_shardId.toString());
-
expectLock();
// Call the splitChunk function asynchronously on a different thread, so that we do not block,
@@ -509,7 +485,6 @@ TEST_F(SplitChunkTest, SplitChunkWithNoErrors) {
}
TEST_F(SplitChunkTest, AttemptSplitWithConfigsvrError) {
-
BSONObj keyPatternObj = BSON("foo" << 1);
// Build a vector of split keys. Note that we start at {"foo" : 0} and end at {"foo" : 1024},
@@ -519,9 +494,6 @@ TEST_F(SplitChunkTest, AttemptSplitWithConfigsvrError) {
splitKeys.push_back(BSON("foo" << i));
}
- // Force-set the sharding state to enabled with the _shardId, for testing purposes.
- ShardingState::get(operationContext())->setEnabledForTest(_shardId.toString());
-
expectLock();
// Call the splitChunk function asynchronously on a different thread, so that we do not block,
@@ -557,7 +529,6 @@ TEST_F(SplitChunkTest, AttemptSplitWithConfigsvrError) {
}
TEST_F(SplitChunkTest, AttemptSplitOnNoDatabases) {
-
BSONObj keyPatternObj = BSON("foo" << 1);
// Build a vector of split keys. Note that we start at {"foo" : 256} and end at {"foo" : 768},
@@ -567,9 +538,6 @@ TEST_F(SplitChunkTest, AttemptSplitOnNoDatabases) {
splitKeys.push_back(BSON("foo" << i));
}
- // Force-set the sharding state to enabled with the _shardId, for testing purposes.
- ShardingState::get(operationContext())->setEnabledForTest(_shardId.toString());
-
expectLock();
// Call the splitChunk function asynchronously on a different thread, so that we do not block,
@@ -595,7 +563,6 @@ TEST_F(SplitChunkTest, AttemptSplitOnNoDatabases) {
}
TEST_F(SplitChunkTest, AttemptSplitOnNoCollections) {
-
BSONObj keyPatternObj = BSON("foo" << 1);
// Build a vector of split keys. Note that we start at {"foo" : 256} and end at {"foo" : 768},
@@ -605,9 +572,6 @@ TEST_F(SplitChunkTest, AttemptSplitOnNoCollections) {
splitKeys.push_back(BSON("foo" << i));
}
- // Force-set the sharding state to enabled with the _shardId, for testing purposes.
- ShardingState::get(operationContext())->setEnabledForTest(_shardId.toString());
-
expectLock();
// Call the splitChunk function asynchronously on a different thread, so that we do not block,
@@ -636,7 +600,6 @@ TEST_F(SplitChunkTest, AttemptSplitOnNoCollections) {
}
TEST_F(SplitChunkTest, AttemptSplitOnNoChunks) {
-
BSONObj keyPatternObj = BSON("foo" << 1);
// Build a vector of split keys. Note that we start at {"foo" : 256} and end at {"foo" : 768},
@@ -646,9 +609,6 @@ TEST_F(SplitChunkTest, AttemptSplitOnNoChunks) {
splitKeys.push_back(BSON("foo" << i));
}
- // Force-set the sharding state to enabled with the _shardId, for testing purposes.
- ShardingState::get(operationContext())->setEnabledForTest(_shardId.toString());
-
expectLock();
// Call the splitChunk function asynchronously on a different thread, so that we do not block,
@@ -683,7 +643,6 @@ TEST_F(SplitChunkTest, AttemptSplitOnNoChunks) {
}
TEST_F(SplitChunkTest, NoCollectionAfterSplit) {
-
BSONObj keyPatternObj = BSON("foo" << 1);
// Build a vector of split keys. Note that we start at {"foo" : 256} and end at {"foo" : 768},
@@ -693,9 +652,6 @@ TEST_F(SplitChunkTest, NoCollectionAfterSplit) {
splitKeys.push_back(BSON("foo" << i));
}
- // Force-set the sharding state to enabled with the _shardId, for testing purposes.
- ShardingState::get(operationContext())->setEnabledForTest(_shardId.toString());
-
expectLock();
// Call the splitChunk function asynchronously on a different thread, so that we do not block,
@@ -731,7 +687,6 @@ TEST_F(SplitChunkTest, NoCollectionAfterSplit) {
}
TEST_F(SplitChunkTest, NoChunksAfterSplit) {
-
BSONObj keyPatternObj = BSON("foo" << 1);
// Build a vector of split keys. Note that we start at {"foo" : 256} and end at {"foo" : 768},
@@ -741,9 +696,6 @@ TEST_F(SplitChunkTest, NoChunksAfterSplit) {
splitKeys.push_back(BSON("foo" << i));
}
- // Force-set the sharding state to enabled with the _shardId, for testing purposes.
- ShardingState::get(operationContext())->setEnabledForTest(_shardId.toString());
-
expectLock();
// Call the splitChunk function asynchronously on a different thread, so that we do not block,
diff --git a/src/mongo/db/s/split_vector_test.cpp b/src/mongo/db/s/split_vector_test.cpp
index ae5f4e6f9b2..98f323846aa 100644
--- a/src/mongo/db/s/split_vector_test.cpp
+++ b/src/mongo/db/s/split_vector_test.cpp
@@ -30,13 +30,13 @@
#include "mongo/platform/basic.h"
-#include "mongo/db/s/split_vector.h"
-
#include "mongo/db/dbdirectclient.h"
+#include "mongo/db/s/split_vector.h"
#include "mongo/s/shard_server_test_fixture.h"
#include "mongo/unittest/unittest.h"
namespace mongo {
+namespace {
const NamespaceString kNss = NamespaceString("foo", "bar");
const std::string kPattern = "_id";
@@ -45,8 +45,10 @@ class SplitVectorTest : public ShardServerTestFixture {
public:
void setUp() {
ShardServerTestFixture::setUp();
+
DBDirectClient dbclient(operationContext());
ASSERT_TRUE(dbclient.createCollection(kNss.ns()));
+ dbclient.createIndex(kNss.ns(), BSON(kPattern << 1));
// Insert 100 documents into the collection so the tests can test splitting with different
// constraints.
@@ -68,8 +70,6 @@ private:
const long long docSizeBytes = BSON(kPattern << 1).objsize();
};
-namespace {
-
TEST_F(SplitVectorTest, SplitVectorInHalf) {
std::vector<BSONObj> splitKeys = unittest::assertGet(splitVector(operationContext(),
kNss,
diff --git a/src/mongo/db/s/type_shard_identity.cpp b/src/mongo/db/s/type_shard_identity.cpp
index c94fa6e5892..b9e739c3bc9 100644
--- a/src/mongo/db/s/type_shard_identity.cpp
+++ b/src/mongo/db/s/type_shard_identity.cpp
@@ -38,122 +38,46 @@ namespace mongo {
const std::string ShardIdentityType::IdName("shardIdentity");
-const BSONField<std::string> ShardIdentityType::configsvrConnString("configsvrConnectionString");
-const BSONField<std::string> ShardIdentityType::shardName("shardName");
-const BSONField<OID> ShardIdentityType::clusterId("clusterId");
-
-StatusWith<ShardIdentityType> ShardIdentityType::fromBSON(const BSONObj& source) {
+StatusWith<ShardIdentityType> ShardIdentityType::fromShardIdentityDocument(const BSONObj& source) {
if (!source.hasField("_id")) {
return {ErrorCodes::NoSuchKey,
str::stream() << "missing _id field for shardIdentity document"};
}
-
- ShardIdentityType shardIdentity;
-
- {
- std::string docId;
- Status status = bsonExtractStringField(source, "_id", &docId);
- if (!status.isOK()) {
- return status;
- }
-
- if (docId != IdName) {
- return {ErrorCodes::FailedToParse,
- str::stream() << "got _id: " << docId << " instead of " << IdName};
- }
- }
-
- {
- std::string connString;
- Status status = bsonExtractStringField(source, configsvrConnString(), &connString);
- if (!status.isOK()) {
- return status;
+ // Strip the id field since it's always the same and we don't store it
+ auto shardIdentityBSON = source.removeField("_id");
+
+ try {
+ ShardIdentityType shardIdentity =
+ ShardIdentity::parse(IDLParserErrorContext("ShardIdentity"), shardIdentityBSON);
+
+ const auto& configsvrConnStr = shardIdentity.getConfigsvrConnectionString();
+ if (configsvrConnStr.type() != ConnectionString::SET) {
+ return Status(ErrorCodes::UnsupportedFormat,
+ str::stream()
+ << "config server connection string can only be replica sets: "
+ << configsvrConnStr.toString());
}
- try {
- // Note: ConnectionString::parse can uassert from HostAndPort constructor.
- auto parsedConfigConnStrStatus = ConnectionString::parse(connString);
- if (!parsedConfigConnStrStatus.isOK()) {
- return parsedConfigConnStrStatus.getStatus();
- }
-
- auto configSvrConnStr = parsedConfigConnStrStatus.getValue();
- if (configSvrConnStr.type() != ConnectionString::SET) {
- return Status(ErrorCodes::UnsupportedFormat,
- str::stream()
- << "config server connection string can only be replica sets: "
- << configSvrConnStr.toString());
- }
-
- shardIdentity.setConfigsvrConnString(std::move(configSvrConnStr));
- } catch (const AssertionException& parseException) {
- return parseException.toStatus();
- }
+ return shardIdentity;
+ } catch (const AssertionException& parseException) {
+ return parseException.toStatus();
}
-
- {
- std::string name;
- Status status = bsonExtractStringField(source, shardName(), &name);
- if (!status.isOK()) {
- return status;
- }
-
- shardIdentity.setShardName(name);
- }
-
- {
- OID oid;
- Status status = bsonExtractOIDField(source, clusterId(), &oid);
- if (!status.isOK()) {
- return status;
- }
-
- shardIdentity.setClusterId(oid);
- }
-
- return shardIdentity;
}
Status ShardIdentityType::validate() const {
- if (!_configsvrConnString) {
- return {ErrorCodes::NoSuchKey,
- str::stream() << "missing " << configsvrConnString() << " field"};
- }
-
- if (_configsvrConnString->type() != ConnectionString::SET) {
+ const auto& configsvrConnStr = getConfigsvrConnectionString();
+ if (configsvrConnStr.type() != ConnectionString::SET) {
return {ErrorCodes::UnsupportedFormat,
str::stream() << "config connection string can only be replica sets, got "
- << ConnectionString::typeToString(_configsvrConnString->type())};
- }
-
- if (!_shardName || _shardName->empty()) {
- return {ErrorCodes::NoSuchKey, str::stream() << "missing " << shardName() << " field"};
- }
-
- if (!_clusterId || !_clusterId->isSet()) {
- return {ErrorCodes::NoSuchKey, str::stream() << "missing " << clusterId() << " field"};
+ << ConnectionString::typeToString(configsvrConnStr.type())};
}
-
return Status::OK();
}
-BSONObj ShardIdentityType::toBSON() const {
+BSONObj ShardIdentityType::toShardIdentityDocument() const {
BSONObjBuilder builder;
-
- builder.append("_id", IdName);
-
- if (_configsvrConnString) {
- builder << configsvrConnString(_configsvrConnString->toString());
- }
-
- if (_shardName) {
- builder << shardName(_shardName.get());
- }
-
- if (_clusterId) {
- builder << clusterId(_clusterId.get());
- }
-
+ builder.append("_id", ShardIdentityType::IdName);
+ builder.appendElements(ShardIdentity::toBSON());
return builder.obj();
}
@@ -161,49 +85,10 @@ std::string ShardIdentityType::toString() const {
return toBSON().toString();
}
-bool ShardIdentityType::isConfigsvrConnStringSet() const {
- return _configsvrConnString.is_initialized();
-}
-
-const ConnectionString& ShardIdentityType::getConfigsvrConnString() const {
- invariant(_configsvrConnString);
- return _configsvrConnString.get();
-}
-
-void ShardIdentityType::setConfigsvrConnString(ConnectionString connString) {
- _configsvrConnString = std::move(connString);
-}
-
-bool ShardIdentityType::isShardNameSet() const {
- return _shardName.is_initialized();
-}
-
-const std::string& ShardIdentityType::getShardName() const {
- invariant(_shardName);
- return _shardName.get();
-}
-
-void ShardIdentityType::setShardName(std::string shardName) {
- _shardName = std::move(shardName);
-}
-
-bool ShardIdentityType::isClusterIdSet() const {
- return _clusterId.is_initialized();
-}
-
-const OID& ShardIdentityType::getClusterId() const {
- invariant(_clusterId);
- return _clusterId.get();
-}
-
-void ShardIdentityType::setClusterId(OID clusterId) {
- _clusterId = std::move(clusterId);
-}
-
BSONObj ShardIdentityType::createConfigServerUpdateObject(const std::string& newConnString) {
BSONObjBuilder builder;
BSONObjBuilder setConfigBuilder(builder.subobjStart("$set"));
- setConfigBuilder.append(configsvrConnString(), newConnString);
+ setConfigBuilder.append(ShardIdentity::kConfigsvrConnectionStringFieldName, newConnString);
setConfigBuilder.doneFast();
return builder.obj();
}
diff --git a/src/mongo/db/s/type_shard_identity.h b/src/mongo/db/s/type_shard_identity.h
index a46049dd287..b29a8c48a63 100644
--- a/src/mongo/db/s/type_shard_identity.h
+++ b/src/mongo/db/s/type_shard_identity.h
@@ -32,73 +32,48 @@
#include "mongo/client/connection_string.h"
#include "mongo/db/jsobj.h"
+#include "mongo/db/s/add_shard_cmd_gen.h"
namespace mongo {
/**
* Contains all the information needed to make a mongod instance shard aware.
*/
-class ShardIdentityType {
+class ShardIdentityType : public ShardIdentity {
public:
// The _id value for this document type.
static const std::string IdName;
- // Field names and types in a shardIdentity document.
- static const BSONField<std::string> configsvrConnString;
- static const BSONField<std::string> shardName;
- static const BSONField<OID> clusterId;
-
ShardIdentityType() = default;
+ ShardIdentityType(const ShardIdentity& sid) : ShardIdentity(sid) {}
/**
- * Constructs a new ShardIdentityType object from BSON.
- * Also does validation of the contents.
- */
- static StatusWith<ShardIdentityType> fromBSON(const BSONObj& source);
-
- /**
- * Returns OK if all fields have been set. Otherwise, returns NoSuchKey
- * and information about the first field that is missing.
+ * Constructs a new ShardIdentityType object from a BSON object containing
+ * a shard identity document. Also does validation of the contents.
*/
- Status validate() const;
+ static StatusWith<ShardIdentityType> fromShardIdentityDocument(const BSONObj& source);
/**
- * Returns the BSON representation of the entry.
+ * Returns the BSON representation of the entry as a shard identity document.
*/
- BSONObj toBSON() const;
+ BSONObj toShardIdentityDocument() const;
/**
* Returns a std::string representation of the current internal state.
*/
std::string toString() const;
- bool isConfigsvrConnStringSet() const;
- const ConnectionString& getConfigsvrConnString() const;
- void setConfigsvrConnString(ConnectionString connString);
-
- bool isShardNameSet() const;
- const std::string& getShardName() const;
- void setShardName(std::string shardName);
-
- bool isClusterIdSet() const;
- const OID& getClusterId() const;
- void setClusterId(OID clusterId);
+ /**
+ * Returns OK if all fields have been set. Otherwise, returns NoSuchKey
+ * and information about the first field that is missing.
+ */
+ Status validate() const;
/**
* Returns an update object that can be used to update the config server field of the
* shardIdentity document with the new connection string.
*/
static BSONObj createConfigServerUpdateObject(const std::string& newConnString);
-
-private:
- // Convention: (M)andatory, (O)ptional, (S)pecial rule.
-
- // (M) connection string to the config server.
- boost::optional<ConnectionString> _configsvrConnString;
- // (M) contains the name of the shard.
- boost::optional<std::string> _shardName;
- // (M) contains the (unique) identifier of the cluster.
- boost::optional<OID> _clusterId;
};
} // namespace mongo
diff --git a/src/mongo/db/s/type_shard_identity_test.cpp b/src/mongo/db/s/type_shard_identity_test.cpp
index cb9af748106..4fdb1e523de 100644
--- a/src/mongo/db/s/type_shard_identity_test.cpp
+++ b/src/mongo/db/s/type_shard_identity_test.cpp
@@ -43,25 +43,22 @@ TEST(ShardIdentityType, RoundTrip) {
auto clusterId(OID::gen());
auto doc = BSON("_id"
<< "shardIdentity"
- << "configsvrConnectionString"
- << "test/a:123"
<< "shardName"
<< "s1"
<< "clusterId"
- << clusterId);
+ << clusterId
+ << "configsvrConnectionString"
+ << "test/a:123");
- auto result = ShardIdentityType::fromBSON(doc);
+ auto result = ShardIdentityType::fromShardIdentityDocument(doc);
ASSERT_OK(result.getStatus());
auto shardIdentity = result.getValue();
- ASSERT_TRUE(shardIdentity.isConfigsvrConnStringSet());
- ASSERT_EQ("test/a:123", shardIdentity.getConfigsvrConnString().toString());
- ASSERT_TRUE(shardIdentity.isShardNameSet());
+ ASSERT_EQ("test/a:123", shardIdentity.getConfigsvrConnectionString().toString());
ASSERT_EQ("s1", shardIdentity.getShardName());
- ASSERT_TRUE(shardIdentity.isClusterIdSet());
ASSERT_EQ(clusterId, shardIdentity.getClusterId());
- ASSERT_BSONOBJ_EQ(doc, shardIdentity.toBSON());
+ ASSERT_BSONOBJ_EQ(doc, shardIdentity.toShardIdentityDocument());
}
TEST(ShardIdentityType, ParseMissingId) {
@@ -72,7 +69,7 @@ TEST(ShardIdentityType, ParseMissingId) {
<< "clusterId"
<< OID::gen());
- auto result = ShardIdentityType::fromBSON(doc);
+ auto result = ShardIdentityType::fromShardIdentityDocument(doc);
ASSERT_NOT_OK(result.getStatus());
}
@@ -84,7 +81,7 @@ TEST(ShardIdentityType, ParseMissingConfigsvrConnString) {
<< "clusterId"
<< OID::gen());
- auto result = ShardIdentityType::fromBSON(doc);
+ auto result = ShardIdentityType::fromShardIdentityDocument(doc);
ASSERT_NOT_OK(result.getStatus());
}
@@ -96,7 +93,7 @@ TEST(ShardIdentityType, ParseMissingShardName) {
<< "clusterId"
<< OID::gen());
- auto result = ShardIdentityType::fromBSON(doc);
+ auto result = ShardIdentityType::fromShardIdentityDocument(doc);
ASSERT_NOT_OK(result.getStatus());
}
@@ -108,7 +105,7 @@ TEST(ShardIdentityType, ParseMissingClusterId) {
<< "shardName"
<< "s1");
- auto result = ShardIdentityType::fromBSON(doc);
+ auto result = ShardIdentityType::fromShardIdentityDocument(doc);
ASSERT_NOT_OK(result.getStatus());
}
@@ -123,7 +120,8 @@ TEST(ShardIdentityType, InvalidConnectionString) {
<< "clusterId"
<< clusterId);
- ASSERT_EQ(ErrorCodes::FailedToParse, ShardIdentityType::fromBSON(doc).getStatus());
+ ASSERT_EQ(ErrorCodes::FailedToParse,
+ ShardIdentityType::fromShardIdentityDocument(doc).getStatus());
}
TEST(ShardIdentityType, NonReplSetConnectionString) {
@@ -137,7 +135,8 @@ TEST(ShardIdentityType, NonReplSetConnectionString) {
<< "clusterId"
<< clusterId);
- ASSERT_EQ(ErrorCodes::UnsupportedFormat, ShardIdentityType::fromBSON(doc).getStatus());
+ ASSERT_EQ(ErrorCodes::UnsupportedFormat,
+ ShardIdentityType::fromShardIdentityDocument(doc).getStatus());
}
TEST(ShardIdentityType, CreateUpdateObject) {
diff --git a/src/mongo/db/service_entry_point_common.cpp b/src/mongo/db/service_entry_point_common.cpp
index 55255971aec..3eb5136b472 100644
--- a/src/mongo/db/service_entry_point_common.cpp
+++ b/src/mongo/db/service_entry_point_common.cpp
@@ -68,6 +68,7 @@
#include "mongo/db/s/scoped_operation_completion_sharding_actions.h"
#include "mongo/db/s/shard_filtering_metadata_refresh.h"
#include "mongo/db/s/sharded_connection_info.h"
+#include "mongo/db/s/sharding_config_optime_gossip.h"
#include "mongo/db/s/sharding_state.h"
#include "mongo/db/service_entry_point_common.h"
#include "mongo/db/session_catalog.h"
@@ -863,7 +864,7 @@ void execCommandDatabase(OperationContext* opCtx,
}
// Handle config optime information that may have been sent along with the command.
- uassertStatusOK(shardingState->updateConfigServerOpTimeFromMetadata(opCtx));
+ rpc::advanceConfigOptimeFromRequestMetadata(opCtx);
}
oss.setAllowImplicitCollectionCreation(allowImplicitCollectionCreationField);
diff --git a/src/mongo/db/views/SConscript b/src/mongo/db/views/SConscript
index 672323fa39a..8024e68c649 100644
--- a/src/mongo/db/views/SConscript
+++ b/src/mongo/db/views/SConscript
@@ -8,12 +8,10 @@ env.Library(
target='views_mongod',
source=[
'durable_view_catalog.cpp',
- 'view_sharding_check.cpp',
],
LIBDEPS=[
'$BUILD_DIR/mongo/db/dbhelpers',
'$BUILD_DIR/mongo/db/views/views',
- '$BUILD_DIR/mongo/db/s/sharding',
],
)
diff --git a/src/mongo/db/views/view_sharding_check.cpp b/src/mongo/db/views/view_sharding_check.cpp
deleted file mode 100644
index 7c1ec2c5f86..00000000000
--- a/src/mongo/db/views/view_sharding_check.cpp
+++ /dev/null
@@ -1,79 +0,0 @@
-/**
- * Copyright (C) 2016 MongoDB Inc.
- *
- * This program is free software: you can redistribute it and/or modify
- * it under the terms of the GNU Affero General Public License, version 3,
- * as published by the Free Software Foundation.
- *
- * This program is distributed in the hope that it will be useful,
- * but WITHOUT ANY WARRANTY; without even the implied warranty of
- * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
- * GNU Affero General Public License for more details.
- *
- * You should have received a copy of the GNU Affero General Public License
- * along with this program. If not, see <http://www.gnu.org/licenses/>.
- *
- * As a special exception, the copyright holders give permission to link the
- * code of portions of this program with the OpenSSL library under certain
- * conditions as described in each individual source file and distribute
- * linked combinations including the program with the OpenSSL library. You
- * must comply with the GNU Affero General Public License in all respects for
- * all of the code used other than as permitted herein. If you modify file(s)
- * with this exception, you may extend this exception to your version of the
- * file(s), but you are not obligated to do so. If you do not wish to do so,
- * delete this exception statement from your version. If you delete this
- * exception statement from all source files in the program, then also delete
- * it in the license file.
- */
-
-#include "mongo/platform/basic.h"
-
-#include "mongo/db/views/view_sharding_check.h"
-
-#include "mongo/db/catalog/database.h"
-#include "mongo/db/commands.h"
-#include "mongo/db/db_raii.h"
-#include "mongo/db/operation_context.h"
-#include "mongo/db/repl/replication_coordinator.h"
-#include "mongo/db/s/collection_sharding_state.h"
-#include "mongo/db/s/sharding_state.h"
-#include "mongo/db/server_options.h"
-#include "mongo/db/views/view_catalog.h"
-#include "mongo/s/stale_exception.h"
-
-namespace mongo {
-
-void ViewShardingCheck::throwResolvedViewIfSharded(OperationContext* opCtx,
- Database* db,
- const ViewDefinition* view) {
- invariant(opCtx);
- invariant(db);
- invariant(view);
-
- if (ClusterRole::ShardServer != serverGlobalParams.clusterRole) {
- // This node is not part of a sharded cluster, so the collection cannot be sharded.
- return;
- }
-
- auto resolvedView = uassertStatusOK(db->getViewCatalog()->resolveView(opCtx, view->name()));
- const auto& sourceNss = resolvedView.getNamespace();
- const auto isPrimary =
- repl::ReplicationCoordinator::get(opCtx->getClient()->getServiceContext())
- ->canAcceptWritesForDatabase(opCtx, db->name());
-
- if (isPrimary && !collectionIsSharded(opCtx, sourceNss)) {
- return;
- }
-
- uasserted(std::move(resolvedView), "Command on view must be executed by mongos");
-}
-
-bool ViewShardingCheck::collectionIsSharded(OperationContext* opCtx, const NamespaceString& nss) {
- // The database is locked at this point but the collection underlying the given view is not
- // and must be for a sharding check.
- dassert(opCtx->lockState()->isDbLockedForMode(nss.db(), MODE_IS));
- AutoGetCollection autoGetCol(opCtx, nss, MODE_IS);
- return CollectionShardingState::get(opCtx, nss)->collectionIsSharded(opCtx);
-}
-
-} // namespace mongo