summaryrefslogtreecommitdiff
path: root/src
diff options
context:
space:
mode:
authorKaloian Manassiev <kaloian.manassiev@mongodb.com>2019-05-10 11:30:21 -0400
committerKaloian Manassiev <kaloian.manassiev@mongodb.com>2019-05-14 07:34:39 -0400
commit3cbd136094249209a212aab06696c2ef0db41b26 (patch)
treee6ec3b0ec58af74c9d7b6e13ebd4920999103554 /src
parent4686f07b4b51cb34156275c44fc1138b950f56dc (diff)
downloadmongo-3cbd136094249209a212aab06696c2ef0db41b26.tar.gz
SERVER-37235 Use an onCommit/Abort handler to complete session kill on direct writes to `config.transactions`
Diffstat (limited to 'src')
-rw-r--r--src/mongo/db/auth/auth_op_observer_test.cpp1
-rw-r--r--src/mongo/db/key_generator_update_test.cpp2
-rw-r--r--src/mongo/db/keys_collection_cache_test.cpp2
-rw-r--r--src/mongo/db/keys_collection_manager_sharding_test.cpp2
-rw-r--r--src/mongo/db/op_observer_impl.cpp11
-rw-r--r--src/mongo/db/op_observer_impl_test.cpp1
-rw-r--r--src/mongo/db/pipeline/mongos_process_interface.cpp1
-rw-r--r--src/mongo/db/repl/rollback_impl.cpp4
-rw-r--r--src/mongo/db/repl/rs_rollback.cpp2
-rw-r--r--src/mongo/db/s/config/initial_split_policy_test.cpp1
-rw-r--r--src/mongo/db/s/config/sharding_catalog_manager_enable_sharding_test.cpp2
-rw-r--r--src/mongo/db/s/config/sharding_catalog_manager_merge_chunks_test.cpp4
-rw-r--r--src/mongo/db/s/config/sharding_catalog_manager_split_chunk_test.cpp4
-rw-r--r--src/mongo/db/session_catalog_mongod.cpp101
-rw-r--r--src/mongo/db/session_catalog_mongod.h23
-rw-r--r--src/mongo/db/transaction_participant.cpp8
-rw-r--r--src/mongo/db/transaction_participant.h36
-rw-r--r--src/mongo/s/session_catalog_router.cpp2
-rw-r--r--src/mongo/s/session_catalog_router.h2
-rw-r--r--src/mongo/s/transaction_router.cpp1
-rw-r--r--src/mongo/s/transaction_router.h1
21 files changed, 122 insertions, 89 deletions
diff --git a/src/mongo/db/auth/auth_op_observer_test.cpp b/src/mongo/db/auth/auth_op_observer_test.cpp
index 63cb7ed42dc..1168ae1836d 100644
--- a/src/mongo/db/auth/auth_op_observer_test.cpp
+++ b/src/mongo/db/auth/auth_op_observer_test.cpp
@@ -48,7 +48,6 @@
#include "mongo/db/session_catalog_mongod.h"
#include "mongo/db/storage/ephemeral_for_test/ephemeral_for_test_recovery_unit.h"
#include "mongo/db/transaction_participant.h"
-#include "mongo/s/config_server_test_fixture.h"
#include "mongo/unittest/death_test.h"
#include "mongo/util/clock_source_mock.h"
diff --git a/src/mongo/db/key_generator_update_test.cpp b/src/mongo/db/key_generator_update_test.cpp
index 2d967efda48..b9cafda3856 100644
--- a/src/mongo/db/key_generator_update_test.cpp
+++ b/src/mongo/db/key_generator_update_test.cpp
@@ -45,6 +45,7 @@
#include "mongo/util/fail_point_service.h"
namespace mongo {
+namespace {
class KeyGeneratorUpdateTest : public ConfigServerTestFixture {
protected:
@@ -416,4 +417,5 @@ TEST_F(KeyGeneratorUpdateTest, ShouldNotCreateKeysWithDisableKeyGenerationFailPo
ASSERT_EQ(0U, allKeys.size());
}
+} // namespace
} // namespace mongo
diff --git a/src/mongo/db/keys_collection_cache_test.cpp b/src/mongo/db/keys_collection_cache_test.cpp
index e68422b8c39..223aa2603a4 100644
--- a/src/mongo/db/keys_collection_cache_test.cpp
+++ b/src/mongo/db/keys_collection_cache_test.cpp
@@ -41,6 +41,7 @@
#include "mongo/util/clock_source_mock.h"
namespace mongo {
+namespace {
class CacheTest : public ConfigServerTestFixture {
protected:
@@ -267,4 +268,5 @@ TEST_F(CacheTest, RefreshCanIncrementallyGetNewKeys) {
}
}
+} // namespace
} // namespace mongo
diff --git a/src/mongo/db/keys_collection_manager_sharding_test.cpp b/src/mongo/db/keys_collection_manager_sharding_test.cpp
index 649d17be78b..f495d33e92a 100644
--- a/src/mongo/db/keys_collection_manager_sharding_test.cpp
+++ b/src/mongo/db/keys_collection_manager_sharding_test.cpp
@@ -45,6 +45,7 @@
#include "mongo/util/fail_point_service.h"
namespace mongo {
+namespace {
class KeysManagerShardedTest : public ConfigServerTestFixture {
public:
@@ -370,4 +371,5 @@ TEST_F(KeysManagerShardedTest, HasSeenKeysIsFalseUntilKeysAreFound) {
ASSERT_EQ(true, keyManager()->hasSeenKeys());
}
+} // namespace
} // namespace mongo
diff --git a/src/mongo/db/op_observer_impl.cpp b/src/mongo/db/op_observer_impl.cpp
index 0965db6074d..7c9d168c378 100644
--- a/src/mongo/db/op_observer_impl.cpp
+++ b/src/mongo/db/op_observer_impl.cpp
@@ -555,7 +555,7 @@ void OpObserverImpl::onInserts(OperationContext* opCtx,
}
} else if (nss == NamespaceString::kSessionTransactionsTableNamespace && !lastOpTime.isNull()) {
for (auto it = first; it != last; it++) {
- MongoDSessionCatalog::invalidateSessions(opCtx, it->doc);
+ MongoDSessionCatalog::observeDirectWriteToConfigTransactions(opCtx, it->doc);
}
}
}
@@ -622,7 +622,8 @@ void OpObserverImpl::onUpdate(OperationContext* opCtx, const OplogUpdateEntryArg
FeatureCompatibilityVersion::onInsertOrUpdate(opCtx, args.updateArgs.updatedDoc);
} else if (args.nss == NamespaceString::kSessionTransactionsTableNamespace &&
!opTime.writeOpTime.isNull()) {
- MongoDSessionCatalog::invalidateSessions(opCtx, args.updateArgs.updatedDoc);
+ MongoDSessionCatalog::observeDirectWriteToConfigTransactions(opCtx,
+ args.updateArgs.updatedDoc);
}
}
@@ -685,7 +686,7 @@ void OpObserverImpl::onDelete(OperationContext* opCtx,
uasserted(40670, "removing FeatureCompatibilityVersion document is not allowed");
} else if (nss == NamespaceString::kSessionTransactionsTableNamespace &&
!opTime.writeOpTime.isNull()) {
- MongoDSessionCatalog::invalidateSessions(opCtx, documentKey);
+ MongoDSessionCatalog::observeDirectWriteToConfigTransactions(opCtx, documentKey);
}
}
@@ -819,7 +820,7 @@ void OpObserverImpl::onDropDatabase(OperationContext* opCtx, const std::string&
50714, "dropping the admin database is not allowed.", dbName != NamespaceString::kAdminDb);
if (dbName == NamespaceString::kSessionTransactionsTableNamespace.db()) {
- MongoDSessionCatalog::invalidateSessions(opCtx, boost::none);
+ MongoDSessionCatalog::invalidateAllSessions(opCtx);
}
}
@@ -857,7 +858,7 @@ repl::OpTime OpObserverImpl::onDropCollection(OperationContext* opCtx,
if (collectionName.coll() == DurableViewCatalog::viewsCollectionName()) {
DurableViewCatalog::onExternalChange(opCtx, collectionName);
} else if (collectionName == NamespaceString::kSessionTransactionsTableNamespace) {
- MongoDSessionCatalog::invalidateSessions(opCtx, boost::none);
+ MongoDSessionCatalog::invalidateAllSessions(opCtx);
}
return {};
diff --git a/src/mongo/db/op_observer_impl_test.cpp b/src/mongo/db/op_observer_impl_test.cpp
index c660fcd30bf..d99fbda821b 100644
--- a/src/mongo/db/op_observer_impl_test.cpp
+++ b/src/mongo/db/op_observer_impl_test.cpp
@@ -48,7 +48,6 @@
#include "mongo/db/storage/ephemeral_for_test/ephemeral_for_test_recovery_unit.h"
#include "mongo/db/transaction_participant.h"
#include "mongo/db/transaction_participant_gen.h"
-#include "mongo/s/config_server_test_fixture.h"
#include "mongo/unittest/death_test.h"
#include "mongo/util/clock_source_mock.h"
diff --git a/src/mongo/db/pipeline/mongos_process_interface.cpp b/src/mongo/db/pipeline/mongos_process_interface.cpp
index d8452f4f7eb..5072c389bc0 100644
--- a/src/mongo/db/pipeline/mongos_process_interface.cpp
+++ b/src/mongo/db/pipeline/mongos_process_interface.cpp
@@ -48,7 +48,6 @@
#include "mongo/s/query/cluster_cursor_manager.h"
#include "mongo/s/query/establish_cursors.h"
#include "mongo/s/query/router_exec_stage.h"
-#include "mongo/s/transaction_router.h"
#include "mongo/util/fail_point.h"
namespace mongo {
diff --git a/src/mongo/db/repl/rollback_impl.cpp b/src/mongo/db/repl/rollback_impl.cpp
index 647988123a1..09cf0345d6d 100644
--- a/src/mongo/db/repl/rollback_impl.cpp
+++ b/src/mongo/db/repl/rollback_impl.cpp
@@ -445,8 +445,8 @@ Status RollbackImpl::_runRollbackCriticalSection(
// If there were rolled back operations on any session, invalidate all sessions.
// We invalidate sessions before we recover so that we avoid invalidating sessions that had
// just recovered prepared transactions.
- if (_observerInfo.rollbackSessionIds.size() > 0) {
- MongoDSessionCatalog::invalidateSessions(opCtx, boost::none);
+ if (!_observerInfo.rollbackSessionIds.empty()) {
+ MongoDSessionCatalog::invalidateAllSessions(opCtx);
}
// Recover to the stable timestamp.
diff --git a/src/mongo/db/repl/rs_rollback.cpp b/src/mongo/db/repl/rs_rollback.cpp
index 44c40e4f40e..e0063ec6ead 100644
--- a/src/mongo/db/repl/rs_rollback.cpp
+++ b/src/mongo/db/repl/rs_rollback.cpp
@@ -1570,7 +1570,7 @@ void rollback_internal::syncFixUp(OperationContext* opCtx,
// If necessary, clear the memory of existing sessions.
if (fixUpInfo.refetchTransactionDocs) {
- MongoDSessionCatalog::invalidateSessions(opCtx, boost::none);
+ MongoDSessionCatalog::invalidateAllSessions(opCtx);
}
if (auto validator = LogicalTimeValidator::get(opCtx)) {
diff --git a/src/mongo/db/s/config/initial_split_policy_test.cpp b/src/mongo/db/s/config/initial_split_policy_test.cpp
index e5eb9cef1fa..45447ecd729 100644
--- a/src/mongo/db/s/config/initial_split_policy_test.cpp
+++ b/src/mongo/db/s/config/initial_split_policy_test.cpp
@@ -34,7 +34,6 @@
#include "mongo/db/s/config/initial_split_policy.h"
#include "mongo/s/catalog/type_shard.h"
#include "mongo/s/catalog/type_tags.h"
-#include "mongo/s/config_server_test_fixture.h"
#include "mongo/unittest/unittest.h"
#include "mongo/util/log.h"
diff --git a/src/mongo/db/s/config/sharding_catalog_manager_enable_sharding_test.cpp b/src/mongo/db/s/config/sharding_catalog_manager_enable_sharding_test.cpp
index 24fb6a641e6..2dbc596b801 100644
--- a/src/mongo/db/s/config/sharding_catalog_manager_enable_sharding_test.cpp
+++ b/src/mongo/db/s/config/sharding_catalog_manager_enable_sharding_test.cpp
@@ -63,7 +63,7 @@ namespace {
using executor::RemoteCommandRequest;
using std::vector;
-class EnableShardingTest : public ConfigServerTestFixture {};
+using EnableShardingTest = ConfigServerTestFixture;
TEST_F(EnableShardingTest, noDBExists) {
ShardType shard;
diff --git a/src/mongo/db/s/config/sharding_catalog_manager_merge_chunks_test.cpp b/src/mongo/db/s/config/sharding_catalog_manager_merge_chunks_test.cpp
index 1970c263a60..b403eca6f7a 100644
--- a/src/mongo/db/s/config/sharding_catalog_manager_merge_chunks_test.cpp
+++ b/src/mongo/db/s/config/sharding_catalog_manager_merge_chunks_test.cpp
@@ -39,10 +39,10 @@
namespace mongo {
namespace {
-using MergeChunkTest = ConfigServerTestFixture;
-
const NamespaceString kNamespace("TestDB.TestColl");
+using MergeChunkTest = ConfigServerTestFixture;
+
TEST_F(MergeChunkTest, MergeExistingChunksCorrectlyShouldSucceed) {
ChunkType chunk;
chunk.setNS(kNamespace);
diff --git a/src/mongo/db/s/config/sharding_catalog_manager_split_chunk_test.cpp b/src/mongo/db/s/config/sharding_catalog_manager_split_chunk_test.cpp
index 5d026d949dd..63095d20d45 100644
--- a/src/mongo/db/s/config/sharding_catalog_manager_split_chunk_test.cpp
+++ b/src/mongo/db/s/config/sharding_catalog_manager_split_chunk_test.cpp
@@ -38,10 +38,10 @@
namespace mongo {
namespace {
-using SplitChunkTest = ConfigServerTestFixture;
-
const NamespaceString kNamespace("TestDB", "TestColl");
+using SplitChunkTest = ConfigServerTestFixture;
+
TEST_F(SplitChunkTest, SplitExistingChunkCorrectlyShouldSucceed) {
ChunkType chunk;
chunk.setNS(kNamespace);
diff --git a/src/mongo/db/session_catalog_mongod.cpp b/src/mongo/db/session_catalog_mongod.cpp
index 89508662baa..05bd53eb01c 100644
--- a/src/mongo/db/session_catalog_mongod.cpp
+++ b/src/mongo/db/session_catalog_mongod.cpp
@@ -81,6 +81,10 @@ auto getThreadPool(OperationContext* opCtx) {
return &sessionTasksExecutor(opCtx->getServiceContext()).threadPool;
}
+/**
+ * Non-blocking call, which schedules asynchronously the work to finish cleaning up the specified
+ * set of kill tokens.
+ */
void killSessionTokens(OperationContext* opCtx,
std::vector<SessionCatalog::KillToken> sessionKillTokens) {
if (sessionKillTokens.empty())
@@ -104,6 +108,18 @@ void killSessionTokens(OperationContext* opCtx,
});
}
+void disallowDirectWritesUnderSession(OperationContext* opCtx) {
+ const auto replCoord = repl::ReplicationCoordinator::get(opCtx);
+ bool isReplSet = replCoord->getReplicationMode() == repl::ReplicationCoordinator::modeReplSet;
+ if (isReplSet) {
+ uassert(40528,
+ str::stream() << "Direct writes against "
+ << NamespaceString::kSessionTransactionsTableNamespace
+ << " cannot be performed using a transaction or on a session.",
+ !opCtx->getLogicalSessionId());
+ }
+}
+
const auto kIdProjection = BSON(SessionTxnRecord::kSessionIdFieldName << 1);
const auto kSortById = BSON(SessionTxnRecord::kSessionIdFieldName << 1);
const auto kLastWriteDateFieldName = SessionTxnRecord::kLastWriteDateFieldName;
@@ -241,41 +257,60 @@ boost::optional<UUID> MongoDSessionCatalog::getTransactionTableUUID(OperationCon
return coll->uuid();
}
-void MongoDSessionCatalog::invalidateSessions(OperationContext* opCtx,
- boost::optional<BSONObj> singleSessionDoc) {
- const auto replCoord = repl::ReplicationCoordinator::get(opCtx);
- bool isReplSet = replCoord->getReplicationMode() == repl::ReplicationCoordinator::modeReplSet;
- if (isReplSet) {
- uassert(40528,
- str::stream() << "Direct writes against "
- << NamespaceString::kSessionTransactionsTableNamespace.ns()
- << " cannot be performed using a transaction or on a session.",
- !opCtx->getLogicalSessionId());
- }
+void MongoDSessionCatalog::observeDirectWriteToConfigTransactions(OperationContext* opCtx,
+ BSONObj singleSessionDoc) {
+ disallowDirectWritesUnderSession(opCtx);
+
+ class KillSessionTokenOnCommit : public RecoveryUnit::Change {
+ public:
+ KillSessionTokenOnCommit(OperationContext* opCtx,
+ SessionCatalog::KillToken sessionKillToken)
+ : _opCtx(opCtx), _sessionKillToken(std::move(sessionKillToken)) {}
+
+ void commit(boost::optional<Timestamp>) override {
+ rollback();
+ }
+
+ void rollback() override {
+ std::vector<SessionCatalog::KillToken> sessionKillTokenVec;
+ sessionKillTokenVec.emplace_back(std::move(_sessionKillToken));
+ killSessionTokens(_opCtx, std::move(sessionKillTokenVec));
+ }
+
+ private:
+ OperationContext* _opCtx;
+ SessionCatalog::KillToken _sessionKillToken;
+ };
+
+ const auto catalog = SessionCatalog::get(opCtx);
+
+ const auto lsid =
+ LogicalSessionId::parse(IDLParserErrorContext("lsid"), singleSessionDoc["_id"].Obj());
+ catalog->scanSession(lsid, [&](const ObservableSession& session) {
+ const auto participant = TransactionParticipant::get(session);
+ uassert(ErrorCodes::PreparedTransactionInProgress,
+ str::stream() << "Cannot modify the entry for session "
+ << session.getSessionId().getId()
+ << " because it is in the prepared state",
+ !participant.transactionIsPrepared());
+
+ opCtx->recoveryUnit()->registerChange(new KillSessionTokenOnCommit(opCtx, session.kill()));
+ });
+}
+
+void MongoDSessionCatalog::invalidateAllSessions(OperationContext* opCtx) {
+ disallowDirectWritesUnderSession(opCtx);
const auto catalog = SessionCatalog::get(opCtx);
std::vector<SessionCatalog::KillToken> sessionKillTokens;
- if (singleSessionDoc) {
- const auto lsid = LogicalSessionId::parse(IDLParserErrorContext("lsid"),
- singleSessionDoc->getField("_id").Obj());
- catalog->scanSession(lsid, [&sessionKillTokens](const ObservableSession& session) {
- const auto participant = TransactionParticipant::get(session);
- uassert(ErrorCodes::PreparedTransactionInProgress,
- str::stream() << "Cannot modify the entry for session "
- << session.getSessionId().getId()
- << " because it is in the prepared state",
- !participant.transactionIsPrepared());
- sessionKillTokens.emplace_back(session.kill());
- });
- } else {
- SessionKiller::Matcher matcher(
- KillAllSessionsByPatternSet{makeKillAllSessionsByPattern(opCtx)});
- catalog->scanSessions(matcher, [&sessionKillTokens](const ObservableSession& session) {
- sessionKillTokens.emplace_back(session.kill());
- });
- }
+ SessionKiller::Matcher matcher(
+ KillAllSessionsByPatternSet{makeKillAllSessionsByPattern(opCtx)});
+ catalog->scanSessions(matcher, [&sessionKillTokens](const ObservableSession& session) {
+ sessionKillTokens.emplace_back(session.kill());
+ });
+
killSessionTokens(opCtx, std::move(sessionKillTokens));
}
@@ -291,11 +326,11 @@ int MongoDSessionCatalog::reapSessionsOlderThan(OperationContext* opCtx,
0,
&kIdProjection);
- // The max batch size is chosen so that a single batch won't exceed the 16MB BSON object
- // size limit
- LogicalSessionIdSet lsids;
+ // The max batch size is chosen so that a single batch won't exceed the 16MB BSON object size
+ // limit
const int kMaxBatchSize = 10'000;
+ LogicalSessionIdSet lsids;
int numReaped = 0;
while (cursor->more()) {
auto transactionSession = SessionsCollectionFetchResultIndividualResult::parse(
diff --git a/src/mongo/db/session_catalog_mongod.h b/src/mongo/db/session_catalog_mongod.h
index e1eb6d8b4fc..b55f8927f3f 100644
--- a/src/mongo/db/session_catalog_mongod.h
+++ b/src/mongo/db/session_catalog_mongod.h
@@ -53,23 +53,24 @@ public:
static boost::optional<UUID> getTransactionTableUUID(OperationContext* opCtx);
/**
- * Callback to be invoked when it is suspected that the on-disk session contents might not be in
- * sync with what is in the sessions cache.
- *
- * If no specific document is available, the method will invalidate all sessions. Otherwise if
- * one is avaiable (which is the case for insert/update/delete), it must contain _id field with
- * a valid session entry, in which case only that particular session will be invalidated. If the
- * _id field is missing or doesn't contain a valid serialization of logical session, the method
- * will throw. This prevents invalid entries from making it in the collection.
+ * Callback to be invoked in response to insert/update/delete of 'config.transactions' in order
+ * to notify the session catalog that the on-disk contents are out of sync with the in-memory
+ * state. The 'singleSessionDoc' must contain the _id of the session which was updated.
+ */
+ static void observeDirectWriteToConfigTransactions(OperationContext* opCtx,
+ BSONObj singleSessionDoc);
+
+ /**
+ * Callback to be invoked when the contents of 'config.transactions' are out of sync with that
+ * in the in-memory catalog, such as when rollback happens or drop of 'config.transactions'.
*/
- static void invalidateSessions(OperationContext* opCtx,
- boost::optional<BSONObj> singleSessionDoc);
+ static void invalidateAllSessions(OperationContext* opCtx);
/**
* Locates session entries from the in-memory catalog and in 'config.transactions' which have
* not been referenced before 'possiblyExpired' and deletes them.
*/
- static int reapSessionsOlderThan(OperationContext* OperationContext,
+ static int reapSessionsOlderThan(OperationContext* opCtx,
SessionsCollection& sessionsCollection,
Date_t possiblyExpired);
};
diff --git a/src/mongo/db/transaction_participant.cpp b/src/mongo/db/transaction_participant.cpp
index 0593fee0897..2301b02d541 100644
--- a/src/mongo/db/transaction_participant.cpp
+++ b/src/mongo/db/transaction_participant.cpp
@@ -58,8 +58,6 @@
#include "mongo/db/retryable_writes_stats.h"
#include "mongo/db/server_recovery.h"
#include "mongo/db/server_transactions_metrics.h"
-#include "mongo/db/session.h"
-#include "mongo/db/session_catalog.h"
#include "mongo/db/stats/fill_locker_info.h"
#include "mongo/db/transaction_history_iterator.h"
#include "mongo/db/transaction_participant_gen.h"
@@ -291,6 +289,9 @@ MONGO_FAIL_POINT_DEFINE(onPrimaryTransactionalWrite);
const BSONObj TransactionParticipant::kDeadEndSentinel(BSON("$incompleteOplogHistory" << 1));
+TransactionParticipant::TransactionParticipant() = default;
+
+TransactionParticipant::~TransactionParticipant() = default;
TransactionParticipant::Observer::Observer(const ObservableSession& osession)
: Observer(&getTransactionParticipant(osession.get())) {}
@@ -307,8 +308,7 @@ TransactionParticipant::Participant::Participant(const SessionToKill& session)
: Observer(&getTransactionParticipant(session.get())) {}
void TransactionParticipant::performNoopWrite(OperationContext* opCtx, StringData msg) {
- repl::ReplicationCoordinator* replCoord =
- repl::ReplicationCoordinator::get(opCtx->getClient()->getServiceContext());
+ const auto replCoord = repl::ReplicationCoordinator::get(opCtx);
// The locker must not have a max lock timeout when this noop write is performed, since if it
// threw LockTimeout, this would be treated as a TransientTransactionError, which would indicate
diff --git a/src/mongo/db/transaction_participant.h b/src/mongo/db/transaction_participant.h
index 9b466083ec8..a76241912fd 100644
--- a/src/mongo/db/transaction_participant.h
+++ b/src/mongo/db/transaction_participant.h
@@ -92,9 +92,6 @@ enum class TerminationCause {
* the comments below for more information.
*/
class TransactionParticipant {
- TransactionParticipant(const TransactionParticipant&) = delete;
- TransactionParticipant& operator=(const TransactionParticipant&) = delete;
-
struct PrivateState;
struct ObservableState;
@@ -192,6 +189,13 @@ class TransactionParticipant {
public:
static inline MutableObeserverRegistry<int32_t> observeTransactionLifetimeLimitSeconds;
+ TransactionParticipant();
+
+ TransactionParticipant(const TransactionParticipant&) = delete;
+ TransactionParticipant& operator=(const TransactionParticipant&) = delete;
+
+ ~TransactionParticipant();
+
/**
* Holds state for a snapshot read or multi-statement transaction in between network
* operations.
@@ -360,10 +364,9 @@ public:
TransactionParticipant* _tp;
}; // class Observer
-
/**
- * Class used by a thread that has checked out the TransactionParticipant's session to
- * observe and modify the transaction participant.
+ * Class used by a thread that has checked out the TransactionParticipant's session to observe
+ * and modify the transaction participant.
*/
class Participant : public Observer {
public:
@@ -778,9 +781,6 @@ public:
// invalidating a transaction, or starting a new transaction.
void _resetTransactionState(WithLock wl, TransactionState::StateFlag state);
- // Helper that updates ServerTransactionsMetrics once a transaction commits.
- void _updateTxnMetricsOnCommit(OperationContext* opCtx, bool isCommittingWithPrepare);
-
// Releases the resources held in *o().txnResources to the operation context.
// o().txnResources must be engaged prior to calling this.
void _releaseTransactionResourcesToOpCtx(OperationContext* opCtx);
@@ -809,7 +809,6 @@ public:
return Observer(osession);
}
-
/**
* Returns the timestamp of the oldest oplog entry written across all open transactions, at the
* time of the stable timestamp. Returns boost::none if there are no active transactions, or an
@@ -824,9 +823,6 @@ public:
*/
static void performNoopWrite(OperationContext* opCtx, StringData msg);
- TransactionParticipant() = default;
- ~TransactionParticipant() = default;
-
private:
/**
* Reserves a slot in the oplog with an open storage-transaction while it is alive. Reserves the
@@ -909,22 +905,25 @@ private:
/**
* State in this struct may be read and written by methods of the Participant, only. It may
- * access the struct via the private p() accessor. No further locking is required in methods
- * of the Participant.
+ * access the struct via the private p() accessor. No further locking is required in methods of
+ * the Participant.
*/
struct PrivateState {
+ // Specifies whether the session information needs to be refreshed from storage
+ bool isValid{false};
+
// Only set if the server is shutting down and it has been ensured that no new requests will
// be accepted. Ensures that any transaction resources will not be stashed from the
// operation context onto the transaction participant when the session is checked-in so that
// locks can automatically get freed.
- bool inShutdown = false;
+ bool inShutdown{false};
// Holds oplog data for operations which have been applied in the current multi-document
// transaction.
std::vector<repl::ReplOperation> transactionOperations;
// Total size in bytes of all operations within the _transactionOperations vector.
- size_t transactionOperationBytes = 0;
+ size_t transactionOperationBytes{0};
// The autocommit setting of this transaction. Should always be false for multi-statement
// transaction. Currently only needed for diagnostics reporting.
@@ -943,9 +942,6 @@ private:
// Retryable writes state
//
- // Specifies whether the session information needs to be refreshed from storage
- bool isValid{false};
-
// Set to true if incomplete history is detected. For example, when the oplog to a write was
// truncated because it was too old.
bool hasIncompleteHistory{false};
diff --git a/src/mongo/s/session_catalog_router.cpp b/src/mongo/s/session_catalog_router.cpp
index f88372078e4..3139b97637a 100644
--- a/src/mongo/s/session_catalog_router.cpp
+++ b/src/mongo/s/session_catalog_router.cpp
@@ -35,7 +35,7 @@
namespace mongo {
-int RouterSessionCatalog::reapSessionsOlderThan(OperationContext* OperationContext,
+int RouterSessionCatalog::reapSessionsOlderThan(OperationContext* opCtx,
SessionsCollection& sessionsCollection,
Date_t possiblyExpired) {
return 0;
diff --git a/src/mongo/s/session_catalog_router.h b/src/mongo/s/session_catalog_router.h
index 75a19324fd1..da503404391 100644
--- a/src/mongo/s/session_catalog_router.h
+++ b/src/mongo/s/session_catalog_router.h
@@ -41,7 +41,7 @@ public:
* Locates session entries from the in-memory catalog which have not been referenced before
* 'possiblyExpired' and deletes them.
*/
- static int reapSessionsOlderThan(OperationContext* OperationContext,
+ static int reapSessionsOlderThan(OperationContext* opCtx,
SessionsCollection& sessionsCollection,
Date_t possiblyExpired);
};
diff --git a/src/mongo/s/transaction_router.cpp b/src/mongo/s/transaction_router.cpp
index b039e20284d..44eb65522ee 100644
--- a/src/mongo/s/transaction_router.cpp
+++ b/src/mongo/s/transaction_router.cpp
@@ -41,7 +41,6 @@
#include "mongo/db/logical_session_id.h"
#include "mongo/db/logical_session_id.h"
#include "mongo/db/repl/read_concern_args.h"
-#include "mongo/db/session_catalog.h"
#include "mongo/executor/task_executor_pool.h"
#include "mongo/rpc/get_status_from_command_result.h"
#include "mongo/s/async_requests_sender.h"
diff --git a/src/mongo/s/transaction_router.h b/src/mongo/s/transaction_router.h
index 19f5398c6d3..853fd0b59ce 100644
--- a/src/mongo/s/transaction_router.h
+++ b/src/mongo/s/transaction_router.h
@@ -30,7 +30,6 @@
#pragma once
#include <boost/optional.hpp>
-#include <map>
#include "mongo/db/commands/txn_cmds_gen.h"
#include "mongo/db/logical_session_id.h"