summaryrefslogtreecommitdiff
path: root/src/mongo/s/transaction_router.h
diff options
context:
space:
mode:
authorBlake Oler <blake.oler@mongodb.com>2019-06-20 11:50:28 -0400
committerBlake Oler <blake.oler@mongodb.com>2019-06-27 17:31:09 -0400
commitc11b97788fcc91288deac647ddcc11625607d256 (patch)
tree3c95e51b7ed04c7a7aa94e6ef56a30ec293e0d9e /src/mongo/s/transaction_router.h
parentd960519275aba7e6611294903cd2b5156710a73b (diff)
downloadmongo-c11b97788fcc91288deac647ddcc11625607d256.tar.gz
SERVER-41676 Convert TransactionRouter to use observer pattern to synchronize internal data
with external observers
Diffstat (limited to 'src/mongo/s/transaction_router.h')
-rw-r--r--src/mongo/s/transaction_router.h671
1 files changed, 366 insertions, 305 deletions
diff --git a/src/mongo/s/transaction_router.h b/src/mongo/s/transaction_router.h
index 1577880a92c..4d442f3a225 100644
--- a/src/mongo/s/transaction_router.h
+++ b/src/mongo/s/transaction_router.h
@@ -47,7 +47,17 @@ namespace mongo {
* Keeps track of the transaction state. A session is in use when it is being used by a request.
*/
class TransactionRouter {
+ struct PrivateState;
+ struct ObservableState;
+
public:
+ TransactionRouter();
+ TransactionRouter(const TransactionRouter&) = delete;
+ TransactionRouter& operator=(const TransactionRouter&) = delete;
+ TransactionRouter(TransactionRouter&&) = delete;
+ TransactionRouter& operator=(TransactionRouter&&) = delete;
+ ~TransactionRouter();
+
// The type of commit initiated for this transaction.
enum class CommitType {
kNotInitiated,
@@ -86,6 +96,7 @@ public:
Participant(bool isCoordinator,
StmtId stmtIdCreatedAt,
+ ReadOnly inReadOnly,
SharedTransactionOptions sharedOptions);
/**
@@ -98,13 +109,13 @@ public:
// Is updated to kReadOnly or kNotReadOnly based on the readOnly field in the participant's
// responses to statements.
- ReadOnly readOnly{ReadOnly::kUnset};
-
- // The highest statement id of the request during which this participant was created.
- const StmtId stmtIdCreatedAt;
+ const ReadOnly readOnly{ReadOnly::kUnset};
// Returns the shared transaction options this participant was created with
const SharedTransactionOptions sharedOptions;
+
+ // The highest statement id of the request during which this participant was created.
+ const StmtId stmtIdCreatedAt;
};
// Container for timing stats for the current transaction. Includes helpers for calculating some
@@ -174,368 +185,418 @@ public:
LogicalTime _atClusterTime;
};
- TransactionRouter();
- TransactionRouter(const TransactionRouter&) = delete;
- TransactionRouter& operator=(const TransactionRouter&) = delete;
- TransactionRouter(TransactionRouter&&) = delete;
- TransactionRouter& operator=(TransactionRouter&&) = delete;
- ~TransactionRouter();
-
/**
- * Extract the runtime state attached to the operation context. Returns nullptr if none is
- * attached.
+ * Class used by observers to examine the state of a TransactionRouter.
*/
- static TransactionRouter* get(OperationContext* opCtx);
- static TransactionRouter* get(const ObservableSession& osession);
+ class Observer {
+ public:
+ explicit Observer(const ObservableSession& session);
- /**
- * Starts a fresh transaction in this session or continue an existing one. Also cleans up the
- * previous transaction state.
- */
- void beginOrContinueTxn(OperationContext* opCtx,
- TxnNumber txnNumber,
- TransactionActions action);
+ protected:
+ explicit Observer(TransactionRouter* tr) : _tr(tr) {}
- /**
- * Attaches the required transaction related fields for a request to be sent to the given
- * shard.
- *
- * Calling this method has the following side effects:
- * 1. Potentially selecting a coordinator.
- * 2. Adding the shard to the list of participants.
- * 3. Also append fields for first statements (ex. startTransaction, readConcern)
- * if the shard was newly added to the list of participants.
- */
- BSONObj attachTxnFieldsIfNeeded(OperationContext* opCtx,
- const ShardId& shardId,
- const BSONObj& cmdObj);
+ const TransactionRouter::ObservableState& o() const {
+ return _tr->_o;
+ }
- /**
- * Processes the transaction metadata in the response from the participant if the response
- * indicates the operation succeeded.
- */
- void processParticipantResponse(const ShardId& shardId, const BSONObj& responseObj);
+ TransactionRouter* _tr;
+ }; // class Observer
/**
- * Returns true if the current transaction can retry on a stale version error from a contacted
- * shard. This is always true except for an error received by a write that is not the first
- * overall statement in the sharded transaction. This is because the entire command will be
- * retried, and shards that were not stale and are targeted again may incorrectly execute the
- * command a second time.
- *
- * Note: Even if this method returns true, the retry attempt may still fail, e.g. if one of the
- * shards that returned a stale version error was involved in a previously completed a statement
- * for this transaction.
- *
- * TODO SERVER-37207: Change batch writes to retry only the failed writes in a batch, to allow
- * retrying writes beyond the first overall statement.
+ * Class used by a thread that has checked out the TransactionRouter's session to observe
+ * and modify the transaction router.
*/
- bool canContinueOnStaleShardOrDbError(StringData cmdName) const;
+ class Router : public Observer {
+ public:
+ explicit Router(OperationContext* opCtx);
- /**
- * Updates the transaction state to allow for a retry of the current command on a stale version
- * error. This includes sending abortTransaction to all cleared participants. Will throw if the
- * transaction cannot be continued.
- */
- void onStaleShardOrDbError(OperationContext* opCtx,
- StringData cmdName,
- const Status& errorStatus);
+ explicit operator bool() const {
+ return _tr;
+ }
- /**
- * Returns true if the current transaction can retry on a snapshot error. This is only true on
- * the first command recevied for a transaction.
- */
- bool canContinueOnSnapshotError() const;
+ /**
+ * Starts a fresh transaction in this session or continue an existing one. Also cleans up the
+ * previous transaction state.
+ */
+ void beginOrContinueTxn(OperationContext* opCtx,
+ TxnNumber txnNumber,
+ TransactionActions action);
- /**
- * Resets the transaction state to allow for a retry attempt. This includes clearing all
- * participants, clearing the coordinator, resetting the global read timestamp, and sending
- * abortTransaction to all cleared participants. Will throw if the transaction cannot be
- * continued.
- */
- void onSnapshotError(OperationContext* opCtx, const Status& errorStatus);
+ /**
+ * Attaches the required transaction related fields for a request to be sent to the given
+ * shard.
+ *
+ * Calling this method has the following side effects:
+ * 1. Potentially selecting a coordinator.
+ * 2. Adding the shard to the list of participants.
+ * 3. Also append fields for first statements (ex. startTransaction, readConcern)
+ * if the shard was newly added to the list of participants.
+ */
+ BSONObj attachTxnFieldsIfNeeded(OperationContext* opCtx,
+ const ShardId& shardId,
+ const BSONObj& cmdObj);
- /**
- * Updates the transaction tracking state to allow for a retry attempt on a view resolution
- * error. This includes sending abortTransaction to all cleared participants.
- */
- void onViewResolutionError(OperationContext* opCtx, const NamespaceString& nss);
+ /**
+ * Processes the transaction metadata in the response from the participant if the response
+ * indicates the operation succeeded.
+ */
+ void processParticipantResponse(OperationContext* opCtx,
+ const ShardId& shardId,
+ const BSONObj& responseObj);
- /**
- * Sets the atClusterTime for the current transaction to the latest time in the router's logical
- * clock. Does nothing if the transaction does not have snapshot read concern or an
- * atClusterTime has already been selected and cannot be changed.
- */
- void setDefaultAtClusterTime(OperationContext* opCtx);
+ /**
+ * Returns true if the current transaction can retry on a stale version error from a
+ * contacted shard. This is always true except for an error received by a write that is not
+ * the first overall statement in the sharded transaction. This is because the entire
+ * command will be retried, and shards that were not stale and are targeted again may
+ * incorrectly execute the command a second time.
+ *
+ * Note: Even if this method returns true, the retry attempt may still fail, e.g. if one of
+ * the shards that returned a stale version error was involved in a previously completed a
+ * statement for this transaction.
+ *
+ * TODO SERVER-37207: Change batch writes to retry only the failed writes in a batch, to
+ * allow retrying writes beyond the first overall statement.
+ */
+ bool canContinueOnStaleShardOrDbError(StringData cmdName) const;
- /**
- * Returns the global read timestamp for this transaction. Returns boost::none for transactions
- * that don't run at snapshot level read concern or if a timestamp has not yet been selected.
- */
- const boost::optional<AtClusterTime>& getAtClusterTime() const;
+ /**
+ * Updates the transaction state to allow for a retry of the current command on a stale
+ * version error. This includes sending abortTransaction to all cleared participants. Will
+ * throw if the transaction cannot be continued.
+ */
+ void onStaleShardOrDbError(OperationContext* opCtx,
+ StringData cmdName,
+ const Status& errorStatus);
- /**
- * If a coordinator has been selected for the current transaction, returns its id.
- */
- const boost::optional<ShardId>& getCoordinatorId() const;
+ /**
+ * Returns true if the current transaction can retry on a snapshot error. This is only true
+ * on the first command recevied for a transaction.
+ */
+ bool canContinueOnSnapshotError() const;
- /**
- * If a recovery shard has been selected for the current transaction, returns its id.
- */
- const boost::optional<ShardId>& getRecoveryShardId() const;
+ /**
+ * Resets the transaction state to allow for a retry attempt. This includes clearing all
+ * participants, clearing the coordinator, resetting the global read timestamp, and sending
+ * abortTransaction to all cleared participants. Will throw if the transaction cannot be
+ * continued.
+ */
+ void onSnapshotError(OperationContext* opCtx, const Status& errorStatus);
- /**
- * Commits the transaction.
- *
- * For transactions that only did reads or only wrote to one shard, sends commit directly to the
- * participants and returns the first error response or the last (success) response.
- *
- * For transactions that performed writes to multiple shards, hands off the participant list to
- * the coordinator to do two-phase commit, and returns the coordinator's response.
- */
- BSONObj commitTransaction(OperationContext* opCtx,
- const boost::optional<TxnRecoveryToken>& recoveryToken);
+ /**
+ * Updates the transaction tracking state to allow for a retry attempt on a view resolution
+ * error. This includes sending abortTransaction to all cleared participants.
+ */
+ void onViewResolutionError(OperationContext* opCtx, const NamespaceString& nss);
- /**
- * Sends abort to all participants.
- *
- * Returns the first error response or the last (success) response.
- */
- BSONObj abortTransaction(OperationContext* opCtx);
+ /**
+ * Returns true if the associated transaction is running at snapshot level read concern.
+ */
+ bool mustUseAtClusterTime() const;
- /**
- * Sends abort to all shards in the current participant list. Will retry on retryable errors,
- * but ignores the responses from each shard.
- */
- void implicitlyAbortTransaction(OperationContext* opCtx, const Status& errorStatus);
+ /**
+ * Returns the read timestamp for this transaction. Callers must verify that the read
+ * timestamp has been selected for this transaction before calling this function.
+ */
+ LogicalTime getSelectedAtClusterTime() const;
- /**
- * Returns the participant for this transaction or nullptr if the specified shard is not
- * participant of this transaction.
- */
- Participant* getParticipant(const ShardId& shard);
+ /**
+ * Sets the atClusterTime for the current transaction to the latest time in the router's
+ * logical clock. Does nothing if the transaction does not have snapshot read concern or an
+ * atClusterTime has already been selected and cannot be changed.
+ */
+ void setDefaultAtClusterTime(OperationContext* opCtx);
- /**
- * If a coordinator has been selected for this transaction already, constructs a recovery token,
- * which can be used to resume commit or abort of the transaction from a different router.
- */
- void appendRecoveryToken(BSONObjBuilder* builder) const;
+ /**
+ * If a coordinator has been selected for the current transaction, returns its id.
+ */
+ const boost::optional<ShardId>& getCoordinatorId() const;
- /**
- * Returns a string with the active transaction's transaction number and logical session id
- * (i.e. the transaction id).
- */
- std::string txnIdToString() const;
+ /**
+ * If a recovery shard has been selected for the current transaction, returns its id.
+ */
+ const boost::optional<ShardId>& getRecoveryShardId() const;
- /**
- * Returns the statement id of the latest received command for this transaction.
- */
- StmtId getLatestStmtId() const {
- return _latestStmtId;
- }
+ /**
+ * Commits the transaction.
+ *
+ * For transactions that only did reads or only wrote to one shard, sends commit directly to
+ * the participants and returns the first error response or the last (success) response.
+ *
+ * For transactions that performed writes to multiple shards, hands off the participant list
+ * to the coordinator to do two-phase commit, and returns the coordinator's response.
+ */
+ BSONObj commitTransaction(OperationContext* opCtx,
+ const boost::optional<TxnRecoveryToken>& recoveryToken);
- /**
- * Returns a copy of the timing stats of the transaction router's active transaction.
- */
- TimingStats getTimingStats() const {
- return _timingStats;
- }
+ /**
+ * Sends abort to all participants.
+ *
+ * Returns the first error response or the last (success) response.
+ */
+ BSONObj abortTransaction(OperationContext* opCtx);
-private:
- // Helper to convert the CommitType enum into a human readable string for diagnostics.
- std::string _commitTypeToString(CommitType state) const {
- switch (state) {
- case CommitType::kNotInitiated:
- return "notInitiated";
- case CommitType::kNoShards:
- return "noShards";
- case CommitType::kSingleShard:
- return "singleShard";
- case CommitType::kSingleWriteShard:
- return "singleWriteShard";
- case CommitType::kReadOnly:
- return "readOnly";
- case CommitType::kTwoPhaseCommit:
- return "twoPhaseCommit";
- case CommitType::kRecoverWithToken:
- return "recoverWithToken";
+ /**
+ * Sends abort to all shards in the current participant list. Will retry on retryable errors,
+ * but ignores the responses from each shard.
+ */
+ void implicitlyAbortTransaction(OperationContext* opCtx, const Status& errorStatus);
+
+ /**
+ * If a coordinator has been selected for this transaction already, constructs a recovery
+ * token, which can be used to resume commit or abort of the transaction from a different
+ * router.
+ */
+ void appendRecoveryToken(BSONObjBuilder* builder) const;
+
+ /**
+ * Returns a string with the active transaction's transaction number and logical session id
+ * (i.e. the transaction id).
+ */
+ std::string txnIdToString() const;
+
+ /**
+ * Returns the participant for this transaction or nullptr if the specified shard is not
+ * participant of this transaction.
+ */
+ const Participant* getParticipant(const ShardId& shard);
+
+ /**
+ * Returns the statement id of the latest received command for this transaction.
+ */
+ StmtId getLatestStmtId() const {
+ return p().latestStmtId;
}
- MONGO_UNREACHABLE;
- }
- /**
- * Prints slow transaction information to the log.
- */
- void _logSlowTransaction(OperationContext* opCtx, TerminationCause terminationCause) const;
+ /**
+ * Returns a copy of the timing stats of the transaction router's active transaction.
+ */
+ const TimingStats& getTimingStats() const {
+ return o().timingStats;
+ }
- /**
- * Returns a string to be logged for slow transactions.
- */
- std::string _transactionInfoForLog(OperationContext* opCtx,
- TerminationCause terminationCause) const;
+ private:
+ /**
+ * Resets the router's state. Used when the router sees a new transaction for the first time.
+ * This is required because we don't create a new router object for each transaction, but
+ * instead reuse the same object across different transactions.
+ */
+ void _resetRouterState(OperationContext* opCtx, const TxnNumber& txnNumber);
- // Shortcut to obtain the id of the session under which this transaction router runs
- const LogicalSessionId& _sessionId() const;
+ /**
+ * Internal method for committing a transaction. Should only throw on failure to send commit.
+ */
+ BSONObj _commitTransaction(OperationContext* opCtx,
+ const boost::optional<TxnRecoveryToken>& recoveryToken);
- /**
- * Resets the router's state. Used when the router sees a new transaction for the first time.
- * This is required because we don't create a new router object for each transaction, but
- * instead reuse the same object across different transactions.
- */
- void _resetRouterState(const TxnNumber& txnNumber);
+ /**
+ * Retrieves the transaction's outcome from the shard specified in the recovery token.
+ */
+ BSONObj _commitWithRecoveryToken(OperationContext* opCtx,
+ const TxnRecoveryToken& recoveryToken);
- /**
- * Internal method for committing a transaction. Should only throw on failure to send commit.
- */
- BSONObj _commitTransaction(OperationContext* opCtx,
- const boost::optional<TxnRecoveryToken>& recoveryToken);
+ /**
+ * Hands off coordinating a two-phase commit across all participants to the coordinator
+ * shard.
+ */
+ BSONObj _handOffCommitToCoordinator(OperationContext* opCtx);
- /**
- * Retrieves the transaction's outcome from the shard specified in the recovery token.
- */
- BSONObj _commitWithRecoveryToken(OperationContext* opCtx,
- const TxnRecoveryToken& recoveryToken);
+ /**
+ * Sets the given logical time as the atClusterTime for the transaction to be the greater of
+ * the given time and the user's afterClusterTime, if one was provided.
+ */
+ void _setAtClusterTime(OperationContext* opCtx,
+ const boost::optional<LogicalTime>& afterClusterTime,
+ LogicalTime candidateTime);
- /**
- * Hands off coordinating a two-phase commit across all participants to the coordinator shard.
- */
- BSONObj _handOffCommitToCoordinator(OperationContext* opCtx);
+ /**
+ * Throws NoSuchTransaction if the response from abortTransaction failed with a code other
+ * than NoSuchTransaction. Does not check for write concern errors.
+ */
+ void _assertAbortStatusIsOkOrNoSuchTransaction(
+ const AsyncRequestsSender::Response& response) const;
- /**
- * Sets the given logical time as the atClusterTime for the transaction to be the greater of the
- * given time and the user's afterClusterTime, if one was provided.
- */
- void _setAtClusterTime(const boost::optional<LogicalTime>& afterClusterTime,
- LogicalTime candidateTime);
+ /**
+ * If the transaction's read concern level is snapshot, asserts the participant's
+ * atClusterTime matches the transaction's.
+ */
+ void _verifyParticipantAtClusterTime(const Participant& participant);
- /**
- * Throws NoSuchTransaction if the response from abortTransaction failed with a code other than
- * NoSuchTransaction. Does not check for write concern errors.
- */
- void _assertAbortStatusIsOkOrNoSuchTransaction(
- const AsyncRequestsSender::Response& response) const;
+ /**
+ * Removes all participants created during the current statement from the participant list
+ * and sends abortTransaction to each. Waits for all responses before returning.
+ */
+ void _clearPendingParticipants(OperationContext* opCtx);
- /**
- * Returns all participants created during the current statement.
- */
- std::vector<ShardId> _getPendingParticipants() const;
+ /**
+ * Creates a new participant for the shard.
+ */
+ TransactionRouter::Participant& _createParticipant(OperationContext* opCtx,
+ const ShardId& shard);
- /**
- * Removes all participants created during the current statement from the participant list and
- * sends abortTransaction to each. Waits for all responses before returning.
- */
- void _clearPendingParticipants(OperationContext* opCtx);
+ /**
+ * Sets the new readOnly value for the current participant on the shard.
+ */
+ void _setReadOnlyForParticipant(OperationContext* opCtx,
+ const ShardId& shard,
+ const Participant::ReadOnly readOnly);
- /**
- * Creates a new participant for the shard.
- */
- Participant& _createParticipant(const ShardId& shard);
+ /**
+ * Updates relevant metrics when a new transaction is begun.
+ */
+ void _onNewTransaction(OperationContext* opCtx);
- /**
- * If the transaction's read concern level is snapshot, asserts the participant's atClusterTime
- * matches the transaction's.
- */
- void _verifyParticipantAtClusterTime(const Participant& participant);
+ /**
+ * Updates relevant metrics when a router receives commit for a higher txnNumber than it has
+ * seen so far.
+ */
+ void _onBeginRecoveringDecision(OperationContext* opCtx);
- /**
- * Updates relevant metrics when a new transaction is begun.
- */
- void _onNewTransaction(OperationContext* opCtx);
+ /**
+ * Updates relevant metrics when the router receives an explicit abort from the client.
+ */
+ void _onExplicitAbort(OperationContext* opCtx);
- /**
- * Updates relevant metrics when a router receives commit for a higher txnNumber than it has
- * seen so far.
- */
- void _onBeginRecoveringDecision(OperationContext* opCtx);
+ /**
+ * Updates relevant metrics when the router begins an implicit abort after an error.
+ */
+ void _onImplicitAbort(OperationContext* opCtx, const Status& errorStatus);
- /**
- * Updates relevant metrics when the router receives an explicit abort from the client.
- */
- void _onExplicitAbort(OperationContext* opCtx);
+ /**
+ * Updates relevant metrics when a transaction is about to begin commit.
+ */
+ void _onStartCommit(WithLock wl, OperationContext* opCtx);
- /**
- * Updates relevant metrics when the router begins an implicit abort after an error.
- */
- void _onImplicitAbort(OperationContext* opCtx, const Status& errorStatus);
+ /**
+ * Updates relevant metrics when a transaction receives a successful response for commit.
+ */
+ void _onSuccessfulCommit(OperationContext* opCtx);
- /**
- * Updates relevant metrics when a transaction is about to begin commit.
- */
- void _onStartCommit(OperationContext* opCtx);
+ /**
+ * Updates relevant metrics when commit receives a response with a non-retryable command
+ * error per the retryable writes specification.
+ */
+ void _onNonRetryableCommitError(OperationContext* opCtx, Status commitStatus);
- /**
- * Updates relevant metrics when a transaction receives a successful response for commit.
- */
- void _onSuccessfulCommit(OperationContext* opCtx);
+ /**
+ * The first time this method is called it marks the transaction as over in the router's
+ * diagnostics and will log transaction information if its duration is over the global slowMS
+ * threshold or the transaction log componenet verbosity >= 1. Only meant to be called when
+ * the router definitively knows the transaction's outcome, e.g. it should not be invoked
+ * after a network error on commit.
+ */
+ void _endTransactionTrackingIfNecessary(OperationContext* opCtx,
+ TerminationCause terminationCause);
- /**
- * Updates relevant metrics when commit receives a response with a non-retryable command error
- * per the retryable writes specification.
- */
- void _onNonRetryableCommitError(OperationContext* opCtx, Status commitStatus);
+ /**
+ * Returns all participants created during the current statement.
+ */
+ std::vector<ShardId> _getPendingParticipants() const;
+
+ /**
+ * Prints slow transaction information to the log.
+ */
+ void _logSlowTransaction(OperationContext* opCtx, TerminationCause terminationCause) const;
+
+ /**
+ * Returns a string to be logged for slow transactions.
+ */
+ std::string _transactionInfoForLog(OperationContext* opCtx,
+ TerminationCause terminationCause) const;
+
+ // Shortcut to obtain the id of the session under which this transaction router runs
+ const LogicalSessionId& _sessionId() const;
+
+ TransactionRouter::PrivateState& p() {
+ return _tr->_p;
+ }
+ const TransactionRouter::PrivateState& p() const {
+ return _tr->_p;
+ }
+ TransactionRouter::ObservableState& o(WithLock) {
+ return _tr->_o;
+ }
+ using Observer::o;
+ }; // class Router
+
+ static Router get(OperationContext* opCtx) {
+ return Router(opCtx);
+ }
+
+ static Observer get(const ObservableSession& osession) {
+ return Observer(osession);
+ }
+
+private:
/**
- * The first time this method is called it marks the transaction as over in the router's
- * diagnostics and will log transaction information if its duration is over the global slowMS
- * threshold or the transaction log componenet verbosity >= 1. Only meant to be called when the
- * router definitively knows the transaction's outcome, e.g. it should not be invoked after a
- * network error on commit.
+ * State in this struct may be read by methods of Observer or Router, and may be written by
+ * methods of Router when they acquire the lock on the opCtx's Client. Access this inside
+ * Observer and Router using the private o() method for reading and (Router only) the
+ * o(WithLock) method for writing.
*/
- void _endTransactionTrackingIfNecessary(OperationContext* opCtx,
- TerminationCause terminationCause);
-
- // The currently active transaction number on this router, if beginOrContinueTxn has been
- // called. Otherwise set to kUninitializedTxnNumber.
- TxnNumber _txnNumber{kUninitializedTxnNumber};
+ struct ObservableState {
+ // The currently active transaction number on this router, if beginOrContinueTxn has been
+ // called. Otherwise set to kUninitializedTxnNumber.
+ TxnNumber txnNumber{kUninitializedTxnNumber};
- // Is updated at commit time to reflect which commit path was taken.
- CommitType _commitType{CommitType::kNotInitiated};
+ // Is updated at commit time to reflect which commit path was taken.
+ CommitType commitType{CommitType::kNotInitiated};
- // Indicates whether this is trying to recover a commitTransaction on the current transaction.
- bool _isRecoveringCommit{false};
+ // Map of current participants of the current transaction.
+ StringMap<Participant> participants;
- // Map of current participants of the current transaction.
- StringMap<Participant> _participants;
+ // The id of participant chosen as the two-phase commit coordinator. If, at commit time,
+ // two-phase commit is required, the participant list is handed off to this shard. Is unset
+ // until the transaction has targeted a participant, and is set to the first participant
+ // targeted. Is reset if the first participant targeted returns a "needs retargeting" error.
+ boost::optional<ShardId> coordinatorId;
- // The id of participant chosen as the two-phase commit coordinator. If, at commit time,
- // two-phase commit is required, the participant list is handed off to this shard. Is unset
- // until the transaction has targeted a participant, and is set to the first participant
- // targeted. Is reset if the first participant targeted returns a "needs retargeting" error.
- boost::optional<ShardId> _coordinatorId;
+ // The read concern the current transaction was started with.
+ repl::ReadConcernArgs readConcernArgs;
- // The id of the recovery participant. Passed in the recoveryToken that is included on responses
- // to the client. Is unset until the transaction has done a write, and is set to the first
- // participant that reports having done a write. Is reset if that participant is removed from
- // the participant list because another participant targeted in the same statement returned a
- // "needs retargeting" error.
- boost::optional<ShardId> _recoveryShardId;
+ // The cluster time of the timestamp all participant shards in the current transaction with
+ // snapshot level read concern must read from. Only set for transactions running with
+ // snapshot level read concern.
+ boost::optional<AtClusterTime> atClusterTime;
- // The read concern the current transaction was started with.
- repl::ReadConcernArgs _readConcernArgs;
+ // String representing the reason a transaction aborted. Either the string name of the error
+ // code that led to an implicit abort or "abort" if the client sent abortTransaction.
+ std::string abortCause;
- // The cluster time of the timestamp all participant shards in the current transaction with
- // snapshot level read concern must read from. Only set for transactions running with snapshot
- // level read concern.
- boost::optional<AtClusterTime> _atClusterTime;
+ // Stats used for calculating durations for the active transaction.
+ TimingStats timingStats;
+ } _o;
- // The statement id of the latest received command for this transaction. For batch writes, this
- // will be the highest stmtId contained in the batch. Incremented by one if new commands do not
- // contain statement ids.
- StmtId _latestStmtId{kDefaultFirstStmtId};
+ /**
+ * State in this struct may be read and written by methods of the Router, only. It may
+ * access the struct via the private p() accessor. No further locking is required in methods of
+ * the Router.
+ */
+ struct PrivateState {
+ // Indicates whether this is trying to recover a commitTransaction on the current
+ // transaction.
+ bool isRecoveringCommit{false};
- // The statement id of the command that began this transaction. Defaults to zero if no statement
- // id was included in the first command.
- StmtId _firstStmtId{kDefaultFirstStmtId};
+ // The id of the recovery participant. Passed in the recoveryToken that is included on
+ // responses to the client. Is unset until the transaction has done a write, and is set to
+ // the first participant that reports having done a write. Is reset if that participant is
+ // removed from the participant list because another participant targeted in the same
+ // statement returned a "needs retargeting" error.
+ boost::optional<ShardId> recoveryShardId;
- // String representing the reason a transaction aborted. Either the string name of the error
- // code that led to an implicit abort or "abort" if the client sent abortTransaction.
- std::string _abortCause;
+ // The statement id of the latest received command for this transaction. For batch writes,
+ // this will be the highest stmtId contained in the batch. Incremented by one if new
+ // commands do not contain statement ids.
+ StmtId latestStmtId{kDefaultFirstStmtId};
- // Stats used for calculating durations for the active transaction.
- TimingStats _timingStats;
+ // The statement id of the command that began this transaction. Defaults to zero if no
+ // statement id was included in the first command.
+ StmtId firstStmtId{kDefaultFirstStmtId};
- // Track whether commit or abort have been initiated.
- bool _terminationInitiated{false};
+ // Track whether commit or abort have been initiated.
+ bool terminationInitiated{false};
+ } _p;
};
} // namespace mongo