diff --git a/SConscript b/SConscript index 3a58fdf..e441c12 100644 --- a/SConscript +++ b/SConscript @@ -9,10 +9,13 @@ conf = Configure(env) if conf.CheckLibWithHeader("lz4", ["lz4.h","lz4hc.h"], "C", "LZ4_versionNumber();", autoadd=False ): dynamic_syslibdeps.append("lz4") -conf.Finish() env.InjectMongoIncludePaths() +env.InjectThirdParty(libraries=['s2',]) # for Encoder and Decoder + +conf.Finish() + env.Library( target= 'storage_rocks_base', source= [ @@ -29,6 +32,10 @@ env.Library( 'src/rocks_oplog_manager.cpp', 'src/rocks_begin_transaction_block.cpp', 'src/rocks_prepare_conflict.cpp', +# TODO(wolfkdy): move totdb files into a seperate compile-unit + 'src/totdb/totransaction_impl.cpp', + 'src/totdb/totransaction_db_impl.cpp', + 'src/totdb/totransaction_prepare_iterator.cpp', env.Idlc('src/rocks_parameters.idl')[0], env.Idlc('src/rocks_global_options.idl')[0], 'src/rocks_parameters.cpp', @@ -53,6 +60,7 @@ env.Library( '$BUILD_DIR/mongo/util/concurrency/ticketholder', '$BUILD_DIR/mongo/util/processinfo', '$BUILD_DIR/third_party/shim_snappy', + '$BUILD_DIR/third_party/s2/util/coding/coding', ], LIBDEPS_PRIVATE= [ '$BUILD_DIR/mongo/db/snapshot_window_options', @@ -132,3 +140,13 @@ env.CppUnitTest( '$BUILD_DIR/mongo/db/repl/replmocks', ], ) + +env.CppUnitTest( + target='totdb_test', + source=[ + 'src/totdb/totransaction_test.cpp', + ], + LIBDEPS=[ + 'storage_rocks_mock', + ], +) diff --git a/src/rocks_begin_transaction_block.h b/src/rocks_begin_transaction_block.h index 85528da..058fadb 100755 --- a/src/rocks_begin_transaction_block.h +++ b/src/rocks_begin_transaction_block.h @@ -29,8 +29,8 @@ #pragma once -#include -#include +#include "mongo/db/modules/rocks/src/totdb/totransaction.h" +#include "mongo/db/modules/rocks/src/totdb/totransaction_db.h" #include "mongo/base/status.h" #include "mongo/bson/timestamp.h" #include "mongo/db/storage/recovery_unit.h" diff --git a/src/rocks_compaction_scheduler.cpp b/src/rocks_compaction_scheduler.cpp index 1218d7d..5a7a0f1 100644 --- a/src/rocks_compaction_scheduler.cpp +++ b/src/rocks_compaction_scheduler.cpp @@ -53,8 +53,8 @@ #include #include #include -#include -#include +#include "mongo/db/modules/rocks/src/totdb/totransaction.h" +#include "mongo/db/modules/rocks/src/totdb/totransaction_db.h" namespace mongo { namespace { @@ -79,8 +79,12 @@ namespace mongo { virtual bool Filter(int level, const rocksdb::Slice& key, const rocksdb::Slice& existing_value, std::string* new_value, bool* value_changed) const { - bool filter = (key.compare(rocksdb::Slice(_until)) <= 0 && - key.compare(rocksdb::Slice(_from)) >= 0); + const rocksdb::Slice stripUserKey = + rocksdb::TOComparator::StripTimestampFromUserKey( + key, rocksdb::TOComparator::TimestampSize()); + + bool filter = (stripUserKey.compare(rocksdb::Slice(_until)) <= 0 && + stripUserKey.compare(rocksdb::Slice(_from)) >= 0); if (filter) { _compactionScheduler->addOplogCompactRemoved(); } else { @@ -189,7 +193,6 @@ namespace mongo { // schedule compact range operation for execution in _compactionThread void scheduleCompactOp(rocksdb::ColumnFamilyHandle* cf, const std::string& begin, const std::string& end, bool rangeDropped, uint32_t order, - const bool trimHistory, boost::optional>>); private: @@ -200,7 +203,6 @@ namespace mongo { std::string _end_str; bool _rangeDropped; uint32_t _order; - bool _trimHistory; boost::optional>> _notification; bool operator>(const CompactOp& other) const { return _order > other._order; } }; @@ -301,11 +303,11 @@ namespace mongo { void CompactionBackgroundJob::scheduleCompactOp( rocksdb::ColumnFamilyHandle* cf, const std::string& begin, const std::string& end, - bool rangeDropped, uint32_t order, bool trimHistory, + bool rangeDropped, uint32_t order, boost::optional>> notification) { { stdx::lock_guard lk(_compactionMutex); - _compactionQueue.push({cf, begin, end, rangeDropped, order, trimHistory, notification}); + _compactionQueue.push({cf, begin, end, rangeDropped, order, notification}); } _compactionWakeUp.notify_one(); } @@ -320,109 +322,113 @@ namespace mongo { const bool isOplog = NamespaceString::oplog(op._cf->GetName()); LOG(1) << "Starting compaction of cf: " << op._cf->GetName() << " range: " << (start ? start->ToString(true) : "") << " .. " - << (end ? end->ToString(true) : "") << " (rangeDropped is " << op._rangeDropped - << ")" - << " (isOplog is " << isOplog << ")" - << " (trimHistory is " << op._trimHistory << ")"; - - if (op._trimHistory) { - invariant(!start && !end && !op._rangeDropped && !isOplog); - s = _db->RollbackToStable(op._cf); - } else { - if (op._rangeDropped || isOplog) { - std::vector beforeDelFiles; - std::vector afterDelFiles; - std::vector diffFiles; - auto queryDelFilesInRange = [&]() -> std::vector { - if (op._start_str.empty() && op._end_str.empty()) { - return {}; + << (end ? end->ToString(true) : "") + << " (rangeDropped is " << op._rangeDropped << ")" + << " (isOplog is " << isOplog << ")"; + + if (op._rangeDropped || isOplog) { + std::vector beforeDelFiles; + std::vector afterDelFiles; + std::vector diffFiles; + auto queryDelFilesInRange = [&]() -> std::vector { + if (op._start_str.empty() && op._end_str.empty()) { + return {}; + } + std::vector toDelFiles; + std::vector allFiles; + _db->GetRootDB()->GetLiveFilesMetaData(&allFiles); + for (const auto& f : allFiles) { + if (!NamespaceString::oplog(f.column_family_name)) { + continue; } - std::vector toDelFiles; - std::vector allFiles; - _db->GetRootDB()->GetLiveFilesMetaData(&allFiles); - for (const auto& f : allFiles) { - if (!NamespaceString::oplog(f.column_family_name)) { - continue; - } - // [start, end] - if (!op._start_str.empty() && !op._end_str.empty()) { - if ((op._start_str <= f.smallestkey) && (f.largestkey <= op._end_str)) { - toDelFiles.push_back(f); - } - } else if (op._start_str.empty() && (f.largestkey <= op._end_str)) { - // [start, max() - toDelFiles.push_back(f); - } else if (op._end_str.empty() && (op._start_str <= f.smallestkey)) { - // min(), end] + // [start, end] + if (!op._start_str.empty() && !op._end_str.empty()) { + if ((op._start_str <= f.smallestkey) && (f.largestkey <= op._end_str)) { toDelFiles.push_back(f); - } else { - // skip } + } else if (op._start_str.empty() && (f.largestkey <= op._end_str)) { + // [start, max() + toDelFiles.push_back(f); + } else if (op._end_str.empty() && (op._start_str <= f.smallestkey)) { + // min(), end] + toDelFiles.push_back(f); + } else { + // skip } - return toDelFiles; - }; - if (isOplog) { - LOG(1) << "Before DeleteFilesInRange Stats: " << op._cf->GetName(); - beforeDelFiles = queryDelFilesInRange(); } + return toDelFiles; + }; + if (isOplog) { + LOG(1) << "Before DeleteFilesInRange Stats: " << op._cf->GetName(); + beforeDelFiles = queryDelFilesInRange(); + } + { + char start_ts_buf[sizeof(rocksdb::RocksTimeStamp)]; + char end_ts_buf[sizeof(rocksdb::RocksTimeStamp)]; + Encoder(start_ts_buf, sizeof(rocksdb::RocksTimeStamp)).put64(0); + Encoder(end_ts_buf, sizeof(rocksdb::RocksTimeStamp)).put64(mongo::Timestamp::max().asULL()); + std::string start_str(op._start_str); + std::string end_str(op._end_str); + start_str.append(start_ts_buf, sizeof(rocksdb::RocksTimeStamp)); + end_str.append(end_ts_buf, sizeof(rocksdb::RocksTimeStamp)); + rocksdb::Slice start_slice(start_str); + rocksdb::Slice end_slice(end_str); + rocksdb::Slice* start = !op._start_str.empty() ? &start_slice : nullptr; + rocksdb::Slice* end = !op._end_str.empty() ? &end_slice : nullptr; auto s1 = rocksdb::DeleteFilesInRange(_db, op._cf, start, end); if (!s1.ok()) { // Do not fail the entire procedure, since there is still chance // to purge the range below, in CompactRange log() << "Failed to delete files in compacted range: " << s1.ToString(); } + } - if (isOplog) { - LOG(1) << "After DeleteFilesInRange Stats: " << op._cf->GetName(); - afterDelFiles = queryDelFilesInRange(); - [&]() { - for (const auto& f : beforeDelFiles) { - invariant(NamespaceString::oplog(f.column_family_name)); - - auto vit = std::find_if(afterDelFiles.begin(), afterDelFiles.end(), - [&](const rocksdb::LiveFileMetaData& a) { - return a.name == f.name; - }); - - // not found - if (vit == afterDelFiles.end()) { - diffFiles.push_back(f); - } + if (isOplog) { + LOG(1) << "After DeleteFilesInRange Stats: " << op._cf->GetName(); + afterDelFiles = queryDelFilesInRange(); + [&]() { + for (const auto& f : beforeDelFiles) { + invariant(NamespaceString::oplog(f.column_family_name)); + + auto vit = std::find_if(afterDelFiles.begin(), afterDelFiles.end(), + [&](const rocksdb::LiveFileMetaData& a) { + return a.name == f.name; + }); + + // not found + if (vit == afterDelFiles.end()) { + diffFiles.push_back(f); } - }(); - auto oplogFilesStats = [&]() { - uint64_t oplogEntries = 0; - uint64_t oplogSizesum = 0; + } + }(); + auto oplogFilesStats = [&]() { + uint64_t oplogEntries = 0; + uint64_t oplogSizesum = 0; - for (const auto& f : diffFiles) { - invariant(NamespaceString::oplog(f.column_family_name)); + for (const auto& f : diffFiles) { + invariant(NamespaceString::oplog(f.column_family_name)); - oplogEntries += f.num_entries; - oplogSizesum += f.size; - } - return std::make_pair(oplogEntries, oplogSizesum); - }(); - _compactionScheduler->addOplogEntriesDeleted(oplogFilesStats.first); - _compactionScheduler->addOplogSizeDeleted(oplogFilesStats.second); - } + oplogEntries += f.num_entries; + oplogSizesum += f.size; + } + return std::make_pair(oplogEntries, oplogSizesum); + }(); + _compactionScheduler->addOplogEntriesDeleted(oplogFilesStats.first); + _compactionScheduler->addOplogSizeDeleted(oplogFilesStats.second); } + } - rocksdb::CompactRangeOptions compact_options; - compact_options.bottommost_level_compaction = - rocksdb::BottommostLevelCompaction::kForce; - // if auto-compaction runs parallelly, oplog compact-range may leave a hole. - compact_options.exclusive_manual_compaction = isOplog; - compact_options.ignore_pin_timestamp = isOplog; + rocksdb::CompactRangeOptions compact_options; + compact_options.bottommost_level_compaction = + rocksdb::BottommostLevelCompaction::kForce; + // if auto-compaction runs parallelly, oplog compact-range may leave a hole. + compact_options.exclusive_manual_compaction = isOplog; - s = _db->CompactRange(compact_options, op._cf, start, end); - } + s = _db->CompactRange(compact_options, op._cf, start, end); + if (!s.ok()) { - if (op._trimHistory) { - log() << "Failed to RollbackToStable: " << s.ToString(); - } else { - log() << "Failed to compact range: " << s.ToString(); - } + log() << "Failed to compact range: " << s.ToString(); if (op._notification != boost::none) { (*op._notification)->set(rocksToMongoStatus(s)); } @@ -454,6 +460,8 @@ namespace mongo { _compactionJob.reset(new CompactionBackgroundJob(db, this)); } + void RocksCompactionScheduler::stop() { _compactionJob.reset(); } + void RocksCompactionScheduler::reportSkippedDeletionsAboveThreshold(rocksdb::ColumnFamilyHandle* cf, const std::string& prefix) { bool schedule = false; @@ -482,9 +490,7 @@ namespace mongo { void RocksCompactionScheduler::compactAll() { // NOTE(wolfkdy): compactAll only compacts DefaultColumnFamily // oplog cf is handled in RocksRecordStore. - const bool trimHistory = false; - compact(_db->DefaultColumnFamily(), std::string(), std::string(), false, kOrderFull, - trimHistory, boost::none); + compact(_db->DefaultColumnFamily(), std::string(), std::string(), false, kOrderFull, boost::none); } Status RocksCompactionScheduler::compactOplog(rocksdb::ColumnFamilyHandle* cf, @@ -499,8 +505,7 @@ namespace mongo { _oplogDeleteUntil = std::make_pair(cf->GetID(), std::make_pair(begin, end)); } auto notification = std::make_shared>(); - const bool trimHistory = false; - compact(cf, begin, end, false, kOrderOplog, trimHistory, notification); + compact(cf, begin, end, false, kOrderOplog, notification); auto s = notification->get(); if (!s.isOK()) { LOG(0) << "compactOplog to " << rocksdb::Slice(end).ToString() << " failed " << s; @@ -508,38 +513,22 @@ namespace mongo { return s; } - Status RocksCompactionScheduler::rollbackToStable(rocksdb::ColumnFamilyHandle* cf) { - auto notification = std::make_shared>(); - const bool trimHistory = true; - compact(cf, std::string(), std::string(), false, kOrderOplog, trimHistory, notification); - auto s = notification->get(); - if (!s.isOK()) { - LOG(0) << "rollbackToStable failed: " << s; - } - return s; - } - void RocksCompactionScheduler::compactPrefix(rocksdb::ColumnFamilyHandle* cf, const std::string& prefix) { - bool trimHistory = false; - compact(cf, prefix, rocksGetNextPrefix(prefix), false, kOrderRange, trimHistory, - boost::none); + compact(cf, prefix, rocksGetNextPrefix(prefix), false, kOrderRange, boost::none); } void RocksCompactionScheduler::compactDroppedPrefix(rocksdb::ColumnFamilyHandle* cf, const std::string& prefix) { LOG(0) << "Compacting dropped prefix: " << rocksdb::Slice(prefix).ToString(true) << " from cf: " << cf->GetName(); - bool trimHistory = false; - compact(cf, prefix, rocksGetNextPrefix(prefix), true, kOrderDroppedRange, trimHistory, - boost::none); + compact(cf, prefix, rocksGetNextPrefix(prefix), true, kOrderDroppedRange, boost::none); } void RocksCompactionScheduler::compact( rocksdb::ColumnFamilyHandle* cf, const std::string& begin, const std::string& end, - bool rangeDropped, uint32_t order, const bool trimHistory, + bool rangeDropped, uint32_t order, boost::optional>> notification) { - _compactionJob->scheduleCompactOp(cf, begin, end, rangeDropped, order, trimHistory, - notification); + _compactionJob->scheduleCompactOp(cf, begin, end, rangeDropped, order, notification); } rocksdb::CompactionFilterFactory* RocksCompactionScheduler::createCompactionFilterFactory() { diff --git a/src/rocks_compaction_scheduler.h b/src/rocks_compaction_scheduler.h index 26fffa0..c867b0f 100644 --- a/src/rocks_compaction_scheduler.h +++ b/src/rocks_compaction_scheduler.h @@ -70,6 +70,7 @@ namespace mongo { ~RocksCompactionScheduler(); void start(rocksdb::TOTransactionDB* db, rocksdb::ColumnFamilyHandle* cf); + void stop(); static int getSkippedDeletionsThreshold() { return kSkippedDeletionsThreshold; } @@ -78,7 +79,6 @@ namespace mongo { // schedule compact range operation for execution in _compactionThread void compactAll(); Status compactOplog(rocksdb::ColumnFamilyHandle* cf, const std::string& begin, const std::string& end); - Status rollbackToStable(rocksdb::ColumnFamilyHandle* cf); rocksdb::CompactionFilterFactory* createCompactionFilterFactory(); std::unordered_map getDroppedPrefixes() const; @@ -111,7 +111,6 @@ namespace mongo { void compactDroppedPrefix(rocksdb::ColumnFamilyHandle* cf, const std::string& prefix); void compact(rocksdb::ColumnFamilyHandle* cf, const std::string& begin, const std::string& end, bool rangeDropped, uint32_t order, - const bool trimHistory, boost::optional>>); void droppedPrefixCompacted(const std::string& prefix, bool opSucceeded); diff --git a/src/rocks_counter_manager.cpp b/src/rocks_counter_manager.cpp index 3010e38..8b69dd3 100644 --- a/src/rocks_counter_manager.cpp +++ b/src/rocks_counter_manager.cpp @@ -57,12 +57,15 @@ namespace mongo { } } std::string value; - auto s = _db->Get(rocksdb::ReadOptions(), _cf, counterKey, &value); - if (s.IsNotFound()) { - return 0; + { + auto txn = _db->makeTxn(); + auto readopts = rocksdb::ReadOptions(); + auto s = txn->Get(readopts, _cf, counterKey, &value); + if (s.IsNotFound()) { + return 0; + } + invariantRocksOK(s); } - invariantRocksOK(s); - int64_t ret; invariant(sizeof(ret) == value.size()); memcpy(&ret, value.data(), sizeof(ret)); @@ -73,7 +76,7 @@ namespace mongo { void RocksCounterManager::updateCounter(const std::string& counterKey, long long count) { if (_crashSafe) { int64_t storage; - auto txn = _makeTxn(); + auto txn = _db->makeTxn(); invariantRocksOK(txn->Put(_cf, counterKey, _encodeCounter(count, &storage))); invariantRocksOK(txn->Commit()); } else { @@ -83,7 +86,7 @@ namespace mongo { if (_syncCounter >= kSyncEvery) { // let's sync this now. piggyback on writeBatch int64_t storage; - auto txn = _makeTxn(); + auto txn = _db->makeTxn(); for (const auto& counter : _counters) { invariantRocksOK( txn->Put(_cf, counter.first, _encodeCounter(counter.second, &storage))); @@ -100,7 +103,7 @@ namespace mongo { if (_counters.size() == 0) { return; } - auto txn = _makeTxn(); + auto txn = _db->makeTxn(); int64_t storage; for (const auto& counter : _counters) { invariantRocksOK(txn->Put(_cf, counter.first, _encodeCounter(counter.second, &storage))); @@ -115,9 +118,4 @@ namespace mongo { return rocksdb::Slice(reinterpret_cast(storage), sizeof(*storage)); } - std::unique_ptr RocksCounterManager::_makeTxn() { - rocksdb::WriteOptions options; - rocksdb::TOTransactionOptions txnOptions; - return std::unique_ptr(_db->BeginTransaction(options, txnOptions)); - } } // namespace mongo diff --git a/src/rocks_counter_manager.h b/src/rocks_counter_manager.h index d8e55cd..d0d0962 100644 --- a/src/rocks_counter_manager.h +++ b/src/rocks_counter_manager.h @@ -37,8 +37,8 @@ #include #include -#include -#include +#include "mongo/db/modules/rocks/src/totdb/totransaction.h" +#include "mongo/db/modules/rocks/src/totdb/totransaction_db.h" #include "mongo/base/string_data.h" #include "mongo/platform/mutex.h" @@ -61,8 +61,6 @@ namespace mongo { private: static rocksdb::Slice _encodeCounter(long long counter, int64_t* storage); - std::unique_ptr _makeTxn(); - rocksdb::TOTransactionDB* _db; // not owned rocksdb::ColumnFamilyHandle* _cf; // not owned diff --git a/src/rocks_engine.cpp b/src/rocks_engine.cpp index cf046c6..34fd002 100644 --- a/src/rocks_engine.cpp +++ b/src/rocks_engine.cpp @@ -73,14 +73,15 @@ #include "mongo/util/log.h" #include "mongo/util/processinfo.h" +#include "mongo/db/dbhelpers.h" #include "mongo/db/modules/rocks/src/rocks_parameters_gen.h" +#include "mongo_rate_limiter_checker.h" #include "rocks_counter_manager.h" #include "rocks_global_options.h" #include "rocks_index.h" #include "rocks_record_store.h" #include "rocks_recovery_unit.h" #include "rocks_util.h" -#include "mongo_rate_limiter_checker.h" #define ROCKS_TRACE log() #define LOG_FOR_RECOVERY(level) \ @@ -88,6 +89,14 @@ #define LOG_FOR_ROLLBACK(level) \ MONGO_LOG_COMPONENT(level, ::mongo::logger::LogComponent::kReplicationRollback) +#define ROCKS_ERR(a) \ + do { \ + s = (a); \ + if (!s.ok()) { \ + return rocksToMongoStatus(s); \ + } \ + } while (0) + namespace mongo { class RocksEngine::RocksJournalFlusher : public BackgroundJob { @@ -132,6 +141,20 @@ namespace mongo { namespace { TicketHolder openWriteTransaction(128); TicketHolder openReadTransaction(128); + rocksdb::TOComparator comparator; + rocksdb::TOComparator comparatorFake(0); + bool isNsEnableTimestamp(const StringData& ns) { + if (ns == "local.replset.minvalid" || ns == "local.oplog.rs") { + return true; + } + if (ns.startsWith("local.")) { + return false; + } + if (ns == "_mdb_catalog") { + return false; + } + return true; + } } // namespace ROpenWriteTransactionParam::ROpenWriteTransactionParam(StringData name, ServerParameterType spt) @@ -176,6 +199,7 @@ namespace mongo { // first four bytes are the default prefix 0 const std::string RocksEngine::kMetadataPrefix("\0\0\0\0metadata-", 13); + const std::string RocksEngine::kStablePrefix("\0\0\0\0stableTs-", 13); const int RocksEngine::kDefaultJournalDelayMillis = 100; @@ -313,7 +337,6 @@ namespace mongo { // open DB rocksdb::TOTransactionDB* db = nullptr; rocksdb::Status s; - std::vector columnFamilies; const bool newDB = [&]() { const auto path = boost::filesystem::path(_path) / "CURRENT"; @@ -322,14 +345,16 @@ namespace mongo { if (newDB) { // init manifest so list column families will not fail when db is empty. invariantRocksOK(rocksdb::TOTransactionDB::Open( - _options(false /* isOplog */), + _options(false /* isOplog */, false /* trimHistory */), rocksdb::TOTransactionDBOptions(rocksGlobalOptions.maxConflictCheckSizeMB), _path, - &db)); + kStablePrefix, &db)); invariantRocksOK(db->Close()); } const bool hasOplog = [&]() { - s = rocksdb::DB::ListColumnFamilies(_options(false /* isOplog */), _path, &columnFamilies); + std::vector columnFamilies; + s = rocksdb::DB::ListColumnFamilies( + _options(false /* isOplog */, false /* trimHIstory */), _path, &columnFamilies); invariantRocksOK(s); auto it = std::find(columnFamilies.begin(), columnFamilies.end(), @@ -341,24 +366,38 @@ namespace mongo { if (!hasOplog) { rocksdb::ColumnFamilyHandle* cf = nullptr; invariantRocksOK(rocksdb::TOTransactionDB::Open( - _options(false /* isOplog */), + _options(false /* isOplog */, false /* trimHistory */), rocksdb::TOTransactionDBOptions(rocksGlobalOptions.maxConflictCheckSizeMB), _path, - &db)); - invariantRocksOK(db->CreateColumnFamily(_options(true /* isOplog */), - NamespaceString::kRsOplogNamespace.toString(), - &cf)); + kStablePrefix, &db)); + invariantRocksOK( + db->CreateColumnFamily(_options(true /* isOplog */, false /* trimHistory */), + NamespaceString::kRsOplogNamespace.toString(), &cf)); invariantRocksOK(db->DestroyColumnFamilyHandle(cf)); invariantRocksOK(db->Close()); log() << "init oplog cf success"; } std::vector cfs; + + std::vector open_cfds = { + {rocksdb::kDefaultColumnFamilyName, + _options(false /* isOplog */, false /* trimHistory */)}, + {NamespaceString::kRsOplogNamespace.toString(), + _options(true /* isOplog */, false /* trimHistory */)}}; + + const bool trimHistory = true; + std::vector trim_cfds = { + {rocksdb::kDefaultColumnFamilyName, _options(false /* isOplog */, trimHistory)}, + {NamespaceString::kRsOplogNamespace.toString(), + _options(true /* isOplog */, trimHistory)}}; + + const std::string trimTs = ""; + s = rocksdb::TOTransactionDB::Open( - _options(false /* isOplog */), + _options(false /* isOplog */, false /* trimHistory */), rocksdb::TOTransactionDBOptions(rocksGlobalOptions.maxConflictCheckSizeMB), _path, - {{rocksdb::kDefaultColumnFamilyName, _options(false /* isOplog */)}, - {NamespaceString::kRsOplogNamespace.toString(), _options(true /* isOplog */)}}, - &cfs, &db); + open_cfds, &cfs, trim_cfds, trimHistory, kStablePrefix, &db); + invariantRocksOK(s); invariant(cfs.size() == 2); invariant(cfs[0]->GetName() == rocksdb::kDefaultColumnFamilyName); @@ -366,12 +405,6 @@ namespace mongo { _db.reset(db); _defaultCf.reset(cfs[0]); _oplogCf.reset(cfs[1]); - - rocksdb::RocksTimeStamp ts(0); - auto status = _db->QueryTimeStamp(rocksdb::TimeStampType::kStable, &ts); - if (!status.IsNotFound() && Timestamp(ts).asULL() >= 1) { - invariantRocksOK(_db->RollbackToStable(_defaultCf.get())); - } } void RocksEngine::appendGlobalStats(BSONObjBuilder& b) { @@ -457,11 +490,7 @@ namespace mongo { _oldestActiveTransactionTimestampCallback = std::move(callback); }; - RecoveryUnit* RocksEngine::newRecoveryUnit() { - return new RocksRecoveryUnit(_db.get(), _oplogManager.get(), &_snapshotManager, - _compactionScheduler.get(), - _durabilityManager.get(), _durable, this); - } + RecoveryUnit* RocksEngine::newRecoveryUnit() { return new RocksRecoveryUnit(_durable, this); } Status RocksEngine::createRecordStore(OperationContext* opCtx, StringData ns, StringData ident, const CollectionOptions& options) { @@ -478,8 +507,10 @@ namespace mongo { // we also need to write out the new prefix to the database. this is just an // optimization std::string encodedPrefix(encodePrefix(oplogTrackerPrefix)); - s = rocksToMongoStatus( - _db->Put(rocksdb::WriteOptions(), _defaultCf.get(), encodedPrefix, rocksdb::Slice())); + auto txn = _db->makeTxn(); + rocksdb::Status s; + ROCKS_ERR(txn->Put(encodedPrefix, rocksdb::Slice())); + ROCKS_ERR(txn->Commit()); } return s; } @@ -509,6 +540,9 @@ namespace mongo { params.isCapped ? (options.cappedMaxDocs ? options.cappedMaxDocs : -1) : -1; params.cappedCallback = nullptr; params.tracksSizeAdjustments = true; + if (isNsEnableTimestamp(params.ns)) { + rocksdb::TOTransaction::enableTimestamp(params.prefix); + } std::unique_ptr recordStore = stdx::make_unique(this, cf, opCtx, params); @@ -538,12 +572,16 @@ namespace mongo { // oplog have no indexes invariant(!desc->parentNS().isOplog()); + if (isNsEnableTimestamp(prefix)) { + rocksdb::TOTransaction::enableTimestamp(prefix); + } + RocksIndexBase* index; if (desc->unique()) { index = new RocksUniqueIndex(_db.get(), _defaultCf.get(), prefix, ident.toString(), Ordering::make(desc->keyPattern()), std::move(config), desc->parentNS().toString(), desc->indexName(), - desc->keyPattern(), desc->isPartial()); + desc->keyPattern(), desc->isPartial(), desc->isIdIndex()); } else { auto si = new RocksStandardIndex(_db.get(), _defaultCf.get(), prefix, ident.toString(), Ordering::make(desc->keyPattern()), std::move(config)); @@ -739,15 +777,16 @@ namespace mongo { BSONObjBuilder builder; - auto s = _db->Put(rocksdb::WriteOptions(), _defaultCf.get(), kMetadataPrefix + ident.toString(), - rocksdb::Slice(config.objdata(), config.objsize())); - - if (s.ok()) { - // As an optimization, add a key to the DB - std::string encodedPrefix(encodePrefix(prefix)); - s = _db->Put(rocksdb::WriteOptions(), _defaultCf.get(), encodedPrefix, rocksdb::Slice()); - } - + auto txn = _db->makeTxn(); + rocksdb::Status s; + ROCKS_ERR(txn->Put(kMetadataPrefix + ident.toString(), + rocksdb::Slice(config.objdata(), config.objsize()))); + ROCKS_ERR(txn->Commit()); + // As an optimization, add a key to the DB + std::string encodedPrefix(encodePrefix(prefix)); + auto txn1 = _db->makeTxn(); + ROCKS_ERR(txn1->Put(encodedPrefix, rocksdb::Slice())); + ROCKS_ERR(txn1->Commit()); return rocksToMongoStatus(s); } @@ -769,7 +808,7 @@ namespace mongo { return encodePrefix(config.getField("prefix").numberInt()); } - rocksdb::Options RocksEngine::_options(bool isOplog) const { + rocksdb::Options RocksEngine::_options(bool isOplog, bool trimHistory) const { // default options rocksdb::Options options; options.rate_limiter = _rateLimiter; @@ -778,6 +817,13 @@ namespace mongo { table_options.filter_policy.reset(rocksdb::NewBloomFilterPolicy(10, false)); table_options.block_size = 16 * 1024; // 16KB table_options.format_version = 2; + + if (isOplog && trimHistory) { + options.comparator = &comparatorFake; + options.disable_auto_compactions = true; + } else { + options.comparator = &comparator; + } options.table_factory.reset(rocksdb::NewBlockBasedTableFactory(table_options)); options.write_buffer_size = rocksGlobalOptions.writeBufferSize; @@ -797,8 +843,6 @@ namespace mongo { options.hard_pending_compaction_bytes_limit = static_cast(rocksGlobalOptions.hardPendingCompactionMBLimit) * 1024 * 1024; options.target_file_size_base = 64 * 1024 * 1024; // 64MB - options.soft_rate_limit = 2.5; - options.hard_rate_limit = 3; options.level_compaction_dynamic_level_bytes = true; options.max_bytes_for_level_base = rocksGlobalOptions.maxBytesForLevelBase; // This means there is no limit on open files. Make sure to always set ulimit so that it can @@ -934,7 +978,10 @@ namespace mongo { rocksdb::RocksTimeStamp ts(oldestTimestamp.asULL()); if (force) { - invariantRocksOK(_db->SetTimeStamp(rocksdb::TimeStampType::kOldest, ts, force)); + { + rocksdb::ShouldNotCheckOldestTsBlock shouldNotCheckOldestTsblock(&comparator, ts); + invariantRocksOK(_db->SetTimeStamp(rocksdb::TimeStampType::kOldest, ts, force)); + } invariantRocksOK(_db->SetTimeStamp(rocksdb::TimeStampType::kCommitted, ts, force)); _oldestTimestamp.store(oldestTimestamp.asULL()); LOG(2) << "oldest_timestamp and commit_timestamp force set to " << oldestTimestamp; @@ -1001,25 +1048,56 @@ namespace mongo { << initialDataTS.toString() << ", Stable timestamp: " << stableTS.toString()); } + invariant(!_oplogManager->isRunning()); + LOG_FOR_ROLLBACK(0) << "RocksEngine::RecoverToStableTimestamp oplogManager is halt."; + + _oplogManager->init(nullptr /* rocksdb::TOTransactionDB* */, nullptr /* RocksDurabilityManager* */); + LOG_FOR_ROLLBACK(0) << "RocksEngine::RecoverToStableTimestamp syncing size storer to disk."; _counterManager->sync(); + _counterManager.reset(); + LOG_FOR_ROLLBACK(0) << "RocksEngine::RecoverToStableTimestamp shutting down counterManager"; if (_journalFlusher) { _journalFlusher->shutdown(); _journalFlusher.reset(); } LOG_FOR_ROLLBACK(0) << "RocksEngine::RecoverToStableTimestamp shutting down journal"; + + _durabilityManager.reset(); + LOG_FOR_ROLLBACK(0) << "RocksEngine::RecoverToStableTimestamp shutting down durabilityManager"; + + _compactionScheduler->stop(); + LOG_FOR_ROLLBACK(0) << "RocksEngine::RecoverToStableTimestamp stop _compactionScheduler"; + + // close db + _defaultCf.reset(); + _oplogCf.reset(); + _db.reset(); + LOG_FOR_ROLLBACK(0) << "RocksEngine::RecoverToStableTimestamp shutting down rocksdb"; + + // compactionScheduler should be create before initDatabase, because + // options.compactionFactor need compactionScheduler + _compactionScheduler.reset(new RocksCompactionScheduler()); + LOG_FOR_ROLLBACK(0) + << "RocksEngine::RecoverToStableTimestamp shutting down _compactionScheduler"; + + _initDatabase(); + LOG_FOR_ROLLBACK(0) << "RocksEngine::RecoverToStableTimestamp open rocksdb"; + + _compactionScheduler->start(_db.get(), _defaultCf.get()); + + _durabilityManager.reset( + new RocksDurabilityManager(_db.get(), _durable, _defaultCf.get(), _oplogCf.get())); + + _oplogManager->init(_db.get(), _durabilityManager.get()); + const auto stableTimestamp = getStableTimestamp(); const auto initialDataTimestamp = getInitialDataTimestamp(); LOG_FOR_ROLLBACK(0) << "Rolling back to the stable timestamp. StableTimestamp: " << stableTimestamp << " Initial Data Timestamp: " << initialDataTimestamp; - auto s = _compactionScheduler->rollbackToStable(_defaultCf.get()); - if (!s.isOK()) { - return {ErrorCodes::UnrecoverableRollbackError, - str::stream() << "Error rolling back to stable. Err: " << s}; - } setInitialDataTimestamp(initialDataTimestamp); setStableTimestamp(stableTimestamp, false); @@ -1030,6 +1108,11 @@ namespace mongo { } _counterManager.reset(new RocksCounterManager(_db.get(), _defaultCf.get(), rocksGlobalOptions.crashSafeCounters)); + opCtx->setRecoveryUnit(std::unique_ptr(newRecoveryUnit()), + WriteUnitOfWork::RecoveryUnitState::kNotInUnitOfWork); + + // oplogManager will be oppend by oplog record store is open, no need open here + return {stableTimestamp}; } diff --git a/src/rocks_engine.h b/src/rocks_engine.h index 454ebf6..feab80a 100644 --- a/src/rocks_engine.h +++ b/src/rocks_engine.h @@ -135,6 +135,8 @@ namespace mongo { return (SnapshotManager*)&_snapshotManager; } + RocksSnapshotManager* getRocksSnapshotManager() { return &_snapshotManager; } + void setJournalListener(JournalListener* jl); void setStableTimestamp(Timestamp stableTimestamp, bool force) override; @@ -301,7 +303,7 @@ namespace mongo { StorageEngine::OldestActiveTransactionTimestampCallback _oldestActiveTransactionTimestampCallback; - rocksdb::Options _options(bool isOplog) const; + rocksdb::Options _options(bool isOplog, bool trimHisotry) const; void _initDatabase(); @@ -363,6 +365,7 @@ namespace mongo { std::unique_ptr _oplogManager; static const std::string kMetadataPrefix; + static const std::string kStablePrefix; std::unique_ptr _durabilityManager; class RocksJournalFlusher; diff --git a/src/rocks_index.cpp b/src/rocks_index.cpp index 66fa355..1bf2606 100755 --- a/src/rocks_index.cpp +++ b/src/rocks_index.cpp @@ -66,6 +66,7 @@ namespace mongo { static const int kKeyStringV1Version = 1; static const int kMinimumIndexVersion = kKeyStringV0Version; static const int kMaximumIndexVersion = kKeyStringV1Version; + static const std::string emptyItem(""); bool hasFieldNames(const BSONObj& obj) { BSONForEach(e, obj) { @@ -159,7 +160,7 @@ namespace mongo { UniqueBulkBuilder(rocksdb::ColumnFamilyHandle* cf, std::string prefix, Ordering ordering, KeyString::Version keyStringVersion, std::string collectionNamespace, std::string indexName, OperationContext* opCtx, bool dupsAllowed, - const BSONObj& keyPattern) + const BSONObj& keyPattern, bool isIdIndex) : _cf(cf), _prefix(std::move(prefix)), _ordering(ordering), @@ -169,16 +170,68 @@ namespace mongo { _opCtx(opCtx), _dupsAllowed(dupsAllowed), _keyString(keyStringVersion), - _keyPattern(keyPattern) {} + _keyPattern(keyPattern), + _isIdIndex(isIdIndex) {} StatusWith addKey(const BSONObj& newKey, const RecordId& loc) { + if (_isIdIndex) { + return addKeyTimestampUnsafe(newKey, loc); + } else { + return addKeyTimestampSafe(newKey, loc); + } + } + + StatusWith addKeyTimestampSafe(const BSONObj& newKey, + const RecordId& loc) { + // Do a duplicate check, but only if dups aren't allowed. + if (!_dupsAllowed) { + const int cmp = newKey.woCompare(_previousKey, _ordering); + if (cmp == 0) { + // Duplicate found! + return buildDupKeyErrorStatus(newKey, + NamespaceString(StringData(_collectionNamespace)), + _indexName, _keyPattern); + } else { + // _previousKey.isEmpty() is only true on the first call to addKey(). + // newKey must be > the last key + invariant(_previousKey.isEmpty() || cmp > 0); + } + } + + _keyString.resetToKey(newKey, _ordering, loc); + std::string prefixedKey(RocksIndexBase::_makePrefixedKey(_prefix, _keyString)); + std::string valueItem = _keyString.getTypeBits().isAllZeros() + ? emptyItem + : std::string(_keyString.getTypeBits().getBuffer(), + _keyString.getTypeBits().getSize()); + + auto ru = RocksRecoveryUnit::getRocksRecoveryUnit(_opCtx); + invariant(ru); + auto transaction = ru->getTransaction(); + invariant(transaction); + invariantRocksOK(transaction->Put(_cf, prefixedKey, valueItem)); + + // Don't copy the key again if dups are allowed. + if (!_dupsAllowed) { + _previousKey = newKey.getOwned(); + } + if (_keyString.getTypeBits().isLongEncoding()) + return StatusWith( + SpecialFormatInserted::LongTypeBitsInserted); + + return StatusWith( + SpecialFormatInserted::NoSpecialFormatInserted); + } + + StatusWith addKeyTimestampUnsafe(const BSONObj& newKey, + const RecordId& loc) { SpecialFormatInserted specialFormatInserted = SpecialFormatInserted::NoSpecialFormatInserted; - const int cmp = newKey.woCompare(_key, _ordering); + const int cmp = newKey.woCompare(_previousKey, _ordering); if (cmp != 0) { - if (!_key.isEmpty()) { // _key.isEmpty() is only true on the first call to - // addKey(). + if (!_previousKey.isEmpty()) { // _previousKey.isEmpty() is only true on the first + // call to addKey(). invariant(cmp > 0); // newKey must be > the last key // We are done with dups of the last key so we can insert it now. specialFormatInserted = doInsert(); @@ -194,11 +247,11 @@ namespace mongo { // If we get here, we are in the weird mode where dups are allowed on a unique // index, so add ourselves to the list of duplicate locs. This also replaces the - // _key which is correct since any dups seen later are likely to be newer. + // _previousKey which is correct since any dups seen later are likely to be newer. } - _key = newKey.getOwned(); - _keyString.resetToKey(_key, _ordering); + _previousKey = newKey.getOwned(); + _keyString.resetToKey(_previousKey, _ordering); _records.push_back(std::make_pair(loc, _keyString.getTypeBits())); return StatusWith( @@ -253,9 +306,10 @@ namespace mongo { std::string _indexName; OperationContext* _opCtx; const bool _dupsAllowed; - BSONObj _key; + BSONObj _previousKey; KeyString _keyString; BSONObj _keyPattern; + const bool _isIdIndex; std::vector> _records; }; @@ -381,7 +435,11 @@ namespace mongo { protected: // Called after _key has been filled in. Must not throw WriteConflictException. - virtual void updateLocAndTypeBits() = 0; + virtual void updateLocAndTypeBits() { + _loc = KeyString::decodeRecordIdAtEnd(_key.getBuffer(), _key.getSize()); + BufReader br(_valueSlice().data(), _valueSlice().size()); + _typeBits.resetFromBuffer(&br); + } boost::optional curr(RequestedInfo parts) const { if (_eof) { @@ -548,24 +606,23 @@ namespace mongo { : RocksCursorBase(opCtx, db, cf, prefix, forward, order, keyStringVersion) { iterator(); } - - virtual void updateLocAndTypeBits() { - _loc = KeyString::decodeRecordIdAtEnd(_key.getBuffer(), _key.getSize()); - BufReader br(_valueSlice().data(), _valueSlice().size()); - _typeBits.resetFromBuffer(&br); - } }; class RocksUniqueCursor final : public RocksCursorBase { public: - RocksUniqueCursor(OperationContext* opCtx, rocksdb::DB* db, rocksdb::ColumnFamilyHandle* cf, - std::string prefix, bool forward, Ordering order, - KeyString::Version keyStringVersion, std::string indexName) + RocksUniqueCursor(OperationContext* opCtx, rocksdb::DB* db, + rocksdb::ColumnFamilyHandle* cf, std::string prefix, bool forward, + Ordering order, KeyString::Version keyStringVersion, + std::string indexName, bool isIdIndex) : RocksCursorBase(opCtx, db, cf, prefix, forward, order, keyStringVersion), - _indexName(std::move(indexName)) {} + _indexName(std::move(indexName)), + _isIdIndex(isIdIndex) {} boost::optional seekExact(const BSONObj& key, RequestedInfo parts) override { + if (!_isIdIndex) { + return RocksCursorBase::seekExact(key, parts); + } _eof = false; _iterator.reset(); @@ -586,10 +643,11 @@ namespace mongo { } void updateLocAndTypeBits() { - // We assume that cursors can only ever see unique indexes in their "pristine" - // state, - // where no duplicates are possible. The cases where dups are allowed should hold - // sufficient locks to ensure that no cursor ever sees them. + // _id indexes remain at the old format + if (!_isIdIndex) { + RocksCursorBase::updateLocAndTypeBits(); + return; + } BufReader br(_valueSlice().data(), _valueSlice().size()); _loc = KeyString::decodeRecordId(&br); _typeBits.resetFromBuffer(&br); @@ -603,13 +661,14 @@ namespace mongo { private: std::string _indexName; + const bool _isIdIndex; }; } // namespace /// RocksIndexBase - RocksIndexBase::RocksIndexBase(rocksdb::DB* db, rocksdb::ColumnFamilyHandle* cf, std::string prefix, - std::string ident, - Ordering order, const BSONObj& config) + RocksIndexBase::RocksIndexBase(rocksdb::DB* db, rocksdb::ColumnFamilyHandle* cf, + std::string prefix, std::string ident, Ordering order, + const BSONObj& config) : _db(db), _cf(cf), _prefix(prefix), _ident(std::move(ident)), _order(order) { uint64_t storageSize; std::string nextPrefix = rocksGetNextPrefix(_prefix); @@ -666,33 +725,85 @@ namespace mongo { /// RocksUniqueIndex - RocksUniqueIndex::RocksUniqueIndex(rocksdb::DB* db, rocksdb::ColumnFamilyHandle* cf, std::string prefix, - std::string ident, Ordering order, const BSONObj& config, - std::string collectionNamespace, std::string indexName, - const BSONObj& keyPattern, bool partial) + RocksUniqueIndex::RocksUniqueIndex(rocksdb::DB* db, rocksdb::ColumnFamilyHandle* cf, + std::string prefix, std::string ident, Ordering order, + const BSONObj& config, std::string collectionNamespace, + std::string indexName, const BSONObj& keyPattern, + bool partial, bool isIdIdx) : RocksIndexBase(db, cf, prefix, ident, order, config), _collectionNamespace(std::move(collectionNamespace)), _indexName(std::move(indexName)), _keyPattern(keyPattern), - _partial(partial) {} + _partial(partial), + _isIdIndex(isIdIdx) {} std::unique_ptr RocksUniqueIndex::newCursor( OperationContext* opCtx, bool forward) const { return stdx::make_unique(opCtx, _db, _cf, _prefix, forward, _order, - _keyStringVersion, _indexName); + _keyStringVersion, _indexName, _isIdIndex); } SortedDataBuilderInterface* RocksUniqueIndex::getBulkBuilder(OperationContext* opCtx, bool dupsAllowed) { return new RocksIndexBase::UniqueBulkBuilder(_cf, _prefix, _order, _keyStringVersion, _collectionNamespace, _indexName, opCtx, - dupsAllowed, _keyPattern); + dupsAllowed, _keyPattern, _isIdIndex); } StatusWith RocksUniqueIndex::insert(OperationContext* opCtx, const BSONObj& key, const RecordId& loc, bool dupsAllowed) { + if (_isIdIndex) { + return _insertTimestampUnsafe(opCtx, key, loc, dupsAllowed); + } else { + return _insertTimestampSafe(opCtx, key, loc, dupsAllowed); + } + } + + bool RocksUniqueIndex::_keyExistsTimestampSafe(OperationContext* opCtx, const KeyString& key) { + std::unique_ptr iter; + iter.reset(RocksRecoveryUnit::getRocksRecoveryUnit(opCtx)->NewIterator(_cf, _prefix)); + auto s = rocksPrepareConflictRetry(opCtx, [&] { + iter->SeekPrefix(rocksdb::Slice(key.getBuffer(), key.getSize())); + return iter->status(); + }); + return iter->Valid(); + } + + StatusWith RocksUniqueIndex::_insertTimestampSafe( + OperationContext* opCtx, const BSONObj& key, const RecordId& loc, bool dupsAllowed) { + auto ru = RocksRecoveryUnit::getRocksRecoveryUnit(opCtx); + invariant(ru); + invariant(ru->getTransaction()); + if (!dupsAllowed) { + const KeyString encodedKey(_keyStringVersion, key, _order); + const std::string prefixedKey(RocksIndexBase::_makePrefixedKey(_prefix, encodedKey)); + invariantRocksOK(ru->getTransaction()->GetForUpdate(_cf, prefixedKey)); + if (_keyExistsTimestampSafe(opCtx, encodedKey)) { + return buildDupKeyErrorStatus(key, + NamespaceString(StringData(_collectionNamespace)), + _indexName, _keyPattern); + } + } + const KeyString tableKey(_keyStringVersion, key, _order, loc); + const std::string prefixedKey(RocksIndexBase::_makePrefixedKey(_prefix, tableKey)); + std::string valueItem = + tableKey.getTypeBits().isAllZeros() + ? emptyItem + : std::string(tableKey.getTypeBits().getBuffer(), tableKey.getTypeBits().getSize()); + invariantRocksOK(ROCKS_OP_CHECK(ru->getTransaction()->Put(_cf, prefixedKey, valueItem))); + _indexStorageSize.fetch_add(static_cast(prefixedKey.size()), + std::memory_order_relaxed); + + if (tableKey.getTypeBits().isLongEncoding()) + return StatusWith(SpecialFormatInserted::LongTypeBitsInserted); + + return StatusWith(SpecialFormatInserted::NoSpecialFormatInserted); + } + + StatusWith RocksUniqueIndex::_insertTimestampUnsafe( + OperationContext* opCtx, const BSONObj& key, const RecordId& loc, bool dupsAllowed) { dassert(opCtx->lockState()->isWriteLocked()); invariant(loc.isValid()); dassert(!hasFieldNames(key)); @@ -717,7 +828,8 @@ namespace mongo { value.appendTypeBits(encodedKey.getTypeBits()); } rocksdb::Slice valueSlice(value.getBuffer(), value.getSize()); - invariantRocksOK(ROCKS_OP_CHECK(ru->getTransaction()->Put(_cf, prefixedKey, valueSlice))); + invariantRocksOK( + ROCKS_OP_CHECK(ru->getTransaction()->Put(_cf, prefixedKey, valueSlice))); return StatusWith( SpecialFormatInserted::NoSpecialFormatInserted); } @@ -772,8 +884,32 @@ namespace mongo { return StatusWith(SpecialFormatInserted::NoSpecialFormatInserted); } + void RocksUniqueIndex::_unindexTimestampSafe(OperationContext* opCtx, const BSONObj& key, + const RecordId& loc, bool dupsAllowed) { + KeyString encodedKey(_keyStringVersion, key, _order, loc); + std::string prefixedKey(_makePrefixedKey(_prefix, encodedKey)); + + auto ru = RocksRecoveryUnit::getRocksRecoveryUnit(opCtx); + invariant(ru); + auto transaction = ru->getTransaction(); + invariant(transaction); + + invariantRocksOK(ROCKS_OP_CHECK(transaction->Delete(_cf, prefixedKey))); + _indexStorageSize.fetch_sub(static_cast(prefixedKey.size()), + std::memory_order_relaxed); + } + void RocksUniqueIndex::unindex(OperationContext* opCtx, const BSONObj& key, const RecordId& loc, bool dupsAllowed) { + if (_isIdIndex) { + _unindexTimestampUnsafe(opCtx, key, loc, dupsAllowed); + } else { + _unindexTimestampSafe(opCtx, key, loc, dupsAllowed); + } + } + + void RocksUniqueIndex::_unindexTimestampUnsafe(OperationContext* opCtx, const BSONObj& key, + const RecordId& loc, bool dupsAllowed) { KeyString encodedKey(_keyStringVersion, key, _order); std::string prefixedKey(_makePrefixedKey(_prefix, encodedKey)); @@ -877,6 +1013,16 @@ namespace mongo { } Status RocksUniqueIndex::dupKeyCheck(OperationContext* opCtx, const BSONObj& key) { + if (!_isIdIndex) { + KeyString encodedKey(_keyStringVersion, key, _order); + if (_keyExistsTimestampSafe(opCtx, encodedKey)) { + return buildDupKeyErrorStatus(key, + NamespaceString(StringData(_collectionNamespace)), + _indexName, _keyPattern); + } else { + return Status::OK(); + } + } KeyString encodedKey(_keyStringVersion, key, _order); std::string prefixedKey(_makePrefixedKey(_prefix, encodedKey)); @@ -913,8 +1059,8 @@ namespace mongo { /// RocksStandardIndex RocksStandardIndex::RocksStandardIndex(rocksdb::DB* db, rocksdb::ColumnFamilyHandle* cf, - std::string prefix, std::string ident, - Ordering order, const BSONObj& config) + std::string prefix, std::string ident, Ordering order, + const BSONObj& config) : RocksIndexBase(db, cf, prefix, ident, order, config), useSingleDelete(false) {} StatusWith RocksStandardIndex::insert(OperationContext* opCtx, diff --git a/src/rocks_index.h b/src/rocks_index.h index 0b8a119..7d4a2c6 100644 --- a/src/rocks_index.h +++ b/src/rocks_index.h @@ -103,9 +103,9 @@ namespace mongo { class RocksUniqueIndex : public RocksIndexBase { public: RocksUniqueIndex(rocksdb::DB* db, rocksdb::ColumnFamilyHandle* cf, std::string prefix, - std::string ident, Ordering order, - const BSONObj& config, std::string collectionNamespace, - std::string indexName, const BSONObj& keyPattern, bool partial = false); + std::string ident, Ordering order, const BSONObj& config, + std::string collectionNamespace, std::string indexName, + const BSONObj& keyPattern, bool partial = false, bool isIdIdx = false); virtual StatusWith insert(OperationContext* opCtx, const BSONObj& key, const RecordId& loc, @@ -121,10 +121,29 @@ namespace mongo { bool dupsAllowed) override; private: + StatusWith _insertTimestampSafe(OperationContext* opCtx, + const BSONObj& key, + const RecordId& loc, + bool dupsAllowed); + + StatusWith _insertTimestampUnsafe(OperationContext* opCtx, + const BSONObj& key, + const RecordId& loc, + bool dupsAllowed); + + void _unindexTimestampUnsafe(OperationContext* opCtx, const BSONObj& key, + const RecordId& loc, bool dupsAllowed); + + void _unindexTimestampSafe(OperationContext* opCtx, const BSONObj& key, const RecordId& loc, + bool dupsAllowed); + + bool _keyExistsTimestampSafe(OperationContext* opCtx, const KeyString& prefixedKey); + std::string _collectionNamespace; std::string _indexName; const BSONObj _keyPattern; const bool _partial; + const bool _isIdIndex; }; class RocksStandardIndex : public RocksIndexBase { diff --git a/src/rocks_index_test.cpp b/src/rocks_index_test.cpp index 633ba7d..d962015 100644 --- a/src/rocks_index_test.cpp +++ b/src/rocks_index_test.cpp @@ -69,22 +69,17 @@ namespace mongo { IndexDescriptor::IndexVersion::kV2); if (unique) { return stdx::make_unique( - _engine.getDB(), _engine.getDefaultCf_ForTest(), - "prefix", "ident", _order, configBuilder.obj(), - "test.rocks", "testIndex", BSONObj(), partial); + _engine.getDB(), _engine.getDefaultCf_ForTest(), "prefix", "ident", _order, + configBuilder.obj(), "test.rocks", "testIndex", BSONObj(), partial); } else { - return stdx::make_unique(_engine.getDB(), _engine.getDefaultCf_ForTest(), - "prefix", "ident", - _order, configBuilder.obj()); + return stdx::make_unique( + _engine.getDB(), _engine.getDefaultCf_ForTest(), "prefix", "ident", _order, + configBuilder.obj()); } } std::unique_ptr newRecoveryUnit() final { - return stdx::make_unique( - _engine.getDB(), _engine.getOplogManager(), - checked_cast(_engine.getSnapshotManager()), - _engine.getCompactionScheduler(), - _engine.getDurabilityManager(), true /* durale */, &_engine); + return stdx::make_unique(true /* durale */, &_engine); } private: diff --git a/src/rocks_oplog_manager.cpp b/src/rocks_oplog_manager.cpp index 8b0f4bf..8a1a288 100755 --- a/src/rocks_oplog_manager.cpp +++ b/src/rocks_oplog_manager.cpp @@ -56,6 +56,11 @@ namespace mongo { RocksDurabilityManager* durabilityManager) : _db(db), _kvEngine(kvEngine), _durabilityManager(durabilityManager) {} + void RocksOplogManager::init(rocksdb::TOTransactionDB* db, RocksDurabilityManager* durabilityManager) { + _db = db; + _durabilityManager = durabilityManager; + } + void RocksOplogManager::start(OperationContext* opCtx, RocksRecordStore* oplogRecordStore) { invariant(!_isRunning); auto reverseOplogCursor = diff --git a/src/rocks_oplog_manager.h b/src/rocks_oplog_manager.h index 17eaf98..2e297e1 100644 --- a/src/rocks_oplog_manager.h +++ b/src/rocks_oplog_manager.h @@ -55,6 +55,8 @@ namespace mongo { RocksDurabilityManager* durabilityManager); virtual ~RocksOplogManager(){}; + void init(rocksdb::TOTransactionDB* db, RocksDurabilityManager* durabilityManager); + void start(OperationContext* opCtx, RocksRecordStore* oplogRecordStore); void halt(); diff --git a/src/rocks_prepare_conflict.h b/src/rocks_prepare_conflict.h index 37d2b32..e83978b 100644 --- a/src/rocks_prepare_conflict.h +++ b/src/rocks_prepare_conflict.h @@ -32,6 +32,7 @@ #include #include "mongo/db/curop.h" +#include "mongo/db/modules/rocks/src/totdb/totransaction_db.h" #include "mongo/db/prepare_conflict_tracker.h" #include "mongo/util/fail_point_service.h" #include "rocks_recovery_unit.h" @@ -76,9 +77,9 @@ namespace mongo { ON_BLOCK_EXIT([opCtx] { PrepareConflictTracker::get(opCtx).endPrepareConflict(); }); // If the failpoint is enabled, don't call the function, just simulate a conflict. rocksdb::Status s = MONGO_FAIL_POINT(RocksPrepareConflictForReads) - ? rocksdb::Status::PrepareConflict("failpoint simulate") + ? rocksdb::PrepareConflict() : ROCKS_READ_CHECK(f()); - if (!s.IsPrepareConflict()) return s; + if (!IsPrepareConflict(s)) return s; PrepareConflictTracker::get(opCtx).beginPrepareConflict(); @@ -136,10 +137,10 @@ namespace mongo { auto lastCount = recoveryUnit->getDurabilityManager()->getPrepareCommitOrAbortCount(); // If the failpoint is enabled, don't call the function, just simulate a conflict. rocksdb::Status s = MONGO_FAIL_POINT(RocksPrepareConflictForReads) - ? rocksdb::Status::PrepareConflict("failpoint simulate") + ? rocksdb::PrepareConflict() : ROCKS_READ_CHECK(f()); - if (!s.IsPrepareConflict()) return s; + if (!IsPrepareConflict(s)) return s; CurOp::get(opCtx)->debug().additiveMetrics.incrementPrepareReadConflicts(1); rocksPrepareConflictLog(attempts); diff --git a/src/rocks_record_store.cpp b/src/rocks_record_store.cpp index eb9c6b3..1ccdf76 100644 --- a/src/rocks_record_store.cpp +++ b/src/rocks_record_store.cpp @@ -60,6 +60,7 @@ #include "mongo/util/log.h" #include "mongo/util/str.h" +#include "mongo/db/modules/rocks/src/totdb/totransaction.h" #include "rocks_compaction_scheduler.h" #include "rocks_counter_manager.h" #include "rocks_durability_manager.h" @@ -108,7 +109,7 @@ namespace mongo { } RocksRecordStore::RocksRecordStore(RocksEngine* engine, rocksdb::ColumnFamilyHandle* cf, - OperationContext* opCtx, Params params) + OperationContext* opCtx, Params params) : RecordStore(params.ns), _engine(engine), _db(engine->getDB()), @@ -127,11 +128,11 @@ namespace mongo { _ident(params.ident), _dataSizeKey(std::string("\0\0\0\0", 4) + "datasize-" + params.ident), _numRecordsKey(std::string("\0\0\0\0", 4) + "numrecords-" + params.ident), - _cappedOldestKey(NamespaceString::oplog(params.ns) ? - std::string("\0\0\0\0", 4) + "cappedOldestKey-" + params.ident : ""), + _cappedOldestKey(NamespaceString::oplog(params.ns) + ? std::string("\0\0\0\0", 4) + "cappedOldestKey-" + params.ident + : ""), _shuttingDown(false), _tracksSizeAdjustments(params.tracksSizeAdjustments) { - LOG(1) << "opening collection " << params.ns << " with prefix " << rocksdb::Slice(_prefix).ToString(true); @@ -319,6 +320,10 @@ namespace mongo { return 0; } + if (!sizeRecoveryState(getGlobalServiceContext()).collectionNeedsSizeAdjustment(_ident)) { + return 0; + } + if (!_isCapped) { return 0; } @@ -504,7 +509,9 @@ namespace mongo { if (!NamespaceString::oplog(f.column_family_name)) { continue; } - auto largestTs = _prefixedKeyToTimestamp(f.largestkey); + auto largestKey = rocksdb::Slice(f.largestkey); + largestKey.remove_suffix(sizeof(rocksdb::RocksTimeStamp)); + auto largestTs = _prefixedKeyToTimestamp(largestKey); if (largestTs > persistedTimestamp) { continue; } @@ -519,7 +526,11 @@ namespace mongo { if (oplogTotalBytes - pendingDelSize > _cappedMaxSize + static_cast(f.size)) { pendingDelSize += f.size; pendingDelFiles.push_back(f); - maxDelKey = std::max(maxDelKey, f.largestkey); + auto largestKey = rocksdb::Slice(f.largestkey); + largestKey.remove_suffix(sizeof(rocksdb::RocksTimeStamp)); + if (largestKey.compare(rocksdb::Slice(maxDelKey)) > 0) { + maxDelKey = largestKey.ToString(); + } } } if (pendingDelFiles.size() < static_cast(minSSTFileCountReserved.load())) { @@ -565,7 +576,7 @@ namespace mongo { invariant(!_cappedOldestKey.empty()); _counterManager->updateCounter(_cappedOldestKey, _cappedOldestKeyHint.repr()); _counterManager->sync(); - LOG(0) << "cuixin: save _cappedOldestKeyHint: " << _cappedOldestKeyHint; + LOG(0) << "save _cappedOldestKeyHint: " << _cappedOldestKeyHint; { // for test _loadCountFromCountManager(opCtx); @@ -616,8 +627,8 @@ namespace mongo { auto s = opCtx->recoveryUnit()->setTimestamp(ts); invariant(s.isOK(), s.reason()); } - invariantRocksOK( - ROCKS_OP_CHECK(txn->Put(_cf, _makePrefixedKey(_prefix, loc), rocksdb::Slice(data, len)))); + invariantRocksOK(ROCKS_OP_CHECK( + txn->Put(_cf, _makePrefixedKey(_prefix, loc), rocksdb::Slice(data, len)))); _changeNumRecords(opCtx, 1); _increaseDataSize(opCtx, len); @@ -758,7 +769,9 @@ namespace mongo { std::string endString(_makePrefixedKey(_prefix, RecordId::max())); rocksdb::Slice beginRange(beginString); rocksdb::Slice endRange(endString); - return rocksToMongoStatus(_db->CompactRange(&beginRange, &endRange)); + // TODO(wolfkdy): support it + return Status(ErrorCodes::InvalidOptions, "not supported, use rocksdbCompact server paramter instead"); + // return rocksToMongoStatus(_db->CompactRange(&beginRange, &endRange)); } void RocksRecordStore::validate(OperationContext* opCtx, ValidateCmdLevel level, @@ -804,6 +817,8 @@ namespace mongo { void RocksRecordStore::updateStatsAfterRepair(OperationContext* opCtx, long long numRecords, long long dataSize) { + sizeRecoveryState(getGlobalServiceContext()) + .markCollectionAsAlwaysNeedsSizeAdjustment(_ident); RocksRecoveryUnit* ru = RocksRecoveryUnit::getRocksRecoveryUnit(opCtx); ru->resetDeltaCounters(); if (!_isOplog) { @@ -1045,12 +1060,16 @@ namespace mongo { return key; } - Timestamp RocksRecordStore::_prefixedKeyToTimestamp(const std::string& key) const { + Timestamp RocksRecordStore::_prefixedKeyToTimestamp(const rocksdb::Slice& key) const { rocksdb::Slice slice(key); slice.remove_prefix(_prefix.size()); return Timestamp(_makeRecordId(slice).repr()); } + Timestamp RocksRecordStore::_prefixedKeyToTimestamp(const std::string& key) const { + return _prefixedKeyToTimestamp(rocksdb::Slice(key)); + } + RecordId RocksRecordStore::_makeRecordId(const rocksdb::Slice& slice) { invariant(slice.size() == sizeof(int64_t)); int64_t repr = endian::bigToNative(*reinterpret_cast(slice.data())); @@ -1088,6 +1107,11 @@ namespace mongo { if (!_tracksSizeAdjustments) { return; } + + if (!sizeRecoveryState(getGlobalServiceContext()).collectionNeedsSizeAdjustment(_ident)) { + return; + } + RocksRecoveryUnit* ru = RocksRecoveryUnit::getRocksRecoveryUnit(opCtx); ru->incrementCounter(_numRecordsKey, &_numRecords, amount); } @@ -1096,6 +1120,9 @@ namespace mongo { if (!_tracksSizeAdjustments) { return; } + if (!sizeRecoveryState(getGlobalServiceContext()).collectionNeedsSizeAdjustment(_ident)) { + return; + } RocksRecoveryUnit* ru = RocksRecoveryUnit::getRocksRecoveryUnit(opCtx); ru->incrementCounter(_dataSizeKey, &_dataSize, amount); } diff --git a/src/rocks_record_store.h b/src/rocks_record_store.h index 446e9cf..fa595b5 100644 --- a/src/rocks_record_store.h +++ b/src/rocks_record_store.h @@ -277,6 +277,7 @@ namespace mongo { static rocksdb::Slice _makeKey(const RecordId& loc, int64_t* storage); static std::string _makePrefixedKey(const std::string& prefix, const RecordId& loc); Timestamp _prefixedKeyToTimestamp(const std::string& key) const; + Timestamp _prefixedKeyToTimestamp(const rocksdb::Slice& key) const; void _changeNumRecords(OperationContext* opCtx, int64_t amount); void _increaseDataSize(OperationContext* opCtx, int64_t amount); diff --git a/src/rocks_record_store_test.cpp b/src/rocks_record_store_test.cpp index f39a8c7..51c836c 100644 --- a/src/rocks_record_store_test.cpp +++ b/src/rocks_record_store_test.cpp @@ -36,8 +36,8 @@ #include #include #include -#include -#include +#include "mongo/db/modules/rocks/src/totdb/totransaction.h" +#include "mongo/db/modules/rocks/src/totdb/totransaction_db.h" #include "mongo/base/checked_cast.h" #include "mongo/base/init.h" diff --git a/src/rocks_recovery_unit.cpp b/src/rocks_recovery_unit.cpp index 14ab297..21acf71 100644 --- a/src/rocks_recovery_unit.cpp +++ b/src/rocks_recovery_unit.cpp @@ -43,6 +43,7 @@ #include "mongo/base/checked_cast.h" #include "mongo/db/concurrency/write_conflict_exception.h" +#include "mongo/db/modules/rocks/src/totdb/totransaction_db.h" #include "mongo/db/operation_context.h" #include "mongo/db/server_options.h" #include "mongo/db/storage/journal_listener.h" @@ -178,7 +179,7 @@ namespace mongo { } virtual rocksdb::Slice value() const { return _baseIterator->value(); } virtual rocksdb::Status status() const { - if (_baseIterator->status().IsPrepareConflict() && !Valid() && + if (IsPrepareConflict(_baseIterator->status()) && !Valid() && _baseIterator->Valid()) { // in this situation, we have seeked to neighbouring prefix invariant(!_baseIterator->key().starts_with(_prefixSlice)); @@ -255,18 +256,8 @@ namespace mongo { std::atomic RocksRecoveryUnit::_totalLiveRecoveryUnits(0); - RocksRecoveryUnit::RocksRecoveryUnit(rocksdb::TOTransactionDB* db, - RocksOplogManager* oplogManager, - RocksSnapshotManager* snapshotManager, - RocksCompactionScheduler* compactionScheduler, - RocksDurabilityManager* durabilityManager, - bool durable, RocksEngine* engine) - : _db(db), - _oplogManager(oplogManager), - _snapshotManager(snapshotManager), - _compactionScheduler(compactionScheduler), - _durabilityManager(durabilityManager), - _durable(durable), + RocksRecoveryUnit::RocksRecoveryUnit(bool durable, RocksEngine* engine) + : _durable(durable), _areWriteUnitOfWorksBanned(false), _isTimestamped(false), _timestampReadSource(ReadSource::kUnset), @@ -301,7 +292,7 @@ namespace mongo { } if (notifyDone) { - _durabilityManager->notifyPreparedUnitOfWorkHasCommittedOrAborted(); + getDurabilityManager()->notifyPreparedUnitOfWorkHasCommittedOrAborted(); } try { @@ -338,7 +329,7 @@ namespace mongo { } if (notifyDone) { - _durabilityManager->notifyPreparedUnitOfWorkHasCommittedOrAborted(); + getDurabilityManager()->notifyPreparedUnitOfWorkHasCommittedOrAborted(); } try { for (Changes::const_reverse_iterator it = _changes.rbegin(), end = _changes.rend(); @@ -392,7 +383,7 @@ namespace mongo { bool RocksRecoveryUnit::waitUntilDurable() { invariant(!_inUnitOfWork(), toString(_state)); - _durabilityManager->waitUntilDurable(false); + getDurabilityManager()->waitUntilDurable(false); return true; } @@ -486,7 +477,7 @@ namespace mongo { if (_isTimestamped) { if (!_orderedCommit) { - _oplogManager->triggerJournalFlush(); + getOplogManager()->triggerJournalFlush(); } _isTimestamped = false; } @@ -519,7 +510,7 @@ namespace mongo { Status RocksRecoveryUnit::obtainMajorityCommittedSnapshot() { invariant(_timestampReadSource == ReadSource::kMajorityCommitted); - auto snapshotName = _snapshotManager->getMinSnapshotForNextCommittedRead(); + auto snapshotName = getSnapshotManager()->getMinSnapshotForNextCommittedRead(); if (!snapshotName) { return {ErrorCodes::ReadConcernMajorityNotAvailableYet, "Read concern majority reads are currently not possible."}; @@ -603,9 +594,9 @@ namespace mongo { case ReadSource::kNoTimestamp: { if (_isOplogReader) { _oplogVisibleTs = - static_cast(_oplogManager->getOplogReadTimestamp()); + static_cast(getOplogManager()->getOplogReadTimestamp()); } - RocksBeginTxnBlock(_db, &_transaction, _prepareConflictBehavior, + RocksBeginTxnBlock(getDB(), &_transaction, _prepareConflictBehavior, _roundUpPreparedTimestamps) .done(); break; @@ -613,16 +604,19 @@ namespace mongo { case ReadSource::kMajorityCommitted: { // We reset _majorityCommittedSnapshot to the actual read timestamp used when the // transaction was started. - _majorityCommittedSnapshot = _snapshotManager->beginTransactionOnCommittedSnapshot( - _db, &_transaction, _prepareConflictBehavior, _roundUpPreparedTimestamps); + _majorityCommittedSnapshot = + getSnapshotManager()->beginTransactionOnCommittedSnapshot( + getDB(), &_transaction, _prepareConflictBehavior, + _roundUpPreparedTimestamps); break; } case ReadSource::kLastApplied: { - if (_snapshotManager->getLocalSnapshot()) { - _readAtTimestamp = _snapshotManager->beginTransactionOnLocalSnapshot( - _db, &_transaction, _prepareConflictBehavior, _roundUpPreparedTimestamps); + if (getSnapshotManager()->getLocalSnapshot()) { + _readAtTimestamp = getSnapshotManager()->beginTransactionOnLocalSnapshot( + getDB(), &_transaction, _prepareConflictBehavior, + _roundUpPreparedTimestamps); } else { - RocksBeginTxnBlock(_db, &_transaction, _prepareConflictBehavior, + RocksBeginTxnBlock(getDB(), &_transaction, _prepareConflictBehavior, _roundUpPreparedTimestamps) .done(); } @@ -640,7 +634,7 @@ namespace mongo { // Intentionally continue to the next case to read at the _readAtTimestamp. } case ReadSource::kProvided: { - RocksBeginTxnBlock txnOpen(_db, &_transaction, _prepareConflictBehavior, + RocksBeginTxnBlock txnOpen(getDB(), &_transaction, _prepareConflictBehavior, _roundUpPreparedTimestamps); auto status = txnOpen.setReadSnapshot(_readAtTimestamp); @@ -660,9 +654,9 @@ namespace mongo { } Timestamp RocksRecoveryUnit::_beginTransactionAtAllDurableTimestamp() { - RocksBeginTxnBlock txnOpen(_db, &_transaction, _prepareConflictBehavior, + RocksBeginTxnBlock txnOpen(getDB(), &_transaction, _prepareConflictBehavior, _roundUpPreparedTimestamps, RoundUpReadTimestamp::kRound); - Timestamp txnTimestamp = _oplogManager->fetchAllDurableValue(); + Timestamp txnTimestamp = getOplogManager()->fetchAllDurableValue(); auto status = txnOpen.setReadSnapshot(txnTimestamp); invariant(status.isOK(), status.reason()); auto readTimestamp = txnOpen.getTimestamp(); @@ -672,8 +666,8 @@ namespace mongo { Timestamp RocksRecoveryUnit::_beginTransactionAtNoOverlapTimestamp( std::unique_ptr* txn) { - auto lastApplied = _snapshotManager->getLocalSnapshot(); - Timestamp allDurable = Timestamp(_oplogManager->fetchAllDurableValue()); + auto lastApplied = getSnapshotManager()->getLocalSnapshot(); + Timestamp allDurable = Timestamp(getOplogManager()->fetchAllDurableValue()); // When using timestamps for reads and writes, it's important that readers and writers don't // overlap with the timestamps they use. In other words, at any point in the system there @@ -706,8 +700,8 @@ namespace mongo { // should read afterward. Timestamp readTimestamp = (lastApplied) ? std::min(*lastApplied, allDurable) : allDurable; - RocksBeginTxnBlock txnOpen(_db, txn, _prepareConflictBehavior, _roundUpPreparedTimestamps, - RoundUpReadTimestamp::kRound); + RocksBeginTxnBlock txnOpen(getDB(), txn, _prepareConflictBehavior, + _roundUpPreparedTimestamps, RoundUpReadTimestamp::kRound); auto status = txnOpen.setReadSnapshot(readTimestamp); fassert(51090, status); @@ -913,7 +907,7 @@ namespace mongo { invariant(getTransaction()); auto it = _transaction->GetIterator(options, cf); return new PrefixStrippingIterator(cf, std::move(prefix), _transaction.get(), it, - isOplog ? nullptr : _compactionScheduler, + isOplog ? nullptr : getCompactionScheduler(), std::move(upperBound)); } @@ -958,4 +952,25 @@ namespace mongo { RocksRecoveryUnit* RocksRecoveryUnit::getRocksRecoveryUnit(OperationContext* opCtx) { return checked_cast(opCtx->recoveryUnit()); } + + rocksdb::TOTransactionDB* RocksRecoveryUnit::getDB() { + invariant(_engine); + return _engine->getDB(); + } + RocksOplogManager* RocksRecoveryUnit::getOplogManager() { + invariant(_engine); + return _engine->getOplogManager(); + } + RocksSnapshotManager* RocksRecoveryUnit::getSnapshotManager() { + invariant(_engine); + return _engine->getRocksSnapshotManager(); + } + RocksCompactionScheduler* RocksRecoveryUnit::getCompactionScheduler() { + invariant(_engine); + return _engine->getCompactionScheduler(); + } + RocksDurabilityManager* RocksRecoveryUnit::getDurabilityManager() { + invariant(_engine); + return _engine->getDurabilityManager(); + } } // namespace mongo diff --git a/src/rocks_recovery_unit.h b/src/rocks_recovery_unit.h index ab8c100..c004221 100644 --- a/src/rocks_recovery_unit.h +++ b/src/rocks_recovery_unit.h @@ -37,11 +37,12 @@ #include #include -#include -#include #include #include +#include "mongo/db/modules/rocks/src/totdb/totransaction.h" +#include "mongo/db/modules/rocks/src/totdb/totransaction_db.h" + #include "mongo/base/owned_pointer_vector.h" #include "mongo/db/record_id.h" #include "mongo/db/storage/recovery_unit.h" @@ -85,11 +86,7 @@ namespace mongo { RocksRecoveryUnit& operator=(const RocksRecoveryUnit&) = delete; public: - RocksRecoveryUnit(rocksdb::TOTransactionDB* db, RocksOplogManager* oplogManager, - RocksSnapshotManager* snapshotManager, - RocksCompactionScheduler* compactionScheduler, - RocksDurabilityManager* durabilityManager, - bool durable, RocksEngine* engine); + RocksRecoveryUnit(bool durable, RocksEngine* engine); virtual ~RocksRecoveryUnit(); void beginUnitOfWork(OperationContext* opCtx) override; @@ -173,8 +170,7 @@ namespace mongo { void resetDeltaCounters(); RocksRecoveryUnit* newRocksRecoveryUnit() { - return new RocksRecoveryUnit(_db, _oplogManager, _snapshotManager, - _compactionScheduler, _durabilityManager, _durable, _engine); + return new RocksRecoveryUnit(_durable, _engine); } struct Counter { @@ -191,16 +187,12 @@ namespace mongo { static int getTotalLiveRecoveryUnits() { return _totalLiveRecoveryUnits.load(); } - rocksdb::DB* getDB() const { return _db; } - rocksdb::TOTransaction* getTransaction(); bool inActiveTxn() const { return _isActive(); } void assertInActiveTxn() const; - RocksDurabilityManager* getDurabilityManager() const { return _durabilityManager; } - boost::optional getOplogVisibilityTs(); /** @@ -250,6 +242,12 @@ namespace mongo { }; State getState_forTest() const; + rocksdb::TOTransactionDB* getDB(); + RocksOplogManager* getOplogManager(); + RocksSnapshotManager* getSnapshotManager(); + RocksCompactionScheduler* getCompactionScheduler(); + RocksDurabilityManager* getDurabilityManager(); + private: void _abort(); void _commit(); @@ -295,12 +293,6 @@ namespace mongo { */ bool _isCommittingOrAborting() const; - rocksdb::TOTransactionDB* _db; // not owned - RocksOplogManager* _oplogManager; // not owned - RocksSnapshotManager* _snapshotManager; // not owned - RocksCompactionScheduler* _compactionScheduler; // not owned - RocksDurabilityManager* _durabilityManager; // not owned - std::unique_ptr _transaction; bool _durable; diff --git a/src/rocks_recovery_unit_test.cpp b/src/rocks_recovery_unit_test.cpp index f20c5f8..622962f 100644 --- a/src/rocks_recovery_unit_test.cpp +++ b/src/rocks_recovery_unit_test.cpp @@ -36,8 +36,8 @@ #include #include #include -#include -#include +#include "mongo/db/modules/rocks/src/totdb/totransaction.h" +#include "mongo/db/modules/rocks/src/totdb/totransaction_db.h" #include "mongo/base/checked_cast.h" #include "mongo/base/init.h" diff --git a/src/rocks_snapshot_manager.h b/src/rocks_snapshot_manager.h index efa8fdb..fe0fe83 100644 --- a/src/rocks_snapshot_manager.h +++ b/src/rocks_snapshot_manager.h @@ -29,8 +29,8 @@ #include #include -#include -#include +#include "mongo/db/modules/rocks/src/totdb/totransaction.h" +#include "mongo/db/modules/rocks/src/totdb/totransaction_db.h" #include "mongo/db/storage/recovery_unit.h" #include "mongo/db/storage/snapshot_manager.h" diff --git a/src/totdb/totransaction.h b/src/totdb/totransaction.h new file mode 100644 index 0000000..bdcad35 --- /dev/null +++ b/src/totdb/totransaction.h @@ -0,0 +1,105 @@ +#pragma once +#ifndef ROCKSDB_LITE + +#include +#include +#include +#include +#include + +#include "rocksdb/comparator.h" +#include "rocksdb/db.h" +#include "rocksdb/status.h" + +namespace rocksdb { + +class Iterator; +class TransactionDB; +class WriteBatchWithIndex; + +using TransactionName = std::string; + +using TransactionID = uint64_t; + +//TimeStamp in rocksdb +using RocksTimeStamp = uint64_t; + +//TimeStamp Ordering Transaction +class TOTransaction { + public: + virtual ~TOTransaction() {} + + // set prepare timestamp for transaction, if the application set the prepare + // timestamp twice, an error will be returned + virtual Status SetPrepareTimeStamp(const RocksTimeStamp& timestamp) = 0; + + virtual Status SetCommitTimeStamp(const RocksTimeStamp& timestamp) = 0; + + virtual Status SetDurableTimeStamp(const RocksTimeStamp& timestamp) = 0; + + // set read timestamp for transaction, if the application set the commit timestamp twice, an error will be returned + virtual Status SetReadTimeStamp(const RocksTimeStamp& timestamp) = 0; + + virtual Status GetReadTimeStamp(RocksTimeStamp* timestamp) const = 0; + + virtual Status Prepare() = 0; + + virtual Status Commit(std::function* hook = nullptr) = 0; + + virtual Status Rollback() = 0; + + virtual Status Get(ReadOptions& options, + ColumnFamilyHandle* column_family, const Slice& key, + std::string* value) = 0; + + virtual Status Get(ReadOptions& options, const Slice& key, + std::string* value) = 0; + + virtual Iterator* GetIterator(ReadOptions& read_options) = 0; + + virtual Iterator* GetIterator(ReadOptions& read_options, + ColumnFamilyHandle* column_family) = 0; + + virtual Status Put(ColumnFamilyHandle* column_family, const Slice& key, + const Slice& value) = 0; + virtual Status Put(const Slice& key, const Slice& value) = 0; + + virtual Status Delete(ColumnFamilyHandle* column_family, const Slice& key) = 0; + virtual Status Delete(const Slice& key) = 0; + + virtual Status GetForUpdate(ColumnFamilyHandle* column_family, const Slice& key) = 0; + virtual Status GetForUpdate(const Slice& key) = 0; + + virtual WriteBatchWithIndex* GetWriteBatch() = 0; + + virtual Status SetName(const TransactionName& name) = 0; + + virtual TransactionName GetName() const { return name_; } + + virtual TransactionID GetID() const { return 0; } + + enum TOTransactionState { + kStarted = 0, + kPrepared = 1, + kCommitted = 2, + kRollback = 3, + }; + + virtual TOTransactionState GetState() const = 0; + + static void enableTimestamp(const std::string& prefix); + static bool isEnableTimestamp(const Slice& key); + static std::set timestampPrefixes; + static std::mutex prefixes_mutex; + + protected: + explicit TOTransaction(const DB* /*db*/) {} + TOTransaction() {} + + TransactionName name_; +}; + +} // namespace rocksdb + +#endif + diff --git a/src/totdb/totransaction_db.h b/src/totdb/totransaction_db.h new file mode 100644 index 0000000..cca9cba --- /dev/null +++ b/src/totdb/totransaction_db.h @@ -0,0 +1,205 @@ +#pragma once +#ifndef ROCKSDB_LITE + +#include +#include +#include +#include + +#include "mongo/db/modules/rocks/src/totdb/totransaction.h" +#include "mongo/util/str.h" +#include "rocksdb/comparator.h" +#include "rocksdb/db.h" +#include "rocksdb/utilities/stackable_db.h" +#include "third_party/s2/util/coding/coder.h" + +namespace rocksdb { + +//TimeStamp Ordering Transaction DB Options +#define DEFAULT_NUM_STRIPES 32 + +struct TOTransactionStat { + size_t max_conflict_bytes; + size_t cur_conflict_bytes; + size_t uk_num; + size_t ck_num; + size_t alive_txns_num; + size_t read_q_num; + size_t commit_q_num; + uint64_t oldest_ts; + uint64_t min_read_ts; + uint64_t max_commit_ts; + uint64_t committed_max_txnid; + uint64_t min_uncommit_ts; + uint64_t update_max_commit_ts_times; + uint64_t update_max_commit_ts_retries; + uint64_t txn_commits; + uint64_t txn_aborts; + uint64_t commit_without_ts_times; + uint64_t read_without_ts_times; + uint64_t read_with_ts_times; + uint64_t read_q_walk_len_sum; + uint64_t read_q_walk_times; + uint64_t commit_q_walk_len_sum; + uint64_t commit_q_walk_times; +}; + +struct TOTransactionDBOptions { + size_t num_stripes = DEFAULT_NUM_STRIPES; + size_t max_conflict_check_bytes_size = 200 * 1024 * 1024; + TOTransactionDBOptions(){}; + TOTransactionDBOptions(int max_conflict_check_bytes_size_mb) + : max_conflict_check_bytes_size(max_conflict_check_bytes_size_mb * 1024 * + 1024) {} +}; + +enum TimeStampType { + kOldest = 0, + kStable = 1, // kStable is not used + kCommitted = 2, + kAllCommitted = 3, + kTimeStampMax, +}; + +Status PrepareConflict(); +bool IsPrepareConflict(const Status& s); + +//TimeStamp Ordering Transaction Options +struct TOTransactionOptions { + size_t max_write_batch_size = 1000; + // Whether or not to round up to the oldest timestamp when the read timestamp + // is behind it. + bool timestamp_round_read = false; + // If true, The prepare timestamp will be rounded up to the oldest timestamp + // if found to be + // and the commit timestamp will be rounded up to the prepare timestamp if + // found to be earlier + // If false, Does not round up prepare and commit timestamp of a prepared + // transaction. + bool timestamp_round_prepared = false; + + bool read_only = false; + + bool ignore_prepare = false; +}; + +class TOComparator : public Comparator { + public: + TOComparator() : Comparator(sizeof(RocksTimeStamp)), cmp_without_ts_(BytewiseComparator()) {} + TOComparator(size_t ts_size) : Comparator(ts_size), cmp_without_ts_(BytewiseComparator()) {} + + static size_t TimestampSize() { return sizeof(RocksTimeStamp); } + const char* Name() const override { return "TOComparator"; } + + void FindShortSuccessor(std::string*) const override {} + + void FindShortestSeparator(std::string*, const Slice&) const override {} + + int Compare(const Slice& a, const Slice& b) const override { + invariant(timestamp_size() > 0); + int r = cmp_without_ts_->Compare(StripTimestampFromUserKey(a, timestamp_size()), + StripTimestampFromUserKey(b, timestamp_size())); + if (r != 0) { + return r; + } + return -CompareTimestamp( + Slice(a.data() + a.size() - timestamp_size(), timestamp_size()), + Slice(b.data() + b.size() - timestamp_size(), timestamp_size())); + } + + int CompareWithoutTimestamp(const Slice& a, bool a_has_ts, const Slice& b, + bool b_has_ts) const override { + invariant(timestamp_size() > 0); + if (a_has_ts) { + invariant(a.size() >= timestamp_size()); + } + if (b_has_ts) { + invariant(b.size() >= timestamp_size()); + } + Slice lhs = a_has_ts ? StripTimestampFromUserKey(a, timestamp_size()) : a; + Slice rhs = b_has_ts ? StripTimestampFromUserKey(b, timestamp_size()) : b; + return cmp_without_ts_->Compare(lhs, rhs); + } + + int CompareTimestamp(const Slice& ts1, const Slice& ts2) const override; + + static const Slice StripTimestampFromUserKey(const Slice& user_key, size_t ts_sz) { + Slice ret = user_key; + ret.remove_suffix(ts_sz); + return ret; + } + + void forceSetOldestTs(RocksTimeStamp ts); + void clearSetOldestTs(); + + private: + const Comparator* cmp_without_ts_; +}; + +class ShouldNotCheckOldestTsBlock { + ShouldNotCheckOldestTsBlock(const ShouldNotCheckOldestTsBlock&) = delete; + ShouldNotCheckOldestTsBlock& operator=(const ShouldNotCheckOldestTsBlock&) = delete; + +public: + explicit ShouldNotCheckOldestTsBlock(TOComparator* comparator, RocksTimeStamp ts) + : _to_comparator(comparator) { + invariant(_to_comparator); + _to_comparator->forceSetOldestTs(ts); + } + + ~ShouldNotCheckOldestTsBlock() { + _to_comparator->clearSetOldestTs(); + } + +private: + TOComparator* const _to_comparator; +}; + + + + +class TOTransactionDB : public StackableDB { + public: + static Status Open(const Options& options, + const TOTransactionDBOptions& txn_db_options, + const std::string& dbname, + const std::string stableTsKey, + TOTransactionDB** dbptr); + + static Status Open(const DBOptions& db_options, + const TOTransactionDBOptions& txn_db_options, + const std::string& dbname, + const std::vector& open_cfds, + std::vector* handles, + const std::vector& trim_cfds, + const bool trimHistory, + const std::string stableTsKey, + TOTransactionDB** dbptr); + + virtual void SetMaxConflictBytes(uint64_t bytes) = 0; + + // The lifecycle of returned pointer should be managed by the application level + virtual TOTransaction* BeginTransaction( + const WriteOptions& write_options, + const TOTransactionOptions& txn_options) = 0; + + virtual Status SetTimeStamp(const TimeStampType& ts_type, + const RocksTimeStamp& ts, bool force = false) = 0; + + virtual Status QueryTimeStamp(const TimeStampType& ts_type, RocksTimeStamp* timestamp) = 0; + + virtual Status Stat(TOTransactionStat* stat) = 0; + //virtual Status Close(); + + virtual std::unique_ptr makeTxn() = 0; + + protected: + //std::shared_ptr info_log_ = nullptr; + // To Create an ToTransactionDB, call Open() + explicit TOTransactionDB(DB* db) : StackableDB(db) {} +}; + +} // namespace rocksdb + +#endif + diff --git a/src/totdb/totransaction_db_impl.cpp b/src/totdb/totransaction_db_impl.cpp new file mode 100644 index 0000000..a6cff9a --- /dev/null +++ b/src/totdb/totransaction_db_impl.cpp @@ -0,0 +1,1337 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). + +#ifndef ROCKSDB_LITE + +#define MONGO_LOG_DEFAULT_COMPONENT ::mongo::logger::LogComponent::kStorage + +#include "third_party/s2/util/coding/coder.h" +#include "mongo/db/modules/rocks/src/totdb/totransaction_db_impl.h" +#include "mongo/db/modules/rocks/src/totdb/totransaction_prepare_iterator.h" +#include "mongo/util/log.h" +#include "mongo/bson/timestamp.h" + +namespace rocksdb { + +namespace { + +using ATN = TOTransactionImpl::ActiveTxnNode; +size_t TxnKeySize(const TxnKey& key) { + return sizeof(key.first) + key.second.size(); +} + +using BatchIteratorCB = std::function; +class BatchIterator : public WriteBatch::Handler { + public: + BatchIterator(BatchIteratorCB cb) : cb_(std::move(cb)) {} + + Status PutCF(uint32_t column_family_id, const Slice& key, + const Slice& value) override { + (void)value; + cb_(column_family_id, key); + return Status::OK(); + } + + Status DeleteCF(uint32_t column_family_id, const Slice& key) override { + cb_(column_family_id, key); + return Status::OK(); + } + + private: + BatchIteratorCB cb_; +}; + +thread_local bool skip_compare_oldest_ts = false; +thread_local RocksTimeStamp last_oldest_ts = 0; +thread_local RocksTimeStamp new_oldest_ts = 0; + +} // namespace + +const TxnKey PrepareHeap::sentinal_ = + std::make_pair(std::numeric_limits::max(), ""); + +PrepareHeap::PrepareHeap() { + // insert sentinal into map_ to simplify the reverse find logic + map_.insert({sentinal_, {}}); +} + +std::shared_ptr PrepareHeap::Find( + const TOTransactionImpl::ActiveTxnNode* core, const TxnKey& key, + TOTransaction::TOTransactionState* state) { + std::shared_lock rl(mutex_); + const auto it = map_.find(key); + if (it == map_.end()) { + return nullptr; + } + for (const auto& v : it->second) { + if (v->txn_id_ == core->txn_id_) { + *state = v->state_.load(std::memory_order_relaxed); + return v; + } + if (v->prepare_ts_ <= core->read_ts_) { + *state = v->state_.load(std::memory_order_relaxed); + return v; + } + } + return nullptr; +} + +void PrepareHeap::Insert(const std::shared_ptr& core) { + int64_t cnt = 0; + // NOTE: need to add PREPARING state to indicate the incomplete state ? + auto iter_cb = [&](uint32_t cf_id, const Slice& key) { + cnt++; + const auto lookup_key = + std::make_pair(cf_id, std::string(key.data(), key.size()-sizeof(RocksTimeStamp))); + auto it = map_.find(lookup_key); + if (it == map_.end()) { + map_.insert({lookup_key, {core}}); + } else { + invariant(it->second.size() > 0); + if (it->second.front()->txn_id_ == core->txn_id_) { + // NOTE(deyukong): there may be duplicated kvs in one write_batch + // because during a txn, same key may be modified several times. + // Though WriteBatchWithIndex sees the lastest one, WriteBatch.Iterate + // sees them all. + } else { + invariant(it->second.front()->txn_id_ < core->txn_id_); + auto state = it->second.front()->state_.load(std::memory_order_relaxed); + (void)state; + invariant(state == TOTransaction::TOTransactionState::kCommitted || + state == TOTransaction::TOTransactionState::kRollback); + it->second.push_front(core); + } + } + }; + BatchIterator it(iter_cb); + { + std::unique_lock wl(mutex_); + auto status = core->write_batch_.GetWriteBatch()->Iterate(&it); + (void)status; + invariant(status.ok() && cnt == core->write_batch_.GetWriteBatch()->Count()); + core->state_.store(TOTransaction::TOTransactionState::kPrepared); + } +} + +uint32_t PrepareHeap::Remove(ATN* core) { + uint32_t cnt = 0; + // NOTE: need to add PREPARING state to indicate the incomplete state ? + auto iter_cb = [&](uint32_t cf_id, const Slice& key) { + const auto lookup_key = + std::make_pair(cf_id, std::string(key.data(), key.size()-sizeof(RocksTimeStamp))); + auto it = map_.find(lookup_key); + if (it != map_.end()) { + invariant(!it->second.empty() && + it->second.front()->txn_id_ == core->txn_id_); + it->second.pop_front(); + cnt++; + if (it->second.empty()) { + map_.erase(it); + } + } + }; + BatchIterator it(iter_cb); + invariant(core->state_.load(std::memory_order_relaxed) == + TOTransaction::TOTransactionState::kPrepared); + { + std::unique_lock wl(mutex_); + auto status = core->write_batch_.GetWriteBatch()->Iterate(&it); + (void)status; + invariant(status.ok()); + } + return cnt; +} + +void PrepareHeap::Purge(TransactionID oldest_txn_id, RocksTimeStamp oldest_ts) { + (void)oldest_ts; + std::unique_lock wl(mutex_); + auto it = map_.begin(); + while (it->first != sentinal_) { + auto& lst = it->second; + auto lst_it = lst.begin(); + while (lst_it != lst.end()) { + auto state = (*lst_it)->state_.load(std::memory_order_relaxed); + invariant(state != TOTransaction::TOTransactionState::kRollback); + if (state == TOTransaction::TOTransactionState::kCommitted && + (*lst_it)->commit_txn_id_ <= + oldest_txn_id /* && lst_it->commit_ts_ < oldest_ts*/) { + // committed-txns with commit_ts_ >= oldest_ts and commit_txn_id <= + // oldest_txn_id are visible + // to new transactions, this purge process is a slice different from + // purging commit_map + lst.erase(lst_it, lst.end()); + break; + } else { + invariant(state == TOTransaction::TOTransactionState::kCommitted || + state == TOTransaction::TOTransactionState::kPrepared); + lst_it++; + } + } + if (lst.empty()) { + it = map_.erase(it); + } else { + it++; + } + } +} + +Status TOTransactionDB::Open(const Options& options, + const TOTransactionDBOptions& txn_db_options, + const std::string& dbname, const std::string stable_ts_key, TOTransactionDB** dbptr) { + DBOptions db_options(options); + ColumnFamilyOptions cf_options(options); + std::vector column_families; + column_families.push_back( + ColumnFamilyDescriptor(kDefaultColumnFamilyName, cf_options)); + std::vector handles; + std::vector trim_cfds; + const bool trimHistory = false; + Status s = Open(db_options, txn_db_options, dbname, column_families, &handles, trim_cfds, trimHistory, stable_ts_key, dbptr); + if (s.ok()) { + invariant(handles.size() == 1); + // I can delete the handle since DBImpl is always holding a reference to + // default column family + delete handles[0]; + } + + return s; +} + +Status TOTransactionDB::Open(const DBOptions& db_options, + const TOTransactionDBOptions& txn_db_options, + const std::string& dbname, + const std::vector& open_cfds, + std::vector* handles, + const std::vector& trim_cfds, + const bool trimHistory, + const std::string stable_ts_key, + TOTransactionDB** dbptr) { + Status s; + for (const auto& cfd : open_cfds) { + if (cfd.options.comparator == nullptr || cfd.options.comparator->timestamp_size() == 0) { + return Status::InvalidArgument("invalid comparator"); + } + } + + if (trimHistory) { + invariant(trim_cfds.size() > 0); + + std::string trim_ts; + // step1: get stableTs + // step2: close db + { + std::vector tmp_cfhs; + DB* tmp_db = nullptr; + s = DB::Open(db_options, dbname, open_cfds, &tmp_cfhs, &tmp_db); + if (!s.ok()) { + return s; + } + + char read_ts_buffer_[sizeof(RocksTimeStamp)]; + Encoder(read_ts_buffer_, sizeof(RocksTimeStamp)).put64(mongo::Timestamp::max().asULL()); + auto read_opt = rocksdb::ReadOptions(); + Slice readTs = rocksdb::Slice(read_ts_buffer_, sizeof(read_ts_buffer_)); + read_opt.timestamp = &readTs; + + s = tmp_db->Get(read_opt, stable_ts_key, &trim_ts); + for (auto handle : tmp_cfhs) { + auto tmp_s = tmp_db->DestroyColumnFamilyHandle(handle); + assert(tmp_s.ok()); + } + tmp_cfhs.clear(); + delete tmp_db; + if (!s.ok() && !s.IsNotFound()) { + return s; + } + } + + // step3: invoke rocksdb::OpenAndTrimHistory + // step4: close db + if(s.ok()) { + LOG(0) << "##### TOTDB recover to stableTs " << rocksdb::Slice(trim_ts).ToString(true); + std::vector tmp_cfhs; + DB* tmp_db = nullptr; + s = DB::OpenAndTrimHistory(db_options, dbname, trim_cfds, &tmp_cfhs, &tmp_db, trim_ts); + if (!s.ok()) { + return s; + } + for (auto handle : tmp_cfhs) { + auto tmp_s = tmp_db->DestroyColumnFamilyHandle(handle); + assert(tmp_s.ok()); + } + tmp_cfhs.clear(); + delete tmp_db; + } + } + + LOG(0) << "##### TOTDB open on normal mode #####"; + DB* db = nullptr; + s = DB::Open(db_options, dbname, open_cfds, handles, &db); + if (s.ok()) { + auto v = new TOTransactionDBImpl(db, txn_db_options, false, stable_ts_key); + v->StartBackgroundCleanThread(); + *dbptr = v; + } + + LOG(0) << "##### TOTDB open success #####"; + return s; +} + +Status TOTransactionDBImpl::UnCommittedKeys::RemoveKeyInLock( + const TxnKey& key, const size_t& stripe_num, + std::atomic* mem_usage) { + UnCommittedLockMapStripe* stripe = lock_map_stripes_.at(stripe_num); + auto iter = stripe->uncommitted_keys_map_.find(key); + invariant(iter != stripe->uncommitted_keys_map_.end()); + auto ccbytes = mem_usage->fetch_sub(TxnKeySize(key) + sizeof(iter->second), + std::memory_order_relaxed); + invariant(ccbytes >= 0); + (void)ccbytes; + stripe->uncommitted_keys_map_.erase(iter); + return Status::OK(); +} + +Status TOTransactionDBImpl::UnCommittedKeys::CheckKeyAndAddInLock( + const TxnKey& key, const TransactionID& txn_id, const size_t& stripe_num, + uint64_t max_mem_usage, std::atomic* mem_usage) { + UnCommittedLockMapStripe* stripe = lock_map_stripes_.at(stripe_num); + + auto iter = stripe->uncommitted_keys_map_.find(key); + if (iter != stripe->uncommitted_keys_map_.end()) { + // Check whether the key is modified by the same txn + if (iter->second != txn_id) { + LOG(2) << "TOTDB WriteConflict another txn id " << iter->second + << " is modifying key " << rocksdb::Slice(key.second).ToString(true); + return Status::Busy(); + } else { + return Status::OK(); + } + } + size_t delta_size = TxnKeySize(key) + sizeof(txn_id); + auto addedSize = mem_usage->load(std::memory_order_relaxed) + delta_size; + if (addedSize > max_mem_usage) { + LOG(2) << "TOTDB WriteConflict mem usage " << addedSize + << " is greater than limit " << max_mem_usage; + return Status::Busy(); + } + mem_usage->fetch_add(delta_size, std::memory_order_relaxed); + stripe->uncommitted_keys_map_.insert({key, txn_id}); + + return Status::OK(); +} + +size_t TOTransactionDBImpl::UnCommittedKeys::CountInLock() const { + size_t res = 0; + for (size_t i = 0; i < lock_map_stripes_.size(); ++i) { + res += lock_map_stripes_[i]->uncommitted_keys_map_.size(); + } + return res; +} + +Status TOTransactionDBImpl::CommittedKeys::AddKeyInLock( + const TxnKey& key, const TransactionID& commit_txn_id, + const RocksTimeStamp& prepare_ts, const RocksTimeStamp& commit_ts, + const size_t& stripe_num, std::atomic* mem_usage) { + CommittedLockMapStripe* stripe = lock_map_stripes_.at(stripe_num); + if (stripe->committed_keys_map_.find(key) == + stripe->committed_keys_map_.end()) { + mem_usage->fetch_add(TxnKeySize(key) + sizeof(prepare_ts) + + sizeof(commit_ts) + sizeof(commit_txn_id)); + } + + stripe->committed_keys_map_[key] = + std::make_tuple(commit_txn_id, prepare_ts, commit_ts); + + return Status::OK(); +} + +Status TOTransactionDBImpl::CommittedKeys::CheckKeyInLock( + const TxnKey& key, const TransactionID& txn_id, + const RocksTimeStamp& timestamp, const size_t& stripe_num) { + // padding to avoid false sharing + CommittedLockMapStripe* stripe = lock_map_stripes_.at(stripe_num); + + auto iter = stripe->committed_keys_map_.find(key); + if (iter != stripe->committed_keys_map_.end()) { + // Check whether some txn commits the key after txn_id began + const TransactionID& committed_txn_id = std::get<0>(iter->second); + // const RocksTimeStamp& prepare_ts = std::get<1>(iter->second); + const RocksTimeStamp& committed_ts = std::get<2>(iter->second); + if (committed_txn_id > txn_id) { + LOG(2) << "TOTDB WriteConflict a committed txn commit_id " << committed_txn_id + << " greater than my txnid " << txn_id; + return Status::Busy("SI conflict"); + } + // Find the latest committed txn for this key and its commit ts + if (committed_ts > timestamp) { + LOG(2) << "TOTDB WriteConflict a committed txn commit_ts " << committed_ts + << " greater than my read_ts " << timestamp; + return Status::Busy("timestamp conflict"); + } + } + return Status::OK(); +} + +size_t TOTransactionDBImpl::CommittedKeys::CountInLock() const { + size_t res = 0; + for (size_t i = 0; i < lock_map_stripes_.size(); ++i) { + res += lock_map_stripes_[i]->committed_keys_map_.size(); + } + return res; +} + +Status TOTransactionDBImpl::AddToActiveTxns(const std::shared_ptr& active_txn) { + active_txns_.insert({active_txn->txn_id_, active_txn}); + return Status::OK(); +} + +TOTransaction* TOTransactionDBImpl::BeginTransaction(const WriteOptions& write_options, + const TOTransactionOptions& txn_options) { + std::shared_ptr newActiveTxnNode = nullptr; + TOTransaction* newTransaction = nullptr; + { + std::lock_guard lock(active_txns_mutex_); + + TOTxnOptions totxn_option; + totxn_option.max_write_batch_size = txn_options.max_write_batch_size; + newActiveTxnNode = std::shared_ptr(new ATN); + newTransaction = new TOTransactionImpl(this, write_options, totxn_option, newActiveTxnNode); + + newActiveTxnNode->txn_id_ = TOTransactionImpl::GenTxnID(); + newActiveTxnNode->txn_snapshot = GetRootDB()->GetSnapshot(); + newActiveTxnNode->timestamp_round_prepared_ = + txn_options.timestamp_round_prepared; + newActiveTxnNode->timestamp_round_read_ = txn_options.timestamp_round_read; + newActiveTxnNode->read_only_ = txn_options.read_only; + newActiveTxnNode->ignore_prepare_ = txn_options.ignore_prepare; + // Add the transaction to active txns + AddToActiveTxns(newActiveTxnNode); + } + + LOG(2) << "TOTDB begin a txn id " << newActiveTxnNode->txn_id_ + << " snapshot " << newActiveTxnNode->txn_snapshot->GetSequenceNumber(); + return newTransaction; + +} + +Status TOTransactionDBImpl::CheckWriteConflict(const TxnKey& key, + const TransactionID& txn_id, + const RocksTimeStamp& readts) { + //if first check the uc key and ck busy ,it will need remove uc key ,so we check commit key first + auto stripe_num = GetStripe(key); + invariant(keys_mutex_.size() > stripe_num); + std::lock_guard lock(*keys_mutex_[stripe_num]); + // Check whether some txn commits the key after current txn started + // Check whether the commit ts of latest committed txn for key is less than my read ts + Status s = committed_keys_.CheckKeyInLock(key, txn_id, readts, stripe_num); + if (!s.ok()) { + LOG(2) << "TOTDB txn id " << txn_id + << " key " << rocksdb::Slice(key.second).ToString(true) << " conflict ck"; + return s; + } + + // Check whether the key is in uncommitted keys + // if not, add the key to uncommitted keys + s = uncommitted_keys_.CheckKeyAndAddInLock( + key, txn_id, stripe_num, + max_conflict_bytes_.load(std::memory_order_relaxed), + ¤t_conflict_bytes_); + if (!s.ok()) { + LOG(2) << "TOTDB txn id " << txn_id + << " key " << rocksdb::Slice(key.second).ToString(true) << " conflict uk"; + return s; + } + + return Status::OK(); +} + +void TOTransactionDBImpl::CleanCommittedKeys() { + TransactionID txn = 0; + RocksTimeStamp ts = 0; + + while(clean_job_.IsRunning()) { + // TEST_SYNC_POINT("TOTransactionDBImpl::CleanCommittedKeys::OnRunning"); + if (clean_job_.NeedToClean(&txn, &ts)) { + uint64_t totalNum = 0; + uint64_t removeKeys = 0; + for (size_t i = 0; i < num_stripes_; i++) { + std::unique_lock lock(*keys_mutex_[i]); + CommittedLockMapStripe* stripe = committed_keys_.lock_map_stripes_.at(i); + totalNum += stripe->committed_keys_map_.size(); + auto map_iter = stripe->committed_keys_map_.begin(); + while (map_iter != stripe->committed_keys_map_.end()) { + auto history = map_iter->second; + const TransactionID& txn_id = std::get<0>(history); + const RocksTimeStamp& prepare_ts = std::get<1>(history); + const RocksTimeStamp& commit_ts = std::get<2>(history); + if (txn_id <= txn && (commit_ts < ts || commit_ts == 0)) { + auto ccbytes = current_conflict_bytes_.fetch_sub( + TxnKeySize(map_iter->first) + sizeof(txn_id) + + sizeof(prepare_ts) + sizeof(commit_ts), + std::memory_order_relaxed); + invariant(ccbytes >= 0); + (void)ccbytes; + map_iter = stripe->committed_keys_map_.erase(map_iter); + removeKeys++; + } else { + map_iter++; + } + } + } + prepare_heap_.Purge(txn, ts); + clean_job_.FinishClean(txn, ts); + if (totalNum < 10000 || removeKeys * 8 < totalNum) { + std::this_thread::sleep_for(std::chrono::milliseconds(100)); + } + } else { + std::this_thread::sleep_for(std::chrono::milliseconds(100)); + } + } + // End run clean thread +} + +Status TOTransactionDBImpl::GetConsiderPrepare( + const std::shared_ptr& core, ReadOptions& options, + ColumnFamilyHandle* column_family, const Slice& key, std::string* value) { + RocksTimeStamp read_ts = core->read_ts_set_ + ? core->read_ts_ + : std::numeric_limits::max(); + (void)read_ts; + const auto lookup_key = std::make_pair(column_family->GetID(), + std::string(key.data(), key.size())); + TOTransaction::TOTransactionState state; + std::shared_ptr it = prepare_heap_.Find(core.get(), lookup_key, &state); + // 1) key is not in prepare_heap_, read data from self and db + if (it == nullptr) { + return core->write_batch_.GetFromBatchAndDB(GetRootDB(), options, + column_family, key, value); + } + + // 2) myself should not appear in prepare_heap because not allowed to read + // after prepared + invariant(it->txn_id_ != core->txn_id_); + invariant(it->prepare_ts_set_); + invariant(state != TOTransaction::TOTransactionState::kRollback); + + // 3) if the txn in prepare_ts_set_ is actually prepared + if (state == TOTransaction::TOTransactionState::kPrepared) { + invariant(it->prepare_ts_set_); + // when picking node from PrepareHeap, we've already considered ts + // relationship. + invariant(it->prepare_ts_ <= read_ts); + if (core->ignore_prepare_) { + SwapSnapshotGuard guard(&options, nullptr); + return core->write_batch_.GetFromBatchAndDB(GetRootDB(), options, + column_family, key, value); + } + return PrepareConflict(); + } + + invariant(state == TOTransaction::TOTransactionState::kCommitted); + SwapSnapshotGuard guard(&options, nullptr); + return core->write_batch_.GetFromBatchAndDB(GetRootDB(), options, + column_family, key, value); +} + +Iterator* TOTransactionDBImpl::NewIteratorConsiderPrepare( + const std::shared_ptr& core, ColumnFamilyHandle* column_family, + Iterator* db_iter) { + std::unique_ptr base_writebatch( + core->write_batch_.NewIteratorWithBase(column_family, db_iter)); + auto pmap_iter = std::unique_ptr( + new PrepareMapIterator(column_family, &prepare_heap_, core.get())); + auto merge_iter = + std::unique_ptr(new PrepareMergingIterator( + std::move(base_writebatch), std::move(pmap_iter))); + return new PrepareFilterIterator(GetRootDB(), column_family, core, std::move(merge_iter)); +} + +void TOTransactionDBImpl::StartBackgroundCleanThread() { + clean_thread_ = std::thread([this] { CleanCommittedKeys(); }); +} + +void TOTransactionDBImpl::AdvanceTS(RocksTimeStamp* pMaxToCleanTs) { + RocksTimeStamp max_to_clean_ts = 0; + + { + std::shared_lock rl(ts_meta_mutex_); + if (oldest_ts_ != nullptr) { + max_to_clean_ts = *oldest_ts_; + } + } + + { + std::shared_lock rl(read_ts_mutex_); + for (auto it = read_q_.begin(); it != read_q_.end();) { + if (it->second->state_.load() == TOTransaction::kStarted) { + invariant(it->second->read_ts_set_); + max_to_clean_ts = std::min(max_to_clean_ts, it->second->read_ts_); + break; + } + it++; + } + } + *pMaxToCleanTs = max_to_clean_ts; + + return; +} + +Status TOTransactionDBImpl::AddReadQueue(const std::shared_ptr& core, + const RocksTimeStamp& ts) { + RocksTimeStamp realTs = ts; + std::shared_lock rl(ts_meta_mutex_); + if (oldest_ts_ != nullptr) { + if (realTs < *oldest_ts_) { + if (!core->timestamp_round_read_) { + return Status::InvalidArgument("read-ts smaller than oldest ts"); + } + realTs = *oldest_ts_; + } else { + // otherwise, realTs >= *oldest_ts_ + } + } + + // take care of the critical area, read_ts_mutex_ is within + // ts_meta_mutex_ + std::unique_lock wl(read_ts_mutex_); + invariant(!core->read_ts_set_); + invariant(core->state_.load() == TOTransaction::kStarted); + // we have to clean commited/aboarted txns, right? + // we just start from the beginning and clean until the first active txn + // This is only a strategy and has nothing to do with the correctness + // You may design other strategies. + for (auto it = read_q_.begin(); it != read_q_.end();) { + if (it->second->state_.load() == TOTransaction::kStarted) { + break; + } + // TODO: add walk stat + it = read_q_.erase(it); + } + core->read_ts_set_ = true; + core->read_ts_ = realTs; + read_q_.insert({{realTs, core->txn_id_}, core}); + return Status::OK(); +} + +Status TOTransactionDBImpl::PublushTimeStamp(const std::shared_ptr& core) { + if (core->timestamp_published_) { + return Status::OK(); + } + + RocksTimeStamp ts; + if (core->commit_ts_set_) { + /* + * If we know for a fact that this is a prepared transaction and we only + * have a commit timestamp, don't add to the durable queue. If we poll + * all_durable after + * setting the commit timestamp of a prepared transaction, that prepared + * transaction + * should NOT be visible. + */ + if (core->state_.load(std::memory_order_relaxed) == + TOTransaction::kPrepared) { + return Status::OK(); + } + ts = core->commit_ts_; + } else { + // no reason to reach here + invariant(0); + } + + { + std::unique_lock wl(commit_ts_mutex_); + invariant(core->state_.load() == TOTransaction::kStarted || + core->state_.load() == TOTransaction::kPrepared); + for (auto it = commit_q_.begin(); it != commit_q_.end();) { + const auto state = it->second->state_.load(std::memory_order_relaxed); + if (state == TOTransaction::kStarted || + state == TOTransaction::kPrepared) { + break; + } + // TODO: add walk stat + it = commit_q_.erase(it); + } + commit_q_.insert({{ts, core->txn_id_}, core}); + core->timestamp_published_ = true; + } + return Status::OK(); +} + +Status TOTransactionDBImpl::SetDurableTimeStamp( + const std::shared_ptr& core, const RocksTimeStamp& ts) { + if (ts == 0) { + return Status::NotSupported("not allowed to set durable-ts to 0"); + } + if (core->state_.load(std::memory_order_relaxed) != + TOTransaction::kPrepared) { + return Status::InvalidArgument( + "durable timestamp should not be specified for non-prepared " + "transaction"); + } + if (!core->commit_ts_set_) { + return Status::InvalidArgument( + "commit timestamp is needed before the durable timestamp"); + } + bool has_oldest = false; + uint64_t tmp_oldest = 0; + { + std::shared_lock rl(ts_meta_mutex_); + has_oldest = (oldest_ts_ != nullptr); + tmp_oldest = has_oldest ? *oldest_ts_ : 0; + } + if (has_oldest && ts < tmp_oldest) { + return Status::InvalidArgument( + "durable timestamp is less than oldest timestamp"); + } + if (ts < core->commit_ts_) { + return Status::InvalidArgument( + "durable timestamp is less than commit timestamp"); + } + core->durable_ts_ = ts; + core->durable_ts_set_ = true; + return PublushTimeStamp(core); +} + +Status TOTransactionDBImpl::SetCommitTimeStamp(const std::shared_ptr& core, + const RocksTimeStamp& ts_orig) { + RocksTimeStamp ts = ts_orig; + if (ts == 0) { + return Status::NotSupported("not allowed to set committs to 0"); + } + bool has_oldest = false; + uint64_t tmp_oldest = 0; + { + std::shared_lock rl(ts_meta_mutex_); + has_oldest = (oldest_ts_ != nullptr); + tmp_oldest = has_oldest ? *oldest_ts_ : 0; + } + if (!core->prepare_ts_set_) { + if (has_oldest && ts < tmp_oldest) { + return Status::InvalidArgument("commit-ts smaller than oldest-ts"); + } + if (core->commit_ts_set_ && ts < core->first_commit_ts_) { + return Status::InvalidArgument("commit-ts smaller than first-commit-ts"); + } + } else { + if (core->prepare_ts_ > ts) { + if (!core->timestamp_round_prepared_) { + return Status::InvalidArgument("commit-ts smaller than prepare-ts"); + } + ts = core->prepare_ts_; + } + } + + invariant(!core->durable_ts_set_ || core->durable_ts_ == core->commit_ts_); + core->commit_ts_ = ts; + if (!core->commit_ts_set_) { + core->first_commit_ts_ = ts; + } + if (!core->durable_ts_set_) { + core->durable_ts_ = ts; + } + core->commit_ts_set_ = true; + return PublushTimeStamp(core); +} + +Status TOTransactionDBImpl::TxnAssertAfterReads( + const std::shared_ptr& core, const char* op, + const RocksTimeStamp& timestamp) { + uint64_t walk_cnt = 0; + std::shared_lock rl(read_ts_mutex_); + for (auto it = read_q_.rbegin(); it != read_q_.rend(); it++) { + walk_cnt++; + // txns with ignore_prepare_ == true should not be considered + if (it->second->state_.load() != TOTransaction::kStarted || + it->second->ignore_prepare_) { + continue; + } + if (it->second->txn_id_ == core->txn_id_) { + continue; + } + invariant(it->second->read_ts_set_); + read_q_walk_len_sum_.fetch_add(read_q_.size(), std::memory_order_relaxed); + read_q_walk_times_.fetch_add(walk_cnt, std::memory_order_relaxed); + if (it->second->read_ts_ >= timestamp) { + LOG(2) << op << " timestamp: " << timestamp + << " must be greater than the latest active readts " << it->second->read_ts_; + return Status::InvalidArgument( + "commit/prepare ts must be greater than latest readts"); + } else { + return Status::OK(); + } + } + + return Status::OK(); +} + +Status TOTransactionDBImpl::PrepareTransaction( + const std::shared_ptr& core) { + invariant(core->prepare_ts_set_); + prepare_heap_.Insert(core); + invariant(core->state_ == TOTransaction::TOTransactionState::kPrepared); + return Status::OK(); +} + +Status TOTransactionDBImpl::SetPrepareTimeStamp( + const std::shared_ptr& core, const RocksTimeStamp& timestamp) { + invariant(!core->prepare_ts_set_); + auto s = TxnAssertAfterReads(core, "prepare", timestamp); + if (!s.ok()) { + return s; + } + + RocksTimeStamp tmp_oldest = 0; + { + std::shared_lock rl(ts_meta_mutex_); + if (oldest_ts_ != nullptr) { + tmp_oldest = *oldest_ts_; + } + } + + // NOTE(deyukong): here wt did not set ts within ts_meta_mutex_, I just + // followed + if (timestamp < tmp_oldest) { + if (core->timestamp_round_prepared_) { + core->prepare_ts_ = tmp_oldest; + core->prepare_ts_set_ = true; + LOG(2) << "TOTDB round txn " << core->txn_id_ << " prepare_ts " << timestamp + << " to oldest " << tmp_oldest; + return Status::OK(); + } else { + return Status::InvalidArgument( + "prepareTs should not be less than oldest ts"); + } + } else { + core->prepare_ts_ = timestamp; + core->prepare_ts_set_ = true; + return Status::OK(); + } + + // void compiler complain, unreachable + invariant(0); + return Status::OK(); +} + +Status TOTransactionDBImpl::CommitTransaction( + const std::shared_ptr& core, const std::set& written_keys, + const std::set& get_for_updates) { + TransactionID max_to_clean_txn_id = 0; + RocksTimeStamp max_to_clean_ts = 0; + RocksTimeStamp candidate_durable_timestamp = 0; + RocksTimeStamp prev_durable_timestamp = 0; + bool update_durable_ts = false; + bool need_clean = false; + LOG(2) << "TOTDB start to commit txn id " << core->txn_id_ + << " commit ts " << core->commit_ts_; + // Update Active Txns + auto state = core->state_.load(std::memory_order_relaxed); + (void)state; + invariant(state == TOTransaction::kStarted || state == TOTransaction::kPrepared); + { + std::lock_guard lock(active_txns_mutex_); + + auto iter = active_txns_.find(core->txn_id_); + invariant(iter != active_txns_.end()); + + GetRootDB()->ReleaseSnapshot(iter->second->txn_snapshot); + core->commit_txn_id_ = TOTransactionImpl::GenTxnID(); + iter->second->state_.store(TOTransaction::kCommitted); + + iter = active_txns_.erase(iter); + + if (core->commit_txn_id_ > committed_max_txnid_) { + committed_max_txnid_ = core->commit_txn_id_; + } + + if (iter == active_txns_.begin()) { + need_clean = true; + if (!active_txns_.empty()) { + max_to_clean_txn_id = active_txns_.begin()->first - 1; + } else { + max_to_clean_txn_id = committed_max_txnid_; + } + } + } + + //it's okey to publish commit_ts a little later + if (core->durable_ts_set_) { + candidate_durable_timestamp = core->durable_ts_; + } else if (core->commit_ts_set_) { + candidate_durable_timestamp = core->commit_ts_; + } + + if (candidate_durable_timestamp != 0) { + prev_durable_timestamp = committed_max_ts_.load(std::memory_order_relaxed); + update_durable_ts = candidate_durable_timestamp > prev_durable_timestamp; + } + if (update_durable_ts) { + while (candidate_durable_timestamp > prev_durable_timestamp) { + update_max_commit_ts_times_.fetch_add(1, std::memory_order_relaxed); + if (committed_max_ts_.compare_exchange_strong( + prev_durable_timestamp, candidate_durable_timestamp)) { + has_commit_ts_.store(true); + break; + } + update_max_commit_ts_retries_.fetch_add(1, std::memory_order_relaxed); + prev_durable_timestamp = + committed_max_ts_.load(std::memory_order_relaxed); + } + } else { + commit_without_ts_times_.fetch_add(1, std::memory_order_relaxed); + } + + AdvanceTS(&max_to_clean_ts); + + // Move uncommited keys for this txn to committed keys + std::map> stripe_commit_keys_map; + auto keys_iter = written_keys.begin(); + while (keys_iter != written_keys.end()) { + auto stripe_num = GetStripe(*keys_iter); + if (stripe_commit_keys_map.find(stripe_num) == stripe_commit_keys_map.end()) { + stripe_commit_keys_map[stripe_num] = {}; + } + stripe_commit_keys_map[stripe_num].insert(std::move(*keys_iter)); + keys_iter++; + } + + // remove get_for_update keys + std::map> stripe_get_for_update_keys_map; + for (const auto& k : get_for_updates) { + auto stripe_num = GetStripe(k); + stripe_get_for_update_keys_map[stripe_num].insert(k); + } + + auto stripe_keys_iter = stripe_commit_keys_map.begin(); + while (stripe_keys_iter != stripe_commit_keys_map.end()) { + std::lock_guard lock(*keys_mutex_[stripe_keys_iter->first]); + // the key in one txn insert to the CK with the max commit ts + for (auto& key : stripe_keys_iter->second) { + committed_keys_.AddKeyInLock(key, core->commit_txn_id_, core->prepare_ts_, + core->commit_ts_, stripe_keys_iter->first, + ¤t_conflict_bytes_); + uncommitted_keys_.RemoveKeyInLock(key, stripe_keys_iter->first, ¤t_conflict_bytes_); + } + + stripe_keys_iter++; + } + + for (const auto& stripe : stripe_get_for_update_keys_map) { + std::lock_guard lock(*keys_mutex_[stripe.first]); + for (const auto& key : stripe.second) { + uncommitted_keys_.RemoveKeyInLock(key, stripe.first, ¤t_conflict_bytes_); + } + } + + LOG(2) << "TOTDB end commit txn id " << core->txn_id_ + << " cid " << core->commit_txn_id_ + << " commit ts " << core->commit_ts_; + // Clean committed keys + if (need_clean) { + // Clean committed keys async + // Clean keys whose commited txnid <= max_to_clean_txn_id + // and committed ts < max_to_clean_ts + LOG(2) << "TOTDB going to clean txnid " << max_to_clean_txn_id + << " ts " << max_to_clean_ts; + clean_job_.SetCleanInfo(max_to_clean_txn_id, max_to_clean_ts); + } + + txn_commits_.fetch_add(1, std::memory_order_relaxed); + if (core->read_ts_set_) { + read_with_ts_times_.fetch_add(1, std::memory_order_relaxed); + } else { + read_without_ts_times_.fetch_add(1, std::memory_order_relaxed); + } + + return Status::OK(); +} + +Status TOTransactionDBImpl::RollbackTransaction( + const std::shared_ptr& core, const std::set& written_keys, + const std::set& get_for_updates) { + LOG(2) << "TOTDB start to rollback txn id " << core->txn_id_; + auto state = core->state_.load(std::memory_order_relaxed); + invariant(state == TOTransaction::kStarted || state == TOTransaction::kPrepared); + // clean prepare_heap_ before set state to keep the invariant that no + // kRollback state in prepare_heap + if (state == TOTransaction::kPrepared) { + uint32_t remove_cnt = prepare_heap_.Remove(core.get()); + (void)remove_cnt; + invariant(remove_cnt == written_keys.size()); + } + // Remove txn for active txns + bool need_clean = false; + TransactionID max_to_clean_txn_id = 0; + RocksTimeStamp max_to_clean_ts = 0; + { + std::lock_guard lock(active_txns_mutex_); + + auto iter = active_txns_.find(core->txn_id_); + invariant(iter != active_txns_.end()); + iter->second->state_.store(TOTransaction::kRollback); + GetRootDB()->ReleaseSnapshot(iter->second->txn_snapshot); + iter = active_txns_.erase(iter); + + if (iter == active_txns_.begin()) { + need_clean = true; + if (!active_txns_.empty()) { + max_to_clean_txn_id = active_txns_.begin()->first - 1; + } else { + max_to_clean_txn_id = committed_max_txnid_; + } + } + } + // Calculation the min clean ts between oldest and the read ts + AdvanceTS(&max_to_clean_ts); + + // Remove written keys from uncommitted keys + std::map> stripe_keys_map; + for (const auto& key : written_keys) { + auto stripe_num = GetStripe(key); + stripe_keys_map[stripe_num].insert(key); + } + for (const auto& key : get_for_updates) { + auto stripe_num = GetStripe(key); + stripe_keys_map[stripe_num].insert(key); + } + + auto stripe_keys_iter = stripe_keys_map.begin(); + while (stripe_keys_iter != stripe_keys_map.end()) { + std::lock_guard lock(*keys_mutex_[stripe_keys_iter->first]); + + for (auto & key : stripe_keys_iter->second) { + uncommitted_keys_.RemoveKeyInLock(key, stripe_keys_iter->first, ¤t_conflict_bytes_); + } + stripe_keys_iter++; + } + + LOG(2) << "TOTDB end rollback txn id " << core->txn_id_; + + if (need_clean) { + LOG(2) << "TOTDB going to clean txnid " << max_to_clean_txn_id + << "ts " << max_to_clean_ts; + clean_job_.SetCleanInfo(max_to_clean_txn_id, max_to_clean_ts); + } + + txn_aborts_.fetch_add(1, std::memory_order_relaxed); + if (core->read_ts_set_) { + read_with_ts_times_.fetch_add(1, std::memory_order_relaxed); + } else { + read_without_ts_times_.fetch_add(1, std::memory_order_relaxed); + } + + return Status::OK(); +} + +Status TOTransactionDBImpl::SetTimeStamp(const TimeStampType& ts_type, + const RocksTimeStamp& ts, bool force) { + if (ts_type == kCommitted) { + // NOTE(deyukong): committed_max_ts_ is not protected by ts_meta_mutex_, + // its change is by atomic cas. So here it's meaningless to Wlock + // ts_meta_mutex_ it's app-level's duty to guarantee that when updating + // kCommitted-timestamp, there is no paralllel running txns that will also + // change it. + if (force) { + LOG(2) << "kCommittedTs force set from " << (has_commit_ts_.load() ? committed_max_ts_.load() : 0) + << " to " << ts; + } + std::shared_lock rl(ts_meta_mutex_); + if ((oldest_ts_ != nullptr && *oldest_ts_ > ts) && !force) { + return Status::InvalidArgument( + "kCommittedTs should not be less than kOldestTs"); + } + committed_max_ts_.store(ts); + has_commit_ts_.store(true); + return Status::OK(); + } + + if (ts_type == kOldest) { + // NOTE(deyukong): here we must take lock, every txn's readTs-setting + // has to be in the same critical area within the set of kOldest + uint64_t original = 0; + { + std::unique_lock wl(ts_meta_mutex_); + if ((oldest_ts_ != nullptr && *oldest_ts_ > ts) && !force) { + LOG(2) << "oldestTs can not travel back, oldest_ts from " << *oldest_ts_ + << " to " << ts + << " keep oldest_ts " << *oldest_ts_; + return Status::OK(); + } + original = (oldest_ts_ == nullptr) ? 0 : *oldest_ts_; + oldest_ts_.reset(new RocksTimeStamp(ts)); + } + if (force) { + LOG(2) << "kOldestTs force set from " << original << " to " << ts; + } + auto pin_ts = ts; + { + std::shared_lock rl(read_ts_mutex_); + uint64_t walk_cnt = 0; + for (auto it = read_q_.begin(); it != read_q_.end();) { + if (it->second->state_.load() == TOTransaction::kStarted) { + invariant(it->second->read_ts_set_); + pin_ts = std::min(pin_ts, it->second->read_ts_); + break; + } + it++; + walk_cnt++; + } + read_q_walk_len_sum_.fetch_add(read_q_.size(), std::memory_order_relaxed); + read_q_walk_times_.fetch_add(walk_cnt, std::memory_order_relaxed); + } + { + char buf[sizeof(RocksTimeStamp)]; + Encoder(buf, sizeof(pin_ts)).put64(pin_ts); + auto s = GetRootDB()->IncreaseFullHistoryTsLow(GetRootDB()->DefaultColumnFamily(), std::string(buf, sizeof(buf))); + if (!s.ok()) { + return s; + } + } + LOG(2) << "TOTDB set oldest ts type " << static_cast(ts_type) + << " value " << pin_ts; + return Status::OK(); + } + + if (ts_type == kStable) { + char val_buf[sizeof(RocksTimeStamp)]; + char ts_buf[sizeof(RocksTimeStamp)]; + Encoder(val_buf, sizeof(ts)).put64(ts); + Encoder(ts_buf, sizeof(ts)).put64(0); + auto write_ops = rocksdb::WriteOptions(); + write_ops.sync = true; + auto s = Put(write_ops, DefaultColumnFamily(), stable_ts_key_, rocksdb::Slice(ts_buf, sizeof(ts_buf)), + rocksdb::Slice(val_buf, sizeof(val_buf))); + if (!s.ok()) { + return s; + } + LOG(2) << "TOTDB set stable ts type " << static_cast(ts_type) + << " value " << ts; + return Status::OK(); + } + + return Status::InvalidArgument("invalid ts type"); +} + +Status TOTransactionDBImpl::QueryTimeStamp(const TimeStampType& ts_type, + RocksTimeStamp* timestamp) { + if (ts_type == kAllCommitted) { + if (!has_commit_ts_.load(/* seq_cst */)) { + return Status::NotFound("not found"); + } + auto tmp = committed_max_ts_.load(std::memory_order_relaxed); + std::shared_lock rl(commit_ts_mutex_); + uint64_t walk_cnt = 0; + for (auto it = commit_q_.begin(); it != commit_q_.end(); ++it) { + const auto state = it->second->state_.load(std::memory_order_relaxed); + if (state != TOTransaction::kStarted && + state != TOTransaction::kPrepared) { + walk_cnt++; + continue; + } + invariant(it->second->commit_ts_set_); + invariant(it->second->first_commit_ts_ > 0); + invariant(it->second->commit_ts_ >= it->second->first_commit_ts_); + tmp = std::min(tmp, it->first.first - 1); + break; + } + commit_q_walk_len_sum_.fetch_add(commit_q_.size(), std::memory_order_relaxed); + commit_q_walk_times_.fetch_add(walk_cnt, std::memory_order_relaxed); + *timestamp = tmp; + return Status::OK(); + } + if (ts_type == kOldest) { + // NOTE: query oldest is not a frequent thing, so I just + // take the rlock + std::shared_lock rl(ts_meta_mutex_); + // todo + std::string ts_holder; + auto s = GetRootDB()->GetFullHistoryTsLow(GetRootDB()->DefaultColumnFamily(), &ts_holder); + if (!s.ok()) { + return s; + } + invariant(ts_holder.size() == sizeof(RocksTimeStamp)); + *timestamp = Decoder(ts_holder.data(), ts_holder.size()).get64(); + oldest_ts_.reset(new RocksTimeStamp(*timestamp)); + LOG(2) << "TOTDB query TS type " << static_cast(ts_type) << " value " << *timestamp; + return Status::OK(); + } + if (ts_type == kStable) { + std::string ts_holder; + + char read_ts_buffer_[sizeof(RocksTimeStamp)]; + Encoder(read_ts_buffer_, sizeof(RocksTimeStamp)).put64(mongo::Timestamp::max().asULL()); + auto read_opt = rocksdb::ReadOptions(); + Slice readTs = rocksdb::Slice(read_ts_buffer_, sizeof(read_ts_buffer_)); + read_opt.timestamp = &readTs; + + auto s = Get(read_opt, stable_ts_key_, &ts_holder); + if (!s.ok()) { + return s; + } + invariant(ts_holder.size() == sizeof(RocksTimeStamp)); + *timestamp = Decoder(ts_holder.data(), ts_holder.size()).get64(); + LOG(2) << "TOTDB query stable TS type " << static_cast(ts_type) << " value " << *timestamp; + return Status::OK(); + } + return Status::InvalidArgument("invalid ts_type"); +} + +Status TOTransactionDBImpl::Stat(TOTransactionStat* stat) { + if (stat == nullptr) { + return Status::InvalidArgument("can not accept null as input"); + } + memset(stat, 0, sizeof(TOTransactionStat)); + stat->max_conflict_bytes = + max_conflict_bytes_.load(std::memory_order_relaxed); + stat->cur_conflict_bytes = current_conflict_bytes_.load(std::memory_order_relaxed); + { + std::vector> lks; + for (size_t i = 0; i < num_stripes_; i++) { + lks.emplace_back(std::unique_lock(*keys_mutex_[i])); + } + stat->uk_num = uncommitted_keys_.CountInLock(); + stat->ck_num = committed_keys_.CountInLock(); + } + { + std::lock_guard lock(active_txns_mutex_); + stat->alive_txns_num = active_txns_.size(); + } + { + std::shared_lock rl(read_ts_mutex_); + stat->read_q_num = read_q_.size(); + for (auto it = read_q_.begin(); it != read_q_.end(); it++) { + invariant(it->second->read_ts_set_); + if (it->second->state_.load(std::memory_order_relaxed) == TOTransaction::kStarted) { + stat->min_read_ts = it->second->read_ts_; + break; + } + } + } + { + std::shared_lock rl(commit_ts_mutex_); + stat->commit_q_num = commit_q_.size(); + for (auto it = commit_q_.begin(); it != commit_q_.end(); it++) { + invariant(it->second->commit_ts_set_); + if (it->second->state_.load(std::memory_order_relaxed) == TOTransaction::kStarted) { + stat->min_uncommit_ts = it->second->commit_ts_; + break; + } + } + } + { + std::shared_lock rl(ts_meta_mutex_); + stat->oldest_ts = oldest_ts_ == nullptr ? 0 : *oldest_ts_; + } + + stat->max_commit_ts = has_commit_ts_.load() ? committed_max_ts_.load() : 0; + stat->update_max_commit_ts_times = update_max_commit_ts_times_.load(std::memory_order_relaxed); + stat->update_max_commit_ts_retries = update_max_commit_ts_retries_.load(std::memory_order_relaxed); + stat->committed_max_txnid = committed_max_txnid_; + stat->txn_commits = txn_commits_.load(std::memory_order_relaxed); + stat->txn_aborts = txn_aborts_.load(std::memory_order_relaxed); + stat->commit_without_ts_times = commit_without_ts_times_.load(std::memory_order_relaxed); + stat->read_without_ts_times = read_without_ts_times_.load(std::memory_order_relaxed); + stat->read_with_ts_times = read_with_ts_times_.load(std::memory_order_relaxed); + stat->read_q_walk_len_sum = read_q_walk_len_sum_.load(std::memory_order_relaxed); + stat->read_q_walk_times = read_q_walk_times_.load(std::memory_order_relaxed); + stat->commit_q_walk_len_sum = commit_q_walk_len_sum_.load(std::memory_order_relaxed); + stat->commit_q_walk_times = commit_q_walk_times_.load(std::memory_order_relaxed); + return Status::OK(); +} + +Status TOTransactionDBImpl::BackgroundCleanJob::SetCleanInfo(const TransactionID& txn_id, + const RocksTimeStamp& time_stamp) { + std::lock_guard lock(thread_mutex_); + txnid_ = txn_id; + ts_ = time_stamp; + return Status::OK(); +} + +bool TOTransactionDBImpl::BackgroundCleanJob::IsRunning() { + std::lock_guard lock(thread_mutex_); + return thread_state_ == kRunning; +} + +bool TOTransactionDBImpl::BackgroundCleanJob::NeedToClean(TransactionID* txn_id, + RocksTimeStamp* time_stamp) { + std::lock_guard lock(thread_mutex_); + if (thread_state_ != kRunning) { + return false; + } + *txn_id = txnid_; + *time_stamp = ts_; + return (txnid_ != 0); +} + +void TOTransactionDBImpl::BackgroundCleanJob::FinishClean(const TransactionID& txn_id, + const RocksTimeStamp& time_stamp) { + std::lock_guard lock(thread_mutex_); + if (txn_id == txnid_ && ts_ == time_stamp) { + txnid_ = 0; + ts_ = 0; + } +} + +void TOTransactionDBImpl::BackgroundCleanJob::StopThread() { + std::lock_guard lock(thread_mutex_); + thread_state_ = kStopped; +} + +std::unique_ptr TOTransactionDBImpl::makeTxn() { + rocksdb::WriteOptions options; + rocksdb::TOTransactionOptions txnOptions; + return std::unique_ptr(BeginTransaction(options, txnOptions)); +} + +int TOComparator::CompareTimestamp(const Slice& ts1, const Slice& ts2) const { + invariant(timestamp_size() > 0); + invariant(ts1.data() && ts2.data()); + invariant(ts1.size() == sizeof(RocksTimeStamp)); + invariant(ts2.size() == sizeof(RocksTimeStamp)); + uint64_t ts1_data = Decoder(ts1.data(), ts1.size()).get64(); + uint64_t ts2_data = Decoder(ts2.data(), ts2.size()).get64(); + + int ret = 0; + if (ts1_data < ts2_data) { + ret = -1; + } else if (ts1_data > ts2_data) { + ret = 1; + } else { + ret = 0; + } + + if (skip_compare_oldest_ts) { + invariant(new_oldest_ts == ts1_data || new_oldest_ts == ts2_data); + if (last_oldest_ts == 0) { + last_oldest_ts = (new_oldest_ts == ts1_data?ts2_data:ts1_data); + } else { + invariant(last_oldest_ts == ts1_data || last_oldest_ts == ts2_data); + } + if(ret != 0) { + return new_oldest_ts == ts1_data?1:-1; + } + } + return ret; +} + + +void TOComparator::forceSetOldestTs(RocksTimeStamp ts){ + invariant(skip_compare_oldest_ts == false); + invariant(new_oldest_ts == 0); + invariant(ts != 0); + skip_compare_oldest_ts = true; + new_oldest_ts = ts; +} + +void TOComparator::clearSetOldestTs() { + invariant(skip_compare_oldest_ts); + invariant(new_oldest_ts != 0); + skip_compare_oldest_ts=false; + last_oldest_ts = 0; + new_oldest_ts = 0; +} + +} // namespace rocksdb + +#endif // ROCKSDB_LITE diff --git a/src/totdb/totransaction_db_impl.h b/src/totdb/totransaction_db_impl.h new file mode 100644 index 0000000..35fd243 --- /dev/null +++ b/src/totdb/totransaction_db_impl.h @@ -0,0 +1,352 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). + +#pragma once +#ifndef ROCKSDB_LITE + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include "rocksdb/db.h" +#include "rocksdb/options.h" +#include "mongo/db/modules/rocks/src/totdb/totransaction_db.h" +#include "mongo/db/modules/rocks/src/totdb/totransaction_impl.h" + +namespace rocksdb { + +class PrepareHeap { + public: + PrepareHeap(); + + ~PrepareHeap() = default; + + std::shared_ptr Find( + const TOTransactionImpl::ActiveTxnNode* core, const TxnKey& key, + TOTransaction::TOTransactionState* state); + + void Insert(const std::shared_ptr& core); + + uint32_t Remove(TOTransactionImpl::ActiveTxnNode* core); + + void Purge(TransactionID oldest_txn_id, RocksTimeStamp oldest_ts); + + private: + friend class PrepareMapIterator; + + std::shared_mutex mutex_; + + using PMAP = + std::map>>; + PMAP map_; + + static const TxnKey sentinal_; +}; + +class TOTransactionDBImpl : public TOTransactionDB { + public: + TOTransactionDBImpl(DB* db, const TOTransactionDBOptions& txn_db_options, + bool read_only, + const std::string stable_ts_key) + : TOTransactionDB(db), + read_only_(read_only), + txn_db_options_(txn_db_options), + num_stripes_(DEFAULT_NUM_STRIPES), + committed_max_txnid_(0), + current_conflict_bytes_(0), + max_conflict_bytes_(1.1 * txn_db_options.max_conflict_check_bytes_size), + txn_commits_(0), + txn_aborts_(0), + committed_max_ts_(0), + has_commit_ts_(false), + update_max_commit_ts_times_(0), + update_max_commit_ts_retries_(0), + commit_without_ts_times_(0), + read_without_ts_times_(0), + read_with_ts_times_(0), + read_q_walk_times_(0), + read_q_walk_len_sum_(0), + commit_q_walk_times_(0), + commit_q_walk_len_sum_(0), + oldest_ts_(nullptr), + stable_ts_key_(stable_ts_key) { + if (max_conflict_bytes_ == 0) { + // we preserve at least 100MB for conflict check + max_conflict_bytes_ = 100 * 1024 * 1024; + } + active_txns_.clear(); + + // Init default num_stripes + num_stripes_ = (txn_db_options.num_stripes > 0) ? txn_db_options.num_stripes + : DEFAULT_NUM_STRIPES; + + uncommitted_keys_.lock_map_stripes_.reserve(num_stripes_); + for (size_t i = 0; i < num_stripes_; i++) { + UnCommittedLockMapStripe* stripe = new UnCommittedLockMapStripe(); + uncommitted_keys_.lock_map_stripes_.push_back(stripe); + } + + committed_keys_.lock_map_stripes_.reserve(num_stripes_); + for (size_t i = 0; i < num_stripes_; i++) { + CommittedLockMapStripe* stripe = new CommittedLockMapStripe(); + committed_keys_.lock_map_stripes_.push_back(stripe); + } + + keys_mutex_.reserve(num_stripes_); + for (size_t i = 0; i < num_stripes_; i++) { + std::mutex* key_mutex = new std::mutex(); + keys_mutex_.push_back(key_mutex); + } + } + + ~TOTransactionDBImpl() { + // Clean resources + clean_job_.StopThread(); + clean_thread_.join(); + + { + for (auto& it : uncommitted_keys_.lock_map_stripes_) { + delete it; + } + uncommitted_keys_.lock_map_stripes_.clear(); + + for (auto& it : committed_keys_.lock_map_stripes_) { + delete it; + } + committed_keys_.lock_map_stripes_.clear(); + + for (auto& it : keys_mutex_) { + delete it; + } + keys_mutex_.clear(); + } + std::lock_guard lock(active_txns_mutex_); + active_txns_.clear(); + } + + void StartBackgroundCleanThread(); + + void SetMaxConflictBytes(uint64_t bytes) override { + max_conflict_bytes_.store(bytes, std::memory_order_relaxed); + } + + virtual TOTransaction* BeginTransaction(const WriteOptions& write_options, + const TOTransactionOptions& txn_options) override; + + using ATN = TOTransactionImpl::ActiveTxnNode; + Status CommitTransaction(const std::shared_ptr& core, + const std::set& written_keys, + const std::set& get_for_updates); + + Status RollbackTransaction(const std::shared_ptr& core, + const std::set& written_keys, + const std::set& get_for_updates); + + Status SetTimeStamp(const TimeStampType& ts_type, const RocksTimeStamp& ts, + bool force) override; + + Status QueryTimeStamp(const TimeStampType& ts_type, RocksTimeStamp* timestamp) override; + + Status Stat(TOTransactionStat* stat) override; + + Status CheckWriteConflict(const TxnKey& key, const TransactionID& txn_id, + const RocksTimeStamp& readts); + + Status PrepareTransaction(const std::shared_ptr& core); + + Status SetCommitTimeStamp(const std::shared_ptr& core, + const RocksTimeStamp& timesamp); + + Status SetDurableTimeStamp(const std::shared_ptr& core, + const RocksTimeStamp& timesamp); + + Status AddReadQueue(const std::shared_ptr& core, + const RocksTimeStamp& ts); + + Status SetPrepareTimeStamp(const std::shared_ptr& core, + const RocksTimeStamp& timestamp); + + void AdvanceTS(RocksTimeStamp* maxToCleanTs); + + void CleanCommittedKeys(); + + bool IsReadOnly() const { return read_only_; } + + Status GetConsiderPrepare(const std::shared_ptr& core, + ReadOptions& options, + ColumnFamilyHandle* column_family, const Slice& key, + std::string* value); + + Iterator* NewIteratorConsiderPrepare(const std::shared_ptr& core, + ColumnFamilyHandle* column_family, + Iterator* db_iter); + + std::unique_ptr makeTxn() override; + + // Committed key, first commit txnid, second prepare ts, third commit ts + // TODO: remove prepare ts from KeyModifyHistory + using KeyModifyHistory = + std::tuple; + using TSTXN = std::pair; + + protected: + bool read_only_; + const TOTransactionDBOptions txn_db_options_; + size_t num_stripes_; + TransactionID committed_max_txnid_; + std::atomic current_conflict_bytes_; + std::atomic max_conflict_bytes_; + std::atomic txn_commits_; + std::atomic txn_aborts_; + + class BackgroundCleanJob { + std::mutex thread_mutex_; + TransactionID txnid_; + RocksTimeStamp ts_; + + enum ThreadState { + kRunning, + kStopped + }; + + ThreadState thread_state_; + public: + BackgroundCleanJob() + :txnid_(0),ts_(0) { + thread_state_ = kRunning; + } + + ~BackgroundCleanJob() { + } + + Status SetCleanInfo(const TransactionID& txn_id, + const RocksTimeStamp& time_stamp); + + bool IsRunning(); + + bool NeedToClean(TransactionID* txn_id, + RocksTimeStamp* time_stamp); + + void FinishClean(const TransactionID& txn_id, + const RocksTimeStamp& time_stamp); + + void StopThread(); + }; + + private: + Status PublushTimeStamp(const std::shared_ptr& active_txn); + + // Add txn to active txns + Status AddToActiveTxns(const std::shared_ptr& active_txn); + + void RemoveUncommittedKeysOnCleanup(const std::set& written_keys); + + Status TxnAssertAfterReads(const std::shared_ptr& core, const char* op, + const RocksTimeStamp& timestamp); + + // Active txns + std::mutex active_txns_mutex_; + std::map> active_txns_; + + // txns sorted by {commit_ts, txnid} + std::shared_mutex commit_ts_mutex_; + std::map> commit_q_; + + // txns sorted by {read_ts, txnid} + std::shared_mutex read_ts_mutex_; + std::map> read_q_; + + PrepareHeap prepare_heap_; + + struct UnCommittedLockMapStripe { + std::map uncommitted_keys_map_; + }; + + size_t GetStripe(const TxnKey& key) const { + invariant(num_stripes_ > 0); + static std::hash hash; + size_t stripe = hash(key.second) % num_stripes_; + return stripe; + } + // Uncommitted keys + struct UnCommittedKeys { + std::vector lock_map_stripes_; + public: + // Remove key from uncommitted keys + Status RemoveKeyInLock(const TxnKey& key, const size_t& stripe_num, + std::atomic* mem_usage); + // Check write conflict and add the key to uncommitted keys + Status CheckKeyAndAddInLock(const TxnKey& key, const TransactionID& txn_id, + const size_t& stripe_num, + uint64_t max_mem_usage, + std::atomic* mem_usage); + + size_t CountInLock() const; + }; + + struct CommittedLockMapStripe { + //std::mutex map_mutex_; + std::map committed_keys_map_; + }; + + struct CommittedKeys { + std::vector lock_map_stripes_; + public: + // Add key to committed keys + Status AddKeyInLock(const TxnKey& key, const TransactionID& commit_txn_id, + const RocksTimeStamp& prepare_ts, + const RocksTimeStamp& commit_ts, + const size_t& stripe_num, + std::atomic* mem_usage); + + // Check write conflict + Status CheckKeyInLock(const TxnKey& key, const TransactionID& txn_id, + const RocksTimeStamp& timestamp, + const size_t& stripe_num); + + size_t CountInLock() const; + }; + + std::vector keys_mutex_; + + UnCommittedKeys uncommitted_keys_; + + CommittedKeys committed_keys_; + + BackgroundCleanJob clean_job_; + + std::thread clean_thread_; + + // NOTE(xxxxxxxx): commit_ts_ is not protected by ts_meta_mutex_ + // remember to publish commit_ts_ before has_commit_ts_ + std::atomic committed_max_ts_; + std::atomic has_commit_ts_; + std::atomic update_max_commit_ts_times_; + std::atomic update_max_commit_ts_retries_; + std::atomic commit_without_ts_times_; + std::atomic read_without_ts_times_; + std::atomic read_with_ts_times_; + std::atomic read_q_walk_times_; + std::atomic read_q_walk_len_sum_; + std::atomic commit_q_walk_times_; + std::atomic commit_q_walk_len_sum_; + + // TODO(xxxxxxxx): use optional<> + std::shared_mutex ts_meta_mutex_; + // protected by ts_meta_mutex_ + std::unique_ptr oldest_ts_; + + std::string stable_ts_key_; +}; + +} // namespace rocksdb +#endif // ROCKSDB_LITE diff --git a/src/totdb/totransaction_impl.cpp b/src/totdb/totransaction_impl.cpp new file mode 100644 index 0000000..eea8f61 --- /dev/null +++ b/src/totdb/totransaction_impl.cpp @@ -0,0 +1,445 @@ +// Copyright (c) 2019-present. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). + +#ifndef ROCKSDB_LITE + +#define MONGO_LOG_DEFAULT_COMPONENT ::mongo::logger::LogComponent::kStorage +#include "mongo/db/modules/rocks/src/totdb/totransaction_impl.h" +#include "mongo/db/modules/rocks/src/totdb/totransaction_db.h" +#include "mongo/db/modules/rocks/src/totdb/totransaction_db_impl.h" +#include "mongo/util/log.h" +#include "mongo/util/stacktrace.h" +#include "rocksdb/comparator.h" +#include "rocksdb/db.h" +#include "rocksdb/status.h" +#include "third_party/s2/util/coding/coder.h" + +namespace rocksdb { + +namespace { + +TOComparator wbwidx_default_comparator; +const RocksTimeStamp none_ts = 1; + +} // namespace + +std::set TOTransaction::timestampPrefixes; +std::mutex TOTransaction::prefixes_mutex; + +void TOTransaction::enableTimestamp(const std::string& prefix) { + std::unique_lock lk(prefixes_mutex); + timestampPrefixes.insert(prefix); +} + +bool TOTransaction::isEnableTimestamp(const Slice& key) { + std::unique_lock lk(prefixes_mutex); + Slice prefix(key.data(), sizeof(uint32_t)); + return timestampPrefixes.count(prefix.ToString()) > 0; +} + +Status PrepareConflict() { + return Status(Status::Code::kInvalidArgument, Status::SubCode::kNone, + Status::Severity::kSoftError, "prepare conflict"); +} + +bool IsPrepareConflict(const Status& s) { + return s.code() == Status::Code::kInvalidArgument && s.subcode() == Status::SubCode::kNone && + s.severity() == Status::Severity::kSoftError && + std::string(s.getState()) == "prepare conflict"; +} + +struct WriteOptions; + +std::atomic TOTransactionImpl::txn_id_counter_(1); + +TransactionID TOTransactionImpl::GenTxnID() { + return txn_id_counter_.fetch_add(1); +} + +TOTransactionImpl::TOTransactionImpl(TOTransactionDB* txn_db, + const WriteOptions& write_options, + const TOTxnOptions& txn_option, + const std::shared_ptr& core) + : txn_id_(0), + db_(txn_db->GetRootDB()), + write_options_(write_options), + txn_option_(txn_option), + core_(core) { + txn_db_impl_ = dynamic_cast(txn_db); + invariant(txn_db_impl_); +} + +TOTransactionImpl::~TOTransactionImpl() { + // Do rollback if this transaction is not committed or rolled back + if (core_->state_ < kCommitted) { + Rollback(); + } +} + +Status TOTransactionImpl::SetReadTimeStamp(const RocksTimeStamp& timestamp) { + if (core_->state_ >= kCommitted) { + return Status::NotSupported("this txn is committed or rollback"); + } + + if (core_->read_ts_set_) { + return Status::NotSupported("set read ts is supposed to be set only once"); + } + + LOG(1) << "TOTDB txn id: " << txn_id_ + << "set read ts: " << timestamp + << "force: " << core_->timestamp_round_read_; + + Status s = txn_db_impl_->AddReadQueue(core_, timestamp); + if (!s.ok()) { + return s; + } + invariant(core_->read_ts_set_); + invariant(core_->read_ts_ >= timestamp); + Encoder(core_->read_ts_buffer_, sizeof(core_->read_ts_)).put64(core_->read_ts_); + // If we already have a snapshot, it may be too early to match + // the timestamp (including the one we just read, if rounding + // to oldest). Get a new one. + invariant(core_->txn_snapshot != nullptr); + txn_db_impl_->ReleaseSnapshot(core_->txn_snapshot); + core_->txn_snapshot = txn_db_impl_->GetSnapshot(); + return s; +} + +Status TOTransactionImpl::SetPrepareTimeStamp(const RocksTimeStamp& timestamp) { + if (core_->state_ != kStarted) { + return Status::NotSupported( + "this txn is prepared or committed or rollback"); + } + + if (core_->prepare_ts_set_) { + return Status::NotSupported("prepare ts is already set"); + } + + if (core_->commit_ts_set_) { + return Status::NotSupported( + "should not have been set before the prepare timestamp"); + } + return txn_db_impl_->SetPrepareTimeStamp(core_, timestamp); +} + +Status TOTransactionImpl::Prepare() { + if (core_->state_ != kStarted) { + return Status::NotSupported( + "this txn is prepared or committed or rollback"); + } + + if (!core_->prepare_ts_set_) { + return Status::NotSupported("prepare ts not set when prepare"); + } + + if (core_->commit_ts_set_) { + return Status::NotSupported( + "commit ts should not have been set when prepare"); + } + + return txn_db_impl_->PrepareTransaction(core_); +} + +Status TOTransactionImpl::SetCommitTimeStamp(const RocksTimeStamp& timestamp) { + if (core_->state_ >= kCommitted) { + return Status::NotSupported("this txn is committed or rollback"); + } + auto s = txn_db_impl_->SetCommitTimeStamp(core_, timestamp); + if (!s.ok()) { + return s; + } + invariant(core_->commit_ts_set_ && + (core_->first_commit_ts_ <= core_->commit_ts_)); + + LOG(2) << "TOTDB txn id " << core_->txn_id_ << "set commit ts " << timestamp; + return Status::OK(); +} + +Status TOTransactionImpl::SetDurableTimeStamp(const RocksTimeStamp& timestamp) { + if (core_->state_ >= kCommitted) { + return Status::NotSupported("this txn is committed or rollback"); + } + + auto s = txn_db_impl_->SetDurableTimeStamp(core_, timestamp); + if (!s.ok()) { + return s; + } + invariant(core_->durable_ts_set_); + + LOG(2) << "TOTDB txn id " << core_->txn_id_ << "set durable ts " << timestamp; + return Status::OK(); +} + +Status TOTransactionImpl::GetReadTimeStamp(RocksTimeStamp* timestamp) const { + if ((!timestamp) || (!core_->read_ts_set_)) { + return Status::InvalidArgument("need set read ts, and parameter should not be null"); + } + + *timestamp = core_->read_ts_; + + return Status::OK(); +} + +WriteBatchWithIndex* TOTransactionImpl::GetWriteBatch() { + return &(core_->write_batch_); +} + +TOTransactionImpl::ActiveTxnNode::ActiveTxnNode() + : txn_id_(0), + commit_txn_id_(0), + commit_ts_set_(false), + commit_ts_(0), + first_commit_ts_(0), + read_ts_set_(false), + read_ts_(std::numeric_limits::max()), + read_ts_slice_(read_ts_buffer_, sizeof(read_ts_buffer_)), + prepare_ts_set_(false), + prepare_ts_(0), + durable_ts_set_(false), + durable_ts_(0), + timestamp_published_(false), + timestamp_round_prepared_(false), + timestamp_round_read_(false), + state_(TOTransaction::kStarted), + txn_snapshot(nullptr), + write_batch_(&wbwidx_default_comparator, 0, true /*overwrite_keys*/, 0) { + Encoder(read_ts_buffer_, sizeof(read_ts_)).put64(read_ts_); +} + +const TOTransactionImpl::ActiveTxnNode* TOTransactionImpl::GetCore() const { + return core_.get(); +} + +Status TOTransactionImpl::Put(ColumnFamilyHandle* column_family, const Slice& key, + const Slice& value) { + if (txn_db_impl_->IsReadOnly()) { + return Status::NotSupported("readonly db cannot accept put"); + } + if (core_->state_ >= kPrepared) { + return Status::NotSupported("txn is already prepared, committed rollback"); + } + if (core_->read_only_) { + if (core_->ignore_prepare_) { + return Status::NotSupported( + "Transactions with ignore_prepare=true cannot perform updates"); + } + return Status::NotSupported("Attempt to update in a read-only transaction"); + } + + const TxnKey txn_key(column_family->GetID(), key.ToString()); + Status s = CheckWriteConflict(txn_key); + + if (s.ok()) { + written_keys_.emplace(std::move(txn_key)); + GetWriteBatch()->Put(column_family, key, value); + auto commit_ts = TOTransaction::isEnableTimestamp(key) ? core_->commit_ts_ : none_ts; + asof_commit_timestamps_.emplace_back(commit_ts); + } + return s; +} + +Status TOTransactionImpl::Put(const Slice& key, const Slice& value) { + return Put(db_->DefaultColumnFamily(), key, value); +} + +Status TOTransactionImpl::Get(ReadOptions& options, + ColumnFamilyHandle* column_family, const Slice& key, + std::string* value) { + if (core_->state_ >= kPrepared) { + return Status::NotSupported("txn is already prepared, committed rollback"); + } + // Check the options, if read ts is set use read ts + options.timestamp = &core_->read_ts_slice_; + invariant(core_->txn_snapshot); + options.snapshot = core_->txn_snapshot; + + invariant(options.timestamp->size() == sizeof(RocksTimeStamp)); + const TxnKey txn_key(column_family->GetID(), key.ToString()); + if (written_keys_.find(txn_key) != written_keys_.end()) { + return GetWriteBatch()->GetFromBatchAndDB(db_, options, column_family, key, + value); + } + + return txn_db_impl_->GetConsiderPrepare(core_, options, column_family, key, + value); +} + +Status TOTransactionImpl::Get(ReadOptions& options, const Slice& key, + std::string* value) { + return Get(options, db_->DefaultColumnFamily(), key, value); +} + +Status TOTransactionImpl::Delete(ColumnFamilyHandle* column_family, const Slice& key) { + if (txn_db_impl_->IsReadOnly()) { + return Status::NotSupported("readonly db cannot accept del"); + } + if (core_->state_ >= kPrepared) { + return Status::NotSupported("txn is already prepared, committed rollback"); + } + if (core_->read_only_) { + if (core_->ignore_prepare_) { + return Status::NotSupported( + "Transactions with ignore_prepare=true cannot perform updates"); + } + return Status::NotSupported("Attempt to update in a read-only transaction"); + } + + const TxnKey txn_key(column_family->GetID(), key.ToString()); + Status s = CheckWriteConflict(txn_key); + + if (s.ok()) { + written_keys_.emplace(std::move(txn_key)); + GetWriteBatch()->Delete(column_family, key); + auto commit_ts = TOTransaction::isEnableTimestamp(key) ? core_->commit_ts_ : none_ts; + asof_commit_timestamps_.emplace_back(commit_ts); + } + return s; +} + +Status TOTransactionImpl::Delete(const Slice& key) { + return Delete(db_->DefaultColumnFamily(), key); +} + +Status TOTransactionImpl::GetForUpdate(ColumnFamilyHandle* column_family, const Slice& key) { + if (txn_db_impl_->IsReadOnly()) { + return Status::NotSupported("readonly db cannot accept del"); + } + if (core_->state_ >= kPrepared) { + return Status::NotSupported("txn is already prepared, committed rollback"); + } + if (core_->read_only_) { + if (core_->ignore_prepare_) { + return Status::NotSupported( + "Transactions with ignore_prepare=true cannot perform updates"); + } + return Status::NotSupported("Attempt to update in a read-only transaction"); + } + + const TxnKey txn_key(column_family->GetID(), key.ToString()); + Status s = CheckWriteConflict(txn_key); + + if (s.ok()) { + get_for_updates_.emplace(std::move(txn_key)); + } + return s; +} + +Status TOTransactionImpl::GetForUpdate(const Slice& key) { + return GetForUpdate(db_->DefaultColumnFamily(), key); +} + +Iterator* TOTransactionImpl::GetIterator(ReadOptions& read_options) { + return GetIterator(read_options, db_->DefaultColumnFamily()); +} + +Iterator* TOTransactionImpl::GetIterator(ReadOptions& read_options, + ColumnFamilyHandle* column_family) { + if (core_->state_ >= kPrepared) { + return nullptr; + } + + read_options.timestamp = &core_->read_ts_slice_; + + invariant(core_->txn_snapshot); + invariant(read_options.timestamp->size() == sizeof(RocksTimeStamp)); + read_options.snapshot = core_->txn_snapshot; + Iterator* db_iter = db_->NewIterator(read_options, column_family); + if (db_iter == nullptr) { + return nullptr; + } + + return txn_db_impl_->NewIteratorConsiderPrepare(core_, column_family, + db_iter); +} + +Status TOTransactionImpl::CheckWriteConflict(const TxnKey& key) { + return txn_db_impl_->CheckWriteConflict(key, GetID(), core_->read_ts_); +} + +Status TOTransactionImpl::Commit(std::function* hook) { + if (core_->state_ >= kCommitted) { + return Status::InvalidArgument("txn already committed or rollback."); + } + + invariant(asof_commit_timestamps_.size() + == static_cast(GetWriteBatch()->GetWriteBatch()->Count())); + if (core_->commit_ts_set_) { + for (size_t i = 0; i < asof_commit_timestamps_.size(); ++i) { + if (asof_commit_timestamps_[i] == 0) { + asof_commit_timestamps_[i] = core_->commit_ts_; + } + } + } + + Status s; + if (GetWriteBatch()->GetWriteBatch()->Count() != 0) { + invariant(!txn_db_impl_->IsReadOnly()); + char ts_buf[sizeof(RocksTimeStamp)]; + size_t cnt = 0; + Slice ts_slice(ts_buf, sizeof(RocksTimeStamp)); + const auto ts_sz_func = [&](uint32_t) { + Encoder(ts_buf, sizeof(RocksTimeStamp)).put64(asof_commit_timestamps_[cnt++]); + return sizeof(RocksTimeStamp); + }; + GetWriteBatch()->GetWriteBatch()->UpdateTimestamps(ts_slice, ts_sz_func); + invariant(cnt == asof_commit_timestamps_.size()); + // NOTE(xxxxxxxx): It's a simple modification for readonly transaction. + // PutLogData will not increase Count. So, If in the future + // PutLogData is added into TOTransactionDB, this shortcut should be redesigned. + s = db_->Write(write_options_, GetWriteBatch()->GetWriteBatch()); + } + if (hook) { + (*hook)(); + } + if (s.ok()) { + // Change active txn set, + // Move uncommitted keys to committed keys, + // Clean data when the committed txn is activeTxnSet's header + // TODO(xxxxxxxx): in fact, here we must not fail + s = txn_db_impl_->CommitTransaction(core_, written_keys_, get_for_updates_); + } else { + s = Status::InvalidArgument("Transaction is fail for commit."); + } + +#ifdef NDEBUG + // prepareHeap needs writeBatch for sanity check in debug mode. + GetWriteBatch()->Clear(); +#endif // NDEBUG + LOG(2) << "TOTDB txn id " << txn_id_ << " committed"; + return s; +} + +Status TOTransactionImpl::Rollback() { + if (core_->state_ >= kCommitted) { + return Status::InvalidArgument("txn is already committed or rollback."); + } + + // Change active txn set, + // Clean uncommitted keys + Status s = txn_db_impl_->RollbackTransaction(core_, written_keys_, get_for_updates_); + + GetWriteBatch()->Clear(); + + LOG(2) << "TOTDB txn id " << txn_id_ << " rollbacked"; + return s; +} + +Status TOTransactionImpl::SetName(const TransactionName& name) { + name_ = name; + return Status::OK(); +} + +TransactionID TOTransactionImpl::GetID() const { + invariant(core_); + return core_->txn_id_; +} + +TOTransaction::TOTransactionState TOTransactionImpl::GetState() const { + invariant(core_); + return core_->state_; +} +} + +#endif diff --git a/src/totdb/totransaction_impl.h b/src/totdb/totransaction_impl.h new file mode 100644 index 0000000..ad6e3e5 --- /dev/null +++ b/src/totdb/totransaction_impl.h @@ -0,0 +1,165 @@ +#pragma once +#ifndef ROCKSDB_LITE + +#include "rocksdb/db.h" +#include "rocksdb/slice.h" +#include "rocksdb/status.h" +#include "rocksdb/types.h" +#include "mongo/db/modules/rocks/src/totdb/totransaction.h" +#include "mongo/db/modules/rocks/src/totdb//totransaction_db.h" +#include "rocksdb/utilities/write_batch_with_index.h" + + +namespace rocksdb { + +using TxnKey = std::pair; + +// TimeStamp Ordering Transaction Options +struct TOTxnOptions { + size_t max_write_batch_size = 1000; + Logger* log_ = nullptr; +}; + +class TOTransactionDBImpl; + +class SwapSnapshotGuard { + public: + // No copying allowed + SwapSnapshotGuard(const SwapSnapshotGuard&) = delete; + SwapSnapshotGuard& operator=(const SwapSnapshotGuard&) = delete; + SwapSnapshotGuard(ReadOptions* readOpt, const Snapshot* newSnapshot) { + read_opt_ = readOpt; + old_snapshot_ = read_opt_->snapshot; + read_opt_->snapshot = newSnapshot; + } + ~SwapSnapshotGuard() { read_opt_->snapshot = old_snapshot_; } + + private: + ReadOptions* read_opt_; + const Snapshot* old_snapshot_; +}; + +class TOTransactionImpl : public TOTransaction { + public: + struct ActiveTxnNode { + // NOTE(deyukong): txn_id_ is indeed duplicated with txn_snapshot + // consider using txn_snapshot + TransactionID txn_id_; + TransactionID commit_txn_id_; + bool commit_ts_set_; + RocksTimeStamp commit_ts_; + RocksTimeStamp first_commit_ts_; + bool read_ts_set_; + RocksTimeStamp read_ts_; + char read_ts_buffer_[sizeof(RocksTimeStamp)]; + Slice read_ts_slice_; + bool prepare_ts_set_; + RocksTimeStamp prepare_ts_; + bool durable_ts_set_; + RocksTimeStamp durable_ts_; + bool timestamp_published_; + bool timestamp_round_prepared_; + bool timestamp_round_read_; + bool read_only_; + bool ignore_prepare_; + std::atomic state_; + const Snapshot* txn_snapshot; + WriteBatchWithIndex write_batch_; + + public: + ActiveTxnNode(const ActiveTxnNode&) = delete; + ActiveTxnNode& operator=(const ActiveTxnNode&) = delete; + ActiveTxnNode(); + }; + + TOTransactionImpl(TOTransactionDB* db, + const WriteOptions& options, + const TOTxnOptions& txn_options, + const std::shared_ptr& core); + + virtual ~TOTransactionImpl(); + + virtual Status SetPrepareTimeStamp(const RocksTimeStamp& timestamp) override; + + virtual Status SetCommitTimeStamp(const RocksTimeStamp& timestamp) override; + + virtual Status SetDurableTimeStamp(const RocksTimeStamp& timestamp) override; + + virtual Status SetReadTimeStamp(const RocksTimeStamp& timestamp) override; + + virtual Status GetReadTimeStamp(RocksTimeStamp* timestamp) const override; + + virtual Status Prepare() override; + + virtual Status Commit(std::function* hook = nullptr) override; + + virtual Status Rollback() override; + + virtual Status Get(ReadOptions& options, + ColumnFamilyHandle* column_family, const Slice& key, + std::string* value) override; + + virtual Status Get(ReadOptions& options, const Slice& key, + std::string* value) override; + + virtual Iterator* GetIterator(ReadOptions& read_options) override; + + virtual Iterator* GetIterator(ReadOptions& read_options, + ColumnFamilyHandle* column_family) override; + + virtual Status Put(ColumnFamilyHandle* column_family, const Slice& key, + const Slice& value) override; + + virtual Status Put(const Slice& key, const Slice& value) override; + + virtual Status Delete(ColumnFamilyHandle* column_family, const Slice& key) override; + + virtual Status Delete(const Slice& key) override; + + virtual Status GetForUpdate(ColumnFamilyHandle* column_family, const Slice& key) override; + + virtual Status GetForUpdate(const Slice& key) override; + + virtual Status SetName(const TransactionName& name) override; + + virtual TransactionID GetID() const override; + + virtual TOTransactionState GetState() const override; + + virtual WriteBatchWithIndex* GetWriteBatch() override; + + const ActiveTxnNode* GetCore() const; + + // Check write conflict. If there is no write conflict, add the key to uncommitted keys + Status CheckWriteConflict(const TxnKey& key); + + // Generate a new unique transaction identifier + static TransactionID GenTxnID(); + + private: + // Used to create unique ids for transactions. + static std::atomic txn_id_counter_; + + // Unique ID for this transaction + TransactionID txn_id_; + + // Updated keys in this transaction + // TODO(deyukong): writtenKeys_ is duplicated with core_->Write_batch_, remove + // this + std::set written_keys_; + + std::set get_for_updates_; + + DB* db_; + TOTransactionDBImpl* txn_db_impl_; + + WriteOptions write_options_; + TOTxnOptions txn_option_; + + std::shared_ptr core_; + + std::vector asof_commit_timestamps_; +}; + +} // namespace rocksdb +#endif diff --git a/src/totdb/totransaction_prepare_iterator.cpp b/src/totdb/totransaction_prepare_iterator.cpp new file mode 100644 index 0000000..48b1d25 --- /dev/null +++ b/src/totdb/totransaction_prepare_iterator.cpp @@ -0,0 +1,686 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). + +#define MONGO_LOG_DEFAULT_COMPONENT ::mongo::logger::LogComponent::kStorage + +#include "mongo/db/modules/rocks/src/totdb/totransaction.h" +#include "mongo/db/modules/rocks/src/totdb/totransaction_impl.h" +#include "mongo/db/modules/rocks/src/totdb/totransaction_db.h" +#include "mongo/db/modules/rocks/src/totdb/totransaction_db_impl.h" +#include "mongo/db/modules/rocks/src/totdb/totransaction_prepare_iterator.h" +#include "mongo/util/log.h" + +namespace rocksdb { + +namespace { +using ATN = TOTransactionImpl::ActiveTxnNode; + +// If we started after prepared-txn commits, due to SI's nature, we can see +// it from db. The value from db may be as updated as the one as prepared-txn. +// Or more updated. If we started before the prepared-txn commits, we cannot +// see the updated data with our own snapshot, so refetch with the latest +// snapshot. +#define MACRO_REFETCH_RETURN_ON_FOUND_ADVANCE_CONTINUE_ON_NOT_FOUND() \ + do { \ + if (pmap_val->commit_txn_id_ < core_->txn_id_) { \ + if (!sval_.has_base) { \ + AdvanceInputNoFilter(); \ + } else { \ + val_ = std::string(sval_.base_value.data(), sval_.base_value.size()); \ + return; \ + } \ + } else { \ + ReadOptions read_opt; \ + read_opt.timestamp = &core_->read_ts_slice_; \ + invariant(read_opt.timestamp->size() == sizeof(RocksTimeStamp)); \ + auto getStatus = core_->write_batch_.GetFromBatchAndDB( \ + db_, read_opt, cf_, key_, &val_); \ + invariant(getStatus.ok() || getStatus.IsNotFound()); \ + if (getStatus.ok()) { \ + return; \ + } else { \ + AdvanceInputNoFilter(); \ + } \ + } \ + } while (0) +} // namespace + +Slice PrepareMapIterator::key() const { + return Slice(pos_.data(), pos_.size()); +} + +const std::shared_ptr& PrepareMapIterator::value() const { return val_; } + +TOTransaction::TOTransactionState PrepareMapIterator::valueState() const { + return val_state_; +} + +TOTransaction::TOTransactionState PrepareMapIterator::UpdatePrepareState() { + invariant(val_state_ == TOTransaction::TOTransactionState::kPrepared); + invariant(Valid()); + val_state_ = val_->state_.load(/*seq_cst*/); + return val_state_; +} + +void PrepareMapIterator::TryPosValueToCorrectMvccVersionInLock( + const std::list>& prepare_mvccs) { + bool found = false; + invariant(!prepare_mvccs.empty()); + invariant(valid_); + for (const auto& c : prepare_mvccs) { + if (c->prepare_ts_ <= core_->read_ts_) { + val_ = c; + found = true; + break; + } + } + if (!found) { + // no satisfied mvcc version, just return arbitory one and let upper-level + // iterator skip to next + val_ = prepare_mvccs.back(); + } + val_state_ = val_->state_.load(/*seq_cst*/); +} + +void PrepareMapIterator::Next() { + std::shared_lock rl(ph_->mutex_); + if (!valid_) { + return; + } + auto it = ph_->map_.upper_bound(std::make_pair(cf_->GetID(), pos_)); + invariant(it != ph_->map_.end()); + if (it->first.first != cf_->GetID()) { + valid_ = false; + } else { + valid_ = true; + pos_ = it->first.second; + TryPosValueToCorrectMvccVersionInLock(it->second); + } +} + +void PrepareMapIterator::Prev() { + std::shared_lock rl(ph_->mutex_); + if (!valid_) { + return; + } + const auto lookup_key = std::make_pair(cf_->GetID(), pos_); + auto it = ph_->map_.lower_bound(lookup_key); + invariant(it != ph_->map_.end() && it->first >= lookup_key); + if (it != ph_->map_.begin()) { + it--; + } + if (it->first >= lookup_key) { + valid_ = false; + } else { + valid_ = true; + pos_ = it->first.second; + TryPosValueToCorrectMvccVersionInLock(it->second); + } +} + +void PrepareMapIterator::SeekToFirst() { Seek(""); } + +void PrepareMapIterator::SeekToLast() { + std::shared_lock rl(ph_->mutex_); + auto it = ph_->map_.lower_bound(std::make_pair(cf_->GetID() + 1, "")); + // because prepare_heap has sentinal at the end, it's impossible to reach the + // end + invariant(it != ph_->map_.end()); + if (it != ph_->map_.begin()) { + it--; + } + if (it->first.first != cf_->GetID()) { + valid_ = false; + } else { + valid_ = true; + pos_ = it->first.second; + TryPosValueToCorrectMvccVersionInLock(it->second); + } +} + +void PrepareMapIterator::SeekForPrev(const Slice& target) { + std::shared_lock rl(ph_->mutex_); + const auto lookup_key = + std::make_pair(cf_->GetID(), std::string(target.data(), target.size())); + auto it = ph_->map_.lower_bound(lookup_key); + // because prepare_heap_ has sentinal at the end, it's impossible to reach the + // end + invariant(it != ph_->map_.end() && it->first >= lookup_key); + if (it->first != lookup_key) { + if (it != ph_->map_.begin()) { + it--; + } + } + + if (it->first.first != cf_->GetID()) { + valid_ = false; + } else { + invariant(it->first <= lookup_key); + valid_ = true; + pos_ = it->first.second; + TryPosValueToCorrectMvccVersionInLock(it->second); + } +} + +void PrepareMapIterator::Seek(const Slice& target) { + std::shared_lock rl(ph_->mutex_); + auto it = ph_->map_.lower_bound( + std::make_pair(cf_->GetID(), std::string(target.data(), target.size()))); + invariant(it != ph_->map_.end()); + if (it->first.first != cf_->GetID()) { + valid_ = false; + } else { + valid_ = true; + pos_ = it->first.second; + TryPosValueToCorrectMvccVersionInLock(it->second); + } +} + +PrepareMergingIterator::PrepareMergingIterator( + std::unique_ptr base_iterator, + std::unique_ptr pmap_iterator) + : forward_(true), + current_at_base_(true), + equal_keys_(false), + status_(Status::OK()), + base_iterator_(std::move(base_iterator)), + delta_iterator_(std::move(pmap_iterator)), + comparator_(BytewiseComparator()) {} + +bool PrepareMergingIterator::BaseValid() const { + return base_iterator_->Valid(); +} + +bool PrepareMergingIterator::DeltaValid() const { + return delta_iterator_->Valid(); +} + +bool PrepareMergingIterator::Valid() const { + return current_at_base_ ? BaseValid() : DeltaValid(); +} + +Slice PrepareMergingIterator::key() const { + return current_at_base_ ? base_iterator_->key() + : delta_iterator_->key(); +} + +ShadowValue PrepareMergingIterator::value() const { + // NOTE(wolfkdy): WriteBatchWithIndexIterator didn't support timestamp() interface. + // TODO(wolfkdy): we need WriteBatchWithIndexIterator's timestamp() interface for sanity-check. + if (equal_keys_) { + return {true /*has_prepare*/, true /*has_base*/, + delta_iterator_->value().get(), delta_iterator_->valueState(), + base_iterator_->value(), base_iterator_->key(), 0}; + } + if (current_at_base_) { + return {false /*has_prepare*/, + true /*has_base*/, + nullptr, + TOTransaction::TOTransactionState::kStarted, + base_iterator_->value(), + base_iterator_->key(), + 0}; + } + return {true /*has_prepare*/, + false /*has_base*/, + delta_iterator_->value().get(), + delta_iterator_->valueState(), + Slice(), + Slice(), + 0}; +} + +void PrepareMergingIterator::SeekToFirst() { + forward_ = true; + base_iterator_->SeekToFirst(); + delta_iterator_->SeekToFirst(); + UpdateCurrent(); +} + +void PrepareMergingIterator::SeekToLast() { + forward_ = false; + base_iterator_->SeekToLast(); + delta_iterator_->SeekToLast(); + UpdateCurrent(); +} + +void PrepareMergingIterator::Seek(const Slice& k) { + forward_ = true; + base_iterator_->Seek(k); + delta_iterator_->Seek(k); + UpdateCurrent(); +} + +void PrepareMergingIterator::SeekForPrev(const Slice& k) { + forward_ = false; + base_iterator_->SeekForPrev(k); + delta_iterator_->SeekForPrev(k); + UpdateCurrent(); +} + +void PrepareMergingIterator::Next() { + if (!Valid()) { + status_ = Status::NotSupported("Next() on invalid iterator"); + return; + } + + if (!forward_) { + // Need to change direction + // if our direction was backward and we're not equal, we have two states: + // * both iterators are valid: we're already in a good state (current + // shows to smaller) + // * only one iterator is valid: we need to advance that iterator + forward_ = true; + equal_keys_ = false; + if (!BaseValid()) { + invariant(DeltaValid()); + base_iterator_->SeekToFirst(); + } else if (!DeltaValid()) { + delta_iterator_->SeekToFirst(); + } else if (current_at_base_) { + // Change delta from larger than base to smaller + AdvanceDelta(); + } else { + // Change base from larger than delta to smaller + AdvanceBase(); + } + if (DeltaValid() && BaseValid()) { + if (comparator_->Equal(delta_iterator_->key(), + base_iterator_->key())) { + equal_keys_ = true; + } + } + } + Advance(); +} + +void PrepareMergingIterator::Prev() { + if (!Valid()) { + status_ = Status::NotSupported("Prev() on invalid iterator"); + return; + } + + if (forward_) { + // Need to change direction + // if our direction was backward and we're not equal, we have two states: + // * both iterators are valid: we're already in a good state (current + // shows to smaller) + // * only one iterator is valid: we need to advance that iterator + forward_ = false; + equal_keys_ = false; + if (!BaseValid()) { + invariant(DeltaValid()); + base_iterator_->SeekToLast(); + } else if (!DeltaValid()) { + delta_iterator_->SeekToLast(); + } else if (current_at_base_) { + // Change delta from less advanced than base to more advanced + AdvanceDelta(); + } else { + // Change base from less advanced than delta to more advanced + AdvanceBase(); + } + if (DeltaValid() && BaseValid()) { + if (comparator_->Equal(delta_iterator_->key(), + base_iterator_->key())) { + equal_keys_ = true; + } + } + } + + Advance(); +} + +Status PrepareMergingIterator::status() const { + if (!status_.ok()) { + return status_; + } + if (!base_iterator_->status().ok()) { + return base_iterator_->status(); + } + return Status::OK(); +} + +TOTransaction::TOTransactionState PrepareMergingIterator::UpdatePrepareState() { + invariant(delta_iterator_->Valid()); + invariant(equal_keys_ || !current_at_base_); + return delta_iterator_->UpdatePrepareState(); +} + +void PrepareMergingIterator::AssertInvariants() { + bool not_ok = false; + if (!base_iterator_->status().ok()) { + invariant(!base_iterator_->Valid()); + not_ok = true; + } + if (not_ok) { + invariant(!Valid()); + invariant(!status().ok()); + return; + } + + if (!Valid()) { + return; + } + if (!BaseValid()) { + invariant(!current_at_base_ && delta_iterator_->Valid()); + return; + } + if (!DeltaValid()) { + invariant(current_at_base_ && base_iterator_->Valid()); + return; + } + int compare = comparator_->Compare(delta_iterator_->key(), + base_iterator_->key()); + (void)compare; + if (forward_) { + // current_at_base -> compare < 0 + invariant(!current_at_base_ || compare < 0); + // !current_at_base -> compare <= 0 + invariant(current_at_base_ && compare >= 0); + } else { + // current_at_base -> compare > 0 + invariant(!current_at_base_ || compare > 0); + // !current_at_base -> compare <= 0 + invariant(current_at_base_ && compare <= 0); + } + // equal_keys_ <=> compare == 0 + invariant((equal_keys_ || compare != 0) && (!equal_keys_ || compare == 0)); +} + +void PrepareMergingIterator::Advance() { + if (equal_keys_) { + invariant(BaseValid() && DeltaValid()); + AdvanceBase(); + AdvanceDelta(); + } else { + if (current_at_base_) { + invariant(BaseValid()); + AdvanceBase(); + } else { + invariant(DeltaValid()); + AdvanceDelta(); + } + } + UpdateCurrent(); +} + +void PrepareMergingIterator::AdvanceDelta() { + if (forward_) { + delta_iterator_->Next(); + } else { + delta_iterator_->Prev(); + } +} + +void PrepareMergingIterator::AdvanceBase() { + if (forward_) { + base_iterator_->Next(); + } else { + base_iterator_->Prev(); + } +} + +void PrepareMergingIterator::UpdateCurrent() { + status_ = Status::OK(); + while (true) { + equal_keys_ = false; + if (!BaseValid()) { + if (!base_iterator_->status().ok()) { + // Expose the error status and stop. + current_at_base_ = true; + return; + } + + // Base has finished. + if (!DeltaValid()) { + // Finished + return; + } + current_at_base_ = false; + return; + } else if (!DeltaValid()) { + // Delta has finished. + current_at_base_ = true; + return; + } else { + int compare = (forward_ ? 1 : -1) * + comparator_->Compare(delta_iterator_->key(), + base_iterator_->key()); + if (compare <= 0) { // delta bigger or equal + if (compare == 0) { + equal_keys_ = true; + } + current_at_base_ = false; + return; + } else { + current_at_base_ = true; + return; + } + } + } + + AssertInvariants(); +} + +PrepareFilterIterator::PrepareFilterIterator( + DB* db, ColumnFamilyHandle* cf, const std::shared_ptr& core, + std::unique_ptr input, Logger* info_log) + : Iterator(), + db_(db), + cf_(cf), + core_(core), + input_(std::move(input)), + valid_(false), + forward_(true) {} + +bool PrepareFilterIterator::Valid() const { return valid_; } + +Slice PrepareFilterIterator::key() const { + invariant(valid_); + return key_; +} + +Slice PrepareFilterIterator::value() const { + invariant(valid_); + return val_; +} + +Status PrepareFilterIterator::status() const { return status_; } + +void PrepareFilterIterator::SeekToFirst() { + forward_ = true; + input_->SeekToFirst(); + UpdateCurrent(); +} + +void PrepareFilterIterator::SeekToLast() { + forward_ = false; + input_->SeekToLast(); + UpdateCurrent(); +} + +void PrepareFilterIterator::Seek(const Slice& k) { + forward_ = true; + input_->Seek(k); + UpdateCurrent(); +} + +void PrepareFilterIterator::SeekForPrev(const Slice& k) { + forward_ = false; + input_->SeekForPrev(k); + UpdateCurrent(); +} + +void PrepareFilterIterator::Next() { + if (!Valid()) { + status_ = Status::NotSupported("Next() on invalid iterator"); + return; + } + if (forward_ && IsPrepareConflict(status_)) { + UpdatePrepareState(); + return; + } + // Suppose at t0, base points at b, pmap points at c with c.state == prepared, + // the merged cursor points at the base key `b`. At t1, c changes to committed + // state. However, the merged cursor can not find that. At t2, the merged + // cursor advances to c, with c.state == prepared, not changed. base: + // ---[a]---[CURRENT(b)]----[c]---------- pmap: + // ---[a]-------------------[c(CURRENT)]- + bool next_may_read_staled_prepared = + forward_ && sval_.has_base && !sval_.has_prepare; + forward_ = true; + AdvanceInputNoFilter(); + UpdateCurrent(); + // The previous cursor state points at base, and after a next call, we get a + // prepare state. This is the only situation we read a staled prepare state. + bool read_staled_prepared = + next_may_read_staled_prepared && Valid() && IsPrepareConflict(status_); + if (read_staled_prepared) { + UpdatePrepareState(); + } +} + +void PrepareFilterIterator::Prev() { + if (!Valid()) { + status_ = Status::NotSupported("Prev() on invalid iterator"); + return; + } + if (!forward_ && IsPrepareConflict(status_)) { + UpdatePrepareState(); + return; + } + bool next_may_read_staled_prepared = + (!forward_) && sval_.has_base && !sval_.has_prepare; + forward_ = false; + AdvanceInputNoFilter(); + UpdateCurrent(); + bool read_staled_prepared = + next_may_read_staled_prepared && Valid() && IsPrepareConflict(status_); + if (read_staled_prepared) { + UpdatePrepareState(); + } +} + +void PrepareFilterIterator::UpdatePrepareState() { + invariant(sval_.prepare_value_state == + TOTransaction::TOTransactionState::kPrepared); + invariant(sval_.has_prepare); + auto new_state = input_->UpdatePrepareState(); + if (new_state == TOTransaction::TOTransactionState::kPrepared) { + return; + } + if (new_state == TOTransaction::TOTransactionState::kCommitted) { + UpdateCurrent(); + return; + } + invariant(new_state == TOTransaction::TOTransactionState::kRollback); + if (forward_) { + Seek(key_); + } else { + SeekForPrev(key_); + } +} + +void PrepareFilterIterator::AdvanceInputNoFilter() { + if (!input_->Valid()) { + valid_ = false; + return; + } + if (forward_) { + input_->Next(); + } else { + input_->Prev(); + } + valid_ = input_->Valid(); +} + +// rocksdb internal api, for sanity check +// WBWIIteratorImpl::Result PrepareFilterIterator::GetFromBatch( +// WriteBatchWithIndex* batch, const Slice& key, std::string* value) { +// Status s; +// WriteBatchWithIndexInternal wbwidx(cf_); +// WBWIIteratorImpl::Result result = wbwidx.GetFromBatch(batch, key, value, &s); +// invariant(s.ok()); +// return result; +// } + +void PrepareFilterIterator::UpdateCurrent() { + while (true) { + status_ = Status::OK(); + if (!input_->Valid()) { + valid_ = false; + return; + } + valid_ = true; + key_ = input_->key(); + sval_ = input_->value(); + if (!sval_.has_prepare) { + // TODO(deyukong): eliminate copy + val_ = std::string(sval_.base_value.data(), sval_.base_value.size()); + return; + } else if (!sval_.has_base) { + const auto pmap_val = sval_.prepare_value; + auto state = sval_.prepare_value_state; + invariant(state != TOTransaction::TOTransactionState::kRollback); + if (state == TOTransaction::TOTransactionState::kCommitted) { +// #ifndef NDEBUG +// auto res = GetFromBatch(&pmap_val->write_batch_, key_, &val_); +// #endif // NDEBUG +// invariant(res == WBWIIteratorImpl::Result::kFound || +// res == WBWIIteratorImpl::Result::kDeleted); + MACRO_REFETCH_RETURN_ON_FOUND_ADVANCE_CONTINUE_ON_NOT_FOUND(); + } else { + invariant(state == TOTransaction::TOTransactionState::kPrepared); + if (pmap_val->prepare_ts_ > core_->read_ts_) { + AdvanceInputNoFilter(); + } else if (core_->ignore_prepare_) { + AdvanceInputNoFilter(); + } else { + status_ = PrepareConflict(); + return; + } + } + } else { + invariant(sval_.has_prepare && sval_.has_base); + const auto pmap_val = sval_.prepare_value; + auto state = sval_.prepare_value_state; + invariant(state != TOTransaction::TOTransactionState::kRollback); + // a key from my own batch has timestamp == 0 to intend the "read-own-writes" rule + // TODO(wolfkdy): WriteBatchWithIndexIterator impls timestamp() interface + // invariant(sval_.base_timestamp <= core_->read_ts_ || sval_.base_timestamp == 0); +// #ifndef NDEBUG +// auto res = GetFromBatch(&pmap_val->write_batch_, key_, &val_); +// #endif // NDEBUG +// invariant(res == WBWIIteratorImpl::Result::kFound || +// res == WBWIIteratorImpl::Result::kDeleted); + if (state == TOTransaction::TOTransactionState::kCommitted) { + MACRO_REFETCH_RETURN_ON_FOUND_ADVANCE_CONTINUE_ON_NOT_FOUND(); + } else { + invariant(state == TOTransaction::TOTransactionState::kPrepared); + // Txn is committed before changing state from kPrepared to kCommitted, + // this is not in a critial section. So another interleaved txn may see + // both its committed data and its kPrepared state. + // TODO(wolfkdy): WriteBatchWithIndexIterator impls timestamp() interface + // invariant(sval_.base_timestamp < pmap_val->prepare_ts_ || + // (pmap_val->commit_ts_set_ && + // sval_.base_timestamp == pmap_val->commit_ts_ && + // sval_.base_value == val_)); + if (pmap_val->prepare_ts_ > core_->read_ts_) { + val_ = std::string(sval_.base_value.data(), sval_.base_value.size()); + return; + } else if (core_->ignore_prepare_) { + val_ = std::string(sval_.base_value.data(), sval_.base_value.size()); + return; + } else { + status_ = PrepareConflict(); + return; + } + } + } + } +} + +} // namespace rocksdb diff --git a/src/totdb/totransaction_prepare_iterator.h b/src/totdb/totransaction_prepare_iterator.h new file mode 100644 index 0000000..410d157 --- /dev/null +++ b/src/totdb/totransaction_prepare_iterator.h @@ -0,0 +1,223 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). + +#pragma once +#ifndef ROCKSDB_LITE + +#include "mongo/db/modules/rocks/src/totdb/totransaction_db_impl.h" +#include "mongo/db/modules/rocks/src/totdb/totransaction_impl.h" + +namespace rocksdb { + +// PrepareFilterIterator +// | +// --PrepareMergingIterator +// | +// -- PrepareMapIterator +// | +// -- BaseIterator +// | +// -- WriteBatchWithIndexIterator +// | +// -- Normal LsmTree Iterator +// | +// -- ...... +// +// 1) `PrepareFilterIterator` checks prepare status of an input entry and +// decides +// to return, wait, or advance to the next record +// 2) PrepareMergingIterator arranges PrepareMapIterator and BaseIterator into +// total-order. if PrepareMapIterator and BaseIterator has the same key, they +// are both returned by `ShadowValue`, it is impossible that the same key +// comes from me(or WriteBatchWithIndexIterator) because the only operations +// after `prepare` is rollback or commit + +class PrepareMapIterator { + public: + PrepareMapIterator(ColumnFamilyHandle* cf, PrepareHeap* ph, + TOTransactionImpl::ActiveTxnNode* core) + : cf_(cf), ph_(ph), core_(core), valid_(false) {} + + bool Valid() const { return valid_; } + + void SeekToFirst(); + + void SeekToLast(); + + void Seek(const Slice& target); + + void SeekForPrev(const Slice& target); + + void Next(); + + void Prev(); + + Slice key() const; + + const std::shared_ptr& value() const; + + TOTransaction::TOTransactionState valueState() const; + + TOTransaction::TOTransactionState UpdatePrepareState(); + + PrepareHeap* getPrepareHeapMap() const; + + private: + void TryPosValueToCorrectMvccVersionInLock( + const std::list>& + prepare_mvccs); + + ColumnFamilyHandle* cf_; // not owned + + PrepareHeap* ph_; // not owned + + TOTransactionImpl::ActiveTxnNode* core_; // not owned + + bool forward_; + + bool valid_; + + std::string pos_; + + std::shared_ptr val_; + + TOTransaction::TOTransactionState val_state_; +}; + +struct ShadowValue { + bool has_prepare; + bool has_base; + TOTransactionImpl::ActiveTxnNode* prepare_value; + TOTransaction::TOTransactionState prepare_value_state; + Slice base_value; + Slice base_key; + RocksTimeStamp base_timestamp; +}; + +class PrepareMergingIterator { + public: + PrepareMergingIterator(std::unique_ptr base_iterator, + std::unique_ptr pmap_iterator); + + bool Valid() const; + + Slice key() const; + + ShadowValue value() const; + + void SeekToFirst(); + + void SeekToLast(); + + void Seek(const Slice& k); + + void SeekForPrev(const Slice& k); + + void Next(); + + void Prev(); + + Status status() const; + + TOTransaction::TOTransactionState UpdatePrepareState(); + + private: + void Advance(); + + void AdvanceDelta(); + + void AdvanceBase(); + + void AssertInvariants(); + + void UpdateCurrent(); + + bool BaseValid() const; + + bool DeltaValid() const; + + bool forward_; + + bool current_at_base_; + + bool equal_keys_; + + Status status_; + + std::unique_ptr base_iterator_; + + std::unique_ptr delta_iterator_; + + const Comparator* comparator_; // not owned +}; + +class PrepareFilterIterator : public Iterator { + public: + PrepareFilterIterator(DB* db, ColumnFamilyHandle* cf, + const std::shared_ptr& core, + std::unique_ptr input, + Logger* info_log = nullptr); + + bool Valid() const final; + + Slice key() const final; + + Slice value() const final; + + void SeekToFirst() final; + + void SeekToLast() final; + + void Seek(const Slice& k) final; + + void SeekForPrev(const Slice& k) final; + + void Next() final; + + void Prev() final; + + Status status() const; + + private: + // rocksdb internal api, for sanity check + // WBWIIteratorImpl::Result GetFromBatch(WriteBatchWithIndex* batch, + // const Slice& key, + // std::string* val); + + void AdvanceInputNoFilter(); + + void UpdateCurrent(); + + void UpdatePrepareState(); + + DB* db_; + + ColumnFamilyHandle* cf_; + + // Iterator's lifetime should be shorter than the Transaction who created it. + // So here core_ should be a raw pointer rather than shared_ptr. However, + // MultiIndexBlock::insertAllDocumentsInCollection breaks this. The cursor + // in exec has longer lifetime than WriteUnitOfWork, so we have to workaround + // with shared_ptr. + std::shared_ptr core_; + + std::unique_ptr input_; + + Slice key_; + + std::string val_; + + ShadowValue sval_; + + bool valid_; + + bool forward_; + + Status status_; +}; + +} // namespace rocksdb + +#endif // ROCKSDB_LITE diff --git a/src/totdb/totransaction_test.cpp b/src/totdb/totransaction_test.cpp new file mode 100644 index 0000000..a5eb0b8 --- /dev/null +++ b/src/totdb/totransaction_test.cpp @@ -0,0 +1,2610 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). + +#include +#include +#include + +#include "rocksdb/db.h" +#include "third_party/s2/util/coding/coder.h" +#include "mongo/db/modules/rocks/src/totdb/totransaction_db.h" +#include "mongo/db/modules/rocks/src/totdb/totransaction_db_impl.h" +#include "mongo/db/modules/rocks/src/totdb/totransaction_impl.h" +#include "mongo/db/modules/rocks/src/rocks_util.h" +#include "mongo/unittest/unittest.h" + +using std::string; + +namespace rocksdb { + +#define ASSERT_ROCKS_OK(EXPRESSION) ASSERT_EQUALS(::mongo::Status::OK(), ::mongo::rocksToMongoStatus(EXPRESSION)) +#define ASSERT_ROCKS_NOK(EXPRESSION) ASSERT_NOT_EQUALS(::mongo::Status::OK(), ::mongo::rocksToMongoStatus(EXPRESSION)) + +class TOTransactionTest : public ::mongo::unittest::Test { + public: + TOTransactionDB* txn_db; + string dbname; + Options options; + TOTransactionDBOptions txndb_options{200}; + TOTransactionOptions txn_options; + + TOTransactionTest() { + options.create_if_missing = true; + options.max_write_buffer_number = 2; + options.comparator = &comparator_; + dbname = /*test::TmpDir() +*/ "./totransaction_testdb"; + + DestroyDB(dbname, options); + Open(); + } + ~TOTransactionTest() { + delete txn_db; + DestroyDB(dbname, options); + } + + void Reopen() { + delete txn_db; + txn_db = nullptr; + Open(); + } + +void Reopen(Options newOptions) { + delete txn_db; + txn_db = nullptr; + Open(newOptions); +} + +private: + void Open() { + Status s = TOTransactionDBImpl::Open(options, txndb_options, dbname, kStablePrefix, &txn_db); + invariant(s.ok()); + invariant(txn_db != nullptr); + } + + void Open(Options newOptions) { + Status s = TOTransactionDBImpl::Open(newOptions, txndb_options, dbname, kStablePrefix, &txn_db); + invariant(s.ok()); + invariant(txn_db != nullptr); + } + + static const std::string kStablePrefix; + TOComparator comparator_; +}; + +const std::string TOTransactionTest::kStablePrefix("\0\0\0\0stableTs-", 13); + +TEST_F(TOTransactionTest, ValidateIOWithoutTimestamp) { + WriteOptions write_options; + ReadOptions read_options; + string value; + Status s; + TOTransactionStat stat; + + ASSERT_ROCKS_OK(s); + // txn1 test put and get + TOTransaction* txn = txn_db->BeginTransaction(write_options, txn_options); + ASSERT_TRUE(txn); + + s = txn->Put(Slice("foo"), Slice("bar")); + ASSERT_ROCKS_OK(s); + + s = txn->Commit(); + ASSERT_ROCKS_OK(s); + delete txn; + txn = txn_db->BeginTransaction(write_options, txn_options); + + ASSERT_ROCKS_OK(txn->Get(read_options, "foo", &value)); + ASSERT_EQ(value, "bar"); + + s = txn->Commit(); + ASSERT_ROCKS_OK(s); + + s = txn_db->Stat(&stat); + ASSERT_ROCKS_OK(s); + ASSERT_EQ(stat.commit_without_ts_times, 2U); + ASSERT_EQ(stat.read_without_ts_times, 2U); + ASSERT_EQ(stat.txn_commits, 2U); + ASSERT_EQ(stat.txn_aborts, 0U); + delete txn; +} + +TEST_F(TOTransactionTest, ValidateIO) { + WriteOptions write_options; + char ts_buf[8]; + Slice ts_slice(ts_buf, sizeof(ts_buf)); + Encoder(ts_buf, sizeof(ts_buf)).put64(50); + ReadOptions read_options; + read_options.timestamp = &ts_slice; + string value; + Status s; + TOTransactionStat stat; + + s = txn_db->SetTimeStamp(kOldest, 10, false); + ASSERT_ROCKS_OK(s); + // txn1 test put and get + TOTransaction* txn = txn_db->BeginTransaction(write_options, txn_options); + ASSERT_TRUE(txn); + + s = txn->SetReadTimeStamp(50); + ASSERT_ROCKS_OK(s); + + s = txn->Put(Slice("foo"), Slice("bar")); + ASSERT_ROCKS_OK(s); + + s = txn->Put(Slice("key3"), Slice("value3")); + ASSERT_ROCKS_OK(s); + + s = txn->SetCommitTimeStamp(100); + ASSERT_ROCKS_OK(s); + + // Read your write + s = txn->Get(read_options, "foo", &value); + ASSERT_ROCKS_OK(s); + ASSERT_EQ(value, "bar"); + + s = txn->Commit(); + ASSERT_ROCKS_OK(s); + + delete txn; + s = txn_db->Stat(&stat); + ASSERT_ROCKS_OK(s); + ASSERT_EQ(stat.commit_without_ts_times, 0U); + ASSERT_EQ(stat.read_without_ts_times, 0U); + ASSERT_EQ(stat.txn_commits, 1U); + ASSERT_EQ(stat.txn_aborts, 0U); + ASSERT_EQ(stat.read_q_walk_times, 0U); + ASSERT_EQ(stat.commit_q_walk_times, 0U); + + // + // txn2 test iterator + s = txn_db->SetTimeStamp(kOldest, 10, false); + + txn = txn_db->BeginTransaction(write_options, txn_options); + txn->SetReadTimeStamp(101); + + txn->Get(read_options, "foo", &value); + ASSERT_EQ(value, "bar"); + + s = txn->Put(Slice("foo"), Slice("bar2")); + ASSERT_ROCKS_OK(s); + + s = txn->Put(Slice("key1"), Slice("value1")); + ASSERT_ROCKS_OK(s); + s = txn->Put(Slice("key2"), Slice("value2")); + ASSERT_ROCKS_OK(s); + + s = txn->Get(read_options, "foo", &value); + ASSERT_ROCKS_OK(s); + ASSERT_EQ(value, "bar2"); + + Iterator* iter = txn->GetIterator(read_options); + ASSERT_TRUE(iter); + + iter->SeekToFirst(); + ASSERT_ROCKS_OK(iter->status()); + ASSERT_TRUE(iter->Valid()); + + Slice key = iter->key(); + Slice val = iter->value(); + + ASSERT_EQ(key.ToString(), "foo"); + ASSERT_EQ(val.ToString(), "bar2"); + + iter->Next(); + ASSERT_TRUE(iter->Valid()); + key = iter->key(); + val = iter->value(); + ASSERT_EQ(key.ToString(), "key1"); + ASSERT_EQ(val.ToString(), "value1"); + + iter->Next(); + key = iter->key(); + val = iter->value(); + ASSERT_EQ(key.ToString(), "key2"); + ASSERT_EQ(val.ToString(), "value2"); + + delete iter; + + s = txn->SetCommitTimeStamp(105); + ASSERT_ROCKS_OK(s); + s = txn->Commit(); + ASSERT_ROCKS_OK(s); + + delete txn; + s = txn_db->Stat(&stat); + ASSERT_ROCKS_OK(s); + ASSERT_EQ(stat.commit_without_ts_times, 0U); + ASSERT_EQ(stat.read_without_ts_times, 0U); + ASSERT_EQ(stat.txn_commits, 2U); + ASSERT_EQ(stat.txn_aborts, 0U); + ASSERT_EQ(stat.read_q_num, 1U); + ASSERT_EQ(stat.commit_q_num, 1U); + + // txn3 test write conflict + txn = txn_db->BeginTransaction(write_options, txn_options); + s = txn->SetReadTimeStamp(101); + ASSERT_ROCKS_OK(s); + + txn->Get(read_options, "foo", &value); + ASSERT_EQ(value, "bar"); + + s = txn->Put(Slice("key4"), Slice("value4")); + ASSERT_ROCKS_OK(s); + + // Write Conflict here, there is a txn committed before + // whose commit ts is greater than my read ts + s = txn->Put(Slice("key1"), Slice("value1")); + ASSERT_TRUE(s.IsBusy()); + + s = txn->Rollback(); + ASSERT_ROCKS_OK(s); + + delete txn; + s = txn_db->Stat(&stat); + ASSERT_ROCKS_OK(s); + ASSERT_EQ(stat.commit_without_ts_times, 0U); + ASSERT_EQ(stat.read_without_ts_times, 0U); + ASSERT_EQ(stat.txn_commits, 2U); + ASSERT_EQ(stat.txn_aborts, 1U); + ASSERT_EQ(stat.read_q_num, 1U); + ASSERT_EQ(stat.commit_q_num, 1U); + + // txn4 + txn = txn_db->BeginTransaction(write_options, txn_options); + s = txn->SetReadTimeStamp(106); + ASSERT_ROCKS_OK(s); + + // No write conflict here + s = txn->Put(Slice("key1"), Slice("value1")); + ASSERT_ROCKS_OK(s); + + s = txn->SetCommitTimeStamp(110); + s = txn->Commit(); + ASSERT_ROCKS_OK(s); + + delete txn; + + // txn5 test delete + txn = txn_db->BeginTransaction(write_options, txn_options); + TOTransaction* txn2 = txn_db->BeginTransaction(write_options, txn_options); + + s = txn2->SetReadTimeStamp(110); + ASSERT_ROCKS_OK(s); + s = txn->SetReadTimeStamp(110); + ASSERT_ROCKS_OK(s); + + s = txn2->Get(read_options, "foo", &value); + ASSERT_EQ(value, "bar2"); + + s = txn->Delete(Slice("foo")); + ASSERT_ROCKS_OK(s); + + s = txn->SetCommitTimeStamp(120); + ASSERT_ROCKS_OK(s); + + s = txn->Commit(); + ASSERT_ROCKS_OK(s); + + // snapshot isolation + s = txn2->Get(read_options, "foo", &value); + ASSERT_EQ(value, "bar2"); + + s = txn2->SetCommitTimeStamp(121); + ASSERT_ROCKS_OK(s); + s = txn2->Commit(); + ASSERT_ROCKS_OK(s); + + s = txn_db->Stat(&stat); + ASSERT_ROCKS_OK(s); + ASSERT_EQ(stat.commit_without_ts_times, 0U); + ASSERT_EQ(stat.read_without_ts_times, 0U); + ASSERT_EQ(stat.txn_commits, 5U); + ASSERT_EQ(stat.txn_aborts, 1U); + + delete txn; + delete txn2; + +} + +TEST_F(TOTransactionTest, ValidateWriteConflict) { + WriteOptions write_options; + char ts_buf[8]; + Slice ts_slice(ts_buf, sizeof(ts_buf)); + Encoder(ts_buf, sizeof(ts_buf)).put64(50); + ReadOptions read_options; + read_options.timestamp = &ts_slice; + + string value; + Status s; + + s = txn_db->SetTimeStamp(kOldest, 10, false); + ASSERT_ROCKS_OK(s); + // txn1 test write conflict + // txn1 and txn2 both modify foo + // first update wins + TOTransaction* txn = txn_db->BeginTransaction(write_options, txn_options); + ASSERT_TRUE(txn); + + TOTransaction* txn2 = txn_db->BeginTransaction(write_options, txn_options); + ASSERT_TRUE(txn2); + + ASSERT_TRUE(txn->GetID() < txn2->GetID()); + + s = txn->SetReadTimeStamp(50); + ASSERT_ROCKS_OK(s); + + s = txn->Put(Slice("foo"), Slice("bar")); + ASSERT_ROCKS_OK(s); + + s = txn2->Put(Slice("foo"), Slice("bar2")); + ASSERT_TRUE(s.IsBusy()); + + s = txn2->Rollback(); + ASSERT_ROCKS_OK(s); + + delete txn2; + + s = txn->SetCommitTimeStamp(100); + ASSERT_ROCKS_OK(s); + + // Read your write + s = txn->Get(read_options, "foo", &value); + ASSERT_ROCKS_OK(s); + ASSERT_EQ(value, "bar"); + + s = txn->Commit(); + ASSERT_ROCKS_OK(s); + + delete txn; + // txn2 test write conflict + // txn1 began before txn2, txn2 modified foo and commit + // txn1 tried to modify foo + s = txn_db->SetTimeStamp(kOldest, 10, false); + + txn = txn_db->BeginTransaction(write_options, txn_options); + txn->SetReadTimeStamp(101); + txn2 = txn_db->BeginTransaction(write_options, txn_options); + txn2->SetReadTimeStamp(101); + + txn->Get(read_options, "foo", &value); + ASSERT_EQ(value, "bar"); + + s = txn2->Put(Slice("foo"), Slice("bar2")); + ASSERT_ROCKS_OK(s); + + s = txn2->Put(Slice("key1"), Slice("value1")); + ASSERT_ROCKS_OK(s); + s = txn2->Put(Slice("key2"), Slice("value2")); + ASSERT_ROCKS_OK(s); + + s = txn2->Get(read_options, "foo", &value); + ASSERT_ROCKS_OK(s); + ASSERT_EQ(value, "bar2"); + + Iterator* iter = txn2->GetIterator(read_options); + ASSERT_TRUE(iter); + + iter->SeekToFirst(); + ASSERT_ROCKS_OK(iter->status()); + ASSERT_TRUE(iter->Valid()); + + Slice key = iter->key(); + Slice val = iter->value(); + + ASSERT_EQ(key.ToString(), "foo"); + ASSERT_EQ(val.ToString(), "bar2"); + + iter->Next(); + key = iter->key(); + val = iter->value(); + ASSERT_EQ(key.ToString(), "key1"); + ASSERT_EQ(val.ToString(), "value1"); + + iter->Next(); + key = iter->key(); + val = iter->value(); + ASSERT_EQ(key.ToString(), "key2"); + ASSERT_EQ(val.ToString(), "value2"); + + delete iter; + + s = txn2->SetCommitTimeStamp(105); + s = txn2->Commit(); + ASSERT_ROCKS_OK(s); + + s = txn->Put(Slice("foo"), Slice("bar3")); + ASSERT_TRUE(s.IsBusy()); + + s = txn->Rollback(); + ASSERT_ROCKS_OK(s); + + delete txn; + delete txn2; + // txn3 test write conflict + txn = txn_db->BeginTransaction(write_options, txn_options); + s = txn->SetReadTimeStamp(101); + ASSERT_ROCKS_OK(s); + + txn->Get(read_options, "foo", &value); + ASSERT_EQ(value, "bar"); + + s = txn->Put(Slice("key4"), Slice("value4")); + ASSERT_ROCKS_OK(s); + + // Write Conflict here, there is a txn committed before + // whose commit ts is greater than my read ts + s = txn->Put(Slice("key1"), Slice("value1_1")); + ASSERT_TRUE(s.IsBusy()); + + s = txn->Rollback(); + ASSERT_ROCKS_OK(s); + + delete txn; + + // txn4 + txn = txn_db->BeginTransaction(write_options, txn_options); + s = txn->SetReadTimeStamp(106); + ASSERT_ROCKS_OK(s); + + // No write conflict here + s = txn->Put(Slice("key1"), Slice("value1")); + ASSERT_ROCKS_OK(s); + + s = txn->SetCommitTimeStamp(110); + s = txn->Commit(); + ASSERT_ROCKS_OK(s); + + delete txn; +} + +TEST_F(TOTransactionTest, ValidateIsolation) { + WriteOptions write_options; + char ts_buf[8]; + Slice ts_slice(ts_buf, sizeof(ts_buf)); + Encoder(ts_buf, sizeof(ts_buf)).put64(50); + ReadOptions read_options; + read_options.timestamp = &ts_slice; + string value; + Status s; + + s = txn_db->SetTimeStamp(kOldest, 10, false); + ASSERT_ROCKS_OK(s); + // txn1 test snapshot isolation + TOTransaction* txn = txn_db->BeginTransaction(write_options, txn_options); + ASSERT_TRUE(txn); + + TOTransaction* txn2 = txn_db->BeginTransaction(write_options, txn_options); + ASSERT_TRUE(txn2); + + ASSERT_TRUE(txn->GetID() < txn2->GetID()); + + s = txn->SetReadTimeStamp(50); + ASSERT_ROCKS_OK(s); + + s = txn->Put(Slice("A"), Slice("A-A")); + ASSERT_ROCKS_OK(s); + + s = txn->SetCommitTimeStamp(100); + ASSERT_ROCKS_OK(s); + + s = txn->Commit(); + ASSERT_ROCKS_OK(s); + + RocksTimeStamp all_committed_ts; + s = txn_db->QueryTimeStamp(kAllCommitted, &all_committed_ts); + ASSERT_ROCKS_OK(s); + + s = txn2->Put(Slice("B"), Slice("B-B")); + ASSERT_ROCKS_OK(s); + + s = txn2->SetCommitTimeStamp(110); + ASSERT_ROCKS_OK(s); + + TOTransaction* txn3 = txn_db->BeginTransaction(write_options, txn_options); + ASSERT_TRUE(txn3); + + TOTransaction* txn4 = txn_db->BeginTransaction(write_options, txn_options); + ASSERT_TRUE(txn4); + + s = txn3->SetReadTimeStamp(60); + ASSERT_ROCKS_OK(s); + + s = txn4->SetReadTimeStamp(110); + ASSERT_ROCKS_OK(s); + + s = txn3->Get(read_options, "A", &value); + ASSERT_TRUE(s.IsNotFound()); + + s = txn4->Get(read_options, "A", &value); + ASSERT_ROCKS_OK(s); + ASSERT_EQ(value, "A-A"); + + s = txn3->Get(read_options, "B", &value); + ASSERT_TRUE(s.IsNotFound()); + + s = txn4->Get(read_options, "B", &value); + ASSERT_TRUE(s.IsNotFound()); + + s = txn2->Commit(); + ASSERT_ROCKS_OK(s); + + s = txn3->Get(read_options, "B", &value); + ASSERT_TRUE(s.IsNotFound()); + + s = txn4->Get(read_options, "B", &value); + ASSERT_TRUE(s.IsNotFound()); + + s = txn3->Rollback(); + ASSERT_ROCKS_OK(s); + + s = txn4->Rollback(); + ASSERT_ROCKS_OK(s); + + delete txn; + delete txn2; + delete txn3; + delete txn4; + + txn = txn_db->BeginTransaction(write_options, txn_options); + ASSERT_TRUE(txn); + + txn2 = txn_db->BeginTransaction(write_options, txn_options); + ASSERT_TRUE(txn2); + + s = txn->SetReadTimeStamp(110); + ASSERT_ROCKS_OK(s); + + s = txn2->SetReadTimeStamp(110); + ASSERT_ROCKS_OK(s); + + s = txn->Put(Slice("C"), Slice("C-C")); + ASSERT_ROCKS_OK(s); + + s = txn->Put(Slice("H"), Slice("H-H")); + ASSERT_ROCKS_OK(s); + + s = txn->Put(Slice("J"), Slice("J-J")); + ASSERT_ROCKS_OK(s); + + Iterator* iter = txn->GetIterator(read_options); + ASSERT_TRUE(iter); + + iter->SeekToFirst(); + ASSERT_ROCKS_OK(iter->status()); + ASSERT_TRUE(iter->Valid()); + + Slice key = iter->key(); + Slice val = iter->value(); + + ASSERT_EQ(key.ToString(), "A"); + ASSERT_EQ(val.ToString(), "A-A"); + + iter->Next(); + key = iter->key(); + val = iter->value(); + ASSERT_EQ(key.ToString(), "B"); + ASSERT_EQ(val.ToString(), "B-B"); + + s = txn2->Put(Slice("E"), Slice("E-E")); + ASSERT_ROCKS_OK(s); + + s = txn2->SetCommitTimeStamp(120); + ASSERT_ROCKS_OK(s); + s = txn2->Commit(); + ASSERT_ROCKS_OK(s); + + iter->Next(); + key = iter->key(); + val = iter->value(); + ASSERT_EQ(key.ToString(), "C"); + ASSERT_EQ(val.ToString(), "C-C"); + + s = txn->Put(Slice("D"), Slice("D-D")); + ASSERT_ROCKS_OK(s); + + iter->Next(); + key = iter->key(); + val = iter->value(); + ASSERT_EQ(key.ToString(), "D"); + ASSERT_EQ(val.ToString(), "D-D"); + + iter->Next(); + key = iter->key(); + val = iter->value(); + ASSERT_EQ(key.ToString(), "H"); + ASSERT_EQ(val.ToString(), "H-H"); + + s = txn->Put(Slice("F"), Slice("F-F")); + ASSERT_ROCKS_OK(s); + + iter->Next(); + key = iter->key(); + val = iter->value(); + ASSERT_EQ(key.ToString(), "J"); + ASSERT_EQ(val.ToString(), "J-J"); + + delete iter; + + s = txn->SetCommitTimeStamp(120); + s = txn->Commit(); + ASSERT_ROCKS_OK(s); + + delete txn; + delete txn2; +} + + +TEST_F(TOTransactionTest, CommitTsCheck) { + WriteOptions write_options; + char ts_buf[8]; + Slice ts_slice(ts_buf, sizeof(ts_buf)); + Encoder(ts_buf, sizeof(ts_buf)).put64(50); + ReadOptions read_options; + read_options.timestamp = &ts_slice; + string value; + Status s; + + s = txn_db->SetTimeStamp(kOldest, 10, false); + ASSERT_ROCKS_OK(s); + + TOTransaction* txn = txn_db->BeginTransaction(write_options, txn_options); + ASSERT_TRUE(txn); + + TOTransaction* txn2 = txn_db->BeginTransaction(write_options, txn_options); + ASSERT_TRUE(txn2); + + s = txn->SetReadTimeStamp(100); + ASSERT_ROCKS_OK(s); + + s = txn->SetCommitTimeStamp(120); + ASSERT_ROCKS_OK(s); + + s = txn2->SetReadTimeStamp(100); + ASSERT_ROCKS_OK(s); + + s = txn2->SetCommitTimeStamp(130); + ASSERT_ROCKS_OK(s); + + RocksTimeStamp all_committed_ts; + s = txn_db->QueryTimeStamp(kAllCommitted, &all_committed_ts); + ASSERT_TRUE(s.IsNotFound()); + + s = txn2->Commit(); + ASSERT_ROCKS_OK(s); + + s = txn_db->QueryTimeStamp(kAllCommitted, &all_committed_ts); + ASSERT_TRUE(s.ok()); + ASSERT_EQ(all_committed_ts, 119U); + + s = txn->Commit(); + ASSERT_ROCKS_OK(s); + + s = txn_db->QueryTimeStamp(kAllCommitted, &all_committed_ts); + ASSERT_ROCKS_OK(s); + ASSERT_EQ(all_committed_ts, 130U); + + delete txn; + delete txn2; + +} + +TEST_F(TOTransactionTest, CommitTsCheck2) { + WriteOptions write_options; + ReadOptions read_options; + string value; + Status s; + + s = txn_db->SetTimeStamp(kOldest, 10, false); + ASSERT_ROCKS_OK(s); + + TOTransaction* txn = txn_db->BeginTransaction(write_options, txn_options); + ASSERT_TRUE(txn); + + TOTransaction* txn2 = txn_db->BeginTransaction(write_options, txn_options); + ASSERT_TRUE(txn2); + + TOTransaction* txn3 = txn_db->BeginTransaction(write_options, txn_options); + ASSERT_TRUE(txn3); + + s = txn->SetReadTimeStamp(100); + ASSERT_ROCKS_OK(s); + + s = txn->SetCommitTimeStamp(100); + ASSERT_ROCKS_OK(s); + + s = txn2->SetReadTimeStamp(100); + ASSERT_ROCKS_OK(s); + + s = txn2->SetCommitTimeStamp(120); + ASSERT_ROCKS_OK(s); + + s = txn3->SetReadTimeStamp(100); + ASSERT_ROCKS_OK(s); + + s = txn3->SetCommitTimeStamp(130); + ASSERT_ROCKS_OK(s); + + RocksTimeStamp all_committed_ts; + s = txn_db->QueryTimeStamp(kAllCommitted, &all_committed_ts); + ASSERT_TRUE(s.IsNotFound()); + s = txn->Put(Slice("1"), Slice("1")); + ASSERT_TRUE(s.ok()); + + s = txn->Commit(); + ASSERT_TRUE(s.ok()); + + s = txn_db->QueryTimeStamp(kAllCommitted, &all_committed_ts); + ASSERT_ROCKS_OK(s); + ASSERT_EQ(all_committed_ts, 100U); + + s = txn3->Put(Slice("3"), Slice("3")); + ASSERT_TRUE(s.ok()); + + s = txn3->Commit(); + ASSERT_ROCKS_OK(s); + + s = txn_db->QueryTimeStamp(kAllCommitted, &all_committed_ts); + ASSERT_ROCKS_OK(s); + ASSERT_EQ(all_committed_ts, 119U); + + s = txn2->Put(Slice("2"), Slice("2")); + ASSERT_TRUE(s.ok()); + + s = txn2->Commit(); + ASSERT_ROCKS_OK(s); + + s = txn_db->QueryTimeStamp(kAllCommitted, &all_committed_ts); + ASSERT_ROCKS_OK(s); + ASSERT_EQ(all_committed_ts, 130U); + + delete txn; + delete txn2; + delete txn3; + +} + +//no put +TEST_F(TOTransactionTest, CommitTsCheck3) { + WriteOptions write_options; + ReadOptions read_options; + string value; + Status s; + + s = txn_db->SetTimeStamp(kOldest, 10, false); + ASSERT_ROCKS_OK(s); + + TOTransaction* txn = txn_db->BeginTransaction(write_options, txn_options); + ASSERT_TRUE(txn); + + TOTransaction* txn2 = txn_db->BeginTransaction(write_options, txn_options); + ASSERT_TRUE(txn2); + + TOTransaction* txn3 = txn_db->BeginTransaction(write_options, txn_options); + ASSERT_TRUE(txn3); + + s = txn->SetReadTimeStamp(100); + ASSERT_ROCKS_OK(s); + + s = txn->SetCommitTimeStamp(100); + ASSERT_ROCKS_OK(s); + + s = txn2->SetReadTimeStamp(100); + ASSERT_ROCKS_OK(s); + + s = txn2->SetCommitTimeStamp(120); + ASSERT_ROCKS_OK(s); + + s = txn3->SetReadTimeStamp(100); + ASSERT_ROCKS_OK(s); + + s = txn3->SetCommitTimeStamp(130); + ASSERT_ROCKS_OK(s); + + RocksTimeStamp all_committed_ts; + s = txn_db->QueryTimeStamp(kAllCommitted, &all_committed_ts); + ASSERT_TRUE(s.IsNotFound()); + + s = txn->Commit(); + ASSERT_TRUE(s.ok()); + + s = txn_db->QueryTimeStamp(kAllCommitted, &all_committed_ts); + ASSERT_ROCKS_OK(s); + ASSERT_EQ(all_committed_ts, 100U); + + ASSERT_TRUE(s.ok()); + + s = txn3->Commit(); + ASSERT_ROCKS_OK(s); + + s = txn_db->QueryTimeStamp(kAllCommitted, &all_committed_ts); + ASSERT_ROCKS_OK(s); + ASSERT_EQ(all_committed_ts, 119U); + + ASSERT_TRUE(s.ok()); + + s = txn2->Commit(); + ASSERT_ROCKS_OK(s); + + s = txn_db->QueryTimeStamp(kAllCommitted, &all_committed_ts); + ASSERT_ROCKS_OK(s); + ASSERT_EQ(all_committed_ts, 130U); + + delete txn; + delete txn2; + delete txn3; + +} + +TEST_F(TOTransactionTest, CommitTsCheck4) { + WriteOptions write_options; + ReadOptions read_options; + RocksTimeStamp all_committed_ts; + string value; + Status s; + + TOTransaction* txn_ori = txn_db->BeginTransaction(write_options, txn_options); + ASSERT_TRUE(txn_ori); + s = txn_ori->SetCommitTimeStamp(4); + ASSERT_ROCKS_OK(s); + s = txn_ori->Commit(); + ASSERT_ROCKS_OK(s); + + s = txn_db->QueryTimeStamp(kAllCommitted, &all_committed_ts); + ASSERT_ROCKS_OK(s); + ASSERT_EQ(all_committed_ts, 4U); + + TOTransaction* txn = txn_db->BeginTransaction(write_options, txn_options); + ASSERT_TRUE(txn); + + TOTransaction* txn2 = txn_db->BeginTransaction(write_options, txn_options); + ASSERT_TRUE(txn2); + + TOTransaction* txn3 = txn_db->BeginTransaction(write_options, txn_options); + ASSERT_TRUE(txn3); + + TOTransaction* txn4 = txn_db->BeginTransaction(write_options, txn_options); + ASSERT_TRUE(txn4); + + s = txn->SetReadTimeStamp(all_committed_ts); + ASSERT_ROCKS_OK(s); + + s = txn2->SetCommitTimeStamp(5); + ASSERT_ROCKS_OK(s); + + s = txn3->SetCommitTimeStamp(6); + ASSERT_ROCKS_OK(s); + + s = txn2->Commit(); + ASSERT_ROCKS_OK(s); + s = txn3->Commit(); + ASSERT_ROCKS_OK(s); + + s = txn4->SetCommitTimeStamp(6); + ASSERT_ROCKS_OK(s); + s = txn4->Commit(); + ASSERT_ROCKS_OK(s); + + s = txn_db->QueryTimeStamp(kAllCommitted, &all_committed_ts); + ASSERT_ROCKS_OK(s); + ASSERT_EQ(all_committed_ts, 6U); + + delete txn_ori; + delete txn; + delete txn2; + delete txn3; + delete txn4; +} + +TEST_F(TOTransactionTest, Rollback) { + WriteOptions write_options; + ReadOptions read_options; + RocksTimeStamp all_committed_ts; + string value; + Status s; + + TOTransaction* txn_ori = txn_db->BeginTransaction(write_options, txn_options); + ASSERT_TRUE(txn_ori); + s = txn_ori->SetCommitTimeStamp(4); + ASSERT_ROCKS_OK(s); + s = txn_ori->Commit(); + ASSERT_ROCKS_OK(s); + + s = txn_db->QueryTimeStamp(kAllCommitted, &all_committed_ts); + ASSERT_ROCKS_OK(s); + ASSERT_EQ(all_committed_ts, 4U); + + TOTransaction* txn = txn_db->BeginTransaction(write_options, txn_options); + ASSERT_TRUE(txn); + + TOTransaction* txn2 = txn_db->BeginTransaction(write_options, txn_options); + ASSERT_TRUE(txn2); + + TOTransaction* txn3 = txn_db->BeginTransaction(write_options, txn_options); + ASSERT_TRUE(txn3); + + TOTransaction* txn4 = txn_db->BeginTransaction(write_options, txn_options); + ASSERT_TRUE(txn4); + + s = txn->SetReadTimeStamp(all_committed_ts); + ASSERT_ROCKS_OK(s); + + s = txn->SetCommitTimeStamp(5); + ASSERT_ROCKS_OK(s); + + s = txn2->SetCommitTimeStamp(6); + ASSERT_ROCKS_OK(s); + + s = txn3->SetCommitTimeStamp(7); + ASSERT_ROCKS_OK(s); + + s = txn2->Commit(); + ASSERT_ROCKS_OK(s); + s = txn3->Commit(); + ASSERT_ROCKS_OK(s); + + s = txn4->SetCommitTimeStamp(8); + ASSERT_ROCKS_OK(s); + s = txn4->Commit(); + ASSERT_ROCKS_OK(s); + + s = txn_db->QueryTimeStamp(kAllCommitted, &all_committed_ts); + ASSERT_ROCKS_OK(s); + ASSERT_EQ(all_committed_ts, 4U); + + s = txn->Rollback(); + ASSERT_ROCKS_OK(s); + + s = txn_db->QueryTimeStamp(kAllCommitted, &all_committed_ts); + ASSERT_ROCKS_OK(s); + ASSERT_EQ(all_committed_ts, 8U); + + delete txn_ori; + delete txn; + delete txn2; + delete txn3; + delete txn4; +} + +TEST_F(TOTransactionTest, AdvanceTSAndCleanInLock) { + WriteOptions write_options; + ReadOptions read_options; + RocksTimeStamp all_committed_ts; + RocksTimeStamp maxToCleanTs = 0; + string value; + Status s; + + s = txn_db->SetTimeStamp(kOldest, 3, false); + ASSERT_ROCKS_OK(s); + TOTransaction* txn_ori = txn_db->BeginTransaction(write_options, txn_options); + ASSERT_TRUE(txn_ori); + s = txn_ori->SetCommitTimeStamp(6); + ASSERT_ROCKS_OK(s); + s = txn_ori->Commit(); + ASSERT_ROCKS_OK(s); + + s = txn_db->QueryTimeStamp(kAllCommitted, &all_committed_ts); + ASSERT_ROCKS_OK(s); + ASSERT_EQ(all_committed_ts, 6U); + + TOTransaction* txn = txn_db->BeginTransaction(write_options, txn_options); + ASSERT_TRUE(txn); + + TOTransaction* txn2 = txn_db->BeginTransaction(write_options, txn_options); + ASSERT_TRUE(txn2); + + TOTransaction* txn3 = txn_db->BeginTransaction(write_options, txn_options); + ASSERT_TRUE(txn3); + + TOTransaction* txn4 = txn_db->BeginTransaction(write_options, txn_options); + ASSERT_TRUE(txn4); + + s = txn->SetReadTimeStamp(0); + ASSERT_TRUE(s.IsInvalidArgument()); + + s = txn->SetReadTimeStamp(3); + ASSERT_ROCKS_OK(s); + + s = txn->SetCommitTimeStamp(8); + ASSERT_ROCKS_OK(s); + + s = txn2->SetReadTimeStamp(5); + ASSERT_ROCKS_OK(s); + + s = txn2->SetCommitTimeStamp(7); + ASSERT_ROCKS_OK(s); + + s = txn3->SetReadTimeStamp(6); + ASSERT_ROCKS_OK(s); + + s = txn3->SetCommitTimeStamp(9); + ASSERT_ROCKS_OK(s); + + s = txn2->Commit(); + ASSERT_ROCKS_OK(s); + ((TOTransactionDBImpl*)txn_db)->AdvanceTS(&maxToCleanTs); + + ASSERT_EQ(maxToCleanTs, 3U); + s = txn_db->QueryTimeStamp(kAllCommitted, &all_committed_ts); + ASSERT_ROCKS_OK(s); + ASSERT_EQ(all_committed_ts, 7U); + + s = txn->Commit(); + ASSERT_ROCKS_OK(s); + ((TOTransactionDBImpl*)txn_db)->AdvanceTS(&maxToCleanTs); + ASSERT_EQ(maxToCleanTs, 3U); + + s = txn_db->SetTimeStamp(kOldest, 7, false); + ASSERT_ROCKS_OK(s); + + ((TOTransactionDBImpl*)txn_db)->AdvanceTS(&maxToCleanTs); + ASSERT_EQ(maxToCleanTs, 6U); + + s = txn3->Commit(); + ASSERT_ROCKS_OK(s); + + ((TOTransactionDBImpl*)txn_db)->AdvanceTS(&maxToCleanTs); + ASSERT_EQ(maxToCleanTs, 7U); + + s = txn4->SetCommitTimeStamp(10); + ASSERT_ROCKS_OK(s); + s = txn4->SetCommitTimeStamp(11); + ASSERT_ROCKS_OK(s); + s = txn4->Commit(); + ASSERT_ROCKS_OK(s); + + s = txn_db->QueryTimeStamp(kAllCommitted, &all_committed_ts); + ASSERT_ROCKS_OK(s); + ASSERT_EQ(all_committed_ts, 11U); + + delete txn_ori; + delete txn; + delete txn2; + delete txn3; + delete txn4; +} + +TEST_F(TOTransactionTest, ThreadsTest) { + + WriteOptions write_options; + ReadOptions read_options; + string value; + + std::vector txns(31); + + for (uint32_t i = 0; i < 31; i++) { + txns[i] = txn_db->BeginTransaction(write_options, txn_options); + ASSERT_TRUE(txns[i]); + auto s = txns[i]->SetCommitTimeStamp(i+1); + ASSERT_ROCKS_OK(s); + } + + std::vector threads; + for (uint32_t i = 0; i < 31; i++) { + std::thread blocking_thread([&, i] { + auto s = txns[i]->Put(std::to_string(i + 1), std::to_string(i + 1)); + ASSERT_ROCKS_OK(s); + + //printf("threads %d\n",i); + s = txns[i]->Commit(); + ASSERT_ROCKS_OK(s); + delete txns[i]; + }); + threads.emplace_back(std::move(blocking_thread)); + } + + //printf("start to join\n"); + + for (auto& t : threads) { + t.join(); + } +} + +TEST_F(TOTransactionTest, MultiCommitTs) { + WriteOptions write_options; + ReadOptions read_options; + RocksTimeStamp all_committed_ts; + string value; + Status s; + + TOTransaction* txn_ori = txn_db->BeginTransaction(write_options, txn_options); + ASSERT_TRUE(txn_ori); + s = txn_ori->SetCommitTimeStamp(4); + ASSERT_ROCKS_OK(s); + s = txn_ori->Commit(); + ASSERT_ROCKS_OK(s); + delete txn_ori; + + s = txn_db->QueryTimeStamp(kAllCommitted, &all_committed_ts); + ASSERT_ROCKS_OK(s); + ASSERT_EQ(all_committed_ts, 4U); + + TOTransaction* txn = txn_db->BeginTransaction(write_options, txn_options); + ASSERT_TRUE(txn != nullptr); + + s = txn->SetCommitTimeStamp(6); + ASSERT_ROCKS_OK(s); + + // commit ts can not set back + s = txn->SetCommitTimeStamp(5); + ASSERT_FALSE(s.ok()); + s = txn->Put("a", "aa"); + ASSERT_ROCKS_OK(s); + s = txn->SetCommitTimeStamp(7); + ASSERT_ROCKS_OK(s); + s = txn->Put("b", "bb"); + ASSERT_ROCKS_OK(s); + ASSERT_ROCKS_OK(txn->Commit()); + delete txn; + + TOTransaction* txn2 = txn_db->BeginTransaction(write_options, txn_options); + ASSERT_TRUE(txn2 != nullptr); + txn2->SetReadTimeStamp(7); + ASSERT_ROCKS_OK(txn2->Get(read_options, "b", &value)); + ASSERT_EQ(value, "bb"); + ASSERT_ROCKS_OK(txn2->Get(read_options, "a", &value)); + ASSERT_EQ(value, "aa"); + delete txn2; + + TOTransaction* txn3 = txn_db->BeginTransaction(write_options, txn_options); + ASSERT_TRUE(txn3 != nullptr); + txn3->SetReadTimeStamp(6); + ASSERT_ROCKS_OK(txn3->Get(read_options, "a", &value)); + ASSERT_EQ(value, "aa"); + ASSERT_FALSE(txn3->Get(read_options, "b", &value).ok()); + delete txn3; + + TOTransaction* txn4 = txn_db->BeginTransaction(write_options, txn_options); + ASSERT_TRUE(txn4 != nullptr); + txn4->SetReadTimeStamp(4); + ASSERT_FALSE(txn4->Get(read_options, "b", &value).ok()); + ASSERT_FALSE(txn4->Get(read_options, "a", &value).ok()); + delete txn4; +} + +TEST_F(TOTransactionTest, PORT_WT_TEST_TIMESTAMP14_TEST_ALL_DURABLE) { + ReadOptions read_options; + WriteOptions write_options; + std::string value; + RocksTimeStamp all_committed_ts; + + // Since this is a non-prepared transaction, we'll be using the commit + // timestamp when calculating all_durable since it's implied that they're + // the same thing. + auto txn1 = std::unique_ptr( + txn_db->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txn1->SetCommitTimeStamp(3)); + ASSERT_ROCKS_OK(txn1->Commit()); + ASSERT_ROCKS_OK(txn_db->QueryTimeStamp(kAllCommitted, &all_committed_ts)); + ASSERT_EQ(all_committed_ts, 3U); + + // We have a running transaction with a lower commit_timestamp than we've + // seen before. So all_durable should return (lowest commit timestamp - 1). + txn1 = std::unique_ptr( + txn_db->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txn1->SetCommitTimeStamp(2)); + ASSERT_ROCKS_OK(txn_db->QueryTimeStamp(kAllCommitted, &all_committed_ts)); + ASSERT_EQ(all_committed_ts, 1U); + ASSERT_ROCKS_OK(txn1->Commit()); + + // After committing, go back to the value we saw previously. + ASSERT_ROCKS_OK(txn_db->QueryTimeStamp(kAllCommitted, &all_committed_ts)); + ASSERT_EQ(all_committed_ts, 3U); + + // For prepared transactions, we take into account the durable timestamp + // when calculating all_durable. + txn1 = std::unique_ptr( + txn_db->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txn1->SetPrepareTimeStamp(6)); + ASSERT_ROCKS_OK(txn1->Prepare()); + ASSERT_ROCKS_OK(txn1->SetCommitTimeStamp(7)); + ASSERT_ROCKS_OK(txn_db->QueryTimeStamp(kAllCommitted, &all_committed_ts)); + ASSERT_EQ(all_committed_ts, 3U); + ASSERT_ROCKS_OK(txn1->SetDurableTimeStamp(8)); + ASSERT_ROCKS_OK(txn1->Commit()); + ASSERT_ROCKS_OK(txn_db->QueryTimeStamp(kAllCommitted, &all_committed_ts)); + ASSERT_EQ(all_committed_ts, 8U); + + // All durable moves back when we have a running prepared transaction + // with a lower durable timestamp than has previously been committed. + txn1 = std::unique_ptr( + txn_db->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txn1->SetPrepareTimeStamp(3)); + ASSERT_ROCKS_OK(txn1->Prepare()); + // If we have a commit timestamp for a prepared transaction, then we + // don't want that to be visible in the all_durable calculation. + ASSERT_ROCKS_OK(txn1->SetCommitTimeStamp(4)); + ASSERT_ROCKS_OK(txn_db->QueryTimeStamp(kAllCommitted, &all_committed_ts)); + ASSERT_EQ(all_committed_ts, 8U); + + // Now take into account the durable timestamp. + ASSERT_ROCKS_OK(txn1->SetDurableTimeStamp(5)); + ASSERT_ROCKS_OK(txn_db->QueryTimeStamp(kAllCommitted, &all_committed_ts)); + //ASSERT_EQ(all_committed_ts, 4); + ASSERT_ROCKS_OK(txn1->Commit()); + ASSERT_ROCKS_OK(txn_db->QueryTimeStamp(kAllCommitted, &all_committed_ts)); + ASSERT_EQ(all_committed_ts, 8U); + + // Now test a scenario with multiple commit timestamps for a single txn. + txn1 = std::unique_ptr( + txn_db->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txn1->SetCommitTimeStamp(6)); + ASSERT_ROCKS_OK(txn_db->QueryTimeStamp(kAllCommitted, &all_committed_ts)); + ASSERT_EQ(all_committed_ts, 5U); + + // Make more changes and set a new commit timestamp. + // Our calculation should use the first commit timestamp so there should + // be no observable difference to the all_durable value. + ASSERT_ROCKS_OK(txn1->SetCommitTimeStamp(7)); + ASSERT_ROCKS_OK(txn_db->QueryTimeStamp(kAllCommitted, &all_committed_ts)); + ASSERT_EQ(all_committed_ts, 5U); + + // Once committed, we go back to 8. + ASSERT_ROCKS_OK(txn1->Commit()); + ASSERT_ROCKS_OK(txn_db->QueryTimeStamp(kAllCommitted, &all_committed_ts)); + ASSERT_EQ(all_committed_ts, 8U); +} + +TEST_F(TOTransactionTest, PORT_WT_TEST_TIMESTAMP14_TEST_ALL_DURABLE_OLD) { + ReadOptions read_options; + WriteOptions write_options; + std::string value; + + // Scenario 0: No commit timestamp has ever been specified therefore + // There is no all_committed timestamp and we will get an error + // Querying for it. + auto txn1 = std::unique_ptr( + txn_db->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txn1->Commit()); + RocksTimeStamp all_committed_ts; + ASSERT_TRUE( + txn_db->QueryTimeStamp(kAllCommitted, &all_committed_ts).IsNotFound()); + + // Scenario 1: A single transaction with a commit timestamp, will + // result in the all_durable timestamp being set. + txn1 = std::unique_ptr( + txn_db->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txn1->SetCommitTimeStamp(1)); + ASSERT_ROCKS_OK(txn1->Commit()); + ASSERT_ROCKS_OK(txn_db->QueryTimeStamp(kAllCommitted, &all_committed_ts)); + ASSERT_EQ(all_committed_ts, 1U); + + // Scenario 2: A transaction begins and specifies that it intends + // to commit at timestamp 2, a second transaction begins and commits + // at timestamp 3. + txn1 = std::unique_ptr( + txn_db->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txn1->SetCommitTimeStamp(2)); + + auto txn2 = std::unique_ptr( + txn_db->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txn2->SetCommitTimeStamp(3)); + ASSERT_ROCKS_OK(txn2->Commit()); + + // As the original transaction is still running the all_commit + // timestamp is being held at 1. + ASSERT_ROCKS_OK(txn_db->QueryTimeStamp(kAllCommitted, &all_committed_ts)); + ASSERT_EQ(all_committed_ts, 1U); + ASSERT_ROCKS_OK(txn1->Commit()); + + // Now that the original transaction has finished the all_commit + // timestamp has moved to 3, skipping 2 as there is a commit with + // a greater timestamp already existing. + ASSERT_ROCKS_OK(txn_db->QueryTimeStamp(kAllCommitted, &all_committed_ts)); + ASSERT_EQ(all_committed_ts, 3U); + + // Senario 3: Commit with a commit timestamp of 5 and then begin a + // transaction intending to commit at 4, the all_commit timestamp + // should move back to 3. Until the transaction at 4 completes. + txn1 = std::unique_ptr( + txn_db->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txn1->SetCommitTimeStamp(5)); + ASSERT_ROCKS_OK(txn1->Commit()); + ASSERT_ROCKS_OK(txn_db->QueryTimeStamp(kAllCommitted, &all_committed_ts)); + ASSERT_EQ(all_committed_ts, 5U); + + txn1 = std::unique_ptr( + txn_db->BeginTransaction(write_options, txn_options)); + // All committed will now move back to 3 as it is the point at which + // all transactions up to that point have committed. + txn1->SetCommitTimeStamp(4); + ASSERT_ROCKS_OK(txn_db->QueryTimeStamp(kAllCommitted, &all_committed_ts)); + ASSERT_EQ(all_committed_ts, 3U); + ASSERT_ROCKS_OK(txn1->Commit()); + + // Now that the transaction at timestamp 4 has completed the + // all committed timestamp is back at 5. + ASSERT_ROCKS_OK(txn_db->QueryTimeStamp(kAllCommitted, &all_committed_ts)); + ASSERT_EQ(all_committed_ts, 5U); + + // Scenario 4: Holding a transaction open without a commit timestamp + // Will not affect the all_durable timestamp. + txn1 = std::unique_ptr( + txn_db->BeginTransaction(write_options, txn_options)); + txn2 = std::unique_ptr( + txn_db->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txn2->SetCommitTimeStamp(6)); + ASSERT_ROCKS_OK(txn2->Commit()); + ASSERT_ROCKS_OK(txn_db->QueryTimeStamp(kAllCommitted, &all_committed_ts)); + ASSERT_EQ(all_committed_ts, 6U); + ASSERT_ROCKS_OK(txn1->Commit()); +} + +TEST_F(TOTransactionTest, PrepareCommitPointRead) { + auto db_imp = dynamic_cast(txn_db); + ReadOptions read_options; + WriteOptions write_options; + std::string value; + auto txnW = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txnW->Put("abc", "abc")); + ASSERT_ROCKS_OK(txnW->Get(read_options, "abc", &value)); + ASSERT_EQ(value, "abc"); + ASSERT_ROCKS_OK(txnW->SetPrepareTimeStamp(100)); + ASSERT_ROCKS_OK(txnW->Prepare()); + // NOTE: Get/Put is not allowed after Prepare + ASSERT_ROCKS_NOK(txnW->Get(read_options, "abc", &value)); + ASSERT_ROCKS_NOK(txnW->Put("abc", "abc")); + + auto txn1 = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + auto txn2 = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + auto txn3 = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txn1->SetReadTimeStamp(99)); + ASSERT_ROCKS_OK(txn2->SetReadTimeStamp(101)); + ASSERT_ROCKS_OK(txn3->SetReadTimeStamp(103)); + ASSERT_TRUE(txn1->Get(read_options, "abc", &value).IsNotFound()); + std::cout << txn2->Get(read_options, "abc", &value).ToString() <Get(read_options, "abc", &value))); + ASSERT_TRUE(IsPrepareConflict(txn3->Get(read_options, "abc", &value))); + ASSERT_ROCKS_NOK(txnW->SetCommitTimeStamp(99)); + ASSERT_ROCKS_OK(txnW->SetCommitTimeStamp(102)); + ASSERT_ROCKS_OK(txnW->Commit()); + ASSERT_TRUE(txn1->Get(read_options, "abc", &value).IsNotFound()); + ASSERT_TRUE(txn2->Get(read_options, "abc", &value).IsNotFound()); + ASSERT_ROCKS_OK(txn3->Get(read_options, "abc", &value)); + ASSERT_EQ(value, "abc"); +} + +TEST_F(TOTransactionTest, DeleteAfterPreparedCommitCursorRead) { + auto db_imp = dynamic_cast(txn_db); + ReadOptions read_options; + WriteOptions write_options; + std::string value; + auto txnW = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txnW->Put("abc", "abc")); + ASSERT_ROCKS_OK(txnW->Get(read_options, "abc", &value)); + ASSERT_EQ(value, "abc"); + ASSERT_ROCKS_OK(txnW->SetPrepareTimeStamp(100)); + ASSERT_ROCKS_OK(txnW->Prepare()); + + auto txnR = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + auto txnR1 = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txnW->SetCommitTimeStamp(100)); + ASSERT_ROCKS_OK(txnW->Commit()); + ASSERT_ROCKS_OK(txnR->SetReadTimeStamp(101)); + ASSERT_ROCKS_OK(txnR->Get(read_options, "abc", &value)); + + auto txnW1 = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txnW1->Delete("abc")); + ASSERT_ROCKS_OK(txnW1->SetCommitTimeStamp(102)); + ASSERT_ROCKS_OK(txnW1->Commit()); + + ASSERT_ROCKS_OK(txnR1->SetReadTimeStamp(103)); + auto iter = std::unique_ptr(txnR1->GetIterator(read_options)); + ASSERT_TRUE(iter != nullptr); + iter->Seek("abc"); + ASSERT_ROCKS_OK(iter->status()); + ASSERT_FALSE(iter->Valid()); +} + +TEST_F(TOTransactionTest, DeleteAfterPreparedCommitPointRead) { + auto db_imp = dynamic_cast(txn_db); + ReadOptions read_options; + WriteOptions write_options; + std::string value; + auto txnW = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txnW->Put("abc", "abc")); + ASSERT_ROCKS_OK(txnW->Get(read_options, "abc", &value)); + ASSERT_EQ(value, "abc"); + ASSERT_ROCKS_OK(txnW->SetPrepareTimeStamp(100)); + ASSERT_ROCKS_OK(txnW->Prepare()); + + auto txnR = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txnW->SetCommitTimeStamp(100)); + ASSERT_ROCKS_OK(txnW->Commit()); + ASSERT_ROCKS_OK(txnR->SetReadTimeStamp(101)); + ASSERT_ROCKS_OK(txnR->Get(read_options, "abc", &value)); + + auto txnW1 = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txnW1->Delete("abc")); + ASSERT_ROCKS_OK(txnW1->SetCommitTimeStamp(102)); + ASSERT_ROCKS_OK(txnW1->Commit()); + + auto txnR1 = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txnR1->SetReadTimeStamp(103)); + ASSERT_TRUE(txnR1->Get(read_options, "abc", &value).IsNotFound()); +} + +TEST_F(TOTransactionTest, UpdateAfterPreparedCommitCursorRead) { + auto db_imp = dynamic_cast(txn_db); + ReadOptions read_options; + WriteOptions write_options; + std::string value; + auto txnW = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txnW->Put("abc", "abc")); + ASSERT_ROCKS_OK(txnW->Get(read_options, "abc", &value)); + ASSERT_EQ(value, "abc"); + ASSERT_ROCKS_OK(txnW->SetPrepareTimeStamp(100)); + ASSERT_ROCKS_OK(txnW->Prepare()); + + auto txnR = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txnW->SetCommitTimeStamp(100)); + ASSERT_ROCKS_OK(txnW->Commit()); + ASSERT_ROCKS_OK(txnR->SetReadTimeStamp(101)); + ASSERT_ROCKS_OK(txnR->Get(read_options, "abc", &value)); + + auto txnW1 = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txnW1->Put("abc", "def")); + ASSERT_ROCKS_OK(txnW1->SetCommitTimeStamp(102)); + ASSERT_ROCKS_OK(txnW1->Commit()); + + auto txnR1 = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txnR1->SetReadTimeStamp(103)); + auto iter = std::unique_ptr(txnR1->GetIterator(read_options)); + ASSERT_TRUE(iter != nullptr); + iter->Seek("abc"); + ASSERT_TRUE(iter->Valid()); + ASSERT_EQ(iter->value().ToString(), "def"); +} + +TEST_F(TOTransactionTest, HangBetweenCommitAndChangePrepareState) { + auto db_imp = dynamic_cast(txn_db); + ReadOptions read_options; + WriteOptions write_options; + std::string value; + auto txnW = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txnW->Put("abc", "abc")); + ASSERT_ROCKS_OK(txnW->SetPrepareTimeStamp(100)); + ASSERT_ROCKS_OK(txnW->Prepare()); + ASSERT_ROCKS_OK(txnW->SetCommitTimeStamp(100)); + + std::atomic pause{true}; + std::atomic written{false}; + std::function commit_hook = [&] { + written.store(true); + while (pause.load()) { + sleep(1); + } + }; + auto thd = std::thread([&] { txnW->Commit(&commit_hook); }); + + while (!written.load()) { + sleep(1); + } + + ReadOptions tmp_read_opt; + char ts_buf[8]; + Slice ts_slice(ts_buf, sizeof(ts_buf)); + Encoder(ts_buf, sizeof(ts_buf)).put64(101); + tmp_read_opt.timestamp = &ts_slice; + ASSERT_ROCKS_OK(txn_db->Get(tmp_read_opt, "abc", &value)); + auto txn1 = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txn1->SetReadTimeStamp(101)); + ASSERT_TRUE(IsPrepareConflict(txn1->Get(read_options, "abc", &value))); + auto iter = std::unique_ptr(txn1->GetIterator(read_options)); + iter->Seek(""); + ASSERT_TRUE(IsPrepareConflict(iter->status())); + pause.store(false); + thd.join(); + ASSERT_ROCKS_OK(txn1->Get(read_options, "abc", &value)); + iter->Seek(""); + ASSERT_ROCKS_OK(iter->status()); +} + +// t0, base cursor state: [a(CURRENT)]------------------------[c] +// t0, delta cursor state: ------------[b(CURRENT)(prepared)]----- +// t1, b.state change from prepared to committed +// t2, merged cursor advance to next, merged cursor should see the latest +// committed value +TEST_F(TOTransactionTest, PreparedCursorReadLatestPMAPState) { + ReadOptions read_options; + WriteOptions write_options; + std::string value; + + auto txn1 = std::unique_ptr( + txn_db->BeginTransaction(write_options, txn_options)); + txn1->Put("a", "1"); + ASSERT_ROCKS_OK(txn1->SetCommitTimeStamp(3)); + ASSERT_ROCKS_OK(txn1->Commit()); + + auto txn2 = std::unique_ptr( + txn_db->BeginTransaction(write_options, txn_options)); + txn2->Put("b", "1"); + ASSERT_ROCKS_OK(txn2->SetPrepareTimeStamp(4)); + ASSERT_ROCKS_OK(txn2->Prepare()); + + auto txn3 = std::unique_ptr( + txn_db->BeginTransaction(write_options, txn_options)); + txn3->Put("c", "1"); + ASSERT_ROCKS_OK(txn3->SetCommitTimeStamp(5)); + ASSERT_ROCKS_OK(txn3->Commit()); + + auto txnR = std::unique_ptr( + txn_db->BeginTransaction(write_options, txn_options)); + char ts_buf[8]; + Slice ts_slice(ts_buf, sizeof(ts_buf)); + Encoder(ts_buf, sizeof(ts_buf)).put64(6); + read_options.timestamp = &ts_slice; + auto iter = std::unique_ptr(txnR->GetIterator(read_options)); + iter->Seek(""); + ASSERT_ROCKS_OK(iter->status()); + ASSERT_TRUE(iter->Valid()); + ASSERT_EQ(iter->key().ToString(), "a"); + + txn2->SetCommitTimeStamp(4); + ASSERT_ROCKS_OK(txn2->Commit()); + iter->Next(); + ASSERT_ROCKS_OK(iter->status()); + ASSERT_TRUE(iter->Valid()); + ASSERT_EQ(iter->key().ToString(), "b"); +} + +TEST_F(TOTransactionTest, PrepareRollbackPointRead) { + auto db_imp = dynamic_cast(txn_db); + ReadOptions read_options; + WriteOptions write_options; + std::string value; + auto txnW = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txnW->Put("abc", "abc")); + ASSERT_ROCKS_OK(txnW->Get(read_options, "abc", &value)); + ASSERT_EQ(value, "abc"); + ASSERT_ROCKS_OK(txnW->SetPrepareTimeStamp(100)); + ASSERT_ROCKS_OK(txnW->Prepare()); + // NOTE: Get/Put is not allowed after Prepare + ASSERT_ROCKS_NOK(txnW->Get(read_options, "abc", &value)); + ASSERT_ROCKS_NOK(txnW->Put("abc", "abc")); + + auto txn1 = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + auto txn2 = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + auto txn3 = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txn1->SetReadTimeStamp(99)); + ASSERT_ROCKS_OK(txn2->SetReadTimeStamp(101)); + ASSERT_TRUE(txn1->Get(read_options, "abc", &value).IsNotFound()); + ASSERT_TRUE(IsPrepareConflict(txn2->Get(read_options, "abc", &value))); + ASSERT_TRUE(IsPrepareConflict(txn3->Get(read_options, "abc", &value))); + ASSERT_ROCKS_NOK(txnW->SetCommitTimeStamp(99)); + ASSERT_ROCKS_OK(txnW->SetCommitTimeStamp(102)); + ASSERT_ROCKS_OK(txnW->Rollback()); + ASSERT_TRUE(txn1->Get(read_options, "abc", &value).IsNotFound()); + ASSERT_TRUE(txn2->Get(read_options, "abc", &value).IsNotFound()); + ASSERT_TRUE(txn3->Get(read_options, "abc", &value).IsNotFound()); +} + +TEST_F(TOTransactionTest, PrepareIteratorSameKey) { + auto db_imp = dynamic_cast(txn_db); + ReadOptions read_options; + WriteOptions write_options; + std::string value; + auto txnW = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txnW->Put("abc", "abc")); + ASSERT_ROCKS_OK(txnW->SetCommitTimeStamp(100)); + ASSERT_ROCKS_OK(txnW->Commit()); + + auto txnW1 = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txnW1->Put("abc", "def")); + ASSERT_ROCKS_OK(txnW1->SetPrepareTimeStamp(101)); + ASSERT_ROCKS_OK(txnW1->Prepare()); + + auto txnR = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + txnR->SetReadTimeStamp(102); + auto iter = std::unique_ptr(txnR->GetIterator(read_options)); + ASSERT_TRUE(iter != nullptr); + iter->Seek(""); + ASSERT_TRUE(IsPrepareConflict(iter->status())); + + auto txnR1 = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + txnR1->SetReadTimeStamp(101); + auto iter1 = std::unique_ptr(txnR1->GetIterator(read_options)); + ASSERT_TRUE(iter1 != nullptr); + iter1->Seek(""); + ASSERT_TRUE(IsPrepareConflict(iter1->status())); + + ASSERT_ROCKS_OK(txnW1->SetCommitTimeStamp(102)); + txnW1->Commit(); + + iter->Seek(""); + ASSERT_ROCKS_OK(iter->status()); + ASSERT_EQ(iter->key().ToString(), "abc"); + ASSERT_EQ(iter->value().ToString(), "def"); + + iter1->Seek(""); + ASSERT_ROCKS_OK(iter1->status()); + ASSERT_EQ(iter1->key().ToString(), "abc"); + ASSERT_EQ(iter1->value().ToString(), "abc"); +} + +TEST_F(TOTransactionTest, PORT_WT_TEST_PREPARE_05) { + WriteOptions write_options; + ReadOptions read_options; + char ts_buf[8]; + Slice ts_slice(ts_buf, sizeof(ts_buf)); + Encoder(ts_buf, sizeof(ts_buf)).put64(50); + read_options.timestamp = &ts_slice; + string value; + Status s; + + ASSERT_ROCKS_OK(txn_db->SetTimeStamp(kOldest, 2)); + auto txn = std::unique_ptr( + txn_db->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_NOK(txn->SetPrepareTimeStamp(1)); + + // Check setting the prepare timestamp same as oldest timestamp is valid. + txn = std::unique_ptr( + txn_db->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txn->SetPrepareTimeStamp(2)); + ASSERT_ROCKS_OK(txn->Prepare()); + ASSERT_ROCKS_OK(txn->SetCommitTimeStamp(3)); + ASSERT_ROCKS_OK(txn->SetDurableTimeStamp(3)); + ASSERT_ROCKS_OK(txn->Commit()); + + // In a single transaction it is illegal to set a commit timestamp + // before invoking prepare for this transaction. + // Note: Values are not important, setting commit timestamp before + // prepare itself is illegal. + txn = std::unique_ptr( + txn_db->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txn->SetCommitTimeStamp(3)); + ASSERT_ROCKS_NOK(txn->SetCommitTimeStamp(2)); + + txn = std::unique_ptr( + txn_db->BeginTransaction(write_options, txn_options)); + auto txnR = std::unique_ptr( + txn_db->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txnR->SetReadTimeStamp(4)); + ASSERT_ROCKS_NOK(txn->SetPrepareTimeStamp(4)); + ASSERT_ROCKS_OK(txn->SetPrepareTimeStamp(5)); + + txn = std::unique_ptr( + txn_db->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txn->SetPrepareTimeStamp(5)); + ASSERT_ROCKS_OK(txn->Prepare()); + ASSERT_ROCKS_OK(txn->SetCommitTimeStamp(5)); + ASSERT_ROCKS_OK(txn->SetDurableTimeStamp(5)); + ASSERT_ROCKS_OK(txn->Commit()); +} + +TEST_F(TOTransactionTest, PORT_WT_TEST_PREPARE_06) { + WriteOptions write_options; + ReadOptions read_options; + string value; + Status s; + + ASSERT_ROCKS_OK(txn_db->SetTimeStamp(kOldest, 20)); + auto txn = std::unique_ptr( + txn_db->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_NOK(txn->SetPrepareTimeStamp(10)); + + TOTransactionOptions new_txn_options; + new_txn_options.timestamp_round_read = false; + new_txn_options.timestamp_round_prepared = true; + // Check setting the prepare timestamp same as oldest timestamp is valid. + txn = std::unique_ptr( + txn_db->BeginTransaction(write_options, new_txn_options)); + ASSERT_ROCKS_OK(txn->SetPrepareTimeStamp(10)); + ASSERT_ROCKS_OK(txn->Prepare()); + ASSERT_ROCKS_OK(txn->SetCommitTimeStamp(15)); + ASSERT_ROCKS_OK(txn->SetDurableTimeStamp(35)); + ASSERT_ROCKS_OK(txn->Commit()); + + // Check the cases with an active reader. + // Start a new reader to have an active read timestamp. + auto txnR = std::unique_ptr( + txn_db->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txnR->SetReadTimeStamp(40)); + txn = std::unique_ptr( + txn_db->BeginTransaction(write_options, txn_options)); + // It is illegal to set the prepare timestamp as earlier than an active + // read timestamp even with roundup_timestamps settings. This is only + // checked in diagnostic builds. + ASSERT_ROCKS_NOK(txn->SetPrepareTimeStamp(10)); +} + +TEST_F(TOTransactionTest, PORT_WT_TEST_PREPARE_CURSOR_01) { + WriteOptions write_options; + ReadOptions read_options; + string value; + Status s; + auto db_imp = dynamic_cast(txn_db); + + [&] { + auto tmp_txn = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(tmp_txn->Put(Slice("45"), Slice("45"))); + ASSERT_ROCKS_OK(tmp_txn->Put(Slice("46"), Slice("46"))); + ASSERT_ROCKS_OK(tmp_txn->Put(Slice("47"), Slice("47"))); + ASSERT_ROCKS_OK(tmp_txn->Put(Slice("48"), Slice("48"))); + ASSERT_ROCKS_OK(tmp_txn->Put(Slice("49"), Slice("49"))); + ASSERT_ROCKS_OK(tmp_txn->Put(Slice("50"), Slice("50"))); + ASSERT_ROCKS_OK(tmp_txn->Commit()); + }(); + + auto txn_prepare = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + auto iter_prepare = + std::unique_ptr(txn_prepare->GetIterator(read_options)); + + // Scenario-1 : Check cursor navigate with insert in prepared transaction. + // Begin of Scenario-1. + // Data set at start has keys {2,3,4 ... 50} + // Insert key 51 to check next operation. + // Insert key 1 to check prev operation. + ASSERT_ROCKS_OK(txn_prepare->Put("51", "51")); + ASSERT_ROCKS_OK(txn_prepare->SetPrepareTimeStamp(100)); + ASSERT_ROCKS_OK(txn_prepare->Prepare()); + + // Txn for timestamped reads before prepare timestamp. + auto txnR_before_ts = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txnR_before_ts->SetReadTimeStamp(50)); + auto iter_before_ts = + std::unique_ptr(txnR_before_ts->GetIterator(read_options)); + + // Txn for timestamped reads between prepare timestamp and commit timestamp. + auto txnR_between_ts = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txnR_between_ts->SetReadTimeStamp(150)); + auto iter_between_ts = + std::unique_ptr(txnR_between_ts->GetIterator(read_options)); + + // Txn for timestamped reads after commit timestamp. + auto txnR_after_ts = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txnR_after_ts->SetReadTimeStamp(250)); + auto iter_after_ts = + std::unique_ptr(txnR_after_ts->GetIterator(read_options)); + + // Point all cursors to key 50. + iter_before_ts->Seek("50"); + iter_between_ts->Seek("50"); + iter_after_ts->Seek("50"); + + ASSERT_TRUE(iter_before_ts->Valid()); + ASSERT_TRUE(iter_between_ts->Valid()); + ASSERT_TRUE(iter_after_ts->Valid()); + + iter_before_ts->Next(); + ASSERT_FALSE(iter_before_ts->Valid()); + ASSERT_ROCKS_OK(iter_before_ts->status()); + + iter_between_ts->Next(); + ASSERT_TRUE(IsPrepareConflict(iter_between_ts->status())); + ASSERT_TRUE(iter_between_ts->Valid()); + + iter_between_ts->Prev(); + ASSERT_ROCKS_OK(iter_between_ts->status()); + ASSERT_TRUE(iter_between_ts->Valid()); + ASSERT_EQ(iter_between_ts->key().ToString(), "50"); + + iter_after_ts->Next(); + ASSERT_TRUE(IsPrepareConflict(iter_after_ts->status())); + ASSERT_TRUE(iter_after_ts->Valid()); + + ASSERT_ROCKS_OK(txn_prepare->SetCommitTimeStamp(200)); + ASSERT_ROCKS_OK(txn_prepare->SetDurableTimeStamp(200)); + ASSERT_ROCKS_OK(txn_prepare->Commit()); + + iter_after_ts->Next(); + ASSERT_ROCKS_OK(iter_after_ts->status()); + ASSERT_TRUE(iter_after_ts->Valid()); + ASSERT_EQ(iter_after_ts->key().ToString(), "51"); + + ASSERT_ROCKS_OK(txnR_before_ts->Commit()); + ASSERT_ROCKS_OK(txnR_between_ts->Commit()); + ASSERT_ROCKS_OK(txnR_after_ts->Commit()); + + // Insert key 44 to check prev operation + txn_prepare = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txn_prepare->Put("44", "44")); + ASSERT_ROCKS_OK(txn_prepare->SetPrepareTimeStamp(100)); + ASSERT_ROCKS_OK(txn_prepare->Prepare()); + + txnR_before_ts = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txnR_before_ts->SetReadTimeStamp(50)); + iter_before_ts = + std::unique_ptr(txnR_before_ts->GetIterator(read_options)); + + txnR_between_ts = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txnR_between_ts->SetReadTimeStamp(150)); + iter_between_ts = + std::unique_ptr(txnR_between_ts->GetIterator(read_options)); + + txnR_after_ts = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txnR_after_ts->SetReadTimeStamp(250)); + iter_after_ts = + std::unique_ptr(txnR_after_ts->GetIterator(read_options)); + + // Point all cursors to key 45. + iter_before_ts->Seek("45"); + iter_between_ts->Seek("45"); + iter_after_ts->Seek("45"); + + ASSERT_TRUE(iter_before_ts->Valid()); + ASSERT_TRUE(iter_between_ts->Valid()); + ASSERT_TRUE(iter_after_ts->Valid()); + + // As read is before prepare timestamp, prev is not found. + iter_before_ts->Prev(); + ASSERT_FALSE(iter_before_ts->Valid()); + ASSERT_ROCKS_OK(iter_before_ts->status()); + + // As read is between, prev will point to prepared update. + iter_between_ts->Prev(); + ASSERT_TRUE(IsPrepareConflict(iter_between_ts->status())); + ASSERT_TRUE(iter_between_ts->Valid()); + + // Check to see next works when a prev returns prepare conflict. + iter_between_ts->Next(); + ASSERT_ROCKS_OK(iter_between_ts->status()); + ASSERT_TRUE(iter_between_ts->Valid()); + ASSERT_EQ(iter_between_ts->key().ToString(), "45"); + + // As read is after, prev will point to prepared update. + iter_after_ts->Prev(); + ASSERT_TRUE(IsPrepareConflict(iter_after_ts->status())); + ASSERT_TRUE(iter_after_ts->Valid()); + + // Commit the prepared transaction. + ASSERT_ROCKS_OK(txn_prepare->SetCommitTimeStamp(200)); + ASSERT_ROCKS_OK(txn_prepare->SetDurableTimeStamp(200)); + ASSERT_ROCKS_OK(txn_prepare->Commit()); + + iter_after_ts->Prev(); + ASSERT_ROCKS_OK(iter_after_ts->status()); + ASSERT_TRUE(iter_after_ts->Valid()); + ASSERT_EQ(iter_after_ts->key().ToString(), "44"); + + // TODO: it may not meet mongodb's requirements + // here we advance oldest to clean prepare_map + ASSERT_ROCKS_OK(txn_db->SetTimeStamp(kOldest, 201)); + ASSERT_ROCKS_OK(txnR_before_ts->Commit()); + ASSERT_ROCKS_OK(txnR_between_ts->Commit()); + ASSERT_ROCKS_OK(txnR_after_ts->Commit()); + // End of Scenario-1. + + // sleep(1) to ensure purged + sleep(1); + // Scenario-2 : Check cursor navigate with update in prepared transaction. + // Begin of Scenario-2. + // Data set at start has keys {44, 45, 46, 47, 48, 49, 50, 51} + // Update key 51 to check next operation. + // Update key 44 to check prev operation. + txn_prepare = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txn_prepare->Put("51", "151")); + ASSERT_ROCKS_OK(txn_prepare->SetPrepareTimeStamp(300)); + ASSERT_ROCKS_OK(txn_prepare->Prepare()); + + txnR_before_ts = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txnR_before_ts->SetReadTimeStamp(250)); + iter_before_ts = + std::unique_ptr(txnR_before_ts->GetIterator(read_options)); + + txnR_between_ts = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txnR_between_ts->SetReadTimeStamp(350)); + iter_between_ts = + std::unique_ptr(txnR_between_ts->GetIterator(read_options)); + + txnR_after_ts = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txnR_after_ts->SetReadTimeStamp(450)); + iter_after_ts = + std::unique_ptr(txnR_after_ts->GetIterator(read_options)); + + // Point all cursors to key 51. + iter_before_ts->Seek("50"); + iter_between_ts->Seek("50"); + iter_after_ts->Seek("50"); + + ASSERT_TRUE(iter_before_ts->Valid()); + ASSERT_TRUE(iter_between_ts->Valid()); + ASSERT_TRUE(iter_after_ts->Valid()); + + iter_before_ts->Next(); + ASSERT_TRUE(iter_before_ts->Valid()); + ASSERT_ROCKS_OK(iter_before_ts->status()); + // As read is before prepare timestamp, next is found with previous value. + ASSERT_EQ(iter_before_ts->key().ToString(), "51"); + ASSERT_EQ(iter_before_ts->value().ToString(), "51"); + + // As read is between, next will point to prepared update. + iter_between_ts->Next(); + ASSERT_TRUE(IsPrepareConflict(iter_between_ts->status())); + ASSERT_TRUE(iter_between_ts->Valid()); + + // As read is after, next will point to prepared update. + iter_after_ts->Next(); + ASSERT_TRUE(IsPrepareConflict(iter_after_ts->status())); + ASSERT_TRUE(iter_after_ts->Valid()); + + // Commit the prepared transaction. + ASSERT_ROCKS_OK(txn_prepare->SetCommitTimeStamp(400)); + ASSERT_ROCKS_OK(txn_prepare->SetDurableTimeStamp(400)); + ASSERT_ROCKS_OK(txn_prepare->Commit()); + + // Check to see before cursor still gets the old value. + iter_before_ts->Seek("51"); + ASSERT_TRUE(iter_before_ts->Valid()); + ASSERT_ROCKS_OK(iter_before_ts->status()); + ASSERT_EQ(iter_before_ts->key().ToString(), "51"); + ASSERT_EQ(iter_before_ts->value().ToString(), "51"); + + // As read is between(i.e before commit), next is not found. + iter_between_ts->Next(); + ASSERT_ROCKS_OK(iter_between_ts->status()); + ASSERT_TRUE(iter_between_ts->Valid()); + ASSERT_EQ(iter_between_ts->key().ToString(), "51"); + ASSERT_EQ(iter_between_ts->value().ToString(), "51"); + + // As read is after, next will point to new key 51. + iter_after_ts->Next(); + ASSERT_ROCKS_OK(iter_after_ts->status()); + ASSERT_TRUE(iter_after_ts->Valid()); + ASSERT_EQ(iter_after_ts->key().ToString(), "51"); + ASSERT_EQ(iter_after_ts->value().ToString(), "151"); + + ASSERT_ROCKS_OK(txnR_before_ts->Commit()); + ASSERT_ROCKS_OK(txnR_between_ts->Commit()); + ASSERT_ROCKS_OK(txnR_after_ts->Commit()); + + // Update key 44 to check prev operation. + txn_prepare = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txn_prepare->Put("44", "444")); + ASSERT_ROCKS_OK(txn_prepare->SetPrepareTimeStamp(300)); + ASSERT_ROCKS_OK(txn_prepare->Prepare()); + + txnR_before_ts = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txnR_before_ts->SetReadTimeStamp(250)); + iter_before_ts = + std::unique_ptr(txnR_before_ts->GetIterator(read_options)); + + txnR_between_ts = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txnR_between_ts->SetReadTimeStamp(350)); + iter_between_ts = + std::unique_ptr(txnR_between_ts->GetIterator(read_options)); + + txnR_after_ts = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txnR_after_ts->SetReadTimeStamp(450)); + iter_after_ts = + std::unique_ptr(txnR_after_ts->GetIterator(read_options)); + + // Check the visibility of new update of prepared transaction. + // Point all cursors to key 45. + iter_before_ts->Seek("45"); + iter_between_ts->Seek("45"); + iter_after_ts->Seek("45"); + + ASSERT_TRUE(iter_before_ts->Valid()); + ASSERT_TRUE(iter_between_ts->Valid()); + ASSERT_TRUE(iter_after_ts->Valid()); + + // As read is before prepare timestamp, prev is not found. + iter_before_ts->Prev(); + ASSERT_TRUE(iter_before_ts->Valid()); + ASSERT_ROCKS_OK(iter_before_ts->status()); + ASSERT_EQ(iter_before_ts->key().ToString(), "44"); + ASSERT_EQ(iter_before_ts->value().ToString(), "44"); + + // As read is between, prev will point to prepared update. + iter_between_ts->Prev(); + ASSERT_TRUE(IsPrepareConflict(iter_between_ts->status())); + ASSERT_TRUE(iter_between_ts->Valid()); + + // As read is after, prev will point to prepared update. + iter_after_ts->Prev(); + ASSERT_TRUE(IsPrepareConflict(iter_after_ts->status())); + ASSERT_TRUE(iter_after_ts->Valid()); + + // Commit the prepared transaction. + ASSERT_ROCKS_OK(txn_prepare->SetCommitTimeStamp(400)); + ASSERT_ROCKS_OK(txn_prepare->SetDurableTimeStamp(400)); + ASSERT_ROCKS_OK(txn_prepare->Commit()); + + // Check to see before cursor still gets the old value. + iter_before_ts->Seek("44"); + ASSERT_ROCKS_OK(iter_before_ts->status()); + ASSERT_TRUE(iter_before_ts->Valid()); + ASSERT_EQ(iter_before_ts->key().ToString(), "44"); + ASSERT_EQ(iter_before_ts->value().ToString(), "44"); + + // As read is between(i.e before commit), next is not found. + iter_between_ts->Prev(); + ASSERT_ROCKS_OK(iter_between_ts->status()); + ASSERT_TRUE(iter_between_ts->Valid()); + ASSERT_EQ(iter_between_ts->key().ToString(), "44"); + ASSERT_EQ(iter_between_ts->value().ToString(), "44"); + + // As read is after, next will point to new key 44. + iter_after_ts->Prev(); + ASSERT_ROCKS_OK(iter_after_ts->status()); + ASSERT_TRUE(iter_after_ts->Valid()); + ASSERT_EQ(iter_after_ts->key().ToString(), "44"); + ASSERT_EQ(iter_after_ts->value().ToString(), "444"); + + // End of Scenario-2. + // TODO: it may not meet mongodb's requirements + // here we advance oldest to clean prepare_map + ASSERT_ROCKS_OK(txn_db->SetTimeStamp(kOldest, 401)); + + ASSERT_ROCKS_OK(txnR_before_ts->Commit()); + ASSERT_ROCKS_OK(txnR_between_ts->Commit()); + ASSERT_ROCKS_OK(txnR_after_ts->Commit()); + + // sleep(1) to ensure purged + sleep(1); + // Scenario-3 : Check cursor navigate with remove in prepared transaction. + // Begin of Scenario-3. + // Data set at start has keys {44, 45, 46, 47, 48, 49, 50, 51} + // Remove key 51 to check next operation. + // Remove key 44 to check prev operation. + txn_prepare = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txn_prepare->Delete("51")); + ASSERT_ROCKS_OK(txn_prepare->SetPrepareTimeStamp(500)); + ASSERT_ROCKS_OK(txn_prepare->Prepare()); + + txnR_before_ts = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txnR_before_ts->SetReadTimeStamp(450)); + iter_before_ts = + std::unique_ptr(txnR_before_ts->GetIterator(read_options)); + + txnR_between_ts = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txnR_between_ts->SetReadTimeStamp(550)); + iter_between_ts = + std::unique_ptr(txnR_between_ts->GetIterator(read_options)); + + txnR_after_ts = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txnR_after_ts->SetReadTimeStamp(650)); + iter_after_ts = + std::unique_ptr(txnR_after_ts->GetIterator(read_options)); + + // Point all cursors to key 51. + iter_before_ts->Seek("50"); + iter_between_ts->Seek("50"); + iter_after_ts->Seek("50"); + + ASSERT_TRUE(iter_before_ts->Valid()); + ASSERT_TRUE(iter_between_ts->Valid()); + ASSERT_TRUE(iter_after_ts->Valid()); + + iter_before_ts->Next(); + ASSERT_TRUE(iter_before_ts->Valid()); + ASSERT_ROCKS_OK(iter_before_ts->status()); + // As read is before prepare timestamp, next is found with previous value. + ASSERT_EQ(iter_before_ts->key().ToString(), "51"); + ASSERT_EQ(iter_before_ts->value().ToString(), "151"); + + // As read is between, next will point to prepared update. + iter_between_ts->Next(); + ASSERT_TRUE(IsPrepareConflict(iter_between_ts->status())); + ASSERT_TRUE(iter_between_ts->Valid()); + + // As read is after, next will point to prepared update. + iter_after_ts->Next(); + ASSERT_TRUE(IsPrepareConflict(iter_after_ts->status())); + ASSERT_TRUE(iter_after_ts->Valid()); + + // Commit the prepared transaction. + ASSERT_ROCKS_OK(txn_prepare->SetCommitTimeStamp(600)); + ASSERT_ROCKS_OK(txn_prepare->SetDurableTimeStamp(600)); + ASSERT_ROCKS_OK(txn_prepare->Commit()); + + // Check to see before cursor still gets the old value. + iter_before_ts->Seek("51"); + ASSERT_TRUE(iter_before_ts->Valid()); + ASSERT_ROCKS_OK(iter_before_ts->status()); + ASSERT_EQ(iter_before_ts->key().ToString(), "51"); + ASSERT_EQ(iter_before_ts->value().ToString(), "151"); + + // As read is between(i.e before commit), next is not found. + iter_between_ts->Next(); + ASSERT_ROCKS_OK(iter_between_ts->status()); + ASSERT_TRUE(iter_between_ts->Valid()); + ASSERT_EQ(iter_between_ts->key().ToString(), "51"); + ASSERT_EQ(iter_between_ts->value().ToString(), "151"); + + // As read is after, next will not be found. + iter_after_ts->Next(); + ASSERT_ROCKS_OK(iter_after_ts->status()); + ASSERT_FALSE(iter_after_ts->Valid()); + + ASSERT_ROCKS_OK(txnR_before_ts->Commit()); + ASSERT_ROCKS_OK(txnR_between_ts->Commit()); + ASSERT_ROCKS_OK(txnR_after_ts->Commit()); + + // remove key 44 to check prev operation. + txn_prepare = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txn_prepare->Delete("44")); + ASSERT_ROCKS_OK(txn_prepare->SetPrepareTimeStamp(500)); + ASSERT_ROCKS_OK(txn_prepare->Prepare()); + + txnR_before_ts = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txnR_before_ts->SetReadTimeStamp(450)); + iter_before_ts = + std::unique_ptr(txnR_before_ts->GetIterator(read_options)); + + txnR_between_ts = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txnR_between_ts->SetReadTimeStamp(550)); + iter_between_ts = + std::unique_ptr(txnR_between_ts->GetIterator(read_options)); + + txnR_after_ts = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txnR_after_ts->SetReadTimeStamp(650)); + iter_after_ts = + std::unique_ptr(txnR_after_ts->GetIterator(read_options)); + + // Check the visibility of new update of prepared transaction. + // Point all cursors to key 45. + iter_before_ts->Seek("45"); + iter_between_ts->Seek("45"); + iter_after_ts->Seek("45"); + + ASSERT_TRUE(iter_before_ts->Valid()); + ASSERT_TRUE(iter_between_ts->Valid()); + ASSERT_TRUE(iter_after_ts->Valid()); + + // As read is before prepare timestamp, prev is not found. + iter_before_ts->Prev(); + ASSERT_TRUE(iter_before_ts->Valid()); + ASSERT_ROCKS_OK(iter_before_ts->status()); + ASSERT_EQ(iter_before_ts->key().ToString(), "44"); + ASSERT_EQ(iter_before_ts->value().ToString(), "444"); + + // As read is between, prev will point to prepared update. + iter_between_ts->Prev(); + ASSERT_TRUE(IsPrepareConflict(iter_between_ts->status())); + ASSERT_TRUE(iter_between_ts->Valid()); + + // As read is after, prev will point to prepared update. + iter_after_ts->Prev(); + ASSERT_TRUE(IsPrepareConflict(iter_after_ts->status())); + ASSERT_TRUE(iter_after_ts->Valid()); + + // Commit the prepared transaction. + ASSERT_ROCKS_OK(txn_prepare->SetCommitTimeStamp(600)); + ASSERT_ROCKS_OK(txn_prepare->SetDurableTimeStamp(600)); + ASSERT_ROCKS_OK(txn_prepare->Commit()); + + // Check to see before cursor still gets the old value. + iter_before_ts->Seek("44"); + ASSERT_ROCKS_OK(iter_before_ts->status()); + ASSERT_TRUE(iter_before_ts->Valid()); + ASSERT_EQ(iter_before_ts->key().ToString(), "44"); + ASSERT_EQ(iter_before_ts->value().ToString(), "444"); + + // As read is between(i.e before commit), next is not found. + iter_between_ts->Prev(); + ASSERT_ROCKS_OK(iter_between_ts->status()); + ASSERT_TRUE(iter_between_ts->Valid()); + ASSERT_EQ(iter_between_ts->key().ToString(), "44"); + ASSERT_EQ(iter_between_ts->value().ToString(), "444"); + + // As read is after, next will point to new key 44. + iter_after_ts->Prev(); + ASSERT_ROCKS_OK(iter_after_ts->status()); + ASSERT_FALSE(iter_after_ts->Valid()); + + // End of Scenario-3. + // TODO: it may not meet mongodb's requirements + // here we advance oldest to clean prepare_map + ASSERT_ROCKS_OK(txn_db->SetTimeStamp(kOldest, 601)); + ASSERT_ROCKS_OK(txnR_before_ts->Commit()); + ASSERT_ROCKS_OK(txnR_between_ts->Commit()); + ASSERT_ROCKS_OK(txnR_after_ts->Commit()); + + // Scenario-4 : Check cursor navigate with remove in prepared transaction. + // remove keys not in the ends. + // Begin of Scenario-4. + // Data set at start has keys {45,46,47,48,49,50} + // Remove key 49 to check next operation. + // Remove key 46 to check prev operation. + txn_prepare = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txn_prepare->Delete("49")); + ASSERT_ROCKS_OK(txn_prepare->SetPrepareTimeStamp(700)); + ASSERT_ROCKS_OK(txn_prepare->Prepare()); + + txnR_before_ts = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txnR_before_ts->SetReadTimeStamp(650)); + iter_before_ts = + std::unique_ptr(txnR_before_ts->GetIterator(read_options)); + + txnR_between_ts = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txnR_between_ts->SetReadTimeStamp(750)); + iter_between_ts = + std::unique_ptr(txnR_between_ts->GetIterator(read_options)); + + txnR_after_ts = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txnR_after_ts->SetReadTimeStamp(850)); + iter_after_ts = + std::unique_ptr(txnR_after_ts->GetIterator(read_options)); + + // Point all cursors to key 48. + iter_before_ts->Seek("48"); + iter_between_ts->Seek("48"); + iter_after_ts->Seek("48"); + + ASSERT_TRUE(iter_before_ts->Valid()); + ASSERT_TRUE(iter_between_ts->Valid()); + ASSERT_TRUE(iter_after_ts->Valid()); + + iter_before_ts->Next(); + ASSERT_TRUE(iter_before_ts->Valid()); + ASSERT_ROCKS_OK(iter_before_ts->status()); + // As read is before prepare timestamp, next is found with 49. + ASSERT_EQ(iter_before_ts->key().ToString(), "49"); + + // As read is between, next will point to prepared update. + iter_between_ts->Next(); + ASSERT_TRUE(IsPrepareConflict(iter_between_ts->status())); + ASSERT_TRUE(iter_between_ts->Valid()); + + // As read is after, next will point to prepared update. + iter_after_ts->Next(); + ASSERT_TRUE(IsPrepareConflict(iter_after_ts->status())); + ASSERT_TRUE(iter_after_ts->Valid()); + + // Commit the prepared transaction. + ASSERT_ROCKS_OK(txn_prepare->SetCommitTimeStamp(800)); + ASSERT_ROCKS_OK(txn_prepare->SetDurableTimeStamp(800)); + ASSERT_ROCKS_OK(txn_prepare->Commit()); + + // Check to see before cursor still gets the old value. + iter_before_ts->Seek("49"); + ASSERT_TRUE(iter_before_ts->Valid()); + ASSERT_ROCKS_OK(iter_before_ts->status()); + ASSERT_EQ(iter_before_ts->key().ToString(), "49"); + + // As read is between(i.e before commit), next is not found. + iter_between_ts->Next(); + ASSERT_ROCKS_OK(iter_between_ts->status()); + ASSERT_TRUE(iter_between_ts->Valid()); + ASSERT_EQ(iter_between_ts->key().ToString(), "49"); + + // As read is after, next will point beyond end. + iter_after_ts->Next(); + ASSERT_ROCKS_OK(iter_after_ts->status()); + ASSERT_TRUE(iter_after_ts->Valid()); + ASSERT_EQ(iter_after_ts->key().ToString(), "50"); + + ASSERT_ROCKS_OK(txnR_before_ts->Commit()); + ASSERT_ROCKS_OK(txnR_between_ts->Commit()); + ASSERT_ROCKS_OK(txnR_after_ts->Commit()); + + // remove key 46 to check prev operation. + txn_prepare = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txn_prepare->Delete("46")); + ASSERT_ROCKS_OK(txn_prepare->SetPrepareTimeStamp(700)); + ASSERT_ROCKS_OK(txn_prepare->Prepare()); + + txnR_before_ts = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txnR_before_ts->SetReadTimeStamp(650)); + iter_before_ts = + std::unique_ptr(txnR_before_ts->GetIterator(read_options)); + + txnR_between_ts = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txnR_between_ts->SetReadTimeStamp(750)); + iter_between_ts = + std::unique_ptr(txnR_between_ts->GetIterator(read_options)); + + txnR_after_ts = std::unique_ptr( + db_imp->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txnR_after_ts->SetReadTimeStamp(850)); + iter_after_ts = + std::unique_ptr(txnR_after_ts->GetIterator(read_options)); + + // Check the visibility of new update of prepared transaction. + // Point all cursors to key 45. + iter_before_ts->Seek("47"); + iter_between_ts->Seek("47"); + iter_after_ts->Seek("47"); + + ASSERT_TRUE(iter_before_ts->Valid()); + ASSERT_TRUE(iter_between_ts->Valid()); + ASSERT_TRUE(iter_after_ts->Valid()); + + // As read is before prepare timestamp, prev is not found. + iter_before_ts->Prev(); + ASSERT_TRUE(iter_before_ts->Valid()); + ASSERT_ROCKS_OK(iter_before_ts->status()); + ASSERT_EQ(iter_before_ts->key().ToString(), "46"); + + // As read is between, prev will point to prepared update. + iter_between_ts->Prev(); + ASSERT_TRUE(IsPrepareConflict(iter_between_ts->status())); + ASSERT_TRUE(iter_between_ts->Valid()); + + // As read is after, prev will point to prepared update. + iter_after_ts->Prev(); + ASSERT_TRUE(IsPrepareConflict(iter_after_ts->status())); + ASSERT_TRUE(iter_after_ts->Valid()); + + // Commit the prepared transaction. + ASSERT_ROCKS_OK(txn_prepare->SetCommitTimeStamp(800)); + ASSERT_ROCKS_OK(txn_prepare->SetDurableTimeStamp(800)); + ASSERT_ROCKS_OK(txn_prepare->Commit()); + + // Check to see before cursor still gets the old value. + iter_before_ts->Seek("46"); + ASSERT_ROCKS_OK(iter_before_ts->status()); + ASSERT_TRUE(iter_before_ts->Valid()); + ASSERT_EQ(iter_before_ts->key().ToString(), "46"); + + // As read is between(i.e before commit), next is not found. + iter_between_ts->Prev(); + ASSERT_ROCKS_OK(iter_between_ts->status()); + ASSERT_TRUE(iter_between_ts->Valid()); + ASSERT_EQ(iter_between_ts->key().ToString(), "46"); + + // As read is after, next will point to new key 45. + iter_after_ts->Prev(); + ASSERT_ROCKS_OK(iter_after_ts->status()); + ASSERT_TRUE(iter_after_ts->Valid()); + ASSERT_EQ(iter_after_ts->key().ToString(), "45"); +} + +TEST_F(TOTransactionTest, MemUsage) { + auto db_imp = dynamic_cast(txn_db); + db_imp->SetMaxConflictBytes(33); + WriteOptions write_options; + ReadOptions read_options; + TOTransactionStat stat; + + TOTransaction* txn = db_imp->BeginTransaction(write_options, txn_options); + // key(3) + cfid(4) + txnid(8) = 15 + ASSERT_ROCKS_OK(txn->Put("abc", "abc")); + memset(&stat, 0, sizeof stat); + db_imp->Stat(&stat); + ASSERT_EQ(stat.uk_num, 1U); + ASSERT_EQ(stat.cur_conflict_bytes, 15U); + // 15+16=31 + ASSERT_ROCKS_OK(txn->Put("defg", "defg")); + memset(&stat, 0, sizeof stat); + db_imp->Stat(&stat); + ASSERT_EQ(stat.uk_num, 2U); + ASSERT_EQ(stat.cur_conflict_bytes, 31U); + auto s = txn->Put("h", "h"); + ASSERT_FALSE(s.ok()); + db_imp->SetMaxConflictBytes(50); + // 31+13=44 + ASSERT_ROCKS_OK(txn->Put("h", "h")); + memset(&stat, 0, sizeof stat); + db_imp->Stat(&stat); + ASSERT_EQ(stat.uk_num, 3U); + ASSERT_EQ(stat.cur_conflict_bytes, 44U); + ASSERT_ROCKS_OK(txn->Commit()); + memset(&stat, 0, sizeof stat); + db_imp->Stat(&stat); + ASSERT_EQ(stat.uk_num, 0U); + ASSERT_EQ(stat.ck_num, 3U); + ASSERT_EQ(stat.cur_conflict_bytes, 3U + 28U + 4U + 28U + 1U + 28U); + delete txn; +} + +// test that when setting readTs, snapshot will be reset to the latest state. +TEST_F(TOTransactionTest, ResetSnapshot) { + WriteOptions write_options; + ReadOptions read_options; + string value; + Status s; + + std::unique_ptr txn1( + txn_db->BeginTransaction(write_options, txn_options)); + std::unique_ptr txn2( + txn_db->BeginTransaction(write_options, txn_options)); + ASSERT_ROCKS_OK(txn1->SetCommitTimeStamp(100)); + ASSERT_ROCKS_OK(txn1->Put("a", "b")); + ASSERT_ROCKS_OK(txn1->Commit()); + // a non-timestamped txn obeys CSI + ASSERT_TRUE(txn2->Get(read_options, "a", &value).IsNotFound()); + ASSERT_ROCKS_OK(txn2->SetReadTimeStamp(100)); + ASSERT_ROCKS_OK(txn2->Get(read_options, "a", &value)); + ASSERT_EQ(value, "b"); +} + +// needs SyncPoint +// TEST_F(TOTransactionTest, tsPinBottomLevelCompaction) { +// // bottom-level files may contain deletions due to snapshots protecting the +// // deleted keys. Once the snapshot is released, we should see files with many +// // such deletions undergo single-file compactions. +// const int kNumKeysPerFile = 1024; +// const int kNumLevelFiles = 4; +// const int kValueSize = 128; +// auto newOptions = options; +// newOptions.compression = kNoCompression; +// newOptions.level0_file_num_compaction_trigger = kNumLevelFiles; +// // inflate it a bit to account for key/metadata overhead +// newOptions.target_file_size_base = 130 * kNumKeysPerFile * kValueSize / 100; +// Reopen(newOptions); +// auto txn_db_imp = dynamic_cast(txn_db); +// auto db_imp = txn_db_imp->getDbImpl(); +// +// WriteOptions write_options; +// ReadOptions read_options; +// //read_options.read_timestamp = 50; +// string value; +// Status s; +// +// Random rnd(301); +// const Snapshot* snapshot = nullptr; +// for (int i = 0; i < kNumLevelFiles; ++i) { +// TOTransaction* txn = txn_db->BeginTransaction(write_options, txn_options); +// for (int j = 0; j < kNumKeysPerFile; ++j) { +// ASSERT_ROCKS_OK( +// txn->Put(DBTestBase::Key(i * kNumKeysPerFile + j), rnd.RandomString(kValueSize))); +// } +// if (i == kNumLevelFiles - 1) { +// snapshot = db_imp->GetSnapshot(); +// // delete every other key after grabbing a snapshot, so these deletions +// // and the keys they cover can't be dropped until after the snapshot is +// // released. +// for (int j = 0; j < kNumLevelFiles * kNumKeysPerFile; j += 2) { +// ASSERT_ROCKS_OK(txn->Delete(DBTestBase::Key(j))); +// } +// } +// ASSERT_ROCKS_OK(txn->SetCommitTimeStamp(i+1)); +// s = txn->Commit(); +// ASSERT_ROCKS_OK(s); +// delete txn; +// db_imp->Flush(FlushOptions()); +// } +// db_imp->TEST_WaitForCompact(); +// std::string level1FileNum; +// db_imp->GetProperty("rocksdb.num-files-at-level1", &level1FileNum); +// ASSERT_EQ(std::to_string(kNumLevelFiles), level1FileNum); +// std::vector pre_release_metadata, post_release_metadata; +// db_imp->GetLiveFilesMetaData(&pre_release_metadata); +// // just need to bump seqnum so ReleaseSnapshot knows the newest key in the SST +// // files does not need to be preserved in case of a future snapshot. +// TOTransaction* txn = txn_db->BeginTransaction(write_options, txn_options); +// ASSERT_ROCKS_OK(txn->Put(DBTestBase::Key(0), "val")); +// s = txn->Commit(); +// ASSERT_ROCKS_OK(s); +// delete txn; +// +// // set the pin_ts to 1, which will make bottom compact make no progress since +// // every sst file's max ts is >= 1. +// // if want test Bottom compaction working well, +// // see CompactBottomLevelFilesWithDeletions +// s = txn_db->SetTimeStamp(kOldest, 1, false); +// ASSERT_ROCKS_OK(s); +// db_imp->ReleaseSnapshot(snapshot); +// rocksdb::SyncPoint::GetInstance()->SetCallBack( +// "LevelCompactionPicker::PickCompaction:Return", [&](void* arg) { +// Compaction* compaction = reinterpret_cast(arg); +// ASSERT_TRUE(compaction->compaction_reason() == +// CompactionReason::kBottommostFiles); +// }); +// rocksdb::SyncPoint::GetInstance()->EnableProcessing(); +// db_imp->TEST_WaitForCompact(); +// db_imp->GetLiveFilesMetaData(&post_release_metadata); +// ASSERT_TRUE(pre_release_metadata.size() == post_release_metadata.size()); +// +// size_t sum_pre = 0, sum_post = 0; +// for (size_t i = 0; i < pre_release_metadata.size(); ++i) { +// ASSERT_EQ(1, pre_release_metadata[i].level); +// sum_pre += pre_release_metadata[i].size; +// } +// for (size_t i = 0; i < post_release_metadata.size(); ++i) { +// sum_post += post_release_metadata[i].size; +// } +// ASSERT_EQ(sum_post, sum_pre); +// rocksdb::SyncPoint::GetInstance()->DisableProcessing(); +// } + +// needs TEST_WaitForCompact +// TEST_F(TOTransactionTest, DISABLED_TrimHistoryToStableTs) { +// const int kNumKeysPerFile = 1024; +// const int kNumLevelFiles = 4; +// const int kValueSize = 128; +// auto newOptions = options; +// newOptions.compression = kNoCompression; +// newOptions.level0_file_num_compaction_trigger = kNumLevelFiles + 1; +// // inflate it a bit to account for key/metadata overhead +// newOptions.target_file_size_base = 130 * kNumKeysPerFile * kValueSize / 100; +// Reopen(newOptions); +// auto txn_db_imp = dynamic_cast(txn_db); +// auto db_imp = txn_db_imp->getDbImpl(); +// +// WriteOptions write_options; +// string value; +// Status s; +// +// Random rnd(301); +// for (int i = 0; i < kNumLevelFiles; ++i) { +// TOTransaction* txn = txn_db->BeginTransaction(write_options, txn_options); +// for (int j = 0; j < kNumKeysPerFile; ++j) { +// ASSERT_ROCKS_OK( +// txn->Put(DBTestBase::Key(i * kNumKeysPerFile + j), rnd.RandomString(kValueSize))); +// } +// ASSERT_ROCKS_OK(txn->SetCommitTimeStamp(i+1)); +// s = txn->Commit(); +// ASSERT_ROCKS_OK(s); +// delete txn; +// db_imp->Flush(FlushOptions()); +// } +// db_imp->TEST_WaitForCompact(); +// std::string level1FileNum; +// db_imp->GetProperty("rocksdb.num-files-at-level0", &level1FileNum); +// ASSERT_EQ(std::to_string(kNumLevelFiles), level1FileNum); +// std::vector pre_release_metadata, post_release_metadata; +// db_imp->GetLiveFilesMetaData(&pre_release_metadata); +// s = txn_db->SetTimeStamp(kStable, 1, false); +// // db_imp->TrimHistoryToStableTs(db_imp->DefaultColumnFamily()); +// db_imp->GetProperty("rocksdb.num-files-at-level0", &level1FileNum); +// ASSERT_EQ("1", level1FileNum); +// +// db_imp->GetLiveFilesMetaData(&post_release_metadata); +// ASSERT_TRUE(1 == post_release_metadata.size()); +// +// size_t sum_pre = 0, sum_post = 0; +// for (size_t i = 0; i < pre_release_metadata.size(); ++i) { +// ASSERT_EQ(0, pre_release_metadata[i].level); +// sum_pre += pre_release_metadata[i].size; +// } +// for (size_t i = 0; i < post_release_metadata.size(); ++i) { +// ASSERT_EQ(0, post_release_metadata[i].level); +// sum_post += post_release_metadata[i].size; +// } +// ASSERT_LT(sum_post, sum_pre); +// } + +} // namespace rocksdb