summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorJames Wahlin <james@mongodb.com>2018-03-27 16:27:34 -0400
committerJames Wahlin <james@mongodb.com>2018-03-28 09:15:21 -0400
commit9b6da195207b71348dd1811dfbd6f8ca5c9b7428 (patch)
treeefd9f073ffcb9d5237bb0a4fc695aefe08eb7c6b
parent82e4873c00f01e4dcb5aad4a6550a82ec4aa4a65 (diff)
downloadmongo-9b6da195207b71348dd1811dfbd6f8ca5c9b7428.tar.gz
SERVER-34101 Revert change to free transaction resources on snapshot read killCursors
-rw-r--r--jstests/noPassthrough/kill_snapshot_cursor.js140
-rw-r--r--src/mongo/db/commands/killcursors_cmd.cpp25
-rw-r--r--src/mongo/db/cursor_manager.cpp104
-rw-r--r--src/mongo/db/cursor_manager.h13
-rw-r--r--src/mongo/db/session.cpp7
-rw-r--r--src/mongo/db/session.h6
-rw-r--r--src/mongo/db/session_catalog.cpp34
-rw-r--r--src/mongo/db/session_catalog.h21
-rw-r--r--src/mongo/dbtests/cursor_manager_test.cpp50
9 files changed, 51 insertions, 349 deletions
diff --git a/jstests/noPassthrough/kill_snapshot_cursor.js b/jstests/noPassthrough/kill_snapshot_cursor.js
deleted file mode 100644
index c1916a0926a..00000000000
--- a/jstests/noPassthrough/kill_snapshot_cursor.js
+++ /dev/null
@@ -1,140 +0,0 @@
-// Tests that destroying a snapshot read cursor aborts its transaction.
-// @tags: [requires_replication]
-(function() {
- "use strict";
-
- const dbName = "test";
- const collName = "coll";
-
- // The cursor timeout thread runs every 'cursorMonitorFrequencySecs' seconds, timing out cursors
- // that have been inactive for 'cursorTimeoutMs' milliseconds.
- const cursorTimeoutMs = 2000;
- const cursorMonitorFrequencySecs = 1;
-
- const options = {
- setParameter: {
- // We use the "cursorTimeoutMillis" server parameter to decrease how long it takes for a
- // non-exhausted cursor to time out. We use the "clientCursorMonitorFrequencySecs"
- // server parameter to make the ClientCursorMonitor that cleans up the timed out cursors
- // run more often. The combination of these server parameters reduces the amount of time
- // we need to wait within this test.
- cursorTimeoutMillis: cursorTimeoutMs,
- clientCursorMonitorFrequencySecs: cursorMonitorFrequencySecs,
- }
- };
-
- const rst = new ReplSetTest({nodes: 1, nodeOptions: options});
- rst.startSet();
- rst.initiate();
-
- const primaryDB = rst.getPrimary().getDB(dbName);
- if (!primaryDB.serverStatus().storageEngine.supportsSnapshotReadConcern) {
- rst.stopSet();
- return;
- }
-
- const session = primaryDB.getMongo().startSession();
- const sessionDB = session.getDatabase(dbName);
- let txnNumber = 0;
-
- //
- // Test killCursors.
- //
-
- for (let i = 0; i < 4; i++) {
- sessionDB.coll.insert({_id: i}, {writeConcern: {w: "majority"}});
- }
-
- // Create a snapshot read cursor.
- let res = assert.commandWorked(sessionDB.runCommand({
- find: collName,
- batchSize: 2,
- readConcern: {level: "snapshot"},
- txnNumber: NumberLong(++txnNumber)
- }));
-
- // It should not be possible to drop the collection.
- assert.commandFailedWithCode(sessionDB.runCommand({drop: collName, maxTimeMS: 500}),
- ErrorCodes.ExceededTimeLimit);
-
- // Kill the cursor.
- assert.commandWorked(sessionDB.runCommand({killCursors: collName, cursors: [res.cursor.id]}));
-
- // It should be possible to drop the collection.
- sessionDB.coll.drop();
-
- //
- // Test legacy killCursors.
- //
-
- for (let i = 0; i < 4; i++) {
- sessionDB.coll.insert({_id: i}, {writeConcern: {w: "majority"}});
- }
-
- // Create a snapshot read cursor.
- res = assert.commandWorked(sessionDB.runCommand({
- find: collName,
- batchSize: 2,
- readConcern: {level: "snapshot"},
- txnNumber: NumberLong(++txnNumber)
- }));
-
- // It should not be possible to drop the collection.
- assert.commandFailedWithCode(sessionDB.runCommand({drop: collName, maxTimeMS: 500}),
- ErrorCodes.ExceededTimeLimit);
-
- // Kill the cursor using legacy.
- primaryDB.getMongo().forceReadMode("legacy");
- let cursor = new DBCommandCursor(primaryDB, res);
- cursor.close();
- primaryDB.getMongo().forceReadMode("commands");
-
- // It should be possible to drop the collection.
- sessionDB.coll.drop();
-
- //
- // Test cursor timeout.
- //
-
- for (let i = 0; i < 4; i++) {
- sessionDB.coll.insert({_id: i}, {writeConcern: {w: "majority"}});
- }
-
- let serverStatus = assert.commandWorked(sessionDB.serverStatus());
- const expectedNumTimedOutCursors = serverStatus.metrics.cursor.timedOut + 1;
-
- // Create a snapshot read cursor.
- res = assert.commandWorked(sessionDB.runCommand({
- find: collName,
- batchSize: 2,
- readConcern: {level: "snapshot"},
- txnNumber: NumberLong(++txnNumber)
- }));
-
- // It should not be possible to drop the collection.
- assert.commandFailedWithCode(sessionDB.runCommand({drop: collName, maxTimeMS: 500}),
- ErrorCodes.ExceededTimeLimit);
-
- // Wait until the idle cursor background job has killed the cursor.
- assert.soon(
- function() {
- serverStatus = assert.commandWorked(sessionDB.serverStatus());
- return +serverStatus.metrics.cursor.timedOut === expectedNumTimedOutCursors;
- },
- function() {
- return "cursor failed to time out: " + tojson(serverStatus.metrics.cursor);
- });
- assert.eq(0, serverStatus.metrics.cursor.open.total, tojson(serverStatus));
-
- // Verify that the cursor was killed.
- assert.commandFailedWithCode(
- sessionDB.runCommand(
- {getMore: res.cursor.id, collection: collName, txnNumber: NumberLong(txnNumber)}),
- ErrorCodes.CursorNotFound);
-
- // It should be possible to drop the collection.
- sessionDB.coll.drop();
-
- session.endSession();
- rst.stopSet();
-})();
diff --git a/src/mongo/db/commands/killcursors_cmd.cpp b/src/mongo/db/commands/killcursors_cmd.cpp
index 453c703dfa4..512ec7e0e94 100644
--- a/src/mongo/db/commands/killcursors_cmd.cpp
+++ b/src/mongo/db/commands/killcursors_cmd.cpp
@@ -36,7 +36,6 @@
#include "mongo/db/cursor_manager.h"
#include "mongo/db/db_raii.h"
#include "mongo/db/query/killcursors_request.h"
-#include "mongo/db/session_catalog.h"
#include "mongo/db/stats/top.h"
#include "mongo/util/scopeguard.h"
@@ -72,28 +71,10 @@ private:
}
}
- boost::optional<std::pair<LogicalSessionId, TxnNumber>> txnToAbort;
-
- auto status = CursorManager::withCursorManager(
- opCtx, id, nss, [opCtx, id, &txnToAbort](CursorManager* manager) {
- auto status = manager->killCursor(opCtx, id, true /* shouldAudit */);
- if (status.isOK()) {
- txnToAbort = status.getValue();
- }
- return status.getStatus();
+ return CursorManager::withCursorManager(
+ opCtx, id, nss, [opCtx, id](CursorManager* manager) {
+ return manager->killCursor(opCtx, id, true /* shouldAudit */);
});
-
- // If the cursor has a corresponding transaction, abort that transaction if it is a snapshot
- // read. This must be done while we are not holding locks.
- invariant(!opCtx->lockState()->isLocked());
- if (txnToAbort) {
- auto session = SessionCatalog::get(opCtx)->getSession(opCtx, txnToAbort->first);
- if (session) {
- (*session)->abortIfSnapshotRead(txnToAbort->second);
- }
- }
-
- return status;
}
} killCursorsCmd;
diff --git a/src/mongo/db/cursor_manager.cpp b/src/mongo/db/cursor_manager.cpp
index 4ae9bafd9ed..39c82910bcb 100644
--- a/src/mongo/db/cursor_manager.cpp
+++ b/src/mongo/db/cursor_manager.cpp
@@ -50,7 +50,6 @@
#include "mongo/db/query/plan_executor.h"
#include "mongo/db/server_parameters.h"
#include "mongo/db/service_context.h"
-#include "mongo/db/session_catalog.h"
#include "mongo/platform/random.h"
#include "mongo/stdx/memory.h"
#include "mongo/util/exit.h"
@@ -216,62 +215,40 @@ bool GlobalCursorIdCache::killCursor(OperationContext* opCtx, CursorId id, bool
}
}
- boost::optional<std::pair<LogicalSessionId, TxnNumber>> txnToAbort;
-
// If this cursor is owned by the global cursor manager, ask it to kill the cursor for us.
if (CursorManager::isGloballyManagedCursor(id)) {
- auto statusWithTxnToAbort = globalCursorManager->killCursor(opCtx, id, checkAuth);
+ Status killStatus = globalCursorManager->killCursor(opCtx, id, checkAuth);
massert(28697,
- statusWithTxnToAbort.getStatus().reason(),
- statusWithTxnToAbort.getStatus().code() == ErrorCodes::OK ||
- statusWithTxnToAbort.getStatus().code() == ErrorCodes::CursorNotFound);
- if (!statusWithTxnToAbort.isOK()) {
- return false;
- }
- txnToAbort = statusWithTxnToAbort.getValue();
- } else {
- // If not, then the cursor must be owned by a collection. Kill the cursor under the
- // collection lock (to prevent the collection from going away during the erase).
- AutoGetCollectionForReadCommand ctx(opCtx, nss);
- Collection* collection = ctx.getCollection();
- if (!collection) {
- if (checkAuth)
- audit::logKillCursorsAuthzCheck(
- opCtx->getClient(), nss, id, ErrorCodes::CursorNotFound);
- return false;
- }
-
- auto statusWithTxnToAbort =
- collection->getCursorManager()->killCursor(opCtx, id, checkAuth);
- uassert(16089,
- statusWithTxnToAbort.getStatus().reason(),
- statusWithTxnToAbort.getStatus().code() == ErrorCodes::OK ||
- statusWithTxnToAbort.getStatus().code() == ErrorCodes::CursorNotFound);
- if (!statusWithTxnToAbort.isOK()) {
- return false;
- }
- txnToAbort = statusWithTxnToAbort.getValue();
- }
-
- // If the cursor has a corresponding transaction, abort that transaction if it is a snapshot
- // read. This must be done while we are not holding locks.
- invariant(!opCtx->lockState()->isLocked());
- if (txnToAbort) {
- auto session = SessionCatalog::get(opCtx)->getSession(opCtx, txnToAbort->first);
- if (session) {
- (*session)->abortIfSnapshotRead(txnToAbort->second);
- }
+ killStatus.reason(),
+ killStatus.code() == ErrorCodes::OK ||
+ killStatus.code() == ErrorCodes::CursorNotFound);
+ return killStatus.isOK();
+ }
+
+ // If not, then the cursor must be owned by a collection. Kill the cursor under the
+ // collection lock (to prevent the collection from going away during the erase).
+ AutoGetCollectionForReadCommand ctx(opCtx, nss);
+ Collection* collection = ctx.getCollection();
+ if (!collection) {
+ if (checkAuth)
+ audit::logKillCursorsAuthzCheck(
+ opCtx->getClient(), nss, id, ErrorCodes::CursorNotFound);
+ return false;
}
- return true;
+ Status eraseStatus = collection->getCursorManager()->killCursor(opCtx, id, checkAuth);
+ uassert(16089,
+ eraseStatus.reason(),
+ eraseStatus.code() == ErrorCodes::OK ||
+ eraseStatus.code() == ErrorCodes::CursorNotFound);
+ return eraseStatus.isOK();
}
std::size_t GlobalCursorIdCache::timeoutCursors(OperationContext* opCtx, Date_t now) {
size_t totalTimedOut = 0;
- std::vector<std::pair<LogicalSessionId, TxnNumber>> txnsToAbort;
// Time out the cursors from the global cursor manager.
- totalTimedOut += globalCursorManager->timeoutCursors(opCtx, now, &txnsToAbort);
+ totalTimedOut += globalCursorManager->timeoutCursors(opCtx, now);
// Compute the set of collection names that we have to time out cursors for.
vector<NamespaceString> todo;
@@ -309,17 +286,7 @@ std::size_t GlobalCursorIdCache::timeoutCursors(OperationContext* opCtx, Date_t
continue;
}
- totalTimedOut += collection->getCursorManager()->timeoutCursors(opCtx, now, &txnsToAbort);
- }
-
- // If the cursors had corresponding transactions, abort the transactions if they are snapshot
- // reads. This must be done while we are not holding locks.
- invariant(!opCtx->lockState()->isLocked());
- for (auto&& txnToAbort : txnsToAbort) {
- auto session = SessionCatalog::get(opCtx)->getSession(opCtx, txnToAbort.first);
- if (session) {
- (*session)->abortIfSnapshotRead(txnToAbort.second);
- }
+ totalTimedOut += collection->getCursorManager()->timeoutCursors(opCtx, now);
}
return totalTimedOut;
@@ -538,10 +505,7 @@ bool CursorManager::cursorShouldTimeout_inlock(const ClientCursor* cursor, Date_
return (now - cursor->_lastUseDate) >= Milliseconds(getCursorTimeoutMillis());
}
-std::size_t CursorManager::timeoutCursors(
- OperationContext* opCtx,
- Date_t now,
- std::vector<std::pair<LogicalSessionId, TxnNumber>>* txnsToAbort) {
+std::size_t CursorManager::timeoutCursors(OperationContext* opCtx, Date_t now) {
std::vector<std::unique_ptr<ClientCursor, ClientCursor::Deleter>> toDelete;
for (size_t partitionId = 0; partitionId < kNumPartitions; ++partitionId) {
@@ -552,10 +516,6 @@ std::size_t CursorManager::timeoutCursors(
// Dispose of the cursor and remove it from the partition.
cursor->dispose(opCtx);
toDelete.push_back(std::unique_ptr<ClientCursor, ClientCursor::Deleter>{cursor});
- if (cursor->getTxnNumber()) {
- invariant(cursor->getSessionId());
- txnsToAbort->emplace_back(*cursor->getSessionId(), *cursor->getTxnNumber());
- }
it = lockedPartition->erase(it);
} else {
++it;
@@ -762,8 +722,7 @@ void CursorManager::deregisterCursor(ClientCursor* cc) {
_cursorMap->erase(cc->cursorid());
}
-StatusWith<boost::optional<std::pair<LogicalSessionId, TxnNumber>>> CursorManager::killCursor(
- OperationContext* opCtx, CursorId id, bool shouldAudit) {
+Status CursorManager::killCursor(OperationContext* opCtx, CursorId id, bool shouldAudit) {
auto lockedPartition = _cursorMap->lockOnePartition(id);
auto it = lockedPartition->find(id);
if (it == lockedPartition->end()) {
@@ -788,23 +747,18 @@ StatusWith<boost::optional<std::pair<LogicalSessionId, TxnNumber>>> CursorManage
if (shouldAudit) {
audit::logKillCursorsAuthzCheck(opCtx->getClient(), _nss, id, ErrorCodes::OK);
}
- return {boost::none};
+ return Status::OK();
}
std::unique_ptr<ClientCursor, ClientCursor::Deleter> ownedCursor(cursor);
- boost::optional<std::pair<LogicalSessionId, TxnNumber>> toReturn;
- if (ownedCursor->getTxnNumber()) {
- invariant(ownedCursor->getSessionId());
- toReturn = std::make_pair(*ownedCursor->getSessionId(), *ownedCursor->getTxnNumber());
- }
-
if (shouldAudit) {
audit::logKillCursorsAuthzCheck(opCtx->getClient(), _nss, id, ErrorCodes::OK);
}
lockedPartition->erase(ownedCursor->cursorid());
ownedCursor->dispose(opCtx);
- return toReturn;
+ return Status::OK();
+ ;
}
Status CursorManager::checkAuthForKillCursors(OperationContext* opCtx, CursorId id) {
diff --git a/src/mongo/db/cursor_manager.h b/src/mongo/db/cursor_manager.h
index 791007134de..90895407f53 100644
--- a/src/mongo/db/cursor_manager.h
+++ b/src/mongo/db/cursor_manager.h
@@ -132,12 +132,9 @@ public:
/**
* Destroys cursors that have been inactive for too long.
*
- * Returns the number of cursors that were timed out. If any of the cursors were running in
- * transactions, appends those transaction IDs to 'txnsToAbort'.
+ * Returns the number of cursors that were timed out.
*/
- std::size_t timeoutCursors(OperationContext* opCtx,
- Date_t now,
- std::vector<std::pair<LogicalSessionId, TxnNumber>>* txnsToAbort);
+ std::size_t timeoutCursors(OperationContext* opCtx, Date_t now);
/**
* Register an executor so that it can be notified of deletions, invalidations, collection
@@ -177,8 +174,7 @@ public:
/**
* Returns an OK status if the cursor was successfully killed, meaning either:
- * (1) The cursor was erased from the cursor registry. In this case, we also return the
- * transaction ID of the transaction the cursor belonged to (if it exists).
+ * (1) The cursor was erased from the cursor registry
* (2) The cursor's operation was interrupted, and the cursor will be cleaned up when the
* operation next checks for interruption.
* Case (2) will only occur if the cursor is pinned.
@@ -188,8 +184,7 @@ public:
*
* If 'shouldAudit' is true, will perform audit logging.
*/
- StatusWith<boost::optional<std::pair<LogicalSessionId, TxnNumber>>> killCursor(
- OperationContext* opCtx, CursorId id, bool shouldAudit);
+ Status killCursor(OperationContext* opCtx, CursorId id, bool shouldAudit);
/**
* Returns an OK status if we're authorized to erase the cursor. Otherwise, returns
diff --git a/src/mongo/db/session.cpp b/src/mongo/db/session.cpp
index 5cecce89bd2..53fbc443744 100644
--- a/src/mongo/db/session.cpp
+++ b/src/mongo/db/session.cpp
@@ -671,13 +671,6 @@ void Session::unstashTransactionResources(OperationContext* opCtx) {
}
}
-void Session::abortIfSnapshotRead(TxnNumber txnNumber) {
- stdx::lock_guard<stdx::mutex> lg(_mutex);
- if (_activeTxnNumber == txnNumber && _autocommit) {
- _abortTransaction(lg);
- }
-}
-
void Session::abortArbitraryTransaction() {
stdx::lock_guard<stdx::mutex> lock(_mutex);
_abortTransaction(lock);
diff --git a/src/mongo/db/session.h b/src/mongo/db/session.h
index 68cc83a9c14..adad7376b43 100644
--- a/src/mongo/db/session.h
+++ b/src/mongo/db/session.h
@@ -223,12 +223,6 @@ public:
void unstashTransactionResources(OperationContext* opCtx);
/**
- * If there is transaction in progress with transaction number 'txnNumber' and _autocommit=true,
- * aborts the transaction.
- */
- void abortIfSnapshotRead(TxnNumber txnNumber);
-
- /**
* Commits the transaction, including committing the write unit of work and updating
* transaction state.
*/
diff --git a/src/mongo/db/session_catalog.cpp b/src/mongo/db/session_catalog.cpp
index 23f830be2e3..48869e4c338 100644
--- a/src/mongo/db/session_catalog.cpp
+++ b/src/mongo/db/session_catalog.cpp
@@ -156,28 +156,6 @@ ScopedSession SessionCatalog::getOrCreateSession(OperationContext* opCtx,
return ss;
}
-boost::optional<ScopedSession> SessionCatalog::getSession(OperationContext* opCtx,
- const LogicalSessionId& lsid) {
- invariant(!opCtx->lockState()->isLocked());
- invariant(!OperationContextSession::get(opCtx));
-
- boost::optional<ScopedSession> ss;
- {
- stdx::unique_lock<stdx::mutex> ul(_mutex);
- auto sri = _getSessionRuntimeInfo(ul, opCtx, lsid);
- if (sri) {
- ss = ScopedSession(sri);
- }
- }
-
- // Perform the refresh outside of the mutex.
- if (ss) {
- (*ss)->refreshFromStorageIfNeeded(opCtx);
- }
-
- return ss;
-}
-
void SessionCatalog::invalidateSessions(OperationContext* opCtx,
boost::optional<BSONObj> singleSessionDoc) {
uassert(40528,
@@ -244,18 +222,6 @@ std::shared_ptr<SessionCatalog::SessionRuntimeInfo> SessionCatalog::_getOrCreate
return it->second;
}
-std::shared_ptr<SessionCatalog::SessionRuntimeInfo> SessionCatalog::_getSessionRuntimeInfo(
- WithLock, OperationContext* opCtx, const LogicalSessionId& lsid) {
- invariant(!opCtx->lockState()->inAWriteUnitOfWork());
-
- auto it = _txnTable.find(lsid);
- if (it == _txnTable.end()) {
- return nullptr;
- }
-
- return it->second;
-}
-
void SessionCatalog::_releaseSession(const LogicalSessionId& lsid) {
stdx::lock_guard<stdx::mutex> lg(_mutex);
diff --git a/src/mongo/db/session_catalog.h b/src/mongo/db/session_catalog.h
index 1c68fdaa623..657f6baf405 100644
--- a/src/mongo/db/session_catalog.h
+++ b/src/mongo/db/session_catalog.h
@@ -106,18 +106,6 @@ public:
ScopedSession getOrCreateSession(OperationContext* opCtx, const LogicalSessionId& lsid);
/**
- * Returns a reference to the specified cached session if it exists, regardless of whether it is
- * checked-out or not. The returned session is not returned checked-out and is allowed to be
- * checked-out concurrently.
- *
- * The intended usage for this method is to allow cursor destruction that may abort the
- * transaction to run in parallel with operations for the same session without blocking it.
- * Because of this, it may not be used from operations which run on a session.
- */
- boost::optional<ScopedSession> getSession(OperationContext* opCtx,
- const LogicalSessionId& lsid);
-
- /**
* 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.
*
@@ -169,15 +157,6 @@ private:
WithLock, OperationContext* opCtx, const LogicalSessionId& lsid);
/**
- * May release and re-acquire it zero or more times before returning. The returned
- * 'SessionRuntimeInfo' is guaranteed to be linked on the catalog's _txnTable as long as the
- * lock is held. If the requested 'SessionRuntimeInfo' does not exist, returns nullptr.
- */
- std::shared_ptr<SessionRuntimeInfo> _getSessionRuntimeInfo(WithLock,
- OperationContext* opCtx,
- const LogicalSessionId& lsid);
-
- /**
* Makes a session, previously checked out through 'checkoutSession', available again.
*/
void _releaseSession(const LogicalSessionId& lsid);
diff --git a/src/mongo/dbtests/cursor_manager_test.cpp b/src/mongo/dbtests/cursor_manager_test.cpp
index 9f28a32e143..ee3933dfa35 100644
--- a/src/mongo/dbtests/cursor_manager_test.cpp
+++ b/src/mongo/dbtests/cursor_manager_test.cpp
@@ -303,7 +303,7 @@ TEST_F(CursorManagerTest, ShouldBeAbleToKillPinnedCursor) {
BSONObj()});
auto cursorId = cursorPin.getCursor()->cursorid();
- ASSERT_OK(cursorManager->killCursor(_opCtx.get(), cursorId, shouldAudit).getStatus());
+ ASSERT_OK(cursorManager->killCursor(_opCtx.get(), cursorId, shouldAudit));
// The original operation should have been interrupted since the cursor was pinned.
ASSERT_EQ(pinningOpCtx->checkForInterruptNoAssert(), ErrorCodes::CursorKilled);
@@ -339,7 +339,7 @@ TEST_F(CursorManagerTest, ShouldBeAbleToKillPinnedCursorMultiClient) {
auto killCursorOpCtx = killCursorClient->makeOperationContext();
invariant(killCursorOpCtx);
- ASSERT_OK(cursorManager->killCursor(killCursorOpCtx.get(), cursorId, shouldAudit).getStatus());
+ ASSERT_OK(cursorManager->killCursor(killCursorOpCtx.get(), cursorId, shouldAudit));
// The original operation should have been interrupted since the cursor was pinned.
ASSERT_EQ(pinningOpCtx->checkForInterruptNoAssert(), ErrorCodes::CursorKilled);
@@ -359,15 +359,11 @@ TEST_F(CursorManagerTest, InactiveCursorShouldTimeout) {
repl::ReadConcernLevel::kLocalReadConcern,
BSONObj()});
- std::vector<std::pair<LogicalSessionId, TxnNumber>> txnsToAbort;
-
- ASSERT_EQ(0UL, cursorManager->timeoutCursors(_opCtx.get(), Date_t(), &txnsToAbort));
- ASSERT_EQ(0UL, txnsToAbort.size());
+ ASSERT_EQ(0UL, cursorManager->timeoutCursors(_opCtx.get(), Date_t()));
clock->advance(getDefaultCursorTimeoutMillis());
- ASSERT_EQ(1UL, cursorManager->timeoutCursors(_opCtx.get(), clock->now(), &txnsToAbort));
+ ASSERT_EQ(1UL, cursorManager->timeoutCursors(_opCtx.get(), clock->now()));
ASSERT_EQ(0UL, cursorManager->numCursors());
- ASSERT_EQ(0UL, txnsToAbort.size());
cursorManager->registerCursor(_opCtx.get(),
{makeFakePlanExecutor(),
@@ -375,9 +371,8 @@ TEST_F(CursorManagerTest, InactiveCursorShouldTimeout) {
{},
repl::ReadConcernLevel::kLocalReadConcern,
BSONObj()});
- ASSERT_EQ(1UL, cursorManager->timeoutCursors(_opCtx.get(), Date_t::max(), &txnsToAbort));
+ ASSERT_EQ(1UL, cursorManager->timeoutCursors(_opCtx.get(), Date_t::max()));
ASSERT_EQ(0UL, cursorManager->numCursors());
- ASSERT_EQ(0UL, txnsToAbort.size());
}
/**
@@ -396,9 +391,7 @@ TEST_F(CursorManagerTest, InactivePinnedCursorShouldNotTimeout) {
// The pin is still in scope, so it should not time out.
clock->advance(getDefaultCursorTimeoutMillis());
- std::vector<std::pair<LogicalSessionId, TxnNumber>> txnsToAbort;
- ASSERT_EQ(0UL, cursorManager->timeoutCursors(_opCtx.get(), clock->now(), &txnsToAbort));
- ASSERT_EQ(0UL, txnsToAbort.size());
+ ASSERT_EQ(0UL, cursorManager->timeoutCursors(_opCtx.get(), clock->now()));
}
/**
@@ -423,10 +416,8 @@ TEST_F(CursorManagerTest, InactiveKilledCursorsShouldTimeout) {
// Advance the clock to simulate time passing.
clock->advance(getDefaultCursorTimeoutMillis());
- std::vector<std::pair<LogicalSessionId, TxnNumber>> txnsToAbort;
- ASSERT_EQ(1UL, cursorManager->timeoutCursors(_opCtx.get(), clock->now(), &txnsToAbort));
+ ASSERT_EQ(1UL, cursorManager->timeoutCursors(_opCtx.get(), clock->now()));
ASSERT_EQ(0UL, cursorManager->numCursors());
- ASSERT_EQ(0UL, txnsToAbort.size());
}
/**
@@ -451,9 +442,7 @@ TEST_F(CursorManagerTest, InactiveKilledCursorsThatAreStillPinnedShouldNotTimeou
clock->advance(getDefaultCursorTimeoutMillis());
// The pin is still in scope, so it should not time out.
- std::vector<std::pair<LogicalSessionId, TxnNumber>> txnsToAbort;
- ASSERT_EQ(0UL, cursorManager->timeoutCursors(_opCtx.get(), clock->now(), &txnsToAbort));
- ASSERT_EQ(0UL, txnsToAbort.size());
+ ASSERT_EQ(0UL, cursorManager->timeoutCursors(_opCtx.get(), clock->now()));
}
/**
@@ -488,20 +477,16 @@ TEST_F(CursorManagerTest, UsingACursorShouldUpdateTimeOfLastUse) {
// Touch the cursor with id 'usedCursorId' to advance its time of last use.
cursorManager->pinCursor(_opCtx.get(), usedCursorId).status_with_transitional_ignore();
- std::vector<std::pair<LogicalSessionId, TxnNumber>> txnsToAbort;
-
// We should be able to time out the unused cursor, but the one we used should stay alive.
ASSERT_EQ(2UL, cursorManager->numCursors());
clock->advance(getDefaultCursorTimeoutMillis() - Milliseconds(1));
- ASSERT_EQ(1UL, cursorManager->timeoutCursors(_opCtx.get(), clock->now(), &txnsToAbort));
+ ASSERT_EQ(1UL, cursorManager->timeoutCursors(_opCtx.get(), clock->now()));
ASSERT_EQ(1UL, cursorManager->numCursors());
- ASSERT_EQ(0UL, txnsToAbort.size());
// We should be able to time out the used cursor after one more millisecond.
clock->advance(Milliseconds(1));
- ASSERT_EQ(1UL, cursorManager->timeoutCursors(_opCtx.get(), clock->now(), &txnsToAbort));
+ ASSERT_EQ(1UL, cursorManager->timeoutCursors(_opCtx.get(), clock->now()));
ASSERT_EQ(0UL, cursorManager->numCursors());
- ASSERT_EQ(0UL, txnsToAbort.size());
}
/**
@@ -523,27 +508,22 @@ TEST_F(CursorManagerTest, CursorShouldNotTimeOutUntilIdleForLongEnoughAfterBeing
// Advance the clock to simulate time passing.
clock->advance(getDefaultCursorTimeoutMillis() + Milliseconds(1));
- std::vector<std::pair<LogicalSessionId, TxnNumber>> txnsToAbort;
-
// Make sure the pinned cursor does not time out, before or after unpinning it.
ASSERT_EQ(1UL, cursorManager->numCursors());
- ASSERT_EQ(0UL, cursorManager->timeoutCursors(_opCtx.get(), clock->now(), &txnsToAbort));
+ ASSERT_EQ(0UL, cursorManager->timeoutCursors(_opCtx.get(), clock->now()));
ASSERT_EQ(1UL, cursorManager->numCursors());
- ASSERT_EQ(0UL, txnsToAbort.size());
cursorPin.release();
ASSERT_EQ(1UL, cursorManager->numCursors());
- ASSERT_EQ(0UL, cursorManager->timeoutCursors(_opCtx.get(), clock->now(), &txnsToAbort));
+ ASSERT_EQ(0UL, cursorManager->timeoutCursors(_opCtx.get(), clock->now()));
ASSERT_EQ(1UL, cursorManager->numCursors());
- ASSERT_EQ(0UL, txnsToAbort.size());
// Advance the clock to simulate more time passing, then assert that the now-inactive cursor
// times out.
clock->advance(getDefaultCursorTimeoutMillis() + Milliseconds(1));
- ASSERT_EQ(1UL, cursorManager->timeoutCursors(_opCtx.get(), clock->now(), &txnsToAbort));
+ ASSERT_EQ(1UL, cursorManager->timeoutCursors(_opCtx.get(), clock->now()));
ASSERT_EQ(0UL, cursorManager->numCursors());
- ASSERT_EQ(0UL, txnsToAbort.size());
}
/**
@@ -606,7 +586,7 @@ TEST_F(CursorManagerTestCustomOpCtx, OneCursorWithASession) {
// Remove the cursor from the manager.
pinned.release();
- ASSERT_OK(useCursorManager()->killCursor(opCtx.get(), cursorId, false).getStatus());
+ ASSERT_OK(useCursorManager()->killCursor(opCtx.get(), cursorId, false));
// There should be no more cursor entries by session id.
LogicalSessionIdSet sessions;
@@ -642,7 +622,7 @@ TEST_F(CursorManagerTestCustomOpCtx, MultipleCursorsWithSameSession) {
// Remove one cursor from the manager.
pinned.release();
- ASSERT_OK(useCursorManager()->killCursor(opCtx.get(), cursorId1, false).getStatus());
+ ASSERT_OK(useCursorManager()->killCursor(opCtx.get(), cursorId1, false));
// Should still be able to retrieve the session.
lsids.clear();