TableBuilder / TableReader support for range deletion

Summary: 1. Range Deletion Tombstone structure 2. Modify Add() in table_builder to make it usable for adding range del tombstones 3. Expose NewTombstoneIterator() API in table_reader

Test Plan: table_test.cc (now BlockBasedTableBuilder::Add() only accepts InternalKey. I make table_test only pass InternalKey to BlockBasedTableBuidler. Also test writing/reading range deletion tombstones in table_test )

Reviewers: sdong, IslamAbdelRahman, lightmark, andrewkr

Reviewed By: andrewkr

Subscribers: andrewkr, dhruba, leveldb

Differential Revision: https://reviews.facebook.net/D61473
main
Wanning Jiang 8 years ago
parent 4cc37f59e5
commit 78837f5d61
  1. 1
      db/db_impl.h
  2. 35
      db/dbformat.h
  3. 1
      include/rocksdb/table_properties.h
  4. 108
      table/block_based_table_builder.cc
  5. 49
      table/block_based_table_reader.cc
  6. 3
      table/block_based_table_reader.h
  7. 2
      table/block_builder.h
  8. 4
      table/plain_table_builder.cc
  9. 30
      table/table_properties.cc
  10. 5
      table/table_properties_internal.h
  11. 6
      table/table_reader.h
  12. 209
      table/table_test.cc
  13. 1
      util/coding.h
  14. 6
      util/env_basic_test.cc
  15. 2
      utilities/transactions/transaction_test.cc

@ -252,6 +252,7 @@ class DBImpl : public DB {
//
// Returns OK or NotFound on success,
// other status on unexpected error.
// TODO(andrewkr): this API need to be aware of range deletion operations
Status GetLatestSequenceForKey(SuperVersion* sv, const Slice& key,
bool cache_only, SequenceNumber* seq,
bool* found_record_for_key);

@ -67,7 +67,7 @@ inline bool IsValueType(ValueType t) {
// Checks whether a type is from user operation
// kTypeRangeDeletion is in meta block so this API is separated from above
inline bool IsExtendedValueType(ValueType t) {
return t <= kTypeMerge || t == kTypeSingleDeletion || t == kTypeRangeDeletion;
return IsValueType(t) || t == kTypeRangeDeletion;
}
// We leave eight bits empty at the bottom so a type and sequence#
@ -497,4 +497,37 @@ extern bool ReadKeyFromWriteBatchEntry(Slice* input, Slice* key,
extern Status ReadRecordFromWriteBatch(Slice* input, char* tag,
uint32_t* column_family, Slice* key,
Slice* value, Slice* blob, Slice* xid);
// When user call DeleteRange() to delete a range of keys,
// we will store a serialized RangeTombstone in MemTable and SST.
// the struct here is a easy-understood form
// start/end_key_ is the start/end user key of the range to be deleted
struct RangeTombstone {
Slice start_key_;
Slice end_key_;
SequenceNumber seq_;
explicit RangeTombstone(Slice sk, Slice ek, SequenceNumber sn)
: start_key_(sk), end_key_(ek), seq_(sn) {}
explicit RangeTombstone(Slice internal_key, Slice value) {
ParsedInternalKey parsed_key;
if (ParseInternalKey(internal_key, &parsed_key)) {
start_key_ = parsed_key.user_key;
seq_ = parsed_key.sequence;
end_key_ = value;
}
}
// be careful to use Serialize(); InternalKey() allocates new memory
std::pair<InternalKey, Slice> Serialize() {
auto key = InternalKey(start_key_, seq_, kTypeRangeDeletion);
Slice value = end_key_;
return std::make_pair(std::move(key), std::move(value));
}
InternalKey SerializeKey() {
return InternalKey(start_key_, seq_, kTypeRangeDeletion);
}
};
} // namespace rocksdb

@ -49,6 +49,7 @@ struct TablePropertiesNames {
extern const std::string kPropertiesBlock;
extern const std::string kCompressionDictBlock;
extern const std::string kRangeDelBlock;
enum EntryType {
kEntryPut,

@ -463,6 +463,7 @@ struct BlockBasedTableBuilder::Rep {
uint64_t offset = 0;
Status status;
BlockBuilder data_block;
BlockBuilder range_del_block;
InternalKeySliceTransform internal_prefix_transform;
std::unique_ptr<IndexBuilder> index_builder;
@ -504,6 +505,7 @@ struct BlockBasedTableBuilder::Rep {
file(f),
data_block(table_options.block_restart_interval,
table_options.use_delta_encoding),
range_del_block(1), // TODO(andrewkr): restart_interval unnecessary
internal_prefix_transform(_ioptions.prefix_extractor),
index_builder(
CreateIndexBuilder(table_options.index_type, &internal_comparator,
@ -577,42 +579,52 @@ void BlockBasedTableBuilder::Add(const Slice& key, const Slice& value) {
Rep* r = rep_;
assert(!r->closed);
if (!ok()) return;
if (r->props.num_entries > 0) {
assert(r->internal_comparator.Compare(key, Slice(r->last_key)) > 0);
}
auto should_flush = r->flush_block_policy->Update(key, value);
if (should_flush) {
assert(!r->data_block.empty());
Flush();
// Add item to index block.
// We do not emit the index entry for a block until we have seen the
// first key for the next data block. This allows us to use shorter
// keys in the index block. For example, consider a block boundary
// between the keys "the quick brown fox" and "the who". We can use
// "the r" as the key for the index block entry since it is >= all
// entries in the first block and < all entries in subsequent
// blocks.
if (ok()) {
r->index_builder->AddIndexEntry(&r->last_key, &key, r->pending_handle);
ValueType value_type = ExtractValueType(key);
if (IsValueType(value_type)) {
if (r->props.num_entries > 0) {
assert(r->internal_comparator.Compare(key, Slice(r->last_key)) > 0);
}
}
if (r->filter_block != nullptr) {
r->filter_block->Add(ExtractUserKey(key));
}
auto should_flush = r->flush_block_policy->Update(key, value);
if (should_flush) {
assert(!r->data_block.empty());
Flush();
// Add item to index block.
// We do not emit the index entry for a block until we have seen the
// first key for the next data block. This allows us to use shorter
// keys in the index block. For example, consider a block boundary
// between the keys "the quick brown fox" and "the who". We can use
// "the r" as the key for the index block entry since it is >= all
// entries in the first block and < all entries in subsequent
// blocks.
if (ok()) {
r->index_builder->AddIndexEntry(&r->last_key, &key, r->pending_handle);
}
}
r->last_key.assign(key.data(), key.size());
r->data_block.Add(key, value);
r->props.num_entries++;
r->props.raw_key_size += key.size();
r->props.raw_value_size += value.size();
if (r->filter_block != nullptr) {
r->filter_block->Add(ExtractUserKey(key));
}
r->index_builder->OnKeyAdded(key);
NotifyCollectTableCollectorsOnAdd(key, value, r->offset,
r->table_properties_collectors,
r->ioptions.info_log);
r->last_key.assign(key.data(), key.size());
r->data_block.Add(key, value);
r->props.num_entries++;
r->props.raw_key_size += key.size();
r->props.raw_value_size += value.size();
r->index_builder->OnKeyAdded(key);
NotifyCollectTableCollectorsOnAdd(key, value, r->offset,
r->table_properties_collectors,
r->ioptions.info_log);
} else if (value_type == kTypeRangeDeletion) {
// TODO(wanning&andrewkr) add num_tomestone to table properties
r->range_del_block.Add(key, value);
++r->props.num_entries;
} else {
assert(false);
}
}
void BlockBasedTableBuilder::Flush() {
@ -651,7 +663,7 @@ void BlockBasedTableBuilder::WriteBlock(const Slice& raw_block_contents,
auto type = r->compression_type;
Slice block_contents;
bool abort_compression = false;
StopWatchNano timer(r->ioptions.env,
ShouldReportDetailedTime(r->ioptions.env, r->ioptions.statistics));
@ -703,14 +715,13 @@ void BlockBasedTableBuilder::WriteBlock(const Slice& raw_block_contents,
RecordTick(r->ioptions.statistics, NUMBER_BLOCK_NOT_COMPRESSED);
type = kNoCompression;
block_contents = raw_block_contents;
}
else if (type != kNoCompression &&
ShouldReportDetailedTime(r->ioptions.env,
r->ioptions.statistics)) {
MeasureTime(r->ioptions.statistics, COMPRESSION_TIMES_NANOS,
timer.ElapsedNanos());
MeasureTime(r->ioptions.statistics, BYTES_COMPRESSED,
raw_block_contents.size());
} else if (type != kNoCompression &&
ShouldReportDetailedTime(r->ioptions.env,
r->ioptions.statistics)) {
MeasureTime(r->ioptions.statistics, COMPRESSION_TIMES_NANOS,
timer.ElapsedNanos());
MeasureTime(r->ioptions.statistics, BYTES_COMPRESSED,
raw_block_contents.size());
RecordTick(r->ioptions.statistics, NUMBER_BLOCK_COMPRESSED);
}
@ -817,7 +828,7 @@ Status BlockBasedTableBuilder::Finish() {
r->closed = true;
BlockHandle filter_block_handle, metaindex_block_handle, index_block_handle,
compression_dict_block_handle;
compression_dict_block_handle, range_del_block_handle;
// Write filter block
if (ok() && r->filter_block != nullptr) {
auto filter_contents = r->filter_block->Finish();
@ -841,9 +852,10 @@ Status BlockBasedTableBuilder::Finish() {
// Write meta blocks and metaindex block with the following order.
// 1. [meta block: filter]
// 2. [other meta blocks]
// 3. [meta block: properties]
// 4. [metaindex block]
// 2. [meta block: properties]
// 3. [meta block: compression dictionary]
// 4. [meta block: range deletion tombstone]
// 5. [metaindex block]
// write meta blocks
MetaIndexBuilder meta_index_builder;
for (const auto& item : index_blocks.meta_blocks) {
@ -920,6 +932,12 @@ Status BlockBasedTableBuilder::Finish() {
compression_dict_block_handle);
}
} // end of properties/compression dictionary block writing
if (ok() && !r->range_del_block.empty()) {
WriteRawBlock(r->range_del_block.Finish(), kNoCompression,
&range_del_block_handle);
meta_index_builder.Add(kRangeDelBlock, range_del_block_handle);
} // range deletion tombstone meta block
} // meta blocks
// Write index block

@ -361,7 +361,8 @@ struct BlockBasedTable::Rep {
internal_comparator(_internal_comparator),
filter_type(FilterType::kNoFilter),
whole_key_filtering(_table_opt.whole_key_filtering),
prefix_filtering(true) {}
prefix_filtering(true),
range_del_block(nullptr) {}
const ImmutableCFOptions& ioptions;
const EnvOptions& env_options;
@ -419,6 +420,7 @@ struct BlockBasedTable::Rep {
// the LRU cache will never push flush them out, hence they're pinned
CachableEntry<FilterBlockReader> filter_entry;
CachableEntry<IndexReader> index_entry;
unique_ptr<Block> range_del_block;
};
BlockBasedTable::~BlockBasedTable() {
@ -581,7 +583,7 @@ Status BlockBasedTable::Open(const ImmutableCFOptions& ioptions,
if (!s.ok()) {
Log(InfoLogLevel::WARN_LEVEL, rep->ioptions.info_log,
"Cannot seek to properties block from file: %s",
"Error when seeking to properties block from file: %s",
s.ToString().c_str());
} else if (found_properties_block) {
s = meta_iter->status();
@ -608,12 +610,15 @@ Status BlockBasedTable::Open(const ImmutableCFOptions& ioptions,
s = SeekToCompressionDictBlock(meta_iter.get(), &found_compression_dict);
if (!s.ok()) {
Log(InfoLogLevel::WARN_LEVEL, rep->ioptions.info_log,
"Cannot seek to compression dictionary block from file: %s",
"Error when seeking to compression dictionary block from file: %s",
s.ToString().c_str());
} else if (found_compression_dict) {
// TODO(andrewkr): Add to block cache if cache_index_and_filter_blocks is
// true.
unique_ptr<BlockContents> compression_dict_block{new BlockContents()};
// TODO(andrewkr): ReadMetaBlock repeats SeekToCompressionDictBlock().
// maybe decode a handle from meta_iter
// and do ReadBlockContents(handle) instead
s = rocksdb::ReadMetaBlock(rep->file.get(), file_size,
kBlockBasedTableMagicNumber, rep->ioptions,
rocksdb::kCompressionDictBlock,
@ -628,6 +633,34 @@ Status BlockBasedTable::Open(const ImmutableCFOptions& ioptions,
}
}
// Read the range del meta block
// TODO(wanning&andrewkr): cache range delete tombstone block
bool found_range_del_block;
BlockHandle range_del_handle;
s = SeekToRangeDelBlock(meta_iter.get(), &found_range_del_block,
&range_del_handle);
if (!s.ok()) {
Log(InfoLogLevel::WARN_LEVEL, rep->ioptions.info_log,
"Error when seeking to range delete tombstones block from file: %s",
s.ToString().c_str());
} else {
if (found_range_del_block && !range_del_handle.IsNull()) {
BlockContents range_del_block_contents;
ReadOptions read_options;
s = ReadBlockContents(rep->file.get(), rep->footer, read_options,
range_del_handle, &range_del_block_contents,
rep->ioptions, false /* decompressed */);
if (!s.ok()) {
Log(InfoLogLevel::WARN_LEVEL, rep->ioptions.info_log,
"Encountered error while reading data from range del block %s",
s.ToString().c_str());
} else {
rep->range_del_block.reset(
new Block(std::move(range_del_block_contents)));
}
}
}
// Determine whether whole key filtering is supported.
if (rep->table_properties) {
rep->whole_key_filtering &=
@ -1350,6 +1383,16 @@ InternalIterator* BlockBasedTable::NewIterator(const ReadOptions& read_options,
NewIndexIterator(read_options), arena);
}
InternalIterator* BlockBasedTable::NewRangeTombstoneIterator(
const ReadOptions& read_options) {
if (rep_->range_del_block.get() != nullptr) {
auto iter =
rep_->range_del_block->NewIterator(&(rep_->internal_comparator));
return iter;
}
return NewEmptyInternalIterator();
}
bool BlockBasedTable::FullFilterKeyMayMatch(const ReadOptions& read_options,
FilterBlockReader* filter,
const Slice& internal_key) const {

@ -96,6 +96,9 @@ class BlockBasedTable : public TableReader {
InternalIterator* NewIterator(const ReadOptions&, Arena* arena = nullptr,
bool skip_filters = false) override;
InternalIterator* NewRangeTombstoneIterator(
const ReadOptions& read_options) override;
// @param skip_filters Disables loading/accessing the filter block
Status Get(const ReadOptions& readOptions, const Slice& key,
GetContext* get_context, bool skip_filters = false) override;

@ -26,7 +26,7 @@ class BlockBuilder {
// Reset the contents as if the BlockBuilder was just constructed.
void Reset();
// REQUIRES: Finish() has not been callled since the last call to Reset().
// REQUIRES: Finish() has not been called since the last call to Reset().
// REQUIRES: key is larger than any previously added key
void Add(const Slice& key, const Slice& value);

@ -125,6 +125,10 @@ void PlainTableBuilder::Add(const Slice& key, const Slice& value) {
ParsedInternalKey internal_key;
ParseInternalKey(key, &internal_key);
if (internal_key.type == kTypeRangeDeletion) {
status_ = Status::NotSupported("Range deletion unsupported");
return;
}
// Store key hash
if (store_index_in_file_) {

@ -3,12 +3,13 @@
// LICENSE file in the root directory of this source tree. An additional grant
// of patent rights can be found in the PATENTS file in the same directory.
#include "table/table_properties_internal.h"
#include "rocksdb/table_properties.h"
#include "rocksdb/iterator.h"
#include "rocksdb/env.h"
#include "port/port.h"
#include "rocksdb/env.h"
#include "rocksdb/iterator.h"
#include "table/block.h"
#include "table/internal_iterator.h"
#include "table/table_properties_internal.h"
#include "util/string_util.h"
namespace rocksdb {
@ -44,12 +45,21 @@ namespace {
// Seek to the specified meta block.
// Return true if it successfully seeks to that block.
Status SeekToMetaBlock(InternalIterator* meta_iter,
const std::string& block_name, bool* is_found) {
const std::string& block_name, bool* is_found,
BlockHandle* block_handle = nullptr) {
*is_found = true;
meta_iter->Seek(block_name);
if (meta_iter->status().ok() &&
(!meta_iter->Valid() || meta_iter->key() != block_name)) {
*is_found = false;
if (meta_iter->status().ok()) {
if (meta_iter->Valid() && meta_iter->key() == block_name) {
*is_found = true;
if (block_handle) {
Slice v = meta_iter->value();
return block_handle->DecodeFrom(&v);
}
} else {
*is_found = false;
return Status::OK();
}
}
return meta_iter->status();
}
@ -166,6 +176,7 @@ extern const std::string kPropertiesBlock = "rocksdb.properties";
// Old property block name for backward compatibility
extern const std::string kPropertiesBlockOldName = "rocksdb.stats";
extern const std::string kCompressionDictBlock = "rocksdb.compression_dict";
extern const std::string kRangeDelBlock = "rocksdb.range_del";
// Seek to the properties block.
// Return true if it successfully seeks to the properties block.
@ -183,4 +194,9 @@ Status SeekToCompressionDictBlock(InternalIterator* meta_iter, bool* is_found) {
return SeekToMetaBlock(meta_iter, kCompressionDictBlock, is_found);
}
Status SeekToRangeDelBlock(InternalIterator* meta_iter, bool* is_found,
BlockHandle* block_handle = nullptr) {
return SeekToMetaBlock(meta_iter, kRangeDelBlock, is_found, block_handle);
}
} // namespace rocksdb

@ -11,6 +11,7 @@
namespace rocksdb {
class InternalIterator;
class BlockHandle;
// Seek to the properties block.
// If it successfully seeks to the properties block, "is_found" will be
@ -22,4 +23,8 @@ Status SeekToPropertiesBlock(InternalIterator* meta_iter, bool* is_found);
// set to true.
Status SeekToCompressionDictBlock(InternalIterator* meta_iter, bool* is_found);
// TODO(andrewkr) should not put all meta block in table_properties.h/cc
Status SeekToRangeDelBlock(InternalIterator* meta_iter, bool* is_found,
BlockHandle* block_handle);
} // namespace rocksdb

@ -9,6 +9,7 @@
#pragma once
#include <memory>
#include "table/internal_iterator.h"
namespace rocksdb {
@ -41,6 +42,11 @@ class TableReader {
Arena* arena = nullptr,
bool skip_filters = false) = 0;
virtual InternalIterator* NewRangeTombstoneIterator(
const ReadOptions& read_options) {
return NewEmptyInternalIterator();
}
// Given a key, return an approximate byte offset in the file where
// the data for that key begins (or would begin if the key were
// present in the file). The returned value is in terms of file

@ -352,6 +352,11 @@ class TableConstructor: public Constructor {
}
uint64_t ApproximateOffsetOf(const Slice& key) const {
if (convert_to_internal_key_) {
InternalKey ikey(key, kMaxSequenceNumber, kTypeValue);
const Slice skey = ikey.Encode();
return table_reader_->ApproximateOffsetOf(skey);
}
return table_reader_->ApproximateOffsetOf(key);
}
@ -373,6 +378,8 @@ class TableConstructor: public Constructor {
void ResetTableReader() { table_reader_.reset(); }
bool ConvertToInternalKey() { return convert_to_internal_key_; }
private:
void Reset() {
uniq_id_ = 0;
@ -685,7 +692,10 @@ class HarnessTest : public testing::Test {
table_options_.format_version = args.format_version;
options_.table_factory.reset(
new BlockBasedTableFactory(table_options_));
constructor_ = new TableConstructor(options_.comparator);
constructor_ = new TableConstructor(
options_.comparator, true /* convert_to_internal_key_ */);
internal_comparator_.reset(
new InternalKeyComparator(options_.comparator));
break;
// Plain table is not supported in ROCKSDB_LITE
#ifndef ROCKSDB_LITE
@ -694,7 +704,8 @@ class HarnessTest : public testing::Test {
only_support_prefix_seek_ = true;
options_.prefix_extractor.reset(new FixedOrLessPrefixTransform(2));
options_.table_factory.reset(NewPlainTableFactory());
constructor_ = new TableConstructor(options_.comparator, true);
constructor_ = new TableConstructor(
options_.comparator, true /* convert_to_internal_key_ */);
internal_comparator_.reset(
new InternalKeyComparator(options_.comparator));
break;
@ -703,7 +714,8 @@ class HarnessTest : public testing::Test {
only_support_prefix_seek_ = true;
options_.prefix_extractor.reset(NewNoopTransform());
options_.table_factory.reset(NewPlainTableFactory());
constructor_ = new TableConstructor(options_.comparator, true);
constructor_ = new TableConstructor(
options_.comparator, true /* convert_to_internal_key_ */);
internal_comparator_.reset(
new InternalKeyComparator(options_.comparator));
break;
@ -721,7 +733,8 @@ class HarnessTest : public testing::Test {
options_.table_factory.reset(
NewPlainTableFactory(plain_table_options));
}
constructor_ = new TableConstructor(options_.comparator, true);
constructor_ = new TableConstructor(
options_.comparator, true /* convert_to_internal_key_ */);
internal_comparator_.reset(
new InternalKeyComparator(options_.comparator));
break;
@ -1019,7 +1032,7 @@ TEST_F(TablePropertyTest, PrefixScanTest) {
// This test include all the basic checks except those for index size and block
// size, which will be conducted in separated unit tests.
TEST_F(BlockBasedTableTest, BasicBlockBasedTableProperties) {
TableConstructor c(BytewiseComparator());
TableConstructor c(BytewiseComparator(), true /* convert_to_internal_key_ */);
c.Add("a1", "val1");
c.Add("b2", "val2");
@ -1030,6 +1043,7 @@ TEST_F(BlockBasedTableTest, BasicBlockBasedTableProperties) {
c.Add("g7", "val7");
c.Add("h8", "val8");
c.Add("j9", "val9");
uint64_t diff_internal_user_bytes = 9 * 8; // 8 is seq size, 9 k-v totally
std::vector<std::string> keys;
stl_wrappers::KVMap kvmap;
@ -1049,7 +1063,7 @@ TEST_F(BlockBasedTableTest, BasicBlockBasedTableProperties) {
auto raw_key_size = kvmap.size() * 2ul;
auto raw_value_size = kvmap.size() * 4ul;
ASSERT_EQ(raw_key_size, props.raw_key_size);
ASSERT_EQ(raw_key_size + diff_internal_user_bytes, props.raw_key_size);
ASSERT_EQ(raw_value_size, props.raw_value_size);
ASSERT_EQ(1ul, props.num_data_blocks);
ASSERT_EQ("", props.filter_policy_name); // no filter policy is used
@ -1060,7 +1074,8 @@ TEST_F(BlockBasedTableTest, BasicBlockBasedTableProperties) {
block_builder.Add(item.first, item.second);
}
Slice content = block_builder.Finish();
ASSERT_EQ(content.size() + kBlockTrailerSize, props.data_size);
ASSERT_EQ(content.size() + kBlockTrailerSize + diff_internal_user_bytes,
props.data_size);
c.ResetTableReader();
}
@ -1120,8 +1135,50 @@ TEST_F(BlockBasedTableTest, BlockBasedTableProperties2) {
}
}
TEST_F(BlockBasedTableTest, RangeDelBlock) {
TableConstructor c(BytewiseComparator());
std::vector<std::string> keys = {"1pika", "2chu"};
std::vector<std::string> vals = {"p", "c"};
for (int i = 0; i < 2; i++) {
RangeTombstone t(keys[i], vals[i], i);
std::pair<InternalKey, Slice> p = t.Serialize();
c.Add(p.first.Encode().ToString(), p.second);
}
std::vector<std::string> sorted_keys;
stl_wrappers::KVMap kvmap;
Options options;
options.compression = kNoCompression;
BlockBasedTableOptions table_options;
table_options.block_restart_interval = 1;
options.table_factory.reset(NewBlockBasedTableFactory(table_options));
const ImmutableCFOptions ioptions(options);
std::unique_ptr<InternalKeyComparator> internal_cmp(
new InternalKeyComparator(options.comparator));
c.Finish(options, ioptions, table_options, *internal_cmp, &sorted_keys,
&kvmap);
std::unique_ptr<InternalIterator> iter(
c.GetTableReader()->NewRangeTombstoneIterator(ReadOptions()));
ASSERT_EQ(false, iter->Valid());
iter->SeekToFirst();
ASSERT_EQ(true, iter->Valid());
for (int i = 0; i < 2; i++) {
ASSERT_TRUE(iter->Valid());
RangeTombstone t(iter->key(), iter->value());
ASSERT_EQ(t.start_key_, keys[i]);
ASSERT_EQ(t.end_key_, vals[i]);
ASSERT_EQ(t.seq_, i);
iter->Next();
}
ASSERT_TRUE(!iter->Valid());
c.ResetTableReader();
}
TEST_F(BlockBasedTableTest, FilterPolicyNameProperties) {
TableConstructor c(BytewiseComparator(), true);
TableConstructor c(BytewiseComparator(), true /* convert_to_internal_key_ */);
c.Add("a1", "val1");
std::vector<std::string> keys;
stl_wrappers::KVMap kvmap;
@ -1143,19 +1200,29 @@ TEST_F(BlockBasedTableTest, FilterPolicyNameProperties) {
//
void AssertKeysInCache(BlockBasedTable* table_reader,
const std::vector<std::string>& keys_in_cache,
const std::vector<std::string>& keys_not_in_cache) {
for (auto key : keys_in_cache) {
ASSERT_TRUE(table_reader->TEST_KeyInCache(ReadOptions(), key));
}
for (auto key : keys_not_in_cache) {
ASSERT_TRUE(!table_reader->TEST_KeyInCache(ReadOptions(), key));
const std::vector<std::string>& keys_not_in_cache,
bool convert = false) {
if (convert) {
for (auto key : keys_in_cache) {
InternalKey ikey(key, kMaxSequenceNumber, kTypeValue);
ASSERT_TRUE(table_reader->TEST_KeyInCache(ReadOptions(), ikey.Encode()));
}
for (auto key : keys_not_in_cache) {
InternalKey ikey(key, kMaxSequenceNumber, kTypeValue);
ASSERT_TRUE(!table_reader->TEST_KeyInCache(ReadOptions(), ikey.Encode()));
}
} else {
for (auto key : keys_in_cache) {
ASSERT_TRUE(table_reader->TEST_KeyInCache(ReadOptions(), key));
}
for (auto key : keys_not_in_cache) {
ASSERT_TRUE(!table_reader->TEST_KeyInCache(ReadOptions(), key));
}
}
}
void PrefetchRange(TableConstructor* c, Options* opt,
BlockBasedTableOptions* table_options,
const std::vector<std::string>& keys, const char* key_begin,
BlockBasedTableOptions* table_options, const char* key_begin,
const char* key_end,
const std::vector<std::string>& keys_in_cache,
const std::vector<std::string>& keys_not_in_cache,
@ -1168,15 +1235,32 @@ void PrefetchRange(TableConstructor* c, Options* opt,
// prefetch
auto* table_reader = dynamic_cast<BlockBasedTable*>(c->GetTableReader());
// empty string replacement is a trick so we don't crash the test
Slice begin(key_begin ? key_begin : "");
Slice end(key_end ? key_end : "");
Status s = table_reader->Prefetch(key_begin ? &begin : nullptr,
key_end ? &end : nullptr);
Status s;
unique_ptr<Slice> begin, end;
unique_ptr<InternalKey> i_begin, i_end;
if (key_begin != nullptr) {
if (c->ConvertToInternalKey()) {
i_begin.reset(new InternalKey(key_begin, kMaxSequenceNumber, kTypeValue));
begin.reset(new Slice(i_begin->Encode()));
} else {
begin.reset(new Slice(key_begin));
}
}
if (key_end != nullptr) {
if (c->ConvertToInternalKey()) {
i_end.reset(new InternalKey(key_end, kMaxSequenceNumber, kTypeValue));
end.reset(new Slice(i_end->Encode()));
} else {
end.reset(new Slice(key_end));
}
}
s = table_reader->Prefetch(begin.get(), end.get());
ASSERT_TRUE(s.code() == expected_status.code());
// assert our expectation in cache warmup
AssertKeysInCache(table_reader, keys_in_cache, keys_not_in_cache);
AssertKeysInCache(table_reader, keys_in_cache, keys_not_in_cache,
c->ConvertToInternalKey());
c->ResetTableReader();
}
@ -1193,7 +1277,7 @@ TEST_F(BlockBasedTableTest, PrefetchTest) {
table_options.block_cache = NewLRUCache(16 * 1024 * 1024, 4);
opt.table_factory.reset(NewBlockBasedTableFactory(table_options));
TableConstructor c(BytewiseComparator());
TableConstructor c(BytewiseComparator(), true /* convert_to_internal_key_ */);
c.Add("k01", "hello");
c.Add("k02", "hello2");
c.Add("k03", std::string(10000, 'x'));
@ -1218,48 +1302,31 @@ TEST_F(BlockBasedTableTest, PrefetchTest) {
// Simple
PrefetchRange(&c, &opt, &table_options, keys,
/*key_range=*/ "k01", "k05",
/*keys_in_cache=*/ {"k01", "k02", "k03", "k04", "k05"},
/*keys_not_in_cache=*/ {"k06", "k07"});
PrefetchRange(&c, &opt, &table_options, keys,
"k01", "k01",
{"k01", "k02", "k03"},
PrefetchRange(&c, &opt, &table_options,
/*key_range=*/"k01", "k05",
/*keys_in_cache=*/{"k01", "k02", "k03", "k04", "k05"},
/*keys_not_in_cache=*/{"k06", "k07"});
PrefetchRange(&c, &opt, &table_options, "k01", "k01", {"k01", "k02", "k03"},
{"k04", "k05", "k06", "k07"});
// odd
PrefetchRange(&c, &opt, &table_options, keys,
"a", "z",
{"k01", "k02", "k03", "k04", "k05", "k06", "k07"},
{});
PrefetchRange(&c, &opt, &table_options, keys,
"k00", "k00",
{"k01", "k02", "k03"},
PrefetchRange(&c, &opt, &table_options, "a", "z",
{"k01", "k02", "k03", "k04", "k05", "k06", "k07"}, {});
PrefetchRange(&c, &opt, &table_options, "k00", "k00", {"k01", "k02", "k03"},
{"k04", "k05", "k06", "k07"});
// Edge cases
PrefetchRange(&c, &opt, &table_options, keys,
"k00", "k06",
{"k01", "k02", "k03", "k04", "k05", "k06", "k07"},
{});
PrefetchRange(&c, &opt, &table_options, keys,
"k00", "zzz",
{"k01", "k02", "k03", "k04", "k05", "k06", "k07"},
{});
PrefetchRange(&c, &opt, &table_options, "k00", "k06",
{"k01", "k02", "k03", "k04", "k05", "k06", "k07"}, {});
PrefetchRange(&c, &opt, &table_options, "k00", "zzz",
{"k01", "k02", "k03", "k04", "k05", "k06", "k07"}, {});
// null keys
PrefetchRange(&c, &opt, &table_options, keys,
nullptr, nullptr,
{"k01", "k02", "k03", "k04", "k05", "k06", "k07"},
{});
PrefetchRange(&c, &opt, &table_options, keys,
"k04", nullptr,
{"k04", "k05", "k06", "k07"},
{"k01", "k02", "k03"});
PrefetchRange(&c, &opt, &table_options, keys,
nullptr, "k05",
{"k01", "k02", "k03", "k04", "k05"},
{"k06", "k07"});
PrefetchRange(&c, &opt, &table_options, nullptr, nullptr,
{"k01", "k02", "k03", "k04", "k05", "k06", "k07"}, {});
PrefetchRange(&c, &opt, &table_options, "k04", nullptr,
{"k04", "k05", "k06", "k07"}, {"k01", "k02", "k03"});
PrefetchRange(&c, &opt, &table_options, nullptr, "k05",
{"k01", "k02", "k03", "k04", "k05"}, {"k06", "k07"});
// invalid
PrefetchRange(&c, &opt, &table_options, keys,
"k06", "k00", {}, {},
PrefetchRange(&c, &opt, &table_options, "k06", "k00", {}, {},
Status::InvalidArgument(Slice("k06 "), Slice("k07")));
c.ResetTableReader();
}
@ -1299,7 +1366,8 @@ TEST_F(BlockBasedTableTest, TotalOrderSeekOnHashIndex) {
break;
}
TableConstructor c(BytewiseComparator(), true);
TableConstructor c(BytewiseComparator(),
true /* convert_to_internal_key_ */);
c.Add("aaaa1", std::string('a', 56));
c.Add("bbaa1", std::string('a', 56));
c.Add("cccc1", std::string('a', 56));
@ -1527,7 +1595,8 @@ TEST_F(BlockBasedTableTest, IndexSizeStat) {
// Each time we load one more key to the table. the table index block
// size is expected to be larger than last time's.
for (size_t i = 1; i < keys.size(); ++i) {
TableConstructor c(BytewiseComparator());
TableConstructor c(BytewiseComparator(),
true /* convert_to_internal_key_ */);
for (size_t j = 0; j < i; ++j) {
c.Add(keys[j], "val");
}
@ -1552,7 +1621,7 @@ TEST_F(BlockBasedTableTest, IndexSizeStat) {
TEST_F(BlockBasedTableTest, NumBlockStat) {
Random rnd(test::RandomSeed());
TableConstructor c(BytewiseComparator());
TableConstructor c(BytewiseComparator(), true /* convert_to_internal_key_ */);
Options options;
options.compression = kNoCompression;
BlockBasedTableOptions table_options;
@ -1652,7 +1721,7 @@ TEST_F(BlockBasedTableTest, BlockCacheDisabledTest) {
std::vector<std::string> keys;
stl_wrappers::KVMap kvmap;
TableConstructor c(BytewiseComparator(), true);
TableConstructor c(BytewiseComparator(), true /* convert_to_internal_key_ */);
c.Add("key", "value");
const ImmutableCFOptions ioptions(options);
c.Finish(options, ioptions, table_options,
@ -1698,7 +1767,7 @@ TEST_F(BlockBasedTableTest, FilterBlockInBlockCache) {
std::vector<std::string> keys;
stl_wrappers::KVMap kvmap;
TableConstructor c(BytewiseComparator());
TableConstructor c(BytewiseComparator(), true /* convert_to_internal_key_ */);
c.Add("key", "value");
const ImmutableCFOptions ioptions(options);
c.Finish(options, ioptions, table_options,
@ -1979,7 +2048,7 @@ TEST_F(BlockBasedTableTest, BlockCacheLeak) {
table_options.block_cache = NewLRUCache(16 * 1024 * 1024, 4);
opt.table_factory.reset(NewBlockBasedTableFactory(table_options));
TableConstructor c(BytewiseComparator());
TableConstructor c(BytewiseComparator(), true /* convert_to_internal_key_ */);
c.Add("k01", "hello");
c.Add("k02", "hello2");
c.Add("k03", std::string(10000, 'x'));
@ -2079,7 +2148,7 @@ TEST_F(PlainTableTest, BasicPlainTableProperties) {
#endif // !ROCKSDB_LITE
TEST_F(GeneralTableTest, ApproximateOffsetOfPlain) {
TableConstructor c(BytewiseComparator());
TableConstructor c(BytewiseComparator(), true /* convert_to_internal_key_ */);
c.Add("k01", "hello");
c.Add("k02", "hello2");
c.Add("k03", std::string(10000, 'x'));
@ -2104,7 +2173,9 @@ TEST_F(GeneralTableTest, ApproximateOffsetOfPlain) {
ASSERT_TRUE(Between(c.ApproximateOffsetOf("k02"), 0, 0));
ASSERT_TRUE(Between(c.ApproximateOffsetOf("k03"), 0, 0));
ASSERT_TRUE(Between(c.ApproximateOffsetOf("k04"), 10000, 11000));
ASSERT_TRUE(Between(c.ApproximateOffsetOf("k04a"), 210000, 211000));
// k04 and k05 will be in two consecutive blocks, the index is
// an arbitrary slice between k04 and k05, either before or after k04a
ASSERT_TRUE(Between(c.ApproximateOffsetOf("k04a"), 10000, 211000));
ASSERT_TRUE(Between(c.ApproximateOffsetOf("k05"), 210000, 211000));
ASSERT_TRUE(Between(c.ApproximateOffsetOf("k06"), 510000, 511000));
ASSERT_TRUE(Between(c.ApproximateOffsetOf("k07"), 510000, 511000));
@ -2114,7 +2185,7 @@ TEST_F(GeneralTableTest, ApproximateOffsetOfPlain) {
static void DoCompressionTest(CompressionType comp) {
Random rnd(301);
TableConstructor c(BytewiseComparator());
TableConstructor c(BytewiseComparator(), true /* convert_to_internal_key_ */);
std::string tmp;
c.Add("k01", "hello");
c.Add("k02", test::CompressibleString(&rnd, 0.25, 10000, &tmp));

@ -320,6 +320,7 @@ inline bool GetLengthPrefixedSlice(Slice* input, Slice* result) {
inline Slice GetLengthPrefixedSlice(const char* data) {
uint32_t len = 0;
// +5: we assume "data" is not corrupted
// unsigned char is 7 bits, uint32_t is 32 bits, need 5 unsigned char
auto p = GetVarint32Ptr(data, data + 5 /* limit */, &len);
return Slice(p, len);
}

@ -318,9 +318,9 @@ TEST_P(EnvMoreTestWithParam, GetChildren) {
ASSERT_EQ(0U, childAttr.size());
// folder with contents returns relative path to test dir
ASSERT_OK(env_->CreateDirIfMissing(test_dir_ + "/linda"));
ASSERT_OK(env_->CreateDirIfMissing(test_dir_ + "/wanning"));
ASSERT_OK(env_->CreateDirIfMissing(test_dir_ + "/jiang"));
ASSERT_OK(env_->CreateDirIfMissing(test_dir_ + "/niu"));
ASSERT_OK(env_->CreateDirIfMissing(test_dir_ + "/you"));
ASSERT_OK(env_->CreateDirIfMissing(test_dir_ + "/guo"));
ASSERT_OK(env_->GetChildren(test_dir_, &children));
ASSERT_OK(env_->GetChildrenFileAttributes(test_dir_, &childAttr));
ASSERT_EQ(3U, children.size());

@ -1380,7 +1380,7 @@ TEST_P(TransactionTest, NoSnapshotTest) {
// Modify key after transaction start
db->Put(write_options, "AAA", "bar1");
// Read and write without a snapshot
// Read and write without a snap
txn->GetForUpdate(read_options, "AAA", &value);
ASSERT_EQ(value, "bar1");
s = txn->Put("AAA", "bar2");

Loading…
Cancel
Save