summaryrefslogtreecommitdiff
path: root/src/mongo/db
diff options
context:
space:
mode:
authorEric Milkie <milkie@10gen.com>2018-02-21 15:24:03 -0500
committerEric Milkie <milkie@10gen.com>2018-03-05 07:49:48 -0500
commit8257a817bfaf596e68204ad858f811e294d986b5 (patch)
tree2af6c44d80b7cdc93b8321b25d1cbef9ea1f1a89 /src/mongo/db
parent1cf874bd73ebb13f753840bb6eea63207076c438 (diff)
downloadmongo-8257a817bfaf596e68204ad858f811e294d986b5.tar.gz
SERVER-33318 simplify oldest_timestamp setting
Diffstat (limited to 'src/mongo/db')
-rw-r--r--src/mongo/db/commands/snapshot_management.cpp12
-rw-r--r--src/mongo/db/storage/snapshot_manager.h25
-rw-r--r--src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.cpp83
-rw-r--r--src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.h20
-rw-r--r--src/mongo/db/storage/wiredtiger/wiredtiger_oplog_manager.cpp1
-rw-r--r--src/mongo/db/storage/wiredtiger/wiredtiger_recovery_unit.cpp22
-rw-r--r--src/mongo/db/storage/wiredtiger/wiredtiger_recovery_unit.h8
-rw-r--r--src/mongo/db/storage/wiredtiger/wiredtiger_session_cache.cpp5
-rw-r--r--src/mongo/db/storage/wiredtiger/wiredtiger_snapshot_manager.cpp21
-rw-r--r--src/mongo/db/storage/wiredtiger/wiredtiger_snapshot_manager.h21
10 files changed, 50 insertions, 168 deletions
diff --git a/src/mongo/db/commands/snapshot_management.cpp b/src/mongo/db/commands/snapshot_management.cpp
index 2614325b6b1..f445b448800 100644
--- a/src/mongo/db/commands/snapshot_management.cpp
+++ b/src/mongo/db/commands/snapshot_management.cpp
@@ -78,13 +78,11 @@ public:
Lock::GlobalLock lk(opCtx, MODE_IX, Date_t::max());
- auto status = snapshotManager->prepareForCreateSnapshot(opCtx);
- if (status.isOK()) {
- const auto name =
- repl::ReplicationCoordinator::get(opCtx)->getMinimumVisibleSnapshot(opCtx);
- result.append("name", static_cast<long long>(name.asULL()));
- }
- return CommandHelpers::appendCommandStatus(result, status);
+ const auto name =
+ repl::ReplicationCoordinator::get(opCtx)->getMinimumVisibleSnapshot(opCtx);
+ result.append("name", static_cast<long long>(name.asULL()));
+
+ return CommandHelpers::appendCommandStatus(result, Status::OK());
}
};
diff --git a/src/mongo/db/storage/snapshot_manager.h b/src/mongo/db/storage/snapshot_manager.h
index 0577236a4ce..4c1def141af 100644
--- a/src/mongo/db/storage/snapshot_manager.h
+++ b/src/mongo/db/storage/snapshot_manager.h
@@ -38,8 +38,6 @@
namespace mongo {
-class RecoveryUnit;
-
/**
* Manages snapshots that can be read from at a later time.
*
@@ -49,38 +47,17 @@ class RecoveryUnit;
class SnapshotManager {
public:
/**
- * Prepares the passed-in OperationContext for snapshot creation.
- *
- * The passed-in OperationContext will be associated with a point-in-time that can be used
- * for creating a snapshot later.
- *
- * This must be the first method called after starting a ScopedTransaction, and it is
- * illegal to start a WriteUnitOfWork inside of the same ScopedTransaction.
- */
- virtual Status prepareForCreateSnapshot(OperationContext* opCtx) = 0;
-
- /**
* Sets the snapshot to be used for committed reads.
*
* Implementations are allowed to assume that all older snapshots have names that compare
* less than the passed in name, and newer ones compare greater.
*
* This is called while holding a very hot mutex. Therefore it should avoid doing any work that
- * can be done later. In particular, cleaning up of old snapshots should be deferred until
- * cleanupUnneededSnapshots is called.
+ * can be done later.
*/
virtual void setCommittedSnapshot(const Timestamp& timestamp) = 0;
/**
- * Cleans up all snapshots older than the current committed snapshot.
- *
- * Operations that have already begun using an older snapshot must continue to work using that
- * snapshot until they would normally start using a newer one. Any implementation that allows
- * that without an unbounded growth of snapshots is permitted.
- */
- virtual void cleanupUnneededSnapshots() = 0;
-
- /**
* Drops all snapshots and clears the "committed" snapshot.
*/
virtual void dropAllSnapshots() = 0;
diff --git a/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.cpp b/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.cpp
index 5c99b5de7ab..a3381606ecf 100644
--- a/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.cpp
+++ b/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.cpp
@@ -1024,28 +1024,8 @@ bool WiredTigerKVEngine::initRsOplogBackgroundThread(StringData ns) {
}
void WiredTigerKVEngine::setOldestTimestamp(Timestamp oldestTimestamp) {
- invariant(oldestTimestamp != Timestamp::min());
-
- char commitTSConfigString["force=true,oldest_timestamp=,commit_timestamp="_sd.size() +
- (2 * 8 * 2) /* 8 hexadecimal characters */ + 1 /* trailing null */];
- auto size = std::snprintf(commitTSConfigString,
- sizeof(commitTSConfigString),
- "force=true,oldest_timestamp=%llx,commit_timestamp=%llx",
- oldestTimestamp.asULL(),
- oldestTimestamp.asULL());
- if (size < 0) {
- int e = errno;
- error() << "error snprintf " << errnoWithDescription(e);
- fassertFailedNoTrace(40677);
- }
-
- invariant(static_cast<std::size_t>(size) < sizeof(commitTSConfigString));
- invariantWTOK(_conn->set_timestamp(_conn, commitTSConfigString));
-
- _oplogManager->setOplogReadTimestamp(oldestTimestamp);
- stdx::unique_lock<stdx::mutex> lock(_oplogManagerMutex);
- _previousSetOldestTimestamp = oldestTimestamp;
- LOG(1) << "Forced a new oldest_timestamp. Value: " << oldestTimestamp;
+ constexpr bool doForce = true;
+ _setOldestTimestamp(oldestTimestamp, doForce);
}
void WiredTigerKVEngine::setStableTimestamp(Timestamp stableTimestamp) {
@@ -1088,45 +1068,32 @@ void WiredTigerKVEngine::setStableTimestamp(Timestamp stableTimestamp) {
// Communicate to WiredTiger that it can clean up timestamp data earlier than the timestamp
// provided. No future queries will need point-in-time reads at a timestamp prior to the one
// provided here.
- _advanceOldestTimestamp(stableTimestamp);
+ _setOldestTimestamp(stableTimestamp);
}
-void WiredTigerKVEngine::_advanceOldestTimestamp(Timestamp oldestTimestamp) {
- Timestamp timestampToSet;
- {
- stdx::unique_lock<stdx::mutex> lock(_oplogManagerMutex);
- if (!_oplogManager) {
- // No oplog yet, so don't bother setting oldest_timestamp.
- return;
- }
- auto oplogReadTimestamp = _oplogManager->getOplogReadTimestamp();
- if (oplogReadTimestamp < oldestTimestamp.asULL()) {
- // For one node replica sets, the commit point might race ahead of the oplog read
- // timestamp.
- oldestTimestamp = Timestamp(oplogReadTimestamp);
- if (_previousSetOldestTimestamp > oldestTimestamp) {
- // Do not go backwards.
- return;
- }
- }
+void WiredTigerKVEngine::_setOldestTimestamp(Timestamp oldestTimestamp, bool force) {
- // Lag the oldest_timestamp by one timestamp set, to give a bit more history.
- invariant(_previousSetOldestTimestamp <= oldestTimestamp);
- timestampToSet = _previousSetOldestTimestamp;
- _previousSetOldestTimestamp = oldestTimestamp;
- }
-
- if (timestampToSet == Timestamp()) {
+ if (oldestTimestamp == Timestamp()) {
// Nothing to set yet.
return;
}
- char oldestTSConfigString["oldest_timestamp="_sd.size() + (8 * 2) /* 16 hexadecimal digits */ +
+ char oldestTSConfigString["force=true,oldest_timestamp=,commit_timestamp="_sd.size() +
+ (2 * 8 * 2) /* 2 timestamps of 16 hexadecimal digits each */ +
1 /* trailing null */];
- auto size = std::snprintf(oldestTSConfigString,
- sizeof(oldestTSConfigString),
- "oldest_timestamp=%llx",
- timestampToSet.asULL());
+ int size = 0;
+ if (force) {
+ size = std::snprintf(oldestTSConfigString,
+ sizeof(oldestTSConfigString),
+ "force=true,oldest_timestamp=%llx,commit_timestamp=%llx",
+ oldestTimestamp.asULL(),
+ oldestTimestamp.asULL());
+ } else {
+ size = std::snprintf(oldestTSConfigString,
+ sizeof(oldestTSConfigString),
+ "oldest_timestamp=%llx",
+ oldestTimestamp.asULL());
+ }
if (size < 0) {
int e = errno;
error() << "error snprintf " << errnoWithDescription(e);
@@ -1134,7 +1101,12 @@ void WiredTigerKVEngine::_advanceOldestTimestamp(Timestamp oldestTimestamp) {
}
invariant(static_cast<std::size_t>(size) < sizeof(oldestTSConfigString));
invariantWTOK(_conn->set_timestamp(_conn, oldestTSConfigString));
- LOG(2) << "oldest_timestamp set to " << timestampToSet;
+
+ if (force) {
+ LOG(2) << "oldest_timestamp and commit_timestamp force set to " << oldestTimestamp;
+ } else {
+ LOG(2) << "oldest_timestamp set to " << oldestTimestamp;
+ }
}
void WiredTigerKVEngine::setInitialDataTimestamp(Timestamp initialDataTimestamp) {
@@ -1194,9 +1166,6 @@ void WiredTigerKVEngine::haltOplogManager() {
invariant(_oplogManagerCount > 0);
_oplogManagerCount--;
if (_oplogManagerCount == 0) {
- // Destructor may lock the mutex, so we must unlock here.
- // Oplog managers only destruct at shutdown or test exit, so it is safe to unlock here.
- lock.unlock();
_oplogManager->halt();
}
}
diff --git a/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.h b/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.h
index bd4e10a4366..39e6c00a85a 100644
--- a/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.h
+++ b/src/mongo/db/storage/wiredtiger/wiredtiger_kv_engine.h
@@ -170,15 +170,10 @@ public:
/**
* This method will force the oldest timestamp to the input value. Callers must be serialized
- * along with `_advanceOldestTimestamp`
+ * along with `setStableTimestamp`
*/
void setOldestTimestamp(Timestamp oldestTimestamp);
- Timestamp getPreviousSetOldestTimestamp() const {
- stdx::unique_lock<stdx::mutex> lock(_oplogManagerMutex);
- return _previousSetOldestTimestamp;
- }
-
virtual bool supportsRecoverToStableTimestamp() const override;
virtual Status recoverToStableTimestamp() override;
@@ -262,24 +257,17 @@ private:
std::string _uri(StringData ident) const;
- // Not threadsafe; callers must be serialized along with `setOldestTimestamp`.
- void _advanceOldestTimestamp(Timestamp oldestTimestamp);
-
- // Protected by _oplogManagerMutex.
- Timestamp _previousSetOldestTimestamp;
+ void _setOldestTimestamp(Timestamp oldestTimestamp, bool force = false);
WT_CONNECTION* _conn;
WT_EVENT_HANDLER _eventHandler;
std::unique_ptr<WiredTigerSessionCache> _sessionCache;
ClockSource* const _clockSource;
- // Mutex to protect use of _oplogManager and _oplogManagerCount by this instance of KV
- // engine.
- // Uses of _oplogManager by the oplog record stores themselves are safe without locking, since
- // those record stores manage the oplogManager lifetime.
+ // Mutex to protect use of _oplogManagerCount by this instance of KV engine.
mutable stdx::mutex _oplogManagerMutex;
- std::unique_ptr<WiredTigerOplogManager> _oplogManager;
std::size_t _oplogManagerCount = 0;
+ std::unique_ptr<WiredTigerOplogManager> _oplogManager;
std::string _canonicalName;
std::string _path;
diff --git a/src/mongo/db/storage/wiredtiger/wiredtiger_oplog_manager.cpp b/src/mongo/db/storage/wiredtiger/wiredtiger_oplog_manager.cpp
index 37881acd173..13e2bc7c32a 100644
--- a/src/mongo/db/storage/wiredtiger/wiredtiger_oplog_manager.cpp
+++ b/src/mongo/db/storage/wiredtiger/wiredtiger_oplog_manager.cpp
@@ -35,6 +35,7 @@
#include "mongo/db/storage/wiredtiger/wiredtiger_kv_engine.h"
#include "mongo/db/storage/wiredtiger/wiredtiger_oplog_manager.h"
+#include "mongo/db/storage/wiredtiger/wiredtiger_util.h"
#include "mongo/stdx/mutex.h"
#include "mongo/util/concurrency/idle_thread_block.h"
#include "mongo/util/log.h"
diff --git a/src/mongo/db/storage/wiredtiger/wiredtiger_recovery_unit.cpp b/src/mongo/db/storage/wiredtiger/wiredtiger_recovery_unit.cpp
index 2f37bad6783..093b9b8b556 100644
--- a/src/mongo/db/storage/wiredtiger/wiredtiger_recovery_unit.cpp
+++ b/src/mongo/db/storage/wiredtiger/wiredtiger_recovery_unit.cpp
@@ -68,16 +68,6 @@ WiredTigerRecoveryUnit::~WiredTigerRecoveryUnit() {
_abort();
}
-void WiredTigerRecoveryUnit::prepareForCreateSnapshot(OperationContext* opCtx) {
- invariant(!_active); // Can't already be in a WT transaction.
- invariant(!_inUnitOfWork);
- invariant(!_readFromMajorityCommittedSnapshot);
-
- // Starts the WT transaction that will be the basis for creating a named snapshot.
- getSession();
- _areWriteUnitOfWorksBanned = true;
-}
-
void WiredTigerRecoveryUnit::_commit() {
try {
if (_session && _active) {
@@ -264,16 +254,14 @@ void WiredTigerRecoveryUnit::_txnOpen() {
auto status =
_sessionCache->snapshotManager().beginTransactionAtTimestamp(_readAtTimestamp, session);
if (!status.isOK() && status.code() == ErrorCodes::BadValue) {
- uasserted(
- ErrorCodes::SnapshotTooOld,
- str::stream()
- << "Read timestamp "
- << _readAtTimestamp.toString()
- << " is older than the oldest available timestamp: "
- << _sessionCache->getKVEngine()->getPreviousSetOldestTimestamp().toString());
+ uasserted(ErrorCodes::SnapshotTooOld,
+ str::stream() << "Read timestamp " << _readAtTimestamp.toString()
+ << " is older than the oldest available timestamp.");
}
uassertStatusOK(status);
} else if (_readFromMajorityCommittedSnapshot) {
+ // We reset _majorityCommittedSnapshot to the actual read timestamp used when the
+ // transaction was started.
_majorityCommittedSnapshot =
_sessionCache->snapshotManager().beginTransactionOnCommittedSnapshot(session);
} else if (_isOplogReader) {
diff --git a/src/mongo/db/storage/wiredtiger/wiredtiger_recovery_unit.h b/src/mongo/db/storage/wiredtiger/wiredtiger_recovery_unit.h
index e41047967e8..c2b65222442 100644
--- a/src/mongo/db/storage/wiredtiger/wiredtiger_recovery_unit.h
+++ b/src/mongo/db/storage/wiredtiger/wiredtiger_recovery_unit.h
@@ -127,14 +127,6 @@ public:
static void appendGlobalStats(BSONObjBuilder& b);
- /**
- * Prepares this RU to be the basis for a named snapshot.
- *
- * Begins a WT transaction, and invariants if we are already in one.
- * Bans being in a WriteUnitOfWork until the next call to abandonSnapshot().
- */
- void prepareForCreateSnapshot(OperationContext* opCtx);
-
private:
void _abort();
void _commit();
diff --git a/src/mongo/db/storage/wiredtiger/wiredtiger_session_cache.cpp b/src/mongo/db/storage/wiredtiger/wiredtiger_session_cache.cpp
index 79b4ad90630..aab054c2371 100644
--- a/src/mongo/db/storage/wiredtiger/wiredtiger_session_cache.cpp
+++ b/src/mongo/db/storage/wiredtiger/wiredtiger_session_cache.cpp
@@ -175,10 +175,10 @@ uint64_t WiredTigerSession::genTableId() {
// -----------------------
WiredTigerSessionCache::WiredTigerSessionCache(WiredTigerKVEngine* engine)
- : _engine(engine), _conn(engine->getConnection()), _snapshotManager(_conn), _shuttingDown(0) {}
+ : _engine(engine), _conn(engine->getConnection()), _shuttingDown(0) {}
WiredTigerSessionCache::WiredTigerSessionCache(WT_CONNECTION* conn)
- : _engine(NULL), _conn(conn), _snapshotManager(_conn), _shuttingDown(0) {}
+ : _engine(NULL), _conn(conn), _shuttingDown(0) {}
WiredTigerSessionCache::~WiredTigerSessionCache() {
shuttingDown();
@@ -202,7 +202,6 @@ void WiredTigerSessionCache::shuttingDown() {
}
closeAll();
- _snapshotManager.shutdown();
}
void WiredTigerSessionCache::waitUntilDurable(bool forceCheckpoint, bool stableCheckpoint) {
diff --git a/src/mongo/db/storage/wiredtiger/wiredtiger_snapshot_manager.cpp b/src/mongo/db/storage/wiredtiger/wiredtiger_snapshot_manager.cpp
index a554015102b..effd6e58ca8 100644
--- a/src/mongo/db/storage/wiredtiger/wiredtiger_snapshot_manager.cpp
+++ b/src/mongo/db/storage/wiredtiger/wiredtiger_snapshot_manager.cpp
@@ -31,25 +31,22 @@
#include "mongo/platform/basic.h"
+#include "mongo/db/storage/wiredtiger/wiredtiger_snapshot_manager.h"
+
#include <algorithm>
+#include <cstdio>
-#include "mongo/base/checked_cast.h"
#include "mongo/db/concurrency/write_conflict_exception.h"
#include "mongo/db/storage/wiredtiger/wiredtiger_oplog_manager.h"
#include "mongo/db/storage/wiredtiger/wiredtiger_record_store.h"
#include "mongo/db/storage/wiredtiger/wiredtiger_recovery_unit.h"
#include "mongo/db/storage/wiredtiger/wiredtiger_session_cache.h"
-#include "mongo/db/storage/wiredtiger/wiredtiger_snapshot_manager.h"
+#include "mongo/db/storage/wiredtiger/wiredtiger_util.h"
#include "mongo/util/log.h"
#include "mongo/util/mongoutils/str.h"
namespace mongo {
-Status WiredTigerSnapshotManager::prepareForCreateSnapshot(OperationContext* opCtx) {
- WiredTigerRecoveryUnit::get(opCtx)->prepareForCreateSnapshot(opCtx);
- return Status::OK();
-}
-
void WiredTigerSnapshotManager::setCommittedSnapshot(const Timestamp& timestamp) {
stdx::lock_guard<stdx::mutex> lock(_mutex);
@@ -57,21 +54,11 @@ void WiredTigerSnapshotManager::setCommittedSnapshot(const Timestamp& timestamp)
_committedSnapshot = timestamp;
}
-void WiredTigerSnapshotManager::cleanupUnneededSnapshots() {}
-
void WiredTigerSnapshotManager::dropAllSnapshots() {
stdx::lock_guard<stdx::mutex> lock(_mutex);
_committedSnapshot = boost::none;
}
-void WiredTigerSnapshotManager::shutdown() {
- stdx::lock_guard<stdx::mutex> lock(_mutex);
- if (!_session)
- return;
- invariantWTOK(_session->close(_session, NULL));
- _session = nullptr;
-}
-
boost::optional<Timestamp> WiredTigerSnapshotManager::getMinSnapshotForNextCommittedRead() const {
stdx::lock_guard<stdx::mutex> lock(_mutex);
return _committedSnapshot;
diff --git a/src/mongo/db/storage/wiredtiger/wiredtiger_snapshot_manager.h b/src/mongo/db/storage/wiredtiger/wiredtiger_snapshot_manager.h
index b3853206bcc..b14f0d2c34f 100644
--- a/src/mongo/db/storage/wiredtiger/wiredtiger_snapshot_manager.h
+++ b/src/mongo/db/storage/wiredtiger/wiredtiger_snapshot_manager.h
@@ -35,7 +35,6 @@
#include "mongo/base/disallow_copying.h"
#include "mongo/bson/timestamp.h"
#include "mongo/db/storage/snapshot_manager.h"
-#include "mongo/db/storage/wiredtiger/wiredtiger_util.h"
#include "mongo/stdx/mutex.h"
namespace mongo {
@@ -46,29 +45,15 @@ class WiredTigerSnapshotManager final : public SnapshotManager {
MONGO_DISALLOW_COPYING(WiredTigerSnapshotManager);
public:
- explicit WiredTigerSnapshotManager(WT_CONNECTION* conn) {
- invariantWTOK(conn->open_session(conn, NULL, NULL, &_session));
- _conn = conn;
- }
+ WiredTigerSnapshotManager() = default;
- ~WiredTigerSnapshotManager() {
- shutdown();
- }
-
- Status prepareForCreateSnapshot(OperationContext* opCtx) final;
void setCommittedSnapshot(const Timestamp& timestamp) final;
- void cleanupUnneededSnapshots() final;
void dropAllSnapshots() final;
//
// WT-specific methods
//
- /**
- * Prepares for a shutdown of the WT_CONNECTION.
- */
- void shutdown();
-
Status beginTransactionAtTimestamp(Timestamp pointInTime, WT_SESSION* session) const;
/**
@@ -94,9 +79,7 @@ public:
boost::optional<Timestamp> getMinSnapshotForNextCommittedRead() const;
private:
- mutable stdx::mutex _mutex; // Guards all members.
+ mutable stdx::mutex _mutex; // Guards _committedSnapshot.
boost::optional<Timestamp> _committedSnapshot;
- WT_SESSION* _session;
- WT_CONNECTION* _conn;
};
}