[RocksDB Performance Branch] A more customized index in PlainTableReader

Summary:
PlainTableReader to use a more customized hash table. This patch assumes the SST file is smaller than 2GB:
(1) Every bucket uses 32-bit integer
(2) no key is stored in bucket
(3) use the first bit of the bucket value to distinguish it points to the file offset or a second level index.
This index schema fits the use case that most of prefixes have very small number of keys

Test Plan: plain_table_db_test

Reviewers: haobo, kailiu, dhruba

Reviewed By: haobo

CC: nkg-, leveldb

Differential Revision: https://reviews.facebook.net/D14343
main
Siying Dong 11 years ago
parent 8c424456fc
commit f040e536e4
  1. 4
      include/rocksdb/plain_table_factory.h
  2. 8
      table/plain_table_builder.cc
  3. 333
      table/plain_table_reader.cc
  4. 48
      table/plain_table_reader.h

@ -23,7 +23,9 @@ class TableBuilder;
// IndexedTable requires fixed length key, configured as a constructor // IndexedTable requires fixed length key, configured as a constructor
// parameter of the factory class. Output file format: // parameter of the factory class. Output file format:
// +--------------------------------------------+ <= key1 offset // +-------------+
// | version |
// +-------------+------------------------------+ <= key1 offset
// | key1 | value_size (4 bytes) | | // | key1 | value_size (4 bytes) | |
// +----------------------------------------+ | // +----------------------------------------+ |
// | value1 | // | value1 |

@ -25,6 +25,10 @@ PlainTableBuilder::PlainTableBuilder(const Options& options,
int user_key_size, int key_prefix_len) : int user_key_size, int key_prefix_len) :
options_(options), file_(file), user_key_size_(user_key_size), options_(options), file_(file), user_key_size_(user_key_size),
key_prefix_len_(key_prefix_len) { key_prefix_len_(key_prefix_len) {
std::string version;
PutFixed32(&version, 1 | 0x80000000);
file_->Append(Slice(version));
offset_ = 4;
} }
PlainTableBuilder::~PlainTableBuilder() { PlainTableBuilder::~PlainTableBuilder() {
@ -43,11 +47,11 @@ void PlainTableBuilder::Add(const Slice& key, const Slice& value) {
std::string size; std::string size;
int value_size = value.size(); int value_size = value.size();
PutFixed32(&size, value_size); PutVarint32(&size, value_size);
Slice sizeSlice(size); Slice sizeSlice(size);
file_->Append(sizeSlice); file_->Append(sizeSlice);
file_->Append(value); file_->Append(value);
offset_ += value_size + 4; offset_ += value_size + size.length();
num_entries_++; num_entries_++;
} }

@ -5,6 +5,7 @@
#include "table/plain_table_reader.h" #include "table/plain_table_reader.h"
#include <unordered_map> #include <unordered_map>
#include <map>
#include "db/dbformat.h" #include "db/dbformat.h"
@ -23,21 +24,35 @@
#include "util/coding.h" #include "util/coding.h"
#include "util/hash.h" #include "util/hash.h"
#include "util/histogram.h" #include "util/histogram.h"
#include "util/murmurhash.h"
#include "util/perf_context_imp.h" #include "util/perf_context_imp.h"
#include "util/stop_watch.h" #include "util/stop_watch.h"
namespace std { namespace std {
template<> template<>
struct hash<rocksdb::Slice> { struct hash<rocksdb::Slice> {
public: public:
std::size_t operator()(rocksdb::Slice const& s) const { std::size_t operator()(rocksdb::Slice const& s) const {
return rocksdb::Hash(s.data(), s.size(), 397); return MurmurHash(s.data(), s.size(), 397);
}
};
class slice_comparator {
public:
bool operator()(rocksdb::Slice const& s1, rocksdb::Slice const& s2) {
return s1.compare(s2) < 0;
} }
}; };
} }
namespace rocksdb { namespace rocksdb {
static uint32_t getBucketId(Slice const& s, size_t prefix_len,
uint32_t num_buckets) {
return MurmurHash(s.data(), prefix_len, 397) % num_buckets;
}
PlainTableReader::PlainTableReader(const EnvOptions& storage_options, PlainTableReader::PlainTableReader(const EnvOptions& storage_options,
uint64_t file_size, int user_key_size, uint64_t file_size, int user_key_size,
int key_prefix_len, int bloom_bits_per_key, int key_prefix_len, int bloom_bits_per_key,
@ -51,6 +66,8 @@ PlainTableReader::PlainTableReader(const EnvOptions& storage_options,
filter_policy_ = nullptr; filter_policy_ = nullptr;
} }
hash_table_ = nullptr; hash_table_ = nullptr;
data_start_offset_ = 0;
data_end_offset_ = file_size;
} }
PlainTableReader::~PlainTableReader() { PlainTableReader::~PlainTableReader() {
@ -73,6 +90,10 @@ Status PlainTableReader::Open(const Options& options,
double hash_table_ratio) { double hash_table_ratio) {
assert(options.allow_mmap_reads); assert(options.allow_mmap_reads);
if (file_size > 2147483646) {
return Status::NotSupported("File is too large for PlainTableReader!");
}
PlainTableReader* t = new PlainTableReader(soptions, file_size, PlainTableReader* t = new PlainTableReader(soptions, file_size,
user_key_size, user_key_size,
key_prefix_len, key_prefix_len,
@ -101,104 +122,111 @@ Iterator* PlainTableReader::NewIterator(const ReadOptions& options) {
} }
Status PlainTableReader::PopulateIndex(uint64_t file_size) { Status PlainTableReader::PopulateIndex(uint64_t file_size) {
// Get mmapped memory to file_data_.
Status s = file_->Read(0, file_size_, &file_data_, nullptr);
if (!s.ok()) {
return s;
}
version_ = DecodeFixed32(file_data_.data());
version_ ^= 0x80000000;
assert(version_ == 1);
data_start_offset_ = 4;
data_end_offset_ = file_size;
Slice key_slice; Slice key_slice;
Slice key_prefix_slice; Slice key_prefix_slice;
Slice key_suffix_slice; Slice key_suffix_slice;
Slice value_slice; Slice value_slice;
Slice tmp_slice;
Slice prev_key_prefix_slice; Slice prev_key_prefix_slice;
uint64_t pos = 0; uint32_t pos = data_start_offset_;
uint64_t data_offset_for_cur_prefix = 0; int key_index_within_prefix = 0;
int count_prefix = 0;
bool first = true; bool first = true;
std::string prefix_sub_index; std::string prefix_sub_index;
HistogramImpl keys_per_prefix_hist; HistogramImpl keys_per_prefix_hist;
std::unordered_map<Slice, uint64_t> tmp_index; // Need map to be ordered to make sure sub indexes generated
// are in order.
std::map<Slice, std::string, std::slice_comparator> prefix2map;
while (pos < file_size) { while (pos < file_size) {
uint64_t key_offset = pos; uint32_t key_offset = pos;
pos = Next(pos, &key_slice, &value_slice, &tmp_slice); status_ = Next(pos, &key_slice, &value_slice, pos);
key_prefix_slice = Slice(key_slice.data(), key_prefix_len_); key_prefix_slice = Slice(key_slice.data(), key_prefix_len_);
if (first || prev_key_prefix_slice != key_prefix_slice) { if (first || prev_key_prefix_slice != key_prefix_slice) {
if (!first) { if (!first) {
if (count_prefix < 8 || key_prefix_len_ == user_key_size_) { keys_per_prefix_hist.Add(key_index_within_prefix);
tmp_index[prev_key_prefix_slice] = data_offset_for_cur_prefix;
} else {
tmp_index[prev_key_prefix_slice] = sub_index_.length()
| kSubIndexMask;
PutFixed32(&sub_index_, (count_prefix - 1) / 8 + 1);
sub_index_.append(prefix_sub_index);
}
prefix_sub_index.clear();
data_offset_for_cur_prefix = key_offset;
keys_per_prefix_hist.Add(count_prefix);
} }
key_index_within_prefix = 0;
prev_key_prefix_slice = key_prefix_slice; prev_key_prefix_slice = key_prefix_slice;
count_prefix = 1;
} else {
count_prefix++;
}
if (key_prefix_len_ < user_key_size_ && count_prefix % 8 == 1) {
prefix_sub_index.append(key_slice.data() + key_prefix_len_,
user_key_size_ - key_prefix_len_);
PutFixed64(&prefix_sub_index, key_offset);
} }
first = false; if (key_index_within_prefix++ % 8 == 0) {
// Add an index key for every 8 keys
std::string& prefix_index = prefix2map[key_prefix_slice];
PutFixed32(&prefix_index, key_offset);
} }
keys_per_prefix_hist.Add(count_prefix); first = false;
if (count_prefix <= 2 || key_prefix_len_ == user_key_size_) {
tmp_index[prev_key_prefix_slice] = data_offset_for_cur_prefix;
} else {
tmp_index[prev_key_prefix_slice] = sub_index_.length() | kSubIndexMask;
PutFixed32(&sub_index_, (count_prefix - 1) / 8 + 1);
sub_index_.append(prefix_sub_index);
} }
keys_per_prefix_hist.Add(key_index_within_prefix);
if (hash_table_ != nullptr) { if (hash_table_ != nullptr) {
delete[] hash_table_; delete[] hash_table_;
} }
// Make the hash table 3/5 full
std::vector<Slice> filter_entries(0); // for creating bloom filter; std::vector<Slice> filter_entries(0); // for creating bloom filter;
if (filter_policy_ != nullptr) { if (filter_policy_ != nullptr) {
filter_entries.reserve(tmp_index.size()); filter_entries.reserve(prefix2map.size());
} }
double hash_table_size_multipier = double hash_table_size_multipier =
(hash_table_ratio_ > 1.0) ? 1.0 : 1.0 / hash_table_ratio_; (hash_table_ratio_ > 1.0) ? 1.0 : 1.0 / hash_table_ratio_;
hash_table_size_ = tmp_index.size() * hash_table_size_multipier + 1; hash_table_size_ = prefix2map.size() * hash_table_size_multipier + 1;
hash_table_ = new char[GetHashTableRecordLen() * hash_table_size_]; hash_table_ = new uint32_t[hash_table_size_];
for (int i = 0; i < hash_table_size_; i++) { std::vector<std::string> hash2map(hash_table_size_);
memcpy(GetHashTableBucketPtr(i) + key_prefix_len_, &file_size_,
kOffsetLen);
}
for (auto it = tmp_index.begin(); it != tmp_index.end(); ++it) { size_t sub_index_size_needed = 0;
for (auto& p: prefix2map) {
auto& sub_index = hash2map[getBucketId(p.first, key_prefix_len_,
hash_table_size_)];
if (sub_index.length() > 0 || p.second.length() > kOffsetLen) {
if (sub_index.length() <= kOffsetLen) {
sub_index_size_needed += sub_index.length() + 4;
}
sub_index_size_needed += p.second.length();
}
sub_index.append(p.second);
if (filter_policy_ != nullptr) { if (filter_policy_ != nullptr) {
filter_entries.push_back(it->first); filter_entries.push_back(p.first);
}
} }
int bucket = GetHashTableBucket(it->first); sub_index_.clear();
uint64_t* hash_value; Log(options_.info_log, "Reserving %zu bytes for sub index",
while (true) { sub_index_size_needed);
GetHashValue(bucket, &hash_value); sub_index_.reserve(sub_index_size_needed);
if (*hash_value == file_size_) { for (int i = 0; i < hash_table_size_; i++) {
uint32_t num_keys_for_bucket = hash2map[i].length() / kOffsetLen;
switch (num_keys_for_bucket) {
case 0:
// No key for bucket
hash_table_[i] = data_end_offset_;
break; break;
case 1:
// point directly to the file offset
hash_table_[i] = DecodeFixed32(hash2map[i].data());
break;
default:
// point to index block
hash_table_[i] = sub_index_.length() | kSubIndexMask;
PutFixed32(&sub_index_, num_keys_for_bucket);
sub_index_.append(hash2map[i]);
} }
bucket = (bucket + 1) % hash_table_size_;
}
char* bucket_ptr = GetHashTableBucketPtr(bucket);
memcpy(bucket_ptr, it->first.data(), key_prefix_len_);
memcpy(bucket_ptr + key_prefix_len_, &it->second, kOffsetLen);
} }
if (filter_policy_ != nullptr) { if (filter_policy_ != nullptr) {
filter_str_.clear();
filter_policy_->CreateFilter(&filter_entries[0], filter_entries.size(), filter_policy_->CreateFilter(&filter_entries[0], filter_entries.size(),
&filter_str_); &filter_str_);
filter_slice_ = Slice(filter_str_.data(), filter_str_.size()); filter_slice_ = Slice(filter_str_.data(), filter_str_.size());
} }
Log(options_.info_log, "Number of prefixes: %d, suffix_map length %ld", Log(options_.info_log, "hash table size: %d, suffix_map length %zu",
hash_table_size_, sub_index_.length()); hash_table_size_, sub_index_.length());
Log(options_.info_log, "Number of Keys per prefix Histogram: %s", Log(options_.info_log, "Number of Keys per prefix Histogram: %s",
keys_per_prefix_hist.ToString().c_str()); keys_per_prefix_hist.ToString().c_str());
@ -206,51 +234,33 @@ Status PlainTableReader::PopulateIndex(uint64_t file_size) {
return Status::OK(); return Status::OK();
} }
inline int PlainTableReader::GetHashTableBucket(Slice key) { uint32_t PlainTableReader::GetOffset(const Slice& target,
return rocksdb::Hash(key.data(), key_prefix_len_, 397) % hash_table_size_; bool& prefix_matched) {
} prefix_matched = false;
int bucket = getBucketId(target, key_prefix_len_, hash_table_size_);
inline void PlainTableReader::GetHashValue(int bucket, uint64_t** ret_value) { uint32_t bucket_value = hash_table_[bucket];
*ret_value = (uint64_t*) (GetHashTableBucketPtr(bucket) + key_prefix_len_); if (bucket_value == data_end_offset_) {
} return data_end_offset_;
} else if ((bucket_value & kSubIndexMask) == 0) {
Status PlainTableReader::GetOffset(const Slice& target, uint64_t* offset) { // point directly to the file
Status s; return bucket_value;
int bucket = GetHashTableBucket(target);
uint64_t* found_value;
Slice hash_key;
while (true) {
GetHashValue(bucket, &found_value);
if (*found_value == file_size_) {
break;
}
GetHashKey(bucket, &hash_key);
if (target.starts_with(hash_key)) {
break;
}
bucket = (bucket + 1) % hash_table_size_;
}
if (*found_value == file_size_ || (*found_value & kSubIndexMask) == 0) {
*offset = *found_value;
return Status::OK();
} }
// point to sub-index, need to do a binary search
uint32_t low = 0; uint32_t low = 0;
uint64_t prefix_index_offset = *found_value ^ kSubIndexMask; uint64_t prefix_index_offset = bucket_value ^ kSubIndexMask;
uint32_t high = DecodeFixed32(sub_index_.data() + prefix_index_offset); uint32_t upper_bound = DecodeFixed32(sub_index_.data() + prefix_index_offset);
uint32_t high = upper_bound;
uint64_t base_offset = prefix_index_offset + 4; uint64_t base_offset = prefix_index_offset + 4;
char* mid_key_str = new char[target.size()]; Slice mid_key;
memcpy(mid_key_str, target.data(), target.size());
Slice mid_key = Slice(mid_key_str, target.size());
// The key is between (low, high). Do a binary search between it. // The key is between [low, high). Do a binary search between it.
while (high - low > 1) { while (high - low > 1) {
uint32_t mid = (high + low) / 2; uint32_t mid = (high + low) / 2;
const char* base = sub_index_.data() + base_offset const char* index_offset = sub_index_.data() + base_offset
+ (user_key_size_ - key_prefix_len_ + kOffsetLen) * mid; + kOffsetLen * mid;
memcpy(mid_key_str + key_prefix_len_, base, uint32_t file_offset = DecodeFixed32(index_offset);
user_key_size_ - key_prefix_len_); mid_key = Slice(file_data_.data() + file_offset, user_key_size_);
int cmp_result = options_.comparator->Compare(target, mid_key); int cmp_result = options_.comparator->Compare(target, mid_key);
if (cmp_result > 0) { if (cmp_result > 0) {
@ -259,21 +269,32 @@ Status PlainTableReader::GetOffset(const Slice& target, uint64_t* offset) {
if (cmp_result == 0) { if (cmp_result == 0) {
// Happen to have found the exact key or target is smaller than the // Happen to have found the exact key or target is smaller than the
// first key after base_offset. // first key after base_offset.
*offset = DecodeFixed64(base + user_key_size_ - key_prefix_len_); prefix_matched = true;
delete[] mid_key_str; return file_offset;
return s;
} else { } else {
high = mid; high = mid;
} }
} }
} }
const char* base = sub_index_.data() + base_offset // The key is between low and low+1 (if exists). Both of them can have the
+ (user_key_size_ - key_prefix_len_ + kOffsetLen) * low; // correct prefix. Need to rule out at least one, to avoid to miss the
*offset = DecodeFixed64(base + user_key_size_ - key_prefix_len_); // correct one.
uint32_t low_key_offset = DecodeFixed32(
delete[] mid_key_str; sub_index_.data() + base_offset + kOffsetLen * low);
return s; if (low + 1 < upper_bound) {
if (Slice(file_data_.data() + low_key_offset, key_prefix_len_)
== Slice(target.data(), key_prefix_len_)) {
prefix_matched = true;
} else {
prefix_matched = false;
return DecodeFixed32(
sub_index_.data() + base_offset + kOffsetLen * (low + 1));
}
} else {
prefix_matched = false;
}
return low_key_offset;
} }
bool PlainTableReader::MayHavePrefix(const Slice& target_prefix) { bool PlainTableReader::MayHavePrefix(const Slice& target_prefix) {
@ -282,46 +303,74 @@ bool PlainTableReader::MayHavePrefix(const Slice& target_prefix) {
} }
uint64_t PlainTableReader::Next(uint64_t offset, Slice* key, Slice* value, Status PlainTableReader::Next(uint32_t offset, Slice* key, Slice* value,
Slice* tmp_slice) { uint32_t& next_offset) {
if (offset >= file_size_) { if (offset == data_end_offset_) {
return file_size_; next_offset = data_end_offset_;
return Status::OK();
} }
int internal_key_size = GetInternalKeyLength();
Status s = file_->Read(offset, internal_key_size, key, nullptr); if (offset > data_end_offset_) {
offset += internal_key_size; return Status::Corruption("Offset is out of file size");
}
s = file_->Read(offset, 4, tmp_slice, nullptr); int internal_key_size = GetInternalKeyLength();
offset += 4; if (offset + internal_key_size >= data_end_offset_) {
uint32_t value_size = DecodeFixed32(tmp_slice->data()); return Status::Corruption("Un able to read the next key");
}
s = file_->Read(offset, value_size, value, nullptr); const char* key_ptr = file_data_.data() + offset;
offset += value_size; *key = Slice(key_ptr, internal_key_size);
return offset; uint32_t value_size;
const char* value_ptr = GetVarint32Ptr(key_ptr + internal_key_size,
file_data_.data() + data_end_offset_,
&value_size);
if (value_ptr == nullptr) {
return Status::Corruption("Error reading value length.");
}
next_offset = offset + (value_ptr - key_ptr) + value_size;
if (next_offset > data_end_offset_) {
return Status::Corruption("Reach end of file when reading value");
}
*value = Slice(value_ptr, value_size);
return Status::OK();
} }
Status PlainTableReader::Get( Status PlainTableReader::Get(
const ReadOptions& ro, const Slice& target, void* arg, const ReadOptions& ro, const Slice& target, void* arg,
bool (*saver)(void*, const Slice&, const Slice&, bool), bool (*saver)(void*, const Slice&, const Slice&, bool),
void (*mark_key_may_exist)(void*)) { void (*mark_key_may_exist)(void*)) {
uint64_t offset; // Check bloom filter first.
Status s = GetOffset(target, &offset); if (!MayHavePrefix(Slice(target.data(), key_prefix_len_))) {
if (!s.ok()) { return Status::OK();
return s;
} }
uint32_t offset;
bool prefix_match;
offset = GetOffset(target, prefix_match);
Slice found_key; Slice found_key;
Slice found_value; Slice found_value;
Slice tmp_slice; while (offset < data_end_offset_) {
while (offset < file_size_) { Status s = Next(offset, &found_key, &found_value, offset);
offset = Next(offset, &found_key, &found_value, &tmp_slice); if (!s.ok()) {
return s;
}
if (!prefix_match) {
// Need to verify prefix for the first key found if it is not yet
// checked.
if (!target.starts_with(Slice(found_key.data(), key_prefix_len_))) {
break;
}
prefix_match = true;
}
if (options_.comparator->Compare(found_key, target) >= 0 if (options_.comparator->Compare(found_key, target) >= 0
&& !(*saver)(arg, found_key, found_value, true)) { && !(*saver)(arg, found_key, found_value, true)) {
break; break;
} }
} }
return s; return Status::OK();
} }
bool PlainTableReader::TEST_KeyInCache(const ReadOptions& options, bool PlainTableReader::TEST_KeyInCache(const ReadOptions& options,
@ -342,11 +391,12 @@ PlainTableIterator::~PlainTableIterator() {
} }
bool PlainTableIterator::Valid() const { bool PlainTableIterator::Valid() const {
return offset_ < table_->file_size_ && offset_ >= 0; return offset_ < table_->data_end_offset_
&& offset_ >= table_->data_start_offset_;
} }
void PlainTableIterator::SeekToFirst() { void PlainTableIterator::SeekToFirst() {
next_offset_ = 0; next_offset_ = table_->data_start_offset_;
Next(); Next();
} }
@ -356,26 +406,35 @@ void PlainTableIterator::SeekToLast() {
void PlainTableIterator::Seek(const Slice& target) { void PlainTableIterator::Seek(const Slice& target) {
if (!table_->MayHavePrefix(Slice(target.data(), table_->key_prefix_len_))) { if (!table_->MayHavePrefix(Slice(target.data(), table_->key_prefix_len_))) {
offset_ = next_offset_ = table_->file_size_; offset_ = next_offset_ = table_->data_end_offset_;
return; return;
} }
bool prefix_match;
next_offset_ = table_->GetOffset(target, prefix_match);
Status s = table_->GetOffset(target, &next_offset_); if (next_offset_ < table_-> data_end_offset_) {
if (!s.ok()) { for (Next(); status_.ok() && Valid(); Next()) {
status_ = s; if (!prefix_match) {
// Need to verify the first key's prefix
if (!target.starts_with(Slice(key().data(), table_->key_prefix_len_))) {
offset_ = next_offset_ = table_->data_end_offset_;
break;
} }
if (next_offset_ < table_->file_size_) { prefix_match = true;
for (Next();
Valid() && table_->options_.comparator->Compare(key(), target) < 0;
Next()) {
} }
if (table_->options_.comparator->Compare(key(), target) >= 0) {
break;
}
}
} else {
offset_ = table_->data_end_offset_;
} }
} }
void PlainTableIterator::Next() { void PlainTableIterator::Next() {
offset_ = next_offset_; offset_ = next_offset_;
Slice tmp_slice; Slice tmp_slice;
next_offset_ = table_->Next(next_offset_, &key_, &value_, &tmp_slice); status_ = table_->Next(next_offset_, &key_, &value_, next_offset_);
} }
void PlainTableIterator::Prev() { void PlainTableIterator::Prev() {

@ -25,7 +25,9 @@ using std::unique_ptr;
using std::unordered_map; using std::unordered_map;
// Based on following output file format: // Based on following output file format:
// +--------------------------------------------+ <= key1_data_offset // +-------------+
// | version |
// +-------------+------------------------------+ <= key1_data_offset
// | key1 | value_size (4 bytes) | | // | key1 | value_size (4 bytes) | |
// +----------------------------------------+ | // +----------------------------------------+ |
// | value1 | // | value1 |
@ -85,7 +87,7 @@ public:
~PlainTableReader(); ~PlainTableReader();
private: private:
char* hash_table_; uint32_t* hash_table_;
int hash_table_size_; int hash_table_size_;
std::string sub_index_; std::string sub_index_;
@ -94,7 +96,11 @@ private:
Status status_; Status status_;
unique_ptr<RandomAccessFile> file_; unique_ptr<RandomAccessFile> file_;
uint64_t file_size_; Slice file_data_;
uint32_t version_;
uint32_t file_size_;
uint32_t data_start_offset_;
uint32_t data_end_offset_;
const size_t user_key_size_; const size_t user_key_size_;
const size_t key_prefix_len_; const size_t key_prefix_len_;
const double hash_table_ratio_; const double hash_table_ratio_;
@ -105,32 +111,34 @@ private:
TableProperties tbl_props; TableProperties tbl_props;
static const size_t kNumInternalBytes = 8; static const size_t kNumInternalBytes = 8;
static const uint64_t kSubIndexMask = 0x8000000000000000; static const uint32_t kSubIndexMask = 0x80000000;
static const size_t kOffsetLen = sizeof(uint64_t); static const size_t kOffsetLen = sizeof(uint32_t);
inline int GetHashTableBucket(Slice key);
inline size_t GetInternalKeyLength() { inline size_t GetInternalKeyLength() {
return user_key_size_ + kNumInternalBytes; return user_key_size_ + kNumInternalBytes;
} }
inline size_t GetHashTableRecordLen() {
return key_prefix_len_ + kOffsetLen;
}
inline char* GetHashTableBucketPtr(int bucket) {
return hash_table_ + GetHashTableRecordLen() * bucket;
}
inline void GetHashKey(int bucket, Slice* slice) {
*slice = Slice(GetHashTableBucketPtr(bucket), key_prefix_len_);
}
inline void GetHashValue(int bucket, uint64_t** ret_value);
friend class TableCache; friend class TableCache;
friend class PlainTableIterator; friend class PlainTableIterator;
// Populate the internal indexes. It must be called before
// any query to the table.
// This query will populate the hash table hash_table_, the second
// level of indexes sub_index_ and bloom filter filter_slice_ if enabled.
Status PopulateIndex(uint64_t file_size); Status PopulateIndex(uint64_t file_size);
uint64_t Next(uint64_t offset, Slice* key, Slice* value, Slice* tmp_slice);
Status GetOffset(const Slice& target, uint64_t* offset); // Check bloom filter to see whether it might contain this prefix
bool MayHavePrefix(const Slice& target_prefix); bool MayHavePrefix(const Slice& target_prefix);
// Read the key and value at offset to key and value.
// tmp_slice is a tmp slice.
// return next_offset as the offset for the next key.
Status Next(uint32_t offset, Slice* key, Slice* value, uint32_t& next_offset);
// Get file offset for key target.
// return value prefix_matched is set to true if the offset is confirmed
// for a key with the same prefix as target.
uint32_t GetOffset(const Slice& target, bool& prefix_matched);
// No copying allowed // No copying allowed
explicit PlainTableReader(const TableReader&) = delete; explicit PlainTableReader(const TableReader&) = delete;
void operator=(const TableReader&) = delete; void operator=(const TableReader&) = delete;
@ -162,8 +170,8 @@ public:
private: private:
PlainTableReader* table_; PlainTableReader* table_;
uint64_t offset_; uint32_t offset_;
uint64_t next_offset_; uint32_t next_offset_;
Slice key_; Slice key_;
Slice value_; Slice value_;
Status status_; Status status_;

Loading…
Cancel
Save