summaryrefslogtreecommitdiff
path: root/src
diff options
context:
space:
mode:
authorKaloian Manassiev <kaloian.manassiev@mongodb.com>2021-11-30 14:29:19 +0100
committerEvergreen Agent <no-reply@evergreen.mongodb.com>2021-12-01 18:36:32 +0000
commitf6cc0e8e02de04e79c4e80aba89418995b6b5f6a (patch)
tree2187522b295d28f17d2766bfe5acd0fddba1ee80 /src
parent4bf8144fb6f3aa0779900769c9b4288935da54e0 (diff)
downloadmongo-f6cc0e8e02de04e79c4e80aba89418995b6b5f6a.tar.gz
SERVER-61759 Cleanup and add comments to MigrationSourceManager
Diffstat (limited to 'src')
-rw-r--r--src/mongo/db/s/balancer/balancer.cpp1
-rw-r--r--src/mongo/db/s/config/configsvr_balancer_collection_status_command.cpp3
-rw-r--r--src/mongo/db/s/flush_database_cache_updates_command.cpp5
-rw-r--r--src/mongo/db/s/flush_routing_table_cache_updates_command.cpp1
-rw-r--r--src/mongo/db/s/migration_source_manager.cpp102
-rw-r--r--src/mongo/db/s/migration_source_manager.h48
-rw-r--r--src/mongo/db/s/set_shard_version_command.cpp1
-rw-r--r--src/mongo/db/s/shard_server_op_observer.cpp2
8 files changed, 70 insertions, 93 deletions
diff --git a/src/mongo/db/s/balancer/balancer.cpp b/src/mongo/db/s/balancer/balancer.cpp
index 63cefbbbbfb..6b75e4a3bbc 100644
--- a/src/mongo/db/s/balancer/balancer.cpp
+++ b/src/mongo/db/s/balancer/balancer.cpp
@@ -500,7 +500,6 @@ void Balancer::_mainThread() {
_actionStreamConsumerThread = stdx::thread([&] { _consumeActionStreamLoop(); });
-
LOGV2(6036606, "Balancer worker thread initialised. Entering main loop.");
// Main balancer loop
diff --git a/src/mongo/db/s/config/configsvr_balancer_collection_status_command.cpp b/src/mongo/db/s/config/configsvr_balancer_collection_status_command.cpp
index 40c4f38f1c2..7a3bcd2b724 100644
--- a/src/mongo/db/s/config/configsvr_balancer_collection_status_command.cpp
+++ b/src/mongo/db/s/config/configsvr_balancer_collection_status_command.cpp
@@ -41,9 +41,6 @@
#include "mongo/db/operation_context.h"
#include "mongo/db/repl/repl_client_info.h"
#include "mongo/db/s/balancer/balancer.h"
-#include "mongo/db/s/database_sharding_state.h"
-#include "mongo/db/s/migration_source_manager.h"
-#include "mongo/db/s/operation_sharding_state.h"
#include "mongo/db/s/shard_filtering_metadata_refresh.h"
#include "mongo/db/s/sharding_state.h"
#include "mongo/s/catalog_cache_loader.h"
diff --git a/src/mongo/db/s/flush_database_cache_updates_command.cpp b/src/mongo/db/s/flush_database_cache_updates_command.cpp
index 361bb6f1132..a80192e7dd3 100644
--- a/src/mongo/db/s/flush_database_cache_updates_command.cpp
+++ b/src/mongo/db/s/flush_database_cache_updates_command.cpp
@@ -41,17 +41,14 @@
#include "mongo/db/operation_context.h"
#include "mongo/db/repl/repl_client_info.h"
#include "mongo/db/s/database_sharding_state.h"
-#include "mongo/db/s/migration_source_manager.h"
#include "mongo/db/s/operation_sharding_state.h"
#include "mongo/db/s/shard_filtering_metadata_refresh.h"
#include "mongo/db/s/sharding_state.h"
+#include "mongo/logv2/log.h"
#include "mongo/s/catalog_cache_loader.h"
#include "mongo/s/grid.h"
#include "mongo/s/request_types/flush_database_cache_updates_gen.h"
-
-#include "mongo/logv2/log.h"
-
namespace mongo {
namespace {
diff --git a/src/mongo/db/s/flush_routing_table_cache_updates_command.cpp b/src/mongo/db/s/flush_routing_table_cache_updates_command.cpp
index 4ebd39a9ae3..06031c336a9 100644
--- a/src/mongo/db/s/flush_routing_table_cache_updates_command.cpp
+++ b/src/mongo/db/s/flush_routing_table_cache_updates_command.cpp
@@ -41,7 +41,6 @@
#include "mongo/db/operation_context.h"
#include "mongo/db/repl/repl_client_info.h"
#include "mongo/db/s/collection_sharding_runtime.h"
-#include "mongo/db/s/migration_source_manager.h"
#include "mongo/db/s/operation_sharding_state.h"
#include "mongo/db/s/shard_filtering_metadata_refresh.h"
#include "mongo/db/s/sharding_state.h"
diff --git a/src/mongo/db/s/migration_source_manager.cpp b/src/mongo/db/s/migration_source_manager.cpp
index cbf61928e9e..a2e085f500d 100644
--- a/src/mongo/db/s/migration_source_manager.cpp
+++ b/src/mongo/db/s/migration_source_manager.cpp
@@ -145,12 +145,12 @@ MigrationSourceManager::MigrationSourceManager(OperationContext* opCtx,
// Make sure the latest shard version is recovered as of the time of the invocation of the
// command.
- onShardVersionMismatch(_opCtx, getNss(), boost::none);
+ onShardVersionMismatch(_opCtx, _args.getNss(), boost::none);
// Snapshot the committed metadata from the time the migration starts
const auto [collectionMetadata, collectionUUID] = [&] {
UninterruptibleLockGuard noInterrupt(_opCtx->lockState());
- AutoGetCollection autoColl(_opCtx, getNss(), MODE_IS);
+ AutoGetCollection autoColl(_opCtx, _args.getNss(), MODE_IS);
uassert(ErrorCodes::InvalidOptions,
"cannot move chunks for a collection that doesn't exist",
autoColl.getCollection());
@@ -158,7 +158,7 @@ MigrationSourceManager::MigrationSourceManager(OperationContext* opCtx,
UUID collectionUUID = autoColl.getCollection()->uuid();
auto optMetadata =
- CollectionShardingRuntime::get(_opCtx, getNss())->getCurrentMetadataIfKnown();
+ CollectionShardingRuntime::get(_opCtx, _args.getNss())->getCurrentMetadataIfKnown();
uassert(ErrorCodes::ConflictingOperationInProgress,
"The collection's sharding state was cleared by a concurrent operation",
optMetadata);
@@ -211,20 +211,16 @@ MigrationSourceManager::~MigrationSourceManager() {
_stats.totalDonorMoveChunkTimeMillis.addAndFetch(_entireOpTimer.millis());
}
-NamespaceString MigrationSourceManager::getNss() const {
- return _args.getNss();
-}
-
Status MigrationSourceManager::startClone() {
invariant(!_opCtx->lockState()->isLocked());
invariant(_state == kCreated);
- ScopeGuard scopedGuard([&] { cleanupOnError(); });
+ ScopeGuard scopedGuard([&] { _cleanupOnError(); });
_stats.countDonorMoveChunkStarted.addAndFetch(1);
const Status logStatus = ShardingLogging::get(_opCtx)->logChangeChecked(
_opCtx,
"moveChunk.start",
- getNss().ns(),
+ _args.getNss().ns(),
BSON("min" << _args.getMinKey() << "max" << _args.getMaxKey() << "from"
<< _args.getFromShardId() << "to" << _args.getToShardId()),
ShardingCatalogClient::kMajorityWriteConcern);
@@ -248,25 +244,24 @@ Status MigrationSourceManager::startClone() {
_args, metadata.getKeyPattern(), _donorConnStr, _recipientHost);
AutoGetCollection autoColl(_opCtx,
- getNss(),
+ _args.getNss(),
replEnabled ? MODE_IX : MODE_X,
AutoGetCollectionViewMode::kViewsForbidden,
_opCtx->getServiceContext()->getPreciseClockSource()->now() +
Milliseconds(migrationLockAcquisitionMaxWaitMS.load()));
- auto csr = CollectionShardingRuntime::get(_opCtx, getNss());
+ auto csr = CollectionShardingRuntime::get(_opCtx, _args.getNss());
auto lockedCsr = CollectionShardingRuntime::CSRLock::lockExclusive(_opCtx, csr);
invariant(nullptr == std::exchange(msmForCsr(csr), this));
- _coordinator = std::make_unique<migrationutil::MigrationCoordinator>(
- _cloneDriver->getSessionId(),
- _args.getFromShardId(),
- _args.getToShardId(),
- getNss(),
- *_collectionUUID,
- ChunkRange(_args.getMinKey(), _args.getMaxKey()),
- _chunkVersion,
- _args.getWaitForDelete());
+ _coordinator.emplace(_cloneDriver->getSessionId(),
+ _args.getFromShardId(),
+ _args.getToShardId(),
+ _args.getNss(),
+ *_collectionUUID,
+ ChunkRange(_args.getMinKey(), _args.getMaxKey()),
+ *_chunkVersion,
+ _args.getWaitForDelete());
_state = kCloning;
}
@@ -301,7 +296,7 @@ Status MigrationSourceManager::startClone() {
Status MigrationSourceManager::awaitToCatchUp() {
invariant(!_opCtx->lockState()->isLocked());
invariant(_state == kCloning);
- ScopeGuard scopedGuard([&] { cleanupOnError(); });
+ ScopeGuard scopedGuard([&] { _cleanupOnError(); });
_stats.totalDonorChunkCloneTimeMillis.addAndFetch(_cloneAndCommitTimer.millis());
_cloneAndCommitTimer.reset();
@@ -320,7 +315,7 @@ Status MigrationSourceManager::awaitToCatchUp() {
Status MigrationSourceManager::enterCriticalSection() {
invariant(!_opCtx->lockState()->isLocked());
invariant(_state == kCloneCaughtUp);
- ScopeGuard scopedGuard([&] { cleanupOnError(); });
+ ScopeGuard scopedGuard([&] { _cleanupOnError(); });
_stats.totalDonorChunkCloneTimeMillis.addAndFetch(_cloneAndCommitTimer.millis());
_cloneAndCommitTimer.reset();
@@ -354,7 +349,7 @@ Status MigrationSourceManager::enterCriticalSection() {
// will stall behind the flag.
Status signalStatus = shardmetadatautil::updateShardCollectionsEntry(
_opCtx,
- BSON(ShardCollectionType::kNssFieldName << getNss().ns()),
+ BSON(ShardCollectionType::kNssFieldName << _args.getNss().ns()),
BSON("$inc" << BSON(ShardCollectionType::kEnterCriticalSectionCounterFieldName << 1)),
false /*upsert*/);
if (!signalStatus.isOK()) {
@@ -375,7 +370,7 @@ Status MigrationSourceManager::enterCriticalSection() {
Status MigrationSourceManager::commitChunkOnRecipient() {
invariant(!_opCtx->lockState()->isLocked());
invariant(_state == kCriticalSection);
- ScopeGuard scopedGuard([&] { cleanupOnError(); });
+ ScopeGuard scopedGuard([&] { _cleanupOnError(); });
// Tell the recipient shard to fetch the latest changes.
auto commitCloneStatus = _cloneDriver->commitClone(_opCtx, _acquireCSOnRecipient);
@@ -399,7 +394,7 @@ Status MigrationSourceManager::commitChunkOnRecipient() {
Status MigrationSourceManager::commitChunkMetadataOnConfig() {
invariant(!_opCtx->lockState()->isLocked());
invariant(_state == kCloneCompleted);
- ScopeGuard scopedGuard([&] { cleanupOnError(); });
+ ScopeGuard scopedGuard([&] { _cleanupOnError(); });
// If we have chunks left on the FROM shard, bump the version of one of them as well. This will
// change the local collection major version, which indicates to other processes that the chunk
@@ -412,11 +407,11 @@ Status MigrationSourceManager::commitChunkMetadataOnConfig() {
ChunkType migratedChunkType;
migratedChunkType.setMin(_args.getMinKey());
migratedChunkType.setMax(_args.getMaxKey());
- migratedChunkType.setVersion(_chunkVersion);
+ migratedChunkType.setVersion(*_chunkVersion);
const auto currentTime = VectorClock::get(_opCtx)->getTime();
CommitChunkMigrationRequest::appendAsCommand(&builder,
- getNss(),
+ _args.getNss(),
_args.getFromShardId(),
_args.getToShardId(),
migratedChunkType,
@@ -458,13 +453,13 @@ Status MigrationSourceManager::commitChunkMetadataOnConfig() {
if (!migrationCommitStatus.isOK()) {
{
UninterruptibleLockGuard noInterrupt(_opCtx->lockState());
- AutoGetCollection autoColl(_opCtx, getNss(), MODE_IX);
- CollectionShardingRuntime::get(_opCtx, getNss())->clearFilteringMetadata(_opCtx);
+ AutoGetCollection autoColl(_opCtx, _args.getNss(), MODE_IX);
+ CollectionShardingRuntime::get(_opCtx, _args.getNss())->clearFilteringMetadata(_opCtx);
}
scopedGuard.dismiss();
_cleanup(false);
// Best-effort recover of the shard version.
- onShardVersionMismatchNoExcept(_opCtx, getNss(), boost::none).ignore();
+ onShardVersionMismatchNoExcept(_opCtx, _args.getNss(), boost::none).ignore();
return migrationCommitStatus;
}
@@ -477,7 +472,7 @@ Status MigrationSourceManager::commitChunkMetadataOnConfig() {
"Starting post-migration commit refresh on the shard",
"migrationId"_attr = _coordinator->getMigrationId());
- forceShardFilteringMetadataRefresh(_opCtx, getNss());
+ forceShardFilteringMetadataRefresh(_opCtx, _args.getNss());
LOGV2_DEBUG_OPTIONS(4817405,
2,
@@ -493,13 +488,13 @@ Status MigrationSourceManager::commitChunkMetadataOnConfig() {
"error"_attr = redact(ex));
{
UninterruptibleLockGuard noInterrupt(_opCtx->lockState());
- AutoGetCollection autoColl(_opCtx, getNss(), MODE_IX);
- CollectionShardingRuntime::get(_opCtx, getNss())->clearFilteringMetadata(_opCtx);
+ AutoGetCollection autoColl(_opCtx, _args.getNss(), MODE_IX);
+ CollectionShardingRuntime::get(_opCtx, _args.getNss())->clearFilteringMetadata(_opCtx);
}
scopedGuard.dismiss();
_cleanup(false);
// Best-effort recover of the shard version.
- onShardVersionMismatchNoExcept(_opCtx, getNss(), boost::none).ignore();
+ onShardVersionMismatchNoExcept(_opCtx, _args.getNss(), boost::none).ignore();
return ex.toStatus();
}
@@ -515,9 +510,9 @@ Status MigrationSourceManager::commitChunkMetadataOnConfig() {
// If the migration has succeeded, clear the BucketCatalog so that the buckets that got migrated
// out are no longer updatable.
- if (getNss().isTimeseriesBucketsCollection()) {
+ if (_args.getNss().isTimeseriesBucketsCollection()) {
auto& bucketCatalog = BucketCatalog::get(_opCtx);
- bucketCatalog.clear(getNss().getTimeseriesViewNamespace());
+ bucketCatalog.clear(_args.getNss().getTimeseriesViewNamespace());
}
_coordinator->setMigrationDecision(DecisionEnum::kCommitted);
@@ -535,7 +530,7 @@ Status MigrationSourceManager::commitChunkMetadataOnConfig() {
ShardingLogging::get(_opCtx)->logChange(
_opCtx,
"moveChunk.commit",
- getNss().ns(),
+ _args.getNss().ns(),
BSON("min" << _args.getMinKey() << "max" << _args.getMaxKey() << "from"
<< _args.getFromShardId() << "to" << _args.getToShardId() << "counts"
<< _recipientCloneCounts),
@@ -547,11 +542,11 @@ Status MigrationSourceManager::commitChunkMetadataOnConfig() {
// Best-effort make the recipient refresh its routing table to the new collection
// version.
refreshRecipientRoutingTable(
- _opCtx, getNss(), _recipientHost, refreshedMetadata.getCollVersion());
+ _opCtx, _args.getNss(), _recipientHost, refreshedMetadata.getCollVersion());
}
std::string orphanedRangeCleanUpErrMsg = str::stream()
- << "Moved chunks successfully but failed to clean up " << getNss().ns() << " range "
+ << "Moved chunks successfully but failed to clean up " << _args.getNss() << " range "
<< redact(range.toString()) << " due to: ";
if (_args.getWaitForDelete()) {
@@ -559,7 +554,7 @@ Status MigrationSourceManager::commitChunkMetadataOnConfig() {
"Waiting for migration cleanup after chunk commit for the namespace {namespace} "
"and range {range}",
"Waiting for migration cleanup after chunk commit",
- "namespace"_attr = getNss().ns(),
+ "namespace"_attr = _args.getNss(),
"range"_attr = redact(range.toString()),
"migrationId"_attr = _coordinator->getMigrationId());
@@ -577,7 +572,7 @@ Status MigrationSourceManager::commitChunkMetadataOnConfig() {
return Status::OK();
}
-void MigrationSourceManager::cleanupOnError() {
+void MigrationSourceManager::_cleanupOnError() {
if (_state == kDone) {
return;
}
@@ -585,7 +580,7 @@ void MigrationSourceManager::cleanupOnError() {
ShardingLogging::get(_opCtx)->logChange(
_opCtx,
"moveChunk.error",
- getNss().ns(),
+ _args.getNss().ns(),
BSON("min" << _args.getMinKey() << "max" << _args.getMaxKey() << "from"
<< _args.getFromShardId() << "to" << _args.getToShardId()),
ShardingCatalogClient::kMajorityWriteConcern);
@@ -594,11 +589,6 @@ void MigrationSourceManager::cleanupOnError() {
}
void MigrationSourceManager::abortDueToConflictingIndexOperation(OperationContext* opCtx) {
- // Index operations sent in the 4.4 protocol from internal clients are versioned and block
- // behind both phases of the critical section, so there should never be an active critical
- // section in this case.
- dassert(!_critSec || !opCtx->getClient()->session() ||
- !(opCtx->getClient()->session()->getTags() & transport::Session::kInternalClient));
stdx::lock_guard<Client> lk(*_opCtx->getClient());
_opCtx->markKilled();
_stats.countDonorMoveChunkAbortConflictingIndexOperation.addAndFetch(1);
@@ -607,8 +597,8 @@ void MigrationSourceManager::abortDueToConflictingIndexOperation(OperationContex
CollectionMetadata MigrationSourceManager::_getCurrentMetadataAndCheckEpoch() {
auto metadata = [&] {
UninterruptibleLockGuard noInterrupt(_opCtx->lockState());
- AutoGetCollection autoColl(_opCtx, getNss(), MODE_IS);
- auto* const css = CollectionShardingRuntime::get(_opCtx, getNss());
+ AutoGetCollection autoColl(_opCtx, _args.getNss(), MODE_IS);
+ auto* const css = CollectionShardingRuntime::get(_opCtx, _args.getNss());
const auto optMetadata = css->getCurrentMetadataIfKnown();
uassert(ErrorCodes::ConflictingOperationInProgress,
@@ -652,7 +642,7 @@ void MigrationSourceManager::_notifyChangeStreamsOnRecipientFirstChunk(
_opCtx, "migrateChunkToNewShard", NamespaceString::kRsOplogNamespace.ns(), [&] {
WriteUnitOfWork uow(_opCtx);
serviceContext->getOpObserver()->onInternalOpMessage(_opCtx,
- getNss(),
+ _args.getNss(),
*_collectionUUID,
BSON("msg" << dbgMessage),
o2Message,
@@ -670,8 +660,8 @@ void MigrationSourceManager::_cleanup(bool completeMigration) noexcept {
auto cloneDriver = [&]() {
// Unregister from the collection's sharding state and exit the migration critical section.
UninterruptibleLockGuard noInterrupt(_opCtx->lockState());
- AutoGetCollection autoColl(_opCtx, getNss(), MODE_IX);
- auto* const csr = CollectionShardingRuntime::get(_opCtx, getNss());
+ AutoGetCollection autoColl(_opCtx, _args.getNss(), MODE_IX);
+ auto* const csr = CollectionShardingRuntime::get(_opCtx, _args.getNss());
auto csrLock = CollectionShardingRuntime::CSRLock::lockExclusive(_opCtx, csr);
if (_state != kCreated) {
@@ -736,7 +726,7 @@ void MigrationSourceManager::_cleanup(bool completeMigration) noexcept {
// is possible that the persisted metadata is rolled back after step down, but the
// write which cleared the 'inMigration' flag is not, a secondary node will report
// itself at an older shard version.
- CatalogCacheLoader::get(newOpCtx).waitForCollectionFlush(newOpCtx, getNss());
+ CatalogCacheLoader::get(newOpCtx).waitForCollectionFlush(newOpCtx, _args.getNss());
// Clear the 'minOpTime recovery' document so that the next time a node from this
// shard becomes a primary, it won't have to recover the config server optime.
@@ -763,13 +753,13 @@ void MigrationSourceManager::_cleanup(bool completeMigration) noexcept {
// Something went really wrong when completing the migration just unset the metadata and let
// the next op to recover.
UninterruptibleLockGuard noInterrupt(_opCtx->lockState());
- AutoGetCollection autoColl(_opCtx, getNss(), MODE_IX);
- CollectionShardingRuntime::get(_opCtx, getNss())->clearFilteringMetadata(_opCtx);
+ AutoGetCollection autoColl(_opCtx, _args.getNss(), MODE_IX);
+ CollectionShardingRuntime::get(_opCtx, _args.getNss())->clearFilteringMetadata(_opCtx);
}
}
BSONObj MigrationSourceManager::getMigrationStatusReport() const {
- return migrationutil::makeMigrationStatusDocument(getNss(),
+ return migrationutil::makeMigrationStatusDocument(_args.getNss(),
_args.getFromShardId(),
_args.getToShardId(),
true,
diff --git a/src/mongo/db/s/migration_source_manager.h b/src/mongo/db/s/migration_source_manager.h
index 0b09335a172..937a636d8c7 100644
--- a/src/mongo/db/s/migration_source_manager.h
+++ b/src/mongo/db/s/migration_source_manager.h
@@ -39,7 +39,6 @@
namespace mongo {
-class OperationContext;
struct ShardingStatistics;
/**
@@ -98,11 +97,6 @@ public:
~MigrationSourceManager();
/**
- * Returns the namespace for which this source manager is active.
- */
- NamespaceString getNss() const;
-
- /**
* Contacts the donor shard and tells it to start cloning the specified chunk. This method will
* fail if for any reason the donor shard fails to initiate the cloning sequence.
*
@@ -155,16 +149,6 @@ public:
Status commitChunkMetadataOnConfig();
/**
- * May be called at any time. Unregisters the migration source manager from the collection,
- * restores the committed metadata (if in critical section) and logs error in the change log to
- * indicate that the migration has failed.
- *
- * Expected state: Any
- * Resulting state: kDone
- */
- void cleanupOnError();
-
- /**
* Aborts the migration after observing a concurrent index operation by marking its operation
* context as killed.
*/
@@ -216,6 +200,16 @@ private:
*/
void _cleanup(bool completeMigration) noexcept;
+ /**
+ * May be called at any time. Unregisters the migration source manager from the collection,
+ * restores the committed metadata (if in critical section) and logs error in the change log to
+ * indicate that the migration has failed.
+ *
+ * Expected state: Any
+ * Resulting state: kDone
+ */
+ void _cleanupOnError();
+
// This is the opCtx of the moveChunk request that constructed the MigrationSourceManager.
// The caller must guarantee it outlives the MigrationSourceManager.
OperationContext* const _opCtx;
@@ -232,6 +226,12 @@ private:
// Stores a reference to the process sharding statistics object which needs to be updated
ShardingStatistics& _stats;
+ // Information about the moveChunk to be used in the critical section.
+ const BSONObj _critSecReason;
+
+ // It states whether the critical section has to be acquired on the recipient.
+ const bool _acquireCSOnRecipient;
+
// Times the entire moveChunk operation
const Timer _entireOpTimer;
@@ -249,11 +249,11 @@ private:
boost::optional<UUID> _collectionUUID;
// The version of the chunk at the time the migration started.
- ChunkVersion _chunkVersion;
+ boost::optional<ChunkVersion> _chunkVersion;
// Contains logic for ensuring the donor's and recipient's config.rangeDeletions entries are
// correctly updated based on whether the migration committed or aborted.
- std::unique_ptr<migrationutil::MigrationCoordinator> _coordinator;
+ boost::optional<migrationutil::MigrationCoordinator> _coordinator;
// The chunk cloner source. Only available if there is an active migration going on. To set and
// remove it, a collection lock and the CSRLock need to be acquired first in order to block all
@@ -261,21 +261,17 @@ private:
// cloning stage has completed.
std::shared_ptr<MigrationChunkClonerSource> _cloneDriver;
+ // Holds the in-memory critical section for the collection. Only set when migration has reached
+ // the critical section phase.
+ boost::optional<CollectionCriticalSection> _critSec;
+
// The statistics about a chunk migration to be included in moveChunk.commit
BSONObj _recipientCloneCounts;
- boost::optional<CollectionCriticalSection> _critSec;
-
// Optional future that is populated if the migration succeeds and range deletion is scheduled
// on this node. The future is set when the range deletion completes. Used if the moveChunk was
// sent with waitForDelete.
boost::optional<SemiFuture<void>> _cleanupCompleteFuture;
-
- // Information about the moveChunk to be used in the critical section.
- const BSONObj _critSecReason;
-
- // It states whether the critical section has to be acquired on the recipient.
- const bool _acquireCSOnRecipient;
};
} // 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 9fa4fd152a0..ac91b524d77 100644
--- a/src/mongo/db/s/set_shard_version_command.cpp
+++ b/src/mongo/db/s/set_shard_version_command.cpp
@@ -42,7 +42,6 @@
#include "mongo/db/operation_context.h"
#include "mongo/db/repl/replication_coordinator.h"
#include "mongo/db/s/collection_sharding_runtime.h"
-#include "mongo/db/s/migration_source_manager.h"
#include "mongo/db/s/shard_filtering_metadata_refresh.h"
#include "mongo/db/s/sharding_state.h"
#include "mongo/db/views/view_catalog.h"
diff --git a/src/mongo/db/s/shard_server_op_observer.cpp b/src/mongo/db/s/shard_server_op_observer.cpp
index 67df49364dc..a81879f8058 100644
--- a/src/mongo/db/s/shard_server_op_observer.cpp
+++ b/src/mongo/db/s/shard_server_op_observer.cpp
@@ -606,7 +606,7 @@ void ShardServerOpObserver::onStartIndexBuild(OperationContext* opCtx,
const std::vector<BSONObj>& indexes,
bool fromMigrate) {
abortOngoingMigrationIfNeeded(opCtx, nss);
-};
+}
void ShardServerOpObserver::onStartIndexBuildSinglePhase(OperationContext* opCtx,
const NamespaceString& nss) {