Skip to content

Commit

Permalink
Modification of WriteBatch to support two phase commit
Browse files Browse the repository at this point in the history
Summary: Adds three new WriteBatch data types: Prepare(xid), Commit(xid), Rollback(xid). Prepare(xid) should precede the (single) operation to which is applies. There can obviously be multiple Prepare(xid) markers. There should only be one Rollback(xid) or Commit(xid) marker yet not both. None of this logic is currently enforced and will most likely be implemented further up such as in the memtableinserter. All three markers are similar to PutLogData in that they are writebatch meta-data, ie stored but not counted. All three markers differ from PutLogData in that they will actually be written to disk. As for WriteBatchWithIndex, Prepare, Commit, Rollback are all implemented just as PutLogData and none are tested just as PutLogData.

Test Plan: single unit test in write_batch_test.

Reviewers: hermanlee4, sdong, anthony

Subscribers: leveldb, dhruba, vasilep, andrewkr

Differential Revision: https://reviews.facebook.net/D57867
  • Loading branch information
reidHoruff committed May 10, 2016
1 parent f548da3 commit 0460e9d
Show file tree
Hide file tree
Showing 10 changed files with 248 additions and 26 deletions.
7 changes: 6 additions & 1 deletion db/dbformat.h
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,11 @@ enum ValueType : unsigned char {
kTypeColumnFamilyMerge = 0x6, // WAL only.
kTypeSingleDeletion = 0x7,
kTypeColumnFamilySingleDeletion = 0x8, // WAL only.
kTypeBeginPrepareXID = 0x9, // WAL only.
kTypeEndPrepareXID = 0xA, // WAL only.
kTypeCommitXID = 0xB, // WAL only.
kTypeRollbackXID = 0xC, // WAL only.
kTypeNoop = 0xD, // WAL only.
kMaxValue = 0x7F // Not used for storing records.
};

Expand Down Expand Up @@ -478,5 +483,5 @@ extern bool ReadKeyFromWriteBatchEntry(Slice* input, Slice* key,
// input will be advanced to after the record.
extern Status ReadRecordFromWriteBatch(Slice* input, char* tag,
uint32_t* column_family, Slice* key,
Slice* value, Slice* blob);
Slice* value, Slice* blob, Slice* xid);
} // namespace rocksdb
142 changes: 134 additions & 8 deletions db/write_batch.cc
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,11 @@
// kTypeColumnFamilyDeletion varint32 varstring varstring
// kTypeColumnFamilySingleDeletion varint32 varstring varstring
// kTypeColumnFamilyMerge varint32 varstring varstring
// kTypeBeginPrepareXID varstring
// kTypeEndPrepareXID
// kTypeCommitXID varstring
// kTypeRollbackXID varstring
// kTypeNoop
// varstring :=
// len: varint32
// data: uint8[len]
Expand Down Expand Up @@ -48,11 +53,15 @@ namespace rocksdb {
namespace {

enum ContentFlags : uint32_t {
DEFERRED = 1,
HAS_PUT = 2,
HAS_DELETE = 4,
HAS_SINGLE_DELETE = 8,
HAS_MERGE = 16,
DEFERRED = 1 << 0,
HAS_PUT = 1 << 1,
HAS_DELETE = 1 << 2,
HAS_SINGLE_DELETE = 1 << 3,
HAS_MERGE = 1 << 4,
HAS_BEGIN_PREPARE = 1 << 5,
HAS_END_PREPARE = 1 << 6,
HAS_COMMIT = 1 << 7,
HAS_ROLLBACK = 1 << 8,
};

struct BatchContentClassifier : public WriteBatch::Handler {
Expand All @@ -77,6 +86,26 @@ struct BatchContentClassifier : public WriteBatch::Handler {
content_flags |= ContentFlags::HAS_MERGE;
return Status::OK();
}

Status MarkBeginPrepare() override {
content_flags |= ContentFlags::HAS_BEGIN_PREPARE;
return Status::OK();
}

Status MarkEndPrepare(const Slice&) override {
content_flags |= ContentFlags::HAS_END_PREPARE;
return Status::OK();
}

Status MarkCommit(const Slice&) override {
content_flags |= ContentFlags::HAS_COMMIT;
return Status::OK();
}

Status MarkRollback(const Slice&) override {
content_flags |= ContentFlags::HAS_ROLLBACK;
return Status::OK();
}
};

} // anon namespace
Expand Down Expand Up @@ -209,9 +238,25 @@ bool ReadKeyFromWriteBatchEntry(Slice* input, Slice* key, bool cf_record) {
return GetLengthPrefixedSlice(input, key);
}

bool WriteBatch::HasBeginPrepare() const {
return (ComputeContentFlags() & ContentFlags::HAS_BEGIN_PREPARE) != 0;
}

bool WriteBatch::HasEndPrepare() const {
return (ComputeContentFlags() & ContentFlags::HAS_END_PREPARE) != 0;
}

bool WriteBatch::HasCommit() const {
return (ComputeContentFlags() & ContentFlags::HAS_COMMIT) != 0;
}

bool WriteBatch::HasRollback() const {
return (ComputeContentFlags() & ContentFlags::HAS_ROLLBACK) != 0;
}

Status ReadRecordFromWriteBatch(Slice* input, char* tag,
uint32_t* column_family, Slice* key,
Slice* value, Slice* blob) {
Slice* value, Slice* blob, Slice* xid) {
assert(key != nullptr && value != nullptr);
*tag = (*input)[0];
input->remove_prefix(1);
Expand Down Expand Up @@ -257,6 +302,24 @@ Status ReadRecordFromWriteBatch(Slice* input, char* tag,
return Status::Corruption("bad WriteBatch Blob");
}
break;
case kTypeNoop:
case kTypeBeginPrepareXID:
break;
case kTypeEndPrepareXID:
if (!GetLengthPrefixedSlice(input, xid)) {
return Status::Corruption("bad EndPrepare XID");
}
break;
case kTypeCommitXID:
if (!GetLengthPrefixedSlice(input, xid)) {
return Status::Corruption("bad Commit XID");
}
break;
case kTypeRollbackXID:
if (!GetLengthPrefixedSlice(input, xid)) {
return Status::Corruption("bad Rollback XID");
}
break;
default:
return Status::Corruption("unknown WriteBatch tag");
}
Expand All @@ -270,15 +333,15 @@ Status WriteBatch::Iterate(Handler* handler) const {
}

input.remove_prefix(WriteBatchInternal::kHeader);
Slice key, value, blob;
Slice key, value, blob, xid;
int found = 0;
Status s;
while (s.ok() && !input.empty() && handler->Continue()) {
char tag = 0;
uint32_t column_family = 0; // default

s = ReadRecordFromWriteBatch(&input, &tag, &column_family, &key, &value,
&blob);
&blob, &xid);
if (!s.ok()) {
return s;
}
Expand Down Expand Up @@ -315,6 +378,28 @@ Status WriteBatch::Iterate(Handler* handler) const {
case kTypeLogData:
handler->LogData(blob);
break;
case kTypeBeginPrepareXID:
assert(content_flags_.load(std::memory_order_relaxed) &
(ContentFlags::DEFERRED | ContentFlags::HAS_BEGIN_PREPARE));
handler->MarkBeginPrepare();
break;
case kTypeEndPrepareXID:
assert(content_flags_.load(std::memory_order_relaxed) &
(ContentFlags::DEFERRED | ContentFlags::HAS_END_PREPARE));
handler->MarkEndPrepare(xid);
break;
case kTypeCommitXID:
assert(content_flags_.load(std::memory_order_relaxed) &
(ContentFlags::DEFERRED | ContentFlags::HAS_COMMIT));
handler->MarkCommit(xid);
break;
case kTypeRollbackXID:
assert(content_flags_.load(std::memory_order_relaxed) &
(ContentFlags::DEFERRED | ContentFlags::HAS_ROLLBACK));
handler->MarkRollback(xid);
break;
case kTypeNoop:
break;
default:
return Status::Corruption("unknown WriteBatch tag");
}
Expand Down Expand Up @@ -391,6 +476,47 @@ void WriteBatch::Put(ColumnFamilyHandle* column_family, const SliceParts& key,
WriteBatchInternal::Put(this, GetColumnFamilyID(column_family), key, value);
}

void WriteBatchInternal::InsertNoop(WriteBatch* b) {
b->rep_.push_back(static_cast<char>(kTypeNoop));
}

void WriteBatchInternal::MarkEndPrepare(WriteBatch* b, const Slice& xid) {
// a manually constructed batch can only contain one prepare section
assert(b->rep_[12] == static_cast<char>(kTypeNoop));

// all savepoints up to this point are cleared
if (b->save_points_ != nullptr) {
while (!b->save_points_->stack.empty()) {
b->save_points_->stack.pop();
}
}

// rewrite noop as begin marker
b->rep_[12] = static_cast<char>(kTypeBeginPrepareXID);
b->rep_.push_back(static_cast<char>(kTypeEndPrepareXID));
PutLengthPrefixedSlice(&b->rep_, xid);
b->content_flags_.store(b->content_flags_.load(std::memory_order_relaxed) |
ContentFlags::HAS_END_PREPARE |
ContentFlags::HAS_BEGIN_PREPARE,
std::memory_order_relaxed);
}

void WriteBatchInternal::MarkCommit(WriteBatch* b, const Slice& xid) {
b->rep_.push_back(static_cast<char>(kTypeCommitXID));
PutLengthPrefixedSlice(&b->rep_, xid);
b->content_flags_.store(b->content_flags_.load(std::memory_order_relaxed) |
ContentFlags::HAS_COMMIT,
std::memory_order_relaxed);
}

void WriteBatchInternal::MarkRollback(WriteBatch* b, const Slice& xid) {
b->rep_.push_back(static_cast<char>(kTypeRollbackXID));
PutLengthPrefixedSlice(&b->rep_, xid);
b->content_flags_.store(b->content_flags_.load(std::memory_order_relaxed) |
ContentFlags::HAS_ROLLBACK,
std::memory_order_relaxed);
}

void WriteBatchInternal::Delete(WriteBatch* b, uint32_t column_family_id,
const Slice& key) {
WriteBatchInternal::SetCount(b, WriteBatchInternal::Count(b) + 1);
Expand Down
8 changes: 8 additions & 0 deletions db/write_batch_internal.h
Original file line number Diff line number Diff line change
Expand Up @@ -92,6 +92,14 @@ class WriteBatchInternal {
static void Merge(WriteBatch* batch, uint32_t column_family_id,
const SliceParts& key, const SliceParts& value);

static void MarkEndPrepare(WriteBatch* batch, const Slice& xid);

static void MarkRollback(WriteBatch* batch, const Slice& xid);

static void MarkCommit(WriteBatch* batch, const Slice& xid);

static void InsertNoop(WriteBatch* batch);

// Return the number of entries in the batch.
static int Count(const WriteBatch* batch);

Expand Down
41 changes: 41 additions & 0 deletions db/write_batch_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -231,6 +231,22 @@ namespace {
virtual void LogData(const Slice& blob) override {
seen += "LogData(" + blob.ToString() + ")";
}
virtual Status MarkBeginPrepare() override {
seen += "MarkBeginPrepare()";
return Status::OK();
}
virtual Status MarkEndPrepare(const Slice& xid) override {
seen += "MarkEndPrepare(" + xid.ToString() + ")";
return Status::OK();
}
virtual Status MarkCommit(const Slice& xid) override {
seen += "MarkCommit(" + xid.ToString() + ")";
return Status::OK();
}
virtual Status MarkRollback(const Slice& xid) override {
seen += "MarkRollback(" + xid.ToString() + ")";
return Status::OK();
}
};
}

Expand Down Expand Up @@ -308,6 +324,31 @@ TEST_F(WriteBatchTest, Blob) {
handler.seen);
}

TEST_F(WriteBatchTest, PrepareCommit) {
WriteBatch batch;
WriteBatchInternal::InsertNoop(&batch);
batch.Put(Slice("k1"), Slice("v1"));
batch.Put(Slice("k2"), Slice("v2"));
batch.SetSavePoint();
WriteBatchInternal::MarkEndPrepare(&batch, Slice("xid1"));
Status s = batch.RollbackToSavePoint();
ASSERT_EQ(s, Status::NotFound());
WriteBatchInternal::MarkCommit(&batch, Slice("xid1"));
WriteBatchInternal::MarkRollback(&batch, Slice("xid1"));
ASSERT_EQ(2, batch.Count());

TestHandler handler;
batch.Iterate(&handler);
ASSERT_EQ(
"MarkBeginPrepare()"
"Put(k1, v1)"
"Put(k2, v2)"
"MarkEndPrepare(xid1)"
"MarkCommit(xid1)"
"MarkRollback(xid1)",
handler.seen);
}

// It requires more than 30GB of memory to run the test. With single memory
// allocation of more than 30GB.
// Not all platform can run it. Also it runs a long time. So disable it.
Expand Down
3 changes: 2 additions & 1 deletion include/rocksdb/utilities/write_batch_with_index.h
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,8 @@ enum WriteType {
kMergeRecord,
kDeleteRecord,
kSingleDeleteRecord,
kLogDataRecord
kLogDataRecord,
kXIDRecord,
};

// an entry for Put, Merge, Delete, or SingleDelete entry for write batches.
Expand Down
29 changes: 29 additions & 0 deletions include/rocksdb/write_batch.h
Original file line number Diff line number Diff line change
Expand Up @@ -186,6 +186,23 @@ class WriteBatch : public WriteBatchBase {
// The default implementation of LogData does nothing.
virtual void LogData(const Slice& blob);

virtual Status MarkBeginPrepare() {
return Status::InvalidArgument("MarkBeginPrepare() handler not defined.");
}

virtual Status MarkEndPrepare(const Slice& xid) {
return Status::InvalidArgument("MarkEndPrepare() handler not defined.");
}

virtual Status MarkRollback(const Slice& xid) {
return Status::InvalidArgument(
"MarkRollbackPrepare() handler not defined.");
}

virtual Status MarkCommit(const Slice& xid) {
return Status::InvalidArgument("MarkCommit() handler not defined.");
}

// Continue is called by WriteBatch::Iterate. If it returns false,
// iteration is halted. Otherwise, it continues iterating. The default
// implementation always returns true.
Expand Down Expand Up @@ -214,6 +231,18 @@ class WriteBatch : public WriteBatchBase {
// Returns trie if MergeCF will be called during Iterate
bool HasMerge() const;

// Returns true if MarkBeginPrepare will be called during Iterate
bool HasBeginPrepare() const;

// Returns true if MarkEndPrepare will be called during Iterate
bool HasEndPrepare() const;

// Returns trie if MarkCommit will be called during Iterate
bool HasCommit() const;

// Returns trie if MarkRollback will be called during Iterate
bool HasRollback() const;

using WriteBatchBase::GetWriteBatch;
WriteBatch* GetWriteBatch() override { return this; }

Expand Down
8 changes: 4 additions & 4 deletions tools/ldb_cmd.cc
Original file line number Diff line number Diff line change
Expand Up @@ -1782,24 +1782,24 @@ class InMemoryHandler : public WriteBatch::Handler {
return Status::OK();
}

virtual Status MarkBeginPrepare() {
virtual Status MarkBeginPrepare() override {
row_ << "BEGIN_PREARE ";
return Status::OK();
}

virtual Status MarkEndPrepare(const Slice& xid) {
virtual Status MarkEndPrepare(const Slice& xid) override {
row_ << "END_PREPARE(";
row_ << LDBCommand::StringToHex(xid.ToString()) << ") ";
return Status::OK();
}

virtual Status MarkRollback(const Slice& xid) {
virtual Status MarkRollback(const Slice& xid) override {
row_ << "ROLLBACK(";
row_ << LDBCommand::StringToHex(xid.ToString()) << ") ";
return Status::OK();
}

virtual Status MarkCommit(const Slice& xid) {
virtual Status MarkCommit(const Slice& xid) override {
row_ << "COMMIT(";
row_ << LDBCommand::StringToHex(xid.ToString()) << ") ";
return Status::OK();
Expand Down
Loading

0 comments on commit 0460e9d

Please sign in to comment.