[RocksDB] Reduce memory footprint of the blockbased table hash index.

Summary:
Currently, the in-memory hash index of blockbased table uses a precise hash map to track the prefix to block range mapping. In some use cases, especially when prefix itself is big, the memory overhead becomes a problem. This diff introduces a fixed hash bucket array that does not store the prefix and allows prefix collision, which is similar to the plaintable hash index, in order to reduce the memory consumption.
Just a quick draft, still testing and refining.

Test Plan: unit test and shadow testing

Reviewers: dhruba, kailiu, sdong

Reviewed By: sdong

Subscribers: leveldb

Differential Revision: https://reviews.facebook.net/D19047
This commit is contained in:
Haobo Xu 2014-06-12 19:03:22 -07:00
parent 3525aac9e5
commit 0f0076ed5a
8 changed files with 475 additions and 33 deletions

View File

@ -74,6 +74,12 @@ struct BlockBasedTableOptions {
IndexType index_type = kBinarySearch;
// Influence the behavior when kHashSearch is used.
// if false, stores a precise prefix to block range mapping
// if true, does not store prefix and allows prefix hash collision
// (less memory consumption)
bool hash_index_allow_collision = true;
// Use the specified checksum type. Newly created table files will be
// protected with this checksum type. Old table files will still be readable,
// even though they have different checksum type.

View File

@ -18,6 +18,7 @@
#include "rocksdb/comparator.h"
#include "table/block_hash_index.h"
#include "table/block_prefix_index.h"
#include "table/format.h"
#include "util/coding.h"
#include "util/logging.h"
@ -97,6 +98,7 @@ class Block::Iter : public Iterator {
Slice value_;
Status status_;
BlockHashIndex* hash_index_;
BlockPrefixIndex* prefix_index_;
inline int Compare(const Slice& a, const Slice& b) const {
return comparator_->Compare(a, b);
@ -124,14 +126,16 @@ class Block::Iter : public Iterator {
public:
Iter(const Comparator* comparator, const char* data, uint32_t restarts,
uint32_t num_restarts, BlockHashIndex* hash_index)
uint32_t num_restarts, BlockHashIndex* hash_index,
BlockPrefixIndex* prefix_index)
: comparator_(comparator),
data_(data),
restarts_(restarts),
num_restarts_(num_restarts),
current_(restarts_),
restart_index_(num_restarts_),
hash_index_(hash_index) {
hash_index_(hash_index),
prefix_index_(prefix_index) {
assert(num_restarts_ > 0);
}
@ -174,8 +178,13 @@ class Block::Iter : public Iterator {
virtual void Seek(const Slice& target) {
uint32_t index = 0;
bool ok = hash_index_ ? HashSeek(target, &index)
: BinarySeek(target, 0, num_restarts_ - 1, &index);
bool ok = false;
if (prefix_index_) {
ok = PrefixSeek(target, &index);
} else {
ok = hash_index_ ? HashSeek(target, &index)
: BinarySeek(target, 0, num_restarts_ - 1, &index);
}
if (!ok) {
return;
@ -238,8 +247,9 @@ class Block::Iter : public Iterator {
return true;
}
}
// Binary search in restart array to find the first restart point
// with a key >= target
// with a key >= target (TODO: this comment is inaccurate)
bool BinarySeek(const Slice& target, uint32_t left, uint32_t right,
uint32_t* index) {
assert(left <= right);
@ -256,14 +266,17 @@ class Block::Iter : public Iterator {
return false;
}
Slice mid_key(key_ptr, non_shared);
if (Compare(mid_key, target) < 0) {
int cmp = Compare(mid_key, target);
if (cmp < 0) {
// Key at "mid" is smaller than "target". Therefore all
// blocks before "mid" are uninteresting.
left = mid;
} else {
} else if (cmp > 0) {
// Key at "mid" is >= "target". Therefore all blocks at or
// after "mid" are uninteresting.
right = mid - 1;
} else {
left = right = mid;
}
}
@ -271,12 +284,56 @@ class Block::Iter : public Iterator {
return true;
}
// Binary search in block_ids to find the first block
// with a key >= target
bool BinaryBlockIndexSeek(const Slice& target, uint32_t* block_ids,
uint32_t left, uint32_t right,
uint32_t* index) {
assert(left <= right);
while (left <= right) {
uint32_t mid = (left + right) / 2;
uint32_t region_offset = GetRestartPoint(block_ids[mid]);
uint32_t shared, non_shared, value_length;
const char* key_ptr =
DecodeEntry(data_ + region_offset, data_ + restarts_, &shared,
&non_shared, &value_length);
if (key_ptr == nullptr || (shared != 0)) {
CorruptionError();
return false;
}
Slice mid_key(key_ptr, non_shared);
int cmp = Compare(mid_key, target);
if (cmp < 0) {
// Key at "target" is larger than "mid". Therefore all
// blocks before or at "mid" are uninteresting.
left = mid + 1;
} else {
// Key at "target" is <= "mid". Therefore all blocks
// after "mid" are uninteresting.
// If there is only one block left, we found it.
if (left == right) break;
right = mid;
}
}
if (left == right) {
*index = block_ids[left];
return true;
} else {
assert(left > right);
// Mark iterator invalid
current_ = restarts_;
return false;
}
}
bool HashSeek(const Slice& target, uint32_t* index) {
assert(hash_index_);
auto restart_index = hash_index_->GetRestartIndex(target);
if (restart_index == nullptr) {
current_ = restarts_;
return 0;
return false;
}
// the elements in restart_array[index : index + num_blocks]
@ -285,6 +342,20 @@ class Block::Iter : public Iterator {
auto right = restart_index->first_index + restart_index->num_blocks - 1;
return BinarySeek(target, left, right, index);
}
bool PrefixSeek(const Slice& target, uint32_t* index) {
assert(prefix_index_);
uint32_t* block_ids = nullptr;
uint32_t num_blocks = prefix_index_->GetBlocks(target, &block_ids);
if (num_blocks == 0) {
current_ = restarts_;
return false;
} else {
return BinaryBlockIndexSeek(target, block_ids, 0, num_blocks - 1, index);
}
}
};
Iterator* Block::NewIterator(const Comparator* cmp) {
@ -296,7 +367,7 @@ Iterator* Block::NewIterator(const Comparator* cmp) {
return NewEmptyIterator();
} else {
return new Iter(cmp, data_, restart_offset_, num_restarts,
hash_index_.get());
hash_index_.get(), prefix_index_.get());
}
}
@ -304,4 +375,8 @@ void Block::SetBlockHashIndex(BlockHashIndex* hash_index) {
hash_index_.reset(hash_index);
}
void Block::SetBlockPrefixIndex(BlockPrefixIndex* prefix_index) {
prefix_index_.reset(prefix_index);
}
} // namespace rocksdb

View File

@ -19,6 +19,7 @@ namespace rocksdb {
struct BlockContents;
class Comparator;
class BlockHashIndex;
class BlockPrefixIndex;
class Block {
public:
@ -41,6 +42,7 @@ class Block {
// the key that is just pass the target key.
Iterator* NewIterator(const Comparator* comparator);
void SetBlockHashIndex(BlockHashIndex* hash_index);
void SetBlockPrefixIndex(BlockPrefixIndex* prefix_index);
private:
const char* data_;
@ -50,6 +52,7 @@ class Block {
bool cachable_;
CompressionType compression_type_;
std::unique_ptr<BlockHashIndex> hash_index_;
std::unique_ptr<BlockPrefixIndex> prefix_index_;
// No copying allowed
Block(const Block&);

View File

@ -476,7 +476,7 @@ void BlockBasedTableBuilder::Add(const Slice& key, const Slice& value) {
if (r->props.num_entries > 0) {
assert(r->internal_comparator.Compare(key, Slice(r->last_key)) > 0);
}
r->index_builder->OnKeyAdded(key);
auto should_flush = r->flush_block_policy->Update(key, value);
if (should_flush) {
assert(!r->data_block.empty());
@ -505,6 +505,7 @@ void BlockBasedTableBuilder::Add(const Slice& key, const Slice& value) {
r->props.raw_key_size += key.size();
r->props.raw_value_size += value.size();
r->index_builder->OnKeyAdded(key);
NotifyCollectTableCollectorsOnAdd(key, value, r->table_properties_collectors,
r->options.info_log.get());
}

View File

@ -27,6 +27,7 @@
#include "table/block.h"
#include "table/filter_block.h"
#include "table/block_hash_index.h"
#include "table/block_prefix_index.h"
#include "table/format.h"
#include "table/meta_blocks.h"
#include "table/two_level_iterator.h"
@ -194,7 +195,8 @@ class HashIndexReader : public IndexReader {
const Footer& footer, RandomAccessFile* file, Env* env,
const Comparator* comparator,
const BlockHandle& index_handle,
Iterator* meta_index_iter, IndexReader** index_reader) {
Iterator* meta_index_iter, IndexReader** index_reader,
bool hash_index_allow_collision) {
Block* index_block = nullptr;
auto s = ReadBlockFromFile(file, footer, ReadOptions(), index_handle,
&index_block, env);
@ -203,12 +205,21 @@ class HashIndexReader : public IndexReader {
return s;
}
// Note, failure to create prefix hash index does not need to be a
// hard error. We can still fall back to the original binary search index.
// So, Create will succeed regardless, from this point on.
auto new_index_reader =
new HashIndexReader(comparator, index_block);
*index_reader = new_index_reader;
// Get prefixes block
BlockHandle prefixes_handle;
s = FindMetaBlock(meta_index_iter, kHashIndexPrefixesBlock,
&prefixes_handle);
if (!s.ok()) {
return s;
// TODO: log error
return Status::OK();
}
// Get index metadata block
@ -216,7 +227,8 @@ class HashIndexReader : public IndexReader {
s = FindMetaBlock(meta_index_iter, kHashIndexPrefixesMetadataBlock,
&prefixes_meta_handle);
if (!s.ok()) {
return s;
// TODO: log error
return Status::OK();
}
// Read contents for the blocks
@ -234,27 +246,47 @@ class HashIndexReader : public IndexReader {
if (prefixes_contents.heap_allocated) {
delete[] prefixes_contents.data.data();
}
return s;
// TODO: log error
return Status::OK();
}
auto new_index_reader =
new HashIndexReader(comparator, index_block, prefixes_contents);
BlockHashIndex* hash_index = nullptr;
s = CreateBlockHashIndex(hash_key_extractor, prefixes_contents.data,
prefixes_meta_contents.data, &hash_index);
if (!s.ok()) {
return s;
if (!hash_index_allow_collision) {
// TODO: deprecate once hash_index_allow_collision proves to be stable.
BlockHashIndex* hash_index = nullptr;
s = CreateBlockHashIndex(hash_key_extractor,
prefixes_contents.data,
prefixes_meta_contents.data,
&hash_index);
// TODO: log error
if (s.ok()) {
new_index_reader->index_block_->SetBlockHashIndex(hash_index);
new_index_reader->OwnPrefixesContents(prefixes_contents);
}
} else {
BlockPrefixIndex* prefix_index = nullptr;
s = BlockPrefixIndex::Create(hash_key_extractor,
prefixes_contents.data,
prefixes_meta_contents.data,
&prefix_index);
// TODO: log error
if (s.ok()) {
new_index_reader->index_block_->SetBlockPrefixIndex(prefix_index);
}
}
new_index_reader->index_block_->SetBlockHashIndex(hash_index);
*index_reader = new_index_reader;
// release resources
// Always release prefix meta block
if (prefixes_meta_contents.heap_allocated) {
delete[] prefixes_meta_contents.data.data();
}
return s;
// Release prefix content block if we don't own it.
if (!new_index_reader->own_prefixes_contents_) {
if (prefixes_contents.heap_allocated) {
delete[] prefixes_contents.data.data();
}
}
return Status::OK();
}
virtual Iterator* NewIterator() override {
@ -264,21 +296,26 @@ class HashIndexReader : public IndexReader {
virtual size_t size() const override { return index_block_->size(); }
private:
HashIndexReader(const Comparator* comparator, Block* index_block,
const BlockContents& prefixes_contents)
HashIndexReader(const Comparator* comparator, Block* index_block)
: IndexReader(comparator),
index_block_(index_block),
prefixes_contents_(prefixes_contents) {
own_prefixes_contents_(false) {
assert(index_block_ != nullptr);
}
~HashIndexReader() {
if (prefixes_contents_.heap_allocated) {
if (own_prefixes_contents_ && prefixes_contents_.heap_allocated) {
delete[] prefixes_contents_.data.data();
}
}
void OwnPrefixesContents(const BlockContents& prefixes_contents) {
prefixes_contents_ = prefixes_contents;
own_prefixes_contents_ = true;
}
std::unique_ptr<Block> index_block_;
bool own_prefixes_contents_;
BlockContents prefixes_contents_;
};
@ -308,6 +345,7 @@ struct BlockBasedTable::Rep {
std::shared_ptr<const TableProperties> table_properties;
BlockBasedTableOptions::IndexType index_type;
bool hash_index_allow_collision;
// TODO(kailiu) It is very ugly to use internal key in table, since table
// module should not be relying on db module. However to make things easier
// and compatible with existing code, we introduce a wrapper that allows
@ -407,6 +445,7 @@ Status BlockBasedTable::Open(const Options& options, const EnvOptions& soptions,
rep->file = std::move(file);
rep->footer = footer;
rep->index_type = table_options.index_type;
rep->hash_index_allow_collision = table_options.hash_index_allow_collision;
SetupCacheKeyPrefix(rep);
unique_ptr<BlockBasedTable> new_table(new BlockBasedTable(rep));
@ -1122,7 +1161,8 @@ Status BlockBasedTable::CreateIndexReader(IndexReader** index_reader,
new InternalKeySliceTransform(rep_->options.prefix_extractor.get()));
return HashIndexReader::Create(
rep_->internal_prefix_transform.get(), footer, file, env, comparator,
footer.index_handle(), meta_index_iter, index_reader);
footer.index_handle(), meta_index_iter, index_reader,
rep_->hash_index_allow_collision);
}
default: {
std::string error_message =

242
table/block_prefix_index.cc Normal file
View File

@ -0,0 +1,242 @@
// Copyright (c) 2014, Facebook, Inc. All rights reserved.
// This source code is licensed under the BSD-style license found in the
// 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/block_prefix_index.h"
#include <vector>
#include "rocksdb/comparator.h"
#include "rocksdb/slice.h"
#include "rocksdb/slice_transform.h"
#include "util/arena.h"
#include "util/coding.h"
#include "util/hash.h"
namespace rocksdb {
namespace {
inline uint32_t Hash(const Slice& s) {
return rocksdb::Hash(s.data(), s.size(), 0);
}
inline uint32_t PrefixToBucket(const Slice& prefix, uint32_t num_buckets) {
return Hash(prefix) % num_buckets;
}
// The prefix block index is simply a bucket array, with each entry pointing to
// the blocks that span the prefixes hashed to this bucket.
//
// To reduce memory footprint, if there is only one block per bucket, the entry
// stores the block id directly. If there are more than one blocks per bucket,
// because of hash collision or a single prefix spanning multiple blocks,
// the entry points to an array of block ids. The block array is an array of
// uint32_t's. The first uint32_t indicates the total number of blocks, followed
// by the block ids.
//
// To differentiate the two cases, the high order bit of the entry indicates
// whether it is a 'pointer' into a separate block array.
// 0x7FFFFFFF is reserved for empty bucket.
const uint32_t kNoneBlock = 0x7FFFFFFF;
const uint32_t kBlockArrayMask = 0x80000000;
inline bool IsNone(uint32_t block_id) {
return block_id == kNoneBlock;
}
inline bool IsBlockId(uint32_t block_id) {
return (block_id & kBlockArrayMask) == 0;
}
inline uint32_t DecodeIndex(uint32_t block_id) {
uint32_t index = block_id ^ kBlockArrayMask;
assert(index < kBlockArrayMask);
return index;
}
inline uint32_t EncodeIndex(uint32_t index) {
assert(index < kBlockArrayMask);
return index | kBlockArrayMask;
}
// temporary storage for prefix information during index building
struct PrefixRecord {
Slice prefix;
uint32_t start_block;
uint32_t end_block;
uint32_t num_blocks;
PrefixRecord* next;
};
} // anonymous namespace
class BlockPrefixIndex::Builder {
public:
explicit Builder(const SliceTransform* internal_prefix_extractor)
: internal_prefix_extractor_(internal_prefix_extractor) {}
void Add(const Slice& key_prefix, uint32_t start_block,
uint32_t num_blocks) {
PrefixRecord* record = reinterpret_cast<PrefixRecord*>(
arena_.AllocateAligned(sizeof(PrefixRecord)));
record->prefix = key_prefix;
record->start_block = start_block;
record->end_block = start_block + num_blocks - 1;
record->num_blocks = num_blocks;
prefixes_.push_back(record);
}
BlockPrefixIndex* Finish() {
// For now, use roughly 1:1 prefix to bucket ratio.
uint32_t num_buckets = prefixes_.size() + 1;
// Collect prefix records that hash to the same bucket, into a single
// linklist.
std::vector<PrefixRecord*> prefixes_per_bucket(num_buckets, nullptr);
std::vector<uint32_t> num_blocks_per_bucket(num_buckets, 0);
for (PrefixRecord* current : prefixes_) {
uint32_t bucket = PrefixToBucket(current->prefix, num_buckets);
// merge the prefix block span if the first block of this prefix is
// connected to the last block of the previous prefix.
PrefixRecord* prev = prefixes_per_bucket[bucket];
if (prev) {
assert(current->start_block >= prev->end_block);
auto distance = current->start_block - prev->end_block;
if (distance <= 1) {
prev->end_block = current->end_block;
prev->num_blocks = prev->end_block - prev->start_block + 1;
continue;
}
}
current->next = prev;
prefixes_per_bucket[bucket] = current;
num_blocks_per_bucket[bucket] += current->num_blocks;
}
// Calculate the block array buffer size
uint32_t total_block_array_entries = 0;
for (uint32_t i = 0; i < num_buckets; i++) {
uint32_t num_blocks = num_blocks_per_bucket[i];
if (num_blocks > 1) {
total_block_array_entries += (num_blocks + 1);
}
}
// Populate the final prefix block index
uint32_t* block_array_buffer = new uint32_t[total_block_array_entries];
uint32_t* buckets = new uint32_t[num_buckets];
uint32_t offset = 0;
for (uint32_t i = 0; i < num_buckets; i++) {
uint32_t num_blocks = num_blocks_per_bucket[i];
if (num_blocks == 0) {
assert(prefixes_per_bucket[i] == nullptr);
buckets[i] = kNoneBlock;
} else if (num_blocks == 1) {
assert(prefixes_per_bucket[i] != nullptr);
assert(prefixes_per_bucket[i]->next == nullptr);
buckets[i] = prefixes_per_bucket[i]->start_block;
} else {
assert(prefixes_per_bucket[i] != nullptr);
buckets[i] = EncodeIndex(offset);
block_array_buffer[offset] = num_blocks;
uint32_t* last_block = &block_array_buffer[offset + num_blocks];
auto current = prefixes_per_bucket[i];
// populate block ids from largest to smallest
while (current != nullptr) {
for (uint32_t i = 0; i < current->num_blocks; i++) {
*last_block = current->end_block - i;
last_block--;
}
current = current->next;
}
assert(last_block == &block_array_buffer[offset]);
offset += (num_blocks + 1);
}
}
assert(offset == total_block_array_entries);
return new BlockPrefixIndex(internal_prefix_extractor_, num_buckets,
buckets, total_block_array_entries,
block_array_buffer);
}
private:
const SliceTransform* internal_prefix_extractor_;
std::vector<PrefixRecord*> prefixes_;
Arena arena_;
};
Status BlockPrefixIndex::Create(const SliceTransform* internal_prefix_extractor,
const Slice& prefixes, const Slice& prefix_meta,
BlockPrefixIndex** prefix_index) {
uint64_t pos = 0;
auto meta_pos = prefix_meta;
Status s;
Builder builder(internal_prefix_extractor);
while (!meta_pos.empty()) {
uint32_t prefix_size = 0;
uint32_t entry_index = 0;
uint32_t num_blocks = 0;
if (!GetVarint32(&meta_pos, &prefix_size) ||
!GetVarint32(&meta_pos, &entry_index) ||
!GetVarint32(&meta_pos, &num_blocks)) {
s = Status::Corruption(
"Corrupted prefix meta block: unable to read from it.");
break;
}
if (pos + prefix_size > prefixes.size()) {
s = Status::Corruption(
"Corrupted prefix meta block: size inconsistency.");
break;
}
Slice prefix(prefixes.data() + pos, prefix_size);
builder.Add(prefix, entry_index, num_blocks);
pos += prefix_size;
}
if (s.ok() && pos != prefixes.size()) {
s = Status::Corruption("Corrupted prefix meta block");
}
if (s.ok()) {
*prefix_index = builder.Finish();
}
return s;
}
const uint32_t BlockPrefixIndex::GetBlocks(const Slice& key,
uint32_t** blocks) {
Slice prefix = internal_prefix_extractor_->Transform(key);
uint32_t bucket = PrefixToBucket(prefix, num_buckets_);
uint32_t block_id = buckets_[bucket];
if (IsNone(block_id)) {
return 0;
} else if (IsBlockId(block_id)) {
*blocks = &buckets_[bucket];
return 1;
} else {
uint32_t index = DecodeIndex(block_id);
assert(index < num_block_array_buffer_entries_);
*blocks = &block_array_buffer_[index+1];
uint32_t num_blocks = block_array_buffer_[index];
assert(num_blocks > 1);
assert(index + num_blocks < num_block_array_buffer_entries_);
return num_blocks;
}
}
} // namespace rocksdb

View File

@ -0,0 +1,67 @@
// Copyright (c) 2013, Facebook, Inc. All rights reserved.
// This source code is licensed under the BSD-style license found in the
// 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.
#pragma once
#include "rocksdb/status.h"
namespace rocksdb {
class Comparator;
class Iterator;
class Slice;
class SliceTransform;
// Build a hash-based index to speed up the lookup for "index block".
// BlockHashIndex accepts a key and, if found, returns its restart index within
// that index block.
class BlockPrefixIndex {
public:
// Maps a key to a list of data blocks that could potentially contain
// the key, based on the prefix.
// Returns the total number of relevant blocks, 0 means the key does
// not exist.
const uint32_t GetBlocks(const Slice& key, uint32_t** blocks);
size_t ApproximateMemoryUsage() const {
return sizeof(BlockPrefixIndex) +
(num_block_array_buffer_entries_ + num_buckets_) * sizeof(uint32_t);
}
// Create hash index by reading from the metadata blocks.
// @params prefixes: a sequence of prefixes.
// @params prefix_meta: contains the "metadata" to of the prefixes.
static Status Create(const SliceTransform* hash_key_extractor,
const Slice& prefixes, const Slice& prefix_meta,
BlockPrefixIndex** prefix_index);
~BlockPrefixIndex() {
delete[] buckets_;
delete[] block_array_buffer_;
}
private:
class Builder;
friend Builder;
BlockPrefixIndex(const SliceTransform* internal_prefix_extractor,
uint32_t num_buckets,
uint32_t* buckets,
uint32_t num_block_array_buffer_entries,
uint32_t* block_array_buffer)
: internal_prefix_extractor_(internal_prefix_extractor),
num_buckets_(num_buckets),
num_block_array_buffer_entries_(num_block_array_buffer_entries),
buckets_(buckets),
block_array_buffer_(block_array_buffer) {}
const SliceTransform* internal_prefix_extractor_;
uint32_t num_buckets_;
uint32_t num_block_array_buffer_entries_;
uint32_t* buckets_;
uint32_t* block_array_buffer_;
};
} // namespace rocksdb

View File

@ -11,6 +11,7 @@
#include <stdio.h>
#include <algorithm>
#include <iostream>
#include <map>
#include <string>
#include <memory>
@ -1086,6 +1087,7 @@ TEST(TableTest, HashIndexTest) {
Options options;
BlockBasedTableOptions table_options;
table_options.index_type = BlockBasedTableOptions::kHashSearch;
table_options.hash_index_allow_collision = true;
options.table_factory.reset(new BlockBasedTableFactory(table_options));
options.prefix_extractor.reset(NewFixedPrefixTransform(3));
@ -1160,7 +1162,13 @@ TEST(TableTest, HashIndexTest) {
// regular_iter->Seek(prefix);
ASSERT_OK(hash_iter->status());
ASSERT_TRUE(!hash_iter->Valid());
// Seek to non-existing prefixes should yield either invalid, or a
// key with prefix greater than the target.
if (hash_iter->Valid()) {
Slice ukey = ExtractUserKey(hash_iter->key());
Slice ukey_prefix = options.prefix_extractor->Transform(ukey);
ASSERT_TRUE(BytewiseComparator()->Compare(prefix, ukey_prefix) < 0);
}
}
}