2017-08-03 17:46:47 +02:00
|
|
|
// 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
|
|
|
|
|
2017-08-08 01:07:40 +02:00
|
|
|
#include "utilities/transactions/write_prepared_txn.h"
|
2017-08-03 17:46:47 +02:00
|
|
|
|
2018-01-09 17:47:46 +01:00
|
|
|
#ifndef __STDC_FORMAT_MACROS
|
|
|
|
#define __STDC_FORMAT_MACROS
|
|
|
|
#endif
|
|
|
|
|
|
|
|
#include <inttypes.h>
|
2017-08-03 17:46:47 +02:00
|
|
|
#include <map>
|
2018-02-06 03:32:54 +01:00
|
|
|
#include <set>
|
2017-08-03 17:46:47 +02:00
|
|
|
|
|
|
|
#include "db/column_family.h"
|
|
|
|
#include "db/db_impl.h"
|
|
|
|
#include "rocksdb/db.h"
|
|
|
|
#include "rocksdb/status.h"
|
|
|
|
#include "rocksdb/utilities/transaction_db.h"
|
2018-04-03 05:19:21 +02:00
|
|
|
#include "util/cast_util.h"
|
2017-08-08 01:07:40 +02:00
|
|
|
#include "utilities/transactions/pessimistic_transaction.h"
|
2017-11-02 19:05:55 +01:00
|
|
|
#include "utilities/transactions/write_prepared_txn_db.h"
|
2017-08-03 17:46:47 +02:00
|
|
|
|
|
|
|
namespace rocksdb {
|
|
|
|
|
|
|
|
struct WriteOptions;
|
|
|
|
|
2017-08-17 01:49:11 +02:00
|
|
|
WritePreparedTxn::WritePreparedTxn(WritePreparedTxnDB* txn_db,
|
|
|
|
const WriteOptions& write_options,
|
|
|
|
const TransactionOptions& txn_options)
|
|
|
|
: PessimisticTransaction(txn_db, write_options, txn_options),
|
2018-02-06 03:32:54 +01:00
|
|
|
wpt_db_(txn_db) {}
|
2017-08-03 17:46:47 +02:00
|
|
|
|
2018-07-24 09:09:18 +02:00
|
|
|
void WritePreparedTxn::Initialize(const TransactionOptions& txn_options) {
|
|
|
|
PessimisticTransaction::Initialize(txn_options);
|
|
|
|
prepare_batch_cnt_ = 0;
|
|
|
|
}
|
|
|
|
|
2017-09-11 17:58:52 +02:00
|
|
|
Status WritePreparedTxn::Get(const ReadOptions& read_options,
|
|
|
|
ColumnFamilyHandle* column_family,
|
|
|
|
const Slice& key, PinnableSlice* pinnable_val) {
|
2017-10-06 23:18:30 +02:00
|
|
|
auto snapshot = read_options.snapshot;
|
2017-09-11 17:58:52 +02:00
|
|
|
auto snap_seq =
|
|
|
|
snapshot != nullptr ? snapshot->GetSequenceNumber() : kMaxSequenceNumber;
|
2018-04-03 05:19:21 +02:00
|
|
|
SequenceNumber min_uncommitted = 0; // by default disable the optimization
|
|
|
|
if (snapshot != nullptr) {
|
|
|
|
min_uncommitted =
|
|
|
|
static_cast_with_check<const SnapshotImpl, const Snapshot>(snapshot)
|
|
|
|
->min_uncommitted_;
|
|
|
|
}
|
2017-09-11 17:58:52 +02:00
|
|
|
|
2018-04-03 05:19:21 +02:00
|
|
|
WritePreparedTxnReadCallback callback(wpt_db_, snap_seq, min_uncommitted);
|
2017-09-11 17:58:52 +02:00
|
|
|
return write_batch_.GetFromBatchAndDB(db_, read_options, column_family, key,
|
|
|
|
pinnable_val, &callback);
|
|
|
|
}
|
|
|
|
|
2017-10-10 02:05:34 +02:00
|
|
|
Iterator* WritePreparedTxn::GetIterator(const ReadOptions& options) {
|
|
|
|
// Make sure to get iterator from WritePrepareTxnDB, not the root db.
|
|
|
|
Iterator* db_iter = wpt_db_->NewIterator(options);
|
|
|
|
assert(db_iter);
|
|
|
|
|
2018-12-04 08:36:32 +01:00
|
|
|
return write_batch_.NewIteratorWithBase(db_iter);
|
2017-10-10 02:05:34 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
Iterator* WritePreparedTxn::GetIterator(const ReadOptions& options,
|
|
|
|
ColumnFamilyHandle* column_family) {
|
|
|
|
// Make sure to get iterator from WritePrepareTxnDB, not the root db.
|
|
|
|
Iterator* db_iter = wpt_db_->NewIterator(options, column_family);
|
|
|
|
assert(db_iter);
|
|
|
|
|
2018-12-04 08:36:32 +01:00
|
|
|
return write_batch_.NewIteratorWithBase(column_family, db_iter);
|
2017-10-10 02:05:34 +02:00
|
|
|
}
|
|
|
|
|
2017-08-08 01:07:40 +02:00
|
|
|
Status WritePreparedTxn::PrepareInternal() {
|
2017-08-17 01:49:11 +02:00
|
|
|
WriteOptions write_options = write_options_;
|
|
|
|
write_options.disableWAL = false;
|
2017-12-01 08:39:56 +01:00
|
|
|
const bool WRITE_AFTER_COMMIT = true;
|
2017-11-11 20:23:43 +01:00
|
|
|
WriteBatchInternal::MarkEndPrepare(GetWriteBatch()->GetWriteBatch(), name_,
|
2017-12-01 08:39:56 +01:00
|
|
|
!WRITE_AFTER_COMMIT);
|
2018-02-06 03:32:54 +01:00
|
|
|
// For each duplicate key we account for a new sub-batch
|
2018-02-23 03:05:14 +01:00
|
|
|
prepare_batch_cnt_ = GetWriteBatch()->SubBatchCnt();
|
2018-04-03 05:19:21 +02:00
|
|
|
// AddPrepared better to be called in the pre-release callback otherwise there
|
|
|
|
// is a non-zero chance of max advancing prepare_seq and readers assume the
|
|
|
|
// data as committed.
|
|
|
|
// Also having it in the PreReleaseCallback allows in-order addition of
|
|
|
|
// prepared entries to PrepareHeap and hence enables an optimization. Refer to
|
|
|
|
// SmallestUnCommittedSeq for more details.
|
|
|
|
AddPreparedCallback add_prepared_callback(
|
|
|
|
wpt_db_, prepare_batch_cnt_,
|
|
|
|
db_impl_->immutable_db_options().two_write_queues);
|
|
|
|
const bool DISABLE_MEMTABLE = true;
|
|
|
|
uint64_t seq_used = kMaxSequenceNumber;
|
|
|
|
Status s = db_impl_->WriteImpl(
|
|
|
|
write_options, GetWriteBatch()->GetWriteBatch(),
|
|
|
|
/*callback*/ nullptr, &log_number_, /*log ref*/ 0, !DISABLE_MEMTABLE,
|
|
|
|
&seq_used, prepare_batch_cnt_, &add_prepared_callback);
|
2018-01-09 17:47:46 +01:00
|
|
|
assert(!s.ok() || seq_used != kMaxSequenceNumber);
|
2017-09-29 01:43:04 +02:00
|
|
|
auto prepare_seq = seq_used;
|
|
|
|
SetId(prepare_seq);
|
2017-08-17 01:49:11 +02:00
|
|
|
return s;
|
2017-08-03 17:46:47 +02:00
|
|
|
}
|
|
|
|
|
2017-08-08 01:07:40 +02:00
|
|
|
Status WritePreparedTxn::CommitWithoutPrepareInternal() {
|
2018-02-06 03:32:54 +01:00
|
|
|
// For each duplicate key we account for a new sub-batch
|
2018-02-23 03:05:14 +01:00
|
|
|
const size_t batch_cnt = GetWriteBatch()->SubBatchCnt();
|
2018-02-06 03:32:54 +01:00
|
|
|
return CommitBatchInternal(GetWriteBatch()->GetWriteBatch(), batch_cnt);
|
2017-09-09 00:53:51 +02:00
|
|
|
}
|
|
|
|
|
2018-02-06 03:32:54 +01:00
|
|
|
Status WritePreparedTxn::CommitBatchInternal(WriteBatch* batch,
|
|
|
|
size_t batch_cnt) {
|
2018-02-13 01:27:39 +01:00
|
|
|
return wpt_db_->WriteInternal(write_options_, batch, batch_cnt, this);
|
2017-08-03 17:46:47 +02:00
|
|
|
}
|
|
|
|
|
2017-08-08 01:07:40 +02:00
|
|
|
Status WritePreparedTxn::CommitInternal() {
|
2018-01-09 17:47:46 +01:00
|
|
|
ROCKS_LOG_DETAILS(db_impl_->immutable_db_options().info_log,
|
|
|
|
"CommitInternal prepare_seq: %" PRIu64, GetID());
|
2017-08-17 01:49:11 +02:00
|
|
|
// We take the commit-time batch and append the Commit marker.
|
|
|
|
// The Memtable will ignore the Commit marker in non-recovery mode
|
|
|
|
WriteBatch* working_batch = GetCommitTimeWriteBatch();
|
2017-10-06 23:18:30 +02:00
|
|
|
const bool empty = working_batch->Count() == 0;
|
2017-08-17 01:49:11 +02:00
|
|
|
WriteBatchInternal::MarkCommit(working_batch, name_);
|
|
|
|
|
2017-11-02 01:23:52 +01:00
|
|
|
const bool for_recovery = use_only_the_last_commit_time_batch_for_recovery_;
|
|
|
|
if (!empty && for_recovery) {
|
|
|
|
// When not writing to memtable, we can still cache the latest write batch.
|
|
|
|
// The cached batch will be written to memtable in WriteRecoverableState
|
|
|
|
// during FlushMemTable
|
|
|
|
WriteBatchInternal::SetAsLastestPersistentState(working_batch);
|
|
|
|
}
|
2017-08-17 01:49:11 +02:00
|
|
|
|
2017-12-01 08:39:56 +01:00
|
|
|
auto prepare_seq = GetId();
|
|
|
|
const bool includes_data = !empty && !for_recovery;
|
2018-02-06 03:32:54 +01:00
|
|
|
assert(prepare_batch_cnt_);
|
|
|
|
size_t commit_batch_cnt = 0;
|
2018-02-16 17:36:47 +01:00
|
|
|
if (UNLIKELY(includes_data)) {
|
|
|
|
ROCKS_LOG_WARN(db_impl_->immutable_db_options().info_log,
|
|
|
|
"Duplicate key overhead");
|
2018-02-06 03:32:54 +01:00
|
|
|
SubBatchCounter counter(*wpt_db_->GetCFComparatorMap());
|
|
|
|
auto s = working_batch->Iterate(&counter);
|
|
|
|
assert(s.ok());
|
|
|
|
commit_batch_cnt = counter.BatchCount();
|
|
|
|
}
|
2017-12-01 08:39:56 +01:00
|
|
|
const bool disable_memtable = !includes_data;
|
2018-03-22 22:27:44 +01:00
|
|
|
const bool do_one_write =
|
|
|
|
!db_impl_->immutable_db_options().two_write_queues || disable_memtable;
|
|
|
|
const bool publish_seq = do_one_write;
|
2018-04-03 05:19:21 +02:00
|
|
|
// Note: CommitTimeWriteBatch does not need AddPrepared since it is written to
|
|
|
|
// DB in one shot. min_uncommitted still works since it requires capturing
|
|
|
|
// data that is written to DB but not yet committed, while
|
|
|
|
// CommitTimeWriteBatch commits with PreReleaseCallback.
|
2018-03-22 22:27:44 +01:00
|
|
|
WritePreparedCommitEntryPreReleaseCallback update_commit_map(
|
|
|
|
wpt_db_, db_impl_, prepare_seq, prepare_batch_cnt_, commit_batch_cnt,
|
2018-04-12 04:59:25 +02:00
|
|
|
publish_seq);
|
2017-09-29 01:43:04 +02:00
|
|
|
uint64_t seq_used = kMaxSequenceNumber;
|
|
|
|
// Since the prepared batch is directly written to memtable, there is already
|
|
|
|
// a connection between the memtable and its WAL, so there is no need to
|
|
|
|
// redundantly reference the log that contains the prepared data.
|
|
|
|
const uint64_t zero_log_number = 0ull;
|
2018-02-16 17:36:47 +01:00
|
|
|
size_t batch_cnt = UNLIKELY(commit_batch_cnt) ? commit_batch_cnt : 1;
|
2017-12-01 08:39:56 +01:00
|
|
|
auto s = db_impl_->WriteImpl(write_options_, working_batch, nullptr, nullptr,
|
|
|
|
zero_log_number, disable_memtable, &seq_used,
|
2018-02-06 03:32:54 +01:00
|
|
|
batch_cnt, &update_commit_map);
|
2018-01-09 17:47:46 +01:00
|
|
|
assert(!s.ok() || seq_used != kMaxSequenceNumber);
|
2018-03-22 22:27:44 +01:00
|
|
|
if (LIKELY(do_one_write || !s.ok())) {
|
2018-04-12 04:59:25 +02:00
|
|
|
if (LIKELY(s.ok())) {
|
|
|
|
// Note RemovePrepared should be called after WriteImpl that publishsed
|
|
|
|
// the seq. Otherwise SmallestUnCommittedSeq optimization breaks.
|
|
|
|
wpt_db_->RemovePrepared(prepare_seq, prepare_batch_cnt_);
|
|
|
|
}
|
2018-03-22 22:27:44 +01:00
|
|
|
return s;
|
|
|
|
} // else do the 2nd write to publish seq
|
|
|
|
// Note: the 2nd write comes with a performance penality. So if we have too
|
|
|
|
// many of commits accompanied with ComitTimeWriteBatch and yet we cannot
|
|
|
|
// enable use_only_the_last_commit_time_batch_for_recovery_ optimization,
|
|
|
|
// two_write_queues should be disabled to avoid many additional writes here.
|
|
|
|
class PublishSeqPreReleaseCallback : public PreReleaseCallback {
|
|
|
|
public:
|
|
|
|
explicit PublishSeqPreReleaseCallback(DBImpl* db_impl)
|
|
|
|
: db_impl_(db_impl) {}
|
|
|
|
virtual Status Callback(SequenceNumber seq, bool is_mem_disabled) override {
|
2018-04-13 02:55:14 +02:00
|
|
|
#ifdef NDEBUG
|
|
|
|
(void)is_mem_disabled;
|
|
|
|
#endif
|
2018-03-22 22:27:44 +01:00
|
|
|
assert(is_mem_disabled);
|
|
|
|
assert(db_impl_->immutable_db_options().two_write_queues);
|
|
|
|
db_impl_->SetLastPublishedSequence(seq);
|
|
|
|
return Status::OK();
|
|
|
|
}
|
|
|
|
|
|
|
|
private:
|
|
|
|
DBImpl* db_impl_;
|
|
|
|
} publish_seq_callback(db_impl_);
|
|
|
|
WriteBatch empty_batch;
|
|
|
|
empty_batch.PutLogData(Slice());
|
|
|
|
// In the absence of Prepare markers, use Noop as a batch separator
|
|
|
|
WriteBatchInternal::InsertNoop(&empty_batch);
|
|
|
|
const bool DISABLE_MEMTABLE = true;
|
|
|
|
const size_t ONE_BATCH = 1;
|
|
|
|
const uint64_t NO_REF_LOG = 0;
|
|
|
|
s = db_impl_->WriteImpl(write_options_, &empty_batch, nullptr, nullptr,
|
|
|
|
NO_REF_LOG, DISABLE_MEMTABLE, &seq_used, ONE_BATCH,
|
|
|
|
&publish_seq_callback);
|
|
|
|
assert(!s.ok() || seq_used != kMaxSequenceNumber);
|
2018-04-12 04:59:25 +02:00
|
|
|
// Note RemovePrepared should be called after WriteImpl that publishsed the
|
|
|
|
// seq. Otherwise SmallestUnCommittedSeq optimization breaks.
|
|
|
|
wpt_db_->RemovePrepared(prepare_seq, prepare_batch_cnt_);
|
2017-08-17 01:49:11 +02:00
|
|
|
return s;
|
2017-08-08 01:07:40 +02:00
|
|
|
}
|
|
|
|
|
2017-10-03 04:46:42 +02:00
|
|
|
Status WritePreparedTxn::RollbackInternal() {
|
2018-01-09 17:47:46 +01:00
|
|
|
ROCKS_LOG_WARN(db_impl_->immutable_db_options().info_log,
|
|
|
|
"RollbackInternal prepare_seq: %" PRIu64, GetId());
|
2017-10-03 04:46:42 +02:00
|
|
|
WriteBatch rollback_batch;
|
|
|
|
assert(GetId() != kMaxSequenceNumber);
|
|
|
|
assert(GetId() > 0);
|
2018-05-03 03:09:55 +02:00
|
|
|
auto cf_map_shared_ptr = wpt_db_->GetCFHandleMap();
|
|
|
|
auto cf_comp_map_shared_ptr = wpt_db_->GetCFComparatorMap();
|
2017-10-03 04:46:42 +02:00
|
|
|
// In WritePrepared, the txn is is the same as prepare seq
|
|
|
|
auto last_visible_txn = GetId() - 1;
|
|
|
|
struct RollbackWriteBatchBuilder : public WriteBatch::Handler {
|
|
|
|
DBImpl* db_;
|
|
|
|
ReadOptions roptions;
|
|
|
|
WritePreparedTxnReadCallback callback;
|
|
|
|
WriteBatch* rollback_batch_;
|
2018-02-06 03:32:54 +01:00
|
|
|
std::map<uint32_t, const Comparator*>& comparators_;
|
2018-05-03 03:09:55 +02:00
|
|
|
std::map<uint32_t, ColumnFamilyHandle*>& handles_;
|
2018-02-06 03:32:54 +01:00
|
|
|
using CFKeys = std::set<Slice, SetComparator>;
|
|
|
|
std::map<uint32_t, CFKeys> keys_;
|
2018-04-12 20:52:15 +02:00
|
|
|
bool rollback_merge_operands_;
|
2018-02-06 03:32:54 +01:00
|
|
|
RollbackWriteBatchBuilder(
|
|
|
|
DBImpl* db, WritePreparedTxnDB* wpt_db, SequenceNumber snap_seq,
|
|
|
|
WriteBatch* dst_batch,
|
2018-04-12 20:52:15 +02:00
|
|
|
std::map<uint32_t, const Comparator*>& comparators,
|
2018-05-03 03:09:55 +02:00
|
|
|
std::map<uint32_t, ColumnFamilyHandle*>& handles,
|
2018-04-12 20:52:15 +02:00
|
|
|
bool rollback_merge_operands)
|
2018-02-06 03:32:54 +01:00
|
|
|
: db_(db),
|
2018-04-03 05:19:21 +02:00
|
|
|
callback(wpt_db, snap_seq,
|
|
|
|
0), // 0 disables min_uncommitted optimization
|
2018-02-06 03:32:54 +01:00
|
|
|
rollback_batch_(dst_batch),
|
2018-04-12 20:52:15 +02:00
|
|
|
comparators_(comparators),
|
2018-05-03 03:09:55 +02:00
|
|
|
handles_(handles),
|
2018-04-12 20:52:15 +02:00
|
|
|
rollback_merge_operands_(rollback_merge_operands) {}
|
2017-10-03 04:46:42 +02:00
|
|
|
|
|
|
|
Status Rollback(uint32_t cf, const Slice& key) {
|
2018-02-06 03:32:54 +01:00
|
|
|
Status s;
|
|
|
|
CFKeys& cf_keys = keys_[cf];
|
|
|
|
if (cf_keys.size() == 0) { // just inserted
|
|
|
|
auto cmp = comparators_[cf];
|
|
|
|
keys_[cf] = CFKeys(SetComparator(cmp));
|
|
|
|
}
|
|
|
|
auto it = cf_keys.insert(key);
|
|
|
|
if (it.second ==
|
|
|
|
false) { // second is false if a element already existed.
|
|
|
|
return s;
|
|
|
|
}
|
|
|
|
|
2017-10-03 04:46:42 +02:00
|
|
|
PinnableSlice pinnable_val;
|
|
|
|
bool not_used;
|
2018-05-03 03:09:55 +02:00
|
|
|
auto cf_handle = handles_[cf];
|
2018-02-06 03:32:54 +01:00
|
|
|
s = db_->GetImpl(roptions, cf_handle, key, &pinnable_val, ¬_used,
|
|
|
|
&callback);
|
2017-10-03 04:46:42 +02:00
|
|
|
assert(s.ok() || s.IsNotFound());
|
|
|
|
if (s.ok()) {
|
|
|
|
s = rollback_batch_->Put(cf_handle, key, pinnable_val);
|
|
|
|
assert(s.ok());
|
|
|
|
} else if (s.IsNotFound()) {
|
|
|
|
// There has been no readable value before txn. By adding a delete we
|
|
|
|
// make sure that there will be none afterwards either.
|
|
|
|
s = rollback_batch_->Delete(cf_handle, key);
|
|
|
|
assert(s.ok());
|
|
|
|
} else {
|
|
|
|
// Unexpected status. Return it to the user.
|
|
|
|
}
|
|
|
|
return s;
|
|
|
|
}
|
|
|
|
|
2018-03-05 22:08:17 +01:00
|
|
|
Status PutCF(uint32_t cf, const Slice& key, const Slice& /*val*/) override {
|
2017-10-03 04:46:42 +02:00
|
|
|
return Rollback(cf, key);
|
|
|
|
}
|
|
|
|
|
|
|
|
Status DeleteCF(uint32_t cf, const Slice& key) override {
|
|
|
|
return Rollback(cf, key);
|
|
|
|
}
|
|
|
|
|
|
|
|
Status SingleDeleteCF(uint32_t cf, const Slice& key) override {
|
|
|
|
return Rollback(cf, key);
|
|
|
|
}
|
|
|
|
|
2018-03-05 22:08:17 +01:00
|
|
|
Status MergeCF(uint32_t cf, const Slice& key,
|
|
|
|
const Slice& /*val*/) override {
|
2018-04-12 20:52:15 +02:00
|
|
|
if (rollback_merge_operands_) {
|
|
|
|
return Rollback(cf, key);
|
|
|
|
} else {
|
|
|
|
return Status::OK();
|
|
|
|
}
|
2017-10-03 04:46:42 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
Status MarkNoop(bool) override { return Status::OK(); }
|
2018-07-07 02:17:36 +02:00
|
|
|
Status MarkBeginPrepare(bool) override { return Status::OK(); }
|
2017-10-03 04:46:42 +02:00
|
|
|
Status MarkEndPrepare(const Slice&) override { return Status::OK(); }
|
|
|
|
Status MarkCommit(const Slice&) override { return Status::OK(); }
|
|
|
|
Status MarkRollback(const Slice&) override {
|
|
|
|
return Status::InvalidArgument();
|
|
|
|
}
|
2017-11-11 20:23:43 +01:00
|
|
|
|
|
|
|
protected:
|
|
|
|
virtual bool WriteAfterCommit() const override { return false; }
|
2018-02-06 03:32:54 +01:00
|
|
|
} rollback_handler(db_impl_, wpt_db_, last_visible_txn, &rollback_batch,
|
2018-05-03 03:09:55 +02:00
|
|
|
*cf_comp_map_shared_ptr.get(), *cf_map_shared_ptr.get(),
|
2018-04-12 20:52:15 +02:00
|
|
|
wpt_db_->txn_db_options_.rollback_merge_operands);
|
2017-10-03 04:46:42 +02:00
|
|
|
auto s = GetWriteBatch()->GetWriteBatch()->Iterate(&rollback_handler);
|
|
|
|
assert(s.ok());
|
|
|
|
if (!s.ok()) {
|
|
|
|
return s;
|
|
|
|
}
|
2017-11-15 17:19:57 +01:00
|
|
|
// The Rollback marker will be used as a batch separator
|
2017-10-03 04:46:42 +02:00
|
|
|
WriteBatchInternal::MarkRollback(&rollback_batch, name_);
|
2017-12-18 17:03:18 +01:00
|
|
|
bool do_one_write = !db_impl_->immutable_db_options().two_write_queues;
|
2017-12-01 08:39:56 +01:00
|
|
|
const bool DISABLE_MEMTABLE = true;
|
2018-03-22 22:27:44 +01:00
|
|
|
const uint64_t NO_REF_LOG = 0;
|
2017-10-03 04:46:42 +02:00
|
|
|
uint64_t seq_used = kMaxSequenceNumber;
|
2018-02-06 03:32:54 +01:00
|
|
|
const size_t ONE_BATCH = 1;
|
2018-04-21 00:25:12 +02:00
|
|
|
// We commit the rolled back prepared batches. ALthough this is
|
|
|
|
// counter-intuitive, i) it is safe to do so, since the prepared batches are
|
|
|
|
// already canceled out by the rollback batch, ii) adding the commit entry to
|
|
|
|
// CommitCache will allow us to benefit from the existing mechanism in
|
|
|
|
// CommitCache that keeps an entry evicted due to max advance and yet overlaps
|
|
|
|
// with a live snapshot around so that the live snapshot properly skips the
|
|
|
|
// entry even if its prepare seq is lower than max_evicted_seq_.
|
2017-12-18 17:03:18 +01:00
|
|
|
WritePreparedCommitEntryPreReleaseCallback update_commit_map(
|
2018-04-21 00:25:12 +02:00
|
|
|
wpt_db_, db_impl_, GetId(), prepare_batch_cnt_, ONE_BATCH);
|
2018-04-03 05:19:21 +02:00
|
|
|
// Note: the rollback batch does not need AddPrepared since it is written to
|
|
|
|
// DB in one shot. min_uncommitted still works since it requires capturing
|
|
|
|
// data that is written to DB but not yet committed, while
|
|
|
|
// the roolback batch commits with PreReleaseCallback.
|
2017-10-03 04:46:42 +02:00
|
|
|
s = db_impl_->WriteImpl(write_options_, &rollback_batch, nullptr, nullptr,
|
2018-03-22 22:27:44 +01:00
|
|
|
NO_REF_LOG, !DISABLE_MEMTABLE, &seq_used, ONE_BATCH,
|
2017-12-18 17:03:18 +01:00
|
|
|
do_one_write ? &update_commit_map : nullptr);
|
2018-01-09 17:47:46 +01:00
|
|
|
assert(!s.ok() || seq_used != kMaxSequenceNumber);
|
2017-12-01 08:39:56 +01:00
|
|
|
if (!s.ok()) {
|
|
|
|
return s;
|
|
|
|
}
|
2017-12-18 17:03:18 +01:00
|
|
|
if (do_one_write) {
|
2018-04-21 00:25:12 +02:00
|
|
|
wpt_db_->RemovePrepared(GetId(), prepare_batch_cnt_);
|
2017-12-18 17:03:18 +01:00
|
|
|
return s;
|
|
|
|
} // else do the 2nd write for commit
|
2017-10-03 04:46:42 +02:00
|
|
|
uint64_t& prepare_seq = seq_used;
|
2018-01-09 17:47:46 +01:00
|
|
|
ROCKS_LOG_DETAILS(db_impl_->immutable_db_options().info_log,
|
|
|
|
"RollbackInternal 2nd write prepare_seq: %" PRIu64,
|
|
|
|
prepare_seq);
|
2017-12-01 08:39:56 +01:00
|
|
|
// Commit the batch by writing an empty batch to the queue that will release
|
|
|
|
// the commit sequence number to readers.
|
2018-02-21 22:40:31 +01:00
|
|
|
const size_t ZERO_COMMITS = 0;
|
2017-12-18 17:03:18 +01:00
|
|
|
WritePreparedCommitEntryPreReleaseCallback update_commit_map_with_prepare(
|
2018-04-12 04:59:25 +02:00
|
|
|
wpt_db_, db_impl_, prepare_seq, ONE_BATCH, ZERO_COMMITS);
|
2017-12-01 08:39:56 +01:00
|
|
|
WriteBatch empty_batch;
|
|
|
|
empty_batch.PutLogData(Slice());
|
|
|
|
// In the absence of Prepare markers, use Noop as a batch separator
|
|
|
|
WriteBatchInternal::InsertNoop(&empty_batch);
|
|
|
|
s = db_impl_->WriteImpl(write_options_, &empty_batch, nullptr, nullptr,
|
2018-03-22 22:27:44 +01:00
|
|
|
NO_REF_LOG, DISABLE_MEMTABLE, &seq_used, ONE_BATCH,
|
2017-12-18 17:03:18 +01:00
|
|
|
&update_commit_map_with_prepare);
|
2018-01-09 17:47:46 +01:00
|
|
|
assert(!s.ok() || seq_used != kMaxSequenceNumber);
|
2017-10-03 04:46:42 +02:00
|
|
|
// Mark the txn as rolled back
|
2017-12-01 08:39:56 +01:00
|
|
|
uint64_t& rollback_seq = seq_used;
|
2018-01-09 17:47:46 +01:00
|
|
|
if (s.ok()) {
|
2018-04-21 00:25:12 +02:00
|
|
|
// Note: it is safe to do it after PreReleaseCallback via WriteImpl since
|
|
|
|
// all the writes by the prpared batch are already blinded by the rollback
|
|
|
|
// batch. The only reason we commit the prepared batch here is to benefit
|
|
|
|
// from the existing mechanism in CommitCache that takes care of the rare
|
|
|
|
// cases that the prepare seq is visible to a snsapshot but max evicted seq
|
|
|
|
// advances that prepare seq.
|
2018-03-24 01:21:47 +01:00
|
|
|
for (size_t i = 0; i < prepare_batch_cnt_; i++) {
|
2018-04-21 00:25:12 +02:00
|
|
|
wpt_db_->AddCommitted(GetId() + i, rollback_seq);
|
2018-03-24 01:21:47 +01:00
|
|
|
}
|
2018-04-21 00:25:12 +02:00
|
|
|
wpt_db_->RemovePrepared(GetId(), prepare_batch_cnt_);
|
2018-01-09 17:47:46 +01:00
|
|
|
}
|
2017-10-03 04:46:42 +02:00
|
|
|
|
|
|
|
return s;
|
2017-08-03 17:46:47 +02:00
|
|
|
}
|
|
|
|
|
2017-11-02 02:56:25 +01:00
|
|
|
Status WritePreparedTxn::ValidateSnapshot(ColumnFamilyHandle* column_family,
|
|
|
|
const Slice& key,
|
2017-11-11 22:08:22 +01:00
|
|
|
SequenceNumber* tracked_at_seq) {
|
2017-11-02 02:56:25 +01:00
|
|
|
assert(snapshot_);
|
|
|
|
|
2018-04-03 05:19:21 +02:00
|
|
|
SequenceNumber min_uncommitted =
|
|
|
|
static_cast_with_check<const SnapshotImpl, const Snapshot>(
|
|
|
|
snapshot_.get())
|
|
|
|
->min_uncommitted_;
|
2017-11-02 02:56:25 +01:00
|
|
|
SequenceNumber snap_seq = snapshot_->GetSequenceNumber();
|
2017-11-11 22:08:22 +01:00
|
|
|
// tracked_at_seq is either max or the last snapshot with which this key was
|
2017-11-02 02:56:25 +01:00
|
|
|
// trackeed so there is no need to apply the IsInSnapshot to this comparison
|
2017-11-11 22:08:22 +01:00
|
|
|
// here as tracked_at_seq is not a prepare seq.
|
|
|
|
if (*tracked_at_seq <= snap_seq) {
|
2017-11-02 02:56:25 +01:00
|
|
|
// If the key has been previous validated at a sequence number earlier
|
|
|
|
// than the curent snapshot's sequence number, we already know it has not
|
|
|
|
// been modified.
|
|
|
|
return Status::OK();
|
|
|
|
}
|
|
|
|
|
2017-11-11 22:08:22 +01:00
|
|
|
*tracked_at_seq = snap_seq;
|
2017-11-02 02:56:25 +01:00
|
|
|
|
|
|
|
ColumnFamilyHandle* cfh =
|
|
|
|
column_family ? column_family : db_impl_->DefaultColumnFamily();
|
|
|
|
|
2018-04-03 05:19:21 +02:00
|
|
|
WritePreparedTxnReadCallback snap_checker(wpt_db_, snap_seq, min_uncommitted);
|
2017-11-11 22:08:22 +01:00
|
|
|
return TransactionUtil::CheckKeyForConflicts(db_impl_, cfh, key.ToString(),
|
|
|
|
snap_seq, false /* cache_only */,
|
|
|
|
&snap_checker);
|
2017-11-02 02:56:25 +01:00
|
|
|
}
|
|
|
|
|
2018-04-03 05:19:21 +02:00
|
|
|
void WritePreparedTxn::SetSnapshot() {
|
2018-04-12 04:59:25 +02:00
|
|
|
// Note: for this optimization setting the last sequence number and obtaining
|
|
|
|
// the smallest uncommitted seq should be done atomically. However to avoid
|
|
|
|
// the mutex overhead, we call SmallestUnCommittedSeq BEFORE taking the
|
|
|
|
// snapshot. Since we always updated the list of unprepared seq (via
|
|
|
|
// AddPrepared) AFTER the last sequence is updated, this guarantees that the
|
|
|
|
// smallest uncommited seq that we pair with the snapshot is smaller or equal
|
|
|
|
// the value that would be obtained otherwise atomically. That is ok since
|
|
|
|
// this optimization works as long as min_uncommitted is less than or equal
|
|
|
|
// than the smallest uncommitted seq when the snapshot was taken.
|
|
|
|
auto min_uncommitted = wpt_db_->SmallestUnCommittedSeq();
|
2018-04-03 05:19:21 +02:00
|
|
|
const bool FOR_WW_CONFLICT_CHECK = true;
|
|
|
|
SnapshotImpl* snapshot = dbimpl_->GetSnapshotImpl(FOR_WW_CONFLICT_CHECK);
|
|
|
|
assert(snapshot);
|
2018-04-12 04:59:25 +02:00
|
|
|
wpt_db_->EnhanceSnapshot(snapshot, min_uncommitted);
|
2018-04-03 05:19:21 +02:00
|
|
|
SetSnapshotInternal(snapshot);
|
|
|
|
}
|
|
|
|
|
2018-02-06 03:32:54 +01:00
|
|
|
Status WritePreparedTxn::RebuildFromWriteBatch(WriteBatch* src_batch) {
|
|
|
|
auto ret = PessimisticTransaction::RebuildFromWriteBatch(src_batch);
|
2018-02-23 03:05:14 +01:00
|
|
|
prepare_batch_cnt_ = GetWriteBatch()->SubBatchCnt();
|
2018-02-06 03:32:54 +01:00
|
|
|
return ret;
|
|
|
|
}
|
|
|
|
|
2017-08-03 17:46:47 +02:00
|
|
|
} // namespace rocksdb
|
|
|
|
|
|
|
|
#endif // ROCKSDB_LITE
|