Skip to content

Commit

Permalink
Integrity protection for live updates to WriteBatch (facebook#7748)
Browse files Browse the repository at this point in the history
Summary:
This PR adds the foundation classes for key-value integrity protection and the first use case: protecting live updates from the source buffers added to `WriteBatch` through the destination buffer in `MemTable`. The width of the protection info is not yet configurable -- only eight bytes per key is supported. This PR allows users to enable protection by constructing `WriteBatch` with `protection_bytes_per_key == 8`. It does not yet expose a way for users to get integrity protection via other write APIs (e.g., `Put()`, `Merge()`, `Delete()`, etc.).

The foundation classes (`ProtectionInfo.*`) embed the coverage info in their type, and provide `Protect.*()` and `Strip.*()` functions to navigate between types with different coverage. For making bytes per key configurable (for powers of two up to eight) in the future, these classes are templated on the unsigned integer type used to store the protection info. That integer contains the XOR'd result of hashes with independent seeds for all covered fields. For integer fields, the hash is computed on the raw unadjusted bytes, so the result is endian-dependent. The most significant bytes are truncated when the hash value (8 bytes) is wider than the protection integer.

When `WriteBatch` is constructed with `protection_bytes_per_key == 8`, we hold a `ProtectionInfoKVOTC` (i.e., one that covers key, value, optype aka `ValueType`, timestamp, and CF ID) for each entry added to the batch. The protection info is generated from the original buffers passed by the user, as well as the original metadata generated internally. When writing to memtable, each entry is transformed to a `ProtectionInfoKVOTS` (i.e., dropping coverage of CF ID and adding coverage of sequence number), since at that point we know the sequence number, and have already selected a memtable corresponding to a particular CF. This protection info is verified once the entry is encoded in the `MemTable` buffer.

Pull Request resolved: facebook#7748

Test Plan:
- an integration test to verify a wide variety of single-byte changes to the encoded `MemTable` buffer are caught
- add to stress/crash test to verify it works in variety of configs/operations without intentional corruption
- [deferred] unit tests for `ProtectionInfo.*` classes for edge cases like KV swap, `SliceParts` and `Slice` APIs are interchangeable, etc.

Reviewed By: pdillinger

Differential Revision: D25754492

Pulled By: ajkr

fbshipit-source-id: e481bac6c03c2ab268be41359730f1ceb9964866
  • Loading branch information
ajkr authored and facebook-github-bot committed Jan 29, 2021
1 parent 4a09d63 commit 78ee856
Show file tree
Hide file tree
Showing 28 changed files with 1,239 additions and 111 deletions.
1 change: 1 addition & 0 deletions CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -1088,6 +1088,7 @@ if(WITH_TESTS)
db/db_iter_test.cc
db/db_iter_stress_test.cc
db/db_iterator_test.cc
db/db_kv_checksum_test.cc
db/db_log_iter_test.cc
db/db_memtable_test.cc
db/db_merge_operator_test.cc
Expand Down
4 changes: 4 additions & 0 deletions HISTORY.md
Original file line number Diff line number Diff line change
Expand Up @@ -3,6 +3,9 @@
### Behavior Changes
* When retryable IO error occurs during compaction, it is mapped to soft error and set the BG error. However, auto resume is not called to clean the soft error since compaction will reschedule by itself. In this change, When retryable IO error occurs during compaction, BG error is not set. User will be informed the error via EventHelper.

### New Features
* Add support for key-value integrity protection in live updates from the user buffers provided to `WriteBatch` through the write to RocksDB's in-memory update buffer (memtable). This is intended to detect some cases of in-memory data corruption, due to either software or hardware errors. Users can enable protection by constructing their `WriteBatch` with `protection_bytes_per_key == 8`.

## 6.17.0 (01/15/2021)
### Behavior Changes
* When verifying full file checksum with `DB::VerifyFileChecksums()`, we now fail with `Status::InvalidArgument` if the name of the checksum generator used for verification does not match the name of the checksum generator used for protecting the file when it was created.
Expand All @@ -16,6 +19,7 @@
* Add a public API WriteBufferManager::dummy_entries_in_cache_usage() which reports the size of dummy entries stored in cache (passed to WriteBufferManager). Dummy entries are used to account for DataBlocks.
* Add a SystemClock class that contains the time-related methods from Env. The original methods in Env may be deprecated in a future release. This class will allow easier testing, development, and expansion of time-related features.
* Add a public API GetRocksBuildProperties and GetRocksBuildInfoAsString to get properties about the current build. These properties may include settings related to the GIT settings (branch, timestamp). This change also sets the "build date" based on the GIT properties, rather than the actual build time, thereby enabling more reproducible builds.

## 6.16.0 (12/18/2020)
### Behavior Changes
* Attempting to write a merge operand without explicitly configuring `merge_operator` now fails immediately, causing the DB to enter read-only mode. Previously, failure was deferred until the `merge_operator` was needed by a user read or a background operation.
Expand Down
4 changes: 4 additions & 0 deletions Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -594,6 +594,7 @@ ifdef ASSERT_STATUS_CHECKED
db_inplace_update_test \
db_io_failure_test \
db_iterator_test \
db_kv_checksum_test \
db_logical_block_size_cache_test \
db_memtable_test \
db_merge_operand_test \
Expand Down Expand Up @@ -1608,6 +1609,9 @@ db_inplace_update_test: $(OBJ_DIR)/db/db_inplace_update_test.o $(TEST_LIBRARY) $
db_iterator_test: $(OBJ_DIR)/db/db_iterator_test.o $(TEST_LIBRARY) $(LIBRARY)
$(AM_LINK)

db_kv_checksum_test: $(OBJ_DIR)/db/db_kv_checksum_test.o $(TEST_LIBRARY) $(LIBRARY)
$(AM_LINK)

db_memtable_test: $(OBJ_DIR)/db/db_memtable_test.o $(TEST_LIBRARY) $(LIBRARY)
$(AM_LINK)

Expand Down
7 changes: 7 additions & 0 deletions TARGETS
Original file line number Diff line number Diff line change
Expand Up @@ -1240,6 +1240,13 @@ ROCKS_TESTS = [
[],
[],
],
[
"db_kv_checksum_test",
"db/db_kv_checksum_test.cc",
"serial",
[],
[],
],
[
"db_log_iter_test",
"db/db_log_iter_test.cc",
Expand Down
2 changes: 1 addition & 1 deletion db/blob/blob_file_cache.cc
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ BlobFileCache::BlobFileCache(Cache* cache,
uint32_t column_family_id,
HistogramImpl* blob_file_read_hist)
: cache_(cache),
mutex_(kNumberOfMutexStripes, GetSliceNPHash64),
mutex_(kNumberOfMutexStripes, kGetSliceNPHash64UnseededFnPtr),
immutable_cf_options_(immutable_cf_options),
file_options_(file_options),
column_family_id_(column_family_id),
Expand Down
199 changes: 199 additions & 0 deletions db/db_kv_checksum_test.cc
Original file line number Diff line number Diff line change
@@ -0,0 +1,199 @@
// Copyright (c) 2020-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 "db/db_test_util.h"
#include "rocksdb/rocksdb_namespace.h"

namespace ROCKSDB_NAMESPACE {

enum class WriteBatchOpType {
kPut = 0,
kDelete,
kSingleDelete,
kDeleteRange,
kMerge,
kBlobIndex,
kNum,
};

// Integer addition is needed for `::testing::Range()` to take the enum type.
WriteBatchOpType operator+(WriteBatchOpType lhs, const int rhs) {
using T = std::underlying_type<WriteBatchOpType>::type;
return static_cast<WriteBatchOpType>(static_cast<T>(lhs) + rhs);
}

class DbKvChecksumTest
: public DBTestBase,
public ::testing::WithParamInterface<std::tuple<WriteBatchOpType, char>> {
public:
DbKvChecksumTest()
: DBTestBase("/db_kv_checksum_test", /*env_do_fsync=*/false) {
op_type_ = std::get<0>(GetParam());
corrupt_byte_addend_ = std::get<1>(GetParam());
}

std::pair<WriteBatch, Status> GetWriteBatch(size_t ts_sz,
ColumnFamilyHandle* cf_handle) {
Status s;
WriteBatch wb(0 /* reserved_bytes */, 0 /* max_bytes */, ts_sz,
8 /* protection_bytes_per_entry */);
switch (op_type_) {
case WriteBatchOpType::kPut:
s = wb.Put(cf_handle, "key", "val");
break;
case WriteBatchOpType::kDelete:
s = wb.Delete(cf_handle, "key");
break;
case WriteBatchOpType::kSingleDelete:
s = wb.SingleDelete(cf_handle, "key");
break;
case WriteBatchOpType::kDeleteRange:
s = wb.DeleteRange(cf_handle, "begin", "end");
break;
case WriteBatchOpType::kMerge:
s = wb.Merge(cf_handle, "key", "val");
break;
case WriteBatchOpType::kBlobIndex:
// TODO(ajkr): use public API once available.
uint32_t cf_id;
if (cf_handle == nullptr) {
cf_id = 0;
} else {
cf_id = cf_handle->GetID();
}
s = WriteBatchInternal::PutBlobIndex(&wb, cf_id, "key", "val");
break;
case WriteBatchOpType::kNum:
assert(false);
}
return {std::move(wb), std::move(s)};
}

void CorruptNextByteCallBack(void* arg) {
Slice encoded = *static_cast<Slice*>(arg);
if (entry_len_ == port::kMaxSizet) {
// We learn the entry size on the first attempt
entry_len_ = encoded.size();
}
// All entries should be the same size
assert(entry_len_ == encoded.size());
char* buf = const_cast<char*>(encoded.data());
buf[corrupt_byte_offset_] += corrupt_byte_addend_;
++corrupt_byte_offset_;
}

bool MoreBytesToCorrupt() { return corrupt_byte_offset_ < entry_len_; }

protected:
WriteBatchOpType op_type_;
char corrupt_byte_addend_;
size_t corrupt_byte_offset_ = 0;
size_t entry_len_ = port::kMaxSizet;
};

std::string GetTestNameSuffix(
::testing::TestParamInfo<std::tuple<WriteBatchOpType, char>> info) {
std::ostringstream oss;
switch (std::get<0>(info.param)) {
case WriteBatchOpType::kPut:
oss << "Put";
break;
case WriteBatchOpType::kDelete:
oss << "Delete";
break;
case WriteBatchOpType::kSingleDelete:
oss << "SingleDelete";
break;
case WriteBatchOpType::kDeleteRange:
oss << "DeleteRange";
break;
case WriteBatchOpType::kMerge:
oss << "Merge";
break;
case WriteBatchOpType::kBlobIndex:
oss << "BlobIndex";
break;
case WriteBatchOpType::kNum:
assert(false);
}
oss << "Add"
<< static_cast<int>(static_cast<unsigned char>(std::get<1>(info.param)));
return oss.str();
}

INSTANTIATE_TEST_CASE_P(
DbKvChecksumTest, DbKvChecksumTest,
::testing::Combine(::testing::Range(static_cast<WriteBatchOpType>(0),
WriteBatchOpType::kNum),
::testing::Values(2, 103, 251)),
GetTestNameSuffix);

TEST_P(DbKvChecksumTest, MemTableAddCorrupted) {
// This test repeatedly attempts to write `WriteBatch`es containing a single
// entry of type `op_type_`. Each attempt has one byte corrupted in its
// memtable entry by adding `corrupt_byte_addend_` to its original value. The
// test repeats until an attempt has been made on each byte in the encoded
// memtable entry. All attempts are expected to fail with `Status::Corruption`
SyncPoint::GetInstance()->SetCallBack(
"MemTable::Add:Encoded",
std::bind(&DbKvChecksumTest::CorruptNextByteCallBack, this,
std::placeholders::_1));

while (MoreBytesToCorrupt()) {
// Failed memtable insert always leads to read-only mode, so we have to
// reopen for every attempt.
Options options = CurrentOptions();
if (op_type_ == WriteBatchOpType::kMerge) {
options.merge_operator = MergeOperators::CreateStringAppendOperator();
}
Reopen(options);

SyncPoint::GetInstance()->EnableProcessing();
auto batch_and_status =
GetWriteBatch(0 /* ts_sz */, nullptr /* cf_handle */);
ASSERT_OK(batch_and_status.second);
ASSERT_TRUE(
db_->Write(WriteOptions(), &batch_and_status.first).IsCorruption());
SyncPoint::GetInstance()->DisableProcessing();
}
}

TEST_P(DbKvChecksumTest, MemTableAddWithColumnFamilyCorrupted) {
// This test repeatedly attempts to write `WriteBatch`es containing a single
// entry of type `op_type_` to a non-default column family. Each attempt has
// one byte corrupted in its memtable entry by adding `corrupt_byte_addend_`
// to its original value. The test repeats until an attempt has been made on
// each byte in the encoded memtable entry. All attempts are expected to fail
// with `Status::Corruption`.
Options options = CurrentOptions();
if (op_type_ == WriteBatchOpType::kMerge) {
options.merge_operator = MergeOperators::CreateStringAppendOperator();
}
CreateAndReopenWithCF({"pikachu"}, options);
SyncPoint::GetInstance()->SetCallBack(
"MemTable::Add:Encoded",
std::bind(&DbKvChecksumTest::CorruptNextByteCallBack, this,
std::placeholders::_1));

while (MoreBytesToCorrupt()) {
// Failed memtable insert always leads to read-only mode, so we have to
// reopen for every attempt.
ReopenWithColumnFamilies({kDefaultColumnFamilyName, "pikachu"}, options);

SyncPoint::GetInstance()->EnableProcessing();
auto batch_and_status = GetWriteBatch(0 /* ts_sz */, handles_[1]);
ASSERT_OK(batch_and_status.second);
ASSERT_TRUE(
db_->Write(WriteOptions(), &batch_and_status.first).IsCorruption());
SyncPoint::GetInstance()->DisableProcessing();
}
}

} // namespace ROCKSDB_NAMESPACE

int main(int argc, char** argv) {
::testing::InitGoogleTest(&argc, argv);
return RUN_ALL_TESTS();
}
52 changes: 33 additions & 19 deletions db/db_memtable_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -145,23 +145,34 @@ TEST_F(DBMemTableTest, DuplicateSeq) {
kMaxSequenceNumber, 0 /* column_family_id */);

// Write some keys and make sure it returns false on duplicates
ASSERT_OK(mem->Add(seq, kTypeValue, "key", "value2"));
ASSERT_TRUE(mem->Add(seq, kTypeValue, "key", "value2").IsTryAgain());
ASSERT_OK(
mem->Add(seq, kTypeValue, "key", "value2", nullptr /* kv_prot_info */));
ASSERT_TRUE(
mem->Add(seq, kTypeValue, "key", "value2", nullptr /* kv_prot_info */)
.IsTryAgain());
// Changing the type should still cause the duplicatae key
ASSERT_TRUE(mem->Add(seq, kTypeMerge, "key", "value2").IsTryAgain());
ASSERT_TRUE(
mem->Add(seq, kTypeMerge, "key", "value2", nullptr /* kv_prot_info */)
.IsTryAgain());
// Changing the seq number will make the key fresh
ASSERT_OK(mem->Add(seq + 1, kTypeMerge, "key", "value2"));
ASSERT_OK(mem->Add(seq + 1, kTypeMerge, "key", "value2",
nullptr /* kv_prot_info */));
// Test with different types for duplicate keys
ASSERT_TRUE(mem->Add(seq, kTypeDeletion, "key", "").IsTryAgain());
ASSERT_TRUE(mem->Add(seq, kTypeSingleDeletion, "key", "").IsTryAgain());
ASSERT_TRUE(
mem->Add(seq, kTypeDeletion, "key", "", nullptr /* kv_prot_info */)
.IsTryAgain());
ASSERT_TRUE(
mem->Add(seq, kTypeSingleDeletion, "key", "", nullptr /* kv_prot_info */)
.IsTryAgain());

// Test the duplicate keys under stress
for (int i = 0; i < 10000; i++) {
bool insert_dup = i % 10 == 1;
if (!insert_dup) {
seq++;
}
Status s = mem->Add(seq, kTypeValue, "foo", "value" + ToString(seq));
Status s = mem->Add(seq, kTypeValue, "foo", "value" + ToString(seq),
nullptr /* kv_prot_info */);
if (insert_dup) {
ASSERT_TRUE(s.IsTryAgain());
} else {
Expand All @@ -177,8 +188,11 @@ TEST_F(DBMemTableTest, DuplicateSeq) {
mem = new MemTable(cmp, ioptions, MutableCFOptions(options), &wb,
kMaxSequenceNumber, 0 /* column_family_id */);
// Insert a duplicate key with _ in it
ASSERT_OK(mem->Add(seq, kTypeValue, "key_1", "value"));
ASSERT_TRUE(mem->Add(seq, kTypeValue, "key_1", "value").IsTryAgain());
ASSERT_OK(
mem->Add(seq, kTypeValue, "key_1", "value", nullptr /* kv_prot_info */));
ASSERT_TRUE(
mem->Add(seq, kTypeValue, "key_1", "value", nullptr /* kv_prot_info */)
.IsTryAgain());
delete mem;

// Test when InsertConcurrently will be invoked
Expand All @@ -187,11 +201,11 @@ TEST_F(DBMemTableTest, DuplicateSeq) {
mem = new MemTable(cmp, ioptions, MutableCFOptions(options), &wb,
kMaxSequenceNumber, 0 /* column_family_id */);
MemTablePostProcessInfo post_process_info;
ASSERT_OK(
mem->Add(seq, kTypeValue, "key", "value", true, &post_process_info));
ASSERT_TRUE(
mem->Add(seq, kTypeValue, "key", "value", true, &post_process_info)
.IsTryAgain());
ASSERT_OK(mem->Add(seq, kTypeValue, "key", "value",
nullptr /* kv_prot_info */, true, &post_process_info));
ASSERT_TRUE(mem->Add(seq, kTypeValue, "key", "value",
nullptr /* kv_prot_info */, true, &post_process_info)
.IsTryAgain());
delete mem;
}

Expand All @@ -217,7 +231,7 @@ TEST_F(DBMemTableTest, ConcurrentMergeWrite) {

// Put 0 as the base
PutFixed64(&value, static_cast<uint64_t>(0));
ASSERT_OK(mem->Add(0, kTypeValue, "key", value));
ASSERT_OK(mem->Add(0, kTypeValue, "key", value, nullptr /* kv_prot_info */));
value.clear();

// Write Merge concurrently
Expand All @@ -226,8 +240,8 @@ TEST_F(DBMemTableTest, ConcurrentMergeWrite) {
std::string v1;
for (int seq = 1; seq < num_ops / 2; seq++) {
PutFixed64(&v1, seq);
ASSERT_OK(
mem->Add(seq, kTypeMerge, "key", v1, true, &post_process_info1));
ASSERT_OK(mem->Add(seq, kTypeMerge, "key", v1, nullptr /* kv_prot_info */,
true, &post_process_info1));
v1.clear();
}
});
Expand All @@ -236,8 +250,8 @@ TEST_F(DBMemTableTest, ConcurrentMergeWrite) {
std::string v2;
for (int seq = num_ops / 2; seq < num_ops; seq++) {
PutFixed64(&v2, seq);
ASSERT_OK(
mem->Add(seq, kTypeMerge, "key", v2, true, &post_process_info2));
ASSERT_OK(mem->Add(seq, kTypeMerge, "key", v2, nullptr /* kv_prot_info */,
true, &post_process_info2));
v2.clear();
}
});
Expand Down
6 changes: 4 additions & 2 deletions db/dbformat.h
Original file line number Diff line number Diff line change
Expand Up @@ -146,8 +146,10 @@ inline void UnPackSequenceAndType(uint64_t packed, uint64_t* seq,
*seq = packed >> 8;
*t = static_cast<ValueType>(packed & 0xff);

assert(*seq <= kMaxSequenceNumber);
assert(IsExtendedValueType(*t));
// Commented the following two assertions in order to test key-value checksum
// on corrupted keys without crashing ("DbKvChecksumTest").
// assert(*seq <= kMaxSequenceNumber);
// assert(IsExtendedValueType(*t));
}

EntryType GetEntryType(ValueType value_type);
Expand Down
Loading

0 comments on commit 78ee856

Please sign in to comment.