summaryrefslogtreecommitdiffstats
path: root/src/rocksdb/table
diff options
context:
space:
mode:
Diffstat (limited to 'src/rocksdb/table')
-rw-r--r--src/rocksdb/table/adaptive/adaptive_table_factory.cc126
-rw-r--r--src/rocksdb/table/adaptive/adaptive_table_factory.h58
-rw-r--r--src/rocksdb/table/block_based/binary_search_index_reader.cc74
-rw-r--r--src/rocksdb/table/block_based/binary_search_index_reader.h48
-rw-r--r--src/rocksdb/table/block_based/block.cc1131
-rw-r--r--src/rocksdb/table/block_based/block.h744
-rw-r--r--src/rocksdb/table/block_based/block_based_table_builder.cc2096
-rw-r--r--src/rocksdb/table/block_based/block_based_table_builder.h203
-rw-r--r--src/rocksdb/table/block_based/block_based_table_factory.cc1058
-rw-r--r--src/rocksdb/table/block_based/block_based_table_factory.h101
-rw-r--r--src/rocksdb/table/block_based/block_based_table_iterator.cc459
-rw-r--r--src/rocksdb/table/block_based/block_based_table_iterator.h280
-rw-r--r--src/rocksdb/table/block_based/block_based_table_reader.cc3092
-rw-r--r--src/rocksdb/table/block_based/block_based_table_reader.h739
-rw-r--r--src/rocksdb/table/block_based/block_based_table_reader_impl.h171
-rw-r--r--src/rocksdb/table/block_based/block_based_table_reader_sync_and_async.h760
-rw-r--r--src/rocksdb/table/block_based/block_based_table_reader_test.cc572
-rw-r--r--src/rocksdb/table/block_based/block_builder.cc234
-rw-r--r--src/rocksdb/table/block_based/block_builder.h102
-rw-r--r--src/rocksdb/table/block_based/block_like_traits.h182
-rw-r--r--src/rocksdb/table/block_based/block_prefetcher.cc120
-rw-r--r--src/rocksdb/table/block_based/block_prefetcher.h72
-rw-r--r--src/rocksdb/table/block_based/block_prefix_index.cc226
-rw-r--r--src/rocksdb/table/block_based/block_prefix_index.h70
-rw-r--r--src/rocksdb/table/block_based/block_test.cc627
-rw-r--r--src/rocksdb/table/block_based/block_type.h34
-rw-r--r--src/rocksdb/table/block_based/cachable_entry.h232
-rw-r--r--src/rocksdb/table/block_based/data_block_footer.cc59
-rw-r--r--src/rocksdb/table/block_based/data_block_footer.h25
-rw-r--r--src/rocksdb/table/block_based/data_block_hash_index.cc94
-rw-r--r--src/rocksdb/table/block_based/data_block_hash_index.h137
-rw-r--r--src/rocksdb/table/block_based/data_block_hash_index_test.cc717
-rw-r--r--src/rocksdb/table/block_based/filter_block.h182
-rw-r--r--src/rocksdb/table/block_based/filter_block_reader_common.cc164
-rw-r--r--src/rocksdb/table/block_based/filter_block_reader_common.h79
-rw-r--r--src/rocksdb/table/block_based/filter_policy.cc1973
-rw-r--r--src/rocksdb/table/block_based/filter_policy_internal.h340
-rw-r--r--src/rocksdb/table/block_based/flush_block_policy.cc146
-rw-r--r--src/rocksdb/table/block_based/flush_block_policy.h40
-rw-r--r--src/rocksdb/table/block_based/full_filter_block.cc296
-rw-r--r--src/rocksdb/table/block_based/full_filter_block.h147
-rw-r--r--src/rocksdb/table/block_based/full_filter_block_test.cc339
-rw-r--r--src/rocksdb/table/block_based/hash_index_reader.cc148
-rw-r--r--src/rocksdb/table/block_based/hash_index_reader.h49
-rw-r--r--src/rocksdb/table/block_based/index_builder.cc282
-rw-r--r--src/rocksdb/table/block_based/index_builder.h455
-rw-r--r--src/rocksdb/table/block_based/index_reader_common.cc56
-rw-r--r--src/rocksdb/table/block_based/index_reader_common.h85
-rw-r--r--src/rocksdb/table/block_based/mock_block_based_table.h62
-rw-r--r--src/rocksdb/table/block_based/parsed_full_filter_block.cc23
-rw-r--r--src/rocksdb/table/block_based/parsed_full_filter_block.h42
-rw-r--r--src/rocksdb/table/block_based/partitioned_filter_block.cc561
-rw-r--r--src/rocksdb/table/block_based/partitioned_filter_block.h178
-rw-r--r--src/rocksdb/table/block_based/partitioned_filter_block_test.cc436
-rw-r--r--src/rocksdb/table/block_based/partitioned_index_iterator.cc163
-rw-r--r--src/rocksdb/table/block_based/partitioned_index_iterator.h160
-rw-r--r--src/rocksdb/table/block_based/partitioned_index_reader.cc215
-rw-r--r--src/rocksdb/table/block_based/partitioned_index_reader.h55
-rw-r--r--src/rocksdb/table/block_based/reader_common.cc52
-rw-r--r--src/rocksdb/table/block_based/reader_common.h38
-rw-r--r--src/rocksdb/table/block_based/uncompression_dict_reader.cc124
-rw-r--r--src/rocksdb/table/block_based/uncompression_dict_reader.h60
-rw-r--r--src/rocksdb/table/block_fetcher.cc399
-rw-r--r--src/rocksdb/table/block_fetcher.h142
-rw-r--r--src/rocksdb/table/block_fetcher_test.cc521
-rw-r--r--src/rocksdb/table/cleanable_test.cc390
-rw-r--r--src/rocksdb/table/cuckoo/cuckoo_table_builder.cc553
-rw-r--r--src/rocksdb/table/cuckoo/cuckoo_table_builder.h138
-rw-r--r--src/rocksdb/table/cuckoo/cuckoo_table_builder_test.cc640
-rw-r--r--src/rocksdb/table/cuckoo/cuckoo_table_factory.cc104
-rw-r--r--src/rocksdb/table/cuckoo/cuckoo_table_factory.h82
-rw-r--r--src/rocksdb/table/cuckoo/cuckoo_table_reader.cc411
-rw-r--r--src/rocksdb/table/cuckoo/cuckoo_table_reader.h100
-rw-r--r--src/rocksdb/table/cuckoo/cuckoo_table_reader_test.cc584
-rw-r--r--src/rocksdb/table/format.cc575
-rw-r--r--src/rocksdb/table/format.h375
-rw-r--r--src/rocksdb/table/get_context.cc604
-rw-r--r--src/rocksdb/table/get_context.h231
-rw-r--r--src/rocksdb/table/internal_iterator.h226
-rw-r--r--src/rocksdb/table/iter_heap.h44
-rw-r--r--src/rocksdb/table/iterator.cc130
-rw-r--r--src/rocksdb/table/iterator_wrapper.h190
-rw-r--r--src/rocksdb/table/merger_test.cc182
-rw-r--r--src/rocksdb/table/merging_iterator.cc1403
-rw-r--r--src/rocksdb/table/merging_iterator.h92
-rw-r--r--src/rocksdb/table/meta_blocks.cc553
-rw-r--r--src/rocksdb/table/meta_blocks.h168
-rw-r--r--src/rocksdb/table/mock_table.cc344
-rw-r--r--src/rocksdb/table/mock_table.h94
-rw-r--r--src/rocksdb/table/multiget_context.h402
-rw-r--r--src/rocksdb/table/persistent_cache_helper.cc111
-rw-r--r--src/rocksdb/table/persistent_cache_helper.h46
-rw-r--r--src/rocksdb/table/persistent_cache_options.h34
-rw-r--r--src/rocksdb/table/plain/plain_table_bloom.cc78
-rw-r--r--src/rocksdb/table/plain/plain_table_bloom.h132
-rw-r--r--src/rocksdb/table/plain/plain_table_builder.cc337
-rw-r--r--src/rocksdb/table/plain/plain_table_builder.h154
-rw-r--r--src/rocksdb/table/plain/plain_table_factory.cc350
-rw-r--r--src/rocksdb/table/plain/plain_table_factory.h182
-rw-r--r--src/rocksdb/table/plain/plain_table_index.cc213
-rw-r--r--src/rocksdb/table/plain/plain_table_index.h248
-rw-r--r--src/rocksdb/table/plain/plain_table_key_coding.cc509
-rw-r--r--src/rocksdb/table/plain/plain_table_key_coding.h201
-rw-r--r--src/rocksdb/table/plain/plain_table_reader.cc765
-rw-r--r--src/rocksdb/table/plain/plain_table_reader.h244
-rw-r--r--src/rocksdb/table/scoped_arena_iterator.h57
-rw-r--r--src/rocksdb/table/sst_file_dumper.cc519
-rw-r--r--src/rocksdb/table/sst_file_dumper.h101
-rw-r--r--src/rocksdb/table/sst_file_reader.cc101
-rw-r--r--src/rocksdb/table/sst_file_reader_test.cc434
-rw-r--r--src/rocksdb/table/sst_file_writer.cc427
-rw-r--r--src/rocksdb/table/sst_file_writer_collectors.h95
-rw-r--r--src/rocksdb/table/table_builder.h219
-rw-r--r--src/rocksdb/table/table_factory.cc65
-rw-r--r--src/rocksdb/table/table_properties.cc349
-rw-r--r--src/rocksdb/table/table_properties_internal.h14
-rw-r--r--src/rocksdb/table/table_reader.h184
-rw-r--r--src/rocksdb/table/table_reader_bench.cc349
-rw-r--r--src/rocksdb/table/table_test.cc5596
-rw-r--r--src/rocksdb/table/two_level_iterator.cc220
-rw-r--r--src/rocksdb/table/two_level_iterator.h43
-rw-r--r--src/rocksdb/table/unique_id.cc223
-rw-r--r--src/rocksdb/table/unique_id_impl.h93
123 files changed, 43963 insertions, 0 deletions
diff --git a/src/rocksdb/table/adaptive/adaptive_table_factory.cc b/src/rocksdb/table/adaptive/adaptive_table_factory.cc
new file mode 100644
index 000000000..5b9fe3dbd
--- /dev/null
+++ b/src/rocksdb/table/adaptive/adaptive_table_factory.cc
@@ -0,0 +1,126 @@
+// Copyright (c) Facebook, Inc. and its affiliates. All Rights Reserved.
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#ifndef ROCKSDB_LITE
+#include "table/adaptive/adaptive_table_factory.h"
+
+#include "port/port.h"
+#include "table/format.h"
+#include "table/table_builder.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+AdaptiveTableFactory::AdaptiveTableFactory(
+ std::shared_ptr<TableFactory> table_factory_to_write,
+ std::shared_ptr<TableFactory> block_based_table_factory,
+ std::shared_ptr<TableFactory> plain_table_factory,
+ std::shared_ptr<TableFactory> cuckoo_table_factory)
+ : table_factory_to_write_(table_factory_to_write),
+ block_based_table_factory_(block_based_table_factory),
+ plain_table_factory_(plain_table_factory),
+ cuckoo_table_factory_(cuckoo_table_factory) {
+ if (!plain_table_factory_) {
+ plain_table_factory_.reset(NewPlainTableFactory());
+ }
+ if (!block_based_table_factory_) {
+ block_based_table_factory_.reset(NewBlockBasedTableFactory());
+ }
+ if (!cuckoo_table_factory_) {
+ cuckoo_table_factory_.reset(NewCuckooTableFactory());
+ }
+ if (!table_factory_to_write_) {
+ table_factory_to_write_ = block_based_table_factory_;
+ }
+}
+
+extern const uint64_t kPlainTableMagicNumber;
+extern const uint64_t kLegacyPlainTableMagicNumber;
+extern const uint64_t kBlockBasedTableMagicNumber;
+extern const uint64_t kLegacyBlockBasedTableMagicNumber;
+extern const uint64_t kCuckooTableMagicNumber;
+
+Status AdaptiveTableFactory::NewTableReader(
+ const ReadOptions& ro, const TableReaderOptions& table_reader_options,
+ std::unique_ptr<RandomAccessFileReader>&& file, uint64_t file_size,
+ std::unique_ptr<TableReader>* table,
+ bool prefetch_index_and_filter_in_cache) const {
+ Footer footer;
+ IOOptions opts;
+ auto s = ReadFooterFromFile(opts, file.get(), nullptr /* prefetch_buffer */,
+ file_size, &footer);
+ if (!s.ok()) {
+ return s;
+ }
+ if (footer.table_magic_number() == kPlainTableMagicNumber ||
+ footer.table_magic_number() == kLegacyPlainTableMagicNumber) {
+ return plain_table_factory_->NewTableReader(
+ table_reader_options, std::move(file), file_size, table);
+ } else if (footer.table_magic_number() == kBlockBasedTableMagicNumber ||
+ footer.table_magic_number() == kLegacyBlockBasedTableMagicNumber) {
+ return block_based_table_factory_->NewTableReader(
+ ro, table_reader_options, std::move(file), file_size, table,
+ prefetch_index_and_filter_in_cache);
+ } else if (footer.table_magic_number() == kCuckooTableMagicNumber) {
+ return cuckoo_table_factory_->NewTableReader(
+ table_reader_options, std::move(file), file_size, table);
+ } else {
+ return Status::NotSupported("Unidentified table format");
+ }
+}
+
+TableBuilder* AdaptiveTableFactory::NewTableBuilder(
+ const TableBuilderOptions& table_builder_options,
+ WritableFileWriter* file) const {
+ return table_factory_to_write_->NewTableBuilder(table_builder_options, file);
+}
+
+std::string AdaptiveTableFactory::GetPrintableOptions() const {
+ std::string ret;
+ ret.reserve(20000);
+ const int kBufferSize = 200;
+ char buffer[kBufferSize];
+
+ if (table_factory_to_write_) {
+ snprintf(buffer, kBufferSize, " write factory (%s) options:\n%s\n",
+ (table_factory_to_write_->Name() ? table_factory_to_write_->Name()
+ : ""),
+ table_factory_to_write_->GetPrintableOptions().c_str());
+ ret.append(buffer);
+ }
+ if (plain_table_factory_) {
+ snprintf(buffer, kBufferSize, " %s options:\n%s\n",
+ plain_table_factory_->Name() ? plain_table_factory_->Name() : "",
+ plain_table_factory_->GetPrintableOptions().c_str());
+ ret.append(buffer);
+ }
+ if (block_based_table_factory_) {
+ snprintf(
+ buffer, kBufferSize, " %s options:\n%s\n",
+ (block_based_table_factory_->Name() ? block_based_table_factory_->Name()
+ : ""),
+ block_based_table_factory_->GetPrintableOptions().c_str());
+ ret.append(buffer);
+ }
+ if (cuckoo_table_factory_) {
+ snprintf(buffer, kBufferSize, " %s options:\n%s\n",
+ cuckoo_table_factory_->Name() ? cuckoo_table_factory_->Name() : "",
+ cuckoo_table_factory_->GetPrintableOptions().c_str());
+ ret.append(buffer);
+ }
+ return ret;
+}
+
+extern TableFactory* NewAdaptiveTableFactory(
+ std::shared_ptr<TableFactory> table_factory_to_write,
+ std::shared_ptr<TableFactory> block_based_table_factory,
+ std::shared_ptr<TableFactory> plain_table_factory,
+ std::shared_ptr<TableFactory> cuckoo_table_factory) {
+ return new AdaptiveTableFactory(table_factory_to_write,
+ block_based_table_factory,
+ plain_table_factory, cuckoo_table_factory);
+}
+
+} // namespace ROCKSDB_NAMESPACE
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/table/adaptive/adaptive_table_factory.h b/src/rocksdb/table/adaptive/adaptive_table_factory.h
new file mode 100644
index 000000000..3b631942d
--- /dev/null
+++ b/src/rocksdb/table/adaptive/adaptive_table_factory.h
@@ -0,0 +1,58 @@
+// Copyright (c) Facebook, Inc. and its affiliates. All Rights Reserved.
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#pragma once
+
+#ifndef ROCKSDB_LITE
+
+#include <string>
+
+#include "rocksdb/options.h"
+#include "rocksdb/table.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+struct EnvOptions;
+
+class Status;
+class RandomAccessFile;
+class WritableFile;
+class Table;
+class TableBuilder;
+
+class AdaptiveTableFactory : public TableFactory {
+ public:
+ ~AdaptiveTableFactory() {}
+
+ explicit AdaptiveTableFactory(
+ std::shared_ptr<TableFactory> table_factory_to_write,
+ std::shared_ptr<TableFactory> block_based_table_factory,
+ std::shared_ptr<TableFactory> plain_table_factory,
+ std::shared_ptr<TableFactory> cuckoo_table_factory);
+
+ const char* Name() const override { return "AdaptiveTableFactory"; }
+
+ using TableFactory::NewTableReader;
+ Status NewTableReader(
+ const ReadOptions& ro, const TableReaderOptions& table_reader_options,
+ std::unique_ptr<RandomAccessFileReader>&& file, uint64_t file_size,
+ std::unique_ptr<TableReader>* table,
+ bool prefetch_index_and_filter_in_cache = true) const override;
+
+ TableBuilder* NewTableBuilder(
+ const TableBuilderOptions& table_builder_options,
+ WritableFileWriter* file) const override;
+
+ std::string GetPrintableOptions() const override;
+
+ private:
+ std::shared_ptr<TableFactory> table_factory_to_write_;
+ std::shared_ptr<TableFactory> block_based_table_factory_;
+ std::shared_ptr<TableFactory> plain_table_factory_;
+ std::shared_ptr<TableFactory> cuckoo_table_factory_;
+};
+
+} // namespace ROCKSDB_NAMESPACE
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/table/block_based/binary_search_index_reader.cc b/src/rocksdb/table/block_based/binary_search_index_reader.cc
new file mode 100644
index 000000000..21787cc1a
--- /dev/null
+++ b/src/rocksdb/table/block_based/binary_search_index_reader.cc
@@ -0,0 +1,74 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+#include "table/block_based/binary_search_index_reader.h"
+
+namespace ROCKSDB_NAMESPACE {
+Status BinarySearchIndexReader::Create(
+ const BlockBasedTable* table, const ReadOptions& ro,
+ FilePrefetchBuffer* prefetch_buffer, bool use_cache, bool prefetch,
+ bool pin, BlockCacheLookupContext* lookup_context,
+ std::unique_ptr<IndexReader>* index_reader) {
+ assert(table != nullptr);
+ assert(table->get_rep());
+ assert(!pin || prefetch);
+ assert(index_reader != nullptr);
+
+ CachableEntry<Block> index_block;
+ if (prefetch || !use_cache) {
+ const Status s =
+ ReadIndexBlock(table, prefetch_buffer, ro, use_cache,
+ /*get_context=*/nullptr, lookup_context, &index_block);
+ if (!s.ok()) {
+ return s;
+ }
+
+ if (use_cache && !pin) {
+ index_block.Reset();
+ }
+ }
+
+ index_reader->reset(
+ new BinarySearchIndexReader(table, std::move(index_block)));
+
+ return Status::OK();
+}
+
+InternalIteratorBase<IndexValue>* BinarySearchIndexReader::NewIterator(
+ const ReadOptions& read_options, bool /* disable_prefix_seek */,
+ IndexBlockIter* iter, GetContext* get_context,
+ BlockCacheLookupContext* lookup_context) {
+ const BlockBasedTable::Rep* rep = table()->get_rep();
+ const bool no_io = (read_options.read_tier == kBlockCacheTier);
+ CachableEntry<Block> index_block;
+ const Status s =
+ GetOrReadIndexBlock(no_io, read_options.rate_limiter_priority,
+ get_context, lookup_context, &index_block);
+ if (!s.ok()) {
+ if (iter != nullptr) {
+ iter->Invalidate(s);
+ return iter;
+ }
+
+ return NewErrorInternalIterator<IndexValue>(s);
+ }
+
+ Statistics* kNullStats = nullptr;
+ // We don't return pinned data from index blocks, so no need
+ // to set `block_contents_pinned`.
+ auto it = index_block.GetValue()->NewIndexIterator(
+ internal_comparator()->user_comparator(),
+ rep->get_global_seqno(BlockType::kIndex), iter, kNullStats, true,
+ index_has_first_key(), index_key_includes_seq(), index_value_is_full());
+
+ assert(it != nullptr);
+ index_block.TransferTo(it);
+
+ return it;
+}
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/binary_search_index_reader.h b/src/rocksdb/table/block_based/binary_search_index_reader.h
new file mode 100644
index 000000000..d4a611ecc
--- /dev/null
+++ b/src/rocksdb/table/block_based/binary_search_index_reader.h
@@ -0,0 +1,48 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+#pragma once
+#include "table/block_based/index_reader_common.h"
+
+namespace ROCKSDB_NAMESPACE {
+// Index that allows binary search lookup for the first key of each block.
+// This class can be viewed as a thin wrapper for `Block` class which already
+// supports binary search.
+class BinarySearchIndexReader : public BlockBasedTable::IndexReaderCommon {
+ public:
+ // Read index from the file and create an intance for
+ // `BinarySearchIndexReader`.
+ // On success, index_reader will be populated; otherwise it will remain
+ // unmodified.
+ static Status Create(const BlockBasedTable* table, const ReadOptions& ro,
+ FilePrefetchBuffer* prefetch_buffer, bool use_cache,
+ bool prefetch, bool pin,
+ BlockCacheLookupContext* lookup_context,
+ std::unique_ptr<IndexReader>* index_reader);
+
+ InternalIteratorBase<IndexValue>* NewIterator(
+ const ReadOptions& read_options, bool /* disable_prefix_seek */,
+ IndexBlockIter* iter, GetContext* get_context,
+ BlockCacheLookupContext* lookup_context) override;
+
+ size_t ApproximateMemoryUsage() const override {
+ size_t usage = ApproximateIndexBlockMemoryUsage();
+#ifdef ROCKSDB_MALLOC_USABLE_SIZE
+ usage += malloc_usable_size(const_cast<BinarySearchIndexReader*>(this));
+#else
+ usage += sizeof(*this);
+#endif // ROCKSDB_MALLOC_USABLE_SIZE
+ return usage;
+ }
+
+ private:
+ BinarySearchIndexReader(const BlockBasedTable* t,
+ CachableEntry<Block>&& index_block)
+ : IndexReaderCommon(t, std::move(index_block)) {}
+};
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/block.cc b/src/rocksdb/table/block_based/block.cc
new file mode 100644
index 000000000..7eb0b010f
--- /dev/null
+++ b/src/rocksdb/table/block_based/block.cc
@@ -0,0 +1,1131 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+//
+// Decodes the blocks generated by block_builder.cc.
+
+#include "table/block_based/block.h"
+
+#include <algorithm>
+#include <string>
+#include <unordered_map>
+#include <vector>
+
+#include "monitoring/perf_context_imp.h"
+#include "port/port.h"
+#include "port/stack_trace.h"
+#include "rocksdb/comparator.h"
+#include "table/block_based/block_prefix_index.h"
+#include "table/block_based/data_block_footer.h"
+#include "table/format.h"
+#include "util/coding.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+// Helper routine: decode the next block entry starting at "p",
+// storing the number of shared key bytes, non_shared key bytes,
+// and the length of the value in "*shared", "*non_shared", and
+// "*value_length", respectively. Will not derefence past "limit".
+//
+// If any errors are detected, returns nullptr. Otherwise, returns a
+// pointer to the key delta (just past the three decoded values).
+struct DecodeEntry {
+ inline const char* operator()(const char* p, const char* limit,
+ uint32_t* shared, uint32_t* non_shared,
+ uint32_t* value_length) {
+ // We need 2 bytes for shared and non_shared size. We also need one more
+ // byte either for value size or the actual value in case of value delta
+ // encoding.
+ assert(limit - p >= 3);
+ *shared = reinterpret_cast<const unsigned char*>(p)[0];
+ *non_shared = reinterpret_cast<const unsigned char*>(p)[1];
+ *value_length = reinterpret_cast<const unsigned char*>(p)[2];
+ if ((*shared | *non_shared | *value_length) < 128) {
+ // Fast path: all three values are encoded in one byte each
+ p += 3;
+ } else {
+ if ((p = GetVarint32Ptr(p, limit, shared)) == nullptr) return nullptr;
+ if ((p = GetVarint32Ptr(p, limit, non_shared)) == nullptr) return nullptr;
+ if ((p = GetVarint32Ptr(p, limit, value_length)) == nullptr) {
+ return nullptr;
+ }
+ }
+
+ // Using an assert in place of "return null" since we should not pay the
+ // cost of checking for corruption on every single key decoding
+ assert(!(static_cast<uint32_t>(limit - p) < (*non_shared + *value_length)));
+ return p;
+ }
+};
+
+// Helper routine: similar to DecodeEntry but does not have assertions.
+// Instead, returns nullptr so that caller can detect and report failure.
+struct CheckAndDecodeEntry {
+ inline const char* operator()(const char* p, const char* limit,
+ uint32_t* shared, uint32_t* non_shared,
+ uint32_t* value_length) {
+ // We need 2 bytes for shared and non_shared size. We also need one more
+ // byte either for value size or the actual value in case of value delta
+ // encoding.
+ if (limit - p < 3) {
+ return nullptr;
+ }
+ *shared = reinterpret_cast<const unsigned char*>(p)[0];
+ *non_shared = reinterpret_cast<const unsigned char*>(p)[1];
+ *value_length = reinterpret_cast<const unsigned char*>(p)[2];
+ if ((*shared | *non_shared | *value_length) < 128) {
+ // Fast path: all three values are encoded in one byte each
+ p += 3;
+ } else {
+ if ((p = GetVarint32Ptr(p, limit, shared)) == nullptr) return nullptr;
+ if ((p = GetVarint32Ptr(p, limit, non_shared)) == nullptr) return nullptr;
+ if ((p = GetVarint32Ptr(p, limit, value_length)) == nullptr) {
+ return nullptr;
+ }
+ }
+
+ if (static_cast<uint32_t>(limit - p) < (*non_shared + *value_length)) {
+ return nullptr;
+ }
+ return p;
+ }
+};
+
+struct DecodeKey {
+ inline const char* operator()(const char* p, const char* limit,
+ uint32_t* shared, uint32_t* non_shared) {
+ uint32_t value_length;
+ return DecodeEntry()(p, limit, shared, non_shared, &value_length);
+ }
+};
+
+// In format_version 4, which is used by index blocks, the value size is not
+// encoded before the entry, as the value is known to be the handle with the
+// known size.
+struct DecodeKeyV4 {
+ inline const char* operator()(const char* p, const char* limit,
+ uint32_t* shared, uint32_t* non_shared) {
+ // We need 2 bytes for shared and non_shared size. We also need one more
+ // byte either for value size or the actual value in case of value delta
+ // encoding.
+ if (limit - p < 3) return nullptr;
+ *shared = reinterpret_cast<const unsigned char*>(p)[0];
+ *non_shared = reinterpret_cast<const unsigned char*>(p)[1];
+ if ((*shared | *non_shared) < 128) {
+ // Fast path: all three values are encoded in one byte each
+ p += 2;
+ } else {
+ if ((p = GetVarint32Ptr(p, limit, shared)) == nullptr) return nullptr;
+ if ((p = GetVarint32Ptr(p, limit, non_shared)) == nullptr) return nullptr;
+ }
+ return p;
+ }
+};
+
+struct DecodeEntryV4 {
+ inline const char* operator()(const char* p, const char* limit,
+ uint32_t* shared, uint32_t* non_shared,
+ uint32_t* value_length) {
+ assert(value_length);
+
+ *value_length = 0;
+ return DecodeKeyV4()(p, limit, shared, non_shared);
+ }
+};
+void DataBlockIter::NextImpl() {
+ bool is_shared = false;
+ ParseNextDataKey(&is_shared);
+}
+
+void MetaBlockIter::NextImpl() {
+ bool is_shared = false;
+ ParseNextKey<CheckAndDecodeEntry>(&is_shared);
+}
+
+void IndexBlockIter::NextImpl() { ParseNextIndexKey(); }
+
+void IndexBlockIter::PrevImpl() {
+ assert(Valid());
+ // Scan backwards to a restart point before current_
+ const uint32_t original = current_;
+ while (GetRestartPoint(restart_index_) >= original) {
+ if (restart_index_ == 0) {
+ // No more entries
+ current_ = restarts_;
+ restart_index_ = num_restarts_;
+ return;
+ }
+ restart_index_--;
+ }
+ SeekToRestartPoint(restart_index_);
+ // Loop until end of current entry hits the start of original entry
+ while (ParseNextIndexKey() && NextEntryOffset() < original) {
+ }
+}
+
+void MetaBlockIter::PrevImpl() {
+ assert(Valid());
+ // Scan backwards to a restart point before current_
+ const uint32_t original = current_;
+ while (GetRestartPoint(restart_index_) >= original) {
+ if (restart_index_ == 0) {
+ // No more entries
+ current_ = restarts_;
+ restart_index_ = num_restarts_;
+ return;
+ }
+ restart_index_--;
+ }
+ SeekToRestartPoint(restart_index_);
+ bool is_shared = false;
+ // Loop until end of current entry hits the start of original entry
+ while (ParseNextKey<CheckAndDecodeEntry>(&is_shared) &&
+ NextEntryOffset() < original) {
+ }
+}
+
+// Similar to IndexBlockIter::PrevImpl but also caches the prev entries
+void DataBlockIter::PrevImpl() {
+ assert(Valid());
+
+ assert(prev_entries_idx_ == -1 ||
+ static_cast<size_t>(prev_entries_idx_) < prev_entries_.size());
+ // Check if we can use cached prev_entries_
+ if (prev_entries_idx_ > 0 &&
+ prev_entries_[prev_entries_idx_].offset == current_) {
+ // Read cached CachedPrevEntry
+ prev_entries_idx_--;
+ const CachedPrevEntry& current_prev_entry =
+ prev_entries_[prev_entries_idx_];
+
+ const char* key_ptr = nullptr;
+ bool raw_key_cached;
+ if (current_prev_entry.key_ptr != nullptr) {
+ // The key is not delta encoded and stored in the data block
+ key_ptr = current_prev_entry.key_ptr;
+ raw_key_cached = false;
+ } else {
+ // The key is delta encoded and stored in prev_entries_keys_buff_
+ key_ptr = prev_entries_keys_buff_.data() + current_prev_entry.key_offset;
+ raw_key_cached = true;
+ }
+ const Slice current_key(key_ptr, current_prev_entry.key_size);
+
+ current_ = current_prev_entry.offset;
+ // TODO(ajkr): the copy when `raw_key_cached` is done here for convenience,
+ // not necessity. It is convenient since this class treats keys as pinned
+ // when `raw_key_` points to an outside buffer. So we cannot allow
+ // `raw_key_` point into Prev cache as it is a transient outside buffer
+ // (i.e., keys in it are not actually pinned).
+ raw_key_.SetKey(current_key, raw_key_cached /* copy */);
+ value_ = current_prev_entry.value;
+
+ return;
+ }
+
+ // Clear prev entries cache
+ prev_entries_idx_ = -1;
+ prev_entries_.clear();
+ prev_entries_keys_buff_.clear();
+
+ // Scan backwards to a restart point before current_
+ const uint32_t original = current_;
+ while (GetRestartPoint(restart_index_) >= original) {
+ if (restart_index_ == 0) {
+ // No more entries
+ current_ = restarts_;
+ restart_index_ = num_restarts_;
+ return;
+ }
+ restart_index_--;
+ }
+
+ SeekToRestartPoint(restart_index_);
+
+ do {
+ bool is_shared = false;
+ if (!ParseNextDataKey(&is_shared)) {
+ break;
+ }
+ Slice current_key = raw_key_.GetKey();
+
+ if (raw_key_.IsKeyPinned()) {
+ // The key is not delta encoded
+ prev_entries_.emplace_back(current_, current_key.data(), 0,
+ current_key.size(), value());
+ } else {
+ // The key is delta encoded, cache decoded key in buffer
+ size_t new_key_offset = prev_entries_keys_buff_.size();
+ prev_entries_keys_buff_.append(current_key.data(), current_key.size());
+
+ prev_entries_.emplace_back(current_, nullptr, new_key_offset,
+ current_key.size(), value());
+ }
+ // Loop until end of current entry hits the start of original entry
+ } while (NextEntryOffset() < original);
+ prev_entries_idx_ = static_cast<int32_t>(prev_entries_.size()) - 1;
+}
+
+void DataBlockIter::SeekImpl(const Slice& target) {
+ Slice seek_key = target;
+ PERF_TIMER_GUARD(block_seek_nanos);
+ if (data_ == nullptr) { // Not init yet
+ return;
+ }
+ uint32_t index = 0;
+ bool skip_linear_scan = false;
+ bool ok = BinarySeek<DecodeKey>(seek_key, &index, &skip_linear_scan);
+
+ if (!ok) {
+ return;
+ }
+ FindKeyAfterBinarySeek(seek_key, index, skip_linear_scan);
+}
+
+void MetaBlockIter::SeekImpl(const Slice& target) {
+ Slice seek_key = target;
+ PERF_TIMER_GUARD(block_seek_nanos);
+ if (data_ == nullptr) { // Not init yet
+ return;
+ }
+ uint32_t index = 0;
+ bool skip_linear_scan = false;
+ bool ok = BinarySeek<DecodeKey>(seek_key, &index, &skip_linear_scan);
+
+ if (!ok) {
+ return;
+ }
+ FindKeyAfterBinarySeek(seek_key, index, skip_linear_scan);
+}
+
+// Optimized Seek for point lookup for an internal key `target`
+// target = "seek_user_key @ type | seqno".
+//
+// For any type other than kTypeValue, kTypeDeletion, kTypeSingleDeletion,
+// kTypeBlobIndex, or kTypeWideColumnEntity, this function behaves identically
+// to Seek().
+//
+// For any type in kTypeValue, kTypeDeletion, kTypeSingleDeletion,
+// kTypeBlobIndex, or kTypeWideColumnEntity:
+//
+// If the return value is FALSE, iter location is undefined, and it means:
+// 1) there is no key in this block falling into the range:
+// ["seek_user_key @ type | seqno", "seek_user_key @ kTypeDeletion | 0"],
+// inclusive; AND
+// 2) the last key of this block has a greater user_key from seek_user_key
+//
+// If the return value is TRUE, iter location has two possibilies:
+// 1) If iter is valid, it is set to a location as if set by BinarySeek. In
+// this case, it points to the first key with a larger user_key or a matching
+// user_key with a seqno no greater than the seeking seqno.
+// 2) If the iter is invalid, it means that either all the user_key is less
+// than the seek_user_key, or the block ends with a matching user_key but
+// with a smaller [ type | seqno ] (i.e. a larger seqno, or the same seqno
+// but larger type).
+bool DataBlockIter::SeekForGetImpl(const Slice& target) {
+ Slice target_user_key = ExtractUserKey(target);
+ uint32_t map_offset = restarts_ + num_restarts_ * sizeof(uint32_t);
+ uint8_t entry =
+ data_block_hash_index_->Lookup(data_, map_offset, target_user_key);
+
+ if (entry == kCollision) {
+ // HashSeek not effective, falling back
+ SeekImpl(target);
+ return true;
+ }
+
+ if (entry == kNoEntry) {
+ // Even if we cannot find the user_key in this block, the result may
+ // exist in the next block. Consider this example:
+ //
+ // Block N: [aab@100, ... , app@120]
+ // boundary key: axy@50 (we make minimal assumption about a boundary key)
+ // Block N+1: [axy@10, ... ]
+ //
+ // If seek_key = axy@60, the search will starts from Block N.
+ // Even if the user_key is not found in the hash map, the caller still
+ // have to continue searching the next block.
+ //
+ // In this case, we pretend the key is the the last restart interval.
+ // The while-loop below will search the last restart interval for the
+ // key. It will stop at the first key that is larger than the seek_key,
+ // or to the end of the block if no one is larger.
+ entry = static_cast<uint8_t>(num_restarts_ - 1);
+ }
+
+ uint32_t restart_index = entry;
+
+ // check if the key is in the restart_interval
+ assert(restart_index < num_restarts_);
+ SeekToRestartPoint(restart_index);
+ current_ = GetRestartPoint(restart_index);
+
+ uint32_t limit = restarts_;
+ if (restart_index + 1 < num_restarts_) {
+ limit = GetRestartPoint(restart_index + 1);
+ }
+ while (current_ < limit) {
+ bool shared;
+ // Here we only linear seek the target key inside the restart interval.
+ // If a key does not exist inside a restart interval, we avoid
+ // further searching the block content across restart interval boundary.
+ //
+ // TODO(fwu): check the left and right boundary of the restart interval
+ // to avoid linear seek a target key that is out of range.
+ if (!ParseNextDataKey(&shared) || CompareCurrentKey(target) >= 0) {
+ // we stop at the first potential matching user key.
+ break;
+ }
+ }
+
+ if (current_ == restarts_) {
+ // Search reaches to the end of the block. There are three possibilites:
+ // 1) there is only one user_key match in the block (otherwise collsion).
+ // the matching user_key resides in the last restart interval, and it
+ // is the last key of the restart interval and of the block as well.
+ // ParseNextKey() skiped it as its [ type | seqno ] is smaller.
+ //
+ // 2) The seek_key is not found in the HashIndex Lookup(), i.e. kNoEntry,
+ // AND all existing user_keys in the restart interval are smaller than
+ // seek_user_key.
+ //
+ // 3) The seek_key is a false positive and happens to be hashed to the
+ // last restart interval, AND all existing user_keys in the restart
+ // interval are smaller than seek_user_key.
+ //
+ // The result may exist in the next block each case, so we return true.
+ return true;
+ }
+
+ if (icmp_->user_comparator()->Compare(raw_key_.GetUserKey(),
+ target_user_key) != 0) {
+ // the key is not in this block and cannot be at the next block either.
+ return false;
+ }
+
+ // Here we are conservative and only support a limited set of cases
+ ValueType value_type = ExtractValueType(raw_key_.GetInternalKey());
+ if (value_type != ValueType::kTypeValue &&
+ value_type != ValueType::kTypeDeletion &&
+ value_type != ValueType::kTypeSingleDeletion &&
+ value_type != ValueType::kTypeBlobIndex &&
+ value_type != ValueType::kTypeWideColumnEntity) {
+ SeekImpl(target);
+ return true;
+ }
+
+ // Result found, and the iter is correctly set.
+ return true;
+}
+
+void IndexBlockIter::SeekImpl(const Slice& target) {
+ TEST_SYNC_POINT("IndexBlockIter::Seek:0");
+ PERF_TIMER_GUARD(block_seek_nanos);
+ if (data_ == nullptr) { // Not init yet
+ return;
+ }
+ Slice seek_key = target;
+ if (raw_key_.IsUserKey()) {
+ seek_key = ExtractUserKey(target);
+ }
+ status_ = Status::OK();
+ uint32_t index = 0;
+ bool skip_linear_scan = false;
+ bool ok = false;
+ if (prefix_index_) {
+ bool prefix_may_exist = true;
+ ok = PrefixSeek(target, &index, &prefix_may_exist);
+ if (!prefix_may_exist) {
+ // This is to let the caller to distinguish between non-existing prefix,
+ // and when key is larger than the last key, which both set Valid() to
+ // false.
+ current_ = restarts_;
+ status_ = Status::NotFound();
+ }
+ // restart interval must be one when hash search is enabled so the binary
+ // search simply lands at the right place.
+ skip_linear_scan = true;
+ } else if (value_delta_encoded_) {
+ ok = BinarySeek<DecodeKeyV4>(seek_key, &index, &skip_linear_scan);
+ } else {
+ ok = BinarySeek<DecodeKey>(seek_key, &index, &skip_linear_scan);
+ }
+
+ if (!ok) {
+ return;
+ }
+ FindKeyAfterBinarySeek(seek_key, index, skip_linear_scan);
+}
+
+void DataBlockIter::SeekForPrevImpl(const Slice& target) {
+ PERF_TIMER_GUARD(block_seek_nanos);
+ Slice seek_key = target;
+ if (data_ == nullptr) { // Not init yet
+ return;
+ }
+ uint32_t index = 0;
+ bool skip_linear_scan = false;
+ bool ok = BinarySeek<DecodeKey>(seek_key, &index, &skip_linear_scan);
+
+ if (!ok) {
+ return;
+ }
+ FindKeyAfterBinarySeek(seek_key, index, skip_linear_scan);
+
+ if (!Valid()) {
+ SeekToLastImpl();
+ } else {
+ while (Valid() && CompareCurrentKey(seek_key) > 0) {
+ PrevImpl();
+ }
+ }
+}
+
+void MetaBlockIter::SeekForPrevImpl(const Slice& target) {
+ PERF_TIMER_GUARD(block_seek_nanos);
+ Slice seek_key = target;
+ if (data_ == nullptr) { // Not init yet
+ return;
+ }
+ uint32_t index = 0;
+ bool skip_linear_scan = false;
+ bool ok = BinarySeek<DecodeKey>(seek_key, &index, &skip_linear_scan);
+
+ if (!ok) {
+ return;
+ }
+ FindKeyAfterBinarySeek(seek_key, index, skip_linear_scan);
+
+ if (!Valid()) {
+ SeekToLastImpl();
+ } else {
+ while (Valid() && CompareCurrentKey(seek_key) > 0) {
+ PrevImpl();
+ }
+ }
+}
+
+void DataBlockIter::SeekToFirstImpl() {
+ if (data_ == nullptr) { // Not init yet
+ return;
+ }
+ SeekToRestartPoint(0);
+ bool is_shared = false;
+ ParseNextDataKey(&is_shared);
+}
+
+void MetaBlockIter::SeekToFirstImpl() {
+ if (data_ == nullptr) { // Not init yet
+ return;
+ }
+ SeekToRestartPoint(0);
+ bool is_shared = false;
+ ParseNextKey<CheckAndDecodeEntry>(&is_shared);
+}
+
+void IndexBlockIter::SeekToFirstImpl() {
+ if (data_ == nullptr) { // Not init yet
+ return;
+ }
+ status_ = Status::OK();
+ SeekToRestartPoint(0);
+ ParseNextIndexKey();
+}
+
+void DataBlockIter::SeekToLastImpl() {
+ if (data_ == nullptr) { // Not init yet
+ return;
+ }
+ SeekToRestartPoint(num_restarts_ - 1);
+ bool is_shared = false;
+ while (ParseNextDataKey(&is_shared) && NextEntryOffset() < restarts_) {
+ // Keep skipping
+ }
+}
+
+void MetaBlockIter::SeekToLastImpl() {
+ if (data_ == nullptr) { // Not init yet
+ return;
+ }
+ SeekToRestartPoint(num_restarts_ - 1);
+ bool is_shared = false;
+ while (ParseNextKey<CheckAndDecodeEntry>(&is_shared) &&
+ NextEntryOffset() < restarts_) {
+ // Keep skipping
+ }
+}
+
+void IndexBlockIter::SeekToLastImpl() {
+ if (data_ == nullptr) { // Not init yet
+ return;
+ }
+ status_ = Status::OK();
+ SeekToRestartPoint(num_restarts_ - 1);
+ while (ParseNextIndexKey() && NextEntryOffset() < restarts_) {
+ // Keep skipping
+ }
+}
+
+template <class TValue>
+void BlockIter<TValue>::CorruptionError() {
+ current_ = restarts_;
+ restart_index_ = num_restarts_;
+ status_ = Status::Corruption("bad entry in block");
+ raw_key_.Clear();
+ value_.clear();
+}
+
+template <class TValue>
+template <typename DecodeEntryFunc>
+bool BlockIter<TValue>::ParseNextKey(bool* is_shared) {
+ current_ = NextEntryOffset();
+ const char* p = data_ + current_;
+ const char* limit = data_ + restarts_; // Restarts come right after data
+
+ if (p >= limit) {
+ // No more entries to return. Mark as invalid.
+ current_ = restarts_;
+ restart_index_ = num_restarts_;
+ return false;
+ }
+ // Decode next entry
+ uint32_t shared, non_shared, value_length;
+ p = DecodeEntryFunc()(p, limit, &shared, &non_shared, &value_length);
+ if (p == nullptr || raw_key_.Size() < shared) {
+ CorruptionError();
+ return false;
+ } else {
+ if (shared == 0) {
+ *is_shared = false;
+ // If this key doesn't share any bytes with prev key then we don't need
+ // to decode it and can use its address in the block directly.
+ raw_key_.SetKey(Slice(p, non_shared), false /* copy */);
+ } else {
+ // This key share `shared` bytes with prev key, we need to decode it
+ *is_shared = true;
+ raw_key_.TrimAppend(shared, p, non_shared);
+ }
+ value_ = Slice(p + non_shared, value_length);
+ if (shared == 0) {
+ while (restart_index_ + 1 < num_restarts_ &&
+ GetRestartPoint(restart_index_ + 1) < current_) {
+ ++restart_index_;
+ }
+ }
+ // else we are in the middle of a restart interval and the restart_index_
+ // thus has not changed
+ return true;
+ }
+}
+
+bool DataBlockIter::ParseNextDataKey(bool* is_shared) {
+ if (ParseNextKey<DecodeEntry>(is_shared)) {
+#ifndef NDEBUG
+ if (global_seqno_ != kDisableGlobalSequenceNumber) {
+ // If we are reading a file with a global sequence number we should
+ // expect that all encoded sequence numbers are zeros and any value
+ // type is kTypeValue, kTypeMerge, kTypeDeletion,
+ // kTypeDeletionWithTimestamp, or kTypeRangeDeletion.
+ uint64_t packed = ExtractInternalKeyFooter(raw_key_.GetKey());
+ SequenceNumber seqno;
+ ValueType value_type;
+ UnPackSequenceAndType(packed, &seqno, &value_type);
+ assert(value_type == ValueType::kTypeValue ||
+ value_type == ValueType::kTypeMerge ||
+ value_type == ValueType::kTypeDeletion ||
+ value_type == ValueType::kTypeDeletionWithTimestamp ||
+ value_type == ValueType::kTypeRangeDeletion);
+ assert(seqno == 0);
+ }
+#endif // NDEBUG
+ return true;
+ } else {
+ return false;
+ }
+}
+
+bool IndexBlockIter::ParseNextIndexKey() {
+ bool is_shared = false;
+ bool ok = (value_delta_encoded_) ? ParseNextKey<DecodeEntryV4>(&is_shared)
+ : ParseNextKey<DecodeEntry>(&is_shared);
+ if (ok) {
+ if (value_delta_encoded_ || global_seqno_state_ != nullptr) {
+ DecodeCurrentValue(is_shared);
+ }
+ }
+ return ok;
+}
+
+// The format:
+// restart_point 0: k, v (off, sz), k, v (delta-sz), ..., k, v (delta-sz)
+// restart_point 1: k, v (off, sz), k, v (delta-sz), ..., k, v (delta-sz)
+// ...
+// restart_point n-1: k, v (off, sz), k, v (delta-sz), ..., k, v (delta-sz)
+// where, k is key, v is value, and its encoding is in parenthesis.
+// The format of each key is (shared_size, non_shared_size, shared, non_shared)
+// The format of each value, i.e., block handle, is (offset, size) whenever the
+// is_shared is false, which included the first entry in each restart point.
+// Otherwise the format is delta-size = block handle size - size of last block
+// handle.
+void IndexBlockIter::DecodeCurrentValue(bool is_shared) {
+ Slice v(value_.data(), data_ + restarts_ - value_.data());
+ // Delta encoding is used if `shared` != 0.
+ Status decode_s __attribute__((__unused__)) = decoded_value_.DecodeFrom(
+ &v, have_first_key_,
+ (value_delta_encoded_ && is_shared) ? &decoded_value_.handle : nullptr);
+ assert(decode_s.ok());
+ value_ = Slice(value_.data(), v.data() - value_.data());
+
+ if (global_seqno_state_ != nullptr) {
+ // Overwrite sequence number the same way as in DataBlockIter.
+
+ IterKey& first_internal_key = global_seqno_state_->first_internal_key;
+ first_internal_key.SetInternalKey(decoded_value_.first_internal_key,
+ /* copy */ true);
+
+ assert(GetInternalKeySeqno(first_internal_key.GetInternalKey()) == 0);
+
+ ValueType value_type = ExtractValueType(first_internal_key.GetKey());
+ assert(value_type == ValueType::kTypeValue ||
+ value_type == ValueType::kTypeMerge ||
+ value_type == ValueType::kTypeDeletion ||
+ value_type == ValueType::kTypeRangeDeletion);
+
+ first_internal_key.UpdateInternalKey(global_seqno_state_->global_seqno,
+ value_type);
+ decoded_value_.first_internal_key = first_internal_key.GetKey();
+ }
+}
+
+template <class TValue>
+void BlockIter<TValue>::FindKeyAfterBinarySeek(const Slice& target,
+ uint32_t index,
+ bool skip_linear_scan) {
+ // SeekToRestartPoint() only does the lookup in the restart block. We need
+ // to follow it up with NextImpl() to position the iterator at the restart
+ // key.
+ SeekToRestartPoint(index);
+ NextImpl();
+
+ if (!skip_linear_scan) {
+ // Linear search (within restart block) for first key >= target
+ uint32_t max_offset;
+ if (index + 1 < num_restarts_) {
+ // We are in a non-last restart interval. Since `BinarySeek()` guarantees
+ // the next restart key is strictly greater than `target`, we can
+ // terminate upon reaching it without any additional key comparison.
+ max_offset = GetRestartPoint(index + 1);
+ } else {
+ // We are in the last restart interval. The while-loop will terminate by
+ // `Valid()` returning false upon advancing past the block's last key.
+ max_offset = std::numeric_limits<uint32_t>::max();
+ }
+ while (true) {
+ NextImpl();
+ if (!Valid()) {
+ break;
+ }
+ if (current_ == max_offset) {
+ assert(CompareCurrentKey(target) > 0);
+ break;
+ } else if (CompareCurrentKey(target) >= 0) {
+ break;
+ }
+ }
+ }
+}
+
+// Binary searches in restart array to find the starting restart point for the
+// linear scan, and stores it in `*index`. Assumes restart array does not
+// contain duplicate keys. It is guaranteed that the restart key at `*index + 1`
+// is strictly greater than `target` or does not exist (this can be used to
+// elide a comparison when linear scan reaches all the way to the next restart
+// key). Furthermore, `*skip_linear_scan` is set to indicate whether the
+// `*index`th restart key is the final result so that key does not need to be
+// compared again later.
+template <class TValue>
+template <typename DecodeKeyFunc>
+bool BlockIter<TValue>::BinarySeek(const Slice& target, uint32_t* index,
+ bool* skip_linear_scan) {
+ if (restarts_ == 0) {
+ // SST files dedicated to range tombstones are written with index blocks
+ // that have no keys while also having `num_restarts_ == 1`. This would
+ // cause a problem for `BinarySeek()` as it'd try to access the first key
+ // which does not exist. We identify such blocks by the offset at which
+ // their restarts are stored, and return false to prevent any attempted
+ // key accesses.
+ return false;
+ }
+
+ *skip_linear_scan = false;
+ // Loop invariants:
+ // - Restart key at index `left` is less than or equal to the target key. The
+ // sentinel index `-1` is considered to have a key that is less than all
+ // keys.
+ // - Any restart keys after index `right` are strictly greater than the target
+ // key.
+ int64_t left = -1, right = num_restarts_ - 1;
+ while (left != right) {
+ // The `mid` is computed by rounding up so it lands in (`left`, `right`].
+ int64_t mid = left + (right - left + 1) / 2;
+ uint32_t region_offset = GetRestartPoint(static_cast<uint32_t>(mid));
+ uint32_t shared, non_shared;
+ const char* key_ptr = DecodeKeyFunc()(
+ data_ + region_offset, data_ + restarts_, &shared, &non_shared);
+ if (key_ptr == nullptr || (shared != 0)) {
+ CorruptionError();
+ return false;
+ }
+ Slice mid_key(key_ptr, non_shared);
+ raw_key_.SetKey(mid_key, false /* copy */);
+ int cmp = CompareCurrentKey(target);
+ if (cmp < 0) {
+ // Key at "mid" is smaller than "target". Therefore all
+ // blocks before "mid" are uninteresting.
+ left = mid;
+ } else if (cmp > 0) {
+ // Key at "mid" is >= "target". Therefore all blocks at or
+ // after "mid" are uninteresting.
+ right = mid - 1;
+ } else {
+ *skip_linear_scan = true;
+ left = right = mid;
+ }
+ }
+
+ if (left == -1) {
+ // All keys in the block were strictly greater than `target`. So the very
+ // first key in the block is the final seek result.
+ *skip_linear_scan = true;
+ *index = 0;
+ } else {
+ *index = static_cast<uint32_t>(left);
+ }
+ return true;
+}
+
+// Compare target key and the block key of the block of `block_index`.
+// Return -1 if error.
+int IndexBlockIter::CompareBlockKey(uint32_t block_index, const Slice& target) {
+ uint32_t region_offset = GetRestartPoint(block_index);
+ uint32_t shared, non_shared;
+ const char* key_ptr =
+ value_delta_encoded_
+ ? DecodeKeyV4()(data_ + region_offset, data_ + restarts_, &shared,
+ &non_shared)
+ : DecodeKey()(data_ + region_offset, data_ + restarts_, &shared,
+ &non_shared);
+ if (key_ptr == nullptr || (shared != 0)) {
+ CorruptionError();
+ return 1; // Return target is smaller
+ }
+ Slice block_key(key_ptr, non_shared);
+ raw_key_.SetKey(block_key, false /* copy */);
+ return CompareCurrentKey(target);
+}
+
+// Binary search in block_ids to find the first block
+// with a key >= target
+bool IndexBlockIter::BinaryBlockIndexSeek(const Slice& target,
+ uint32_t* block_ids, uint32_t left,
+ uint32_t right, uint32_t* index,
+ bool* prefix_may_exist) {
+ assert(left <= right);
+ assert(index);
+ assert(prefix_may_exist);
+ *prefix_may_exist = true;
+ uint32_t left_bound = left;
+
+ while (left <= right) {
+ uint32_t mid = (right + left) / 2;
+
+ int cmp = CompareBlockKey(block_ids[mid], target);
+ if (!status_.ok()) {
+ return false;
+ }
+ 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) {
+ // In one of the two following cases:
+ // (1) left is the first one of block_ids
+ // (2) there is a gap of blocks between block of `left` and `left-1`.
+ // we can further distinguish the case of key in the block or key not
+ // existing, by comparing the target key and the key of the previous
+ // block to the left of the block found.
+ if (block_ids[left] > 0 &&
+ (left == left_bound || block_ids[left - 1] != block_ids[left] - 1) &&
+ CompareBlockKey(block_ids[left] - 1, target) > 0) {
+ current_ = restarts_;
+ *prefix_may_exist = false;
+ return false;
+ }
+
+ *index = block_ids[left];
+ return true;
+ } else {
+ assert(left > right);
+
+ // If the next block key is larger than seek key, it is possible that
+ // no key shares the prefix with `target`, or all keys with the same
+ // prefix as `target` are smaller than prefix. In the latter case,
+ // we are mandated to set the position the same as the total order.
+ // In the latter case, either:
+ // (1) `target` falls into the range of the next block. In this case,
+ // we can place the iterator to the next block, or
+ // (2) `target` is larger than all block keys. In this case we can
+ // keep the iterator invalidate without setting `prefix_may_exist`
+ // to false.
+ // We might sometimes end up with setting the total order position
+ // while there is no key sharing the prefix as `target`, but it
+ // still follows the contract.
+ uint32_t right_index = block_ids[right];
+ assert(right_index + 1 <= num_restarts_);
+ if (right_index + 1 < num_restarts_) {
+ if (CompareBlockKey(right_index + 1, target) >= 0) {
+ *index = right_index + 1;
+ return true;
+ } else {
+ // We have to set the flag here because we are not positioning
+ // the iterator to the total order position.
+ *prefix_may_exist = false;
+ }
+ }
+
+ // Mark iterator invalid
+ current_ = restarts_;
+ return false;
+ }
+}
+
+bool IndexBlockIter::PrefixSeek(const Slice& target, uint32_t* index,
+ bool* prefix_may_exist) {
+ assert(index);
+ assert(prefix_may_exist);
+ assert(prefix_index_);
+ *prefix_may_exist = true;
+ Slice seek_key = target;
+ if (raw_key_.IsUserKey()) {
+ seek_key = ExtractUserKey(target);
+ }
+ uint32_t* block_ids = nullptr;
+ uint32_t num_blocks = prefix_index_->GetBlocks(target, &block_ids);
+
+ if (num_blocks == 0) {
+ current_ = restarts_;
+ *prefix_may_exist = false;
+ return false;
+ } else {
+ assert(block_ids);
+ return BinaryBlockIndexSeek(seek_key, block_ids, 0, num_blocks - 1, index,
+ prefix_may_exist);
+ }
+}
+
+uint32_t Block::NumRestarts() const {
+ assert(size_ >= 2 * sizeof(uint32_t));
+ uint32_t block_footer = DecodeFixed32(data_ + size_ - sizeof(uint32_t));
+ uint32_t num_restarts = block_footer;
+ if (size_ > kMaxBlockSizeSupportedByHashIndex) {
+ // In BlockBuilder, we have ensured a block with HashIndex is less than
+ // kMaxBlockSizeSupportedByHashIndex (64KiB).
+ //
+ // Therefore, if we encounter a block with a size > 64KiB, the block
+ // cannot have HashIndex. So the footer will directly interpreted as
+ // num_restarts.
+ //
+ // Such check is for backward compatibility. We can ensure legacy block
+ // with a vary large num_restarts i.e. >= 0x80000000 can be interpreted
+ // correctly as no HashIndex even if the MSB of num_restarts is set.
+ return num_restarts;
+ }
+ BlockBasedTableOptions::DataBlockIndexType index_type;
+ UnPackIndexTypeAndNumRestarts(block_footer, &index_type, &num_restarts);
+ return num_restarts;
+}
+
+BlockBasedTableOptions::DataBlockIndexType Block::IndexType() const {
+ assert(size_ >= 2 * sizeof(uint32_t));
+ if (size_ > kMaxBlockSizeSupportedByHashIndex) {
+ // The check is for the same reason as that in NumRestarts()
+ return BlockBasedTableOptions::kDataBlockBinarySearch;
+ }
+ uint32_t block_footer = DecodeFixed32(data_ + size_ - sizeof(uint32_t));
+ uint32_t num_restarts = block_footer;
+ BlockBasedTableOptions::DataBlockIndexType index_type;
+ UnPackIndexTypeAndNumRestarts(block_footer, &index_type, &num_restarts);
+ return index_type;
+}
+
+Block::~Block() {
+ // This sync point can be re-enabled if RocksDB can control the
+ // initialization order of any/all static options created by the user.
+ // TEST_SYNC_POINT("Block::~Block");
+}
+
+Block::Block(BlockContents&& contents, size_t read_amp_bytes_per_bit,
+ Statistics* statistics)
+ : contents_(std::move(contents)),
+ data_(contents_.data.data()),
+ size_(contents_.data.size()),
+ restart_offset_(0),
+ num_restarts_(0) {
+ TEST_SYNC_POINT("Block::Block:0");
+ if (size_ < sizeof(uint32_t)) {
+ size_ = 0; // Error marker
+ } else {
+ // Should only decode restart points for uncompressed blocks
+ num_restarts_ = NumRestarts();
+ switch (IndexType()) {
+ case BlockBasedTableOptions::kDataBlockBinarySearch:
+ restart_offset_ = static_cast<uint32_t>(size_) -
+ (1 + num_restarts_) * sizeof(uint32_t);
+ if (restart_offset_ > size_ - sizeof(uint32_t)) {
+ // The size is too small for NumRestarts() and therefore
+ // restart_offset_ wrapped around.
+ size_ = 0;
+ }
+ break;
+ case BlockBasedTableOptions::kDataBlockBinaryAndHash:
+ if (size_ < sizeof(uint32_t) /* block footer */ +
+ sizeof(uint16_t) /* NUM_BUCK */) {
+ size_ = 0;
+ break;
+ }
+
+ uint16_t map_offset;
+ data_block_hash_index_.Initialize(
+ contents.data.data(),
+ static_cast<uint16_t>(contents.data.size() -
+ sizeof(uint32_t)), /*chop off
+ NUM_RESTARTS*/
+ &map_offset);
+
+ restart_offset_ = map_offset - num_restarts_ * sizeof(uint32_t);
+
+ if (restart_offset_ > map_offset) {
+ // map_offset is too small for NumRestarts() and
+ // therefore restart_offset_ wrapped around.
+ size_ = 0;
+ break;
+ }
+ break;
+ default:
+ size_ = 0; // Error marker
+ }
+ }
+ if (read_amp_bytes_per_bit != 0 && statistics && size_ != 0) {
+ read_amp_bitmap_.reset(new BlockReadAmpBitmap(
+ restart_offset_, read_amp_bytes_per_bit, statistics));
+ }
+}
+
+MetaBlockIter* Block::NewMetaIterator(bool block_contents_pinned) {
+ MetaBlockIter* iter = new MetaBlockIter();
+ if (size_ < 2 * sizeof(uint32_t)) {
+ iter->Invalidate(Status::Corruption("bad block contents"));
+ return iter;
+ } else if (num_restarts_ == 0) {
+ // Empty block.
+ iter->Invalidate(Status::OK());
+ } else {
+ iter->Initialize(data_, restart_offset_, num_restarts_,
+ block_contents_pinned);
+ }
+ return iter;
+}
+
+DataBlockIter* Block::NewDataIterator(const Comparator* raw_ucmp,
+ SequenceNumber global_seqno,
+ DataBlockIter* iter, Statistics* stats,
+ bool block_contents_pinned) {
+ DataBlockIter* ret_iter;
+ if (iter != nullptr) {
+ ret_iter = iter;
+ } else {
+ ret_iter = new DataBlockIter;
+ }
+ if (size_ < 2 * sizeof(uint32_t)) {
+ ret_iter->Invalidate(Status::Corruption("bad block contents"));
+ return ret_iter;
+ }
+ if (num_restarts_ == 0) {
+ // Empty block.
+ ret_iter->Invalidate(Status::OK());
+ return ret_iter;
+ } else {
+ ret_iter->Initialize(
+ raw_ucmp, data_, restart_offset_, num_restarts_, global_seqno,
+ read_amp_bitmap_.get(), block_contents_pinned,
+ data_block_hash_index_.Valid() ? &data_block_hash_index_ : nullptr);
+ if (read_amp_bitmap_) {
+ if (read_amp_bitmap_->GetStatistics() != stats) {
+ // DB changed the Statistics pointer, we need to notify read_amp_bitmap_
+ read_amp_bitmap_->SetStatistics(stats);
+ }
+ }
+ }
+
+ return ret_iter;
+}
+
+IndexBlockIter* Block::NewIndexIterator(
+ const Comparator* raw_ucmp, SequenceNumber global_seqno,
+ IndexBlockIter* iter, Statistics* /*stats*/, bool total_order_seek,
+ bool have_first_key, bool key_includes_seq, bool value_is_full,
+ bool block_contents_pinned, BlockPrefixIndex* prefix_index) {
+ IndexBlockIter* ret_iter;
+ if (iter != nullptr) {
+ ret_iter = iter;
+ } else {
+ ret_iter = new IndexBlockIter;
+ }
+ if (size_ < 2 * sizeof(uint32_t)) {
+ ret_iter->Invalidate(Status::Corruption("bad block contents"));
+ return ret_iter;
+ }
+ if (num_restarts_ == 0) {
+ // Empty block.
+ ret_iter->Invalidate(Status::OK());
+ return ret_iter;
+ } else {
+ BlockPrefixIndex* prefix_index_ptr =
+ total_order_seek ? nullptr : prefix_index;
+ ret_iter->Initialize(raw_ucmp, data_, restart_offset_, num_restarts_,
+ global_seqno, prefix_index_ptr, have_first_key,
+ key_includes_seq, value_is_full,
+ block_contents_pinned);
+ }
+
+ return ret_iter;
+}
+
+size_t Block::ApproximateMemoryUsage() const {
+ size_t usage = usable_size();
+#ifdef ROCKSDB_MALLOC_USABLE_SIZE
+ usage += malloc_usable_size((void*)this);
+#else
+ usage += sizeof(*this);
+#endif // ROCKSDB_MALLOC_USABLE_SIZE
+ if (read_amp_bitmap_) {
+ usage += read_amp_bitmap_->ApproximateMemoryUsage();
+ }
+ return usage;
+}
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/block.h b/src/rocksdb/table/block_based/block.h
new file mode 100644
index 000000000..5d73f72f6
--- /dev/null
+++ b/src/rocksdb/table/block_based/block.h
@@ -0,0 +1,744 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#pragma once
+#include <stddef.h>
+#include <stdint.h>
+
+#include <string>
+#include <vector>
+
+#include "db/pinned_iterators_manager.h"
+#include "port/malloc.h"
+#include "rocksdb/iterator.h"
+#include "rocksdb/options.h"
+#include "rocksdb/statistics.h"
+#include "rocksdb/table.h"
+#include "table/block_based/block_prefix_index.h"
+#include "table/block_based/data_block_hash_index.h"
+#include "table/format.h"
+#include "table/internal_iterator.h"
+#include "test_util/sync_point.h"
+#include "util/random.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+struct BlockContents;
+class Comparator;
+template <class TValue>
+class BlockIter;
+class DataBlockIter;
+class IndexBlockIter;
+class MetaBlockIter;
+class BlockPrefixIndex;
+
+// BlockReadAmpBitmap is a bitmap that map the ROCKSDB_NAMESPACE::Block data
+// bytes to a bitmap with ratio bytes_per_bit. Whenever we access a range of
+// bytes in the Block we update the bitmap and increment
+// READ_AMP_ESTIMATE_USEFUL_BYTES.
+class BlockReadAmpBitmap {
+ public:
+ explicit BlockReadAmpBitmap(size_t block_size, size_t bytes_per_bit,
+ Statistics* statistics)
+ : bitmap_(nullptr),
+ bytes_per_bit_pow_(0),
+ statistics_(statistics),
+ rnd_(Random::GetTLSInstance()->Uniform(
+ static_cast<int>(bytes_per_bit))) {
+ TEST_SYNC_POINT_CALLBACK("BlockReadAmpBitmap:rnd", &rnd_);
+ assert(block_size > 0 && bytes_per_bit > 0);
+
+ // convert bytes_per_bit to be a power of 2
+ while (bytes_per_bit >>= 1) {
+ bytes_per_bit_pow_++;
+ }
+
+ // num_bits_needed = ceil(block_size / bytes_per_bit)
+ size_t num_bits_needed = ((block_size - 1) >> bytes_per_bit_pow_) + 1;
+ assert(num_bits_needed > 0);
+
+ // bitmap_size = ceil(num_bits_needed / kBitsPerEntry)
+ size_t bitmap_size = (num_bits_needed - 1) / kBitsPerEntry + 1;
+
+ // Create bitmap and set all the bits to 0
+ bitmap_ = new std::atomic<uint32_t>[bitmap_size]();
+
+ RecordTick(GetStatistics(), READ_AMP_TOTAL_READ_BYTES, block_size);
+ }
+
+ ~BlockReadAmpBitmap() { delete[] bitmap_; }
+
+ void Mark(uint32_t start_offset, uint32_t end_offset) {
+ assert(end_offset >= start_offset);
+ // Index of first bit in mask
+ uint32_t start_bit =
+ (start_offset + (1 << bytes_per_bit_pow_) - rnd_ - 1) >>
+ bytes_per_bit_pow_;
+ // Index of last bit in mask + 1
+ uint32_t exclusive_end_bit =
+ (end_offset + (1 << bytes_per_bit_pow_) - rnd_) >> bytes_per_bit_pow_;
+ if (start_bit >= exclusive_end_bit) {
+ return;
+ }
+ assert(exclusive_end_bit > 0);
+
+ if (GetAndSet(start_bit) == 0) {
+ uint32_t new_useful_bytes = (exclusive_end_bit - start_bit)
+ << bytes_per_bit_pow_;
+ RecordTick(GetStatistics(), READ_AMP_ESTIMATE_USEFUL_BYTES,
+ new_useful_bytes);
+ }
+ }
+
+ Statistics* GetStatistics() {
+ return statistics_.load(std::memory_order_relaxed);
+ }
+
+ void SetStatistics(Statistics* stats) { statistics_.store(stats); }
+
+ uint32_t GetBytesPerBit() { return 1 << bytes_per_bit_pow_; }
+
+ size_t ApproximateMemoryUsage() const {
+#ifdef ROCKSDB_MALLOC_USABLE_SIZE
+ return malloc_usable_size((void*)this);
+#endif // ROCKSDB_MALLOC_USABLE_SIZE
+ return sizeof(*this);
+ }
+
+ private:
+ // Get the current value of bit at `bit_idx` and set it to 1
+ inline bool GetAndSet(uint32_t bit_idx) {
+ const uint32_t byte_idx = bit_idx / kBitsPerEntry;
+ const uint32_t bit_mask = 1 << (bit_idx % kBitsPerEntry);
+
+ return bitmap_[byte_idx].fetch_or(bit_mask, std::memory_order_relaxed) &
+ bit_mask;
+ }
+
+ const uint32_t kBytesPersEntry = sizeof(uint32_t); // 4 bytes
+ const uint32_t kBitsPerEntry = kBytesPersEntry * 8; // 32 bits
+
+ // Bitmap used to record the bytes that we read, use atomic to protect
+ // against multiple threads updating the same bit
+ std::atomic<uint32_t>* bitmap_;
+ // (1 << bytes_per_bit_pow_) is bytes_per_bit. Use power of 2 to optimize
+ // muliplication and division
+ uint8_t bytes_per_bit_pow_;
+ // Pointer to DB Statistics object, Since this bitmap may outlive the DB
+ // this pointer maybe invalid, but the DB will update it to a valid pointer
+ // by using SetStatistics() before calling Mark()
+ std::atomic<Statistics*> statistics_;
+ uint32_t rnd_;
+};
+
+// class Block is the uncompressed and "parsed" form for blocks containing
+// key-value pairs. (See BlockContents comments for more on terminology.)
+// This includes the in-memory representation of data blocks, index blocks
+// (including partitions), range deletion blocks, properties blocks, metaindex
+// blocks, as well as the top level of the partitioned filter structure (which
+// is actually an index of the filter partitions). It is NOT suitable for
+// compressed blocks in general, filter blocks/partitions, or compression
+// dictionaries.
+//
+// See https://github.com/facebook/rocksdb/wiki/Rocksdb-BlockBasedTable-Format
+// for details of the format and the various block types.
+//
+// TODO: Rename to ParsedKvBlock?
+class Block {
+ public:
+ // Initialize the block with the specified contents.
+ explicit Block(BlockContents&& contents, size_t read_amp_bytes_per_bit = 0,
+ Statistics* statistics = nullptr);
+ // No copying allowed
+ Block(const Block&) = delete;
+ void operator=(const Block&) = delete;
+
+ ~Block();
+
+ size_t size() const { return size_; }
+ const char* data() const { return data_; }
+ // The additional memory space taken by the block data.
+ size_t usable_size() const { return contents_.usable_size(); }
+ uint32_t NumRestarts() const;
+ bool own_bytes() const { return contents_.own_bytes(); }
+
+ BlockBasedTableOptions::DataBlockIndexType IndexType() const;
+
+ // raw_ucmp is a raw (i.e., not wrapped by `UserComparatorWrapper`) user key
+ // comparator.
+ //
+ // If iter is null, return new Iterator
+ // If iter is not null, update this one and return it as Iterator*
+ //
+ // Updates read_amp_bitmap_ if it is not nullptr.
+ //
+ // If `block_contents_pinned` is true, the caller will guarantee that when
+ // the cleanup functions are transferred from the iterator to other
+ // classes, e.g. PinnableSlice, the pointer to the bytes will still be
+ // valid. Either the iterator holds cache handle or ownership of some resource
+ // and release them in a release function, or caller is sure that the data
+ // will not go away (for example, it's from mmapped file which will not be
+ // closed).
+ //
+ // NOTE: for the hash based lookup, if a key prefix doesn't match any key,
+ // the iterator will simply be set as "invalid", rather than returning
+ // the key that is just pass the target key.
+ DataBlockIter* NewDataIterator(const Comparator* raw_ucmp,
+ SequenceNumber global_seqno,
+ DataBlockIter* iter = nullptr,
+ Statistics* stats = nullptr,
+ bool block_contents_pinned = false);
+
+ // Returns an MetaBlockIter for iterating over blocks containing metadata
+ // (like Properties blocks). Unlike data blocks, the keys for these blocks
+ // do not contain sequence numbers, do not use a user-define comparator, and
+ // do not track read amplification/statistics. Additionally, MetaBlocks will
+ // not assert if the block is formatted improperly.
+ //
+ // If `block_contents_pinned` is true, the caller will guarantee that when
+ // the cleanup functions are transferred from the iterator to other
+ // classes, e.g. PinnableSlice, the pointer to the bytes will still be
+ // valid. Either the iterator holds cache handle or ownership of some resource
+ // and release them in a release function, or caller is sure that the data
+ // will not go away (for example, it's from mmapped file which will not be
+ // closed).
+ MetaBlockIter* NewMetaIterator(bool block_contents_pinned = false);
+
+ // raw_ucmp is a raw (i.e., not wrapped by `UserComparatorWrapper`) user key
+ // comparator.
+ //
+ // key_includes_seq, default true, means that the keys are in internal key
+ // format.
+ // value_is_full, default true, means that no delta encoding is
+ // applied to values.
+ //
+ // If `prefix_index` is not nullptr this block will do hash lookup for the key
+ // prefix. If total_order_seek is true, prefix_index_ is ignored.
+ //
+ // `have_first_key` controls whether IndexValue will contain
+ // first_internal_key. It affects data serialization format, so the same value
+ // have_first_key must be used when writing and reading index.
+ // It is determined by IndexType property of the table.
+ IndexBlockIter* NewIndexIterator(const Comparator* raw_ucmp,
+ SequenceNumber global_seqno,
+ IndexBlockIter* iter, Statistics* stats,
+ bool total_order_seek, bool have_first_key,
+ bool key_includes_seq, bool value_is_full,
+ bool block_contents_pinned = false,
+ BlockPrefixIndex* prefix_index = nullptr);
+
+ // Report an approximation of how much memory has been used.
+ size_t ApproximateMemoryUsage() const;
+
+ private:
+ BlockContents contents_;
+ const char* data_; // contents_.data.data()
+ size_t size_; // contents_.data.size()
+ uint32_t restart_offset_; // Offset in data_ of restart array
+ uint32_t num_restarts_;
+ std::unique_ptr<BlockReadAmpBitmap> read_amp_bitmap_;
+ DataBlockHashIndex data_block_hash_index_;
+};
+
+// A `BlockIter` iterates over the entries in a `Block`'s data buffer. The
+// format of this data buffer is an uncompressed, sorted sequence of key-value
+// pairs (see `Block` API for more details).
+//
+// Notably, the keys may either be in internal key format or user key format.
+// Subclasses are responsible for configuring the key format.
+//
+// `BlockIter` intends to provide final overrides for all of
+// `InternalIteratorBase` functions that can move the iterator. It does
+// this to guarantee `UpdateKey()` is called exactly once after each key
+// movement potentially visible to users. In this step, the key is prepared
+// (e.g., serialized if global seqno is in effect) so it can be returned
+// immediately when the user asks for it via calling `key() const`.
+//
+// For its subclasses, it provides protected variants of the above-mentioned
+// final-overridden methods. They are named with the "Impl" suffix, e.g.,
+// `Seek()` logic would be implemented by subclasses in `SeekImpl()`. These
+// "Impl" functions are responsible for positioning `raw_key_` but not
+// invoking `UpdateKey()`.
+template <class TValue>
+class BlockIter : public InternalIteratorBase<TValue> {
+ public:
+ // Makes Valid() return false, status() return `s`, and Seek()/Prev()/etc do
+ // nothing. Calls cleanup functions.
+ virtual void Invalidate(const Status& s) {
+ // Assert that the BlockIter is never deleted while Pinning is Enabled.
+ assert(!pinned_iters_mgr_ || !pinned_iters_mgr_->PinningEnabled());
+
+ data_ = nullptr;
+ current_ = restarts_;
+ status_ = s;
+
+ // Call cleanup callbacks.
+ Cleanable::Reset();
+ }
+
+ bool Valid() const override { return current_ < restarts_; }
+
+ virtual void SeekToFirst() override final {
+ SeekToFirstImpl();
+ UpdateKey();
+ }
+
+ virtual void SeekToLast() override final {
+ SeekToLastImpl();
+ UpdateKey();
+ }
+
+ virtual void Seek(const Slice& target) override final {
+ SeekImpl(target);
+ UpdateKey();
+ }
+
+ virtual void SeekForPrev(const Slice& target) override final {
+ SeekForPrevImpl(target);
+ UpdateKey();
+ }
+
+ virtual void Next() override final {
+ NextImpl();
+ UpdateKey();
+ }
+
+ virtual bool NextAndGetResult(IterateResult* result) override final {
+ // This does not need to call `UpdateKey()` as the parent class only has
+ // access to the `UpdateKey()`-invoking functions.
+ return InternalIteratorBase<TValue>::NextAndGetResult(result);
+ }
+
+ virtual void Prev() override final {
+ PrevImpl();
+ UpdateKey();
+ }
+
+ Status status() const override { return status_; }
+ Slice key() const override {
+ assert(Valid());
+ return key_;
+ }
+
+#ifndef NDEBUG
+ ~BlockIter() override {
+ // Assert that the BlockIter is never deleted while Pinning is Enabled.
+ assert(!pinned_iters_mgr_ ||
+ (pinned_iters_mgr_ && !pinned_iters_mgr_->PinningEnabled()));
+ status_.PermitUncheckedError();
+ }
+ void SetPinnedItersMgr(PinnedIteratorsManager* pinned_iters_mgr) override {
+ pinned_iters_mgr_ = pinned_iters_mgr;
+ }
+ PinnedIteratorsManager* pinned_iters_mgr_ = nullptr;
+#endif
+
+ bool IsKeyPinned() const override {
+ return block_contents_pinned_ && key_pinned_;
+ }
+
+ bool IsValuePinned() const override { return block_contents_pinned_; }
+
+ size_t TEST_CurrentEntrySize() { return NextEntryOffset() - current_; }
+
+ uint32_t ValueOffset() const {
+ return static_cast<uint32_t>(value_.data() - data_);
+ }
+
+ void SetCacheHandle(Cache::Handle* handle) { cache_handle_ = handle; }
+
+ Cache::Handle* cache_handle() { return cache_handle_; }
+
+ protected:
+ std::unique_ptr<InternalKeyComparator> icmp_;
+ const char* data_; // underlying block contents
+ uint32_t num_restarts_; // Number of uint32_t entries in restart array
+
+ // Index of restart block in which current_ or current_-1 falls
+ uint32_t restart_index_;
+ uint32_t restarts_; // Offset of restart array (list of fixed32)
+ // current_ is offset in data_ of current entry. >= restarts_ if !Valid
+ uint32_t current_;
+ // Raw key from block.
+ IterKey raw_key_;
+ // Buffer for key data when global seqno assignment is enabled.
+ IterKey key_buf_;
+ Slice value_;
+ Status status_;
+ // Key to be exposed to users.
+ Slice key_;
+ bool key_pinned_;
+ // Whether the block data is guaranteed to outlive this iterator, and
+ // as long as the cleanup functions are transferred to another class,
+ // e.g. PinnableSlice, the pointer to the bytes will still be valid.
+ bool block_contents_pinned_;
+ SequenceNumber global_seqno_;
+
+ virtual void SeekToFirstImpl() = 0;
+ virtual void SeekToLastImpl() = 0;
+ virtual void SeekImpl(const Slice& target) = 0;
+ virtual void SeekForPrevImpl(const Slice& target) = 0;
+ virtual void NextImpl() = 0;
+
+ virtual void PrevImpl() = 0;
+
+ template <typename DecodeEntryFunc>
+ inline bool ParseNextKey(bool* is_shared);
+
+ void InitializeBase(const Comparator* raw_ucmp, const char* data,
+ uint32_t restarts, uint32_t num_restarts,
+ SequenceNumber global_seqno, bool block_contents_pinned) {
+ assert(data_ == nullptr); // Ensure it is called only once
+ assert(num_restarts > 0); // Ensure the param is valid
+
+ icmp_ = std::make_unique<InternalKeyComparator>(raw_ucmp);
+ data_ = data;
+ restarts_ = restarts;
+ num_restarts_ = num_restarts;
+ current_ = restarts_;
+ restart_index_ = num_restarts_;
+ global_seqno_ = global_seqno;
+ block_contents_pinned_ = block_contents_pinned;
+ cache_handle_ = nullptr;
+ }
+
+ // Must be called every time a key is found that needs to be returned to user,
+ // and may be called when no key is found (as a no-op). Updates `key_`,
+ // `key_buf_`, and `key_pinned_` with info about the found key.
+ void UpdateKey() {
+ key_buf_.Clear();
+ if (!Valid()) {
+ return;
+ }
+ if (raw_key_.IsUserKey()) {
+ assert(global_seqno_ == kDisableGlobalSequenceNumber);
+ key_ = raw_key_.GetUserKey();
+ key_pinned_ = raw_key_.IsKeyPinned();
+ } else if (global_seqno_ == kDisableGlobalSequenceNumber) {
+ key_ = raw_key_.GetInternalKey();
+ key_pinned_ = raw_key_.IsKeyPinned();
+ } else {
+ key_buf_.SetInternalKey(raw_key_.GetUserKey(), global_seqno_,
+ ExtractValueType(raw_key_.GetInternalKey()));
+ key_ = key_buf_.GetInternalKey();
+ key_pinned_ = false;
+ }
+ }
+
+ // Returns the result of `Comparator::Compare()`, where the appropriate
+ // comparator is used for the block contents, the LHS argument is the current
+ // key with global seqno applied, and the RHS argument is `other`.
+ int CompareCurrentKey(const Slice& other) {
+ if (raw_key_.IsUserKey()) {
+ assert(global_seqno_ == kDisableGlobalSequenceNumber);
+ return icmp_->user_comparator()->Compare(raw_key_.GetUserKey(), other);
+ } else if (global_seqno_ == kDisableGlobalSequenceNumber) {
+ return icmp_->Compare(raw_key_.GetInternalKey(), other);
+ }
+ return icmp_->Compare(raw_key_.GetInternalKey(), global_seqno_, other,
+ kDisableGlobalSequenceNumber);
+ }
+
+ private:
+ // Store the cache handle, if the block is cached. We need this since the
+ // only other place the handle is stored is as an argument to the Cleanable
+ // function callback, which is hard to retrieve. When multiple value
+ // PinnableSlices reference the block, they need the cache handle in order
+ // to bump up the ref count
+ Cache::Handle* cache_handle_;
+
+ public:
+ // Return the offset in data_ just past the end of the current entry.
+ inline uint32_t NextEntryOffset() const {
+ // NOTE: We don't support blocks bigger than 2GB
+ return static_cast<uint32_t>((value_.data() + value_.size()) - data_);
+ }
+
+ uint32_t GetRestartPoint(uint32_t index) {
+ assert(index < num_restarts_);
+ return DecodeFixed32(data_ + restarts_ + index * sizeof(uint32_t));
+ }
+
+ void SeekToRestartPoint(uint32_t index) {
+ raw_key_.Clear();
+ restart_index_ = index;
+ // current_ will be fixed by ParseNextKey();
+
+ // ParseNextKey() starts at the end of value_, so set value_ accordingly
+ uint32_t offset = GetRestartPoint(index);
+ value_ = Slice(data_ + offset, 0);
+ }
+
+ void CorruptionError();
+
+ protected:
+ template <typename DecodeKeyFunc>
+ inline bool BinarySeek(const Slice& target, uint32_t* index,
+ bool* is_index_key_result);
+
+ void FindKeyAfterBinarySeek(const Slice& target, uint32_t index,
+ bool is_index_key_result);
+};
+
+class DataBlockIter final : public BlockIter<Slice> {
+ public:
+ DataBlockIter()
+ : BlockIter(), read_amp_bitmap_(nullptr), last_bitmap_offset_(0) {}
+ DataBlockIter(const Comparator* raw_ucmp, const char* data, uint32_t restarts,
+ uint32_t num_restarts, SequenceNumber global_seqno,
+ BlockReadAmpBitmap* read_amp_bitmap, bool block_contents_pinned,
+ DataBlockHashIndex* data_block_hash_index)
+ : DataBlockIter() {
+ Initialize(raw_ucmp, data, restarts, num_restarts, global_seqno,
+ read_amp_bitmap, block_contents_pinned, data_block_hash_index);
+ }
+ void Initialize(const Comparator* raw_ucmp, const char* data,
+ uint32_t restarts, uint32_t num_restarts,
+ SequenceNumber global_seqno,
+ BlockReadAmpBitmap* read_amp_bitmap,
+ bool block_contents_pinned,
+ DataBlockHashIndex* data_block_hash_index) {
+ InitializeBase(raw_ucmp, data, restarts, num_restarts, global_seqno,
+ block_contents_pinned);
+ raw_key_.SetIsUserKey(false);
+ read_amp_bitmap_ = read_amp_bitmap;
+ last_bitmap_offset_ = current_ + 1;
+ data_block_hash_index_ = data_block_hash_index;
+ }
+
+ Slice value() const override {
+ assert(Valid());
+ if (read_amp_bitmap_ && current_ < restarts_ &&
+ current_ != last_bitmap_offset_) {
+ read_amp_bitmap_->Mark(current_ /* current entry offset */,
+ NextEntryOffset() - 1);
+ last_bitmap_offset_ = current_;
+ }
+ return value_;
+ }
+
+ inline bool SeekForGet(const Slice& target) {
+ if (!data_block_hash_index_) {
+ SeekImpl(target);
+ UpdateKey();
+ return true;
+ }
+ bool res = SeekForGetImpl(target);
+ UpdateKey();
+ return res;
+ }
+
+ void Invalidate(const Status& s) override {
+ BlockIter::Invalidate(s);
+ // Clear prev entries cache.
+ prev_entries_keys_buff_.clear();
+ prev_entries_.clear();
+ prev_entries_idx_ = -1;
+ }
+
+ protected:
+ friend Block;
+ inline bool ParseNextDataKey(bool* is_shared);
+ void SeekToFirstImpl() override;
+ void SeekToLastImpl() override;
+ void SeekImpl(const Slice& target) override;
+ void SeekForPrevImpl(const Slice& target) override;
+ void NextImpl() override;
+ void PrevImpl() override;
+
+ private:
+ // read-amp bitmap
+ BlockReadAmpBitmap* read_amp_bitmap_;
+ // last `current_` value we report to read-amp bitmp
+ mutable uint32_t last_bitmap_offset_;
+ struct CachedPrevEntry {
+ explicit CachedPrevEntry(uint32_t _offset, const char* _key_ptr,
+ size_t _key_offset, size_t _key_size, Slice _value)
+ : offset(_offset),
+ key_ptr(_key_ptr),
+ key_offset(_key_offset),
+ key_size(_key_size),
+ value(_value) {}
+
+ // offset of entry in block
+ uint32_t offset;
+ // Pointer to key data in block (nullptr if key is delta-encoded)
+ const char* key_ptr;
+ // offset of key in prev_entries_keys_buff_ (0 if key_ptr is not nullptr)
+ size_t key_offset;
+ // size of key
+ size_t key_size;
+ // value slice pointing to data in block
+ Slice value;
+ };
+ std::string prev_entries_keys_buff_;
+ std::vector<CachedPrevEntry> prev_entries_;
+ int32_t prev_entries_idx_ = -1;
+
+ DataBlockHashIndex* data_block_hash_index_;
+
+ bool SeekForGetImpl(const Slice& target);
+};
+
+// Iterator over MetaBlocks. MetaBlocks are similar to Data Blocks and
+// are used to store Properties associated with table.
+// Meta blocks always store user keys (no sequence number) and always
+// use the BytewiseComparator. Additionally, MetaBlock accesses are
+// not recorded in the Statistics or for Read-Amplification.
+class MetaBlockIter final : public BlockIter<Slice> {
+ public:
+ MetaBlockIter() : BlockIter() { raw_key_.SetIsUserKey(true); }
+ void Initialize(const char* data, uint32_t restarts, uint32_t num_restarts,
+ bool block_contents_pinned) {
+ // Initializes the iterator with a BytewiseComparator and
+ // the raw key being a user key.
+ InitializeBase(BytewiseComparator(), data, restarts, num_restarts,
+ kDisableGlobalSequenceNumber, block_contents_pinned);
+ raw_key_.SetIsUserKey(true);
+ }
+
+ Slice value() const override {
+ assert(Valid());
+ return value_;
+ }
+
+ protected:
+ void SeekToFirstImpl() override;
+ void SeekToLastImpl() override;
+ void SeekImpl(const Slice& target) override;
+ void SeekForPrevImpl(const Slice& target) override;
+ void NextImpl() override;
+ void PrevImpl() override;
+};
+
+class IndexBlockIter final : public BlockIter<IndexValue> {
+ public:
+ IndexBlockIter() : BlockIter(), prefix_index_(nullptr) {}
+
+ // key_includes_seq, default true, means that the keys are in internal key
+ // format.
+ // value_is_full, default true, means that no delta encoding is
+ // applied to values.
+ void Initialize(const Comparator* raw_ucmp, const char* data,
+ uint32_t restarts, uint32_t num_restarts,
+ SequenceNumber global_seqno, BlockPrefixIndex* prefix_index,
+ bool have_first_key, bool key_includes_seq,
+ bool value_is_full, bool block_contents_pinned) {
+ InitializeBase(raw_ucmp, data, restarts, num_restarts,
+ kDisableGlobalSequenceNumber, block_contents_pinned);
+ raw_key_.SetIsUserKey(!key_includes_seq);
+ prefix_index_ = prefix_index;
+ value_delta_encoded_ = !value_is_full;
+ have_first_key_ = have_first_key;
+ if (have_first_key_ && global_seqno != kDisableGlobalSequenceNumber) {
+ global_seqno_state_.reset(new GlobalSeqnoState(global_seqno));
+ } else {
+ global_seqno_state_.reset();
+ }
+ }
+
+ Slice user_key() const override {
+ assert(Valid());
+ return raw_key_.GetUserKey();
+ }
+
+ IndexValue value() const override {
+ assert(Valid());
+ if (value_delta_encoded_ || global_seqno_state_ != nullptr) {
+ return decoded_value_;
+ } else {
+ IndexValue entry;
+ Slice v = value_;
+ Status decode_s __attribute__((__unused__)) =
+ entry.DecodeFrom(&v, have_first_key_, nullptr);
+ assert(decode_s.ok());
+ return entry;
+ }
+ }
+
+ bool IsValuePinned() const override {
+ return global_seqno_state_ != nullptr ? false : BlockIter::IsValuePinned();
+ }
+
+ protected:
+ // IndexBlockIter follows a different contract for prefix iterator
+ // from data iterators.
+ // If prefix of the seek key `target` exists in the file, it must
+ // return the same result as total order seek.
+ // If the prefix of `target` doesn't exist in the file, it can either
+ // return the result of total order seek, or set both of Valid() = false
+ // and status() = NotFound().
+ void SeekImpl(const Slice& target) override;
+
+ void SeekForPrevImpl(const Slice&) override {
+ assert(false);
+ current_ = restarts_;
+ restart_index_ = num_restarts_;
+ status_ = Status::InvalidArgument(
+ "RocksDB internal error: should never call SeekForPrev() on index "
+ "blocks");
+ raw_key_.Clear();
+ value_.clear();
+ }
+
+ void PrevImpl() override;
+
+ void NextImpl() override;
+
+ void SeekToFirstImpl() override;
+
+ void SeekToLastImpl() override;
+
+ private:
+ bool value_delta_encoded_;
+ bool have_first_key_; // value includes first_internal_key
+ BlockPrefixIndex* prefix_index_;
+ // Whether the value is delta encoded. In that case the value is assumed to be
+ // BlockHandle. The first value in each restart interval is the full encoded
+ // BlockHandle; the restart of encoded size part of the BlockHandle. The
+ // offset of delta encoded BlockHandles is computed by adding the size of
+ // previous delta encoded values in the same restart interval to the offset of
+ // the first value in that restart interval.
+ IndexValue decoded_value_;
+
+ // When sequence number overwriting is enabled, this struct contains the seqno
+ // to overwrite with, and current first_internal_key with overwritten seqno.
+ // This is rarely used, so we put it behind a pointer and only allocate when
+ // needed.
+ struct GlobalSeqnoState {
+ // First internal key according to current index entry, but with sequence
+ // number overwritten to global_seqno.
+ IterKey first_internal_key;
+ SequenceNumber global_seqno;
+
+ explicit GlobalSeqnoState(SequenceNumber seqno) : global_seqno(seqno) {}
+ };
+
+ std::unique_ptr<GlobalSeqnoState> global_seqno_state_;
+
+ // Set *prefix_may_exist to false if no key possibly share the same prefix
+ // as `target`. If not set, the result position should be the same as total
+ // order Seek.
+ bool PrefixSeek(const Slice& target, uint32_t* index, bool* prefix_may_exist);
+ // Set *prefix_may_exist to false if no key can possibly share the same
+ // prefix as `target`. If not set, the result position should be the same
+ // as total order seek.
+ bool BinaryBlockIndexSeek(const Slice& target, uint32_t* block_ids,
+ uint32_t left, uint32_t right, uint32_t* index,
+ bool* prefix_may_exist);
+ inline int CompareBlockKey(uint32_t block_index, const Slice& target);
+
+ inline bool ParseNextIndexKey();
+
+ // When value_delta_encoded_ is enabled it decodes the value which is assumed
+ // to be BlockHandle and put it to decoded_value_
+ inline void DecodeCurrentValue(bool is_shared);
+};
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/block_based_table_builder.cc b/src/rocksdb/table/block_based/block_based_table_builder.cc
new file mode 100644
index 000000000..fed69af07
--- /dev/null
+++ b/src/rocksdb/table/block_based/block_based_table_builder.cc
@@ -0,0 +1,2096 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#include "table/block_based/block_based_table_builder.h"
+
+#include <assert.h>
+#include <stdio.h>
+
+#include <atomic>
+#include <list>
+#include <map>
+#include <memory>
+#include <numeric>
+#include <string>
+#include <unordered_map>
+#include <utility>
+
+#include "cache/cache_entry_roles.h"
+#include "cache/cache_helpers.h"
+#include "cache/cache_key.h"
+#include "cache/cache_reservation_manager.h"
+#include "db/dbformat.h"
+#include "index_builder.h"
+#include "logging/logging.h"
+#include "memory/memory_allocator.h"
+#include "rocksdb/cache.h"
+#include "rocksdb/comparator.h"
+#include "rocksdb/env.h"
+#include "rocksdb/filter_policy.h"
+#include "rocksdb/flush_block_policy.h"
+#include "rocksdb/merge_operator.h"
+#include "rocksdb/table.h"
+#include "rocksdb/types.h"
+#include "table/block_based/block.h"
+#include "table/block_based/block_based_table_factory.h"
+#include "table/block_based/block_based_table_reader.h"
+#include "table/block_based/block_builder.h"
+#include "table/block_based/block_like_traits.h"
+#include "table/block_based/filter_block.h"
+#include "table/block_based/filter_policy_internal.h"
+#include "table/block_based/full_filter_block.h"
+#include "table/block_based/partitioned_filter_block.h"
+#include "table/format.h"
+#include "table/meta_blocks.h"
+#include "table/table_builder.h"
+#include "util/coding.h"
+#include "util/compression.h"
+#include "util/stop_watch.h"
+#include "util/string_util.h"
+#include "util/work_queue.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+extern const std::string kHashIndexPrefixesBlock;
+extern const std::string kHashIndexPrefixesMetadataBlock;
+
+// Without anonymous namespace here, we fail the warning -Wmissing-prototypes
+namespace {
+
+constexpr size_t kBlockTrailerSize = BlockBasedTable::kBlockTrailerSize;
+
+// Create a filter block builder based on its type.
+FilterBlockBuilder* CreateFilterBlockBuilder(
+ const ImmutableCFOptions& /*opt*/, const MutableCFOptions& mopt,
+ const FilterBuildingContext& context,
+ const bool use_delta_encoding_for_index_values,
+ PartitionedIndexBuilder* const p_index_builder) {
+ const BlockBasedTableOptions& table_opt = context.table_options;
+ assert(table_opt.filter_policy); // precondition
+
+ FilterBitsBuilder* filter_bits_builder =
+ BloomFilterPolicy::GetBuilderFromContext(context);
+ if (filter_bits_builder == nullptr) {
+ return nullptr;
+ } else {
+ if (table_opt.partition_filters) {
+ assert(p_index_builder != nullptr);
+ // Since after partition cut request from filter builder it takes time
+ // until index builder actully cuts the partition, until the end of a
+ // data block potentially with many keys, we take the lower bound as
+ // partition size.
+ assert(table_opt.block_size_deviation <= 100);
+ auto partition_size =
+ static_cast<uint32_t>(((table_opt.metadata_block_size *
+ (100 - table_opt.block_size_deviation)) +
+ 99) /
+ 100);
+ partition_size = std::max(partition_size, static_cast<uint32_t>(1));
+ return new PartitionedFilterBlockBuilder(
+ mopt.prefix_extractor.get(), table_opt.whole_key_filtering,
+ filter_bits_builder, table_opt.index_block_restart_interval,
+ use_delta_encoding_for_index_values, p_index_builder, partition_size);
+ } else {
+ return new FullFilterBlockBuilder(mopt.prefix_extractor.get(),
+ table_opt.whole_key_filtering,
+ filter_bits_builder);
+ }
+ }
+}
+
+bool GoodCompressionRatio(size_t compressed_size, size_t uncomp_size) {
+ // Check to see if compressed less than 12.5%
+ return compressed_size < uncomp_size - (uncomp_size / 8u);
+}
+
+} // namespace
+
+// format_version is the block format as defined in include/rocksdb/table.h
+Slice CompressBlock(const Slice& uncompressed_data, const CompressionInfo& info,
+ CompressionType* type, uint32_t format_version,
+ bool do_sample, std::string* compressed_output,
+ std::string* sampled_output_fast,
+ std::string* sampled_output_slow) {
+ assert(type);
+ assert(compressed_output);
+ assert(compressed_output->empty());
+
+ // If requested, we sample one in every N block with a
+ // fast and slow compression algorithm and report the stats.
+ // The users can use these stats to decide if it is worthwhile
+ // enabling compression and they also get a hint about which
+ // compression algorithm wil be beneficial.
+ if (do_sample && info.SampleForCompression() &&
+ Random::GetTLSInstance()->OneIn(
+ static_cast<int>(info.SampleForCompression()))) {
+ // Sampling with a fast compression algorithm
+ if (sampled_output_fast && (LZ4_Supported() || Snappy_Supported())) {
+ CompressionType c =
+ LZ4_Supported() ? kLZ4Compression : kSnappyCompression;
+ CompressionContext context(c);
+ CompressionOptions options;
+ CompressionInfo info_tmp(options, context,
+ CompressionDict::GetEmptyDict(), c,
+ info.SampleForCompression());
+
+ CompressData(uncompressed_data, info_tmp,
+ GetCompressFormatForVersion(format_version),
+ sampled_output_fast);
+ }
+
+ // Sampling with a slow but high-compression algorithm
+ if (sampled_output_slow && (ZSTD_Supported() || Zlib_Supported())) {
+ CompressionType c = ZSTD_Supported() ? kZSTD : kZlibCompression;
+ CompressionContext context(c);
+ CompressionOptions options;
+ CompressionInfo info_tmp(options, context,
+ CompressionDict::GetEmptyDict(), c,
+ info.SampleForCompression());
+
+ CompressData(uncompressed_data, info_tmp,
+ GetCompressFormatForVersion(format_version),
+ sampled_output_slow);
+ }
+ }
+
+ if (info.type() == kNoCompression) {
+ *type = kNoCompression;
+ return uncompressed_data;
+ }
+
+ // Actually compress the data; if the compression method is not supported,
+ // or the compression fails etc., just fall back to uncompressed
+ if (!CompressData(uncompressed_data, info,
+ GetCompressFormatForVersion(format_version),
+ compressed_output)) {
+ *type = kNoCompression;
+ return uncompressed_data;
+ }
+
+ // Check the compression ratio; if it's not good enough, just fall back to
+ // uncompressed
+ if (!GoodCompressionRatio(compressed_output->size(),
+ uncompressed_data.size())) {
+ *type = kNoCompression;
+ return uncompressed_data;
+ }
+
+ *type = info.type();
+ return *compressed_output;
+}
+
+// kBlockBasedTableMagicNumber was picked by running
+// echo rocksdb.table.block_based | sha1sum
+// and taking the leading 64 bits.
+// Please note that kBlockBasedTableMagicNumber may also be accessed by other
+// .cc files
+// for that reason we declare it extern in the header but to get the space
+// allocated
+// it must be not extern in one place.
+const uint64_t kBlockBasedTableMagicNumber = 0x88e241b785f4cff7ull;
+// We also support reading and writing legacy block based table format (for
+// backwards compatibility)
+const uint64_t kLegacyBlockBasedTableMagicNumber = 0xdb4775248b80fb57ull;
+
+// A collector that collects properties of interest to block-based table.
+// For now this class looks heavy-weight since we only write one additional
+// property.
+// But in the foreseeable future, we will add more and more properties that are
+// specific to block-based table.
+class BlockBasedTableBuilder::BlockBasedTablePropertiesCollector
+ : public IntTblPropCollector {
+ public:
+ explicit BlockBasedTablePropertiesCollector(
+ BlockBasedTableOptions::IndexType index_type, bool whole_key_filtering,
+ bool prefix_filtering)
+ : index_type_(index_type),
+ whole_key_filtering_(whole_key_filtering),
+ prefix_filtering_(prefix_filtering) {}
+
+ Status InternalAdd(const Slice& /*key*/, const Slice& /*value*/,
+ uint64_t /*file_size*/) override {
+ // Intentionally left blank. Have no interest in collecting stats for
+ // individual key/value pairs.
+ return Status::OK();
+ }
+
+ virtual void BlockAdd(uint64_t /* block_uncomp_bytes */,
+ uint64_t /* block_compressed_bytes_fast */,
+ uint64_t /* block_compressed_bytes_slow */) override {
+ // Intentionally left blank. No interest in collecting stats for
+ // blocks.
+ return;
+ }
+
+ Status Finish(UserCollectedProperties* properties) override {
+ std::string val;
+ PutFixed32(&val, static_cast<uint32_t>(index_type_));
+ properties->insert({BlockBasedTablePropertyNames::kIndexType, val});
+ properties->insert({BlockBasedTablePropertyNames::kWholeKeyFiltering,
+ whole_key_filtering_ ? kPropTrue : kPropFalse});
+ properties->insert({BlockBasedTablePropertyNames::kPrefixFiltering,
+ prefix_filtering_ ? kPropTrue : kPropFalse});
+ return Status::OK();
+ }
+
+ // The name of the properties collector can be used for debugging purpose.
+ const char* Name() const override {
+ return "BlockBasedTablePropertiesCollector";
+ }
+
+ UserCollectedProperties GetReadableProperties() const override {
+ // Intentionally left blank.
+ return UserCollectedProperties();
+ }
+
+ private:
+ BlockBasedTableOptions::IndexType index_type_;
+ bool whole_key_filtering_;
+ bool prefix_filtering_;
+};
+
+struct BlockBasedTableBuilder::Rep {
+ const ImmutableOptions ioptions;
+ const MutableCFOptions moptions;
+ const BlockBasedTableOptions table_options;
+ const InternalKeyComparator& internal_comparator;
+ WritableFileWriter* file;
+ std::atomic<uint64_t> offset;
+ size_t alignment;
+ BlockBuilder data_block;
+ // Buffers uncompressed data blocks to replay later. Needed when
+ // compression dictionary is enabled so we can finalize the dictionary before
+ // compressing any data blocks.
+ std::vector<std::string> data_block_buffers;
+ BlockBuilder range_del_block;
+
+ InternalKeySliceTransform internal_prefix_transform;
+ std::unique_ptr<IndexBuilder> index_builder;
+ PartitionedIndexBuilder* p_index_builder_ = nullptr;
+
+ std::string last_key;
+ const Slice* first_key_in_next_block = nullptr;
+ CompressionType compression_type;
+ uint64_t sample_for_compression;
+ std::atomic<uint64_t> compressible_input_data_bytes;
+ std::atomic<uint64_t> uncompressible_input_data_bytes;
+ std::atomic<uint64_t> sampled_input_data_bytes;
+ std::atomic<uint64_t> sampled_output_slow_data_bytes;
+ std::atomic<uint64_t> sampled_output_fast_data_bytes;
+ CompressionOptions compression_opts;
+ std::unique_ptr<CompressionDict> compression_dict;
+ std::vector<std::unique_ptr<CompressionContext>> compression_ctxs;
+ std::vector<std::unique_ptr<UncompressionContext>> verify_ctxs;
+ std::unique_ptr<UncompressionDict> verify_dict;
+
+ size_t data_begin_offset = 0;
+
+ TableProperties props;
+
+ // States of the builder.
+ //
+ // - `kBuffered`: This is the initial state where zero or more data blocks are
+ // accumulated uncompressed in-memory. From this state, call
+ // `EnterUnbuffered()` to finalize the compression dictionary if enabled,
+ // compress/write out any buffered blocks, and proceed to the `kUnbuffered`
+ // state.
+ //
+ // - `kUnbuffered`: This is the state when compression dictionary is finalized
+ // either because it wasn't enabled in the first place or it's been created
+ // from sampling previously buffered data. In this state, blocks are simply
+ // compressed/written out as they fill up. From this state, call `Finish()`
+ // to complete the file (write meta-blocks, etc.), or `Abandon()` to delete
+ // the partially created file.
+ //
+ // - `kClosed`: This indicates either `Finish()` or `Abandon()` has been
+ // called, so the table builder is no longer usable. We must be in this
+ // state by the time the destructor runs.
+ enum class State {
+ kBuffered,
+ kUnbuffered,
+ kClosed,
+ };
+ State state;
+ // `kBuffered` state is allowed only as long as the buffering of uncompressed
+ // data blocks (see `data_block_buffers`) does not exceed `buffer_limit`.
+ uint64_t buffer_limit;
+ std::shared_ptr<CacheReservationManager>
+ compression_dict_buffer_cache_res_mgr;
+ const bool use_delta_encoding_for_index_values;
+ std::unique_ptr<FilterBlockBuilder> filter_builder;
+ OffsetableCacheKey base_cache_key;
+ const TableFileCreationReason reason;
+
+ BlockHandle pending_handle; // Handle to add to index block
+
+ std::string compressed_output;
+ std::unique_ptr<FlushBlockPolicy> flush_block_policy;
+
+ std::vector<std::unique_ptr<IntTblPropCollector>> table_properties_collectors;
+
+ std::unique_ptr<ParallelCompressionRep> pc_rep;
+
+ uint64_t get_offset() { return offset.load(std::memory_order_relaxed); }
+ void set_offset(uint64_t o) { offset.store(o, std::memory_order_relaxed); }
+
+ bool IsParallelCompressionEnabled() const {
+ return compression_opts.parallel_threads > 1;
+ }
+
+ Status GetStatus() {
+ // We need to make modifications of status visible when status_ok is set
+ // to false, and this is ensured by status_mutex, so no special memory
+ // order for status_ok is required.
+ if (status_ok.load(std::memory_order_relaxed)) {
+ return Status::OK();
+ } else {
+ return CopyStatus();
+ }
+ }
+
+ Status CopyStatus() {
+ std::lock_guard<std::mutex> lock(status_mutex);
+ return status;
+ }
+
+ IOStatus GetIOStatus() {
+ // We need to make modifications of io_status visible when status_ok is set
+ // to false, and this is ensured by io_status_mutex, so no special memory
+ // order for io_status_ok is required.
+ if (io_status_ok.load(std::memory_order_relaxed)) {
+ return IOStatus::OK();
+ } else {
+ return CopyIOStatus();
+ }
+ }
+
+ IOStatus CopyIOStatus() {
+ std::lock_guard<std::mutex> lock(io_status_mutex);
+ return io_status;
+ }
+
+ // Never erase an existing status that is not OK.
+ void SetStatus(Status s) {
+ if (!s.ok() && status_ok.load(std::memory_order_relaxed)) {
+ // Locking is an overkill for non compression_opts.parallel_threads
+ // case but since it's unlikely that s is not OK, we take this cost
+ // to be simplicity.
+ std::lock_guard<std::mutex> lock(status_mutex);
+ status = s;
+ status_ok.store(false, std::memory_order_relaxed);
+ }
+ }
+
+ // Never erase an existing I/O status that is not OK.
+ // Calling this will also SetStatus(ios)
+ void SetIOStatus(IOStatus ios) {
+ if (!ios.ok() && io_status_ok.load(std::memory_order_relaxed)) {
+ // Locking is an overkill for non compression_opts.parallel_threads
+ // case but since it's unlikely that s is not OK, we take this cost
+ // to be simplicity.
+ std::lock_guard<std::mutex> lock(io_status_mutex);
+ io_status = ios;
+ io_status_ok.store(false, std::memory_order_relaxed);
+ }
+ SetStatus(ios);
+ }
+
+ Rep(const BlockBasedTableOptions& table_opt, const TableBuilderOptions& tbo,
+ WritableFileWriter* f)
+ : ioptions(tbo.ioptions),
+ moptions(tbo.moptions),
+ table_options(table_opt),
+ internal_comparator(tbo.internal_comparator),
+ file(f),
+ offset(0),
+ alignment(table_options.block_align
+ ? std::min(static_cast<size_t>(table_options.block_size),
+ kDefaultPageSize)
+ : 0),
+ data_block(table_options.block_restart_interval,
+ table_options.use_delta_encoding,
+ false /* use_value_delta_encoding */,
+ tbo.internal_comparator.user_comparator()
+ ->CanKeysWithDifferentByteContentsBeEqual()
+ ? BlockBasedTableOptions::kDataBlockBinarySearch
+ : table_options.data_block_index_type,
+ table_options.data_block_hash_table_util_ratio),
+ range_del_block(1 /* block_restart_interval */),
+ internal_prefix_transform(tbo.moptions.prefix_extractor.get()),
+ compression_type(tbo.compression_type),
+ sample_for_compression(tbo.moptions.sample_for_compression),
+ compressible_input_data_bytes(0),
+ uncompressible_input_data_bytes(0),
+ sampled_input_data_bytes(0),
+ sampled_output_slow_data_bytes(0),
+ sampled_output_fast_data_bytes(0),
+ compression_opts(tbo.compression_opts),
+ compression_dict(),
+ compression_ctxs(tbo.compression_opts.parallel_threads),
+ verify_ctxs(tbo.compression_opts.parallel_threads),
+ verify_dict(),
+ state((tbo.compression_opts.max_dict_bytes > 0) ? State::kBuffered
+ : State::kUnbuffered),
+ use_delta_encoding_for_index_values(table_opt.format_version >= 4 &&
+ !table_opt.block_align),
+ reason(tbo.reason),
+ flush_block_policy(
+ table_options.flush_block_policy_factory->NewFlushBlockPolicy(
+ table_options, data_block)),
+ status_ok(true),
+ io_status_ok(true) {
+ if (tbo.target_file_size == 0) {
+ buffer_limit = compression_opts.max_dict_buffer_bytes;
+ } else if (compression_opts.max_dict_buffer_bytes == 0) {
+ buffer_limit = tbo.target_file_size;
+ } else {
+ buffer_limit = std::min(tbo.target_file_size,
+ compression_opts.max_dict_buffer_bytes);
+ }
+
+ const auto compress_dict_build_buffer_charged =
+ table_options.cache_usage_options.options_overrides
+ .at(CacheEntryRole::kCompressionDictionaryBuildingBuffer)
+ .charged;
+ if (table_options.block_cache &&
+ (compress_dict_build_buffer_charged ==
+ CacheEntryRoleOptions::Decision::kEnabled ||
+ compress_dict_build_buffer_charged ==
+ CacheEntryRoleOptions::Decision::kFallback)) {
+ compression_dict_buffer_cache_res_mgr =
+ std::make_shared<CacheReservationManagerImpl<
+ CacheEntryRole::kCompressionDictionaryBuildingBuffer>>(
+ table_options.block_cache);
+ } else {
+ compression_dict_buffer_cache_res_mgr = nullptr;
+ }
+
+ for (uint32_t i = 0; i < compression_opts.parallel_threads; i++) {
+ compression_ctxs[i].reset(new CompressionContext(compression_type));
+ }
+ if (table_options.index_type ==
+ BlockBasedTableOptions::kTwoLevelIndexSearch) {
+ p_index_builder_ = PartitionedIndexBuilder::CreateIndexBuilder(
+ &internal_comparator, use_delta_encoding_for_index_values,
+ table_options);
+ index_builder.reset(p_index_builder_);
+ } else {
+ index_builder.reset(IndexBuilder::CreateIndexBuilder(
+ table_options.index_type, &internal_comparator,
+ &this->internal_prefix_transform, use_delta_encoding_for_index_values,
+ table_options));
+ }
+ if (ioptions.optimize_filters_for_hits && tbo.is_bottommost) {
+ // Apply optimize_filters_for_hits setting here when applicable by
+ // skipping filter generation
+ filter_builder.reset();
+ } else if (tbo.skip_filters) {
+ // For SstFileWriter skip_filters
+ filter_builder.reset();
+ } else if (!table_options.filter_policy) {
+ // Null filter_policy -> no filter
+ filter_builder.reset();
+ } else {
+ FilterBuildingContext filter_context(table_options);
+
+ filter_context.info_log = ioptions.logger;
+ filter_context.column_family_name = tbo.column_family_name;
+ filter_context.reason = reason;
+
+ // Only populate other fields if known to be in LSM rather than
+ // generating external SST file
+ if (reason != TableFileCreationReason::kMisc) {
+ filter_context.compaction_style = ioptions.compaction_style;
+ filter_context.num_levels = ioptions.num_levels;
+ filter_context.level_at_creation = tbo.level_at_creation;
+ filter_context.is_bottommost = tbo.is_bottommost;
+ assert(filter_context.level_at_creation < filter_context.num_levels);
+ }
+
+ filter_builder.reset(CreateFilterBlockBuilder(
+ ioptions, moptions, filter_context,
+ use_delta_encoding_for_index_values, p_index_builder_));
+ }
+
+ assert(tbo.int_tbl_prop_collector_factories);
+ for (auto& factory : *tbo.int_tbl_prop_collector_factories) {
+ assert(factory);
+
+ table_properties_collectors.emplace_back(
+ factory->CreateIntTblPropCollector(tbo.column_family_id,
+ tbo.level_at_creation));
+ }
+ table_properties_collectors.emplace_back(
+ new BlockBasedTablePropertiesCollector(
+ table_options.index_type, table_options.whole_key_filtering,
+ moptions.prefix_extractor != nullptr));
+ const Comparator* ucmp = tbo.internal_comparator.user_comparator();
+ assert(ucmp);
+ if (ucmp->timestamp_size() > 0) {
+ table_properties_collectors.emplace_back(
+ new TimestampTablePropertiesCollector(ucmp));
+ }
+ if (table_options.verify_compression) {
+ for (uint32_t i = 0; i < compression_opts.parallel_threads; i++) {
+ verify_ctxs[i].reset(new UncompressionContext(compression_type));
+ }
+ }
+
+ // These are only needed for populating table properties
+ props.column_family_id = tbo.column_family_id;
+ props.column_family_name = tbo.column_family_name;
+ props.oldest_key_time = tbo.oldest_key_time;
+ props.file_creation_time = tbo.file_creation_time;
+ props.orig_file_number = tbo.cur_file_num;
+ props.db_id = tbo.db_id;
+ props.db_session_id = tbo.db_session_id;
+ props.db_host_id = ioptions.db_host_id;
+ if (!ReifyDbHostIdProperty(ioptions.env, &props.db_host_id).ok()) {
+ ROCKS_LOG_INFO(ioptions.logger, "db_host_id property will not be set");
+ }
+ }
+
+ Rep(const Rep&) = delete;
+ Rep& operator=(const Rep&) = delete;
+
+ private:
+ // Synchronize status & io_status accesses across threads from main thread,
+ // compression thread and write thread in parallel compression.
+ std::mutex status_mutex;
+ std::atomic<bool> status_ok;
+ Status status;
+ std::mutex io_status_mutex;
+ std::atomic<bool> io_status_ok;
+ IOStatus io_status;
+};
+
+struct BlockBasedTableBuilder::ParallelCompressionRep {
+ // Keys is a wrapper of vector of strings avoiding
+ // releasing string memories during vector clear()
+ // in order to save memory allocation overhead
+ class Keys {
+ public:
+ Keys() : keys_(kKeysInitSize), size_(0) {}
+ void PushBack(const Slice& key) {
+ if (size_ == keys_.size()) {
+ keys_.emplace_back(key.data(), key.size());
+ } else {
+ keys_[size_].assign(key.data(), key.size());
+ }
+ size_++;
+ }
+ void SwapAssign(std::vector<std::string>& keys) {
+ size_ = keys.size();
+ std::swap(keys_, keys);
+ }
+ void Clear() { size_ = 0; }
+ size_t Size() { return size_; }
+ std::string& Back() { return keys_[size_ - 1]; }
+ std::string& operator[](size_t idx) {
+ assert(idx < size_);
+ return keys_[idx];
+ }
+
+ private:
+ const size_t kKeysInitSize = 32;
+ std::vector<std::string> keys_;
+ size_t size_;
+ };
+ std::unique_ptr<Keys> curr_block_keys;
+
+ class BlockRepSlot;
+
+ // BlockRep instances are fetched from and recycled to
+ // block_rep_pool during parallel compression.
+ struct BlockRep {
+ Slice contents;
+ Slice compressed_contents;
+ std::unique_ptr<std::string> data;
+ std::unique_ptr<std::string> compressed_data;
+ CompressionType compression_type;
+ std::unique_ptr<std::string> first_key_in_next_block;
+ std::unique_ptr<Keys> keys;
+ std::unique_ptr<BlockRepSlot> slot;
+ Status status;
+ };
+ // Use a vector of BlockRep as a buffer for a determined number
+ // of BlockRep structures. All data referenced by pointers in
+ // BlockRep will be freed when this vector is destructed.
+ using BlockRepBuffer = std::vector<BlockRep>;
+ BlockRepBuffer block_rep_buf;
+ // Use a thread-safe queue for concurrent access from block
+ // building thread and writer thread.
+ using BlockRepPool = WorkQueue<BlockRep*>;
+ BlockRepPool block_rep_pool;
+
+ // Use BlockRepSlot to keep block order in write thread.
+ // slot_ will pass references to BlockRep
+ class BlockRepSlot {
+ public:
+ BlockRepSlot() : slot_(1) {}
+ template <typename T>
+ void Fill(T&& rep) {
+ slot_.push(std::forward<T>(rep));
+ };
+ void Take(BlockRep*& rep) { slot_.pop(rep); }
+
+ private:
+ // slot_ will pass references to BlockRep in block_rep_buf,
+ // and those references are always valid before the destruction of
+ // block_rep_buf.
+ WorkQueue<BlockRep*> slot_;
+ };
+
+ // Compression queue will pass references to BlockRep in block_rep_buf,
+ // and those references are always valid before the destruction of
+ // block_rep_buf.
+ using CompressQueue = WorkQueue<BlockRep*>;
+ CompressQueue compress_queue;
+ std::vector<port::Thread> compress_thread_pool;
+
+ // Write queue will pass references to BlockRep::slot in block_rep_buf,
+ // and those references are always valid before the corresponding
+ // BlockRep::slot is destructed, which is before the destruction of
+ // block_rep_buf.
+ using WriteQueue = WorkQueue<BlockRepSlot*>;
+ WriteQueue write_queue;
+ std::unique_ptr<port::Thread> write_thread;
+
+ // Estimate output file size when parallel compression is enabled. This is
+ // necessary because compression & flush are no longer synchronized,
+ // and BlockBasedTableBuilder::FileSize() is no longer accurate.
+ // memory_order_relaxed suffices because accurate statistics is not required.
+ class FileSizeEstimator {
+ public:
+ explicit FileSizeEstimator()
+ : uncomp_bytes_compressed(0),
+ uncomp_bytes_curr_block(0),
+ uncomp_bytes_curr_block_set(false),
+ uncomp_bytes_inflight(0),
+ blocks_inflight(0),
+ curr_compression_ratio(0),
+ estimated_file_size(0) {}
+
+ // Estimate file size when a block is about to be emitted to
+ // compression thread
+ void EmitBlock(uint64_t uncomp_block_size, uint64_t curr_file_size) {
+ uint64_t new_uncomp_bytes_inflight =
+ uncomp_bytes_inflight.fetch_add(uncomp_block_size,
+ std::memory_order_relaxed) +
+ uncomp_block_size;
+
+ uint64_t new_blocks_inflight =
+ blocks_inflight.fetch_add(1, std::memory_order_relaxed) + 1;
+
+ estimated_file_size.store(
+ curr_file_size +
+ static_cast<uint64_t>(
+ static_cast<double>(new_uncomp_bytes_inflight) *
+ curr_compression_ratio.load(std::memory_order_relaxed)) +
+ new_blocks_inflight * kBlockTrailerSize,
+ std::memory_order_relaxed);
+ }
+
+ // Estimate file size when a block is already reaped from
+ // compression thread
+ void ReapBlock(uint64_t compressed_block_size, uint64_t curr_file_size) {
+ assert(uncomp_bytes_curr_block_set);
+
+ uint64_t new_uncomp_bytes_compressed =
+ uncomp_bytes_compressed + uncomp_bytes_curr_block;
+ assert(new_uncomp_bytes_compressed > 0);
+
+ curr_compression_ratio.store(
+ (curr_compression_ratio.load(std::memory_order_relaxed) *
+ uncomp_bytes_compressed +
+ compressed_block_size) /
+ static_cast<double>(new_uncomp_bytes_compressed),
+ std::memory_order_relaxed);
+ uncomp_bytes_compressed = new_uncomp_bytes_compressed;
+
+ uint64_t new_uncomp_bytes_inflight =
+ uncomp_bytes_inflight.fetch_sub(uncomp_bytes_curr_block,
+ std::memory_order_relaxed) -
+ uncomp_bytes_curr_block;
+
+ uint64_t new_blocks_inflight =
+ blocks_inflight.fetch_sub(1, std::memory_order_relaxed) - 1;
+
+ estimated_file_size.store(
+ curr_file_size +
+ static_cast<uint64_t>(
+ static_cast<double>(new_uncomp_bytes_inflight) *
+ curr_compression_ratio.load(std::memory_order_relaxed)) +
+ new_blocks_inflight * kBlockTrailerSize,
+ std::memory_order_relaxed);
+
+ uncomp_bytes_curr_block_set = false;
+ }
+
+ void SetEstimatedFileSize(uint64_t size) {
+ estimated_file_size.store(size, std::memory_order_relaxed);
+ }
+
+ uint64_t GetEstimatedFileSize() {
+ return estimated_file_size.load(std::memory_order_relaxed);
+ }
+
+ void SetCurrBlockUncompSize(uint64_t size) {
+ uncomp_bytes_curr_block = size;
+ uncomp_bytes_curr_block_set = true;
+ }
+
+ private:
+ // Input bytes compressed so far.
+ uint64_t uncomp_bytes_compressed;
+ // Size of current block being appended.
+ uint64_t uncomp_bytes_curr_block;
+ // Whether uncomp_bytes_curr_block has been set for next
+ // ReapBlock call.
+ bool uncomp_bytes_curr_block_set;
+ // Input bytes under compression and not appended yet.
+ std::atomic<uint64_t> uncomp_bytes_inflight;
+ // Number of blocks under compression and not appended yet.
+ std::atomic<uint64_t> blocks_inflight;
+ // Current compression ratio, maintained by BGWorkWriteMaybeCompressedBlock.
+ std::atomic<double> curr_compression_ratio;
+ // Estimated SST file size.
+ std::atomic<uint64_t> estimated_file_size;
+ };
+ FileSizeEstimator file_size_estimator;
+
+ // Facilities used for waiting first block completion. Need to Wait for
+ // the completion of first block compression and flush to get a non-zero
+ // compression ratio.
+ std::atomic<bool> first_block_processed;
+ std::condition_variable first_block_cond;
+ std::mutex first_block_mutex;
+
+ explicit ParallelCompressionRep(uint32_t parallel_threads)
+ : curr_block_keys(new Keys()),
+ block_rep_buf(parallel_threads),
+ block_rep_pool(parallel_threads),
+ compress_queue(parallel_threads),
+ write_queue(parallel_threads),
+ first_block_processed(false) {
+ for (uint32_t i = 0; i < parallel_threads; i++) {
+ block_rep_buf[i].contents = Slice();
+ block_rep_buf[i].compressed_contents = Slice();
+ block_rep_buf[i].data.reset(new std::string());
+ block_rep_buf[i].compressed_data.reset(new std::string());
+ block_rep_buf[i].compression_type = CompressionType();
+ block_rep_buf[i].first_key_in_next_block.reset(new std::string());
+ block_rep_buf[i].keys.reset(new Keys());
+ block_rep_buf[i].slot.reset(new BlockRepSlot());
+ block_rep_buf[i].status = Status::OK();
+ block_rep_pool.push(&block_rep_buf[i]);
+ }
+ }
+
+ ~ParallelCompressionRep() { block_rep_pool.finish(); }
+
+ // Make a block prepared to be emitted to compression thread
+ // Used in non-buffered mode
+ BlockRep* PrepareBlock(CompressionType compression_type,
+ const Slice* first_key_in_next_block,
+ BlockBuilder* data_block) {
+ BlockRep* block_rep =
+ PrepareBlockInternal(compression_type, first_key_in_next_block);
+ assert(block_rep != nullptr);
+ data_block->SwapAndReset(*(block_rep->data));
+ block_rep->contents = *(block_rep->data);
+ std::swap(block_rep->keys, curr_block_keys);
+ curr_block_keys->Clear();
+ return block_rep;
+ }
+
+ // Used in EnterUnbuffered
+ BlockRep* PrepareBlock(CompressionType compression_type,
+ const Slice* first_key_in_next_block,
+ std::string* data_block,
+ std::vector<std::string>* keys) {
+ BlockRep* block_rep =
+ PrepareBlockInternal(compression_type, first_key_in_next_block);
+ assert(block_rep != nullptr);
+ std::swap(*(block_rep->data), *data_block);
+ block_rep->contents = *(block_rep->data);
+ block_rep->keys->SwapAssign(*keys);
+ return block_rep;
+ }
+
+ // Emit a block to compression thread
+ void EmitBlock(BlockRep* block_rep) {
+ assert(block_rep != nullptr);
+ assert(block_rep->status.ok());
+ if (!write_queue.push(block_rep->slot.get())) {
+ return;
+ }
+ if (!compress_queue.push(block_rep)) {
+ return;
+ }
+
+ if (!first_block_processed.load(std::memory_order_relaxed)) {
+ std::unique_lock<std::mutex> lock(first_block_mutex);
+ first_block_cond.wait(lock, [this] {
+ return first_block_processed.load(std::memory_order_relaxed);
+ });
+ }
+ }
+
+ // Reap a block from compression thread
+ void ReapBlock(BlockRep* block_rep) {
+ assert(block_rep != nullptr);
+ block_rep->compressed_data->clear();
+ block_rep_pool.push(block_rep);
+
+ if (!first_block_processed.load(std::memory_order_relaxed)) {
+ std::lock_guard<std::mutex> lock(first_block_mutex);
+ first_block_processed.store(true, std::memory_order_relaxed);
+ first_block_cond.notify_one();
+ }
+ }
+
+ private:
+ BlockRep* PrepareBlockInternal(CompressionType compression_type,
+ const Slice* first_key_in_next_block) {
+ BlockRep* block_rep = nullptr;
+ block_rep_pool.pop(block_rep);
+ assert(block_rep != nullptr);
+
+ assert(block_rep->data);
+
+ block_rep->compression_type = compression_type;
+
+ if (first_key_in_next_block == nullptr) {
+ block_rep->first_key_in_next_block.reset(nullptr);
+ } else {
+ block_rep->first_key_in_next_block->assign(
+ first_key_in_next_block->data(), first_key_in_next_block->size());
+ }
+
+ return block_rep;
+ }
+};
+
+BlockBasedTableBuilder::BlockBasedTableBuilder(
+ const BlockBasedTableOptions& table_options, const TableBuilderOptions& tbo,
+ WritableFileWriter* file) {
+ BlockBasedTableOptions sanitized_table_options(table_options);
+ if (sanitized_table_options.format_version == 0 &&
+ sanitized_table_options.checksum != kCRC32c) {
+ ROCKS_LOG_WARN(
+ tbo.ioptions.logger,
+ "Silently converting format_version to 1 because checksum is "
+ "non-default");
+ // silently convert format_version to 1 to keep consistent with current
+ // behavior
+ sanitized_table_options.format_version = 1;
+ }
+
+ rep_ = new Rep(sanitized_table_options, tbo, file);
+
+ TEST_SYNC_POINT_CALLBACK(
+ "BlockBasedTableBuilder::BlockBasedTableBuilder:PreSetupBaseCacheKey",
+ const_cast<TableProperties*>(&rep_->props));
+
+ BlockBasedTable::SetupBaseCacheKey(&rep_->props, tbo.db_session_id,
+ tbo.cur_file_num, &rep_->base_cache_key);
+
+ if (rep_->IsParallelCompressionEnabled()) {
+ StartParallelCompression();
+ }
+}
+
+BlockBasedTableBuilder::~BlockBasedTableBuilder() {
+ // Catch errors where caller forgot to call Finish()
+ assert(rep_->state == Rep::State::kClosed);
+ delete rep_;
+}
+
+void BlockBasedTableBuilder::Add(const Slice& key, const Slice& value) {
+ Rep* r = rep_;
+ assert(rep_->state != Rep::State::kClosed);
+ if (!ok()) return;
+ ValueType value_type = ExtractValueType(key);
+ if (IsValueType(value_type)) {
+#ifndef NDEBUG
+ if (r->props.num_entries > r->props.num_range_deletions) {
+ assert(r->internal_comparator.Compare(key, Slice(r->last_key)) > 0);
+ }
+#endif // !NDEBUG
+
+ auto should_flush = r->flush_block_policy->Update(key, value);
+ if (should_flush) {
+ assert(!r->data_block.empty());
+ r->first_key_in_next_block = &key;
+ Flush();
+ if (r->state == Rep::State::kBuffered) {
+ bool exceeds_buffer_limit =
+ (r->buffer_limit != 0 && r->data_begin_offset > r->buffer_limit);
+ bool exceeds_global_block_cache_limit = false;
+
+ // Increase cache charging for the last buffered data block
+ // only if the block is not going to be unbuffered immediately
+ // and there exists a cache reservation manager
+ if (!exceeds_buffer_limit &&
+ r->compression_dict_buffer_cache_res_mgr != nullptr) {
+ Status s =
+ r->compression_dict_buffer_cache_res_mgr->UpdateCacheReservation(
+ r->data_begin_offset);
+ exceeds_global_block_cache_limit = s.IsMemoryLimit();
+ }
+
+ if (exceeds_buffer_limit || exceeds_global_block_cache_limit) {
+ EnterUnbuffered();
+ }
+ }
+
+ // 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->state == Rep::State::kUnbuffered) {
+ if (r->IsParallelCompressionEnabled()) {
+ r->pc_rep->curr_block_keys->Clear();
+ } else {
+ r->index_builder->AddIndexEntry(&r->last_key, &key,
+ r->pending_handle);
+ }
+ }
+ }
+
+ // Note: PartitionedFilterBlockBuilder requires key being added to filter
+ // builder after being added to index builder.
+ if (r->state == Rep::State::kUnbuffered) {
+ if (r->IsParallelCompressionEnabled()) {
+ r->pc_rep->curr_block_keys->PushBack(key);
+ } else {
+ if (r->filter_builder != nullptr) {
+ size_t ts_sz =
+ r->internal_comparator.user_comparator()->timestamp_size();
+ r->filter_builder->Add(ExtractUserKeyAndStripTimestamp(key, ts_sz));
+ }
+ }
+ }
+
+ r->data_block.AddWithLastKey(key, value, r->last_key);
+ r->last_key.assign(key.data(), key.size());
+ if (r->state == Rep::State::kBuffered) {
+ // Buffered keys will be replayed from data_block_buffers during
+ // `Finish()` once compression dictionary has been finalized.
+ } else {
+ if (!r->IsParallelCompressionEnabled()) {
+ r->index_builder->OnKeyAdded(key);
+ }
+ }
+ // TODO offset passed in is not accurate for parallel compression case
+ NotifyCollectTableCollectorsOnAdd(key, value, r->get_offset(),
+ r->table_properties_collectors,
+ r->ioptions.logger);
+
+ } else if (value_type == kTypeRangeDeletion) {
+ r->range_del_block.Add(key, value);
+ // TODO offset passed in is not accurate for parallel compression case
+ NotifyCollectTableCollectorsOnAdd(key, value, r->get_offset(),
+ r->table_properties_collectors,
+ r->ioptions.logger);
+ } else {
+ assert(false);
+ }
+
+ r->props.num_entries++;
+ r->props.raw_key_size += key.size();
+ r->props.raw_value_size += value.size();
+ if (value_type == kTypeDeletion || value_type == kTypeSingleDeletion ||
+ value_type == kTypeDeletionWithTimestamp) {
+ r->props.num_deletions++;
+ } else if (value_type == kTypeRangeDeletion) {
+ r->props.num_deletions++;
+ r->props.num_range_deletions++;
+ } else if (value_type == kTypeMerge) {
+ r->props.num_merge_operands++;
+ }
+}
+
+void BlockBasedTableBuilder::Flush() {
+ Rep* r = rep_;
+ assert(rep_->state != Rep::State::kClosed);
+ if (!ok()) return;
+ if (r->data_block.empty()) return;
+ if (r->IsParallelCompressionEnabled() &&
+ r->state == Rep::State::kUnbuffered) {
+ r->data_block.Finish();
+ ParallelCompressionRep::BlockRep* block_rep = r->pc_rep->PrepareBlock(
+ r->compression_type, r->first_key_in_next_block, &(r->data_block));
+ assert(block_rep != nullptr);
+ r->pc_rep->file_size_estimator.EmitBlock(block_rep->data->size(),
+ r->get_offset());
+ r->pc_rep->EmitBlock(block_rep);
+ } else {
+ WriteBlock(&r->data_block, &r->pending_handle, BlockType::kData);
+ }
+}
+
+void BlockBasedTableBuilder::WriteBlock(BlockBuilder* block,
+ BlockHandle* handle,
+ BlockType block_type) {
+ block->Finish();
+ std::string uncompressed_block_data;
+ uncompressed_block_data.reserve(rep_->table_options.block_size);
+ block->SwapAndReset(uncompressed_block_data);
+ if (rep_->state == Rep::State::kBuffered) {
+ assert(block_type == BlockType::kData);
+ rep_->data_block_buffers.emplace_back(std::move(uncompressed_block_data));
+ rep_->data_begin_offset += rep_->data_block_buffers.back().size();
+ return;
+ }
+ WriteBlock(uncompressed_block_data, handle, block_type);
+}
+
+void BlockBasedTableBuilder::WriteBlock(const Slice& uncompressed_block_data,
+ BlockHandle* handle,
+ BlockType block_type) {
+ Rep* r = rep_;
+ assert(r->state == Rep::State::kUnbuffered);
+ Slice block_contents;
+ CompressionType type;
+ Status compress_status;
+ bool is_data_block = block_type == BlockType::kData;
+ CompressAndVerifyBlock(uncompressed_block_data, is_data_block,
+ *(r->compression_ctxs[0]), r->verify_ctxs[0].get(),
+ &(r->compressed_output), &(block_contents), &type,
+ &compress_status);
+ r->SetStatus(compress_status);
+ if (!ok()) {
+ return;
+ }
+
+ WriteMaybeCompressedBlock(block_contents, type, handle, block_type,
+ &uncompressed_block_data);
+ r->compressed_output.clear();
+ if (is_data_block) {
+ r->props.data_size = r->get_offset();
+ ++r->props.num_data_blocks;
+ }
+}
+
+void BlockBasedTableBuilder::BGWorkCompression(
+ const CompressionContext& compression_ctx,
+ UncompressionContext* verify_ctx) {
+ ParallelCompressionRep::BlockRep* block_rep = nullptr;
+ while (rep_->pc_rep->compress_queue.pop(block_rep)) {
+ assert(block_rep != nullptr);
+ CompressAndVerifyBlock(block_rep->contents, true, /* is_data_block*/
+ compression_ctx, verify_ctx,
+ block_rep->compressed_data.get(),
+ &block_rep->compressed_contents,
+ &(block_rep->compression_type), &block_rep->status);
+ block_rep->slot->Fill(block_rep);
+ }
+}
+
+void BlockBasedTableBuilder::CompressAndVerifyBlock(
+ const Slice& uncompressed_block_data, bool is_data_block,
+ const CompressionContext& compression_ctx, UncompressionContext* verify_ctx,
+ std::string* compressed_output, Slice* block_contents,
+ CompressionType* type, Status* out_status) {
+ // File format contains a sequence of blocks where each block has:
+ // block_data: uint8[n]
+ // type: uint8
+ // crc: uint32
+ Rep* r = rep_;
+ bool is_status_ok = ok();
+ if (!r->IsParallelCompressionEnabled()) {
+ assert(is_status_ok);
+ }
+
+ *type = r->compression_type;
+ uint64_t sample_for_compression = r->sample_for_compression;
+ bool abort_compression = false;
+
+ StopWatchNano timer(
+ r->ioptions.clock,
+ ShouldReportDetailedTime(r->ioptions.env, r->ioptions.stats));
+
+ if (is_status_ok && uncompressed_block_data.size() < kCompressionSizeLimit) {
+ if (is_data_block) {
+ r->compressible_input_data_bytes.fetch_add(uncompressed_block_data.size(),
+ std::memory_order_relaxed);
+ }
+ const CompressionDict* compression_dict;
+ if (!is_data_block || r->compression_dict == nullptr) {
+ compression_dict = &CompressionDict::GetEmptyDict();
+ } else {
+ compression_dict = r->compression_dict.get();
+ }
+ assert(compression_dict != nullptr);
+ CompressionInfo compression_info(r->compression_opts, compression_ctx,
+ *compression_dict, *type,
+ sample_for_compression);
+
+ std::string sampled_output_fast;
+ std::string sampled_output_slow;
+ *block_contents = CompressBlock(
+ uncompressed_block_data, compression_info, type,
+ r->table_options.format_version, is_data_block /* do_sample */,
+ compressed_output, &sampled_output_fast, &sampled_output_slow);
+
+ if (sampled_output_slow.size() > 0 || sampled_output_fast.size() > 0) {
+ // Currently compression sampling is only enabled for data block.
+ assert(is_data_block);
+ r->sampled_input_data_bytes.fetch_add(uncompressed_block_data.size(),
+ std::memory_order_relaxed);
+ r->sampled_output_slow_data_bytes.fetch_add(sampled_output_slow.size(),
+ std::memory_order_relaxed);
+ r->sampled_output_fast_data_bytes.fetch_add(sampled_output_fast.size(),
+ std::memory_order_relaxed);
+ }
+ // notify collectors on block add
+ NotifyCollectTableCollectorsOnBlockAdd(
+ r->table_properties_collectors, uncompressed_block_data.size(),
+ sampled_output_fast.size(), sampled_output_slow.size());
+
+ // Some of the compression algorithms are known to be unreliable. If
+ // the verify_compression flag is set then try to de-compress the
+ // compressed data and compare to the input.
+ if (*type != kNoCompression && r->table_options.verify_compression) {
+ // Retrieve the uncompressed contents into a new buffer
+ const UncompressionDict* verify_dict;
+ if (!is_data_block || r->verify_dict == nullptr) {
+ verify_dict = &UncompressionDict::GetEmptyDict();
+ } else {
+ verify_dict = r->verify_dict.get();
+ }
+ assert(verify_dict != nullptr);
+ BlockContents contents;
+ UncompressionInfo uncompression_info(*verify_ctx, *verify_dict,
+ r->compression_type);
+ Status stat = UncompressBlockData(
+ uncompression_info, block_contents->data(), block_contents->size(),
+ &contents, r->table_options.format_version, r->ioptions);
+
+ if (stat.ok()) {
+ bool compressed_ok =
+ contents.data.compare(uncompressed_block_data) == 0;
+ if (!compressed_ok) {
+ // The result of the compression was invalid. abort.
+ abort_compression = true;
+ const char* const msg =
+ "Decompressed block did not match pre-compression block";
+ ROCKS_LOG_ERROR(r->ioptions.logger, "%s", msg);
+ *out_status = Status::Corruption(msg);
+ }
+ } else {
+ // Decompression reported an error. abort.
+ *out_status = Status::Corruption(std::string("Could not decompress: ") +
+ stat.getState());
+ abort_compression = true;
+ }
+ }
+ } else {
+ // Block is too big to be compressed.
+ if (is_data_block) {
+ r->uncompressible_input_data_bytes.fetch_add(
+ uncompressed_block_data.size(), std::memory_order_relaxed);
+ }
+ abort_compression = true;
+ }
+ if (is_data_block) {
+ r->uncompressible_input_data_bytes.fetch_add(kBlockTrailerSize,
+ std::memory_order_relaxed);
+ }
+
+ // Abort compression if the block is too big, or did not pass
+ // verification.
+ if (abort_compression) {
+ RecordTick(r->ioptions.stats, NUMBER_BLOCK_NOT_COMPRESSED);
+ *type = kNoCompression;
+ *block_contents = uncompressed_block_data;
+ } else if (*type != kNoCompression) {
+ if (ShouldReportDetailedTime(r->ioptions.env, r->ioptions.stats)) {
+ RecordTimeToHistogram(r->ioptions.stats, COMPRESSION_TIMES_NANOS,
+ timer.ElapsedNanos());
+ }
+ RecordInHistogram(r->ioptions.stats, BYTES_COMPRESSED,
+ uncompressed_block_data.size());
+ RecordTick(r->ioptions.stats, NUMBER_BLOCK_COMPRESSED);
+ } else if (*type != r->compression_type) {
+ RecordTick(r->ioptions.stats, NUMBER_BLOCK_NOT_COMPRESSED);
+ }
+}
+
+void BlockBasedTableBuilder::WriteMaybeCompressedBlock(
+ const Slice& block_contents, CompressionType type, BlockHandle* handle,
+ BlockType block_type, const Slice* uncompressed_block_data) {
+ Rep* r = rep_;
+ bool is_data_block = block_type == BlockType::kData;
+ // Old, misleading name of this function: WriteRawBlock
+ StopWatch sw(r->ioptions.clock, r->ioptions.stats, WRITE_RAW_BLOCK_MICROS);
+ handle->set_offset(r->get_offset());
+ handle->set_size(block_contents.size());
+ assert(status().ok());
+ assert(io_status().ok());
+
+ {
+ IOStatus io_s = r->file->Append(block_contents);
+ if (!io_s.ok()) {
+ r->SetIOStatus(io_s);
+ return;
+ }
+ }
+
+ std::array<char, kBlockTrailerSize> trailer;
+ trailer[0] = type;
+ uint32_t checksum = ComputeBuiltinChecksumWithLastByte(
+ r->table_options.checksum, block_contents.data(), block_contents.size(),
+ /*last_byte*/ type);
+
+ if (block_type == BlockType::kFilter) {
+ Status s = r->filter_builder->MaybePostVerifyFilter(block_contents);
+ if (!s.ok()) {
+ r->SetStatus(s);
+ return;
+ }
+ }
+
+ EncodeFixed32(trailer.data() + 1, checksum);
+ TEST_SYNC_POINT_CALLBACK(
+ "BlockBasedTableBuilder::WriteMaybeCompressedBlock:TamperWithChecksum",
+ trailer.data());
+ {
+ IOStatus io_s = r->file->Append(Slice(trailer.data(), trailer.size()));
+ if (!io_s.ok()) {
+ r->SetIOStatus(io_s);
+ return;
+ }
+ }
+
+ {
+ Status s = Status::OK();
+ bool warm_cache;
+ switch (r->table_options.prepopulate_block_cache) {
+ case BlockBasedTableOptions::PrepopulateBlockCache::kFlushOnly:
+ warm_cache = (r->reason == TableFileCreationReason::kFlush);
+ break;
+ case BlockBasedTableOptions::PrepopulateBlockCache::kDisable:
+ warm_cache = false;
+ break;
+ default:
+ // missing case
+ assert(false);
+ warm_cache = false;
+ }
+ if (warm_cache) {
+ if (type == kNoCompression) {
+ s = InsertBlockInCacheHelper(block_contents, handle, block_type);
+ } else if (uncompressed_block_data != nullptr) {
+ s = InsertBlockInCacheHelper(*uncompressed_block_data, handle,
+ block_type);
+ }
+ if (!s.ok()) {
+ r->SetStatus(s);
+ return;
+ }
+ }
+ s = InsertBlockInCompressedCache(block_contents, type, handle);
+ if (!s.ok()) {
+ r->SetStatus(s);
+ return;
+ }
+ }
+
+ r->set_offset(r->get_offset() + block_contents.size() + kBlockTrailerSize);
+ if (r->table_options.block_align && is_data_block) {
+ size_t pad_bytes =
+ (r->alignment -
+ ((block_contents.size() + kBlockTrailerSize) & (r->alignment - 1))) &
+ (r->alignment - 1);
+ IOStatus io_s = r->file->Pad(pad_bytes);
+ if (io_s.ok()) {
+ r->set_offset(r->get_offset() + pad_bytes);
+ } else {
+ r->SetIOStatus(io_s);
+ return;
+ }
+ }
+
+ if (r->IsParallelCompressionEnabled()) {
+ if (is_data_block) {
+ r->pc_rep->file_size_estimator.ReapBlock(block_contents.size(),
+ r->get_offset());
+ } else {
+ r->pc_rep->file_size_estimator.SetEstimatedFileSize(r->get_offset());
+ }
+ }
+}
+
+void BlockBasedTableBuilder::BGWorkWriteMaybeCompressedBlock() {
+ Rep* r = rep_;
+ ParallelCompressionRep::BlockRepSlot* slot = nullptr;
+ ParallelCompressionRep::BlockRep* block_rep = nullptr;
+ while (r->pc_rep->write_queue.pop(slot)) {
+ assert(slot != nullptr);
+ slot->Take(block_rep);
+ assert(block_rep != nullptr);
+ if (!block_rep->status.ok()) {
+ r->SetStatus(block_rep->status);
+ // Reap block so that blocked Flush() can finish
+ // if there is one, and Flush() will notice !ok() next time.
+ block_rep->status = Status::OK();
+ r->pc_rep->ReapBlock(block_rep);
+ continue;
+ }
+
+ for (size_t i = 0; i < block_rep->keys->Size(); i++) {
+ auto& key = (*block_rep->keys)[i];
+ if (r->filter_builder != nullptr) {
+ size_t ts_sz =
+ r->internal_comparator.user_comparator()->timestamp_size();
+ r->filter_builder->Add(ExtractUserKeyAndStripTimestamp(key, ts_sz));
+ }
+ r->index_builder->OnKeyAdded(key);
+ }
+
+ r->pc_rep->file_size_estimator.SetCurrBlockUncompSize(
+ block_rep->data->size());
+ WriteMaybeCompressedBlock(block_rep->compressed_contents,
+ block_rep->compression_type, &r->pending_handle,
+ BlockType::kData, &block_rep->contents);
+ if (!ok()) {
+ break;
+ }
+
+ r->props.data_size = r->get_offset();
+ ++r->props.num_data_blocks;
+
+ if (block_rep->first_key_in_next_block == nullptr) {
+ r->index_builder->AddIndexEntry(&(block_rep->keys->Back()), nullptr,
+ r->pending_handle);
+ } else {
+ Slice first_key_in_next_block =
+ Slice(*block_rep->first_key_in_next_block);
+ r->index_builder->AddIndexEntry(&(block_rep->keys->Back()),
+ &first_key_in_next_block,
+ r->pending_handle);
+ }
+
+ r->pc_rep->ReapBlock(block_rep);
+ }
+}
+
+void BlockBasedTableBuilder::StartParallelCompression() {
+ rep_->pc_rep.reset(
+ new ParallelCompressionRep(rep_->compression_opts.parallel_threads));
+ rep_->pc_rep->compress_thread_pool.reserve(
+ rep_->compression_opts.parallel_threads);
+ for (uint32_t i = 0; i < rep_->compression_opts.parallel_threads; i++) {
+ rep_->pc_rep->compress_thread_pool.emplace_back([this, i] {
+ BGWorkCompression(*(rep_->compression_ctxs[i]),
+ rep_->verify_ctxs[i].get());
+ });
+ }
+ rep_->pc_rep->write_thread.reset(
+ new port::Thread([this] { BGWorkWriteMaybeCompressedBlock(); }));
+}
+
+void BlockBasedTableBuilder::StopParallelCompression() {
+ rep_->pc_rep->compress_queue.finish();
+ for (auto& thread : rep_->pc_rep->compress_thread_pool) {
+ thread.join();
+ }
+ rep_->pc_rep->write_queue.finish();
+ rep_->pc_rep->write_thread->join();
+}
+
+Status BlockBasedTableBuilder::status() const { return rep_->GetStatus(); }
+
+IOStatus BlockBasedTableBuilder::io_status() const {
+ return rep_->GetIOStatus();
+}
+
+//
+// Make a copy of the block contents and insert into compressed block cache
+//
+Status BlockBasedTableBuilder::InsertBlockInCompressedCache(
+ const Slice& block_contents, const CompressionType type,
+ const BlockHandle* handle) {
+ Rep* r = rep_;
+ Cache* block_cache_compressed = r->table_options.block_cache_compressed.get();
+ Status s;
+ if (type != kNoCompression && block_cache_compressed != nullptr) {
+ size_t size = block_contents.size();
+
+ auto ubuf =
+ AllocateBlock(size + 1, block_cache_compressed->memory_allocator());
+ memcpy(ubuf.get(), block_contents.data(), size);
+ ubuf[size] = type;
+
+ BlockContents* block_contents_to_cache =
+ new BlockContents(std::move(ubuf), size);
+#ifndef NDEBUG
+ block_contents_to_cache->has_trailer = true;
+#endif // NDEBUG
+
+ CacheKey key = BlockBasedTable::GetCacheKey(rep_->base_cache_key, *handle);
+
+ s = block_cache_compressed->Insert(
+ key.AsSlice(), block_contents_to_cache,
+ block_contents_to_cache->ApproximateMemoryUsage(),
+ &DeleteCacheEntry<BlockContents>);
+ if (s.ok()) {
+ RecordTick(rep_->ioptions.stats, BLOCK_CACHE_COMPRESSED_ADD);
+ } else {
+ RecordTick(rep_->ioptions.stats, BLOCK_CACHE_COMPRESSED_ADD_FAILURES);
+ }
+ // Invalidate OS cache.
+ r->file->InvalidateCache(static_cast<size_t>(r->get_offset()), size)
+ .PermitUncheckedError();
+ }
+ return s;
+}
+
+Status BlockBasedTableBuilder::InsertBlockInCacheHelper(
+ const Slice& block_contents, const BlockHandle* handle,
+ BlockType block_type) {
+ Status s;
+ switch (block_type) {
+ case BlockType::kData:
+ case BlockType::kIndex:
+ case BlockType::kFilterPartitionIndex:
+ s = InsertBlockInCache<Block>(block_contents, handle, block_type);
+ break;
+ case BlockType::kFilter:
+ s = InsertBlockInCache<ParsedFullFilterBlock>(block_contents, handle,
+ block_type);
+ break;
+ case BlockType::kCompressionDictionary:
+ s = InsertBlockInCache<UncompressionDict>(block_contents, handle,
+ block_type);
+ break;
+ default:
+ // no-op / not cached
+ break;
+ }
+ return s;
+}
+
+template <typename TBlocklike>
+Status BlockBasedTableBuilder::InsertBlockInCache(const Slice& block_contents,
+ const BlockHandle* handle,
+ BlockType block_type) {
+ // Uncompressed regular block cache
+ Cache* block_cache = rep_->table_options.block_cache.get();
+ Status s;
+ if (block_cache != nullptr) {
+ size_t size = block_contents.size();
+ auto buf = AllocateBlock(size, block_cache->memory_allocator());
+ memcpy(buf.get(), block_contents.data(), size);
+ BlockContents results(std::move(buf), size);
+
+ CacheKey key = BlockBasedTable::GetCacheKey(rep_->base_cache_key, *handle);
+
+ const size_t read_amp_bytes_per_bit =
+ rep_->table_options.read_amp_bytes_per_bit;
+
+ // TODO akanksha:: Dedup below code by calling
+ // BlockBasedTable::PutDataBlockToCache.
+ std::unique_ptr<TBlocklike> block_holder(
+ BlocklikeTraits<TBlocklike>::Create(
+ std::move(results), read_amp_bytes_per_bit,
+ rep_->ioptions.statistics.get(),
+ false /*rep_->blocks_definitely_zstd_compressed*/,
+ rep_->table_options.filter_policy.get()));
+
+ assert(block_holder->own_bytes());
+ size_t charge = block_holder->ApproximateMemoryUsage();
+ s = block_cache->Insert(
+ key.AsSlice(), block_holder.get(),
+ BlocklikeTraits<TBlocklike>::GetCacheItemHelper(block_type), charge,
+ nullptr, Cache::Priority::LOW);
+
+ if (s.ok()) {
+ // Release ownership of block_holder.
+ block_holder.release();
+ BlockBasedTable::UpdateCacheInsertionMetrics(
+ block_type, nullptr /*get_context*/, charge, s.IsOkOverwritten(),
+ rep_->ioptions.stats);
+ } else {
+ RecordTick(rep_->ioptions.stats, BLOCK_CACHE_ADD_FAILURES);
+ }
+ }
+ return s;
+}
+
+void BlockBasedTableBuilder::WriteFilterBlock(
+ MetaIndexBuilder* meta_index_builder) {
+ if (rep_->filter_builder == nullptr || rep_->filter_builder->IsEmpty()) {
+ // No filter block needed
+ return;
+ }
+ BlockHandle filter_block_handle;
+ bool is_partitioned_filter = rep_->table_options.partition_filters;
+ if (ok()) {
+ rep_->props.num_filter_entries +=
+ rep_->filter_builder->EstimateEntriesAdded();
+ Status s = Status::Incomplete();
+ while (ok() && s.IsIncomplete()) {
+ // filter_data is used to store the transferred filter data payload from
+ // FilterBlockBuilder and deallocate the payload by going out of scope.
+ // Otherwise, the payload will unnecessarily remain until
+ // BlockBasedTableBuilder is deallocated.
+ //
+ // See FilterBlockBuilder::Finish() for more on the difference in
+ // transferred filter data payload among different FilterBlockBuilder
+ // subtypes.
+ std::unique_ptr<const char[]> filter_data;
+ Slice filter_content =
+ rep_->filter_builder->Finish(filter_block_handle, &s, &filter_data);
+
+ assert(s.ok() || s.IsIncomplete() || s.IsCorruption());
+ if (s.IsCorruption()) {
+ rep_->SetStatus(s);
+ break;
+ }
+
+ rep_->props.filter_size += filter_content.size();
+
+ BlockType btype = is_partitioned_filter && /* last */ s.ok()
+ ? BlockType::kFilterPartitionIndex
+ : BlockType::kFilter;
+ WriteMaybeCompressedBlock(filter_content, kNoCompression,
+ &filter_block_handle, btype);
+ }
+ rep_->filter_builder->ResetFilterBitsBuilder();
+ }
+ if (ok()) {
+ // Add mapping from "<filter_block_prefix>.Name" to location
+ // of filter data.
+ std::string key;
+ key = is_partitioned_filter ? BlockBasedTable::kPartitionedFilterBlockPrefix
+ : BlockBasedTable::kFullFilterBlockPrefix;
+ key.append(rep_->table_options.filter_policy->CompatibilityName());
+ meta_index_builder->Add(key, filter_block_handle);
+ }
+}
+
+void BlockBasedTableBuilder::WriteIndexBlock(
+ MetaIndexBuilder* meta_index_builder, BlockHandle* index_block_handle) {
+ if (!ok()) {
+ return;
+ }
+ IndexBuilder::IndexBlocks index_blocks;
+ auto index_builder_status = rep_->index_builder->Finish(&index_blocks);
+ if (index_builder_status.IsIncomplete()) {
+ // We we have more than one index partition then meta_blocks are not
+ // supported for the index. Currently meta_blocks are used only by
+ // HashIndexBuilder which is not multi-partition.
+ assert(index_blocks.meta_blocks.empty());
+ } else if (ok() && !index_builder_status.ok()) {
+ rep_->SetStatus(index_builder_status);
+ }
+ if (ok()) {
+ for (const auto& item : index_blocks.meta_blocks) {
+ BlockHandle block_handle;
+ WriteBlock(item.second, &block_handle, BlockType::kIndex);
+ if (!ok()) {
+ break;
+ }
+ meta_index_builder->Add(item.first, block_handle);
+ }
+ }
+ if (ok()) {
+ if (rep_->table_options.enable_index_compression) {
+ WriteBlock(index_blocks.index_block_contents, index_block_handle,
+ BlockType::kIndex);
+ } else {
+ WriteMaybeCompressedBlock(index_blocks.index_block_contents,
+ kNoCompression, index_block_handle,
+ BlockType::kIndex);
+ }
+ }
+ // If there are more index partitions, finish them and write them out
+ if (index_builder_status.IsIncomplete()) {
+ bool index_building_finished = false;
+ while (ok() && !index_building_finished) {
+ Status s =
+ rep_->index_builder->Finish(&index_blocks, *index_block_handle);
+ if (s.ok()) {
+ index_building_finished = true;
+ } else if (s.IsIncomplete()) {
+ // More partitioned index after this one
+ assert(!index_building_finished);
+ } else {
+ // Error
+ rep_->SetStatus(s);
+ return;
+ }
+
+ if (rep_->table_options.enable_index_compression) {
+ WriteBlock(index_blocks.index_block_contents, index_block_handle,
+ BlockType::kIndex);
+ } else {
+ WriteMaybeCompressedBlock(index_blocks.index_block_contents,
+ kNoCompression, index_block_handle,
+ BlockType::kIndex);
+ }
+ // The last index_block_handle will be for the partition index block
+ }
+ }
+}
+
+void BlockBasedTableBuilder::WritePropertiesBlock(
+ MetaIndexBuilder* meta_index_builder) {
+ BlockHandle properties_block_handle;
+ if (ok()) {
+ PropertyBlockBuilder property_block_builder;
+ rep_->props.filter_policy_name =
+ rep_->table_options.filter_policy != nullptr
+ ? rep_->table_options.filter_policy->Name()
+ : "";
+ rep_->props.index_size =
+ rep_->index_builder->IndexSize() + kBlockTrailerSize;
+ rep_->props.comparator_name = rep_->ioptions.user_comparator != nullptr
+ ? rep_->ioptions.user_comparator->Name()
+ : "nullptr";
+ rep_->props.merge_operator_name =
+ rep_->ioptions.merge_operator != nullptr
+ ? rep_->ioptions.merge_operator->Name()
+ : "nullptr";
+ rep_->props.compression_name =
+ CompressionTypeToString(rep_->compression_type);
+ rep_->props.compression_options =
+ CompressionOptionsToString(rep_->compression_opts);
+ rep_->props.prefix_extractor_name =
+ rep_->moptions.prefix_extractor != nullptr
+ ? rep_->moptions.prefix_extractor->AsString()
+ : "nullptr";
+ std::string property_collectors_names = "[";
+ for (size_t i = 0;
+ i < rep_->ioptions.table_properties_collector_factories.size(); ++i) {
+ if (i != 0) {
+ property_collectors_names += ",";
+ }
+ property_collectors_names +=
+ rep_->ioptions.table_properties_collector_factories[i]->Name();
+ }
+ property_collectors_names += "]";
+ rep_->props.property_collectors_names = property_collectors_names;
+ if (rep_->table_options.index_type ==
+ BlockBasedTableOptions::kTwoLevelIndexSearch) {
+ assert(rep_->p_index_builder_ != nullptr);
+ rep_->props.index_partitions = rep_->p_index_builder_->NumPartitions();
+ rep_->props.top_level_index_size =
+ rep_->p_index_builder_->TopLevelIndexSize(rep_->offset);
+ }
+ rep_->props.index_key_is_user_key =
+ !rep_->index_builder->seperator_is_key_plus_seq();
+ rep_->props.index_value_is_delta_encoded =
+ rep_->use_delta_encoding_for_index_values;
+ if (rep_->sampled_input_data_bytes > 0) {
+ rep_->props.slow_compression_estimated_data_size = static_cast<uint64_t>(
+ static_cast<double>(rep_->sampled_output_slow_data_bytes) /
+ rep_->sampled_input_data_bytes *
+ rep_->compressible_input_data_bytes +
+ rep_->uncompressible_input_data_bytes + 0.5);
+ rep_->props.fast_compression_estimated_data_size = static_cast<uint64_t>(
+ static_cast<double>(rep_->sampled_output_fast_data_bytes) /
+ rep_->sampled_input_data_bytes *
+ rep_->compressible_input_data_bytes +
+ rep_->uncompressible_input_data_bytes + 0.5);
+ } else if (rep_->sample_for_compression > 0) {
+ // We tried to sample but none were found. Assume worst-case (compression
+ // ratio 1.0) so data is complete and aggregatable.
+ rep_->props.slow_compression_estimated_data_size =
+ rep_->compressible_input_data_bytes +
+ rep_->uncompressible_input_data_bytes;
+ rep_->props.fast_compression_estimated_data_size =
+ rep_->compressible_input_data_bytes +
+ rep_->uncompressible_input_data_bytes;
+ }
+
+ // Add basic properties
+ property_block_builder.AddTableProperty(rep_->props);
+
+ // Add use collected properties
+ NotifyCollectTableCollectorsOnFinish(rep_->table_properties_collectors,
+ rep_->ioptions.logger,
+ &property_block_builder);
+
+ Slice block_data = property_block_builder.Finish();
+ TEST_SYNC_POINT_CALLBACK(
+ "BlockBasedTableBuilder::WritePropertiesBlock:BlockData", &block_data);
+ WriteMaybeCompressedBlock(block_data, kNoCompression,
+ &properties_block_handle, BlockType::kProperties);
+ }
+ if (ok()) {
+#ifndef NDEBUG
+ {
+ uint64_t props_block_offset = properties_block_handle.offset();
+ uint64_t props_block_size = properties_block_handle.size();
+ TEST_SYNC_POINT_CALLBACK(
+ "BlockBasedTableBuilder::WritePropertiesBlock:GetPropsBlockOffset",
+ &props_block_offset);
+ TEST_SYNC_POINT_CALLBACK(
+ "BlockBasedTableBuilder::WritePropertiesBlock:GetPropsBlockSize",
+ &props_block_size);
+ }
+#endif // !NDEBUG
+
+ const std::string* properties_block_meta = &kPropertiesBlockName;
+ TEST_SYNC_POINT_CALLBACK(
+ "BlockBasedTableBuilder::WritePropertiesBlock:Meta",
+ &properties_block_meta);
+ meta_index_builder->Add(*properties_block_meta, properties_block_handle);
+ }
+}
+
+void BlockBasedTableBuilder::WriteCompressionDictBlock(
+ MetaIndexBuilder* meta_index_builder) {
+ if (rep_->compression_dict != nullptr &&
+ rep_->compression_dict->GetRawDict().size()) {
+ BlockHandle compression_dict_block_handle;
+ if (ok()) {
+ WriteMaybeCompressedBlock(rep_->compression_dict->GetRawDict(),
+ kNoCompression, &compression_dict_block_handle,
+ BlockType::kCompressionDictionary);
+#ifndef NDEBUG
+ Slice compression_dict = rep_->compression_dict->GetRawDict();
+ TEST_SYNC_POINT_CALLBACK(
+ "BlockBasedTableBuilder::WriteCompressionDictBlock:RawDict",
+ &compression_dict);
+#endif // NDEBUG
+ }
+ if (ok()) {
+ meta_index_builder->Add(kCompressionDictBlockName,
+ compression_dict_block_handle);
+ }
+ }
+}
+
+void BlockBasedTableBuilder::WriteRangeDelBlock(
+ MetaIndexBuilder* meta_index_builder) {
+ if (ok() && !rep_->range_del_block.empty()) {
+ BlockHandle range_del_block_handle;
+ WriteMaybeCompressedBlock(rep_->range_del_block.Finish(), kNoCompression,
+ &range_del_block_handle,
+ BlockType::kRangeDeletion);
+ meta_index_builder->Add(kRangeDelBlockName, range_del_block_handle);
+ }
+}
+
+void BlockBasedTableBuilder::WriteFooter(BlockHandle& metaindex_block_handle,
+ BlockHandle& index_block_handle) {
+ Rep* r = rep_;
+ // this is guaranteed by BlockBasedTableBuilder's constructor
+ assert(r->table_options.checksum == kCRC32c ||
+ r->table_options.format_version != 0);
+ assert(ok());
+
+ FooterBuilder footer;
+ footer.Build(kBlockBasedTableMagicNumber, r->table_options.format_version,
+ r->get_offset(), r->table_options.checksum,
+ metaindex_block_handle, index_block_handle);
+ IOStatus ios = r->file->Append(footer.GetSlice());
+ if (ios.ok()) {
+ r->set_offset(r->get_offset() + footer.GetSlice().size());
+ } else {
+ r->SetIOStatus(ios);
+ }
+}
+
+void BlockBasedTableBuilder::EnterUnbuffered() {
+ Rep* r = rep_;
+ assert(r->state == Rep::State::kBuffered);
+ r->state = Rep::State::kUnbuffered;
+ const size_t kSampleBytes = r->compression_opts.zstd_max_train_bytes > 0
+ ? r->compression_opts.zstd_max_train_bytes
+ : r->compression_opts.max_dict_bytes;
+ const size_t kNumBlocksBuffered = r->data_block_buffers.size();
+ if (kNumBlocksBuffered == 0) {
+ // The below code is neither safe nor necessary for handling zero data
+ // blocks.
+ return;
+ }
+
+ // Abstract algebra teaches us that a finite cyclic group (such as the
+ // additive group of integers modulo N) can be generated by a number that is
+ // coprime with N. Since N is variable (number of buffered data blocks), we
+ // must then pick a prime number in order to guarantee coprimeness with any N.
+ //
+ // One downside of this approach is the spread will be poor when
+ // `kPrimeGeneratorRemainder` is close to zero or close to
+ // `kNumBlocksBuffered`.
+ //
+ // Picked a random number between one and one trillion and then chose the
+ // next prime number greater than or equal to it.
+ const uint64_t kPrimeGenerator = 545055921143ull;
+ // Can avoid repeated division by just adding the remainder repeatedly.
+ const size_t kPrimeGeneratorRemainder = static_cast<size_t>(
+ kPrimeGenerator % static_cast<uint64_t>(kNumBlocksBuffered));
+ const size_t kInitSampleIdx = kNumBlocksBuffered / 2;
+
+ std::string compression_dict_samples;
+ std::vector<size_t> compression_dict_sample_lens;
+ size_t buffer_idx = kInitSampleIdx;
+ for (size_t i = 0;
+ i < kNumBlocksBuffered && compression_dict_samples.size() < kSampleBytes;
+ ++i) {
+ size_t copy_len = std::min(kSampleBytes - compression_dict_samples.size(),
+ r->data_block_buffers[buffer_idx].size());
+ compression_dict_samples.append(r->data_block_buffers[buffer_idx], 0,
+ copy_len);
+ compression_dict_sample_lens.emplace_back(copy_len);
+
+ buffer_idx += kPrimeGeneratorRemainder;
+ if (buffer_idx >= kNumBlocksBuffered) {
+ buffer_idx -= kNumBlocksBuffered;
+ }
+ }
+
+ // final data block flushed, now we can generate dictionary from the samples.
+ // OK if compression_dict_samples is empty, we'll just get empty dictionary.
+ std::string dict;
+ if (r->compression_opts.zstd_max_train_bytes > 0) {
+ if (r->compression_opts.use_zstd_dict_trainer) {
+ dict = ZSTD_TrainDictionary(compression_dict_samples,
+ compression_dict_sample_lens,
+ r->compression_opts.max_dict_bytes);
+ } else {
+ dict = ZSTD_FinalizeDictionary(
+ compression_dict_samples, compression_dict_sample_lens,
+ r->compression_opts.max_dict_bytes, r->compression_opts.level);
+ }
+ } else {
+ dict = std::move(compression_dict_samples);
+ }
+ r->compression_dict.reset(new CompressionDict(dict, r->compression_type,
+ r->compression_opts.level));
+ r->verify_dict.reset(new UncompressionDict(
+ dict, r->compression_type == kZSTD ||
+ r->compression_type == kZSTDNotFinalCompression));
+
+ auto get_iterator_for_block = [&r](size_t i) {
+ auto& data_block = r->data_block_buffers[i];
+ assert(!data_block.empty());
+
+ Block reader{BlockContents{data_block}};
+ DataBlockIter* iter = reader.NewDataIterator(
+ r->internal_comparator.user_comparator(), kDisableGlobalSequenceNumber);
+
+ iter->SeekToFirst();
+ assert(iter->Valid());
+ return std::unique_ptr<DataBlockIter>(iter);
+ };
+
+ std::unique_ptr<DataBlockIter> iter = nullptr, next_block_iter = nullptr;
+
+ for (size_t i = 0; ok() && i < r->data_block_buffers.size(); ++i) {
+ if (iter == nullptr) {
+ iter = get_iterator_for_block(i);
+ assert(iter != nullptr);
+ };
+
+ if (i + 1 < r->data_block_buffers.size()) {
+ next_block_iter = get_iterator_for_block(i + 1);
+ }
+
+ auto& data_block = r->data_block_buffers[i];
+ if (r->IsParallelCompressionEnabled()) {
+ Slice first_key_in_next_block;
+ const Slice* first_key_in_next_block_ptr = &first_key_in_next_block;
+ if (i + 1 < r->data_block_buffers.size()) {
+ assert(next_block_iter != nullptr);
+ first_key_in_next_block = next_block_iter->key();
+ } else {
+ first_key_in_next_block_ptr = r->first_key_in_next_block;
+ }
+
+ std::vector<std::string> keys;
+ for (; iter->Valid(); iter->Next()) {
+ keys.emplace_back(iter->key().ToString());
+ }
+
+ ParallelCompressionRep::BlockRep* block_rep = r->pc_rep->PrepareBlock(
+ r->compression_type, first_key_in_next_block_ptr, &data_block, &keys);
+
+ assert(block_rep != nullptr);
+ r->pc_rep->file_size_estimator.EmitBlock(block_rep->data->size(),
+ r->get_offset());
+ r->pc_rep->EmitBlock(block_rep);
+ } else {
+ for (; iter->Valid(); iter->Next()) {
+ Slice key = iter->key();
+ if (r->filter_builder != nullptr) {
+ size_t ts_sz =
+ r->internal_comparator.user_comparator()->timestamp_size();
+ r->filter_builder->Add(ExtractUserKeyAndStripTimestamp(key, ts_sz));
+ }
+ r->index_builder->OnKeyAdded(key);
+ }
+ WriteBlock(Slice(data_block), &r->pending_handle, BlockType::kData);
+ if (ok() && i + 1 < r->data_block_buffers.size()) {
+ assert(next_block_iter != nullptr);
+ Slice first_key_in_next_block = next_block_iter->key();
+
+ Slice* first_key_in_next_block_ptr = &first_key_in_next_block;
+
+ iter->SeekToLast();
+ std::string last_key = iter->key().ToString();
+ r->index_builder->AddIndexEntry(&last_key, first_key_in_next_block_ptr,
+ r->pending_handle);
+ }
+ }
+ std::swap(iter, next_block_iter);
+ }
+ r->data_block_buffers.clear();
+ r->data_begin_offset = 0;
+ // Release all reserved cache for data block buffers
+ if (r->compression_dict_buffer_cache_res_mgr != nullptr) {
+ Status s = r->compression_dict_buffer_cache_res_mgr->UpdateCacheReservation(
+ r->data_begin_offset);
+ s.PermitUncheckedError();
+ }
+}
+
+Status BlockBasedTableBuilder::Finish() {
+ Rep* r = rep_;
+ assert(r->state != Rep::State::kClosed);
+ bool empty_data_block = r->data_block.empty();
+ r->first_key_in_next_block = nullptr;
+ Flush();
+ if (r->state == Rep::State::kBuffered) {
+ EnterUnbuffered();
+ }
+ if (r->IsParallelCompressionEnabled()) {
+ StopParallelCompression();
+#ifndef NDEBUG
+ for (const auto& br : r->pc_rep->block_rep_buf) {
+ assert(br.status.ok());
+ }
+#endif // !NDEBUG
+ } else {
+ // To make sure properties block is able to keep the accurate size of index
+ // block, we will finish writing all index entries first.
+ if (ok() && !empty_data_block) {
+ r->index_builder->AddIndexEntry(
+ &r->last_key, nullptr /* no next data block */, r->pending_handle);
+ }
+ }
+
+ // Write meta blocks, metaindex block and footer in the following order.
+ // 1. [meta block: filter]
+ // 2. [meta block: index]
+ // 3. [meta block: compression dictionary]
+ // 4. [meta block: range deletion tombstone]
+ // 5. [meta block: properties]
+ // 6. [metaindex block]
+ // 7. Footer
+ BlockHandle metaindex_block_handle, index_block_handle;
+ MetaIndexBuilder meta_index_builder;
+ WriteFilterBlock(&meta_index_builder);
+ WriteIndexBlock(&meta_index_builder, &index_block_handle);
+ WriteCompressionDictBlock(&meta_index_builder);
+ WriteRangeDelBlock(&meta_index_builder);
+ WritePropertiesBlock(&meta_index_builder);
+ if (ok()) {
+ // flush the meta index block
+ WriteMaybeCompressedBlock(meta_index_builder.Finish(), kNoCompression,
+ &metaindex_block_handle, BlockType::kMetaIndex);
+ }
+ if (ok()) {
+ WriteFooter(metaindex_block_handle, index_block_handle);
+ }
+ r->state = Rep::State::kClosed;
+ r->SetStatus(r->CopyIOStatus());
+ Status ret_status = r->CopyStatus();
+ assert(!ret_status.ok() || io_status().ok());
+ return ret_status;
+}
+
+void BlockBasedTableBuilder::Abandon() {
+ assert(rep_->state != Rep::State::kClosed);
+ if (rep_->IsParallelCompressionEnabled()) {
+ StopParallelCompression();
+ }
+ rep_->state = Rep::State::kClosed;
+ rep_->CopyStatus().PermitUncheckedError();
+ rep_->CopyIOStatus().PermitUncheckedError();
+}
+
+uint64_t BlockBasedTableBuilder::NumEntries() const {
+ return rep_->props.num_entries;
+}
+
+bool BlockBasedTableBuilder::IsEmpty() const {
+ return rep_->props.num_entries == 0 && rep_->props.num_range_deletions == 0;
+}
+
+uint64_t BlockBasedTableBuilder::FileSize() const { return rep_->offset; }
+
+uint64_t BlockBasedTableBuilder::EstimatedFileSize() const {
+ if (rep_->IsParallelCompressionEnabled()) {
+ // Use compression ratio so far and inflight uncompressed bytes to estimate
+ // final SST size.
+ return rep_->pc_rep->file_size_estimator.GetEstimatedFileSize();
+ } else {
+ return FileSize();
+ }
+}
+
+bool BlockBasedTableBuilder::NeedCompact() const {
+ for (const auto& collector : rep_->table_properties_collectors) {
+ if (collector->NeedCompact()) {
+ return true;
+ }
+ }
+ return false;
+}
+
+TableProperties BlockBasedTableBuilder::GetTableProperties() const {
+ TableProperties ret = rep_->props;
+ for (const auto& collector : rep_->table_properties_collectors) {
+ for (const auto& prop : collector->GetReadableProperties()) {
+ ret.readable_properties.insert(prop);
+ }
+ collector->Finish(&ret.user_collected_properties).PermitUncheckedError();
+ }
+ return ret;
+}
+
+std::string BlockBasedTableBuilder::GetFileChecksum() const {
+ if (rep_->file != nullptr) {
+ return rep_->file->GetFileChecksum();
+ } else {
+ return kUnknownFileChecksum;
+ }
+}
+
+const char* BlockBasedTableBuilder::GetFileChecksumFuncName() const {
+ if (rep_->file != nullptr) {
+ return rep_->file->GetFileChecksumFuncName();
+ } else {
+ return kUnknownFileChecksumFuncName;
+ }
+}
+void BlockBasedTableBuilder::SetSeqnoTimeTableProperties(
+ const std::string& encoded_seqno_to_time_mapping,
+ uint64_t oldest_ancestor_time) {
+ rep_->props.seqno_to_time_mapping = encoded_seqno_to_time_mapping;
+ rep_->props.creation_time = oldest_ancestor_time;
+}
+
+const std::string BlockBasedTable::kObsoleteFilterBlockPrefix = "filter.";
+const std::string BlockBasedTable::kFullFilterBlockPrefix = "fullfilter.";
+const std::string BlockBasedTable::kPartitionedFilterBlockPrefix =
+ "partitionedfilter.";
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/block_based_table_builder.h b/src/rocksdb/table/block_based/block_based_table_builder.h
new file mode 100644
index 000000000..ecc13d0f7
--- /dev/null
+++ b/src/rocksdb/table/block_based/block_based_table_builder.h
@@ -0,0 +1,203 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#pragma once
+#include <stdint.h>
+
+#include <array>
+#include <limits>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include "db/version_edit.h"
+#include "rocksdb/flush_block_policy.h"
+#include "rocksdb/listener.h"
+#include "rocksdb/options.h"
+#include "rocksdb/status.h"
+#include "rocksdb/table.h"
+#include "table/meta_blocks.h"
+#include "table/table_builder.h"
+#include "util/compression.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class BlockBuilder;
+class BlockHandle;
+class WritableFile;
+struct BlockBasedTableOptions;
+
+extern const uint64_t kBlockBasedTableMagicNumber;
+extern const uint64_t kLegacyBlockBasedTableMagicNumber;
+
+class BlockBasedTableBuilder : public TableBuilder {
+ public:
+ // Create a builder that will store the contents of the table it is
+ // building in *file. Does not close the file. It is up to the
+ // caller to close the file after calling Finish().
+ BlockBasedTableBuilder(const BlockBasedTableOptions& table_options,
+ const TableBuilderOptions& table_builder_options,
+ WritableFileWriter* file);
+
+ // No copying allowed
+ BlockBasedTableBuilder(const BlockBasedTableBuilder&) = delete;
+ BlockBasedTableBuilder& operator=(const BlockBasedTableBuilder&) = delete;
+
+ // REQUIRES: Either Finish() or Abandon() has been called.
+ ~BlockBasedTableBuilder();
+
+ // Add key,value to the table being constructed.
+ // REQUIRES: key is after any previously added key according to comparator.
+ // REQUIRES: Finish(), Abandon() have not been called
+ void Add(const Slice& key, const Slice& value) override;
+
+ // Return non-ok iff some error has been detected.
+ Status status() const override;
+
+ // Return non-ok iff some error happens during IO.
+ IOStatus io_status() const override;
+
+ // Finish building the table. Stops using the file passed to the
+ // constructor after this function returns.
+ // REQUIRES: Finish(), Abandon() have not been called
+ Status Finish() override;
+
+ // Indicate that the contents of this builder should be abandoned. Stops
+ // using the file passed to the constructor after this function returns.
+ // If the caller is not going to call Finish(), it must call Abandon()
+ // before destroying this builder.
+ // REQUIRES: Finish(), Abandon() have not been called
+ void Abandon() override;
+
+ // Number of calls to Add() so far.
+ uint64_t NumEntries() const override;
+
+ bool IsEmpty() const override;
+
+ // Size of the file generated so far. If invoked after a successful
+ // Finish() call, returns the size of the final generated file.
+ uint64_t FileSize() const override;
+
+ // Estimated size of the file generated so far. This is used when
+ // FileSize() cannot estimate final SST size, e.g. parallel compression
+ // is enabled.
+ uint64_t EstimatedFileSize() const override;
+
+ bool NeedCompact() const override;
+
+ // Get table properties
+ TableProperties GetTableProperties() const override;
+
+ // Get file checksum
+ std::string GetFileChecksum() const override;
+
+ // Get file checksum function name
+ const char* GetFileChecksumFuncName() const override;
+
+ void SetSeqnoTimeTableProperties(
+ const std::string& encoded_seqno_to_time_mapping,
+ uint64_t oldest_ancestor_time) override;
+
+ private:
+ bool ok() const { return status().ok(); }
+
+ // Transition state from buffered to unbuffered. See `Rep::State` API comment
+ // for details of the states.
+ // REQUIRES: `rep_->state == kBuffered`
+ void EnterUnbuffered();
+
+ // Call block's Finish() method and then
+ // - in buffered mode, buffer the uncompressed block contents.
+ // - in unbuffered mode, write the compressed block contents to file.
+ void WriteBlock(BlockBuilder* block, BlockHandle* handle,
+ BlockType blocktype);
+
+ // Compress and write block content to the file.
+ void WriteBlock(const Slice& block_contents, BlockHandle* handle,
+ BlockType block_type);
+ // Directly write data to the file.
+ void WriteMaybeCompressedBlock(const Slice& data, CompressionType,
+ BlockHandle* handle, BlockType block_type,
+ const Slice* raw_data = nullptr);
+
+ void SetupCacheKeyPrefix(const TableBuilderOptions& tbo);
+
+ template <typename TBlocklike>
+ Status InsertBlockInCache(const Slice& block_contents,
+ const BlockHandle* handle, BlockType block_type);
+
+ Status InsertBlockInCacheHelper(const Slice& block_contents,
+ const BlockHandle* handle,
+ BlockType block_type);
+
+ Status InsertBlockInCompressedCache(const Slice& block_contents,
+ const CompressionType type,
+ const BlockHandle* handle);
+
+ void WriteFilterBlock(MetaIndexBuilder* meta_index_builder);
+ void WriteIndexBlock(MetaIndexBuilder* meta_index_builder,
+ BlockHandle* index_block_handle);
+ void WritePropertiesBlock(MetaIndexBuilder* meta_index_builder);
+ void WriteCompressionDictBlock(MetaIndexBuilder* meta_index_builder);
+ void WriteRangeDelBlock(MetaIndexBuilder* meta_index_builder);
+ void WriteFooter(BlockHandle& metaindex_block_handle,
+ BlockHandle& index_block_handle);
+
+ struct Rep;
+ class BlockBasedTablePropertiesCollectorFactory;
+ class BlockBasedTablePropertiesCollector;
+ Rep* rep_;
+
+ struct ParallelCompressionRep;
+
+ // Advanced operation: flush any buffered key/value pairs to file.
+ // Can be used to ensure that two adjacent entries never live in
+ // the same data block. Most clients should not need to use this method.
+ // REQUIRES: Finish(), Abandon() have not been called
+ void Flush();
+
+ // Some compression libraries fail when the uncompressed size is bigger than
+ // int. If uncompressed size is bigger than kCompressionSizeLimit, don't
+ // compress it
+ const uint64_t kCompressionSizeLimit = std::numeric_limits<int>::max();
+
+ // Get blocks from mem-table walking thread, compress them and
+ // pass them to the write thread. Used in parallel compression mode only
+ void BGWorkCompression(const CompressionContext& compression_ctx,
+ UncompressionContext* verify_ctx);
+
+ // Given uncompressed block content, try to compress it and return result and
+ // compression type
+ void CompressAndVerifyBlock(const Slice& uncompressed_block_data,
+ bool is_data_block,
+ const CompressionContext& compression_ctx,
+ UncompressionContext* verify_ctx,
+ std::string* compressed_output,
+ Slice* result_block_contents,
+ CompressionType* result_compression_type,
+ Status* out_status);
+
+ // Get compressed blocks from BGWorkCompression and write them into SST
+ void BGWorkWriteMaybeCompressedBlock();
+
+ // Initialize parallel compression context and
+ // start BGWorkCompression and BGWorkWriteMaybeCompressedBlock threads
+ void StartParallelCompression();
+
+ // Stop BGWorkCompression and BGWorkWriteMaybeCompressedBlock threads
+ void StopParallelCompression();
+};
+
+Slice CompressBlock(const Slice& uncompressed_data, const CompressionInfo& info,
+ CompressionType* type, uint32_t format_version,
+ bool do_sample, std::string* compressed_output,
+ std::string* sampled_output_fast,
+ std::string* sampled_output_slow);
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/block_based_table_factory.cc b/src/rocksdb/table/block_based/block_based_table_factory.cc
new file mode 100644
index 000000000..09c1d2f62
--- /dev/null
+++ b/src/rocksdb/table/block_based/block_based_table_factory.cc
@@ -0,0 +1,1058 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#include "table/block_based/block_based_table_factory.h"
+
+#include <stdint.h>
+
+#include <cinttypes>
+#include <memory>
+#include <string>
+
+#include "cache/cache_entry_roles.h"
+#include "cache/cache_reservation_manager.h"
+#include "logging/logging.h"
+#include "options/options_helper.h"
+#include "port/port.h"
+#include "rocksdb/cache.h"
+#include "rocksdb/convenience.h"
+#include "rocksdb/filter_policy.h"
+#include "rocksdb/flush_block_policy.h"
+#include "rocksdb/rocksdb_namespace.h"
+#include "rocksdb/table.h"
+#include "rocksdb/utilities/options_type.h"
+#include "table/block_based/block_based_table_builder.h"
+#include "table/block_based/block_based_table_reader.h"
+#include "table/format.h"
+#include "util/mutexlock.h"
+#include "util/string_util.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+void TailPrefetchStats::RecordEffectiveSize(size_t len) {
+ MutexLock l(&mutex_);
+ if (num_records_ < kNumTracked) {
+ num_records_++;
+ }
+ records_[next_++] = len;
+ if (next_ == kNumTracked) {
+ next_ = 0;
+ }
+}
+
+size_t TailPrefetchStats::GetSuggestedPrefetchSize() {
+ std::vector<size_t> sorted;
+ {
+ MutexLock l(&mutex_);
+
+ if (num_records_ == 0) {
+ return 0;
+ }
+ sorted.assign(records_, records_ + num_records_);
+ }
+
+ // Of the historic size, we find the maximum one that satisifis the condtiion
+ // that if prefetching all, less than 1/8 will be wasted.
+ std::sort(sorted.begin(), sorted.end());
+
+ // Assuming we have 5 data points, and after sorting it looks like this:
+ //
+ // +---+
+ // +---+ | |
+ // | | | |
+ // | | | |
+ // | | | |
+ // | | | |
+ // +---+ | | | |
+ // | | | | | |
+ // +---+ | | | | | |
+ // | | | | | | | |
+ // +---+ | | | | | | | |
+ // | | | | | | | | | |
+ // | | | | | | | | | |
+ // | | | | | | | | | |
+ // | | | | | | | | | |
+ // | | | | | | | | | |
+ // +---+ +---+ +---+ +---+ +---+
+ //
+ // and we use every of the value as a candidate, and estimate how much we
+ // wasted, compared to read. For example, when we use the 3rd record
+ // as candiate. This area is what we read:
+ // +---+
+ // +---+ | |
+ // | | | |
+ // | | | |
+ // | | | |
+ // | | | |
+ // *** *** *** ***+ *** *** *** *** **
+ // * | | | | | |
+ // +---+ | | | | | *
+ // * | | | | | | | |
+ // +---+ | | | | | | | *
+ // * | | | | X | | | | |
+ // | | | | | | | | | *
+ // * | | | | | | | | |
+ // | | | | | | | | | *
+ // * | | | | | | | | |
+ // *** *** ***-*** ***--*** ***--*** +****
+ // which is (size of the record) X (number of records).
+ //
+ // While wasted is this area:
+ // +---+
+ // +---+ | |
+ // | | | |
+ // | | | |
+ // | | | |
+ // | | | |
+ // *** *** *** ****---+ | | | |
+ // * * | | | | |
+ // * *-*** *** | | | | |
+ // * * | | | | | | |
+ // *--** *** | | | | | | |
+ // | | | | | X | | | | |
+ // | | | | | | | | | |
+ // | | | | | | | | | |
+ // | | | | | | | | | |
+ // | | | | | | | | | |
+ // +---+ +---+ +---+ +---+ +---+
+ //
+ // Which can be calculated iteratively.
+ // The difference between wasted using 4st and 3rd record, will
+ // be following area:
+ // +---+
+ // +--+ +-+ ++ +-+ +-+ +---+ | |
+ // + xxxxxxxxxxxxxxxxxxxxxxxx | | | |
+ // xxxxxxxxxxxxxxxxxxxxxxxx | | | |
+ // + xxxxxxxxxxxxxxxxxxxxxxxx | | | |
+ // | xxxxxxxxxxxxxxxxxxxxxxxx | | | |
+ // +-+ +-+ +-+ ++ +---+ +--+ | | |
+ // | | | | | | |
+ // +---+ ++ | | | | | |
+ // | | | | | | X | | |
+ // +---+ ++ | | | | | | | |
+ // | | | | | | | | | |
+ // | | | | | | | | | |
+ // | | | | | | | | | |
+ // | | | | | | | | | |
+ // | | | | | | | | | |
+ // +---+ +---+ +---+ +---+ +---+
+ //
+ // which will be the size difference between 4st and 3rd record,
+ // times 3, which is number of records before the 4st.
+ // Here we assume that all data within the prefetch range will be useful. In
+ // reality, it may not be the case when a partial block is inside the range,
+ // or there are data in the middle that is not read. We ignore those cases
+ // for simplicity.
+ assert(!sorted.empty());
+ size_t prev_size = sorted[0];
+ size_t max_qualified_size = sorted[0];
+ size_t wasted = 0;
+ for (size_t i = 1; i < sorted.size(); i++) {
+ size_t read = sorted[i] * sorted.size();
+ wasted += (sorted[i] - prev_size) * i;
+ if (wasted <= read / 8) {
+ max_qualified_size = sorted[i];
+ }
+ prev_size = sorted[i];
+ }
+ const size_t kMaxPrefetchSize = 512 * 1024; // Never exceed 512KB
+ return std::min(kMaxPrefetchSize, max_qualified_size);
+}
+
+#ifndef ROCKSDB_LITE
+
+const std::string kOptNameMetadataCacheOpts = "metadata_cache_options";
+
+static std::unordered_map<std::string, PinningTier>
+ pinning_tier_type_string_map = {
+ {"kFallback", PinningTier::kFallback},
+ {"kNone", PinningTier::kNone},
+ {"kFlushedAndSimilar", PinningTier::kFlushedAndSimilar},
+ {"kAll", PinningTier::kAll}};
+
+static std::unordered_map<std::string, BlockBasedTableOptions::IndexType>
+ block_base_table_index_type_string_map = {
+ {"kBinarySearch", BlockBasedTableOptions::IndexType::kBinarySearch},
+ {"kHashSearch", BlockBasedTableOptions::IndexType::kHashSearch},
+ {"kTwoLevelIndexSearch",
+ BlockBasedTableOptions::IndexType::kTwoLevelIndexSearch},
+ {"kBinarySearchWithFirstKey",
+ BlockBasedTableOptions::IndexType::kBinarySearchWithFirstKey}};
+
+static std::unordered_map<std::string,
+ BlockBasedTableOptions::DataBlockIndexType>
+ block_base_table_data_block_index_type_string_map = {
+ {"kDataBlockBinarySearch",
+ BlockBasedTableOptions::DataBlockIndexType::kDataBlockBinarySearch},
+ {"kDataBlockBinaryAndHash",
+ BlockBasedTableOptions::DataBlockIndexType::kDataBlockBinaryAndHash}};
+
+static std::unordered_map<std::string,
+ BlockBasedTableOptions::IndexShorteningMode>
+ block_base_table_index_shortening_mode_string_map = {
+ {"kNoShortening",
+ BlockBasedTableOptions::IndexShorteningMode::kNoShortening},
+ {"kShortenSeparators",
+ BlockBasedTableOptions::IndexShorteningMode::kShortenSeparators},
+ {"kShortenSeparatorsAndSuccessor",
+ BlockBasedTableOptions::IndexShorteningMode::
+ kShortenSeparatorsAndSuccessor}};
+
+static std::unordered_map<std::string, OptionTypeInfo>
+ metadata_cache_options_type_info = {
+ {"top_level_index_pinning",
+ OptionTypeInfo::Enum<PinningTier>(
+ offsetof(struct MetadataCacheOptions, top_level_index_pinning),
+ &pinning_tier_type_string_map)},
+ {"partition_pinning",
+ OptionTypeInfo::Enum<PinningTier>(
+ offsetof(struct MetadataCacheOptions, partition_pinning),
+ &pinning_tier_type_string_map)},
+ {"unpartitioned_pinning",
+ OptionTypeInfo::Enum<PinningTier>(
+ offsetof(struct MetadataCacheOptions, unpartitioned_pinning),
+ &pinning_tier_type_string_map)}};
+
+static std::unordered_map<std::string,
+ BlockBasedTableOptions::PrepopulateBlockCache>
+ block_base_table_prepopulate_block_cache_string_map = {
+ {"kDisable", BlockBasedTableOptions::PrepopulateBlockCache::kDisable},
+ {"kFlushOnly",
+ BlockBasedTableOptions::PrepopulateBlockCache::kFlushOnly}};
+
+#endif // ROCKSDB_LITE
+
+static std::unordered_map<std::string, OptionTypeInfo>
+ block_based_table_type_info = {
+#ifndef ROCKSDB_LITE
+ /* currently not supported
+ std::shared_ptr<Cache> block_cache = nullptr;
+ std::shared_ptr<Cache> block_cache_compressed = nullptr;
+ CacheUsageOptions cache_usage_options;
+ */
+ {"flush_block_policy_factory",
+ OptionTypeInfo::AsCustomSharedPtr<FlushBlockPolicyFactory>(
+ offsetof(struct BlockBasedTableOptions,
+ flush_block_policy_factory),
+ OptionVerificationType::kByName, OptionTypeFlags::kCompareNever)},
+ {"cache_index_and_filter_blocks",
+ {offsetof(struct BlockBasedTableOptions,
+ cache_index_and_filter_blocks),
+ OptionType::kBoolean, OptionVerificationType::kNormal,
+ OptionTypeFlags::kNone}},
+ {"cache_index_and_filter_blocks_with_high_priority",
+ {offsetof(struct BlockBasedTableOptions,
+ cache_index_and_filter_blocks_with_high_priority),
+ OptionType::kBoolean, OptionVerificationType::kNormal,
+ OptionTypeFlags::kNone}},
+ {"pin_l0_filter_and_index_blocks_in_cache",
+ {offsetof(struct BlockBasedTableOptions,
+ pin_l0_filter_and_index_blocks_in_cache),
+ OptionType::kBoolean, OptionVerificationType::kNormal,
+ OptionTypeFlags::kNone}},
+ {"index_type", OptionTypeInfo::Enum<BlockBasedTableOptions::IndexType>(
+ offsetof(struct BlockBasedTableOptions, index_type),
+ &block_base_table_index_type_string_map)},
+ {"hash_index_allow_collision",
+ {0, OptionType::kBoolean, OptionVerificationType::kDeprecated,
+ OptionTypeFlags::kNone}},
+ {"data_block_index_type",
+ OptionTypeInfo::Enum<BlockBasedTableOptions::DataBlockIndexType>(
+ offsetof(struct BlockBasedTableOptions, data_block_index_type),
+ &block_base_table_data_block_index_type_string_map)},
+ {"index_shortening",
+ OptionTypeInfo::Enum<BlockBasedTableOptions::IndexShorteningMode>(
+ offsetof(struct BlockBasedTableOptions, index_shortening),
+ &block_base_table_index_shortening_mode_string_map)},
+ {"data_block_hash_table_util_ratio",
+ {offsetof(struct BlockBasedTableOptions,
+ data_block_hash_table_util_ratio),
+ OptionType::kDouble, OptionVerificationType::kNormal,
+ OptionTypeFlags::kNone}},
+ {"checksum",
+ {offsetof(struct BlockBasedTableOptions, checksum),
+ OptionType::kChecksumType, OptionVerificationType::kNormal,
+ OptionTypeFlags::kNone}},
+ {"no_block_cache",
+ {offsetof(struct BlockBasedTableOptions, no_block_cache),
+ OptionType::kBoolean, OptionVerificationType::kNormal,
+ OptionTypeFlags::kNone}},
+ {"block_size",
+ {offsetof(struct BlockBasedTableOptions, block_size),
+ OptionType::kSizeT, OptionVerificationType::kNormal,
+ OptionTypeFlags::kMutable}},
+ {"block_size_deviation",
+ {offsetof(struct BlockBasedTableOptions, block_size_deviation),
+ OptionType::kInt, OptionVerificationType::kNormal,
+ OptionTypeFlags::kNone}},
+ {"block_restart_interval",
+ {offsetof(struct BlockBasedTableOptions, block_restart_interval),
+ OptionType::kInt, OptionVerificationType::kNormal,
+ OptionTypeFlags::kMutable}},
+ {"index_block_restart_interval",
+ {offsetof(struct BlockBasedTableOptions, index_block_restart_interval),
+ OptionType::kInt, OptionVerificationType::kNormal,
+ OptionTypeFlags::kNone}},
+ {"index_per_partition",
+ {0, OptionType::kUInt64T, OptionVerificationType::kDeprecated,
+ OptionTypeFlags::kNone}},
+ {"metadata_block_size",
+ {offsetof(struct BlockBasedTableOptions, metadata_block_size),
+ OptionType::kUInt64T, OptionVerificationType::kNormal,
+ OptionTypeFlags::kNone}},
+ {"partition_filters",
+ {offsetof(struct BlockBasedTableOptions, partition_filters),
+ OptionType::kBoolean, OptionVerificationType::kNormal,
+ OptionTypeFlags::kNone}},
+ {"optimize_filters_for_memory",
+ {offsetof(struct BlockBasedTableOptions, optimize_filters_for_memory),
+ OptionType::kBoolean, OptionVerificationType::kNormal,
+ OptionTypeFlags::kNone}},
+ {"filter_policy",
+ OptionTypeInfo::AsCustomSharedPtr<const FilterPolicy>(
+ offsetof(struct BlockBasedTableOptions, filter_policy),
+ OptionVerificationType::kByNameAllowFromNull,
+ OptionTypeFlags::kNone)},
+ {"whole_key_filtering",
+ {offsetof(struct BlockBasedTableOptions, whole_key_filtering),
+ OptionType::kBoolean, OptionVerificationType::kNormal,
+ OptionTypeFlags::kNone}},
+ {"detect_filter_construct_corruption",
+ {offsetof(struct BlockBasedTableOptions,
+ detect_filter_construct_corruption),
+ OptionType::kBoolean, OptionVerificationType::kNormal,
+ OptionTypeFlags::kMutable}},
+ {"reserve_table_builder_memory",
+ {0, OptionType::kBoolean, OptionVerificationType::kDeprecated,
+ OptionTypeFlags::kNone}},
+ {"reserve_table_reader_memory",
+ {0, OptionType::kBoolean, OptionVerificationType::kDeprecated,
+ OptionTypeFlags::kNone}},
+ {"skip_table_builder_flush",
+ {0, OptionType::kBoolean, OptionVerificationType::kDeprecated,
+ OptionTypeFlags::kNone}},
+ {"format_version",
+ {offsetof(struct BlockBasedTableOptions, format_version),
+ OptionType::kUInt32T, OptionVerificationType::kNormal,
+ OptionTypeFlags::kNone}},
+ {"verify_compression",
+ {offsetof(struct BlockBasedTableOptions, verify_compression),
+ OptionType::kBoolean, OptionVerificationType::kNormal,
+ OptionTypeFlags::kNone}},
+ {"read_amp_bytes_per_bit",
+ {offsetof(struct BlockBasedTableOptions, read_amp_bytes_per_bit),
+ OptionType::kUInt32T, OptionVerificationType::kNormal,
+ OptionTypeFlags::kNone,
+ [](const ConfigOptions& /*opts*/, const std::string& /*name*/,
+ const std::string& value, void* addr) {
+ // A workaround to fix a bug in 6.10, 6.11, 6.12, 6.13
+ // and 6.14. The bug will write out 8 bytes to OPTIONS file from the
+ // starting address of BlockBasedTableOptions.read_amp_bytes_per_bit
+ // which is actually a uint32. Consequently, the value of
+ // read_amp_bytes_per_bit written in the OPTIONS file is wrong.
+ // From 6.15, RocksDB will try to parse the read_amp_bytes_per_bit
+ // from OPTIONS file as a uint32. To be able to load OPTIONS file
+ // generated by affected releases before the fix, we need to
+ // manually parse read_amp_bytes_per_bit with this special hack.
+ uint64_t read_amp_bytes_per_bit = ParseUint64(value);
+ *(static_cast<uint32_t*>(addr)) =
+ static_cast<uint32_t>(read_amp_bytes_per_bit);
+ return Status::OK();
+ }}},
+ {"enable_index_compression",
+ {offsetof(struct BlockBasedTableOptions, enable_index_compression),
+ OptionType::kBoolean, OptionVerificationType::kNormal,
+ OptionTypeFlags::kNone}},
+ {"block_align",
+ {offsetof(struct BlockBasedTableOptions, block_align),
+ OptionType::kBoolean, OptionVerificationType::kNormal,
+ OptionTypeFlags::kNone}},
+ {"pin_top_level_index_and_filter",
+ {offsetof(struct BlockBasedTableOptions,
+ pin_top_level_index_and_filter),
+ OptionType::kBoolean, OptionVerificationType::kNormal,
+ OptionTypeFlags::kNone}},
+ {kOptNameMetadataCacheOpts,
+ OptionTypeInfo::Struct(
+ kOptNameMetadataCacheOpts, &metadata_cache_options_type_info,
+ offsetof(struct BlockBasedTableOptions, metadata_cache_options),
+ OptionVerificationType::kNormal, OptionTypeFlags::kNone)},
+ {"block_cache",
+ {offsetof(struct BlockBasedTableOptions, block_cache),
+ OptionType::kUnknown, OptionVerificationType::kNormal,
+ (OptionTypeFlags::kCompareNever | OptionTypeFlags::kDontSerialize),
+ // Parses the input value as a Cache
+ [](const ConfigOptions& opts, const std::string&,
+ const std::string& value, void* addr) {
+ auto* cache = static_cast<std::shared_ptr<Cache>*>(addr);
+ return Cache::CreateFromString(opts, value, cache);
+ }}},
+ {"block_cache_compressed",
+ {offsetof(struct BlockBasedTableOptions, block_cache_compressed),
+ OptionType::kUnknown, OptionVerificationType::kNormal,
+ (OptionTypeFlags::kCompareNever | OptionTypeFlags::kDontSerialize),
+ // Parses the input value as a Cache
+ [](const ConfigOptions& opts, const std::string&,
+ const std::string& value, void* addr) {
+ auto* cache = static_cast<std::shared_ptr<Cache>*>(addr);
+ return Cache::CreateFromString(opts, value, cache);
+ }}},
+ {"max_auto_readahead_size",
+ {offsetof(struct BlockBasedTableOptions, max_auto_readahead_size),
+ OptionType::kSizeT, OptionVerificationType::kNormal,
+ OptionTypeFlags::kMutable}},
+ {"prepopulate_block_cache",
+ OptionTypeInfo::Enum<BlockBasedTableOptions::PrepopulateBlockCache>(
+ offsetof(struct BlockBasedTableOptions, prepopulate_block_cache),
+ &block_base_table_prepopulate_block_cache_string_map,
+ OptionTypeFlags::kMutable)},
+ {"initial_auto_readahead_size",
+ {offsetof(struct BlockBasedTableOptions, initial_auto_readahead_size),
+ OptionType::kSizeT, OptionVerificationType::kNormal,
+ OptionTypeFlags::kMutable}},
+ {"num_file_reads_for_auto_readahead",
+ {offsetof(struct BlockBasedTableOptions,
+ num_file_reads_for_auto_readahead),
+ OptionType::kUInt64T, OptionVerificationType::kNormal,
+ OptionTypeFlags::kMutable}},
+
+#endif // ROCKSDB_LITE
+};
+
+// TODO(myabandeh): We should return an error instead of silently changing the
+// options
+BlockBasedTableFactory::BlockBasedTableFactory(
+ const BlockBasedTableOptions& _table_options)
+ : table_options_(_table_options) {
+ InitializeOptions();
+ RegisterOptions(&table_options_, &block_based_table_type_info);
+
+ const auto table_reader_charged =
+ table_options_.cache_usage_options.options_overrides
+ .at(CacheEntryRole::kBlockBasedTableReader)
+ .charged;
+ if (table_options_.block_cache &&
+ table_reader_charged == CacheEntryRoleOptions::Decision::kEnabled) {
+ table_reader_cache_res_mgr_.reset(new ConcurrentCacheReservationManager(
+ std::make_shared<CacheReservationManagerImpl<
+ CacheEntryRole::kBlockBasedTableReader>>(
+ table_options_.block_cache)));
+ }
+}
+
+void BlockBasedTableFactory::InitializeOptions() {
+ if (table_options_.flush_block_policy_factory == nullptr) {
+ table_options_.flush_block_policy_factory.reset(
+ new FlushBlockBySizePolicyFactory());
+ }
+ if (table_options_.no_block_cache) {
+ table_options_.block_cache.reset();
+ } else if (table_options_.block_cache == nullptr) {
+ LRUCacheOptions co;
+ co.capacity = 8 << 20;
+ // It makes little sense to pay overhead for mid-point insertion while the
+ // block size is only 8MB.
+ co.high_pri_pool_ratio = 0.0;
+ co.low_pri_pool_ratio = 0.0;
+ table_options_.block_cache = NewLRUCache(co);
+ }
+ if (table_options_.block_size_deviation < 0 ||
+ table_options_.block_size_deviation > 100) {
+ table_options_.block_size_deviation = 0;
+ }
+ if (table_options_.block_restart_interval < 1) {
+ table_options_.block_restart_interval = 1;
+ }
+ if (table_options_.index_block_restart_interval < 1) {
+ table_options_.index_block_restart_interval = 1;
+ }
+ if (table_options_.index_type == BlockBasedTableOptions::kHashSearch &&
+ table_options_.index_block_restart_interval != 1) {
+ // Currently kHashSearch is incompatible with
+ // index_block_restart_interval > 1
+ table_options_.index_block_restart_interval = 1;
+ }
+ if (table_options_.partition_filters &&
+ table_options_.index_type !=
+ BlockBasedTableOptions::kTwoLevelIndexSearch) {
+ // We do not support partitioned filters without partitioning indexes
+ table_options_.partition_filters = false;
+ }
+ auto& options_overrides =
+ table_options_.cache_usage_options.options_overrides;
+ const auto options = table_options_.cache_usage_options.options;
+ for (std::uint32_t i = 0; i < kNumCacheEntryRoles; ++i) {
+ CacheEntryRole role = static_cast<CacheEntryRole>(i);
+ auto options_overrides_iter = options_overrides.find(role);
+ if (options_overrides_iter == options_overrides.end()) {
+ options_overrides.insert({role, options});
+ } else if (options_overrides_iter->second.charged ==
+ CacheEntryRoleOptions::Decision::kFallback) {
+ options_overrides_iter->second.charged = options.charged;
+ }
+ }
+}
+
+Status BlockBasedTableFactory::PrepareOptions(const ConfigOptions& opts) {
+ InitializeOptions();
+ return TableFactory::PrepareOptions(opts);
+}
+
+namespace {
+// Different cache kinds use the same keys for physically different values, so
+// they must not share an underlying key space with each other.
+Status CheckCacheOptionCompatibility(const BlockBasedTableOptions& bbto) {
+ int cache_count = (bbto.block_cache != nullptr) +
+ (bbto.block_cache_compressed != nullptr) +
+ (bbto.persistent_cache != nullptr);
+ if (cache_count <= 1) {
+ // Nothing to share / overlap
+ return Status::OK();
+ }
+
+ // Simple pointer equality
+ if (bbto.block_cache == bbto.block_cache_compressed) {
+ return Status::InvalidArgument(
+ "block_cache same as block_cache_compressed not currently supported, "
+ "and would be bad for performance anyway");
+ }
+
+ // More complex test of shared key space, in case the instances are wrappers
+ // for some shared underlying cache.
+ CacheKey sentinel_key = CacheKey::CreateUniqueForProcessLifetime();
+ static char kRegularBlockCacheMarker = 'b';
+ static char kCompressedBlockCacheMarker = 'c';
+ static char kPersistentCacheMarker = 'p';
+ if (bbto.block_cache) {
+ bbto.block_cache
+ ->Insert(sentinel_key.AsSlice(), &kRegularBlockCacheMarker, 1,
+ GetNoopDeleterForRole<CacheEntryRole::kMisc>())
+ .PermitUncheckedError();
+ }
+ if (bbto.block_cache_compressed) {
+ bbto.block_cache_compressed
+ ->Insert(sentinel_key.AsSlice(), &kCompressedBlockCacheMarker, 1,
+ GetNoopDeleterForRole<CacheEntryRole::kMisc>())
+ .PermitUncheckedError();
+ }
+ if (bbto.persistent_cache) {
+ // Note: persistent cache copies the data, not keeping the pointer
+ bbto.persistent_cache
+ ->Insert(sentinel_key.AsSlice(), &kPersistentCacheMarker, 1)
+ .PermitUncheckedError();
+ }
+ // If we get something different from what we inserted, that indicates
+ // dangerously overlapping key spaces.
+ if (bbto.block_cache) {
+ auto handle = bbto.block_cache->Lookup(sentinel_key.AsSlice());
+ if (handle) {
+ auto v = static_cast<char*>(bbto.block_cache->Value(handle));
+ char c = *v;
+ bbto.block_cache->Release(handle);
+ if (v == &kCompressedBlockCacheMarker) {
+ return Status::InvalidArgument(
+ "block_cache and block_cache_compressed share the same key space, "
+ "which is not supported");
+ } else if (c == kPersistentCacheMarker) {
+ return Status::InvalidArgument(
+ "block_cache and persistent_cache share the same key space, "
+ "which is not supported");
+ } else if (v != &kRegularBlockCacheMarker) {
+ return Status::Corruption("Unexpected mutation to block_cache");
+ }
+ }
+ }
+ if (bbto.block_cache_compressed) {
+ auto handle = bbto.block_cache_compressed->Lookup(sentinel_key.AsSlice());
+ if (handle) {
+ auto v = static_cast<char*>(bbto.block_cache_compressed->Value(handle));
+ char c = *v;
+ bbto.block_cache_compressed->Release(handle);
+ if (v == &kRegularBlockCacheMarker) {
+ return Status::InvalidArgument(
+ "block_cache_compressed and block_cache share the same key space, "
+ "which is not supported");
+ } else if (c == kPersistentCacheMarker) {
+ return Status::InvalidArgument(
+ "block_cache_compressed and persistent_cache share the same key "
+ "space, "
+ "which is not supported");
+ } else if (v != &kCompressedBlockCacheMarker) {
+ return Status::Corruption(
+ "Unexpected mutation to block_cache_compressed");
+ }
+ }
+ }
+ if (bbto.persistent_cache) {
+ std::unique_ptr<char[]> data;
+ size_t size = 0;
+ bbto.persistent_cache->Lookup(sentinel_key.AsSlice(), &data, &size)
+ .PermitUncheckedError();
+ if (data && size > 0) {
+ if (data[0] == kRegularBlockCacheMarker) {
+ return Status::InvalidArgument(
+ "persistent_cache and block_cache share the same key space, "
+ "which is not supported");
+ } else if (data[0] == kCompressedBlockCacheMarker) {
+ return Status::InvalidArgument(
+ "persistent_cache and block_cache_compressed share the same key "
+ "space, "
+ "which is not supported");
+ } else if (data[0] != kPersistentCacheMarker) {
+ return Status::Corruption("Unexpected mutation to persistent_cache");
+ }
+ }
+ }
+ return Status::OK();
+}
+
+} // namespace
+
+Status BlockBasedTableFactory::NewTableReader(
+ const ReadOptions& ro, const TableReaderOptions& table_reader_options,
+ std::unique_ptr<RandomAccessFileReader>&& file, uint64_t file_size,
+ std::unique_ptr<TableReader>* table_reader,
+ bool prefetch_index_and_filter_in_cache) const {
+ return BlockBasedTable::Open(
+ ro, table_reader_options.ioptions, table_reader_options.env_options,
+ table_options_, table_reader_options.internal_comparator, std::move(file),
+ file_size, table_reader, table_reader_cache_res_mgr_,
+ table_reader_options.prefix_extractor, prefetch_index_and_filter_in_cache,
+ table_reader_options.skip_filters, table_reader_options.level,
+ table_reader_options.immortal, table_reader_options.largest_seqno,
+ table_reader_options.force_direct_prefetch, &tail_prefetch_stats_,
+ table_reader_options.block_cache_tracer,
+ table_reader_options.max_file_size_for_l0_meta_pin,
+ table_reader_options.cur_db_session_id, table_reader_options.cur_file_num,
+ table_reader_options.unique_id);
+}
+
+TableBuilder* BlockBasedTableFactory::NewTableBuilder(
+ const TableBuilderOptions& table_builder_options,
+ WritableFileWriter* file) const {
+ return new BlockBasedTableBuilder(table_options_, table_builder_options,
+ file);
+}
+
+Status BlockBasedTableFactory::ValidateOptions(
+ const DBOptions& db_opts, const ColumnFamilyOptions& cf_opts) const {
+ if (table_options_.index_type == BlockBasedTableOptions::kHashSearch &&
+ cf_opts.prefix_extractor == nullptr) {
+ return Status::InvalidArgument(
+ "Hash index is specified for block-based "
+ "table, but prefix_extractor is not given");
+ }
+ if (table_options_.cache_index_and_filter_blocks &&
+ table_options_.no_block_cache) {
+ return Status::InvalidArgument(
+ "Enable cache_index_and_filter_blocks, "
+ ", but block cache is disabled");
+ }
+ if (table_options_.pin_l0_filter_and_index_blocks_in_cache &&
+ table_options_.no_block_cache) {
+ return Status::InvalidArgument(
+ "Enable pin_l0_filter_and_index_blocks_in_cache, "
+ ", but block cache is disabled");
+ }
+ if (!IsSupportedFormatVersion(table_options_.format_version)) {
+ return Status::InvalidArgument(
+ "Unsupported BlockBasedTable format_version. Please check "
+ "include/rocksdb/table.h for more info");
+ }
+ if (table_options_.block_align && (cf_opts.compression != kNoCompression)) {
+ return Status::InvalidArgument(
+ "Enable block_align, but compression "
+ "enabled");
+ }
+ if (table_options_.block_align &&
+ (table_options_.block_size & (table_options_.block_size - 1))) {
+ return Status::InvalidArgument(
+ "Block alignment requested but block size is not a power of 2");
+ }
+ if (table_options_.block_size > std::numeric_limits<uint32_t>::max()) {
+ return Status::InvalidArgument(
+ "block size exceeds maximum number (4GiB) allowed");
+ }
+ if (table_options_.data_block_index_type ==
+ BlockBasedTableOptions::kDataBlockBinaryAndHash &&
+ table_options_.data_block_hash_table_util_ratio <= 0) {
+ return Status::InvalidArgument(
+ "data_block_hash_table_util_ratio should be greater than 0 when "
+ "data_block_index_type is set to kDataBlockBinaryAndHash");
+ }
+ if (db_opts.unordered_write && cf_opts.max_successive_merges > 0) {
+ // TODO(myabandeh): support it
+ return Status::InvalidArgument(
+ "max_successive_merges larger than 0 is currently inconsistent with "
+ "unordered_write");
+ }
+ const auto& options_overrides =
+ table_options_.cache_usage_options.options_overrides;
+ for (auto options_overrides_iter = options_overrides.cbegin();
+ options_overrides_iter != options_overrides.cend();
+ ++options_overrides_iter) {
+ const CacheEntryRole role = options_overrides_iter->first;
+ const CacheEntryRoleOptions options = options_overrides_iter->second;
+ static const std::set<CacheEntryRole> kMemoryChargingSupported = {
+ CacheEntryRole::kCompressionDictionaryBuildingBuffer,
+ CacheEntryRole::kFilterConstruction,
+ CacheEntryRole::kBlockBasedTableReader, CacheEntryRole::kFileMetadata,
+ CacheEntryRole::kBlobCache};
+ if (options.charged != CacheEntryRoleOptions::Decision::kFallback &&
+ kMemoryChargingSupported.count(role) == 0) {
+ return Status::NotSupported(
+ "Enable/Disable CacheEntryRoleOptions::charged"
+ " for CacheEntryRole " +
+ kCacheEntryRoleToCamelString[static_cast<uint32_t>(role)] +
+ " is not supported");
+ }
+ if (table_options_.no_block_cache &&
+ options.charged == CacheEntryRoleOptions::Decision::kEnabled) {
+ return Status::InvalidArgument(
+ "Enable CacheEntryRoleOptions::charged"
+ " for CacheEntryRole " +
+ kCacheEntryRoleToCamelString[static_cast<uint32_t>(role)] +
+ " but block cache is disabled");
+ }
+ if (role == CacheEntryRole::kBlobCache &&
+ options.charged == CacheEntryRoleOptions::Decision::kEnabled) {
+ if (cf_opts.blob_cache == nullptr) {
+ return Status::InvalidArgument(
+ "Enable CacheEntryRoleOptions::charged"
+ " for CacheEntryRole " +
+ kCacheEntryRoleToCamelString[static_cast<uint32_t>(role)] +
+ " but blob cache is not configured");
+ }
+ if (table_options_.no_block_cache) {
+ return Status::InvalidArgument(
+ "Enable CacheEntryRoleOptions::charged"
+ " for CacheEntryRole " +
+ kCacheEntryRoleToCamelString[static_cast<uint32_t>(role)] +
+ " but block cache is disabled");
+ }
+ if (table_options_.block_cache == cf_opts.blob_cache) {
+ return Status::InvalidArgument(
+ "Enable CacheEntryRoleOptions::charged"
+ " for CacheEntryRole " +
+ kCacheEntryRoleToCamelString[static_cast<uint32_t>(role)] +
+ " but blob cache is the same as block cache");
+ }
+ if (cf_opts.blob_cache->GetCapacity() >
+ table_options_.block_cache->GetCapacity()) {
+ return Status::InvalidArgument(
+ "Enable CacheEntryRoleOptions::charged"
+ " for CacheEntryRole " +
+ kCacheEntryRoleToCamelString[static_cast<uint32_t>(role)] +
+ " but blob cache capacity is larger than block cache capacity");
+ }
+ }
+ }
+ {
+ Status s = CheckCacheOptionCompatibility(table_options_);
+ if (!s.ok()) {
+ return s;
+ }
+ }
+ std::string garbage;
+ if (!SerializeEnum<ChecksumType>(checksum_type_string_map,
+ table_options_.checksum, &garbage)) {
+ return Status::InvalidArgument(
+ "Unrecognized ChecksumType for checksum: " +
+ std::to_string(static_cast<uint32_t>(table_options_.checksum)));
+ }
+ return TableFactory::ValidateOptions(db_opts, cf_opts);
+}
+
+std::string BlockBasedTableFactory::GetPrintableOptions() const {
+ std::string ret;
+ ret.reserve(20000);
+ const int kBufferSize = 200;
+ char buffer[kBufferSize];
+
+ snprintf(buffer, kBufferSize, " flush_block_policy_factory: %s (%p)\n",
+ table_options_.flush_block_policy_factory->Name(),
+ static_cast<void*>(table_options_.flush_block_policy_factory.get()));
+ ret.append(buffer);
+ snprintf(buffer, kBufferSize, " cache_index_and_filter_blocks: %d\n",
+ table_options_.cache_index_and_filter_blocks);
+ ret.append(buffer);
+ snprintf(buffer, kBufferSize,
+ " cache_index_and_filter_blocks_with_high_priority: %d\n",
+ table_options_.cache_index_and_filter_blocks_with_high_priority);
+ ret.append(buffer);
+ snprintf(buffer, kBufferSize,
+ " pin_l0_filter_and_index_blocks_in_cache: %d\n",
+ table_options_.pin_l0_filter_and_index_blocks_in_cache);
+ ret.append(buffer);
+ snprintf(buffer, kBufferSize, " pin_top_level_index_and_filter: %d\n",
+ table_options_.pin_top_level_index_and_filter);
+ ret.append(buffer);
+ snprintf(buffer, kBufferSize, " index_type: %d\n",
+ table_options_.index_type);
+ ret.append(buffer);
+ snprintf(buffer, kBufferSize, " data_block_index_type: %d\n",
+ table_options_.data_block_index_type);
+ ret.append(buffer);
+ snprintf(buffer, kBufferSize, " index_shortening: %d\n",
+ static_cast<int>(table_options_.index_shortening));
+ ret.append(buffer);
+ snprintf(buffer, kBufferSize, " data_block_hash_table_util_ratio: %lf\n",
+ table_options_.data_block_hash_table_util_ratio);
+ ret.append(buffer);
+ snprintf(buffer, kBufferSize, " checksum: %d\n", table_options_.checksum);
+ ret.append(buffer);
+ snprintf(buffer, kBufferSize, " no_block_cache: %d\n",
+ table_options_.no_block_cache);
+ ret.append(buffer);
+ snprintf(buffer, kBufferSize, " block_cache: %p\n",
+ static_cast<void*>(table_options_.block_cache.get()));
+ ret.append(buffer);
+ if (table_options_.block_cache) {
+ const char* block_cache_name = table_options_.block_cache->Name();
+ if (block_cache_name != nullptr) {
+ snprintf(buffer, kBufferSize, " block_cache_name: %s\n",
+ block_cache_name);
+ ret.append(buffer);
+ }
+ ret.append(" block_cache_options:\n");
+ ret.append(table_options_.block_cache->GetPrintableOptions());
+ }
+ snprintf(buffer, kBufferSize, " block_cache_compressed: %p\n",
+ static_cast<void*>(table_options_.block_cache_compressed.get()));
+ ret.append(buffer);
+ if (table_options_.block_cache_compressed) {
+ const char* block_cache_compressed_name =
+ table_options_.block_cache_compressed->Name();
+ if (block_cache_compressed_name != nullptr) {
+ snprintf(buffer, kBufferSize, " block_cache_name: %s\n",
+ block_cache_compressed_name);
+ ret.append(buffer);
+ }
+ ret.append(" block_cache_compressed_options:\n");
+ ret.append(table_options_.block_cache_compressed->GetPrintableOptions());
+ }
+ snprintf(buffer, kBufferSize, " persistent_cache: %p\n",
+ static_cast<void*>(table_options_.persistent_cache.get()));
+ ret.append(buffer);
+ if (table_options_.persistent_cache) {
+ snprintf(buffer, kBufferSize, " persistent_cache_options:\n");
+ ret.append(buffer);
+ ret.append(table_options_.persistent_cache->GetPrintableOptions());
+ }
+ snprintf(buffer, kBufferSize, " block_size: %" PRIu64 "\n",
+ table_options_.block_size);
+ ret.append(buffer);
+ snprintf(buffer, kBufferSize, " block_size_deviation: %d\n",
+ table_options_.block_size_deviation);
+ ret.append(buffer);
+ snprintf(buffer, kBufferSize, " block_restart_interval: %d\n",
+ table_options_.block_restart_interval);
+ ret.append(buffer);
+ snprintf(buffer, kBufferSize, " index_block_restart_interval: %d\n",
+ table_options_.index_block_restart_interval);
+ ret.append(buffer);
+ snprintf(buffer, kBufferSize, " metadata_block_size: %" PRIu64 "\n",
+ table_options_.metadata_block_size);
+ ret.append(buffer);
+ snprintf(buffer, kBufferSize, " partition_filters: %d\n",
+ table_options_.partition_filters);
+ ret.append(buffer);
+ snprintf(buffer, kBufferSize, " use_delta_encoding: %d\n",
+ table_options_.use_delta_encoding);
+ ret.append(buffer);
+ snprintf(buffer, kBufferSize, " filter_policy: %s\n",
+ table_options_.filter_policy == nullptr
+ ? "nullptr"
+ : table_options_.filter_policy->Name());
+ ret.append(buffer);
+ snprintf(buffer, kBufferSize, " whole_key_filtering: %d\n",
+ table_options_.whole_key_filtering);
+ ret.append(buffer);
+ snprintf(buffer, kBufferSize, " verify_compression: %d\n",
+ table_options_.verify_compression);
+ ret.append(buffer);
+ snprintf(buffer, kBufferSize, " read_amp_bytes_per_bit: %d\n",
+ table_options_.read_amp_bytes_per_bit);
+ ret.append(buffer);
+ snprintf(buffer, kBufferSize, " format_version: %d\n",
+ table_options_.format_version);
+ ret.append(buffer);
+ snprintf(buffer, kBufferSize, " enable_index_compression: %d\n",
+ table_options_.enable_index_compression);
+ ret.append(buffer);
+ snprintf(buffer, kBufferSize, " block_align: %d\n",
+ table_options_.block_align);
+ ret.append(buffer);
+ snprintf(buffer, kBufferSize,
+ " max_auto_readahead_size: %" ROCKSDB_PRIszt "\n",
+ table_options_.max_auto_readahead_size);
+ ret.append(buffer);
+ snprintf(buffer, kBufferSize, " prepopulate_block_cache: %d\n",
+ static_cast<int>(table_options_.prepopulate_block_cache));
+ ret.append(buffer);
+ snprintf(buffer, kBufferSize,
+ " initial_auto_readahead_size: %" ROCKSDB_PRIszt "\n",
+ table_options_.initial_auto_readahead_size);
+ ret.append(buffer);
+ snprintf(buffer, kBufferSize,
+ " num_file_reads_for_auto_readahead: %" PRIu64 "\n",
+ table_options_.num_file_reads_for_auto_readahead);
+ ret.append(buffer);
+ return ret;
+}
+
+const void* BlockBasedTableFactory::GetOptionsPtr(
+ const std::string& name) const {
+ if (name == kBlockCacheOpts()) {
+ if (table_options_.no_block_cache) {
+ return nullptr;
+ } else {
+ return table_options_.block_cache.get();
+ }
+ } else {
+ return TableFactory::GetOptionsPtr(name);
+ }
+}
+
+#ifndef ROCKSDB_LITE
+// Take a default BlockBasedTableOptions "table_options" in addition to a
+// map "opts_map" of option name to option value to construct the new
+// BlockBasedTableOptions "new_table_options".
+//
+// Below are the instructions of how to config some non-primitive-typed
+// options in BlockBasedTableOptions:
+//
+// * filter_policy:
+// We currently only support the following FilterPolicy in the convenience
+// functions:
+// - BloomFilter: use "bloomfilter:[bits_per_key]:[use_block_based_builder]"
+// to specify BloomFilter. The above string is equivalent to calling
+// NewBloomFilterPolicy(bits_per_key, use_block_based_builder).
+// [Example]:
+// - Pass {"filter_policy", "bloomfilter:4:true"} in
+// GetBlockBasedTableOptionsFromMap to use a BloomFilter with 4-bits
+// per key and use_block_based_builder enabled.
+//
+// * block_cache / block_cache_compressed:
+// We currently only support LRU cache in the GetOptions API. The LRU
+// cache can be set by directly specifying its size.
+// [Example]:
+// - Passing {"block_cache", "1M"} in GetBlockBasedTableOptionsFromMap is
+// equivalent to setting block_cache using NewLRUCache(1024 * 1024).
+//
+// @param table_options the default options of the output "new_table_options".
+// @param opts_map an option name to value map for specifying how
+// "new_table_options" should be set.
+// @param new_table_options the resulting options based on "table_options"
+// with the change specified in "opts_map".
+// @param input_strings_escaped when set to true, each escaped characters
+// prefixed by '\' in the values of the opts_map will be further converted
+// back to the raw string before assigning to the associated options.
+// @param ignore_unknown_options when set to true, unknown options are ignored
+// instead of resulting in an unknown-option error.
+// @return Status::OK() on success. Otherwise, a non-ok status indicating
+// error will be returned, and "new_table_options" will be set to
+// "table_options".
+Status BlockBasedTableFactory::ParseOption(const ConfigOptions& config_options,
+ const OptionTypeInfo& opt_info,
+ const std::string& opt_name,
+ const std::string& opt_value,
+ void* opt_ptr) {
+ Status status = TableFactory::ParseOption(config_options, opt_info, opt_name,
+ opt_value, opt_ptr);
+ if (config_options.input_strings_escaped && !status.ok()) { // Got an error
+ // !input_strings_escaped indicates the old API, where everything is
+ // parsable.
+ if (opt_info.IsByName()) {
+ status = Status::OK();
+ }
+ }
+ return status;
+}
+
+Status GetBlockBasedTableOptionsFromString(
+ const BlockBasedTableOptions& table_options, const std::string& opts_str,
+ BlockBasedTableOptions* new_table_options) {
+ ConfigOptions config_options;
+ config_options.input_strings_escaped = false;
+ config_options.ignore_unknown_options = false;
+ config_options.invoke_prepare_options = false;
+ config_options.ignore_unsupported_options = false;
+
+ return GetBlockBasedTableOptionsFromString(config_options, table_options,
+ opts_str, new_table_options);
+}
+Status GetBlockBasedTableOptionsFromString(
+ const ConfigOptions& config_options,
+ const BlockBasedTableOptions& table_options, const std::string& opts_str,
+ BlockBasedTableOptions* new_table_options) {
+ std::unordered_map<std::string, std::string> opts_map;
+ Status s = StringToMap(opts_str, &opts_map);
+ if (!s.ok()) {
+ return s;
+ }
+ s = GetBlockBasedTableOptionsFromMap(config_options, table_options, opts_map,
+ new_table_options);
+ // Translate any errors (NotFound, NotSupported, to InvalidArgument
+ if (s.ok() || s.IsInvalidArgument()) {
+ return s;
+ } else {
+ return Status::InvalidArgument(s.getState());
+ }
+}
+
+Status GetBlockBasedTableOptionsFromMap(
+ const BlockBasedTableOptions& table_options,
+ const std::unordered_map<std::string, std::string>& opts_map,
+ BlockBasedTableOptions* new_table_options, bool input_strings_escaped,
+ bool ignore_unknown_options) {
+ ConfigOptions config_options;
+ config_options.input_strings_escaped = input_strings_escaped;
+ config_options.ignore_unknown_options = ignore_unknown_options;
+ config_options.invoke_prepare_options = false;
+
+ return GetBlockBasedTableOptionsFromMap(config_options, table_options,
+ opts_map, new_table_options);
+}
+
+Status GetBlockBasedTableOptionsFromMap(
+ const ConfigOptions& config_options,
+ const BlockBasedTableOptions& table_options,
+ const std::unordered_map<std::string, std::string>& opts_map,
+ BlockBasedTableOptions* new_table_options) {
+ assert(new_table_options);
+ BlockBasedTableFactory bbtf(table_options);
+ Status s = bbtf.ConfigureFromMap(config_options, opts_map);
+ if (s.ok()) {
+ *new_table_options = *(bbtf.GetOptions<BlockBasedTableOptions>());
+ } else {
+ *new_table_options = table_options;
+ }
+ return s;
+}
+#endif // !ROCKSDB_LITE
+
+TableFactory* NewBlockBasedTableFactory(
+ const BlockBasedTableOptions& _table_options) {
+ return new BlockBasedTableFactory(_table_options);
+}
+
+const std::string BlockBasedTablePropertyNames::kIndexType =
+ "rocksdb.block.based.table.index.type";
+const std::string BlockBasedTablePropertyNames::kWholeKeyFiltering =
+ "rocksdb.block.based.table.whole.key.filtering";
+const std::string BlockBasedTablePropertyNames::kPrefixFiltering =
+ "rocksdb.block.based.table.prefix.filtering";
+const std::string kHashIndexPrefixesBlock = "rocksdb.hashindex.prefixes";
+const std::string kHashIndexPrefixesMetadataBlock =
+ "rocksdb.hashindex.metadata";
+const std::string kPropTrue = "1";
+const std::string kPropFalse = "0";
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/block_based_table_factory.h b/src/rocksdb/table/block_based/block_based_table_factory.h
new file mode 100644
index 000000000..3166cd3cc
--- /dev/null
+++ b/src/rocksdb/table/block_based/block_based_table_factory.h
@@ -0,0 +1,101 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#pragma once
+#include <stdint.h>
+
+#include <memory>
+#include <string>
+
+#include "cache/cache_reservation_manager.h"
+#include "port/port.h"
+#include "rocksdb/flush_block_policy.h"
+#include "rocksdb/table.h"
+
+namespace ROCKSDB_NAMESPACE {
+struct ColumnFamilyOptions;
+struct ConfigOptions;
+struct DBOptions;
+struct EnvOptions;
+
+class BlockBasedTableBuilder;
+class RandomAccessFileReader;
+class WritableFileWriter;
+
+// A class used to track actual bytes written from the tail in the recent SST
+// file opens, and provide a suggestion for following open.
+class TailPrefetchStats {
+ public:
+ void RecordEffectiveSize(size_t len);
+ // 0 indicates no information to determine.
+ size_t GetSuggestedPrefetchSize();
+
+ private:
+ const static size_t kNumTracked = 32;
+ size_t records_[kNumTracked];
+ port::Mutex mutex_;
+ size_t next_ = 0;
+ size_t num_records_ = 0;
+};
+
+class BlockBasedTableFactory : public TableFactory {
+ public:
+ explicit BlockBasedTableFactory(
+ const BlockBasedTableOptions& table_options = BlockBasedTableOptions());
+
+ ~BlockBasedTableFactory() {}
+
+ // Method to allow CheckedCast to work for this class
+ static const char* kClassName() { return kBlockBasedTableName(); }
+
+ const char* Name() const override { return kBlockBasedTableName(); }
+
+ using TableFactory::NewTableReader;
+ Status NewTableReader(
+ const ReadOptions& ro, const TableReaderOptions& table_reader_options,
+ std::unique_ptr<RandomAccessFileReader>&& file, uint64_t file_size,
+ std::unique_ptr<TableReader>* table_reader,
+ bool prefetch_index_and_filter_in_cache = true) const override;
+
+ TableBuilder* NewTableBuilder(
+ const TableBuilderOptions& table_builder_options,
+ WritableFileWriter* file) const override;
+
+ // Valdates the specified DB Options.
+ Status ValidateOptions(const DBOptions& db_opts,
+ const ColumnFamilyOptions& cf_opts) const override;
+ Status PrepareOptions(const ConfigOptions& opts) override;
+
+ std::string GetPrintableOptions() const override;
+
+ bool IsDeleteRangeSupported() const override { return true; }
+
+ TailPrefetchStats* tail_prefetch_stats() { return &tail_prefetch_stats_; }
+
+ protected:
+ const void* GetOptionsPtr(const std::string& name) const override;
+#ifndef ROCKSDB_LITE
+ Status ParseOption(const ConfigOptions& config_options,
+ const OptionTypeInfo& opt_info,
+ const std::string& opt_name, const std::string& opt_value,
+ void* opt_ptr) override;
+#endif
+ void InitializeOptions();
+
+ private:
+ BlockBasedTableOptions table_options_;
+ std::shared_ptr<CacheReservationManager> table_reader_cache_res_mgr_;
+ mutable TailPrefetchStats tail_prefetch_stats_;
+};
+
+extern const std::string kHashIndexPrefixesBlock;
+extern const std::string kHashIndexPrefixesMetadataBlock;
+extern const std::string kPropTrue;
+extern const std::string kPropFalse;
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/block_based_table_iterator.cc b/src/rocksdb/table/block_based/block_based_table_iterator.cc
new file mode 100644
index 000000000..d2605670f
--- /dev/null
+++ b/src/rocksdb/table/block_based/block_based_table_iterator.cc
@@ -0,0 +1,459 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+#include "table/block_based/block_based_table_iterator.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+void BlockBasedTableIterator::SeekToFirst() { SeekImpl(nullptr, false); }
+
+void BlockBasedTableIterator::Seek(const Slice& target) {
+ SeekImpl(&target, true);
+}
+
+void BlockBasedTableIterator::SeekImpl(const Slice* target,
+ bool async_prefetch) {
+ bool is_first_pass = true;
+ if (async_read_in_progress_) {
+ AsyncInitDataBlock(false);
+ is_first_pass = false;
+ }
+
+ is_out_of_bound_ = false;
+ is_at_first_key_from_index_ = false;
+ if (target && !CheckPrefixMayMatch(*target, IterDirection::kForward)) {
+ ResetDataIter();
+ return;
+ }
+
+ bool need_seek_index = true;
+ if (block_iter_points_to_real_block_ && block_iter_.Valid()) {
+ // Reseek.
+ prev_block_offset_ = index_iter_->value().handle.offset();
+
+ if (target) {
+ // We can avoid an index seek if:
+ // 1. The new seek key is larger than the current key
+ // 2. The new seek key is within the upper bound of the block
+ // Since we don't necessarily know the internal key for either
+ // the current key or the upper bound, we check user keys and
+ // exclude the equality case. Considering internal keys can
+ // improve for the boundary cases, but it would complicate the
+ // code.
+ if (user_comparator_.Compare(ExtractUserKey(*target),
+ block_iter_.user_key()) > 0 &&
+ user_comparator_.Compare(ExtractUserKey(*target),
+ index_iter_->user_key()) < 0) {
+ need_seek_index = false;
+ }
+ }
+ }
+
+ if (need_seek_index) {
+ if (target) {
+ index_iter_->Seek(*target);
+ } else {
+ index_iter_->SeekToFirst();
+ }
+
+ if (!index_iter_->Valid()) {
+ ResetDataIter();
+ return;
+ }
+ }
+
+ IndexValue v = index_iter_->value();
+ const bool same_block = block_iter_points_to_real_block_ &&
+ v.handle.offset() == prev_block_offset_;
+
+ if (!v.first_internal_key.empty() && !same_block &&
+ (!target || icomp_.Compare(*target, v.first_internal_key) <= 0) &&
+ allow_unprepared_value_) {
+ // Index contains the first key of the block, and it's >= target.
+ // We can defer reading the block.
+ is_at_first_key_from_index_ = true;
+ // ResetDataIter() will invalidate block_iter_. Thus, there is no need to
+ // call CheckDataBlockWithinUpperBound() to check for iterate_upper_bound
+ // as that will be done later when the data block is actually read.
+ ResetDataIter();
+ } else {
+ // Need to use the data block.
+ if (!same_block) {
+ if (read_options_.async_io && async_prefetch) {
+ if (is_first_pass) {
+ AsyncInitDataBlock(is_first_pass);
+ }
+ if (async_read_in_progress_) {
+ // Status::TryAgain indicates asynchronous request for retrieval of
+ // data blocks has been submitted. So it should return at this point
+ // and Seek should be called again to retrieve the requested block and
+ // execute the remaining code.
+ return;
+ }
+ } else {
+ InitDataBlock();
+ }
+ } else {
+ // When the user does a reseek, the iterate_upper_bound might have
+ // changed. CheckDataBlockWithinUpperBound() needs to be called
+ // explicitly if the reseek ends up in the same data block.
+ // If the reseek ends up in a different block, InitDataBlock() will do
+ // the iterator upper bound check.
+ CheckDataBlockWithinUpperBound();
+ }
+
+ if (target) {
+ block_iter_.Seek(*target);
+ } else {
+ block_iter_.SeekToFirst();
+ }
+ FindKeyForward();
+ }
+
+ CheckOutOfBound();
+
+ if (target) {
+ assert(!Valid() || icomp_.Compare(*target, key()) <= 0);
+ }
+}
+
+void BlockBasedTableIterator::SeekForPrev(const Slice& target) {
+ is_out_of_bound_ = false;
+ is_at_first_key_from_index_ = false;
+ // For now totally disable prefix seek in auto prefix mode because we don't
+ // have logic
+ if (!CheckPrefixMayMatch(target, IterDirection::kBackward)) {
+ ResetDataIter();
+ return;
+ }
+
+ SavePrevIndexValue();
+
+ // Call Seek() rather than SeekForPrev() in the index block, because the
+ // target data block will likely to contain the position for `target`, the
+ // same as Seek(), rather than than before.
+ // For example, if we have three data blocks, each containing two keys:
+ // [2, 4] [6, 8] [10, 12]
+ // (the keys in the index block would be [4, 8, 12])
+ // and the user calls SeekForPrev(7), we need to go to the second block,
+ // just like if they call Seek(7).
+ // The only case where the block is difference is when they seek to a position
+ // in the boundary. For example, if they SeekForPrev(5), we should go to the
+ // first block, rather than the second. However, we don't have the information
+ // to distinguish the two unless we read the second block. In this case, we'll
+ // end up with reading two blocks.
+ index_iter_->Seek(target);
+
+ if (!index_iter_->Valid()) {
+ auto seek_status = index_iter_->status();
+ // Check for IO error
+ if (!seek_status.IsNotFound() && !seek_status.ok()) {
+ ResetDataIter();
+ return;
+ }
+
+ // With prefix index, Seek() returns NotFound if the prefix doesn't exist
+ if (seek_status.IsNotFound()) {
+ // Any key less than the target is fine for prefix seek
+ ResetDataIter();
+ return;
+ } else {
+ index_iter_->SeekToLast();
+ }
+ // Check for IO error
+ if (!index_iter_->Valid()) {
+ ResetDataIter();
+ return;
+ }
+ }
+
+ InitDataBlock();
+
+ block_iter_.SeekForPrev(target);
+
+ FindKeyBackward();
+ CheckDataBlockWithinUpperBound();
+ assert(!block_iter_.Valid() ||
+ icomp_.Compare(target, block_iter_.key()) >= 0);
+}
+
+void BlockBasedTableIterator::SeekToLast() {
+ is_out_of_bound_ = false;
+ is_at_first_key_from_index_ = false;
+ SavePrevIndexValue();
+ index_iter_->SeekToLast();
+ if (!index_iter_->Valid()) {
+ ResetDataIter();
+ return;
+ }
+ InitDataBlock();
+ block_iter_.SeekToLast();
+ FindKeyBackward();
+ CheckDataBlockWithinUpperBound();
+}
+
+void BlockBasedTableIterator::Next() {
+ if (is_at_first_key_from_index_ && !MaterializeCurrentBlock()) {
+ return;
+ }
+ assert(block_iter_points_to_real_block_);
+ block_iter_.Next();
+ FindKeyForward();
+ CheckOutOfBound();
+}
+
+bool BlockBasedTableIterator::NextAndGetResult(IterateResult* result) {
+ Next();
+ bool is_valid = Valid();
+ if (is_valid) {
+ result->key = key();
+ result->bound_check_result = UpperBoundCheckResult();
+ result->value_prepared = !is_at_first_key_from_index_;
+ }
+ return is_valid;
+}
+
+void BlockBasedTableIterator::Prev() {
+ if (is_at_first_key_from_index_) {
+ is_at_first_key_from_index_ = false;
+
+ index_iter_->Prev();
+ if (!index_iter_->Valid()) {
+ return;
+ }
+
+ InitDataBlock();
+ block_iter_.SeekToLast();
+ } else {
+ assert(block_iter_points_to_real_block_);
+ block_iter_.Prev();
+ }
+
+ FindKeyBackward();
+}
+
+void BlockBasedTableIterator::InitDataBlock() {
+ BlockHandle data_block_handle = index_iter_->value().handle;
+ if (!block_iter_points_to_real_block_ ||
+ data_block_handle.offset() != prev_block_offset_ ||
+ // if previous attempt of reading the block missed cache, try again
+ block_iter_.status().IsIncomplete()) {
+ if (block_iter_points_to_real_block_) {
+ ResetDataIter();
+ }
+ auto* rep = table_->get_rep();
+
+ bool is_for_compaction =
+ lookup_context_.caller == TableReaderCaller::kCompaction;
+ // Prefetch additional data for range scans (iterators).
+ // Implicit auto readahead:
+ // Enabled after 2 sequential IOs when ReadOptions.readahead_size == 0.
+ // Explicit user requested readahead:
+ // Enabled from the very first IO when ReadOptions.readahead_size is set.
+ block_prefetcher_.PrefetchIfNeeded(
+ rep, data_block_handle, read_options_.readahead_size, is_for_compaction,
+ /*no_sequential_checking=*/false, read_options_.rate_limiter_priority);
+ Status s;
+ table_->NewDataBlockIterator<DataBlockIter>(
+ read_options_, data_block_handle, &block_iter_, BlockType::kData,
+ /*get_context=*/nullptr, &lookup_context_,
+ block_prefetcher_.prefetch_buffer(),
+ /*for_compaction=*/is_for_compaction, /*async_read=*/false, s);
+ block_iter_points_to_real_block_ = true;
+ CheckDataBlockWithinUpperBound();
+ }
+}
+
+void BlockBasedTableIterator::AsyncInitDataBlock(bool is_first_pass) {
+ BlockHandle data_block_handle = index_iter_->value().handle;
+ bool is_for_compaction =
+ lookup_context_.caller == TableReaderCaller::kCompaction;
+ if (is_first_pass) {
+ if (!block_iter_points_to_real_block_ ||
+ data_block_handle.offset() != prev_block_offset_ ||
+ // if previous attempt of reading the block missed cache, try again
+ block_iter_.status().IsIncomplete()) {
+ if (block_iter_points_to_real_block_) {
+ ResetDataIter();
+ }
+ auto* rep = table_->get_rep();
+ // Prefetch additional data for range scans (iterators).
+ // Implicit auto readahead:
+ // Enabled after 2 sequential IOs when ReadOptions.readahead_size == 0.
+ // Explicit user requested readahead:
+ // Enabled from the very first IO when ReadOptions.readahead_size is
+ // set.
+ // In case of async_io with Implicit readahead, block_prefetcher_ will
+ // always the create the prefetch buffer by setting no_sequential_checking
+ // = true.
+ block_prefetcher_.PrefetchIfNeeded(
+ rep, data_block_handle, read_options_.readahead_size,
+ is_for_compaction, /*no_sequential_checking=*/read_options_.async_io,
+ read_options_.rate_limiter_priority);
+
+ Status s;
+ table_->NewDataBlockIterator<DataBlockIter>(
+ read_options_, data_block_handle, &block_iter_, BlockType::kData,
+ /*get_context=*/nullptr, &lookup_context_,
+ block_prefetcher_.prefetch_buffer(),
+ /*for_compaction=*/is_for_compaction, /*async_read=*/true, s);
+
+ if (s.IsTryAgain()) {
+ async_read_in_progress_ = true;
+ return;
+ }
+ }
+ } else {
+ // Second pass will call the Poll to get the data block which has been
+ // requested asynchronously.
+ Status s;
+ table_->NewDataBlockIterator<DataBlockIter>(
+ read_options_, data_block_handle, &block_iter_, BlockType::kData,
+ /*get_context=*/nullptr, &lookup_context_,
+ block_prefetcher_.prefetch_buffer(),
+ /*for_compaction=*/is_for_compaction, /*async_read=*/false, s);
+ }
+ block_iter_points_to_real_block_ = true;
+ CheckDataBlockWithinUpperBound();
+ async_read_in_progress_ = false;
+}
+
+bool BlockBasedTableIterator::MaterializeCurrentBlock() {
+ assert(is_at_first_key_from_index_);
+ assert(!block_iter_points_to_real_block_);
+ assert(index_iter_->Valid());
+
+ is_at_first_key_from_index_ = false;
+ InitDataBlock();
+ assert(block_iter_points_to_real_block_);
+
+ if (!block_iter_.status().ok()) {
+ return false;
+ }
+
+ block_iter_.SeekToFirst();
+
+ if (!block_iter_.Valid() ||
+ icomp_.Compare(block_iter_.key(),
+ index_iter_->value().first_internal_key) != 0) {
+ block_iter_.Invalidate(Status::Corruption(
+ "first key in index doesn't match first key in block"));
+ return false;
+ }
+
+ return true;
+}
+
+void BlockBasedTableIterator::FindKeyForward() {
+ // This method's code is kept short to make it likely to be inlined.
+
+ assert(!is_out_of_bound_);
+ assert(block_iter_points_to_real_block_);
+
+ if (!block_iter_.Valid()) {
+ // This is the only call site of FindBlockForward(), but it's extracted into
+ // a separate method to keep FindKeyForward() short and likely to be
+ // inlined. When transitioning to a different block, we call
+ // FindBlockForward(), which is much longer and is probably not inlined.
+ FindBlockForward();
+ } else {
+ // This is the fast path that avoids a function call.
+ }
+}
+
+void BlockBasedTableIterator::FindBlockForward() {
+ // TODO the while loop inherits from two-level-iterator. We don't know
+ // whether a block can be empty so it can be replaced by an "if".
+ do {
+ if (!block_iter_.status().ok()) {
+ return;
+ }
+ // Whether next data block is out of upper bound, if there is one.
+ const bool next_block_is_out_of_bound =
+ read_options_.iterate_upper_bound != nullptr &&
+ block_iter_points_to_real_block_ &&
+ block_upper_bound_check_ == BlockUpperBound::kUpperBoundInCurBlock;
+ assert(!next_block_is_out_of_bound ||
+ user_comparator_.CompareWithoutTimestamp(
+ *read_options_.iterate_upper_bound, /*a_has_ts=*/false,
+ index_iter_->user_key(), /*b_has_ts=*/true) <= 0);
+ ResetDataIter();
+ index_iter_->Next();
+ if (next_block_is_out_of_bound) {
+ // The next block is out of bound. No need to read it.
+ TEST_SYNC_POINT_CALLBACK("BlockBasedTableIterator:out_of_bound", nullptr);
+ // We need to make sure this is not the last data block before setting
+ // is_out_of_bound_, since the index key for the last data block can be
+ // larger than smallest key of the next file on the same level.
+ if (index_iter_->Valid()) {
+ is_out_of_bound_ = true;
+ }
+ return;
+ }
+
+ if (!index_iter_->Valid()) {
+ return;
+ }
+
+ IndexValue v = index_iter_->value();
+
+ if (!v.first_internal_key.empty() && allow_unprepared_value_) {
+ // Index contains the first key of the block. Defer reading the block.
+ is_at_first_key_from_index_ = true;
+ return;
+ }
+
+ InitDataBlock();
+ block_iter_.SeekToFirst();
+ } while (!block_iter_.Valid());
+}
+
+void BlockBasedTableIterator::FindKeyBackward() {
+ while (!block_iter_.Valid()) {
+ if (!block_iter_.status().ok()) {
+ return;
+ }
+
+ ResetDataIter();
+ index_iter_->Prev();
+
+ if (index_iter_->Valid()) {
+ InitDataBlock();
+ block_iter_.SeekToLast();
+ } else {
+ return;
+ }
+ }
+
+ // We could have check lower bound here too, but we opt not to do it for
+ // code simplicity.
+}
+
+void BlockBasedTableIterator::CheckOutOfBound() {
+ if (read_options_.iterate_upper_bound != nullptr &&
+ block_upper_bound_check_ != BlockUpperBound::kUpperBoundBeyondCurBlock &&
+ Valid()) {
+ is_out_of_bound_ =
+ user_comparator_.CompareWithoutTimestamp(
+ *read_options_.iterate_upper_bound, /*a_has_ts=*/false, user_key(),
+ /*b_has_ts=*/true) <= 0;
+ }
+}
+
+void BlockBasedTableIterator::CheckDataBlockWithinUpperBound() {
+ if (read_options_.iterate_upper_bound != nullptr &&
+ block_iter_points_to_real_block_) {
+ block_upper_bound_check_ = (user_comparator_.CompareWithoutTimestamp(
+ *read_options_.iterate_upper_bound,
+ /*a_has_ts=*/false, index_iter_->user_key(),
+ /*b_has_ts=*/true) > 0)
+ ? BlockUpperBound::kUpperBoundBeyondCurBlock
+ : BlockUpperBound::kUpperBoundInCurBlock;
+ }
+}
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/block_based_table_iterator.h b/src/rocksdb/table/block_based/block_based_table_iterator.h
new file mode 100644
index 000000000..a2918b248
--- /dev/null
+++ b/src/rocksdb/table/block_based/block_based_table_iterator.h
@@ -0,0 +1,280 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+#pragma once
+#include "table/block_based/block_based_table_reader.h"
+#include "table/block_based/block_based_table_reader_impl.h"
+#include "table/block_based/block_prefetcher.h"
+#include "table/block_based/reader_common.h"
+
+namespace ROCKSDB_NAMESPACE {
+// Iterates over the contents of BlockBasedTable.
+class BlockBasedTableIterator : public InternalIteratorBase<Slice> {
+ // compaction_readahead_size: its value will only be used if for_compaction =
+ // true
+ // @param read_options Must outlive this iterator.
+ public:
+ BlockBasedTableIterator(
+ const BlockBasedTable* table, const ReadOptions& read_options,
+ const InternalKeyComparator& icomp,
+ std::unique_ptr<InternalIteratorBase<IndexValue>>&& index_iter,
+ bool check_filter, bool need_upper_bound_check,
+ const SliceTransform* prefix_extractor, TableReaderCaller caller,
+ size_t compaction_readahead_size = 0, bool allow_unprepared_value = false)
+ : index_iter_(std::move(index_iter)),
+ table_(table),
+ read_options_(read_options),
+ icomp_(icomp),
+ user_comparator_(icomp.user_comparator()),
+ pinned_iters_mgr_(nullptr),
+ prefix_extractor_(prefix_extractor),
+ lookup_context_(caller),
+ block_prefetcher_(
+ compaction_readahead_size,
+ table_->get_rep()->table_options.initial_auto_readahead_size),
+ allow_unprepared_value_(allow_unprepared_value),
+ block_iter_points_to_real_block_(false),
+ check_filter_(check_filter),
+ need_upper_bound_check_(need_upper_bound_check),
+ async_read_in_progress_(false) {}
+
+ ~BlockBasedTableIterator() {}
+
+ void Seek(const Slice& target) override;
+ void SeekForPrev(const Slice& target) override;
+ void SeekToFirst() override;
+ void SeekToLast() override;
+ void Next() final override;
+ bool NextAndGetResult(IterateResult* result) override;
+ void Prev() override;
+ bool Valid() const override {
+ return !is_out_of_bound_ &&
+ (is_at_first_key_from_index_ ||
+ (block_iter_points_to_real_block_ && block_iter_.Valid()));
+ }
+ Slice key() const override {
+ assert(Valid());
+ if (is_at_first_key_from_index_) {
+ return index_iter_->value().first_internal_key;
+ } else {
+ return block_iter_.key();
+ }
+ }
+ Slice user_key() const override {
+ assert(Valid());
+ if (is_at_first_key_from_index_) {
+ return ExtractUserKey(index_iter_->value().first_internal_key);
+ } else {
+ return block_iter_.user_key();
+ }
+ }
+ bool PrepareValue() override {
+ assert(Valid());
+
+ if (!is_at_first_key_from_index_) {
+ return true;
+ }
+
+ return const_cast<BlockBasedTableIterator*>(this)
+ ->MaterializeCurrentBlock();
+ }
+ Slice value() const override {
+ // PrepareValue() must have been called.
+ assert(!is_at_first_key_from_index_);
+ assert(Valid());
+
+ return block_iter_.value();
+ }
+ Status status() const override {
+ // Prefix index set status to NotFound when the prefix does not exist
+ if (!index_iter_->status().ok() && !index_iter_->status().IsNotFound()) {
+ return index_iter_->status();
+ } else if (block_iter_points_to_real_block_) {
+ return block_iter_.status();
+ } else if (async_read_in_progress_) {
+ return Status::TryAgain();
+ } else {
+ return Status::OK();
+ }
+ }
+
+ inline IterBoundCheck UpperBoundCheckResult() override {
+ if (is_out_of_bound_) {
+ return IterBoundCheck::kOutOfBound;
+ } else if (block_upper_bound_check_ ==
+ BlockUpperBound::kUpperBoundBeyondCurBlock) {
+ assert(!is_out_of_bound_);
+ return IterBoundCheck::kInbound;
+ } else {
+ return IterBoundCheck::kUnknown;
+ }
+ }
+
+ void SetPinnedItersMgr(PinnedIteratorsManager* pinned_iters_mgr) override {
+ pinned_iters_mgr_ = pinned_iters_mgr;
+ }
+ bool IsKeyPinned() const override {
+ // Our key comes either from block_iter_'s current key
+ // or index_iter_'s current *value*.
+ return pinned_iters_mgr_ && pinned_iters_mgr_->PinningEnabled() &&
+ ((is_at_first_key_from_index_ && index_iter_->IsValuePinned()) ||
+ (block_iter_points_to_real_block_ && block_iter_.IsKeyPinned()));
+ }
+ bool IsValuePinned() const override {
+ assert(!is_at_first_key_from_index_);
+ assert(Valid());
+
+ // BlockIter::IsValuePinned() is always true. No need to check
+ return pinned_iters_mgr_ && pinned_iters_mgr_->PinningEnabled() &&
+ block_iter_points_to_real_block_;
+ }
+
+ void ResetDataIter() {
+ if (block_iter_points_to_real_block_) {
+ if (pinned_iters_mgr_ != nullptr && pinned_iters_mgr_->PinningEnabled()) {
+ block_iter_.DelegateCleanupsTo(pinned_iters_mgr_);
+ }
+ block_iter_.Invalidate(Status::OK());
+ block_iter_points_to_real_block_ = false;
+ }
+ block_upper_bound_check_ = BlockUpperBound::kUnknown;
+ }
+
+ void SavePrevIndexValue() {
+ if (block_iter_points_to_real_block_) {
+ // Reseek. If they end up with the same data block, we shouldn't re-fetch
+ // the same data block.
+ prev_block_offset_ = index_iter_->value().handle.offset();
+ }
+ }
+
+ void GetReadaheadState(ReadaheadFileInfo* readahead_file_info) override {
+ if (block_prefetcher_.prefetch_buffer() != nullptr &&
+ read_options_.adaptive_readahead) {
+ block_prefetcher_.prefetch_buffer()->GetReadaheadState(
+ &(readahead_file_info->data_block_readahead_info));
+ if (index_iter_) {
+ index_iter_->GetReadaheadState(readahead_file_info);
+ }
+ }
+ }
+
+ void SetReadaheadState(ReadaheadFileInfo* readahead_file_info) override {
+ if (read_options_.adaptive_readahead) {
+ block_prefetcher_.SetReadaheadState(
+ &(readahead_file_info->data_block_readahead_info));
+ if (index_iter_) {
+ index_iter_->SetReadaheadState(readahead_file_info);
+ }
+ }
+ }
+
+ std::unique_ptr<InternalIteratorBase<IndexValue>> index_iter_;
+
+ private:
+ enum class IterDirection {
+ kForward,
+ kBackward,
+ };
+ // This enum indicates whether the upper bound falls into current block
+ // or beyond.
+ // +-------------+
+ // | cur block | <-- (1)
+ // +-------------+
+ // <-- (2)
+ // --- <boundary key> ---
+ // <-- (3)
+ // +-------------+
+ // | next block | <-- (4)
+ // ......
+ //
+ // When the block is smaller than <boundary key>, kUpperBoundInCurBlock
+ // is the value to use. The examples are (1) or (2) in the graph. It means
+ // all keys in the next block or beyond will be out of bound. Keys within
+ // the current block may or may not be out of bound.
+ // When the block is larger or equal to <boundary key>,
+ // kUpperBoundBeyondCurBlock is to be used. The examples are (3) and (4)
+ // in the graph. It means that all keys in the current block is within the
+ // upper bound and keys in the next block may or may not be within the uppder
+ // bound.
+ // If the boundary key hasn't been checked against the upper bound,
+ // kUnknown can be used.
+ enum class BlockUpperBound {
+ kUpperBoundInCurBlock,
+ kUpperBoundBeyondCurBlock,
+ kUnknown,
+ };
+
+ const BlockBasedTable* table_;
+ const ReadOptions& read_options_;
+ const InternalKeyComparator& icomp_;
+ UserComparatorWrapper user_comparator_;
+ PinnedIteratorsManager* pinned_iters_mgr_;
+ DataBlockIter block_iter_;
+ const SliceTransform* prefix_extractor_;
+ uint64_t prev_block_offset_ = std::numeric_limits<uint64_t>::max();
+ BlockCacheLookupContext lookup_context_;
+
+ BlockPrefetcher block_prefetcher_;
+
+ const bool allow_unprepared_value_;
+ // True if block_iter_ is initialized and points to the same block
+ // as index iterator.
+ bool block_iter_points_to_real_block_;
+ // See InternalIteratorBase::IsOutOfBound().
+ bool is_out_of_bound_ = false;
+ // How current data block's boundary key with the next block is compared with
+ // iterate upper bound.
+ BlockUpperBound block_upper_bound_check_ = BlockUpperBound::kUnknown;
+ // True if we're standing at the first key of a block, and we haven't loaded
+ // that block yet. A call to PrepareValue() will trigger loading the block.
+ bool is_at_first_key_from_index_ = false;
+ bool check_filter_;
+ // TODO(Zhongyi): pick a better name
+ bool need_upper_bound_check_;
+
+ bool async_read_in_progress_;
+
+ // If `target` is null, seek to first.
+ void SeekImpl(const Slice* target, bool async_prefetch);
+
+ void InitDataBlock();
+ void AsyncInitDataBlock(bool is_first_pass);
+ bool MaterializeCurrentBlock();
+ void FindKeyForward();
+ void FindBlockForward();
+ void FindKeyBackward();
+ void CheckOutOfBound();
+
+ // Check if data block is fully within iterate_upper_bound.
+ //
+ // Note MyRocks may update iterate bounds between seek. To workaround it,
+ // we need to check and update data_block_within_upper_bound_ accordingly.
+ void CheckDataBlockWithinUpperBound();
+
+ bool CheckPrefixMayMatch(const Slice& ikey, IterDirection direction) {
+ if (need_upper_bound_check_ && direction == IterDirection::kBackward) {
+ // Upper bound check isn't sufficient for backward direction to
+ // guarantee the same result as total order, so disable prefix
+ // check.
+ return true;
+ }
+ if (check_filter_ && !table_->PrefixRangeMayMatch(
+ ikey, read_options_, prefix_extractor_,
+ need_upper_bound_check_, &lookup_context_)) {
+ // TODO remember the iterator is invalidated because of prefix
+ // match. This can avoid the upper level file iterator to falsely
+ // believe the position is the end of the SST file and move to
+ // the first key of the next file.
+ ResetDataIter();
+ return false;
+ }
+ return true;
+ }
+};
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/block_based_table_reader.cc b/src/rocksdb/table/block_based/block_based_table_reader.cc
new file mode 100644
index 000000000..43962ba1d
--- /dev/null
+++ b/src/rocksdb/table/block_based/block_based_table_reader.cc
@@ -0,0 +1,3092 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+#include "table/block_based/block_based_table_reader.h"
+
+#include <algorithm>
+#include <array>
+#include <atomic>
+#include <cstdint>
+#include <limits>
+#include <memory>
+#include <string>
+#include <unordered_set>
+#include <utility>
+#include <vector>
+
+#include "cache/cache_entry_roles.h"
+#include "cache/cache_key.h"
+#include "db/compaction/compaction_picker.h"
+#include "db/dbformat.h"
+#include "db/pinned_iterators_manager.h"
+#include "file/file_prefetch_buffer.h"
+#include "file/file_util.h"
+#include "file/random_access_file_reader.h"
+#include "logging/logging.h"
+#include "monitoring/perf_context_imp.h"
+#include "port/lang.h"
+#include "rocksdb/cache.h"
+#include "rocksdb/comparator.h"
+#include "rocksdb/convenience.h"
+#include "rocksdb/env.h"
+#include "rocksdb/file_system.h"
+#include "rocksdb/filter_policy.h"
+#include "rocksdb/iterator.h"
+#include "rocksdb/options.h"
+#include "rocksdb/snapshot.h"
+#include "rocksdb/statistics.h"
+#include "rocksdb/system_clock.h"
+#include "rocksdb/table.h"
+#include "rocksdb/table_properties.h"
+#include "rocksdb/trace_record.h"
+#include "table/block_based/binary_search_index_reader.h"
+#include "table/block_based/block.h"
+#include "table/block_based/block_based_table_factory.h"
+#include "table/block_based/block_based_table_iterator.h"
+#include "table/block_based/block_like_traits.h"
+#include "table/block_based/block_prefix_index.h"
+#include "table/block_based/block_type.h"
+#include "table/block_based/filter_block.h"
+#include "table/block_based/filter_policy_internal.h"
+#include "table/block_based/full_filter_block.h"
+#include "table/block_based/hash_index_reader.h"
+#include "table/block_based/partitioned_filter_block.h"
+#include "table/block_based/partitioned_index_reader.h"
+#include "table/block_fetcher.h"
+#include "table/format.h"
+#include "table/get_context.h"
+#include "table/internal_iterator.h"
+#include "table/meta_blocks.h"
+#include "table/multiget_context.h"
+#include "table/persistent_cache_helper.h"
+#include "table/persistent_cache_options.h"
+#include "table/sst_file_writer_collectors.h"
+#include "table/two_level_iterator.h"
+#include "test_util/sync_point.h"
+#include "util/coding.h"
+#include "util/crc32c.h"
+#include "util/stop_watch.h"
+#include "util/string_util.h"
+
+namespace ROCKSDB_NAMESPACE {
+namespace {
+
+CacheAllocationPtr CopyBufferToHeap(MemoryAllocator* allocator, Slice& buf) {
+ CacheAllocationPtr heap_buf;
+ heap_buf = AllocateBlock(buf.size(), allocator);
+ memcpy(heap_buf.get(), buf.data(), buf.size());
+ return heap_buf;
+}
+} // namespace
+} // namespace ROCKSDB_NAMESPACE
+
+// Generate the regular and coroutine versions of some methods by
+// including block_based_table_reader_sync_and_async.h twice
+// Macros in the header will expand differently based on whether
+// WITH_COROUTINES or WITHOUT_COROUTINES is defined
+// clang-format off
+#define WITHOUT_COROUTINES
+#include "table/block_based/block_based_table_reader_sync_and_async.h"
+#undef WITHOUT_COROUTINES
+#define WITH_COROUTINES
+#include "table/block_based/block_based_table_reader_sync_and_async.h"
+#undef WITH_COROUTINES
+// clang-format on
+
+namespace ROCKSDB_NAMESPACE {
+
+extern const uint64_t kBlockBasedTableMagicNumber;
+extern const std::string kHashIndexPrefixesBlock;
+extern const std::string kHashIndexPrefixesMetadataBlock;
+
+BlockBasedTable::~BlockBasedTable() { delete rep_; }
+
+namespace {
+// Read the block identified by "handle" from "file".
+// The only relevant option is options.verify_checksums for now.
+// On failure return non-OK.
+// On success fill *result and return OK - caller owns *result
+// @param uncompression_dict Data for presetting the compression library's
+// dictionary.
+template <typename TBlocklike>
+Status ReadBlockFromFile(
+ RandomAccessFileReader* file, FilePrefetchBuffer* prefetch_buffer,
+ const Footer& footer, const ReadOptions& options, const BlockHandle& handle,
+ std::unique_ptr<TBlocklike>* result, const ImmutableOptions& ioptions,
+ bool do_uncompress, bool maybe_compressed, BlockType block_type,
+ const UncompressionDict& uncompression_dict,
+ const PersistentCacheOptions& cache_options, size_t read_amp_bytes_per_bit,
+ MemoryAllocator* memory_allocator, bool for_compaction, bool using_zstd,
+ const FilterPolicy* filter_policy, bool async_read) {
+ assert(result);
+
+ BlockContents contents;
+ BlockFetcher block_fetcher(
+ file, prefetch_buffer, footer, options, handle, &contents, ioptions,
+ do_uncompress, maybe_compressed, block_type, uncompression_dict,
+ cache_options, memory_allocator, nullptr, for_compaction);
+ Status s;
+ // If prefetch_buffer is not allocated, it will fallback to synchronous
+ // reading of block contents.
+ if (async_read && prefetch_buffer != nullptr) {
+ s = block_fetcher.ReadAsyncBlockContents();
+ if (!s.ok()) {
+ return s;
+ }
+ } else {
+ s = block_fetcher.ReadBlockContents();
+ }
+ if (s.ok()) {
+ result->reset(BlocklikeTraits<TBlocklike>::Create(
+ std::move(contents), read_amp_bytes_per_bit, ioptions.stats, using_zstd,
+ filter_policy));
+ }
+
+ return s;
+}
+
+// For hash based index, return false if table_properties->prefix_extractor_name
+// and prefix_extractor both exist and match, otherwise true.
+inline bool PrefixExtractorChangedHelper(
+ const TableProperties* table_properties,
+ const SliceTransform* prefix_extractor) {
+ // BlockBasedTableOptions::kHashSearch requires prefix_extractor to be set.
+ // Turn off hash index in prefix_extractor is not set; if prefix_extractor
+ // is set but prefix_extractor_block is not set, also disable hash index
+ if (prefix_extractor == nullptr || table_properties == nullptr ||
+ table_properties->prefix_extractor_name.empty()) {
+ return true;
+ }
+
+ // prefix_extractor and prefix_extractor_block are both non-empty
+ if (table_properties->prefix_extractor_name != prefix_extractor->AsString()) {
+ return true;
+ } else {
+ return false;
+ }
+}
+
+} // namespace
+
+void BlockBasedTable::UpdateCacheHitMetrics(BlockType block_type,
+ GetContext* get_context,
+ size_t usage) const {
+ Statistics* const statistics = rep_->ioptions.stats;
+
+ PERF_COUNTER_ADD(block_cache_hit_count, 1);
+ PERF_COUNTER_BY_LEVEL_ADD(block_cache_hit_count, 1,
+ static_cast<uint32_t>(rep_->level));
+
+ if (get_context) {
+ ++get_context->get_context_stats_.num_cache_hit;
+ get_context->get_context_stats_.num_cache_bytes_read += usage;
+ } else {
+ RecordTick(statistics, BLOCK_CACHE_HIT);
+ RecordTick(statistics, BLOCK_CACHE_BYTES_READ, usage);
+ }
+
+ switch (block_type) {
+ case BlockType::kFilter:
+ case BlockType::kFilterPartitionIndex:
+ PERF_COUNTER_ADD(block_cache_filter_hit_count, 1);
+
+ if (get_context) {
+ ++get_context->get_context_stats_.num_cache_filter_hit;
+ } else {
+ RecordTick(statistics, BLOCK_CACHE_FILTER_HIT);
+ }
+ break;
+
+ case BlockType::kCompressionDictionary:
+ // TODO: introduce perf counter for compression dictionary hit count
+ if (get_context) {
+ ++get_context->get_context_stats_.num_cache_compression_dict_hit;
+ } else {
+ RecordTick(statistics, BLOCK_CACHE_COMPRESSION_DICT_HIT);
+ }
+ break;
+
+ case BlockType::kIndex:
+ PERF_COUNTER_ADD(block_cache_index_hit_count, 1);
+
+ if (get_context) {
+ ++get_context->get_context_stats_.num_cache_index_hit;
+ } else {
+ RecordTick(statistics, BLOCK_CACHE_INDEX_HIT);
+ }
+ break;
+
+ default:
+ // TODO: introduce dedicated tickers/statistics/counters
+ // for range tombstones
+ if (get_context) {
+ ++get_context->get_context_stats_.num_cache_data_hit;
+ } else {
+ RecordTick(statistics, BLOCK_CACHE_DATA_HIT);
+ }
+ break;
+ }
+}
+
+void BlockBasedTable::UpdateCacheMissMetrics(BlockType block_type,
+ GetContext* get_context) const {
+ Statistics* const statistics = rep_->ioptions.stats;
+
+ // TODO: introduce aggregate (not per-level) block cache miss count
+ PERF_COUNTER_BY_LEVEL_ADD(block_cache_miss_count, 1,
+ static_cast<uint32_t>(rep_->level));
+
+ if (get_context) {
+ ++get_context->get_context_stats_.num_cache_miss;
+ } else {
+ RecordTick(statistics, BLOCK_CACHE_MISS);
+ }
+
+ // TODO: introduce perf counters for misses per block type
+ switch (block_type) {
+ case BlockType::kFilter:
+ case BlockType::kFilterPartitionIndex:
+ if (get_context) {
+ ++get_context->get_context_stats_.num_cache_filter_miss;
+ } else {
+ RecordTick(statistics, BLOCK_CACHE_FILTER_MISS);
+ }
+ break;
+
+ case BlockType::kCompressionDictionary:
+ if (get_context) {
+ ++get_context->get_context_stats_.num_cache_compression_dict_miss;
+ } else {
+ RecordTick(statistics, BLOCK_CACHE_COMPRESSION_DICT_MISS);
+ }
+ break;
+
+ case BlockType::kIndex:
+ if (get_context) {
+ ++get_context->get_context_stats_.num_cache_index_miss;
+ } else {
+ RecordTick(statistics, BLOCK_CACHE_INDEX_MISS);
+ }
+ break;
+
+ default:
+ // TODO: introduce dedicated tickers/statistics/counters
+ // for range tombstones
+ if (get_context) {
+ ++get_context->get_context_stats_.num_cache_data_miss;
+ } else {
+ RecordTick(statistics, BLOCK_CACHE_DATA_MISS);
+ }
+ break;
+ }
+}
+
+void BlockBasedTable::UpdateCacheInsertionMetrics(
+ BlockType block_type, GetContext* get_context, size_t usage, bool redundant,
+ Statistics* const statistics) {
+ // TODO: introduce perf counters for block cache insertions
+ if (get_context) {
+ ++get_context->get_context_stats_.num_cache_add;
+ if (redundant) {
+ ++get_context->get_context_stats_.num_cache_add_redundant;
+ }
+ get_context->get_context_stats_.num_cache_bytes_write += usage;
+ } else {
+ RecordTick(statistics, BLOCK_CACHE_ADD);
+ if (redundant) {
+ RecordTick(statistics, BLOCK_CACHE_ADD_REDUNDANT);
+ }
+ RecordTick(statistics, BLOCK_CACHE_BYTES_WRITE, usage);
+ }
+
+ switch (block_type) {
+ case BlockType::kFilter:
+ case BlockType::kFilterPartitionIndex:
+ if (get_context) {
+ ++get_context->get_context_stats_.num_cache_filter_add;
+ if (redundant) {
+ ++get_context->get_context_stats_.num_cache_filter_add_redundant;
+ }
+ get_context->get_context_stats_.num_cache_filter_bytes_insert += usage;
+ } else {
+ RecordTick(statistics, BLOCK_CACHE_FILTER_ADD);
+ if (redundant) {
+ RecordTick(statistics, BLOCK_CACHE_FILTER_ADD_REDUNDANT);
+ }
+ RecordTick(statistics, BLOCK_CACHE_FILTER_BYTES_INSERT, usage);
+ }
+ break;
+
+ case BlockType::kCompressionDictionary:
+ if (get_context) {
+ ++get_context->get_context_stats_.num_cache_compression_dict_add;
+ if (redundant) {
+ ++get_context->get_context_stats_
+ .num_cache_compression_dict_add_redundant;
+ }
+ get_context->get_context_stats_
+ .num_cache_compression_dict_bytes_insert += usage;
+ } else {
+ RecordTick(statistics, BLOCK_CACHE_COMPRESSION_DICT_ADD);
+ if (redundant) {
+ RecordTick(statistics, BLOCK_CACHE_COMPRESSION_DICT_ADD_REDUNDANT);
+ }
+ RecordTick(statistics, BLOCK_CACHE_COMPRESSION_DICT_BYTES_INSERT,
+ usage);
+ }
+ break;
+
+ case BlockType::kIndex:
+ if (get_context) {
+ ++get_context->get_context_stats_.num_cache_index_add;
+ if (redundant) {
+ ++get_context->get_context_stats_.num_cache_index_add_redundant;
+ }
+ get_context->get_context_stats_.num_cache_index_bytes_insert += usage;
+ } else {
+ RecordTick(statistics, BLOCK_CACHE_INDEX_ADD);
+ if (redundant) {
+ RecordTick(statistics, BLOCK_CACHE_INDEX_ADD_REDUNDANT);
+ }
+ RecordTick(statistics, BLOCK_CACHE_INDEX_BYTES_INSERT, usage);
+ }
+ break;
+
+ default:
+ // TODO: introduce dedicated tickers/statistics/counters
+ // for range tombstones
+ if (get_context) {
+ ++get_context->get_context_stats_.num_cache_data_add;
+ if (redundant) {
+ ++get_context->get_context_stats_.num_cache_data_add_redundant;
+ }
+ get_context->get_context_stats_.num_cache_data_bytes_insert += usage;
+ } else {
+ RecordTick(statistics, BLOCK_CACHE_DATA_ADD);
+ if (redundant) {
+ RecordTick(statistics, BLOCK_CACHE_DATA_ADD_REDUNDANT);
+ }
+ RecordTick(statistics, BLOCK_CACHE_DATA_BYTES_INSERT, usage);
+ }
+ break;
+ }
+}
+
+Cache::Handle* BlockBasedTable::GetEntryFromCache(
+ const CacheTier& cache_tier, Cache* block_cache, const Slice& key,
+ BlockType block_type, const bool wait, GetContext* get_context,
+ const Cache::CacheItemHelper* cache_helper,
+ const Cache::CreateCallback& create_cb, Cache::Priority priority) const {
+ Cache::Handle* cache_handle = nullptr;
+ if (cache_tier == CacheTier::kNonVolatileBlockTier) {
+ cache_handle = block_cache->Lookup(key, cache_helper, create_cb, priority,
+ wait, rep_->ioptions.statistics.get());
+ } else {
+ cache_handle = block_cache->Lookup(key, rep_->ioptions.statistics.get());
+ }
+
+ // Avoid updating metrics here if the handle is not complete yet. This
+ // happens with MultiGet and secondary cache. So update the metrics only
+ // if its a miss, or a hit and value is ready
+ if (!cache_handle || block_cache->Value(cache_handle)) {
+ if (cache_handle != nullptr) {
+ UpdateCacheHitMetrics(block_type, get_context,
+ block_cache->GetUsage(cache_handle));
+ } else {
+ UpdateCacheMissMetrics(block_type, get_context);
+ }
+ }
+
+ return cache_handle;
+}
+
+template <typename TBlocklike>
+Status BlockBasedTable::InsertEntryToCache(
+ const CacheTier& cache_tier, Cache* block_cache, const Slice& key,
+ const Cache::CacheItemHelper* cache_helper,
+ std::unique_ptr<TBlocklike>&& block_holder, size_t charge,
+ Cache::Handle** cache_handle, Cache::Priority priority) const {
+ Status s = Status::OK();
+ if (cache_tier == CacheTier::kNonVolatileBlockTier) {
+ s = block_cache->Insert(key, block_holder.get(), cache_helper, charge,
+ cache_handle, priority);
+ } else {
+ s = block_cache->Insert(key, block_holder.get(), charge,
+ cache_helper->del_cb, cache_handle, priority);
+ }
+ if (s.ok()) {
+ // Cache took ownership
+ block_holder.release();
+ }
+ s.MustCheck();
+ return s;
+}
+
+namespace {
+// Return True if table_properties has `user_prop_name` has a `true` value
+// or it doesn't contain this property (for backward compatible).
+bool IsFeatureSupported(const TableProperties& table_properties,
+ const std::string& user_prop_name, Logger* info_log) {
+ auto& props = table_properties.user_collected_properties;
+ auto pos = props.find(user_prop_name);
+ // Older version doesn't have this value set. Skip this check.
+ if (pos != props.end()) {
+ if (pos->second == kPropFalse) {
+ return false;
+ } else if (pos->second != kPropTrue) {
+ ROCKS_LOG_WARN(info_log, "Property %s has invalidate value %s",
+ user_prop_name.c_str(), pos->second.c_str());
+ }
+ }
+ return true;
+}
+
+// Caller has to ensure seqno is not nullptr.
+Status GetGlobalSequenceNumber(const TableProperties& table_properties,
+ SequenceNumber largest_seqno,
+ SequenceNumber* seqno) {
+ const auto& props = table_properties.user_collected_properties;
+ const auto version_pos = props.find(ExternalSstFilePropertyNames::kVersion);
+ const auto seqno_pos = props.find(ExternalSstFilePropertyNames::kGlobalSeqno);
+
+ *seqno = kDisableGlobalSequenceNumber;
+ if (version_pos == props.end()) {
+ if (seqno_pos != props.end()) {
+ std::array<char, 200> msg_buf;
+ // This is not an external sst file, global_seqno is not supported.
+ snprintf(
+ msg_buf.data(), msg_buf.max_size(),
+ "A non-external sst file have global seqno property with value %s",
+ seqno_pos->second.c_str());
+ return Status::Corruption(msg_buf.data());
+ }
+ return Status::OK();
+ }
+
+ uint32_t version = DecodeFixed32(version_pos->second.c_str());
+ if (version < 2) {
+ if (seqno_pos != props.end() || version != 1) {
+ std::array<char, 200> msg_buf;
+ // This is a v1 external sst file, global_seqno is not supported.
+ snprintf(msg_buf.data(), msg_buf.max_size(),
+ "An external sst file with version %u have global seqno "
+ "property with value %s",
+ version, seqno_pos->second.c_str());
+ return Status::Corruption(msg_buf.data());
+ }
+ return Status::OK();
+ }
+
+ // Since we have a plan to deprecate global_seqno, we do not return failure
+ // if seqno_pos == props.end(). We rely on version_pos to detect whether the
+ // SST is external.
+ SequenceNumber global_seqno(0);
+ if (seqno_pos != props.end()) {
+ global_seqno = DecodeFixed64(seqno_pos->second.c_str());
+ }
+ // SstTableReader open table reader with kMaxSequenceNumber as largest_seqno
+ // to denote it is unknown.
+ if (largest_seqno < kMaxSequenceNumber) {
+ if (global_seqno == 0) {
+ global_seqno = largest_seqno;
+ }
+ if (global_seqno != largest_seqno) {
+ std::array<char, 200> msg_buf;
+ snprintf(
+ msg_buf.data(), msg_buf.max_size(),
+ "An external sst file with version %u have global seqno property "
+ "with value %s, while largest seqno in the file is %llu",
+ version, seqno_pos->second.c_str(),
+ static_cast<unsigned long long>(largest_seqno));
+ return Status::Corruption(msg_buf.data());
+ }
+ }
+ *seqno = global_seqno;
+
+ if (global_seqno > kMaxSequenceNumber) {
+ std::array<char, 200> msg_buf;
+ snprintf(msg_buf.data(), msg_buf.max_size(),
+ "An external sst file with version %u have global seqno property "
+ "with value %llu, which is greater than kMaxSequenceNumber",
+ version, static_cast<unsigned long long>(global_seqno));
+ return Status::Corruption(msg_buf.data());
+ }
+
+ return Status::OK();
+}
+} // namespace
+
+void BlockBasedTable::SetupBaseCacheKey(const TableProperties* properties,
+ const std::string& cur_db_session_id,
+ uint64_t cur_file_number,
+ OffsetableCacheKey* out_base_cache_key,
+ bool* out_is_stable) {
+ // Use a stable cache key if sufficient data is in table properties
+ std::string db_session_id;
+ uint64_t file_num;
+ std::string db_id;
+ if (properties && !properties->db_session_id.empty() &&
+ properties->orig_file_number > 0) {
+ // (Newer SST file case)
+ // We must have both properties to get a stable unique id because
+ // CreateColumnFamilyWithImport or IngestExternalFiles can change the
+ // file numbers on a file.
+ db_session_id = properties->db_session_id;
+ file_num = properties->orig_file_number;
+ // Less critical, populated in earlier release than above
+ db_id = properties->db_id;
+ if (out_is_stable) {
+ *out_is_stable = true;
+ }
+ } else {
+ // (Old SST file case)
+ // We use (unique) cache keys based on current identifiers. These are at
+ // least stable across table file close and re-open, but not across
+ // different DBs nor DB close and re-open.
+ db_session_id = cur_db_session_id;
+ file_num = cur_file_number;
+ // Plumbing through the DB ID to here would be annoying, and of limited
+ // value because of the case of VersionSet::Recover opening some table
+ // files and later setting the DB ID. So we just rely on uniqueness
+ // level provided by session ID.
+ db_id = "unknown";
+ if (out_is_stable) {
+ *out_is_stable = false;
+ }
+ }
+
+ // Too many tests to update to get these working
+ // assert(file_num > 0);
+ // assert(!db_session_id.empty());
+ // assert(!db_id.empty());
+
+ // Minimum block size is 5 bytes; therefore we can trim off two lower bits
+ // from offsets. See GetCacheKey.
+ *out_base_cache_key = OffsetableCacheKey(db_id, db_session_id, file_num);
+}
+
+CacheKey BlockBasedTable::GetCacheKey(const OffsetableCacheKey& base_cache_key,
+ const BlockHandle& handle) {
+ // Minimum block size is 5 bytes; therefore we can trim off two lower bits
+ // from offet.
+ return base_cache_key.WithOffset(handle.offset() >> 2);
+}
+
+Status BlockBasedTable::Open(
+ const ReadOptions& read_options, const ImmutableOptions& ioptions,
+ const EnvOptions& env_options, const BlockBasedTableOptions& table_options,
+ const InternalKeyComparator& internal_comparator,
+ std::unique_ptr<RandomAccessFileReader>&& file, uint64_t file_size,
+ std::unique_ptr<TableReader>* table_reader,
+ std::shared_ptr<CacheReservationManager> table_reader_cache_res_mgr,
+ const std::shared_ptr<const SliceTransform>& prefix_extractor,
+ const bool prefetch_index_and_filter_in_cache, const bool skip_filters,
+ const int level, const bool immortal_table,
+ const SequenceNumber largest_seqno, const bool force_direct_prefetch,
+ TailPrefetchStats* tail_prefetch_stats,
+ BlockCacheTracer* const block_cache_tracer,
+ size_t max_file_size_for_l0_meta_pin, const std::string& cur_db_session_id,
+ uint64_t cur_file_num, UniqueId64x2 expected_unique_id) {
+ table_reader->reset();
+
+ Status s;
+ Footer footer;
+ std::unique_ptr<FilePrefetchBuffer> prefetch_buffer;
+
+ // From read_options, retain deadline, io_timeout, and rate_limiter_priority.
+ // In future, we may retain more
+ // options. Specifically, we ignore verify_checksums and default to
+ // checksum verification anyway when creating the index and filter
+ // readers.
+ ReadOptions ro;
+ ro.deadline = read_options.deadline;
+ ro.io_timeout = read_options.io_timeout;
+ ro.rate_limiter_priority = read_options.rate_limiter_priority;
+
+ // prefetch both index and filters, down to all partitions
+ const bool prefetch_all = prefetch_index_and_filter_in_cache || level == 0;
+ const bool preload_all = !table_options.cache_index_and_filter_blocks;
+
+ if (!ioptions.allow_mmap_reads) {
+ s = PrefetchTail(ro, file.get(), file_size, force_direct_prefetch,
+ tail_prefetch_stats, prefetch_all, preload_all,
+ &prefetch_buffer);
+ // Return error in prefetch path to users.
+ if (!s.ok()) {
+ return s;
+ }
+ } else {
+ // Should not prefetch for mmap mode.
+ prefetch_buffer.reset(new FilePrefetchBuffer(
+ 0 /* readahead_size */, 0 /* max_readahead_size */, false /* enable */,
+ true /* track_min_offset */));
+ }
+
+ // Read in the following order:
+ // 1. Footer
+ // 2. [metaindex block]
+ // 3. [meta block: properties]
+ // 4. [meta block: range deletion tombstone]
+ // 5. [meta block: compression dictionary]
+ // 6. [meta block: index]
+ // 7. [meta block: filter]
+ IOOptions opts;
+ s = file->PrepareIOOptions(ro, opts);
+ if (s.ok()) {
+ s = ReadFooterFromFile(opts, file.get(), prefetch_buffer.get(), file_size,
+ &footer, kBlockBasedTableMagicNumber);
+ }
+ if (!s.ok()) {
+ return s;
+ }
+ if (!IsSupportedFormatVersion(footer.format_version())) {
+ return Status::Corruption(
+ "Unknown Footer version. Maybe this file was created with newer "
+ "version of RocksDB?");
+ }
+
+ BlockCacheLookupContext lookup_context{TableReaderCaller::kPrefetch};
+ Rep* rep = new BlockBasedTable::Rep(ioptions, env_options, table_options,
+ internal_comparator, skip_filters,
+ file_size, level, immortal_table);
+ rep->file = std::move(file);
+ rep->footer = footer;
+
+ // For fully portable/stable cache keys, we need to read the properties
+ // block before setting up cache keys. TODO: consider setting up a bootstrap
+ // cache key for PersistentCache to use for metaindex and properties blocks.
+ rep->persistent_cache_options = PersistentCacheOptions();
+
+ // Meta-blocks are not dictionary compressed. Explicitly set the dictionary
+ // handle to null, otherwise it may be seen as uninitialized during the below
+ // meta-block reads.
+ rep->compression_dict_handle = BlockHandle::NullBlockHandle();
+
+ // Read metaindex
+ std::unique_ptr<BlockBasedTable> new_table(
+ new BlockBasedTable(rep, block_cache_tracer));
+ std::unique_ptr<Block> metaindex;
+ std::unique_ptr<InternalIterator> metaindex_iter;
+ s = new_table->ReadMetaIndexBlock(ro, prefetch_buffer.get(), &metaindex,
+ &metaindex_iter);
+ if (!s.ok()) {
+ return s;
+ }
+
+ // Populates table_properties and some fields that depend on it,
+ // such as index_type.
+ s = new_table->ReadPropertiesBlock(ro, prefetch_buffer.get(),
+ metaindex_iter.get(), largest_seqno);
+ if (!s.ok()) {
+ return s;
+ }
+
+ // Check expected unique id if provided
+ if (expected_unique_id != kNullUniqueId64x2) {
+ auto props = rep->table_properties;
+ if (!props) {
+ return Status::Corruption("Missing table properties on file " +
+ std::to_string(cur_file_num) +
+ " with known unique ID");
+ }
+ UniqueId64x2 actual_unique_id{};
+ s = GetSstInternalUniqueId(props->db_id, props->db_session_id,
+ props->orig_file_number, &actual_unique_id,
+ /*force*/ true);
+ assert(s.ok()); // because force=true
+ if (expected_unique_id != actual_unique_id) {
+ return Status::Corruption(
+ "Mismatch in unique ID on table file " +
+ std::to_string(cur_file_num) +
+ ". Expected: " + InternalUniqueIdToHumanString(&expected_unique_id) +
+ " Actual: " + InternalUniqueIdToHumanString(&actual_unique_id));
+ }
+ TEST_SYNC_POINT_CALLBACK("BlockBasedTable::Open::PassedVerifyUniqueId",
+ &actual_unique_id);
+ } else {
+ TEST_SYNC_POINT_CALLBACK("BlockBasedTable::Open::SkippedVerifyUniqueId",
+ nullptr);
+ if (ioptions.verify_sst_unique_id_in_manifest && ioptions.logger) {
+ // A crude but isolated way of reporting unverified files. This should not
+ // be an ongoing concern so doesn't deserve a place in Statistics IMHO.
+ static std::atomic<uint64_t> unverified_count{0};
+ auto prev_count =
+ unverified_count.fetch_add(1, std::memory_order_relaxed);
+ if (prev_count == 0) {
+ ROCKS_LOG_WARN(
+ ioptions.logger,
+ "At least one SST file opened without unique ID to verify: %" PRIu64
+ ".sst",
+ cur_file_num);
+ } else if (prev_count % 1000 == 0) {
+ ROCKS_LOG_WARN(
+ ioptions.logger,
+ "Another ~1000 SST files opened without unique ID to verify");
+ }
+ }
+ }
+
+ // Set up prefix extracto as needed
+ bool force_null_table_prefix_extractor = false;
+ TEST_SYNC_POINT_CALLBACK(
+ "BlockBasedTable::Open::ForceNullTablePrefixExtractor",
+ &force_null_table_prefix_extractor);
+ if (force_null_table_prefix_extractor) {
+ assert(!rep->table_prefix_extractor);
+ } else if (!PrefixExtractorChangedHelper(rep->table_properties.get(),
+ prefix_extractor.get())) {
+ // Establish fast path for unchanged prefix_extractor
+ rep->table_prefix_extractor = prefix_extractor;
+ } else {
+ // Current prefix_extractor doesn't match table
+#ifndef ROCKSDB_LITE
+ if (rep->table_properties) {
+ //**TODO: If/When the DBOptions has a registry in it, the ConfigOptions
+ // will need to use it
+ ConfigOptions config_options;
+ Status st = SliceTransform::CreateFromString(
+ config_options, rep->table_properties->prefix_extractor_name,
+ &(rep->table_prefix_extractor));
+ if (!st.ok()) {
+ //**TODO: Should this be error be returned or swallowed?
+ ROCKS_LOG_ERROR(rep->ioptions.logger,
+ "Failed to create prefix extractor[%s]: %s",
+ rep->table_properties->prefix_extractor_name.c_str(),
+ st.ToString().c_str());
+ }
+ }
+#endif // ROCKSDB_LITE
+ }
+
+ // With properties loaded, we can set up portable/stable cache keys
+ SetupBaseCacheKey(rep->table_properties.get(), cur_db_session_id,
+ cur_file_num, &rep->base_cache_key);
+
+ rep->persistent_cache_options =
+ PersistentCacheOptions(rep->table_options.persistent_cache,
+ rep->base_cache_key, rep->ioptions.stats);
+
+ s = new_table->ReadRangeDelBlock(ro, prefetch_buffer.get(),
+ metaindex_iter.get(), internal_comparator,
+ &lookup_context);
+ if (!s.ok()) {
+ return s;
+ }
+ s = new_table->PrefetchIndexAndFilterBlocks(
+ ro, prefetch_buffer.get(), metaindex_iter.get(), new_table.get(),
+ prefetch_all, table_options, level, file_size,
+ max_file_size_for_l0_meta_pin, &lookup_context);
+
+ if (s.ok()) {
+ // Update tail prefetch stats
+ assert(prefetch_buffer.get() != nullptr);
+ if (tail_prefetch_stats != nullptr) {
+ assert(prefetch_buffer->min_offset_read() < file_size);
+ tail_prefetch_stats->RecordEffectiveSize(
+ static_cast<size_t>(file_size) - prefetch_buffer->min_offset_read());
+ }
+ }
+
+ if (s.ok() && table_reader_cache_res_mgr) {
+ std::size_t mem_usage = new_table->ApproximateMemoryUsage();
+ s = table_reader_cache_res_mgr->MakeCacheReservation(
+ mem_usage, &(rep->table_reader_cache_res_handle));
+ if (s.IsMemoryLimit()) {
+ s = Status::MemoryLimit(
+ "Can't allocate " +
+ kCacheEntryRoleToCamelString[static_cast<std::uint32_t>(
+ CacheEntryRole::kBlockBasedTableReader)] +
+ " due to memory limit based on "
+ "cache capacity for memory allocation");
+ }
+ }
+
+ if (s.ok()) {
+ *table_reader = std::move(new_table);
+ }
+ return s;
+}
+
+Status BlockBasedTable::PrefetchTail(
+ const ReadOptions& ro, RandomAccessFileReader* file, uint64_t file_size,
+ bool force_direct_prefetch, TailPrefetchStats* tail_prefetch_stats,
+ const bool prefetch_all, const bool preload_all,
+ std::unique_ptr<FilePrefetchBuffer>* prefetch_buffer) {
+ size_t tail_prefetch_size = 0;
+ if (tail_prefetch_stats != nullptr) {
+ // Multiple threads may get a 0 (no history) when running in parallel,
+ // but it will get cleared after the first of them finishes.
+ tail_prefetch_size = tail_prefetch_stats->GetSuggestedPrefetchSize();
+ }
+ if (tail_prefetch_size == 0) {
+ // Before read footer, readahead backwards to prefetch data. Do more
+ // readahead if we're going to read index/filter.
+ // TODO: This may incorrectly select small readahead in case partitioned
+ // index/filter is enabled and top-level partition pinning is enabled.
+ // That's because we need to issue readahead before we read the properties,
+ // at which point we don't yet know the index type.
+ tail_prefetch_size = prefetch_all || preload_all ? 512 * 1024 : 4 * 1024;
+ }
+ size_t prefetch_off;
+ size_t prefetch_len;
+ if (file_size < tail_prefetch_size) {
+ prefetch_off = 0;
+ prefetch_len = static_cast<size_t>(file_size);
+ } else {
+ prefetch_off = static_cast<size_t>(file_size - tail_prefetch_size);
+ prefetch_len = tail_prefetch_size;
+ }
+ TEST_SYNC_POINT_CALLBACK("BlockBasedTable::Open::TailPrefetchLen",
+ &tail_prefetch_size);
+
+ // Try file system prefetch
+ if (!file->use_direct_io() && !force_direct_prefetch) {
+ if (!file->Prefetch(prefetch_off, prefetch_len, ro.rate_limiter_priority)
+ .IsNotSupported()) {
+ prefetch_buffer->reset(new FilePrefetchBuffer(
+ 0 /* readahead_size */, 0 /* max_readahead_size */,
+ false /* enable */, true /* track_min_offset */));
+ return Status::OK();
+ }
+ }
+
+ // Use `FilePrefetchBuffer`
+ prefetch_buffer->reset(
+ new FilePrefetchBuffer(0 /* readahead_size */, 0 /* max_readahead_size */,
+ true /* enable */, true /* track_min_offset */));
+
+ IOOptions opts;
+ Status s = file->PrepareIOOptions(ro, opts);
+ if (s.ok()) {
+ s = (*prefetch_buffer)
+ ->Prefetch(opts, file, prefetch_off, prefetch_len,
+ ro.rate_limiter_priority);
+ }
+ return s;
+}
+
+Status BlockBasedTable::ReadPropertiesBlock(
+ const ReadOptions& ro, FilePrefetchBuffer* prefetch_buffer,
+ InternalIterator* meta_iter, const SequenceNumber largest_seqno) {
+ Status s;
+ BlockHandle handle;
+ s = FindOptionalMetaBlock(meta_iter, kPropertiesBlockName, &handle);
+
+ if (!s.ok()) {
+ ROCKS_LOG_WARN(rep_->ioptions.logger,
+ "Error when seeking to properties block from file: %s",
+ s.ToString().c_str());
+ } else if (!handle.IsNull()) {
+ s = meta_iter->status();
+ std::unique_ptr<TableProperties> table_properties;
+ if (s.ok()) {
+ s = ReadTablePropertiesHelper(
+ ro, handle, rep_->file.get(), prefetch_buffer, rep_->footer,
+ rep_->ioptions, &table_properties, nullptr /* memory_allocator */);
+ }
+ IGNORE_STATUS_IF_ERROR(s);
+
+ if (!s.ok()) {
+ ROCKS_LOG_WARN(rep_->ioptions.logger,
+ "Encountered error while reading data from properties "
+ "block %s",
+ s.ToString().c_str());
+ } else {
+ assert(table_properties != nullptr);
+ rep_->table_properties = std::move(table_properties);
+ rep_->blocks_maybe_compressed =
+ rep_->table_properties->compression_name !=
+ CompressionTypeToString(kNoCompression);
+ rep_->blocks_definitely_zstd_compressed =
+ (rep_->table_properties->compression_name ==
+ CompressionTypeToString(kZSTD) ||
+ rep_->table_properties->compression_name ==
+ CompressionTypeToString(kZSTDNotFinalCompression));
+ }
+ } else {
+ ROCKS_LOG_ERROR(rep_->ioptions.logger,
+ "Cannot find Properties block from file.");
+ }
+
+ // Read the table properties, if provided.
+ if (rep_->table_properties) {
+ rep_->whole_key_filtering &=
+ IsFeatureSupported(*(rep_->table_properties),
+ BlockBasedTablePropertyNames::kWholeKeyFiltering,
+ rep_->ioptions.logger);
+ rep_->prefix_filtering &= IsFeatureSupported(
+ *(rep_->table_properties),
+ BlockBasedTablePropertyNames::kPrefixFiltering, rep_->ioptions.logger);
+
+ rep_->index_key_includes_seq =
+ rep_->table_properties->index_key_is_user_key == 0;
+ rep_->index_value_is_full =
+ rep_->table_properties->index_value_is_delta_encoded == 0;
+
+ // Update index_type with the true type.
+ // If table properties don't contain index type, we assume that the table
+ // is in very old format and has kBinarySearch index type.
+ auto& props = rep_->table_properties->user_collected_properties;
+ auto pos = props.find(BlockBasedTablePropertyNames::kIndexType);
+ if (pos != props.end()) {
+ rep_->index_type = static_cast<BlockBasedTableOptions::IndexType>(
+ DecodeFixed32(pos->second.c_str()));
+ }
+
+ rep_->index_has_first_key =
+ rep_->index_type == BlockBasedTableOptions::kBinarySearchWithFirstKey;
+
+ s = GetGlobalSequenceNumber(*(rep_->table_properties), largest_seqno,
+ &(rep_->global_seqno));
+ if (!s.ok()) {
+ ROCKS_LOG_ERROR(rep_->ioptions.logger, "%s", s.ToString().c_str());
+ }
+ }
+ return s;
+}
+
+Status BlockBasedTable::ReadRangeDelBlock(
+ const ReadOptions& read_options, FilePrefetchBuffer* prefetch_buffer,
+ InternalIterator* meta_iter,
+ const InternalKeyComparator& internal_comparator,
+ BlockCacheLookupContext* lookup_context) {
+ Status s;
+ BlockHandle range_del_handle;
+ s = FindOptionalMetaBlock(meta_iter, kRangeDelBlockName, &range_del_handle);
+ if (!s.ok()) {
+ ROCKS_LOG_WARN(
+ rep_->ioptions.logger,
+ "Error when seeking to range delete tombstones block from file: %s",
+ s.ToString().c_str());
+ } else if (!range_del_handle.IsNull()) {
+ Status tmp_status;
+ std::unique_ptr<InternalIterator> iter(NewDataBlockIterator<DataBlockIter>(
+ read_options, range_del_handle,
+ /*input_iter=*/nullptr, BlockType::kRangeDeletion,
+ /*get_context=*/nullptr, lookup_context, prefetch_buffer,
+ /*for_compaction= */ false, /*async_read= */ false, tmp_status));
+ assert(iter != nullptr);
+ s = iter->status();
+ if (!s.ok()) {
+ ROCKS_LOG_WARN(
+ rep_->ioptions.logger,
+ "Encountered error while reading data from range del block %s",
+ s.ToString().c_str());
+ IGNORE_STATUS_IF_ERROR(s);
+ } else {
+ rep_->fragmented_range_dels =
+ std::make_shared<FragmentedRangeTombstoneList>(std::move(iter),
+ internal_comparator);
+ }
+ }
+ return s;
+}
+
+Status BlockBasedTable::PrefetchIndexAndFilterBlocks(
+ const ReadOptions& ro, FilePrefetchBuffer* prefetch_buffer,
+ InternalIterator* meta_iter, BlockBasedTable* new_table, bool prefetch_all,
+ const BlockBasedTableOptions& table_options, const int level,
+ size_t file_size, size_t max_file_size_for_l0_meta_pin,
+ BlockCacheLookupContext* lookup_context) {
+ // Find filter handle and filter type
+ if (rep_->filter_policy) {
+ auto name = rep_->filter_policy->CompatibilityName();
+ bool builtin_compatible =
+ strcmp(name, BuiltinFilterPolicy::kCompatibilityName()) == 0;
+
+ for (const auto& [filter_type, prefix] :
+ {std::make_pair(Rep::FilterType::kFullFilter, kFullFilterBlockPrefix),
+ std::make_pair(Rep::FilterType::kPartitionedFilter,
+ kPartitionedFilterBlockPrefix),
+ std::make_pair(Rep::FilterType::kNoFilter,
+ kObsoleteFilterBlockPrefix)}) {
+ if (builtin_compatible) {
+ // This code is only here to deal with a hiccup in early 7.0.x where
+ // there was an unintentional name change in the SST files metadata.
+ // It should be OK to remove this in the future (late 2022) and just
+ // have the 'else' code.
+ // NOTE: the test:: names below are likely not needed but included
+ // out of caution
+ static const std::unordered_set<std::string> kBuiltinNameAndAliases = {
+ BuiltinFilterPolicy::kCompatibilityName(),
+ test::LegacyBloomFilterPolicy::kClassName(),
+ test::FastLocalBloomFilterPolicy::kClassName(),
+ test::Standard128RibbonFilterPolicy::kClassName(),
+ "rocksdb.internal.DeprecatedBlockBasedBloomFilter",
+ BloomFilterPolicy::kClassName(),
+ RibbonFilterPolicy::kClassName(),
+ };
+
+ // For efficiency, do a prefix seek and see if the first match is
+ // good.
+ meta_iter->Seek(prefix);
+ if (meta_iter->status().ok() && meta_iter->Valid()) {
+ Slice key = meta_iter->key();
+ if (key.starts_with(prefix)) {
+ key.remove_prefix(prefix.size());
+ if (kBuiltinNameAndAliases.find(key.ToString()) !=
+ kBuiltinNameAndAliases.end()) {
+ Slice v = meta_iter->value();
+ Status s = rep_->filter_handle.DecodeFrom(&v);
+ if (s.ok()) {
+ rep_->filter_type = filter_type;
+ if (filter_type == Rep::FilterType::kNoFilter) {
+ ROCKS_LOG_WARN(rep_->ioptions.logger,
+ "Detected obsolete filter type in %s. Read "
+ "performance might suffer until DB is fully "
+ "re-compacted.",
+ rep_->file->file_name().c_str());
+ }
+ break;
+ }
+ }
+ }
+ }
+ } else {
+ std::string filter_block_key = prefix + name;
+ if (FindMetaBlock(meta_iter, filter_block_key, &rep_->filter_handle)
+ .ok()) {
+ rep_->filter_type = filter_type;
+ if (filter_type == Rep::FilterType::kNoFilter) {
+ ROCKS_LOG_WARN(
+ rep_->ioptions.logger,
+ "Detected obsolete filter type in %s. Read performance might "
+ "suffer until DB is fully re-compacted.",
+ rep_->file->file_name().c_str());
+ }
+ break;
+ }
+ }
+ }
+ }
+ // Partition filters cannot be enabled without partition indexes
+ assert(rep_->filter_type != Rep::FilterType::kPartitionedFilter ||
+ rep_->index_type == BlockBasedTableOptions::kTwoLevelIndexSearch);
+
+ // Find compression dictionary handle
+ Status s = FindOptionalMetaBlock(meta_iter, kCompressionDictBlockName,
+ &rep_->compression_dict_handle);
+ if (!s.ok()) {
+ return s;
+ }
+
+ BlockBasedTableOptions::IndexType index_type = rep_->index_type;
+
+ const bool use_cache = table_options.cache_index_and_filter_blocks;
+
+ const bool maybe_flushed =
+ level == 0 && file_size <= max_file_size_for_l0_meta_pin;
+ std::function<bool(PinningTier, PinningTier)> is_pinned =
+ [maybe_flushed, &is_pinned](PinningTier pinning_tier,
+ PinningTier fallback_pinning_tier) {
+ // Fallback to fallback would lead to infinite recursion. Disallow it.
+ assert(fallback_pinning_tier != PinningTier::kFallback);
+
+ switch (pinning_tier) {
+ case PinningTier::kFallback:
+ return is_pinned(fallback_pinning_tier,
+ PinningTier::kNone /* fallback_pinning_tier */);
+ case PinningTier::kNone:
+ return false;
+ case PinningTier::kFlushedAndSimilar:
+ return maybe_flushed;
+ case PinningTier::kAll:
+ return true;
+ };
+
+ // In GCC, this is needed to suppress `control reaches end of non-void
+ // function [-Werror=return-type]`.
+ assert(false);
+ return false;
+ };
+ const bool pin_top_level_index = is_pinned(
+ table_options.metadata_cache_options.top_level_index_pinning,
+ table_options.pin_top_level_index_and_filter ? PinningTier::kAll
+ : PinningTier::kNone);
+ const bool pin_partition =
+ is_pinned(table_options.metadata_cache_options.partition_pinning,
+ table_options.pin_l0_filter_and_index_blocks_in_cache
+ ? PinningTier::kFlushedAndSimilar
+ : PinningTier::kNone);
+ const bool pin_unpartitioned =
+ is_pinned(table_options.metadata_cache_options.unpartitioned_pinning,
+ table_options.pin_l0_filter_and_index_blocks_in_cache
+ ? PinningTier::kFlushedAndSimilar
+ : PinningTier::kNone);
+
+ // pin the first level of index
+ const bool pin_index =
+ index_type == BlockBasedTableOptions::kTwoLevelIndexSearch
+ ? pin_top_level_index
+ : pin_unpartitioned;
+ // prefetch the first level of index
+ // WART: this might be redundant (unnecessary cache hit) if !pin_index,
+ // depending on prepopulate_block_cache option
+ const bool prefetch_index = prefetch_all || pin_index;
+
+ std::unique_ptr<IndexReader> index_reader;
+ s = new_table->CreateIndexReader(ro, prefetch_buffer, meta_iter, use_cache,
+ prefetch_index, pin_index, lookup_context,
+ &index_reader);
+ if (!s.ok()) {
+ return s;
+ }
+
+ rep_->index_reader = std::move(index_reader);
+
+ // The partitions of partitioned index are always stored in cache. They
+ // are hence follow the configuration for pin and prefetch regardless of
+ // the value of cache_index_and_filter_blocks
+ if (prefetch_all || pin_partition) {
+ s = rep_->index_reader->CacheDependencies(ro, pin_partition);
+ }
+ if (!s.ok()) {
+ return s;
+ }
+
+ // pin the first level of filter
+ const bool pin_filter =
+ rep_->filter_type == Rep::FilterType::kPartitionedFilter
+ ? pin_top_level_index
+ : pin_unpartitioned;
+ // prefetch the first level of filter
+ // WART: this might be redundant (unnecessary cache hit) if !pin_filter,
+ // depending on prepopulate_block_cache option
+ const bool prefetch_filter = prefetch_all || pin_filter;
+
+ if (rep_->filter_policy) {
+ auto filter = new_table->CreateFilterBlockReader(
+ ro, prefetch_buffer, use_cache, prefetch_filter, pin_filter,
+ lookup_context);
+
+ if (filter) {
+ // Refer to the comment above about paritioned indexes always being cached
+ if (prefetch_all || pin_partition) {
+ s = filter->CacheDependencies(ro, pin_partition);
+ if (!s.ok()) {
+ return s;
+ }
+ }
+ rep_->filter = std::move(filter);
+ }
+ }
+
+ if (!rep_->compression_dict_handle.IsNull()) {
+ std::unique_ptr<UncompressionDictReader> uncompression_dict_reader;
+ s = UncompressionDictReader::Create(
+ this, ro, prefetch_buffer, use_cache, prefetch_all || pin_unpartitioned,
+ pin_unpartitioned, lookup_context, &uncompression_dict_reader);
+ if (!s.ok()) {
+ return s;
+ }
+
+ rep_->uncompression_dict_reader = std::move(uncompression_dict_reader);
+ }
+
+ assert(s.ok());
+ return s;
+}
+
+void BlockBasedTable::SetupForCompaction() {
+ switch (rep_->ioptions.access_hint_on_compaction_start) {
+ case Options::NONE:
+ break;
+ case Options::NORMAL:
+ rep_->file->file()->Hint(FSRandomAccessFile::kNormal);
+ break;
+ case Options::SEQUENTIAL:
+ rep_->file->file()->Hint(FSRandomAccessFile::kSequential);
+ break;
+ case Options::WILLNEED:
+ rep_->file->file()->Hint(FSRandomAccessFile::kWillNeed);
+ break;
+ default:
+ assert(false);
+ }
+}
+
+std::shared_ptr<const TableProperties> BlockBasedTable::GetTableProperties()
+ const {
+ return rep_->table_properties;
+}
+
+size_t BlockBasedTable::ApproximateMemoryUsage() const {
+ size_t usage = 0;
+ if (rep_) {
+ usage += rep_->ApproximateMemoryUsage();
+ } else {
+ return usage;
+ }
+ if (rep_->filter) {
+ usage += rep_->filter->ApproximateMemoryUsage();
+ }
+ if (rep_->index_reader) {
+ usage += rep_->index_reader->ApproximateMemoryUsage();
+ }
+ if (rep_->uncompression_dict_reader) {
+ usage += rep_->uncompression_dict_reader->ApproximateMemoryUsage();
+ }
+ if (rep_->table_properties) {
+ usage += rep_->table_properties->ApproximateMemoryUsage();
+ }
+ return usage;
+}
+
+// Load the meta-index-block from the file. On success, return the loaded
+// metaindex
+// block and its iterator.
+Status BlockBasedTable::ReadMetaIndexBlock(
+ const ReadOptions& ro, FilePrefetchBuffer* prefetch_buffer,
+ std::unique_ptr<Block>* metaindex_block,
+ std::unique_ptr<InternalIterator>* iter) {
+ // TODO(sanjay): Skip this if footer.metaindex_handle() size indicates
+ // it is an empty block.
+ std::unique_ptr<Block> metaindex;
+ Status s = ReadBlockFromFile(
+ rep_->file.get(), prefetch_buffer, rep_->footer, ro,
+ rep_->footer.metaindex_handle(), &metaindex, rep_->ioptions,
+ true /* decompress */, true /*maybe_compressed*/, BlockType::kMetaIndex,
+ UncompressionDict::GetEmptyDict(), rep_->persistent_cache_options,
+ 0 /* read_amp_bytes_per_bit */, GetMemoryAllocator(rep_->table_options),
+ false /* for_compaction */, rep_->blocks_definitely_zstd_compressed,
+ nullptr /* filter_policy */, false /* async_read */);
+
+ if (!s.ok()) {
+ ROCKS_LOG_ERROR(rep_->ioptions.logger,
+ "Encountered error while reading data from properties"
+ " block %s",
+ s.ToString().c_str());
+ return s;
+ }
+
+ *metaindex_block = std::move(metaindex);
+ // meta block uses bytewise comparator.
+ iter->reset(metaindex_block->get()->NewMetaIterator());
+ return Status::OK();
+}
+
+template <typename TBlocklike>
+Status BlockBasedTable::GetDataBlockFromCache(
+ const Slice& cache_key, Cache* block_cache, Cache* block_cache_compressed,
+ const ReadOptions& read_options,
+ CachableEntry<TBlocklike>* out_parsed_block,
+ const UncompressionDict& uncompression_dict, BlockType block_type,
+ const bool wait, GetContext* get_context) const {
+ const size_t read_amp_bytes_per_bit =
+ block_type == BlockType::kData
+ ? rep_->table_options.read_amp_bytes_per_bit
+ : 0;
+ assert(out_parsed_block);
+ assert(out_parsed_block->IsEmpty());
+ // Here we treat the legacy name "...index_and_filter_blocks..." to mean all
+ // metadata blocks that might go into block cache, EXCEPT only those needed
+ // for the read path (Get, etc.). TableProperties should not be needed on the
+ // read path (prefix extractor setting is an O(1) size special case that we
+ // are working not to require from TableProperties), so it is not given
+ // high-priority treatment if it should go into BlockCache.
+ const Cache::Priority priority =
+ rep_->table_options.cache_index_and_filter_blocks_with_high_priority &&
+ block_type != BlockType::kData &&
+ block_type != BlockType::kProperties
+ ? Cache::Priority::HIGH
+ : Cache::Priority::LOW;
+
+ Status s;
+ BlockContents* compressed_block = nullptr;
+ Cache::Handle* block_cache_compressed_handle = nullptr;
+ Statistics* statistics = rep_->ioptions.statistics.get();
+ bool using_zstd = rep_->blocks_definitely_zstd_compressed;
+ const FilterPolicy* filter_policy = rep_->filter_policy;
+ Cache::CreateCallback create_cb = GetCreateCallback<TBlocklike>(
+ read_amp_bytes_per_bit, statistics, using_zstd, filter_policy);
+
+ // Lookup uncompressed cache first
+ if (block_cache != nullptr) {
+ assert(!cache_key.empty());
+ Cache::Handle* cache_handle = nullptr;
+ cache_handle = GetEntryFromCache(
+ rep_->ioptions.lowest_used_cache_tier, block_cache, cache_key,
+ block_type, wait, get_context,
+ BlocklikeTraits<TBlocklike>::GetCacheItemHelper(block_type), create_cb,
+ priority);
+ if (cache_handle != nullptr) {
+ out_parsed_block->SetCachedValue(
+ reinterpret_cast<TBlocklike*>(block_cache->Value(cache_handle)),
+ block_cache, cache_handle);
+ return s;
+ }
+ }
+
+ // If not found, search from the compressed block cache.
+ assert(out_parsed_block->IsEmpty());
+
+ if (block_cache_compressed == nullptr) {
+ return s;
+ }
+
+ assert(!cache_key.empty());
+ BlockContents contents;
+ block_cache_compressed_handle =
+ block_cache_compressed->Lookup(cache_key, statistics);
+
+ // if we found in the compressed cache, then uncompress and insert into
+ // uncompressed cache
+ if (block_cache_compressed_handle == nullptr) {
+ RecordTick(statistics, BLOCK_CACHE_COMPRESSED_MISS);
+ return s;
+ }
+
+ // found compressed block
+ RecordTick(statistics, BLOCK_CACHE_COMPRESSED_HIT);
+ compressed_block = reinterpret_cast<BlockContents*>(
+ block_cache_compressed->Value(block_cache_compressed_handle));
+ CompressionType compression_type = GetBlockCompressionType(*compressed_block);
+ assert(compression_type != kNoCompression);
+
+ // Retrieve the uncompressed contents into a new buffer
+ UncompressionContext context(compression_type);
+ UncompressionInfo info(context, uncompression_dict, compression_type);
+ s = UncompressSerializedBlock(
+ info, compressed_block->data.data(), compressed_block->data.size(),
+ &contents, rep_->table_options.format_version, rep_->ioptions,
+ GetMemoryAllocator(rep_->table_options));
+
+ // Insert parsed block into block cache, the priority is based on the
+ // data block type.
+ if (s.ok()) {
+ std::unique_ptr<TBlocklike> block_holder(
+ BlocklikeTraits<TBlocklike>::Create(
+ std::move(contents), read_amp_bytes_per_bit, statistics,
+ rep_->blocks_definitely_zstd_compressed,
+ rep_->table_options.filter_policy.get()));
+
+ if (block_cache != nullptr && block_holder->own_bytes() &&
+ read_options.fill_cache) {
+ size_t charge = block_holder->ApproximateMemoryUsage();
+ Cache::Handle* cache_handle = nullptr;
+ auto block_holder_raw_ptr = block_holder.get();
+ s = InsertEntryToCache(
+ rep_->ioptions.lowest_used_cache_tier, block_cache, cache_key,
+ BlocklikeTraits<TBlocklike>::GetCacheItemHelper(block_type),
+ std::move(block_holder), charge, &cache_handle, priority);
+ if (s.ok()) {
+ assert(cache_handle != nullptr);
+ out_parsed_block->SetCachedValue(block_holder_raw_ptr, block_cache,
+ cache_handle);
+
+ UpdateCacheInsertionMetrics(block_type, get_context, charge,
+ s.IsOkOverwritten(), rep_->ioptions.stats);
+ } else {
+ RecordTick(statistics, BLOCK_CACHE_ADD_FAILURES);
+ }
+ } else {
+ out_parsed_block->SetOwnedValue(std::move(block_holder));
+ }
+ }
+
+ // Release hold on compressed cache entry
+ block_cache_compressed->Release(block_cache_compressed_handle);
+ return s;
+}
+
+template <typename TBlocklike>
+Status BlockBasedTable::PutDataBlockToCache(
+ const Slice& cache_key, Cache* block_cache, Cache* block_cache_compressed,
+ CachableEntry<TBlocklike>* out_parsed_block, BlockContents&& block_contents,
+ CompressionType block_comp_type,
+ const UncompressionDict& uncompression_dict,
+ MemoryAllocator* memory_allocator, BlockType block_type,
+ GetContext* get_context) const {
+ const ImmutableOptions& ioptions = rep_->ioptions;
+ const uint32_t format_version = rep_->table_options.format_version;
+ const size_t read_amp_bytes_per_bit =
+ block_type == BlockType::kData
+ ? rep_->table_options.read_amp_bytes_per_bit
+ : 0;
+ const Cache::Priority priority =
+ rep_->table_options.cache_index_and_filter_blocks_with_high_priority &&
+ block_type != BlockType::kData
+ ? Cache::Priority::HIGH
+ : Cache::Priority::LOW;
+ assert(out_parsed_block);
+ assert(out_parsed_block->IsEmpty());
+
+ Status s;
+ Statistics* statistics = ioptions.stats;
+
+ std::unique_ptr<TBlocklike> block_holder;
+ if (block_comp_type != kNoCompression) {
+ // Retrieve the uncompressed contents into a new buffer
+ BlockContents uncompressed_block_contents;
+ UncompressionContext context(block_comp_type);
+ UncompressionInfo info(context, uncompression_dict, block_comp_type);
+ s = UncompressBlockData(info, block_contents.data.data(),
+ block_contents.data.size(),
+ &uncompressed_block_contents, format_version,
+ ioptions, memory_allocator);
+ if (!s.ok()) {
+ return s;
+ }
+
+ block_holder.reset(BlocklikeTraits<TBlocklike>::Create(
+ std::move(uncompressed_block_contents), read_amp_bytes_per_bit,
+ statistics, rep_->blocks_definitely_zstd_compressed,
+ rep_->table_options.filter_policy.get()));
+ } else {
+ block_holder.reset(BlocklikeTraits<TBlocklike>::Create(
+ std::move(block_contents), read_amp_bytes_per_bit, statistics,
+ rep_->blocks_definitely_zstd_compressed,
+ rep_->table_options.filter_policy.get()));
+ }
+
+ // Insert compressed block into compressed block cache.
+ // Release the hold on the compressed cache entry immediately.
+ if (block_cache_compressed != nullptr && block_comp_type != kNoCompression &&
+ block_contents.own_bytes()) {
+ assert(block_contents.has_trailer);
+ assert(!cache_key.empty());
+
+ // We cannot directly put block_contents because this could point to
+ // an object in the stack.
+ auto block_cont_for_comp_cache =
+ std::make_unique<BlockContents>(std::move(block_contents));
+ size_t charge = block_cont_for_comp_cache->ApproximateMemoryUsage();
+
+ s = block_cache_compressed->Insert(
+ cache_key, block_cont_for_comp_cache.get(), charge,
+ &DeleteCacheEntry<BlockContents>, nullptr /*handle*/,
+ Cache::Priority::LOW);
+
+ if (s.ok()) {
+ // Cache took ownership
+ block_cont_for_comp_cache.release();
+ RecordTick(statistics, BLOCK_CACHE_COMPRESSED_ADD);
+ } else {
+ RecordTick(statistics, BLOCK_CACHE_COMPRESSED_ADD_FAILURES);
+ }
+ }
+
+ // insert into uncompressed block cache
+ if (block_cache != nullptr && block_holder->own_bytes()) {
+ size_t charge = block_holder->ApproximateMemoryUsage();
+ auto block_holder_raw_ptr = block_holder.get();
+ Cache::Handle* cache_handle = nullptr;
+ s = InsertEntryToCache(
+ rep_->ioptions.lowest_used_cache_tier, block_cache, cache_key,
+ BlocklikeTraits<TBlocklike>::GetCacheItemHelper(block_type),
+ std::move(block_holder), charge, &cache_handle, priority);
+ if (s.ok()) {
+ assert(cache_handle != nullptr);
+ out_parsed_block->SetCachedValue(block_holder_raw_ptr, block_cache,
+ cache_handle);
+
+ UpdateCacheInsertionMetrics(block_type, get_context, charge,
+ s.IsOkOverwritten(), rep_->ioptions.stats);
+ } else {
+ RecordTick(statistics, BLOCK_CACHE_ADD_FAILURES);
+ }
+ } else {
+ out_parsed_block->SetOwnedValue(std::move(block_holder));
+ }
+
+ return s;
+}
+
+std::unique_ptr<FilterBlockReader> BlockBasedTable::CreateFilterBlockReader(
+ const ReadOptions& ro, FilePrefetchBuffer* prefetch_buffer, bool use_cache,
+ bool prefetch, bool pin, BlockCacheLookupContext* lookup_context) {
+ auto& rep = rep_;
+ auto filter_type = rep->filter_type;
+ if (filter_type == Rep::FilterType::kNoFilter) {
+ return std::unique_ptr<FilterBlockReader>();
+ }
+
+ assert(rep->filter_policy);
+
+ switch (filter_type) {
+ case Rep::FilterType::kPartitionedFilter:
+ return PartitionedFilterBlockReader::Create(
+ this, ro, prefetch_buffer, use_cache, prefetch, pin, lookup_context);
+
+ case Rep::FilterType::kFullFilter:
+ return FullFilterBlockReader::Create(this, ro, prefetch_buffer, use_cache,
+ prefetch, pin, lookup_context);
+
+ default:
+ // filter_type is either kNoFilter (exited the function at the first if),
+ // or it must be covered in this switch block
+ assert(false);
+ return std::unique_ptr<FilterBlockReader>();
+ }
+}
+
+// disable_prefix_seek should be set to true when prefix_extractor found in SST
+// differs from the one in mutable_cf_options and index type is HashBasedIndex
+InternalIteratorBase<IndexValue>* BlockBasedTable::NewIndexIterator(
+ const ReadOptions& read_options, bool disable_prefix_seek,
+ IndexBlockIter* input_iter, GetContext* get_context,
+ BlockCacheLookupContext* lookup_context) const {
+ assert(rep_ != nullptr);
+ assert(rep_->index_reader != nullptr);
+
+ // We don't return pinned data from index blocks, so no need
+ // to set `block_contents_pinned`.
+ return rep_->index_reader->NewIterator(read_options, disable_prefix_seek,
+ input_iter, get_context,
+ lookup_context);
+}
+
+template <>
+DataBlockIter* BlockBasedTable::InitBlockIterator<DataBlockIter>(
+ const Rep* rep, Block* block, BlockType block_type,
+ DataBlockIter* input_iter, bool block_contents_pinned) {
+ return block->NewDataIterator(rep->internal_comparator.user_comparator(),
+ rep->get_global_seqno(block_type), input_iter,
+ rep->ioptions.stats, block_contents_pinned);
+}
+
+template <>
+IndexBlockIter* BlockBasedTable::InitBlockIterator<IndexBlockIter>(
+ const Rep* rep, Block* block, BlockType block_type,
+ IndexBlockIter* input_iter, bool block_contents_pinned) {
+ return block->NewIndexIterator(
+ rep->internal_comparator.user_comparator(),
+ rep->get_global_seqno(block_type), input_iter, rep->ioptions.stats,
+ /* total_order_seek */ true, rep->index_has_first_key,
+ rep->index_key_includes_seq, rep->index_value_is_full,
+ block_contents_pinned);
+}
+
+// If contents is nullptr, this function looks up the block caches for the
+// data block referenced by handle, and read the block from disk if necessary.
+// If contents is non-null, it skips the cache lookup and disk read, since
+// the caller has already read it. In both cases, if ro.fill_cache is true,
+// it inserts the block into the block cache.
+template <typename TBlocklike>
+Status BlockBasedTable::MaybeReadBlockAndLoadToCache(
+ FilePrefetchBuffer* prefetch_buffer, const ReadOptions& ro,
+ const BlockHandle& handle, const UncompressionDict& uncompression_dict,
+ const bool wait, const bool for_compaction,
+ CachableEntry<TBlocklike>* out_parsed_block, BlockType block_type,
+ GetContext* get_context, BlockCacheLookupContext* lookup_context,
+ BlockContents* contents, bool async_read) const {
+ assert(out_parsed_block != nullptr);
+ const bool no_io = (ro.read_tier == kBlockCacheTier);
+ Cache* block_cache = rep_->table_options.block_cache.get();
+ Cache* block_cache_compressed =
+ rep_->table_options.block_cache_compressed.get();
+
+ // First, try to get the block from the cache
+ //
+ // If either block cache is enabled, we'll try to read from it.
+ Status s;
+ CacheKey key_data;
+ Slice key;
+ bool is_cache_hit = false;
+ if (block_cache != nullptr || block_cache_compressed != nullptr) {
+ // create key for block cache
+ key_data = GetCacheKey(rep_->base_cache_key, handle);
+ key = key_data.AsSlice();
+
+ if (!contents) {
+ s = GetDataBlockFromCache(key, block_cache, block_cache_compressed, ro,
+ out_parsed_block, uncompression_dict,
+ block_type, wait, get_context);
+ // Value could still be null at this point, so check the cache handle
+ // and update the read pattern for prefetching
+ if (out_parsed_block->GetValue() || out_parsed_block->GetCacheHandle()) {
+ // TODO(haoyu): Differentiate cache hit on uncompressed block cache and
+ // compressed block cache.
+ is_cache_hit = true;
+ if (prefetch_buffer) {
+ // Update the block details so that PrefetchBuffer can use the read
+ // pattern to determine if reads are sequential or not for
+ // prefetching. It should also take in account blocks read from cache.
+ prefetch_buffer->UpdateReadPattern(
+ handle.offset(), BlockSizeWithTrailer(handle),
+ ro.adaptive_readahead /*decrease_readahead_size*/);
+ }
+ }
+ }
+
+ // Can't find the block from the cache. If I/O is allowed, read from the
+ // file.
+ if (out_parsed_block->GetValue() == nullptr &&
+ out_parsed_block->GetCacheHandle() == nullptr && !no_io &&
+ ro.fill_cache) {
+ Statistics* statistics = rep_->ioptions.stats;
+ const bool maybe_compressed =
+ block_type != BlockType::kFilter &&
+ block_type != BlockType::kCompressionDictionary &&
+ rep_->blocks_maybe_compressed;
+ const bool do_uncompress = maybe_compressed && !block_cache_compressed;
+ CompressionType contents_comp_type;
+ // Maybe serialized or uncompressed
+ BlockContents tmp_contents;
+ if (!contents) {
+ Histograms histogram = for_compaction ? READ_BLOCK_COMPACTION_MICROS
+ : READ_BLOCK_GET_MICROS;
+ StopWatch sw(rep_->ioptions.clock, statistics, histogram);
+ BlockFetcher block_fetcher(
+ rep_->file.get(), prefetch_buffer, rep_->footer, ro, handle,
+ &tmp_contents, rep_->ioptions, do_uncompress, maybe_compressed,
+ block_type, uncompression_dict, rep_->persistent_cache_options,
+ GetMemoryAllocator(rep_->table_options),
+ GetMemoryAllocatorForCompressedBlock(rep_->table_options));
+
+ // If prefetch_buffer is not allocated, it will fallback to synchronous
+ // reading of block contents.
+ if (async_read && prefetch_buffer != nullptr) {
+ s = block_fetcher.ReadAsyncBlockContents();
+ if (!s.ok()) {
+ return s;
+ }
+ } else {
+ s = block_fetcher.ReadBlockContents();
+ }
+
+ contents_comp_type = block_fetcher.get_compression_type();
+ contents = &tmp_contents;
+ if (get_context) {
+ switch (block_type) {
+ case BlockType::kIndex:
+ ++get_context->get_context_stats_.num_index_read;
+ break;
+ case BlockType::kFilter:
+ case BlockType::kFilterPartitionIndex:
+ ++get_context->get_context_stats_.num_filter_read;
+ break;
+ default:
+ break;
+ }
+ }
+ } else {
+ contents_comp_type = GetBlockCompressionType(*contents);
+ }
+
+ if (s.ok()) {
+ // If filling cache is allowed and a cache is configured, try to put the
+ // block to the cache.
+ s = PutDataBlockToCache(
+ key, block_cache, block_cache_compressed, out_parsed_block,
+ std::move(*contents), contents_comp_type, uncompression_dict,
+ GetMemoryAllocator(rep_->table_options), block_type, get_context);
+ }
+ }
+ }
+
+ // Fill lookup_context.
+ if (block_cache_tracer_ && block_cache_tracer_->is_tracing_enabled() &&
+ lookup_context) {
+ size_t usage = 0;
+ uint64_t nkeys = 0;
+ if (out_parsed_block->GetValue()) {
+ // Approximate the number of keys in the block using restarts.
+ nkeys = rep_->table_options.block_restart_interval *
+ BlocklikeTraits<TBlocklike>::GetNumRestarts(
+ *out_parsed_block->GetValue());
+ usage = out_parsed_block->GetValue()->ApproximateMemoryUsage();
+ }
+ TraceType trace_block_type = TraceType::kTraceMax;
+ switch (block_type) {
+ case BlockType::kData:
+ trace_block_type = TraceType::kBlockTraceDataBlock;
+ break;
+ case BlockType::kFilter:
+ case BlockType::kFilterPartitionIndex:
+ trace_block_type = TraceType::kBlockTraceFilterBlock;
+ break;
+ case BlockType::kCompressionDictionary:
+ trace_block_type = TraceType::kBlockTraceUncompressionDictBlock;
+ break;
+ case BlockType::kRangeDeletion:
+ trace_block_type = TraceType::kBlockTraceRangeDeletionBlock;
+ break;
+ case BlockType::kIndex:
+ trace_block_type = TraceType::kBlockTraceIndexBlock;
+ break;
+ default:
+ // This cannot happen.
+ assert(false);
+ break;
+ }
+ bool no_insert = no_io || !ro.fill_cache;
+ if (BlockCacheTraceHelper::IsGetOrMultiGetOnDataBlock(
+ trace_block_type, lookup_context->caller)) {
+ // Defer logging the access to Get() and MultiGet() to trace additional
+ // information, e.g., referenced_key_exist_in_block.
+
+ // Make a copy of the block key here since it will be logged later.
+ lookup_context->FillLookupContext(
+ is_cache_hit, no_insert, trace_block_type,
+ /*block_size=*/usage, /*block_key=*/key.ToString(), nkeys);
+ } else {
+ // Avoid making copy of block_key and cf_name when constructing the access
+ // record.
+ BlockCacheTraceRecord access_record(
+ rep_->ioptions.clock->NowMicros(),
+ /*block_key=*/"", trace_block_type,
+ /*block_size=*/usage, rep_->cf_id_for_tracing(),
+ /*cf_name=*/"", rep_->level_for_tracing(),
+ rep_->sst_number_for_tracing(), lookup_context->caller, is_cache_hit,
+ no_insert, lookup_context->get_id,
+ lookup_context->get_from_user_specified_snapshot,
+ /*referenced_key=*/"");
+ // TODO: Should handle this error?
+ block_cache_tracer_
+ ->WriteBlockAccess(access_record, key, rep_->cf_name_for_tracing(),
+ lookup_context->referenced_key)
+ .PermitUncheckedError();
+ }
+ }
+
+ assert(s.ok() || out_parsed_block->GetValue() == nullptr);
+ return s;
+}
+
+template <typename TBlocklike>
+Status BlockBasedTable::RetrieveBlock(
+ FilePrefetchBuffer* prefetch_buffer, const ReadOptions& ro,
+ const BlockHandle& handle, const UncompressionDict& uncompression_dict,
+ CachableEntry<TBlocklike>* out_parsed_block, BlockType block_type,
+ GetContext* get_context, BlockCacheLookupContext* lookup_context,
+ bool for_compaction, bool use_cache, bool wait_for_cache,
+ bool async_read) const {
+ assert(out_parsed_block);
+ assert(out_parsed_block->IsEmpty());
+
+ Status s;
+ if (use_cache) {
+ s = MaybeReadBlockAndLoadToCache(prefetch_buffer, ro, handle,
+ uncompression_dict, wait_for_cache,
+ for_compaction, out_parsed_block,
+ block_type, get_context, lookup_context,
+ /*contents=*/nullptr, async_read);
+
+ if (!s.ok()) {
+ return s;
+ }
+
+ if (out_parsed_block->GetValue() != nullptr ||
+ out_parsed_block->GetCacheHandle() != nullptr) {
+ assert(s.ok());
+ return s;
+ }
+ }
+
+ assert(out_parsed_block->IsEmpty());
+
+ const bool no_io = ro.read_tier == kBlockCacheTier;
+ if (no_io) {
+ return Status::Incomplete("no blocking io");
+ }
+
+ const bool maybe_compressed =
+ block_type != BlockType::kFilter &&
+ block_type != BlockType::kCompressionDictionary &&
+ rep_->blocks_maybe_compressed;
+ const bool do_uncompress = maybe_compressed;
+ std::unique_ptr<TBlocklike> block;
+
+ {
+ Histograms histogram =
+ for_compaction ? READ_BLOCK_COMPACTION_MICROS : READ_BLOCK_GET_MICROS;
+ StopWatch sw(rep_->ioptions.clock, rep_->ioptions.stats, histogram);
+ s = ReadBlockFromFile(
+ rep_->file.get(), prefetch_buffer, rep_->footer, ro, handle, &block,
+ rep_->ioptions, do_uncompress, maybe_compressed, block_type,
+ uncompression_dict, rep_->persistent_cache_options,
+ block_type == BlockType::kData
+ ? rep_->table_options.read_amp_bytes_per_bit
+ : 0,
+ GetMemoryAllocator(rep_->table_options), for_compaction,
+ rep_->blocks_definitely_zstd_compressed,
+ rep_->table_options.filter_policy.get(), async_read);
+
+ if (get_context) {
+ switch (block_type) {
+ case BlockType::kIndex:
+ ++(get_context->get_context_stats_.num_index_read);
+ break;
+ case BlockType::kFilter:
+ case BlockType::kFilterPartitionIndex:
+ ++(get_context->get_context_stats_.num_filter_read);
+ break;
+ default:
+ break;
+ }
+ }
+ }
+
+ if (!s.ok()) {
+ return s;
+ }
+
+ out_parsed_block->SetOwnedValue(std::move(block));
+
+ assert(s.ok());
+ return s;
+}
+
+// Explicitly instantiate templates for each "blocklike" type we use.
+// This makes it possible to keep the template definitions in the .cc file.
+template Status BlockBasedTable::RetrieveBlock<ParsedFullFilterBlock>(
+ FilePrefetchBuffer* prefetch_buffer, const ReadOptions& ro,
+ const BlockHandle& handle, const UncompressionDict& uncompression_dict,
+ CachableEntry<ParsedFullFilterBlock>* out_parsed_block,
+ BlockType block_type, GetContext* get_context,
+ BlockCacheLookupContext* lookup_context, bool for_compaction,
+ bool use_cache, bool wait_for_cache, bool async_read) const;
+
+template Status BlockBasedTable::RetrieveBlock<Block>(
+ FilePrefetchBuffer* prefetch_buffer, const ReadOptions& ro,
+ const BlockHandle& handle, const UncompressionDict& uncompression_dict,
+ CachableEntry<Block>* out_parsed_block, BlockType block_type,
+ GetContext* get_context, BlockCacheLookupContext* lookup_context,
+ bool for_compaction, bool use_cache, bool wait_for_cache,
+ bool async_read) const;
+
+template Status BlockBasedTable::RetrieveBlock<UncompressionDict>(
+ FilePrefetchBuffer* prefetch_buffer, const ReadOptions& ro,
+ const BlockHandle& handle, const UncompressionDict& uncompression_dict,
+ CachableEntry<UncompressionDict>* out_parsed_block, BlockType block_type,
+ GetContext* get_context, BlockCacheLookupContext* lookup_context,
+ bool for_compaction, bool use_cache, bool wait_for_cache,
+ bool async_read) const;
+
+BlockBasedTable::PartitionedIndexIteratorState::PartitionedIndexIteratorState(
+ const BlockBasedTable* table,
+ UnorderedMap<uint64_t, CachableEntry<Block>>* block_map)
+ : table_(table), block_map_(block_map) {}
+
+InternalIteratorBase<IndexValue>*
+BlockBasedTable::PartitionedIndexIteratorState::NewSecondaryIterator(
+ const BlockHandle& handle) {
+ // Return a block iterator on the index partition
+ auto block = block_map_->find(handle.offset());
+ // block_map_ must be exhaustive
+ if (block == block_map_->end()) {
+ assert(false);
+ // Signal problem to caller
+ return nullptr;
+ }
+ const Rep* rep = table_->get_rep();
+ assert(rep);
+
+ Statistics* kNullStats = nullptr;
+ // We don't return pinned data from index blocks, so no need
+ // to set `block_contents_pinned`.
+ return block->second.GetValue()->NewIndexIterator(
+ rep->internal_comparator.user_comparator(),
+ rep->get_global_seqno(BlockType::kIndex), nullptr, kNullStats, true,
+ rep->index_has_first_key, rep->index_key_includes_seq,
+ rep->index_value_is_full);
+}
+
+// This will be broken if the user specifies an unusual implementation
+// of Options.comparator, or if the user specifies an unusual
+// definition of prefixes in BlockBasedTableOptions.filter_policy.
+// In particular, we require the following three properties:
+//
+// 1) key.starts_with(prefix(key))
+// 2) Compare(prefix(key), key) <= 0.
+// 3) If Compare(key1, key2) <= 0, then Compare(prefix(key1), prefix(key2)) <= 0
+//
+// If read_options.read_tier == kBlockCacheTier, this method will do no I/O and
+// will return true if the filter block is not in memory and not found in block
+// cache.
+//
+// REQUIRES: this method shouldn't be called while the DB lock is held.
+bool BlockBasedTable::PrefixRangeMayMatch(
+ const Slice& internal_key, const ReadOptions& read_options,
+ const SliceTransform* options_prefix_extractor,
+ const bool need_upper_bound_check,
+ BlockCacheLookupContext* lookup_context) const {
+ if (!rep_->filter_policy) {
+ return true;
+ }
+
+ const SliceTransform* prefix_extractor;
+
+ if (rep_->table_prefix_extractor == nullptr) {
+ if (need_upper_bound_check) {
+ return true;
+ }
+ prefix_extractor = options_prefix_extractor;
+ } else {
+ prefix_extractor = rep_->table_prefix_extractor.get();
+ }
+ auto ts_sz = rep_->internal_comparator.user_comparator()->timestamp_size();
+ auto user_key_without_ts =
+ ExtractUserKeyAndStripTimestamp(internal_key, ts_sz);
+ if (!prefix_extractor->InDomain(user_key_without_ts)) {
+ return true;
+ }
+
+ bool may_match = true;
+
+ FilterBlockReader* const filter = rep_->filter.get();
+ bool filter_checked = false;
+ if (filter != nullptr) {
+ const bool no_io = read_options.read_tier == kBlockCacheTier;
+
+ const Slice* const const_ikey_ptr = &internal_key;
+ may_match = filter->RangeMayExist(
+ read_options.iterate_upper_bound, user_key_without_ts, prefix_extractor,
+ rep_->internal_comparator.user_comparator(), const_ikey_ptr,
+ &filter_checked, need_upper_bound_check, no_io, lookup_context,
+ read_options.rate_limiter_priority);
+ }
+
+ if (filter_checked) {
+ Statistics* statistics = rep_->ioptions.stats;
+ RecordTick(statistics, BLOOM_FILTER_PREFIX_CHECKED);
+ if (!may_match) {
+ RecordTick(statistics, BLOOM_FILTER_PREFIX_USEFUL);
+ }
+ }
+
+ return may_match;
+}
+
+bool BlockBasedTable::PrefixExtractorChanged(
+ const SliceTransform* prefix_extractor) const {
+ if (prefix_extractor == nullptr) {
+ return true;
+ } else if (prefix_extractor == rep_->table_prefix_extractor.get()) {
+ return false;
+ } else {
+ return PrefixExtractorChangedHelper(rep_->table_properties.get(),
+ prefix_extractor);
+ }
+}
+
+InternalIterator* BlockBasedTable::NewIterator(
+ const ReadOptions& read_options, const SliceTransform* prefix_extractor,
+ Arena* arena, bool skip_filters, TableReaderCaller caller,
+ size_t compaction_readahead_size, bool allow_unprepared_value) {
+ BlockCacheLookupContext lookup_context{caller};
+ bool need_upper_bound_check =
+ read_options.auto_prefix_mode || PrefixExtractorChanged(prefix_extractor);
+ std::unique_ptr<InternalIteratorBase<IndexValue>> index_iter(NewIndexIterator(
+ read_options,
+ /*disable_prefix_seek=*/need_upper_bound_check &&
+ rep_->index_type == BlockBasedTableOptions::kHashSearch,
+ /*input_iter=*/nullptr, /*get_context=*/nullptr, &lookup_context));
+ if (arena == nullptr) {
+ return new BlockBasedTableIterator(
+ this, read_options, rep_->internal_comparator, std::move(index_iter),
+ !skip_filters && !read_options.total_order_seek &&
+ prefix_extractor != nullptr,
+ need_upper_bound_check, prefix_extractor, caller,
+ compaction_readahead_size, allow_unprepared_value);
+ } else {
+ auto* mem = arena->AllocateAligned(sizeof(BlockBasedTableIterator));
+ return new (mem) BlockBasedTableIterator(
+ this, read_options, rep_->internal_comparator, std::move(index_iter),
+ !skip_filters && !read_options.total_order_seek &&
+ prefix_extractor != nullptr,
+ need_upper_bound_check, prefix_extractor, caller,
+ compaction_readahead_size, allow_unprepared_value);
+ }
+}
+
+FragmentedRangeTombstoneIterator* BlockBasedTable::NewRangeTombstoneIterator(
+ const ReadOptions& read_options) {
+ if (rep_->fragmented_range_dels == nullptr) {
+ return nullptr;
+ }
+ SequenceNumber snapshot = kMaxSequenceNumber;
+ if (read_options.snapshot != nullptr) {
+ snapshot = read_options.snapshot->GetSequenceNumber();
+ }
+ return new FragmentedRangeTombstoneIterator(rep_->fragmented_range_dels,
+ rep_->internal_comparator,
+ snapshot, read_options.timestamp);
+}
+
+bool BlockBasedTable::FullFilterKeyMayMatch(
+ FilterBlockReader* filter, const Slice& internal_key, const bool no_io,
+ const SliceTransform* prefix_extractor, GetContext* get_context,
+ BlockCacheLookupContext* lookup_context,
+ Env::IOPriority rate_limiter_priority) const {
+ if (filter == nullptr) {
+ return true;
+ }
+ Slice user_key = ExtractUserKey(internal_key);
+ const Slice* const const_ikey_ptr = &internal_key;
+ bool may_match = true;
+ size_t ts_sz = rep_->internal_comparator.user_comparator()->timestamp_size();
+ Slice user_key_without_ts = StripTimestampFromUserKey(user_key, ts_sz);
+ if (rep_->whole_key_filtering) {
+ may_match =
+ filter->KeyMayMatch(user_key_without_ts, no_io, const_ikey_ptr,
+ get_context, lookup_context, rate_limiter_priority);
+ } else if (!PrefixExtractorChanged(prefix_extractor) &&
+ prefix_extractor->InDomain(user_key_without_ts) &&
+ !filter->PrefixMayMatch(
+ prefix_extractor->Transform(user_key_without_ts), no_io,
+ const_ikey_ptr, get_context, lookup_context,
+ rate_limiter_priority)) {
+ // FIXME ^^^: there should be no reason for Get() to depend on current
+ // prefix_extractor at all. It should always use table_prefix_extractor.
+ may_match = false;
+ }
+ if (may_match) {
+ RecordTick(rep_->ioptions.stats, BLOOM_FILTER_FULL_POSITIVE);
+ PERF_COUNTER_BY_LEVEL_ADD(bloom_filter_full_positive, 1, rep_->level);
+ }
+ return may_match;
+}
+
+void BlockBasedTable::FullFilterKeysMayMatch(
+ FilterBlockReader* filter, MultiGetRange* range, const bool no_io,
+ const SliceTransform* prefix_extractor,
+ BlockCacheLookupContext* lookup_context,
+ Env::IOPriority rate_limiter_priority) const {
+ if (filter == nullptr) {
+ return;
+ }
+ uint64_t before_keys = range->KeysLeft();
+ assert(before_keys > 0); // Caller should ensure
+ if (rep_->whole_key_filtering) {
+ filter->KeysMayMatch(range, no_io, lookup_context, rate_limiter_priority);
+ uint64_t after_keys = range->KeysLeft();
+ if (after_keys) {
+ RecordTick(rep_->ioptions.stats, BLOOM_FILTER_FULL_POSITIVE, after_keys);
+ PERF_COUNTER_BY_LEVEL_ADD(bloom_filter_full_positive, after_keys,
+ rep_->level);
+ }
+ uint64_t filtered_keys = before_keys - after_keys;
+ if (filtered_keys) {
+ RecordTick(rep_->ioptions.stats, BLOOM_FILTER_USEFUL, filtered_keys);
+ PERF_COUNTER_BY_LEVEL_ADD(bloom_filter_useful, filtered_keys,
+ rep_->level);
+ }
+ } else if (!PrefixExtractorChanged(prefix_extractor)) {
+ // FIXME ^^^: there should be no reason for MultiGet() to depend on current
+ // prefix_extractor at all. It should always use table_prefix_extractor.
+ filter->PrefixesMayMatch(range, prefix_extractor, false, lookup_context,
+ rate_limiter_priority);
+ RecordTick(rep_->ioptions.stats, BLOOM_FILTER_PREFIX_CHECKED, before_keys);
+ uint64_t after_keys = range->KeysLeft();
+ uint64_t filtered_keys = before_keys - after_keys;
+ if (filtered_keys) {
+ RecordTick(rep_->ioptions.stats, BLOOM_FILTER_PREFIX_USEFUL,
+ filtered_keys);
+ }
+ }
+}
+
+Status BlockBasedTable::ApproximateKeyAnchors(const ReadOptions& read_options,
+ std::vector<Anchor>& anchors) {
+ // We iterator the whole index block here. More efficient implementation
+ // is possible if we push this operation into IndexReader. For example, we
+ // can directly sample from restart block entries in the index block and
+ // only read keys needed. Here we take a simple solution. Performance is
+ // likely not to be a problem. We are compacting the whole file, so all
+ // keys will be read out anyway. An extra read to index block might be
+ // a small share of the overhead. We can try to optimize if needed.
+ IndexBlockIter iiter_on_stack;
+ auto iiter = NewIndexIterator(
+ read_options, /*disable_prefix_seek=*/false, &iiter_on_stack,
+ /*get_context=*/nullptr, /*lookup_context=*/nullptr);
+ std::unique_ptr<InternalIteratorBase<IndexValue>> iiter_unique_ptr;
+ if (iiter != &iiter_on_stack) {
+ iiter_unique_ptr.reset(iiter);
+ }
+
+ // If needed the threshold could be more adaptive. For example, it can be
+ // based on size, so that a larger will be sampled to more partitions than a
+ // smaller file. The size might also need to be passed in by the caller based
+ // on total compaction size.
+ const uint64_t kMaxNumAnchors = uint64_t{128};
+ uint64_t num_blocks = this->GetTableProperties()->num_data_blocks;
+ uint64_t num_blocks_per_anchor = num_blocks / kMaxNumAnchors;
+ if (num_blocks_per_anchor == 0) {
+ num_blocks_per_anchor = 1;
+ }
+
+ uint64_t count = 0;
+ std::string last_key;
+ uint64_t range_size = 0;
+ uint64_t prev_offset = 0;
+ for (iiter->SeekToFirst(); iiter->Valid(); iiter->Next()) {
+ const BlockHandle& bh = iiter->value().handle;
+ range_size += bh.offset() + bh.size() - prev_offset;
+ prev_offset = bh.offset() + bh.size();
+ if (++count % num_blocks_per_anchor == 0) {
+ count = 0;
+ anchors.emplace_back(iiter->user_key(), range_size);
+ range_size = 0;
+ } else {
+ last_key = iiter->user_key().ToString();
+ }
+ }
+ if (count != 0) {
+ anchors.emplace_back(last_key, range_size);
+ }
+ return Status::OK();
+}
+
+Status BlockBasedTable::Get(const ReadOptions& read_options, const Slice& key,
+ GetContext* get_context,
+ const SliceTransform* prefix_extractor,
+ bool skip_filters) {
+ assert(key.size() >= 8); // key must be internal key
+ assert(get_context != nullptr);
+ Status s;
+ const bool no_io = read_options.read_tier == kBlockCacheTier;
+
+ FilterBlockReader* const filter =
+ !skip_filters ? rep_->filter.get() : nullptr;
+
+ // First check the full filter
+ // If full filter not useful, Then go into each block
+ uint64_t tracing_get_id = get_context->get_tracing_get_id();
+ BlockCacheLookupContext lookup_context{
+ TableReaderCaller::kUserGet, tracing_get_id,
+ /*get_from_user_specified_snapshot=*/read_options.snapshot != nullptr};
+ if (block_cache_tracer_ && block_cache_tracer_->is_tracing_enabled()) {
+ // Trace the key since it contains both user key and sequence number.
+ lookup_context.referenced_key = key.ToString();
+ lookup_context.get_from_user_specified_snapshot =
+ read_options.snapshot != nullptr;
+ }
+ TEST_SYNC_POINT("BlockBasedTable::Get:BeforeFilterMatch");
+ const bool may_match = FullFilterKeyMayMatch(
+ filter, key, no_io, prefix_extractor, get_context, &lookup_context,
+ read_options.rate_limiter_priority);
+ TEST_SYNC_POINT("BlockBasedTable::Get:AfterFilterMatch");
+ if (!may_match) {
+ RecordTick(rep_->ioptions.stats, BLOOM_FILTER_USEFUL);
+ PERF_COUNTER_BY_LEVEL_ADD(bloom_filter_useful, 1, rep_->level);
+ } else {
+ IndexBlockIter iiter_on_stack;
+ // if prefix_extractor found in block differs from options, disable
+ // BlockPrefixIndex. Only do this check when index_type is kHashSearch.
+ bool need_upper_bound_check = false;
+ if (rep_->index_type == BlockBasedTableOptions::kHashSearch) {
+ need_upper_bound_check = PrefixExtractorChanged(prefix_extractor);
+ }
+ auto iiter =
+ NewIndexIterator(read_options, need_upper_bound_check, &iiter_on_stack,
+ get_context, &lookup_context);
+ std::unique_ptr<InternalIteratorBase<IndexValue>> iiter_unique_ptr;
+ if (iiter != &iiter_on_stack) {
+ iiter_unique_ptr.reset(iiter);
+ }
+
+ size_t ts_sz =
+ rep_->internal_comparator.user_comparator()->timestamp_size();
+ bool matched = false; // if such user key matched a key in SST
+ bool done = false;
+ for (iiter->Seek(key); iiter->Valid() && !done; iiter->Next()) {
+ IndexValue v = iiter->value();
+
+ if (!v.first_internal_key.empty() && !skip_filters &&
+ UserComparatorWrapper(rep_->internal_comparator.user_comparator())
+ .CompareWithoutTimestamp(
+ ExtractUserKey(key),
+ ExtractUserKey(v.first_internal_key)) < 0) {
+ // The requested key falls between highest key in previous block and
+ // lowest key in current block.
+ break;
+ }
+
+ BlockCacheLookupContext lookup_data_block_context{
+ TableReaderCaller::kUserGet, tracing_get_id,
+ /*get_from_user_specified_snapshot=*/read_options.snapshot !=
+ nullptr};
+ bool does_referenced_key_exist = false;
+ DataBlockIter biter;
+ uint64_t referenced_data_size = 0;
+ Status tmp_status;
+ NewDataBlockIterator<DataBlockIter>(
+ read_options, v.handle, &biter, BlockType::kData, get_context,
+ &lookup_data_block_context, /*prefetch_buffer=*/nullptr,
+ /*for_compaction=*/false, /*async_read=*/false, tmp_status);
+
+ if (no_io && biter.status().IsIncomplete()) {
+ // couldn't get block from block_cache
+ // Update Saver.state to Found because we are only looking for
+ // whether we can guarantee the key is not there when "no_io" is set
+ get_context->MarkKeyMayExist();
+ s = biter.status();
+ break;
+ }
+ if (!biter.status().ok()) {
+ s = biter.status();
+ break;
+ }
+
+ bool may_exist = biter.SeekForGet(key);
+ // If user-specified timestamp is supported, we cannot end the search
+ // just because hash index lookup indicates the key+ts does not exist.
+ if (!may_exist && ts_sz == 0) {
+ // HashSeek cannot find the key this block and the the iter is not
+ // the end of the block, i.e. cannot be in the following blocks
+ // either. In this case, the seek_key cannot be found, so we break
+ // from the top level for-loop.
+ done = true;
+ } else {
+ // Call the *saver function on each entry/block until it returns false
+ for (; biter.Valid(); biter.Next()) {
+ ParsedInternalKey parsed_key;
+ Status pik_status = ParseInternalKey(
+ biter.key(), &parsed_key, false /* log_err_key */); // TODO
+ if (!pik_status.ok()) {
+ s = pik_status;
+ }
+
+ if (!get_context->SaveValue(
+ parsed_key, biter.value(), &matched,
+ biter.IsValuePinned() ? &biter : nullptr)) {
+ if (get_context->State() == GetContext::GetState::kFound) {
+ does_referenced_key_exist = true;
+ referenced_data_size = biter.key().size() + biter.value().size();
+ }
+ done = true;
+ break;
+ }
+ }
+ s = biter.status();
+ }
+ // Write the block cache access record.
+ if (block_cache_tracer_ && block_cache_tracer_->is_tracing_enabled()) {
+ // Avoid making copy of block_key, cf_name, and referenced_key when
+ // constructing the access record.
+ Slice referenced_key;
+ if (does_referenced_key_exist) {
+ referenced_key = biter.key();
+ } else {
+ referenced_key = key;
+ }
+ BlockCacheTraceRecord access_record(
+ rep_->ioptions.clock->NowMicros(),
+ /*block_key=*/"", lookup_data_block_context.block_type,
+ lookup_data_block_context.block_size, rep_->cf_id_for_tracing(),
+ /*cf_name=*/"", rep_->level_for_tracing(),
+ rep_->sst_number_for_tracing(), lookup_data_block_context.caller,
+ lookup_data_block_context.is_cache_hit,
+ lookup_data_block_context.no_insert,
+ lookup_data_block_context.get_id,
+ lookup_data_block_context.get_from_user_specified_snapshot,
+ /*referenced_key=*/"", referenced_data_size,
+ lookup_data_block_context.num_keys_in_block,
+ does_referenced_key_exist);
+ // TODO: Should handle status here?
+ block_cache_tracer_
+ ->WriteBlockAccess(access_record,
+ lookup_data_block_context.block_key,
+ rep_->cf_name_for_tracing(), referenced_key)
+ .PermitUncheckedError();
+ }
+
+ if (done) {
+ // Avoid the extra Next which is expensive in two-level indexes
+ break;
+ }
+ }
+ if (matched && filter != nullptr) {
+ RecordTick(rep_->ioptions.stats, BLOOM_FILTER_FULL_TRUE_POSITIVE);
+ PERF_COUNTER_BY_LEVEL_ADD(bloom_filter_full_true_positive, 1,
+ rep_->level);
+ }
+ if (s.ok() && !iiter->status().IsNotFound()) {
+ s = iiter->status();
+ }
+ }
+
+ return s;
+}
+
+Status BlockBasedTable::MultiGetFilter(const ReadOptions& read_options,
+ const SliceTransform* prefix_extractor,
+ MultiGetRange* mget_range) {
+ if (mget_range->empty()) {
+ // Caller should ensure non-empty (performance bug)
+ assert(false);
+ return Status::OK(); // Nothing to do
+ }
+
+ FilterBlockReader* const filter = rep_->filter.get();
+ if (!filter) {
+ return Status::OK();
+ }
+
+ // First check the full filter
+ // If full filter not useful, Then go into each block
+ const bool no_io = read_options.read_tier == kBlockCacheTier;
+ uint64_t tracing_mget_id = BlockCacheTraceHelper::kReservedGetId;
+ if (mget_range->begin()->get_context) {
+ tracing_mget_id = mget_range->begin()->get_context->get_tracing_get_id();
+ }
+ BlockCacheLookupContext lookup_context{
+ TableReaderCaller::kUserMultiGet, tracing_mget_id,
+ /*_get_from_user_specified_snapshot=*/read_options.snapshot != nullptr};
+ FullFilterKeysMayMatch(filter, mget_range, no_io, prefix_extractor,
+ &lookup_context, read_options.rate_limiter_priority);
+
+ return Status::OK();
+}
+
+Status BlockBasedTable::Prefetch(const Slice* const begin,
+ const Slice* const end) {
+ auto& comparator = rep_->internal_comparator;
+ UserComparatorWrapper user_comparator(comparator.user_comparator());
+ // pre-condition
+ if (begin && end && comparator.Compare(*begin, *end) > 0) {
+ return Status::InvalidArgument(*begin, *end);
+ }
+ BlockCacheLookupContext lookup_context{TableReaderCaller::kPrefetch};
+ IndexBlockIter iiter_on_stack;
+ auto iiter = NewIndexIterator(ReadOptions(), /*need_upper_bound_check=*/false,
+ &iiter_on_stack, /*get_context=*/nullptr,
+ &lookup_context);
+ std::unique_ptr<InternalIteratorBase<IndexValue>> iiter_unique_ptr;
+ if (iiter != &iiter_on_stack) {
+ iiter_unique_ptr = std::unique_ptr<InternalIteratorBase<IndexValue>>(iiter);
+ }
+
+ if (!iiter->status().ok()) {
+ // error opening index iterator
+ return iiter->status();
+ }
+
+ // indicates if we are on the last page that need to be pre-fetched
+ bool prefetching_boundary_page = false;
+
+ for (begin ? iiter->Seek(*begin) : iiter->SeekToFirst(); iiter->Valid();
+ iiter->Next()) {
+ BlockHandle block_handle = iiter->value().handle;
+ const bool is_user_key = !rep_->index_key_includes_seq;
+ if (end &&
+ ((!is_user_key && comparator.Compare(iiter->key(), *end) >= 0) ||
+ (is_user_key &&
+ user_comparator.Compare(iiter->key(), ExtractUserKey(*end)) >= 0))) {
+ if (prefetching_boundary_page) {
+ break;
+ }
+
+ // The index entry represents the last key in the data block.
+ // We should load this page into memory as well, but no more
+ prefetching_boundary_page = true;
+ }
+
+ // Load the block specified by the block_handle into the block cache
+ DataBlockIter biter;
+ Status tmp_status;
+ NewDataBlockIterator<DataBlockIter>(
+ ReadOptions(), block_handle, &biter, /*type=*/BlockType::kData,
+ /*get_context=*/nullptr, &lookup_context,
+ /*prefetch_buffer=*/nullptr, /*for_compaction=*/false,
+ /*async_read=*/false, tmp_status);
+
+ if (!biter.status().ok()) {
+ // there was an unexpected error while pre-fetching
+ return biter.status();
+ }
+ }
+
+ return Status::OK();
+}
+
+Status BlockBasedTable::VerifyChecksum(const ReadOptions& read_options,
+ TableReaderCaller caller) {
+ Status s;
+ // Check Meta blocks
+ std::unique_ptr<Block> metaindex;
+ std::unique_ptr<InternalIterator> metaindex_iter;
+ ReadOptions ro;
+ s = ReadMetaIndexBlock(ro, nullptr /* prefetch buffer */, &metaindex,
+ &metaindex_iter);
+ if (s.ok()) {
+ s = VerifyChecksumInMetaBlocks(metaindex_iter.get());
+ if (!s.ok()) {
+ return s;
+ }
+ } else {
+ return s;
+ }
+ // Check Data blocks
+ IndexBlockIter iiter_on_stack;
+ BlockCacheLookupContext context{caller};
+ InternalIteratorBase<IndexValue>* iiter = NewIndexIterator(
+ read_options, /*disable_prefix_seek=*/false, &iiter_on_stack,
+ /*get_context=*/nullptr, &context);
+ std::unique_ptr<InternalIteratorBase<IndexValue>> iiter_unique_ptr;
+ if (iiter != &iiter_on_stack) {
+ iiter_unique_ptr = std::unique_ptr<InternalIteratorBase<IndexValue>>(iiter);
+ }
+ if (!iiter->status().ok()) {
+ // error opening index iterator
+ return iiter->status();
+ }
+ s = VerifyChecksumInBlocks(read_options, iiter);
+ return s;
+}
+
+Status BlockBasedTable::VerifyChecksumInBlocks(
+ const ReadOptions& read_options,
+ InternalIteratorBase<IndexValue>* index_iter) {
+ Status s;
+ // We are scanning the whole file, so no need to do exponential
+ // increasing of the buffer size.
+ size_t readahead_size = (read_options.readahead_size != 0)
+ ? read_options.readahead_size
+ : rep_->table_options.max_auto_readahead_size;
+ // FilePrefetchBuffer doesn't work in mmap mode and readahead is not
+ // needed there.
+ FilePrefetchBuffer prefetch_buffer(
+ readahead_size /* readahead_size */,
+ readahead_size /* max_readahead_size */,
+ !rep_->ioptions.allow_mmap_reads /* enable */);
+
+ for (index_iter->SeekToFirst(); index_iter->Valid(); index_iter->Next()) {
+ s = index_iter->status();
+ if (!s.ok()) {
+ break;
+ }
+ BlockHandle handle = index_iter->value().handle;
+ BlockContents contents;
+ BlockFetcher block_fetcher(
+ rep_->file.get(), &prefetch_buffer, rep_->footer, read_options, handle,
+ &contents, rep_->ioptions, false /* decompress */,
+ false /*maybe_compressed*/, BlockType::kData,
+ UncompressionDict::GetEmptyDict(), rep_->persistent_cache_options);
+ s = block_fetcher.ReadBlockContents();
+ if (!s.ok()) {
+ break;
+ }
+ }
+ if (s.ok()) {
+ // In the case of two level indexes, we would have exited the above loop
+ // by checking index_iter->Valid(), but Valid() might have returned false
+ // due to an IO error. So check the index_iter status
+ s = index_iter->status();
+ }
+ return s;
+}
+
+BlockType BlockBasedTable::GetBlockTypeForMetaBlockByName(
+ const Slice& meta_block_name) {
+ if (meta_block_name.starts_with(kFullFilterBlockPrefix)) {
+ return BlockType::kFilter;
+ }
+
+ if (meta_block_name.starts_with(kPartitionedFilterBlockPrefix)) {
+ return BlockType::kFilterPartitionIndex;
+ }
+
+ if (meta_block_name == kPropertiesBlockName) {
+ return BlockType::kProperties;
+ }
+
+ if (meta_block_name == kCompressionDictBlockName) {
+ return BlockType::kCompressionDictionary;
+ }
+
+ if (meta_block_name == kRangeDelBlockName) {
+ return BlockType::kRangeDeletion;
+ }
+
+ if (meta_block_name == kHashIndexPrefixesBlock) {
+ return BlockType::kHashIndexPrefixes;
+ }
+
+ if (meta_block_name == kHashIndexPrefixesMetadataBlock) {
+ return BlockType::kHashIndexMetadata;
+ }
+
+ if (meta_block_name.starts_with(kObsoleteFilterBlockPrefix)) {
+ // Obsolete but possible in old files
+ return BlockType::kInvalid;
+ }
+
+ assert(false);
+ return BlockType::kInvalid;
+}
+
+Status BlockBasedTable::VerifyChecksumInMetaBlocks(
+ InternalIteratorBase<Slice>* index_iter) {
+ Status s;
+ for (index_iter->SeekToFirst(); index_iter->Valid(); index_iter->Next()) {
+ s = index_iter->status();
+ if (!s.ok()) {
+ break;
+ }
+ BlockHandle handle;
+ Slice input = index_iter->value();
+ s = handle.DecodeFrom(&input);
+ BlockContents contents;
+ const Slice meta_block_name = index_iter->key();
+ if (meta_block_name == kPropertiesBlockName) {
+ // Unfortunate special handling for properties block checksum w/
+ // global seqno
+ std::unique_ptr<TableProperties> table_properties;
+ s = ReadTablePropertiesHelper(ReadOptions(), handle, rep_->file.get(),
+ nullptr /* prefetch_buffer */, rep_->footer,
+ rep_->ioptions, &table_properties,
+ nullptr /* memory_allocator */);
+ } else {
+ s = BlockFetcher(
+ rep_->file.get(), nullptr /* prefetch buffer */, rep_->footer,
+ ReadOptions(), handle, &contents, rep_->ioptions,
+ false /* decompress */, false /*maybe_compressed*/,
+ GetBlockTypeForMetaBlockByName(meta_block_name),
+ UncompressionDict::GetEmptyDict(), rep_->persistent_cache_options)
+ .ReadBlockContents();
+ }
+ if (!s.ok()) {
+ break;
+ }
+ }
+ return s;
+}
+
+bool BlockBasedTable::TEST_BlockInCache(const BlockHandle& handle) const {
+ assert(rep_ != nullptr);
+
+ Cache* const cache = rep_->table_options.block_cache.get();
+ if (cache == nullptr) {
+ return false;
+ }
+
+ CacheKey key = GetCacheKey(rep_->base_cache_key, handle);
+
+ Cache::Handle* const cache_handle = cache->Lookup(key.AsSlice());
+ if (cache_handle == nullptr) {
+ return false;
+ }
+
+ cache->Release(cache_handle);
+
+ return true;
+}
+
+bool BlockBasedTable::TEST_KeyInCache(const ReadOptions& options,
+ const Slice& key) {
+ std::unique_ptr<InternalIteratorBase<IndexValue>> iiter(NewIndexIterator(
+ options, /*need_upper_bound_check=*/false, /*input_iter=*/nullptr,
+ /*get_context=*/nullptr, /*lookup_context=*/nullptr));
+ iiter->Seek(key);
+ assert(iiter->Valid());
+
+ return TEST_BlockInCache(iiter->value().handle);
+}
+
+// REQUIRES: The following fields of rep_ should have already been populated:
+// 1. file
+// 2. index_handle,
+// 3. options
+// 4. internal_comparator
+// 5. index_type
+Status BlockBasedTable::CreateIndexReader(
+ const ReadOptions& ro, FilePrefetchBuffer* prefetch_buffer,
+ InternalIterator* meta_iter, bool use_cache, bool prefetch, bool pin,
+ BlockCacheLookupContext* lookup_context,
+ std::unique_ptr<IndexReader>* index_reader) {
+ switch (rep_->index_type) {
+ case BlockBasedTableOptions::kTwoLevelIndexSearch: {
+ return PartitionIndexReader::Create(this, ro, prefetch_buffer, use_cache,
+ prefetch, pin, lookup_context,
+ index_reader);
+ }
+ case BlockBasedTableOptions::kBinarySearch:
+ FALLTHROUGH_INTENDED;
+ case BlockBasedTableOptions::kBinarySearchWithFirstKey: {
+ return BinarySearchIndexReader::Create(this, ro, prefetch_buffer,
+ use_cache, prefetch, pin,
+ lookup_context, index_reader);
+ }
+ case BlockBasedTableOptions::kHashSearch: {
+ if (!rep_->table_prefix_extractor) {
+ ROCKS_LOG_WARN(rep_->ioptions.logger,
+ "Missing prefix extractor for hash index. Fall back to"
+ " binary search index.");
+ return BinarySearchIndexReader::Create(this, ro, prefetch_buffer,
+ use_cache, prefetch, pin,
+ lookup_context, index_reader);
+ } else {
+ return HashIndexReader::Create(this, ro, prefetch_buffer, meta_iter,
+ use_cache, prefetch, pin, lookup_context,
+ index_reader);
+ }
+ }
+ default: {
+ std::string error_message =
+ "Unrecognized index type: " + std::to_string(rep_->index_type);
+ return Status::InvalidArgument(error_message.c_str());
+ }
+ }
+}
+
+uint64_t BlockBasedTable::ApproximateDataOffsetOf(
+ const InternalIteratorBase<IndexValue>& index_iter,
+ uint64_t data_size) const {
+ assert(index_iter.status().ok());
+ if (index_iter.Valid()) {
+ BlockHandle handle = index_iter.value().handle;
+ return handle.offset();
+ } else {
+ // The iterator is past the last key in the file.
+ return data_size;
+ }
+}
+
+uint64_t BlockBasedTable::GetApproximateDataSize() {
+ // Should be in table properties unless super old version
+ if (rep_->table_properties) {
+ return rep_->table_properties->data_size;
+ }
+ // Fall back to rough estimate from footer
+ return rep_->footer.metaindex_handle().offset();
+}
+
+uint64_t BlockBasedTable::ApproximateOffsetOf(const Slice& key,
+ TableReaderCaller caller) {
+ uint64_t data_size = GetApproximateDataSize();
+ if (UNLIKELY(data_size == 0)) {
+ // Hmm. Let's just split in half to avoid skewing one way or another,
+ // since we don't know whether we're operating on lower bound or
+ // upper bound.
+ return rep_->file_size / 2;
+ }
+
+ BlockCacheLookupContext context(caller);
+ IndexBlockIter iiter_on_stack;
+ ReadOptions ro;
+ ro.total_order_seek = true;
+ auto index_iter =
+ NewIndexIterator(ro, /*disable_prefix_seek=*/true,
+ /*input_iter=*/&iiter_on_stack, /*get_context=*/nullptr,
+ /*lookup_context=*/&context);
+ std::unique_ptr<InternalIteratorBase<IndexValue>> iiter_unique_ptr;
+ if (index_iter != &iiter_on_stack) {
+ iiter_unique_ptr.reset(index_iter);
+ }
+
+ index_iter->Seek(key);
+ uint64_t offset;
+ if (index_iter->status().ok()) {
+ offset = ApproximateDataOffsetOf(*index_iter, data_size);
+ } else {
+ // Split in half to avoid skewing one way or another,
+ // since we don't know whether we're operating on lower bound or
+ // upper bound.
+ return rep_->file_size / 2;
+ }
+
+ // Pro-rate file metadata (incl filters) size-proportionally across data
+ // blocks.
+ double size_ratio =
+ static_cast<double>(offset) / static_cast<double>(data_size);
+ return static_cast<uint64_t>(size_ratio *
+ static_cast<double>(rep_->file_size));
+}
+
+uint64_t BlockBasedTable::ApproximateSize(const Slice& start, const Slice& end,
+ TableReaderCaller caller) {
+ assert(rep_->internal_comparator.Compare(start, end) <= 0);
+
+ uint64_t data_size = GetApproximateDataSize();
+ if (UNLIKELY(data_size == 0)) {
+ // Hmm. Assume whole file is involved, since we have lower and upper
+ // bound. This likely skews the estimate if we consider that this function
+ // is typically called with `[start, end]` fully contained in the file's
+ // key-range.
+ return rep_->file_size;
+ }
+
+ BlockCacheLookupContext context(caller);
+ IndexBlockIter iiter_on_stack;
+ ReadOptions ro;
+ ro.total_order_seek = true;
+ auto index_iter =
+ NewIndexIterator(ro, /*disable_prefix_seek=*/true,
+ /*input_iter=*/&iiter_on_stack, /*get_context=*/nullptr,
+ /*lookup_context=*/&context);
+ std::unique_ptr<InternalIteratorBase<IndexValue>> iiter_unique_ptr;
+ if (index_iter != &iiter_on_stack) {
+ iiter_unique_ptr.reset(index_iter);
+ }
+
+ index_iter->Seek(start);
+ uint64_t start_offset;
+ if (index_iter->status().ok()) {
+ start_offset = ApproximateDataOffsetOf(*index_iter, data_size);
+ } else {
+ // Assume file is involved from the start. This likely skews the estimate
+ // but is consistent with the above error handling.
+ start_offset = 0;
+ }
+
+ index_iter->Seek(end);
+ uint64_t end_offset;
+ if (index_iter->status().ok()) {
+ end_offset = ApproximateDataOffsetOf(*index_iter, data_size);
+ } else {
+ // Assume file is involved until the end. This likely skews the estimate
+ // but is consistent with the above error handling.
+ end_offset = data_size;
+ }
+
+ assert(end_offset >= start_offset);
+ // Pro-rate file metadata (incl filters) size-proportionally across data
+ // blocks.
+ double size_ratio = static_cast<double>(end_offset - start_offset) /
+ static_cast<double>(data_size);
+ return static_cast<uint64_t>(size_ratio *
+ static_cast<double>(rep_->file_size));
+}
+
+bool BlockBasedTable::TEST_FilterBlockInCache() const {
+ assert(rep_ != nullptr);
+ return rep_->filter_type != Rep::FilterType::kNoFilter &&
+ TEST_BlockInCache(rep_->filter_handle);
+}
+
+bool BlockBasedTable::TEST_IndexBlockInCache() const {
+ assert(rep_ != nullptr);
+
+ return TEST_BlockInCache(rep_->footer.index_handle());
+}
+
+Status BlockBasedTable::GetKVPairsFromDataBlocks(
+ std::vector<KVPairBlock>* kv_pair_blocks) {
+ std::unique_ptr<InternalIteratorBase<IndexValue>> blockhandles_iter(
+ NewIndexIterator(ReadOptions(), /*need_upper_bound_check=*/false,
+ /*input_iter=*/nullptr, /*get_context=*/nullptr,
+ /*lookup_contex=*/nullptr));
+
+ Status s = blockhandles_iter->status();
+ if (!s.ok()) {
+ // Cannot read Index Block
+ return s;
+ }
+
+ for (blockhandles_iter->SeekToFirst(); blockhandles_iter->Valid();
+ blockhandles_iter->Next()) {
+ s = blockhandles_iter->status();
+
+ if (!s.ok()) {
+ break;
+ }
+
+ std::unique_ptr<InternalIterator> datablock_iter;
+ Status tmp_status;
+ datablock_iter.reset(NewDataBlockIterator<DataBlockIter>(
+ ReadOptions(), blockhandles_iter->value().handle,
+ /*input_iter=*/nullptr, /*type=*/BlockType::kData,
+ /*get_context=*/nullptr, /*lookup_context=*/nullptr,
+ /*prefetch_buffer=*/nullptr, /*for_compaction=*/false,
+ /*async_read=*/false, tmp_status));
+ s = datablock_iter->status();
+
+ if (!s.ok()) {
+ // Error reading the block - Skipped
+ continue;
+ }
+
+ KVPairBlock kv_pair_block;
+ for (datablock_iter->SeekToFirst(); datablock_iter->Valid();
+ datablock_iter->Next()) {
+ s = datablock_iter->status();
+ if (!s.ok()) {
+ // Error reading the block - Skipped
+ break;
+ }
+ const Slice& key = datablock_iter->key();
+ const Slice& value = datablock_iter->value();
+ std::string key_copy = std::string(key.data(), key.size());
+ std::string value_copy = std::string(value.data(), value.size());
+
+ kv_pair_block.push_back(
+ std::make_pair(std::move(key_copy), std::move(value_copy)));
+ }
+ kv_pair_blocks->push_back(std::move(kv_pair_block));
+ }
+ return Status::OK();
+}
+
+Status BlockBasedTable::DumpTable(WritableFile* out_file) {
+ WritableFileStringStreamAdapter out_file_wrapper(out_file);
+ std::ostream out_stream(&out_file_wrapper);
+ // Output Footer
+ out_stream << "Footer Details:\n"
+ "--------------------------------------\n";
+ out_stream << " " << rep_->footer.ToString() << "\n";
+
+ // Output MetaIndex
+ out_stream << "Metaindex Details:\n"
+ "--------------------------------------\n";
+ std::unique_ptr<Block> metaindex;
+ std::unique_ptr<InternalIterator> metaindex_iter;
+ ReadOptions ro;
+ Status s = ReadMetaIndexBlock(ro, nullptr /* prefetch_buffer */, &metaindex,
+ &metaindex_iter);
+ if (s.ok()) {
+ for (metaindex_iter->SeekToFirst(); metaindex_iter->Valid();
+ metaindex_iter->Next()) {
+ s = metaindex_iter->status();
+ if (!s.ok()) {
+ return s;
+ }
+ if (metaindex_iter->key() == kPropertiesBlockName) {
+ out_stream << " Properties block handle: "
+ << metaindex_iter->value().ToString(true) << "\n";
+ } else if (metaindex_iter->key() == kCompressionDictBlockName) {
+ out_stream << " Compression dictionary block handle: "
+ << metaindex_iter->value().ToString(true) << "\n";
+ } else if (strstr(metaindex_iter->key().ToString().c_str(),
+ "filter.rocksdb.") != nullptr) {
+ out_stream << " Filter block handle: "
+ << metaindex_iter->value().ToString(true) << "\n";
+ } else if (metaindex_iter->key() == kRangeDelBlockName) {
+ out_stream << " Range deletion block handle: "
+ << metaindex_iter->value().ToString(true) << "\n";
+ }
+ }
+ out_stream << "\n";
+ } else {
+ return s;
+ }
+
+ // Output TableProperties
+ const ROCKSDB_NAMESPACE::TableProperties* table_properties;
+ table_properties = rep_->table_properties.get();
+
+ if (table_properties != nullptr) {
+ out_stream << "Table Properties:\n"
+ "--------------------------------------\n";
+ out_stream << " " << table_properties->ToString("\n ", ": ") << "\n";
+ }
+
+ if (rep_->filter) {
+ out_stream << "Filter Details:\n"
+ "--------------------------------------\n";
+ out_stream << " " << rep_->filter->ToString() << "\n";
+ }
+
+ // Output Index block
+ s = DumpIndexBlock(out_stream);
+ if (!s.ok()) {
+ return s;
+ }
+
+ // Output compression dictionary
+ if (rep_->uncompression_dict_reader) {
+ CachableEntry<UncompressionDict> uncompression_dict;
+ s = rep_->uncompression_dict_reader->GetOrReadUncompressionDictionary(
+ nullptr /* prefetch_buffer */, false /* no_io */,
+ false, /* verify_checksums */
+ nullptr /* get_context */, nullptr /* lookup_context */,
+ &uncompression_dict);
+ if (!s.ok()) {
+ return s;
+ }
+
+ assert(uncompression_dict.GetValue());
+
+ const Slice& raw_dict = uncompression_dict.GetValue()->GetRawDict();
+ out_stream << "Compression Dictionary:\n"
+ "--------------------------------------\n";
+ out_stream << " size (bytes): " << raw_dict.size() << "\n\n";
+ out_stream << " HEX " << raw_dict.ToString(true) << "\n\n";
+ }
+
+ // Output range deletions block
+ auto* range_del_iter = NewRangeTombstoneIterator(ReadOptions());
+ if (range_del_iter != nullptr) {
+ range_del_iter->SeekToFirst();
+ if (range_del_iter->Valid()) {
+ out_stream << "Range deletions:\n"
+ "--------------------------------------\n";
+ for (; range_del_iter->Valid(); range_del_iter->Next()) {
+ DumpKeyValue(range_del_iter->key(), range_del_iter->value(),
+ out_stream);
+ }
+ out_stream << "\n";
+ }
+ delete range_del_iter;
+ }
+ // Output Data blocks
+ s = DumpDataBlocks(out_stream);
+
+ if (!s.ok()) {
+ return s;
+ }
+
+ if (!out_stream.good()) {
+ return Status::IOError("Failed to write to output file");
+ }
+ return Status::OK();
+}
+
+Status BlockBasedTable::DumpIndexBlock(std::ostream& out_stream) {
+ out_stream << "Index Details:\n"
+ "--------------------------------------\n";
+ std::unique_ptr<InternalIteratorBase<IndexValue>> blockhandles_iter(
+ NewIndexIterator(ReadOptions(), /*need_upper_bound_check=*/false,
+ /*input_iter=*/nullptr, /*get_context=*/nullptr,
+ /*lookup_contex=*/nullptr));
+ Status s = blockhandles_iter->status();
+ if (!s.ok()) {
+ out_stream << "Can not read Index Block \n\n";
+ return s;
+ }
+
+ out_stream << " Block key hex dump: Data block handle\n";
+ out_stream << " Block key ascii\n\n";
+ for (blockhandles_iter->SeekToFirst(); blockhandles_iter->Valid();
+ blockhandles_iter->Next()) {
+ s = blockhandles_iter->status();
+ if (!s.ok()) {
+ break;
+ }
+ Slice key = blockhandles_iter->key();
+ Slice user_key;
+ InternalKey ikey;
+ if (!rep_->index_key_includes_seq) {
+ user_key = key;
+ } else {
+ ikey.DecodeFrom(key);
+ user_key = ikey.user_key();
+ }
+
+ out_stream << " HEX " << user_key.ToString(true) << ": "
+ << blockhandles_iter->value().ToString(true,
+ rep_->index_has_first_key)
+ << " offset " << blockhandles_iter->value().handle.offset()
+ << " size " << blockhandles_iter->value().handle.size() << "\n";
+
+ std::string str_key = user_key.ToString();
+ std::string res_key("");
+ char cspace = ' ';
+ for (size_t i = 0; i < str_key.size(); i++) {
+ res_key.append(&str_key[i], 1);
+ res_key.append(1, cspace);
+ }
+ out_stream << " ASCII " << res_key << "\n";
+ out_stream << " ------\n";
+ }
+ out_stream << "\n";
+ return Status::OK();
+}
+
+Status BlockBasedTable::DumpDataBlocks(std::ostream& out_stream) {
+ std::unique_ptr<InternalIteratorBase<IndexValue>> blockhandles_iter(
+ NewIndexIterator(ReadOptions(), /*need_upper_bound_check=*/false,
+ /*input_iter=*/nullptr, /*get_context=*/nullptr,
+ /*lookup_contex=*/nullptr));
+ Status s = blockhandles_iter->status();
+ if (!s.ok()) {
+ out_stream << "Can not read Index Block \n\n";
+ return s;
+ }
+
+ uint64_t datablock_size_min = std::numeric_limits<uint64_t>::max();
+ uint64_t datablock_size_max = 0;
+ uint64_t datablock_size_sum = 0;
+
+ size_t block_id = 1;
+ for (blockhandles_iter->SeekToFirst(); blockhandles_iter->Valid();
+ block_id++, blockhandles_iter->Next()) {
+ s = blockhandles_iter->status();
+ if (!s.ok()) {
+ break;
+ }
+
+ BlockHandle bh = blockhandles_iter->value().handle;
+ uint64_t datablock_size = bh.size();
+ datablock_size_min = std::min(datablock_size_min, datablock_size);
+ datablock_size_max = std::max(datablock_size_max, datablock_size);
+ datablock_size_sum += datablock_size;
+
+ out_stream << "Data Block # " << block_id << " @ "
+ << blockhandles_iter->value().handle.ToString(true) << "\n";
+ out_stream << "--------------------------------------\n";
+
+ std::unique_ptr<InternalIterator> datablock_iter;
+ Status tmp_status;
+ datablock_iter.reset(NewDataBlockIterator<DataBlockIter>(
+ ReadOptions(), blockhandles_iter->value().handle,
+ /*input_iter=*/nullptr, /*type=*/BlockType::kData,
+ /*get_context=*/nullptr, /*lookup_context=*/nullptr,
+ /*prefetch_buffer=*/nullptr, /*for_compaction=*/false,
+ /*async_read=*/false, tmp_status));
+ s = datablock_iter->status();
+
+ if (!s.ok()) {
+ out_stream << "Error reading the block - Skipped \n\n";
+ continue;
+ }
+
+ for (datablock_iter->SeekToFirst(); datablock_iter->Valid();
+ datablock_iter->Next()) {
+ s = datablock_iter->status();
+ if (!s.ok()) {
+ out_stream << "Error reading the block - Skipped \n";
+ break;
+ }
+ DumpKeyValue(datablock_iter->key(), datablock_iter->value(), out_stream);
+ }
+ out_stream << "\n";
+ }
+
+ uint64_t num_datablocks = block_id - 1;
+ if (num_datablocks) {
+ double datablock_size_avg =
+ static_cast<double>(datablock_size_sum) / num_datablocks;
+ out_stream << "Data Block Summary:\n";
+ out_stream << "--------------------------------------\n";
+ out_stream << " # data blocks: " << num_datablocks << "\n";
+ out_stream << " min data block size: " << datablock_size_min << "\n";
+ out_stream << " max data block size: " << datablock_size_max << "\n";
+ out_stream << " avg data block size: "
+ << std::to_string(datablock_size_avg) << "\n";
+ }
+
+ return Status::OK();
+}
+
+void BlockBasedTable::DumpKeyValue(const Slice& key, const Slice& value,
+ std::ostream& out_stream) {
+ InternalKey ikey;
+ ikey.DecodeFrom(key);
+
+ out_stream << " HEX " << ikey.user_key().ToString(true) << ": "
+ << value.ToString(true) << "\n";
+
+ std::string str_key = ikey.user_key().ToString();
+ std::string str_value = value.ToString();
+ std::string res_key(""), res_value("");
+ char cspace = ' ';
+ for (size_t i = 0; i < str_key.size(); i++) {
+ if (str_key[i] == '\0') {
+ res_key.append("\\0", 2);
+ } else {
+ res_key.append(&str_key[i], 1);
+ }
+ res_key.append(1, cspace);
+ }
+ for (size_t i = 0; i < str_value.size(); i++) {
+ if (str_value[i] == '\0') {
+ res_value.append("\\0", 2);
+ } else {
+ res_value.append(&str_value[i], 1);
+ }
+ res_value.append(1, cspace);
+ }
+
+ out_stream << " ASCII " << res_key << ": " << res_value << "\n";
+ out_stream << " ------\n";
+}
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/block_based_table_reader.h b/src/rocksdb/table/block_based/block_based_table_reader.h
new file mode 100644
index 000000000..89de891c9
--- /dev/null
+++ b/src/rocksdb/table/block_based/block_based_table_reader.h
@@ -0,0 +1,739 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#pragma once
+
+#include <cstdint>
+#include <memory>
+
+#include "cache/cache_entry_roles.h"
+#include "cache/cache_key.h"
+#include "cache/cache_reservation_manager.h"
+#include "db/range_tombstone_fragmenter.h"
+#include "file/filename.h"
+#include "rocksdb/slice_transform.h"
+#include "rocksdb/table_properties.h"
+#include "table/block_based/block.h"
+#include "table/block_based/block_based_table_factory.h"
+#include "table/block_based/block_type.h"
+#include "table/block_based/cachable_entry.h"
+#include "table/block_based/filter_block.h"
+#include "table/block_based/uncompression_dict_reader.h"
+#include "table/format.h"
+#include "table/persistent_cache_options.h"
+#include "table/table_properties_internal.h"
+#include "table/table_reader.h"
+#include "table/two_level_iterator.h"
+#include "trace_replay/block_cache_tracer.h"
+#include "util/coro_utils.h"
+#include "util/hash_containers.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class Cache;
+class FilterBlockReader;
+class FullFilterBlockReader;
+class Footer;
+class InternalKeyComparator;
+class Iterator;
+class FSRandomAccessFile;
+class TableCache;
+class TableReader;
+class WritableFile;
+struct BlockBasedTableOptions;
+struct EnvOptions;
+struct ReadOptions;
+class GetContext;
+
+using KVPairBlock = std::vector<std::pair<std::string, std::string>>;
+
+// Reader class for BlockBasedTable format.
+// For the format of BlockBasedTable refer to
+// https://github.com/facebook/rocksdb/wiki/Rocksdb-BlockBasedTable-Format.
+// This is the default table type. Data is chucked into fixed size blocks and
+// each block in-turn stores entries. When storing data, we can compress and/or
+// encode data efficiently within a block, which often results in a much smaller
+// data size compared with the raw data size. As for the record retrieval, we'll
+// first locate the block where target record may reside, then read the block to
+// memory, and finally search that record within the block. Of course, to avoid
+// frequent reads of the same block, we introduced the block cache to keep the
+// loaded blocks in the memory.
+class BlockBasedTable : public TableReader {
+ public:
+ static const std::string kObsoleteFilterBlockPrefix;
+ static const std::string kFullFilterBlockPrefix;
+ static const std::string kPartitionedFilterBlockPrefix;
+
+ // 1-byte compression type + 32-bit checksum
+ static constexpr size_t kBlockTrailerSize = 5;
+
+ // Attempt to open the table that is stored in bytes [0..file_size)
+ // of "file", and read the metadata entries necessary to allow
+ // retrieving data from the table.
+ //
+ // If successful, returns ok and sets "*table_reader" to the newly opened
+ // table. The client should delete "*table_reader" when no longer needed.
+ // If there was an error while initializing the table, sets "*table_reader"
+ // to nullptr and returns a non-ok status.
+ //
+ // @param file must remain live while this Table is in use.
+ // @param prefetch_index_and_filter_in_cache can be used to disable
+ // prefetching of
+ // index and filter blocks into block cache at startup
+ // @param skip_filters Disables loading/accessing the filter block. Overrides
+ // prefetch_index_and_filter_in_cache, so filter will be skipped if both
+ // are set.
+ // @param force_direct_prefetch if true, always prefetching to RocksDB
+ // buffer, rather than calling RandomAccessFile::Prefetch().
+ static Status Open(
+ const ReadOptions& ro, const ImmutableOptions& ioptions,
+ const EnvOptions& env_options,
+ const BlockBasedTableOptions& table_options,
+ const InternalKeyComparator& internal_key_comparator,
+ std::unique_ptr<RandomAccessFileReader>&& file, uint64_t file_size,
+ std::unique_ptr<TableReader>* table_reader,
+ std::shared_ptr<CacheReservationManager> table_reader_cache_res_mgr =
+ nullptr,
+ const std::shared_ptr<const SliceTransform>& prefix_extractor = nullptr,
+ bool prefetch_index_and_filter_in_cache = true, bool skip_filters = false,
+ int level = -1, const bool immortal_table = false,
+ const SequenceNumber largest_seqno = 0,
+ bool force_direct_prefetch = false,
+ TailPrefetchStats* tail_prefetch_stats = nullptr,
+ BlockCacheTracer* const block_cache_tracer = nullptr,
+ size_t max_file_size_for_l0_meta_pin = 0,
+ const std::string& cur_db_session_id = "", uint64_t cur_file_num = 0,
+ UniqueId64x2 expected_unique_id = {});
+
+ bool PrefixRangeMayMatch(const Slice& internal_key,
+ const ReadOptions& read_options,
+ const SliceTransform* options_prefix_extractor,
+ const bool need_upper_bound_check,
+ BlockCacheLookupContext* lookup_context) const;
+
+ // Returns a new iterator over the table contents.
+ // The result of NewIterator() is initially invalid (caller must
+ // call one of the Seek methods on the iterator before using it).
+ // @param read_options Must outlive the returned iterator.
+ // @param skip_filters Disables loading/accessing the filter block
+ // compaction_readahead_size: its value will only be used if caller =
+ // kCompaction.
+ InternalIterator* NewIterator(const ReadOptions&,
+ const SliceTransform* prefix_extractor,
+ Arena* arena, bool skip_filters,
+ TableReaderCaller caller,
+ size_t compaction_readahead_size = 0,
+ bool allow_unprepared_value = false) override;
+
+ FragmentedRangeTombstoneIterator* 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, const SliceTransform* prefix_extractor,
+ bool skip_filters = false) override;
+
+ Status MultiGetFilter(const ReadOptions& read_options,
+ const SliceTransform* prefix_extractor,
+ MultiGetRange* mget_range) override;
+
+ DECLARE_SYNC_AND_ASYNC_OVERRIDE(void, MultiGet,
+ const ReadOptions& readOptions,
+ const MultiGetContext::Range* mget_range,
+ const SliceTransform* prefix_extractor,
+ bool skip_filters = false);
+
+ // Pre-fetch the disk blocks that correspond to the key range specified by
+ // (kbegin, kend). The call will return error status in the event of
+ // IO or iteration error.
+ Status Prefetch(const Slice* begin, const Slice* end) override;
+
+ // 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
+ // bytes, and so includes effects like compression of the underlying data.
+ // E.g., the approximate offset of the last key in the table will
+ // be close to the file length.
+ uint64_t ApproximateOffsetOf(const Slice& key,
+ TableReaderCaller caller) override;
+
+ // Given start and end keys, return the approximate data size in the file
+ // between the keys. The returned value is in terms of file bytes, and so
+ // includes effects like compression of the underlying data.
+ // The start key must not be greater than the end key.
+ uint64_t ApproximateSize(const Slice& start, const Slice& end,
+ TableReaderCaller caller) override;
+
+ Status ApproximateKeyAnchors(const ReadOptions& read_options,
+ std::vector<Anchor>& anchors) override;
+
+ bool TEST_BlockInCache(const BlockHandle& handle) const;
+
+ // Returns true if the block for the specified key is in cache.
+ // REQUIRES: key is in this table && block cache enabled
+ bool TEST_KeyInCache(const ReadOptions& options, const Slice& key);
+
+ // Set up the table for Compaction. Might change some parameters with
+ // posix_fadvise
+ void SetupForCompaction() override;
+
+ std::shared_ptr<const TableProperties> GetTableProperties() const override;
+
+ size_t ApproximateMemoryUsage() const override;
+
+ // convert SST file to a human readable form
+ Status DumpTable(WritableFile* out_file) override;
+
+ Status VerifyChecksum(const ReadOptions& readOptions,
+ TableReaderCaller caller) override;
+
+ ~BlockBasedTable();
+
+ bool TEST_FilterBlockInCache() const;
+ bool TEST_IndexBlockInCache() const;
+
+ // IndexReader is the interface that provides the functionality for index
+ // access.
+ class IndexReader {
+ public:
+ virtual ~IndexReader() = default;
+
+ // Create an iterator for index access. If iter is null, then a new object
+ // is created on the heap, and the callee will have the ownership.
+ // If a non-null iter is passed in, it will be used, and the returned value
+ // is either the same as iter or a new on-heap object that
+ // wraps the passed iter. In the latter case the return value points
+ // to a different object then iter, and the callee has the ownership of the
+ // returned object.
+ virtual InternalIteratorBase<IndexValue>* NewIterator(
+ const ReadOptions& read_options, bool disable_prefix_seek,
+ IndexBlockIter* iter, GetContext* get_context,
+ BlockCacheLookupContext* lookup_context) = 0;
+
+ // Report an approximation of how much memory has been used other than
+ // memory that was allocated in block cache.
+ virtual size_t ApproximateMemoryUsage() const = 0;
+ // Cache the dependencies of the index reader (e.g. the partitions
+ // of a partitioned index).
+ virtual Status CacheDependencies(const ReadOptions& /*ro*/,
+ bool /* pin */) {
+ return Status::OK();
+ }
+ };
+
+ class IndexReaderCommon;
+
+ static void SetupBaseCacheKey(const TableProperties* properties,
+ const std::string& cur_db_session_id,
+ uint64_t cur_file_number,
+ OffsetableCacheKey* out_base_cache_key,
+ bool* out_is_stable = nullptr);
+
+ static CacheKey GetCacheKey(const OffsetableCacheKey& base_cache_key,
+ const BlockHandle& handle);
+
+ static void UpdateCacheInsertionMetrics(BlockType block_type,
+ GetContext* get_context, size_t usage,
+ bool redundant,
+ Statistics* const statistics);
+
+ // Get the size to read from storage for a BlockHandle. size_t because we
+ // are about to load into memory.
+ static inline size_t BlockSizeWithTrailer(const BlockHandle& handle) {
+ return static_cast<size_t>(handle.size() + kBlockTrailerSize);
+ }
+
+ // It is the caller's responsibility to make sure that this is called with
+ // block-based table serialized block contents, which contains the compression
+ // byte in the trailer after `block_size`.
+ static inline CompressionType GetBlockCompressionType(const char* block_data,
+ size_t block_size) {
+ return static_cast<CompressionType>(block_data[block_size]);
+ }
+ static inline CompressionType GetBlockCompressionType(
+ const BlockContents& contents) {
+ assert(contents.has_trailer);
+ return GetBlockCompressionType(contents.data.data(), contents.data.size());
+ }
+
+ // Retrieve all key value pairs from data blocks in the table.
+ // The key retrieved are internal keys.
+ Status GetKVPairsFromDataBlocks(std::vector<KVPairBlock>* kv_pair_blocks);
+
+ struct Rep;
+
+ Rep* get_rep() { return rep_; }
+ const Rep* get_rep() const { return rep_; }
+
+ // input_iter: if it is not null, update this one and return it as Iterator
+ template <typename TBlockIter>
+ TBlockIter* NewDataBlockIterator(const ReadOptions& ro,
+ const BlockHandle& block_handle,
+ TBlockIter* input_iter, BlockType block_type,
+ GetContext* get_context,
+ BlockCacheLookupContext* lookup_context,
+ FilePrefetchBuffer* prefetch_buffer,
+ bool for_compaction, bool async_read,
+ Status& s) const;
+
+ // input_iter: if it is not null, update this one and return it as Iterator
+ template <typename TBlockIter>
+ TBlockIter* NewDataBlockIterator(const ReadOptions& ro,
+ CachableEntry<Block>& block,
+ TBlockIter* input_iter, Status s) const;
+
+ class PartitionedIndexIteratorState;
+
+ template <typename TBlocklike>
+ friend class FilterBlockReaderCommon;
+
+ friend class PartitionIndexReader;
+
+ friend class UncompressionDictReader;
+
+ protected:
+ Rep* rep_;
+ explicit BlockBasedTable(Rep* rep, BlockCacheTracer* const block_cache_tracer)
+ : rep_(rep), block_cache_tracer_(block_cache_tracer) {}
+ // No copying allowed
+ explicit BlockBasedTable(const TableReader&) = delete;
+ void operator=(const TableReader&) = delete;
+
+ private:
+ friend class MockedBlockBasedTable;
+ friend class BlockBasedTableReaderTestVerifyChecksum_ChecksumMismatch_Test;
+ BlockCacheTracer* const block_cache_tracer_;
+
+ void UpdateCacheHitMetrics(BlockType block_type, GetContext* get_context,
+ size_t usage) const;
+ void UpdateCacheMissMetrics(BlockType block_type,
+ GetContext* get_context) const;
+
+ Cache::Handle* GetEntryFromCache(const CacheTier& cache_tier,
+ Cache* block_cache, const Slice& key,
+ BlockType block_type, const bool wait,
+ GetContext* get_context,
+ const Cache::CacheItemHelper* cache_helper,
+ const Cache::CreateCallback& create_cb,
+ Cache::Priority priority) const;
+
+ template <typename TBlocklike>
+ Status InsertEntryToCache(const CacheTier& cache_tier, Cache* block_cache,
+ const Slice& key,
+ const Cache::CacheItemHelper* cache_helper,
+ std::unique_ptr<TBlocklike>&& block_holder,
+ size_t charge, Cache::Handle** cache_handle,
+ Cache::Priority priority) const;
+
+ // Either Block::NewDataIterator() or Block::NewIndexIterator().
+ template <typename TBlockIter>
+ static TBlockIter* InitBlockIterator(const Rep* rep, Block* block,
+ BlockType block_type,
+ TBlockIter* input_iter,
+ bool block_contents_pinned);
+
+ // If block cache enabled (compressed or uncompressed), looks for the block
+ // identified by handle in (1) uncompressed cache, (2) compressed cache, and
+ // then (3) file. If found, inserts into the cache(s) that were searched
+ // unsuccessfully (e.g., if found in file, will add to both uncompressed and
+ // compressed caches if they're enabled).
+ //
+ // @param block_entry value is set to the uncompressed block if found. If
+ // in uncompressed block cache, also sets cache_handle to reference that
+ // block.
+ template <typename TBlocklike>
+ Status MaybeReadBlockAndLoadToCache(
+ FilePrefetchBuffer* prefetch_buffer, const ReadOptions& ro,
+ const BlockHandle& handle, const UncompressionDict& uncompression_dict,
+ const bool wait, const bool for_compaction,
+ CachableEntry<TBlocklike>* block_entry, BlockType block_type,
+ GetContext* get_context, BlockCacheLookupContext* lookup_context,
+ BlockContents* contents, bool async_read) const;
+
+ // Similar to the above, with one crucial difference: it will retrieve the
+ // block from the file even if there are no caches configured (assuming the
+ // read options allow I/O).
+ template <typename TBlocklike>
+ Status RetrieveBlock(FilePrefetchBuffer* prefetch_buffer,
+ const ReadOptions& ro, const BlockHandle& handle,
+ const UncompressionDict& uncompression_dict,
+ CachableEntry<TBlocklike>* block_entry,
+ BlockType block_type, GetContext* get_context,
+ BlockCacheLookupContext* lookup_context,
+ bool for_compaction, bool use_cache, bool wait_for_cache,
+ bool async_read) const;
+
+ DECLARE_SYNC_AND_ASYNC_CONST(
+ void, RetrieveMultipleBlocks, const ReadOptions& options,
+ const MultiGetRange* batch,
+ const autovector<BlockHandle, MultiGetContext::MAX_BATCH_SIZE>* handles,
+ autovector<Status, MultiGetContext::MAX_BATCH_SIZE>* statuses,
+ autovector<CachableEntry<Block>, MultiGetContext::MAX_BATCH_SIZE>*
+ results,
+ char* scratch, const UncompressionDict& uncompression_dict);
+
+ // Get the iterator from the index reader.
+ //
+ // If input_iter is not set, return a new Iterator.
+ // If input_iter is set, try to update it and return it as Iterator.
+ // However note that in some cases the returned iterator may be different
+ // from input_iter. In such case the returned iterator should be freed.
+ //
+ // Note: ErrorIterator with Status::Incomplete shall be returned if all the
+ // following conditions are met:
+ // 1. We enabled table_options.cache_index_and_filter_blocks.
+ // 2. index is not present in block cache.
+ // 3. We disallowed any io to be performed, that is, read_options ==
+ // kBlockCacheTier
+ InternalIteratorBase<IndexValue>* NewIndexIterator(
+ const ReadOptions& read_options, bool need_upper_bound_check,
+ IndexBlockIter* input_iter, GetContext* get_context,
+ BlockCacheLookupContext* lookup_context) const;
+
+ // Read block cache from block caches (if set): block_cache and
+ // block_cache_compressed.
+ // On success, Status::OK with be returned and @block will be populated with
+ // pointer to the block as well as its block handle.
+ // @param uncompression_dict Data for presetting the compression library's
+ // dictionary.
+ template <typename TBlocklike>
+ Status GetDataBlockFromCache(const Slice& cache_key, Cache* block_cache,
+ Cache* block_cache_compressed,
+ const ReadOptions& read_options,
+ CachableEntry<TBlocklike>* block,
+ const UncompressionDict& uncompression_dict,
+ BlockType block_type, const bool wait,
+ GetContext* get_context) const;
+
+ // Put a maybe compressed block to the corresponding block caches.
+ // This method will perform decompression against block_contents if needed
+ // and then populate the block caches.
+ // On success, Status::OK will be returned; also @block will be populated with
+ // uncompressed block and its cache handle.
+ //
+ // Allocated memory managed by block_contents will be transferred to
+ // PutDataBlockToCache(). After the call, the object will be invalid.
+ // @param uncompression_dict Data for presetting the compression library's
+ // dictionary.
+ template <typename TBlocklike>
+ Status PutDataBlockToCache(const Slice& cache_key, Cache* block_cache,
+ Cache* block_cache_compressed,
+ CachableEntry<TBlocklike>* cached_block,
+ BlockContents&& block_contents,
+ CompressionType block_comp_type,
+ const UncompressionDict& uncompression_dict,
+ MemoryAllocator* memory_allocator,
+ BlockType block_type,
+ GetContext* get_context) const;
+
+ // Calls (*handle_result)(arg, ...) repeatedly, starting with the entry found
+ // after a call to Seek(key), until handle_result returns false.
+ // May not make such a call if filter policy says that key is not present.
+ friend class TableCache;
+ friend class BlockBasedTableBuilder;
+
+ // Create a index reader based on the index type stored in the table.
+ // Optionally, user can pass a preloaded meta_index_iter for the index that
+ // need to access extra meta blocks for index construction. This parameter
+ // helps avoid re-reading meta index block if caller already created one.
+ Status CreateIndexReader(const ReadOptions& ro,
+ FilePrefetchBuffer* prefetch_buffer,
+ InternalIterator* preloaded_meta_index_iter,
+ bool use_cache, bool prefetch, bool pin,
+ BlockCacheLookupContext* lookup_context,
+ std::unique_ptr<IndexReader>* index_reader);
+
+ bool FullFilterKeyMayMatch(FilterBlockReader* filter, const Slice& user_key,
+ const bool no_io,
+ const SliceTransform* prefix_extractor,
+ GetContext* get_context,
+ BlockCacheLookupContext* lookup_context,
+ Env::IOPriority rate_limiter_priority) const;
+
+ void FullFilterKeysMayMatch(FilterBlockReader* filter, MultiGetRange* range,
+ const bool no_io,
+ const SliceTransform* prefix_extractor,
+ BlockCacheLookupContext* lookup_context,
+ Env::IOPriority rate_limiter_priority) const;
+
+ // If force_direct_prefetch is true, always prefetching to RocksDB
+ // buffer, rather than calling RandomAccessFile::Prefetch().
+ static Status PrefetchTail(
+ const ReadOptions& ro, RandomAccessFileReader* file, uint64_t file_size,
+ bool force_direct_prefetch, TailPrefetchStats* tail_prefetch_stats,
+ const bool prefetch_all, const bool preload_all,
+ std::unique_ptr<FilePrefetchBuffer>* prefetch_buffer);
+ Status ReadMetaIndexBlock(const ReadOptions& ro,
+ FilePrefetchBuffer* prefetch_buffer,
+ std::unique_ptr<Block>* metaindex_block,
+ std::unique_ptr<InternalIterator>* iter);
+ Status ReadPropertiesBlock(const ReadOptions& ro,
+ FilePrefetchBuffer* prefetch_buffer,
+ InternalIterator* meta_iter,
+ const SequenceNumber largest_seqno);
+ Status ReadRangeDelBlock(const ReadOptions& ro,
+ FilePrefetchBuffer* prefetch_buffer,
+ InternalIterator* meta_iter,
+ const InternalKeyComparator& internal_comparator,
+ BlockCacheLookupContext* lookup_context);
+ Status PrefetchIndexAndFilterBlocks(
+ const ReadOptions& ro, FilePrefetchBuffer* prefetch_buffer,
+ InternalIterator* meta_iter, BlockBasedTable* new_table,
+ bool prefetch_all, const BlockBasedTableOptions& table_options,
+ const int level, size_t file_size, size_t max_file_size_for_l0_meta_pin,
+ BlockCacheLookupContext* lookup_context);
+
+ static BlockType GetBlockTypeForMetaBlockByName(const Slice& meta_block_name);
+
+ Status VerifyChecksumInMetaBlocks(InternalIteratorBase<Slice>* index_iter);
+ Status VerifyChecksumInBlocks(const ReadOptions& read_options,
+ InternalIteratorBase<IndexValue>* index_iter);
+
+ // Create the filter from the filter block.
+ std::unique_ptr<FilterBlockReader> CreateFilterBlockReader(
+ const ReadOptions& ro, FilePrefetchBuffer* prefetch_buffer,
+ bool use_cache, bool prefetch, bool pin,
+ BlockCacheLookupContext* lookup_context);
+
+ // Size of all data blocks, maybe approximate
+ uint64_t GetApproximateDataSize();
+
+ // Given an iterator return its offset in data block section of file.
+ uint64_t ApproximateDataOffsetOf(
+ const InternalIteratorBase<IndexValue>& index_iter,
+ uint64_t data_size) const;
+
+ // Helper functions for DumpTable()
+ Status DumpIndexBlock(std::ostream& out_stream);
+ Status DumpDataBlocks(std::ostream& out_stream);
+ void DumpKeyValue(const Slice& key, const Slice& value,
+ std::ostream& out_stream);
+
+ // Returns false if prefix_extractor exists and is compatible with that used
+ // in building the table file, otherwise true.
+ bool PrefixExtractorChanged(const SliceTransform* prefix_extractor) const;
+
+ // A cumulative data block file read in MultiGet lower than this size will
+ // use a stack buffer
+ static constexpr size_t kMultiGetReadStackBufSize = 8192;
+
+ friend class PartitionedFilterBlockReader;
+ friend class PartitionedFilterBlockTest;
+ friend class DBBasicTest_MultiGetIOBufferOverrun_Test;
+};
+
+// Maintaining state of a two-level iteration on a partitioned index structure.
+class BlockBasedTable::PartitionedIndexIteratorState
+ : public TwoLevelIteratorState {
+ public:
+ PartitionedIndexIteratorState(
+ const BlockBasedTable* table,
+ UnorderedMap<uint64_t, CachableEntry<Block>>* block_map);
+ InternalIteratorBase<IndexValue>* NewSecondaryIterator(
+ const BlockHandle& index_value) override;
+
+ private:
+ // Don't own table_
+ const BlockBasedTable* table_;
+ UnorderedMap<uint64_t, CachableEntry<Block>>* block_map_;
+};
+
+// Stores all the properties associated with a BlockBasedTable.
+// These are immutable.
+struct BlockBasedTable::Rep {
+ Rep(const ImmutableOptions& _ioptions, const EnvOptions& _env_options,
+ const BlockBasedTableOptions& _table_opt,
+ const InternalKeyComparator& _internal_comparator, bool skip_filters,
+ uint64_t _file_size, int _level, const bool _immortal_table)
+ : ioptions(_ioptions),
+ env_options(_env_options),
+ table_options(_table_opt),
+ filter_policy(skip_filters ? nullptr : _table_opt.filter_policy.get()),
+ internal_comparator(_internal_comparator),
+ filter_type(FilterType::kNoFilter),
+ index_type(BlockBasedTableOptions::IndexType::kBinarySearch),
+ whole_key_filtering(_table_opt.whole_key_filtering),
+ prefix_filtering(true),
+ global_seqno(kDisableGlobalSequenceNumber),
+ file_size(_file_size),
+ level(_level),
+ immortal_table(_immortal_table) {}
+ ~Rep() { status.PermitUncheckedError(); }
+ const ImmutableOptions& ioptions;
+ const EnvOptions& env_options;
+ const BlockBasedTableOptions table_options;
+ const FilterPolicy* const filter_policy;
+ const InternalKeyComparator& internal_comparator;
+ Status status;
+ std::unique_ptr<RandomAccessFileReader> file;
+ OffsetableCacheKey base_cache_key;
+ PersistentCacheOptions persistent_cache_options;
+
+ // Footer contains the fixed table information
+ Footer footer;
+
+ std::unique_ptr<IndexReader> index_reader;
+ std::unique_ptr<FilterBlockReader> filter;
+ std::unique_ptr<UncompressionDictReader> uncompression_dict_reader;
+
+ enum class FilterType {
+ kNoFilter,
+ kFullFilter,
+ kPartitionedFilter,
+ };
+ FilterType filter_type;
+ BlockHandle filter_handle;
+ BlockHandle compression_dict_handle;
+
+ std::shared_ptr<const TableProperties> table_properties;
+ BlockBasedTableOptions::IndexType index_type;
+ bool whole_key_filtering;
+ bool prefix_filtering;
+ std::shared_ptr<const SliceTransform> table_prefix_extractor;
+
+ std::shared_ptr<FragmentedRangeTombstoneList> fragmented_range_dels;
+
+ // If global_seqno is used, all Keys in this file will have the same
+ // seqno with value `global_seqno`.
+ //
+ // A value of kDisableGlobalSequenceNumber means that this feature is disabled
+ // and every key have it's own seqno.
+ SequenceNumber global_seqno;
+
+ // Size of the table file on disk
+ uint64_t file_size;
+
+ // the level when the table is opened, could potentially change when trivial
+ // move is involved
+ int level;
+
+ // If false, blocks in this file are definitely all uncompressed. Knowing this
+ // before reading individual blocks enables certain optimizations.
+ bool blocks_maybe_compressed = true;
+
+ // If true, data blocks in this file are definitely ZSTD compressed. If false
+ // they might not be. When false we skip creating a ZSTD digested
+ // uncompression dictionary. Even if we get a false negative, things should
+ // still work, just not as quickly.
+ bool blocks_definitely_zstd_compressed = false;
+
+ // These describe how index is encoded.
+ bool index_has_first_key = false;
+ bool index_key_includes_seq = true;
+ bool index_value_is_full = true;
+
+ const bool immortal_table;
+
+ std::unique_ptr<CacheReservationManager::CacheReservationHandle>
+ table_reader_cache_res_handle = nullptr;
+
+ SequenceNumber get_global_seqno(BlockType block_type) const {
+ return (block_type == BlockType::kFilterPartitionIndex ||
+ block_type == BlockType::kCompressionDictionary)
+ ? kDisableGlobalSequenceNumber
+ : global_seqno;
+ }
+
+ uint64_t cf_id_for_tracing() const {
+ return table_properties
+ ? table_properties->column_family_id
+ : ROCKSDB_NAMESPACE::TablePropertiesCollectorFactory::Context::
+ kUnknownColumnFamily;
+ }
+
+ Slice cf_name_for_tracing() const {
+ return table_properties ? table_properties->column_family_name
+ : BlockCacheTraceHelper::kUnknownColumnFamilyName;
+ }
+
+ uint32_t level_for_tracing() const { return level >= 0 ? level : UINT32_MAX; }
+
+ uint64_t sst_number_for_tracing() const {
+ return file ? TableFileNameToNumber(file->file_name()) : UINT64_MAX;
+ }
+ void CreateFilePrefetchBuffer(
+ size_t readahead_size, size_t max_readahead_size,
+ std::unique_ptr<FilePrefetchBuffer>* fpb, bool implicit_auto_readahead,
+ uint64_t num_file_reads,
+ uint64_t num_file_reads_for_auto_readahead) const {
+ fpb->reset(new FilePrefetchBuffer(
+ readahead_size, max_readahead_size,
+ !ioptions.allow_mmap_reads /* enable */, false /* track_min_offset */,
+ implicit_auto_readahead, num_file_reads,
+ num_file_reads_for_auto_readahead, ioptions.fs.get(), ioptions.clock,
+ ioptions.stats));
+ }
+
+ void CreateFilePrefetchBufferIfNotExists(
+ size_t readahead_size, size_t max_readahead_size,
+ std::unique_ptr<FilePrefetchBuffer>* fpb, bool implicit_auto_readahead,
+ uint64_t num_file_reads,
+ uint64_t num_file_reads_for_auto_readahead) const {
+ if (!(*fpb)) {
+ CreateFilePrefetchBuffer(readahead_size, max_readahead_size, fpb,
+ implicit_auto_readahead, num_file_reads,
+ num_file_reads_for_auto_readahead);
+ }
+ }
+
+ std::size_t ApproximateMemoryUsage() const {
+ std::size_t usage = 0;
+#ifdef ROCKSDB_MALLOC_USABLE_SIZE
+ usage += malloc_usable_size(const_cast<BlockBasedTable::Rep*>(this));
+#else
+ usage += sizeof(*this);
+#endif // ROCKSDB_MALLOC_USABLE_SIZE
+ return usage;
+ }
+};
+
+// This is an adapter class for `WritableFile` to be used for `std::ostream`.
+// The adapter wraps a `WritableFile`, which can be passed to a `std::ostream`
+// constructor for storing streaming data.
+// Note:
+// * This adapter doesn't provide any buffering, each write is forwarded to
+// `WritableFile->Append()` directly.
+// * For a failed write, the user needs to check the status by `ostream.good()`
+class WritableFileStringStreamAdapter : public std::stringbuf {
+ public:
+ explicit WritableFileStringStreamAdapter(WritableFile* writable_file)
+ : file_(writable_file) {}
+
+ // Override overflow() to handle `sputc()`. There are cases that will not go
+ // through `xsputn()` e.g. `std::endl` or an unsigned long long is written by
+ // `os.put()` directly and will call `sputc()` By internal implementation:
+ // int_type __CLR_OR_THIS_CALL sputc(_Elem _Ch) { // put a character
+ // return 0 < _Pnavail() ? _Traits::to_int_type(*_Pninc() = _Ch) :
+ // overflow(_Traits::to_int_type(_Ch));
+ // }
+ // As we explicitly disabled buffering (_Pnavail() is always 0), every write,
+ // not captured by xsputn(), becomes an overflow here.
+ int overflow(int ch = EOF) override {
+ if (ch != EOF) {
+ Status s = file_->Append(Slice((char*)&ch, 1));
+ if (s.ok()) {
+ return ch;
+ }
+ }
+ return EOF;
+ }
+
+ std::streamsize xsputn(char const* p, std::streamsize n) override {
+ Status s = file_->Append(Slice(p, n));
+ if (!s.ok()) {
+ return 0;
+ }
+ return n;
+ }
+
+ private:
+ WritableFile* file_;
+};
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/block_based_table_reader_impl.h b/src/rocksdb/table/block_based/block_based_table_reader_impl.h
new file mode 100644
index 000000000..1f6f5f223
--- /dev/null
+++ b/src/rocksdb/table/block_based/block_based_table_reader_impl.h
@@ -0,0 +1,171 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+#pragma once
+#include "table/block_based/block_based_table_reader.h"
+#include "table/block_based/reader_common.h"
+
+// The file contains some member functions of BlockBasedTable that
+// cannot be implemented in block_based_table_reader.cc because
+// it's called by other files (e.g. block_based_iterator.h) and
+// are templates.
+
+namespace ROCKSDB_NAMESPACE {
+// Convert an index iterator value (i.e., an encoded BlockHandle)
+// into an iterator over the contents of the corresponding block.
+// If input_iter is null, new a iterator
+// If input_iter is not null, update this iter and return it
+template <typename TBlockIter>
+TBlockIter* BlockBasedTable::NewDataBlockIterator(
+ const ReadOptions& ro, const BlockHandle& handle, TBlockIter* input_iter,
+ BlockType block_type, GetContext* get_context,
+ BlockCacheLookupContext* lookup_context,
+ FilePrefetchBuffer* prefetch_buffer, bool for_compaction, bool async_read,
+ Status& s) const {
+ PERF_TIMER_GUARD(new_table_block_iter_nanos);
+
+ TBlockIter* iter = input_iter != nullptr ? input_iter : new TBlockIter;
+ if (!s.ok()) {
+ iter->Invalidate(s);
+ return iter;
+ }
+
+ CachableEntry<Block> block;
+ if (rep_->uncompression_dict_reader && block_type == BlockType::kData) {
+ CachableEntry<UncompressionDict> uncompression_dict;
+ const bool no_io = (ro.read_tier == kBlockCacheTier);
+ s = rep_->uncompression_dict_reader->GetOrReadUncompressionDictionary(
+ prefetch_buffer, no_io, ro.verify_checksums, get_context,
+ lookup_context, &uncompression_dict);
+ if (!s.ok()) {
+ iter->Invalidate(s);
+ return iter;
+ }
+ const UncompressionDict& dict = uncompression_dict.GetValue()
+ ? *uncompression_dict.GetValue()
+ : UncompressionDict::GetEmptyDict();
+ s = RetrieveBlock(prefetch_buffer, ro, handle, dict, &block, block_type,
+ get_context, lookup_context, for_compaction,
+ /* use_cache */ true, /* wait_for_cache */ true,
+ async_read);
+ } else {
+ s = RetrieveBlock(
+ prefetch_buffer, ro, handle, UncompressionDict::GetEmptyDict(), &block,
+ block_type, get_context, lookup_context, for_compaction,
+ /* use_cache */ true, /* wait_for_cache */ true, async_read);
+ }
+
+ if (s.IsTryAgain() && async_read) {
+ return iter;
+ }
+
+ if (!s.ok()) {
+ assert(block.IsEmpty());
+ iter->Invalidate(s);
+ return iter;
+ }
+
+ assert(block.GetValue() != nullptr);
+
+ // Block contents are pinned and it is still pinned after the iterator
+ // is destroyed as long as cleanup functions are moved to another object,
+ // when:
+ // 1. block cache handle is set to be released in cleanup function, or
+ // 2. it's pointing to immortal source. If own_bytes is true then we are
+ // not reading data from the original source, whether immortal or not.
+ // Otherwise, the block is pinned iff the source is immortal.
+ const bool block_contents_pinned =
+ block.IsCached() ||
+ (!block.GetValue()->own_bytes() && rep_->immortal_table);
+ iter = InitBlockIterator<TBlockIter>(rep_, block.GetValue(), block_type, iter,
+ block_contents_pinned);
+
+ if (!block.IsCached()) {
+ if (!ro.fill_cache) {
+ Cache* const block_cache = rep_->table_options.block_cache.get();
+ if (block_cache) {
+ // insert a dummy record to block cache to track the memory usage
+ Cache::Handle* cache_handle = nullptr;
+ CacheKey key = CacheKey::CreateUniqueForCacheLifetime(block_cache);
+ s = block_cache->Insert(key.AsSlice(), nullptr,
+ block.GetValue()->ApproximateMemoryUsage(),
+ nullptr, &cache_handle);
+
+ if (s.ok()) {
+ assert(cache_handle != nullptr);
+ iter->RegisterCleanup(&ForceReleaseCachedEntry, block_cache,
+ cache_handle);
+ }
+ }
+ }
+ } else {
+ iter->SetCacheHandle(block.GetCacheHandle());
+ }
+
+ block.TransferTo(iter);
+
+ return iter;
+}
+
+// Convert an uncompressed data block (i.e CachableEntry<Block>)
+// into an iterator over the contents of the corresponding block.
+// If input_iter is null, new a iterator
+// If input_iter is not null, update this iter and return it
+template <typename TBlockIter>
+TBlockIter* BlockBasedTable::NewDataBlockIterator(const ReadOptions& ro,
+ CachableEntry<Block>& block,
+ TBlockIter* input_iter,
+ Status s) const {
+ PERF_TIMER_GUARD(new_table_block_iter_nanos);
+
+ TBlockIter* iter = input_iter != nullptr ? input_iter : new TBlockIter;
+ if (!s.ok()) {
+ iter->Invalidate(s);
+ return iter;
+ }
+
+ assert(block.GetValue() != nullptr);
+ // Block contents are pinned and it is still pinned after the iterator
+ // is destroyed as long as cleanup functions are moved to another object,
+ // when:
+ // 1. block cache handle is set to be released in cleanup function, or
+ // 2. it's pointing to immortal source. If own_bytes is true then we are
+ // not reading data from the original source, whether immortal or not.
+ // Otherwise, the block is pinned iff the source is immortal.
+ const bool block_contents_pinned =
+ block.IsCached() ||
+ (!block.GetValue()->own_bytes() && rep_->immortal_table);
+ iter = InitBlockIterator<TBlockIter>(rep_, block.GetValue(), BlockType::kData,
+ iter, block_contents_pinned);
+
+ if (!block.IsCached()) {
+ if (!ro.fill_cache) {
+ Cache* const block_cache = rep_->table_options.block_cache.get();
+ if (block_cache) {
+ // insert a dummy record to block cache to track the memory usage
+ Cache::Handle* cache_handle = nullptr;
+ CacheKey key = CacheKey::CreateUniqueForCacheLifetime(block_cache);
+ s = block_cache->Insert(key.AsSlice(), nullptr,
+ block.GetValue()->ApproximateMemoryUsage(),
+ nullptr, &cache_handle);
+
+ if (s.ok()) {
+ assert(cache_handle != nullptr);
+ iter->RegisterCleanup(&ForceReleaseCachedEntry, block_cache,
+ cache_handle);
+ }
+ }
+ }
+ } else {
+ iter->SetCacheHandle(block.GetCacheHandle());
+ }
+
+ block.TransferTo(iter);
+ return iter;
+}
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/block_based_table_reader_sync_and_async.h b/src/rocksdb/table/block_based/block_based_table_reader_sync_and_async.h
new file mode 100644
index 000000000..8c7547a2a
--- /dev/null
+++ b/src/rocksdb/table/block_based/block_based_table_reader_sync_and_async.h
@@ -0,0 +1,760 @@
+// Copyright (c) Meta Platforms, Inc. and affiliates.
+//
+// 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 "util/async_file_reader.h"
+#include "util/coro_utils.h"
+
+#if defined(WITHOUT_COROUTINES) || \
+ (defined(USE_COROUTINES) && defined(WITH_COROUTINES))
+
+namespace ROCKSDB_NAMESPACE {
+
+// This function reads multiple data blocks from disk using Env::MultiRead()
+// and optionally inserts them into the block cache. It uses the scratch
+// buffer provided by the caller, which is contiguous. If scratch is a nullptr
+// it allocates a separate buffer for each block. Typically, if the blocks
+// need to be uncompressed and there is no compressed block cache, callers
+// can allocate a temporary scratch buffer in order to minimize memory
+// allocations.
+// If options.fill_cache is true, it inserts the blocks into cache. If its
+// false and scratch is non-null and the blocks are uncompressed, it copies
+// the buffers to heap. In any case, the CachableEntry<Block> returned will
+// own the data bytes.
+// If compression is enabled and also there is no compressed block cache,
+// the adjacent blocks are read out in one IO (combined read)
+// batch - A MultiGetRange with only those keys with unique data blocks not
+// found in cache
+// handles - A vector of block handles. Some of them me be NULL handles
+// scratch - An optional contiguous buffer to read compressed blocks into
+DEFINE_SYNC_AND_ASYNC(void, BlockBasedTable::RetrieveMultipleBlocks)
+(const ReadOptions& options, const MultiGetRange* batch,
+ const autovector<BlockHandle, MultiGetContext::MAX_BATCH_SIZE>* handles,
+ autovector<Status, MultiGetContext::MAX_BATCH_SIZE>* statuses,
+ autovector<CachableEntry<Block>, MultiGetContext::MAX_BATCH_SIZE>* results,
+ char* scratch, const UncompressionDict& uncompression_dict) const {
+ RandomAccessFileReader* file = rep_->file.get();
+ const Footer& footer = rep_->footer;
+ const ImmutableOptions& ioptions = rep_->ioptions;
+ size_t read_amp_bytes_per_bit = rep_->table_options.read_amp_bytes_per_bit;
+ MemoryAllocator* memory_allocator = GetMemoryAllocator(rep_->table_options);
+
+ if (ioptions.allow_mmap_reads) {
+ size_t idx_in_batch = 0;
+ for (auto mget_iter = batch->begin(); mget_iter != batch->end();
+ ++mget_iter, ++idx_in_batch) {
+ BlockCacheLookupContext lookup_data_block_context(
+ TableReaderCaller::kUserMultiGet);
+ const BlockHandle& handle = (*handles)[idx_in_batch];
+ if (handle.IsNull()) {
+ continue;
+ }
+
+ (*statuses)[idx_in_batch] =
+ RetrieveBlock(nullptr, options, handle, uncompression_dict,
+ &(*results)[idx_in_batch], BlockType::kData,
+ mget_iter->get_context, &lookup_data_block_context,
+ /* for_compaction */ false, /* use_cache */ true,
+ /* wait_for_cache */ true, /* async_read */ false);
+ }
+ CO_RETURN;
+ }
+
+ // In direct IO mode, blocks share the direct io buffer.
+ // Otherwise, blocks share the scratch buffer.
+ const bool use_shared_buffer = file->use_direct_io() || scratch != nullptr;
+
+ autovector<FSReadRequest, MultiGetContext::MAX_BATCH_SIZE> read_reqs;
+ size_t buf_offset = 0;
+ size_t idx_in_batch = 0;
+
+ uint64_t prev_offset = 0;
+ size_t prev_len = 0;
+ autovector<size_t, MultiGetContext::MAX_BATCH_SIZE> req_idx_for_block;
+ autovector<size_t, MultiGetContext::MAX_BATCH_SIZE> req_offset_for_block;
+ for (auto mget_iter = batch->begin(); mget_iter != batch->end();
+ ++mget_iter, ++idx_in_batch) {
+ const BlockHandle& handle = (*handles)[idx_in_batch];
+ if (handle.IsNull()) {
+ continue;
+ }
+
+ size_t prev_end = static_cast<size_t>(prev_offset) + prev_len;
+
+ // If current block is adjacent to the previous one, at the same time,
+ // compression is enabled and there is no compressed cache, we combine
+ // the two block read as one.
+ // We don't combine block reads here in direct IO mode, because when doing
+ // direct IO read, the block requests will be realigned and merged when
+ // necessary.
+ if (use_shared_buffer && !file->use_direct_io() &&
+ prev_end == handle.offset()) {
+ req_offset_for_block.emplace_back(prev_len);
+ prev_len += BlockSizeWithTrailer(handle);
+ } else {
+ // No compression or current block and previous one is not adjacent:
+ // Step 1, create a new request for previous blocks
+ if (prev_len != 0) {
+ FSReadRequest req;
+ req.offset = prev_offset;
+ req.len = prev_len;
+ if (file->use_direct_io()) {
+ req.scratch = nullptr;
+ } else if (use_shared_buffer) {
+ req.scratch = scratch + buf_offset;
+ buf_offset += req.len;
+ } else {
+ req.scratch = new char[req.len];
+ }
+ read_reqs.emplace_back(req);
+ }
+
+ // Step 2, remeber the previous block info
+ prev_offset = handle.offset();
+ prev_len = BlockSizeWithTrailer(handle);
+ req_offset_for_block.emplace_back(0);
+ }
+ req_idx_for_block.emplace_back(read_reqs.size());
+
+ PERF_COUNTER_ADD(block_read_count, 1);
+ PERF_COUNTER_ADD(block_read_byte, BlockSizeWithTrailer(handle));
+ }
+ // Handle the last block and process the pending last request
+ if (prev_len != 0) {
+ FSReadRequest req;
+ req.offset = prev_offset;
+ req.len = prev_len;
+ if (file->use_direct_io()) {
+ req.scratch = nullptr;
+ } else if (use_shared_buffer) {
+ req.scratch = scratch + buf_offset;
+ } else {
+ req.scratch = new char[req.len];
+ }
+ read_reqs.emplace_back(req);
+ }
+
+ AlignedBuf direct_io_buf;
+ {
+ IOOptions opts;
+ IOStatus s = file->PrepareIOOptions(options, opts);
+ if (s.ok()) {
+#if defined(WITH_COROUTINES)
+ if (file->use_direct_io()) {
+#endif // WITH_COROUTINES
+ s = file->MultiRead(opts, &read_reqs[0], read_reqs.size(),
+ &direct_io_buf, options.rate_limiter_priority);
+#if defined(WITH_COROUTINES)
+ } else {
+ co_await batch->context()->reader().MultiReadAsync(
+ file, opts, &read_reqs[0], read_reqs.size(), &direct_io_buf);
+ }
+#endif // WITH_COROUTINES
+ }
+ if (!s.ok()) {
+ // Discard all the results in this batch if there is any time out
+ // or overall MultiRead error
+ for (FSReadRequest& req : read_reqs) {
+ req.status = s;
+ }
+ }
+ }
+
+ idx_in_batch = 0;
+ size_t valid_batch_idx = 0;
+ for (auto mget_iter = batch->begin(); mget_iter != batch->end();
+ ++mget_iter, ++idx_in_batch) {
+ const BlockHandle& handle = (*handles)[idx_in_batch];
+
+ if (handle.IsNull()) {
+ continue;
+ }
+
+ assert(valid_batch_idx < req_idx_for_block.size());
+ assert(valid_batch_idx < req_offset_for_block.size());
+ assert(req_idx_for_block[valid_batch_idx] < read_reqs.size());
+ size_t& req_idx = req_idx_for_block[valid_batch_idx];
+ size_t& req_offset = req_offset_for_block[valid_batch_idx];
+ valid_batch_idx++;
+ FSReadRequest& req = read_reqs[req_idx];
+ Status s = req.status;
+ if (s.ok()) {
+ if ((req.result.size() != req.len) ||
+ (req_offset + BlockSizeWithTrailer(handle) > req.result.size())) {
+ s = Status::Corruption("truncated block read from " +
+ rep_->file->file_name() + " offset " +
+ std::to_string(handle.offset()) + ", expected " +
+ std::to_string(req.len) + " bytes, got " +
+ std::to_string(req.result.size()));
+ }
+ }
+
+ BlockContents serialized_block;
+ if (s.ok()) {
+ if (!use_shared_buffer) {
+ // We allocated a buffer for this block. Give ownership of it to
+ // BlockContents so it can free the memory
+ assert(req.result.data() == req.scratch);
+ assert(req.result.size() == BlockSizeWithTrailer(handle));
+ assert(req_offset == 0);
+ serialized_block =
+ BlockContents(std::unique_ptr<char[]>(req.scratch), handle.size());
+ } else {
+ // We used the scratch buffer or direct io buffer
+ // which are shared by the blocks.
+ // serialized_block does not have the ownership.
+ serialized_block =
+ BlockContents(Slice(req.result.data() + req_offset, handle.size()));
+ }
+#ifndef NDEBUG
+ serialized_block.has_trailer = true;
+#endif
+
+ if (options.verify_checksums) {
+ PERF_TIMER_GUARD(block_checksum_time);
+ const char* data = req.result.data();
+ // Since the scratch might be shared, the offset of the data block in
+ // the buffer might not be 0. req.result.data() only point to the
+ // begin address of each read request, we need to add the offset
+ // in each read request. Checksum is stored in the block trailer,
+ // beyond the payload size.
+ s = VerifyBlockChecksum(footer.checksum_type(), data + req_offset,
+ handle.size(), rep_->file->file_name(),
+ handle.offset());
+ TEST_SYNC_POINT_CALLBACK("RetrieveMultipleBlocks:VerifyChecksum", &s);
+ }
+ } else if (!use_shared_buffer) {
+ // Free the allocated scratch buffer.
+ delete[] req.scratch;
+ }
+
+ if (s.ok()) {
+ // When the blocks share the same underlying buffer (scratch or direct io
+ // buffer), we may need to manually copy the block into heap if the
+ // serialized block has to be inserted into a cache. That falls into the
+ // following cases -
+ // 1. serialized block is not compressed, it needs to be inserted into
+ // the uncompressed block cache if there is one
+ // 2. If the serialized block is compressed, it needs to be inserted
+ // into the compressed block cache if there is one
+ //
+ // In all other cases, the serialized block is either uncompressed into a
+ // heap buffer or there is no cache at all.
+ CompressionType compression_type =
+ GetBlockCompressionType(serialized_block);
+ if (use_shared_buffer && (compression_type == kNoCompression ||
+ (compression_type != kNoCompression &&
+ rep_->table_options.block_cache_compressed))) {
+ Slice serialized =
+ Slice(req.result.data() + req_offset, BlockSizeWithTrailer(handle));
+ serialized_block = BlockContents(
+ CopyBufferToHeap(GetMemoryAllocator(rep_->table_options),
+ serialized),
+ handle.size());
+#ifndef NDEBUG
+ serialized_block.has_trailer = true;
+#endif
+ }
+ }
+
+ if (s.ok()) {
+ if (options.fill_cache) {
+ BlockCacheLookupContext lookup_data_block_context(
+ TableReaderCaller::kUserMultiGet);
+ CachableEntry<Block>* block_entry = &(*results)[idx_in_batch];
+ // MaybeReadBlockAndLoadToCache will insert into the block caches if
+ // necessary. Since we're passing the serialized block contents, it
+ // will avoid looking up the block cache
+ s = MaybeReadBlockAndLoadToCache(
+ nullptr, options, handle, uncompression_dict, /*wait=*/true,
+ /*for_compaction=*/false, block_entry, BlockType::kData,
+ mget_iter->get_context, &lookup_data_block_context,
+ &serialized_block, /*async_read=*/false);
+
+ // block_entry value could be null if no block cache is present, i.e
+ // BlockBasedTableOptions::no_block_cache is true and no compressed
+ // block cache is configured. In that case, fall
+ // through and set up the block explicitly
+ if (block_entry->GetValue() != nullptr) {
+ s.PermitUncheckedError();
+ continue;
+ }
+ }
+
+ CompressionType compression_type =
+ GetBlockCompressionType(serialized_block);
+ BlockContents contents;
+ if (compression_type != kNoCompression) {
+ UncompressionContext context(compression_type);
+ UncompressionInfo info(context, uncompression_dict, compression_type);
+ s = UncompressSerializedBlock(
+ info, req.result.data() + req_offset, handle.size(), &contents,
+ footer.format_version(), rep_->ioptions, memory_allocator);
+ } else {
+ // There are two cases here:
+ // 1) caller uses the shared buffer (scratch or direct io buffer);
+ // 2) we use the requst buffer.
+ // If scratch buffer or direct io buffer is used, we ensure that
+ // all serialized blocks are copyed to the heap as single blocks. If
+ // scratch buffer is not used, we also have no combined read, so the
+ // serialized block can be used directly.
+ contents = std::move(serialized_block);
+ }
+ if (s.ok()) {
+ (*results)[idx_in_batch].SetOwnedValue(std::make_unique<Block>(
+ std::move(contents), read_amp_bytes_per_bit, ioptions.stats));
+ }
+ }
+ (*statuses)[idx_in_batch] = s;
+ }
+}
+
+using MultiGetRange = MultiGetContext::Range;
+DEFINE_SYNC_AND_ASYNC(void, BlockBasedTable::MultiGet)
+(const ReadOptions& read_options, const MultiGetRange* mget_range,
+ const SliceTransform* prefix_extractor, bool skip_filters) {
+ if (mget_range->empty()) {
+ // Caller should ensure non-empty (performance bug)
+ assert(false);
+ CO_RETURN; // Nothing to do
+ }
+
+ FilterBlockReader* const filter =
+ !skip_filters ? rep_->filter.get() : nullptr;
+ MultiGetRange sst_file_range(*mget_range, mget_range->begin(),
+ mget_range->end());
+
+ // First check the full filter
+ // If full filter not useful, Then go into each block
+ const bool no_io = read_options.read_tier == kBlockCacheTier;
+ uint64_t tracing_mget_id = BlockCacheTraceHelper::kReservedGetId;
+ if (sst_file_range.begin()->get_context) {
+ tracing_mget_id = sst_file_range.begin()->get_context->get_tracing_get_id();
+ }
+ BlockCacheLookupContext lookup_context{
+ TableReaderCaller::kUserMultiGet, tracing_mget_id,
+ /*_get_from_user_specified_snapshot=*/read_options.snapshot != nullptr};
+ FullFilterKeysMayMatch(filter, &sst_file_range, no_io, prefix_extractor,
+ &lookup_context, read_options.rate_limiter_priority);
+
+ if (!sst_file_range.empty()) {
+ IndexBlockIter iiter_on_stack;
+ // if prefix_extractor found in block differs from options, disable
+ // BlockPrefixIndex. Only do this check when index_type is kHashSearch.
+ bool need_upper_bound_check = false;
+ if (rep_->index_type == BlockBasedTableOptions::kHashSearch) {
+ need_upper_bound_check = PrefixExtractorChanged(prefix_extractor);
+ }
+ auto iiter =
+ NewIndexIterator(read_options, need_upper_bound_check, &iiter_on_stack,
+ sst_file_range.begin()->get_context, &lookup_context);
+ std::unique_ptr<InternalIteratorBase<IndexValue>> iiter_unique_ptr;
+ if (iiter != &iiter_on_stack) {
+ iiter_unique_ptr.reset(iiter);
+ }
+
+ uint64_t prev_offset = std::numeric_limits<uint64_t>::max();
+ autovector<BlockHandle, MultiGetContext::MAX_BATCH_SIZE> block_handles;
+ autovector<CachableEntry<Block>, MultiGetContext::MAX_BATCH_SIZE> results;
+ autovector<Status, MultiGetContext::MAX_BATCH_SIZE> statuses;
+ MultiGetContext::Mask reused_mask = 0;
+ char stack_buf[kMultiGetReadStackBufSize];
+ std::unique_ptr<char[]> block_buf;
+ {
+ MultiGetRange data_block_range(sst_file_range, sst_file_range.begin(),
+ sst_file_range.end());
+ std::vector<Cache::Handle*> cache_handles;
+ bool wait_for_cache_results = false;
+
+ CachableEntry<UncompressionDict> uncompression_dict;
+ Status uncompression_dict_status;
+ uncompression_dict_status.PermitUncheckedError();
+ bool uncompression_dict_inited = false;
+ size_t total_len = 0;
+ ReadOptions ro = read_options;
+ ro.read_tier = kBlockCacheTier;
+
+ for (auto miter = data_block_range.begin();
+ miter != data_block_range.end(); ++miter) {
+ const Slice& key = miter->ikey;
+ iiter->Seek(miter->ikey);
+
+ IndexValue v;
+ if (iiter->Valid()) {
+ v = iiter->value();
+ }
+ if (!iiter->Valid() ||
+ (!v.first_internal_key.empty() && !skip_filters &&
+ UserComparatorWrapper(rep_->internal_comparator.user_comparator())
+ .CompareWithoutTimestamp(
+ ExtractUserKey(key),
+ ExtractUserKey(v.first_internal_key)) < 0)) {
+ // The requested key falls between highest key in previous block and
+ // lowest key in current block.
+ if (!iiter->status().IsNotFound()) {
+ *(miter->s) = iiter->status();
+ }
+ data_block_range.SkipKey(miter);
+ sst_file_range.SkipKey(miter);
+ continue;
+ }
+
+ if (!uncompression_dict_inited && rep_->uncompression_dict_reader) {
+ uncompression_dict_status =
+ rep_->uncompression_dict_reader->GetOrReadUncompressionDictionary(
+ nullptr /* prefetch_buffer */, no_io,
+ read_options.verify_checksums,
+ sst_file_range.begin()->get_context, &lookup_context,
+ &uncompression_dict);
+ uncompression_dict_inited = true;
+ }
+
+ if (!uncompression_dict_status.ok()) {
+ assert(!uncompression_dict_status.IsNotFound());
+ *(miter->s) = uncompression_dict_status;
+ data_block_range.SkipKey(miter);
+ sst_file_range.SkipKey(miter);
+ continue;
+ }
+
+ statuses.emplace_back();
+ results.emplace_back();
+ if (v.handle.offset() == prev_offset) {
+ // This key can reuse the previous block (later on).
+ // Mark previous as "reused"
+ reused_mask |= MultiGetContext::Mask{1} << (block_handles.size() - 1);
+ // Use null handle to indicate this one reuses same block as
+ // previous.
+ block_handles.emplace_back(BlockHandle::NullBlockHandle());
+ continue;
+ }
+ // Lookup the cache for the given data block referenced by an index
+ // iterator value (i.e BlockHandle). If it exists in the cache,
+ // initialize block to the contents of the data block.
+ prev_offset = v.handle.offset();
+ BlockHandle handle = v.handle;
+ BlockCacheLookupContext lookup_data_block_context(
+ TableReaderCaller::kUserMultiGet);
+ const UncompressionDict& dict = uncompression_dict.GetValue()
+ ? *uncompression_dict.GetValue()
+ : UncompressionDict::GetEmptyDict();
+ Status s = RetrieveBlock(
+ nullptr, ro, handle, dict, &(results.back()), BlockType::kData,
+ miter->get_context, &lookup_data_block_context,
+ /* for_compaction */ false, /* use_cache */ true,
+ /* wait_for_cache */ false, /* async_read */ false);
+ if (s.IsIncomplete()) {
+ s = Status::OK();
+ }
+ if (s.ok() && !results.back().IsEmpty()) {
+ // Since we have a valid handle, check the value. If its nullptr,
+ // it means the cache is waiting for the final result and we're
+ // supposed to call WaitAll() to wait for the result.
+ if (results.back().GetValue() != nullptr) {
+ // Found it in the cache. Add NULL handle to indicate there is
+ // nothing to read from disk.
+ if (results.back().GetCacheHandle()) {
+ results.back().UpdateCachedValue();
+ }
+ block_handles.emplace_back(BlockHandle::NullBlockHandle());
+ } else {
+ // We have to wait for the cache lookup to finish in the
+ // background, and then we may have to read the block from disk
+ // anyway
+ assert(results.back().GetCacheHandle());
+ wait_for_cache_results = true;
+ block_handles.emplace_back(handle);
+ cache_handles.emplace_back(results.back().GetCacheHandle());
+ }
+ } else {
+ block_handles.emplace_back(handle);
+ total_len += BlockSizeWithTrailer(handle);
+ }
+ }
+
+ if (wait_for_cache_results) {
+ Cache* block_cache = rep_->table_options.block_cache.get();
+ block_cache->WaitAll(cache_handles);
+ for (size_t i = 0; i < block_handles.size(); ++i) {
+ // If this block was a success or failure or not needed because
+ // the corresponding key is in the same block as a prior key, skip
+ if (block_handles[i] == BlockHandle::NullBlockHandle() ||
+ results[i].IsEmpty()) {
+ continue;
+ }
+ results[i].UpdateCachedValue();
+ void* val = results[i].GetValue();
+ Cache::Handle* handle = results[i].GetCacheHandle();
+ // GetContext for any key will do, as the stats will be aggregated
+ // anyway
+ GetContext* get_context = sst_file_range.begin()->get_context;
+ if (!val) {
+ // The async cache lookup failed - could be due to an error
+ // or a false positive. We need to read the data block from
+ // the SST file
+ results[i].Reset();
+ total_len += BlockSizeWithTrailer(block_handles[i]);
+ UpdateCacheMissMetrics(BlockType::kData, get_context);
+ } else {
+ block_handles[i] = BlockHandle::NullBlockHandle();
+ UpdateCacheHitMetrics(BlockType::kData, get_context,
+ block_cache->GetUsage(handle));
+ }
+ }
+ }
+
+ if (total_len) {
+ char* scratch = nullptr;
+ const UncompressionDict& dict = uncompression_dict.GetValue()
+ ? *uncompression_dict.GetValue()
+ : UncompressionDict::GetEmptyDict();
+ assert(uncompression_dict_inited || !rep_->uncompression_dict_reader);
+ assert(uncompression_dict_status.ok());
+ // If using direct IO, then scratch is not used, so keep it nullptr.
+ // If the blocks need to be uncompressed and we don't need the
+ // compressed blocks, then we can use a contiguous block of
+ // memory to read in all the blocks as it will be temporary
+ // storage
+ // 1. If blocks are compressed and compressed block cache is there,
+ // alloc heap bufs
+ // 2. If blocks are uncompressed, alloc heap bufs
+ // 3. If blocks are compressed and no compressed block cache, use
+ // stack buf
+ if (!rep_->file->use_direct_io() &&
+ rep_->table_options.block_cache_compressed == nullptr &&
+ rep_->blocks_maybe_compressed) {
+ if (total_len <= kMultiGetReadStackBufSize) {
+ scratch = stack_buf;
+ } else {
+ scratch = new char[total_len];
+ block_buf.reset(scratch);
+ }
+ }
+ CO_AWAIT(RetrieveMultipleBlocks)
+ (read_options, &data_block_range, &block_handles, &statuses, &results,
+ scratch, dict);
+ if (sst_file_range.begin()->get_context) {
+ ++(sst_file_range.begin()
+ ->get_context->get_context_stats_.num_sst_read);
+ }
+ }
+ }
+
+ DataBlockIter first_biter;
+ DataBlockIter next_biter;
+ size_t idx_in_batch = 0;
+ SharedCleanablePtr shared_cleanable;
+ for (auto miter = sst_file_range.begin(); miter != sst_file_range.end();
+ ++miter) {
+ Status s;
+ GetContext* get_context = miter->get_context;
+ const Slice& key = miter->ikey;
+ bool matched = false; // if such user key matched a key in SST
+ bool done = false;
+ bool first_block = true;
+ do {
+ DataBlockIter* biter = nullptr;
+ bool reusing_prev_block;
+ bool later_reused;
+ uint64_t referenced_data_size = 0;
+ bool does_referenced_key_exist = false;
+ BlockCacheLookupContext lookup_data_block_context(
+ TableReaderCaller::kUserMultiGet, tracing_mget_id,
+ /*_get_from_user_specified_snapshot=*/read_options.snapshot !=
+ nullptr);
+ if (first_block) {
+ if (!block_handles[idx_in_batch].IsNull() ||
+ !results[idx_in_batch].IsEmpty()) {
+ first_biter.Invalidate(Status::OK());
+ NewDataBlockIterator<DataBlockIter>(
+ read_options, results[idx_in_batch], &first_biter,
+ statuses[idx_in_batch]);
+ reusing_prev_block = false;
+ } else {
+ // If handler is null and result is empty, then the status is never
+ // set, which should be the initial value: ok().
+ assert(statuses[idx_in_batch].ok());
+ reusing_prev_block = true;
+ }
+ biter = &first_biter;
+ later_reused =
+ (reused_mask & (MultiGetContext::Mask{1} << idx_in_batch)) != 0;
+ idx_in_batch++;
+ } else {
+ IndexValue v = iiter->value();
+ if (!v.first_internal_key.empty() && !skip_filters &&
+ UserComparatorWrapper(rep_->internal_comparator.user_comparator())
+ .CompareWithoutTimestamp(
+ ExtractUserKey(key),
+ ExtractUserKey(v.first_internal_key)) < 0) {
+ // The requested key falls between highest key in previous block and
+ // lowest key in current block.
+ break;
+ }
+
+ next_biter.Invalidate(Status::OK());
+ Status tmp_s;
+ NewDataBlockIterator<DataBlockIter>(
+ read_options, iiter->value().handle, &next_biter,
+ BlockType::kData, get_context, &lookup_data_block_context,
+ /* prefetch_buffer= */ nullptr, /* for_compaction = */ false,
+ /*async_read = */ false, tmp_s);
+ biter = &next_biter;
+ reusing_prev_block = false;
+ later_reused = false;
+ }
+
+ if (read_options.read_tier == kBlockCacheTier &&
+ biter->status().IsIncomplete()) {
+ // couldn't get block from block_cache
+ // Update Saver.state to Found because we are only looking for
+ // whether we can guarantee the key is not there when "no_io" is set
+ get_context->MarkKeyMayExist();
+ break;
+ }
+ if (!biter->status().ok()) {
+ s = biter->status();
+ break;
+ }
+
+ // Reusing blocks complicates pinning/Cleanable, because the cache
+ // entry referenced by biter can only be released once all returned
+ // pinned values are released. This code previously did an extra
+ // block_cache Ref for each reuse, but that unnecessarily increases
+ // block cache contention. Instead we can use a variant of shared_ptr
+ // to release in block cache only once.
+ //
+ // Although the biter loop below might SaveValue multiple times for
+ // merges, just one value_pinner suffices, as MultiGet will merge
+ // the operands before returning to the API user.
+ Cleanable* value_pinner;
+ if (biter->IsValuePinned()) {
+ if (reusing_prev_block) {
+ // Note that we don't yet know if the MultiGet results will need
+ // to pin this block, so we might wrap a block for sharing and
+ // still end up with 1 (or 0) pinning ref. Not ideal but OK.
+ //
+ // Here we avoid adding redundant cleanups if we didn't end up
+ // delegating the cleanup from last time around.
+ if (!biter->HasCleanups()) {
+ assert(shared_cleanable.get());
+ if (later_reused) {
+ shared_cleanable.RegisterCopyWith(biter);
+ } else {
+ shared_cleanable.MoveAsCleanupTo(biter);
+ }
+ }
+ } else if (later_reused) {
+ assert(biter->HasCleanups());
+ // Make the existing cleanups on `biter` sharable:
+ shared_cleanable.Allocate();
+ // Move existing `biter` cleanup(s) to `shared_cleanable`
+ biter->DelegateCleanupsTo(&*shared_cleanable);
+ // Reference `shared_cleanable` as new cleanup for `biter`
+ shared_cleanable.RegisterCopyWith(biter);
+ }
+ assert(biter->HasCleanups());
+ value_pinner = biter;
+ } else {
+ value_pinner = nullptr;
+ }
+
+ bool may_exist = biter->SeekForGet(key);
+ if (!may_exist) {
+ // HashSeek cannot find the key this block and the the iter is not
+ // the end of the block, i.e. cannot be in the following blocks
+ // either. In this case, the seek_key cannot be found, so we break
+ // from the top level for-loop.
+ break;
+ }
+
+ // Call the *saver function on each entry/block until it returns false
+ for (; biter->Valid(); biter->Next()) {
+ ParsedInternalKey parsed_key;
+ Status pik_status = ParseInternalKey(
+ biter->key(), &parsed_key, false /* log_err_key */); // TODO
+ if (!pik_status.ok()) {
+ s = pik_status;
+ }
+ if (!get_context->SaveValue(parsed_key, biter->value(), &matched,
+ value_pinner)) {
+ if (get_context->State() == GetContext::GetState::kFound) {
+ does_referenced_key_exist = true;
+ referenced_data_size =
+ biter->key().size() + biter->value().size();
+ }
+ done = true;
+ break;
+ }
+ s = biter->status();
+ }
+ // Write the block cache access.
+ // XXX: There appear to be 'break' statements above that bypass this
+ // writing of the block cache trace record
+ if (block_cache_tracer_ && block_cache_tracer_->is_tracing_enabled() &&
+ !reusing_prev_block) {
+ // Avoid making copy of block_key, cf_name, and referenced_key when
+ // constructing the access record.
+ Slice referenced_key;
+ if (does_referenced_key_exist) {
+ referenced_key = biter->key();
+ } else {
+ referenced_key = key;
+ }
+ BlockCacheTraceRecord access_record(
+ rep_->ioptions.clock->NowMicros(),
+ /*_block_key=*/"", lookup_data_block_context.block_type,
+ lookup_data_block_context.block_size, rep_->cf_id_for_tracing(),
+ /*_cf_name=*/"", rep_->level_for_tracing(),
+ rep_->sst_number_for_tracing(), lookup_data_block_context.caller,
+ lookup_data_block_context.is_cache_hit,
+ lookup_data_block_context.no_insert,
+ lookup_data_block_context.get_id,
+ lookup_data_block_context.get_from_user_specified_snapshot,
+ /*_referenced_key=*/"", referenced_data_size,
+ lookup_data_block_context.num_keys_in_block,
+ does_referenced_key_exist);
+ // TODO: Should handle status here?
+ block_cache_tracer_
+ ->WriteBlockAccess(access_record,
+ lookup_data_block_context.block_key,
+ rep_->cf_name_for_tracing(), referenced_key)
+ .PermitUncheckedError();
+ }
+ s = biter->status();
+ if (done) {
+ // Avoid the extra Next which is expensive in two-level indexes
+ break;
+ }
+ if (first_block) {
+ iiter->Seek(key);
+ if (!iiter->Valid()) {
+ break;
+ }
+ }
+ first_block = false;
+ iiter->Next();
+ } while (iiter->Valid());
+
+ if (matched && filter != nullptr) {
+ RecordTick(rep_->ioptions.stats, BLOOM_FILTER_FULL_TRUE_POSITIVE);
+ PERF_COUNTER_BY_LEVEL_ADD(bloom_filter_full_true_positive, 1,
+ rep_->level);
+ }
+ if (s.ok() && !iiter->status().IsNotFound()) {
+ s = iiter->status();
+ }
+ *(miter->s) = s;
+ }
+#ifdef ROCKSDB_ASSERT_STATUS_CHECKED
+ // Not sure why we need to do it. Should investigate more.
+ for (auto& st : statuses) {
+ st.PermitUncheckedError();
+ }
+#endif // ROCKSDB_ASSERT_STATUS_CHECKED
+ }
+}
+} // namespace ROCKSDB_NAMESPACE
+#endif
diff --git a/src/rocksdb/table/block_based/block_based_table_reader_test.cc b/src/rocksdb/table/block_based/block_based_table_reader_test.cc
new file mode 100644
index 000000000..c5a615dfc
--- /dev/null
+++ b/src/rocksdb/table/block_based/block_based_table_reader_test.cc
@@ -0,0 +1,572 @@
+// Copyright (c) 2011-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 "table/block_based/block_based_table_reader.h"
+
+#include <cmath>
+#include <memory>
+#include <string>
+
+#include "cache/cache_reservation_manager.h"
+#include "db/db_test_util.h"
+#include "db/table_properties_collector.h"
+#include "file/file_util.h"
+#include "options/options_helper.h"
+#include "port/port.h"
+#include "port/stack_trace.h"
+#include "rocksdb/compression_type.h"
+#include "rocksdb/db.h"
+#include "rocksdb/file_system.h"
+#include "table/block_based/block_based_table_builder.h"
+#include "table/block_based/block_based_table_factory.h"
+#include "table/block_based/partitioned_index_iterator.h"
+#include "table/format.h"
+#include "test_util/testharness.h"
+#include "test_util/testutil.h"
+#include "util/random.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class BlockBasedTableReaderBaseTest : public testing::Test {
+ protected:
+ // Prepare key-value pairs to occupy multiple blocks.
+ // Each value is 256B, every 16 pairs constitute 1 block.
+ // If mixed_with_human_readable_string_value == true,
+ // then adjacent blocks contain values with different compression
+ // complexity: human readable strings are easier to compress than random
+ // strings.
+ static std::map<std::string, std::string> GenerateKVMap(
+ int num_block = 100,
+ bool mixed_with_human_readable_string_value = false) {
+ std::map<std::string, std::string> kv;
+
+ Random rnd(101);
+ uint32_t key = 0;
+ for (int block = 0; block < num_block; block++) {
+ for (int i = 0; i < 16; i++) {
+ char k[9] = {0};
+ // Internal key is constructed directly from this key,
+ // and internal key size is required to be >= 8 bytes,
+ // so use %08u as the format string.
+ sprintf(k, "%08u", key);
+ std::string v;
+ if (mixed_with_human_readable_string_value) {
+ v = (block % 2) ? rnd.HumanReadableString(256)
+ : rnd.RandomString(256);
+ } else {
+ v = rnd.RandomString(256);
+ }
+ kv[std::string(k)] = v;
+ key++;
+ }
+ }
+ return kv;
+ }
+
+ void SetUp() override {
+ SetupSyncPointsToMockDirectIO();
+ test_dir_ = test::PerThreadDBPath("block_based_table_reader_test");
+ env_ = Env::Default();
+ fs_ = FileSystem::Default();
+ ASSERT_OK(fs_->CreateDir(test_dir_, IOOptions(), nullptr));
+ ConfigureTableFactory();
+ }
+
+ virtual void ConfigureTableFactory() = 0;
+
+ void TearDown() override { EXPECT_OK(DestroyDir(env_, test_dir_)); }
+
+ // Creates a table with the specificied key value pairs (kv).
+ void CreateTable(const std::string& table_name,
+ const CompressionType& compression_type,
+ const std::map<std::string, std::string>& kv) {
+ std::unique_ptr<WritableFileWriter> writer;
+ NewFileWriter(table_name, &writer);
+
+ // Create table builder.
+ ImmutableOptions ioptions(options_);
+ InternalKeyComparator comparator(options_.comparator);
+ ColumnFamilyOptions cf_options;
+ MutableCFOptions moptions(cf_options);
+ IntTblPropCollectorFactories factories;
+ std::unique_ptr<TableBuilder> table_builder(
+ options_.table_factory->NewTableBuilder(
+ TableBuilderOptions(ioptions, moptions, comparator, &factories,
+ compression_type, CompressionOptions(),
+ 0 /* column_family_id */,
+ kDefaultColumnFamilyName, -1 /* level */),
+ writer.get()));
+
+ // Build table.
+ for (auto it = kv.begin(); it != kv.end(); it++) {
+ std::string k = ToInternalKey(it->first);
+ std::string v = it->second;
+ table_builder->Add(k, v);
+ }
+ ASSERT_OK(table_builder->Finish());
+ }
+
+ void NewBlockBasedTableReader(const FileOptions& foptions,
+ const ImmutableOptions& ioptions,
+ const InternalKeyComparator& comparator,
+ const std::string& table_name,
+ std::unique_ptr<BlockBasedTable>* table,
+ bool prefetch_index_and_filter_in_cache = true,
+ Status* status = nullptr) {
+ const MutableCFOptions moptions(options_);
+ TableReaderOptions table_reader_options = TableReaderOptions(
+ ioptions, moptions.prefix_extractor, EnvOptions(), comparator);
+
+ std::unique_ptr<RandomAccessFileReader> file;
+ NewFileReader(table_name, foptions, &file);
+
+ uint64_t file_size = 0;
+ ASSERT_OK(env_->GetFileSize(Path(table_name), &file_size));
+
+ std::unique_ptr<TableReader> general_table;
+ Status s = options_.table_factory->NewTableReader(
+ ReadOptions(), table_reader_options, std::move(file), file_size,
+ &general_table, prefetch_index_and_filter_in_cache);
+
+ if (s.ok()) {
+ table->reset(reinterpret_cast<BlockBasedTable*>(general_table.release()));
+ }
+
+ if (status) {
+ *status = s;
+ }
+ }
+
+ std::string Path(const std::string& fname) { return test_dir_ + "/" + fname; }
+
+ std::string test_dir_;
+ Env* env_;
+ std::shared_ptr<FileSystem> fs_;
+ Options options_;
+
+ private:
+ void WriteToFile(const std::string& content, const std::string& filename) {
+ std::unique_ptr<FSWritableFile> f;
+ ASSERT_OK(fs_->NewWritableFile(Path(filename), FileOptions(), &f, nullptr));
+ ASSERT_OK(f->Append(content, IOOptions(), nullptr));
+ ASSERT_OK(f->Close(IOOptions(), nullptr));
+ }
+
+ void NewFileWriter(const std::string& filename,
+ std::unique_ptr<WritableFileWriter>* writer) {
+ std::string path = Path(filename);
+ EnvOptions env_options;
+ FileOptions foptions;
+ std::unique_ptr<FSWritableFile> file;
+ ASSERT_OK(fs_->NewWritableFile(path, foptions, &file, nullptr));
+ writer->reset(new WritableFileWriter(std::move(file), path, env_options));
+ }
+
+ void NewFileReader(const std::string& filename, const FileOptions& opt,
+ std::unique_ptr<RandomAccessFileReader>* reader) {
+ std::string path = Path(filename);
+ std::unique_ptr<FSRandomAccessFile> f;
+ ASSERT_OK(fs_->NewRandomAccessFile(path, opt, &f, nullptr));
+ reader->reset(new RandomAccessFileReader(std::move(f), path,
+ env_->GetSystemClock().get()));
+ }
+
+ std::string ToInternalKey(const std::string& key) {
+ InternalKey internal_key(key, 0, ValueType::kTypeValue);
+ return internal_key.Encode().ToString();
+ }
+};
+
+class BlockBasedTableReaderTest
+ : public BlockBasedTableReaderBaseTest,
+ public testing::WithParamInterface<std::tuple<
+ CompressionType, bool, BlockBasedTableOptions::IndexType, bool>> {
+ protected:
+ void SetUp() override {
+ compression_type_ = std::get<0>(GetParam());
+ use_direct_reads_ = std::get<1>(GetParam());
+ BlockBasedTableReaderBaseTest::SetUp();
+ }
+
+ void ConfigureTableFactory() override {
+ BlockBasedTableOptions opts;
+ opts.index_type = std::get<2>(GetParam());
+ opts.no_block_cache = std::get<3>(GetParam());
+ options_.table_factory.reset(
+ static_cast<BlockBasedTableFactory*>(NewBlockBasedTableFactory(opts)));
+ }
+
+ CompressionType compression_type_;
+ bool use_direct_reads_;
+};
+
+// Tests MultiGet in both direct IO and non-direct IO mode.
+// The keys should be in cache after MultiGet.
+TEST_P(BlockBasedTableReaderTest, MultiGet) {
+ std::map<std::string, std::string> kv =
+ BlockBasedTableReaderBaseTest::GenerateKVMap(
+ 100 /* num_block */,
+ true /* mixed_with_human_readable_string_value */);
+
+ // Prepare keys, values, and statuses for MultiGet.
+ autovector<Slice, MultiGetContext::MAX_BATCH_SIZE> keys;
+ autovector<PinnableSlice, MultiGetContext::MAX_BATCH_SIZE> values;
+ autovector<Status, MultiGetContext::MAX_BATCH_SIZE> statuses;
+ {
+ const int step =
+ static_cast<int>(kv.size()) / MultiGetContext::MAX_BATCH_SIZE;
+ auto it = kv.begin();
+ for (int i = 0; i < MultiGetContext::MAX_BATCH_SIZE; i++) {
+ keys.emplace_back(it->first);
+ values.emplace_back();
+ statuses.emplace_back();
+ std::advance(it, step);
+ }
+ }
+
+ std::string table_name =
+ "BlockBasedTableReaderTest" + CompressionTypeToString(compression_type_);
+ CreateTable(table_name, compression_type_, kv);
+
+ std::unique_ptr<BlockBasedTable> table;
+ Options options;
+ ImmutableOptions ioptions(options);
+ FileOptions foptions;
+ foptions.use_direct_reads = use_direct_reads_;
+ InternalKeyComparator comparator(options.comparator);
+ NewBlockBasedTableReader(foptions, ioptions, comparator, table_name, &table);
+
+ // Ensure that keys are not in cache before MultiGet.
+ for (auto& key : keys) {
+ ASSERT_FALSE(table->TEST_KeyInCache(ReadOptions(), key));
+ }
+
+ // Prepare MultiGetContext.
+ autovector<GetContext, MultiGetContext::MAX_BATCH_SIZE> get_context;
+ autovector<KeyContext, MultiGetContext::MAX_BATCH_SIZE> key_context;
+ autovector<KeyContext*, MultiGetContext::MAX_BATCH_SIZE> sorted_keys;
+ for (size_t i = 0; i < keys.size(); ++i) {
+ get_context.emplace_back(BytewiseComparator(), nullptr, nullptr, nullptr,
+ GetContext::kNotFound, keys[i], &values[i],
+ nullptr, nullptr, nullptr, nullptr,
+ true /* do_merge */, nullptr, nullptr, nullptr,
+ nullptr, nullptr, nullptr);
+ key_context.emplace_back(nullptr, keys[i], &values[i], nullptr,
+ &statuses.back());
+ key_context.back().get_context = &get_context.back();
+ }
+ for (auto& key_ctx : key_context) {
+ sorted_keys.emplace_back(&key_ctx);
+ }
+ MultiGetContext ctx(&sorted_keys, 0, sorted_keys.size(), 0, ReadOptions(),
+ fs_.get(), nullptr);
+
+ // Execute MultiGet.
+ MultiGetContext::Range range = ctx.GetMultiGetRange();
+ PerfContext* perf_ctx = get_perf_context();
+ perf_ctx->Reset();
+ table->MultiGet(ReadOptions(), &range, nullptr);
+
+ ASSERT_GE(perf_ctx->block_read_count - perf_ctx->index_block_read_count -
+ perf_ctx->filter_block_read_count -
+ perf_ctx->compression_dict_block_read_count,
+ 1);
+ ASSERT_GE(perf_ctx->block_read_byte, 1);
+
+ for (const Status& status : statuses) {
+ ASSERT_OK(status);
+ }
+ // Check that keys are in cache after MultiGet.
+ for (size_t i = 0; i < keys.size(); i++) {
+ ASSERT_TRUE(table->TEST_KeyInCache(ReadOptions(), keys[i]));
+ ASSERT_EQ(values[i].ToString(), kv[keys[i].ToString()]);
+ }
+}
+
+class ChargeTableReaderTest
+ : public BlockBasedTableReaderBaseTest,
+ public testing::WithParamInterface<
+ CacheEntryRoleOptions::Decision /* charge_table_reader_mem */> {
+ protected:
+ static std::size_t CalculateMaxTableReaderNumBeforeCacheFull(
+ std::size_t cache_capacity, std::size_t approx_table_reader_mem) {
+ // To make calculation easier for testing
+ assert(cache_capacity % CacheReservationManagerImpl<
+ CacheEntryRole::kBlockBasedTableReader>::
+ GetDummyEntrySize() ==
+ 0 &&
+ cache_capacity >= 2 * CacheReservationManagerImpl<
+ CacheEntryRole::kBlockBasedTableReader>::
+ GetDummyEntrySize());
+
+ // We need to subtract 1 for max_num_dummy_entry to account for dummy
+ // entries' overhead, assumed the overhead is no greater than 1 dummy entry
+ // size
+ std::size_t max_num_dummy_entry =
+ (size_t)std::floor((
+ 1.0 * cache_capacity /
+ CacheReservationManagerImpl<
+ CacheEntryRole::kBlockBasedTableReader>::GetDummyEntrySize())) -
+ 1;
+ std::size_t cache_capacity_rounded_to_dummy_entry_multiples =
+ max_num_dummy_entry *
+ CacheReservationManagerImpl<
+ CacheEntryRole::kBlockBasedTableReader>::GetDummyEntrySize();
+ std::size_t max_table_reader_num_capped = static_cast<std::size_t>(
+ std::floor(1.0 * cache_capacity_rounded_to_dummy_entry_multiples /
+ approx_table_reader_mem));
+
+ return max_table_reader_num_capped;
+ }
+
+ void SetUp() override {
+ // To cache and re-use the same kv map and compression type in the test
+ // suite for elimiating variance caused by these two factors
+ kv_ = BlockBasedTableReaderBaseTest::GenerateKVMap();
+ compression_type_ = CompressionType::kNoCompression;
+
+ table_reader_charge_tracking_cache_ = std::make_shared<
+ TargetCacheChargeTrackingCache<
+ CacheEntryRole::kBlockBasedTableReader>>((NewLRUCache(
+ 4 * CacheReservationManagerImpl<
+ CacheEntryRole::kBlockBasedTableReader>::GetDummyEntrySize(),
+ 0 /* num_shard_bits */, true /* strict_capacity_limit */)));
+
+ // To ApproximateTableReaderMem() without being affected by
+ // the feature of charging its memory, we turn off the feature
+ charge_table_reader_ = CacheEntryRoleOptions::Decision::kDisabled;
+ BlockBasedTableReaderBaseTest::SetUp();
+ approx_table_reader_mem_ = ApproximateTableReaderMem();
+
+ // Now we condtionally turn on the feature to test
+ charge_table_reader_ = GetParam();
+ ConfigureTableFactory();
+ }
+
+ void ConfigureTableFactory() override {
+ BlockBasedTableOptions table_options;
+ table_options.cache_usage_options.options_overrides.insert(
+ {CacheEntryRole::kBlockBasedTableReader,
+ {/*.charged = */ charge_table_reader_}});
+ table_options.block_cache = table_reader_charge_tracking_cache_;
+
+ table_options.cache_index_and_filter_blocks = false;
+ table_options.filter_policy.reset(NewBloomFilterPolicy(10, false));
+ table_options.partition_filters = true;
+ table_options.index_type = BlockBasedTableOptions::kTwoLevelIndexSearch;
+
+ options_.table_factory.reset(NewBlockBasedTableFactory(table_options));
+ }
+
+ CacheEntryRoleOptions::Decision charge_table_reader_;
+ std::shared_ptr<
+ TargetCacheChargeTrackingCache<CacheEntryRole::kBlockBasedTableReader>>
+ table_reader_charge_tracking_cache_;
+ std::size_t approx_table_reader_mem_;
+ std::map<std::string, std::string> kv_;
+ CompressionType compression_type_;
+
+ private:
+ std::size_t ApproximateTableReaderMem() {
+ std::size_t approx_table_reader_mem = 0;
+
+ std::string table_name = "table_for_approx_table_reader_mem";
+ CreateTable(table_name, compression_type_, kv_);
+
+ std::unique_ptr<BlockBasedTable> table;
+ Status s;
+ NewBlockBasedTableReader(
+ FileOptions(), ImmutableOptions(options_),
+ InternalKeyComparator(options_.comparator), table_name, &table,
+ false /* prefetch_index_and_filter_in_cache */, &s);
+ assert(s.ok());
+
+ approx_table_reader_mem = table->ApproximateMemoryUsage();
+ assert(approx_table_reader_mem > 0);
+ return approx_table_reader_mem;
+ }
+};
+
+INSTANTIATE_TEST_CASE_P(
+ ChargeTableReaderTest, ChargeTableReaderTest,
+ ::testing::Values(CacheEntryRoleOptions::Decision::kEnabled,
+ CacheEntryRoleOptions::Decision::kDisabled));
+
+TEST_P(ChargeTableReaderTest, Basic) {
+ const std::size_t max_table_reader_num_capped =
+ ChargeTableReaderTest::CalculateMaxTableReaderNumBeforeCacheFull(
+ table_reader_charge_tracking_cache_->GetCapacity(),
+ approx_table_reader_mem_);
+
+ // Acceptable estimtation errors coming from
+ // 1. overstimate max_table_reader_num_capped due to # dummy entries is high
+ // and results in metadata charge overhead greater than 1 dummy entry size
+ // (violating our assumption in calculating max_table_reader_num_capped)
+ // 2. overestimate/underestimate max_table_reader_num_capped due to the gap
+ // between ApproximateTableReaderMem() and actual table reader mem
+ std::size_t max_table_reader_num_capped_upper_bound =
+ (std::size_t)(max_table_reader_num_capped * 1.05);
+ std::size_t max_table_reader_num_capped_lower_bound =
+ (std::size_t)(max_table_reader_num_capped * 0.95);
+ std::size_t max_table_reader_num_uncapped =
+ (std::size_t)(max_table_reader_num_capped * 1.1);
+ ASSERT_GT(max_table_reader_num_uncapped,
+ max_table_reader_num_capped_upper_bound)
+ << "We need `max_table_reader_num_uncapped` > "
+ "`max_table_reader_num_capped_upper_bound` to differentiate cases "
+ "between "
+ "charge_table_reader_ == kDisabled and == kEnabled)";
+
+ Status s = Status::OK();
+ std::size_t opened_table_reader_num = 0;
+ std::string table_name;
+ std::vector<std::unique_ptr<BlockBasedTable>> tables;
+ // Keep creating BlockBasedTableReader till hiting the memory limit based on
+ // cache capacity and creation fails (when charge_table_reader_ ==
+ // kEnabled) or reaching a specfied big number of table readers (when
+ // charge_table_reader_ == kDisabled)
+ while (s.ok() && opened_table_reader_num < max_table_reader_num_uncapped) {
+ table_name = "table_" + std::to_string(opened_table_reader_num);
+ CreateTable(table_name, compression_type_, kv_);
+ tables.push_back(std::unique_ptr<BlockBasedTable>());
+ NewBlockBasedTableReader(
+ FileOptions(), ImmutableOptions(options_),
+ InternalKeyComparator(options_.comparator), table_name, &tables.back(),
+ false /* prefetch_index_and_filter_in_cache */, &s);
+ if (s.ok()) {
+ ++opened_table_reader_num;
+ }
+ }
+
+ if (charge_table_reader_ == CacheEntryRoleOptions::Decision::kEnabled) {
+ EXPECT_TRUE(s.IsMemoryLimit()) << "s: " << s.ToString();
+ EXPECT_TRUE(s.ToString().find(
+ kCacheEntryRoleToCamelString[static_cast<std::uint32_t>(
+ CacheEntryRole::kBlockBasedTableReader)]) !=
+ std::string::npos);
+ EXPECT_TRUE(s.ToString().find("memory limit based on cache capacity") !=
+ std::string::npos);
+
+ EXPECT_GE(opened_table_reader_num, max_table_reader_num_capped_lower_bound);
+ EXPECT_LE(opened_table_reader_num, max_table_reader_num_capped_upper_bound);
+
+ std::size_t updated_max_table_reader_num_capped =
+ ChargeTableReaderTest::CalculateMaxTableReaderNumBeforeCacheFull(
+ table_reader_charge_tracking_cache_->GetCapacity() / 2,
+ approx_table_reader_mem_);
+
+ // Keep deleting BlockBasedTableReader to lower down memory usage from the
+ // memory limit to make the next creation succeeds
+ while (opened_table_reader_num >= updated_max_table_reader_num_capped) {
+ tables.pop_back();
+ --opened_table_reader_num;
+ }
+ table_name = "table_for_successful_table_reader_open";
+ CreateTable(table_name, compression_type_, kv_);
+ tables.push_back(std::unique_ptr<BlockBasedTable>());
+ NewBlockBasedTableReader(
+ FileOptions(), ImmutableOptions(options_),
+ InternalKeyComparator(options_.comparator), table_name, &tables.back(),
+ false /* prefetch_index_and_filter_in_cache */, &s);
+ EXPECT_TRUE(s.ok()) << s.ToString();
+
+ tables.clear();
+ EXPECT_EQ(table_reader_charge_tracking_cache_->GetCacheCharge(), 0);
+ } else {
+ EXPECT_TRUE(s.ok() &&
+ opened_table_reader_num == max_table_reader_num_uncapped)
+ << "s: " << s.ToString() << " opened_table_reader_num: "
+ << std::to_string(opened_table_reader_num);
+ EXPECT_EQ(table_reader_charge_tracking_cache_->GetCacheCharge(), 0);
+ }
+}
+
+class BlockBasedTableReaderTestVerifyChecksum
+ : public BlockBasedTableReaderTest {
+ public:
+ BlockBasedTableReaderTestVerifyChecksum() : BlockBasedTableReaderTest() {}
+};
+
+TEST_P(BlockBasedTableReaderTestVerifyChecksum, ChecksumMismatch) {
+ std::map<std::string, std::string> kv =
+ BlockBasedTableReaderBaseTest::GenerateKVMap(800 /* num_block */);
+
+ std::string table_name =
+ "BlockBasedTableReaderTest" + CompressionTypeToString(compression_type_);
+ CreateTable(table_name, compression_type_, kv);
+
+ std::unique_ptr<BlockBasedTable> table;
+ Options options;
+ ImmutableOptions ioptions(options);
+ FileOptions foptions;
+ foptions.use_direct_reads = use_direct_reads_;
+ InternalKeyComparator comparator(options.comparator);
+ NewBlockBasedTableReader(foptions, ioptions, comparator, table_name, &table);
+
+ // Use the top level iterator to find the offset/size of the first
+ // 2nd level index block and corrupt the block
+ IndexBlockIter iiter_on_stack;
+ BlockCacheLookupContext context{TableReaderCaller::kUserVerifyChecksum};
+ InternalIteratorBase<IndexValue>* iiter = table->NewIndexIterator(
+ ReadOptions(), /*disable_prefix_seek=*/false, &iiter_on_stack,
+ /*get_context=*/nullptr, &context);
+ std::unique_ptr<InternalIteratorBase<IndexValue>> iiter_unique_ptr;
+ if (iiter != &iiter_on_stack) {
+ iiter_unique_ptr = std::unique_ptr<InternalIteratorBase<IndexValue>>(iiter);
+ }
+ ASSERT_OK(iiter->status());
+ iiter->SeekToFirst();
+ BlockHandle handle = static_cast<PartitionedIndexIterator*>(iiter)
+ ->index_iter_->value()
+ .handle;
+ table.reset();
+
+ // Corrupt the block pointed to by handle
+ ASSERT_OK(test::CorruptFile(options.env, Path(table_name),
+ static_cast<int>(handle.offset()), 128));
+
+ NewBlockBasedTableReader(foptions, ioptions, comparator, table_name, &table);
+ Status s = table->VerifyChecksum(ReadOptions(),
+ TableReaderCaller::kUserVerifyChecksum);
+ ASSERT_EQ(s.code(), Status::kCorruption);
+}
+
+// Param 1: compression type
+// Param 2: whether to use direct reads
+// Param 3: Block Based Table Index type
+// Param 4: BBTO no_block_cache option
+#ifdef ROCKSDB_LITE
+// Skip direct I/O tests in lite mode since direct I/O is unsupported.
+INSTANTIATE_TEST_CASE_P(
+ MultiGet, BlockBasedTableReaderTest,
+ ::testing::Combine(
+ ::testing::ValuesIn(GetSupportedCompressions()),
+ ::testing::Values(false),
+ ::testing::Values(BlockBasedTableOptions::IndexType::kBinarySearch),
+ ::testing::Values(false)));
+#else // ROCKSDB_LITE
+INSTANTIATE_TEST_CASE_P(
+ MultiGet, BlockBasedTableReaderTest,
+ ::testing::Combine(
+ ::testing::ValuesIn(GetSupportedCompressions()), ::testing::Bool(),
+ ::testing::Values(BlockBasedTableOptions::IndexType::kBinarySearch),
+ ::testing::Values(false)));
+#endif // ROCKSDB_LITE
+INSTANTIATE_TEST_CASE_P(
+ VerifyChecksum, BlockBasedTableReaderTestVerifyChecksum,
+ ::testing::Combine(
+ ::testing::ValuesIn(GetSupportedCompressions()),
+ ::testing::Values(false),
+ ::testing::Values(
+ BlockBasedTableOptions::IndexType::kTwoLevelIndexSearch),
+ ::testing::Values(true)));
+
+} // namespace ROCKSDB_NAMESPACE
+
+int main(int argc, char** argv) {
+ ROCKSDB_NAMESPACE::port::InstallStackTraceHandler();
+ ::testing::InitGoogleTest(&argc, argv);
+ return RUN_ALL_TESTS();
+}
diff --git a/src/rocksdb/table/block_based/block_builder.cc b/src/rocksdb/table/block_based/block_builder.cc
new file mode 100644
index 000000000..92702b17d
--- /dev/null
+++ b/src/rocksdb/table/block_based/block_builder.cc
@@ -0,0 +1,234 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+//
+// BlockBuilder generates blocks where keys are prefix-compressed:
+//
+// When we store a key, we drop the prefix shared with the previous
+// string. This helps reduce the space requirement significantly.
+// Furthermore, once every K keys, we do not apply the prefix
+// compression and store the entire key. We call this a "restart
+// point". The tail end of the block stores the offsets of all of the
+// restart points, and can be used to do a binary search when looking
+// for a particular key. Values are stored as-is (without compression)
+// immediately following the corresponding key.
+//
+// An entry for a particular key-value pair has the form:
+// shared_bytes: varint32
+// unshared_bytes: varint32
+// value_length: varint32
+// key_delta: char[unshared_bytes]
+// value: char[value_length]
+// shared_bytes == 0 for restart points.
+//
+// The trailer of the block has the form:
+// restarts: uint32[num_restarts]
+// num_restarts: uint32
+// restarts[i] contains the offset within the block of the ith restart point.
+
+#include "table/block_based/block_builder.h"
+
+#include <assert.h>
+
+#include <algorithm>
+
+#include "db/dbformat.h"
+#include "rocksdb/comparator.h"
+#include "table/block_based/data_block_footer.h"
+#include "util/coding.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+BlockBuilder::BlockBuilder(
+ int block_restart_interval, bool use_delta_encoding,
+ bool use_value_delta_encoding,
+ BlockBasedTableOptions::DataBlockIndexType index_type,
+ double data_block_hash_table_util_ratio)
+ : block_restart_interval_(block_restart_interval),
+ use_delta_encoding_(use_delta_encoding),
+ use_value_delta_encoding_(use_value_delta_encoding),
+ restarts_(1, 0), // First restart point is at offset 0
+ counter_(0),
+ finished_(false) {
+ switch (index_type) {
+ case BlockBasedTableOptions::kDataBlockBinarySearch:
+ break;
+ case BlockBasedTableOptions::kDataBlockBinaryAndHash:
+ data_block_hash_index_builder_.Initialize(
+ data_block_hash_table_util_ratio);
+ break;
+ default:
+ assert(0);
+ }
+ assert(block_restart_interval_ >= 1);
+ estimate_ = sizeof(uint32_t) + sizeof(uint32_t);
+}
+
+void BlockBuilder::Reset() {
+ buffer_.clear();
+ restarts_.resize(1); // First restart point is at offset 0
+ assert(restarts_[0] == 0);
+ estimate_ = sizeof(uint32_t) + sizeof(uint32_t);
+ counter_ = 0;
+ finished_ = false;
+ last_key_.clear();
+ if (data_block_hash_index_builder_.Valid()) {
+ data_block_hash_index_builder_.Reset();
+ }
+#ifndef NDEBUG
+ add_with_last_key_called_ = false;
+#endif
+}
+
+void BlockBuilder::SwapAndReset(std::string& buffer) {
+ std::swap(buffer_, buffer);
+ Reset();
+}
+
+size_t BlockBuilder::EstimateSizeAfterKV(const Slice& key,
+ const Slice& value) const {
+ size_t estimate = CurrentSizeEstimate();
+ // Note: this is an imprecise estimate as it accounts for the whole key size
+ // instead of non-shared key size.
+ estimate += key.size();
+ // In value delta encoding we estimate the value delta size as half the full
+ // value size since only the size field of block handle is encoded.
+ estimate +=
+ !use_value_delta_encoding_ || (counter_ >= block_restart_interval_)
+ ? value.size()
+ : value.size() / 2;
+
+ if (counter_ >= block_restart_interval_) {
+ estimate += sizeof(uint32_t); // a new restart entry.
+ }
+
+ estimate += sizeof(int32_t); // varint for shared prefix length.
+ // Note: this is an imprecise estimate as we will have to encoded size, one
+ // for shared key and one for non-shared key.
+ estimate += VarintLength(key.size()); // varint for key length.
+ if (!use_value_delta_encoding_ || (counter_ >= block_restart_interval_)) {
+ estimate += VarintLength(value.size()); // varint for value length.
+ }
+
+ return estimate;
+}
+
+Slice BlockBuilder::Finish() {
+ // Append restart array
+ for (size_t i = 0; i < restarts_.size(); i++) {
+ PutFixed32(&buffer_, restarts_[i]);
+ }
+
+ uint32_t num_restarts = static_cast<uint32_t>(restarts_.size());
+ BlockBasedTableOptions::DataBlockIndexType index_type =
+ BlockBasedTableOptions::kDataBlockBinarySearch;
+ if (data_block_hash_index_builder_.Valid() &&
+ CurrentSizeEstimate() <= kMaxBlockSizeSupportedByHashIndex) {
+ data_block_hash_index_builder_.Finish(buffer_);
+ index_type = BlockBasedTableOptions::kDataBlockBinaryAndHash;
+ }
+
+ // footer is a packed format of data_block_index_type and num_restarts
+ uint32_t block_footer = PackIndexTypeAndNumRestarts(index_type, num_restarts);
+
+ PutFixed32(&buffer_, block_footer);
+ finished_ = true;
+ return Slice(buffer_);
+}
+
+void BlockBuilder::Add(const Slice& key, const Slice& value,
+ const Slice* const delta_value) {
+ // Ensure no unsafe mixing of Add and AddWithLastKey
+ assert(!add_with_last_key_called_);
+
+ AddWithLastKeyImpl(key, value, last_key_, delta_value, buffer_.size());
+ if (use_delta_encoding_) {
+ // Update state
+ // We used to just copy the changed data, but it appears to be
+ // faster to just copy the whole thing.
+ last_key_.assign(key.data(), key.size());
+ }
+}
+
+void BlockBuilder::AddWithLastKey(const Slice& key, const Slice& value,
+ const Slice& last_key_param,
+ const Slice* const delta_value) {
+ // Ensure no unsafe mixing of Add and AddWithLastKey
+ assert(last_key_.empty());
+#ifndef NDEBUG
+ add_with_last_key_called_ = false;
+#endif
+
+ // Here we make sure to use an empty `last_key` on first call after creation
+ // or Reset. This is more convenient for the caller and we can be more
+ // clever inside BlockBuilder. On this hot code path, we want to avoid
+ // conditional jumps like `buffer_.empty() ? ... : ...` so we can use a
+ // fast min operation instead, with an assertion to be sure our logic is
+ // sound.
+ size_t buffer_size = buffer_.size();
+ size_t last_key_size = last_key_param.size();
+ assert(buffer_size == 0 || buffer_size >= last_key_size);
+
+ Slice last_key(last_key_param.data(), std::min(buffer_size, last_key_size));
+
+ AddWithLastKeyImpl(key, value, last_key, delta_value, buffer_size);
+}
+
+inline void BlockBuilder::AddWithLastKeyImpl(const Slice& key,
+ const Slice& value,
+ const Slice& last_key,
+ const Slice* const delta_value,
+ size_t buffer_size) {
+ assert(!finished_);
+ assert(counter_ <= block_restart_interval_);
+ assert(!use_value_delta_encoding_ || delta_value);
+ size_t shared = 0; // number of bytes shared with prev key
+ if (counter_ >= block_restart_interval_) {
+ // Restart compression
+ restarts_.push_back(static_cast<uint32_t>(buffer_size));
+ estimate_ += sizeof(uint32_t);
+ counter_ = 0;
+ } else if (use_delta_encoding_) {
+ // See how much sharing to do with previous string
+ shared = key.difference_offset(last_key);
+ }
+
+ const size_t non_shared = key.size() - shared;
+
+ if (use_value_delta_encoding_) {
+ // Add "<shared><non_shared>" to buffer_
+ PutVarint32Varint32(&buffer_, static_cast<uint32_t>(shared),
+ static_cast<uint32_t>(non_shared));
+ } else {
+ // Add "<shared><non_shared><value_size>" to buffer_
+ PutVarint32Varint32Varint32(&buffer_, static_cast<uint32_t>(shared),
+ static_cast<uint32_t>(non_shared),
+ static_cast<uint32_t>(value.size()));
+ }
+
+ // Add string delta to buffer_ followed by value
+ buffer_.append(key.data() + shared, non_shared);
+ // Use value delta encoding only when the key has shared bytes. This would
+ // simplify the decoding, where it can figure which decoding to use simply by
+ // looking at the shared bytes size.
+ if (shared != 0 && use_value_delta_encoding_) {
+ buffer_.append(delta_value->data(), delta_value->size());
+ } else {
+ buffer_.append(value.data(), value.size());
+ }
+
+ if (data_block_hash_index_builder_.Valid()) {
+ data_block_hash_index_builder_.Add(ExtractUserKey(key),
+ restarts_.size() - 1);
+ }
+
+ counter_++;
+ estimate_ += buffer_.size() - buffer_size;
+}
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/block_builder.h b/src/rocksdb/table/block_based/block_builder.h
new file mode 100644
index 000000000..5f68b449b
--- /dev/null
+++ b/src/rocksdb/table/block_based/block_builder.h
@@ -0,0 +1,102 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#pragma once
+#include <stdint.h>
+
+#include <vector>
+
+#include "rocksdb/slice.h"
+#include "rocksdb/table.h"
+#include "table/block_based/data_block_hash_index.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class BlockBuilder {
+ public:
+ BlockBuilder(const BlockBuilder&) = delete;
+ void operator=(const BlockBuilder&) = delete;
+
+ explicit BlockBuilder(int block_restart_interval,
+ bool use_delta_encoding = true,
+ bool use_value_delta_encoding = false,
+ BlockBasedTableOptions::DataBlockIndexType index_type =
+ BlockBasedTableOptions::kDataBlockBinarySearch,
+ double data_block_hash_table_util_ratio = 0.75);
+
+ // Reset the contents as if the BlockBuilder was just constructed.
+ void Reset();
+
+ // Swap the contents in BlockBuilder with buffer, then reset the BlockBuilder.
+ void SwapAndReset(std::string& buffer);
+
+ // REQUIRES: Finish() has not been called since the last call to Reset().
+ // REQUIRES: key is larger than any previously added key
+ // DO NOT mix with AddWithLastKey() between Resets. For efficiency, use
+ // AddWithLastKey() in contexts where previous added key is already known
+ // and delta encoding might be used.
+ void Add(const Slice& key, const Slice& value,
+ const Slice* const delta_value = nullptr);
+
+ // A faster version of Add() if the previous key is already known for all
+ // Add()s.
+ // REQUIRES: Finish() has not been called since the last call to Reset().
+ // REQUIRES: key is larger than any previously added key
+ // REQUIRES: if AddWithLastKey has been called since last Reset(), last_key
+ // is the key from most recent AddWithLastKey. (For convenience, last_key
+ // is ignored on first call after creation or Reset().)
+ // DO NOT mix with Add() between Resets.
+ void AddWithLastKey(const Slice& key, const Slice& value,
+ const Slice& last_key,
+ const Slice* const delta_value = nullptr);
+
+ // Finish building the block and return a slice that refers to the
+ // block contents. The returned slice will remain valid for the
+ // lifetime of this builder or until Reset() is called.
+ Slice Finish();
+
+ // Returns an estimate of the current (uncompressed) size of the block
+ // we are building.
+ inline size_t CurrentSizeEstimate() const {
+ return estimate_ + (data_block_hash_index_builder_.Valid()
+ ? data_block_hash_index_builder_.EstimateSize()
+ : 0);
+ }
+
+ // Returns an estimated block size after appending key and value.
+ size_t EstimateSizeAfterKV(const Slice& key, const Slice& value) const;
+
+ // Return true iff no entries have been added since the last Reset()
+ bool empty() const { return buffer_.empty(); }
+
+ private:
+ inline void AddWithLastKeyImpl(const Slice& key, const Slice& value,
+ const Slice& last_key,
+ const Slice* const delta_value,
+ size_t buffer_size);
+
+ const int block_restart_interval_;
+ // TODO(myabandeh): put it into a separate IndexBlockBuilder
+ const bool use_delta_encoding_;
+ // Refer to BlockIter::DecodeCurrentValue for format of delta encoded values
+ const bool use_value_delta_encoding_;
+
+ std::string buffer_; // Destination buffer
+ std::vector<uint32_t> restarts_; // Restart points
+ size_t estimate_;
+ int counter_; // Number of entries emitted since restart
+ bool finished_; // Has Finish() been called?
+ std::string last_key_;
+ DataBlockHashIndexBuilder data_block_hash_index_builder_;
+#ifndef NDEBUG
+ bool add_with_last_key_called_ = false;
+#endif
+};
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/block_like_traits.h b/src/rocksdb/table/block_based/block_like_traits.h
new file mode 100644
index 000000000..d406dbb5d
--- /dev/null
+++ b/src/rocksdb/table/block_based/block_like_traits.h
@@ -0,0 +1,182 @@
+// Copyright (c) Facebook, Inc. and its affiliates. 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).
+
+#pragma once
+
+#include "cache/cache_entry_roles.h"
+#include "port/lang.h"
+#include "table/block_based/block.h"
+#include "table/block_based/block_type.h"
+#include "table/block_based/parsed_full_filter_block.h"
+#include "table/format.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+template <typename TBlocklike>
+class BlocklikeTraits;
+
+template <typename T, CacheEntryRole R>
+Cache::CacheItemHelper* GetCacheItemHelperForRole();
+
+template <typename TBlocklike>
+Cache::CreateCallback GetCreateCallback(size_t read_amp_bytes_per_bit,
+ Statistics* statistics, bool using_zstd,
+ const FilterPolicy* filter_policy) {
+ return [read_amp_bytes_per_bit, statistics, using_zstd, filter_policy](
+ const void* buf, size_t size, void** out_obj,
+ size_t* charge) -> Status {
+ assert(buf != nullptr);
+ std::unique_ptr<char[]> buf_data(new char[size]());
+ memcpy(buf_data.get(), buf, size);
+ BlockContents bc = BlockContents(std::move(buf_data), size);
+ TBlocklike* ucd_ptr = BlocklikeTraits<TBlocklike>::Create(
+ std::move(bc), read_amp_bytes_per_bit, statistics, using_zstd,
+ filter_policy);
+ *out_obj = reinterpret_cast<void*>(ucd_ptr);
+ *charge = size;
+ return Status::OK();
+ };
+}
+
+template <>
+class BlocklikeTraits<ParsedFullFilterBlock> {
+ public:
+ static ParsedFullFilterBlock* Create(BlockContents&& contents,
+ size_t /* read_amp_bytes_per_bit */,
+ Statistics* /* statistics */,
+ bool /* using_zstd */,
+ const FilterPolicy* filter_policy) {
+ return new ParsedFullFilterBlock(filter_policy, std::move(contents));
+ }
+
+ static uint32_t GetNumRestarts(const ParsedFullFilterBlock& /* block */) {
+ return 0;
+ }
+
+ static size_t SizeCallback(void* obj) {
+ assert(obj != nullptr);
+ ParsedFullFilterBlock* ptr = static_cast<ParsedFullFilterBlock*>(obj);
+ return ptr->GetBlockContentsData().size();
+ }
+
+ static Status SaveToCallback(void* from_obj, size_t from_offset,
+ size_t length, void* out) {
+ assert(from_obj != nullptr);
+ ParsedFullFilterBlock* ptr = static_cast<ParsedFullFilterBlock*>(from_obj);
+ const char* buf = ptr->GetBlockContentsData().data();
+ assert(length == ptr->GetBlockContentsData().size());
+ (void)from_offset;
+ memcpy(out, buf, length);
+ return Status::OK();
+ }
+
+ static Cache::CacheItemHelper* GetCacheItemHelper(BlockType block_type) {
+ (void)block_type;
+ assert(block_type == BlockType::kFilter);
+ return GetCacheItemHelperForRole<ParsedFullFilterBlock,
+ CacheEntryRole::kFilterBlock>();
+ }
+};
+
+template <>
+class BlocklikeTraits<Block> {
+ public:
+ static Block* Create(BlockContents&& contents, size_t read_amp_bytes_per_bit,
+ Statistics* statistics, bool /* using_zstd */,
+ const FilterPolicy* /* filter_policy */) {
+ return new Block(std::move(contents), read_amp_bytes_per_bit, statistics);
+ }
+
+ static uint32_t GetNumRestarts(const Block& block) {
+ return block.NumRestarts();
+ }
+
+ static size_t SizeCallback(void* obj) {
+ assert(obj != nullptr);
+ Block* ptr = static_cast<Block*>(obj);
+ return ptr->size();
+ }
+
+ static Status SaveToCallback(void* from_obj, size_t from_offset,
+ size_t length, void* out) {
+ assert(from_obj != nullptr);
+ Block* ptr = static_cast<Block*>(from_obj);
+ const char* buf = ptr->data();
+ assert(length == ptr->size());
+ (void)from_offset;
+ memcpy(out, buf, length);
+ return Status::OK();
+ }
+
+ static Cache::CacheItemHelper* GetCacheItemHelper(BlockType block_type) {
+ switch (block_type) {
+ case BlockType::kData:
+ return GetCacheItemHelperForRole<Block, CacheEntryRole::kDataBlock>();
+ case BlockType::kIndex:
+ return GetCacheItemHelperForRole<Block, CacheEntryRole::kIndexBlock>();
+ case BlockType::kFilterPartitionIndex:
+ return GetCacheItemHelperForRole<Block,
+ CacheEntryRole::kFilterMetaBlock>();
+ default:
+ // Not a recognized combination
+ assert(false);
+ FALLTHROUGH_INTENDED;
+ case BlockType::kRangeDeletion:
+ return GetCacheItemHelperForRole<Block, CacheEntryRole::kOtherBlock>();
+ }
+ }
+};
+
+template <>
+class BlocklikeTraits<UncompressionDict> {
+ public:
+ static UncompressionDict* Create(BlockContents&& contents,
+ size_t /* read_amp_bytes_per_bit */,
+ Statistics* /* statistics */,
+ bool using_zstd,
+ const FilterPolicy* /* filter_policy */) {
+ return new UncompressionDict(contents.data, std::move(contents.allocation),
+ using_zstd);
+ }
+
+ static uint32_t GetNumRestarts(const UncompressionDict& /* dict */) {
+ return 0;
+ }
+
+ static size_t SizeCallback(void* obj) {
+ assert(obj != nullptr);
+ UncompressionDict* ptr = static_cast<UncompressionDict*>(obj);
+ return ptr->slice_.size();
+ }
+
+ static Status SaveToCallback(void* from_obj, size_t from_offset,
+ size_t length, void* out) {
+ assert(from_obj != nullptr);
+ UncompressionDict* ptr = static_cast<UncompressionDict*>(from_obj);
+ const char* buf = ptr->slice_.data();
+ assert(length == ptr->slice_.size());
+ (void)from_offset;
+ memcpy(out, buf, length);
+ return Status::OK();
+ }
+
+ static Cache::CacheItemHelper* GetCacheItemHelper(BlockType block_type) {
+ (void)block_type;
+ assert(block_type == BlockType::kCompressionDictionary);
+ return GetCacheItemHelperForRole<UncompressionDict,
+ CacheEntryRole::kOtherBlock>();
+ }
+};
+
+// Get an CacheItemHelper pointer for value type T and role R.
+template <typename T, CacheEntryRole R>
+Cache::CacheItemHelper* GetCacheItemHelperForRole() {
+ static Cache::CacheItemHelper cache_helper(
+ BlocklikeTraits<T>::SizeCallback, BlocklikeTraits<T>::SaveToCallback,
+ GetCacheEntryDeleterForRole<T, R>());
+ return &cache_helper;
+}
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/block_prefetcher.cc b/src/rocksdb/table/block_based/block_prefetcher.cc
new file mode 100644
index 000000000..83ec2cb06
--- /dev/null
+++ b/src/rocksdb/table/block_based/block_prefetcher.cc
@@ -0,0 +1,120 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+#include "table/block_based/block_prefetcher.h"
+
+#include "rocksdb/file_system.h"
+#include "table/block_based/block_based_table_reader.h"
+
+namespace ROCKSDB_NAMESPACE {
+void BlockPrefetcher::PrefetchIfNeeded(
+ const BlockBasedTable::Rep* rep, const BlockHandle& handle,
+ const size_t readahead_size, bool is_for_compaction,
+ const bool no_sequential_checking,
+ const Env::IOPriority rate_limiter_priority) {
+ // num_file_reads is used by FilePrefetchBuffer only when
+ // implicit_auto_readahead is set.
+ if (is_for_compaction) {
+ rep->CreateFilePrefetchBufferIfNotExists(
+ compaction_readahead_size_, compaction_readahead_size_,
+ &prefetch_buffer_, /*implicit_auto_readahead=*/false,
+ /*num_file_reads=*/0, /*num_file_reads_for_auto_readahead=*/0);
+ return;
+ }
+
+ // Explicit user requested readahead.
+ if (readahead_size > 0) {
+ rep->CreateFilePrefetchBufferIfNotExists(
+ readahead_size, readahead_size, &prefetch_buffer_,
+ /*implicit_auto_readahead=*/false, /*num_file_reads=*/0,
+ /*num_file_reads_for_auto_readahead=*/0);
+ return;
+ }
+
+ // Implicit readahead.
+
+ // If max_auto_readahead_size is set to be 0 by user, no data will be
+ // prefetched.
+ size_t max_auto_readahead_size = rep->table_options.max_auto_readahead_size;
+ if (max_auto_readahead_size == 0 || initial_auto_readahead_size_ == 0) {
+ return;
+ }
+
+ if (initial_auto_readahead_size_ > max_auto_readahead_size) {
+ initial_auto_readahead_size_ = max_auto_readahead_size;
+ }
+
+ // In case of no_sequential_checking, it will skip the num_file_reads_ and
+ // will always creates the FilePrefetchBuffer.
+ if (no_sequential_checking) {
+ rep->CreateFilePrefetchBufferIfNotExists(
+ initial_auto_readahead_size_, max_auto_readahead_size,
+ &prefetch_buffer_, /*implicit_auto_readahead=*/true,
+ /*num_file_reads=*/0,
+ rep->table_options.num_file_reads_for_auto_readahead);
+ return;
+ }
+
+ size_t len = BlockBasedTable::BlockSizeWithTrailer(handle);
+ size_t offset = handle.offset();
+
+ // If FS supports prefetching (readahead_limit_ will be non zero in that case)
+ // and current block exists in prefetch buffer then return.
+ if (offset + len <= readahead_limit_) {
+ UpdateReadPattern(offset, len);
+ return;
+ }
+
+ if (!IsBlockSequential(offset)) {
+ UpdateReadPattern(offset, len);
+ ResetValues(rep->table_options.initial_auto_readahead_size);
+ return;
+ }
+ UpdateReadPattern(offset, len);
+
+ // Implicit auto readahead, which will be enabled if the number of reads
+ // reached `table_options.num_file_reads_for_auto_readahead` (default: 2) and
+ // scans are sequential.
+ num_file_reads_++;
+ if (num_file_reads_ <= rep->table_options.num_file_reads_for_auto_readahead) {
+ return;
+ }
+
+ if (rep->file->use_direct_io()) {
+ rep->CreateFilePrefetchBufferIfNotExists(
+ initial_auto_readahead_size_, max_auto_readahead_size,
+ &prefetch_buffer_, /*implicit_auto_readahead=*/true, num_file_reads_,
+ rep->table_options.num_file_reads_for_auto_readahead);
+ return;
+ }
+
+ if (readahead_size_ > max_auto_readahead_size) {
+ readahead_size_ = max_auto_readahead_size;
+ }
+
+ // If prefetch is not supported, fall back to use internal prefetch buffer.
+ // Discarding other return status of Prefetch calls intentionally, as
+ // we can fallback to reading from disk if Prefetch fails.
+ Status s = rep->file->Prefetch(
+ handle.offset(),
+ BlockBasedTable::BlockSizeWithTrailer(handle) + readahead_size_,
+ rate_limiter_priority);
+ if (s.IsNotSupported()) {
+ rep->CreateFilePrefetchBufferIfNotExists(
+ initial_auto_readahead_size_, max_auto_readahead_size,
+ &prefetch_buffer_, /*implicit_auto_readahead=*/true, num_file_reads_,
+ rep->table_options.num_file_reads_for_auto_readahead);
+ return;
+ }
+
+ readahead_limit_ = offset + len + readahead_size_;
+ // Keep exponentially increasing readahead size until
+ // max_auto_readahead_size.
+ readahead_size_ = std::min(max_auto_readahead_size, readahead_size_ * 2);
+}
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/block_prefetcher.h b/src/rocksdb/table/block_based/block_prefetcher.h
new file mode 100644
index 000000000..518868a30
--- /dev/null
+++ b/src/rocksdb/table/block_based/block_prefetcher.h
@@ -0,0 +1,72 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+#pragma once
+#include "table/block_based/block_based_table_reader.h"
+
+namespace ROCKSDB_NAMESPACE {
+class BlockPrefetcher {
+ public:
+ explicit BlockPrefetcher(size_t compaction_readahead_size,
+ size_t initial_auto_readahead_size)
+ : compaction_readahead_size_(compaction_readahead_size),
+ readahead_size_(initial_auto_readahead_size),
+ initial_auto_readahead_size_(initial_auto_readahead_size) {}
+
+ void PrefetchIfNeeded(const BlockBasedTable::Rep* rep,
+ const BlockHandle& handle, size_t readahead_size,
+ bool is_for_compaction,
+ const bool no_sequential_checking,
+ Env::IOPriority rate_limiter_priority);
+ FilePrefetchBuffer* prefetch_buffer() { return prefetch_buffer_.get(); }
+
+ void UpdateReadPattern(const uint64_t& offset, const size_t& len) {
+ prev_offset_ = offset;
+ prev_len_ = len;
+ }
+
+ bool IsBlockSequential(const uint64_t& offset) {
+ return (prev_len_ == 0 || (prev_offset_ + prev_len_ == offset));
+ }
+
+ void ResetValues(size_t initial_auto_readahead_size) {
+ num_file_reads_ = 1;
+ // Since initial_auto_readahead_size_ can be different from
+ // the value passed to BlockBasedTableOptions.initial_auto_readahead_size in
+ // case of adaptive_readahead, so fallback the readahead_size_ to that value
+ // in case of reset.
+ initial_auto_readahead_size_ = initial_auto_readahead_size;
+ readahead_size_ = initial_auto_readahead_size_;
+ readahead_limit_ = 0;
+ return;
+ }
+
+ void SetReadaheadState(ReadaheadFileInfo::ReadaheadInfo* readahead_info) {
+ num_file_reads_ = readahead_info->num_file_reads;
+ initial_auto_readahead_size_ = readahead_info->readahead_size;
+ TEST_SYNC_POINT_CALLBACK("BlockPrefetcher::SetReadaheadState",
+ &initial_auto_readahead_size_);
+ }
+
+ private:
+ // Readahead size used in compaction, its value is used only if
+ // lookup_context_.caller = kCompaction.
+ size_t compaction_readahead_size_;
+
+ // readahead_size_ is used if underlying FS supports prefetching.
+ size_t readahead_size_;
+ size_t readahead_limit_ = 0;
+ // initial_auto_readahead_size_ is used if RocksDB uses internal prefetch
+ // buffer.
+ uint64_t initial_auto_readahead_size_;
+ uint64_t num_file_reads_ = 0;
+ uint64_t prev_offset_ = 0;
+ size_t prev_len_ = 0;
+ std::unique_ptr<FilePrefetchBuffer> prefetch_buffer_;
+};
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/block_prefix_index.cc b/src/rocksdb/table/block_based/block_prefix_index.cc
new file mode 100644
index 000000000..c83701d69
--- /dev/null
+++ b/src/rocksdb/table/block_based/block_prefix_index.cc
@@ -0,0 +1,226 @@
+// Copyright (c) 2011-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 "table/block_based/block_prefix_index.h"
+
+#include <vector>
+
+#include "memory/arena.h"
+#include "rocksdb/comparator.h"
+#include "rocksdb/slice.h"
+#include "rocksdb/slice_transform.h"
+#include "util/coding.h"
+#include "util/hash.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+inline uint32_t Hash(const Slice& s) {
+ return ROCKSDB_NAMESPACE::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;
+};
+
+class BlockPrefixIndex::Builder {
+ public:
+ 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(const SliceTransform* prefix_extractor) {
+ // For now, use roughly 1:1 prefix to bucket ratio.
+ uint32_t num_buckets = static_cast<uint32_t>(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;
+ num_blocks_per_bucket[bucket] += (current->num_blocks + distance - 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(total_block_array_entries > 0);
+ 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 iter = 0; iter < current->num_blocks; iter++) {
+ *last_block = current->end_block - iter;
+ 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(prefix_extractor, num_buckets, buckets,
+ total_block_array_entries, block_array_buffer);
+ }
+
+ private:
+ std::vector<PrefixRecord*> prefixes_;
+ Arena arena_;
+};
+
+Status BlockPrefixIndex::Create(const SliceTransform* 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;
+
+ 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(prefix_extractor);
+ }
+
+ return s;
+}
+
+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_NAMESPACE
diff --git a/src/rocksdb/table/block_based/block_prefix_index.h b/src/rocksdb/table/block_based/block_prefix_index.h
new file mode 100644
index 000000000..4db8e2c65
--- /dev/null
+++ b/src/rocksdb/table/block_based/block_prefix_index.h
@@ -0,0 +1,70 @@
+// Copyright (c) 2011-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).
+#pragma once
+
+#include <stdint.h>
+
+#include "db/dbformat.h"
+#include "rocksdb/status.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+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.
+ 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.
+ // Note: table reader (caller) is responsible for keeping shared_ptr to
+ // underlying prefix extractor
+ // @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* prefix_extractor, uint32_t num_buckets,
+ uint32_t* buckets, uint32_t num_block_array_buffer_entries,
+ uint32_t* block_array_buffer)
+ : internal_prefix_extractor_(prefix_extractor),
+ num_buckets_(num_buckets),
+ num_block_array_buffer_entries_(num_block_array_buffer_entries),
+ buckets_(buckets),
+ block_array_buffer_(block_array_buffer) {}
+
+ InternalKeySliceTransform internal_prefix_extractor_;
+
+ uint32_t num_buckets_;
+ uint32_t num_block_array_buffer_entries_;
+ uint32_t* buckets_;
+ uint32_t* block_array_buffer_;
+};
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/block_test.cc b/src/rocksdb/table/block_based/block_test.cc
new file mode 100644
index 000000000..83b87fe79
--- /dev/null
+++ b/src/rocksdb/table/block_based/block_test.cc
@@ -0,0 +1,627 @@
+// Copyright (c) 2011-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 "table/block_based/block.h"
+
+#include <stdio.h>
+
+#include <algorithm>
+#include <set>
+#include <string>
+#include <unordered_set>
+#include <utility>
+#include <vector>
+
+#include "db/dbformat.h"
+#include "db/memtable.h"
+#include "db/write_batch_internal.h"
+#include "rocksdb/db.h"
+#include "rocksdb/env.h"
+#include "rocksdb/iterator.h"
+#include "rocksdb/slice_transform.h"
+#include "rocksdb/table.h"
+#include "table/block_based/block_based_table_reader.h"
+#include "table/block_based/block_builder.h"
+#include "table/format.h"
+#include "test_util/testharness.h"
+#include "test_util/testutil.h"
+#include "util/random.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+std::string GenerateInternalKey(int primary_key, int secondary_key,
+ int padding_size, Random *rnd) {
+ char buf[50];
+ char *p = &buf[0];
+ snprintf(buf, sizeof(buf), "%6d%4d", primary_key, secondary_key);
+ std::string k(p);
+ if (padding_size) {
+ k += rnd->RandomString(padding_size);
+ }
+ AppendInternalKeyFooter(&k, 0 /* seqno */, kTypeValue);
+
+ return k;
+}
+
+// Generate random key value pairs.
+// The generated key will be sorted. You can tune the parameters to generated
+// different kinds of test key/value pairs for different scenario.
+void GenerateRandomKVs(std::vector<std::string> *keys,
+ std::vector<std::string> *values, const int from,
+ const int len, const int step = 1,
+ const int padding_size = 0,
+ const int keys_share_prefix = 1) {
+ Random rnd(302);
+
+ // generate different prefix
+ for (int i = from; i < from + len; i += step) {
+ // generating keys that shares the prefix
+ for (int j = 0; j < keys_share_prefix; ++j) {
+ // `DataBlockIter` assumes it reads only internal keys.
+ keys->emplace_back(GenerateInternalKey(i, j, padding_size, &rnd));
+
+ // 100 bytes values
+ values->emplace_back(rnd.RandomString(100));
+ }
+ }
+}
+
+class BlockTest : public testing::Test {};
+
+// block test
+TEST_F(BlockTest, SimpleTest) {
+ Random rnd(301);
+ Options options = Options();
+
+ std::vector<std::string> keys;
+ std::vector<std::string> values;
+ BlockBuilder builder(16);
+ int num_records = 100000;
+
+ GenerateRandomKVs(&keys, &values, 0, num_records);
+ // add a bunch of records to a block
+ for (int i = 0; i < num_records; i++) {
+ builder.Add(keys[i], values[i]);
+ }
+
+ // read serialized contents of the block
+ Slice rawblock = builder.Finish();
+
+ // create block reader
+ BlockContents contents;
+ contents.data = rawblock;
+ Block reader(std::move(contents));
+
+ // read contents of block sequentially
+ int count = 0;
+ InternalIterator *iter =
+ reader.NewDataIterator(options.comparator, kDisableGlobalSequenceNumber);
+ for (iter->SeekToFirst(); iter->Valid(); count++, iter->Next()) {
+ // read kv from block
+ Slice k = iter->key();
+ Slice v = iter->value();
+
+ // compare with lookaside array
+ ASSERT_EQ(k.ToString().compare(keys[count]), 0);
+ ASSERT_EQ(v.ToString().compare(values[count]), 0);
+ }
+ delete iter;
+
+ // read block contents randomly
+ iter =
+ reader.NewDataIterator(options.comparator, kDisableGlobalSequenceNumber);
+ for (int i = 0; i < num_records; i++) {
+ // find a random key in the lookaside array
+ int index = rnd.Uniform(num_records);
+ Slice k(keys[index]);
+
+ // search in block for this key
+ iter->Seek(k);
+ ASSERT_TRUE(iter->Valid());
+ Slice v = iter->value();
+ ASSERT_EQ(v.ToString().compare(values[index]), 0);
+ }
+ delete iter;
+}
+
+// return the block contents
+BlockContents GetBlockContents(std::unique_ptr<BlockBuilder> *builder,
+ const std::vector<std::string> &keys,
+ const std::vector<std::string> &values,
+ const int /*prefix_group_size*/ = 1) {
+ builder->reset(new BlockBuilder(1 /* restart interval */));
+
+ // Add only half of the keys
+ for (size_t i = 0; i < keys.size(); ++i) {
+ (*builder)->Add(keys[i], values[i]);
+ }
+ Slice rawblock = (*builder)->Finish();
+
+ BlockContents contents;
+ contents.data = rawblock;
+
+ return contents;
+}
+
+void CheckBlockContents(BlockContents contents, const int max_key,
+ const std::vector<std::string> &keys,
+ const std::vector<std::string> &values) {
+ const size_t prefix_size = 6;
+ // create block reader
+ BlockContents contents_ref(contents.data);
+ Block reader1(std::move(contents));
+ Block reader2(std::move(contents_ref));
+
+ std::unique_ptr<const SliceTransform> prefix_extractor(
+ NewFixedPrefixTransform(prefix_size));
+
+ std::unique_ptr<InternalIterator> regular_iter(reader2.NewDataIterator(
+ BytewiseComparator(), kDisableGlobalSequenceNumber));
+
+ // Seek existent keys
+ for (size_t i = 0; i < keys.size(); i++) {
+ regular_iter->Seek(keys[i]);
+ ASSERT_OK(regular_iter->status());
+ ASSERT_TRUE(regular_iter->Valid());
+
+ Slice v = regular_iter->value();
+ ASSERT_EQ(v.ToString().compare(values[i]), 0);
+ }
+
+ // Seek non-existent keys.
+ // For hash index, if no key with a given prefix is not found, iterator will
+ // simply be set as invalid; whereas the binary search based iterator will
+ // return the one that is closest.
+ for (int i = 1; i < max_key - 1; i += 2) {
+ // `DataBlockIter` assumes its APIs receive only internal keys.
+ auto key = GenerateInternalKey(i, 0, 0, nullptr);
+ regular_iter->Seek(key);
+ ASSERT_TRUE(regular_iter->Valid());
+ }
+}
+
+// In this test case, no two key share same prefix.
+TEST_F(BlockTest, SimpleIndexHash) {
+ const int kMaxKey = 100000;
+ std::vector<std::string> keys;
+ std::vector<std::string> values;
+ GenerateRandomKVs(&keys, &values, 0 /* first key id */,
+ kMaxKey /* last key id */, 2 /* step */,
+ 8 /* padding size (8 bytes randomly generated suffix) */);
+
+ std::unique_ptr<BlockBuilder> builder;
+ auto contents = GetBlockContents(&builder, keys, values);
+
+ CheckBlockContents(std::move(contents), kMaxKey, keys, values);
+}
+
+TEST_F(BlockTest, IndexHashWithSharedPrefix) {
+ const int kMaxKey = 100000;
+ // for each prefix, there will be 5 keys starts with it.
+ const int kPrefixGroup = 5;
+ std::vector<std::string> keys;
+ std::vector<std::string> values;
+ // Generate keys with same prefix.
+ GenerateRandomKVs(&keys, &values, 0, // first key id
+ kMaxKey, // last key id
+ 2, // step
+ 10, // padding size,
+ kPrefixGroup);
+
+ std::unique_ptr<BlockBuilder> builder;
+ auto contents = GetBlockContents(&builder, keys, values, kPrefixGroup);
+
+ CheckBlockContents(std::move(contents), kMaxKey, keys, values);
+}
+
+// A slow and accurate version of BlockReadAmpBitmap that simply store
+// all the marked ranges in a set.
+class BlockReadAmpBitmapSlowAndAccurate {
+ public:
+ void Mark(size_t start_offset, size_t end_offset) {
+ assert(end_offset >= start_offset);
+ marked_ranges_.emplace(end_offset, start_offset);
+ }
+
+ void ResetCheckSequence() { iter_valid_ = false; }
+
+ // Return true if any byte in this range was Marked
+ // This does linear search from the previous position. When calling
+ // multiple times, `offset` needs to be incremental to get correct results.
+ // Call ResetCheckSequence() to reset it.
+ bool IsPinMarked(size_t offset) {
+ if (iter_valid_) {
+ // Has existing iterator, try linear search from
+ // the iterator.
+ for (int i = 0; i < 64; i++) {
+ if (offset < iter_->second) {
+ return false;
+ }
+ if (offset <= iter_->first) {
+ return true;
+ }
+
+ iter_++;
+ if (iter_ == marked_ranges_.end()) {
+ iter_valid_ = false;
+ return false;
+ }
+ }
+ }
+ // Initial call or have linear searched too many times.
+ // Do binary search.
+ iter_ = marked_ranges_.lower_bound(
+ std::make_pair(offset, static_cast<size_t>(0)));
+ if (iter_ == marked_ranges_.end()) {
+ iter_valid_ = false;
+ return false;
+ }
+ iter_valid_ = true;
+ return offset <= iter_->first && offset >= iter_->second;
+ }
+
+ private:
+ std::set<std::pair<size_t, size_t>> marked_ranges_;
+ std::set<std::pair<size_t, size_t>>::iterator iter_;
+ bool iter_valid_ = false;
+};
+
+TEST_F(BlockTest, BlockReadAmpBitmap) {
+ uint32_t pin_offset = 0;
+ SyncPoint::GetInstance()->SetCallBack(
+ "BlockReadAmpBitmap:rnd", [&pin_offset](void *arg) {
+ pin_offset = *(static_cast<uint32_t *>(arg));
+ });
+ SyncPoint::GetInstance()->EnableProcessing();
+ std::vector<size_t> block_sizes = {
+ 1, // 1 byte
+ 32, // 32 bytes
+ 61, // 61 bytes
+ 64, // 64 bytes
+ 512, // 0.5 KB
+ 1024, // 1 KB
+ 1024 * 4, // 4 KB
+ 1024 * 10, // 10 KB
+ 1024 * 50, // 50 KB
+ 1024 * 1024 * 4, // 5 MB
+ 777,
+ 124653,
+ };
+ const size_t kBytesPerBit = 64;
+
+ Random rnd(301);
+ for (size_t block_size : block_sizes) {
+ std::shared_ptr<Statistics> stats = ROCKSDB_NAMESPACE::CreateDBStatistics();
+ BlockReadAmpBitmap read_amp_bitmap(block_size, kBytesPerBit, stats.get());
+ BlockReadAmpBitmapSlowAndAccurate read_amp_slow_and_accurate;
+
+ size_t needed_bits = (block_size / kBytesPerBit);
+ if (block_size % kBytesPerBit != 0) {
+ needed_bits++;
+ }
+
+ ASSERT_EQ(stats->getTickerCount(READ_AMP_TOTAL_READ_BYTES), block_size);
+
+ // Generate some random entries
+ std::vector<size_t> random_entry_offsets;
+ for (int i = 0; i < 1000; i++) {
+ random_entry_offsets.push_back(rnd.Next() % block_size);
+ }
+ std::sort(random_entry_offsets.begin(), random_entry_offsets.end());
+ auto it =
+ std::unique(random_entry_offsets.begin(), random_entry_offsets.end());
+ random_entry_offsets.resize(
+ std::distance(random_entry_offsets.begin(), it));
+
+ std::vector<std::pair<size_t, size_t>> random_entries;
+ for (size_t i = 0; i < random_entry_offsets.size(); i++) {
+ size_t entry_start = random_entry_offsets[i];
+ size_t entry_end;
+ if (i + 1 < random_entry_offsets.size()) {
+ entry_end = random_entry_offsets[i + 1] - 1;
+ } else {
+ entry_end = block_size - 1;
+ }
+ random_entries.emplace_back(entry_start, entry_end);
+ }
+
+ for (size_t i = 0; i < random_entries.size(); i++) {
+ read_amp_slow_and_accurate.ResetCheckSequence();
+ auto &current_entry = random_entries[rnd.Next() % random_entries.size()];
+
+ read_amp_bitmap.Mark(static_cast<uint32_t>(current_entry.first),
+ static_cast<uint32_t>(current_entry.second));
+ read_amp_slow_and_accurate.Mark(current_entry.first,
+ current_entry.second);
+
+ size_t total_bits = 0;
+ for (size_t bit_idx = 0; bit_idx < needed_bits; bit_idx++) {
+ total_bits += read_amp_slow_and_accurate.IsPinMarked(
+ bit_idx * kBytesPerBit + pin_offset);
+ }
+ size_t expected_estimate_useful = total_bits * kBytesPerBit;
+ size_t got_estimate_useful =
+ stats->getTickerCount(READ_AMP_ESTIMATE_USEFUL_BYTES);
+ ASSERT_EQ(expected_estimate_useful, got_estimate_useful);
+ }
+ }
+ SyncPoint::GetInstance()->DisableProcessing();
+ SyncPoint::GetInstance()->ClearAllCallBacks();
+}
+
+TEST_F(BlockTest, BlockWithReadAmpBitmap) {
+ Random rnd(301);
+ Options options = Options();
+
+ std::vector<std::string> keys;
+ std::vector<std::string> values;
+ BlockBuilder builder(16);
+ int num_records = 10000;
+
+ GenerateRandomKVs(&keys, &values, 0, num_records, 1);
+ // add a bunch of records to a block
+ for (int i = 0; i < num_records; i++) {
+ builder.Add(keys[i], values[i]);
+ }
+
+ Slice rawblock = builder.Finish();
+ const size_t kBytesPerBit = 8;
+
+ // Read the block sequentially using Next()
+ {
+ std::shared_ptr<Statistics> stats = ROCKSDB_NAMESPACE::CreateDBStatistics();
+
+ // create block reader
+ BlockContents contents;
+ contents.data = rawblock;
+ Block reader(std::move(contents), kBytesPerBit, stats.get());
+
+ // read contents of block sequentially
+ size_t read_bytes = 0;
+ DataBlockIter *iter = reader.NewDataIterator(
+ options.comparator, kDisableGlobalSequenceNumber, nullptr, stats.get());
+ for (iter->SeekToFirst(); iter->Valid(); iter->Next()) {
+ iter->value();
+ read_bytes += iter->TEST_CurrentEntrySize();
+
+ double semi_acc_read_amp =
+ static_cast<double>(read_bytes) / rawblock.size();
+ double read_amp = static_cast<double>(stats->getTickerCount(
+ READ_AMP_ESTIMATE_USEFUL_BYTES)) /
+ stats->getTickerCount(READ_AMP_TOTAL_READ_BYTES);
+
+ // Error in read amplification will be less than 1% if we are reading
+ // sequentially
+ double error_pct = fabs(semi_acc_read_amp - read_amp) * 100;
+ EXPECT_LT(error_pct, 1);
+ }
+
+ delete iter;
+ }
+
+ // Read the block sequentially using Seek()
+ {
+ std::shared_ptr<Statistics> stats = ROCKSDB_NAMESPACE::CreateDBStatistics();
+
+ // create block reader
+ BlockContents contents;
+ contents.data = rawblock;
+ Block reader(std::move(contents), kBytesPerBit, stats.get());
+
+ size_t read_bytes = 0;
+ DataBlockIter *iter = reader.NewDataIterator(
+ options.comparator, kDisableGlobalSequenceNumber, nullptr, stats.get());
+ for (int i = 0; i < num_records; i++) {
+ Slice k(keys[i]);
+
+ // search in block for this key
+ iter->Seek(k);
+ iter->value();
+ read_bytes += iter->TEST_CurrentEntrySize();
+
+ double semi_acc_read_amp =
+ static_cast<double>(read_bytes) / rawblock.size();
+ double read_amp = static_cast<double>(stats->getTickerCount(
+ READ_AMP_ESTIMATE_USEFUL_BYTES)) /
+ stats->getTickerCount(READ_AMP_TOTAL_READ_BYTES);
+
+ // Error in read amplification will be less than 1% if we are reading
+ // sequentially
+ double error_pct = fabs(semi_acc_read_amp - read_amp) * 100;
+ EXPECT_LT(error_pct, 1);
+ }
+ delete iter;
+ }
+
+ // Read the block randomly
+ {
+ std::shared_ptr<Statistics> stats = ROCKSDB_NAMESPACE::CreateDBStatistics();
+
+ // create block reader
+ BlockContents contents;
+ contents.data = rawblock;
+ Block reader(std::move(contents), kBytesPerBit, stats.get());
+
+ size_t read_bytes = 0;
+ DataBlockIter *iter = reader.NewDataIterator(
+ options.comparator, kDisableGlobalSequenceNumber, nullptr, stats.get());
+ std::unordered_set<int> read_keys;
+ for (int i = 0; i < num_records; i++) {
+ int index = rnd.Uniform(num_records);
+ Slice k(keys[index]);
+
+ iter->Seek(k);
+ iter->value();
+ if (read_keys.find(index) == read_keys.end()) {
+ read_keys.insert(index);
+ read_bytes += iter->TEST_CurrentEntrySize();
+ }
+
+ double semi_acc_read_amp =
+ static_cast<double>(read_bytes) / rawblock.size();
+ double read_amp = static_cast<double>(stats->getTickerCount(
+ READ_AMP_ESTIMATE_USEFUL_BYTES)) /
+ stats->getTickerCount(READ_AMP_TOTAL_READ_BYTES);
+
+ double error_pct = fabs(semi_acc_read_amp - read_amp) * 100;
+ // Error in read amplification will be less than 2% if we are reading
+ // randomly
+ EXPECT_LT(error_pct, 2);
+ }
+ delete iter;
+ }
+}
+
+TEST_F(BlockTest, ReadAmpBitmapPow2) {
+ std::shared_ptr<Statistics> stats = ROCKSDB_NAMESPACE::CreateDBStatistics();
+ ASSERT_EQ(BlockReadAmpBitmap(100, 1, stats.get()).GetBytesPerBit(), 1u);
+ ASSERT_EQ(BlockReadAmpBitmap(100, 2, stats.get()).GetBytesPerBit(), 2u);
+ ASSERT_EQ(BlockReadAmpBitmap(100, 4, stats.get()).GetBytesPerBit(), 4u);
+ ASSERT_EQ(BlockReadAmpBitmap(100, 8, stats.get()).GetBytesPerBit(), 8u);
+ ASSERT_EQ(BlockReadAmpBitmap(100, 16, stats.get()).GetBytesPerBit(), 16u);
+ ASSERT_EQ(BlockReadAmpBitmap(100, 32, stats.get()).GetBytesPerBit(), 32u);
+
+ ASSERT_EQ(BlockReadAmpBitmap(100, 3, stats.get()).GetBytesPerBit(), 2u);
+ ASSERT_EQ(BlockReadAmpBitmap(100, 7, stats.get()).GetBytesPerBit(), 4u);
+ ASSERT_EQ(BlockReadAmpBitmap(100, 11, stats.get()).GetBytesPerBit(), 8u);
+ ASSERT_EQ(BlockReadAmpBitmap(100, 17, stats.get()).GetBytesPerBit(), 16u);
+ ASSERT_EQ(BlockReadAmpBitmap(100, 33, stats.get()).GetBytesPerBit(), 32u);
+ ASSERT_EQ(BlockReadAmpBitmap(100, 35, stats.get()).GetBytesPerBit(), 32u);
+}
+
+class IndexBlockTest
+ : public testing::Test,
+ public testing::WithParamInterface<std::tuple<bool, bool>> {
+ public:
+ IndexBlockTest() = default;
+
+ bool useValueDeltaEncoding() const { return std::get<0>(GetParam()); }
+ bool includeFirstKey() const { return std::get<1>(GetParam()); }
+};
+
+// Similar to GenerateRandomKVs but for index block contents.
+void GenerateRandomIndexEntries(std::vector<std::string> *separators,
+ std::vector<BlockHandle> *block_handles,
+ std::vector<std::string> *first_keys,
+ const int len) {
+ Random rnd(42);
+
+ // For each of `len` blocks, we need to generate a first and last key.
+ // Let's generate n*2 random keys, sort them, group into consecutive pairs.
+ std::set<std::string> keys;
+ while ((int)keys.size() < len * 2) {
+ // Keys need to be at least 8 bytes long to look like internal keys.
+ keys.insert(test::RandomKey(&rnd, 12));
+ }
+
+ uint64_t offset = 0;
+ for (auto it = keys.begin(); it != keys.end();) {
+ first_keys->emplace_back(*it++);
+ separators->emplace_back(*it++);
+ uint64_t size = rnd.Uniform(1024 * 16);
+ BlockHandle handle(offset, size);
+ offset += size + BlockBasedTable::kBlockTrailerSize;
+ block_handles->emplace_back(handle);
+ }
+}
+
+TEST_P(IndexBlockTest, IndexValueEncodingTest) {
+ Random rnd(301);
+ Options options = Options();
+
+ std::vector<std::string> separators;
+ std::vector<BlockHandle> block_handles;
+ std::vector<std::string> first_keys;
+ const bool kUseDeltaEncoding = true;
+ BlockBuilder builder(16, kUseDeltaEncoding, useValueDeltaEncoding());
+ int num_records = 100;
+
+ GenerateRandomIndexEntries(&separators, &block_handles, &first_keys,
+ num_records);
+ BlockHandle last_encoded_handle;
+ for (int i = 0; i < num_records; i++) {
+ IndexValue entry(block_handles[i], first_keys[i]);
+ std::string encoded_entry;
+ std::string delta_encoded_entry;
+ entry.EncodeTo(&encoded_entry, includeFirstKey(), nullptr);
+ if (useValueDeltaEncoding() && i > 0) {
+ entry.EncodeTo(&delta_encoded_entry, includeFirstKey(),
+ &last_encoded_handle);
+ }
+ last_encoded_handle = entry.handle;
+ const Slice delta_encoded_entry_slice(delta_encoded_entry);
+ builder.Add(separators[i], encoded_entry, &delta_encoded_entry_slice);
+ }
+
+ // read serialized contents of the block
+ Slice rawblock = builder.Finish();
+
+ // create block reader
+ BlockContents contents;
+ contents.data = rawblock;
+ Block reader(std::move(contents));
+
+ const bool kTotalOrderSeek = true;
+ const bool kIncludesSeq = true;
+ const bool kValueIsFull = !useValueDeltaEncoding();
+ IndexBlockIter *kNullIter = nullptr;
+ Statistics *kNullStats = nullptr;
+ // read contents of block sequentially
+ InternalIteratorBase<IndexValue> *iter = reader.NewIndexIterator(
+ options.comparator, kDisableGlobalSequenceNumber, kNullIter, kNullStats,
+ kTotalOrderSeek, includeFirstKey(), kIncludesSeq, kValueIsFull);
+ iter->SeekToFirst();
+ for (int index = 0; index < num_records; ++index) {
+ ASSERT_TRUE(iter->Valid());
+
+ Slice k = iter->key();
+ IndexValue v = iter->value();
+
+ EXPECT_EQ(separators[index], k.ToString());
+ EXPECT_EQ(block_handles[index].offset(), v.handle.offset());
+ EXPECT_EQ(block_handles[index].size(), v.handle.size());
+ EXPECT_EQ(includeFirstKey() ? first_keys[index] : "",
+ v.first_internal_key.ToString());
+
+ iter->Next();
+ }
+ delete iter;
+
+ // read block contents randomly
+ iter = reader.NewIndexIterator(
+ options.comparator, kDisableGlobalSequenceNumber, kNullIter, kNullStats,
+ kTotalOrderSeek, includeFirstKey(), kIncludesSeq, kValueIsFull);
+ for (int i = 0; i < num_records * 2; i++) {
+ // find a random key in the lookaside array
+ int index = rnd.Uniform(num_records);
+ Slice k(separators[index]);
+
+ // search in block for this key
+ iter->Seek(k);
+ ASSERT_TRUE(iter->Valid());
+ IndexValue v = iter->value();
+ EXPECT_EQ(separators[index], iter->key().ToString());
+ EXPECT_EQ(block_handles[index].offset(), v.handle.offset());
+ EXPECT_EQ(block_handles[index].size(), v.handle.size());
+ EXPECT_EQ(includeFirstKey() ? first_keys[index] : "",
+ v.first_internal_key.ToString());
+ }
+ delete iter;
+}
+
+INSTANTIATE_TEST_CASE_P(P, IndexBlockTest,
+ ::testing::Values(std::make_tuple(false, false),
+ std::make_tuple(false, true),
+ std::make_tuple(true, false),
+ std::make_tuple(true, true)));
+
+} // namespace ROCKSDB_NAMESPACE
+
+int main(int argc, char **argv) {
+ ROCKSDB_NAMESPACE::port::InstallStackTraceHandler();
+ ::testing::InitGoogleTest(&argc, argv);
+ return RUN_ALL_TESTS();
+}
diff --git a/src/rocksdb/table/block_based/block_type.h b/src/rocksdb/table/block_based/block_type.h
new file mode 100644
index 000000000..a9d6a1a77
--- /dev/null
+++ b/src/rocksdb/table/block_based/block_type.h
@@ -0,0 +1,34 @@
+// Copyright (c) 2011-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).
+
+#pragma once
+
+#include <cstdint>
+
+#include "rocksdb/rocksdb_namespace.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+// Represents the types of blocks used in the block based table format.
+// See https://github.com/facebook/rocksdb/wiki/Rocksdb-BlockBasedTable-Format
+// for details.
+// For code sanity, BlockType should imply a specific TBlocklike for
+// BlocklikeTraits.
+enum class BlockType : uint8_t {
+ kData,
+ kFilter, // for second level partitioned filters and full filters
+ kFilterPartitionIndex, // for top-level index of filter partitions
+ kProperties,
+ kCompressionDictionary,
+ kRangeDeletion,
+ kHashIndexPrefixes,
+ kHashIndexMetadata,
+ kMetaIndex,
+ kIndex,
+ // Note: keep kInvalid the last value when adding new enum values.
+ kInvalid
+};
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/cachable_entry.h b/src/rocksdb/table/block_based/cachable_entry.h
new file mode 100644
index 000000000..ad8acb18d
--- /dev/null
+++ b/src/rocksdb/table/block_based/cachable_entry.h
@@ -0,0 +1,232 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2012 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#pragma once
+
+#include <cassert>
+
+#include "port/likely.h"
+#include "rocksdb/cache.h"
+#include "rocksdb/cleanable.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+// CachableEntry is a handle to an object that may or may not be in the block
+// cache. It is used in a variety of ways:
+//
+// 1) It may refer to an object in the block cache. In this case, cache_ and
+// cache_handle_ are not nullptr, and the cache handle has to be released when
+// the CachableEntry is destroyed (the lifecycle of the cached object, on the
+// other hand, is managed by the cache itself).
+// 2) It may uniquely own the (non-cached) object it refers to (examples include
+// a block read directly from file, or uncompressed blocks when there is a
+// compressed block cache but no uncompressed block cache). In such cases, the
+// object has to be destroyed when the CachableEntry is destroyed.
+// 3) It may point to an object (cached or not) without owning it. In this case,
+// no action is needed when the CachableEntry is destroyed.
+// 4) Sometimes, management of a cached or owned object (see #1 and #2 above)
+// is transferred to some other object. This is used for instance with iterators
+// (where cleanup is performed using a chain of cleanup functions,
+// see Cleanable).
+//
+// Because of #1 and #2 above, copying a CachableEntry is not safe (and thus not
+// allowed); hence, this is a move-only type, where a move transfers the
+// management responsibilities, and leaves the source object in an empty state.
+
+template <class T>
+class CachableEntry {
+ public:
+ CachableEntry() = default;
+
+ CachableEntry(T* value, Cache* cache, Cache::Handle* cache_handle,
+ bool own_value)
+ : value_(value),
+ cache_(cache),
+ cache_handle_(cache_handle),
+ own_value_(own_value) {
+ assert(value_ != nullptr ||
+ (cache_ == nullptr && cache_handle_ == nullptr && !own_value_));
+ assert(!!cache_ == !!cache_handle_);
+ assert(!cache_handle_ || !own_value_);
+ }
+
+ CachableEntry(const CachableEntry&) = delete;
+ CachableEntry& operator=(const CachableEntry&) = delete;
+
+ CachableEntry(CachableEntry&& rhs) noexcept
+ : value_(rhs.value_),
+ cache_(rhs.cache_),
+ cache_handle_(rhs.cache_handle_),
+ own_value_(rhs.own_value_) {
+ assert(value_ != nullptr ||
+ (cache_ == nullptr && cache_handle_ == nullptr && !own_value_));
+ assert(!!cache_ == !!cache_handle_);
+ assert(!cache_handle_ || !own_value_);
+
+ rhs.ResetFields();
+ }
+
+ CachableEntry& operator=(CachableEntry&& rhs) noexcept {
+ if (UNLIKELY(this == &rhs)) {
+ return *this;
+ }
+
+ ReleaseResource();
+
+ value_ = rhs.value_;
+ cache_ = rhs.cache_;
+ cache_handle_ = rhs.cache_handle_;
+ own_value_ = rhs.own_value_;
+
+ assert(value_ != nullptr ||
+ (cache_ == nullptr && cache_handle_ == nullptr && !own_value_));
+ assert(!!cache_ == !!cache_handle_);
+ assert(!cache_handle_ || !own_value_);
+
+ rhs.ResetFields();
+
+ return *this;
+ }
+
+ ~CachableEntry() { ReleaseResource(); }
+
+ bool IsEmpty() const {
+ return value_ == nullptr && cache_ == nullptr && cache_handle_ == nullptr &&
+ !own_value_;
+ }
+
+ bool IsCached() const {
+ assert(!!cache_ == !!cache_handle_);
+
+ return cache_handle_ != nullptr;
+ }
+
+ T* GetValue() const { return value_; }
+ Cache* GetCache() const { return cache_; }
+ Cache::Handle* GetCacheHandle() const { return cache_handle_; }
+ bool GetOwnValue() const { return own_value_; }
+
+ void Reset() {
+ ReleaseResource();
+ ResetFields();
+ }
+
+ void TransferTo(Cleanable* cleanable) {
+ if (cleanable) {
+ if (cache_handle_ != nullptr) {
+ assert(cache_ != nullptr);
+ cleanable->RegisterCleanup(&ReleaseCacheHandle, cache_, cache_handle_);
+ } else if (own_value_) {
+ cleanable->RegisterCleanup(&DeleteValue, value_, nullptr);
+ }
+ }
+
+ ResetFields();
+ }
+
+ void SetOwnedValue(std::unique_ptr<T>&& value) {
+ assert(value.get() != nullptr);
+
+ if (UNLIKELY(value_ == value.get() && own_value_)) {
+ assert(cache_ == nullptr && cache_handle_ == nullptr);
+ return;
+ }
+
+ Reset();
+
+ value_ = value.release();
+ own_value_ = true;
+ }
+
+ void SetUnownedValue(T* value) {
+ assert(value != nullptr);
+
+ if (UNLIKELY(value_ == value && cache_ == nullptr &&
+ cache_handle_ == nullptr && !own_value_)) {
+ return;
+ }
+
+ Reset();
+
+ value_ = value;
+ assert(!own_value_);
+ }
+
+ void SetCachedValue(T* value, Cache* cache, Cache::Handle* cache_handle) {
+ assert(cache != nullptr);
+ assert(cache_handle != nullptr);
+
+ if (UNLIKELY(value_ == value && cache_ == cache &&
+ cache_handle_ == cache_handle && !own_value_)) {
+ return;
+ }
+
+ Reset();
+
+ value_ = value;
+ cache_ = cache;
+ cache_handle_ = cache_handle;
+ assert(!own_value_);
+ }
+
+ void UpdateCachedValue() {
+ assert(cache_ != nullptr);
+ assert(cache_handle_ != nullptr);
+
+ value_ = static_cast<T*>(cache_->Value(cache_handle_));
+ }
+
+ bool IsReady() {
+ if (!own_value_) {
+ assert(cache_ != nullptr);
+ assert(cache_handle_ != nullptr);
+ return cache_->IsReady(cache_handle_);
+ }
+ return true;
+ }
+
+ private:
+ void ReleaseResource() noexcept {
+ if (LIKELY(cache_handle_ != nullptr)) {
+ assert(cache_ != nullptr);
+ cache_->Release(cache_handle_);
+ } else if (own_value_) {
+ delete value_;
+ }
+ }
+
+ void ResetFields() noexcept {
+ value_ = nullptr;
+ cache_ = nullptr;
+ cache_handle_ = nullptr;
+ own_value_ = false;
+ }
+
+ static void ReleaseCacheHandle(void* arg1, void* arg2) {
+ Cache* const cache = static_cast<Cache*>(arg1);
+ assert(cache);
+
+ Cache::Handle* const cache_handle = static_cast<Cache::Handle*>(arg2);
+ assert(cache_handle);
+
+ cache->Release(cache_handle);
+ }
+
+ static void DeleteValue(void* arg1, void* /* arg2 */) {
+ delete static_cast<T*>(arg1);
+ }
+
+ private:
+ T* value_ = nullptr;
+ Cache* cache_ = nullptr;
+ Cache::Handle* cache_handle_ = nullptr;
+ bool own_value_ = false;
+};
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/data_block_footer.cc b/src/rocksdb/table/block_based/data_block_footer.cc
new file mode 100644
index 000000000..5d5d8ed55
--- /dev/null
+++ b/src/rocksdb/table/block_based/data_block_footer.cc
@@ -0,0 +1,59 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#include "table/block_based/data_block_footer.h"
+
+#include "rocksdb/table.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+const int kDataBlockIndexTypeBitShift = 31;
+
+// 0x7FFFFFFF
+const uint32_t kMaxNumRestarts = (1u << kDataBlockIndexTypeBitShift) - 1u;
+
+// 0x7FFFFFFF
+const uint32_t kNumRestartsMask = (1u << kDataBlockIndexTypeBitShift) - 1u;
+
+uint32_t PackIndexTypeAndNumRestarts(
+ BlockBasedTableOptions::DataBlockIndexType index_type,
+ uint32_t num_restarts) {
+ if (num_restarts > kMaxNumRestarts) {
+ assert(0); // mute travis "unused" warning
+ }
+
+ uint32_t block_footer = num_restarts;
+ if (index_type == BlockBasedTableOptions::kDataBlockBinaryAndHash) {
+ block_footer |= 1u << kDataBlockIndexTypeBitShift;
+ } else if (index_type != BlockBasedTableOptions::kDataBlockBinarySearch) {
+ assert(0);
+ }
+
+ return block_footer;
+}
+
+void UnPackIndexTypeAndNumRestarts(
+ uint32_t block_footer,
+ BlockBasedTableOptions::DataBlockIndexType* index_type,
+ uint32_t* num_restarts) {
+ if (index_type) {
+ if (block_footer & 1u << kDataBlockIndexTypeBitShift) {
+ *index_type = BlockBasedTableOptions::kDataBlockBinaryAndHash;
+ } else {
+ *index_type = BlockBasedTableOptions::kDataBlockBinarySearch;
+ }
+ }
+
+ if (num_restarts) {
+ *num_restarts = block_footer & kNumRestartsMask;
+ assert(*num_restarts <= kMaxNumRestarts);
+ }
+}
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/data_block_footer.h b/src/rocksdb/table/block_based/data_block_footer.h
new file mode 100644
index 000000000..c1cfd4730
--- /dev/null
+++ b/src/rocksdb/table/block_based/data_block_footer.h
@@ -0,0 +1,25 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#pragma once
+
+#include "rocksdb/table.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+uint32_t PackIndexTypeAndNumRestarts(
+ BlockBasedTableOptions::DataBlockIndexType index_type,
+ uint32_t num_restarts);
+
+void UnPackIndexTypeAndNumRestarts(
+ uint32_t block_footer,
+ BlockBasedTableOptions::DataBlockIndexType* index_type,
+ uint32_t* num_restarts);
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/data_block_hash_index.cc b/src/rocksdb/table/block_based/data_block_hash_index.cc
new file mode 100644
index 000000000..c579dcc43
--- /dev/null
+++ b/src/rocksdb/table/block_based/data_block_hash_index.cc
@@ -0,0 +1,94 @@
+// Copyright (c) 2011-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 "table/block_based/data_block_hash_index.h"
+
+#include <string>
+#include <vector>
+
+#include "rocksdb/slice.h"
+#include "util/coding.h"
+#include "util/hash.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+void DataBlockHashIndexBuilder::Add(const Slice& key,
+ const size_t restart_index) {
+ assert(Valid());
+ if (restart_index > kMaxRestartSupportedByHashIndex) {
+ valid_ = false;
+ return;
+ }
+
+ uint32_t hash_value = GetSliceHash(key);
+ hash_and_restart_pairs_.emplace_back(hash_value,
+ static_cast<uint8_t>(restart_index));
+ estimated_num_buckets_ += bucket_per_key_;
+}
+
+void DataBlockHashIndexBuilder::Finish(std::string& buffer) {
+ assert(Valid());
+ uint16_t num_buckets = static_cast<uint16_t>(estimated_num_buckets_);
+
+ if (num_buckets == 0) {
+ num_buckets = 1; // sanity check
+ }
+
+ // The build-in hash cannot well distribute strings when into different
+ // buckets when num_buckets is power of two, resulting in high hash
+ // collision.
+ // We made the num_buckets to be odd to avoid this issue.
+ num_buckets |= 1;
+
+ std::vector<uint8_t> buckets(num_buckets, kNoEntry);
+ // write the restart_index array
+ for (auto& entry : hash_and_restart_pairs_) {
+ uint32_t hash_value = entry.first;
+ uint8_t restart_index = entry.second;
+ uint16_t buck_idx = static_cast<uint16_t>(hash_value % num_buckets);
+ if (buckets[buck_idx] == kNoEntry) {
+ buckets[buck_idx] = restart_index;
+ } else if (buckets[buck_idx] != restart_index) {
+ // same bucket cannot store two different restart_index, mark collision
+ buckets[buck_idx] = kCollision;
+ }
+ }
+
+ for (uint8_t restart_index : buckets) {
+ buffer.append(
+ const_cast<const char*>(reinterpret_cast<char*>(&restart_index)),
+ sizeof(restart_index));
+ }
+
+ // write NUM_BUCK
+ PutFixed16(&buffer, num_buckets);
+
+ assert(buffer.size() <= kMaxBlockSizeSupportedByHashIndex);
+}
+
+void DataBlockHashIndexBuilder::Reset() {
+ estimated_num_buckets_ = 0;
+ valid_ = true;
+ hash_and_restart_pairs_.clear();
+}
+
+void DataBlockHashIndex::Initialize(const char* data, uint16_t size,
+ uint16_t* map_offset) {
+ assert(size >= sizeof(uint16_t)); // NUM_BUCKETS
+ num_buckets_ = DecodeFixed16(data + size - sizeof(uint16_t));
+ assert(num_buckets_ > 0);
+ assert(size > num_buckets_ * sizeof(uint8_t));
+ *map_offset = static_cast<uint16_t>(size - sizeof(uint16_t) -
+ num_buckets_ * sizeof(uint8_t));
+}
+
+uint8_t DataBlockHashIndex::Lookup(const char* data, uint32_t map_offset,
+ const Slice& key) const {
+ uint32_t hash_value = GetSliceHash(key);
+ uint16_t idx = static_cast<uint16_t>(hash_value % num_buckets_);
+ const char* bucket_table = data + map_offset;
+ return static_cast<uint8_t>(*(bucket_table + idx * sizeof(uint8_t)));
+}
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/data_block_hash_index.h b/src/rocksdb/table/block_based/data_block_hash_index.h
new file mode 100644
index 000000000..321522175
--- /dev/null
+++ b/src/rocksdb/table/block_based/data_block_hash_index.h
@@ -0,0 +1,137 @@
+// Copyright (c) 2011-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).
+
+#pragma once
+
+#include <cstdint>
+#include <string>
+#include <vector>
+
+#include "rocksdb/slice.h"
+
+namespace ROCKSDB_NAMESPACE {
+// This is an experimental feature aiming to reduce the CPU utilization of
+// point-lookup within a data-block. It is only used in data blocks, and not
+// in meta-data blocks or per-table index blocks.
+//
+// It only used to support BlockBasedTable::Get().
+//
+// A serialized hash index is appended to the data-block. The new block data
+// format is as follows:
+//
+// DATA_BLOCK: [RI RI RI ... RI RI_IDX HASH_IDX FOOTER]
+//
+// RI: Restart Interval (the same as the default data-block format)
+// RI_IDX: Restart Interval index (the same as the default data-block format)
+// HASH_IDX: The new data-block hash index feature.
+// FOOTER: A 32bit block footer, which is the NUM_RESTARTS with the MSB as
+// the flag indicating if this hash index is in use. Note that
+// given a data block < 32KB, the MSB is never used. So we can
+// borrow the MSB as the hash index flag. Therefore, this format is
+// compatible with the legacy data-blocks with num_restarts < 32768,
+// as the MSB is 0.
+//
+// The format of the data-block hash index is as follows:
+//
+// HASH_IDX: [B B B ... B NUM_BUCK]
+//
+// B: bucket, an array of restart index. Each buckets is uint8_t.
+// NUM_BUCK: Number of buckets, which is the length of the bucket array.
+//
+// We reserve two special flag:
+// kNoEntry=255,
+// kCollision=254.
+//
+// Therefore, the max number of restarts this hash index can supoport is 253.
+//
+// Buckets are initialized to be kNoEntry.
+//
+// When storing a key in the hash index, the key is first hashed to a bucket.
+// If there the bucket is empty (kNoEntry), the restart index is stored in
+// the bucket. If there is already a restart index there, we will update the
+// existing restart index to a collision marker (kCollision). If the
+// the bucket is already marked as collision, we do not store the restart
+// index either.
+//
+// During query process, a key is first hashed to a bucket. Then we examine if
+// the buckets store nothing (kNoEntry) or the bucket had a collision
+// (kCollision). If either of those happens, we get the restart index of
+// the key and will directly go to the restart interval to search the key.
+//
+// Note that we only support blocks with #restart_interval < 254. If a block
+// has more restart interval than that, hash index will not be create for it.
+
+const uint8_t kNoEntry = 255;
+const uint8_t kCollision = 254;
+const uint8_t kMaxRestartSupportedByHashIndex = 253;
+
+// Because we use uint16_t address, we only support block no more than 64KB
+const size_t kMaxBlockSizeSupportedByHashIndex = 1u << 16;
+const double kDefaultUtilRatio = 0.75;
+
+class DataBlockHashIndexBuilder {
+ public:
+ DataBlockHashIndexBuilder()
+ : bucket_per_key_(-1 /*uninitialized marker*/),
+ estimated_num_buckets_(0),
+ valid_(false) {}
+
+ void Initialize(double util_ratio) {
+ if (util_ratio <= 0) {
+ util_ratio = kDefaultUtilRatio; // sanity check
+ }
+ bucket_per_key_ = 1 / util_ratio;
+ valid_ = true;
+ }
+
+ inline bool Valid() const { return valid_ && bucket_per_key_ > 0; }
+ void Add(const Slice& key, const size_t restart_index);
+ void Finish(std::string& buffer);
+ void Reset();
+ inline size_t EstimateSize() const {
+ uint16_t estimated_num_buckets =
+ static_cast<uint16_t>(estimated_num_buckets_);
+
+ // Maching the num_buckets number in DataBlockHashIndexBuilder::Finish.
+ estimated_num_buckets |= 1;
+
+ return sizeof(uint16_t) +
+ static_cast<size_t>(estimated_num_buckets * sizeof(uint8_t));
+ }
+
+ private:
+ double bucket_per_key_; // is the multiplicative inverse of util_ratio_
+ double estimated_num_buckets_;
+
+ // Now the only usage for `valid_` is to mark false when the inserted
+ // restart_index is larger than supported. In this case HashIndex is not
+ // appended to the block content.
+ bool valid_;
+
+ std::vector<std::pair<uint32_t, uint8_t>> hash_and_restart_pairs_;
+ friend class DataBlockHashIndex_DataBlockHashTestSmall_Test;
+};
+
+class DataBlockHashIndex {
+ public:
+ DataBlockHashIndex() : num_buckets_(0) {}
+
+ void Initialize(const char* data, uint16_t size, uint16_t* map_offset);
+
+ uint8_t Lookup(const char* data, uint32_t map_offset, const Slice& key) const;
+
+ inline bool Valid() { return num_buckets_ != 0; }
+
+ private:
+ // To make the serialized hash index compact and to save the space overhead,
+ // here all the data fields persisted in the block are in uint16 format.
+ // We find that a uint16 is large enough to index every offset of a 64KiB
+ // block.
+ // So in other words, DataBlockHashIndex does not support block size equal
+ // or greater then 64KiB.
+ uint16_t num_buckets_;
+};
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/data_block_hash_index_test.cc b/src/rocksdb/table/block_based/data_block_hash_index_test.cc
new file mode 100644
index 000000000..cd2e30833
--- /dev/null
+++ b/src/rocksdb/table/block_based/data_block_hash_index_test.cc
@@ -0,0 +1,717 @@
+// Copyright (c) 2011-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 "table/block_based/data_block_hash_index.h"
+
+#include <cstdlib>
+#include <string>
+#include <unordered_map>
+
+#include "db/table_properties_collector.h"
+#include "rocksdb/slice.h"
+#include "table/block_based/block.h"
+#include "table/block_based/block_based_table_reader.h"
+#include "table/block_based/block_builder.h"
+#include "table/get_context.h"
+#include "table/table_builder.h"
+#include "test_util/testharness.h"
+#include "test_util/testutil.h"
+#include "util/random.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+bool SearchForOffset(DataBlockHashIndex& index, const char* data,
+ uint16_t map_offset, const Slice& key,
+ uint8_t& restart_point) {
+ uint8_t entry = index.Lookup(data, map_offset, key);
+ if (entry == kCollision) {
+ return true;
+ }
+
+ if (entry == kNoEntry) {
+ return false;
+ }
+
+ return entry == restart_point;
+}
+
+std::string GenerateKey(int primary_key, int secondary_key, int padding_size,
+ Random* rnd) {
+ char buf[50];
+ char* p = &buf[0];
+ snprintf(buf, sizeof(buf), "%6d%4d", primary_key, secondary_key);
+ std::string k(p);
+ if (padding_size) {
+ k += rnd->RandomString(padding_size);
+ }
+
+ return k;
+}
+
+// Generate random key value pairs.
+// The generated key will be sorted. You can tune the parameters to generated
+// different kinds of test key/value pairs for different scenario.
+void GenerateRandomKVs(std::vector<std::string>* keys,
+ std::vector<std::string>* values, const int from,
+ const int len, const int step = 1,
+ const int padding_size = 0,
+ const int keys_share_prefix = 1) {
+ Random rnd(302);
+
+ // generate different prefix
+ for (int i = from; i < from + len; i += step) {
+ // generating keys that shares the prefix
+ for (int j = 0; j < keys_share_prefix; ++j) {
+ keys->emplace_back(GenerateKey(i, j, padding_size, &rnd));
+
+ // 100 bytes values
+ values->emplace_back(rnd.RandomString(100));
+ }
+ }
+}
+
+TEST(DataBlockHashIndex, DataBlockHashTestSmall) {
+ DataBlockHashIndexBuilder builder;
+ builder.Initialize(0.75 /*util_ratio*/);
+ for (int j = 0; j < 5; j++) {
+ for (uint8_t i = 0; i < 2 + j; i++) {
+ std::string key("key" + std::to_string(i));
+ uint8_t restart_point = i;
+ builder.Add(key, restart_point);
+ }
+
+ size_t estimated_size = builder.EstimateSize();
+
+ std::string buffer("fake"), buffer2;
+ size_t original_size = buffer.size();
+ estimated_size += original_size;
+ builder.Finish(buffer);
+
+ ASSERT_EQ(buffer.size(), estimated_size);
+
+ buffer2 = buffer; // test for the correctness of relative offset
+
+ Slice s(buffer2);
+ DataBlockHashIndex index;
+ uint16_t map_offset;
+ index.Initialize(s.data(), static_cast<uint16_t>(s.size()), &map_offset);
+
+ // the additional hash map should start at the end of the buffer
+ ASSERT_EQ(original_size, map_offset);
+ for (uint8_t i = 0; i < 2; i++) {
+ std::string key("key" + std::to_string(i));
+ uint8_t restart_point = i;
+ ASSERT_TRUE(
+ SearchForOffset(index, s.data(), map_offset, key, restart_point));
+ }
+ builder.Reset();
+ }
+}
+
+TEST(DataBlockHashIndex, DataBlockHashTest) {
+ // bucket_num = 200, #keys = 100. 50% utilization
+ DataBlockHashIndexBuilder builder;
+ builder.Initialize(0.75 /*util_ratio*/);
+
+ for (uint8_t i = 0; i < 100; i++) {
+ std::string key("key" + std::to_string(i));
+ uint8_t restart_point = i;
+ builder.Add(key, restart_point);
+ }
+
+ size_t estimated_size = builder.EstimateSize();
+
+ std::string buffer("fake content"), buffer2;
+ size_t original_size = buffer.size();
+ estimated_size += original_size;
+ builder.Finish(buffer);
+
+ ASSERT_EQ(buffer.size(), estimated_size);
+
+ buffer2 = buffer; // test for the correctness of relative offset
+
+ Slice s(buffer2);
+ DataBlockHashIndex index;
+ uint16_t map_offset;
+ index.Initialize(s.data(), static_cast<uint16_t>(s.size()), &map_offset);
+
+ // the additional hash map should start at the end of the buffer
+ ASSERT_EQ(original_size, map_offset);
+ for (uint8_t i = 0; i < 100; i++) {
+ std::string key("key" + std::to_string(i));
+ uint8_t restart_point = i;
+ ASSERT_TRUE(
+ SearchForOffset(index, s.data(), map_offset, key, restart_point));
+ }
+}
+
+TEST(DataBlockHashIndex, DataBlockHashTestCollision) {
+ // bucket_num = 2. There will be intense hash collisions
+ DataBlockHashIndexBuilder builder;
+ builder.Initialize(0.75 /*util_ratio*/);
+
+ for (uint8_t i = 0; i < 100; i++) {
+ std::string key("key" + std::to_string(i));
+ uint8_t restart_point = i;
+ builder.Add(key, restart_point);
+ }
+
+ size_t estimated_size = builder.EstimateSize();
+
+ std::string buffer("some other fake content to take up space"), buffer2;
+ size_t original_size = buffer.size();
+ estimated_size += original_size;
+ builder.Finish(buffer);
+
+ ASSERT_EQ(buffer.size(), estimated_size);
+
+ buffer2 = buffer; // test for the correctness of relative offset
+
+ Slice s(buffer2);
+ DataBlockHashIndex index;
+ uint16_t map_offset;
+ index.Initialize(s.data(), static_cast<uint16_t>(s.size()), &map_offset);
+
+ // the additional hash map should start at the end of the buffer
+ ASSERT_EQ(original_size, map_offset);
+ for (uint8_t i = 0; i < 100; i++) {
+ std::string key("key" + std::to_string(i));
+ uint8_t restart_point = i;
+ ASSERT_TRUE(
+ SearchForOffset(index, s.data(), map_offset, key, restart_point));
+ }
+}
+
+TEST(DataBlockHashIndex, DataBlockHashTestLarge) {
+ DataBlockHashIndexBuilder builder;
+ builder.Initialize(0.75 /*util_ratio*/);
+ std::unordered_map<std::string, uint8_t> m;
+
+ for (uint8_t i = 0; i < 100; i++) {
+ if (i % 2) {
+ continue; // leave half of the keys out
+ }
+ std::string key = "key" + std::to_string(i);
+ uint8_t restart_point = i;
+ builder.Add(key, restart_point);
+ m[key] = restart_point;
+ }
+
+ size_t estimated_size = builder.EstimateSize();
+
+ std::string buffer("filling stuff"), buffer2;
+ size_t original_size = buffer.size();
+ estimated_size += original_size;
+ builder.Finish(buffer);
+
+ ASSERT_EQ(buffer.size(), estimated_size);
+
+ buffer2 = buffer; // test for the correctness of relative offset
+
+ Slice s(buffer2);
+ DataBlockHashIndex index;
+ uint16_t map_offset;
+ index.Initialize(s.data(), static_cast<uint16_t>(s.size()), &map_offset);
+
+ // the additional hash map should start at the end of the buffer
+ ASSERT_EQ(original_size, map_offset);
+ for (uint8_t i = 0; i < 100; i++) {
+ std::string key = "key" + std::to_string(i);
+ uint8_t restart_point = i;
+ if (m.count(key)) {
+ ASSERT_TRUE(m[key] == restart_point);
+ ASSERT_TRUE(
+ SearchForOffset(index, s.data(), map_offset, key, restart_point));
+ } else {
+ // we allow false positve, so don't test the nonexisting keys.
+ // when false positive happens, the search will continue to the
+ // restart intervals to see if the key really exist.
+ }
+ }
+}
+
+TEST(DataBlockHashIndex, RestartIndexExceedMax) {
+ DataBlockHashIndexBuilder builder;
+ builder.Initialize(0.75 /*util_ratio*/);
+ std::unordered_map<std::string, uint8_t> m;
+
+ for (uint8_t i = 0; i <= 253; i++) {
+ std::string key = "key" + std::to_string(i);
+ uint8_t restart_point = i;
+ builder.Add(key, restart_point);
+ }
+ ASSERT_TRUE(builder.Valid());
+
+ builder.Reset();
+
+ for (uint8_t i = 0; i <= 254; i++) {
+ std::string key = "key" + std::to_string(i);
+ uint8_t restart_point = i;
+ builder.Add(key, restart_point);
+ }
+
+ ASSERT_FALSE(builder.Valid());
+
+ builder.Reset();
+ ASSERT_TRUE(builder.Valid());
+}
+
+TEST(DataBlockHashIndex, BlockRestartIndexExceedMax) {
+ Options options = Options();
+
+ BlockBuilder builder(1 /* block_restart_interval */,
+ true /* use_delta_encoding */,
+ false /* use_value_delta_encoding */,
+ BlockBasedTableOptions::kDataBlockBinaryAndHash);
+
+ // #restarts <= 253. HashIndex is valid
+ for (int i = 0; i <= 253; i++) {
+ std::string ukey = "key" + std::to_string(i);
+ InternalKey ikey(ukey, 0, kTypeValue);
+ builder.Add(ikey.Encode().ToString(), "value");
+ }
+
+ {
+ // read serialized contents of the block
+ Slice rawblock = builder.Finish();
+
+ // create block reader
+ BlockContents contents;
+ contents.data = rawblock;
+ Block reader(std::move(contents));
+
+ ASSERT_EQ(reader.IndexType(),
+ BlockBasedTableOptions::kDataBlockBinaryAndHash);
+ }
+
+ builder.Reset();
+
+ // #restarts > 253. HashIndex is not used
+ for (int i = 0; i <= 254; i++) {
+ std::string ukey = "key" + std::to_string(i);
+ InternalKey ikey(ukey, 0, kTypeValue);
+ builder.Add(ikey.Encode().ToString(), "value");
+ }
+
+ {
+ // read serialized contents of the block
+ Slice rawblock = builder.Finish();
+
+ // create block reader
+ BlockContents contents;
+ contents.data = rawblock;
+ Block reader(std::move(contents));
+
+ ASSERT_EQ(reader.IndexType(),
+ BlockBasedTableOptions::kDataBlockBinarySearch);
+ }
+}
+
+TEST(DataBlockHashIndex, BlockSizeExceedMax) {
+ Options options = Options();
+ std::string ukey(10, 'k');
+ InternalKey ikey(ukey, 0, kTypeValue);
+
+ BlockBuilder builder(1 /* block_restart_interval */,
+ false /* use_delta_encoding */,
+ false /* use_value_delta_encoding */,
+ BlockBasedTableOptions::kDataBlockBinaryAndHash);
+
+ {
+ // insert a large value. The block size plus HashIndex is 65536.
+ std::string value(65502, 'v');
+
+ builder.Add(ikey.Encode().ToString(), value);
+
+ // read serialized contents of the block
+ Slice rawblock = builder.Finish();
+ ASSERT_LE(rawblock.size(), kMaxBlockSizeSupportedByHashIndex);
+ std::cerr << "block size: " << rawblock.size() << std::endl;
+
+ // create block reader
+ BlockContents contents;
+ contents.data = rawblock;
+ Block reader(std::move(contents));
+
+ ASSERT_EQ(reader.IndexType(),
+ BlockBasedTableOptions::kDataBlockBinaryAndHash);
+ }
+
+ builder.Reset();
+
+ {
+ // insert a large value. The block size plus HashIndex would be 65537.
+ // This excceed the max block size supported by HashIndex (65536).
+ // So when build finishes HashIndex will not be created for the block.
+ std::string value(65503, 'v');
+
+ builder.Add(ikey.Encode().ToString(), value);
+
+ // read serialized contents of the block
+ Slice rawblock = builder.Finish();
+ ASSERT_LE(rawblock.size(), kMaxBlockSizeSupportedByHashIndex);
+ std::cerr << "block size: " << rawblock.size() << std::endl;
+
+ // create block reader
+ BlockContents contents;
+ contents.data = rawblock;
+ Block reader(std::move(contents));
+
+ // the index type have fallen back to binary when build finish.
+ ASSERT_EQ(reader.IndexType(),
+ BlockBasedTableOptions::kDataBlockBinarySearch);
+ }
+}
+
+TEST(DataBlockHashIndex, BlockTestSingleKey) {
+ Options options = Options();
+
+ BlockBuilder builder(16 /* block_restart_interval */,
+ true /* use_delta_encoding */,
+ false /* use_value_delta_encoding */,
+ BlockBasedTableOptions::kDataBlockBinaryAndHash);
+
+ std::string ukey("gopher");
+ std::string value("gold");
+ InternalKey ikey(ukey, 10, kTypeValue);
+ builder.Add(ikey.Encode().ToString(), value /*value*/);
+
+ // read serialized contents of the block
+ Slice rawblock = builder.Finish();
+
+ // create block reader
+ BlockContents contents;
+ contents.data = rawblock;
+ Block reader(std::move(contents));
+
+ const InternalKeyComparator icmp(BytewiseComparator());
+ auto iter = reader.NewDataIterator(icmp.user_comparator(),
+ kDisableGlobalSequenceNumber);
+ bool may_exist;
+ // search in block for the key just inserted
+ {
+ InternalKey seek_ikey(ukey, 10, kValueTypeForSeek);
+ may_exist = iter->SeekForGet(seek_ikey.Encode().ToString());
+ ASSERT_TRUE(may_exist);
+ ASSERT_TRUE(iter->Valid());
+ ASSERT_EQ(
+ options.comparator->Compare(iter->key(), ikey.Encode().ToString()), 0);
+ ASSERT_EQ(iter->value(), value);
+ }
+
+ // search in block for the existing ukey, but with higher seqno
+ {
+ InternalKey seek_ikey(ukey, 20, kValueTypeForSeek);
+
+ // HashIndex should be able to set the iter correctly
+ may_exist = iter->SeekForGet(seek_ikey.Encode().ToString());
+ ASSERT_TRUE(may_exist);
+ ASSERT_TRUE(iter->Valid());
+
+ // user key should match
+ ASSERT_EQ(options.comparator->Compare(ExtractUserKey(iter->key()), ukey),
+ 0);
+
+ // seek_key seqno number should be greater than that of iter result
+ ASSERT_GT(GetInternalKeySeqno(seek_ikey.Encode()),
+ GetInternalKeySeqno(iter->key()));
+
+ ASSERT_EQ(iter->value(), value);
+ }
+
+ // Search in block for the existing ukey, but with lower seqno
+ // in this case, hash can find the only occurrence of the user_key, but
+ // ParseNextDataKey() will skip it as it does not have a older seqno.
+ // In this case, GetForSeek() is effective to locate the user_key, and
+ // iter->Valid() == false indicates that we've reached to the end of
+ // the block and the caller should continue searching the next block.
+ {
+ InternalKey seek_ikey(ukey, 5, kValueTypeForSeek);
+ may_exist = iter->SeekForGet(seek_ikey.Encode().ToString());
+ ASSERT_TRUE(may_exist);
+ ASSERT_FALSE(iter->Valid()); // should have reached to the end of block
+ }
+
+ delete iter;
+}
+
+TEST(DataBlockHashIndex, BlockTestLarge) {
+ Random rnd(1019);
+ Options options = Options();
+ std::vector<std::string> keys;
+ std::vector<std::string> values;
+
+ BlockBuilder builder(16 /* block_restart_interval */,
+ true /* use_delta_encoding */,
+ false /* use_value_delta_encoding */,
+ BlockBasedTableOptions::kDataBlockBinaryAndHash);
+ int num_records = 500;
+
+ GenerateRandomKVs(&keys, &values, 0, num_records);
+
+ // Generate keys. Adding a trailing "1" to indicate existent keys.
+ // Later will Seeking for keys with a trailing "0" to test seeking
+ // non-existent keys.
+ for (int i = 0; i < num_records; i++) {
+ std::string ukey(keys[i] + "1" /* existing key marker */);
+ InternalKey ikey(ukey, 0, kTypeValue);
+ builder.Add(ikey.Encode().ToString(), values[i]);
+ }
+
+ // read serialized contents of the block
+ Slice rawblock = builder.Finish();
+
+ // create block reader
+ BlockContents contents;
+ contents.data = rawblock;
+ Block reader(std::move(contents));
+ const InternalKeyComparator icmp(BytewiseComparator());
+
+ // random seek existent keys
+ for (int i = 0; i < num_records; i++) {
+ auto iter = reader.NewDataIterator(icmp.user_comparator(),
+ kDisableGlobalSequenceNumber);
+ // find a random key in the lookaside array
+ int index = rnd.Uniform(num_records);
+ std::string ukey(keys[index] + "1" /* existing key marker */);
+ InternalKey ikey(ukey, 0, kTypeValue);
+
+ // search in block for this key
+ bool may_exist = iter->SeekForGet(ikey.Encode().ToString());
+ ASSERT_TRUE(may_exist);
+ ASSERT_TRUE(iter->Valid());
+ ASSERT_EQ(values[index], iter->value());
+
+ delete iter;
+ }
+
+ // random seek non-existent user keys
+ // In this case A), the user_key cannot be found in HashIndex. The key may
+ // exist in the next block. So the iter is set invalidated to tell the
+ // caller to search the next block. This test case belongs to this case A).
+ //
+ // Note that for non-existent keys, there is possibility of false positive,
+ // i.e. the key is still hashed into some restart interval.
+ // Two additional possible outcome:
+ // B) linear seek the restart interval and not found, the iter stops at the
+ // starting of the next restart interval. The key does not exist
+ // anywhere.
+ // C) linear seek the restart interval and not found, the iter stops at the
+ // the end of the block, i.e. restarts_. The key may exist in the next
+ // block.
+ // So these combinations are possible when searching non-existent user_key:
+ //
+ // case# may_exist iter->Valid()
+ // A true false
+ // B false true
+ // C true false
+
+ for (int i = 0; i < num_records; i++) {
+ auto iter = reader.NewDataIterator(icmp.user_comparator(),
+ kDisableGlobalSequenceNumber);
+ // find a random key in the lookaside array
+ int index = rnd.Uniform(num_records);
+ std::string ukey(keys[index] + "0" /* non-existing key marker */);
+ InternalKey ikey(ukey, 0, kTypeValue);
+
+ // search in block for this key
+ bool may_exist = iter->SeekForGet(ikey.Encode().ToString());
+ if (!may_exist) {
+ ASSERT_TRUE(iter->Valid());
+ }
+ if (!iter->Valid()) {
+ ASSERT_TRUE(may_exist);
+ }
+
+ delete iter;
+ }
+}
+
+// helper routine for DataBlockHashIndex.BlockBoundary
+void TestBoundary(InternalKey& ik1, std::string& v1, InternalKey& ik2,
+ std::string& v2, InternalKey& seek_ikey,
+ GetContext& get_context, Options& options) {
+ std::unique_ptr<WritableFileWriter> file_writer;
+ std::unique_ptr<RandomAccessFileReader> file_reader;
+ std::unique_ptr<TableReader> table_reader;
+ int level_ = -1;
+
+ std::vector<std::string> keys;
+ const ImmutableOptions ioptions(options);
+ const MutableCFOptions moptions(options);
+ const InternalKeyComparator internal_comparator(options.comparator);
+
+ EnvOptions soptions;
+
+ soptions.use_mmap_reads = ioptions.allow_mmap_reads;
+ test::StringSink* sink = new test::StringSink();
+ std::unique_ptr<FSWritableFile> f(sink);
+ file_writer.reset(
+ new WritableFileWriter(std::move(f), "" /* don't care */, FileOptions()));
+ std::unique_ptr<TableBuilder> builder;
+ IntTblPropCollectorFactories int_tbl_prop_collector_factories;
+ std::string column_family_name;
+ builder.reset(ioptions.table_factory->NewTableBuilder(
+ TableBuilderOptions(
+ ioptions, moptions, internal_comparator,
+ &int_tbl_prop_collector_factories, options.compression,
+ CompressionOptions(),
+ TablePropertiesCollectorFactory::Context::kUnknownColumnFamily,
+ column_family_name, level_),
+ file_writer.get()));
+
+ builder->Add(ik1.Encode().ToString(), v1);
+ builder->Add(ik2.Encode().ToString(), v2);
+ EXPECT_TRUE(builder->status().ok());
+
+ Status s = builder->Finish();
+ ASSERT_OK(file_writer->Flush());
+ EXPECT_TRUE(s.ok()) << s.ToString();
+
+ EXPECT_EQ(sink->contents().size(), builder->FileSize());
+
+ // Open the table
+ test::StringSource* source = new test::StringSource(
+ sink->contents(), 0 /*uniq_id*/, ioptions.allow_mmap_reads);
+ std::unique_ptr<FSRandomAccessFile> file(source);
+ file_reader.reset(new RandomAccessFileReader(std::move(file), "test"));
+ const bool kSkipFilters = true;
+ const bool kImmortal = true;
+ ASSERT_OK(ioptions.table_factory->NewTableReader(
+ TableReaderOptions(ioptions, moptions.prefix_extractor, soptions,
+ internal_comparator, !kSkipFilters, !kImmortal,
+ level_),
+ std::move(file_reader), sink->contents().size(), &table_reader));
+ // Search using Get()
+ ReadOptions ro;
+
+ ASSERT_OK(table_reader->Get(ro, seek_ikey.Encode().ToString(), &get_context,
+ moptions.prefix_extractor.get()));
+}
+
+TEST(DataBlockHashIndex, BlockBoundary) {
+ BlockBasedTableOptions table_options;
+ table_options.data_block_index_type =
+ BlockBasedTableOptions::kDataBlockBinaryAndHash;
+ table_options.block_restart_interval = 1;
+ table_options.block_size = 4096;
+
+ Options options;
+ options.comparator = BytewiseComparator();
+
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+
+ // insert two large k/v pair. Given that the block_size is 4096, one k/v
+ // pair will take up one block.
+ // [ k1/v1 ][ k2/v2 ]
+ // [ Block N ][ Block N+1 ]
+
+ {
+ // [ "aab"@100 ][ "axy"@10 ]
+ // | Block N ][ Block N+1 ]
+ // seek for "axy"@60
+ std::string uk1("aab");
+ InternalKey ik1(uk1, 100, kTypeValue);
+ std::string v1(4100, '1'); // large value
+
+ std::string uk2("axy");
+ InternalKey ik2(uk2, 10, kTypeValue);
+ std::string v2(4100, '2'); // large value
+
+ PinnableSlice value;
+ std::string seek_ukey("axy");
+ InternalKey seek_ikey(seek_ukey, 60, kTypeValue);
+ GetContext get_context(options.comparator, nullptr, nullptr, nullptr,
+ GetContext::kNotFound, seek_ukey, &value, nullptr,
+ nullptr, nullptr, true, nullptr, nullptr);
+
+ TestBoundary(ik1, v1, ik2, v2, seek_ikey, get_context, options);
+ ASSERT_EQ(get_context.State(), GetContext::kFound);
+ ASSERT_EQ(value, v2);
+ value.Reset();
+ }
+
+ {
+ // [ "axy"@100 ][ "axy"@10 ]
+ // | Block N ][ Block N+1 ]
+ // seek for "axy"@60
+ std::string uk1("axy");
+ InternalKey ik1(uk1, 100, kTypeValue);
+ std::string v1(4100, '1'); // large value
+
+ std::string uk2("axy");
+ InternalKey ik2(uk2, 10, kTypeValue);
+ std::string v2(4100, '2'); // large value
+
+ PinnableSlice value;
+ std::string seek_ukey("axy");
+ InternalKey seek_ikey(seek_ukey, 60, kTypeValue);
+ GetContext get_context(options.comparator, nullptr, nullptr, nullptr,
+ GetContext::kNotFound, seek_ukey, &value, nullptr,
+ nullptr, nullptr, true, nullptr, nullptr);
+
+ TestBoundary(ik1, v1, ik2, v2, seek_ikey, get_context, options);
+ ASSERT_EQ(get_context.State(), GetContext::kFound);
+ ASSERT_EQ(value, v2);
+ value.Reset();
+ }
+
+ {
+ // [ "axy"@100 ][ "axy"@10 ]
+ // | Block N ][ Block N+1 ]
+ // seek for "axy"@120
+ std::string uk1("axy");
+ InternalKey ik1(uk1, 100, kTypeValue);
+ std::string v1(4100, '1'); // large value
+
+ std::string uk2("axy");
+ InternalKey ik2(uk2, 10, kTypeValue);
+ std::string v2(4100, '2'); // large value
+
+ PinnableSlice value;
+ std::string seek_ukey("axy");
+ InternalKey seek_ikey(seek_ukey, 120, kTypeValue);
+ GetContext get_context(options.comparator, nullptr, nullptr, nullptr,
+ GetContext::kNotFound, seek_ukey, &value, nullptr,
+ nullptr, nullptr, true, nullptr, nullptr);
+
+ TestBoundary(ik1, v1, ik2, v2, seek_ikey, get_context, options);
+ ASSERT_EQ(get_context.State(), GetContext::kFound);
+ ASSERT_EQ(value, v1);
+ value.Reset();
+ }
+
+ {
+ // [ "axy"@100 ][ "axy"@10 ]
+ // | Block N ][ Block N+1 ]
+ // seek for "axy"@5
+ std::string uk1("axy");
+ InternalKey ik1(uk1, 100, kTypeValue);
+ std::string v1(4100, '1'); // large value
+
+ std::string uk2("axy");
+ InternalKey ik2(uk2, 10, kTypeValue);
+ std::string v2(4100, '2'); // large value
+
+ PinnableSlice value;
+ std::string seek_ukey("axy");
+ InternalKey seek_ikey(seek_ukey, 5, kTypeValue);
+ GetContext get_context(options.comparator, nullptr, nullptr, nullptr,
+ GetContext::kNotFound, seek_ukey, &value, nullptr,
+ nullptr, nullptr, true, nullptr, nullptr);
+
+ TestBoundary(ik1, v1, ik2, v2, seek_ikey, get_context, options);
+ ASSERT_EQ(get_context.State(), GetContext::kNotFound);
+ value.Reset();
+ }
+}
+
+} // namespace ROCKSDB_NAMESPACE
+
+int main(int argc, char** argv) {
+ ROCKSDB_NAMESPACE::port::InstallStackTraceHandler();
+ ::testing::InitGoogleTest(&argc, argv);
+ return RUN_ALL_TESTS();
+}
diff --git a/src/rocksdb/table/block_based/filter_block.h b/src/rocksdb/table/block_based/filter_block.h
new file mode 100644
index 000000000..e1e206990
--- /dev/null
+++ b/src/rocksdb/table/block_based/filter_block.h
@@ -0,0 +1,182 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2012 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+//
+// A filter block is stored near the end of a Table file. It contains
+// filters (e.g., bloom filters) for all data blocks in the table combined
+// into a single filter block.
+
+#pragma once
+
+#include <stddef.h>
+#include <stdint.h>
+
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "rocksdb/options.h"
+#include "rocksdb/slice.h"
+#include "rocksdb/slice_transform.h"
+#include "rocksdb/table.h"
+#include "table/format.h"
+#include "table/multiget_context.h"
+#include "trace_replay/block_cache_tracer.h"
+#include "util/hash.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+const uint64_t kNotValid = ULLONG_MAX;
+class FilterPolicy;
+
+class GetContext;
+using MultiGetRange = MultiGetContext::Range;
+
+// A FilterBlockBuilder is used to construct all of the filters for a
+// particular Table. It generates a single string which is stored as
+// a special block in the Table, or partitioned into smaller filters.
+//
+// The sequence of calls to FilterBlockBuilder must match the regexp:
+// Add* Finish
+class FilterBlockBuilder {
+ public:
+ explicit FilterBlockBuilder() {}
+ // No copying allowed
+ FilterBlockBuilder(const FilterBlockBuilder&) = delete;
+ void operator=(const FilterBlockBuilder&) = delete;
+
+ virtual ~FilterBlockBuilder() {}
+
+ virtual void Add(
+ const Slice& key_without_ts) = 0; // Add a key to current filter
+ virtual bool IsEmpty() const = 0; // Empty == none added
+ // For reporting stats on how many entries the builder considered unique
+ virtual size_t EstimateEntriesAdded() = 0;
+ Slice Finish() { // Generate Filter
+ const BlockHandle empty_handle;
+ Status dont_care_status;
+ auto ret = Finish(empty_handle, &dont_care_status);
+ assert(dont_care_status.ok());
+ return ret;
+ }
+ // If filter_data is not nullptr, Finish() may transfer ownership of
+ // underlying filter data to the caller, so that it can be freed as soon as
+ // possible. BlockBasedFilterBlock will ignore this parameter.
+ //
+ virtual Slice Finish(
+ const BlockHandle& tmp /* only used in PartitionedFilterBlock as
+ last_partition_block_handle */
+ ,
+ Status* status, std::unique_ptr<const char[]>* filter_data = nullptr) = 0;
+
+ // This is called when finishes using the FilterBitsBuilder
+ // in order to release memory usage and cache charge
+ // associated with it timely
+ virtual void ResetFilterBitsBuilder() {}
+
+ // To optionally post-verify the filter returned from
+ // FilterBlockBuilder::Finish.
+ // Return Status::OK() if skipped.
+ virtual Status MaybePostVerifyFilter(const Slice& /* filter_content */) {
+ return Status::OK();
+ }
+};
+
+// A FilterBlockReader is used to parse filter from SST table.
+// KeyMayMatch and PrefixMayMatch would trigger filter checking
+//
+// BlockBased/Full FilterBlock would be called in the same way.
+class FilterBlockReader {
+ public:
+ FilterBlockReader() = default;
+ virtual ~FilterBlockReader() = default;
+
+ FilterBlockReader(const FilterBlockReader&) = delete;
+ FilterBlockReader& operator=(const FilterBlockReader&) = delete;
+
+ /**
+ * If no_io is set, then it returns true if it cannot answer the query without
+ * reading data from disk. This is used in PartitionedFilterBlockReader to
+ * avoid reading partitions that are not in block cache already
+ *
+ * Normally filters are built on only the user keys and the InternalKey is not
+ * needed for a query. The index in PartitionedFilterBlockReader however is
+ * built upon InternalKey and must be provided via const_ikey_ptr when running
+ * queries.
+ */
+ virtual bool KeyMayMatch(const Slice& key, const bool no_io,
+ const Slice* const const_ikey_ptr,
+ GetContext* get_context,
+ BlockCacheLookupContext* lookup_context,
+ Env::IOPriority rate_limiter_priority) = 0;
+
+ virtual void KeysMayMatch(MultiGetRange* range, const bool no_io,
+ BlockCacheLookupContext* lookup_context,
+ Env::IOPriority rate_limiter_priority) {
+ for (auto iter = range->begin(); iter != range->end(); ++iter) {
+ const Slice ukey_without_ts = iter->ukey_without_ts;
+ const Slice ikey = iter->ikey;
+ GetContext* const get_context = iter->get_context;
+ if (!KeyMayMatch(ukey_without_ts, no_io, &ikey, get_context,
+ lookup_context, rate_limiter_priority)) {
+ range->SkipKey(iter);
+ }
+ }
+ }
+
+ /**
+ * no_io and const_ikey_ptr here means the same as in KeyMayMatch
+ */
+ virtual bool PrefixMayMatch(const Slice& prefix, const bool no_io,
+ const Slice* const const_ikey_ptr,
+ GetContext* get_context,
+ BlockCacheLookupContext* lookup_context,
+ Env::IOPriority rate_limiter_priority) = 0;
+
+ virtual void PrefixesMayMatch(MultiGetRange* range,
+ const SliceTransform* prefix_extractor,
+ const bool no_io,
+ BlockCacheLookupContext* lookup_context,
+ Env::IOPriority rate_limiter_priority) {
+ for (auto iter = range->begin(); iter != range->end(); ++iter) {
+ const Slice ukey_without_ts = iter->ukey_without_ts;
+ const Slice ikey = iter->ikey;
+ GetContext* const get_context = iter->get_context;
+ if (prefix_extractor->InDomain(ukey_without_ts) &&
+ !PrefixMayMatch(prefix_extractor->Transform(ukey_without_ts), no_io,
+ &ikey, get_context, lookup_context,
+ rate_limiter_priority)) {
+ range->SkipKey(iter);
+ }
+ }
+ }
+
+ virtual size_t ApproximateMemoryUsage() const = 0;
+
+ // convert this object to a human readable form
+ virtual std::string ToString() const {
+ std::string error_msg("Unsupported filter \n");
+ return error_msg;
+ }
+
+ virtual Status CacheDependencies(const ReadOptions& /*ro*/, bool /*pin*/) {
+ return Status::OK();
+ }
+
+ virtual bool RangeMayExist(const Slice* /*iterate_upper_bound*/,
+ const Slice& user_key_without_ts,
+ const SliceTransform* prefix_extractor,
+ const Comparator* /*comparator*/,
+ const Slice* const const_ikey_ptr,
+ bool* filter_checked, bool need_upper_bound_check,
+ bool no_io,
+ BlockCacheLookupContext* lookup_context,
+ Env::IOPriority rate_limiter_priority) = 0;
+};
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/filter_block_reader_common.cc b/src/rocksdb/table/block_based/filter_block_reader_common.cc
new file mode 100644
index 000000000..7dc49e83e
--- /dev/null
+++ b/src/rocksdb/table/block_based/filter_block_reader_common.cc
@@ -0,0 +1,164 @@
+// Copyright (c) 2011-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 "table/block_based/filter_block_reader_common.h"
+
+#include "monitoring/perf_context_imp.h"
+#include "table/block_based/block_based_table_reader.h"
+#include "table/block_based/parsed_full_filter_block.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+template <typename TBlocklike>
+Status FilterBlockReaderCommon<TBlocklike>::ReadFilterBlock(
+ const BlockBasedTable* table, FilePrefetchBuffer* prefetch_buffer,
+ const ReadOptions& read_options, bool use_cache, GetContext* get_context,
+ BlockCacheLookupContext* lookup_context,
+ CachableEntry<TBlocklike>* filter_block, BlockType block_type) {
+ PERF_TIMER_GUARD(read_filter_block_nanos);
+
+ assert(table);
+ assert(filter_block);
+ assert(filter_block->IsEmpty());
+
+ const BlockBasedTable::Rep* const rep = table->get_rep();
+ assert(rep);
+
+ const Status s =
+ table->RetrieveBlock(prefetch_buffer, read_options, rep->filter_handle,
+ UncompressionDict::GetEmptyDict(), filter_block,
+ block_type, get_context, lookup_context,
+ /* for_compaction */ false, use_cache,
+ /* wait_for_cache */ true, /* async_read */ false);
+
+ return s;
+}
+
+template <typename TBlocklike>
+const SliceTransform*
+FilterBlockReaderCommon<TBlocklike>::table_prefix_extractor() const {
+ assert(table_);
+
+ const BlockBasedTable::Rep* const rep = table_->get_rep();
+ assert(rep);
+
+ return rep->prefix_filtering ? rep->table_prefix_extractor.get() : nullptr;
+}
+
+template <typename TBlocklike>
+bool FilterBlockReaderCommon<TBlocklike>::whole_key_filtering() const {
+ assert(table_);
+ assert(table_->get_rep());
+
+ return table_->get_rep()->whole_key_filtering;
+}
+
+template <typename TBlocklike>
+bool FilterBlockReaderCommon<TBlocklike>::cache_filter_blocks() const {
+ assert(table_);
+ assert(table_->get_rep());
+
+ return table_->get_rep()->table_options.cache_index_and_filter_blocks;
+}
+
+template <typename TBlocklike>
+Status FilterBlockReaderCommon<TBlocklike>::GetOrReadFilterBlock(
+ bool no_io, GetContext* get_context,
+ BlockCacheLookupContext* lookup_context,
+ CachableEntry<TBlocklike>* filter_block, BlockType block_type,
+ Env::IOPriority rate_limiter_priority) const {
+ assert(filter_block);
+
+ if (!filter_block_.IsEmpty()) {
+ filter_block->SetUnownedValue(filter_block_.GetValue());
+ return Status::OK();
+ }
+
+ ReadOptions read_options;
+ read_options.rate_limiter_priority = rate_limiter_priority;
+ if (no_io) {
+ read_options.read_tier = kBlockCacheTier;
+ }
+
+ return ReadFilterBlock(table_, nullptr /* prefetch_buffer */, read_options,
+ cache_filter_blocks(), get_context, lookup_context,
+ filter_block, block_type);
+}
+
+template <typename TBlocklike>
+size_t FilterBlockReaderCommon<TBlocklike>::ApproximateFilterBlockMemoryUsage()
+ const {
+ assert(!filter_block_.GetOwnValue() || filter_block_.GetValue() != nullptr);
+ return filter_block_.GetOwnValue()
+ ? filter_block_.GetValue()->ApproximateMemoryUsage()
+ : 0;
+}
+
+template <typename TBlocklike>
+bool FilterBlockReaderCommon<TBlocklike>::RangeMayExist(
+ const Slice* iterate_upper_bound, const Slice& user_key_without_ts,
+ const SliceTransform* prefix_extractor, const Comparator* comparator,
+ const Slice* const const_ikey_ptr, bool* filter_checked,
+ bool need_upper_bound_check, bool no_io,
+ BlockCacheLookupContext* lookup_context,
+ Env::IOPriority rate_limiter_priority) {
+ if (!prefix_extractor || !prefix_extractor->InDomain(user_key_without_ts)) {
+ *filter_checked = false;
+ return true;
+ }
+ Slice prefix = prefix_extractor->Transform(user_key_without_ts);
+ if (need_upper_bound_check &&
+ !IsFilterCompatible(iterate_upper_bound, prefix, comparator)) {
+ *filter_checked = false;
+ return true;
+ } else {
+ *filter_checked = true;
+ return PrefixMayMatch(prefix, no_io, const_ikey_ptr,
+ /* get_context */ nullptr, lookup_context,
+ rate_limiter_priority);
+ }
+}
+
+template <typename TBlocklike>
+bool FilterBlockReaderCommon<TBlocklike>::IsFilterCompatible(
+ const Slice* iterate_upper_bound, const Slice& prefix,
+ const Comparator* comparator) const {
+ // Try to reuse the bloom filter in the SST table if prefix_extractor in
+ // mutable_cf_options has changed. If range [user_key, upper_bound) all
+ // share the same prefix then we may still be able to use the bloom filter.
+ const SliceTransform* const prefix_extractor = table_prefix_extractor();
+ if (iterate_upper_bound != nullptr && prefix_extractor) {
+ if (!prefix_extractor->InDomain(*iterate_upper_bound)) {
+ return false;
+ }
+ Slice upper_bound_xform = prefix_extractor->Transform(*iterate_upper_bound);
+ // first check if user_key and upper_bound all share the same prefix
+ if (comparator->CompareWithoutTimestamp(prefix, false, upper_bound_xform,
+ false) != 0) {
+ // second check if user_key's prefix is the immediate predecessor of
+ // upper_bound and have the same length. If so, we know for sure all
+ // keys in the range [user_key, upper_bound) share the same prefix.
+ // Also need to make sure upper_bound are full length to ensure
+ // correctness
+ if (!full_length_enabled_ ||
+ iterate_upper_bound->size() != prefix_extractor_full_length_ ||
+ !comparator->IsSameLengthImmediateSuccessor(prefix,
+ *iterate_upper_bound)) {
+ return false;
+ }
+ }
+ return true;
+ } else {
+ return false;
+ }
+}
+
+// Explicitly instantiate templates for both "blocklike" types we use.
+// This makes it possible to keep the template definitions in the .cc file.
+template class FilterBlockReaderCommon<Block>;
+template class FilterBlockReaderCommon<ParsedFullFilterBlock>;
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/filter_block_reader_common.h b/src/rocksdb/table/block_based/filter_block_reader_common.h
new file mode 100644
index 000000000..ca07f5050
--- /dev/null
+++ b/src/rocksdb/table/block_based/filter_block_reader_common.h
@@ -0,0 +1,79 @@
+// Copyright (c) 2011-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).
+//
+
+#pragma once
+
+#include <cassert>
+
+#include "block_type.h"
+#include "table/block_based/cachable_entry.h"
+#include "table/block_based/filter_block.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class BlockBasedTable;
+class FilePrefetchBuffer;
+
+// Encapsulates common functionality for the various filter block reader
+// implementations. Provides access to the filter block regardless of whether
+// it is owned by the reader or stored in the cache, or whether it is pinned
+// in the cache or not.
+template <typename TBlocklike>
+class FilterBlockReaderCommon : public FilterBlockReader {
+ public:
+ FilterBlockReaderCommon(const BlockBasedTable* t,
+ CachableEntry<TBlocklike>&& filter_block)
+ : table_(t), filter_block_(std::move(filter_block)) {
+ assert(table_);
+ const SliceTransform* const prefix_extractor = table_prefix_extractor();
+ if (prefix_extractor) {
+ full_length_enabled_ =
+ prefix_extractor->FullLengthEnabled(&prefix_extractor_full_length_);
+ }
+ }
+
+ bool RangeMayExist(const Slice* iterate_upper_bound, const Slice& user_key,
+ const SliceTransform* prefix_extractor,
+ const Comparator* comparator,
+ const Slice* const const_ikey_ptr, bool* filter_checked,
+ bool need_upper_bound_check, bool no_io,
+ BlockCacheLookupContext* lookup_context,
+ Env::IOPriority rate_limiter_priority) override;
+
+ protected:
+ static Status ReadFilterBlock(const BlockBasedTable* table,
+ FilePrefetchBuffer* prefetch_buffer,
+ const ReadOptions& read_options, bool use_cache,
+ GetContext* get_context,
+ BlockCacheLookupContext* lookup_context,
+ CachableEntry<TBlocklike>* filter_block,
+ BlockType block_type);
+
+ const BlockBasedTable* table() const { return table_; }
+ const SliceTransform* table_prefix_extractor() const;
+ bool whole_key_filtering() const;
+ bool cache_filter_blocks() const;
+
+ Status GetOrReadFilterBlock(bool no_io, GetContext* get_context,
+ BlockCacheLookupContext* lookup_context,
+ CachableEntry<TBlocklike>* filter_block,
+ BlockType block_type,
+ Env::IOPriority rate_limiter_priority) const;
+
+ size_t ApproximateFilterBlockMemoryUsage() const;
+
+ private:
+ bool IsFilterCompatible(const Slice* iterate_upper_bound, const Slice& prefix,
+ const Comparator* comparator) const;
+
+ private:
+ const BlockBasedTable* table_;
+ CachableEntry<TBlocklike> filter_block_;
+ size_t prefix_extractor_full_length_ = 0;
+ bool full_length_enabled_ = false;
+};
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/filter_policy.cc b/src/rocksdb/table/block_based/filter_policy.cc
new file mode 100644
index 000000000..f84f804dd
--- /dev/null
+++ b/src/rocksdb/table/block_based/filter_policy.cc
@@ -0,0 +1,1973 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2012 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#include "rocksdb/filter_policy.h"
+
+#include <array>
+#include <climits>
+#include <cstring>
+#include <deque>
+#include <limits>
+#include <memory>
+
+#include "cache/cache_entry_roles.h"
+#include "cache/cache_reservation_manager.h"
+#include "logging/logging.h"
+#include "port/lang.h"
+#include "rocksdb/convenience.h"
+#include "rocksdb/rocksdb_namespace.h"
+#include "rocksdb/slice.h"
+#include "rocksdb/utilities/object_registry.h"
+#include "table/block_based/block_based_table_reader.h"
+#include "table/block_based/filter_policy_internal.h"
+#include "table/block_based/full_filter_block.h"
+#include "util/bloom_impl.h"
+#include "util/coding.h"
+#include "util/hash.h"
+#include "util/math.h"
+#include "util/ribbon_config.h"
+#include "util/ribbon_impl.h"
+#include "util/string_util.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+namespace {
+
+// Metadata trailer size for built-in filters. (This is separate from
+// block-based table block trailer.)
+//
+// Originally this was 1 byte for num_probes and 4 bytes for number of
+// cache lines in the Bloom filter, but now the first trailer byte is
+// usually an implementation marker and remaining 4 bytes have various
+// meanings.
+static constexpr uint32_t kMetadataLen = 5;
+
+Slice FinishAlwaysFalse(std::unique_ptr<const char[]>* /*buf*/) {
+ // Missing metadata, treated as zero entries
+ return Slice(nullptr, 0);
+}
+
+Slice FinishAlwaysTrue(std::unique_ptr<const char[]>* /*buf*/) {
+ return Slice("\0\0\0\0\0\0", 6);
+}
+
+// Base class for filter builders using the XXH3 preview hash,
+// also known as Hash64 or GetSliceHash64.
+class XXPH3FilterBitsBuilder : public BuiltinFilterBitsBuilder {
+ public:
+ explicit XXPH3FilterBitsBuilder(
+ std::atomic<int64_t>* aggregate_rounding_balance,
+ std::shared_ptr<CacheReservationManager> cache_res_mgr,
+ bool detect_filter_construct_corruption)
+ : aggregate_rounding_balance_(aggregate_rounding_balance),
+ cache_res_mgr_(cache_res_mgr),
+ detect_filter_construct_corruption_(
+ detect_filter_construct_corruption) {}
+
+ ~XXPH3FilterBitsBuilder() override {}
+
+ virtual void AddKey(const Slice& key) override {
+ uint64_t hash = GetSliceHash64(key);
+ // Especially with prefixes, it is common to have repetition,
+ // though only adjacent repetition, which we want to immediately
+ // recognize and collapse for estimating true filter space
+ // requirements.
+ if (hash_entries_info_.entries.empty() ||
+ hash != hash_entries_info_.entries.back()) {
+ if (detect_filter_construct_corruption_) {
+ hash_entries_info_.xor_checksum ^= hash;
+ }
+ hash_entries_info_.entries.push_back(hash);
+ if (cache_res_mgr_ &&
+ // Traditional rounding to whole bucket size
+ ((hash_entries_info_.entries.size() %
+ kUint64tHashEntryCacheResBucketSize) ==
+ kUint64tHashEntryCacheResBucketSize / 2)) {
+ hash_entries_info_.cache_res_bucket_handles.emplace_back(nullptr);
+ Status s = cache_res_mgr_->MakeCacheReservation(
+ kUint64tHashEntryCacheResBucketSize * sizeof(hash),
+ &hash_entries_info_.cache_res_bucket_handles.back());
+ s.PermitUncheckedError();
+ }
+ }
+ }
+
+ virtual size_t EstimateEntriesAdded() override {
+ return hash_entries_info_.entries.size();
+ }
+
+ virtual Status MaybePostVerify(const Slice& filter_content) override;
+
+ protected:
+ static constexpr uint32_t kMetadataLen = 5;
+
+ // Number of hash entries to accumulate before charging their memory usage to
+ // the cache when cache charging is available
+ static const std::size_t kUint64tHashEntryCacheResBucketSize =
+ CacheReservationManagerImpl<
+ CacheEntryRole::kFilterConstruction>::GetDummyEntrySize() /
+ sizeof(uint64_t);
+
+ // For delegating between XXPH3FilterBitsBuilders
+ void SwapEntriesWith(XXPH3FilterBitsBuilder* other) {
+ assert(other != nullptr);
+ hash_entries_info_.Swap(&(other->hash_entries_info_));
+ }
+
+ void ResetEntries() { hash_entries_info_.Reset(); }
+
+ virtual size_t RoundDownUsableSpace(size_t available_size) = 0;
+
+ // To choose size using malloc_usable_size, we have to actually allocate.
+ size_t AllocateMaybeRounding(size_t target_len_with_metadata,
+ size_t num_entries,
+ std::unique_ptr<char[]>* buf) {
+ // Return value set to a default; overwritten in some cases
+ size_t rv = target_len_with_metadata;
+#ifdef ROCKSDB_MALLOC_USABLE_SIZE
+ if (aggregate_rounding_balance_ != nullptr) {
+ // Do optimize_filters_for_memory, using malloc_usable_size.
+ // Approach: try to keep FP rate balance better than or on
+ // target (negative aggregate_rounding_balance_). We can then select a
+ // lower bound filter size (within reasonable limits) that gets us as
+ // close to on target as possible. We request allocation for that filter
+ // size and use malloc_usable_size to "round up" to the actual
+ // allocation size.
+
+ // Although it can be considered bad practice to use malloc_usable_size
+ // to access an object beyond its original size, this approach should be
+ // quite general: working for all allocators that properly support
+ // malloc_usable_size.
+
+ // Race condition on balance is OK because it can only cause temporary
+ // skew in rounding up vs. rounding down, as long as updates are atomic
+ // and relative.
+ int64_t balance = aggregate_rounding_balance_->load();
+
+ double target_fp_rate =
+ EstimatedFpRate(num_entries, target_len_with_metadata);
+ double rv_fp_rate = target_fp_rate;
+
+ if (balance < 0) {
+ // See formula for BloomFilterPolicy::aggregate_rounding_balance_
+ double for_balance_fp_rate =
+ -balance / double{0x100000000} + target_fp_rate;
+
+ // To simplify, we just try a few modified smaller sizes. This also
+ // caps how much we vary filter size vs. target, to avoid outlier
+ // behavior from excessive variance.
+ size_t target_len = target_len_with_metadata - kMetadataLen;
+ assert(target_len < target_len_with_metadata); // check underflow
+ for (uint64_t maybe_len_rough :
+ {uint64_t{3} * target_len / 4, uint64_t{13} * target_len / 16,
+ uint64_t{7} * target_len / 8, uint64_t{15} * target_len / 16}) {
+ size_t maybe_len_with_metadata =
+ RoundDownUsableSpace(maybe_len_rough + kMetadataLen);
+ double maybe_fp_rate =
+ EstimatedFpRate(num_entries, maybe_len_with_metadata);
+ if (maybe_fp_rate <= for_balance_fp_rate) {
+ rv = maybe_len_with_metadata;
+ rv_fp_rate = maybe_fp_rate;
+ break;
+ }
+ }
+ }
+
+ // Filter blocks are loaded into block cache with their block trailer.
+ // We need to make sure that's accounted for in choosing a
+ // fragmentation-friendly size.
+ const size_t kExtraPadding = BlockBasedTable::kBlockTrailerSize;
+ size_t requested = rv + kExtraPadding;
+
+ // Allocate and get usable size
+ buf->reset(new char[requested]);
+ size_t usable = malloc_usable_size(buf->get());
+
+ if (usable - usable / 4 > requested) {
+ // Ratio greater than 4/3 is too much for utilizing, if it's
+ // not a buggy or mislinked malloc_usable_size implementation.
+ // Non-linearity of FP rates with bits/key means rapidly
+ // diminishing returns in overall accuracy for additional
+ // storage on disk.
+ // Nothing to do, except assert that the result is accurate about
+ // the usable size. (Assignment never used.)
+ assert(((*buf)[usable - 1] = 'x'));
+ } else if (usable > requested) {
+ rv = RoundDownUsableSpace(usable - kExtraPadding);
+ assert(rv <= usable - kExtraPadding);
+ rv_fp_rate = EstimatedFpRate(num_entries, rv);
+ } else {
+ // Too small means bad malloc_usable_size
+ assert(usable == requested);
+ }
+ memset(buf->get(), 0, rv);
+
+ // Update balance
+ int64_t diff = static_cast<int64_t>((rv_fp_rate - target_fp_rate) *
+ double{0x100000000});
+ *aggregate_rounding_balance_ += diff;
+ } else {
+ buf->reset(new char[rv]());
+ }
+#else
+ (void)num_entries;
+ buf->reset(new char[rv]());
+#endif // ROCKSDB_MALLOC_USABLE_SIZE
+ return rv;
+ }
+
+ // TODO: Ideally we want to verify the hash entry
+ // as it is added to the filter and eliminate this function
+ // for speeding up and leaving fewer spaces for undetected memory/CPU
+ // corruption. For Ribbon Filter, it's bit harder.
+ // Possible solution:
+ // pass a custom iterator that tracks the xor checksum as
+ // it iterates to ResetAndFindSeedToSolve
+ Status MaybeVerifyHashEntriesChecksum() {
+ if (!detect_filter_construct_corruption_) {
+ return Status::OK();
+ }
+
+ uint64_t actual_hash_entries_xor_checksum = 0;
+ for (uint64_t h : hash_entries_info_.entries) {
+ actual_hash_entries_xor_checksum ^= h;
+ }
+
+ if (actual_hash_entries_xor_checksum == hash_entries_info_.xor_checksum) {
+ return Status::OK();
+ } else {
+ // Since these hash entries are corrupted and they will not be used
+ // anymore, we can reset them and release memory.
+ ResetEntries();
+ return Status::Corruption("Filter's hash entries checksum mismatched");
+ }
+ }
+
+ // See BloomFilterPolicy::aggregate_rounding_balance_. If nullptr,
+ // always "round up" like historic behavior.
+ std::atomic<int64_t>* aggregate_rounding_balance_;
+
+ // For reserving memory used in (new) Bloom and Ribbon Filter construction
+ std::shared_ptr<CacheReservationManager> cache_res_mgr_;
+
+ // For managing cache charge for final filter in (new) Bloom and Ribbon
+ // Filter construction
+ std::deque<std::unique_ptr<CacheReservationManager::CacheReservationHandle>>
+ final_filter_cache_res_handles_;
+
+ bool detect_filter_construct_corruption_;
+
+ struct HashEntriesInfo {
+ // A deque avoids unnecessary copying of already-saved values
+ // and has near-minimal peak memory use.
+ std::deque<uint64_t> entries;
+
+ // If cache_res_mgr_ != nullptr,
+ // it manages cache charge for buckets of hash entries in (new) Bloom
+ // or Ribbon Filter construction.
+ // Otherwise, it is empty.
+ std::deque<std::unique_ptr<CacheReservationManager::CacheReservationHandle>>
+ cache_res_bucket_handles;
+
+ // If detect_filter_construct_corruption_ == true,
+ // it records the xor checksum of hash entries.
+ // Otherwise, it is 0.
+ uint64_t xor_checksum = 0;
+
+ void Swap(HashEntriesInfo* other) {
+ assert(other != nullptr);
+ std::swap(entries, other->entries);
+ std::swap(cache_res_bucket_handles, other->cache_res_bucket_handles);
+ std::swap(xor_checksum, other->xor_checksum);
+ }
+
+ void Reset() {
+ entries.clear();
+ cache_res_bucket_handles.clear();
+ xor_checksum = 0;
+ }
+ };
+
+ HashEntriesInfo hash_entries_info_;
+};
+
+// #################### FastLocalBloom implementation ################## //
+// ############## also known as format_version=5 Bloom filter ########## //
+
+// See description in FastLocalBloomImpl
+class FastLocalBloomBitsBuilder : public XXPH3FilterBitsBuilder {
+ public:
+ // Non-null aggregate_rounding_balance implies optimize_filters_for_memory
+ explicit FastLocalBloomBitsBuilder(
+ const int millibits_per_key,
+ std::atomic<int64_t>* aggregate_rounding_balance,
+ std::shared_ptr<CacheReservationManager> cache_res_mgr,
+ bool detect_filter_construct_corruption)
+ : XXPH3FilterBitsBuilder(aggregate_rounding_balance, cache_res_mgr,
+ detect_filter_construct_corruption),
+ millibits_per_key_(millibits_per_key) {
+ assert(millibits_per_key >= 1000);
+ }
+
+ // No Copy allowed
+ FastLocalBloomBitsBuilder(const FastLocalBloomBitsBuilder&) = delete;
+ void operator=(const FastLocalBloomBitsBuilder&) = delete;
+
+ ~FastLocalBloomBitsBuilder() override {}
+
+ using FilterBitsBuilder::Finish;
+
+ virtual Slice Finish(std::unique_ptr<const char[]>* buf) override {
+ return Finish(buf, nullptr);
+ }
+
+ virtual Slice Finish(std::unique_ptr<const char[]>* buf,
+ Status* status) override {
+ size_t num_entries = hash_entries_info_.entries.size();
+ size_t len_with_metadata = CalculateSpace(num_entries);
+
+ std::unique_ptr<char[]> mutable_buf;
+ std::unique_ptr<CacheReservationManager::CacheReservationHandle>
+ final_filter_cache_res_handle;
+ len_with_metadata =
+ AllocateMaybeRounding(len_with_metadata, num_entries, &mutable_buf);
+ // Cache charging for mutable_buf
+ if (cache_res_mgr_) {
+ Status s = cache_res_mgr_->MakeCacheReservation(
+ len_with_metadata * sizeof(char), &final_filter_cache_res_handle);
+ s.PermitUncheckedError();
+ }
+
+ assert(mutable_buf);
+ assert(len_with_metadata >= kMetadataLen);
+
+ // Max size supported by implementation
+ assert(len_with_metadata <= 0xffffffffU);
+
+ // Compute num_probes after any rounding / adjustments
+ int num_probes = GetNumProbes(num_entries, len_with_metadata);
+
+ uint32_t len = static_cast<uint32_t>(len_with_metadata - kMetadataLen);
+ if (len > 0) {
+ TEST_SYNC_POINT_CALLBACK(
+ "XXPH3FilterBitsBuilder::Finish::"
+ "TamperHashEntries",
+ &hash_entries_info_.entries);
+ AddAllEntries(mutable_buf.get(), len, num_probes);
+ Status verify_hash_entries_checksum_status =
+ MaybeVerifyHashEntriesChecksum();
+ if (!verify_hash_entries_checksum_status.ok()) {
+ if (status) {
+ *status = verify_hash_entries_checksum_status;
+ }
+ return FinishAlwaysTrue(buf);
+ }
+ }
+
+ bool keep_entries_for_postverify = detect_filter_construct_corruption_;
+ if (!keep_entries_for_postverify) {
+ ResetEntries();
+ }
+
+ // See BloomFilterPolicy::GetBloomBitsReader re: metadata
+ // -1 = Marker for newer Bloom implementations
+ mutable_buf[len] = static_cast<char>(-1);
+ // 0 = Marker for this sub-implementation
+ mutable_buf[len + 1] = static_cast<char>(0);
+ // num_probes (and 0 in upper bits for 64-byte block size)
+ mutable_buf[len + 2] = static_cast<char>(num_probes);
+ // rest of metadata stays zero
+
+ auto TEST_arg_pair __attribute__((__unused__)) =
+ std::make_pair(&mutable_buf, len_with_metadata);
+ TEST_SYNC_POINT_CALLBACK("XXPH3FilterBitsBuilder::Finish::TamperFilter",
+ &TEST_arg_pair);
+
+ Slice rv(mutable_buf.get(), len_with_metadata);
+ *buf = std::move(mutable_buf);
+ final_filter_cache_res_handles_.push_back(
+ std::move(final_filter_cache_res_handle));
+ if (status) {
+ *status = Status::OK();
+ }
+ return rv;
+ }
+
+ size_t ApproximateNumEntries(size_t bytes) override {
+ size_t bytes_no_meta =
+ bytes >= kMetadataLen ? RoundDownUsableSpace(bytes) - kMetadataLen : 0;
+ return static_cast<size_t>(uint64_t{8000} * bytes_no_meta /
+ millibits_per_key_);
+ }
+
+ size_t CalculateSpace(size_t num_entries) override {
+ // If not for cache line blocks in the filter, what would the target
+ // length in bytes be?
+ size_t raw_target_len = static_cast<size_t>(
+ (uint64_t{num_entries} * millibits_per_key_ + 7999) / 8000);
+
+ if (raw_target_len >= size_t{0xffffffc0}) {
+ // Max supported for this data structure implementation
+ raw_target_len = size_t{0xffffffc0};
+ }
+
+ // Round up to nearest multiple of 64 (block size). This adjustment is
+ // used for target FP rate only so that we don't receive complaints about
+ // lower FP rate vs. historic Bloom filter behavior.
+ return ((raw_target_len + 63) & ~size_t{63}) + kMetadataLen;
+ }
+
+ double EstimatedFpRate(size_t keys, size_t len_with_metadata) override {
+ int num_probes = GetNumProbes(keys, len_with_metadata);
+ return FastLocalBloomImpl::EstimatedFpRate(
+ keys, len_with_metadata - kMetadataLen, num_probes, /*hash bits*/ 64);
+ }
+
+ protected:
+ size_t RoundDownUsableSpace(size_t available_size) override {
+ size_t rv = available_size - kMetadataLen;
+
+ if (rv >= size_t{0xffffffc0}) {
+ // Max supported for this data structure implementation
+ rv = size_t{0xffffffc0};
+ }
+
+ // round down to multiple of 64 (block size)
+ rv &= ~size_t{63};
+
+ return rv + kMetadataLen;
+ }
+
+ private:
+ // Compute num_probes after any rounding / adjustments
+ int GetNumProbes(size_t keys, size_t len_with_metadata) {
+ uint64_t millibits = uint64_t{len_with_metadata - kMetadataLen} * 8000;
+ int actual_millibits_per_key =
+ static_cast<int>(millibits / std::max(keys, size_t{1}));
+ // BEGIN XXX/TODO(peterd): preserving old/default behavior for now to
+ // minimize unit test churn. Remove this some time.
+ if (!aggregate_rounding_balance_) {
+ actual_millibits_per_key = millibits_per_key_;
+ }
+ // END XXX/TODO
+ return FastLocalBloomImpl::ChooseNumProbes(actual_millibits_per_key);
+ }
+
+ void AddAllEntries(char* data, uint32_t len, int num_probes) {
+ // Simple version without prefetching:
+ //
+ // for (auto h : hash_entries_info_.entries) {
+ // FastLocalBloomImpl::AddHash(Lower32of64(h), Upper32of64(h), len,
+ // num_probes, data);
+ // }
+
+ const size_t num_entries = hash_entries_info_.entries.size();
+ constexpr size_t kBufferMask = 7;
+ static_assert(((kBufferMask + 1) & kBufferMask) == 0,
+ "Must be power of 2 minus 1");
+
+ std::array<uint32_t, kBufferMask + 1> hashes;
+ std::array<uint32_t, kBufferMask + 1> byte_offsets;
+
+ // Prime the buffer
+ size_t i = 0;
+ std::deque<uint64_t>::iterator hash_entries_it =
+ hash_entries_info_.entries.begin();
+ for (; i <= kBufferMask && i < num_entries; ++i) {
+ uint64_t h = *hash_entries_it;
+ FastLocalBloomImpl::PrepareHash(Lower32of64(h), len, data,
+ /*out*/ &byte_offsets[i]);
+ hashes[i] = Upper32of64(h);
+ ++hash_entries_it;
+ }
+
+ // Process and buffer
+ for (; i < num_entries; ++i) {
+ uint32_t& hash_ref = hashes[i & kBufferMask];
+ uint32_t& byte_offset_ref = byte_offsets[i & kBufferMask];
+ // Process (add)
+ FastLocalBloomImpl::AddHashPrepared(hash_ref, num_probes,
+ data + byte_offset_ref);
+ // And buffer
+ uint64_t h = *hash_entries_it;
+ FastLocalBloomImpl::PrepareHash(Lower32of64(h), len, data,
+ /*out*/ &byte_offset_ref);
+ hash_ref = Upper32of64(h);
+ ++hash_entries_it;
+ }
+
+ // Finish processing
+ for (i = 0; i <= kBufferMask && i < num_entries; ++i) {
+ FastLocalBloomImpl::AddHashPrepared(hashes[i], num_probes,
+ data + byte_offsets[i]);
+ }
+ }
+
+ // Target allocation per added key, in thousandths of a bit.
+ int millibits_per_key_;
+};
+
+// See description in FastLocalBloomImpl
+class FastLocalBloomBitsReader : public BuiltinFilterBitsReader {
+ public:
+ FastLocalBloomBitsReader(const char* data, int num_probes, uint32_t len_bytes)
+ : data_(data), num_probes_(num_probes), len_bytes_(len_bytes) {}
+
+ // No Copy allowed
+ FastLocalBloomBitsReader(const FastLocalBloomBitsReader&) = delete;
+ void operator=(const FastLocalBloomBitsReader&) = delete;
+
+ ~FastLocalBloomBitsReader() override {}
+
+ bool MayMatch(const Slice& key) override {
+ uint64_t h = GetSliceHash64(key);
+ uint32_t byte_offset;
+ FastLocalBloomImpl::PrepareHash(Lower32of64(h), len_bytes_, data_,
+ /*out*/ &byte_offset);
+ return FastLocalBloomImpl::HashMayMatchPrepared(Upper32of64(h), num_probes_,
+ data_ + byte_offset);
+ }
+
+ virtual void MayMatch(int num_keys, Slice** keys, bool* may_match) override {
+ std::array<uint32_t, MultiGetContext::MAX_BATCH_SIZE> hashes;
+ std::array<uint32_t, MultiGetContext::MAX_BATCH_SIZE> byte_offsets;
+ for (int i = 0; i < num_keys; ++i) {
+ uint64_t h = GetSliceHash64(*keys[i]);
+ FastLocalBloomImpl::PrepareHash(Lower32of64(h), len_bytes_, data_,
+ /*out*/ &byte_offsets[i]);
+ hashes[i] = Upper32of64(h);
+ }
+ for (int i = 0; i < num_keys; ++i) {
+ may_match[i] = FastLocalBloomImpl::HashMayMatchPrepared(
+ hashes[i], num_probes_, data_ + byte_offsets[i]);
+ }
+ }
+
+ bool HashMayMatch(const uint64_t h) override {
+ return FastLocalBloomImpl::HashMayMatch(Lower32of64(h), Upper32of64(h),
+ len_bytes_, num_probes_, data_);
+ }
+
+ private:
+ const char* data_;
+ const int num_probes_;
+ const uint32_t len_bytes_;
+};
+
+// ##################### Ribbon filter implementation ################### //
+
+// Implements concept RehasherTypesAndSettings in ribbon_impl.h
+struct Standard128RibbonRehasherTypesAndSettings {
+ // These are schema-critical. Any change almost certainly changes
+ // underlying data.
+ static constexpr bool kIsFilter = true;
+ static constexpr bool kHomogeneous = false;
+ static constexpr bool kFirstCoeffAlwaysOne = true;
+ static constexpr bool kUseSmash = false;
+ using CoeffRow = ROCKSDB_NAMESPACE::Unsigned128;
+ using Hash = uint64_t;
+ using Seed = uint32_t;
+ // Changing these doesn't necessarily change underlying data,
+ // but might affect supported scalability of those dimensions.
+ using Index = uint32_t;
+ using ResultRow = uint32_t;
+ // Save a conditional in Ribbon queries
+ static constexpr bool kAllowZeroStarts = false;
+};
+
+using Standard128RibbonTypesAndSettings =
+ ribbon::StandardRehasherAdapter<Standard128RibbonRehasherTypesAndSettings>;
+
+class Standard128RibbonBitsBuilder : public XXPH3FilterBitsBuilder {
+ public:
+ explicit Standard128RibbonBitsBuilder(
+ double desired_one_in_fp_rate, int bloom_millibits_per_key,
+ std::atomic<int64_t>* aggregate_rounding_balance,
+ std::shared_ptr<CacheReservationManager> cache_res_mgr,
+ bool detect_filter_construct_corruption, Logger* info_log)
+ : XXPH3FilterBitsBuilder(aggregate_rounding_balance, cache_res_mgr,
+ detect_filter_construct_corruption),
+ desired_one_in_fp_rate_(desired_one_in_fp_rate),
+ info_log_(info_log),
+ bloom_fallback_(bloom_millibits_per_key, aggregate_rounding_balance,
+ cache_res_mgr, detect_filter_construct_corruption) {
+ assert(desired_one_in_fp_rate >= 1.0);
+ }
+
+ // No Copy allowed
+ Standard128RibbonBitsBuilder(const Standard128RibbonBitsBuilder&) = delete;
+ void operator=(const Standard128RibbonBitsBuilder&) = delete;
+
+ ~Standard128RibbonBitsBuilder() override {}
+
+ using FilterBitsBuilder::Finish;
+
+ virtual Slice Finish(std::unique_ptr<const char[]>* buf) override {
+ return Finish(buf, nullptr);
+ }
+
+ virtual Slice Finish(std::unique_ptr<const char[]>* buf,
+ Status* status) override {
+ if (hash_entries_info_.entries.size() > kMaxRibbonEntries) {
+ ROCKS_LOG_WARN(
+ info_log_, "Too many keys for Ribbon filter: %llu",
+ static_cast<unsigned long long>(hash_entries_info_.entries.size()));
+ SwapEntriesWith(&bloom_fallback_);
+ assert(hash_entries_info_.entries.empty());
+ return bloom_fallback_.Finish(buf, status);
+ }
+ if (hash_entries_info_.entries.size() == 0) {
+ // Save a conditional in Ribbon queries by using alternate reader
+ // for zero entries added.
+ if (status) {
+ *status = Status::OK();
+ }
+ return FinishAlwaysFalse(buf);
+ }
+ uint32_t num_entries =
+ static_cast<uint32_t>(hash_entries_info_.entries.size());
+ uint32_t num_slots;
+ size_t len_with_metadata;
+
+ CalculateSpaceAndSlots(num_entries, &len_with_metadata, &num_slots);
+
+ // Bloom fall-back indicator
+ if (num_slots == 0) {
+ SwapEntriesWith(&bloom_fallback_);
+ assert(hash_entries_info_.entries.empty());
+ return bloom_fallback_.Finish(buf, status);
+ }
+
+ uint32_t entropy = 0;
+ if (!hash_entries_info_.entries.empty()) {
+ entropy = Lower32of64(hash_entries_info_.entries.front());
+ }
+
+ BandingType banding;
+ std::size_t bytes_banding = ribbon::StandardBanding<
+ Standard128RibbonTypesAndSettings>::EstimateMemoryUsage(num_slots);
+ Status status_banding_cache_res = Status::OK();
+
+ // Cache charging for banding
+ std::unique_ptr<CacheReservationManager::CacheReservationHandle>
+ banding_res_handle;
+ if (cache_res_mgr_) {
+ status_banding_cache_res = cache_res_mgr_->MakeCacheReservation(
+ bytes_banding, &banding_res_handle);
+ }
+
+ if (status_banding_cache_res.IsMemoryLimit()) {
+ ROCKS_LOG_WARN(info_log_,
+ "Cache charging for Ribbon filter banding failed due "
+ "to cache full");
+ SwapEntriesWith(&bloom_fallback_);
+ assert(hash_entries_info_.entries.empty());
+ // Release cache for banding since the banding won't be allocated
+ banding_res_handle.reset();
+ return bloom_fallback_.Finish(buf, status);
+ }
+
+ TEST_SYNC_POINT_CALLBACK(
+ "XXPH3FilterBitsBuilder::Finish::"
+ "TamperHashEntries",
+ &hash_entries_info_.entries);
+
+ bool success = banding.ResetAndFindSeedToSolve(
+ num_slots, hash_entries_info_.entries.begin(),
+ hash_entries_info_.entries.end(),
+ /*starting seed*/ entropy & 255, /*seed mask*/ 255);
+ if (!success) {
+ ROCKS_LOG_WARN(
+ info_log_, "Too many re-seeds (256) for Ribbon filter, %llu / %llu",
+ static_cast<unsigned long long>(hash_entries_info_.entries.size()),
+ static_cast<unsigned long long>(num_slots));
+ SwapEntriesWith(&bloom_fallback_);
+ assert(hash_entries_info_.entries.empty());
+ return bloom_fallback_.Finish(buf, status);
+ }
+
+ Status verify_hash_entries_checksum_status =
+ MaybeVerifyHashEntriesChecksum();
+ if (!verify_hash_entries_checksum_status.ok()) {
+ ROCKS_LOG_WARN(info_log_, "Verify hash entries checksum error: %s",
+ verify_hash_entries_checksum_status.getState());
+ if (status) {
+ *status = verify_hash_entries_checksum_status;
+ }
+ return FinishAlwaysTrue(buf);
+ }
+
+ bool keep_entries_for_postverify = detect_filter_construct_corruption_;
+ if (!keep_entries_for_postverify) {
+ ResetEntries();
+ }
+
+ uint32_t seed = banding.GetOrdinalSeed();
+ assert(seed < 256);
+
+ std::unique_ptr<char[]> mutable_buf;
+ std::unique_ptr<CacheReservationManager::CacheReservationHandle>
+ final_filter_cache_res_handle;
+ len_with_metadata =
+ AllocateMaybeRounding(len_with_metadata, num_entries, &mutable_buf);
+ // Cache charging for mutable_buf
+ if (cache_res_mgr_) {
+ Status s = cache_res_mgr_->MakeCacheReservation(
+ len_with_metadata * sizeof(char), &final_filter_cache_res_handle);
+ s.PermitUncheckedError();
+ }
+
+ SolnType soln(mutable_buf.get(), len_with_metadata);
+ soln.BackSubstFrom(banding);
+ uint32_t num_blocks = soln.GetNumBlocks();
+ // This should be guaranteed:
+ // num_entries < 2^30
+ // => (overhead_factor < 2.0)
+ // num_entries * overhead_factor == num_slots < 2^31
+ // => (num_blocks = num_slots / 128)
+ // num_blocks < 2^24
+ assert(num_blocks < 0x1000000U);
+
+ // See BloomFilterPolicy::GetBloomBitsReader re: metadata
+ // -2 = Marker for Standard128 Ribbon
+ mutable_buf[len_with_metadata - 5] = static_cast<char>(-2);
+ // Hash seed
+ mutable_buf[len_with_metadata - 4] = static_cast<char>(seed);
+ // Number of blocks, in 24 bits
+ // (Along with bytes, we can derive other settings)
+ mutable_buf[len_with_metadata - 3] = static_cast<char>(num_blocks & 255);
+ mutable_buf[len_with_metadata - 2] =
+ static_cast<char>((num_blocks >> 8) & 255);
+ mutable_buf[len_with_metadata - 1] =
+ static_cast<char>((num_blocks >> 16) & 255);
+
+ auto TEST_arg_pair __attribute__((__unused__)) =
+ std::make_pair(&mutable_buf, len_with_metadata);
+ TEST_SYNC_POINT_CALLBACK("XXPH3FilterBitsBuilder::Finish::TamperFilter",
+ &TEST_arg_pair);
+
+ Slice rv(mutable_buf.get(), len_with_metadata);
+ *buf = std::move(mutable_buf);
+ final_filter_cache_res_handles_.push_back(
+ std::move(final_filter_cache_res_handle));
+ if (status) {
+ *status = Status::OK();
+ }
+ return rv;
+ }
+
+ // Setting num_slots to 0 means "fall back on Bloom filter."
+ // And note this implementation does not support num_entries or num_slots
+ // beyond uint32_t; see kMaxRibbonEntries.
+ void CalculateSpaceAndSlots(size_t num_entries,
+ size_t* target_len_with_metadata,
+ uint32_t* num_slots) {
+ if (num_entries > kMaxRibbonEntries) {
+ // More entries than supported by this Ribbon
+ *num_slots = 0; // use Bloom
+ *target_len_with_metadata = bloom_fallback_.CalculateSpace(num_entries);
+ return;
+ }
+ uint32_t entropy = 0;
+ if (!hash_entries_info_.entries.empty()) {
+ entropy = Upper32of64(hash_entries_info_.entries.front());
+ }
+
+ *num_slots = NumEntriesToNumSlots(static_cast<uint32_t>(num_entries));
+ *target_len_with_metadata =
+ SolnType::GetBytesForOneInFpRate(*num_slots, desired_one_in_fp_rate_,
+ /*rounding*/ entropy) +
+ kMetadataLen;
+
+ // Consider possible Bloom fallback for small filters
+ if (*num_slots < 1024) {
+ size_t bloom = bloom_fallback_.CalculateSpace(num_entries);
+ if (bloom < *target_len_with_metadata) {
+ *num_slots = 0; // use Bloom
+ *target_len_with_metadata = bloom;
+ return;
+ }
+ }
+ }
+
+ size_t CalculateSpace(size_t num_entries) override {
+ if (num_entries == 0) {
+ // See FinishAlwaysFalse
+ return 0;
+ }
+ size_t target_len_with_metadata;
+ uint32_t num_slots;
+ CalculateSpaceAndSlots(num_entries, &target_len_with_metadata, &num_slots);
+ (void)num_slots;
+ return target_len_with_metadata;
+ }
+
+ // This is a somewhat ugly but reasonably fast and reasonably accurate
+ // reversal of CalculateSpace.
+ size_t ApproximateNumEntries(size_t bytes) override {
+ size_t len_no_metadata =
+ RoundDownUsableSpace(std::max(bytes, size_t{kMetadataLen})) -
+ kMetadataLen;
+
+ if (!(desired_one_in_fp_rate_ > 1.0)) {
+ // Effectively asking for 100% FP rate, or NaN etc.
+ // Note that NaN is neither < 1.0 nor > 1.0
+ return kMaxRibbonEntries;
+ }
+
+ // Find a slight under-estimate for actual average bits per slot
+ double min_real_bits_per_slot;
+ if (desired_one_in_fp_rate_ >= 1.0 + std::numeric_limits<uint32_t>::max()) {
+ // Max of 32 solution columns (result bits)
+ min_real_bits_per_slot = 32.0;
+ } else {
+ // Account for mix of b and b+1 solution columns being slightly
+ // suboptimal vs. ideal log2(1/fp_rate) bits.
+ uint32_t rounded = static_cast<uint32_t>(desired_one_in_fp_rate_);
+ int upper_bits_per_key = 1 + FloorLog2(rounded);
+ double fp_rate_for_upper = std::pow(2.0, -upper_bits_per_key);
+ double portion_lower =
+ (1.0 / desired_one_in_fp_rate_ - fp_rate_for_upper) /
+ fp_rate_for_upper;
+ min_real_bits_per_slot = upper_bits_per_key - portion_lower;
+ assert(min_real_bits_per_slot > 0.0);
+ assert(min_real_bits_per_slot <= 32.0);
+ }
+
+ // An overestimate, but this should only be O(1) slots away from truth.
+ double max_slots = len_no_metadata * 8.0 / min_real_bits_per_slot;
+
+ // Let's not bother accounting for overflow to Bloom filter
+ // (Includes NaN case)
+ if (!(max_slots < ConfigHelper::GetNumSlots(kMaxRibbonEntries))) {
+ return kMaxRibbonEntries;
+ }
+
+ // Set up for short iteration
+ uint32_t slots = static_cast<uint32_t>(max_slots);
+ slots = SolnType::RoundUpNumSlots(slots);
+
+ // Assert that we have a valid upper bound on slots
+ assert(SolnType::GetBytesForOneInFpRate(
+ SolnType::RoundUpNumSlots(slots + 1), desired_one_in_fp_rate_,
+ /*rounding*/ 0) > len_no_metadata);
+
+ // Iterate up to a few times to rather precisely account for small effects
+ for (int i = 0; slots > 0; ++i) {
+ size_t reqd_bytes =
+ SolnType::GetBytesForOneInFpRate(slots, desired_one_in_fp_rate_,
+ /*rounding*/ 0);
+ if (reqd_bytes <= len_no_metadata) {
+ break; // done
+ }
+ if (i >= 2) {
+ // should have been enough iterations
+ assert(false);
+ break;
+ }
+ slots = SolnType::RoundDownNumSlots(slots - 1);
+ }
+
+ uint32_t num_entries = ConfigHelper::GetNumToAdd(slots);
+
+ // Consider possible Bloom fallback for small filters
+ if (slots < 1024) {
+ size_t bloom = bloom_fallback_.ApproximateNumEntries(bytes);
+ if (bloom > num_entries) {
+ return bloom;
+ } else {
+ return num_entries;
+ }
+ } else {
+ return std::min(num_entries, kMaxRibbonEntries);
+ }
+ }
+
+ double EstimatedFpRate(size_t num_entries,
+ size_t len_with_metadata) override {
+ if (num_entries > kMaxRibbonEntries) {
+ // More entries than supported by this Ribbon
+ return bloom_fallback_.EstimatedFpRate(num_entries, len_with_metadata);
+ }
+ uint32_t num_slots =
+ NumEntriesToNumSlots(static_cast<uint32_t>(num_entries));
+ SolnType fake_soln(nullptr, len_with_metadata);
+ fake_soln.ConfigureForNumSlots(num_slots);
+ return fake_soln.ExpectedFpRate();
+ }
+
+ Status MaybePostVerify(const Slice& filter_content) override {
+ bool fall_back = (bloom_fallback_.EstimateEntriesAdded() > 0);
+ return fall_back ? bloom_fallback_.MaybePostVerify(filter_content)
+ : XXPH3FilterBitsBuilder::MaybePostVerify(filter_content);
+ }
+
+ protected:
+ size_t RoundDownUsableSpace(size_t available_size) override {
+ size_t rv = available_size - kMetadataLen;
+
+ // round down to multiple of 16 (segment size)
+ rv &= ~size_t{15};
+
+ return rv + kMetadataLen;
+ }
+
+ private:
+ using TS = Standard128RibbonTypesAndSettings;
+ using SolnType = ribbon::SerializableInterleavedSolution<TS>;
+ using BandingType = ribbon::StandardBanding<TS>;
+ using ConfigHelper = ribbon::BandingConfigHelper1TS<ribbon::kOneIn20, TS>;
+
+ static uint32_t NumEntriesToNumSlots(uint32_t num_entries) {
+ uint32_t num_slots1 = ConfigHelper::GetNumSlots(num_entries);
+ return SolnType::RoundUpNumSlots(num_slots1);
+ }
+
+ // Approximate num_entries to ensure number of bytes fits in 32 bits, which
+ // is not an inherent limitation but does ensure somewhat graceful Bloom
+ // fallback for crazy high number of entries, since the Bloom implementation
+ // does not support number of bytes bigger than fits in 32 bits. This is
+ // within an order of magnitude of implementation limit on num_slots
+ // fitting in 32 bits, and even closer for num_blocks fitting in 24 bits
+ // (for filter metadata).
+ static constexpr uint32_t kMaxRibbonEntries = 950000000; // ~ 1 billion
+
+ // A desired value for 1/fp_rate. For example, 100 -> 1% fp rate.
+ double desired_one_in_fp_rate_;
+
+ // For warnings, or can be nullptr
+ Logger* info_log_;
+
+ // For falling back on Bloom filter in some exceptional cases and
+ // very small filter cases
+ FastLocalBloomBitsBuilder bloom_fallback_;
+};
+
+// for the linker, at least with DEBUG_LEVEL=2
+constexpr uint32_t Standard128RibbonBitsBuilder::kMaxRibbonEntries;
+
+class Standard128RibbonBitsReader : public BuiltinFilterBitsReader {
+ public:
+ Standard128RibbonBitsReader(const char* data, size_t len_bytes,
+ uint32_t num_blocks, uint32_t seed)
+ : soln_(const_cast<char*>(data), len_bytes) {
+ soln_.ConfigureForNumBlocks(num_blocks);
+ hasher_.SetOrdinalSeed(seed);
+ }
+
+ // No Copy allowed
+ Standard128RibbonBitsReader(const Standard128RibbonBitsReader&) = delete;
+ void operator=(const Standard128RibbonBitsReader&) = delete;
+
+ ~Standard128RibbonBitsReader() override {}
+
+ bool MayMatch(const Slice& key) override {
+ uint64_t h = GetSliceHash64(key);
+ return soln_.FilterQuery(h, hasher_);
+ }
+
+ virtual void MayMatch(int num_keys, Slice** keys, bool* may_match) override {
+ struct SavedData {
+ uint64_t seeded_hash;
+ uint32_t segment_num;
+ uint32_t num_columns;
+ uint32_t start_bits;
+ };
+ std::array<SavedData, MultiGetContext::MAX_BATCH_SIZE> saved;
+ for (int i = 0; i < num_keys; ++i) {
+ ribbon::InterleavedPrepareQuery(
+ GetSliceHash64(*keys[i]), hasher_, soln_, &saved[i].seeded_hash,
+ &saved[i].segment_num, &saved[i].num_columns, &saved[i].start_bits);
+ }
+ for (int i = 0; i < num_keys; ++i) {
+ may_match[i] = ribbon::InterleavedFilterQuery(
+ saved[i].seeded_hash, saved[i].segment_num, saved[i].num_columns,
+ saved[i].start_bits, hasher_, soln_);
+ }
+ }
+
+ bool HashMayMatch(const uint64_t h) override {
+ return soln_.FilterQuery(h, hasher_);
+ }
+
+ private:
+ using TS = Standard128RibbonTypesAndSettings;
+ ribbon::SerializableInterleavedSolution<TS> soln_;
+ ribbon::StandardHasher<TS> hasher_;
+};
+
+// ##################### Legacy Bloom implementation ################### //
+
+using LegacyBloomImpl = LegacyLocalityBloomImpl</*ExtraRotates*/ false>;
+
+class LegacyBloomBitsBuilder : public BuiltinFilterBitsBuilder {
+ public:
+ explicit LegacyBloomBitsBuilder(const int bits_per_key, Logger* info_log);
+
+ // No Copy allowed
+ LegacyBloomBitsBuilder(const LegacyBloomBitsBuilder&) = delete;
+ void operator=(const LegacyBloomBitsBuilder&) = delete;
+
+ ~LegacyBloomBitsBuilder() override;
+
+ void AddKey(const Slice& key) override;
+
+ virtual size_t EstimateEntriesAdded() override {
+ return hash_entries_.size();
+ }
+
+ using FilterBitsBuilder::Finish;
+
+ Slice Finish(std::unique_ptr<const char[]>* buf) override;
+
+ size_t CalculateSpace(size_t num_entries) override {
+ uint32_t dont_care1;
+ uint32_t dont_care2;
+ return CalculateSpace(num_entries, &dont_care1, &dont_care2);
+ }
+
+ double EstimatedFpRate(size_t keys, size_t bytes) override {
+ return LegacyBloomImpl::EstimatedFpRate(keys, bytes - kMetadataLen,
+ num_probes_);
+ }
+
+ size_t ApproximateNumEntries(size_t bytes) override;
+
+ private:
+ int bits_per_key_;
+ int num_probes_;
+ std::vector<uint32_t> hash_entries_;
+ Logger* info_log_;
+
+ // Get totalbits that optimized for cpu cache line
+ uint32_t GetTotalBitsForLocality(uint32_t total_bits);
+
+ // Reserve space for new filter
+ char* ReserveSpace(size_t num_entries, uint32_t* total_bits,
+ uint32_t* num_lines);
+
+ // Implementation-specific variant of public CalculateSpace
+ uint32_t CalculateSpace(size_t num_entries, uint32_t* total_bits,
+ uint32_t* num_lines);
+
+ // Assuming single threaded access to this function.
+ void AddHash(uint32_t h, char* data, uint32_t num_lines, uint32_t total_bits);
+};
+
+LegacyBloomBitsBuilder::LegacyBloomBitsBuilder(const int bits_per_key,
+ Logger* info_log)
+ : bits_per_key_(bits_per_key),
+ num_probes_(LegacyNoLocalityBloomImpl::ChooseNumProbes(bits_per_key_)),
+ info_log_(info_log) {
+ assert(bits_per_key_);
+}
+
+LegacyBloomBitsBuilder::~LegacyBloomBitsBuilder() {}
+
+void LegacyBloomBitsBuilder::AddKey(const Slice& key) {
+ uint32_t hash = BloomHash(key);
+ if (hash_entries_.size() == 0 || hash != hash_entries_.back()) {
+ hash_entries_.push_back(hash);
+ }
+}
+
+Slice LegacyBloomBitsBuilder::Finish(std::unique_ptr<const char[]>* buf) {
+ uint32_t total_bits, num_lines;
+ size_t num_entries = hash_entries_.size();
+ char* data =
+ ReserveSpace(static_cast<int>(num_entries), &total_bits, &num_lines);
+ assert(data);
+
+ if (total_bits != 0 && num_lines != 0) {
+ for (auto h : hash_entries_) {
+ AddHash(h, data, num_lines, total_bits);
+ }
+
+ // Check for excessive entries for 32-bit hash function
+ if (num_entries >= /* minimum of 3 million */ 3000000U) {
+ // More specifically, we can detect that the 32-bit hash function
+ // is causing significant increase in FP rate by comparing current
+ // estimated FP rate to what we would get with a normal number of
+ // keys at same memory ratio.
+ double est_fp_rate = LegacyBloomImpl::EstimatedFpRate(
+ num_entries, total_bits / 8, num_probes_);
+ double vs_fp_rate = LegacyBloomImpl::EstimatedFpRate(
+ 1U << 16, (1U << 16) * bits_per_key_ / 8, num_probes_);
+
+ if (est_fp_rate >= 1.50 * vs_fp_rate) {
+ // For more details, see
+ // https://github.com/facebook/rocksdb/wiki/RocksDB-Bloom-Filter
+ ROCKS_LOG_WARN(
+ info_log_,
+ "Using legacy SST/BBT Bloom filter with excessive key count "
+ "(%.1fM @ %dbpk), causing estimated %.1fx higher filter FP rate. "
+ "Consider using new Bloom with format_version>=5, smaller SST "
+ "file size, or partitioned filters.",
+ num_entries / 1000000.0, bits_per_key_, est_fp_rate / vs_fp_rate);
+ }
+ }
+ }
+ // See BloomFilterPolicy::GetFilterBitsReader for metadata
+ data[total_bits / 8] = static_cast<char>(num_probes_);
+ EncodeFixed32(data + total_bits / 8 + 1, static_cast<uint32_t>(num_lines));
+
+ const char* const_data = data;
+ buf->reset(const_data);
+ hash_entries_.clear();
+
+ return Slice(data, total_bits / 8 + kMetadataLen);
+}
+
+size_t LegacyBloomBitsBuilder::ApproximateNumEntries(size_t bytes) {
+ assert(bits_per_key_);
+ assert(bytes > 0);
+
+ uint64_t total_bits_tmp = bytes * 8;
+ // total bits, including temporary computations, cannot exceed 2^32
+ // for compatibility
+ total_bits_tmp = std::min(total_bits_tmp, uint64_t{0xffff0000});
+
+ uint32_t high = static_cast<uint32_t>(total_bits_tmp) /
+ static_cast<uint32_t>(bits_per_key_) +
+ 1;
+ uint32_t low = 1;
+ uint32_t n = high;
+ for (; n >= low; n--) {
+ if (CalculateSpace(n) <= bytes) {
+ break;
+ }
+ }
+ return n;
+}
+
+uint32_t LegacyBloomBitsBuilder::GetTotalBitsForLocality(uint32_t total_bits) {
+ uint32_t num_lines =
+ (total_bits + CACHE_LINE_SIZE * 8 - 1) / (CACHE_LINE_SIZE * 8);
+
+ // Make num_lines an odd number to make sure more bits are involved
+ // when determining which block.
+ if (num_lines % 2 == 0) {
+ num_lines++;
+ }
+ return num_lines * (CACHE_LINE_SIZE * 8);
+}
+
+uint32_t LegacyBloomBitsBuilder::CalculateSpace(size_t num_entries,
+ uint32_t* total_bits,
+ uint32_t* num_lines) {
+ assert(bits_per_key_);
+ if (num_entries != 0) {
+ size_t total_bits_tmp = num_entries * bits_per_key_;
+ // total bits, including temporary computations, cannot exceed 2^32
+ // for compatibility
+ total_bits_tmp = std::min(total_bits_tmp, size_t{0xffff0000});
+
+ *total_bits =
+ GetTotalBitsForLocality(static_cast<uint32_t>(total_bits_tmp));
+ *num_lines = *total_bits / (CACHE_LINE_SIZE * 8);
+ assert(*total_bits > 0 && *total_bits % 8 == 0);
+ } else {
+ // filter is empty, just leave space for metadata
+ *total_bits = 0;
+ *num_lines = 0;
+ }
+
+ // Reserve space for Filter
+ uint32_t sz = *total_bits / 8;
+ sz += kMetadataLen; // 4 bytes for num_lines, 1 byte for num_probes
+ return sz;
+}
+
+char* LegacyBloomBitsBuilder::ReserveSpace(size_t num_entries,
+ uint32_t* total_bits,
+ uint32_t* num_lines) {
+ uint32_t sz = CalculateSpace(num_entries, total_bits, num_lines);
+ char* data = new char[sz];
+ memset(data, 0, sz);
+ return data;
+}
+
+inline void LegacyBloomBitsBuilder::AddHash(uint32_t h, char* data,
+ uint32_t num_lines,
+ uint32_t total_bits) {
+#ifdef NDEBUG
+ static_cast<void>(total_bits);
+#endif
+ assert(num_lines > 0 && total_bits > 0);
+
+ LegacyBloomImpl::AddHash(h, num_lines, num_probes_, data,
+ ConstexprFloorLog2(CACHE_LINE_SIZE));
+}
+
+class LegacyBloomBitsReader : public BuiltinFilterBitsReader {
+ public:
+ LegacyBloomBitsReader(const char* data, int num_probes, uint32_t num_lines,
+ uint32_t log2_cache_line_size)
+ : data_(data),
+ num_probes_(num_probes),
+ num_lines_(num_lines),
+ log2_cache_line_size_(log2_cache_line_size) {}
+
+ // No Copy allowed
+ LegacyBloomBitsReader(const LegacyBloomBitsReader&) = delete;
+ void operator=(const LegacyBloomBitsReader&) = delete;
+
+ ~LegacyBloomBitsReader() override {}
+
+ // "contents" contains the data built by a preceding call to
+ // FilterBitsBuilder::Finish. MayMatch must return true if the key was
+ // passed to FilterBitsBuilder::AddKey. This method may return true or false
+ // if the key was not on the list, but it should aim to return false with a
+ // high probability.
+ bool MayMatch(const Slice& key) override {
+ uint32_t hash = BloomHash(key);
+ uint32_t byte_offset;
+ LegacyBloomImpl::PrepareHashMayMatch(
+ hash, num_lines_, data_, /*out*/ &byte_offset, log2_cache_line_size_);
+ return LegacyBloomImpl::HashMayMatchPrepared(
+ hash, num_probes_, data_ + byte_offset, log2_cache_line_size_);
+ }
+
+ virtual void MayMatch(int num_keys, Slice** keys, bool* may_match) override {
+ std::array<uint32_t, MultiGetContext::MAX_BATCH_SIZE> hashes;
+ std::array<uint32_t, MultiGetContext::MAX_BATCH_SIZE> byte_offsets;
+ for (int i = 0; i < num_keys; ++i) {
+ hashes[i] = BloomHash(*keys[i]);
+ LegacyBloomImpl::PrepareHashMayMatch(hashes[i], num_lines_, data_,
+ /*out*/ &byte_offsets[i],
+ log2_cache_line_size_);
+ }
+ for (int i = 0; i < num_keys; ++i) {
+ may_match[i] = LegacyBloomImpl::HashMayMatchPrepared(
+ hashes[i], num_probes_, data_ + byte_offsets[i],
+ log2_cache_line_size_);
+ }
+ }
+
+ bool HashMayMatch(const uint64_t /* h */) override { return false; }
+
+ private:
+ const char* data_;
+ const int num_probes_;
+ const uint32_t num_lines_;
+ const uint32_t log2_cache_line_size_;
+};
+
+class AlwaysTrueFilter : public BuiltinFilterBitsReader {
+ public:
+ bool MayMatch(const Slice&) override { return true; }
+ using FilterBitsReader::MayMatch; // inherit overload
+ bool HashMayMatch(const uint64_t) override { return true; }
+ using BuiltinFilterBitsReader::HashMayMatch; // inherit overload
+};
+
+class AlwaysFalseFilter : public BuiltinFilterBitsReader {
+ public:
+ bool MayMatch(const Slice&) override { return false; }
+ using FilterBitsReader::MayMatch; // inherit overload
+ bool HashMayMatch(const uint64_t) override { return false; }
+ using BuiltinFilterBitsReader::HashMayMatch; // inherit overload
+};
+
+Status XXPH3FilterBitsBuilder::MaybePostVerify(const Slice& filter_content) {
+ Status s = Status::OK();
+
+ if (!detect_filter_construct_corruption_) {
+ return s;
+ }
+
+ std::unique_ptr<BuiltinFilterBitsReader> bits_reader(
+ BuiltinFilterPolicy::GetBuiltinFilterBitsReader(filter_content));
+
+ for (uint64_t h : hash_entries_info_.entries) {
+ // The current approach will not detect corruption from XXPH3Filter to
+ // AlwaysTrueFilter, which can lead to performance cost later due to
+ // AlwaysTrueFilter not filtering anything. But this cost is acceptable
+ // given the extra implementation complixity to detect such case.
+ bool may_match = bits_reader->HashMayMatch(h);
+ if (!may_match) {
+ s = Status::Corruption("Corrupted filter content");
+ break;
+ }
+ }
+
+ ResetEntries();
+ return s;
+}
+} // namespace
+
+const char* BuiltinFilterPolicy::kClassName() {
+ return "rocksdb.internal.BuiltinFilter";
+}
+
+bool BuiltinFilterPolicy::IsInstanceOf(const std::string& name) const {
+ if (name == kClassName()) {
+ return true;
+ } else {
+ return FilterPolicy::IsInstanceOf(name);
+ }
+}
+
+static const char* kBuiltinFilterMetadataName = "rocksdb.BuiltinBloomFilter";
+
+const char* BuiltinFilterPolicy::kCompatibilityName() {
+ return kBuiltinFilterMetadataName;
+}
+
+const char* BuiltinFilterPolicy::CompatibilityName() const {
+ return kBuiltinFilterMetadataName;
+}
+
+BloomLikeFilterPolicy::BloomLikeFilterPolicy(double bits_per_key)
+ : warned_(false), aggregate_rounding_balance_(0) {
+ // Sanitize bits_per_key
+ if (bits_per_key < 0.5) {
+ // Round down to no filter
+ bits_per_key = 0;
+ } else if (bits_per_key < 1.0) {
+ // Minimum 1 bit per key (equiv) when creating filter
+ bits_per_key = 1.0;
+ } else if (!(bits_per_key < 100.0)) { // including NaN
+ bits_per_key = 100.0;
+ }
+
+ // Includes a nudge toward rounding up, to ensure on all platforms
+ // that doubles specified with three decimal digits after the decimal
+ // point are interpreted accurately.
+ millibits_per_key_ = static_cast<int>(bits_per_key * 1000.0 + 0.500001);
+
+ // For now configure Ribbon filter to match Bloom FP rate and save
+ // memory. (Ribbon bits per key will be ~30% less than Bloom bits per key
+ // for same FP rate.)
+ desired_one_in_fp_rate_ =
+ 1.0 / BloomMath::CacheLocalFpRate(
+ bits_per_key,
+ FastLocalBloomImpl::ChooseNumProbes(millibits_per_key_),
+ /*cache_line_bits*/ 512);
+
+ // For better or worse, this is a rounding up of a nudged rounding up,
+ // e.g. 7.4999999999999 will round up to 8, but that provides more
+ // predictability against small arithmetic errors in floating point.
+ whole_bits_per_key_ = (millibits_per_key_ + 500) / 1000;
+}
+
+BloomLikeFilterPolicy::~BloomLikeFilterPolicy() {}
+const char* BloomLikeFilterPolicy::kClassName() {
+ return "rocksdb.internal.BloomLikeFilter";
+}
+
+bool BloomLikeFilterPolicy::IsInstanceOf(const std::string& name) const {
+ if (name == kClassName()) {
+ return true;
+ } else {
+ return BuiltinFilterPolicy::IsInstanceOf(name);
+ }
+}
+
+const char* ReadOnlyBuiltinFilterPolicy::kClassName() {
+ return kBuiltinFilterMetadataName;
+}
+
+std::string BloomLikeFilterPolicy::GetId() const {
+ return Name() + GetBitsPerKeySuffix();
+}
+
+BloomFilterPolicy::BloomFilterPolicy(double bits_per_key)
+ : BloomLikeFilterPolicy(bits_per_key) {}
+
+FilterBitsBuilder* BloomFilterPolicy::GetBuilderWithContext(
+ const FilterBuildingContext& context) const {
+ if (GetMillibitsPerKey() == 0) {
+ // "No filter" special case
+ return nullptr;
+ } else if (context.table_options.format_version < 5) {
+ return GetLegacyBloomBuilderWithContext(context);
+ } else {
+ return GetFastLocalBloomBuilderWithContext(context);
+ }
+}
+
+const char* BloomFilterPolicy::kClassName() { return "bloomfilter"; }
+const char* BloomFilterPolicy::kNickName() { return "rocksdb.BloomFilter"; }
+
+std::string BloomFilterPolicy::GetId() const {
+ // Including ":false" for better forward-compatibility with 6.29 and earlier
+ // which required a boolean `use_block_based_builder` parameter
+ return BloomLikeFilterPolicy::GetId() + ":false";
+}
+
+FilterBitsBuilder* BloomLikeFilterPolicy::GetFastLocalBloomBuilderWithContext(
+ const FilterBuildingContext& context) const {
+ bool offm = context.table_options.optimize_filters_for_memory;
+ const auto options_overrides_iter =
+ context.table_options.cache_usage_options.options_overrides.find(
+ CacheEntryRole::kFilterConstruction);
+ const auto filter_construction_charged =
+ options_overrides_iter !=
+ context.table_options.cache_usage_options.options_overrides.end()
+ ? options_overrides_iter->second.charged
+ : context.table_options.cache_usage_options.options.charged;
+
+ std::shared_ptr<CacheReservationManager> cache_res_mgr;
+ if (context.table_options.block_cache &&
+ filter_construction_charged ==
+ CacheEntryRoleOptions::Decision::kEnabled) {
+ cache_res_mgr = std::make_shared<
+ CacheReservationManagerImpl<CacheEntryRole::kFilterConstruction>>(
+ context.table_options.block_cache);
+ }
+ return new FastLocalBloomBitsBuilder(
+ millibits_per_key_, offm ? &aggregate_rounding_balance_ : nullptr,
+ cache_res_mgr, context.table_options.detect_filter_construct_corruption);
+}
+
+FilterBitsBuilder* BloomLikeFilterPolicy::GetLegacyBloomBuilderWithContext(
+ const FilterBuildingContext& context) const {
+ if (whole_bits_per_key_ >= 14 && context.info_log &&
+ !warned_.load(std::memory_order_relaxed)) {
+ warned_ = true;
+ const char* adjective;
+ if (whole_bits_per_key_ >= 20) {
+ adjective = "Dramatic";
+ } else {
+ adjective = "Significant";
+ }
+ // For more details, see
+ // https://github.com/facebook/rocksdb/wiki/RocksDB-Bloom-Filter
+ ROCKS_LOG_WARN(context.info_log,
+ "Using legacy Bloom filter with high (%d) bits/key. "
+ "%s filter space and/or accuracy improvement is available "
+ "with format_version>=5.",
+ whole_bits_per_key_, adjective);
+ }
+ return new LegacyBloomBitsBuilder(whole_bits_per_key_, context.info_log);
+}
+
+FilterBitsBuilder*
+BloomLikeFilterPolicy::GetStandard128RibbonBuilderWithContext(
+ const FilterBuildingContext& context) const {
+ // FIXME: code duplication with GetFastLocalBloomBuilderWithContext
+ bool offm = context.table_options.optimize_filters_for_memory;
+ const auto options_overrides_iter =
+ context.table_options.cache_usage_options.options_overrides.find(
+ CacheEntryRole::kFilterConstruction);
+ const auto filter_construction_charged =
+ options_overrides_iter !=
+ context.table_options.cache_usage_options.options_overrides.end()
+ ? options_overrides_iter->second.charged
+ : context.table_options.cache_usage_options.options.charged;
+
+ std::shared_ptr<CacheReservationManager> cache_res_mgr;
+ if (context.table_options.block_cache &&
+ filter_construction_charged ==
+ CacheEntryRoleOptions::Decision::kEnabled) {
+ cache_res_mgr = std::make_shared<
+ CacheReservationManagerImpl<CacheEntryRole::kFilterConstruction>>(
+ context.table_options.block_cache);
+ }
+ return new Standard128RibbonBitsBuilder(
+ desired_one_in_fp_rate_, millibits_per_key_,
+ offm ? &aggregate_rounding_balance_ : nullptr, cache_res_mgr,
+ context.table_options.detect_filter_construct_corruption,
+ context.info_log);
+}
+
+std::string BloomLikeFilterPolicy::GetBitsPerKeySuffix() const {
+ std::string rv = ":" + std::to_string(millibits_per_key_ / 1000);
+ int frac = millibits_per_key_ % 1000;
+ if (frac > 0) {
+ rv.push_back('.');
+ rv.push_back(static_cast<char>('0' + (frac / 100)));
+ frac %= 100;
+ if (frac > 0) {
+ rv.push_back(static_cast<char>('0' + (frac / 10)));
+ frac %= 10;
+ if (frac > 0) {
+ rv.push_back(static_cast<char>('0' + frac));
+ }
+ }
+ }
+ return rv;
+}
+
+FilterBitsBuilder* BuiltinFilterPolicy::GetBuilderFromContext(
+ const FilterBuildingContext& context) {
+ if (context.table_options.filter_policy) {
+ return context.table_options.filter_policy->GetBuilderWithContext(context);
+ } else {
+ return nullptr;
+ }
+}
+
+// For testing only, but always constructable with internal names
+namespace test {
+
+const char* LegacyBloomFilterPolicy::kClassName() {
+ return "rocksdb.internal.LegacyBloomFilter";
+}
+
+FilterBitsBuilder* LegacyBloomFilterPolicy::GetBuilderWithContext(
+ const FilterBuildingContext& context) const {
+ if (GetMillibitsPerKey() == 0) {
+ // "No filter" special case
+ return nullptr;
+ }
+ return GetLegacyBloomBuilderWithContext(context);
+}
+
+const char* FastLocalBloomFilterPolicy::kClassName() {
+ return "rocksdb.internal.FastLocalBloomFilter";
+}
+
+FilterBitsBuilder* FastLocalBloomFilterPolicy::GetBuilderWithContext(
+ const FilterBuildingContext& context) const {
+ if (GetMillibitsPerKey() == 0) {
+ // "No filter" special case
+ return nullptr;
+ }
+ return GetFastLocalBloomBuilderWithContext(context);
+}
+
+const char* Standard128RibbonFilterPolicy::kClassName() {
+ return "rocksdb.internal.Standard128RibbonFilter";
+}
+
+FilterBitsBuilder* Standard128RibbonFilterPolicy::GetBuilderWithContext(
+ const FilterBuildingContext& context) const {
+ if (GetMillibitsPerKey() == 0) {
+ // "No filter" special case
+ return nullptr;
+ }
+ return GetStandard128RibbonBuilderWithContext(context);
+}
+
+} // namespace test
+
+BuiltinFilterBitsReader* BuiltinFilterPolicy::GetBuiltinFilterBitsReader(
+ const Slice& contents) {
+ uint32_t len_with_meta = static_cast<uint32_t>(contents.size());
+ if (len_with_meta <= kMetadataLen) {
+ // filter is empty or broken. Treat like zero keys added.
+ return new AlwaysFalseFilter();
+ }
+
+ // Legacy Bloom filter data:
+ // 0 +-----------------------------------+
+ // | Raw Bloom filter data |
+ // | ... |
+ // len +-----------------------------------+
+ // | byte for num_probes or |
+ // | marker for new implementations |
+ // len+1 +-----------------------------------+
+ // | four bytes for number of cache |
+ // | lines |
+ // len_with_meta +-----------------------------------+
+
+ int8_t raw_num_probes =
+ static_cast<int8_t>(contents.data()[len_with_meta - kMetadataLen]);
+ // NB: *num_probes > 30 and < 128 probably have not been used, because of
+ // BloomFilterPolicy::initialize, unless directly calling
+ // LegacyBloomBitsBuilder as an API, but we are leaving those cases in
+ // limbo with LegacyBloomBitsReader for now.
+
+ if (raw_num_probes < 1) {
+ // Note: < 0 (or unsigned > 127) indicate special new implementations
+ // (or reserved for future use)
+ switch (raw_num_probes) {
+ case 0:
+ // Treat as zero probes (always FP)
+ return new AlwaysTrueFilter();
+ case -1:
+ // Marker for newer Bloom implementations
+ return GetBloomBitsReader(contents);
+ case -2:
+ // Marker for Ribbon implementations
+ return GetRibbonBitsReader(contents);
+ default:
+ // Reserved (treat as zero probes, always FP, for now)
+ return new AlwaysTrueFilter();
+ }
+ }
+ // else attempt decode for LegacyBloomBitsReader
+
+ int num_probes = raw_num_probes;
+ assert(num_probes >= 1);
+ assert(num_probes <= 127);
+
+ uint32_t len = len_with_meta - kMetadataLen;
+ assert(len > 0);
+
+ uint32_t num_lines = DecodeFixed32(contents.data() + len_with_meta - 4);
+ uint32_t log2_cache_line_size;
+
+ if (num_lines * CACHE_LINE_SIZE == len) {
+ // Common case
+ log2_cache_line_size = ConstexprFloorLog2(CACHE_LINE_SIZE);
+ } else if (num_lines == 0 || len % num_lines != 0) {
+ // Invalid (no solution to num_lines * x == len)
+ // Treat as zero probes (always FP) for now.
+ return new AlwaysTrueFilter();
+ } else {
+ // Determine the non-native cache line size (from another system)
+ log2_cache_line_size = 0;
+ while ((num_lines << log2_cache_line_size) < len) {
+ ++log2_cache_line_size;
+ }
+ if ((num_lines << log2_cache_line_size) != len) {
+ // Invalid (block size not a power of two)
+ // Treat as zero probes (always FP) for now.
+ return new AlwaysTrueFilter();
+ }
+ }
+ // if not early return
+ return new LegacyBloomBitsReader(contents.data(), num_probes, num_lines,
+ log2_cache_line_size);
+}
+
+// Read metadata to determine what kind of FilterBitsReader is needed
+// and return a new one.
+FilterBitsReader* BuiltinFilterPolicy::GetFilterBitsReader(
+ const Slice& contents) const {
+ return BuiltinFilterPolicy::GetBuiltinFilterBitsReader(contents);
+}
+
+BuiltinFilterBitsReader* BuiltinFilterPolicy::GetRibbonBitsReader(
+ const Slice& contents) {
+ uint32_t len_with_meta = static_cast<uint32_t>(contents.size());
+ uint32_t len = len_with_meta - kMetadataLen;
+
+ assert(len > 0); // precondition
+
+ uint32_t seed = static_cast<uint8_t>(contents.data()[len + 1]);
+ uint32_t num_blocks = static_cast<uint8_t>(contents.data()[len + 2]);
+ num_blocks |= static_cast<uint8_t>(contents.data()[len + 3]) << 8;
+ num_blocks |= static_cast<uint8_t>(contents.data()[len + 4]) << 16;
+ if (num_blocks < 2) {
+ // Not supported
+ // num_blocks == 1 is not used because num_starts == 1 is problematic
+ // for the hashing scheme. num_blocks == 0 is unused because there's
+ // already a concise encoding of an "always false" filter.
+ // Return something safe:
+ return new AlwaysTrueFilter();
+ }
+ return new Standard128RibbonBitsReader(contents.data(), len, num_blocks,
+ seed);
+}
+
+// For newer Bloom filter implementations
+BuiltinFilterBitsReader* BuiltinFilterPolicy::GetBloomBitsReader(
+ const Slice& contents) {
+ uint32_t len_with_meta = static_cast<uint32_t>(contents.size());
+ uint32_t len = len_with_meta - kMetadataLen;
+
+ assert(len > 0); // precondition
+
+ // New Bloom filter data:
+ // 0 +-----------------------------------+
+ // | Raw Bloom filter data |
+ // | ... |
+ // len +-----------------------------------+
+ // | char{-1} byte -> new Bloom filter |
+ // len+1 +-----------------------------------+
+ // | byte for subimplementation |
+ // | 0: FastLocalBloom |
+ // | other: reserved |
+ // len+2 +-----------------------------------+
+ // | byte for block_and_probes |
+ // | 0 in top 3 bits -> 6 -> 64-byte |
+ // | reserved: |
+ // | 1 in top 3 bits -> 7 -> 128-byte|
+ // | 2 in top 3 bits -> 8 -> 256-byte|
+ // | ... |
+ // | num_probes in bottom 5 bits, |
+ // | except 0 and 31 reserved |
+ // len+3 +-----------------------------------+
+ // | two bytes reserved |
+ // | possibly for hash seed |
+ // len_with_meta +-----------------------------------+
+
+ // Read more metadata (see above)
+ char sub_impl_val = contents.data()[len_with_meta - 4];
+ char block_and_probes = contents.data()[len_with_meta - 3];
+ int log2_block_bytes = ((block_and_probes >> 5) & 7) + 6;
+
+ int num_probes = (block_and_probes & 31);
+ if (num_probes < 1 || num_probes > 30) {
+ // Reserved / future safe
+ return new AlwaysTrueFilter();
+ }
+
+ uint16_t rest = DecodeFixed16(contents.data() + len_with_meta - 2);
+ if (rest != 0) {
+ // Reserved, possibly for hash seed
+ // Future safe
+ return new AlwaysTrueFilter();
+ }
+
+ if (sub_impl_val == 0) { // FastLocalBloom
+ if (log2_block_bytes == 6) { // Only block size supported for now
+ return new FastLocalBloomBitsReader(contents.data(), num_probes, len);
+ }
+ }
+ // otherwise
+ // Reserved / future safe
+ return new AlwaysTrueFilter();
+}
+
+const FilterPolicy* NewBloomFilterPolicy(double bits_per_key,
+ bool /*use_block_based_builder*/) {
+ // NOTE: use_block_based_builder now ignored so block-based filter is no
+ // longer accessible in public API.
+ return new BloomFilterPolicy(bits_per_key);
+}
+
+RibbonFilterPolicy::RibbonFilterPolicy(double bloom_equivalent_bits_per_key,
+ int bloom_before_level)
+ : BloomLikeFilterPolicy(bloom_equivalent_bits_per_key),
+ bloom_before_level_(bloom_before_level) {}
+
+FilterBitsBuilder* RibbonFilterPolicy::GetBuilderWithContext(
+ const FilterBuildingContext& context) const {
+ if (GetMillibitsPerKey() == 0) {
+ // "No filter" special case
+ return nullptr;
+ }
+ // Treat unknown same as bottommost
+ int levelish = INT_MAX;
+
+ switch (context.compaction_style) {
+ case kCompactionStyleLevel:
+ case kCompactionStyleUniversal: {
+ if (context.reason == TableFileCreationReason::kFlush) {
+ // Treat flush as level -1
+ assert(context.level_at_creation == 0);
+ levelish = -1;
+ } else if (context.level_at_creation == -1) {
+ // Unknown level
+ assert(levelish == INT_MAX);
+ } else {
+ levelish = context.level_at_creation;
+ }
+ break;
+ }
+ case kCompactionStyleFIFO:
+ case kCompactionStyleNone:
+ // Treat as bottommost
+ assert(levelish == INT_MAX);
+ break;
+ }
+ if (levelish < bloom_before_level_) {
+ return GetFastLocalBloomBuilderWithContext(context);
+ } else {
+ return GetStandard128RibbonBuilderWithContext(context);
+ }
+}
+
+const char* RibbonFilterPolicy::kClassName() { return "ribbonfilter"; }
+const char* RibbonFilterPolicy::kNickName() { return "rocksdb.RibbonFilter"; }
+
+std::string RibbonFilterPolicy::GetId() const {
+ return BloomLikeFilterPolicy::GetId() + ":" +
+ std::to_string(bloom_before_level_);
+}
+
+const FilterPolicy* NewRibbonFilterPolicy(double bloom_equivalent_bits_per_key,
+ int bloom_before_level) {
+ return new RibbonFilterPolicy(bloom_equivalent_bits_per_key,
+ bloom_before_level);
+}
+
+FilterBuildingContext::FilterBuildingContext(
+ const BlockBasedTableOptions& _table_options)
+ : table_options(_table_options) {}
+
+FilterPolicy::~FilterPolicy() {}
+
+std::shared_ptr<const FilterPolicy> BloomLikeFilterPolicy::Create(
+ const std::string& name, double bits_per_key) {
+ if (name == test::LegacyBloomFilterPolicy::kClassName()) {
+ return std::make_shared<test::LegacyBloomFilterPolicy>(bits_per_key);
+ } else if (name == test::FastLocalBloomFilterPolicy::kClassName()) {
+ return std::make_shared<test::FastLocalBloomFilterPolicy>(bits_per_key);
+ } else if (name == test::Standard128RibbonFilterPolicy::kClassName()) {
+ return std::make_shared<test::Standard128RibbonFilterPolicy>(bits_per_key);
+ } else if (name == BloomFilterPolicy::kClassName()) {
+ // For testing
+ return std::make_shared<BloomFilterPolicy>(bits_per_key);
+ } else if (name == RibbonFilterPolicy::kClassName()) {
+ // For testing
+ return std::make_shared<RibbonFilterPolicy>(bits_per_key,
+ /*bloom_before_level*/ 0);
+ } else {
+ return nullptr;
+ }
+}
+
+#ifndef ROCKSDB_LITE
+namespace {
+static ObjectLibrary::PatternEntry FilterPatternEntryWithBits(
+ const char* name) {
+ return ObjectLibrary::PatternEntry(name, false).AddNumber(":", false);
+}
+
+template <typename T>
+T* NewBuiltinFilterPolicyWithBits(const std::string& uri) {
+ const std::vector<std::string> vals = StringSplit(uri, ':');
+ double bits_per_key = ParseDouble(vals[1]);
+ return new T(bits_per_key);
+}
+static int RegisterBuiltinFilterPolicies(ObjectLibrary& library,
+ const std::string& /*arg*/) {
+ library.AddFactory<const FilterPolicy>(
+ ReadOnlyBuiltinFilterPolicy::kClassName(),
+ [](const std::string& /*uri*/, std::unique_ptr<const FilterPolicy>* guard,
+ std::string* /* errmsg */) {
+ guard->reset(new ReadOnlyBuiltinFilterPolicy());
+ return guard->get();
+ });
+
+ library.AddFactory<const FilterPolicy>(
+ FilterPatternEntryWithBits(BloomFilterPolicy::kClassName())
+ .AnotherName(BloomFilterPolicy::kNickName()),
+ [](const std::string& uri, std::unique_ptr<const FilterPolicy>* guard,
+ std::string* /* errmsg */) {
+ guard->reset(NewBuiltinFilterPolicyWithBits<BloomFilterPolicy>(uri));
+ return guard->get();
+ });
+ library.AddFactory<const FilterPolicy>(
+ FilterPatternEntryWithBits(BloomFilterPolicy::kClassName())
+ .AnotherName(BloomFilterPolicy::kNickName())
+ .AddSuffix(":false"),
+ [](const std::string& uri, std::unique_ptr<const FilterPolicy>* guard,
+ std::string* /* errmsg */) {
+ guard->reset(NewBuiltinFilterPolicyWithBits<BloomFilterPolicy>(uri));
+ return guard->get();
+ });
+ library.AddFactory<const FilterPolicy>(
+ FilterPatternEntryWithBits(BloomFilterPolicy::kClassName())
+ .AnotherName(BloomFilterPolicy::kNickName())
+ .AddSuffix(":true"),
+ [](const std::string& uri, std::unique_ptr<const FilterPolicy>* guard,
+ std::string* /* errmsg */) {
+ const std::vector<std::string> vals = StringSplit(uri, ':');
+ double bits_per_key = ParseDouble(vals[1]);
+ // NOTE: This case previously configured the deprecated block-based
+ // filter, but old ways of configuring that now map to full filter. We
+ // defer to the corresponding API to ensure consistency in case that
+ // change is reverted.
+ guard->reset(NewBloomFilterPolicy(bits_per_key, true));
+ return guard->get();
+ });
+ library.AddFactory<const FilterPolicy>(
+ FilterPatternEntryWithBits(RibbonFilterPolicy::kClassName())
+ .AnotherName(RibbonFilterPolicy::kNickName()),
+ [](const std::string& uri, std::unique_ptr<const FilterPolicy>* guard,
+ std::string* /* errmsg */) {
+ const std::vector<std::string> vals = StringSplit(uri, ':');
+ double bits_per_key = ParseDouble(vals[1]);
+ guard->reset(NewRibbonFilterPolicy(bits_per_key));
+ return guard->get();
+ });
+ library.AddFactory<const FilterPolicy>(
+ FilterPatternEntryWithBits(RibbonFilterPolicy::kClassName())
+ .AnotherName(RibbonFilterPolicy::kNickName())
+ .AddNumber(":", true),
+ [](const std::string& uri, std::unique_ptr<const FilterPolicy>* guard,
+ std::string* /* errmsg */) {
+ const std::vector<std::string> vals = StringSplit(uri, ':');
+ double bits_per_key = ParseDouble(vals[1]);
+ int bloom_before_level = ParseInt(vals[2]);
+ guard->reset(NewRibbonFilterPolicy(bits_per_key, bloom_before_level));
+ return guard->get();
+ });
+ library.AddFactory<const FilterPolicy>(
+ FilterPatternEntryWithBits(test::LegacyBloomFilterPolicy::kClassName()),
+ [](const std::string& uri, std::unique_ptr<const FilterPolicy>* guard,
+ std::string* /* errmsg */) {
+ guard->reset(
+ NewBuiltinFilterPolicyWithBits<test::LegacyBloomFilterPolicy>(uri));
+ return guard->get();
+ });
+ library.AddFactory<const FilterPolicy>(
+ FilterPatternEntryWithBits(
+ test::FastLocalBloomFilterPolicy::kClassName()),
+ [](const std::string& uri, std::unique_ptr<const FilterPolicy>* guard,
+ std::string* /* errmsg */) {
+ guard->reset(
+ NewBuiltinFilterPolicyWithBits<test::FastLocalBloomFilterPolicy>(
+ uri));
+ return guard->get();
+ });
+ library.AddFactory<const FilterPolicy>(
+ FilterPatternEntryWithBits(
+ test::Standard128RibbonFilterPolicy::kClassName()),
+ [](const std::string& uri, std::unique_ptr<const FilterPolicy>* guard,
+ std::string* /* errmsg */) {
+ guard->reset(
+ NewBuiltinFilterPolicyWithBits<test::Standard128RibbonFilterPolicy>(
+ uri));
+ return guard->get();
+ });
+ size_t num_types;
+ return static_cast<int>(library.GetFactoryCount(&num_types));
+}
+} // namespace
+#endif // ROCKSDB_LITE
+
+Status FilterPolicy::CreateFromString(
+ const ConfigOptions& options, const std::string& value,
+ std::shared_ptr<const FilterPolicy>* policy) {
+ if (value == kNullptrString || value.empty()) {
+ policy->reset();
+ return Status::OK();
+ } else if (value == ReadOnlyBuiltinFilterPolicy::kClassName()) {
+ *policy = std::make_shared<ReadOnlyBuiltinFilterPolicy>();
+ return Status::OK();
+ }
+
+ std::string id;
+ std::unordered_map<std::string, std::string> opt_map;
+ Status status =
+ Customizable::GetOptionsMap(options, policy->get(), value, &id, &opt_map);
+ if (!status.ok()) { // GetOptionsMap failed
+ return status;
+ } else if (id.empty()) { // We have no Id but have options. Not good
+ return Status::NotSupported("Cannot reset object ", id);
+ } else {
+#ifndef ROCKSDB_LITE
+ static std::once_flag loaded;
+ std::call_once(loaded, [&]() {
+ RegisterBuiltinFilterPolicies(*(ObjectLibrary::Default().get()), "");
+ });
+ status = options.registry->NewSharedObject(id, policy);
+#else
+ status =
+ Status::NotSupported("Cannot load filter policy in LITE mode ", value);
+#endif // ROCKSDB_LITE
+ }
+ if (options.ignore_unsupported_options && status.IsNotSupported()) {
+ return Status::OK();
+ } else if (status.ok()) {
+ status = Customizable::ConfigureNewObject(
+ options, const_cast<FilterPolicy*>(policy->get()), opt_map);
+ }
+ return status;
+}
+
+const std::vector<std::string>& BloomLikeFilterPolicy::GetAllFixedImpls() {
+ STATIC_AVOID_DESTRUCTION(std::vector<std::string>, impls){
+ // Match filter_bench -impl=x ordering
+ test::LegacyBloomFilterPolicy::kClassName(),
+ test::FastLocalBloomFilterPolicy::kClassName(),
+ test::Standard128RibbonFilterPolicy::kClassName(),
+ };
+ return impls;
+}
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/filter_policy_internal.h b/src/rocksdb/table/block_based/filter_policy_internal.h
new file mode 100644
index 000000000..9bc3a2482
--- /dev/null
+++ b/src/rocksdb/table/block_based/filter_policy_internal.h
@@ -0,0 +1,340 @@
+// Copyright (c) 2011-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).
+// Copyright (c) 2012 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#pragma once
+
+#include <atomic>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "rocksdb/filter_policy.h"
+#include "rocksdb/table.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+// A class that takes a bunch of keys, then generates filter
+class FilterBitsBuilder {
+ public:
+ virtual ~FilterBitsBuilder() {}
+
+ // Add a key (or prefix) to the filter. Typically, a builder will keep
+ // a set of 64-bit key hashes and only build the filter in Finish
+ // when the final number of keys is known. Keys are added in sorted order
+ // and duplicated keys are possible, so typically, the builder will
+ // only add this key if its hash is different from the most recently
+ // added.
+ virtual void AddKey(const Slice& key) = 0;
+
+ // Called by RocksDB before Finish to populate
+ // TableProperties::num_filter_entries, so should represent the
+ // number of unique keys (and/or prefixes) added, but does not have
+ // to be exact. `return 0;` may be used to conspicuously indicate "unknown".
+ virtual size_t EstimateEntriesAdded() = 0;
+
+ // Generate the filter using the keys that are added
+ // The return value of this function would be the filter bits,
+ // The ownership of actual data is set to buf
+ virtual Slice Finish(std::unique_ptr<const char[]>* buf) = 0;
+
+ // Similar to Finish(std::unique_ptr<const char[]>* buf), except that
+ // for a non-null status pointer argument, it will point to
+ // Status::Corruption() when there is any corruption during filter
+ // construction or Status::OK() otherwise.
+ //
+ // WARNING: do not use a filter resulted from a corrupted construction
+ // TODO: refactor this to have a better signature, consolidate
+ virtual Slice Finish(std::unique_ptr<const char[]>* buf,
+ Status* /* status */) {
+ return Finish(buf);
+ }
+
+ // Verify the filter returned from calling FilterBitsBuilder::Finish.
+ // The function returns Status::Corruption() if there is any corruption in the
+ // constructed filter or Status::OK() otherwise.
+ //
+ // Implementations should normally consult
+ // FilterBuildingContext::table_options.detect_filter_construct_corruption
+ // to determine whether to perform verification or to skip by returning
+ // Status::OK(). The decision is left to the FilterBitsBuilder so that
+ // verification prerequisites before PostVerify can be skipped when not
+ // configured.
+ //
+ // RocksDB internal will always call MaybePostVerify() on the filter after
+ // it is returned from calling FilterBitsBuilder::Finish
+ // except for FilterBitsBuilder::Finish resulting a corruption
+ // status, which indicates the filter is already in a corrupted state and
+ // there is no need to post-verify
+ virtual Status MaybePostVerify(const Slice& /* filter_content */) {
+ return Status::OK();
+ }
+
+ // Approximate the number of keys that can be added and generate a filter
+ // <= the specified number of bytes. Callers (including RocksDB) should
+ // only use this result for optimizing performance and not as a guarantee.
+ virtual size_t ApproximateNumEntries(size_t bytes) = 0;
+};
+
+// A class that checks if a key can be in filter
+// It should be initialized by Slice generated by BitsBuilder
+class FilterBitsReader {
+ public:
+ virtual ~FilterBitsReader() {}
+
+ // Check if the entry match the bits in filter
+ virtual bool MayMatch(const Slice& entry) = 0;
+
+ // Check if an array of entries match the bits in filter
+ virtual void MayMatch(int num_keys, Slice** keys, bool* may_match) {
+ for (int i = 0; i < num_keys; ++i) {
+ may_match[i] = MayMatch(*keys[i]);
+ }
+ }
+};
+
+// Exposes any extra information needed for testing built-in
+// FilterBitsBuilders
+class BuiltinFilterBitsBuilder : public FilterBitsBuilder {
+ public:
+ // Calculate number of bytes needed for a new filter, including
+ // metadata. Passing the result to ApproximateNumEntries should
+ // (ideally, usually) return >= the num_entry passed in.
+ // When optimize_filters_for_memory is enabled, this function
+ // is not authoritative but represents a target size that should
+ // be close to the average size.
+ virtual size_t CalculateSpace(size_t num_entries) = 0;
+
+ // Returns an estimate of the FP rate of the returned filter if
+ // `num_entries` keys are added and the filter returned by Finish
+ // is `bytes` bytes.
+ virtual double EstimatedFpRate(size_t num_entries, size_t bytes) = 0;
+};
+
+// Base class for RocksDB built-in filter reader with
+// extra useful functionalities for inernal.
+class BuiltinFilterBitsReader : public FilterBitsReader {
+ public:
+ // Check if the hash of the entry match the bits in filter
+ virtual bool HashMayMatch(const uint64_t /* h */) { return true; }
+};
+
+// Base class for RocksDB built-in filter policies. This provides the
+// ability to read all kinds of built-in filters (so that old filters can
+// be used even when you change between built-in policies).
+class BuiltinFilterPolicy : public FilterPolicy {
+ public: // overrides
+ // Read metadata to determine what kind of FilterBitsReader is needed
+ // and return a new one. This must successfully process any filter data
+ // generated by a built-in FilterBitsBuilder, regardless of the impl
+ // chosen for this BloomFilterPolicy.
+ FilterBitsReader* GetFilterBitsReader(const Slice& contents) const override;
+ static const char* kClassName();
+ bool IsInstanceOf(const std::string& id) const override;
+ // All variants of BuiltinFilterPolicy can read each others filters.
+ const char* CompatibilityName() const override;
+ static const char* kCompatibilityName();
+
+ public: // new
+ // An internal function for the implementation of
+ // BuiltinFilterBitsReader::GetFilterBitsReader without requiring an instance
+ // or working around potential virtual overrides.
+ static BuiltinFilterBitsReader* GetBuiltinFilterBitsReader(
+ const Slice& contents);
+
+ // Returns a new FilterBitsBuilder from the filter_policy in
+ // table_options of a context, or nullptr if not applicable.
+ // (An internal convenience function to save boilerplate.)
+ static FilterBitsBuilder* GetBuilderFromContext(const FilterBuildingContext&);
+
+ private:
+ // For Bloom filter implementation(s)
+ static BuiltinFilterBitsReader* GetBloomBitsReader(const Slice& contents);
+
+ // For Ribbon filter implementation(s)
+ static BuiltinFilterBitsReader* GetRibbonBitsReader(const Slice& contents);
+};
+
+// A "read only" filter policy used for backward compatibility with old
+// OPTIONS files, which did not specifying a Bloom configuration, just
+// "rocksdb.BuiltinBloomFilter". Although this can read existing filters,
+// this policy does not build new filters, so new SST files generated
+// under the policy will get no filters (like nullptr FilterPolicy).
+// This class is considered internal API and subject to change.
+class ReadOnlyBuiltinFilterPolicy : public BuiltinFilterPolicy {
+ public:
+ const char* Name() const override { return kClassName(); }
+ static const char* kClassName();
+
+ // Does not write filters.
+ FilterBitsBuilder* GetBuilderWithContext(
+ const FilterBuildingContext&) const override {
+ return nullptr;
+ }
+};
+
+// RocksDB built-in filter policy for Bloom or Bloom-like filters including
+// Ribbon filters.
+// This class is considered internal API and subject to change.
+// See NewBloomFilterPolicy and NewRibbonFilterPolicy.
+class BloomLikeFilterPolicy : public BuiltinFilterPolicy {
+ public:
+ explicit BloomLikeFilterPolicy(double bits_per_key);
+
+ ~BloomLikeFilterPolicy() override;
+ static const char* kClassName();
+ bool IsInstanceOf(const std::string& id) const override;
+
+ std::string GetId() const override;
+
+ // Essentially for testing only: configured millibits/key
+ int GetMillibitsPerKey() const { return millibits_per_key_; }
+ // Essentially for testing only: legacy whole bits/key
+ int GetWholeBitsPerKey() const { return whole_bits_per_key_; }
+
+ // All the different underlying implementations that a BloomLikeFilterPolicy
+ // might use, as a configuration string name for a testing mode for
+ // "always use this implementation." Only appropriate for unit tests.
+ static const std::vector<std::string>& GetAllFixedImpls();
+
+ // Convenience function for creating by name for fixed impls
+ static std::shared_ptr<const FilterPolicy> Create(const std::string& name,
+ double bits_per_key);
+
+ protected:
+ // Some implementations used by aggregating policies
+ FilterBitsBuilder* GetLegacyBloomBuilderWithContext(
+ const FilterBuildingContext& context) const;
+ FilterBitsBuilder* GetFastLocalBloomBuilderWithContext(
+ const FilterBuildingContext& context) const;
+ FilterBitsBuilder* GetStandard128RibbonBuilderWithContext(
+ const FilterBuildingContext& context) const;
+
+ std::string GetBitsPerKeySuffix() const;
+
+ private:
+ // Bits per key settings are for configuring Bloom filters.
+
+ // Newer filters support fractional bits per key. For predictable behavior
+ // of 0.001-precision values across floating point implementations, we
+ // round to thousandths of a bit (on average) per key.
+ int millibits_per_key_;
+
+ // Older filters round to whole number bits per key. (There *should* be no
+ // compatibility issue with fractional bits per key, but preserving old
+ // behavior with format_version < 5 just in case.)
+ int whole_bits_per_key_;
+
+ // For configuring Ribbon filter: a desired value for 1/fp_rate. For
+ // example, 100 -> 1% fp rate.
+ double desired_one_in_fp_rate_;
+
+ // Whether relevant warnings have been logged already. (Remember so we
+ // only report once per BloomFilterPolicy instance, to keep the noise down.)
+ mutable std::atomic<bool> warned_;
+
+ // State for implementing optimize_filters_for_memory. Essentially, this
+ // tracks a surplus or deficit in total FP rate of filters generated by
+ // builders under this policy vs. what would have been generated without
+ // optimize_filters_for_memory.
+ //
+ // To avoid floating point weirdness, the actual value is
+ // Sum over all generated filters f:
+ // (predicted_fp_rate(f) - predicted_fp_rate(f|o_f_f_m=false)) * 2^32
+ mutable std::atomic<int64_t> aggregate_rounding_balance_;
+};
+
+// For NewBloomFilterPolicy
+//
+// This is a user-facing policy that automatically choose between
+// LegacyBloom and FastLocalBloom based on context at build time,
+// including compatibility with format_version.
+class BloomFilterPolicy : public BloomLikeFilterPolicy {
+ public:
+ explicit BloomFilterPolicy(double bits_per_key);
+
+ // To use this function, call BuiltinFilterPolicy::GetBuilderFromContext().
+ //
+ // Neither the context nor any objects therein should be saved beyond
+ // the call to this function, unless it's shared_ptr.
+ FilterBitsBuilder* GetBuilderWithContext(
+ const FilterBuildingContext&) const override;
+
+ static const char* kClassName();
+ const char* Name() const override { return kClassName(); }
+ static const char* kNickName();
+ const char* NickName() const override { return kNickName(); }
+ std::string GetId() const override;
+};
+
+// For NewRibbonFilterPolicy
+//
+// This is a user-facing policy that chooses between Standard128Ribbon
+// and FastLocalBloom based on context at build time (LSM level and other
+// factors in extreme cases).
+class RibbonFilterPolicy : public BloomLikeFilterPolicy {
+ public:
+ explicit RibbonFilterPolicy(double bloom_equivalent_bits_per_key,
+ int bloom_before_level);
+
+ FilterBitsBuilder* GetBuilderWithContext(
+ const FilterBuildingContext&) const override;
+
+ int GetBloomBeforeLevel() const { return bloom_before_level_; }
+
+ static const char* kClassName();
+ const char* Name() const override { return kClassName(); }
+ static const char* kNickName();
+ const char* NickName() const override { return kNickName(); }
+ std::string GetId() const override;
+
+ private:
+ const int bloom_before_level_;
+};
+
+// For testing only, but always constructable with internal names
+namespace test {
+
+class LegacyBloomFilterPolicy : public BloomLikeFilterPolicy {
+ public:
+ explicit LegacyBloomFilterPolicy(double bits_per_key)
+ : BloomLikeFilterPolicy(bits_per_key) {}
+
+ FilterBitsBuilder* GetBuilderWithContext(
+ const FilterBuildingContext& context) const override;
+
+ static const char* kClassName();
+ const char* Name() const override { return kClassName(); }
+};
+
+class FastLocalBloomFilterPolicy : public BloomLikeFilterPolicy {
+ public:
+ explicit FastLocalBloomFilterPolicy(double bits_per_key)
+ : BloomLikeFilterPolicy(bits_per_key) {}
+
+ FilterBitsBuilder* GetBuilderWithContext(
+ const FilterBuildingContext& context) const override;
+
+ static const char* kClassName();
+ const char* Name() const override { return kClassName(); }
+};
+
+class Standard128RibbonFilterPolicy : public BloomLikeFilterPolicy {
+ public:
+ explicit Standard128RibbonFilterPolicy(double bloom_equiv_bits_per_key)
+ : BloomLikeFilterPolicy(bloom_equiv_bits_per_key) {}
+
+ FilterBitsBuilder* GetBuilderWithContext(
+ const FilterBuildingContext& context) const override;
+
+ static const char* kClassName();
+ const char* Name() const override { return kClassName(); }
+};
+
+} // namespace test
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/flush_block_policy.cc b/src/rocksdb/table/block_based/flush_block_policy.cc
new file mode 100644
index 000000000..9bb1f334b
--- /dev/null
+++ b/src/rocksdb/table/block_based/flush_block_policy.cc
@@ -0,0 +1,146 @@
+// Copyright (c) 2011-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 "rocksdb/flush_block_policy.h"
+
+#include <cassert>
+#include <mutex>
+
+#include "rocksdb/options.h"
+#include "rocksdb/slice.h"
+#include "rocksdb/utilities/customizable_util.h"
+#include "table/block_based/block_based_table_reader.h"
+#include "table/block_based/block_builder.h"
+#include "table/block_based/flush_block_policy.h"
+#include "table/format.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+// Flush block by size
+class FlushBlockBySizePolicy : public FlushBlockPolicy {
+ public:
+ // @params block_size: Approximate size of user data packed per
+ // block.
+ // @params block_size_deviation: This is used to close a block before it
+ // reaches the configured
+ FlushBlockBySizePolicy(const uint64_t block_size,
+ const uint64_t block_size_deviation, const bool align,
+ const BlockBuilder& data_block_builder)
+ : block_size_(block_size),
+ block_size_deviation_limit_(
+ ((block_size * (100 - block_size_deviation)) + 99) / 100),
+ align_(align),
+ data_block_builder_(data_block_builder) {}
+
+ bool Update(const Slice& key, const Slice& value) override {
+ // it makes no sense to flush when the data block is empty
+ if (data_block_builder_.empty()) {
+ return false;
+ }
+
+ auto curr_size = data_block_builder_.CurrentSizeEstimate();
+
+ // Do flush if one of the below two conditions is true:
+ // 1) if the current estimated size already exceeds the block size,
+ // 2) block_size_deviation is set and the estimated size after appending
+ // the kv will exceed the block size and the current size is under the
+ // the deviation.
+ return curr_size >= block_size_ || BlockAlmostFull(key, value);
+ }
+
+ private:
+ bool BlockAlmostFull(const Slice& key, const Slice& value) const {
+ if (block_size_deviation_limit_ == 0) {
+ return false;
+ }
+
+ const auto curr_size = data_block_builder_.CurrentSizeEstimate();
+ auto estimated_size_after =
+ data_block_builder_.EstimateSizeAfterKV(key, value);
+
+ if (align_) {
+ estimated_size_after += BlockBasedTable::kBlockTrailerSize;
+ return estimated_size_after > block_size_;
+ }
+
+ return estimated_size_after > block_size_ &&
+ curr_size > block_size_deviation_limit_;
+ }
+
+ const uint64_t block_size_;
+ const uint64_t block_size_deviation_limit_;
+ const bool align_;
+ const BlockBuilder& data_block_builder_;
+};
+
+FlushBlockPolicy* FlushBlockBySizePolicyFactory::NewFlushBlockPolicy(
+ const BlockBasedTableOptions& table_options,
+ const BlockBuilder& data_block_builder) const {
+ return new FlushBlockBySizePolicy(
+ table_options.block_size, table_options.block_size_deviation,
+ table_options.block_align, data_block_builder);
+}
+
+FlushBlockPolicy* FlushBlockBySizePolicyFactory::NewFlushBlockPolicy(
+ const uint64_t size, const int deviation,
+ const BlockBuilder& data_block_builder) {
+ return new FlushBlockBySizePolicy(size, deviation, false, data_block_builder);
+}
+
+#ifndef ROCKSDB_LITE
+static int RegisterFlushBlockPolicyFactories(ObjectLibrary& library,
+ const std::string& /*arg*/) {
+ library.AddFactory<FlushBlockPolicyFactory>(
+ FlushBlockBySizePolicyFactory::kClassName(),
+ [](const std::string& /*uri*/,
+ std::unique_ptr<FlushBlockPolicyFactory>* guard,
+ std::string* /* errmsg */) {
+ guard->reset(new FlushBlockBySizePolicyFactory());
+ return guard->get();
+ });
+ library.AddFactory<FlushBlockPolicyFactory>(
+ FlushBlockEveryKeyPolicyFactory::kClassName(),
+ [](const std::string& /*uri*/,
+ std::unique_ptr<FlushBlockPolicyFactory>* guard,
+ std::string* /* errmsg */) {
+ guard->reset(new FlushBlockEveryKeyPolicyFactory());
+ return guard->get();
+ });
+ return 2;
+}
+#endif // ROCKSDB_LITE
+
+static bool LoadFlushPolicyFactory(
+ const std::string& id, std::shared_ptr<FlushBlockPolicyFactory>* result) {
+ if (id.empty()) {
+ result->reset(new FlushBlockBySizePolicyFactory());
+#ifdef ROCKSDB_LITE
+ } else if (id == FlushBlockBySizePolicyFactory::kClassName()) {
+ result->reset(new FlushBlockBySizePolicyFactory());
+ } else if (id == FlushBlockEveryKeyPolicyFactory::kClassName()) {
+ result->reset(new FlushBlockEveryKeyPolicyFactory());
+#endif // ROCKSDB_LITE
+ } else {
+ return false;
+ }
+ return true;
+}
+
+FlushBlockBySizePolicyFactory::FlushBlockBySizePolicyFactory()
+ : FlushBlockPolicyFactory() {}
+
+Status FlushBlockPolicyFactory::CreateFromString(
+ const ConfigOptions& config_options, const std::string& value,
+ std::shared_ptr<FlushBlockPolicyFactory>* factory) {
+#ifndef ROCKSDB_LITE
+ static std::once_flag once;
+ std::call_once(once, [&]() {
+ RegisterFlushBlockPolicyFactories(*(ObjectLibrary::Default().get()), "");
+ });
+#endif // ROCKSDB_LITE
+ return LoadSharedObject<FlushBlockPolicyFactory>(
+ config_options, value, LoadFlushPolicyFactory, factory);
+}
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/flush_block_policy.h b/src/rocksdb/table/block_based/flush_block_policy.h
new file mode 100644
index 000000000..4f79682bc
--- /dev/null
+++ b/src/rocksdb/table/block_based/flush_block_policy.h
@@ -0,0 +1,40 @@
+// Copyright (c) 2011-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 "rocksdb/flush_block_policy.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+// FlushBlockEveryKeyPolicy currently used only in tests.
+
+class FlushBlockEveryKeyPolicy : public FlushBlockPolicy {
+ public:
+ bool Update(const Slice& /*key*/, const Slice& /*value*/) override {
+ if (!start_) {
+ start_ = true;
+ return false;
+ }
+ return true;
+ }
+
+ private:
+ bool start_ = false;
+};
+
+class FlushBlockEveryKeyPolicyFactory : public FlushBlockPolicyFactory {
+ public:
+ explicit FlushBlockEveryKeyPolicyFactory() {}
+
+ static const char* kClassName() { return "FlushBlockEveryKeyPolicyFactory"; }
+ const char* Name() const override { return kClassName(); }
+
+ FlushBlockPolicy* NewFlushBlockPolicy(
+ const BlockBasedTableOptions& /*table_options*/,
+ const BlockBuilder& /*data_block_builder*/) const override {
+ return new FlushBlockEveryKeyPolicy;
+ }
+};
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/full_filter_block.cc b/src/rocksdb/table/block_based/full_filter_block.cc
new file mode 100644
index 000000000..62b7a9eca
--- /dev/null
+++ b/src/rocksdb/table/block_based/full_filter_block.cc
@@ -0,0 +1,296 @@
+// Copyright (c) 2011-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 "table/block_based/full_filter_block.h"
+
+#include <array>
+
+#include "block_type.h"
+#include "monitoring/perf_context_imp.h"
+#include "port/malloc.h"
+#include "port/port.h"
+#include "rocksdb/filter_policy.h"
+#include "table/block_based/block_based_table_reader.h"
+#include "util/coding.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+FullFilterBlockBuilder::FullFilterBlockBuilder(
+ const SliceTransform* _prefix_extractor, bool whole_key_filtering,
+ FilterBitsBuilder* filter_bits_builder)
+ : prefix_extractor_(_prefix_extractor),
+ whole_key_filtering_(whole_key_filtering),
+ last_whole_key_recorded_(false),
+ last_prefix_recorded_(false),
+ last_key_in_domain_(false),
+ any_added_(false) {
+ assert(filter_bits_builder != nullptr);
+ filter_bits_builder_.reset(filter_bits_builder);
+}
+
+size_t FullFilterBlockBuilder::EstimateEntriesAdded() {
+ return filter_bits_builder_->EstimateEntriesAdded();
+}
+
+void FullFilterBlockBuilder::Add(const Slice& key_without_ts) {
+ const bool add_prefix =
+ prefix_extractor_ && prefix_extractor_->InDomain(key_without_ts);
+
+ if (!last_prefix_recorded_ && last_key_in_domain_) {
+ // We can reach here when a new filter partition starts in partitioned
+ // filter. The last prefix in the previous partition should be added if
+ // necessary regardless of key_without_ts, to support prefix SeekForPrev.
+ AddKey(last_prefix_str_);
+ last_prefix_recorded_ = true;
+ }
+
+ if (whole_key_filtering_) {
+ if (!add_prefix) {
+ AddKey(key_without_ts);
+ } else {
+ // if both whole_key and prefix are added to bloom then we will have whole
+ // key_without_ts and prefix addition being interleaved and thus cannot
+ // rely on the bits builder to properly detect the duplicates by comparing
+ // with the last item.
+ Slice last_whole_key = Slice(last_whole_key_str_);
+ if (!last_whole_key_recorded_ ||
+ last_whole_key.compare(key_without_ts) != 0) {
+ AddKey(key_without_ts);
+ last_whole_key_recorded_ = true;
+ last_whole_key_str_.assign(key_without_ts.data(),
+ key_without_ts.size());
+ }
+ }
+ }
+ if (add_prefix) {
+ last_key_in_domain_ = true;
+ AddPrefix(key_without_ts);
+ } else {
+ last_key_in_domain_ = false;
+ }
+}
+
+// Add key to filter if needed
+inline void FullFilterBlockBuilder::AddKey(const Slice& key) {
+ filter_bits_builder_->AddKey(key);
+ any_added_ = true;
+}
+
+// Add prefix to filter if needed
+void FullFilterBlockBuilder::AddPrefix(const Slice& key) {
+ assert(prefix_extractor_ && prefix_extractor_->InDomain(key));
+ Slice prefix = prefix_extractor_->Transform(key);
+ if (whole_key_filtering_) {
+ // if both whole_key and prefix are added to bloom then we will have whole
+ // key and prefix addition being interleaved and thus cannot rely on the
+ // bits builder to properly detect the duplicates by comparing with the last
+ // item.
+ Slice last_prefix = Slice(last_prefix_str_);
+ if (!last_prefix_recorded_ || last_prefix.compare(prefix) != 0) {
+ AddKey(prefix);
+ last_prefix_recorded_ = true;
+ last_prefix_str_.assign(prefix.data(), prefix.size());
+ }
+ } else {
+ AddKey(prefix);
+ }
+}
+
+void FullFilterBlockBuilder::Reset() {
+ last_whole_key_recorded_ = false;
+ last_prefix_recorded_ = false;
+}
+
+Slice FullFilterBlockBuilder::Finish(
+ const BlockHandle& /*tmp*/, Status* status,
+ std::unique_ptr<const char[]>* filter_data) {
+ Reset();
+ // In this impl we ignore BlockHandle
+ *status = Status::OK();
+ if (any_added_) {
+ any_added_ = false;
+ Slice filter_content = filter_bits_builder_->Finish(
+ filter_data ? filter_data : &filter_data_, status);
+ return filter_content;
+ }
+ return Slice();
+}
+
+FullFilterBlockReader::FullFilterBlockReader(
+ const BlockBasedTable* t,
+ CachableEntry<ParsedFullFilterBlock>&& filter_block)
+ : FilterBlockReaderCommon(t, std::move(filter_block)) {}
+
+bool FullFilterBlockReader::KeyMayMatch(const Slice& key, const bool no_io,
+ const Slice* const /*const_ikey_ptr*/,
+ GetContext* get_context,
+ BlockCacheLookupContext* lookup_context,
+ Env::IOPriority rate_limiter_priority) {
+ if (!whole_key_filtering()) {
+ return true;
+ }
+ return MayMatch(key, no_io, get_context, lookup_context,
+ rate_limiter_priority);
+}
+
+std::unique_ptr<FilterBlockReader> FullFilterBlockReader::Create(
+ const BlockBasedTable* table, const ReadOptions& ro,
+ FilePrefetchBuffer* prefetch_buffer, bool use_cache, bool prefetch,
+ bool pin, BlockCacheLookupContext* lookup_context) {
+ assert(table);
+ assert(table->get_rep());
+ assert(!pin || prefetch);
+
+ CachableEntry<ParsedFullFilterBlock> filter_block;
+ if (prefetch || !use_cache) {
+ const Status s = ReadFilterBlock(table, prefetch_buffer, ro, use_cache,
+ nullptr /* get_context */, lookup_context,
+ &filter_block, BlockType::kFilter);
+ if (!s.ok()) {
+ IGNORE_STATUS_IF_ERROR(s);
+ return std::unique_ptr<FilterBlockReader>();
+ }
+
+ if (use_cache && !pin) {
+ filter_block.Reset();
+ }
+ }
+
+ return std::unique_ptr<FilterBlockReader>(
+ new FullFilterBlockReader(table, std::move(filter_block)));
+}
+
+bool FullFilterBlockReader::PrefixMayMatch(
+ const Slice& prefix, const bool no_io,
+ const Slice* const /*const_ikey_ptr*/, GetContext* get_context,
+ BlockCacheLookupContext* lookup_context,
+ Env::IOPriority rate_limiter_priority) {
+ return MayMatch(prefix, no_io, get_context, lookup_context,
+ rate_limiter_priority);
+}
+
+bool FullFilterBlockReader::MayMatch(
+ const Slice& entry, bool no_io, GetContext* get_context,
+ BlockCacheLookupContext* lookup_context,
+ Env::IOPriority rate_limiter_priority) const {
+ CachableEntry<ParsedFullFilterBlock> filter_block;
+
+ const Status s =
+ GetOrReadFilterBlock(no_io, get_context, lookup_context, &filter_block,
+ BlockType::kFilter, rate_limiter_priority);
+ if (!s.ok()) {
+ IGNORE_STATUS_IF_ERROR(s);
+ return true;
+ }
+
+ assert(filter_block.GetValue());
+
+ FilterBitsReader* const filter_bits_reader =
+ filter_block.GetValue()->filter_bits_reader();
+
+ if (filter_bits_reader) {
+ if (filter_bits_reader->MayMatch(entry)) {
+ PERF_COUNTER_ADD(bloom_sst_hit_count, 1);
+ return true;
+ } else {
+ PERF_COUNTER_ADD(bloom_sst_miss_count, 1);
+ return false;
+ }
+ }
+ return true;
+}
+
+void FullFilterBlockReader::KeysMayMatch(
+ MultiGetRange* range, const bool no_io,
+ BlockCacheLookupContext* lookup_context,
+ Env::IOPriority rate_limiter_priority) {
+ if (!whole_key_filtering()) {
+ // Simply return. Don't skip any key - consider all keys as likely to be
+ // present
+ return;
+ }
+ MayMatch(range, no_io, nullptr, lookup_context, rate_limiter_priority);
+}
+
+void FullFilterBlockReader::PrefixesMayMatch(
+ MultiGetRange* range, const SliceTransform* prefix_extractor,
+ const bool no_io, BlockCacheLookupContext* lookup_context,
+ Env::IOPriority rate_limiter_priority) {
+ MayMatch(range, no_io, prefix_extractor, lookup_context,
+ rate_limiter_priority);
+}
+
+void FullFilterBlockReader::MayMatch(
+ MultiGetRange* range, bool no_io, const SliceTransform* prefix_extractor,
+ BlockCacheLookupContext* lookup_context,
+ Env::IOPriority rate_limiter_priority) const {
+ CachableEntry<ParsedFullFilterBlock> filter_block;
+
+ const Status s = GetOrReadFilterBlock(
+ no_io, range->begin()->get_context, lookup_context, &filter_block,
+ BlockType::kFilter, rate_limiter_priority);
+ if (!s.ok()) {
+ IGNORE_STATUS_IF_ERROR(s);
+ return;
+ }
+
+ assert(filter_block.GetValue());
+
+ FilterBitsReader* const filter_bits_reader =
+ filter_block.GetValue()->filter_bits_reader();
+
+ if (!filter_bits_reader) {
+ return;
+ }
+
+ // We need to use an array instead of autovector for may_match since
+ // &may_match[0] doesn't work for autovector<bool> (compiler error). So
+ // declare both keys and may_match as arrays, which is also slightly less
+ // expensive compared to autovector
+ std::array<Slice*, MultiGetContext::MAX_BATCH_SIZE> keys;
+ std::array<bool, MultiGetContext::MAX_BATCH_SIZE> may_match = {{true}};
+ autovector<Slice, MultiGetContext::MAX_BATCH_SIZE> prefixes;
+ int num_keys = 0;
+ MultiGetRange filter_range(*range, range->begin(), range->end());
+ for (auto iter = filter_range.begin(); iter != filter_range.end(); ++iter) {
+ if (!prefix_extractor) {
+ keys[num_keys++] = &iter->ukey_without_ts;
+ } else if (prefix_extractor->InDomain(iter->ukey_without_ts)) {
+ prefixes.emplace_back(prefix_extractor->Transform(iter->ukey_without_ts));
+ keys[num_keys++] = &prefixes.back();
+ } else {
+ filter_range.SkipKey(iter);
+ }
+ }
+
+ filter_bits_reader->MayMatch(num_keys, &keys[0], &may_match[0]);
+
+ int i = 0;
+ for (auto iter = filter_range.begin(); iter != filter_range.end(); ++iter) {
+ if (!may_match[i]) {
+ // Update original MultiGet range to skip this key. The filter_range
+ // was temporarily used just to skip keys not in prefix_extractor domain
+ range->SkipKey(iter);
+ PERF_COUNTER_ADD(bloom_sst_miss_count, 1);
+ } else {
+ // PERF_COUNTER_ADD(bloom_sst_hit_count, 1);
+ PerfContext* perf_ctx = get_perf_context();
+ perf_ctx->bloom_sst_hit_count++;
+ }
+ ++i;
+ }
+}
+
+size_t FullFilterBlockReader::ApproximateMemoryUsage() const {
+ size_t usage = ApproximateFilterBlockMemoryUsage();
+#ifdef ROCKSDB_MALLOC_USABLE_SIZE
+ usage += malloc_usable_size(const_cast<FullFilterBlockReader*>(this));
+#else
+ usage += sizeof(*this);
+#endif // ROCKSDB_MALLOC_USABLE_SIZE
+ return usage;
+}
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/full_filter_block.h b/src/rocksdb/table/block_based/full_filter_block.h
new file mode 100644
index 000000000..cd1771a38
--- /dev/null
+++ b/src/rocksdb/table/block_based/full_filter_block.h
@@ -0,0 +1,147 @@
+// Copyright (c) 2011-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).
+
+#pragma once
+
+#include <stddef.h>
+#include <stdint.h>
+
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "rocksdb/options.h"
+#include "rocksdb/slice.h"
+#include "rocksdb/slice_transform.h"
+#include "table/block_based/filter_block_reader_common.h"
+#include "table/block_based/filter_policy_internal.h"
+#include "table/block_based/parsed_full_filter_block.h"
+#include "util/hash.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class FilterPolicy;
+class FilterBitsBuilder;
+class FilterBitsReader;
+
+// A FullFilterBlockBuilder is used to construct a full filter for a
+// particular Table. It generates a single string which is stored as
+// a special block in the Table.
+// The format of full filter block is:
+// +----------------------------------------------------------------+
+// | full filter for all keys in sst file |
+// +----------------------------------------------------------------+
+// The full filter can be very large. At the end of it, we put
+// num_probes: how many hash functions are used in bloom filter
+//
+class FullFilterBlockBuilder : public FilterBlockBuilder {
+ public:
+ explicit FullFilterBlockBuilder(const SliceTransform* prefix_extractor,
+ bool whole_key_filtering,
+ FilterBitsBuilder* filter_bits_builder);
+ // No copying allowed
+ FullFilterBlockBuilder(const FullFilterBlockBuilder&) = delete;
+ void operator=(const FullFilterBlockBuilder&) = delete;
+
+ // bits_builder is created in filter_policy, it should be passed in here
+ // directly. and be deleted here
+ ~FullFilterBlockBuilder() {}
+
+ virtual void Add(const Slice& key_without_ts) override;
+ virtual bool IsEmpty() const override { return !any_added_; }
+ virtual size_t EstimateEntriesAdded() override;
+ virtual Slice Finish(
+ const BlockHandle& tmp, Status* status,
+ std::unique_ptr<const char[]>* filter_data = nullptr) override;
+ using FilterBlockBuilder::Finish;
+
+ virtual void ResetFilterBitsBuilder() override {
+ filter_bits_builder_.reset();
+ }
+
+ virtual Status MaybePostVerifyFilter(const Slice& filter_content) override {
+ return filter_bits_builder_->MaybePostVerify(filter_content);
+ }
+
+ protected:
+ virtual void AddKey(const Slice& key);
+ std::unique_ptr<FilterBitsBuilder> filter_bits_builder_;
+ virtual void Reset();
+ void AddPrefix(const Slice& key);
+ const SliceTransform* prefix_extractor() { return prefix_extractor_; }
+ const std::string& last_prefix_str() const { return last_prefix_str_; }
+
+ private:
+ // important: all of these might point to invalid addresses
+ // at the time of destruction of this filter block. destructor
+ // should NOT dereference them.
+ const SliceTransform* prefix_extractor_;
+ bool whole_key_filtering_;
+ bool last_whole_key_recorded_;
+ std::string last_whole_key_str_;
+ bool last_prefix_recorded_;
+ std::string last_prefix_str_;
+ // Whether prefix_extractor_->InDomain(last_whole_key_) is true.
+ // Used in partitioned filters so that the last prefix from the previous
+ // filter partition will be added to the current partition if
+ // last_key_in_domain_ is true, regardless of the current key.
+ bool last_key_in_domain_;
+ bool any_added_;
+ std::unique_ptr<const char[]> filter_data_;
+};
+
+// A FilterBlockReader is used to parse filter from SST table.
+// KeyMayMatch and PrefixMayMatch would trigger filter checking
+class FullFilterBlockReader
+ : public FilterBlockReaderCommon<ParsedFullFilterBlock> {
+ public:
+ FullFilterBlockReader(const BlockBasedTable* t,
+ CachableEntry<ParsedFullFilterBlock>&& filter_block);
+
+ static std::unique_ptr<FilterBlockReader> Create(
+ const BlockBasedTable* table, const ReadOptions& ro,
+ FilePrefetchBuffer* prefetch_buffer, bool use_cache, bool prefetch,
+ bool pin, BlockCacheLookupContext* lookup_context);
+
+ bool KeyMayMatch(const Slice& key, const bool no_io,
+ const Slice* const const_ikey_ptr, GetContext* get_context,
+ BlockCacheLookupContext* lookup_context,
+ Env::IOPriority rate_limiter_priority) override;
+
+ bool PrefixMayMatch(const Slice& prefix, const bool no_io,
+ const Slice* const const_ikey_ptr,
+ GetContext* get_context,
+ BlockCacheLookupContext* lookup_context,
+ Env::IOPriority rate_limiter_priority) override;
+
+ void KeysMayMatch(MultiGetRange* range, const bool no_io,
+ BlockCacheLookupContext* lookup_context,
+ Env::IOPriority rate_limiter_priority) override;
+ // Used in partitioned filter code
+ void KeysMayMatch2(MultiGetRange* range,
+ const SliceTransform* /*prefix_extractor*/,
+ const bool no_io, BlockCacheLookupContext* lookup_context,
+ Env::IOPriority rate_limiter_priority) {
+ KeysMayMatch(range, no_io, lookup_context, rate_limiter_priority);
+ }
+
+ void PrefixesMayMatch(MultiGetRange* range,
+ const SliceTransform* prefix_extractor,
+ const bool no_io,
+ BlockCacheLookupContext* lookup_context,
+ Env::IOPriority rate_limiter_priority) override;
+ size_t ApproximateMemoryUsage() const override;
+
+ private:
+ bool MayMatch(const Slice& entry, bool no_io, GetContext* get_context,
+ BlockCacheLookupContext* lookup_context,
+ Env::IOPriority rate_limiter_priority) const;
+ void MayMatch(MultiGetRange* range, bool no_io,
+ const SliceTransform* prefix_extractor,
+ BlockCacheLookupContext* lookup_context,
+ Env::IOPriority rate_limiter_priority) const;
+};
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/full_filter_block_test.cc b/src/rocksdb/table/block_based/full_filter_block_test.cc
new file mode 100644
index 000000000..bd98638e5
--- /dev/null
+++ b/src/rocksdb/table/block_based/full_filter_block_test.cc
@@ -0,0 +1,339 @@
+// Copyright (c) 2011-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 "table/block_based/full_filter_block.h"
+
+#include <set>
+
+#include "rocksdb/filter_policy.h"
+#include "rocksdb/status.h"
+#include "table/block_based/block_based_table_reader.h"
+#include "table/block_based/filter_policy_internal.h"
+#include "table/block_based/mock_block_based_table.h"
+#include "table/format.h"
+#include "test_util/testharness.h"
+#include "test_util/testutil.h"
+#include "util/coding.h"
+#include "util/hash.h"
+#include "util/string_util.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class TestFilterBitsBuilder : public FilterBitsBuilder {
+ public:
+ explicit TestFilterBitsBuilder() {}
+
+ // Add Key to filter
+ void AddKey(const Slice& key) override {
+ hash_entries_.push_back(Hash(key.data(), key.size(), 1));
+ }
+
+ using FilterBitsBuilder::Finish;
+
+ // Generate the filter using the keys that are added
+ Slice Finish(std::unique_ptr<const char[]>* buf) override {
+ uint32_t len = static_cast<uint32_t>(hash_entries_.size()) * 4;
+ char* data = new char[len];
+ for (size_t i = 0; i < hash_entries_.size(); i++) {
+ EncodeFixed32(data + i * 4, hash_entries_[i]);
+ }
+ const char* const_data = data;
+ buf->reset(const_data);
+ return Slice(data, len);
+ }
+
+ size_t EstimateEntriesAdded() override { return hash_entries_.size(); }
+
+ size_t ApproximateNumEntries(size_t bytes) override { return bytes / 4; }
+
+ private:
+ std::vector<uint32_t> hash_entries_;
+};
+
+class MockBlockBasedTable : public BlockBasedTable {
+ public:
+ explicit MockBlockBasedTable(Rep* rep)
+ : BlockBasedTable(rep, nullptr /* block_cache_tracer */) {}
+};
+
+class TestFilterBitsReader : public FilterBitsReader {
+ public:
+ explicit TestFilterBitsReader(const Slice& contents)
+ : data_(contents.data()), len_(static_cast<uint32_t>(contents.size())) {}
+
+ // Silence compiler warning about overloaded virtual
+ using FilterBitsReader::MayMatch;
+ bool MayMatch(const Slice& entry) override {
+ uint32_t h = Hash(entry.data(), entry.size(), 1);
+ for (size_t i = 0; i + 4 <= len_; i += 4) {
+ if (h == DecodeFixed32(data_ + i)) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ private:
+ const char* data_;
+ uint32_t len_;
+};
+
+class TestHashFilter : public FilterPolicy {
+ public:
+ const char* Name() const override { return "TestHashFilter"; }
+ const char* CompatibilityName() const override { return Name(); }
+
+ FilterBitsBuilder* GetBuilderWithContext(
+ const FilterBuildingContext&) const override {
+ return new TestFilterBitsBuilder();
+ }
+
+ FilterBitsReader* GetFilterBitsReader(const Slice& contents) const override {
+ return new TestFilterBitsReader(contents);
+ }
+};
+
+class PluginFullFilterBlockTest : public mock::MockBlockBasedTableTester,
+ public testing::Test {
+ public:
+ PluginFullFilterBlockTest()
+ : mock::MockBlockBasedTableTester(new TestHashFilter) {}
+};
+
+TEST_F(PluginFullFilterBlockTest, PluginEmptyBuilder) {
+ FullFilterBlockBuilder builder(nullptr, true, GetBuilder());
+ Slice slice = builder.Finish();
+ ASSERT_EQ("", EscapeString(slice));
+
+ CachableEntry<ParsedFullFilterBlock> block(
+ new ParsedFullFilterBlock(table_options_.filter_policy.get(),
+ BlockContents(slice)),
+ nullptr /* cache */, nullptr /* cache_handle */, true /* own_value */);
+
+ FullFilterBlockReader reader(table_.get(), std::move(block));
+ // Remain same symantic with blockbased filter
+ ASSERT_TRUE(reader.KeyMayMatch("foo",
+ /*no_io=*/false, /*const_ikey_ptr=*/nullptr,
+ /*get_context=*/nullptr,
+ /*lookup_context=*/nullptr, Env::IO_TOTAL));
+}
+
+TEST_F(PluginFullFilterBlockTest, PluginSingleChunk) {
+ FullFilterBlockBuilder builder(nullptr, true, GetBuilder());
+ builder.Add("foo");
+ builder.Add("bar");
+ builder.Add("box");
+ builder.Add("box");
+ builder.Add("hello");
+ Slice slice = builder.Finish();
+
+ CachableEntry<ParsedFullFilterBlock> block(
+ new ParsedFullFilterBlock(table_options_.filter_policy.get(),
+ BlockContents(slice)),
+ nullptr /* cache */, nullptr /* cache_handle */, true /* own_value */);
+
+ FullFilterBlockReader reader(table_.get(), std::move(block));
+ Env::IOPriority rate_limiter_priority = Env::IO_TOTAL;
+ ASSERT_TRUE(reader.KeyMayMatch("foo",
+ /*no_io=*/false, /*const_ikey_ptr=*/nullptr,
+ /*get_context=*/nullptr,
+ /*lookup_context=*/nullptr,
+ rate_limiter_priority));
+ ASSERT_TRUE(reader.KeyMayMatch("bar",
+ /*no_io=*/false, /*const_ikey_ptr=*/nullptr,
+ /*get_context=*/nullptr,
+ /*lookup_context=*/nullptr,
+ rate_limiter_priority));
+ ASSERT_TRUE(reader.KeyMayMatch("box",
+ /*no_io=*/false, /*const_ikey_ptr=*/nullptr,
+ /*get_context=*/nullptr,
+ /*lookup_context=*/nullptr,
+ rate_limiter_priority));
+ ASSERT_TRUE(reader.KeyMayMatch("hello",
+ /*no_io=*/false, /*const_ikey_ptr=*/nullptr,
+ /*get_context=*/nullptr,
+ /*lookup_context=*/nullptr,
+ rate_limiter_priority));
+ ASSERT_TRUE(reader.KeyMayMatch("foo",
+ /*no_io=*/false, /*const_ikey_ptr=*/nullptr,
+ /*get_context=*/nullptr,
+ /*lookup_context=*/nullptr,
+ rate_limiter_priority));
+ ASSERT_TRUE(!reader.KeyMayMatch("missing",
+ /*no_io=*/false, /*const_ikey_ptr=*/nullptr,
+ /*get_context=*/nullptr,
+ /*lookup_context=*/nullptr,
+ rate_limiter_priority));
+ ASSERT_TRUE(!reader.KeyMayMatch("other",
+ /*no_io=*/false, /*const_ikey_ptr=*/nullptr,
+ /*get_context=*/nullptr,
+ /*lookup_context=*/nullptr,
+ rate_limiter_priority));
+}
+
+class FullFilterBlockTest : public mock::MockBlockBasedTableTester,
+ public testing::Test {
+ public:
+ FullFilterBlockTest()
+ : mock::MockBlockBasedTableTester(NewBloomFilterPolicy(10, false)) {}
+};
+
+TEST_F(FullFilterBlockTest, EmptyBuilder) {
+ FullFilterBlockBuilder builder(nullptr, true, GetBuilder());
+ Slice slice = builder.Finish();
+ ASSERT_EQ("", EscapeString(slice));
+
+ CachableEntry<ParsedFullFilterBlock> block(
+ new ParsedFullFilterBlock(table_options_.filter_policy.get(),
+ BlockContents(slice)),
+ nullptr /* cache */, nullptr /* cache_handle */, true /* own_value */);
+
+ FullFilterBlockReader reader(table_.get(), std::move(block));
+ // Remain same symantic with blockbased filter
+ ASSERT_TRUE(reader.KeyMayMatch("foo",
+ /*no_io=*/false, /*const_ikey_ptr=*/nullptr,
+ /*get_context=*/nullptr,
+ /*lookup_context=*/nullptr, Env::IO_TOTAL));
+}
+
+class CountUniqueFilterBitsBuilderWrapper : public FilterBitsBuilder {
+ std::unique_ptr<FilterBitsBuilder> b_;
+ std::set<std::string> uniq_;
+
+ public:
+ explicit CountUniqueFilterBitsBuilderWrapper(FilterBitsBuilder* b) : b_(b) {}
+
+ ~CountUniqueFilterBitsBuilderWrapper() override {}
+
+ void AddKey(const Slice& key) override {
+ b_->AddKey(key);
+ uniq_.insert(key.ToString());
+ }
+
+ using FilterBitsBuilder::Finish;
+
+ Slice Finish(std::unique_ptr<const char[]>* buf) override {
+ Slice rv = b_->Finish(buf);
+ Status s_dont_care = b_->MaybePostVerify(rv);
+ s_dont_care.PermitUncheckedError();
+ uniq_.clear();
+ return rv;
+ }
+
+ size_t EstimateEntriesAdded() override { return b_->EstimateEntriesAdded(); }
+
+ size_t ApproximateNumEntries(size_t bytes) override {
+ return b_->ApproximateNumEntries(bytes);
+ }
+
+ size_t CountUnique() { return uniq_.size(); }
+};
+
+TEST_F(FullFilterBlockTest, DuplicateEntries) {
+ { // empty prefixes
+ std::unique_ptr<const SliceTransform> prefix_extractor(
+ NewFixedPrefixTransform(0));
+ auto bits_builder = new CountUniqueFilterBitsBuilderWrapper(GetBuilder());
+ const bool WHOLE_KEY = true;
+ FullFilterBlockBuilder builder(prefix_extractor.get(), WHOLE_KEY,
+ bits_builder);
+ ASSERT_EQ(0, bits_builder->CountUnique());
+ // adds key and empty prefix; both abstractions count them
+ builder.Add("key1");
+ ASSERT_EQ(2, bits_builder->CountUnique());
+ // Add different key (unique) and also empty prefix (not unique).
+ // From here in this test, it's immaterial whether the block builder
+ // can count unique keys.
+ builder.Add("key2");
+ ASSERT_EQ(3, bits_builder->CountUnique());
+ // Empty key -> nothing unique
+ builder.Add("");
+ ASSERT_EQ(3, bits_builder->CountUnique());
+ }
+
+ // mix of empty and non-empty
+ std::unique_ptr<const SliceTransform> prefix_extractor(
+ NewFixedPrefixTransform(7));
+ auto bits_builder = new CountUniqueFilterBitsBuilderWrapper(GetBuilder());
+ const bool WHOLE_KEY = true;
+ FullFilterBlockBuilder builder(prefix_extractor.get(), WHOLE_KEY,
+ bits_builder);
+ builder.Add(""); // test with empty key too
+ builder.Add("prefix1key1");
+ builder.Add("prefix1key1");
+ builder.Add("prefix1key2");
+ builder.Add("prefix1key3");
+ builder.Add("prefix2key4");
+ // 1 empty, 2 non-empty prefixes, and 4 non-empty keys
+ ASSERT_EQ(1 + 2 + 4, bits_builder->CountUnique());
+}
+
+TEST_F(FullFilterBlockTest, SingleChunk) {
+ FullFilterBlockBuilder builder(nullptr, true, GetBuilder());
+ ASSERT_TRUE(builder.IsEmpty());
+ builder.Add("foo");
+ ASSERT_FALSE(builder.IsEmpty());
+ builder.Add("bar");
+ builder.Add("box");
+ builder.Add("box");
+ builder.Add("hello");
+ // "box" only counts once
+ ASSERT_EQ(4, builder.EstimateEntriesAdded());
+ ASSERT_FALSE(builder.IsEmpty());
+ Status s;
+ Slice slice = builder.Finish(BlockHandle(), &s);
+ ASSERT_OK(s);
+
+ CachableEntry<ParsedFullFilterBlock> block(
+ new ParsedFullFilterBlock(table_options_.filter_policy.get(),
+ BlockContents(slice)),
+ nullptr /* cache */, nullptr /* cache_handle */, true /* own_value */);
+
+ FullFilterBlockReader reader(table_.get(), std::move(block));
+ Env::IOPriority rate_limiter_priority = Env::IO_TOTAL;
+ ASSERT_TRUE(reader.KeyMayMatch("foo",
+ /*no_io=*/false, /*const_ikey_ptr=*/nullptr,
+ /*get_context=*/nullptr,
+ /*lookup_context=*/nullptr,
+ rate_limiter_priority));
+ ASSERT_TRUE(reader.KeyMayMatch("bar",
+ /*no_io=*/false, /*const_ikey_ptr=*/nullptr,
+ /*get_context=*/nullptr,
+ /*lookup_context=*/nullptr,
+ rate_limiter_priority));
+ ASSERT_TRUE(reader.KeyMayMatch("box",
+ /*no_io=*/false, /*const_ikey_ptr=*/nullptr,
+ /*get_context=*/nullptr,
+ /*lookup_context=*/nullptr,
+ rate_limiter_priority));
+ ASSERT_TRUE(reader.KeyMayMatch("hello",
+ /*no_io=*/false, /*const_ikey_ptr=*/nullptr,
+ /*get_context=*/nullptr,
+ /*lookup_context=*/nullptr,
+ rate_limiter_priority));
+ ASSERT_TRUE(reader.KeyMayMatch("foo",
+ /*no_io=*/false, /*const_ikey_ptr=*/nullptr,
+ /*get_context=*/nullptr,
+ /*lookup_context=*/nullptr,
+ rate_limiter_priority));
+ ASSERT_TRUE(!reader.KeyMayMatch("missing",
+ /*no_io=*/false, /*const_ikey_ptr=*/nullptr,
+ /*get_context=*/nullptr,
+ /*lookup_context=*/nullptr,
+ rate_limiter_priority));
+ ASSERT_TRUE(!reader.KeyMayMatch("other",
+ /*no_io=*/false, /*const_ikey_ptr=*/nullptr,
+ /*get_context=*/nullptr,
+ /*lookup_context=*/nullptr,
+ rate_limiter_priority));
+}
+
+} // namespace ROCKSDB_NAMESPACE
+
+int main(int argc, char** argv) {
+ ROCKSDB_NAMESPACE::port::InstallStackTraceHandler();
+ ::testing::InitGoogleTest(&argc, argv);
+ return RUN_ALL_TESTS();
+}
diff --git a/src/rocksdb/table/block_based/hash_index_reader.cc b/src/rocksdb/table/block_based/hash_index_reader.cc
new file mode 100644
index 000000000..bcaba17a2
--- /dev/null
+++ b/src/rocksdb/table/block_based/hash_index_reader.cc
@@ -0,0 +1,148 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+#include "table/block_based/hash_index_reader.h"
+
+#include "table/block_fetcher.h"
+#include "table/meta_blocks.h"
+
+namespace ROCKSDB_NAMESPACE {
+Status HashIndexReader::Create(const BlockBasedTable* table,
+ const ReadOptions& ro,
+ FilePrefetchBuffer* prefetch_buffer,
+ InternalIterator* meta_index_iter,
+ bool use_cache, bool prefetch, bool pin,
+ BlockCacheLookupContext* lookup_context,
+ std::unique_ptr<IndexReader>* index_reader) {
+ assert(table != nullptr);
+ assert(index_reader != nullptr);
+ assert(!pin || prefetch);
+
+ const BlockBasedTable::Rep* rep = table->get_rep();
+ assert(rep != nullptr);
+
+ CachableEntry<Block> index_block;
+ if (prefetch || !use_cache) {
+ const Status s =
+ ReadIndexBlock(table, prefetch_buffer, ro, use_cache,
+ /*get_context=*/nullptr, lookup_context, &index_block);
+ if (!s.ok()) {
+ return s;
+ }
+
+ if (use_cache && !pin) {
+ index_block.Reset();
+ }
+ }
+
+ // 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.
+
+ index_reader->reset(new HashIndexReader(table, std::move(index_block)));
+
+ // Get prefixes block
+ BlockHandle prefixes_handle;
+ Status s =
+ FindMetaBlock(meta_index_iter, kHashIndexPrefixesBlock, &prefixes_handle);
+ if (!s.ok()) {
+ // TODO: log error
+ return Status::OK();
+ }
+
+ // Get index metadata block
+ BlockHandle prefixes_meta_handle;
+ s = FindMetaBlock(meta_index_iter, kHashIndexPrefixesMetadataBlock,
+ &prefixes_meta_handle);
+ if (!s.ok()) {
+ // TODO: log error
+ return Status::OK();
+ }
+
+ RandomAccessFileReader* const file = rep->file.get();
+ const Footer& footer = rep->footer;
+ const ImmutableOptions& ioptions = rep->ioptions;
+ const PersistentCacheOptions& cache_options = rep->persistent_cache_options;
+ MemoryAllocator* const memory_allocator =
+ GetMemoryAllocator(rep->table_options);
+
+ // Read contents for the blocks
+ BlockContents prefixes_contents;
+ BlockFetcher prefixes_block_fetcher(
+ file, prefetch_buffer, footer, ReadOptions(), prefixes_handle,
+ &prefixes_contents, ioptions, true /*decompress*/,
+ true /*maybe_compressed*/, BlockType::kHashIndexPrefixes,
+ UncompressionDict::GetEmptyDict(), cache_options, memory_allocator);
+ s = prefixes_block_fetcher.ReadBlockContents();
+ if (!s.ok()) {
+ return s;
+ }
+ BlockContents prefixes_meta_contents;
+ BlockFetcher prefixes_meta_block_fetcher(
+ file, prefetch_buffer, footer, ReadOptions(), prefixes_meta_handle,
+ &prefixes_meta_contents, ioptions, true /*decompress*/,
+ true /*maybe_compressed*/, BlockType::kHashIndexMetadata,
+ UncompressionDict::GetEmptyDict(), cache_options, memory_allocator);
+ s = prefixes_meta_block_fetcher.ReadBlockContents();
+ if (!s.ok()) {
+ // TODO: log error
+ return Status::OK();
+ }
+
+ BlockPrefixIndex* prefix_index = nullptr;
+ assert(rep->table_prefix_extractor);
+ s = BlockPrefixIndex::Create(rep->table_prefix_extractor.get(),
+ prefixes_contents.data,
+ prefixes_meta_contents.data, &prefix_index);
+ // TODO: log error
+ if (s.ok()) {
+ HashIndexReader* const hash_index_reader =
+ static_cast<HashIndexReader*>(index_reader->get());
+ hash_index_reader->prefix_index_.reset(prefix_index);
+ }
+
+ return Status::OK();
+}
+
+InternalIteratorBase<IndexValue>* HashIndexReader::NewIterator(
+ const ReadOptions& read_options, bool disable_prefix_seek,
+ IndexBlockIter* iter, GetContext* get_context,
+ BlockCacheLookupContext* lookup_context) {
+ const BlockBasedTable::Rep* rep = table()->get_rep();
+ const bool no_io = (read_options.read_tier == kBlockCacheTier);
+ CachableEntry<Block> index_block;
+ const Status s =
+ GetOrReadIndexBlock(no_io, read_options.rate_limiter_priority,
+ get_context, lookup_context, &index_block);
+ if (!s.ok()) {
+ if (iter != nullptr) {
+ iter->Invalidate(s);
+ return iter;
+ }
+
+ return NewErrorInternalIterator<IndexValue>(s);
+ }
+
+ Statistics* kNullStats = nullptr;
+ const bool total_order_seek =
+ read_options.total_order_seek || disable_prefix_seek;
+ // We don't return pinned data from index blocks, so no need
+ // to set `block_contents_pinned`.
+ auto it = index_block.GetValue()->NewIndexIterator(
+ internal_comparator()->user_comparator(),
+ rep->get_global_seqno(BlockType::kIndex), iter, kNullStats,
+ total_order_seek, index_has_first_key(), index_key_includes_seq(),
+ index_value_is_full(), false /* block_contents_pinned */,
+ prefix_index_.get());
+
+ assert(it != nullptr);
+ index_block.TransferTo(it);
+
+ return it;
+}
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/hash_index_reader.h b/src/rocksdb/table/block_based/hash_index_reader.h
new file mode 100644
index 000000000..9037efc87
--- /dev/null
+++ b/src/rocksdb/table/block_based/hash_index_reader.h
@@ -0,0 +1,49 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+#pragma once
+
+#include "table/block_based/index_reader_common.h"
+
+namespace ROCKSDB_NAMESPACE {
+// Index that leverages an internal hash table to quicken the lookup for a given
+// key.
+class HashIndexReader : public BlockBasedTable::IndexReaderCommon {
+ public:
+ static Status Create(const BlockBasedTable* table, const ReadOptions& ro,
+ FilePrefetchBuffer* prefetch_buffer,
+ InternalIterator* meta_index_iter, bool use_cache,
+ bool prefetch, bool pin,
+ BlockCacheLookupContext* lookup_context,
+ std::unique_ptr<IndexReader>* index_reader);
+
+ InternalIteratorBase<IndexValue>* NewIterator(
+ const ReadOptions& read_options, bool disable_prefix_seek,
+ IndexBlockIter* iter, GetContext* get_context,
+ BlockCacheLookupContext* lookup_context) override;
+
+ size_t ApproximateMemoryUsage() const override {
+ size_t usage = ApproximateIndexBlockMemoryUsage();
+#ifdef ROCKSDB_MALLOC_USABLE_SIZE
+ usage += malloc_usable_size(const_cast<HashIndexReader*>(this));
+#else
+ if (prefix_index_) {
+ usage += prefix_index_->ApproximateMemoryUsage();
+ }
+ usage += sizeof(*this);
+#endif // ROCKSDB_MALLOC_USABLE_SIZE
+ return usage;
+ }
+
+ private:
+ HashIndexReader(const BlockBasedTable* t, CachableEntry<Block>&& index_block)
+ : IndexReaderCommon(t, std::move(index_block)) {}
+
+ std::unique_ptr<BlockPrefixIndex> prefix_index_;
+};
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/index_builder.cc b/src/rocksdb/table/block_based/index_builder.cc
new file mode 100644
index 000000000..024730178
--- /dev/null
+++ b/src/rocksdb/table/block_based/index_builder.cc
@@ -0,0 +1,282 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#include "table/block_based/index_builder.h"
+
+#include <assert.h>
+
+#include <cinttypes>
+#include <list>
+#include <string>
+
+#include "db/dbformat.h"
+#include "rocksdb/comparator.h"
+#include "rocksdb/flush_block_policy.h"
+#include "table/block_based/partitioned_filter_block.h"
+#include "table/format.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+// Create a index builder based on its type.
+IndexBuilder* IndexBuilder::CreateIndexBuilder(
+ BlockBasedTableOptions::IndexType index_type,
+ const InternalKeyComparator* comparator,
+ const InternalKeySliceTransform* int_key_slice_transform,
+ const bool use_value_delta_encoding,
+ const BlockBasedTableOptions& table_opt) {
+ IndexBuilder* result = nullptr;
+ switch (index_type) {
+ case BlockBasedTableOptions::kBinarySearch: {
+ result = new ShortenedIndexBuilder(
+ comparator, table_opt.index_block_restart_interval,
+ table_opt.format_version, use_value_delta_encoding,
+ table_opt.index_shortening, /* include_first_key */ false);
+ break;
+ }
+ case BlockBasedTableOptions::kHashSearch: {
+ // Currently kHashSearch is incompatible with index_block_restart_interval
+ // > 1
+ assert(table_opt.index_block_restart_interval == 1);
+ result = new HashIndexBuilder(
+ comparator, int_key_slice_transform,
+ table_opt.index_block_restart_interval, table_opt.format_version,
+ use_value_delta_encoding, table_opt.index_shortening);
+ break;
+ }
+ case BlockBasedTableOptions::kTwoLevelIndexSearch: {
+ result = PartitionedIndexBuilder::CreateIndexBuilder(
+ comparator, use_value_delta_encoding, table_opt);
+ break;
+ }
+ case BlockBasedTableOptions::kBinarySearchWithFirstKey: {
+ result = new ShortenedIndexBuilder(
+ comparator, table_opt.index_block_restart_interval,
+ table_opt.format_version, use_value_delta_encoding,
+ table_opt.index_shortening, /* include_first_key */ true);
+ break;
+ }
+ default: {
+ assert(!"Do not recognize the index type ");
+ break;
+ }
+ }
+ return result;
+}
+
+void ShortenedIndexBuilder::FindShortestInternalKeySeparator(
+ const Comparator& comparator, std::string* start, const Slice& limit) {
+ // Attempt to shorten the user portion of the key
+ Slice user_start = ExtractUserKey(*start);
+ Slice user_limit = ExtractUserKey(limit);
+ std::string tmp(user_start.data(), user_start.size());
+ comparator.FindShortestSeparator(&tmp, user_limit);
+ if (tmp.size() <= user_start.size() &&
+ comparator.Compare(user_start, tmp) < 0) {
+ // User key has become shorter physically, but larger logically.
+ // Tack on the earliest possible number to the shortened user key.
+ PutFixed64(&tmp,
+ PackSequenceAndType(kMaxSequenceNumber, kValueTypeForSeek));
+ assert(InternalKeyComparator(&comparator).Compare(*start, tmp) < 0);
+ assert(InternalKeyComparator(&comparator).Compare(tmp, limit) < 0);
+ start->swap(tmp);
+ }
+}
+
+void ShortenedIndexBuilder::FindShortInternalKeySuccessor(
+ const Comparator& comparator, std::string* key) {
+ Slice user_key = ExtractUserKey(*key);
+ std::string tmp(user_key.data(), user_key.size());
+ comparator.FindShortSuccessor(&tmp);
+ if (tmp.size() <= user_key.size() && comparator.Compare(user_key, tmp) < 0) {
+ // User key has become shorter physically, but larger logically.
+ // Tack on the earliest possible number to the shortened user key.
+ PutFixed64(&tmp,
+ PackSequenceAndType(kMaxSequenceNumber, kValueTypeForSeek));
+ assert(InternalKeyComparator(&comparator).Compare(*key, tmp) < 0);
+ key->swap(tmp);
+ }
+}
+
+PartitionedIndexBuilder* PartitionedIndexBuilder::CreateIndexBuilder(
+ const InternalKeyComparator* comparator,
+ const bool use_value_delta_encoding,
+ const BlockBasedTableOptions& table_opt) {
+ return new PartitionedIndexBuilder(comparator, table_opt,
+ use_value_delta_encoding);
+}
+
+PartitionedIndexBuilder::PartitionedIndexBuilder(
+ const InternalKeyComparator* comparator,
+ const BlockBasedTableOptions& table_opt,
+ const bool use_value_delta_encoding)
+ : IndexBuilder(comparator),
+ index_block_builder_(table_opt.index_block_restart_interval,
+ true /*use_delta_encoding*/,
+ use_value_delta_encoding),
+ index_block_builder_without_seq_(table_opt.index_block_restart_interval,
+ true /*use_delta_encoding*/,
+ use_value_delta_encoding),
+ sub_index_builder_(nullptr),
+ table_opt_(table_opt),
+ // We start by false. After each partition we revise the value based on
+ // what the sub_index_builder has decided. If the feature is disabled
+ // entirely, this will be set to true after switching the first
+ // sub_index_builder. Otherwise, it could be set to true even one of the
+ // sub_index_builders could not safely exclude seq from the keys, then it
+ // wil be enforced on all sub_index_builders on ::Finish.
+ seperator_is_key_plus_seq_(false),
+ use_value_delta_encoding_(use_value_delta_encoding) {}
+
+PartitionedIndexBuilder::~PartitionedIndexBuilder() {
+ delete sub_index_builder_;
+}
+
+void PartitionedIndexBuilder::MakeNewSubIndexBuilder() {
+ assert(sub_index_builder_ == nullptr);
+ sub_index_builder_ = new ShortenedIndexBuilder(
+ comparator_, table_opt_.index_block_restart_interval,
+ table_opt_.format_version, use_value_delta_encoding_,
+ table_opt_.index_shortening, /* include_first_key */ false);
+
+ // Set sub_index_builder_->seperator_is_key_plus_seq_ to true if
+ // seperator_is_key_plus_seq_ is true (internal-key mode) (set to false by
+ // default on Creation) so that flush policy can point to
+ // sub_index_builder_->index_block_builder_
+ if (seperator_is_key_plus_seq_) {
+ sub_index_builder_->seperator_is_key_plus_seq_ = true;
+ }
+
+ flush_policy_.reset(FlushBlockBySizePolicyFactory::NewFlushBlockPolicy(
+ table_opt_.metadata_block_size, table_opt_.block_size_deviation,
+ // Note: this is sub-optimal since sub_index_builder_ could later reset
+ // seperator_is_key_plus_seq_ but the probability of that is low.
+ sub_index_builder_->seperator_is_key_plus_seq_
+ ? sub_index_builder_->index_block_builder_
+ : sub_index_builder_->index_block_builder_without_seq_));
+ partition_cut_requested_ = false;
+}
+
+void PartitionedIndexBuilder::RequestPartitionCut() {
+ partition_cut_requested_ = true;
+}
+
+void PartitionedIndexBuilder::AddIndexEntry(
+ std::string* last_key_in_current_block,
+ const Slice* first_key_in_next_block, const BlockHandle& block_handle) {
+ // Note: to avoid two consecuitive flush in the same method call, we do not
+ // check flush policy when adding the last key
+ if (UNLIKELY(first_key_in_next_block == nullptr)) { // no more keys
+ if (sub_index_builder_ == nullptr) {
+ MakeNewSubIndexBuilder();
+ }
+ sub_index_builder_->AddIndexEntry(last_key_in_current_block,
+ first_key_in_next_block, block_handle);
+ if (!seperator_is_key_plus_seq_ &&
+ sub_index_builder_->seperator_is_key_plus_seq_) {
+ // then we need to apply it to all sub-index builders and reset
+ // flush_policy to point to Block Builder of sub_index_builder_ that store
+ // internal keys.
+ seperator_is_key_plus_seq_ = true;
+ flush_policy_.reset(FlushBlockBySizePolicyFactory::NewFlushBlockPolicy(
+ table_opt_.metadata_block_size, table_opt_.block_size_deviation,
+ sub_index_builder_->index_block_builder_));
+ }
+ sub_index_last_key_ = std::string(*last_key_in_current_block);
+ entries_.push_back(
+ {sub_index_last_key_,
+ std::unique_ptr<ShortenedIndexBuilder>(sub_index_builder_)});
+ sub_index_builder_ = nullptr;
+ cut_filter_block = true;
+ } else {
+ // apply flush policy only to non-empty sub_index_builder_
+ if (sub_index_builder_ != nullptr) {
+ std::string handle_encoding;
+ block_handle.EncodeTo(&handle_encoding);
+ bool do_flush =
+ partition_cut_requested_ ||
+ flush_policy_->Update(*last_key_in_current_block, handle_encoding);
+ if (do_flush) {
+ entries_.push_back(
+ {sub_index_last_key_,
+ std::unique_ptr<ShortenedIndexBuilder>(sub_index_builder_)});
+ cut_filter_block = true;
+ sub_index_builder_ = nullptr;
+ }
+ }
+ if (sub_index_builder_ == nullptr) {
+ MakeNewSubIndexBuilder();
+ }
+ sub_index_builder_->AddIndexEntry(last_key_in_current_block,
+ first_key_in_next_block, block_handle);
+ sub_index_last_key_ = std::string(*last_key_in_current_block);
+ if (!seperator_is_key_plus_seq_ &&
+ sub_index_builder_->seperator_is_key_plus_seq_) {
+ // then we need to apply it to all sub-index builders and reset
+ // flush_policy to point to Block Builder of sub_index_builder_ that store
+ // internal keys.
+ seperator_is_key_plus_seq_ = true;
+ flush_policy_.reset(FlushBlockBySizePolicyFactory::NewFlushBlockPolicy(
+ table_opt_.metadata_block_size, table_opt_.block_size_deviation,
+ sub_index_builder_->index_block_builder_));
+ }
+ }
+}
+
+Status PartitionedIndexBuilder::Finish(
+ IndexBlocks* index_blocks, const BlockHandle& last_partition_block_handle) {
+ if (partition_cnt_ == 0) {
+ partition_cnt_ = entries_.size();
+ }
+ // It must be set to null after last key is added
+ assert(sub_index_builder_ == nullptr);
+ if (finishing_indexes == true) {
+ Entry& last_entry = entries_.front();
+ std::string handle_encoding;
+ last_partition_block_handle.EncodeTo(&handle_encoding);
+ std::string handle_delta_encoding;
+ PutVarsignedint64(
+ &handle_delta_encoding,
+ last_partition_block_handle.size() - last_encoded_handle_.size());
+ last_encoded_handle_ = last_partition_block_handle;
+ const Slice handle_delta_encoding_slice(handle_delta_encoding);
+ index_block_builder_.Add(last_entry.key, handle_encoding,
+ &handle_delta_encoding_slice);
+ if (!seperator_is_key_plus_seq_) {
+ index_block_builder_without_seq_.Add(ExtractUserKey(last_entry.key),
+ handle_encoding,
+ &handle_delta_encoding_slice);
+ }
+ entries_.pop_front();
+ }
+ // If there is no sub_index left, then return the 2nd level index.
+ if (UNLIKELY(entries_.empty())) {
+ if (seperator_is_key_plus_seq_) {
+ index_blocks->index_block_contents = index_block_builder_.Finish();
+ } else {
+ index_blocks->index_block_contents =
+ index_block_builder_without_seq_.Finish();
+ }
+ top_level_index_size_ = index_blocks->index_block_contents.size();
+ index_size_ += top_level_index_size_;
+ return Status::OK();
+ } else {
+ // Finish the next partition index in line and Incomplete() to indicate we
+ // expect more calls to Finish
+ Entry& entry = entries_.front();
+ // Apply the policy to all sub-indexes
+ entry.value->seperator_is_key_plus_seq_ = seperator_is_key_plus_seq_;
+ auto s = entry.value->Finish(index_blocks);
+ index_size_ += index_blocks->index_block_contents.size();
+ finishing_indexes = true;
+ return s.ok() ? Status::Incomplete() : s;
+ }
+}
+
+size_t PartitionedIndexBuilder::NumPartitions() const { return partition_cnt_; }
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/index_builder.h b/src/rocksdb/table/block_based/index_builder.h
new file mode 100644
index 000000000..dd3be0331
--- /dev/null
+++ b/src/rocksdb/table/block_based/index_builder.h
@@ -0,0 +1,455 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#pragma once
+
+#include <assert.h>
+
+#include <cinttypes>
+#include <list>
+#include <string>
+#include <unordered_map>
+
+#include "rocksdb/comparator.h"
+#include "table/block_based/block_based_table_factory.h"
+#include "table/block_based/block_builder.h"
+#include "table/format.h"
+
+namespace ROCKSDB_NAMESPACE {
+// The interface for building index.
+// Instruction for adding a new concrete IndexBuilder:
+// 1. Create a subclass instantiated from IndexBuilder.
+// 2. Add a new entry associated with that subclass in TableOptions::IndexType.
+// 3. Add a create function for the new subclass in CreateIndexBuilder.
+// Note: we can devise more advanced design to simplify the process for adding
+// new subclass, which will, on the other hand, increase the code complexity and
+// catch unwanted attention from readers. Given that we won't add/change
+// indexes frequently, it makes sense to just embrace a more straightforward
+// design that just works.
+class IndexBuilder {
+ public:
+ static IndexBuilder* CreateIndexBuilder(
+ BlockBasedTableOptions::IndexType index_type,
+ const ROCKSDB_NAMESPACE::InternalKeyComparator* comparator,
+ const InternalKeySliceTransform* int_key_slice_transform,
+ const bool use_value_delta_encoding,
+ const BlockBasedTableOptions& table_opt);
+
+ // Index builder will construct a set of blocks which contain:
+ // 1. One primary index block.
+ // 2. (Optional) a set of metablocks that contains the metadata of the
+ // primary index.
+ struct IndexBlocks {
+ Slice index_block_contents;
+ std::unordered_map<std::string, Slice> meta_blocks;
+ };
+ explicit IndexBuilder(const InternalKeyComparator* comparator)
+ : comparator_(comparator) {}
+
+ virtual ~IndexBuilder() {}
+
+ // Add a new index entry to index block.
+ // To allow further optimization, we provide `last_key_in_current_block` and
+ // `first_key_in_next_block`, based on which the specific implementation can
+ // determine the best index key to be used for the index block.
+ // Called before the OnKeyAdded() call for first_key_in_next_block.
+ // @last_key_in_current_block: this parameter maybe overridden with the value
+ // "substitute key".
+ // @first_key_in_next_block: it will be nullptr if the entry being added is
+ // the last one in the table
+ //
+ // REQUIRES: Finish() has not yet been called.
+ virtual void AddIndexEntry(std::string* last_key_in_current_block,
+ const Slice* first_key_in_next_block,
+ const BlockHandle& block_handle) = 0;
+
+ // This method will be called whenever a key is added. The subclasses may
+ // override OnKeyAdded() if they need to collect additional information.
+ virtual void OnKeyAdded(const Slice& /*key*/) {}
+
+ // Inform the index builder that all entries has been written. Block builder
+ // may therefore perform any operation required for block finalization.
+ //
+ // REQUIRES: Finish() has not yet been called.
+ inline Status Finish(IndexBlocks* index_blocks) {
+ // Throw away the changes to last_partition_block_handle. It has no effect
+ // on the first call to Finish anyway.
+ BlockHandle last_partition_block_handle;
+ return Finish(index_blocks, last_partition_block_handle);
+ }
+
+ // This override of Finish can be utilized to build the 2nd level index in
+ // PartitionIndexBuilder.
+ //
+ // index_blocks will be filled with the resulting index data. If the return
+ // value is Status::InComplete() then it means that the index is partitioned
+ // and the callee should keep calling Finish until Status::OK() is returned.
+ // In that case, last_partition_block_handle is pointer to the block written
+ // with the result of the last call to Finish. This can be utilized to build
+ // the second level index pointing to each block of partitioned indexes. The
+ // last call to Finish() that returns Status::OK() populates index_blocks with
+ // the 2nd level index content.
+ virtual Status Finish(IndexBlocks* index_blocks,
+ const BlockHandle& last_partition_block_handle) = 0;
+
+ // Get the size for index block. Must be called after ::Finish.
+ virtual size_t IndexSize() const = 0;
+
+ virtual bool seperator_is_key_plus_seq() { return true; }
+
+ protected:
+ const InternalKeyComparator* comparator_;
+ // Set after ::Finish is called
+ size_t index_size_ = 0;
+};
+
+// This index builder builds space-efficient index block.
+//
+// Optimizations:
+// 1. Made block's `block_restart_interval` to be 1, which will avoid linear
+// search when doing index lookup (can be disabled by setting
+// index_block_restart_interval).
+// 2. Shorten the key length for index block. Other than honestly using the
+// last key in the data block as the index key, we instead find a shortest
+// substitute key that serves the same function.
+class ShortenedIndexBuilder : public IndexBuilder {
+ public:
+ explicit ShortenedIndexBuilder(
+ const InternalKeyComparator* comparator,
+ const int index_block_restart_interval, const uint32_t format_version,
+ const bool use_value_delta_encoding,
+ BlockBasedTableOptions::IndexShorteningMode shortening_mode,
+ bool include_first_key)
+ : IndexBuilder(comparator),
+ index_block_builder_(index_block_restart_interval,
+ true /*use_delta_encoding*/,
+ use_value_delta_encoding),
+ index_block_builder_without_seq_(index_block_restart_interval,
+ true /*use_delta_encoding*/,
+ use_value_delta_encoding),
+ use_value_delta_encoding_(use_value_delta_encoding),
+ include_first_key_(include_first_key),
+ shortening_mode_(shortening_mode) {
+ // Making the default true will disable the feature for old versions
+ seperator_is_key_plus_seq_ = (format_version <= 2);
+ }
+
+ virtual void OnKeyAdded(const Slice& key) override {
+ if (include_first_key_ && current_block_first_internal_key_.empty()) {
+ current_block_first_internal_key_.assign(key.data(), key.size());
+ }
+ }
+
+ virtual void AddIndexEntry(std::string* last_key_in_current_block,
+ const Slice* first_key_in_next_block,
+ const BlockHandle& block_handle) override {
+ if (first_key_in_next_block != nullptr) {
+ if (shortening_mode_ !=
+ BlockBasedTableOptions::IndexShorteningMode::kNoShortening) {
+ FindShortestInternalKeySeparator(*comparator_->user_comparator(),
+ last_key_in_current_block,
+ *first_key_in_next_block);
+ }
+ if (!seperator_is_key_plus_seq_ &&
+ comparator_->user_comparator()->Compare(
+ ExtractUserKey(*last_key_in_current_block),
+ ExtractUserKey(*first_key_in_next_block)) == 0) {
+ seperator_is_key_plus_seq_ = true;
+ }
+ } else {
+ if (shortening_mode_ == BlockBasedTableOptions::IndexShorteningMode::
+ kShortenSeparatorsAndSuccessor) {
+ FindShortInternalKeySuccessor(*comparator_->user_comparator(),
+ last_key_in_current_block);
+ }
+ }
+ auto sep = Slice(*last_key_in_current_block);
+
+ assert(!include_first_key_ || !current_block_first_internal_key_.empty());
+ IndexValue entry(block_handle, current_block_first_internal_key_);
+ std::string encoded_entry;
+ std::string delta_encoded_entry;
+ entry.EncodeTo(&encoded_entry, include_first_key_, nullptr);
+ if (use_value_delta_encoding_ && !last_encoded_handle_.IsNull()) {
+ entry.EncodeTo(&delta_encoded_entry, include_first_key_,
+ &last_encoded_handle_);
+ } else {
+ // If it's the first block, or delta encoding is disabled,
+ // BlockBuilder::Add() below won't use delta-encoded slice.
+ }
+ last_encoded_handle_ = block_handle;
+ const Slice delta_encoded_entry_slice(delta_encoded_entry);
+ index_block_builder_.Add(sep, encoded_entry, &delta_encoded_entry_slice);
+ if (!seperator_is_key_plus_seq_) {
+ index_block_builder_without_seq_.Add(ExtractUserKey(sep), encoded_entry,
+ &delta_encoded_entry_slice);
+ }
+
+ current_block_first_internal_key_.clear();
+ }
+
+ using IndexBuilder::Finish;
+ virtual Status Finish(
+ IndexBlocks* index_blocks,
+ const BlockHandle& /*last_partition_block_handle*/) override {
+ if (seperator_is_key_plus_seq_) {
+ index_blocks->index_block_contents = index_block_builder_.Finish();
+ } else {
+ index_blocks->index_block_contents =
+ index_block_builder_without_seq_.Finish();
+ }
+ index_size_ = index_blocks->index_block_contents.size();
+ return Status::OK();
+ }
+
+ virtual size_t IndexSize() const override { return index_size_; }
+
+ virtual bool seperator_is_key_plus_seq() override {
+ return seperator_is_key_plus_seq_;
+ }
+
+ // Changes *key to a short string >= *key.
+ //
+ static void FindShortestInternalKeySeparator(const Comparator& comparator,
+ std::string* start,
+ const Slice& limit);
+
+ static void FindShortInternalKeySuccessor(const Comparator& comparator,
+ std::string* key);
+
+ friend class PartitionedIndexBuilder;
+
+ private:
+ BlockBuilder index_block_builder_;
+ BlockBuilder index_block_builder_without_seq_;
+ const bool use_value_delta_encoding_;
+ bool seperator_is_key_plus_seq_;
+ const bool include_first_key_;
+ BlockBasedTableOptions::IndexShorteningMode shortening_mode_;
+ BlockHandle last_encoded_handle_ = BlockHandle::NullBlockHandle();
+ std::string current_block_first_internal_key_;
+};
+
+// HashIndexBuilder contains a binary-searchable primary index and the
+// metadata for secondary hash index construction.
+// The metadata for hash index consists two parts:
+// - a metablock that compactly contains a sequence of prefixes. All prefixes
+// are stored consectively without any metadata (like, prefix sizes) being
+// stored, which is kept in the other metablock.
+// - a metablock contains the metadata of the prefixes, including prefix size,
+// restart index and number of block it spans. The format looks like:
+//
+// +-----------------+---------------------------+---------------------+
+// <=prefix 1
+// | length: 4 bytes | restart interval: 4 bytes | num-blocks: 4 bytes |
+// +-----------------+---------------------------+---------------------+
+// <=prefix 2
+// | length: 4 bytes | restart interval: 4 bytes | num-blocks: 4 bytes |
+// +-----------------+---------------------------+---------------------+
+// | |
+// | .... |
+// | |
+// +-----------------+---------------------------+---------------------+
+// <=prefix n
+// | length: 4 bytes | restart interval: 4 bytes | num-blocks: 4 bytes |
+// +-----------------+---------------------------+---------------------+
+//
+// The reason of separating these two metablocks is to enable the efficiently
+// reuse the first metablock during hash index construction without unnecessary
+// data copy or small heap allocations for prefixes.
+class HashIndexBuilder : public IndexBuilder {
+ public:
+ explicit HashIndexBuilder(
+ const InternalKeyComparator* comparator,
+ const SliceTransform* hash_key_extractor,
+ int index_block_restart_interval, int format_version,
+ bool use_value_delta_encoding,
+ BlockBasedTableOptions::IndexShorteningMode shortening_mode)
+ : IndexBuilder(comparator),
+ primary_index_builder_(comparator, index_block_restart_interval,
+ format_version, use_value_delta_encoding,
+ shortening_mode, /* include_first_key */ false),
+ hash_key_extractor_(hash_key_extractor) {}
+
+ virtual void AddIndexEntry(std::string* last_key_in_current_block,
+ const Slice* first_key_in_next_block,
+ const BlockHandle& block_handle) override {
+ ++current_restart_index_;
+ primary_index_builder_.AddIndexEntry(last_key_in_current_block,
+ first_key_in_next_block, block_handle);
+ }
+
+ virtual void OnKeyAdded(const Slice& key) override {
+ auto key_prefix = hash_key_extractor_->Transform(key);
+ bool is_first_entry = pending_block_num_ == 0;
+
+ // Keys may share the prefix
+ if (is_first_entry || pending_entry_prefix_ != key_prefix) {
+ if (!is_first_entry) {
+ FlushPendingPrefix();
+ }
+
+ // need a hard copy otherwise the underlying data changes all the time.
+ // TODO(kailiu) std::to_string() is expensive. We may speed up can avoid
+ // data copy.
+ pending_entry_prefix_ = key_prefix.ToString();
+ pending_block_num_ = 1;
+ pending_entry_index_ = static_cast<uint32_t>(current_restart_index_);
+ } else {
+ // entry number increments when keys share the prefix reside in
+ // different data blocks.
+ auto last_restart_index = pending_entry_index_ + pending_block_num_ - 1;
+ assert(last_restart_index <= current_restart_index_);
+ if (last_restart_index != current_restart_index_) {
+ ++pending_block_num_;
+ }
+ }
+ }
+
+ virtual Status Finish(
+ IndexBlocks* index_blocks,
+ const BlockHandle& last_partition_block_handle) override {
+ if (pending_block_num_ != 0) {
+ FlushPendingPrefix();
+ }
+ Status s = primary_index_builder_.Finish(index_blocks,
+ last_partition_block_handle);
+ index_blocks->meta_blocks.insert(
+ {kHashIndexPrefixesBlock.c_str(), prefix_block_});
+ index_blocks->meta_blocks.insert(
+ {kHashIndexPrefixesMetadataBlock.c_str(), prefix_meta_block_});
+ return s;
+ }
+
+ virtual size_t IndexSize() const override {
+ return primary_index_builder_.IndexSize() + prefix_block_.size() +
+ prefix_meta_block_.size();
+ }
+
+ virtual bool seperator_is_key_plus_seq() override {
+ return primary_index_builder_.seperator_is_key_plus_seq();
+ }
+
+ private:
+ void FlushPendingPrefix() {
+ prefix_block_.append(pending_entry_prefix_.data(),
+ pending_entry_prefix_.size());
+ PutVarint32Varint32Varint32(
+ &prefix_meta_block_,
+ static_cast<uint32_t>(pending_entry_prefix_.size()),
+ pending_entry_index_, pending_block_num_);
+ }
+
+ ShortenedIndexBuilder primary_index_builder_;
+ const SliceTransform* hash_key_extractor_;
+
+ // stores a sequence of prefixes
+ std::string prefix_block_;
+ // stores the metadata of prefixes
+ std::string prefix_meta_block_;
+
+ // The following 3 variables keeps unflushed prefix and its metadata.
+ // The details of block_num and entry_index can be found in
+ // "block_hash_index.{h,cc}"
+ uint32_t pending_block_num_ = 0;
+ uint32_t pending_entry_index_ = 0;
+ std::string pending_entry_prefix_;
+
+ uint64_t current_restart_index_ = 0;
+};
+
+/**
+ * IndexBuilder for two-level indexing. Internally it creates a new index for
+ * each partition and Finish then in order when Finish is called on it
+ * continiously until Status::OK() is returned.
+ *
+ * The format on the disk would be I I I I I I IP where I is block containing a
+ * partition of indexes built using ShortenedIndexBuilder and IP is a block
+ * containing a secondary index on the partitions, built using
+ * ShortenedIndexBuilder.
+ */
+class PartitionedIndexBuilder : public IndexBuilder {
+ public:
+ static PartitionedIndexBuilder* CreateIndexBuilder(
+ const ROCKSDB_NAMESPACE::InternalKeyComparator* comparator,
+ const bool use_value_delta_encoding,
+ const BlockBasedTableOptions& table_opt);
+
+ explicit PartitionedIndexBuilder(const InternalKeyComparator* comparator,
+ const BlockBasedTableOptions& table_opt,
+ const bool use_value_delta_encoding);
+
+ virtual ~PartitionedIndexBuilder();
+
+ virtual void AddIndexEntry(std::string* last_key_in_current_block,
+ const Slice* first_key_in_next_block,
+ const BlockHandle& block_handle) override;
+
+ virtual Status Finish(
+ IndexBlocks* index_blocks,
+ const BlockHandle& last_partition_block_handle) override;
+
+ virtual size_t IndexSize() const override { return index_size_; }
+ size_t TopLevelIndexSize(uint64_t) const { return top_level_index_size_; }
+ size_t NumPartitions() const;
+
+ inline bool ShouldCutFilterBlock() {
+ // Current policy is to align the partitions of index and filters
+ if (cut_filter_block) {
+ cut_filter_block = false;
+ return true;
+ }
+ return false;
+ }
+
+ std::string& GetPartitionKey() { return sub_index_last_key_; }
+
+ // Called when an external entity (such as filter partition builder) request
+ // cutting the next partition
+ void RequestPartitionCut();
+
+ virtual bool seperator_is_key_plus_seq() override {
+ return seperator_is_key_plus_seq_;
+ }
+
+ bool get_use_value_delta_encoding() { return use_value_delta_encoding_; }
+
+ private:
+ // Set after ::Finish is called
+ size_t top_level_index_size_ = 0;
+ // Set after ::Finish is called
+ size_t partition_cnt_ = 0;
+
+ void MakeNewSubIndexBuilder();
+
+ struct Entry {
+ std::string key;
+ std::unique_ptr<ShortenedIndexBuilder> value;
+ };
+ std::list<Entry> entries_; // list of partitioned indexes and their keys
+ BlockBuilder index_block_builder_; // top-level index builder
+ BlockBuilder index_block_builder_without_seq_; // same for user keys
+ // the active partition index builder
+ ShortenedIndexBuilder* sub_index_builder_;
+ // the last key in the active partition index builder
+ std::string sub_index_last_key_;
+ std::unique_ptr<FlushBlockPolicy> flush_policy_;
+ // true if Finish is called once but not complete yet.
+ bool finishing_indexes = false;
+ const BlockBasedTableOptions& table_opt_;
+ bool seperator_is_key_plus_seq_;
+ bool use_value_delta_encoding_;
+ // true if an external entity (such as filter partition builder) request
+ // cutting the next partition
+ bool partition_cut_requested_ = true;
+ // true if it should cut the next filter partition block
+ bool cut_filter_block = false;
+ BlockHandle last_encoded_handle_;
+};
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/index_reader_common.cc b/src/rocksdb/table/block_based/index_reader_common.cc
new file mode 100644
index 000000000..6584586c9
--- /dev/null
+++ b/src/rocksdb/table/block_based/index_reader_common.cc
@@ -0,0 +1,56 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+#include "table/block_based/index_reader_common.h"
+
+namespace ROCKSDB_NAMESPACE {
+Status BlockBasedTable::IndexReaderCommon::ReadIndexBlock(
+ const BlockBasedTable* table, FilePrefetchBuffer* prefetch_buffer,
+ const ReadOptions& read_options, bool use_cache, GetContext* get_context,
+ BlockCacheLookupContext* lookup_context,
+ CachableEntry<Block>* index_block) {
+ PERF_TIMER_GUARD(read_index_block_nanos);
+
+ assert(table != nullptr);
+ assert(index_block != nullptr);
+ assert(index_block->IsEmpty());
+
+ const Rep* const rep = table->get_rep();
+ assert(rep != nullptr);
+
+ const Status s = table->RetrieveBlock(
+ prefetch_buffer, read_options, rep->footer.index_handle(),
+ UncompressionDict::GetEmptyDict(), index_block, BlockType::kIndex,
+ get_context, lookup_context, /* for_compaction */ false, use_cache,
+ /* wait_for_cache */ true, /* async_read */ false);
+
+ return s;
+}
+
+Status BlockBasedTable::IndexReaderCommon::GetOrReadIndexBlock(
+ bool no_io, Env::IOPriority rate_limiter_priority, GetContext* get_context,
+ BlockCacheLookupContext* lookup_context,
+ CachableEntry<Block>* index_block) const {
+ assert(index_block != nullptr);
+
+ if (!index_block_.IsEmpty()) {
+ index_block->SetUnownedValue(index_block_.GetValue());
+ return Status::OK();
+ }
+
+ ReadOptions read_options;
+ read_options.rate_limiter_priority = rate_limiter_priority;
+ if (no_io) {
+ read_options.read_tier = kBlockCacheTier;
+ }
+
+ return ReadIndexBlock(table_, /*prefetch_buffer=*/nullptr, read_options,
+ cache_index_blocks(), get_context, lookup_context,
+ index_block);
+}
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/index_reader_common.h b/src/rocksdb/table/block_based/index_reader_common.h
new file mode 100644
index 000000000..5627b0eeb
--- /dev/null
+++ b/src/rocksdb/table/block_based/index_reader_common.h
@@ -0,0 +1,85 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+#pragma once
+
+#include "table/block_based/block_based_table_reader.h"
+#include "table/block_based/reader_common.h"
+
+namespace ROCKSDB_NAMESPACE {
+// Encapsulates common functionality for the various index reader
+// implementations. Provides access to the index block regardless of whether
+// it is owned by the reader or stored in the cache, or whether it is pinned
+// in the cache or not.
+class BlockBasedTable::IndexReaderCommon : public BlockBasedTable::IndexReader {
+ public:
+ IndexReaderCommon(const BlockBasedTable* t,
+ CachableEntry<Block>&& index_block)
+ : table_(t), index_block_(std::move(index_block)) {
+ assert(table_ != nullptr);
+ }
+
+ protected:
+ static Status ReadIndexBlock(const BlockBasedTable* table,
+ FilePrefetchBuffer* prefetch_buffer,
+ const ReadOptions& read_options, bool use_cache,
+ GetContext* get_context,
+ BlockCacheLookupContext* lookup_context,
+ CachableEntry<Block>* index_block);
+
+ const BlockBasedTable* table() const { return table_; }
+
+ const InternalKeyComparator* internal_comparator() const {
+ assert(table_ != nullptr);
+ assert(table_->get_rep() != nullptr);
+
+ return &table_->get_rep()->internal_comparator;
+ }
+
+ bool index_has_first_key() const {
+ assert(table_ != nullptr);
+ assert(table_->get_rep() != nullptr);
+ return table_->get_rep()->index_has_first_key;
+ }
+
+ bool index_key_includes_seq() const {
+ assert(table_ != nullptr);
+ assert(table_->get_rep() != nullptr);
+ return table_->get_rep()->index_key_includes_seq;
+ }
+
+ bool index_value_is_full() const {
+ assert(table_ != nullptr);
+ assert(table_->get_rep() != nullptr);
+ return table_->get_rep()->index_value_is_full;
+ }
+
+ bool cache_index_blocks() const {
+ assert(table_ != nullptr);
+ assert(table_->get_rep() != nullptr);
+ return table_->get_rep()->table_options.cache_index_and_filter_blocks;
+ }
+
+ Status GetOrReadIndexBlock(bool no_io, Env::IOPriority rate_limiter_priority,
+ GetContext* get_context,
+ BlockCacheLookupContext* lookup_context,
+ CachableEntry<Block>* index_block) const;
+
+ size_t ApproximateIndexBlockMemoryUsage() const {
+ assert(!index_block_.GetOwnValue() || index_block_.GetValue() != nullptr);
+ return index_block_.GetOwnValue()
+ ? index_block_.GetValue()->ApproximateMemoryUsage()
+ : 0;
+ }
+
+ private:
+ const BlockBasedTable* table_;
+ CachableEntry<Block> index_block_;
+};
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/mock_block_based_table.h b/src/rocksdb/table/block_based/mock_block_based_table.h
new file mode 100644
index 000000000..13f3dfaee
--- /dev/null
+++ b/src/rocksdb/table/block_based/mock_block_based_table.h
@@ -0,0 +1,62 @@
+// Copyright (c) 2019-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).
+#pragma once
+
+#include <memory>
+
+#include "rocksdb/filter_policy.h"
+#include "table/block_based/block_based_table_reader.h"
+#include "table/block_based/filter_policy_internal.h"
+
+namespace ROCKSDB_NAMESPACE {
+namespace mock {
+
+class MockBlockBasedTable : public BlockBasedTable {
+ public:
+ explicit MockBlockBasedTable(Rep* rep)
+ : BlockBasedTable(rep, nullptr /* block_cache_tracer */) {}
+};
+
+class MockBlockBasedTableTester {
+ static constexpr int kMockLevel = 0;
+
+ public:
+ Options options_;
+ ImmutableOptions ioptions_;
+ EnvOptions env_options_;
+ BlockBasedTableOptions table_options_;
+ InternalKeyComparator icomp_;
+ std::unique_ptr<BlockBasedTable> table_;
+
+ explicit MockBlockBasedTableTester(const FilterPolicy* filter_policy)
+ : MockBlockBasedTableTester(
+ std::shared_ptr<const FilterPolicy>(filter_policy)){};
+
+ explicit MockBlockBasedTableTester(
+ std::shared_ptr<const FilterPolicy> filter_policy)
+ : ioptions_(options_),
+ env_options_(options_),
+ icomp_(options_.comparator) {
+ table_options_.filter_policy = std::move(filter_policy);
+
+ constexpr bool skip_filters = false;
+ constexpr bool immortal_table = false;
+ table_.reset(new MockBlockBasedTable(new BlockBasedTable::Rep(
+ ioptions_, env_options_, table_options_, icomp_, skip_filters,
+ 12345 /*file_size*/, kMockLevel, immortal_table)));
+ }
+
+ FilterBitsBuilder* GetBuilder() const {
+ FilterBuildingContext context(table_options_);
+ context.column_family_name = "mock_cf";
+ context.compaction_style = ioptions_.compaction_style;
+ context.level_at_creation = kMockLevel;
+ context.info_log = ioptions_.logger;
+ return BloomFilterPolicy::GetBuilderFromContext(context);
+ }
+};
+
+} // namespace mock
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/parsed_full_filter_block.cc b/src/rocksdb/table/block_based/parsed_full_filter_block.cc
new file mode 100644
index 000000000..9184a48d2
--- /dev/null
+++ b/src/rocksdb/table/block_based/parsed_full_filter_block.cc
@@ -0,0 +1,23 @@
+// Copyright (c) 2011-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 "table/block_based/parsed_full_filter_block.h"
+
+#include "table/block_based/filter_policy_internal.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+ParsedFullFilterBlock::ParsedFullFilterBlock(const FilterPolicy* filter_policy,
+ BlockContents&& contents)
+ : block_contents_(std::move(contents)),
+ filter_bits_reader_(
+ !block_contents_.data.empty()
+ ? filter_policy->GetFilterBitsReader(block_contents_.data)
+ : nullptr) {}
+
+ParsedFullFilterBlock::~ParsedFullFilterBlock() = default;
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/parsed_full_filter_block.h b/src/rocksdb/table/block_based/parsed_full_filter_block.h
new file mode 100644
index 000000000..95d7b5208
--- /dev/null
+++ b/src/rocksdb/table/block_based/parsed_full_filter_block.h
@@ -0,0 +1,42 @@
+// Copyright (c) 2011-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).
+
+#pragma once
+
+#include <memory>
+
+#include "table/format.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class FilterBitsReader;
+class FilterPolicy;
+
+// The sharable/cachable part of the full filter.
+class ParsedFullFilterBlock {
+ public:
+ ParsedFullFilterBlock(const FilterPolicy* filter_policy,
+ BlockContents&& contents);
+ ~ParsedFullFilterBlock();
+
+ FilterBitsReader* filter_bits_reader() const {
+ return filter_bits_reader_.get();
+ }
+
+ // TODO: consider memory usage of the FilterBitsReader
+ size_t ApproximateMemoryUsage() const {
+ return block_contents_.ApproximateMemoryUsage();
+ }
+
+ bool own_bytes() const { return block_contents_.own_bytes(); }
+
+ const Slice GetBlockContentsData() const { return block_contents_.data; }
+
+ private:
+ BlockContents block_contents_;
+ std::unique_ptr<FilterBitsReader> filter_bits_reader_;
+};
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/partitioned_filter_block.cc b/src/rocksdb/table/block_based/partitioned_filter_block.cc
new file mode 100644
index 000000000..af30925b7
--- /dev/null
+++ b/src/rocksdb/table/block_based/partitioned_filter_block.cc
@@ -0,0 +1,561 @@
+// Copyright (c) 2011-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 "table/block_based/partitioned_filter_block.h"
+
+#include <utility>
+
+#include "block_type.h"
+#include "file/random_access_file_reader.h"
+#include "logging/logging.h"
+#include "monitoring/perf_context_imp.h"
+#include "port/malloc.h"
+#include "port/port.h"
+#include "rocksdb/filter_policy.h"
+#include "table/block_based/block.h"
+#include "table/block_based/block_based_table_reader.h"
+#include "util/coding.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+PartitionedFilterBlockBuilder::PartitionedFilterBlockBuilder(
+ const SliceTransform* _prefix_extractor, bool whole_key_filtering,
+ FilterBitsBuilder* filter_bits_builder, int index_block_restart_interval,
+ const bool use_value_delta_encoding,
+ PartitionedIndexBuilder* const p_index_builder,
+ const uint32_t partition_size)
+ : FullFilterBlockBuilder(_prefix_extractor, whole_key_filtering,
+ filter_bits_builder),
+ index_on_filter_block_builder_(index_block_restart_interval,
+ true /*use_delta_encoding*/,
+ use_value_delta_encoding),
+ index_on_filter_block_builder_without_seq_(index_block_restart_interval,
+ true /*use_delta_encoding*/,
+ use_value_delta_encoding),
+ p_index_builder_(p_index_builder),
+ keys_added_to_partition_(0),
+ total_added_in_built_(0) {
+ keys_per_partition_ = static_cast<uint32_t>(
+ filter_bits_builder_->ApproximateNumEntries(partition_size));
+ if (keys_per_partition_ < 1) {
+ // partition_size (minus buffer, ~10%) might be smaller than minimum
+ // filter size, sometimes based on cache line size. Try to find that
+ // minimum size without CalculateSpace (not necessarily available).
+ uint32_t larger = std::max(partition_size + 4, uint32_t{16});
+ for (;;) {
+ keys_per_partition_ = static_cast<uint32_t>(
+ filter_bits_builder_->ApproximateNumEntries(larger));
+ if (keys_per_partition_ >= 1) {
+ break;
+ }
+ larger += larger / 4;
+ if (larger > 100000) {
+ // might be a broken implementation. substitute something reasonable:
+ // 1 key / byte.
+ keys_per_partition_ = partition_size;
+ break;
+ }
+ }
+ }
+}
+
+PartitionedFilterBlockBuilder::~PartitionedFilterBlockBuilder() {
+ partitioned_filters_construction_status_.PermitUncheckedError();
+}
+
+void PartitionedFilterBlockBuilder::MaybeCutAFilterBlock(
+ const Slice* next_key) {
+ // Use == to send the request only once
+ if (keys_added_to_partition_ == keys_per_partition_) {
+ // Currently only index builder is in charge of cutting a partition. We keep
+ // requesting until it is granted.
+ p_index_builder_->RequestPartitionCut();
+ }
+ if (!p_index_builder_->ShouldCutFilterBlock()) {
+ return;
+ }
+
+ // Add the prefix of the next key before finishing the partition without
+ // updating last_prefix_str_. This hack, fixes a bug with format_verison=3
+ // where seeking for the prefix would lead us to the previous partition.
+ const bool maybe_add_prefix =
+ next_key && prefix_extractor() && prefix_extractor()->InDomain(*next_key);
+ if (maybe_add_prefix) {
+ const Slice next_key_prefix = prefix_extractor()->Transform(*next_key);
+ if (next_key_prefix.compare(last_prefix_str()) != 0) {
+ AddKey(next_key_prefix);
+ }
+ }
+
+ total_added_in_built_ += filter_bits_builder_->EstimateEntriesAdded();
+ std::unique_ptr<const char[]> filter_data;
+ Status filter_construction_status = Status::OK();
+ Slice filter =
+ filter_bits_builder_->Finish(&filter_data, &filter_construction_status);
+ if (filter_construction_status.ok()) {
+ filter_construction_status = filter_bits_builder_->MaybePostVerify(filter);
+ }
+ std::string& index_key = p_index_builder_->GetPartitionKey();
+ filters.push_back({index_key, std::move(filter_data), filter});
+ if (!filter_construction_status.ok() &&
+ partitioned_filters_construction_status_.ok()) {
+ partitioned_filters_construction_status_ = filter_construction_status;
+ }
+ keys_added_to_partition_ = 0;
+ Reset();
+}
+
+void PartitionedFilterBlockBuilder::Add(const Slice& key) {
+ MaybeCutAFilterBlock(&key);
+ FullFilterBlockBuilder::Add(key);
+}
+
+void PartitionedFilterBlockBuilder::AddKey(const Slice& key) {
+ FullFilterBlockBuilder::AddKey(key);
+ keys_added_to_partition_++;
+}
+
+size_t PartitionedFilterBlockBuilder::EstimateEntriesAdded() {
+ return total_added_in_built_ + filter_bits_builder_->EstimateEntriesAdded();
+}
+
+Slice PartitionedFilterBlockBuilder::Finish(
+ const BlockHandle& last_partition_block_handle, Status* status,
+ std::unique_ptr<const char[]>* filter_data) {
+ if (finishing_filters == true) {
+ // Record the handle of the last written filter block in the index
+ std::string handle_encoding;
+ last_partition_block_handle.EncodeTo(&handle_encoding);
+ std::string handle_delta_encoding;
+ PutVarsignedint64(
+ &handle_delta_encoding,
+ last_partition_block_handle.size() - last_encoded_handle_.size());
+ last_encoded_handle_ = last_partition_block_handle;
+ const Slice handle_delta_encoding_slice(handle_delta_encoding);
+ index_on_filter_block_builder_.Add(last_filter_entry_key, handle_encoding,
+ &handle_delta_encoding_slice);
+ if (!p_index_builder_->seperator_is_key_plus_seq()) {
+ index_on_filter_block_builder_without_seq_.Add(
+ ExtractUserKey(last_filter_entry_key), handle_encoding,
+ &handle_delta_encoding_slice);
+ }
+ } else {
+ MaybeCutAFilterBlock(nullptr);
+ }
+
+ if (!partitioned_filters_construction_status_.ok()) {
+ *status = partitioned_filters_construction_status_;
+ return Slice();
+ }
+
+ // If there is no filter partition left, then return the index on filter
+ // partitions
+ if (UNLIKELY(filters.empty())) {
+ *status = Status::OK();
+ last_filter_data.reset();
+ if (finishing_filters) {
+ // Simplest to just add them all at the end
+ total_added_in_built_ = 0;
+ if (p_index_builder_->seperator_is_key_plus_seq()) {
+ return index_on_filter_block_builder_.Finish();
+ } else {
+ return index_on_filter_block_builder_without_seq_.Finish();
+ }
+ } else {
+ // This is the rare case where no key was added to the filter
+ return Slice();
+ }
+ } else {
+ // Return the next filter partition in line and set Incomplete() status to
+ // indicate we expect more calls to Finish
+ *status = Status::Incomplete();
+ finishing_filters = true;
+
+ last_filter_entry_key = filters.front().key;
+ Slice filter = filters.front().filter;
+ last_filter_data = std::move(filters.front().filter_data);
+ if (filter_data != nullptr) {
+ *filter_data = std::move(last_filter_data);
+ }
+ filters.pop_front();
+ return filter;
+ }
+}
+
+PartitionedFilterBlockReader::PartitionedFilterBlockReader(
+ const BlockBasedTable* t, CachableEntry<Block>&& filter_block)
+ : FilterBlockReaderCommon(t, std::move(filter_block)) {}
+
+std::unique_ptr<FilterBlockReader> PartitionedFilterBlockReader::Create(
+ const BlockBasedTable* table, const ReadOptions& ro,
+ FilePrefetchBuffer* prefetch_buffer, bool use_cache, bool prefetch,
+ bool pin, BlockCacheLookupContext* lookup_context) {
+ assert(table);
+ assert(table->get_rep());
+ assert(!pin || prefetch);
+
+ CachableEntry<Block> filter_block;
+ if (prefetch || !use_cache) {
+ const Status s = ReadFilterBlock(
+ table, prefetch_buffer, ro, use_cache, nullptr /* get_context */,
+ lookup_context, &filter_block, BlockType::kFilterPartitionIndex);
+ if (!s.ok()) {
+ IGNORE_STATUS_IF_ERROR(s);
+ return std::unique_ptr<FilterBlockReader>();
+ }
+
+ if (use_cache && !pin) {
+ filter_block.Reset();
+ }
+ }
+
+ return std::unique_ptr<FilterBlockReader>(
+ new PartitionedFilterBlockReader(table, std::move(filter_block)));
+}
+
+bool PartitionedFilterBlockReader::KeyMayMatch(
+ const Slice& key, const bool no_io, const Slice* const const_ikey_ptr,
+ GetContext* get_context, BlockCacheLookupContext* lookup_context,
+ Env::IOPriority rate_limiter_priority) {
+ assert(const_ikey_ptr != nullptr);
+ if (!whole_key_filtering()) {
+ return true;
+ }
+
+ return MayMatch(key, no_io, const_ikey_ptr, get_context, lookup_context,
+ rate_limiter_priority, &FullFilterBlockReader::KeyMayMatch);
+}
+
+void PartitionedFilterBlockReader::KeysMayMatch(
+ MultiGetRange* range, const bool no_io,
+ BlockCacheLookupContext* lookup_context,
+ Env::IOPriority rate_limiter_priority) {
+ if (!whole_key_filtering()) {
+ return; // Any/all may match
+ }
+
+ MayMatch(range, nullptr, no_io, lookup_context, rate_limiter_priority,
+ &FullFilterBlockReader::KeysMayMatch2);
+}
+
+bool PartitionedFilterBlockReader::PrefixMayMatch(
+ const Slice& prefix, const bool no_io, const Slice* const const_ikey_ptr,
+ GetContext* get_context, BlockCacheLookupContext* lookup_context,
+ Env::IOPriority rate_limiter_priority) {
+ assert(const_ikey_ptr != nullptr);
+ return MayMatch(prefix, no_io, const_ikey_ptr, get_context, lookup_context,
+ rate_limiter_priority,
+ &FullFilterBlockReader::PrefixMayMatch);
+}
+
+void PartitionedFilterBlockReader::PrefixesMayMatch(
+ MultiGetRange* range, const SliceTransform* prefix_extractor,
+ const bool no_io, BlockCacheLookupContext* lookup_context,
+ Env::IOPriority rate_limiter_priority) {
+ assert(prefix_extractor);
+ MayMatch(range, prefix_extractor, no_io, lookup_context,
+ rate_limiter_priority, &FullFilterBlockReader::PrefixesMayMatch);
+}
+
+BlockHandle PartitionedFilterBlockReader::GetFilterPartitionHandle(
+ const CachableEntry<Block>& filter_block, const Slice& entry) const {
+ IndexBlockIter iter;
+ const InternalKeyComparator* const comparator = internal_comparator();
+ Statistics* kNullStats = nullptr;
+ filter_block.GetValue()->NewIndexIterator(
+ comparator->user_comparator(),
+ table()->get_rep()->get_global_seqno(BlockType::kFilterPartitionIndex),
+ &iter, kNullStats, true /* total_order_seek */,
+ false /* have_first_key */, index_key_includes_seq(),
+ index_value_is_full());
+ iter.Seek(entry);
+ if (UNLIKELY(!iter.Valid())) {
+ // entry is larger than all the keys. However its prefix might still be
+ // present in the last partition. If this is called by PrefixMayMatch this
+ // is necessary for correct behavior. Otherwise it is unnecessary but safe.
+ // Assuming this is an unlikely case for full key search, the performance
+ // overhead should be negligible.
+ iter.SeekToLast();
+ }
+ assert(iter.Valid());
+ BlockHandle fltr_blk_handle = iter.value().handle;
+ return fltr_blk_handle;
+}
+
+Status PartitionedFilterBlockReader::GetFilterPartitionBlock(
+ FilePrefetchBuffer* prefetch_buffer, const BlockHandle& fltr_blk_handle,
+ bool no_io, GetContext* get_context,
+ BlockCacheLookupContext* lookup_context,
+ Env::IOPriority rate_limiter_priority,
+ CachableEntry<ParsedFullFilterBlock>* filter_block) const {
+ assert(table());
+ assert(filter_block);
+ assert(filter_block->IsEmpty());
+
+ if (!filter_map_.empty()) {
+ auto iter = filter_map_.find(fltr_blk_handle.offset());
+ // This is a possible scenario since block cache might not have had space
+ // for the partition
+ if (iter != filter_map_.end()) {
+ filter_block->SetUnownedValue(iter->second.GetValue());
+ return Status::OK();
+ }
+ }
+
+ ReadOptions read_options;
+ read_options.rate_limiter_priority = rate_limiter_priority;
+ if (no_io) {
+ read_options.read_tier = kBlockCacheTier;
+ }
+
+ const Status s =
+ table()->RetrieveBlock(prefetch_buffer, read_options, fltr_blk_handle,
+ UncompressionDict::GetEmptyDict(), filter_block,
+ BlockType::kFilter, get_context, lookup_context,
+ /* for_compaction */ false, /* use_cache */ true,
+ /* wait_for_cache */ true, /* async_read */ false);
+
+ return s;
+}
+
+bool PartitionedFilterBlockReader::MayMatch(
+ const Slice& slice, bool no_io, const Slice* const_ikey_ptr,
+ GetContext* get_context, BlockCacheLookupContext* lookup_context,
+ Env::IOPriority rate_limiter_priority,
+ FilterFunction filter_function) const {
+ CachableEntry<Block> filter_block;
+ Status s = GetOrReadFilterBlock(
+ no_io, get_context, lookup_context, &filter_block,
+ BlockType::kFilterPartitionIndex, rate_limiter_priority);
+ if (UNLIKELY(!s.ok())) {
+ IGNORE_STATUS_IF_ERROR(s);
+ return true;
+ }
+
+ if (UNLIKELY(filter_block.GetValue()->size() == 0)) {
+ return true;
+ }
+
+ auto filter_handle = GetFilterPartitionHandle(filter_block, *const_ikey_ptr);
+ if (UNLIKELY(filter_handle.size() == 0)) { // key is out of range
+ return false;
+ }
+
+ CachableEntry<ParsedFullFilterBlock> filter_partition_block;
+ s = GetFilterPartitionBlock(nullptr /* prefetch_buffer */, filter_handle,
+ no_io, get_context, lookup_context,
+ rate_limiter_priority, &filter_partition_block);
+ if (UNLIKELY(!s.ok())) {
+ IGNORE_STATUS_IF_ERROR(s);
+ return true;
+ }
+
+ FullFilterBlockReader filter_partition(table(),
+ std::move(filter_partition_block));
+ return (filter_partition.*filter_function)(slice, no_io, const_ikey_ptr,
+ get_context, lookup_context,
+ rate_limiter_priority);
+}
+
+void PartitionedFilterBlockReader::MayMatch(
+ MultiGetRange* range, const SliceTransform* prefix_extractor, bool no_io,
+ BlockCacheLookupContext* lookup_context,
+ Env::IOPriority rate_limiter_priority,
+ FilterManyFunction filter_function) const {
+ CachableEntry<Block> filter_block;
+ Status s = GetOrReadFilterBlock(
+ no_io, range->begin()->get_context, lookup_context, &filter_block,
+ BlockType::kFilterPartitionIndex, rate_limiter_priority);
+ if (UNLIKELY(!s.ok())) {
+ IGNORE_STATUS_IF_ERROR(s);
+ return; // Any/all may match
+ }
+
+ if (UNLIKELY(filter_block.GetValue()->size() == 0)) {
+ return; // Any/all may match
+ }
+
+ auto start_iter_same_handle = range->begin();
+ BlockHandle prev_filter_handle = BlockHandle::NullBlockHandle();
+
+ // For all keys mapping to same partition (must be adjacent in sorted order)
+ // share block cache lookup and use full filter multiget on the partition
+ // filter.
+ for (auto iter = start_iter_same_handle; iter != range->end(); ++iter) {
+ // TODO: re-use one top-level index iterator
+ BlockHandle this_filter_handle =
+ GetFilterPartitionHandle(filter_block, iter->ikey);
+ if (!prev_filter_handle.IsNull() &&
+ this_filter_handle != prev_filter_handle) {
+ MultiGetRange subrange(*range, start_iter_same_handle, iter);
+ MayMatchPartition(&subrange, prefix_extractor, prev_filter_handle, no_io,
+ lookup_context, rate_limiter_priority, filter_function);
+ range->AddSkipsFrom(subrange);
+ start_iter_same_handle = iter;
+ }
+ if (UNLIKELY(this_filter_handle.size() == 0)) { // key is out of range
+ // Not reachable with current behavior of GetFilterPartitionHandle
+ assert(false);
+ range->SkipKey(iter);
+ prev_filter_handle = BlockHandle::NullBlockHandle();
+ } else {
+ prev_filter_handle = this_filter_handle;
+ }
+ }
+ if (!prev_filter_handle.IsNull()) {
+ MultiGetRange subrange(*range, start_iter_same_handle, range->end());
+ MayMatchPartition(&subrange, prefix_extractor, prev_filter_handle, no_io,
+ lookup_context, rate_limiter_priority, filter_function);
+ range->AddSkipsFrom(subrange);
+ }
+}
+
+void PartitionedFilterBlockReader::MayMatchPartition(
+ MultiGetRange* range, const SliceTransform* prefix_extractor,
+ BlockHandle filter_handle, bool no_io,
+ BlockCacheLookupContext* lookup_context,
+ Env::IOPriority rate_limiter_priority,
+ FilterManyFunction filter_function) const {
+ CachableEntry<ParsedFullFilterBlock> filter_partition_block;
+ Status s = GetFilterPartitionBlock(
+ nullptr /* prefetch_buffer */, filter_handle, no_io,
+ range->begin()->get_context, lookup_context, rate_limiter_priority,
+ &filter_partition_block);
+ if (UNLIKELY(!s.ok())) {
+ IGNORE_STATUS_IF_ERROR(s);
+ return; // Any/all may match
+ }
+
+ FullFilterBlockReader filter_partition(table(),
+ std::move(filter_partition_block));
+ (filter_partition.*filter_function)(range, prefix_extractor, no_io,
+ lookup_context, rate_limiter_priority);
+}
+
+size_t PartitionedFilterBlockReader::ApproximateMemoryUsage() const {
+ size_t usage = ApproximateFilterBlockMemoryUsage();
+#ifdef ROCKSDB_MALLOC_USABLE_SIZE
+ usage += malloc_usable_size(const_cast<PartitionedFilterBlockReader*>(this));
+#else
+ usage += sizeof(*this);
+#endif // ROCKSDB_MALLOC_USABLE_SIZE
+ return usage;
+ // TODO(myabandeh): better estimation for filter_map_ size
+}
+
+// TODO(myabandeh): merge this with the same function in IndexReader
+Status PartitionedFilterBlockReader::CacheDependencies(const ReadOptions& ro,
+ bool pin) {
+ assert(table());
+
+ const BlockBasedTable::Rep* const rep = table()->get_rep();
+ assert(rep);
+
+ BlockCacheLookupContext lookup_context{TableReaderCaller::kPrefetch};
+
+ CachableEntry<Block> filter_block;
+
+ Status s = GetOrReadFilterBlock(false /* no_io */, nullptr /* get_context */,
+ &lookup_context, &filter_block,
+ BlockType::kFilterPartitionIndex,
+ ro.rate_limiter_priority);
+ if (!s.ok()) {
+ ROCKS_LOG_ERROR(rep->ioptions.logger,
+ "Error retrieving top-level filter block while trying to "
+ "cache filter partitions: %s",
+ s.ToString().c_str());
+ return s;
+ }
+
+ // Before read partitions, prefetch them to avoid lots of IOs
+ assert(filter_block.GetValue());
+
+ IndexBlockIter biter;
+ const InternalKeyComparator* const comparator = internal_comparator();
+ Statistics* kNullStats = nullptr;
+ filter_block.GetValue()->NewIndexIterator(
+ comparator->user_comparator(),
+ rep->get_global_seqno(BlockType::kFilterPartitionIndex), &biter,
+ kNullStats, true /* total_order_seek */, false /* have_first_key */,
+ index_key_includes_seq(), index_value_is_full());
+ // Index partitions are assumed to be consecuitive. Prefetch them all.
+ // Read the first block offset
+ biter.SeekToFirst();
+ BlockHandle handle = biter.value().handle;
+ uint64_t prefetch_off = handle.offset();
+
+ // Read the last block's offset
+ biter.SeekToLast();
+ handle = biter.value().handle;
+ uint64_t last_off =
+ handle.offset() + handle.size() + BlockBasedTable::kBlockTrailerSize;
+ uint64_t prefetch_len = last_off - prefetch_off;
+ std::unique_ptr<FilePrefetchBuffer> prefetch_buffer;
+ rep->CreateFilePrefetchBuffer(
+ 0, 0, &prefetch_buffer, false /* Implicit autoreadahead */,
+ 0 /*num_reads_*/, 0 /*num_file_reads_for_auto_readahead*/);
+
+ IOOptions opts;
+ s = rep->file->PrepareIOOptions(ro, opts);
+ if (s.ok()) {
+ s = prefetch_buffer->Prefetch(opts, rep->file.get(), prefetch_off,
+ static_cast<size_t>(prefetch_len),
+ ro.rate_limiter_priority);
+ }
+ if (!s.ok()) {
+ return s;
+ }
+
+ // After prefetch, read the partitions one by one
+ for (biter.SeekToFirst(); biter.Valid(); biter.Next()) {
+ handle = biter.value().handle;
+
+ CachableEntry<ParsedFullFilterBlock> block;
+ // TODO: Support counter batch update for partitioned index and
+ // filter blocks
+ s = table()->MaybeReadBlockAndLoadToCache(
+ prefetch_buffer.get(), ro, handle, UncompressionDict::GetEmptyDict(),
+ /* wait */ true, /* for_compaction */ false, &block, BlockType::kFilter,
+ nullptr /* get_context */, &lookup_context, nullptr /* contents */,
+ false);
+ if (!s.ok()) {
+ return s;
+ }
+ assert(s.ok() || block.GetValue() == nullptr);
+
+ if (block.GetValue() != nullptr) {
+ if (block.IsCached()) {
+ if (pin) {
+ filter_map_[handle.offset()] = std::move(block);
+ }
+ }
+ }
+ }
+ return biter.status();
+}
+
+const InternalKeyComparator* PartitionedFilterBlockReader::internal_comparator()
+ const {
+ assert(table());
+ assert(table()->get_rep());
+
+ return &table()->get_rep()->internal_comparator;
+}
+
+bool PartitionedFilterBlockReader::index_key_includes_seq() const {
+ assert(table());
+ assert(table()->get_rep());
+
+ return table()->get_rep()->index_key_includes_seq;
+}
+
+bool PartitionedFilterBlockReader::index_value_is_full() const {
+ assert(table());
+ assert(table()->get_rep());
+
+ return table()->get_rep()->index_value_is_full;
+}
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/partitioned_filter_block.h b/src/rocksdb/table/block_based/partitioned_filter_block.h
new file mode 100644
index 000000000..955b50739
--- /dev/null
+++ b/src/rocksdb/table/block_based/partitioned_filter_block.h
@@ -0,0 +1,178 @@
+// Copyright (c) 2011-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).
+
+#pragma once
+
+#include <deque>
+#include <list>
+#include <string>
+#include <unordered_map>
+
+#include "rocksdb/options.h"
+#include "rocksdb/slice.h"
+#include "rocksdb/slice_transform.h"
+#include "table/block_based/block.h"
+#include "table/block_based/filter_block_reader_common.h"
+#include "table/block_based/full_filter_block.h"
+#include "table/block_based/index_builder.h"
+#include "util/autovector.h"
+#include "util/hash_containers.h"
+
+namespace ROCKSDB_NAMESPACE {
+class InternalKeyComparator;
+
+class PartitionedFilterBlockBuilder : public FullFilterBlockBuilder {
+ public:
+ explicit PartitionedFilterBlockBuilder(
+ const SliceTransform* prefix_extractor, bool whole_key_filtering,
+ FilterBitsBuilder* filter_bits_builder, int index_block_restart_interval,
+ const bool use_value_delta_encoding,
+ PartitionedIndexBuilder* const p_index_builder,
+ const uint32_t partition_size);
+
+ virtual ~PartitionedFilterBlockBuilder();
+
+ void AddKey(const Slice& key) override;
+ void Add(const Slice& key) override;
+ size_t EstimateEntriesAdded() override;
+
+ virtual Slice Finish(
+ const BlockHandle& last_partition_block_handle, Status* status,
+ std::unique_ptr<const char[]>* filter_data = nullptr) override;
+
+ virtual void ResetFilterBitsBuilder() override {
+ // Previously constructed partitioned filters by
+ // this to-be-reset FiterBitsBuilder can also be
+ // cleared
+ filters.clear();
+ FullFilterBlockBuilder::ResetFilterBitsBuilder();
+ }
+
+ // For PartitionFilter, optional post-verifing the filter is done
+ // as part of PartitionFilterBlockBuilder::Finish
+ // to avoid implementation complexity of doing it elsewhere.
+ // Therefore we are skipping it in here.
+ virtual Status MaybePostVerifyFilter(
+ const Slice& /* filter_content */) override {
+ return Status::OK();
+ }
+
+ private:
+ // Filter data
+ BlockBuilder index_on_filter_block_builder_; // top-level index builder
+ BlockBuilder
+ index_on_filter_block_builder_without_seq_; // same for user keys
+ struct FilterEntry {
+ std::string key;
+ std::unique_ptr<const char[]> filter_data;
+ Slice filter;
+ };
+ std::deque<FilterEntry> filters; // list of partitioned filters and keys used
+ // in building the index
+
+ // Set to the first non-okay status if any of the filter
+ // partitions experiences construction error.
+ // If partitioned_filters_construction_status_ is non-okay,
+ // then the whole partitioned filters should not be used.
+ Status partitioned_filters_construction_status_;
+ std::string last_filter_entry_key;
+ std::unique_ptr<const char[]> last_filter_data;
+ std::unique_ptr<IndexBuilder> value;
+ bool finishing_filters =
+ false; // true if Finish is called once but not complete yet.
+ // The policy of when cut a filter block and Finish it
+ void MaybeCutAFilterBlock(const Slice* next_key);
+ // Currently we keep the same number of partitions for filters and indexes.
+ // This would allow for some potentioal optimizations in future. If such
+ // optimizations did not realize we can use different number of partitions and
+ // eliminate p_index_builder_
+ PartitionedIndexBuilder* const p_index_builder_;
+ // The desired number of keys per partition
+ uint32_t keys_per_partition_;
+ // The number of keys added to the last partition so far
+ uint32_t keys_added_to_partition_;
+ // According to the bits builders, how many keys/prefixes added
+ // in all the filters we have fully built
+ uint64_t total_added_in_built_;
+ BlockHandle last_encoded_handle_;
+};
+
+class PartitionedFilterBlockReader : public FilterBlockReaderCommon<Block> {
+ public:
+ PartitionedFilterBlockReader(const BlockBasedTable* t,
+ CachableEntry<Block>&& filter_block);
+
+ static std::unique_ptr<FilterBlockReader> Create(
+ const BlockBasedTable* table, const ReadOptions& ro,
+ FilePrefetchBuffer* prefetch_buffer, bool use_cache, bool prefetch,
+ bool pin, BlockCacheLookupContext* lookup_context);
+
+ bool KeyMayMatch(const Slice& key, const bool no_io,
+ const Slice* const const_ikey_ptr, GetContext* get_context,
+ BlockCacheLookupContext* lookup_context,
+ Env::IOPriority rate_limiter_priority) override;
+ void KeysMayMatch(MultiGetRange* range, const bool no_io,
+ BlockCacheLookupContext* lookup_context,
+ Env::IOPriority rate_limiter_priority) override;
+
+ bool PrefixMayMatch(const Slice& prefix, const bool no_io,
+ const Slice* const const_ikey_ptr,
+ GetContext* get_context,
+ BlockCacheLookupContext* lookup_context,
+ Env::IOPriority rate_limiter_priority) override;
+ void PrefixesMayMatch(MultiGetRange* range,
+ const SliceTransform* prefix_extractor,
+ const bool no_io,
+ BlockCacheLookupContext* lookup_context,
+ Env::IOPriority rate_limiter_priority) override;
+
+ size_t ApproximateMemoryUsage() const override;
+
+ private:
+ BlockHandle GetFilterPartitionHandle(const CachableEntry<Block>& filter_block,
+ const Slice& entry) const;
+ Status GetFilterPartitionBlock(
+ FilePrefetchBuffer* prefetch_buffer, const BlockHandle& handle,
+ bool no_io, GetContext* get_context,
+ BlockCacheLookupContext* lookup_context,
+ Env::IOPriority rate_limiter_priority,
+ CachableEntry<ParsedFullFilterBlock>* filter_block) const;
+
+ using FilterFunction = bool (FullFilterBlockReader::*)(
+ const Slice& slice, const bool no_io, const Slice* const const_ikey_ptr,
+ GetContext* get_context, BlockCacheLookupContext* lookup_context,
+ Env::IOPriority rate_limiter_priority);
+ bool MayMatch(const Slice& slice, bool no_io, const Slice* const_ikey_ptr,
+ GetContext* get_context,
+ BlockCacheLookupContext* lookup_context,
+ Env::IOPriority rate_limiter_priority,
+ FilterFunction filter_function) const;
+ using FilterManyFunction = void (FullFilterBlockReader::*)(
+ MultiGetRange* range, const SliceTransform* prefix_extractor,
+ const bool no_io, BlockCacheLookupContext* lookup_context,
+ Env::IOPriority rate_limiter_priority);
+ void MayMatch(MultiGetRange* range, const SliceTransform* prefix_extractor,
+ bool no_io, BlockCacheLookupContext* lookup_context,
+ Env::IOPriority rate_limiter_priority,
+ FilterManyFunction filter_function) const;
+ void MayMatchPartition(MultiGetRange* range,
+ const SliceTransform* prefix_extractor,
+ BlockHandle filter_handle, bool no_io,
+ BlockCacheLookupContext* lookup_context,
+ Env::IOPriority rate_limiter_priority,
+ FilterManyFunction filter_function) const;
+ Status CacheDependencies(const ReadOptions& ro, bool pin) override;
+
+ const InternalKeyComparator* internal_comparator() const;
+ bool index_key_includes_seq() const;
+ bool index_value_is_full() const;
+
+ protected:
+ // For partition blocks pinned in cache. Can be a subset of blocks
+ // in case some fail insertion on attempt to pin.
+ UnorderedMap<uint64_t, CachableEntry<ParsedFullFilterBlock>> filter_map_;
+};
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/partitioned_filter_block_test.cc b/src/rocksdb/table/block_based/partitioned_filter_block_test.cc
new file mode 100644
index 000000000..0ce50d2bc
--- /dev/null
+++ b/src/rocksdb/table/block_based/partitioned_filter_block_test.cc
@@ -0,0 +1,436 @@
+// Copyright (c) 2011-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 "table/block_based/partitioned_filter_block.h"
+
+#include <map>
+
+#include "index_builder.h"
+#include "rocksdb/filter_policy.h"
+#include "table/block_based/block_based_table_reader.h"
+#include "table/block_based/filter_policy_internal.h"
+#include "table/format.h"
+#include "test_util/testharness.h"
+#include "test_util/testutil.h"
+#include "util/coding.h"
+#include "util/hash.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+std::map<uint64_t, std::string> blooms;
+
+class MockedBlockBasedTable : public BlockBasedTable {
+ public:
+ MockedBlockBasedTable(Rep* rep, PartitionedIndexBuilder* pib)
+ : BlockBasedTable(rep, /*block_cache_tracer=*/nullptr) {
+ // Initialize what Open normally does as much as necessary for the test
+ rep->index_key_includes_seq = pib->seperator_is_key_plus_seq();
+ rep->index_value_is_full = !pib->get_use_value_delta_encoding();
+ }
+};
+
+class MyPartitionedFilterBlockReader : public PartitionedFilterBlockReader {
+ public:
+ MyPartitionedFilterBlockReader(BlockBasedTable* t,
+ CachableEntry<Block>&& filter_block)
+ : PartitionedFilterBlockReader(t, std::move(filter_block)) {
+ for (const auto& pair : blooms) {
+ const uint64_t offset = pair.first;
+ const std::string& bloom = pair.second;
+
+ assert(t);
+ assert(t->get_rep());
+ CachableEntry<ParsedFullFilterBlock> block(
+ new ParsedFullFilterBlock(
+ t->get_rep()->table_options.filter_policy.get(),
+ BlockContents(Slice(bloom))),
+ nullptr /* cache */, nullptr /* cache_handle */,
+ true /* own_value */);
+ filter_map_[offset] = std::move(block);
+ }
+ }
+};
+
+class PartitionedFilterBlockTest
+ : public testing::Test,
+ virtual public ::testing::WithParamInterface<uint32_t> {
+ public:
+ Options options_;
+ ImmutableOptions ioptions_;
+ EnvOptions env_options_;
+ BlockBasedTableOptions table_options_;
+ InternalKeyComparator icomp_;
+ std::unique_ptr<BlockBasedTable> table_;
+ std::shared_ptr<Cache> cache_;
+ int bits_per_key_;
+
+ PartitionedFilterBlockTest()
+ : ioptions_(options_),
+ env_options_(options_),
+ icomp_(options_.comparator),
+ bits_per_key_(10) {
+ table_options_.filter_policy.reset(
+ NewBloomFilterPolicy(bits_per_key_, false));
+ table_options_.format_version = GetParam();
+ table_options_.index_block_restart_interval = 3;
+ }
+
+ ~PartitionedFilterBlockTest() override {}
+
+ const std::string keys[4] = {"afoo", "bar", "box", "hello"};
+ const std::string missing_keys[2] = {"missing", "other"};
+
+ uint64_t MaxIndexSize() {
+ int num_keys = sizeof(keys) / sizeof(*keys);
+ uint64_t max_key_size = 0;
+ for (int i = 1; i < num_keys; i++) {
+ max_key_size =
+ std::max(max_key_size, static_cast<uint64_t>(keys[i].size()));
+ }
+ uint64_t max_index_size = num_keys * (max_key_size + 8 /*handle*/);
+ return max_index_size;
+ }
+
+ uint64_t MaxFilterSize() {
+ int num_keys = sizeof(keys) / sizeof(*keys);
+ // General, rough over-approximation
+ return num_keys * bits_per_key_ + (CACHE_LINE_SIZE * 8 + /*metadata*/ 5);
+ }
+
+ uint64_t last_offset = 10;
+ BlockHandle Write(const Slice& slice) {
+ BlockHandle bh(last_offset + 1, slice.size());
+ blooms[bh.offset()] = slice.ToString();
+ last_offset += bh.size();
+ return bh;
+ }
+
+ PartitionedIndexBuilder* NewIndexBuilder() {
+ const bool kValueDeltaEncoded = true;
+ return PartitionedIndexBuilder::CreateIndexBuilder(
+ &icomp_, !kValueDeltaEncoded, table_options_);
+ }
+
+ PartitionedFilterBlockBuilder* NewBuilder(
+ PartitionedIndexBuilder* const p_index_builder,
+ const SliceTransform* prefix_extractor = nullptr) {
+ assert(table_options_.block_size_deviation <= 100);
+ auto partition_size =
+ static_cast<uint32_t>(((table_options_.metadata_block_size *
+ (100 - table_options_.block_size_deviation)) +
+ 99) /
+ 100);
+ partition_size = std::max(partition_size, static_cast<uint32_t>(1));
+ const bool kValueDeltaEncoded = true;
+ return new PartitionedFilterBlockBuilder(
+ prefix_extractor, table_options_.whole_key_filtering,
+ BloomFilterPolicy::GetBuilderFromContext(
+ FilterBuildingContext(table_options_)),
+ table_options_.index_block_restart_interval, !kValueDeltaEncoded,
+ p_index_builder, partition_size);
+ }
+
+ PartitionedFilterBlockReader* NewReader(
+ PartitionedFilterBlockBuilder* builder, PartitionedIndexBuilder* pib) {
+ BlockHandle bh;
+ Status status;
+ Slice slice;
+ std::unique_ptr<const char[]> filter_data;
+ do {
+ slice = builder->Finish(bh, &status, &filter_data);
+ bh = Write(slice);
+ } while (status.IsIncomplete());
+
+ constexpr bool skip_filters = false;
+ constexpr uint64_t file_size = 12345;
+ constexpr int level = 0;
+ constexpr bool immortal_table = false;
+ table_.reset(new MockedBlockBasedTable(
+ new BlockBasedTable::Rep(ioptions_, env_options_, table_options_,
+ icomp_, skip_filters, file_size, level,
+ immortal_table),
+ pib));
+ BlockContents contents(slice);
+ CachableEntry<Block> block(
+ new Block(std::move(contents), 0 /* read_amp_bytes_per_bit */, nullptr),
+ nullptr /* cache */, nullptr /* cache_handle */, true /* own_value */);
+ auto reader =
+ new MyPartitionedFilterBlockReader(table_.get(), std::move(block));
+ return reader;
+ }
+
+ void VerifyReader(PartitionedFilterBlockBuilder* builder,
+ PartitionedIndexBuilder* pib, bool empty = false) {
+ std::unique_ptr<PartitionedFilterBlockReader> reader(
+ NewReader(builder, pib));
+ Env::IOPriority rate_limiter_priority = Env::IO_TOTAL;
+ // Querying added keys
+ const bool no_io = true;
+ for (auto key : keys) {
+ auto ikey = InternalKey(key, 0, ValueType::kTypeValue);
+ const Slice ikey_slice = Slice(*ikey.rep());
+ ASSERT_TRUE(reader->KeyMayMatch(key, !no_io, &ikey_slice,
+ /*get_context=*/nullptr,
+ /*lookup_context=*/nullptr,
+ rate_limiter_priority));
+ }
+ {
+ // querying a key twice
+ auto ikey = InternalKey(keys[0], 0, ValueType::kTypeValue);
+ const Slice ikey_slice = Slice(*ikey.rep());
+ ASSERT_TRUE(reader->KeyMayMatch(keys[0], !no_io, &ikey_slice,
+ /*get_context=*/nullptr,
+ /*lookup_context=*/nullptr,
+ rate_limiter_priority));
+ }
+ // querying missing keys
+ for (auto key : missing_keys) {
+ auto ikey = InternalKey(key, 0, ValueType::kTypeValue);
+ const Slice ikey_slice = Slice(*ikey.rep());
+ if (empty) {
+ ASSERT_TRUE(reader->KeyMayMatch(key, !no_io, &ikey_slice,
+ /*get_context=*/nullptr,
+ /*lookup_context=*/nullptr,
+ rate_limiter_priority));
+ } else {
+ // assuming a good hash function
+ ASSERT_FALSE(reader->KeyMayMatch(key, !no_io, &ikey_slice,
+ /*get_context=*/nullptr,
+ /*lookup_context=*/nullptr,
+ rate_limiter_priority));
+ }
+ }
+ }
+
+ int TestBlockPerKey() {
+ std::unique_ptr<PartitionedIndexBuilder> pib(NewIndexBuilder());
+ std::unique_ptr<PartitionedFilterBlockBuilder> builder(
+ NewBuilder(pib.get()));
+ int i = 0;
+ builder->Add(keys[i]);
+ CutABlock(pib.get(), keys[i], keys[i + 1]);
+ i++;
+ builder->Add(keys[i]);
+ CutABlock(pib.get(), keys[i], keys[i + 1]);
+ i++;
+ builder->Add(keys[i]);
+ builder->Add(keys[i]);
+ CutABlock(pib.get(), keys[i], keys[i + 1]);
+ i++;
+ builder->Add(keys[i]);
+ CutABlock(pib.get(), keys[i]);
+
+ VerifyReader(builder.get(), pib.get());
+ return CountNumOfIndexPartitions(pib.get());
+ }
+
+ void TestBlockPerTwoKeys(const SliceTransform* prefix_extractor = nullptr) {
+ std::unique_ptr<PartitionedIndexBuilder> pib(NewIndexBuilder());
+ std::unique_ptr<PartitionedFilterBlockBuilder> builder(
+ NewBuilder(pib.get(), prefix_extractor));
+ int i = 0;
+ builder->Add(keys[i]);
+ i++;
+ builder->Add(keys[i]);
+ CutABlock(pib.get(), keys[i], keys[i + 1]);
+ i++;
+ builder->Add(keys[i]);
+ builder->Add(keys[i]);
+ i++;
+ builder->Add(keys[i]);
+ CutABlock(pib.get(), keys[i]);
+
+ VerifyReader(builder.get(), pib.get(), prefix_extractor);
+ }
+
+ void TestBlockPerAllKeys() {
+ std::unique_ptr<PartitionedIndexBuilder> pib(NewIndexBuilder());
+ std::unique_ptr<PartitionedFilterBlockBuilder> builder(
+ NewBuilder(pib.get()));
+ int i = 0;
+ builder->Add(keys[i]);
+ i++;
+ builder->Add(keys[i]);
+ i++;
+ builder->Add(keys[i]);
+ builder->Add(keys[i]);
+ i++;
+ builder->Add(keys[i]);
+ CutABlock(pib.get(), keys[i]);
+
+ VerifyReader(builder.get(), pib.get());
+ }
+
+ void CutABlock(PartitionedIndexBuilder* builder,
+ const std::string& user_key) {
+ // Assuming a block is cut, add an entry to the index
+ std::string key =
+ std::string(*InternalKey(user_key, 0, ValueType::kTypeValue).rep());
+ BlockHandle dont_care_block_handle(1, 1);
+ builder->AddIndexEntry(&key, nullptr, dont_care_block_handle);
+ }
+
+ void CutABlock(PartitionedIndexBuilder* builder, const std::string& user_key,
+ const std::string& next_user_key) {
+ // Assuming a block is cut, add an entry to the index
+ std::string key =
+ std::string(*InternalKey(user_key, 0, ValueType::kTypeValue).rep());
+ std::string next_key = std::string(
+ *InternalKey(next_user_key, 0, ValueType::kTypeValue).rep());
+ BlockHandle dont_care_block_handle(1, 1);
+ Slice slice = Slice(next_key.data(), next_key.size());
+ builder->AddIndexEntry(&key, &slice, dont_care_block_handle);
+ }
+
+ int CountNumOfIndexPartitions(PartitionedIndexBuilder* builder) {
+ IndexBuilder::IndexBlocks dont_care_ib;
+ BlockHandle dont_care_bh(10, 10);
+ Status s;
+ int cnt = 0;
+ do {
+ s = builder->Finish(&dont_care_ib, dont_care_bh);
+ cnt++;
+ } while (s.IsIncomplete());
+ return cnt - 1; // 1 is 2nd level index
+ }
+};
+
+// Format versions potentially intersting to partitioning
+INSTANTIATE_TEST_CASE_P(FormatVersions, PartitionedFilterBlockTest,
+ testing::ValuesIn(std::set<uint32_t>{
+ 2, 3, 4, test::kDefaultFormatVersion,
+ kLatestFormatVersion}));
+
+TEST_P(PartitionedFilterBlockTest, EmptyBuilder) {
+ std::unique_ptr<PartitionedIndexBuilder> pib(NewIndexBuilder());
+ std::unique_ptr<PartitionedFilterBlockBuilder> builder(NewBuilder(pib.get()));
+ const bool empty = true;
+ VerifyReader(builder.get(), pib.get(), empty);
+}
+
+TEST_P(PartitionedFilterBlockTest, OneBlock) {
+ uint64_t max_index_size = MaxIndexSize();
+ for (uint64_t i = 1; i < max_index_size + 1; i++) {
+ table_options_.metadata_block_size = i;
+ TestBlockPerAllKeys();
+ }
+}
+
+TEST_P(PartitionedFilterBlockTest, TwoBlocksPerKey) {
+ uint64_t max_index_size = MaxIndexSize();
+ for (uint64_t i = 1; i < max_index_size + 1; i++) {
+ table_options_.metadata_block_size = i;
+ TestBlockPerTwoKeys();
+ }
+}
+
+// This reproduces the bug that a prefix is the same among multiple consecutive
+// blocks but the bug would add it only to the first block.
+TEST_P(PartitionedFilterBlockTest, SamePrefixInMultipleBlocks) {
+ // some small number to cause partition cuts
+ table_options_.metadata_block_size = 1;
+ std::unique_ptr<const SliceTransform> prefix_extractor(
+ ROCKSDB_NAMESPACE::NewFixedPrefixTransform(1));
+ std::unique_ptr<PartitionedIndexBuilder> pib(NewIndexBuilder());
+ std::unique_ptr<PartitionedFilterBlockBuilder> builder(
+ NewBuilder(pib.get(), prefix_extractor.get()));
+ const std::string pkeys[3] = {"p-key10", "p-key20", "p-key30"};
+ builder->Add(pkeys[0]);
+ CutABlock(pib.get(), pkeys[0], pkeys[1]);
+ builder->Add(pkeys[1]);
+ CutABlock(pib.get(), pkeys[1], pkeys[2]);
+ builder->Add(pkeys[2]);
+ CutABlock(pib.get(), pkeys[2]);
+ std::unique_ptr<PartitionedFilterBlockReader> reader(
+ NewReader(builder.get(), pib.get()));
+ for (auto key : pkeys) {
+ auto ikey = InternalKey(key, 0, ValueType::kTypeValue);
+ const Slice ikey_slice = Slice(*ikey.rep());
+ ASSERT_TRUE(reader->PrefixMayMatch(prefix_extractor->Transform(key),
+ /*no_io=*/false, &ikey_slice,
+ /*get_context=*/nullptr,
+ /*lookup_context=*/nullptr,
+ Env::IO_TOTAL));
+ }
+ // Non-existent keys but with the same prefix
+ const std::string pnonkeys[4] = {"p-key9", "p-key11", "p-key21", "p-key31"};
+ for (auto key : pnonkeys) {
+ auto ikey = InternalKey(key, 0, ValueType::kTypeValue);
+ const Slice ikey_slice = Slice(*ikey.rep());
+ ASSERT_TRUE(reader->PrefixMayMatch(prefix_extractor->Transform(key),
+ /*no_io=*/false, &ikey_slice,
+ /*get_context=*/nullptr,
+ /*lookup_context=*/nullptr,
+ Env::IO_TOTAL));
+ }
+}
+
+// This reproduces the bug in format_version=3 that the seeking the prefix will
+// lead us to the partition before the one that has filter for the prefix.
+TEST_P(PartitionedFilterBlockTest, PrefixInWrongPartitionBug) {
+ // some small number to cause partition cuts
+ table_options_.metadata_block_size = 1;
+ std::unique_ptr<const SliceTransform> prefix_extractor(
+ ROCKSDB_NAMESPACE::NewFixedPrefixTransform(2));
+ std::unique_ptr<PartitionedIndexBuilder> pib(NewIndexBuilder());
+ std::unique_ptr<PartitionedFilterBlockBuilder> builder(
+ NewBuilder(pib.get(), prefix_extractor.get()));
+ // In the bug, searching for prefix "p3" on an index with format version 3,
+ // will give the key "p3" and the partition of the keys that are <= p3, i.e.,
+ // p2-keys, where the filter for prefix "p3" does not exist.
+ const std::string pkeys[] = {"p1-key1", "p2-key2", "p3-key3", "p4-key3",
+ "p5-key3"};
+ builder->Add(pkeys[0]);
+ CutABlock(pib.get(), pkeys[0], pkeys[1]);
+ builder->Add(pkeys[1]);
+ CutABlock(pib.get(), pkeys[1], pkeys[2]);
+ builder->Add(pkeys[2]);
+ CutABlock(pib.get(), pkeys[2], pkeys[3]);
+ builder->Add(pkeys[3]);
+ CutABlock(pib.get(), pkeys[3], pkeys[4]);
+ builder->Add(pkeys[4]);
+ CutABlock(pib.get(), pkeys[4]);
+ std::unique_ptr<PartitionedFilterBlockReader> reader(
+ NewReader(builder.get(), pib.get()));
+ Env::IOPriority rate_limiter_priority = Env::IO_TOTAL;
+ for (auto key : pkeys) {
+ auto prefix = prefix_extractor->Transform(key);
+ auto ikey = InternalKey(prefix, 0, ValueType::kTypeValue);
+ const Slice ikey_slice = Slice(*ikey.rep());
+ ASSERT_TRUE(reader->PrefixMayMatch(prefix,
+ /*no_io=*/false, &ikey_slice,
+ /*get_context=*/nullptr,
+ /*lookup_context=*/nullptr,
+ rate_limiter_priority));
+ }
+}
+
+TEST_P(PartitionedFilterBlockTest, OneBlockPerKey) {
+ uint64_t max_index_size = MaxIndexSize();
+ for (uint64_t i = 1; i < max_index_size + 1; i++) {
+ table_options_.metadata_block_size = i;
+ TestBlockPerKey();
+ }
+}
+
+TEST_P(PartitionedFilterBlockTest, PartitionCount) {
+ int num_keys = sizeof(keys) / sizeof(*keys);
+ table_options_.metadata_block_size =
+ std::max(MaxIndexSize(), MaxFilterSize());
+ int partitions = TestBlockPerKey();
+ ASSERT_EQ(partitions, 1);
+ // A low number ensures cutting a block after each key
+ table_options_.metadata_block_size = 1;
+ partitions = TestBlockPerKey();
+ ASSERT_EQ(partitions, num_keys - 1 /* last two keys make one flush */);
+}
+
+} // namespace ROCKSDB_NAMESPACE
+
+int main(int argc, char** argv) {
+ ROCKSDB_NAMESPACE::port::InstallStackTraceHandler();
+ ::testing::InitGoogleTest(&argc, argv);
+ return RUN_ALL_TESTS();
+}
diff --git a/src/rocksdb/table/block_based/partitioned_index_iterator.cc b/src/rocksdb/table/block_based/partitioned_index_iterator.cc
new file mode 100644
index 000000000..b9bc2155a
--- /dev/null
+++ b/src/rocksdb/table/block_based/partitioned_index_iterator.cc
@@ -0,0 +1,163 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+#include "table/block_based/partitioned_index_iterator.h"
+
+namespace ROCKSDB_NAMESPACE {
+void PartitionedIndexIterator::Seek(const Slice& target) { SeekImpl(&target); }
+
+void PartitionedIndexIterator::SeekToFirst() { SeekImpl(nullptr); }
+
+void PartitionedIndexIterator::SeekImpl(const Slice* target) {
+ SavePrevIndexValue();
+
+ if (target) {
+ index_iter_->Seek(*target);
+ } else {
+ index_iter_->SeekToFirst();
+ }
+
+ if (!index_iter_->Valid()) {
+ ResetPartitionedIndexIter();
+ return;
+ }
+
+ InitPartitionedIndexBlock();
+
+ if (target) {
+ block_iter_.Seek(*target);
+ } else {
+ block_iter_.SeekToFirst();
+ }
+ FindKeyForward();
+
+ // We could check upper bound here, but that would be too complicated
+ // and checking index upper bound is less useful than for data blocks.
+
+ if (target) {
+ assert(!Valid() || (table_->get_rep()->index_key_includes_seq
+ ? (icomp_.Compare(*target, key()) <= 0)
+ : (user_comparator_.Compare(ExtractUserKey(*target),
+ key()) <= 0)));
+ }
+}
+
+void PartitionedIndexIterator::SeekToLast() {
+ SavePrevIndexValue();
+ index_iter_->SeekToLast();
+ if (!index_iter_->Valid()) {
+ ResetPartitionedIndexIter();
+ return;
+ }
+ InitPartitionedIndexBlock();
+ block_iter_.SeekToLast();
+ FindKeyBackward();
+}
+
+void PartitionedIndexIterator::Next() {
+ assert(block_iter_points_to_real_block_);
+ block_iter_.Next();
+ FindKeyForward();
+}
+
+void PartitionedIndexIterator::Prev() {
+ assert(block_iter_points_to_real_block_);
+ block_iter_.Prev();
+
+ FindKeyBackward();
+}
+
+void PartitionedIndexIterator::InitPartitionedIndexBlock() {
+ BlockHandle partitioned_index_handle = index_iter_->value().handle;
+ if (!block_iter_points_to_real_block_ ||
+ partitioned_index_handle.offset() != prev_block_offset_ ||
+ // if previous attempt of reading the block missed cache, try again
+ block_iter_.status().IsIncomplete()) {
+ if (block_iter_points_to_real_block_) {
+ ResetPartitionedIndexIter();
+ }
+ auto* rep = table_->get_rep();
+ bool is_for_compaction =
+ lookup_context_.caller == TableReaderCaller::kCompaction;
+ // Prefetch additional data for range scans (iterators).
+ // Implicit auto readahead:
+ // Enabled after 2 sequential IOs when ReadOptions.readahead_size == 0.
+ // Explicit user requested readahead:
+ // Enabled from the very first IO when ReadOptions.readahead_size is set.
+ block_prefetcher_.PrefetchIfNeeded(
+ rep, partitioned_index_handle, read_options_.readahead_size,
+ is_for_compaction, /*no_sequential_checking=*/false,
+ read_options_.rate_limiter_priority);
+ Status s;
+ table_->NewDataBlockIterator<IndexBlockIter>(
+ read_options_, partitioned_index_handle, &block_iter_,
+ BlockType::kIndex,
+ /*get_context=*/nullptr, &lookup_context_,
+ block_prefetcher_.prefetch_buffer(),
+ /*for_compaction=*/is_for_compaction, /*async_read=*/false, s);
+ block_iter_points_to_real_block_ = true;
+ // We could check upper bound here but it is complicated to reason about
+ // upper bound in index iterator. On the other than, in large scans, index
+ // iterators are moved much less frequently compared to data blocks. So
+ // the upper bound check is skipped for simplicity.
+ }
+}
+
+void PartitionedIndexIterator::FindKeyForward() {
+ // This method's code is kept short to make it likely to be inlined.
+
+ assert(block_iter_points_to_real_block_);
+
+ if (!block_iter_.Valid()) {
+ // This is the only call site of FindBlockForward(), but it's extracted into
+ // a separate method to keep FindKeyForward() short and likely to be
+ // inlined. When transitioning to a different block, we call
+ // FindBlockForward(), which is much longer and is probably not inlined.
+ FindBlockForward();
+ } else {
+ // This is the fast path that avoids a function call.
+ }
+}
+
+void PartitionedIndexIterator::FindBlockForward() {
+ // TODO the while loop inherits from two-level-iterator. We don't know
+ // whether a block can be empty so it can be replaced by an "if".
+ do {
+ if (!block_iter_.status().ok()) {
+ return;
+ }
+ ResetPartitionedIndexIter();
+ index_iter_->Next();
+
+ if (!index_iter_->Valid()) {
+ return;
+ }
+
+ InitPartitionedIndexBlock();
+ block_iter_.SeekToFirst();
+ } while (!block_iter_.Valid());
+}
+
+void PartitionedIndexIterator::FindKeyBackward() {
+ while (!block_iter_.Valid()) {
+ if (!block_iter_.status().ok()) {
+ return;
+ }
+
+ ResetPartitionedIndexIter();
+ index_iter_->Prev();
+
+ if (index_iter_->Valid()) {
+ InitPartitionedIndexBlock();
+ block_iter_.SeekToLast();
+ } else {
+ return;
+ }
+ }
+}
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/partitioned_index_iterator.h b/src/rocksdb/table/block_based/partitioned_index_iterator.h
new file mode 100644
index 000000000..6412fe239
--- /dev/null
+++ b/src/rocksdb/table/block_based/partitioned_index_iterator.h
@@ -0,0 +1,160 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+#pragma once
+#include "table/block_based/block_based_table_reader.h"
+#include "table/block_based/block_based_table_reader_impl.h"
+#include "table/block_based/block_prefetcher.h"
+#include "table/block_based/reader_common.h"
+
+namespace ROCKSDB_NAMESPACE {
+// Iterator that iterates over partitioned index.
+// Some upper and lower bound tricks played in block based table iterators
+// could be played here, but it's too complicated to reason about index
+// keys with upper or lower bound, so we skip it for simplicity.
+class PartitionedIndexIterator : public InternalIteratorBase<IndexValue> {
+ // compaction_readahead_size: its value will only be used if for_compaction =
+ // true
+ public:
+ PartitionedIndexIterator(
+ const BlockBasedTable* table, const ReadOptions& read_options,
+ const InternalKeyComparator& icomp,
+ std::unique_ptr<InternalIteratorBase<IndexValue>>&& index_iter,
+ TableReaderCaller caller, size_t compaction_readahead_size = 0)
+ : index_iter_(std::move(index_iter)),
+ table_(table),
+ read_options_(read_options),
+#ifndef NDEBUG
+ icomp_(icomp),
+#endif
+ user_comparator_(icomp.user_comparator()),
+ block_iter_points_to_real_block_(false),
+ lookup_context_(caller),
+ block_prefetcher_(
+ compaction_readahead_size,
+ table_->get_rep()->table_options.initial_auto_readahead_size) {
+ }
+
+ ~PartitionedIndexIterator() override {}
+
+ void Seek(const Slice& target) override;
+ void SeekForPrev(const Slice&) override {
+ // Shouldn't be called.
+ assert(false);
+ }
+ void SeekToFirst() override;
+ void SeekToLast() override;
+ void Next() final override;
+ bool NextAndGetResult(IterateResult*) override {
+ assert(false);
+ return false;
+ }
+ void Prev() override;
+ bool Valid() const override {
+ return block_iter_points_to_real_block_ && block_iter_.Valid();
+ }
+ Slice key() const override {
+ assert(Valid());
+ return block_iter_.key();
+ }
+ Slice user_key() const override {
+ assert(Valid());
+ return block_iter_.user_key();
+ }
+ IndexValue value() const override {
+ assert(Valid());
+ return block_iter_.value();
+ }
+ Status status() const override {
+ // Prefix index set status to NotFound when the prefix does not exist
+ if (!index_iter_->status().ok() && !index_iter_->status().IsNotFound()) {
+ return index_iter_->status();
+ } else if (block_iter_points_to_real_block_) {
+ return block_iter_.status();
+ } else {
+ return Status::OK();
+ }
+ }
+ inline IterBoundCheck UpperBoundCheckResult() override {
+ // Shouldn't be called.
+ assert(false);
+ return IterBoundCheck::kUnknown;
+ }
+ void SetPinnedItersMgr(PinnedIteratorsManager*) override {
+ // Shouldn't be called.
+ assert(false);
+ }
+ bool IsKeyPinned() const override {
+ // Shouldn't be called.
+ assert(false);
+ return false;
+ }
+ bool IsValuePinned() const override {
+ // Shouldn't be called.
+ assert(false);
+ return false;
+ }
+
+ void ResetPartitionedIndexIter() {
+ if (block_iter_points_to_real_block_) {
+ block_iter_.Invalidate(Status::OK());
+ block_iter_points_to_real_block_ = false;
+ }
+ }
+
+ void SavePrevIndexValue() {
+ if (block_iter_points_to_real_block_) {
+ // Reseek. If they end up with the same data block, we shouldn't re-fetch
+ // the same data block.
+ prev_block_offset_ = index_iter_->value().handle.offset();
+ }
+ }
+
+ void GetReadaheadState(ReadaheadFileInfo* readahead_file_info) override {
+ if (block_prefetcher_.prefetch_buffer() != nullptr &&
+ read_options_.adaptive_readahead) {
+ block_prefetcher_.prefetch_buffer()->GetReadaheadState(
+ &(readahead_file_info->index_block_readahead_info));
+ }
+ }
+
+ void SetReadaheadState(ReadaheadFileInfo* readahead_file_info) override {
+ if (read_options_.adaptive_readahead) {
+ block_prefetcher_.SetReadaheadState(
+ &(readahead_file_info->index_block_readahead_info));
+ }
+ }
+
+ std::unique_ptr<InternalIteratorBase<IndexValue>> index_iter_;
+
+ private:
+ friend class BlockBasedTableReaderTestVerifyChecksum_ChecksumMismatch_Test;
+ const BlockBasedTable* table_;
+ const ReadOptions read_options_;
+#ifndef NDEBUG
+ const InternalKeyComparator& icomp_;
+#endif
+ UserComparatorWrapper user_comparator_;
+ IndexBlockIter block_iter_;
+
+ // True if block_iter_ is initialized and points to the same block
+ // as index iterator.
+ bool block_iter_points_to_real_block_;
+ uint64_t prev_block_offset_ = std::numeric_limits<uint64_t>::max();
+ BlockCacheLookupContext lookup_context_;
+ BlockPrefetcher block_prefetcher_;
+
+ // If `target` is null, seek to first.
+ void SeekImpl(const Slice* target);
+
+ void InitPartitionedIndexBlock();
+ void FindKeyForward();
+ void FindBlockForward();
+ void FindKeyBackward();
+};
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/partitioned_index_reader.cc b/src/rocksdb/table/block_based/partitioned_index_reader.cc
new file mode 100644
index 000000000..017ea4a3a
--- /dev/null
+++ b/src/rocksdb/table/block_based/partitioned_index_reader.cc
@@ -0,0 +1,215 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+#include "table/block_based/partitioned_index_reader.h"
+
+#include "file/random_access_file_reader.h"
+#include "table/block_based/block_based_table_reader.h"
+#include "table/block_based/partitioned_index_iterator.h"
+
+namespace ROCKSDB_NAMESPACE {
+Status PartitionIndexReader::Create(
+ const BlockBasedTable* table, const ReadOptions& ro,
+ FilePrefetchBuffer* prefetch_buffer, bool use_cache, bool prefetch,
+ bool pin, BlockCacheLookupContext* lookup_context,
+ std::unique_ptr<IndexReader>* index_reader) {
+ assert(table != nullptr);
+ assert(table->get_rep());
+ assert(!pin || prefetch);
+ assert(index_reader != nullptr);
+
+ CachableEntry<Block> index_block;
+ if (prefetch || !use_cache) {
+ const Status s =
+ ReadIndexBlock(table, prefetch_buffer, ro, use_cache,
+ /*get_context=*/nullptr, lookup_context, &index_block);
+ if (!s.ok()) {
+ return s;
+ }
+
+ if (use_cache && !pin) {
+ index_block.Reset();
+ }
+ }
+
+ index_reader->reset(new PartitionIndexReader(table, std::move(index_block)));
+
+ return Status::OK();
+}
+
+InternalIteratorBase<IndexValue>* PartitionIndexReader::NewIterator(
+ const ReadOptions& read_options, bool /* disable_prefix_seek */,
+ IndexBlockIter* iter, GetContext* get_context,
+ BlockCacheLookupContext* lookup_context) {
+ const bool no_io = (read_options.read_tier == kBlockCacheTier);
+ CachableEntry<Block> index_block;
+ const Status s =
+ GetOrReadIndexBlock(no_io, read_options.rate_limiter_priority,
+ get_context, lookup_context, &index_block);
+ if (!s.ok()) {
+ if (iter != nullptr) {
+ iter->Invalidate(s);
+ return iter;
+ }
+
+ return NewErrorInternalIterator<IndexValue>(s);
+ }
+
+ const BlockBasedTable::Rep* rep = table()->rep_;
+ InternalIteratorBase<IndexValue>* it = nullptr;
+
+ Statistics* kNullStats = nullptr;
+ // Filters are already checked before seeking the index
+ if (!partition_map_.empty()) {
+ // We don't return pinned data from index blocks, so no need
+ // to set `block_contents_pinned`.
+ it = NewTwoLevelIterator(
+ new BlockBasedTable::PartitionedIndexIteratorState(table(),
+ &partition_map_),
+ index_block.GetValue()->NewIndexIterator(
+ internal_comparator()->user_comparator(),
+ rep->get_global_seqno(BlockType::kIndex), nullptr, kNullStats, true,
+ index_has_first_key(), index_key_includes_seq(),
+ index_value_is_full()));
+ } else {
+ ReadOptions ro;
+ ro.fill_cache = read_options.fill_cache;
+ ro.deadline = read_options.deadline;
+ ro.io_timeout = read_options.io_timeout;
+ ro.adaptive_readahead = read_options.adaptive_readahead;
+ ro.async_io = read_options.async_io;
+ ro.rate_limiter_priority = read_options.rate_limiter_priority;
+
+ // We don't return pinned data from index blocks, so no need
+ // to set `block_contents_pinned`.
+ std::unique_ptr<InternalIteratorBase<IndexValue>> index_iter(
+ index_block.GetValue()->NewIndexIterator(
+ internal_comparator()->user_comparator(),
+ rep->get_global_seqno(BlockType::kIndex), nullptr, kNullStats, true,
+ index_has_first_key(), index_key_includes_seq(),
+ index_value_is_full()));
+
+ it = new PartitionedIndexIterator(
+ table(), ro, *internal_comparator(), std::move(index_iter),
+ lookup_context ? lookup_context->caller
+ : TableReaderCaller::kUncategorized);
+ }
+
+ assert(it != nullptr);
+ index_block.TransferTo(it);
+
+ return it;
+
+ // TODO(myabandeh): Update TwoLevelIterator to be able to make use of
+ // on-stack BlockIter while the state is on heap. Currentlly it assumes
+ // the first level iter is always on heap and will attempt to delete it
+ // in its destructor.
+}
+Status PartitionIndexReader::CacheDependencies(const ReadOptions& ro,
+ bool pin) {
+ // Before read partitions, prefetch them to avoid lots of IOs
+ BlockCacheLookupContext lookup_context{TableReaderCaller::kPrefetch};
+ const BlockBasedTable::Rep* rep = table()->rep_;
+ IndexBlockIter biter;
+ BlockHandle handle;
+ Statistics* kNullStats = nullptr;
+
+ CachableEntry<Block> index_block;
+ {
+ Status s = GetOrReadIndexBlock(false /* no_io */, ro.rate_limiter_priority,
+ nullptr /* get_context */, &lookup_context,
+ &index_block);
+ if (!s.ok()) {
+ return s;
+ }
+ }
+
+ // We don't return pinned data from index blocks, so no need
+ // to set `block_contents_pinned`.
+ index_block.GetValue()->NewIndexIterator(
+ internal_comparator()->user_comparator(),
+ rep->get_global_seqno(BlockType::kIndex), &biter, kNullStats, true,
+ index_has_first_key(), index_key_includes_seq(), index_value_is_full());
+ // Index partitions are assumed to be consecuitive. Prefetch them all.
+ // Read the first block offset
+ biter.SeekToFirst();
+ if (!biter.Valid()) {
+ // Empty index.
+ return biter.status();
+ }
+ handle = biter.value().handle;
+ uint64_t prefetch_off = handle.offset();
+
+ // Read the last block's offset
+ biter.SeekToLast();
+ if (!biter.Valid()) {
+ // Empty index.
+ return biter.status();
+ }
+ handle = biter.value().handle;
+ uint64_t last_off =
+ handle.offset() + BlockBasedTable::BlockSizeWithTrailer(handle);
+ uint64_t prefetch_len = last_off - prefetch_off;
+ std::unique_ptr<FilePrefetchBuffer> prefetch_buffer;
+ rep->CreateFilePrefetchBuffer(
+ 0, 0, &prefetch_buffer, false /*Implicit auto readahead*/,
+ 0 /*num_reads_*/, 0 /*num_file_reads_for_auto_readahead*/);
+ IOOptions opts;
+ {
+ Status s = rep->file->PrepareIOOptions(ro, opts);
+ if (s.ok()) {
+ s = prefetch_buffer->Prefetch(opts, rep->file.get(), prefetch_off,
+ static_cast<size_t>(prefetch_len),
+ ro.rate_limiter_priority);
+ }
+ if (!s.ok()) {
+ return s;
+ }
+ }
+
+ // For saving "all or nothing" to partition_map_
+ UnorderedMap<uint64_t, CachableEntry<Block>> map_in_progress;
+
+ // After prefetch, read the partitions one by one
+ biter.SeekToFirst();
+ size_t partition_count = 0;
+ for (; biter.Valid(); biter.Next()) {
+ handle = biter.value().handle;
+ CachableEntry<Block> block;
+ ++partition_count;
+ // TODO: Support counter batch update for partitioned index and
+ // filter blocks
+ Status s = table()->MaybeReadBlockAndLoadToCache(
+ prefetch_buffer.get(), ro, handle, UncompressionDict::GetEmptyDict(),
+ /*wait=*/true, /*for_compaction=*/false, &block, BlockType::kIndex,
+ /*get_context=*/nullptr, &lookup_context, /*contents=*/nullptr,
+ /*async_read=*/false);
+
+ if (!s.ok()) {
+ return s;
+ }
+ if (block.GetValue() != nullptr) {
+ // Might need to "pin" some mmap-read blocks (GetOwnValue) if some
+ // partitions are successfully compressed (cached) and some are not
+ // compressed (mmap eligible)
+ if (block.IsCached() || block.GetOwnValue()) {
+ if (pin) {
+ map_in_progress[handle.offset()] = std::move(block);
+ }
+ }
+ }
+ }
+ Status s = biter.status();
+ // Save (pin) them only if everything checks out
+ if (map_in_progress.size() == partition_count && s.ok()) {
+ std::swap(partition_map_, map_in_progress);
+ }
+ return s;
+}
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/partitioned_index_reader.h b/src/rocksdb/table/block_based/partitioned_index_reader.h
new file mode 100644
index 000000000..58a7877ab
--- /dev/null
+++ b/src/rocksdb/table/block_based/partitioned_index_reader.h
@@ -0,0 +1,55 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+#pragma once
+#include "table/block_based/index_reader_common.h"
+#include "util/hash_containers.h"
+
+namespace ROCKSDB_NAMESPACE {
+// Index that allows binary search lookup in a two-level index structure.
+class PartitionIndexReader : public BlockBasedTable::IndexReaderCommon {
+ public:
+ // Read the partition index from the file and create an instance for
+ // `PartitionIndexReader`.
+ // On success, index_reader will be populated; otherwise it will remain
+ // unmodified.
+ static Status Create(const BlockBasedTable* table, const ReadOptions& ro,
+ FilePrefetchBuffer* prefetch_buffer, bool use_cache,
+ bool prefetch, bool pin,
+ BlockCacheLookupContext* lookup_context,
+ std::unique_ptr<IndexReader>* index_reader);
+
+ // return a two-level iterator: first level is on the partition index
+ InternalIteratorBase<IndexValue>* NewIterator(
+ const ReadOptions& read_options, bool /* disable_prefix_seek */,
+ IndexBlockIter* iter, GetContext* get_context,
+ BlockCacheLookupContext* lookup_context) override;
+
+ Status CacheDependencies(const ReadOptions& ro, bool pin) override;
+ size_t ApproximateMemoryUsage() const override {
+ size_t usage = ApproximateIndexBlockMemoryUsage();
+#ifdef ROCKSDB_MALLOC_USABLE_SIZE
+ usage += malloc_usable_size(const_cast<PartitionIndexReader*>(this));
+#else
+ usage += sizeof(*this);
+#endif // ROCKSDB_MALLOC_USABLE_SIZE
+ // TODO(myabandeh): more accurate estimate of partition_map_ mem usage
+ return usage;
+ }
+
+ private:
+ PartitionIndexReader(const BlockBasedTable* t,
+ CachableEntry<Block>&& index_block)
+ : IndexReaderCommon(t, std::move(index_block)) {}
+
+ // For partition blocks pinned in cache. This is expected to be "all or
+ // none" so that !partition_map_.empty() can use an iterator expecting
+ // all partitions to be saved here.
+ UnorderedMap<uint64_t, CachableEntry<Block>> partition_map_;
+};
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/reader_common.cc b/src/rocksdb/table/block_based/reader_common.cc
new file mode 100644
index 000000000..0ff43e9b4
--- /dev/null
+++ b/src/rocksdb/table/block_based/reader_common.cc
@@ -0,0 +1,52 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+#include "table/block_based/reader_common.h"
+
+#include "monitoring/perf_context_imp.h"
+#include "rocksdb/table.h"
+#include "table/format.h"
+#include "util/coding.h"
+#include "util/crc32c.h"
+#include "util/string_util.h"
+
+namespace ROCKSDB_NAMESPACE {
+void ForceReleaseCachedEntry(void* arg, void* h) {
+ Cache* cache = reinterpret_cast<Cache*>(arg);
+ Cache::Handle* handle = reinterpret_cast<Cache::Handle*>(h);
+ cache->Release(handle, true /* erase_if_last_ref */);
+}
+
+// WART: this is specific to block-based table
+Status VerifyBlockChecksum(ChecksumType type, const char* data,
+ size_t block_size, const std::string& file_name,
+ uint64_t offset) {
+ PERF_TIMER_GUARD(block_checksum_time);
+ // After block_size bytes is compression type (1 byte), which is part of
+ // the checksummed section.
+ size_t len = block_size + 1;
+ // And then the stored checksum value (4 bytes).
+ uint32_t stored = DecodeFixed32(data + len);
+
+ uint32_t computed = ComputeBuiltinChecksum(type, data, len);
+ if (stored == computed) {
+ return Status::OK();
+ } else {
+ // Unmask for people who might look for reference crc value
+ if (type == kCRC32c) {
+ stored = crc32c::Unmask(stored);
+ computed = crc32c::Unmask(computed);
+ }
+ return Status::Corruption(
+ "block checksum mismatch: stored = " + std::to_string(stored) +
+ ", computed = " + std::to_string(computed) +
+ ", type = " + std::to_string(type) + " in " + file_name + " offset " +
+ std::to_string(offset) + " size " + std::to_string(block_size));
+ }
+}
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/reader_common.h b/src/rocksdb/table/block_based/reader_common.h
new file mode 100644
index 000000000..5bb199f28
--- /dev/null
+++ b/src/rocksdb/table/block_based/reader_common.h
@@ -0,0 +1,38 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+#pragma once
+
+#include "rocksdb/cache.h"
+#include "rocksdb/table.h"
+
+namespace ROCKSDB_NAMESPACE {
+// Release the cached entry and decrement its ref count.
+extern void ForceReleaseCachedEntry(void* arg, void* h);
+
+inline MemoryAllocator* GetMemoryAllocator(
+ const BlockBasedTableOptions& table_options) {
+ return table_options.block_cache.get()
+ ? table_options.block_cache->memory_allocator()
+ : nullptr;
+}
+
+inline MemoryAllocator* GetMemoryAllocatorForCompressedBlock(
+ const BlockBasedTableOptions& table_options) {
+ return table_options.block_cache_compressed.get()
+ ? table_options.block_cache_compressed->memory_allocator()
+ : nullptr;
+}
+
+// Assumes block has a trailer as in format.h. file_name and offset provided
+// for generating a diagnostic message in returned status.
+extern Status VerifyBlockChecksum(ChecksumType type, const char* data,
+ size_t block_size,
+ const std::string& file_name,
+ uint64_t offset);
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/uncompression_dict_reader.cc b/src/rocksdb/table/block_based/uncompression_dict_reader.cc
new file mode 100644
index 000000000..dc9a47ec7
--- /dev/null
+++ b/src/rocksdb/table/block_based/uncompression_dict_reader.cc
@@ -0,0 +1,124 @@
+// Copyright (c) 2011-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 "table/block_based/uncompression_dict_reader.h"
+
+#include "logging/logging.h"
+#include "monitoring/perf_context_imp.h"
+#include "table/block_based/block_based_table_reader.h"
+#include "util/compression.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+Status UncompressionDictReader::Create(
+ const BlockBasedTable* table, const ReadOptions& ro,
+ FilePrefetchBuffer* prefetch_buffer, bool use_cache, bool prefetch,
+ bool pin, BlockCacheLookupContext* lookup_context,
+ std::unique_ptr<UncompressionDictReader>* uncompression_dict_reader) {
+ assert(table);
+ assert(table->get_rep());
+ assert(!pin || prefetch);
+ assert(uncompression_dict_reader);
+
+ CachableEntry<UncompressionDict> uncompression_dict;
+ if (prefetch || !use_cache) {
+ const Status s = ReadUncompressionDictionary(
+ table, prefetch_buffer, ro, use_cache, nullptr /* get_context */,
+ lookup_context, &uncompression_dict);
+ if (!s.ok()) {
+ return s;
+ }
+
+ if (use_cache && !pin) {
+ uncompression_dict.Reset();
+ }
+ }
+
+ uncompression_dict_reader->reset(
+ new UncompressionDictReader(table, std::move(uncompression_dict)));
+
+ return Status::OK();
+}
+
+Status UncompressionDictReader::ReadUncompressionDictionary(
+ const BlockBasedTable* table, FilePrefetchBuffer* prefetch_buffer,
+ const ReadOptions& read_options, bool use_cache, GetContext* get_context,
+ BlockCacheLookupContext* lookup_context,
+ CachableEntry<UncompressionDict>* uncompression_dict) {
+ // TODO: add perf counter for compression dictionary read time
+
+ assert(table);
+ assert(uncompression_dict);
+ assert(uncompression_dict->IsEmpty());
+
+ const BlockBasedTable::Rep* const rep = table->get_rep();
+ assert(rep);
+ assert(!rep->compression_dict_handle.IsNull());
+
+ const Status s = table->RetrieveBlock(
+ prefetch_buffer, read_options, rep->compression_dict_handle,
+ UncompressionDict::GetEmptyDict(), uncompression_dict,
+ BlockType::kCompressionDictionary, get_context, lookup_context,
+ /* for_compaction */ false, use_cache, /* wait_for_cache */ true,
+ /* async_read */ false);
+
+ if (!s.ok()) {
+ ROCKS_LOG_WARN(
+ rep->ioptions.logger,
+ "Encountered error while reading data from compression dictionary "
+ "block %s",
+ s.ToString().c_str());
+ }
+
+ return s;
+}
+
+Status UncompressionDictReader::GetOrReadUncompressionDictionary(
+ FilePrefetchBuffer* prefetch_buffer, bool no_io, bool verify_checksums,
+ GetContext* get_context, BlockCacheLookupContext* lookup_context,
+ CachableEntry<UncompressionDict>* uncompression_dict) const {
+ assert(uncompression_dict);
+
+ if (!uncompression_dict_.IsEmpty()) {
+ uncompression_dict->SetUnownedValue(uncompression_dict_.GetValue());
+ return Status::OK();
+ }
+
+ ReadOptions read_options;
+ if (no_io) {
+ read_options.read_tier = kBlockCacheTier;
+ }
+ read_options.verify_checksums = verify_checksums;
+
+ return ReadUncompressionDictionary(table_, prefetch_buffer, read_options,
+ cache_dictionary_blocks(), get_context,
+ lookup_context, uncompression_dict);
+}
+
+size_t UncompressionDictReader::ApproximateMemoryUsage() const {
+ assert(!uncompression_dict_.GetOwnValue() ||
+ uncompression_dict_.GetValue() != nullptr);
+ size_t usage = uncompression_dict_.GetOwnValue()
+ ? uncompression_dict_.GetValue()->ApproximateMemoryUsage()
+ : 0;
+
+#ifdef ROCKSDB_MALLOC_USABLE_SIZE
+ usage += malloc_usable_size(const_cast<UncompressionDictReader*>(this));
+#else
+ usage += sizeof(*this);
+#endif // ROCKSDB_MALLOC_USABLE_SIZE
+
+ return usage;
+}
+
+bool UncompressionDictReader::cache_dictionary_blocks() const {
+ assert(table_);
+ assert(table_->get_rep());
+
+ return table_->get_rep()->table_options.cache_index_and_filter_blocks;
+}
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_based/uncompression_dict_reader.h b/src/rocksdb/table/block_based/uncompression_dict_reader.h
new file mode 100644
index 000000000..416d25e2d
--- /dev/null
+++ b/src/rocksdb/table/block_based/uncompression_dict_reader.h
@@ -0,0 +1,60 @@
+// Copyright (c) 2011-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).
+//
+
+#pragma once
+
+#include <cassert>
+
+#include "table/block_based/cachable_entry.h"
+#include "table/format.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class BlockBasedTable;
+struct BlockCacheLookupContext;
+class FilePrefetchBuffer;
+class GetContext;
+struct ReadOptions;
+struct UncompressionDict;
+
+// Provides access to the uncompression dictionary regardless of whether
+// it is owned by the reader or stored in the cache, or whether it is pinned
+// in the cache or not.
+class UncompressionDictReader {
+ public:
+ static Status Create(
+ const BlockBasedTable* table, const ReadOptions& ro,
+ FilePrefetchBuffer* prefetch_buffer, bool use_cache, bool prefetch,
+ bool pin, BlockCacheLookupContext* lookup_context,
+ std::unique_ptr<UncompressionDictReader>* uncompression_dict_reader);
+
+ Status GetOrReadUncompressionDictionary(
+ FilePrefetchBuffer* prefetch_buffer, bool no_io, bool verify_checksums,
+ GetContext* get_context, BlockCacheLookupContext* lookup_context,
+ CachableEntry<UncompressionDict>* uncompression_dict) const;
+
+ size_t ApproximateMemoryUsage() const;
+
+ private:
+ UncompressionDictReader(const BlockBasedTable* t,
+ CachableEntry<UncompressionDict>&& uncompression_dict)
+ : table_(t), uncompression_dict_(std::move(uncompression_dict)) {
+ assert(table_);
+ }
+
+ bool cache_dictionary_blocks() const;
+
+ static Status ReadUncompressionDictionary(
+ const BlockBasedTable* table, FilePrefetchBuffer* prefetch_buffer,
+ const ReadOptions& read_options, bool use_cache, GetContext* get_context,
+ BlockCacheLookupContext* lookup_context,
+ CachableEntry<UncompressionDict>* uncompression_dict);
+
+ const BlockBasedTable* table_;
+ CachableEntry<UncompressionDict> uncompression_dict_;
+};
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_fetcher.cc b/src/rocksdb/table/block_fetcher.cc
new file mode 100644
index 000000000..8df0850b3
--- /dev/null
+++ b/src/rocksdb/table/block_fetcher.cc
@@ -0,0 +1,399 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#include "table/block_fetcher.h"
+
+#include <cassert>
+#include <cinttypes>
+#include <string>
+
+#include "logging/logging.h"
+#include "memory/memory_allocator.h"
+#include "monitoring/perf_context_imp.h"
+#include "rocksdb/compression_type.h"
+#include "rocksdb/env.h"
+#include "table/block_based/block.h"
+#include "table/block_based/block_based_table_reader.h"
+#include "table/block_based/block_type.h"
+#include "table/block_based/reader_common.h"
+#include "table/format.h"
+#include "table/persistent_cache_helper.h"
+#include "util/compression.h"
+#include "util/stop_watch.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+inline void BlockFetcher::ProcessTrailerIfPresent() {
+ if (footer_.GetBlockTrailerSize() > 0) {
+ assert(footer_.GetBlockTrailerSize() == BlockBasedTable::kBlockTrailerSize);
+ if (read_options_.verify_checksums) {
+ io_status_ = status_to_io_status(VerifyBlockChecksum(
+ footer_.checksum_type(), slice_.data(), block_size_,
+ file_->file_name(), handle_.offset()));
+ RecordTick(ioptions_.stats, BLOCK_CHECKSUM_COMPUTE_COUNT);
+ }
+ compression_type_ =
+ BlockBasedTable::GetBlockCompressionType(slice_.data(), block_size_);
+ } else {
+ // E.g. plain table or cuckoo table
+ compression_type_ = kNoCompression;
+ }
+}
+
+inline bool BlockFetcher::TryGetUncompressBlockFromPersistentCache() {
+ if (cache_options_.persistent_cache &&
+ !cache_options_.persistent_cache->IsCompressed()) {
+ Status status = PersistentCacheHelper::LookupUncompressed(
+ cache_options_, handle_, contents_);
+ if (status.ok()) {
+ // uncompressed page is found for the block handle
+ return true;
+ } else {
+ // uncompressed page is not found
+ if (ioptions_.logger && !status.IsNotFound()) {
+ assert(!status.ok());
+ ROCKS_LOG_INFO(ioptions_.logger,
+ "Error reading from persistent cache. %s",
+ status.ToString().c_str());
+ }
+ }
+ }
+ return false;
+}
+
+inline bool BlockFetcher::TryGetFromPrefetchBuffer() {
+ if (prefetch_buffer_ != nullptr) {
+ IOOptions opts;
+ IOStatus io_s = file_->PrepareIOOptions(read_options_, opts);
+ if (io_s.ok()) {
+ bool read_from_prefetch_buffer = false;
+ if (read_options_.async_io && !for_compaction_) {
+ read_from_prefetch_buffer = prefetch_buffer_->TryReadFromCacheAsync(
+ opts, file_, handle_.offset(), block_size_with_trailer_, &slice_,
+ &io_s, read_options_.rate_limiter_priority);
+ } else {
+ read_from_prefetch_buffer = prefetch_buffer_->TryReadFromCache(
+ opts, file_, handle_.offset(), block_size_with_trailer_, &slice_,
+ &io_s, read_options_.rate_limiter_priority, for_compaction_);
+ }
+ if (read_from_prefetch_buffer) {
+ ProcessTrailerIfPresent();
+ if (!io_status_.ok()) {
+ return true;
+ }
+ got_from_prefetch_buffer_ = true;
+ used_buf_ = const_cast<char*>(slice_.data());
+ }
+ }
+ if (!io_s.ok()) {
+ io_status_ = io_s;
+ return true;
+ }
+ }
+ return got_from_prefetch_buffer_;
+}
+
+inline bool BlockFetcher::TryGetSerializedBlockFromPersistentCache() {
+ if (cache_options_.persistent_cache &&
+ cache_options_.persistent_cache->IsCompressed()) {
+ std::unique_ptr<char[]> buf;
+ io_status_ = status_to_io_status(PersistentCacheHelper::LookupSerialized(
+ cache_options_, handle_, &buf, block_size_with_trailer_));
+ if (io_status_.ok()) {
+ heap_buf_ = CacheAllocationPtr(buf.release());
+ used_buf_ = heap_buf_.get();
+ slice_ = Slice(heap_buf_.get(), block_size_);
+ ProcessTrailerIfPresent();
+ return true;
+ } else if (!io_status_.IsNotFound() && ioptions_.logger) {
+ assert(!io_status_.ok());
+ ROCKS_LOG_INFO(ioptions_.logger,
+ "Error reading from persistent cache. %s",
+ io_status_.ToString().c_str());
+ }
+ }
+ return false;
+}
+
+inline void BlockFetcher::PrepareBufferForBlockFromFile() {
+ // cache miss read from device
+ if ((do_uncompress_ || ioptions_.allow_mmap_reads) &&
+ block_size_with_trailer_ < kDefaultStackBufferSize) {
+ // If we've got a small enough chunk of data, read it in to the
+ // trivially allocated stack buffer instead of needing a full malloc()
+ //
+ // `GetBlockContents()` cannot return this data as its lifetime is tied to
+ // this `BlockFetcher`'s lifetime. That is fine because this is only used
+ // in cases where we do not expect the `GetBlockContents()` result to be the
+ // same buffer we are assigning here. If we guess incorrectly, there will be
+ // a heap allocation and memcpy in `GetBlockContents()` to obtain the final
+ // result. Considering we are eliding a heap allocation here by using the
+ // stack buffer, the cost of guessing incorrectly here is one extra memcpy.
+ //
+ // When `do_uncompress_` is true, we expect the uncompression step will
+ // allocate heap memory for the final result. However this expectation will
+ // be wrong if the block turns out to already be uncompressed, which we
+ // won't know for sure until after reading it.
+ //
+ // When `ioptions_.allow_mmap_reads` is true, we do not expect the file
+ // reader to use the scratch buffer at all, but instead return a pointer
+ // into the mapped memory. This expectation will be wrong when using a
+ // file reader that does not implement mmap reads properly.
+ used_buf_ = &stack_buf_[0];
+ } else if (maybe_compressed_ && !do_uncompress_) {
+ compressed_buf_ =
+ AllocateBlock(block_size_with_trailer_, memory_allocator_compressed_);
+ used_buf_ = compressed_buf_.get();
+ } else {
+ heap_buf_ = AllocateBlock(block_size_with_trailer_, memory_allocator_);
+ used_buf_ = heap_buf_.get();
+ }
+}
+
+inline void BlockFetcher::InsertCompressedBlockToPersistentCacheIfNeeded() {
+ if (io_status_.ok() && read_options_.fill_cache &&
+ cache_options_.persistent_cache &&
+ cache_options_.persistent_cache->IsCompressed()) {
+ PersistentCacheHelper::InsertSerialized(cache_options_, handle_, used_buf_,
+ block_size_with_trailer_);
+ }
+}
+
+inline void BlockFetcher::InsertUncompressedBlockToPersistentCacheIfNeeded() {
+ if (io_status_.ok() && !got_from_prefetch_buffer_ &&
+ read_options_.fill_cache && cache_options_.persistent_cache &&
+ !cache_options_.persistent_cache->IsCompressed()) {
+ // insert to uncompressed cache
+ PersistentCacheHelper::InsertUncompressed(cache_options_, handle_,
+ *contents_);
+ }
+}
+
+inline void BlockFetcher::CopyBufferToHeapBuf() {
+ assert(used_buf_ != heap_buf_.get());
+ heap_buf_ = AllocateBlock(block_size_with_trailer_, memory_allocator_);
+ memcpy(heap_buf_.get(), used_buf_, block_size_with_trailer_);
+#ifndef NDEBUG
+ num_heap_buf_memcpy_++;
+#endif
+}
+
+inline void BlockFetcher::CopyBufferToCompressedBuf() {
+ assert(used_buf_ != compressed_buf_.get());
+ compressed_buf_ =
+ AllocateBlock(block_size_with_trailer_, memory_allocator_compressed_);
+ memcpy(compressed_buf_.get(), used_buf_, block_size_with_trailer_);
+#ifndef NDEBUG
+ num_compressed_buf_memcpy_++;
+#endif
+}
+
+// Entering this method means the block is not compressed or do not need to be
+// uncompressed. The block can be in one of the following buffers:
+// 1. prefetch buffer if prefetch is enabled and the block is prefetched before
+// 2. stack_buf_ if block size is smaller than the stack_buf_ size and block
+// is not compressed
+// 3. heap_buf_ if the block is not compressed
+// 4. compressed_buf_ if the block is compressed
+// 5. direct_io_buf_ if direct IO is enabled
+// After this method, if the block is compressed, it should be in
+// compressed_buf_, otherwise should be in heap_buf_.
+inline void BlockFetcher::GetBlockContents() {
+ if (slice_.data() != used_buf_) {
+ // the slice content is not the buffer provided
+ *contents_ = BlockContents(Slice(slice_.data(), block_size_));
+ } else {
+ // page can be either uncompressed or compressed, the buffer either stack
+ // or heap provided. Refer to https://github.com/facebook/rocksdb/pull/4096
+ if (got_from_prefetch_buffer_ || used_buf_ == &stack_buf_[0]) {
+ CopyBufferToHeapBuf();
+ } else if (used_buf_ == compressed_buf_.get()) {
+ if (compression_type_ == kNoCompression &&
+ memory_allocator_ != memory_allocator_compressed_) {
+ CopyBufferToHeapBuf();
+ } else {
+ heap_buf_ = std::move(compressed_buf_);
+ }
+ } else if (direct_io_buf_.get() != nullptr) {
+ if (compression_type_ == kNoCompression) {
+ CopyBufferToHeapBuf();
+ } else {
+ CopyBufferToCompressedBuf();
+ heap_buf_ = std::move(compressed_buf_);
+ }
+ }
+ *contents_ = BlockContents(std::move(heap_buf_), block_size_);
+ }
+#ifndef NDEBUG
+ contents_->has_trailer = footer_.GetBlockTrailerSize() > 0;
+#endif
+}
+
+IOStatus BlockFetcher::ReadBlockContents() {
+ if (TryGetUncompressBlockFromPersistentCache()) {
+ compression_type_ = kNoCompression;
+#ifndef NDEBUG
+ contents_->has_trailer = footer_.GetBlockTrailerSize() > 0;
+#endif // NDEBUG
+ return IOStatus::OK();
+ }
+ if (TryGetFromPrefetchBuffer()) {
+ if (!io_status_.ok()) {
+ return io_status_;
+ }
+ } else if (!TryGetSerializedBlockFromPersistentCache()) {
+ IOOptions opts;
+ io_status_ = file_->PrepareIOOptions(read_options_, opts);
+ // Actual file read
+ if (io_status_.ok()) {
+ if (file_->use_direct_io()) {
+ PERF_TIMER_GUARD(block_read_time);
+ io_status_ = file_->Read(
+ opts, handle_.offset(), block_size_with_trailer_, &slice_, nullptr,
+ &direct_io_buf_, read_options_.rate_limiter_priority);
+ PERF_COUNTER_ADD(block_read_count, 1);
+ used_buf_ = const_cast<char*>(slice_.data());
+ } else {
+ PrepareBufferForBlockFromFile();
+ PERF_TIMER_GUARD(block_read_time);
+ io_status_ = file_->Read(opts, handle_.offset(),
+ block_size_with_trailer_, &slice_, used_buf_,
+ nullptr, read_options_.rate_limiter_priority);
+ PERF_COUNTER_ADD(block_read_count, 1);
+#ifndef NDEBUG
+ if (slice_.data() == &stack_buf_[0]) {
+ num_stack_buf_memcpy_++;
+ } else if (slice_.data() == heap_buf_.get()) {
+ num_heap_buf_memcpy_++;
+ } else if (slice_.data() == compressed_buf_.get()) {
+ num_compressed_buf_memcpy_++;
+ }
+#endif
+ }
+ }
+
+ // TODO: introduce dedicated perf counter for range tombstones
+ switch (block_type_) {
+ case BlockType::kFilter:
+ case BlockType::kFilterPartitionIndex:
+ PERF_COUNTER_ADD(filter_block_read_count, 1);
+ break;
+
+ case BlockType::kCompressionDictionary:
+ PERF_COUNTER_ADD(compression_dict_block_read_count, 1);
+ break;
+
+ case BlockType::kIndex:
+ PERF_COUNTER_ADD(index_block_read_count, 1);
+ break;
+
+ // Nothing to do here as we don't have counters for the other types.
+ default:
+ break;
+ }
+
+ PERF_COUNTER_ADD(block_read_byte, block_size_with_trailer_);
+ if (!io_status_.ok()) {
+ return io_status_;
+ }
+
+ if (slice_.size() != block_size_with_trailer_) {
+ return IOStatus::Corruption(
+ "truncated block read from " + file_->file_name() + " offset " +
+ std::to_string(handle_.offset()) + ", expected " +
+ std::to_string(block_size_with_trailer_) + " bytes, got " +
+ std::to_string(slice_.size()));
+ }
+
+ ProcessTrailerIfPresent();
+ if (io_status_.ok()) {
+ InsertCompressedBlockToPersistentCacheIfNeeded();
+ } else {
+ return io_status_;
+ }
+ }
+
+ if (do_uncompress_ && compression_type_ != kNoCompression) {
+ PERF_TIMER_GUARD(block_decompress_time);
+ // compressed page, uncompress, update cache
+ UncompressionContext context(compression_type_);
+ UncompressionInfo info(context, uncompression_dict_, compression_type_);
+ io_status_ = status_to_io_status(UncompressSerializedBlock(
+ info, slice_.data(), block_size_, contents_, footer_.format_version(),
+ ioptions_, memory_allocator_));
+#ifndef NDEBUG
+ num_heap_buf_memcpy_++;
+#endif
+ compression_type_ = kNoCompression;
+ } else {
+ GetBlockContents();
+ }
+
+ InsertUncompressedBlockToPersistentCacheIfNeeded();
+
+ return io_status_;
+}
+
+IOStatus BlockFetcher::ReadAsyncBlockContents() {
+ if (TryGetUncompressBlockFromPersistentCache()) {
+ compression_type_ = kNoCompression;
+#ifndef NDEBUG
+ contents_->has_trailer = footer_.GetBlockTrailerSize() > 0;
+#endif // NDEBUG
+ return IOStatus::OK();
+ } else if (!TryGetSerializedBlockFromPersistentCache()) {
+ assert(prefetch_buffer_ != nullptr);
+ if (!for_compaction_) {
+ IOOptions opts;
+ IOStatus io_s = file_->PrepareIOOptions(read_options_, opts);
+ if (!io_s.ok()) {
+ return io_s;
+ }
+ io_s = status_to_io_status(prefetch_buffer_->PrefetchAsync(
+ opts, file_, handle_.offset(), block_size_with_trailer_, &slice_));
+ if (io_s.IsTryAgain()) {
+ return io_s;
+ }
+ if (io_s.ok()) {
+ // Data Block is already in prefetch.
+ got_from_prefetch_buffer_ = true;
+ ProcessTrailerIfPresent();
+ if (!io_status_.ok()) {
+ return io_status_;
+ }
+ used_buf_ = const_cast<char*>(slice_.data());
+
+ if (do_uncompress_ && compression_type_ != kNoCompression) {
+ PERF_TIMER_GUARD(block_decompress_time);
+ // compressed page, uncompress, update cache
+ UncompressionContext context(compression_type_);
+ UncompressionInfo info(context, uncompression_dict_,
+ compression_type_);
+ io_status_ = status_to_io_status(UncompressSerializedBlock(
+ info, slice_.data(), block_size_, contents_,
+ footer_.format_version(), ioptions_, memory_allocator_));
+#ifndef NDEBUG
+ num_heap_buf_memcpy_++;
+#endif
+ compression_type_ = kNoCompression;
+ } else {
+ GetBlockContents();
+ }
+ InsertUncompressedBlockToPersistentCacheIfNeeded();
+ return io_status_;
+ }
+ }
+ // Fallback to sequential reading of data blocks in case of io_s returns
+ // error or for_compaction_is true.
+ return ReadBlockContents();
+ }
+ return io_status_;
+}
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_fetcher.h b/src/rocksdb/table/block_fetcher.h
new file mode 100644
index 000000000..72adced30
--- /dev/null
+++ b/src/rocksdb/table/block_fetcher.h
@@ -0,0 +1,142 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#pragma once
+#include "memory/memory_allocator.h"
+#include "table/block_based/block.h"
+#include "table/block_based/block_type.h"
+#include "table/format.h"
+#include "table/persistent_cache_options.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+// Retrieves a single block of a given file. Utilizes the prefetch buffer and/or
+// persistent cache provided (if any) to try to avoid reading from the file
+// directly. Note that both the prefetch buffer and the persistent cache are
+// optional; also, note that the persistent cache may be configured to store
+// either compressed or uncompressed blocks.
+//
+// If the retrieved block is compressed and the do_uncompress flag is set,
+// BlockFetcher uncompresses the block (using the uncompression dictionary,
+// if provided, to prime the compression algorithm), and returns the resulting
+// uncompressed block data. Otherwise, it returns the original block.
+//
+// Two read options affect the behavior of BlockFetcher: if verify_checksums is
+// true, the checksum of the (original) block is checked; if fill_cache is true,
+// the block is added to the persistent cache if needed.
+//
+// Memory for uncompressed and compressed blocks is allocated as needed
+// using memory_allocator and memory_allocator_compressed, respectively
+// (if provided; otherwise, the default allocator is used).
+
+class BlockFetcher {
+ public:
+ BlockFetcher(RandomAccessFileReader* file,
+ FilePrefetchBuffer* prefetch_buffer,
+ const Footer& footer /* ref retained */,
+ const ReadOptions& read_options,
+ const BlockHandle& handle /* ref retained */,
+ BlockContents* contents,
+ const ImmutableOptions& ioptions /* ref retained */,
+ bool do_uncompress, bool maybe_compressed, BlockType block_type,
+ const UncompressionDict& uncompression_dict /* ref retained */,
+ const PersistentCacheOptions& cache_options /* ref retained */,
+ MemoryAllocator* memory_allocator = nullptr,
+ MemoryAllocator* memory_allocator_compressed = nullptr,
+ bool for_compaction = false)
+ : file_(file),
+ prefetch_buffer_(prefetch_buffer),
+ footer_(footer),
+ read_options_(read_options),
+ handle_(handle),
+ contents_(contents),
+ ioptions_(ioptions),
+ do_uncompress_(do_uncompress),
+ maybe_compressed_(maybe_compressed),
+ block_type_(block_type),
+ block_size_(static_cast<size_t>(handle_.size())),
+ block_size_with_trailer_(block_size_ + footer.GetBlockTrailerSize()),
+ uncompression_dict_(uncompression_dict),
+ cache_options_(cache_options),
+ memory_allocator_(memory_allocator),
+ memory_allocator_compressed_(memory_allocator_compressed),
+ for_compaction_(for_compaction) {
+ io_status_.PermitUncheckedError(); // TODO(AR) can we improve on this?
+ }
+
+ IOStatus ReadBlockContents();
+ IOStatus ReadAsyncBlockContents();
+
+ inline CompressionType get_compression_type() const {
+ return compression_type_;
+ }
+ inline size_t GetBlockSizeWithTrailer() const {
+ return block_size_with_trailer_;
+ }
+
+#ifndef NDEBUG
+ int TEST_GetNumStackBufMemcpy() const { return num_stack_buf_memcpy_; }
+ int TEST_GetNumHeapBufMemcpy() const { return num_heap_buf_memcpy_; }
+ int TEST_GetNumCompressedBufMemcpy() const {
+ return num_compressed_buf_memcpy_;
+ }
+
+#endif
+ private:
+#ifndef NDEBUG
+ int num_stack_buf_memcpy_ = 0;
+ int num_heap_buf_memcpy_ = 0;
+ int num_compressed_buf_memcpy_ = 0;
+
+#endif
+ static const uint32_t kDefaultStackBufferSize = 5000;
+
+ RandomAccessFileReader* file_;
+ FilePrefetchBuffer* prefetch_buffer_;
+ const Footer& footer_;
+ const ReadOptions read_options_;
+ const BlockHandle& handle_;
+ BlockContents* contents_;
+ const ImmutableOptions& ioptions_;
+ const bool do_uncompress_;
+ const bool maybe_compressed_;
+ const BlockType block_type_;
+ const size_t block_size_;
+ const size_t block_size_with_trailer_;
+ const UncompressionDict& uncompression_dict_;
+ const PersistentCacheOptions& cache_options_;
+ MemoryAllocator* memory_allocator_;
+ MemoryAllocator* memory_allocator_compressed_;
+ IOStatus io_status_;
+ Slice slice_;
+ char* used_buf_ = nullptr;
+ AlignedBuf direct_io_buf_;
+ CacheAllocationPtr heap_buf_;
+ CacheAllocationPtr compressed_buf_;
+ char stack_buf_[kDefaultStackBufferSize];
+ bool got_from_prefetch_buffer_ = false;
+ CompressionType compression_type_;
+ bool for_compaction_ = false;
+
+ // return true if found
+ bool TryGetUncompressBlockFromPersistentCache();
+ // return true if found
+ bool TryGetFromPrefetchBuffer();
+ bool TryGetSerializedBlockFromPersistentCache();
+ void PrepareBufferForBlockFromFile();
+ // Copy content from used_buf_ to new heap_buf_.
+ void CopyBufferToHeapBuf();
+ // Copy content from used_buf_ to new compressed_buf_.
+ void CopyBufferToCompressedBuf();
+ void GetBlockContents();
+ void InsertCompressedBlockToPersistentCacheIfNeeded();
+ void InsertUncompressedBlockToPersistentCacheIfNeeded();
+ void ProcessTrailerIfPresent();
+};
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/block_fetcher_test.cc b/src/rocksdb/table/block_fetcher_test.cc
new file mode 100644
index 000000000..82caee282
--- /dev/null
+++ b/src/rocksdb/table/block_fetcher_test.cc
@@ -0,0 +1,521 @@
+// Copyright (c) 2011-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 "table/block_fetcher.h"
+
+#include "db/table_properties_collector.h"
+#include "file/file_util.h"
+#include "options/options_helper.h"
+#include "port/port.h"
+#include "port/stack_trace.h"
+#include "rocksdb/db.h"
+#include "rocksdb/file_system.h"
+#include "table/block_based/binary_search_index_reader.h"
+#include "table/block_based/block_based_table_builder.h"
+#include "table/block_based/block_based_table_factory.h"
+#include "table/block_based/block_based_table_reader.h"
+#include "table/format.h"
+#include "test_util/testharness.h"
+#include "utilities/memory_allocators.h"
+
+namespace ROCKSDB_NAMESPACE {
+namespace {
+struct MemcpyStats {
+ int num_stack_buf_memcpy;
+ int num_heap_buf_memcpy;
+ int num_compressed_buf_memcpy;
+};
+
+struct BufAllocationStats {
+ int num_heap_buf_allocations;
+ int num_compressed_buf_allocations;
+};
+
+struct TestStats {
+ MemcpyStats memcpy_stats;
+ BufAllocationStats buf_allocation_stats;
+};
+
+class BlockFetcherTest : public testing::Test {
+ public:
+ enum class Mode {
+ kBufferedRead = 0,
+ kBufferedMmap,
+ kDirectRead,
+ kNumModes,
+ };
+ // use NumModes as array size to avoid "size of array '...' has non-integral
+ // type" errors.
+ const static int NumModes = static_cast<int>(Mode::kNumModes);
+
+ protected:
+ void SetUp() override {
+ SetupSyncPointsToMockDirectIO();
+ test_dir_ = test::PerThreadDBPath("block_fetcher_test");
+ env_ = Env::Default();
+ fs_ = FileSystem::Default();
+ ASSERT_OK(fs_->CreateDir(test_dir_, IOOptions(), nullptr));
+ }
+
+ void TearDown() override { EXPECT_OK(DestroyDir(env_, test_dir_)); }
+
+ void AssertSameBlock(const std::string& block1, const std::string& block2) {
+ ASSERT_EQ(block1, block2);
+ }
+
+ // Creates a table with kv pairs (i, i) where i ranges from 0 to 9, inclusive.
+ void CreateTable(const std::string& table_name,
+ const CompressionType& compression_type) {
+ std::unique_ptr<WritableFileWriter> writer;
+ NewFileWriter(table_name, &writer);
+
+ // Create table builder.
+ ImmutableOptions ioptions(options_);
+ InternalKeyComparator comparator(options_.comparator);
+ ColumnFamilyOptions cf_options(options_);
+ MutableCFOptions moptions(cf_options);
+ IntTblPropCollectorFactories factories;
+ std::unique_ptr<TableBuilder> table_builder(table_factory_.NewTableBuilder(
+ TableBuilderOptions(ioptions, moptions, comparator, &factories,
+ compression_type, CompressionOptions(),
+ 0 /* column_family_id */, kDefaultColumnFamilyName,
+ -1 /* level */),
+ writer.get()));
+
+ // Build table.
+ for (int i = 0; i < 9; i++) {
+ std::string key = ToInternalKey(std::to_string(i));
+ // Append "00000000" to string value to enhance compression ratio
+ std::string value = "00000000" + std::to_string(i);
+ table_builder->Add(key, value);
+ }
+ ASSERT_OK(table_builder->Finish());
+ }
+
+ void FetchIndexBlock(const std::string& table_name,
+ CountedMemoryAllocator* heap_buf_allocator,
+ CountedMemoryAllocator* compressed_buf_allocator,
+ MemcpyStats* memcpy_stats, BlockContents* index_block,
+ std::string* result) {
+ FileOptions fopt(options_);
+ std::unique_ptr<RandomAccessFileReader> file;
+ NewFileReader(table_name, fopt, &file);
+
+ // Get handle of the index block.
+ Footer footer;
+ ReadFooter(file.get(), &footer);
+ const BlockHandle& index_handle = footer.index_handle();
+
+ CompressionType compression_type;
+ FetchBlock(file.get(), index_handle, BlockType::kIndex,
+ false /* compressed */, false /* do_uncompress */,
+ heap_buf_allocator, compressed_buf_allocator, index_block,
+ memcpy_stats, &compression_type);
+ ASSERT_EQ(compression_type, CompressionType::kNoCompression);
+ result->assign(index_block->data.ToString());
+ }
+
+ // Fetches the first data block in both direct IO and non-direct IO mode.
+ //
+ // compressed: whether the data blocks are compressed;
+ // do_uncompress: whether the data blocks should be uncompressed on fetching.
+ // compression_type: the expected compression type.
+ //
+ // Expects:
+ // Block contents are the same.
+ // Bufferr allocation and memory copy statistics are expected.
+ void TestFetchDataBlock(
+ const std::string& table_name_prefix, bool compressed, bool do_uncompress,
+ std::array<TestStats, NumModes> expected_stats_by_mode) {
+ for (CompressionType compression_type : GetSupportedCompressions()) {
+ bool do_compress = compression_type != kNoCompression;
+ if (compressed != do_compress) continue;
+ std::string compression_type_str =
+ CompressionTypeToString(compression_type);
+
+ std::string table_name = table_name_prefix + compression_type_str;
+ CreateTable(table_name, compression_type);
+
+ CompressionType expected_compression_type_after_fetch =
+ (compressed && !do_uncompress) ? compression_type : kNoCompression;
+
+ BlockContents blocks[NumModes];
+ std::string block_datas[NumModes];
+ MemcpyStats memcpy_stats[NumModes];
+ CountedMemoryAllocator heap_buf_allocators[NumModes];
+ CountedMemoryAllocator compressed_buf_allocators[NumModes];
+ for (int i = 0; i < NumModes; ++i) {
+ SetMode(static_cast<Mode>(i));
+ FetchFirstDataBlock(table_name, compressed, do_uncompress,
+ expected_compression_type_after_fetch,
+ &heap_buf_allocators[i],
+ &compressed_buf_allocators[i], &blocks[i],
+ &block_datas[i], &memcpy_stats[i]);
+ }
+
+ for (int i = 0; i < NumModes - 1; ++i) {
+ AssertSameBlock(block_datas[i], block_datas[i + 1]);
+ }
+
+ // Check memcpy and buffer allocation statistics.
+ for (int i = 0; i < NumModes; ++i) {
+ const TestStats& expected_stats = expected_stats_by_mode[i];
+
+ ASSERT_EQ(memcpy_stats[i].num_stack_buf_memcpy,
+ expected_stats.memcpy_stats.num_stack_buf_memcpy);
+ ASSERT_EQ(memcpy_stats[i].num_heap_buf_memcpy,
+ expected_stats.memcpy_stats.num_heap_buf_memcpy);
+ ASSERT_EQ(memcpy_stats[i].num_compressed_buf_memcpy,
+ expected_stats.memcpy_stats.num_compressed_buf_memcpy);
+
+ if (kXpressCompression == compression_type) {
+ // XPRESS allocates memory internally, thus does not support for
+ // custom allocator verification
+ continue;
+ } else {
+ ASSERT_EQ(
+ heap_buf_allocators[i].GetNumAllocations(),
+ expected_stats.buf_allocation_stats.num_heap_buf_allocations);
+ ASSERT_EQ(compressed_buf_allocators[i].GetNumAllocations(),
+ expected_stats.buf_allocation_stats
+ .num_compressed_buf_allocations);
+
+ // The allocated buffers are not deallocated until
+ // the block content is deleted.
+ ASSERT_EQ(heap_buf_allocators[i].GetNumDeallocations(), 0);
+ ASSERT_EQ(compressed_buf_allocators[i].GetNumDeallocations(), 0);
+ blocks[i].allocation.reset();
+ ASSERT_EQ(
+ heap_buf_allocators[i].GetNumDeallocations(),
+ expected_stats.buf_allocation_stats.num_heap_buf_allocations);
+ ASSERT_EQ(compressed_buf_allocators[i].GetNumDeallocations(),
+ expected_stats.buf_allocation_stats
+ .num_compressed_buf_allocations);
+ }
+ }
+ }
+ }
+
+ void SetMode(Mode mode) {
+ switch (mode) {
+ case Mode::kBufferedRead:
+ options_.use_direct_reads = false;
+ options_.allow_mmap_reads = false;
+ break;
+ case Mode::kBufferedMmap:
+ options_.use_direct_reads = false;
+ options_.allow_mmap_reads = true;
+ break;
+ case Mode::kDirectRead:
+ options_.use_direct_reads = true;
+ options_.allow_mmap_reads = false;
+ break;
+ case Mode::kNumModes:
+ assert(false);
+ }
+ }
+
+ private:
+ std::string test_dir_;
+ Env* env_;
+ std::shared_ptr<FileSystem> fs_;
+ BlockBasedTableFactory table_factory_;
+ Options options_;
+
+ std::string Path(const std::string& fname) { return test_dir_ + "/" + fname; }
+
+ void WriteToFile(const std::string& content, const std::string& filename) {
+ std::unique_ptr<FSWritableFile> f;
+ ASSERT_OK(fs_->NewWritableFile(Path(filename), FileOptions(), &f, nullptr));
+ ASSERT_OK(f->Append(content, IOOptions(), nullptr));
+ ASSERT_OK(f->Close(IOOptions(), nullptr));
+ }
+
+ void NewFileWriter(const std::string& filename,
+ std::unique_ptr<WritableFileWriter>* writer) {
+ std::string path = Path(filename);
+ FileOptions file_options;
+ ASSERT_OK(WritableFileWriter::Create(env_->GetFileSystem(), path,
+ file_options, writer, nullptr));
+ }
+
+ void NewFileReader(const std::string& filename, const FileOptions& opt,
+ std::unique_ptr<RandomAccessFileReader>* reader) {
+ std::string path = Path(filename);
+ std::unique_ptr<FSRandomAccessFile> f;
+ ASSERT_OK(fs_->NewRandomAccessFile(path, opt, &f, nullptr));
+ reader->reset(new RandomAccessFileReader(std::move(f), path,
+ env_->GetSystemClock().get()));
+ }
+
+ void NewTableReader(const ImmutableOptions& ioptions,
+ const FileOptions& foptions,
+ const InternalKeyComparator& comparator,
+ const std::string& table_name,
+ std::unique_ptr<BlockBasedTable>* table) {
+ std::unique_ptr<RandomAccessFileReader> file;
+ NewFileReader(table_name, foptions, &file);
+
+ uint64_t file_size = 0;
+ ASSERT_OK(env_->GetFileSize(Path(table_name), &file_size));
+
+ std::unique_ptr<TableReader> table_reader;
+ ReadOptions ro;
+ const auto* table_options =
+ table_factory_.GetOptions<BlockBasedTableOptions>();
+ ASSERT_NE(table_options, nullptr);
+ ASSERT_OK(BlockBasedTable::Open(ro, ioptions, EnvOptions(), *table_options,
+ comparator, std::move(file), file_size,
+ &table_reader));
+
+ table->reset(reinterpret_cast<BlockBasedTable*>(table_reader.release()));
+ }
+
+ std::string ToInternalKey(const std::string& key) {
+ InternalKey internal_key(key, 0, ValueType::kTypeValue);
+ return internal_key.Encode().ToString();
+ }
+
+ void ReadFooter(RandomAccessFileReader* file, Footer* footer) {
+ uint64_t file_size = 0;
+ ASSERT_OK(env_->GetFileSize(file->file_name(), &file_size));
+ IOOptions opts;
+ ASSERT_OK(ReadFooterFromFile(opts, file, nullptr /* prefetch_buffer */,
+ file_size, footer,
+ kBlockBasedTableMagicNumber));
+ }
+
+ // NOTE: compression_type returns the compression type of the fetched block
+ // contents, so if the block is fetched and uncompressed, then it's
+ // kNoCompression.
+ void FetchBlock(RandomAccessFileReader* file, const BlockHandle& block,
+ BlockType block_type, bool compressed, bool do_uncompress,
+ MemoryAllocator* heap_buf_allocator,
+ MemoryAllocator* compressed_buf_allocator,
+ BlockContents* contents, MemcpyStats* stats,
+ CompressionType* compresstion_type) {
+ ImmutableOptions ioptions(options_);
+ ReadOptions roptions;
+ PersistentCacheOptions persistent_cache_options;
+ Footer footer;
+ ReadFooter(file, &footer);
+ std::unique_ptr<BlockFetcher> fetcher(new BlockFetcher(
+ file, nullptr /* prefetch_buffer */, footer, roptions, block, contents,
+ ioptions, do_uncompress, compressed, block_type,
+ UncompressionDict::GetEmptyDict(), persistent_cache_options,
+ heap_buf_allocator, compressed_buf_allocator));
+
+ ASSERT_OK(fetcher->ReadBlockContents());
+
+ stats->num_stack_buf_memcpy = fetcher->TEST_GetNumStackBufMemcpy();
+ stats->num_heap_buf_memcpy = fetcher->TEST_GetNumHeapBufMemcpy();
+ stats->num_compressed_buf_memcpy =
+ fetcher->TEST_GetNumCompressedBufMemcpy();
+
+ *compresstion_type = fetcher->get_compression_type();
+ }
+
+ // NOTE: expected_compression_type is the expected compression
+ // type of the fetched block content, if the block is uncompressed,
+ // then the expected compression type is kNoCompression.
+ void FetchFirstDataBlock(const std::string& table_name, bool compressed,
+ bool do_uncompress,
+ CompressionType expected_compression_type,
+ MemoryAllocator* heap_buf_allocator,
+ MemoryAllocator* compressed_buf_allocator,
+ BlockContents* block, std::string* result,
+ MemcpyStats* memcpy_stats) {
+ ImmutableOptions ioptions(options_);
+ InternalKeyComparator comparator(options_.comparator);
+ FileOptions foptions(options_);
+
+ // Get block handle for the first data block.
+ std::unique_ptr<BlockBasedTable> table;
+ NewTableReader(ioptions, foptions, comparator, table_name, &table);
+
+ std::unique_ptr<BlockBasedTable::IndexReader> index_reader;
+ ReadOptions ro;
+ ASSERT_OK(BinarySearchIndexReader::Create(
+ table.get(), ro, nullptr /* prefetch_buffer */, false /* use_cache */,
+ false /* prefetch */, false /* pin */, nullptr /* lookup_context */,
+ &index_reader));
+
+ std::unique_ptr<InternalIteratorBase<IndexValue>> iter(
+ index_reader->NewIterator(
+ ReadOptions(), false /* disable_prefix_seek */, nullptr /* iter */,
+ nullptr /* get_context */, nullptr /* lookup_context */));
+ ASSERT_OK(iter->status());
+ iter->SeekToFirst();
+ BlockHandle first_block_handle = iter->value().handle;
+
+ // Fetch first data block.
+ std::unique_ptr<RandomAccessFileReader> file;
+ NewFileReader(table_name, foptions, &file);
+ CompressionType compression_type;
+ FetchBlock(file.get(), first_block_handle, BlockType::kData, compressed,
+ do_uncompress, heap_buf_allocator, compressed_buf_allocator,
+ block, memcpy_stats, &compression_type);
+ ASSERT_EQ(compression_type, expected_compression_type);
+ result->assign(block->data.ToString());
+ }
+};
+
+// Skip the following tests in lite mode since direct I/O is unsupported.
+#ifndef ROCKSDB_LITE
+
+// Fetch index block under both direct IO and non-direct IO.
+// Expects:
+// the index block contents are the same for both read modes.
+TEST_F(BlockFetcherTest, FetchIndexBlock) {
+ for (CompressionType compression : GetSupportedCompressions()) {
+ std::string table_name =
+ "FetchIndexBlock" + CompressionTypeToString(compression);
+ CreateTable(table_name, compression);
+
+ CountedMemoryAllocator allocator;
+ MemcpyStats memcpy_stats;
+ BlockContents indexes[NumModes];
+ std::string index_datas[NumModes];
+ for (int i = 0; i < NumModes; ++i) {
+ SetMode(static_cast<Mode>(i));
+ FetchIndexBlock(table_name, &allocator, &allocator, &memcpy_stats,
+ &indexes[i], &index_datas[i]);
+ }
+ for (int i = 0; i < NumModes - 1; ++i) {
+ AssertSameBlock(index_datas[i], index_datas[i + 1]);
+ }
+ }
+}
+
+// Data blocks are not compressed,
+// fetch data block under direct IO, mmap IO,and non-direct IO.
+// Expects:
+// 1. in non-direct IO mode, allocate a heap buffer and memcpy the block
+// into the buffer;
+// 2. in direct IO mode, allocate a heap buffer and memcpy from the
+// direct IO buffer to the heap buffer.
+TEST_F(BlockFetcherTest, FetchUncompressedDataBlock) {
+ TestStats expected_non_mmap_stats = {
+ {
+ 0 /* num_stack_buf_memcpy */,
+ 1 /* num_heap_buf_memcpy */,
+ 0 /* num_compressed_buf_memcpy */,
+ },
+ {
+ 1 /* num_heap_buf_allocations */,
+ 0 /* num_compressed_buf_allocations */,
+ }};
+ TestStats expected_mmap_stats = {{
+ 0 /* num_stack_buf_memcpy */,
+ 0 /* num_heap_buf_memcpy */,
+ 0 /* num_compressed_buf_memcpy */,
+ },
+ {
+ 0 /* num_heap_buf_allocations */,
+ 0 /* num_compressed_buf_allocations */,
+ }};
+ std::array<TestStats, NumModes> expected_stats_by_mode{{
+ expected_non_mmap_stats /* kBufferedRead */,
+ expected_mmap_stats /* kBufferedMmap */,
+ expected_non_mmap_stats /* kDirectRead */,
+ }};
+ TestFetchDataBlock("FetchUncompressedDataBlock", false, false,
+ expected_stats_by_mode);
+}
+
+// Data blocks are compressed,
+// fetch data block under both direct IO and non-direct IO,
+// but do not uncompress.
+// Expects:
+// 1. in non-direct IO mode, allocate a compressed buffer and memcpy the block
+// into the buffer;
+// 2. in direct IO mode, allocate a compressed buffer and memcpy from the
+// direct IO buffer to the compressed buffer.
+TEST_F(BlockFetcherTest, FetchCompressedDataBlock) {
+ TestStats expected_non_mmap_stats = {
+ {
+ 0 /* num_stack_buf_memcpy */,
+ 0 /* num_heap_buf_memcpy */,
+ 1 /* num_compressed_buf_memcpy */,
+ },
+ {
+ 0 /* num_heap_buf_allocations */,
+ 1 /* num_compressed_buf_allocations */,
+ }};
+ TestStats expected_mmap_stats = {{
+ 0 /* num_stack_buf_memcpy */,
+ 0 /* num_heap_buf_memcpy */,
+ 0 /* num_compressed_buf_memcpy */,
+ },
+ {
+ 0 /* num_heap_buf_allocations */,
+ 0 /* num_compressed_buf_allocations */,
+ }};
+ std::array<TestStats, NumModes> expected_stats_by_mode{{
+ expected_non_mmap_stats /* kBufferedRead */,
+ expected_mmap_stats /* kBufferedMmap */,
+ expected_non_mmap_stats /* kDirectRead */,
+ }};
+ TestFetchDataBlock("FetchCompressedDataBlock", true, false,
+ expected_stats_by_mode);
+}
+
+// Data blocks are compressed,
+// fetch and uncompress data block under both direct IO and non-direct IO.
+// Expects:
+// 1. in non-direct IO mode, since the block is small, so it's first memcpyed
+// to the stack buffer, then a heap buffer is allocated and the block is
+// uncompressed into the heap.
+// 2. in direct IO mode mode, allocate a heap buffer, then directly uncompress
+// and memcpy from the direct IO buffer to the heap buffer.
+TEST_F(BlockFetcherTest, FetchAndUncompressCompressedDataBlock) {
+ TestStats expected_buffered_read_stats = {
+ {
+ 1 /* num_stack_buf_memcpy */,
+ 1 /* num_heap_buf_memcpy */,
+ 0 /* num_compressed_buf_memcpy */,
+ },
+ {
+ 1 /* num_heap_buf_allocations */,
+ 0 /* num_compressed_buf_allocations */,
+ }};
+ TestStats expected_mmap_stats = {{
+ 0 /* num_stack_buf_memcpy */,
+ 1 /* num_heap_buf_memcpy */,
+ 0 /* num_compressed_buf_memcpy */,
+ },
+ {
+ 1 /* num_heap_buf_allocations */,
+ 0 /* num_compressed_buf_allocations */,
+ }};
+ TestStats expected_direct_read_stats = {
+ {
+ 0 /* num_stack_buf_memcpy */,
+ 1 /* num_heap_buf_memcpy */,
+ 0 /* num_compressed_buf_memcpy */,
+ },
+ {
+ 1 /* num_heap_buf_allocations */,
+ 0 /* num_compressed_buf_allocations */,
+ }};
+ std::array<TestStats, NumModes> expected_stats_by_mode{{
+ expected_buffered_read_stats,
+ expected_mmap_stats,
+ expected_direct_read_stats,
+ }};
+ TestFetchDataBlock("FetchAndUncompressCompressedDataBlock", true, true,
+ expected_stats_by_mode);
+}
+
+#endif // ROCKSDB_LITE
+
+} // namespace
+} // namespace ROCKSDB_NAMESPACE
+
+int main(int argc, char** argv) {
+ ROCKSDB_NAMESPACE::port::InstallStackTraceHandler();
+ ::testing::InitGoogleTest(&argc, argv);
+ return RUN_ALL_TESTS();
+}
diff --git a/src/rocksdb/table/cleanable_test.cc b/src/rocksdb/table/cleanable_test.cc
new file mode 100644
index 000000000..b58eb7dc6
--- /dev/null
+++ b/src/rocksdb/table/cleanable_test.cc
@@ -0,0 +1,390 @@
+// Copyright (c) 2011-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 "rocksdb/cleanable.h"
+
+#include <gtest/gtest.h>
+
+#include <functional>
+
+#include "port/port.h"
+#include "port/stack_trace.h"
+#include "rocksdb/iostats_context.h"
+#include "rocksdb/perf_context.h"
+#include "test_util/testharness.h"
+#include "test_util/testutil.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class CleanableTest : public testing::Test {};
+
+// Use this to keep track of the cleanups that were actually performed
+void Multiplier(void* arg1, void* arg2) {
+ int* res = reinterpret_cast<int*>(arg1);
+ int* num = reinterpret_cast<int*>(arg2);
+ *res *= *num;
+}
+
+// the first Cleanup is on stack and the rest on heap, so test with both cases
+TEST_F(CleanableTest, Register) {
+ int n2 = 2, n3 = 3;
+ int res = 1;
+ { Cleanable c1; }
+ // ~Cleanable
+ ASSERT_EQ(1, res);
+
+ res = 1;
+ {
+ Cleanable c1;
+ c1.RegisterCleanup(Multiplier, &res, &n2); // res = 2;
+ }
+ // ~Cleanable
+ ASSERT_EQ(2, res);
+
+ res = 1;
+ {
+ Cleanable c1;
+ c1.RegisterCleanup(Multiplier, &res, &n2); // res = 2;
+ c1.RegisterCleanup(Multiplier, &res, &n3); // res = 2 * 3;
+ }
+ // ~Cleanable
+ ASSERT_EQ(6, res);
+
+ // Test the Reset does cleanup
+ res = 1;
+ {
+ Cleanable c1;
+ c1.RegisterCleanup(Multiplier, &res, &n2); // res = 2;
+ c1.RegisterCleanup(Multiplier, &res, &n3); // res = 2 * 3;
+ c1.Reset();
+ ASSERT_EQ(6, res);
+ }
+ // ~Cleanable
+ ASSERT_EQ(6, res);
+
+ // Test Clenable is usable after Reset
+ res = 1;
+ {
+ Cleanable c1;
+ c1.RegisterCleanup(Multiplier, &res, &n2); // res = 2;
+ c1.Reset();
+ ASSERT_EQ(2, res);
+ c1.RegisterCleanup(Multiplier, &res, &n3); // res = 2 * 3;
+ }
+ // ~Cleanable
+ ASSERT_EQ(6, res);
+}
+
+// the first Cleanup is on stack and the rest on heap,
+// so test all the combinations of them
+TEST_F(CleanableTest, Delegation) {
+ int n2 = 2, n3 = 3, n5 = 5, n7 = 7;
+ int res = 1;
+ {
+ Cleanable c2;
+ {
+ Cleanable c1;
+ c1.RegisterCleanup(Multiplier, &res, &n2); // res = 2;
+ c1.DelegateCleanupsTo(&c2);
+ }
+ // ~Cleanable
+ ASSERT_EQ(1, res);
+ }
+ // ~Cleanable
+ ASSERT_EQ(2, res);
+
+ res = 1;
+ {
+ Cleanable c2;
+ {
+ Cleanable c1;
+ c1.DelegateCleanupsTo(&c2);
+ }
+ // ~Cleanable
+ ASSERT_EQ(1, res);
+ }
+ // ~Cleanable
+ ASSERT_EQ(1, res);
+
+ res = 1;
+ {
+ Cleanable c2;
+ {
+ Cleanable c1;
+ c1.RegisterCleanup(Multiplier, &res, &n2); // res = 2;
+ c1.RegisterCleanup(Multiplier, &res, &n3); // res = 2 * 3;
+ c1.DelegateCleanupsTo(&c2);
+ }
+ // ~Cleanable
+ ASSERT_EQ(1, res);
+ }
+ // ~Cleanable
+ ASSERT_EQ(6, res);
+
+ res = 1;
+ {
+ Cleanable c2;
+ c2.RegisterCleanup(Multiplier, &res, &n5); // res = 5;
+ {
+ Cleanable c1;
+ c1.RegisterCleanup(Multiplier, &res, &n2); // res = 2;
+ c1.RegisterCleanup(Multiplier, &res, &n3); // res = 2 * 3;
+ c1.DelegateCleanupsTo(&c2); // res = 2 * 3 * 5;
+ }
+ // ~Cleanable
+ ASSERT_EQ(1, res);
+ }
+ // ~Cleanable
+ ASSERT_EQ(30, res);
+
+ res = 1;
+ {
+ Cleanable c2;
+ c2.RegisterCleanup(Multiplier, &res, &n5); // res = 5;
+ c2.RegisterCleanup(Multiplier, &res, &n7); // res = 5 * 7;
+ {
+ Cleanable c1;
+ c1.RegisterCleanup(Multiplier, &res, &n2); // res = 2;
+ c1.RegisterCleanup(Multiplier, &res, &n3); // res = 2 * 3;
+ c1.DelegateCleanupsTo(&c2); // res = 2 * 3 * 5 * 7;
+ }
+ // ~Cleanable
+ ASSERT_EQ(1, res);
+ }
+ // ~Cleanable
+ ASSERT_EQ(210, res);
+
+ res = 1;
+ {
+ Cleanable c2;
+ c2.RegisterCleanup(Multiplier, &res, &n5); // res = 5;
+ c2.RegisterCleanup(Multiplier, &res, &n7); // res = 5 * 7;
+ {
+ Cleanable c1;
+ c1.RegisterCleanup(Multiplier, &res, &n2); // res = 2;
+ c1.DelegateCleanupsTo(&c2); // res = 2 * 5 * 7;
+ }
+ // ~Cleanable
+ ASSERT_EQ(1, res);
+ }
+ // ~Cleanable
+ ASSERT_EQ(70, res);
+
+ res = 1;
+ {
+ Cleanable c2;
+ c2.RegisterCleanup(Multiplier, &res, &n5); // res = 5;
+ c2.RegisterCleanup(Multiplier, &res, &n7); // res = 5 * 7;
+ {
+ Cleanable c1;
+ c1.DelegateCleanupsTo(&c2); // res = 5 * 7;
+ }
+ // ~Cleanable
+ ASSERT_EQ(1, res);
+ }
+ // ~Cleanable
+ ASSERT_EQ(35, res);
+
+ res = 1;
+ {
+ Cleanable c2;
+ c2.RegisterCleanup(Multiplier, &res, &n5); // res = 5;
+ {
+ Cleanable c1;
+ c1.DelegateCleanupsTo(&c2); // res = 5;
+ }
+ // ~Cleanable
+ ASSERT_EQ(1, res);
+ }
+ // ~Cleanable
+ ASSERT_EQ(5, res);
+}
+
+static void ReleaseStringHeap(void* s, void*) {
+ delete reinterpret_cast<const std::string*>(s);
+}
+
+class PinnableSlice4Test : public PinnableSlice {
+ public:
+ void TestStringIsRegistered(std::string* s) {
+ ASSERT_TRUE(cleanup_.function == ReleaseStringHeap);
+ ASSERT_EQ(cleanup_.arg1, s);
+ ASSERT_EQ(cleanup_.arg2, nullptr);
+ ASSERT_EQ(cleanup_.next, nullptr);
+ }
+};
+
+// Putting the PinnableSlice tests here due to similarity to Cleanable tests
+TEST_F(CleanableTest, PinnableSlice) {
+ int n2 = 2;
+ int res = 1;
+ const std::string const_str = "123";
+
+ {
+ res = 1;
+ PinnableSlice4Test value;
+ Slice slice(const_str);
+ value.PinSlice(slice, Multiplier, &res, &n2);
+ std::string str;
+ str.assign(value.data(), value.size());
+ ASSERT_EQ(const_str, str);
+ }
+ // ~Cleanable
+ ASSERT_EQ(2, res);
+
+ {
+ res = 1;
+ PinnableSlice4Test value;
+ Slice slice(const_str);
+ {
+ Cleanable c1;
+ c1.RegisterCleanup(Multiplier, &res, &n2); // res = 2;
+ value.PinSlice(slice, &c1);
+ }
+ // ~Cleanable
+ ASSERT_EQ(1, res); // cleanups must have be delegated to value
+ std::string str;
+ str.assign(value.data(), value.size());
+ ASSERT_EQ(const_str, str);
+ }
+ // ~Cleanable
+ ASSERT_EQ(2, res);
+
+ {
+ PinnableSlice4Test value;
+ Slice slice(const_str);
+ value.PinSelf(slice);
+ std::string str;
+ str.assign(value.data(), value.size());
+ ASSERT_EQ(const_str, str);
+ }
+
+ {
+ PinnableSlice4Test value;
+ std::string* self_str_ptr = value.GetSelf();
+ self_str_ptr->assign(const_str);
+ value.PinSelf();
+ std::string str;
+ str.assign(value.data(), value.size());
+ ASSERT_EQ(const_str, str);
+ }
+}
+
+static void Decrement(void* intptr, void*) { --*static_cast<int*>(intptr); }
+
+// Allow unit testing moved-from data
+template <class T>
+void MarkInitializedForClangAnalyze(T& t) {
+ // No net effect, but confuse analyzer. (Published advice doesn't work.)
+ char* p = reinterpret_cast<char*>(&t);
+ std::swap(*p, *p);
+}
+
+TEST_F(CleanableTest, SharedWrapCleanables) {
+ int val = 5;
+ Cleanable c1, c2;
+ c1.RegisterCleanup(&Decrement, &val, nullptr);
+ c1.RegisterCleanup(&Decrement, &val, nullptr);
+ ASSERT_TRUE(c1.HasCleanups());
+ ASSERT_FALSE(c2.HasCleanups());
+
+ SharedCleanablePtr scp1;
+ ASSERT_EQ(scp1.get(), nullptr);
+
+ // No-ops
+ scp1.RegisterCopyWith(&c2);
+ scp1.MoveAsCleanupTo(&c2);
+
+ ASSERT_FALSE(c2.HasCleanups());
+ c2.RegisterCleanup(&Decrement, &val, nullptr);
+ c2.RegisterCleanup(&Decrement, &val, nullptr);
+ c2.RegisterCleanup(&Decrement, &val, nullptr);
+
+ scp1.Allocate();
+ ASSERT_NE(scp1.get(), nullptr);
+ ASSERT_FALSE(scp1->HasCleanups());
+
+ // Copy ctor (alias scp2 = scp1)
+ SharedCleanablePtr scp2{scp1};
+ ASSERT_EQ(scp1.get(), scp2.get());
+
+ c1.DelegateCleanupsTo(&*scp1);
+ ASSERT_TRUE(scp1->HasCleanups());
+ ASSERT_TRUE(scp2->HasCleanups());
+ ASSERT_FALSE(c1.HasCleanups());
+
+ SharedCleanablePtr scp3;
+ ASSERT_EQ(scp3.get(), nullptr);
+
+ // Copy operator (alias scp3 = scp2 = scp1)
+ scp3 = scp2;
+
+ // Make scp2 point elsewhere
+ scp2.Allocate();
+ c2.DelegateCleanupsTo(&*scp2);
+
+ ASSERT_EQ(val, 5);
+ // Move operator, invoke old c2 cleanups
+ scp2 = std::move(scp1);
+ ASSERT_EQ(val, 2);
+ MarkInitializedForClangAnalyze(scp1);
+ ASSERT_EQ(scp1.get(), nullptr);
+
+ // Move ctor
+ {
+ SharedCleanablePtr scp4{std::move(scp3)};
+ MarkInitializedForClangAnalyze(scp3);
+ ASSERT_EQ(scp3.get(), nullptr);
+ ASSERT_EQ(scp4.get(), scp2.get());
+
+ scp2.Reset();
+ ASSERT_EQ(val, 2);
+ // invoke old c1 cleanups
+ }
+ ASSERT_EQ(val, 0);
+}
+
+TEST_F(CleanableTest, CleanableWrapShared) {
+ int val = 5;
+ SharedCleanablePtr scp1, scp2;
+ scp1.Allocate();
+ scp1->RegisterCleanup(&Decrement, &val, nullptr);
+ scp1->RegisterCleanup(&Decrement, &val, nullptr);
+
+ scp2.Allocate();
+ scp2->RegisterCleanup(&Decrement, &val, nullptr);
+ scp2->RegisterCleanup(&Decrement, &val, nullptr);
+ scp2->RegisterCleanup(&Decrement, &val, nullptr);
+
+ {
+ Cleanable c1;
+ {
+ Cleanable c2, c3;
+ scp1.RegisterCopyWith(&c1);
+ scp1.MoveAsCleanupTo(&c2);
+ ASSERT_TRUE(c1.HasCleanups());
+ ASSERT_TRUE(c2.HasCleanups());
+ ASSERT_EQ(scp1.get(), nullptr);
+ scp2.MoveAsCleanupTo(&c3);
+ ASSERT_TRUE(c3.HasCleanups());
+ ASSERT_EQ(scp2.get(), nullptr);
+ c2.Reset();
+ ASSERT_FALSE(c2.HasCleanups());
+ ASSERT_EQ(val, 5);
+ // invoke cleanups from scp2
+ }
+ ASSERT_EQ(val, 2);
+ // invoke cleanups from scp1
+ }
+ ASSERT_EQ(val, 0);
+}
+
+} // namespace ROCKSDB_NAMESPACE
+
+int main(int argc, char** argv) {
+ ROCKSDB_NAMESPACE::port::InstallStackTraceHandler();
+ ::testing::InitGoogleTest(&argc, argv);
+ return RUN_ALL_TESTS();
+}
diff --git a/src/rocksdb/table/cuckoo/cuckoo_table_builder.cc b/src/rocksdb/table/cuckoo/cuckoo_table_builder.cc
new file mode 100644
index 000000000..296825d94
--- /dev/null
+++ b/src/rocksdb/table/cuckoo/cuckoo_table_builder.cc
@@ -0,0 +1,553 @@
+// Copyright (c) 2011-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).
+
+#ifndef ROCKSDB_LITE
+#include "table/cuckoo/cuckoo_table_builder.h"
+
+#include <assert.h>
+
+#include <algorithm>
+#include <limits>
+#include <string>
+#include <vector>
+
+#include "db/dbformat.h"
+#include "file/writable_file_writer.h"
+#include "rocksdb/env.h"
+#include "rocksdb/table.h"
+#include "table/block_based/block_builder.h"
+#include "table/cuckoo/cuckoo_table_factory.h"
+#include "table/format.h"
+#include "table/meta_blocks.h"
+#include "util/autovector.h"
+#include "util/random.h"
+#include "util/string_util.h"
+
+namespace ROCKSDB_NAMESPACE {
+const std::string CuckooTablePropertyNames::kEmptyKey =
+ "rocksdb.cuckoo.bucket.empty.key";
+const std::string CuckooTablePropertyNames::kNumHashFunc =
+ "rocksdb.cuckoo.hash.num";
+const std::string CuckooTablePropertyNames::kHashTableSize =
+ "rocksdb.cuckoo.hash.size";
+const std::string CuckooTablePropertyNames::kValueLength =
+ "rocksdb.cuckoo.value.length";
+const std::string CuckooTablePropertyNames::kIsLastLevel =
+ "rocksdb.cuckoo.file.islastlevel";
+const std::string CuckooTablePropertyNames::kCuckooBlockSize =
+ "rocksdb.cuckoo.hash.cuckooblocksize";
+const std::string CuckooTablePropertyNames::kIdentityAsFirstHash =
+ "rocksdb.cuckoo.hash.identityfirst";
+const std::string CuckooTablePropertyNames::kUseModuleHash =
+ "rocksdb.cuckoo.hash.usemodule";
+const std::string CuckooTablePropertyNames::kUserKeyLength =
+ "rocksdb.cuckoo.hash.userkeylength";
+
+// Obtained by running echo rocksdb.table.cuckoo | sha1sum
+extern const uint64_t kCuckooTableMagicNumber = 0x926789d0c5f17873ull;
+
+CuckooTableBuilder::CuckooTableBuilder(
+ WritableFileWriter* file, double max_hash_table_ratio,
+ uint32_t max_num_hash_table, uint32_t max_search_depth,
+ const Comparator* user_comparator, uint32_t cuckoo_block_size,
+ bool use_module_hash, bool identity_as_first_hash,
+ uint64_t (*get_slice_hash)(const Slice&, uint32_t, uint64_t),
+ uint32_t column_family_id, const std::string& column_family_name,
+ const std::string& db_id, const std::string& db_session_id,
+ uint64_t file_number)
+ : num_hash_func_(2),
+ file_(file),
+ max_hash_table_ratio_(max_hash_table_ratio),
+ max_num_hash_func_(max_num_hash_table),
+ max_search_depth_(max_search_depth),
+ cuckoo_block_size_(std::max(1U, cuckoo_block_size)),
+ hash_table_size_(use_module_hash ? 0 : 2),
+ is_last_level_file_(false),
+ has_seen_first_key_(false),
+ has_seen_first_value_(false),
+ key_size_(0),
+ value_size_(0),
+ num_entries_(0),
+ num_values_(0),
+ ucomp_(user_comparator),
+ use_module_hash_(use_module_hash),
+ identity_as_first_hash_(identity_as_first_hash),
+ get_slice_hash_(get_slice_hash),
+ closed_(false) {
+ // Data is in a huge block.
+ properties_.num_data_blocks = 1;
+ properties_.index_size = 0;
+ properties_.filter_size = 0;
+ properties_.column_family_id = column_family_id;
+ properties_.column_family_name = column_family_name;
+ properties_.db_id = db_id;
+ properties_.db_session_id = db_session_id;
+ properties_.orig_file_number = file_number;
+ status_.PermitUncheckedError();
+ io_status_.PermitUncheckedError();
+}
+
+void CuckooTableBuilder::Add(const Slice& key, const Slice& value) {
+ if (num_entries_ >= kMaxVectorIdx - 1) {
+ status_ = Status::NotSupported("Number of keys in a file must be < 2^32-1");
+ return;
+ }
+ ParsedInternalKey ikey;
+ Status pik_status =
+ ParseInternalKey(key, &ikey, false /* log_err_key */); // TODO
+ if (!pik_status.ok()) {
+ status_ = Status::Corruption("Unable to parse key into internal key. ",
+ pik_status.getState());
+ return;
+ }
+ if (ikey.type != kTypeDeletion && ikey.type != kTypeValue) {
+ status_ = Status::NotSupported("Unsupported key type " +
+ std::to_string(ikey.type));
+ return;
+ }
+
+ // Determine if we can ignore the sequence number and value type from
+ // internal keys by looking at sequence number from first key. We assume
+ // that if first key has a zero sequence number, then all the remaining
+ // keys will have zero seq. no.
+ if (!has_seen_first_key_) {
+ is_last_level_file_ = ikey.sequence == 0;
+ has_seen_first_key_ = true;
+ smallest_user_key_.assign(ikey.user_key.data(), ikey.user_key.size());
+ largest_user_key_.assign(ikey.user_key.data(), ikey.user_key.size());
+ key_size_ = is_last_level_file_ ? ikey.user_key.size() : key.size();
+ }
+ if (key_size_ != (is_last_level_file_ ? ikey.user_key.size() : key.size())) {
+ status_ = Status::NotSupported("all keys have to be the same size");
+ return;
+ }
+
+ if (ikey.type == kTypeValue) {
+ if (!has_seen_first_value_) {
+ has_seen_first_value_ = true;
+ value_size_ = value.size();
+ }
+ if (value_size_ != value.size()) {
+ status_ = Status::NotSupported("all values have to be the same size");
+ return;
+ }
+
+ if (is_last_level_file_) {
+ kvs_.append(ikey.user_key.data(), ikey.user_key.size());
+ } else {
+ kvs_.append(key.data(), key.size());
+ }
+ kvs_.append(value.data(), value.size());
+ ++num_values_;
+ } else {
+ if (is_last_level_file_) {
+ deleted_keys_.append(ikey.user_key.data(), ikey.user_key.size());
+ } else {
+ deleted_keys_.append(key.data(), key.size());
+ }
+ }
+ ++num_entries_;
+
+ // In order to fill the empty buckets in the hash table, we identify a
+ // key which is not used so far (unused_user_key). We determine this by
+ // maintaining smallest and largest keys inserted so far in bytewise order
+ // and use them to find a key outside this range in Finish() operation.
+ // Note that this strategy is independent of user comparator used here.
+ if (ikey.user_key.compare(smallest_user_key_) < 0) {
+ smallest_user_key_.assign(ikey.user_key.data(), ikey.user_key.size());
+ } else if (ikey.user_key.compare(largest_user_key_) > 0) {
+ largest_user_key_.assign(ikey.user_key.data(), ikey.user_key.size());
+ }
+ if (!use_module_hash_) {
+ if (hash_table_size_ < num_entries_ / max_hash_table_ratio_) {
+ hash_table_size_ *= 2;
+ }
+ }
+}
+
+bool CuckooTableBuilder::IsDeletedKey(uint64_t idx) const {
+ assert(closed_);
+ return idx >= num_values_;
+}
+
+Slice CuckooTableBuilder::GetKey(uint64_t idx) const {
+ assert(closed_);
+ if (IsDeletedKey(idx)) {
+ return Slice(
+ &deleted_keys_[static_cast<size_t>((idx - num_values_) * key_size_)],
+ static_cast<size_t>(key_size_));
+ }
+ return Slice(&kvs_[static_cast<size_t>(idx * (key_size_ + value_size_))],
+ static_cast<size_t>(key_size_));
+}
+
+Slice CuckooTableBuilder::GetUserKey(uint64_t idx) const {
+ assert(closed_);
+ return is_last_level_file_ ? GetKey(idx) : ExtractUserKey(GetKey(idx));
+}
+
+Slice CuckooTableBuilder::GetValue(uint64_t idx) const {
+ assert(closed_);
+ if (IsDeletedKey(idx)) {
+ static std::string empty_value(static_cast<unsigned int>(value_size_), 'a');
+ return Slice(empty_value);
+ }
+ return Slice(
+ &kvs_[static_cast<size_t>(idx * (key_size_ + value_size_) + key_size_)],
+ static_cast<size_t>(value_size_));
+}
+
+Status CuckooTableBuilder::MakeHashTable(std::vector<CuckooBucket>* buckets) {
+ buckets->resize(
+ static_cast<size_t>(hash_table_size_ + cuckoo_block_size_ - 1));
+ uint32_t make_space_for_key_call_id = 0;
+ for (uint32_t vector_idx = 0; vector_idx < num_entries_; vector_idx++) {
+ uint64_t bucket_id = 0;
+ bool bucket_found = false;
+ autovector<uint64_t> hash_vals;
+ Slice user_key = GetUserKey(vector_idx);
+ for (uint32_t hash_cnt = 0; hash_cnt < num_hash_func_ && !bucket_found;
+ ++hash_cnt) {
+ uint64_t hash_val =
+ CuckooHash(user_key, hash_cnt, use_module_hash_, hash_table_size_,
+ identity_as_first_hash_, get_slice_hash_);
+ // If there is a collision, check next cuckoo_block_size_ locations for
+ // empty locations. While checking, if we reach end of the hash table,
+ // stop searching and proceed for next hash function.
+ for (uint32_t block_idx = 0; block_idx < cuckoo_block_size_;
+ ++block_idx, ++hash_val) {
+ if ((*buckets)[static_cast<size_t>(hash_val)].vector_idx ==
+ kMaxVectorIdx) {
+ bucket_id = hash_val;
+ bucket_found = true;
+ break;
+ } else {
+ if (ucomp_->Compare(
+ user_key, GetUserKey((*buckets)[static_cast<size_t>(hash_val)]
+ .vector_idx)) == 0) {
+ return Status::NotSupported("Same key is being inserted again.");
+ }
+ hash_vals.push_back(hash_val);
+ }
+ }
+ }
+ while (!bucket_found &&
+ !MakeSpaceForKey(hash_vals, ++make_space_for_key_call_id, buckets,
+ &bucket_id)) {
+ // Rehash by increashing number of hash tables.
+ if (num_hash_func_ >= max_num_hash_func_) {
+ return Status::NotSupported("Too many collisions. Unable to hash.");
+ }
+ // We don't really need to rehash the entire table because old hashes are
+ // still valid and we only increased the number of hash functions.
+ uint64_t hash_val = CuckooHash(user_key, num_hash_func_, use_module_hash_,
+ hash_table_size_, identity_as_first_hash_,
+ get_slice_hash_);
+ ++num_hash_func_;
+ for (uint32_t block_idx = 0; block_idx < cuckoo_block_size_;
+ ++block_idx, ++hash_val) {
+ if ((*buckets)[static_cast<size_t>(hash_val)].vector_idx ==
+ kMaxVectorIdx) {
+ bucket_found = true;
+ bucket_id = hash_val;
+ break;
+ } else {
+ hash_vals.push_back(hash_val);
+ }
+ }
+ }
+ (*buckets)[static_cast<size_t>(bucket_id)].vector_idx = vector_idx;
+ }
+ return Status::OK();
+}
+
+Status CuckooTableBuilder::Finish() {
+ assert(!closed_);
+ closed_ = true;
+ std::vector<CuckooBucket> buckets;
+ std::string unused_bucket;
+ if (num_entries_ > 0) {
+ // Calculate the real hash size if module hash is enabled.
+ if (use_module_hash_) {
+ hash_table_size_ =
+ static_cast<uint64_t>(num_entries_ / max_hash_table_ratio_);
+ }
+ status_ = MakeHashTable(&buckets);
+ if (!status_.ok()) {
+ return status_;
+ }
+ // Determine unused_user_key to fill empty buckets.
+ std::string unused_user_key = smallest_user_key_;
+ int curr_pos = static_cast<int>(unused_user_key.size()) - 1;
+ while (curr_pos >= 0) {
+ --unused_user_key[curr_pos];
+ if (Slice(unused_user_key).compare(smallest_user_key_) < 0) {
+ break;
+ }
+ --curr_pos;
+ }
+ if (curr_pos < 0) {
+ // Try using the largest key to identify an unused key.
+ unused_user_key = largest_user_key_;
+ curr_pos = static_cast<int>(unused_user_key.size()) - 1;
+ while (curr_pos >= 0) {
+ ++unused_user_key[curr_pos];
+ if (Slice(unused_user_key).compare(largest_user_key_) > 0) {
+ break;
+ }
+ --curr_pos;
+ }
+ }
+ if (curr_pos < 0) {
+ return Status::Corruption("Unable to find unused key");
+ }
+ if (is_last_level_file_) {
+ unused_bucket = unused_user_key;
+ } else {
+ ParsedInternalKey ikey(unused_user_key, 0, kTypeValue);
+ AppendInternalKey(&unused_bucket, ikey);
+ }
+ }
+ properties_.num_entries = num_entries_;
+ properties_.num_deletions = num_entries_ - num_values_;
+ properties_.fixed_key_len = key_size_;
+ properties_.user_collected_properties[CuckooTablePropertyNames::kValueLength]
+ .assign(reinterpret_cast<const char*>(&value_size_), sizeof(value_size_));
+
+ uint64_t bucket_size = key_size_ + value_size_;
+ unused_bucket.resize(static_cast<size_t>(bucket_size), 'a');
+ // Write the table.
+ uint32_t num_added = 0;
+ for (auto& bucket : buckets) {
+ if (bucket.vector_idx == kMaxVectorIdx) {
+ io_status_ = file_->Append(Slice(unused_bucket));
+ } else {
+ ++num_added;
+ io_status_ = file_->Append(GetKey(bucket.vector_idx));
+ if (io_status_.ok()) {
+ if (value_size_ > 0) {
+ io_status_ = file_->Append(GetValue(bucket.vector_idx));
+ }
+ }
+ }
+ if (!io_status_.ok()) {
+ status_ = io_status_;
+ return status_;
+ }
+ }
+ assert(num_added == NumEntries());
+ properties_.raw_key_size = num_added * properties_.fixed_key_len;
+ properties_.raw_value_size = num_added * value_size_;
+
+ uint64_t offset = buckets.size() * bucket_size;
+ properties_.data_size = offset;
+ unused_bucket.resize(static_cast<size_t>(properties_.fixed_key_len));
+ properties_.user_collected_properties[CuckooTablePropertyNames::kEmptyKey] =
+ unused_bucket;
+ properties_.user_collected_properties[CuckooTablePropertyNames::kNumHashFunc]
+ .assign(reinterpret_cast<char*>(&num_hash_func_), sizeof(num_hash_func_));
+
+ properties_
+ .user_collected_properties[CuckooTablePropertyNames::kHashTableSize]
+ .assign(reinterpret_cast<const char*>(&hash_table_size_),
+ sizeof(hash_table_size_));
+ properties_.user_collected_properties[CuckooTablePropertyNames::kIsLastLevel]
+ .assign(reinterpret_cast<const char*>(&is_last_level_file_),
+ sizeof(is_last_level_file_));
+ properties_
+ .user_collected_properties[CuckooTablePropertyNames::kCuckooBlockSize]
+ .assign(reinterpret_cast<const char*>(&cuckoo_block_size_),
+ sizeof(cuckoo_block_size_));
+ properties_
+ .user_collected_properties[CuckooTablePropertyNames::kIdentityAsFirstHash]
+ .assign(reinterpret_cast<const char*>(&identity_as_first_hash_),
+ sizeof(identity_as_first_hash_));
+ properties_
+ .user_collected_properties[CuckooTablePropertyNames::kUseModuleHash]
+ .assign(reinterpret_cast<const char*>(&use_module_hash_),
+ sizeof(use_module_hash_));
+ uint32_t user_key_len = static_cast<uint32_t>(smallest_user_key_.size());
+ properties_
+ .user_collected_properties[CuckooTablePropertyNames::kUserKeyLength]
+ .assign(reinterpret_cast<const char*>(&user_key_len),
+ sizeof(user_key_len));
+
+ // Write meta blocks.
+ MetaIndexBuilder meta_index_builder;
+ PropertyBlockBuilder property_block_builder;
+
+ property_block_builder.AddTableProperty(properties_);
+ property_block_builder.Add(properties_.user_collected_properties);
+ Slice property_block = property_block_builder.Finish();
+ BlockHandle property_block_handle;
+ property_block_handle.set_offset(offset);
+ property_block_handle.set_size(property_block.size());
+ io_status_ = file_->Append(property_block);
+ offset += property_block.size();
+ if (!io_status_.ok()) {
+ status_ = io_status_;
+ return status_;
+ }
+
+ meta_index_builder.Add(kPropertiesBlockName, property_block_handle);
+ Slice meta_index_block = meta_index_builder.Finish();
+
+ BlockHandle meta_index_block_handle;
+ meta_index_block_handle.set_offset(offset);
+ meta_index_block_handle.set_size(meta_index_block.size());
+ io_status_ = file_->Append(meta_index_block);
+ if (!io_status_.ok()) {
+ status_ = io_status_;
+ return status_;
+ }
+
+ FooterBuilder footer;
+ footer.Build(kCuckooTableMagicNumber, /* format_version */ 1, offset,
+ kNoChecksum, meta_index_block_handle);
+ io_status_ = file_->Append(footer.GetSlice());
+ status_ = io_status_;
+ return status_;
+}
+
+void CuckooTableBuilder::Abandon() {
+ assert(!closed_);
+ closed_ = true;
+}
+
+uint64_t CuckooTableBuilder::NumEntries() const { return num_entries_; }
+
+uint64_t CuckooTableBuilder::FileSize() const {
+ if (closed_) {
+ return file_->GetFileSize();
+ } else if (num_entries_ == 0) {
+ return 0;
+ }
+
+ if (use_module_hash_) {
+ return static_cast<uint64_t>((key_size_ + value_size_) * num_entries_ /
+ max_hash_table_ratio_);
+ } else {
+ // Account for buckets being a power of two.
+ // As elements are added, file size remains constant for a while and
+ // doubles its size. Since compaction algorithm stops adding elements
+ // only after it exceeds the file limit, we account for the extra element
+ // being added here.
+ uint64_t expected_hash_table_size = hash_table_size_;
+ if (expected_hash_table_size < (num_entries_ + 1) / max_hash_table_ratio_) {
+ expected_hash_table_size *= 2;
+ }
+ return (key_size_ + value_size_) * expected_hash_table_size - 1;
+ }
+}
+
+// This method is invoked when there is no place to insert the target key.
+// It searches for a set of elements that can be moved to accommodate target
+// key. The search is a BFS graph traversal with first level (hash_vals)
+// being all the buckets target key could go to.
+// Then, from each node (curr_node), we find all the buckets that curr_node
+// could go to. They form the children of curr_node in the tree.
+// We continue the traversal until we find an empty bucket, in which case, we
+// move all elements along the path from first level to this empty bucket, to
+// make space for target key which is inserted at first level (*bucket_id).
+// If tree depth exceedes max depth, we return false indicating failure.
+bool CuckooTableBuilder::MakeSpaceForKey(
+ const autovector<uint64_t>& hash_vals,
+ const uint32_t make_space_for_key_call_id,
+ std::vector<CuckooBucket>* buckets, uint64_t* bucket_id) {
+ struct CuckooNode {
+ uint64_t bucket_id;
+ uint32_t depth;
+ uint32_t parent_pos;
+ CuckooNode(uint64_t _bucket_id, uint32_t _depth, int _parent_pos)
+ : bucket_id(_bucket_id), depth(_depth), parent_pos(_parent_pos) {}
+ };
+ // This is BFS search tree that is stored simply as a vector.
+ // Each node stores the index of parent node in the vector.
+ std::vector<CuckooNode> tree;
+ // We want to identify already visited buckets in the current method call so
+ // that we don't add same buckets again for exploration in the tree.
+ // We do this by maintaining a count of current method call in
+ // make_space_for_key_call_id, which acts as a unique id for this invocation
+ // of the method. We store this number into the nodes that we explore in
+ // current method call.
+ // It is unlikely for the increment operation to overflow because the maximum
+ // no. of times this will be called is <= max_num_hash_func_ + num_entries_.
+ for (uint32_t hash_cnt = 0; hash_cnt < num_hash_func_; ++hash_cnt) {
+ uint64_t bid = hash_vals[hash_cnt];
+ (*buckets)[static_cast<size_t>(bid)].make_space_for_key_call_id =
+ make_space_for_key_call_id;
+ tree.push_back(CuckooNode(bid, 0, 0));
+ }
+ bool null_found = false;
+ uint32_t curr_pos = 0;
+ while (!null_found && curr_pos < tree.size()) {
+ CuckooNode& curr_node = tree[curr_pos];
+ uint32_t curr_depth = curr_node.depth;
+ if (curr_depth >= max_search_depth_) {
+ break;
+ }
+ CuckooBucket& curr_bucket =
+ (*buckets)[static_cast<size_t>(curr_node.bucket_id)];
+ for (uint32_t hash_cnt = 0; hash_cnt < num_hash_func_ && !null_found;
+ ++hash_cnt) {
+ uint64_t child_bucket_id = CuckooHash(
+ GetUserKey(curr_bucket.vector_idx), hash_cnt, use_module_hash_,
+ hash_table_size_, identity_as_first_hash_, get_slice_hash_);
+ // Iterate inside Cuckoo Block.
+ for (uint32_t block_idx = 0; block_idx < cuckoo_block_size_;
+ ++block_idx, ++child_bucket_id) {
+ if ((*buckets)[static_cast<size_t>(child_bucket_id)]
+ .make_space_for_key_call_id == make_space_for_key_call_id) {
+ continue;
+ }
+ (*buckets)[static_cast<size_t>(child_bucket_id)]
+ .make_space_for_key_call_id = make_space_for_key_call_id;
+ tree.push_back(CuckooNode(child_bucket_id, curr_depth + 1, curr_pos));
+ if ((*buckets)[static_cast<size_t>(child_bucket_id)].vector_idx ==
+ kMaxVectorIdx) {
+ null_found = true;
+ break;
+ }
+ }
+ }
+ ++curr_pos;
+ }
+
+ if (null_found) {
+ // There is an empty node in tree.back(). Now, traverse the path from this
+ // empty node to top of the tree and at every node in the path, replace
+ // child with the parent. Stop when first level is reached in the tree
+ // (happens when 0 <= bucket_to_replace_pos < num_hash_func_) and return
+ // this location in first level for target key to be inserted.
+ uint32_t bucket_to_replace_pos = static_cast<uint32_t>(tree.size()) - 1;
+ while (bucket_to_replace_pos >= num_hash_func_) {
+ CuckooNode& curr_node = tree[bucket_to_replace_pos];
+ (*buckets)[static_cast<size_t>(curr_node.bucket_id)] =
+ (*buckets)[static_cast<size_t>(tree[curr_node.parent_pos].bucket_id)];
+ bucket_to_replace_pos = curr_node.parent_pos;
+ }
+ *bucket_id = tree[bucket_to_replace_pos].bucket_id;
+ }
+ return null_found;
+}
+
+std::string CuckooTableBuilder::GetFileChecksum() const {
+ if (file_ != nullptr) {
+ return file_->GetFileChecksum();
+ } else {
+ return kUnknownFileChecksum;
+ }
+}
+
+const char* CuckooTableBuilder::GetFileChecksumFuncName() const {
+ if (file_ != nullptr) {
+ return file_->GetFileChecksumFuncName();
+ } else {
+ return kUnknownFileChecksumFuncName;
+ }
+}
+
+} // namespace ROCKSDB_NAMESPACE
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/table/cuckoo/cuckoo_table_builder.h b/src/rocksdb/table/cuckoo/cuckoo_table_builder.h
new file mode 100644
index 000000000..a125e1f4c
--- /dev/null
+++ b/src/rocksdb/table/cuckoo/cuckoo_table_builder.h
@@ -0,0 +1,138 @@
+// Copyright (c) 2011-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).
+
+#pragma once
+#ifndef ROCKSDB_LITE
+#include <stdint.h>
+
+#include <limits>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include "db/version_edit.h"
+#include "port/port.h"
+#include "rocksdb/status.h"
+#include "rocksdb/table.h"
+#include "rocksdb/table_properties.h"
+#include "table/table_builder.h"
+#include "util/autovector.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class CuckooTableBuilder : public TableBuilder {
+ public:
+ CuckooTableBuilder(
+ WritableFileWriter* file, double max_hash_table_ratio,
+ uint32_t max_num_hash_func, uint32_t max_search_depth,
+ const Comparator* user_comparator, uint32_t cuckoo_block_size,
+ bool use_module_hash, bool identity_as_first_hash,
+ uint64_t (*get_slice_hash)(const Slice&, uint32_t, uint64_t),
+ uint32_t column_family_id, const std::string& column_family_name,
+ const std::string& db_id = "", const std::string& db_session_id = "",
+ uint64_t file_number = 0);
+ // No copying allowed
+ CuckooTableBuilder(const CuckooTableBuilder&) = delete;
+ void operator=(const CuckooTableBuilder&) = delete;
+
+ // REQUIRES: Either Finish() or Abandon() has been called.
+ ~CuckooTableBuilder() {}
+
+ // Add key,value to the table being constructed.
+ // REQUIRES: key is after any previously added key according to comparator.
+ // REQUIRES: Finish(), Abandon() have not been called
+ void Add(const Slice& key, const Slice& value) override;
+
+ // Return non-ok iff some error has been detected.
+ Status status() const override { return status_; }
+
+ // Return non-ok iff some error happens during IO.
+ IOStatus io_status() const override { return io_status_; }
+
+ // Finish building the table. Stops using the file passed to the
+ // constructor after this function returns.
+ // REQUIRES: Finish(), Abandon() have not been called
+ Status Finish() override;
+
+ // Indicate that the contents of this builder should be abandoned. Stops
+ // using the file passed to the constructor after this function returns.
+ // If the caller is not going to call Finish(), it must call Abandon()
+ // before destroying this builder.
+ // REQUIRES: Finish(), Abandon() have not been called
+ void Abandon() override;
+
+ // Number of calls to Add() so far.
+ uint64_t NumEntries() const override;
+
+ // Size of the file generated so far. If invoked after a successful
+ // Finish() call, returns the size of the final generated file.
+ uint64_t FileSize() const override;
+
+ TableProperties GetTableProperties() const override { return properties_; }
+
+ // Get file checksum
+ std::string GetFileChecksum() const override;
+
+ // Get file checksum function name
+ const char* GetFileChecksumFuncName() const override;
+
+ private:
+ struct CuckooBucket {
+ CuckooBucket() : vector_idx(kMaxVectorIdx), make_space_for_key_call_id(0) {}
+ uint32_t vector_idx;
+ // This number will not exceed kvs_.size() + max_num_hash_func_.
+ // We assume number of items is <= 2^32.
+ uint32_t make_space_for_key_call_id;
+ };
+ static const uint32_t kMaxVectorIdx = std::numeric_limits<int32_t>::max();
+
+ bool MakeSpaceForKey(const autovector<uint64_t>& hash_vals,
+ const uint32_t call_id,
+ std::vector<CuckooBucket>* buckets, uint64_t* bucket_id);
+ Status MakeHashTable(std::vector<CuckooBucket>* buckets);
+
+ inline bool IsDeletedKey(uint64_t idx) const;
+ inline Slice GetKey(uint64_t idx) const;
+ inline Slice GetUserKey(uint64_t idx) const;
+ inline Slice GetValue(uint64_t idx) const;
+
+ uint32_t num_hash_func_;
+ WritableFileWriter* file_;
+ const double max_hash_table_ratio_;
+ const uint32_t max_num_hash_func_;
+ const uint32_t max_search_depth_;
+ const uint32_t cuckoo_block_size_;
+ uint64_t hash_table_size_;
+ bool is_last_level_file_;
+ bool has_seen_first_key_;
+ bool has_seen_first_value_;
+ uint64_t key_size_;
+ uint64_t value_size_;
+ // A list of fixed-size key-value pairs concatenating into a string.
+ // Use GetKey(), GetUserKey(), and GetValue() to retrieve a specific
+ // key / value given an index
+ std::string kvs_;
+ std::string deleted_keys_;
+ // Number of key-value pairs stored in kvs_ + number of deleted keys
+ uint64_t num_entries_;
+ // Number of keys that contain value (non-deletion op)
+ uint64_t num_values_;
+ Status status_;
+ IOStatus io_status_;
+ TableProperties properties_;
+ const Comparator* ucomp_;
+ bool use_module_hash_;
+ bool identity_as_first_hash_;
+ uint64_t (*get_slice_hash_)(const Slice& s, uint32_t index,
+ uint64_t max_num_buckets);
+ std::string largest_user_key_ = "";
+ std::string smallest_user_key_ = "";
+
+ bool closed_; // Either Finish() or Abandon() has been called.
+};
+
+} // namespace ROCKSDB_NAMESPACE
+
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/table/cuckoo/cuckoo_table_builder_test.cc b/src/rocksdb/table/cuckoo/cuckoo_table_builder_test.cc
new file mode 100644
index 000000000..be1c62117
--- /dev/null
+++ b/src/rocksdb/table/cuckoo/cuckoo_table_builder_test.cc
@@ -0,0 +1,640 @@
+// Copyright (c) 2011-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).
+
+#ifndef ROCKSDB_LITE
+
+#include "table/cuckoo/cuckoo_table_builder.h"
+
+#include <map>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include "file/random_access_file_reader.h"
+#include "file/writable_file_writer.h"
+#include "rocksdb/db.h"
+#include "rocksdb/file_system.h"
+#include "table/meta_blocks.h"
+#include "test_util/testharness.h"
+#include "test_util/testutil.h"
+
+namespace ROCKSDB_NAMESPACE {
+extern const uint64_t kCuckooTableMagicNumber;
+
+namespace {
+std::unordered_map<std::string, std::vector<uint64_t>> hash_map;
+
+uint64_t GetSliceHash(const Slice& s, uint32_t index,
+ uint64_t /*max_num_buckets*/) {
+ return hash_map[s.ToString()][index];
+}
+} // namespace
+
+class CuckooBuilderTest : public testing::Test {
+ public:
+ CuckooBuilderTest() {
+ env_ = Env::Default();
+ Options options;
+ options.allow_mmap_reads = true;
+ file_options_ = FileOptions(options);
+ }
+
+ void CheckFileContents(const std::vector<std::string>& keys,
+ const std::vector<std::string>& values,
+ const std::vector<uint64_t>& expected_locations,
+ std::string expected_unused_bucket,
+ uint64_t expected_table_size,
+ uint32_t expected_num_hash_func,
+ bool expected_is_last_level,
+ uint32_t expected_cuckoo_block_size = 1) {
+ uint64_t num_deletions = 0;
+ for (const auto& key : keys) {
+ ParsedInternalKey parsed;
+ Status pik_status =
+ ParseInternalKey(key, &parsed, true /* log_err_key */);
+ if (pik_status.ok() && parsed.type == kTypeDeletion) {
+ num_deletions++;
+ }
+ }
+ // Read file
+ uint64_t read_file_size;
+ ASSERT_OK(env_->GetFileSize(fname, &read_file_size));
+ std::unique_ptr<RandomAccessFileReader> file_reader;
+ ASSERT_OK(RandomAccessFileReader::Create(
+ env_->GetFileSystem(), fname, file_options_, &file_reader, nullptr));
+
+ Options options;
+ options.allow_mmap_reads = true;
+ ImmutableOptions ioptions(options);
+
+ // Assert Table Properties.
+ std::unique_ptr<TableProperties> props;
+ ASSERT_OK(ReadTableProperties(file_reader.get(), read_file_size,
+ kCuckooTableMagicNumber, ioptions, &props));
+ // Check unused bucket.
+ std::string unused_key =
+ props->user_collected_properties[CuckooTablePropertyNames::kEmptyKey];
+ ASSERT_EQ(expected_unused_bucket.substr(0, props->fixed_key_len),
+ unused_key);
+
+ uint64_t value_len_found = *reinterpret_cast<const uint64_t*>(
+ props->user_collected_properties[CuckooTablePropertyNames::kValueLength]
+ .data());
+ ASSERT_EQ(values.empty() ? 0 : values[0].size(), value_len_found);
+ ASSERT_EQ(props->raw_value_size, values.size() * value_len_found);
+ const uint64_t table_size = *reinterpret_cast<const uint64_t*>(
+ props
+ ->user_collected_properties
+ [CuckooTablePropertyNames::kHashTableSize]
+ .data());
+ ASSERT_EQ(expected_table_size, table_size);
+ const uint32_t num_hash_func_found = *reinterpret_cast<const uint32_t*>(
+ props->user_collected_properties[CuckooTablePropertyNames::kNumHashFunc]
+ .data());
+ ASSERT_EQ(expected_num_hash_func, num_hash_func_found);
+ const uint32_t cuckoo_block_size = *reinterpret_cast<const uint32_t*>(
+ props
+ ->user_collected_properties
+ [CuckooTablePropertyNames::kCuckooBlockSize]
+ .data());
+ ASSERT_EQ(expected_cuckoo_block_size, cuckoo_block_size);
+ const bool is_last_level_found = *reinterpret_cast<const bool*>(
+ props->user_collected_properties[CuckooTablePropertyNames::kIsLastLevel]
+ .data());
+ ASSERT_EQ(expected_is_last_level, is_last_level_found);
+
+ ASSERT_EQ(props->num_entries, keys.size());
+ ASSERT_EQ(props->num_deletions, num_deletions);
+ ASSERT_EQ(props->fixed_key_len, keys.empty() ? 0 : keys[0].size());
+ ASSERT_EQ(props->data_size,
+ expected_unused_bucket.size() *
+ (expected_table_size + expected_cuckoo_block_size - 1));
+ ASSERT_EQ(props->raw_key_size, keys.size() * props->fixed_key_len);
+ ASSERT_EQ(props->column_family_id, 0);
+ ASSERT_EQ(props->column_family_name, kDefaultColumnFamilyName);
+
+ // Check contents of the bucket.
+ std::vector<bool> keys_found(keys.size(), false);
+ size_t bucket_size = expected_unused_bucket.size();
+ for (uint32_t i = 0; i + 1 < table_size + cuckoo_block_size; ++i) {
+ Slice read_slice;
+ ASSERT_OK(file_reader->Read(IOOptions(), i * bucket_size, bucket_size,
+ &read_slice, nullptr, nullptr,
+ Env::IO_TOTAL /* rate_limiter_priority */));
+ size_t key_idx =
+ std::find(expected_locations.begin(), expected_locations.end(), i) -
+ expected_locations.begin();
+ if (key_idx == keys.size()) {
+ // i is not one of the expected locations. Empty bucket.
+ if (read_slice.data() == nullptr) {
+ ASSERT_EQ(0, expected_unused_bucket.size());
+ } else {
+ ASSERT_EQ(read_slice.compare(expected_unused_bucket), 0);
+ }
+ } else {
+ keys_found[key_idx] = true;
+ ASSERT_EQ(read_slice.compare(keys[key_idx] + values[key_idx]), 0);
+ }
+ }
+ for (auto key_found : keys_found) {
+ // Check that all keys wereReader found.
+ ASSERT_TRUE(key_found);
+ }
+ }
+
+ std::string GetInternalKey(Slice user_key, bool zero_seqno,
+ ValueType type = kTypeValue) {
+ IterKey ikey;
+ ikey.SetInternalKey(user_key, zero_seqno ? 0 : 1000, type);
+ return ikey.GetInternalKey().ToString();
+ }
+
+ uint64_t NextPowOf2(uint64_t num) {
+ uint64_t n = 2;
+ while (n <= num) {
+ n *= 2;
+ }
+ return n;
+ }
+
+ uint64_t GetExpectedTableSize(uint64_t num) {
+ return NextPowOf2(static_cast<uint64_t>(num / kHashTableRatio));
+ }
+
+ Env* env_;
+ FileOptions file_options_;
+ std::string fname;
+ const double kHashTableRatio = 0.9;
+};
+
+TEST_F(CuckooBuilderTest, SuccessWithEmptyFile) {
+ std::unique_ptr<WritableFile> writable_file;
+ fname = test::PerThreadDBPath("EmptyFile");
+ std::unique_ptr<WritableFileWriter> file_writer;
+ ASSERT_OK(WritableFileWriter::Create(env_->GetFileSystem(), fname,
+ file_options_, &file_writer, nullptr));
+ CuckooTableBuilder builder(file_writer.get(), kHashTableRatio, 4, 100,
+ BytewiseComparator(), 1, false, false,
+ GetSliceHash, 0 /* column_family_id */,
+ kDefaultColumnFamilyName);
+ ASSERT_OK(builder.status());
+ ASSERT_EQ(0UL, builder.FileSize());
+ ASSERT_OK(builder.Finish());
+ ASSERT_OK(file_writer->Close());
+ CheckFileContents({}, {}, {}, "", 2, 2, false);
+}
+
+TEST_F(CuckooBuilderTest, WriteSuccessNoCollisionFullKey) {
+ for (auto type : {kTypeValue, kTypeDeletion}) {
+ uint32_t num_hash_fun = 4;
+ std::vector<std::string> user_keys = {"key01", "key02", "key03", "key04"};
+ std::vector<std::string> values;
+ if (type == kTypeValue) {
+ values = {"v01", "v02", "v03", "v04"};
+ } else {
+ values = {"", "", "", ""};
+ }
+ // Need to have a temporary variable here as VS compiler does not currently
+ // support operator= with initializer_list as a parameter
+ std::unordered_map<std::string, std::vector<uint64_t>> hm = {
+ {user_keys[0], {0, 1, 2, 3}},
+ {user_keys[1], {1, 2, 3, 4}},
+ {user_keys[2], {2, 3, 4, 5}},
+ {user_keys[3], {3, 4, 5, 6}}};
+ hash_map = std::move(hm);
+
+ std::vector<uint64_t> expected_locations = {0, 1, 2, 3};
+ std::vector<std::string> keys;
+ for (auto& user_key : user_keys) {
+ keys.push_back(GetInternalKey(user_key, false, type));
+ }
+ uint64_t expected_table_size = GetExpectedTableSize(keys.size());
+
+ fname = test::PerThreadDBPath("NoCollisionFullKey");
+ std::unique_ptr<WritableFileWriter> file_writer;
+ ASSERT_OK(WritableFileWriter::Create(env_->GetFileSystem(), fname,
+ file_options_, &file_writer, nullptr));
+ CuckooTableBuilder builder(file_writer.get(), kHashTableRatio, num_hash_fun,
+ 100, BytewiseComparator(), 1, false, false,
+ GetSliceHash, 0 /* column_family_id */,
+ kDefaultColumnFamilyName);
+ ASSERT_OK(builder.status());
+ for (uint32_t i = 0; i < user_keys.size(); i++) {
+ builder.Add(Slice(keys[i]), Slice(values[i]));
+ ASSERT_EQ(builder.NumEntries(), i + 1);
+ ASSERT_OK(builder.status());
+ }
+ size_t bucket_size = keys[0].size() + values[0].size();
+ ASSERT_EQ(expected_table_size * bucket_size - 1, builder.FileSize());
+ ASSERT_OK(builder.Finish());
+ ASSERT_OK(file_writer->Close());
+ ASSERT_LE(expected_table_size * bucket_size, builder.FileSize());
+
+ std::string expected_unused_bucket = GetInternalKey("key00", true);
+ expected_unused_bucket += std::string(values[0].size(), 'a');
+ CheckFileContents(keys, values, expected_locations, expected_unused_bucket,
+ expected_table_size, 2, false);
+ }
+}
+
+TEST_F(CuckooBuilderTest, WriteSuccessWithCollisionFullKey) {
+ uint32_t num_hash_fun = 4;
+ std::vector<std::string> user_keys = {"key01", "key02", "key03", "key04"};
+ std::vector<std::string> values = {"v01", "v02", "v03", "v04"};
+ // Need to have a temporary variable here as VS compiler does not currently
+ // support operator= with initializer_list as a parameter
+ std::unordered_map<std::string, std::vector<uint64_t>> hm = {
+ {user_keys[0], {0, 1, 2, 3}},
+ {user_keys[1], {0, 1, 2, 3}},
+ {user_keys[2], {0, 1, 2, 3}},
+ {user_keys[3], {0, 1, 2, 3}},
+ };
+ hash_map = std::move(hm);
+
+ std::vector<uint64_t> expected_locations = {0, 1, 2, 3};
+ std::vector<std::string> keys;
+ for (auto& user_key : user_keys) {
+ keys.push_back(GetInternalKey(user_key, false));
+ }
+ uint64_t expected_table_size = GetExpectedTableSize(keys.size());
+
+ fname = test::PerThreadDBPath("WithCollisionFullKey");
+ std::unique_ptr<WritableFileWriter> file_writer;
+ ASSERT_OK(WritableFileWriter::Create(env_->GetFileSystem(), fname,
+ file_options_, &file_writer, nullptr));
+ CuckooTableBuilder builder(file_writer.get(), kHashTableRatio, num_hash_fun,
+ 100, BytewiseComparator(), 1, false, false,
+ GetSliceHash, 0 /* column_family_id */,
+ kDefaultColumnFamilyName);
+ ASSERT_OK(builder.status());
+ for (uint32_t i = 0; i < user_keys.size(); i++) {
+ builder.Add(Slice(keys[i]), Slice(values[i]));
+ ASSERT_EQ(builder.NumEntries(), i + 1);
+ ASSERT_OK(builder.status());
+ }
+ size_t bucket_size = keys[0].size() + values[0].size();
+ ASSERT_EQ(expected_table_size * bucket_size - 1, builder.FileSize());
+ ASSERT_OK(builder.Finish());
+ ASSERT_OK(file_writer->Close());
+ ASSERT_LE(expected_table_size * bucket_size, builder.FileSize());
+
+ std::string expected_unused_bucket = GetInternalKey("key00", true);
+ expected_unused_bucket += std::string(values[0].size(), 'a');
+ CheckFileContents(keys, values, expected_locations, expected_unused_bucket,
+ expected_table_size, 4, false);
+}
+
+TEST_F(CuckooBuilderTest, WriteSuccessWithCollisionAndCuckooBlock) {
+ uint32_t num_hash_fun = 4;
+ std::vector<std::string> user_keys = {"key01", "key02", "key03", "key04"};
+ std::vector<std::string> values = {"v01", "v02", "v03", "v04"};
+ // Need to have a temporary variable here as VS compiler does not currently
+ // support operator= with initializer_list as a parameter
+ std::unordered_map<std::string, std::vector<uint64_t>> hm = {
+ {user_keys[0], {0, 1, 2, 3}},
+ {user_keys[1], {0, 1, 2, 3}},
+ {user_keys[2], {0, 1, 2, 3}},
+ {user_keys[3], {0, 1, 2, 3}},
+ };
+ hash_map = std::move(hm);
+
+ std::vector<uint64_t> expected_locations = {0, 1, 2, 3};
+ std::vector<std::string> keys;
+ for (auto& user_key : user_keys) {
+ keys.push_back(GetInternalKey(user_key, false));
+ }
+ uint64_t expected_table_size = GetExpectedTableSize(keys.size());
+
+ std::unique_ptr<WritableFileWriter> file_writer;
+ uint32_t cuckoo_block_size = 2;
+ fname = test::PerThreadDBPath("WithCollisionFullKey2");
+ ASSERT_OK(WritableFileWriter::Create(env_->GetFileSystem(), fname,
+ file_options_, &file_writer, nullptr));
+ CuckooTableBuilder builder(
+ file_writer.get(), kHashTableRatio, num_hash_fun, 100,
+ BytewiseComparator(), cuckoo_block_size, false, false, GetSliceHash,
+ 0 /* column_family_id */, kDefaultColumnFamilyName);
+ ASSERT_OK(builder.status());
+ for (uint32_t i = 0; i < user_keys.size(); i++) {
+ builder.Add(Slice(keys[i]), Slice(values[i]));
+ ASSERT_EQ(builder.NumEntries(), i + 1);
+ ASSERT_OK(builder.status());
+ }
+ size_t bucket_size = keys[0].size() + values[0].size();
+ ASSERT_EQ(expected_table_size * bucket_size - 1, builder.FileSize());
+ ASSERT_OK(builder.Finish());
+ ASSERT_OK(file_writer->Close());
+ ASSERT_LE(expected_table_size * bucket_size, builder.FileSize());
+
+ std::string expected_unused_bucket = GetInternalKey("key00", true);
+ expected_unused_bucket += std::string(values[0].size(), 'a');
+ CheckFileContents(keys, values, expected_locations, expected_unused_bucket,
+ expected_table_size, 3, false, cuckoo_block_size);
+}
+
+TEST_F(CuckooBuilderTest, WithCollisionPathFullKey) {
+ // Have two hash functions. Insert elements with overlapping hashes.
+ // Finally insert an element with hash value somewhere in the middle
+ // so that it displaces all the elements after that.
+ uint32_t num_hash_fun = 2;
+ std::vector<std::string> user_keys = {"key01", "key02", "key03", "key04",
+ "key05"};
+ std::vector<std::string> values = {"v01", "v02", "v03", "v04", "v05"};
+ // Need to have a temporary variable here as VS compiler does not currently
+ // support operator= with initializer_list as a parameter
+ std::unordered_map<std::string, std::vector<uint64_t>> hm = {
+ {user_keys[0], {0, 1}}, {user_keys[1], {1, 2}}, {user_keys[2], {2, 3}},
+ {user_keys[3], {3, 4}}, {user_keys[4], {0, 2}},
+ };
+ hash_map = std::move(hm);
+
+ std::vector<uint64_t> expected_locations = {0, 1, 3, 4, 2};
+ std::vector<std::string> keys;
+ for (auto& user_key : user_keys) {
+ keys.push_back(GetInternalKey(user_key, false));
+ }
+ uint64_t expected_table_size = GetExpectedTableSize(keys.size());
+
+ std::unique_ptr<WritableFileWriter> file_writer;
+ fname = test::PerThreadDBPath("WithCollisionPathFullKey");
+ ASSERT_OK(WritableFileWriter::Create(env_->GetFileSystem(), fname,
+ file_options_, &file_writer, nullptr));
+ CuckooTableBuilder builder(file_writer.get(), kHashTableRatio, num_hash_fun,
+ 100, BytewiseComparator(), 1, false, false,
+ GetSliceHash, 0 /* column_family_id */,
+ kDefaultColumnFamilyName);
+ ASSERT_OK(builder.status());
+ for (uint32_t i = 0; i < user_keys.size(); i++) {
+ builder.Add(Slice(keys[i]), Slice(values[i]));
+ ASSERT_EQ(builder.NumEntries(), i + 1);
+ ASSERT_OK(builder.status());
+ }
+ size_t bucket_size = keys[0].size() + values[0].size();
+ ASSERT_EQ(expected_table_size * bucket_size - 1, builder.FileSize());
+ ASSERT_OK(builder.Finish());
+ ASSERT_OK(file_writer->Close());
+ ASSERT_LE(expected_table_size * bucket_size, builder.FileSize());
+
+ std::string expected_unused_bucket = GetInternalKey("key00", true);
+ expected_unused_bucket += std::string(values[0].size(), 'a');
+ CheckFileContents(keys, values, expected_locations, expected_unused_bucket,
+ expected_table_size, 2, false);
+}
+
+TEST_F(CuckooBuilderTest, WithCollisionPathFullKeyAndCuckooBlock) {
+ uint32_t num_hash_fun = 2;
+ std::vector<std::string> user_keys = {"key01", "key02", "key03", "key04",
+ "key05"};
+ std::vector<std::string> values = {"v01", "v02", "v03", "v04", "v05"};
+ // Need to have a temporary variable here as VS compiler does not currently
+ // support operator= with initializer_list as a parameter
+ std::unordered_map<std::string, std::vector<uint64_t>> hm = {
+ {user_keys[0], {0, 1}}, {user_keys[1], {1, 2}}, {user_keys[2], {3, 4}},
+ {user_keys[3], {4, 5}}, {user_keys[4], {0, 3}},
+ };
+ hash_map = std::move(hm);
+
+ std::vector<uint64_t> expected_locations = {2, 1, 3, 4, 0};
+ std::vector<std::string> keys;
+ for (auto& user_key : user_keys) {
+ keys.push_back(GetInternalKey(user_key, false));
+ }
+ uint64_t expected_table_size = GetExpectedTableSize(keys.size());
+
+ std::unique_ptr<WritableFileWriter> file_writer;
+ fname = test::PerThreadDBPath("WithCollisionPathFullKeyAndCuckooBlock");
+ ASSERT_OK(WritableFileWriter::Create(env_->GetFileSystem(), fname,
+ file_options_, &file_writer, nullptr));
+ CuckooTableBuilder builder(file_writer.get(), kHashTableRatio, num_hash_fun,
+ 100, BytewiseComparator(), 2, false, false,
+ GetSliceHash, 0 /* column_family_id */,
+ kDefaultColumnFamilyName);
+ ASSERT_OK(builder.status());
+ for (uint32_t i = 0; i < user_keys.size(); i++) {
+ builder.Add(Slice(keys[i]), Slice(values[i]));
+ ASSERT_EQ(builder.NumEntries(), i + 1);
+ ASSERT_OK(builder.status());
+ }
+ size_t bucket_size = keys[0].size() + values[0].size();
+ ASSERT_EQ(expected_table_size * bucket_size - 1, builder.FileSize());
+ ASSERT_OK(builder.Finish());
+ ASSERT_OK(file_writer->Close());
+ ASSERT_LE(expected_table_size * bucket_size, builder.FileSize());
+
+ std::string expected_unused_bucket = GetInternalKey("key00", true);
+ expected_unused_bucket += std::string(values[0].size(), 'a');
+ CheckFileContents(keys, values, expected_locations, expected_unused_bucket,
+ expected_table_size, 2, false, 2);
+}
+
+TEST_F(CuckooBuilderTest, WriteSuccessNoCollisionUserKey) {
+ uint32_t num_hash_fun = 4;
+ std::vector<std::string> user_keys = {"key01", "key02", "key03", "key04"};
+ std::vector<std::string> values = {"v01", "v02", "v03", "v04"};
+ // Need to have a temporary variable here as VS compiler does not currently
+ // support operator= with initializer_list as a parameter
+ std::unordered_map<std::string, std::vector<uint64_t>> hm = {
+ {user_keys[0], {0, 1, 2, 3}},
+ {user_keys[1], {1, 2, 3, 4}},
+ {user_keys[2], {2, 3, 4, 5}},
+ {user_keys[3], {3, 4, 5, 6}}};
+ hash_map = std::move(hm);
+
+ std::vector<uint64_t> expected_locations = {0, 1, 2, 3};
+ uint64_t expected_table_size = GetExpectedTableSize(user_keys.size());
+
+ std::unique_ptr<WritableFileWriter> file_writer;
+ fname = test::PerThreadDBPath("NoCollisionUserKey");
+ ASSERT_OK(WritableFileWriter::Create(env_->GetFileSystem(), fname,
+ file_options_, &file_writer, nullptr));
+
+ CuckooTableBuilder builder(file_writer.get(), kHashTableRatio, num_hash_fun,
+ 100, BytewiseComparator(), 1, false, false,
+ GetSliceHash, 0 /* column_family_id */,
+ kDefaultColumnFamilyName);
+ ASSERT_OK(builder.status());
+ for (uint32_t i = 0; i < user_keys.size(); i++) {
+ builder.Add(Slice(GetInternalKey(user_keys[i], true)), Slice(values[i]));
+ ASSERT_EQ(builder.NumEntries(), i + 1);
+ ASSERT_OK(builder.status());
+ }
+ size_t bucket_size = user_keys[0].size() + values[0].size();
+ ASSERT_EQ(expected_table_size * bucket_size - 1, builder.FileSize());
+ ASSERT_OK(builder.Finish());
+ ASSERT_OK(file_writer->Close());
+ ASSERT_LE(expected_table_size * bucket_size, builder.FileSize());
+
+ std::string expected_unused_bucket = "key00";
+ expected_unused_bucket += std::string(values[0].size(), 'a');
+ CheckFileContents(user_keys, values, expected_locations,
+ expected_unused_bucket, expected_table_size, 2, true);
+}
+
+TEST_F(CuckooBuilderTest, WriteSuccessWithCollisionUserKey) {
+ uint32_t num_hash_fun = 4;
+ std::vector<std::string> user_keys = {"key01", "key02", "key03", "key04"};
+ std::vector<std::string> values = {"v01", "v02", "v03", "v04"};
+ // Need to have a temporary variable here as VS compiler does not currently
+ // support operator= with initializer_list as a parameter
+ std::unordered_map<std::string, std::vector<uint64_t>> hm = {
+ {user_keys[0], {0, 1, 2, 3}},
+ {user_keys[1], {0, 1, 2, 3}},
+ {user_keys[2], {0, 1, 2, 3}},
+ {user_keys[3], {0, 1, 2, 3}},
+ };
+ hash_map = std::move(hm);
+
+ std::vector<uint64_t> expected_locations = {0, 1, 2, 3};
+ uint64_t expected_table_size = GetExpectedTableSize(user_keys.size());
+
+ std::unique_ptr<WritableFileWriter> file_writer;
+ fname = test::PerThreadDBPath("WithCollisionUserKey");
+ ASSERT_OK(WritableFileWriter::Create(env_->GetFileSystem(), fname,
+ file_options_, &file_writer, nullptr));
+
+ CuckooTableBuilder builder(file_writer.get(), kHashTableRatio, num_hash_fun,
+ 100, BytewiseComparator(), 1, false, false,
+ GetSliceHash, 0 /* column_family_id */,
+ kDefaultColumnFamilyName);
+ ASSERT_OK(builder.status());
+ for (uint32_t i = 0; i < user_keys.size(); i++) {
+ builder.Add(Slice(GetInternalKey(user_keys[i], true)), Slice(values[i]));
+ ASSERT_EQ(builder.NumEntries(), i + 1);
+ ASSERT_OK(builder.status());
+ }
+ size_t bucket_size = user_keys[0].size() + values[0].size();
+ ASSERT_EQ(expected_table_size * bucket_size - 1, builder.FileSize());
+ ASSERT_OK(builder.Finish());
+ ASSERT_OK(file_writer->Close());
+ ASSERT_LE(expected_table_size * bucket_size, builder.FileSize());
+
+ std::string expected_unused_bucket = "key00";
+ expected_unused_bucket += std::string(values[0].size(), 'a');
+ CheckFileContents(user_keys, values, expected_locations,
+ expected_unused_bucket, expected_table_size, 4, true);
+}
+
+TEST_F(CuckooBuilderTest, WithCollisionPathUserKey) {
+ uint32_t num_hash_fun = 2;
+ std::vector<std::string> user_keys = {"key01", "key02", "key03", "key04",
+ "key05"};
+ std::vector<std::string> values = {"v01", "v02", "v03", "v04", "v05"};
+ // Need to have a temporary variable here as VS compiler does not currently
+ // support operator= with initializer_list as a parameter
+ std::unordered_map<std::string, std::vector<uint64_t>> hm = {
+ {user_keys[0], {0, 1}}, {user_keys[1], {1, 2}}, {user_keys[2], {2, 3}},
+ {user_keys[3], {3, 4}}, {user_keys[4], {0, 2}},
+ };
+ hash_map = std::move(hm);
+
+ std::vector<uint64_t> expected_locations = {0, 1, 3, 4, 2};
+ uint64_t expected_table_size = GetExpectedTableSize(user_keys.size());
+
+ std::unique_ptr<WritableFileWriter> file_writer;
+ fname = test::PerThreadDBPath("WithCollisionPathUserKey");
+ ASSERT_OK(WritableFileWriter::Create(env_->GetFileSystem(), fname,
+ file_options_, &file_writer, nullptr));
+
+ CuckooTableBuilder builder(file_writer.get(), kHashTableRatio, num_hash_fun,
+ 2, BytewiseComparator(), 1, false, false,
+ GetSliceHash, 0 /* column_family_id */,
+ kDefaultColumnFamilyName);
+ ASSERT_OK(builder.status());
+ for (uint32_t i = 0; i < user_keys.size(); i++) {
+ builder.Add(Slice(GetInternalKey(user_keys[i], true)), Slice(values[i]));
+ ASSERT_EQ(builder.NumEntries(), i + 1);
+ ASSERT_OK(builder.status());
+ }
+ size_t bucket_size = user_keys[0].size() + values[0].size();
+ ASSERT_EQ(expected_table_size * bucket_size - 1, builder.FileSize());
+ ASSERT_OK(builder.Finish());
+ ASSERT_OK(file_writer->Close());
+ ASSERT_LE(expected_table_size * bucket_size, builder.FileSize());
+
+ std::string expected_unused_bucket = "key00";
+ expected_unused_bucket += std::string(values[0].size(), 'a');
+ CheckFileContents(user_keys, values, expected_locations,
+ expected_unused_bucket, expected_table_size, 2, true);
+}
+
+TEST_F(CuckooBuilderTest, FailWhenCollisionPathTooLong) {
+ // Have two hash functions. Insert elements with overlapping hashes.
+ // Finally try inserting an element with hash value somewhere in the middle
+ // and it should fail because the no. of elements to displace is too high.
+ uint32_t num_hash_fun = 2;
+ std::vector<std::string> user_keys = {"key01", "key02", "key03", "key04",
+ "key05"};
+ // Need to have a temporary variable here as VS compiler does not currently
+ // support operator= with initializer_list as a parameter
+ std::unordered_map<std::string, std::vector<uint64_t>> hm = {
+ {user_keys[0], {0, 1}}, {user_keys[1], {1, 2}}, {user_keys[2], {2, 3}},
+ {user_keys[3], {3, 4}}, {user_keys[4], {0, 1}},
+ };
+ hash_map = std::move(hm);
+
+ std::unique_ptr<WritableFileWriter> file_writer;
+ fname = test::PerThreadDBPath("WithCollisionPathUserKey");
+ ASSERT_OK(WritableFileWriter::Create(env_->GetFileSystem(), fname,
+ file_options_, &file_writer, nullptr));
+ CuckooTableBuilder builder(file_writer.get(), kHashTableRatio, num_hash_fun,
+ 2, BytewiseComparator(), 1, false, false,
+ GetSliceHash, 0 /* column_family_id */,
+ kDefaultColumnFamilyName);
+ ASSERT_OK(builder.status());
+ for (uint32_t i = 0; i < user_keys.size(); i++) {
+ builder.Add(Slice(GetInternalKey(user_keys[i], false)), Slice("value"));
+ ASSERT_EQ(builder.NumEntries(), i + 1);
+ ASSERT_OK(builder.status());
+ }
+ ASSERT_TRUE(builder.Finish().IsNotSupported());
+ ASSERT_OK(file_writer->Close());
+}
+
+TEST_F(CuckooBuilderTest, FailWhenSameKeyInserted) {
+ // Need to have a temporary variable here as VS compiler does not currently
+ // support operator= with initializer_list as a parameter
+ std::unordered_map<std::string, std::vector<uint64_t>> hm = {
+ {"repeatedkey", {0, 1, 2, 3}}};
+ hash_map = std::move(hm);
+ uint32_t num_hash_fun = 4;
+ std::string user_key = "repeatedkey";
+
+ std::unique_ptr<WritableFileWriter> file_writer;
+ fname = test::PerThreadDBPath("FailWhenSameKeyInserted");
+ ASSERT_OK(WritableFileWriter::Create(env_->GetFileSystem(), fname,
+ file_options_, &file_writer, nullptr));
+ CuckooTableBuilder builder(file_writer.get(), kHashTableRatio, num_hash_fun,
+ 100, BytewiseComparator(), 1, false, false,
+ GetSliceHash, 0 /* column_family_id */,
+ kDefaultColumnFamilyName);
+ ASSERT_OK(builder.status());
+
+ builder.Add(Slice(GetInternalKey(user_key, false)), Slice("value1"));
+ ASSERT_EQ(builder.NumEntries(), 1u);
+ ASSERT_OK(builder.status());
+ builder.Add(Slice(GetInternalKey(user_key, true)), Slice("value2"));
+ ASSERT_EQ(builder.NumEntries(), 2u);
+ ASSERT_OK(builder.status());
+
+ ASSERT_TRUE(builder.Finish().IsNotSupported());
+ ASSERT_OK(file_writer->Close());
+}
+} // namespace ROCKSDB_NAMESPACE
+
+int main(int argc, char** argv) {
+ ROCKSDB_NAMESPACE::port::InstallStackTraceHandler();
+ ::testing::InitGoogleTest(&argc, argv);
+ return RUN_ALL_TESTS();
+}
+
+#else
+#include <stdio.h>
+
+int main(int /*argc*/, char** /*argv*/) {
+ fprintf(stderr, "SKIPPED as Cuckoo table is not supported in ROCKSDB_LITE\n");
+ return 0;
+}
+
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/table/cuckoo/cuckoo_table_factory.cc b/src/rocksdb/table/cuckoo/cuckoo_table_factory.cc
new file mode 100644
index 000000000..1253c92dd
--- /dev/null
+++ b/src/rocksdb/table/cuckoo/cuckoo_table_factory.cc
@@ -0,0 +1,104 @@
+// Copyright (c) 2011-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).
+
+#ifndef ROCKSDB_LITE
+#include "table/cuckoo/cuckoo_table_factory.h"
+
+#include "db/dbformat.h"
+#include "options/configurable_helper.h"
+#include "rocksdb/utilities/options_type.h"
+#include "table/cuckoo/cuckoo_table_builder.h"
+#include "table/cuckoo/cuckoo_table_reader.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+Status CuckooTableFactory::NewTableReader(
+ const ReadOptions& /*ro*/, const TableReaderOptions& table_reader_options,
+ std::unique_ptr<RandomAccessFileReader>&& file, uint64_t file_size,
+ std::unique_ptr<TableReader>* table,
+ bool /*prefetch_index_and_filter_in_cache*/) const {
+ std::unique_ptr<CuckooTableReader> new_reader(new CuckooTableReader(
+ table_reader_options.ioptions, std::move(file), file_size,
+ table_reader_options.internal_comparator.user_comparator(), nullptr));
+ Status s = new_reader->status();
+ if (s.ok()) {
+ *table = std::move(new_reader);
+ }
+ return s;
+}
+
+TableBuilder* CuckooTableFactory::NewTableBuilder(
+ const TableBuilderOptions& table_builder_options,
+ WritableFileWriter* file) const {
+ // TODO: change builder to take the option struct
+ return new CuckooTableBuilder(
+ file, table_options_.hash_table_ratio, 64,
+ table_options_.max_search_depth,
+ table_builder_options.internal_comparator.user_comparator(),
+ table_options_.cuckoo_block_size, table_options_.use_module_hash,
+ table_options_.identity_as_first_hash, nullptr /* get_slice_hash */,
+ table_builder_options.column_family_id,
+ table_builder_options.column_family_name, table_builder_options.db_id,
+ table_builder_options.db_session_id, table_builder_options.cur_file_num);
+}
+
+std::string CuckooTableFactory::GetPrintableOptions() const {
+ std::string ret;
+ ret.reserve(2000);
+ const int kBufferSize = 200;
+ char buffer[kBufferSize];
+
+ snprintf(buffer, kBufferSize, " hash_table_ratio: %lf\n",
+ table_options_.hash_table_ratio);
+ ret.append(buffer);
+ snprintf(buffer, kBufferSize, " max_search_depth: %u\n",
+ table_options_.max_search_depth);
+ ret.append(buffer);
+ snprintf(buffer, kBufferSize, " cuckoo_block_size: %u\n",
+ table_options_.cuckoo_block_size);
+ ret.append(buffer);
+ snprintf(buffer, kBufferSize, " identity_as_first_hash: %d\n",
+ table_options_.identity_as_first_hash);
+ ret.append(buffer);
+ return ret;
+}
+
+static std::unordered_map<std::string, OptionTypeInfo> cuckoo_table_type_info =
+ {
+#ifndef ROCKSDB_LITE
+ {"hash_table_ratio",
+ {offsetof(struct CuckooTableOptions, hash_table_ratio),
+ OptionType::kDouble, OptionVerificationType::kNormal,
+ OptionTypeFlags::kNone}},
+ {"max_search_depth",
+ {offsetof(struct CuckooTableOptions, max_search_depth),
+ OptionType::kUInt32T, OptionVerificationType::kNormal,
+ OptionTypeFlags::kNone}},
+ {"cuckoo_block_size",
+ {offsetof(struct CuckooTableOptions, cuckoo_block_size),
+ OptionType::kUInt32T, OptionVerificationType::kNormal,
+ OptionTypeFlags::kNone}},
+ {"identity_as_first_hash",
+ {offsetof(struct CuckooTableOptions, identity_as_first_hash),
+ OptionType::kBoolean, OptionVerificationType::kNormal,
+ OptionTypeFlags::kNone}},
+ {"use_module_hash",
+ {offsetof(struct CuckooTableOptions, use_module_hash),
+ OptionType::kBoolean, OptionVerificationType::kNormal,
+ OptionTypeFlags::kNone}},
+#endif // ROCKSDB_LITE
+};
+
+CuckooTableFactory::CuckooTableFactory(const CuckooTableOptions& table_options)
+ : table_options_(table_options) {
+ RegisterOptions(&table_options_, &cuckoo_table_type_info);
+}
+
+TableFactory* NewCuckooTableFactory(const CuckooTableOptions& table_options) {
+ return new CuckooTableFactory(table_options);
+}
+
+} // namespace ROCKSDB_NAMESPACE
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/table/cuckoo/cuckoo_table_factory.h b/src/rocksdb/table/cuckoo/cuckoo_table_factory.h
new file mode 100644
index 000000000..9937c28dd
--- /dev/null
+++ b/src/rocksdb/table/cuckoo/cuckoo_table_factory.h
@@ -0,0 +1,82 @@
+// Copyright (c) 2011-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).
+
+#pragma once
+#ifndef ROCKSDB_LITE
+
+#include <string>
+
+#include "rocksdb/options.h"
+#include "rocksdb/table.h"
+#include "util/murmurhash.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+const uint32_t kCuckooMurmurSeedMultiplier = 816922183;
+static inline uint64_t CuckooHash(
+ const Slice& user_key, uint32_t hash_cnt, bool use_module_hash,
+ uint64_t table_size_, bool identity_as_first_hash,
+ uint64_t (*get_slice_hash)(const Slice&, uint32_t, uint64_t)) {
+#if !defined NDEBUG || defined OS_WIN
+ // This part is used only in unit tests but we have to keep it for Windows
+ // build as we run test in both debug and release modes under Windows.
+ if (get_slice_hash != nullptr) {
+ return get_slice_hash(user_key, hash_cnt, table_size_);
+ }
+#else
+ (void)get_slice_hash;
+#endif
+
+ uint64_t value = 0;
+ if (hash_cnt == 0 && identity_as_first_hash) {
+ value = (*reinterpret_cast<const int64_t*>(user_key.data()));
+ } else {
+ value = MurmurHash(user_key.data(), static_cast<int>(user_key.size()),
+ kCuckooMurmurSeedMultiplier * hash_cnt);
+ }
+ if (use_module_hash) {
+ return value % table_size_;
+ } else {
+ return value & (table_size_ - 1);
+ }
+}
+
+// Cuckoo Table is designed for applications that require fast point lookups
+// but not fast range scans.
+//
+// Some assumptions:
+// - Key length and Value length are fixed.
+// - Does not support Snapshot.
+// - Does not support Merge operations.
+// - Does not support prefix bloom filters.
+class CuckooTableFactory : public TableFactory {
+ public:
+ explicit CuckooTableFactory(
+ const CuckooTableOptions& table_option = CuckooTableOptions());
+ ~CuckooTableFactory() {}
+
+ // Method to allow CheckedCast to work for this class
+ static const char* kClassName() { return kCuckooTableName(); }
+ const char* Name() const override { return kCuckooTableName(); }
+
+ using TableFactory::NewTableReader;
+ Status NewTableReader(
+ const ReadOptions& ro, const TableReaderOptions& table_reader_options,
+ std::unique_ptr<RandomAccessFileReader>&& file, uint64_t file_size,
+ std::unique_ptr<TableReader>* table,
+ bool prefetch_index_and_filter_in_cache = true) const override;
+
+ TableBuilder* NewTableBuilder(
+ const TableBuilderOptions& table_builder_options,
+ WritableFileWriter* file) const override;
+
+ std::string GetPrintableOptions() const override;
+
+ private:
+ CuckooTableOptions table_options_;
+};
+
+} // namespace ROCKSDB_NAMESPACE
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/table/cuckoo/cuckoo_table_reader.cc b/src/rocksdb/table/cuckoo/cuckoo_table_reader.cc
new file mode 100644
index 000000000..1d70909a6
--- /dev/null
+++ b/src/rocksdb/table/cuckoo/cuckoo_table_reader.cc
@@ -0,0 +1,411 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#ifndef ROCKSDB_LITE
+#include "table/cuckoo/cuckoo_table_reader.h"
+
+#include <algorithm>
+#include <limits>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include "memory/arena.h"
+#include "options/cf_options.h"
+#include "rocksdb/iterator.h"
+#include "rocksdb/table.h"
+#include "table/cuckoo/cuckoo_table_factory.h"
+#include "table/get_context.h"
+#include "table/internal_iterator.h"
+#include "table/meta_blocks.h"
+#include "util/coding.h"
+
+namespace ROCKSDB_NAMESPACE {
+namespace {
+const uint64_t CACHE_LINE_MASK = ~((uint64_t)CACHE_LINE_SIZE - 1);
+const uint32_t kInvalidIndex = std::numeric_limits<uint32_t>::max();
+} // namespace
+
+extern const uint64_t kCuckooTableMagicNumber;
+
+CuckooTableReader::CuckooTableReader(
+ const ImmutableOptions& ioptions,
+ std::unique_ptr<RandomAccessFileReader>&& file, uint64_t file_size,
+ const Comparator* comparator,
+ uint64_t (*get_slice_hash)(const Slice&, uint32_t, uint64_t))
+ : file_(std::move(file)),
+ is_last_level_(false),
+ identity_as_first_hash_(false),
+ use_module_hash_(false),
+ num_hash_func_(0),
+ unused_key_(""),
+ key_length_(0),
+ user_key_length_(0),
+ value_length_(0),
+ bucket_length_(0),
+ cuckoo_block_size_(0),
+ cuckoo_block_bytes_minus_one_(0),
+ table_size_(0),
+ ucomp_(comparator),
+ get_slice_hash_(get_slice_hash) {
+ if (!ioptions.allow_mmap_reads) {
+ status_ = Status::InvalidArgument("File is not mmaped");
+ return;
+ }
+ {
+ std::unique_ptr<TableProperties> props;
+ status_ = ReadTableProperties(file_.get(), file_size,
+ kCuckooTableMagicNumber, ioptions, &props);
+ if (!status_.ok()) {
+ return;
+ }
+ table_props_ = std::move(props);
+ }
+ auto& user_props = table_props_->user_collected_properties;
+ auto hash_funs = user_props.find(CuckooTablePropertyNames::kNumHashFunc);
+ if (hash_funs == user_props.end()) {
+ status_ = Status::Corruption("Number of hash functions not found");
+ return;
+ }
+ num_hash_func_ = *reinterpret_cast<const uint32_t*>(hash_funs->second.data());
+ auto unused_key = user_props.find(CuckooTablePropertyNames::kEmptyKey);
+ if (unused_key == user_props.end()) {
+ status_ = Status::Corruption("Empty bucket value not found");
+ return;
+ }
+ unused_key_ = unused_key->second;
+
+ key_length_ = static_cast<uint32_t>(table_props_->fixed_key_len);
+ auto user_key_len = user_props.find(CuckooTablePropertyNames::kUserKeyLength);
+ if (user_key_len == user_props.end()) {
+ status_ = Status::Corruption("User key length not found");
+ return;
+ }
+ user_key_length_ =
+ *reinterpret_cast<const uint32_t*>(user_key_len->second.data());
+
+ auto value_length = user_props.find(CuckooTablePropertyNames::kValueLength);
+ if (value_length == user_props.end()) {
+ status_ = Status::Corruption("Value length not found");
+ return;
+ }
+ value_length_ =
+ *reinterpret_cast<const uint32_t*>(value_length->second.data());
+ bucket_length_ = key_length_ + value_length_;
+
+ auto hash_table_size =
+ user_props.find(CuckooTablePropertyNames::kHashTableSize);
+ if (hash_table_size == user_props.end()) {
+ status_ = Status::Corruption("Hash table size not found");
+ return;
+ }
+ table_size_ =
+ *reinterpret_cast<const uint64_t*>(hash_table_size->second.data());
+
+ auto is_last_level = user_props.find(CuckooTablePropertyNames::kIsLastLevel);
+ if (is_last_level == user_props.end()) {
+ status_ = Status::Corruption("Is last level not found");
+ return;
+ }
+ is_last_level_ = *reinterpret_cast<const bool*>(is_last_level->second.data());
+
+ auto identity_as_first_hash =
+ user_props.find(CuckooTablePropertyNames::kIdentityAsFirstHash);
+ if (identity_as_first_hash == user_props.end()) {
+ status_ = Status::Corruption("identity as first hash not found");
+ return;
+ }
+ identity_as_first_hash_ =
+ *reinterpret_cast<const bool*>(identity_as_first_hash->second.data());
+
+ auto use_module_hash =
+ user_props.find(CuckooTablePropertyNames::kUseModuleHash);
+ if (use_module_hash == user_props.end()) {
+ status_ = Status::Corruption("hash type is not found");
+ return;
+ }
+ use_module_hash_ =
+ *reinterpret_cast<const bool*>(use_module_hash->second.data());
+ auto cuckoo_block_size =
+ user_props.find(CuckooTablePropertyNames::kCuckooBlockSize);
+ if (cuckoo_block_size == user_props.end()) {
+ status_ = Status::Corruption("Cuckoo block size not found");
+ return;
+ }
+ cuckoo_block_size_ =
+ *reinterpret_cast<const uint32_t*>(cuckoo_block_size->second.data());
+ cuckoo_block_bytes_minus_one_ = cuckoo_block_size_ * bucket_length_ - 1;
+ // TODO: rate limit reads of whole cuckoo tables.
+ status_ =
+ file_->Read(IOOptions(), 0, static_cast<size_t>(file_size), &file_data_,
+ nullptr, nullptr, Env::IO_TOTAL /* rate_limiter_priority */);
+}
+
+Status CuckooTableReader::Get(const ReadOptions& /*readOptions*/,
+ const Slice& key, GetContext* get_context,
+ const SliceTransform* /* prefix_extractor */,
+ bool /*skip_filters*/) {
+ assert(key.size() == key_length_ + (is_last_level_ ? 8 : 0));
+ Slice user_key = ExtractUserKey(key);
+ for (uint32_t hash_cnt = 0; hash_cnt < num_hash_func_; ++hash_cnt) {
+ uint64_t offset =
+ bucket_length_ * CuckooHash(user_key, hash_cnt, use_module_hash_,
+ table_size_, identity_as_first_hash_,
+ get_slice_hash_);
+ const char* bucket = &file_data_.data()[offset];
+ for (uint32_t block_idx = 0; block_idx < cuckoo_block_size_;
+ ++block_idx, bucket += bucket_length_) {
+ if (ucomp_->Equal(Slice(unused_key_.data(), user_key.size()),
+ Slice(bucket, user_key.size()))) {
+ return Status::OK();
+ }
+ // Here, we compare only the user key part as we support only one entry
+ // per user key and we don't support snapshot.
+ if (ucomp_->Equal(user_key, Slice(bucket, user_key.size()))) {
+ Slice value(bucket + key_length_, value_length_);
+ if (is_last_level_) {
+ // Sequence number is not stored at the last level, so we will use
+ // kMaxSequenceNumber since it is unknown. This could cause some
+ // transactions to fail to lock a key due to known sequence number.
+ // However, it is expected for anyone to use a CuckooTable in a
+ // TransactionDB.
+ get_context->SaveValue(value, kMaxSequenceNumber);
+ } else {
+ Slice full_key(bucket, key_length_);
+ ParsedInternalKey found_ikey;
+ Status s = ParseInternalKey(full_key, &found_ikey,
+ false /* log_err_key */); // TODO
+ if (!s.ok()) return s;
+ bool dont_care __attribute__((__unused__));
+ get_context->SaveValue(found_ikey, value, &dont_care);
+ }
+ // We don't support merge operations. So, we return here.
+ return Status::OK();
+ }
+ }
+ }
+ return Status::OK();
+}
+
+void CuckooTableReader::Prepare(const Slice& key) {
+ // Prefetch the first Cuckoo Block.
+ Slice user_key = ExtractUserKey(key);
+ uint64_t addr =
+ reinterpret_cast<uint64_t>(file_data_.data()) +
+ bucket_length_ * CuckooHash(user_key, 0, use_module_hash_, table_size_,
+ identity_as_first_hash_, nullptr);
+ uint64_t end_addr = addr + cuckoo_block_bytes_minus_one_;
+ for (addr &= CACHE_LINE_MASK; addr < end_addr; addr += CACHE_LINE_SIZE) {
+ PREFETCH(reinterpret_cast<const char*>(addr), 0, 3);
+ }
+}
+
+class CuckooTableIterator : public InternalIterator {
+ public:
+ explicit CuckooTableIterator(CuckooTableReader* reader);
+ // No copying allowed
+ CuckooTableIterator(const CuckooTableIterator&) = delete;
+ void operator=(const Iterator&) = delete;
+ ~CuckooTableIterator() override {}
+ bool Valid() const override;
+ void SeekToFirst() override;
+ void SeekToLast() override;
+ void Seek(const Slice& target) override;
+ void SeekForPrev(const Slice& target) override;
+ void Next() override;
+ void Prev() override;
+ Slice key() const override;
+ Slice value() const override;
+ Status status() const override { return Status::OK(); }
+ void InitIfNeeded();
+
+ private:
+ struct BucketComparator {
+ BucketComparator(const Slice& file_data, const Comparator* ucomp,
+ uint32_t bucket_len, uint32_t user_key_len,
+ const Slice& target = Slice())
+ : file_data_(file_data),
+ ucomp_(ucomp),
+ bucket_len_(bucket_len),
+ user_key_len_(user_key_len),
+ target_(target) {}
+ bool operator()(const uint32_t first, const uint32_t second) const {
+ const char* first_bucket = (first == kInvalidIndex)
+ ? target_.data()
+ : &file_data_.data()[first * bucket_len_];
+ const char* second_bucket =
+ (second == kInvalidIndex) ? target_.data()
+ : &file_data_.data()[second * bucket_len_];
+ return ucomp_->Compare(Slice(first_bucket, user_key_len_),
+ Slice(second_bucket, user_key_len_)) < 0;
+ }
+
+ private:
+ const Slice file_data_;
+ const Comparator* ucomp_;
+ const uint32_t bucket_len_;
+ const uint32_t user_key_len_;
+ const Slice target_;
+ };
+
+ const BucketComparator bucket_comparator_;
+ void PrepareKVAtCurrIdx();
+ CuckooTableReader* reader_;
+ bool initialized_;
+ // Contains a map of keys to bucket_id sorted in key order.
+ std::vector<uint32_t> sorted_bucket_ids_;
+ // We assume that the number of items can be stored in uint32 (4 Billion).
+ uint32_t curr_key_idx_;
+ Slice curr_value_;
+ IterKey curr_key_;
+};
+
+CuckooTableIterator::CuckooTableIterator(CuckooTableReader* reader)
+ : bucket_comparator_(reader->file_data_, reader->ucomp_,
+ reader->bucket_length_, reader->user_key_length_),
+ reader_(reader),
+ initialized_(false),
+ curr_key_idx_(kInvalidIndex) {
+ sorted_bucket_ids_.clear();
+ curr_value_.clear();
+ curr_key_.Clear();
+}
+
+void CuckooTableIterator::InitIfNeeded() {
+ if (initialized_) {
+ return;
+ }
+ sorted_bucket_ids_.reserve(
+ static_cast<size_t>(reader_->GetTableProperties()->num_entries));
+ uint64_t num_buckets = reader_->table_size_ + reader_->cuckoo_block_size_ - 1;
+ assert(num_buckets < kInvalidIndex);
+ const char* bucket = reader_->file_data_.data();
+ for (uint32_t bucket_id = 0; bucket_id < num_buckets; ++bucket_id) {
+ if (Slice(bucket, reader_->key_length_) != Slice(reader_->unused_key_)) {
+ sorted_bucket_ids_.push_back(bucket_id);
+ }
+ bucket += reader_->bucket_length_;
+ }
+ assert(sorted_bucket_ids_.size() ==
+ reader_->GetTableProperties()->num_entries);
+ std::sort(sorted_bucket_ids_.begin(), sorted_bucket_ids_.end(),
+ bucket_comparator_);
+ curr_key_idx_ = kInvalidIndex;
+ initialized_ = true;
+}
+
+void CuckooTableIterator::SeekToFirst() {
+ InitIfNeeded();
+ curr_key_idx_ = 0;
+ PrepareKVAtCurrIdx();
+}
+
+void CuckooTableIterator::SeekToLast() {
+ InitIfNeeded();
+ curr_key_idx_ = static_cast<uint32_t>(sorted_bucket_ids_.size()) - 1;
+ PrepareKVAtCurrIdx();
+}
+
+void CuckooTableIterator::Seek(const Slice& target) {
+ InitIfNeeded();
+ const BucketComparator seek_comparator(
+ reader_->file_data_, reader_->ucomp_, reader_->bucket_length_,
+ reader_->user_key_length_, ExtractUserKey(target));
+ auto seek_it =
+ std::lower_bound(sorted_bucket_ids_.begin(), sorted_bucket_ids_.end(),
+ kInvalidIndex, seek_comparator);
+ curr_key_idx_ =
+ static_cast<uint32_t>(std::distance(sorted_bucket_ids_.begin(), seek_it));
+ PrepareKVAtCurrIdx();
+}
+
+void CuckooTableIterator::SeekForPrev(const Slice& /*target*/) {
+ // Not supported
+ assert(false);
+}
+
+bool CuckooTableIterator::Valid() const {
+ return curr_key_idx_ < sorted_bucket_ids_.size();
+}
+
+void CuckooTableIterator::PrepareKVAtCurrIdx() {
+ if (!Valid()) {
+ curr_value_.clear();
+ curr_key_.Clear();
+ return;
+ }
+ uint32_t id = sorted_bucket_ids_[curr_key_idx_];
+ const char* offset =
+ reader_->file_data_.data() + id * reader_->bucket_length_;
+ if (reader_->is_last_level_) {
+ // Always return internal key.
+ curr_key_.SetInternalKey(Slice(offset, reader_->user_key_length_), 0,
+ kTypeValue);
+ } else {
+ curr_key_.SetInternalKey(Slice(offset, reader_->key_length_));
+ }
+ curr_value_ = Slice(offset + reader_->key_length_, reader_->value_length_);
+}
+
+void CuckooTableIterator::Next() {
+ if (!Valid()) {
+ curr_value_.clear();
+ curr_key_.Clear();
+ return;
+ }
+ ++curr_key_idx_;
+ PrepareKVAtCurrIdx();
+}
+
+void CuckooTableIterator::Prev() {
+ if (curr_key_idx_ == 0) {
+ curr_key_idx_ = static_cast<uint32_t>(sorted_bucket_ids_.size());
+ }
+ if (!Valid()) {
+ curr_value_.clear();
+ curr_key_.Clear();
+ return;
+ }
+ --curr_key_idx_;
+ PrepareKVAtCurrIdx();
+}
+
+Slice CuckooTableIterator::key() const {
+ assert(Valid());
+ return curr_key_.GetInternalKey();
+}
+
+Slice CuckooTableIterator::value() const {
+ assert(Valid());
+ return curr_value_;
+}
+
+InternalIterator* CuckooTableReader::NewIterator(
+ const ReadOptions& /*read_options*/,
+ const SliceTransform* /* prefix_extractor */, Arena* arena,
+ bool /*skip_filters*/, TableReaderCaller /*caller*/,
+ size_t /*compaction_readahead_size*/, bool /* allow_unprepared_value */) {
+ if (!status().ok()) {
+ return NewErrorInternalIterator<Slice>(
+ Status::Corruption("CuckooTableReader status is not okay."), arena);
+ }
+ CuckooTableIterator* iter;
+ if (arena == nullptr) {
+ iter = new CuckooTableIterator(this);
+ } else {
+ auto iter_mem = arena->AllocateAligned(sizeof(CuckooTableIterator));
+ iter = new (iter_mem) CuckooTableIterator(this);
+ }
+ return iter;
+}
+
+size_t CuckooTableReader::ApproximateMemoryUsage() const { return 0; }
+
+} // namespace ROCKSDB_NAMESPACE
+#endif
diff --git a/src/rocksdb/table/cuckoo/cuckoo_table_reader.h b/src/rocksdb/table/cuckoo/cuckoo_table_reader.h
new file mode 100644
index 000000000..f6c599ae8
--- /dev/null
+++ b/src/rocksdb/table/cuckoo/cuckoo_table_reader.h
@@ -0,0 +1,100 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#pragma once
+#ifndef ROCKSDB_LITE
+#include <memory>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include "file/random_access_file_reader.h"
+#include "rocksdb/env.h"
+#include "rocksdb/options.h"
+#include "table/table_reader.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class Arena;
+class TableReader;
+struct ImmutableOptions;
+
+class CuckooTableReader : public TableReader {
+ public:
+ CuckooTableReader(const ImmutableOptions& ioptions,
+ std::unique_ptr<RandomAccessFileReader>&& file,
+ uint64_t file_size, const Comparator* user_comparator,
+ uint64_t (*get_slice_hash)(const Slice&, uint32_t,
+ uint64_t));
+ ~CuckooTableReader() {}
+
+ std::shared_ptr<const TableProperties> GetTableProperties() const override {
+ return table_props_;
+ }
+
+ Status status() const { return status_; }
+
+ Status Get(const ReadOptions& readOptions, const Slice& key,
+ GetContext* get_context, const SliceTransform* prefix_extractor,
+ bool skip_filters = false) override;
+
+ // Returns a new iterator over table contents
+ // compaction_readahead_size: its value will only be used if for_compaction =
+ // true
+ InternalIterator* NewIterator(const ReadOptions&,
+ const SliceTransform* prefix_extractor,
+ Arena* arena, bool skip_filters,
+ TableReaderCaller caller,
+ size_t compaction_readahead_size = 0,
+ bool allow_unprepared_value = false) override;
+ void Prepare(const Slice& target) override;
+
+ // Report an approximation of how much memory has been used.
+ size_t ApproximateMemoryUsage() const override;
+
+ // Following methods are not implemented for Cuckoo Table Reader
+ uint64_t ApproximateOffsetOf(const Slice& /*key*/,
+ TableReaderCaller /*caller*/) override {
+ return 0;
+ }
+
+ uint64_t ApproximateSize(const Slice& /*start*/, const Slice& /*end*/,
+ TableReaderCaller /*caller*/) override {
+ return 0;
+ }
+
+ void SetupForCompaction() override {}
+ // End of methods not implemented.
+
+ private:
+ friend class CuckooTableIterator;
+ void LoadAllKeys(std::vector<std::pair<Slice, uint32_t>>* key_to_bucket_id);
+ std::unique_ptr<RandomAccessFileReader> file_;
+ Slice file_data_;
+ bool is_last_level_;
+ bool identity_as_first_hash_;
+ bool use_module_hash_;
+ std::shared_ptr<const TableProperties> table_props_;
+ Status status_;
+ uint32_t num_hash_func_;
+ std::string unused_key_;
+ uint32_t key_length_;
+ uint32_t user_key_length_;
+ uint32_t value_length_;
+ uint32_t bucket_length_;
+ uint32_t cuckoo_block_size_;
+ uint32_t cuckoo_block_bytes_minus_one_;
+ uint64_t table_size_;
+ const Comparator* ucomp_;
+ uint64_t (*get_slice_hash_)(const Slice& s, uint32_t index,
+ uint64_t max_num_buckets);
+};
+
+} // namespace ROCKSDB_NAMESPACE
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/table/cuckoo/cuckoo_table_reader_test.cc b/src/rocksdb/table/cuckoo/cuckoo_table_reader_test.cc
new file mode 100644
index 000000000..d3d1490c6
--- /dev/null
+++ b/src/rocksdb/table/cuckoo/cuckoo_table_reader_test.cc
@@ -0,0 +1,584 @@
+// Copyright (c) 2011-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).
+
+#ifndef ROCKSDB_LITE
+
+#ifndef GFLAGS
+#include <cstdio>
+int main() {
+ fprintf(stderr, "Please install gflags to run this test... Skipping...\n");
+ return 0;
+}
+#else
+
+#include <cinttypes>
+#include <map>
+#include <string>
+#include <vector>
+
+#include "memory/arena.h"
+#include "rocksdb/db.h"
+#include "table/cuckoo/cuckoo_table_builder.h"
+#include "table/cuckoo/cuckoo_table_factory.h"
+#include "table/cuckoo/cuckoo_table_reader.h"
+#include "table/get_context.h"
+#include "table/meta_blocks.h"
+#include "test_util/testharness.h"
+#include "test_util/testutil.h"
+#include "util/gflags_compat.h"
+#include "util/random.h"
+#include "util/string_util.h"
+
+using GFLAGS_NAMESPACE::ParseCommandLineFlags;
+
+DEFINE_string(file_dir, "",
+ "Directory where the files will be created"
+ " for benchmark. Added for using tmpfs.");
+DEFINE_bool(enable_perf, false, "Run Benchmark Tests too.");
+DEFINE_bool(write, false,
+ "Should write new values to file in performance tests?");
+DEFINE_bool(identity_as_first_hash, true, "use identity as first hash");
+
+namespace ROCKSDB_NAMESPACE {
+
+namespace {
+const uint32_t kNumHashFunc = 10;
+// Methods, variables related to Hash functions.
+std::unordered_map<std::string, std::vector<uint64_t> > hash_map;
+
+void AddHashLookups(const std::string& s, uint64_t bucket_id,
+ uint32_t num_hash_fun) {
+ std::vector<uint64_t> v;
+ for (uint32_t i = 0; i < num_hash_fun; i++) {
+ v.push_back(bucket_id + i);
+ }
+ hash_map[s] = v;
+}
+
+uint64_t GetSliceHash(const Slice& s, uint32_t index,
+ uint64_t /*max_num_buckets*/) {
+ return hash_map[s.ToString()][index];
+}
+} // namespace
+
+class CuckooReaderTest : public testing::Test {
+ public:
+ using testing::Test::SetUp;
+
+ CuckooReaderTest() {
+ options.allow_mmap_reads = true;
+ env = options.env;
+ file_options = FileOptions(options);
+ }
+
+ void SetUp(int num) {
+ num_items = num;
+ hash_map.clear();
+ keys.clear();
+ keys.resize(num_items);
+ user_keys.clear();
+ user_keys.resize(num_items);
+ values.clear();
+ values.resize(num_items);
+ }
+
+ std::string NumToStr(int64_t i) {
+ return std::string(reinterpret_cast<char*>(&i), sizeof(i));
+ }
+
+ void CreateCuckooFileAndCheckReader(
+ const Comparator* ucomp = BytewiseComparator()) {
+ std::unique_ptr<WritableFileWriter> file_writer;
+ ASSERT_OK(WritableFileWriter::Create(env->GetFileSystem(), fname,
+ file_options, &file_writer, nullptr));
+ CuckooTableBuilder builder(
+ file_writer.get(), 0.9, kNumHashFunc, 100, ucomp, 2, false, false,
+ GetSliceHash, 0 /* column_family_id */, kDefaultColumnFamilyName);
+ ASSERT_OK(builder.status());
+ for (uint32_t key_idx = 0; key_idx < num_items; ++key_idx) {
+ builder.Add(Slice(keys[key_idx]), Slice(values[key_idx]));
+ ASSERT_OK(builder.status());
+ ASSERT_EQ(builder.NumEntries(), key_idx + 1);
+ }
+ ASSERT_OK(builder.Finish());
+ ASSERT_EQ(num_items, builder.NumEntries());
+ file_size = builder.FileSize();
+ ASSERT_OK(file_writer->Close());
+
+ // Check reader now.
+ std::unique_ptr<RandomAccessFileReader> file_reader;
+ ASSERT_OK(RandomAccessFileReader::Create(
+ env->GetFileSystem(), fname, file_options, &file_reader, nullptr));
+ const ImmutableOptions ioptions(options);
+ CuckooTableReader reader(ioptions, std::move(file_reader), file_size, ucomp,
+ GetSliceHash);
+ ASSERT_OK(reader.status());
+ // Assume no merge/deletion
+ for (uint32_t i = 0; i < num_items; ++i) {
+ PinnableSlice value;
+ GetContext get_context(ucomp, nullptr, nullptr, nullptr,
+ GetContext::kNotFound, Slice(user_keys[i]), &value,
+ nullptr, nullptr, nullptr, nullptr, true, nullptr,
+ nullptr);
+ ASSERT_OK(
+ reader.Get(ReadOptions(), Slice(keys[i]), &get_context, nullptr));
+ ASSERT_STREQ(values[i].c_str(), value.data());
+ }
+ }
+ void UpdateKeys(bool with_zero_seqno) {
+ for (uint32_t i = 0; i < num_items; i++) {
+ ParsedInternalKey ikey(user_keys[i], with_zero_seqno ? 0 : i + 1000,
+ kTypeValue);
+ keys[i].clear();
+ AppendInternalKey(&keys[i], ikey);
+ }
+ }
+
+ void CheckIterator(const Comparator* ucomp = BytewiseComparator()) {
+ std::unique_ptr<RandomAccessFileReader> file_reader;
+ ASSERT_OK(RandomAccessFileReader::Create(
+ env->GetFileSystem(), fname, file_options, &file_reader, nullptr));
+ const ImmutableOptions ioptions(options);
+ CuckooTableReader reader(ioptions, std::move(file_reader), file_size, ucomp,
+ GetSliceHash);
+ ASSERT_OK(reader.status());
+ InternalIterator* it = reader.NewIterator(
+ ReadOptions(), /*prefix_extractor=*/nullptr, /*arena=*/nullptr,
+ /*skip_filters=*/false, TableReaderCaller::kUncategorized);
+ ASSERT_OK(it->status());
+ ASSERT_TRUE(!it->Valid());
+ it->SeekToFirst();
+ int cnt = 0;
+ while (it->Valid()) {
+ ASSERT_OK(it->status());
+ ASSERT_TRUE(Slice(keys[cnt]) == it->key());
+ ASSERT_TRUE(Slice(values[cnt]) == it->value());
+ ++cnt;
+ it->Next();
+ }
+ ASSERT_EQ(static_cast<uint32_t>(cnt), num_items);
+
+ it->SeekToLast();
+ cnt = static_cast<int>(num_items) - 1;
+ ASSERT_TRUE(it->Valid());
+ while (it->Valid()) {
+ ASSERT_OK(it->status());
+ ASSERT_TRUE(Slice(keys[cnt]) == it->key());
+ ASSERT_TRUE(Slice(values[cnt]) == it->value());
+ --cnt;
+ it->Prev();
+ }
+ ASSERT_EQ(cnt, -1);
+
+ cnt = static_cast<int>(num_items) / 2;
+ it->Seek(keys[cnt]);
+ while (it->Valid()) {
+ ASSERT_OK(it->status());
+ ASSERT_TRUE(Slice(keys[cnt]) == it->key());
+ ASSERT_TRUE(Slice(values[cnt]) == it->value());
+ ++cnt;
+ it->Next();
+ }
+ ASSERT_EQ(static_cast<uint32_t>(cnt), num_items);
+ delete it;
+
+ Arena arena;
+ it = reader.NewIterator(ReadOptions(), /*prefix_extractor=*/nullptr, &arena,
+ /*skip_filters=*/false,
+ TableReaderCaller::kUncategorized);
+ ASSERT_OK(it->status());
+ ASSERT_TRUE(!it->Valid());
+ it->Seek(keys[num_items / 2]);
+ ASSERT_TRUE(it->Valid());
+ ASSERT_OK(it->status());
+ ASSERT_TRUE(keys[num_items / 2] == it->key());
+ ASSERT_TRUE(values[num_items / 2] == it->value());
+ ASSERT_OK(it->status());
+ it->~InternalIterator();
+ }
+
+ std::vector<std::string> keys;
+ std::vector<std::string> user_keys;
+ std::vector<std::string> values;
+ uint64_t num_items;
+ std::string fname;
+ uint64_t file_size;
+ Options options;
+ Env* env;
+ FileOptions file_options;
+};
+
+TEST_F(CuckooReaderTest, FileNotMmaped) {
+ options.allow_mmap_reads = false;
+ ImmutableOptions ioptions(options);
+ CuckooTableReader reader(ioptions, nullptr, 0, nullptr, nullptr);
+ ASSERT_TRUE(reader.status().IsInvalidArgument());
+ ASSERT_STREQ("File is not mmaped", reader.status().getState());
+}
+
+TEST_F(CuckooReaderTest, WhenKeyExists) {
+ SetUp(kNumHashFunc);
+ fname = test::PerThreadDBPath("CuckooReader_WhenKeyExists");
+ for (uint64_t i = 0; i < num_items; i++) {
+ user_keys[i] = "key" + NumToStr(i);
+ ParsedInternalKey ikey(user_keys[i], i + 1000, kTypeValue);
+ AppendInternalKey(&keys[i], ikey);
+ values[i] = "value" + NumToStr(i);
+ // Give disjoint hash values.
+ AddHashLookups(user_keys[i], i, kNumHashFunc);
+ }
+ CreateCuckooFileAndCheckReader();
+ // Last level file.
+ UpdateKeys(true);
+ CreateCuckooFileAndCheckReader();
+ // Test with collision. Make all hash values collide.
+ hash_map.clear();
+ for (uint32_t i = 0; i < num_items; i++) {
+ AddHashLookups(user_keys[i], 0, kNumHashFunc);
+ }
+ UpdateKeys(false);
+ CreateCuckooFileAndCheckReader();
+ // Last level file.
+ UpdateKeys(true);
+ CreateCuckooFileAndCheckReader();
+}
+
+TEST_F(CuckooReaderTest, WhenKeyExistsWithUint64Comparator) {
+ SetUp(kNumHashFunc);
+ fname = test::PerThreadDBPath("CuckooReaderUint64_WhenKeyExists");
+ for (uint64_t i = 0; i < num_items; i++) {
+ user_keys[i].resize(8);
+ memcpy(&user_keys[i][0], static_cast<void*>(&i), 8);
+ ParsedInternalKey ikey(user_keys[i], i + 1000, kTypeValue);
+ AppendInternalKey(&keys[i], ikey);
+ values[i] = "value" + NumToStr(i);
+ // Give disjoint hash values.
+ AddHashLookups(user_keys[i], i, kNumHashFunc);
+ }
+ CreateCuckooFileAndCheckReader(test::Uint64Comparator());
+ // Last level file.
+ UpdateKeys(true);
+ CreateCuckooFileAndCheckReader(test::Uint64Comparator());
+ // Test with collision. Make all hash values collide.
+ hash_map.clear();
+ for (uint32_t i = 0; i < num_items; i++) {
+ AddHashLookups(user_keys[i], 0, kNumHashFunc);
+ }
+ UpdateKeys(false);
+ CreateCuckooFileAndCheckReader(test::Uint64Comparator());
+ // Last level file.
+ UpdateKeys(true);
+ CreateCuckooFileAndCheckReader(test::Uint64Comparator());
+}
+
+TEST_F(CuckooReaderTest, CheckIterator) {
+ SetUp(2 * kNumHashFunc);
+ fname = test::PerThreadDBPath("CuckooReader_CheckIterator");
+ for (uint64_t i = 0; i < num_items; i++) {
+ user_keys[i] = "key" + NumToStr(i);
+ ParsedInternalKey ikey(user_keys[i], 1000, kTypeValue);
+ AppendInternalKey(&keys[i], ikey);
+ values[i] = "value" + NumToStr(i);
+ // Give disjoint hash values, in reverse order.
+ AddHashLookups(user_keys[i], num_items - i - 1, kNumHashFunc);
+ }
+ CreateCuckooFileAndCheckReader();
+ CheckIterator();
+ // Last level file.
+ UpdateKeys(true);
+ CreateCuckooFileAndCheckReader();
+ CheckIterator();
+}
+
+TEST_F(CuckooReaderTest, CheckIteratorUint64) {
+ SetUp(2 * kNumHashFunc);
+ fname = test::PerThreadDBPath("CuckooReader_CheckIterator");
+ for (uint64_t i = 0; i < num_items; i++) {
+ user_keys[i].resize(8);
+ memcpy(&user_keys[i][0], static_cast<void*>(&i), 8);
+ ParsedInternalKey ikey(user_keys[i], 1000, kTypeValue);
+ AppendInternalKey(&keys[i], ikey);
+ values[i] = "value" + NumToStr(i);
+ // Give disjoint hash values, in reverse order.
+ AddHashLookups(user_keys[i], num_items - i - 1, kNumHashFunc);
+ }
+ CreateCuckooFileAndCheckReader(test::Uint64Comparator());
+ CheckIterator(test::Uint64Comparator());
+ // Last level file.
+ UpdateKeys(true);
+ CreateCuckooFileAndCheckReader(test::Uint64Comparator());
+ CheckIterator(test::Uint64Comparator());
+}
+
+TEST_F(CuckooReaderTest, WhenKeyNotFound) {
+ // Add keys with colliding hash values.
+ SetUp(kNumHashFunc);
+ fname = test::PerThreadDBPath("CuckooReader_WhenKeyNotFound");
+ for (uint64_t i = 0; i < num_items; i++) {
+ user_keys[i] = "key" + NumToStr(i);
+ ParsedInternalKey ikey(user_keys[i], i + 1000, kTypeValue);
+ AppendInternalKey(&keys[i], ikey);
+ values[i] = "value" + NumToStr(i);
+ // Make all hash values collide.
+ AddHashLookups(user_keys[i], 0, kNumHashFunc);
+ }
+ auto* ucmp = BytewiseComparator();
+ CreateCuckooFileAndCheckReader();
+
+ std::unique_ptr<RandomAccessFileReader> file_reader;
+ ASSERT_OK(RandomAccessFileReader::Create(
+ env->GetFileSystem(), fname, file_options, &file_reader, nullptr));
+
+ const ImmutableOptions ioptions(options);
+ CuckooTableReader reader(ioptions, std::move(file_reader), file_size, ucmp,
+ GetSliceHash);
+ ASSERT_OK(reader.status());
+ // Search for a key with colliding hash values.
+ std::string not_found_user_key = "key" + NumToStr(num_items);
+ std::string not_found_key;
+ AddHashLookups(not_found_user_key, 0, kNumHashFunc);
+ ParsedInternalKey ikey(not_found_user_key, 1000, kTypeValue);
+ AppendInternalKey(&not_found_key, ikey);
+ PinnableSlice value;
+ GetContext get_context(ucmp, nullptr, nullptr, nullptr, GetContext::kNotFound,
+ Slice(not_found_key), &value, nullptr, nullptr,
+ nullptr, nullptr, true, nullptr, nullptr);
+ ASSERT_OK(
+ reader.Get(ReadOptions(), Slice(not_found_key), &get_context, nullptr));
+ ASSERT_TRUE(value.empty());
+ ASSERT_OK(reader.status());
+ // Search for a key with an independent hash value.
+ std::string not_found_user_key2 = "key" + NumToStr(num_items + 1);
+ AddHashLookups(not_found_user_key2, kNumHashFunc, kNumHashFunc);
+ ParsedInternalKey ikey2(not_found_user_key2, 1000, kTypeValue);
+ std::string not_found_key2;
+ AppendInternalKey(&not_found_key2, ikey2);
+ value.Reset();
+ GetContext get_context2(ucmp, nullptr, nullptr, nullptr,
+ GetContext::kNotFound, Slice(not_found_key2), &value,
+ nullptr, nullptr, nullptr, nullptr, true, nullptr,
+ nullptr);
+ ASSERT_OK(
+ reader.Get(ReadOptions(), Slice(not_found_key2), &get_context2, nullptr));
+ ASSERT_TRUE(value.empty());
+ ASSERT_OK(reader.status());
+
+ // Test read when key is unused key.
+ std::string unused_key =
+ reader.GetTableProperties()->user_collected_properties.at(
+ CuckooTablePropertyNames::kEmptyKey);
+ // Add hash values that map to empty buckets.
+ AddHashLookups(ExtractUserKey(unused_key).ToString(), kNumHashFunc,
+ kNumHashFunc);
+ value.Reset();
+ GetContext get_context3(
+ ucmp, nullptr, nullptr, nullptr, GetContext::kNotFound, Slice(unused_key),
+ &value, nullptr, nullptr, nullptr, nullptr, true, nullptr, nullptr);
+ ASSERT_OK(
+ reader.Get(ReadOptions(), Slice(unused_key), &get_context3, nullptr));
+ ASSERT_TRUE(value.empty());
+ ASSERT_OK(reader.status());
+}
+
+// Performance tests
+namespace {
+void GetKeys(uint64_t num, std::vector<std::string>* keys) {
+ keys->clear();
+ IterKey k;
+ k.SetInternalKey("", 0, kTypeValue);
+ std::string internal_key_suffix = k.GetInternalKey().ToString();
+ ASSERT_EQ(static_cast<size_t>(8), internal_key_suffix.size());
+ for (uint64_t key_idx = 0; key_idx < num; ++key_idx) {
+ uint64_t value = 2 * key_idx;
+ std::string new_key(reinterpret_cast<char*>(&value), sizeof(value));
+ new_key += internal_key_suffix;
+ keys->push_back(new_key);
+ }
+}
+
+std::string GetFileName(uint64_t num) {
+ if (FLAGS_file_dir.empty()) {
+ FLAGS_file_dir = test::TmpDir();
+ }
+ return test::PerThreadDBPath(FLAGS_file_dir, "cuckoo_read_benchmark") +
+ std::to_string(num / 1000000) + "Mkeys";
+}
+
+// Create last level file as we are interested in measuring performance of
+// last level file only.
+void WriteFile(const std::vector<std::string>& keys, const uint64_t num,
+ double hash_ratio) {
+ Options options;
+ options.allow_mmap_reads = true;
+ const auto& fs = options.env->GetFileSystem();
+ FileOptions file_options(options);
+ std::string fname = GetFileName(num);
+
+ std::unique_ptr<WritableFileWriter> file_writer;
+ ASSERT_OK(WritableFileWriter::Create(fs, fname, file_options, &file_writer,
+ nullptr));
+ CuckooTableBuilder builder(
+ file_writer.get(), hash_ratio, 64, 1000, test::Uint64Comparator(), 5,
+ false, FLAGS_identity_as_first_hash, nullptr, 0 /* column_family_id */,
+ kDefaultColumnFamilyName);
+ ASSERT_OK(builder.status());
+ for (uint64_t key_idx = 0; key_idx < num; ++key_idx) {
+ // Value is just a part of key.
+ builder.Add(Slice(keys[key_idx]), Slice(&keys[key_idx][0], 4));
+ ASSERT_EQ(builder.NumEntries(), key_idx + 1);
+ ASSERT_OK(builder.status());
+ }
+ ASSERT_OK(builder.Finish());
+ ASSERT_EQ(num, builder.NumEntries());
+ ASSERT_OK(file_writer->Close());
+
+ uint64_t file_size;
+ ASSERT_OK(
+ fs->GetFileSize(fname, file_options.io_options, &file_size, nullptr));
+ std::unique_ptr<RandomAccessFileReader> file_reader;
+ ASSERT_OK(RandomAccessFileReader::Create(fs, fname, file_options,
+ &file_reader, nullptr));
+
+ const ImmutableOptions ioptions(options);
+ CuckooTableReader reader(ioptions, std::move(file_reader), file_size,
+ test::Uint64Comparator(), nullptr);
+ ASSERT_OK(reader.status());
+ ReadOptions r_options;
+ PinnableSlice value;
+ // Assume only the fast path is triggered
+ GetContext get_context(nullptr, nullptr, nullptr, nullptr,
+ GetContext::kNotFound, Slice(), &value, nullptr,
+ nullptr, nullptr, true, nullptr, nullptr);
+ for (uint64_t i = 0; i < num; ++i) {
+ value.Reset();
+ value.clear();
+ ASSERT_OK(reader.Get(r_options, Slice(keys[i]), &get_context, nullptr));
+ ASSERT_TRUE(Slice(keys[i]) == Slice(&keys[i][0], 4));
+ }
+}
+
+void ReadKeys(uint64_t num, uint32_t batch_size) {
+ Options options;
+ options.allow_mmap_reads = true;
+ Env* env = options.env;
+ const auto& fs = options.env->GetFileSystem();
+ FileOptions file_options(options);
+ std::string fname = GetFileName(num);
+
+ uint64_t file_size;
+ ASSERT_OK(
+ fs->GetFileSize(fname, file_options.io_options, &file_size, nullptr));
+ std::unique_ptr<RandomAccessFileReader> file_reader;
+ ASSERT_OK(RandomAccessFileReader::Create(fs, fname, file_options,
+ &file_reader, nullptr));
+
+ const ImmutableOptions ioptions(options);
+ CuckooTableReader reader(ioptions, std::move(file_reader), file_size,
+ test::Uint64Comparator(), nullptr);
+ ASSERT_OK(reader.status());
+ const UserCollectedProperties user_props =
+ reader.GetTableProperties()->user_collected_properties;
+ const uint32_t num_hash_fun = *reinterpret_cast<const uint32_t*>(
+ user_props.at(CuckooTablePropertyNames::kNumHashFunc).data());
+ const uint64_t table_size = *reinterpret_cast<const uint64_t*>(
+ user_props.at(CuckooTablePropertyNames::kHashTableSize).data());
+ fprintf(stderr,
+ "With %" PRIu64
+ " items, utilization is %.2f%%, number of"
+ " hash functions: %u.\n",
+ num, num * 100.0 / (table_size), num_hash_fun);
+ ReadOptions r_options;
+
+ std::vector<uint64_t> keys;
+ keys.reserve(num);
+ for (uint64_t i = 0; i < num; ++i) {
+ keys.push_back(2 * i);
+ }
+ RandomShuffle(keys.begin(), keys.end());
+
+ PinnableSlice value;
+ // Assume only the fast path is triggered
+ GetContext get_context(nullptr, nullptr, nullptr, nullptr,
+ GetContext::kNotFound, Slice(), &value, nullptr,
+ nullptr, nullptr, true, nullptr, nullptr);
+ uint64_t start_time = env->NowMicros();
+ if (batch_size > 0) {
+ for (uint64_t i = 0; i < num; i += batch_size) {
+ for (uint64_t j = i; j < i + batch_size && j < num; ++j) {
+ reader.Prepare(Slice(reinterpret_cast<char*>(&keys[j]), 16));
+ }
+ for (uint64_t j = i; j < i + batch_size && j < num; ++j) {
+ reader.Get(r_options, Slice(reinterpret_cast<char*>(&keys[j]), 16),
+ &get_context, nullptr);
+ }
+ }
+ } else {
+ for (uint64_t i = 0; i < num; i++) {
+ reader.Get(r_options, Slice(reinterpret_cast<char*>(&keys[i]), 16),
+ &get_context, nullptr);
+ }
+ }
+ float time_per_op = (env->NowMicros() - start_time) * 1.0f / num;
+ fprintf(stderr,
+ "Time taken per op is %.3fus (%.1f Mqps) with batch size of %u\n",
+ time_per_op, 1.0 / time_per_op, batch_size);
+}
+} // namespace.
+
+TEST_F(CuckooReaderTest, TestReadPerformance) {
+ if (!FLAGS_enable_perf) {
+ return;
+ }
+ double hash_ratio = 0.95;
+ // These numbers are chosen to have a hash utilization % close to
+ // 0.9, 0.75, 0.6 and 0.5 respectively.
+ // They all create 128 M buckets.
+ std::vector<uint64_t> nums = {120 * 1024 * 1024, 100 * 1024 * 1024,
+ 80 * 1024 * 1024, 70 * 1024 * 1024};
+#ifndef NDEBUG
+ fprintf(
+ stdout,
+ "WARNING: Not compiled with DNDEBUG. Performance tests may be slow.\n");
+#endif
+ for (uint64_t num : nums) {
+ if (FLAGS_write ||
+ Env::Default()->FileExists(GetFileName(num)).IsNotFound()) {
+ std::vector<std::string> all_keys;
+ GetKeys(num, &all_keys);
+ WriteFile(all_keys, num, hash_ratio);
+ }
+ ReadKeys(num, 0);
+ ReadKeys(num, 10);
+ ReadKeys(num, 25);
+ ReadKeys(num, 50);
+ ReadKeys(num, 100);
+ fprintf(stderr, "\n");
+ }
+}
+} // namespace ROCKSDB_NAMESPACE
+
+int main(int argc, char** argv) {
+ if (ROCKSDB_NAMESPACE::port::kLittleEndian) {
+ ROCKSDB_NAMESPACE::port::InstallStackTraceHandler();
+ ::testing::InitGoogleTest(&argc, argv);
+ ParseCommandLineFlags(&argc, &argv, true);
+ return RUN_ALL_TESTS();
+ } else {
+ fprintf(stderr, "SKIPPED as Cuckoo table doesn't support Big Endian\n");
+ return 0;
+ }
+}
+
+#endif // GFLAGS.
+
+#else
+#include <stdio.h>
+
+int main(int /*argc*/, char** /*argv*/) {
+ fprintf(stderr, "SKIPPED as Cuckoo table is not supported in ROCKSDB_LITE\n");
+ return 0;
+}
+
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/table/format.cc b/src/rocksdb/table/format.cc
new file mode 100644
index 000000000..efde5e169
--- /dev/null
+++ b/src/rocksdb/table/format.cc
@@ -0,0 +1,575 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#include "table/format.h"
+
+#include <cinttypes>
+#include <string>
+
+#include "block_fetcher.h"
+#include "file/random_access_file_reader.h"
+#include "memory/memory_allocator.h"
+#include "monitoring/perf_context_imp.h"
+#include "monitoring/statistics.h"
+#include "options/options_helper.h"
+#include "rocksdb/env.h"
+#include "rocksdb/options.h"
+#include "rocksdb/table.h"
+#include "table/block_based/block.h"
+#include "table/block_based/block_based_table_reader.h"
+#include "table/persistent_cache_helper.h"
+#include "util/cast_util.h"
+#include "util/coding.h"
+#include "util/compression.h"
+#include "util/crc32c.h"
+#include "util/hash.h"
+#include "util/stop_watch.h"
+#include "util/string_util.h"
+#include "util/xxhash.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+extern const uint64_t kLegacyBlockBasedTableMagicNumber;
+extern const uint64_t kBlockBasedTableMagicNumber;
+
+#ifndef ROCKSDB_LITE
+extern const uint64_t kLegacyPlainTableMagicNumber;
+extern const uint64_t kPlainTableMagicNumber;
+#else
+// ROCKSDB_LITE doesn't have plain table
+const uint64_t kLegacyPlainTableMagicNumber = 0;
+const uint64_t kPlainTableMagicNumber = 0;
+#endif
+const char* kHostnameForDbHostId = "__hostname__";
+
+bool ShouldReportDetailedTime(Env* env, Statistics* stats) {
+ return env != nullptr && stats != nullptr &&
+ stats->get_stats_level() > kExceptDetailedTimers;
+}
+
+void BlockHandle::EncodeTo(std::string* dst) const {
+ // Sanity check that all fields have been set
+ assert(offset_ != ~uint64_t{0});
+ assert(size_ != ~uint64_t{0});
+ PutVarint64Varint64(dst, offset_, size_);
+}
+
+char* BlockHandle::EncodeTo(char* dst) const {
+ // Sanity check that all fields have been set
+ assert(offset_ != ~uint64_t{0});
+ assert(size_ != ~uint64_t{0});
+ char* cur = EncodeVarint64(dst, offset_);
+ cur = EncodeVarint64(cur, size_);
+ return cur;
+}
+
+Status BlockHandle::DecodeFrom(Slice* input) {
+ if (GetVarint64(input, &offset_) && GetVarint64(input, &size_)) {
+ return Status::OK();
+ } else {
+ // reset in case failure after partially decoding
+ offset_ = 0;
+ size_ = 0;
+ return Status::Corruption("bad block handle");
+ }
+}
+
+Status BlockHandle::DecodeSizeFrom(uint64_t _offset, Slice* input) {
+ if (GetVarint64(input, &size_)) {
+ offset_ = _offset;
+ return Status::OK();
+ } else {
+ // reset in case failure after partially decoding
+ offset_ = 0;
+ size_ = 0;
+ return Status::Corruption("bad block handle");
+ }
+}
+
+// Return a string that contains the copy of handle.
+std::string BlockHandle::ToString(bool hex) const {
+ std::string handle_str;
+ EncodeTo(&handle_str);
+ if (hex) {
+ return Slice(handle_str).ToString(true);
+ } else {
+ return handle_str;
+ }
+}
+
+const BlockHandle BlockHandle::kNullBlockHandle(0, 0);
+
+void IndexValue::EncodeTo(std::string* dst, bool have_first_key,
+ const BlockHandle* previous_handle) const {
+ if (previous_handle) {
+ // WART: this is specific to Block-based table
+ assert(handle.offset() == previous_handle->offset() +
+ previous_handle->size() +
+ BlockBasedTable::kBlockTrailerSize);
+ PutVarsignedint64(dst, handle.size() - previous_handle->size());
+ } else {
+ handle.EncodeTo(dst);
+ }
+ assert(dst->size() != 0);
+
+ if (have_first_key) {
+ PutLengthPrefixedSlice(dst, first_internal_key);
+ }
+}
+
+Status IndexValue::DecodeFrom(Slice* input, bool have_first_key,
+ const BlockHandle* previous_handle) {
+ if (previous_handle) {
+ int64_t delta;
+ if (!GetVarsignedint64(input, &delta)) {
+ return Status::Corruption("bad delta-encoded index value");
+ }
+ // WART: this is specific to Block-based table
+ handle = BlockHandle(previous_handle->offset() + previous_handle->size() +
+ BlockBasedTable::kBlockTrailerSize,
+ previous_handle->size() + delta);
+ } else {
+ Status s = handle.DecodeFrom(input);
+ if (!s.ok()) {
+ return s;
+ }
+ }
+
+ if (!have_first_key) {
+ first_internal_key = Slice();
+ } else if (!GetLengthPrefixedSlice(input, &first_internal_key)) {
+ return Status::Corruption("bad first key in block info");
+ }
+
+ return Status::OK();
+}
+
+std::string IndexValue::ToString(bool hex, bool have_first_key) const {
+ std::string s;
+ EncodeTo(&s, have_first_key, nullptr);
+ if (hex) {
+ return Slice(s).ToString(true);
+ } else {
+ return s;
+ }
+}
+
+namespace {
+inline bool IsLegacyFooterFormat(uint64_t magic_number) {
+ return magic_number == kLegacyBlockBasedTableMagicNumber ||
+ magic_number == kLegacyPlainTableMagicNumber;
+}
+inline uint64_t UpconvertLegacyFooterFormat(uint64_t magic_number) {
+ if (magic_number == kLegacyBlockBasedTableMagicNumber) {
+ return kBlockBasedTableMagicNumber;
+ }
+ if (magic_number == kLegacyPlainTableMagicNumber) {
+ return kPlainTableMagicNumber;
+ }
+ assert(false);
+ return magic_number;
+}
+inline uint64_t DownconvertToLegacyFooterFormat(uint64_t magic_number) {
+ if (magic_number == kBlockBasedTableMagicNumber) {
+ return kLegacyBlockBasedTableMagicNumber;
+ }
+ if (magic_number == kPlainTableMagicNumber) {
+ return kLegacyPlainTableMagicNumber;
+ }
+ assert(false);
+ return magic_number;
+}
+inline uint8_t BlockTrailerSizeForMagicNumber(uint64_t magic_number) {
+ if (magic_number == kBlockBasedTableMagicNumber ||
+ magic_number == kLegacyBlockBasedTableMagicNumber) {
+ return static_cast<uint8_t>(BlockBasedTable::kBlockTrailerSize);
+ } else {
+ return 0;
+ }
+}
+
+// Footer format, in three parts:
+// * Part1
+// -> format_version == 0 (inferred from legacy magic number)
+// <empty> (0 bytes)
+// -> format_version >= 1
+// checksum type (char, 1 byte)
+// * Part2
+// metaindex handle (varint64 offset, varint64 size)
+// index handle (varint64 offset, varint64 size)
+// <zero padding> for part2 size = 2 * BlockHandle::kMaxEncodedLength = 40
+// * Part3
+// -> format_version == 0 (inferred from legacy magic number)
+// legacy magic number (8 bytes)
+// -> format_version >= 1 (inferred from NOT legacy magic number)
+// format_version (uint32LE, 4 bytes), also called "footer version"
+// newer magic number (8 bytes)
+
+constexpr size_t kFooterPart2Size = 2 * BlockHandle::kMaxEncodedLength;
+} // namespace
+
+void FooterBuilder::Build(uint64_t magic_number, uint32_t format_version,
+ uint64_t footer_offset, ChecksumType checksum_type,
+ const BlockHandle& metaindex_handle,
+ const BlockHandle& index_handle) {
+ (void)footer_offset; // Future use
+
+ assert(magic_number != Footer::kNullTableMagicNumber);
+ assert(IsSupportedFormatVersion(format_version));
+
+ char* part2;
+ char* part3;
+ if (format_version > 0) {
+ slice_ = Slice(data_.data(), Footer::kNewVersionsEncodedLength);
+ // Generate parts 1 and 3
+ char* cur = data_.data();
+ // Part 1
+ *(cur++) = checksum_type;
+ // Part 2
+ part2 = cur;
+ // Skip over part 2 for now
+ cur += kFooterPart2Size;
+ // Part 3
+ part3 = cur;
+ EncodeFixed32(cur, format_version);
+ cur += 4;
+ EncodeFixed64(cur, magic_number);
+ assert(cur + 8 == slice_.data() + slice_.size());
+ } else {
+ slice_ = Slice(data_.data(), Footer::kVersion0EncodedLength);
+ // Legacy SST files use kCRC32c checksum but it's not stored in footer.
+ assert(checksum_type == kNoChecksum || checksum_type == kCRC32c);
+ // Generate part 3 (part 1 empty, skip part 2 for now)
+ part2 = data_.data();
+ part3 = part2 + kFooterPart2Size;
+ char* cur = part3;
+ // Use legacy magic numbers to indicate format_version=0, for
+ // compatibility. No other cases should use format_version=0.
+ EncodeFixed64(cur, DownconvertToLegacyFooterFormat(magic_number));
+ assert(cur + 8 == slice_.data() + slice_.size());
+ }
+
+ {
+ char* cur = part2;
+ cur = metaindex_handle.EncodeTo(cur);
+ cur = index_handle.EncodeTo(cur);
+ // Zero pad remainder
+ std::fill(cur, part3, char{0});
+ }
+}
+
+Status Footer::DecodeFrom(Slice input, uint64_t input_offset) {
+ (void)input_offset; // Future use
+
+ // Only decode to unused Footer
+ assert(table_magic_number_ == kNullTableMagicNumber);
+ assert(input != nullptr);
+ assert(input.size() >= kMinEncodedLength);
+
+ const char* magic_ptr = input.data() + input.size() - kMagicNumberLengthByte;
+ uint64_t magic = DecodeFixed64(magic_ptr);
+
+ // We check for legacy formats here and silently upconvert them
+ bool legacy = IsLegacyFooterFormat(magic);
+ if (legacy) {
+ magic = UpconvertLegacyFooterFormat(magic);
+ }
+ table_magic_number_ = magic;
+ block_trailer_size_ = BlockTrailerSizeForMagicNumber(magic);
+
+ // Parse Part3
+ if (legacy) {
+ // The size is already asserted to be at least kMinEncodedLength
+ // at the beginning of the function
+ input.remove_prefix(input.size() - kVersion0EncodedLength);
+ format_version_ = 0 /* legacy */;
+ checksum_type_ = kCRC32c;
+ } else {
+ const char* part3_ptr = magic_ptr - 4;
+ format_version_ = DecodeFixed32(part3_ptr);
+ if (!IsSupportedFormatVersion(format_version_)) {
+ return Status::Corruption("Corrupt or unsupported format_version: " +
+ std::to_string(format_version_));
+ }
+ // All known format versions >= 1 occupy exactly this many bytes.
+ if (input.size() < kNewVersionsEncodedLength) {
+ return Status::Corruption("Input is too short to be an SST file");
+ }
+ uint64_t adjustment = input.size() - kNewVersionsEncodedLength;
+ input.remove_prefix(adjustment);
+
+ // Parse Part1
+ char chksum = input.data()[0];
+ checksum_type_ = lossless_cast<ChecksumType>(chksum);
+ if (!IsSupportedChecksumType(checksum_type())) {
+ return Status::Corruption("Corrupt or unsupported checksum type: " +
+ std::to_string(lossless_cast<uint8_t>(chksum)));
+ }
+ // Consume checksum type field
+ input.remove_prefix(1);
+ }
+
+ // Parse Part2
+ Status result = metaindex_handle_.DecodeFrom(&input);
+ if (result.ok()) {
+ result = index_handle_.DecodeFrom(&input);
+ }
+ return result;
+ // Padding in part2 is ignored
+}
+
+std::string Footer::ToString() const {
+ std::string result;
+ result.reserve(1024);
+
+ bool legacy = IsLegacyFooterFormat(table_magic_number_);
+ if (legacy) {
+ result.append("metaindex handle: " + metaindex_handle_.ToString() + "\n ");
+ result.append("index handle: " + index_handle_.ToString() + "\n ");
+ result.append("table_magic_number: " + std::to_string(table_magic_number_) +
+ "\n ");
+ } else {
+ result.append("metaindex handle: " + metaindex_handle_.ToString() + "\n ");
+ result.append("index handle: " + index_handle_.ToString() + "\n ");
+ result.append("table_magic_number: " + std::to_string(table_magic_number_) +
+ "\n ");
+ result.append("format version: " + std::to_string(format_version_) +
+ "\n ");
+ }
+ return result;
+}
+
+Status ReadFooterFromFile(const IOOptions& opts, RandomAccessFileReader* file,
+ FilePrefetchBuffer* prefetch_buffer,
+ uint64_t file_size, Footer* footer,
+ uint64_t enforce_table_magic_number) {
+ if (file_size < Footer::kMinEncodedLength) {
+ return Status::Corruption("file is too short (" +
+ std::to_string(file_size) +
+ " bytes) to be an "
+ "sstable: " +
+ file->file_name());
+ }
+
+ std::string footer_buf;
+ AlignedBuf internal_buf;
+ Slice footer_input;
+ uint64_t read_offset = (file_size > Footer::kMaxEncodedLength)
+ ? file_size - Footer::kMaxEncodedLength
+ : 0;
+ Status s;
+ // TODO: Need to pass appropriate deadline to TryReadFromCache(). Right now,
+ // there is no readahead for point lookups, so TryReadFromCache will fail if
+ // the required data is not in the prefetch buffer. Once deadline is enabled
+ // for iterator, TryReadFromCache might do a readahead. Revisit to see if we
+ // need to pass a timeout at that point
+ // TODO: rate limit footer reads.
+ if (prefetch_buffer == nullptr ||
+ !prefetch_buffer->TryReadFromCache(
+ opts, file, read_offset, Footer::kMaxEncodedLength, &footer_input,
+ nullptr, opts.rate_limiter_priority)) {
+ if (file->use_direct_io()) {
+ s = file->Read(opts, read_offset, Footer::kMaxEncodedLength,
+ &footer_input, nullptr, &internal_buf,
+ opts.rate_limiter_priority);
+ } else {
+ footer_buf.reserve(Footer::kMaxEncodedLength);
+ s = file->Read(opts, read_offset, Footer::kMaxEncodedLength,
+ &footer_input, &footer_buf[0], nullptr,
+ opts.rate_limiter_priority);
+ }
+ if (!s.ok()) return s;
+ }
+
+ // Check that we actually read the whole footer from the file. It may be
+ // that size isn't correct.
+ if (footer_input.size() < Footer::kMinEncodedLength) {
+ // FIXME: this error message is bad. We should be checking whether the
+ // provided file_size matches what's on disk, at least in this case.
+ // Unfortunately FileSystem/Env does not provide a way to get the size
+ // of an open file, so getting file size requires a full path seek.
+ return Status::Corruption("file is too short (" +
+ std::to_string(file_size) +
+ " bytes) to be an "
+ "sstable" +
+ file->file_name());
+ }
+
+ s = footer->DecodeFrom(footer_input, read_offset);
+ if (!s.ok()) {
+ return s;
+ }
+ if (enforce_table_magic_number != 0 &&
+ enforce_table_magic_number != footer->table_magic_number()) {
+ return Status::Corruption("Bad table magic number: expected " +
+ std::to_string(enforce_table_magic_number) +
+ ", found " +
+ std::to_string(footer->table_magic_number()) +
+ " in " + file->file_name());
+ }
+ return Status::OK();
+}
+
+namespace {
+// Custom handling for the last byte of a block, to avoid invoking streaming
+// API to get an effective block checksum. This function is its own inverse
+// because it uses xor.
+inline uint32_t ModifyChecksumForLastByte(uint32_t checksum, char last_byte) {
+ // This strategy bears some resemblance to extending a CRC checksum by one
+ // more byte, except we don't need to re-mix the input checksum as long as
+ // we do this step only once (per checksum).
+ const uint32_t kRandomPrime = 0x6b9083d9;
+ return checksum ^ lossless_cast<uint8_t>(last_byte) * kRandomPrime;
+}
+} // namespace
+
+uint32_t ComputeBuiltinChecksum(ChecksumType type, const char* data,
+ size_t data_size) {
+ switch (type) {
+ case kCRC32c:
+ return crc32c::Mask(crc32c::Value(data, data_size));
+ case kxxHash:
+ return XXH32(data, data_size, /*seed*/ 0);
+ case kxxHash64:
+ return Lower32of64(XXH64(data, data_size, /*seed*/ 0));
+ case kXXH3: {
+ if (data_size == 0) {
+ // Special case because of special handling for last byte, not
+ // present in this case. Can be any value different from other
+ // small input size checksums.
+ return 0;
+ } else {
+ // See corresponding code in ComputeBuiltinChecksumWithLastByte
+ uint32_t v = Lower32of64(XXH3_64bits(data, data_size - 1));
+ return ModifyChecksumForLastByte(v, data[data_size - 1]);
+ }
+ }
+ default: // including kNoChecksum
+ return 0;
+ }
+}
+
+uint32_t ComputeBuiltinChecksumWithLastByte(ChecksumType type, const char* data,
+ size_t data_size, char last_byte) {
+ switch (type) {
+ case kCRC32c: {
+ uint32_t crc = crc32c::Value(data, data_size);
+ // Extend to cover last byte (compression type)
+ crc = crc32c::Extend(crc, &last_byte, 1);
+ return crc32c::Mask(crc);
+ }
+ case kxxHash: {
+ XXH32_state_t* const state = XXH32_createState();
+ XXH32_reset(state, 0);
+ XXH32_update(state, data, data_size);
+ // Extend to cover last byte (compression type)
+ XXH32_update(state, &last_byte, 1);
+ uint32_t v = XXH32_digest(state);
+ XXH32_freeState(state);
+ return v;
+ }
+ case kxxHash64: {
+ XXH64_state_t* const state = XXH64_createState();
+ XXH64_reset(state, 0);
+ XXH64_update(state, data, data_size);
+ // Extend to cover last byte (compression type)
+ XXH64_update(state, &last_byte, 1);
+ uint32_t v = Lower32of64(XXH64_digest(state));
+ XXH64_freeState(state);
+ return v;
+ }
+ case kXXH3: {
+ // XXH3 is a complicated hash function that is extremely fast on
+ // contiguous input, but that makes its streaming support rather
+ // complex. It is worth custom handling of the last byte (`type`)
+ // in order to avoid allocating a large state object and bringing
+ // that code complexity into CPU working set.
+ uint32_t v = Lower32of64(XXH3_64bits(data, data_size));
+ return ModifyChecksumForLastByte(v, last_byte);
+ }
+ default: // including kNoChecksum
+ return 0;
+ }
+}
+
+Status UncompressBlockData(const UncompressionInfo& uncompression_info,
+ const char* data, size_t size,
+ BlockContents* out_contents, uint32_t format_version,
+ const ImmutableOptions& ioptions,
+ MemoryAllocator* allocator) {
+ Status ret = Status::OK();
+
+ assert(uncompression_info.type() != kNoCompression &&
+ "Invalid compression type");
+
+ StopWatchNano timer(ioptions.clock,
+ ShouldReportDetailedTime(ioptions.env, ioptions.stats));
+ size_t uncompressed_size = 0;
+ CacheAllocationPtr ubuf =
+ UncompressData(uncompression_info, data, size, &uncompressed_size,
+ GetCompressFormatForVersion(format_version), allocator);
+ if (!ubuf) {
+ if (!CompressionTypeSupported(uncompression_info.type())) {
+ return Status::NotSupported(
+ "Unsupported compression method for this build",
+ CompressionTypeToString(uncompression_info.type()));
+ } else {
+ return Status::Corruption(
+ "Corrupted compressed block contents",
+ CompressionTypeToString(uncompression_info.type()));
+ }
+ }
+
+ *out_contents = BlockContents(std::move(ubuf), uncompressed_size);
+
+ if (ShouldReportDetailedTime(ioptions.env, ioptions.stats)) {
+ RecordTimeToHistogram(ioptions.stats, DECOMPRESSION_TIMES_NANOS,
+ timer.ElapsedNanos());
+ }
+ RecordTimeToHistogram(ioptions.stats, BYTES_DECOMPRESSED,
+ out_contents->data.size());
+ RecordTick(ioptions.stats, NUMBER_BLOCK_DECOMPRESSED);
+
+ TEST_SYNC_POINT_CALLBACK("UncompressBlockData:TamperWithReturnValue",
+ static_cast<void*>(&ret));
+ TEST_SYNC_POINT_CALLBACK(
+ "UncompressBlockData:"
+ "TamperWithDecompressionOutput",
+ static_cast<void*>(out_contents));
+
+ return ret;
+}
+
+Status UncompressSerializedBlock(const UncompressionInfo& uncompression_info,
+ const char* data, size_t size,
+ BlockContents* out_contents,
+ uint32_t format_version,
+ const ImmutableOptions& ioptions,
+ MemoryAllocator* allocator) {
+ assert(data[size] != kNoCompression);
+ assert(data[size] == static_cast<char>(uncompression_info.type()));
+ return UncompressBlockData(uncompression_info, data, size, out_contents,
+ format_version, ioptions, allocator);
+}
+
+// Replace the contents of db_host_id with the actual hostname, if db_host_id
+// matches the keyword kHostnameForDbHostId
+Status ReifyDbHostIdProperty(Env* env, std::string* db_host_id) {
+ assert(db_host_id);
+ if (*db_host_id == kHostnameForDbHostId) {
+ Status s = env->GetHostNameString(db_host_id);
+ if (!s.ok()) {
+ db_host_id->clear();
+ }
+ return s;
+ }
+
+ return Status::OK();
+}
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/format.h b/src/rocksdb/table/format.h
new file mode 100644
index 000000000..ffb9fb0ca
--- /dev/null
+++ b/src/rocksdb/table/format.h
@@ -0,0 +1,375 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#pragma once
+
+#include <array>
+#include <cstdint>
+#include <string>
+
+#include "file/file_prefetch_buffer.h"
+#include "file/random_access_file_reader.h"
+#include "memory/memory_allocator.h"
+#include "options/cf_options.h"
+#include "port/malloc.h"
+#include "port/port.h" // noexcept
+#include "rocksdb/slice.h"
+#include "rocksdb/status.h"
+#include "rocksdb/table.h"
+#include "util/hash.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class RandomAccessFile;
+struct ReadOptions;
+
+bool ShouldReportDetailedTime(Env* env, Statistics* stats);
+
+// the length of the magic number in bytes.
+constexpr uint32_t kMagicNumberLengthByte = 8;
+
+// BlockHandle is a pointer to the extent of a file that stores a data
+// block or a meta block.
+class BlockHandle {
+ public:
+ // Creates a block handle with special values indicating "uninitialized,"
+ // distinct from the "null" block handle.
+ BlockHandle();
+ BlockHandle(uint64_t offset, uint64_t size);
+
+ // The offset of the block in the file.
+ uint64_t offset() const { return offset_; }
+ void set_offset(uint64_t _offset) { offset_ = _offset; }
+
+ // The size of the stored block
+ uint64_t size() const { return size_; }
+ void set_size(uint64_t _size) { size_ = _size; }
+
+ void EncodeTo(std::string* dst) const;
+ char* EncodeTo(char* dst) const;
+ Status DecodeFrom(Slice* input);
+ Status DecodeSizeFrom(uint64_t offset, Slice* input);
+
+ // Return a string that contains the copy of handle.
+ std::string ToString(bool hex = true) const;
+
+ // if the block handle's offset and size are both "0", we will view it
+ // as a null block handle that points to no where.
+ bool IsNull() const { return offset_ == 0 && size_ == 0; }
+
+ static const BlockHandle& NullBlockHandle() { return kNullBlockHandle; }
+
+ // Maximum encoding length of a BlockHandle
+ static constexpr uint32_t kMaxEncodedLength = 2 * kMaxVarint64Length;
+
+ inline bool operator==(const BlockHandle& rhs) const {
+ return offset_ == rhs.offset_ && size_ == rhs.size_;
+ }
+ inline bool operator!=(const BlockHandle& rhs) const {
+ return !(*this == rhs);
+ }
+
+ private:
+ uint64_t offset_;
+ uint64_t size_;
+
+ static const BlockHandle kNullBlockHandle;
+};
+
+// Value in block-based table file index.
+//
+// The index entry for block n is: y -> h, [x],
+// where: y is some key between the last key of block n (inclusive) and the
+// first key of block n+1 (exclusive); h is BlockHandle pointing to block n;
+// x, if present, is the first key of block n (unshortened).
+// This struct represents the "h, [x]" part.
+struct IndexValue {
+ BlockHandle handle;
+ // Empty means unknown.
+ Slice first_internal_key;
+
+ IndexValue() = default;
+ IndexValue(BlockHandle _handle, Slice _first_internal_key)
+ : handle(_handle), first_internal_key(_first_internal_key) {}
+
+ // have_first_key indicates whether the `first_internal_key` is used.
+ // If previous_handle is not null, delta encoding is used;
+ // in this case, the two handles must point to consecutive blocks:
+ // handle.offset() ==
+ // previous_handle->offset() + previous_handle->size() + kBlockTrailerSize
+ void EncodeTo(std::string* dst, bool have_first_key,
+ const BlockHandle* previous_handle) const;
+ Status DecodeFrom(Slice* input, bool have_first_key,
+ const BlockHandle* previous_handle);
+
+ std::string ToString(bool hex, bool have_first_key) const;
+};
+
+inline uint32_t GetCompressFormatForVersion(uint32_t format_version) {
+ // As of format_version 2, we encode compressed block with
+ // compress_format_version == 2. Before that, the version is 1.
+ // DO NOT CHANGE THIS FUNCTION, it affects disk format
+ return format_version >= 2 ? 2 : 1;
+}
+
+constexpr uint32_t kLatestFormatVersion = 5;
+
+inline bool IsSupportedFormatVersion(uint32_t version) {
+ return version <= kLatestFormatVersion;
+}
+
+// Footer encapsulates the fixed information stored at the tail end of every
+// SST file. In general, it should only include things that cannot go
+// elsewhere under the metaindex block. For example, checksum_type is
+// required for verifying metaindex block checksum (when applicable), but
+// index block handle can easily go in metaindex block (possible future).
+// See also FooterBuilder below.
+class Footer {
+ public:
+ // Create empty. Populate using DecodeFrom.
+ Footer() {}
+
+ // Deserialize a footer (populate fields) from `input` and check for various
+ // corruptions. `input_offset` is the offset within the target file of
+ // `input` buffer (future use).
+ Status DecodeFrom(Slice input, uint64_t input_offset);
+
+ // Table magic number identifies file as RocksDB SST file and which kind of
+ // SST format is use.
+ uint64_t table_magic_number() const { return table_magic_number_; }
+
+ // A version (footer and more) within a kind of SST. (It would add more
+ // unnecessary complexity to separate footer versions and
+ // BBTO::format_version.)
+ uint32_t format_version() const { return format_version_; }
+
+ // Block handle for metaindex block.
+ const BlockHandle& metaindex_handle() const { return metaindex_handle_; }
+
+ // Block handle for (top-level) index block.
+ const BlockHandle& index_handle() const { return index_handle_; }
+
+ // Checksum type used in the file.
+ ChecksumType checksum_type() const {
+ return static_cast<ChecksumType>(checksum_type_);
+ }
+
+ // Block trailer size used by file with this footer (e.g. 5 for block-based
+ // table and 0 for plain table). This is inferred from magic number so
+ // not in the serialized form.
+ inline size_t GetBlockTrailerSize() const { return block_trailer_size_; }
+
+ // Convert this object to a human readable form
+ std::string ToString() const;
+
+ // Encoded lengths of Footers. Bytes for serialized Footer will always be
+ // >= kMinEncodedLength and <= kMaxEncodedLength.
+ //
+ // Footer version 0 (legacy) will always occupy exactly this many bytes.
+ // It consists of two block handles, padding, and a magic number.
+ static constexpr uint32_t kVersion0EncodedLength =
+ 2 * BlockHandle::kMaxEncodedLength + kMagicNumberLengthByte;
+ static constexpr uint32_t kMinEncodedLength = kVersion0EncodedLength;
+
+ // Footer of versions 1 and higher will always occupy exactly this many
+ // bytes. It originally consisted of the checksum type, two block handles,
+ // padding (to maximum handle encoding size), a format version number, and a
+ // magic number.
+ static constexpr uint32_t kNewVersionsEncodedLength =
+ 1 + 2 * BlockHandle::kMaxEncodedLength + 4 + kMagicNumberLengthByte;
+ static constexpr uint32_t kMaxEncodedLength = kNewVersionsEncodedLength;
+
+ static constexpr uint64_t kNullTableMagicNumber = 0;
+
+ static constexpr uint32_t kInvalidFormatVersion = 0xffffffffU;
+
+ private:
+ static constexpr int kInvalidChecksumType =
+ (1 << (sizeof(ChecksumType) * 8)) | kNoChecksum;
+
+ uint64_t table_magic_number_ = kNullTableMagicNumber;
+ uint32_t format_version_ = kInvalidFormatVersion;
+ BlockHandle metaindex_handle_;
+ BlockHandle index_handle_;
+ int checksum_type_ = kInvalidChecksumType;
+ uint8_t block_trailer_size_ = 0;
+};
+
+// Builder for Footer
+class FooterBuilder {
+ public:
+ // Run builder in inputs. This is a single step with lots of parameters for
+ // efficiency (based on perf testing).
+ // * table_magic_number identifies file as RocksDB SST file and which kind of
+ // SST format is use.
+ // * format_version is a version for the footer and can also apply to other
+ // aspects of the SST file (see BlockBasedTableOptions::format_version).
+ // NOTE: To save complexity in the caller, when format_version == 0 and
+ // there is a corresponding legacy magic number to the one specified, the
+ // legacy magic number will be written for forward compatibility.
+ // * footer_offset is the file offset where the footer will be written
+ // (for future use).
+ // * checksum_type is for formats using block checksums.
+ // * index_handle is optional for some kinds of SST files.
+ void Build(uint64_t table_magic_number, uint32_t format_version,
+ uint64_t footer_offset, ChecksumType checksum_type,
+ const BlockHandle& metaindex_handle,
+ const BlockHandle& index_handle = BlockHandle::NullBlockHandle());
+
+ // After Builder, get a Slice for the serialized Footer, backed by this
+ // FooterBuilder.
+ const Slice& GetSlice() const {
+ assert(slice_.size());
+ return slice_;
+ }
+
+ private:
+ Slice slice_;
+ std::array<char, Footer::kMaxEncodedLength> data_;
+};
+
+// Read the footer from file
+// If enforce_table_magic_number != 0, ReadFooterFromFile() will return
+// corruption if table_magic number is not equal to enforce_table_magic_number
+Status ReadFooterFromFile(const IOOptions& opts, RandomAccessFileReader* file,
+ FilePrefetchBuffer* prefetch_buffer,
+ uint64_t file_size, Footer* footer,
+ uint64_t enforce_table_magic_number = 0);
+
+// Computes a checksum using the given ChecksumType. Sometimes we need to
+// include one more input byte logically at the end but not part of the main
+// data buffer. If data_size >= 1, then
+// ComputeBuiltinChecksum(type, data, size)
+// ==
+// ComputeBuiltinChecksumWithLastByte(type, data, size - 1, data[size - 1])
+uint32_t ComputeBuiltinChecksum(ChecksumType type, const char* data,
+ size_t size);
+uint32_t ComputeBuiltinChecksumWithLastByte(ChecksumType type, const char* data,
+ size_t size, char last_byte);
+
+// Represents the contents of a block read from an SST file. Depending on how
+// it's created, it may or may not own the actual block bytes. As an example,
+// BlockContents objects representing data read from mmapped files only point
+// into the mmapped region. Depending on context, it might be a serialized
+// (potentially compressed) block, including a trailer beyond `size`, or an
+// uncompressed block.
+//
+// Please try to use this terminology when dealing with blocks:
+// * "Serialized block" - bytes that go into storage. For block-based table
+// (usually the case) this includes the block trailer. Here the `size` does
+// not include the trailer, but other places in code might include the trailer
+// in the size.
+// * "Maybe compressed block" - like a serialized block, but without the
+// trailer (or no promise of including a trailer). Must be accompanied by a
+// CompressionType in some other variable or field.
+// * "Uncompressed block" - "payload" bytes that are either stored with no
+// compression, used as input to compression function, or result of
+// decompression function.
+// * "Parsed block" - an in-memory form of a block in block cache, as it is
+// used by the table reader. Different C++ types are used depending on the
+// block type (see block_like_traits.h). Only trivially parsable block types
+// use BlockContents as the parsed form.
+//
+struct BlockContents {
+ // Points to block payload (without trailer)
+ Slice data;
+ CacheAllocationPtr allocation;
+
+#ifndef NDEBUG
+ // Whether there is a known trailer after what is pointed to by `data`.
+ // See BlockBasedTable::GetCompressionType.
+ bool has_trailer = false;
+#endif // NDEBUG
+
+ BlockContents() {}
+
+ // Does not take ownership of the underlying data bytes.
+ BlockContents(const Slice& _data) : data(_data) {}
+
+ // Takes ownership of the underlying data bytes.
+ BlockContents(CacheAllocationPtr&& _data, size_t _size)
+ : data(_data.get(), _size), allocation(std::move(_data)) {}
+
+ // Takes ownership of the underlying data bytes.
+ BlockContents(std::unique_ptr<char[]>&& _data, size_t _size)
+ : data(_data.get(), _size) {
+ allocation.reset(_data.release());
+ }
+
+ // Returns whether the object has ownership of the underlying data bytes.
+ bool own_bytes() const { return allocation.get() != nullptr; }
+
+ // The additional memory space taken by the block data.
+ size_t usable_size() const {
+ if (allocation.get() != nullptr) {
+ auto allocator = allocation.get_deleter().allocator;
+ if (allocator) {
+ return allocator->UsableSize(allocation.get(), data.size());
+ }
+#ifdef ROCKSDB_MALLOC_USABLE_SIZE
+ return malloc_usable_size(allocation.get());
+#else
+ return data.size();
+#endif // ROCKSDB_MALLOC_USABLE_SIZE
+ } else {
+ return 0; // no extra memory is occupied by the data
+ }
+ }
+
+ size_t ApproximateMemoryUsage() const {
+ return usable_size() + sizeof(*this);
+ }
+
+ BlockContents(BlockContents&& other) noexcept { *this = std::move(other); }
+
+ BlockContents& operator=(BlockContents&& other) {
+ data = std::move(other.data);
+ allocation = std::move(other.allocation);
+#ifndef NDEBUG
+ has_trailer = other.has_trailer;
+#endif // NDEBUG
+ return *this;
+ }
+};
+
+// The `data` points to serialized block contents read in from file, which
+// must be compressed and include a trailer beyond `size`. A new buffer is
+// allocated with the given allocator (or default) and the uncompressed
+// contents are returned in `out_contents`.
+// format_version is as defined in include/rocksdb/table.h, which is
+// used to determine compression format version.
+Status UncompressSerializedBlock(const UncompressionInfo& info,
+ const char* data, size_t size,
+ BlockContents* out_contents,
+ uint32_t format_version,
+ const ImmutableOptions& ioptions,
+ MemoryAllocator* allocator = nullptr);
+
+// This is a variant of UncompressSerializedBlock that does not expect a
+// block trailer beyond `size`. (CompressionType is taken from `info`.)
+Status UncompressBlockData(const UncompressionInfo& info, const char* data,
+ size_t size, BlockContents* out_contents,
+ uint32_t format_version,
+ const ImmutableOptions& ioptions,
+ MemoryAllocator* allocator = nullptr);
+
+// Replace db_host_id contents with the real hostname if necessary
+Status ReifyDbHostIdProperty(Env* env, std::string* db_host_id);
+
+// Implementation details follow. Clients should ignore,
+
+// TODO(andrewkr): we should prefer one way of representing a null/uninitialized
+// BlockHandle. Currently we use zeros for null and use negation-of-zeros for
+// uninitialized.
+inline BlockHandle::BlockHandle() : BlockHandle(~uint64_t{0}, ~uint64_t{0}) {}
+
+inline BlockHandle::BlockHandle(uint64_t _offset, uint64_t _size)
+ : offset_(_offset), size_(_size) {}
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/get_context.cc b/src/rocksdb/table/get_context.cc
new file mode 100644
index 000000000..69e752714
--- /dev/null
+++ b/src/rocksdb/table/get_context.cc
@@ -0,0 +1,604 @@
+// Copyright (c) 2011-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 "table/get_context.h"
+
+#include "db/blob//blob_fetcher.h"
+#include "db/merge_helper.h"
+#include "db/pinned_iterators_manager.h"
+#include "db/read_callback.h"
+#include "db/wide/wide_column_serialization.h"
+#include "monitoring/file_read_sample.h"
+#include "monitoring/perf_context_imp.h"
+#include "monitoring/statistics.h"
+#include "rocksdb/merge_operator.h"
+#include "rocksdb/statistics.h"
+#include "rocksdb/system_clock.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+namespace {
+
+void appendToReplayLog(std::string* replay_log, ValueType type, Slice value) {
+#ifndef ROCKSDB_LITE
+ if (replay_log) {
+ if (replay_log->empty()) {
+ // Optimization: in the common case of only one operation in the
+ // log, we allocate the exact amount of space needed.
+ replay_log->reserve(1 + VarintLength(value.size()) + value.size());
+ }
+ replay_log->push_back(type);
+ PutLengthPrefixedSlice(replay_log, value);
+ }
+#else
+ (void)replay_log;
+ (void)type;
+ (void)value;
+#endif // ROCKSDB_LITE
+}
+
+} // namespace
+
+GetContext::GetContext(
+ const Comparator* ucmp, const MergeOperator* merge_operator, Logger* logger,
+ Statistics* statistics, GetState init_state, const Slice& user_key,
+ PinnableSlice* pinnable_val, PinnableWideColumns* columns,
+ std::string* timestamp, bool* value_found, MergeContext* merge_context,
+ bool do_merge, SequenceNumber* _max_covering_tombstone_seq,
+ SystemClock* clock, SequenceNumber* seq,
+ PinnedIteratorsManager* _pinned_iters_mgr, ReadCallback* callback,
+ bool* is_blob_index, uint64_t tracing_get_id, BlobFetcher* blob_fetcher)
+ : ucmp_(ucmp),
+ merge_operator_(merge_operator),
+ logger_(logger),
+ statistics_(statistics),
+ state_(init_state),
+ user_key_(user_key),
+ pinnable_val_(pinnable_val),
+ columns_(columns),
+ timestamp_(timestamp),
+ value_found_(value_found),
+ merge_context_(merge_context),
+ max_covering_tombstone_seq_(_max_covering_tombstone_seq),
+ clock_(clock),
+ seq_(seq),
+ replay_log_(nullptr),
+ pinned_iters_mgr_(_pinned_iters_mgr),
+ callback_(callback),
+ do_merge_(do_merge),
+ is_blob_index_(is_blob_index),
+ tracing_get_id_(tracing_get_id),
+ blob_fetcher_(blob_fetcher) {
+ if (seq_) {
+ *seq_ = kMaxSequenceNumber;
+ }
+ sample_ = should_sample_file_read();
+}
+
+GetContext::GetContext(const Comparator* ucmp,
+ const MergeOperator* merge_operator, Logger* logger,
+ Statistics* statistics, GetState init_state,
+ const Slice& user_key, PinnableSlice* pinnable_val,
+ PinnableWideColumns* columns, bool* value_found,
+ MergeContext* merge_context, bool do_merge,
+ SequenceNumber* _max_covering_tombstone_seq,
+ SystemClock* clock, SequenceNumber* seq,
+ PinnedIteratorsManager* _pinned_iters_mgr,
+ ReadCallback* callback, bool* is_blob_index,
+ uint64_t tracing_get_id, BlobFetcher* blob_fetcher)
+ : GetContext(ucmp, merge_operator, logger, statistics, init_state, user_key,
+ pinnable_val, columns, /*timestamp=*/nullptr, value_found,
+ merge_context, do_merge, _max_covering_tombstone_seq, clock,
+ seq, _pinned_iters_mgr, callback, is_blob_index,
+ tracing_get_id, blob_fetcher) {}
+
+// Called from TableCache::Get and Table::Get when file/block in which
+// key may exist are not there in TableCache/BlockCache respectively. In this
+// case we can't guarantee that key does not exist and are not permitted to do
+// IO to be certain.Set the status=kFound and value_found=false to let the
+// caller know that key may exist but is not there in memory
+void GetContext::MarkKeyMayExist() {
+ state_ = kFound;
+ if (value_found_ != nullptr) {
+ *value_found_ = false;
+ }
+}
+
+void GetContext::SaveValue(const Slice& value, SequenceNumber /*seq*/) {
+ assert(state_ == kNotFound);
+ appendToReplayLog(replay_log_, kTypeValue, value);
+
+ state_ = kFound;
+ if (LIKELY(pinnable_val_ != nullptr)) {
+ pinnable_val_->PinSelf(value);
+ }
+}
+
+void GetContext::ReportCounters() {
+ if (get_context_stats_.num_cache_hit > 0) {
+ RecordTick(statistics_, BLOCK_CACHE_HIT, get_context_stats_.num_cache_hit);
+ }
+ if (get_context_stats_.num_cache_index_hit > 0) {
+ RecordTick(statistics_, BLOCK_CACHE_INDEX_HIT,
+ get_context_stats_.num_cache_index_hit);
+ }
+ if (get_context_stats_.num_cache_data_hit > 0) {
+ RecordTick(statistics_, BLOCK_CACHE_DATA_HIT,
+ get_context_stats_.num_cache_data_hit);
+ }
+ if (get_context_stats_.num_cache_filter_hit > 0) {
+ RecordTick(statistics_, BLOCK_CACHE_FILTER_HIT,
+ get_context_stats_.num_cache_filter_hit);
+ }
+ if (get_context_stats_.num_cache_compression_dict_hit > 0) {
+ RecordTick(statistics_, BLOCK_CACHE_COMPRESSION_DICT_HIT,
+ get_context_stats_.num_cache_compression_dict_hit);
+ }
+ if (get_context_stats_.num_cache_index_miss > 0) {
+ RecordTick(statistics_, BLOCK_CACHE_INDEX_MISS,
+ get_context_stats_.num_cache_index_miss);
+ }
+ if (get_context_stats_.num_cache_filter_miss > 0) {
+ RecordTick(statistics_, BLOCK_CACHE_FILTER_MISS,
+ get_context_stats_.num_cache_filter_miss);
+ }
+ if (get_context_stats_.num_cache_data_miss > 0) {
+ RecordTick(statistics_, BLOCK_CACHE_DATA_MISS,
+ get_context_stats_.num_cache_data_miss);
+ }
+ if (get_context_stats_.num_cache_compression_dict_miss > 0) {
+ RecordTick(statistics_, BLOCK_CACHE_COMPRESSION_DICT_MISS,
+ get_context_stats_.num_cache_compression_dict_miss);
+ }
+ if (get_context_stats_.num_cache_bytes_read > 0) {
+ RecordTick(statistics_, BLOCK_CACHE_BYTES_READ,
+ get_context_stats_.num_cache_bytes_read);
+ }
+ if (get_context_stats_.num_cache_miss > 0) {
+ RecordTick(statistics_, BLOCK_CACHE_MISS,
+ get_context_stats_.num_cache_miss);
+ }
+ if (get_context_stats_.num_cache_add > 0) {
+ RecordTick(statistics_, BLOCK_CACHE_ADD, get_context_stats_.num_cache_add);
+ }
+ if (get_context_stats_.num_cache_add_redundant > 0) {
+ RecordTick(statistics_, BLOCK_CACHE_ADD_REDUNDANT,
+ get_context_stats_.num_cache_add_redundant);
+ }
+ if (get_context_stats_.num_cache_bytes_write > 0) {
+ RecordTick(statistics_, BLOCK_CACHE_BYTES_WRITE,
+ get_context_stats_.num_cache_bytes_write);
+ }
+ if (get_context_stats_.num_cache_index_add > 0) {
+ RecordTick(statistics_, BLOCK_CACHE_INDEX_ADD,
+ get_context_stats_.num_cache_index_add);
+ }
+ if (get_context_stats_.num_cache_index_add_redundant > 0) {
+ RecordTick(statistics_, BLOCK_CACHE_INDEX_ADD_REDUNDANT,
+ get_context_stats_.num_cache_index_add_redundant);
+ }
+ if (get_context_stats_.num_cache_index_bytes_insert > 0) {
+ RecordTick(statistics_, BLOCK_CACHE_INDEX_BYTES_INSERT,
+ get_context_stats_.num_cache_index_bytes_insert);
+ }
+ if (get_context_stats_.num_cache_data_add > 0) {
+ RecordTick(statistics_, BLOCK_CACHE_DATA_ADD,
+ get_context_stats_.num_cache_data_add);
+ }
+ if (get_context_stats_.num_cache_data_add_redundant > 0) {
+ RecordTick(statistics_, BLOCK_CACHE_DATA_ADD_REDUNDANT,
+ get_context_stats_.num_cache_data_add_redundant);
+ }
+ if (get_context_stats_.num_cache_data_bytes_insert > 0) {
+ RecordTick(statistics_, BLOCK_CACHE_DATA_BYTES_INSERT,
+ get_context_stats_.num_cache_data_bytes_insert);
+ }
+ if (get_context_stats_.num_cache_filter_add > 0) {
+ RecordTick(statistics_, BLOCK_CACHE_FILTER_ADD,
+ get_context_stats_.num_cache_filter_add);
+ }
+ if (get_context_stats_.num_cache_filter_add_redundant > 0) {
+ RecordTick(statistics_, BLOCK_CACHE_FILTER_ADD_REDUNDANT,
+ get_context_stats_.num_cache_filter_add_redundant);
+ }
+ if (get_context_stats_.num_cache_filter_bytes_insert > 0) {
+ RecordTick(statistics_, BLOCK_CACHE_FILTER_BYTES_INSERT,
+ get_context_stats_.num_cache_filter_bytes_insert);
+ }
+ if (get_context_stats_.num_cache_compression_dict_add > 0) {
+ RecordTick(statistics_, BLOCK_CACHE_COMPRESSION_DICT_ADD,
+ get_context_stats_.num_cache_compression_dict_add);
+ }
+ if (get_context_stats_.num_cache_compression_dict_add_redundant > 0) {
+ RecordTick(statistics_, BLOCK_CACHE_COMPRESSION_DICT_ADD_REDUNDANT,
+ get_context_stats_.num_cache_compression_dict_add_redundant);
+ }
+ if (get_context_stats_.num_cache_compression_dict_bytes_insert > 0) {
+ RecordTick(statistics_, BLOCK_CACHE_COMPRESSION_DICT_BYTES_INSERT,
+ get_context_stats_.num_cache_compression_dict_bytes_insert);
+ }
+}
+
+bool GetContext::SaveValue(const ParsedInternalKey& parsed_key,
+ const Slice& value, bool* matched,
+ Cleanable* value_pinner) {
+ assert(matched);
+ assert((state_ != kMerge && parsed_key.type != kTypeMerge) ||
+ merge_context_ != nullptr);
+ if (ucmp_->EqualWithoutTimestamp(parsed_key.user_key, user_key_)) {
+ *matched = true;
+ // If the value is not in the snapshot, skip it
+ if (!CheckCallback(parsed_key.sequence)) {
+ return true; // to continue to the next seq
+ }
+
+ appendToReplayLog(replay_log_, parsed_key.type, value);
+
+ if (seq_ != nullptr) {
+ // Set the sequence number if it is uninitialized
+ if (*seq_ == kMaxSequenceNumber) {
+ *seq_ = parsed_key.sequence;
+ }
+ if (max_covering_tombstone_seq_) {
+ *seq_ = std::max(*seq_, *max_covering_tombstone_seq_);
+ }
+ }
+
+ size_t ts_sz = ucmp_->timestamp_size();
+ if (ts_sz > 0 && timestamp_ != nullptr) {
+ if (!timestamp_->empty()) {
+ assert(ts_sz == timestamp_->size());
+ // `timestamp` can be set before `SaveValue` is ever called
+ // when max_covering_tombstone_seq_ was set.
+ // If this key has a higher sequence number than range tombstone,
+ // then timestamp should be updated. `ts_from_rangetombstone_` is
+ // set to false afterwards so that only the key with highest seqno
+ // updates the timestamp.
+ if (ts_from_rangetombstone_) {
+ assert(max_covering_tombstone_seq_);
+ if (parsed_key.sequence > *max_covering_tombstone_seq_) {
+ Slice ts = ExtractTimestampFromUserKey(parsed_key.user_key, ts_sz);
+ timestamp_->assign(ts.data(), ts.size());
+ ts_from_rangetombstone_ = false;
+ }
+ }
+ }
+ // TODO optimize for small size ts
+ const std::string kMaxTs(ts_sz, '\xff');
+ if (timestamp_->empty() ||
+ ucmp_->CompareTimestamp(*timestamp_, kMaxTs) == 0) {
+ Slice ts = ExtractTimestampFromUserKey(parsed_key.user_key, ts_sz);
+ timestamp_->assign(ts.data(), ts.size());
+ }
+ }
+
+ auto type = parsed_key.type;
+ // Key matches. Process it
+ if ((type == kTypeValue || type == kTypeMerge || type == kTypeBlobIndex ||
+ type == kTypeWideColumnEntity || type == kTypeDeletion ||
+ type == kTypeDeletionWithTimestamp || type == kTypeSingleDeletion) &&
+ max_covering_tombstone_seq_ != nullptr &&
+ *max_covering_tombstone_seq_ > parsed_key.sequence) {
+ // Note that deletion types are also considered, this is for the case
+ // when we need to return timestamp to user. If a range tombstone has a
+ // higher seqno than point tombstone, its timestamp should be returned.
+ type = kTypeRangeDeletion;
+ }
+ switch (type) {
+ case kTypeValue:
+ case kTypeBlobIndex:
+ case kTypeWideColumnEntity:
+ assert(state_ == kNotFound || state_ == kMerge);
+ if (type == kTypeBlobIndex) {
+ if (is_blob_index_ == nullptr) {
+ // Blob value not supported. Stop.
+ state_ = kUnexpectedBlobIndex;
+ return false;
+ }
+ }
+
+ if (is_blob_index_ != nullptr) {
+ *is_blob_index_ = (type == kTypeBlobIndex);
+ }
+
+ if (kNotFound == state_) {
+ state_ = kFound;
+ if (do_merge_) {
+ if (LIKELY(pinnable_val_ != nullptr)) {
+ Slice value_to_use = value;
+
+ if (type == kTypeWideColumnEntity) {
+ Slice value_copy = value;
+
+ if (!WideColumnSerialization::GetValueOfDefaultColumn(
+ value_copy, value_to_use)
+ .ok()) {
+ state_ = kCorrupt;
+ return false;
+ }
+ }
+
+ if (LIKELY(value_pinner != nullptr)) {
+ // If the backing resources for the value are provided, pin them
+ pinnable_val_->PinSlice(value_to_use, value_pinner);
+ } else {
+ TEST_SYNC_POINT_CALLBACK("GetContext::SaveValue::PinSelf",
+ this);
+ // Otherwise copy the value
+ pinnable_val_->PinSelf(value_to_use);
+ }
+ } else if (columns_ != nullptr) {
+ if (type == kTypeWideColumnEntity) {
+ if (!columns_->SetWideColumnValue(value, value_pinner).ok()) {
+ state_ = kCorrupt;
+ return false;
+ }
+ } else {
+ columns_->SetPlainValue(value, value_pinner);
+ }
+ }
+ } else {
+ // It means this function is called as part of DB GetMergeOperands
+ // API and the current value should be part of
+ // merge_context_->operand_list
+ if (type == kTypeBlobIndex) {
+ PinnableSlice pin_val;
+ if (GetBlobValue(value, &pin_val) == false) {
+ return false;
+ }
+ Slice blob_value(pin_val);
+ push_operand(blob_value, nullptr);
+ } else if (type == kTypeWideColumnEntity) {
+ Slice value_copy = value;
+ Slice value_of_default;
+
+ if (!WideColumnSerialization::GetValueOfDefaultColumn(
+ value_copy, value_of_default)
+ .ok()) {
+ state_ = kCorrupt;
+ return false;
+ }
+
+ push_operand(value_of_default, value_pinner);
+ } else {
+ assert(type == kTypeValue);
+ push_operand(value, value_pinner);
+ }
+ }
+ } else if (kMerge == state_) {
+ assert(merge_operator_ != nullptr);
+ if (type == kTypeBlobIndex) {
+ PinnableSlice pin_val;
+ if (GetBlobValue(value, &pin_val) == false) {
+ return false;
+ }
+ Slice blob_value(pin_val);
+ state_ = kFound;
+ if (do_merge_) {
+ Merge(&blob_value);
+ } else {
+ // It means this function is called as part of DB GetMergeOperands
+ // API and the current value should be part of
+ // merge_context_->operand_list
+ push_operand(blob_value, nullptr);
+ }
+ } else if (type == kTypeWideColumnEntity) {
+ state_ = kFound;
+
+ if (do_merge_) {
+ MergeWithEntity(value);
+ } else {
+ // It means this function is called as part of DB GetMergeOperands
+ // API and the current value should be part of
+ // merge_context_->operand_list
+ Slice value_copy = value;
+ Slice value_of_default;
+
+ if (!WideColumnSerialization::GetValueOfDefaultColumn(
+ value_copy, value_of_default)
+ .ok()) {
+ state_ = kCorrupt;
+ return false;
+ }
+
+ push_operand(value_of_default, value_pinner);
+ }
+ } else {
+ assert(type == kTypeValue);
+
+ state_ = kFound;
+ if (do_merge_) {
+ Merge(&value);
+ } else {
+ // It means this function is called as part of DB GetMergeOperands
+ // API and the current value should be part of
+ // merge_context_->operand_list
+ push_operand(value, value_pinner);
+ }
+ }
+ }
+ return false;
+
+ case kTypeDeletion:
+ case kTypeDeletionWithTimestamp:
+ case kTypeSingleDeletion:
+ case kTypeRangeDeletion:
+ // TODO(noetzli): Verify correctness once merge of single-deletes
+ // is supported
+ assert(state_ == kNotFound || state_ == kMerge);
+ if (kNotFound == state_) {
+ state_ = kDeleted;
+ } else if (kMerge == state_) {
+ state_ = kFound;
+ if (do_merge_) {
+ Merge(nullptr);
+ }
+ // If do_merge_ = false then the current value shouldn't be part of
+ // merge_context_->operand_list
+ }
+ return false;
+
+ case kTypeMerge:
+ assert(state_ == kNotFound || state_ == kMerge);
+ state_ = kMerge;
+ // value_pinner is not set from plain_table_reader.cc for example.
+ push_operand(value, value_pinner);
+ if (do_merge_ && merge_operator_ != nullptr &&
+ merge_operator_->ShouldMerge(
+ merge_context_->GetOperandsDirectionBackward())) {
+ state_ = kFound;
+ Merge(nullptr);
+ return false;
+ }
+ return true;
+
+ default:
+ assert(false);
+ break;
+ }
+ }
+
+ // state_ could be Corrupt, merge or notfound
+ return false;
+}
+
+void GetContext::Merge(const Slice* value) {
+ assert(do_merge_);
+ assert(!pinnable_val_ || !columns_);
+
+ std::string result;
+ const Status s = MergeHelper::TimedFullMerge(
+ merge_operator_, user_key_, value, merge_context_->GetOperands(), &result,
+ logger_, statistics_, clock_, /* result_operand */ nullptr,
+ /* update_num_ops_stats */ true);
+ if (!s.ok()) {
+ state_ = kCorrupt;
+ return;
+ }
+
+ if (LIKELY(pinnable_val_ != nullptr)) {
+ *(pinnable_val_->GetSelf()) = std::move(result);
+ pinnable_val_->PinSelf();
+ return;
+ }
+
+ assert(columns_);
+ columns_->SetPlainValue(result);
+}
+
+void GetContext::MergeWithEntity(Slice entity) {
+ assert(do_merge_);
+ assert(!pinnable_val_ || !columns_);
+
+ if (LIKELY(pinnable_val_ != nullptr)) {
+ Slice value_of_default;
+
+ {
+ const Status s = WideColumnSerialization::GetValueOfDefaultColumn(
+ entity, value_of_default);
+ if (!s.ok()) {
+ state_ = kCorrupt;
+ return;
+ }
+ }
+
+ {
+ const Status s = MergeHelper::TimedFullMerge(
+ merge_operator_, user_key_, &value_of_default,
+ merge_context_->GetOperands(), pinnable_val_->GetSelf(), logger_,
+ statistics_, clock_, /* result_operand */ nullptr,
+ /* update_num_ops_stats */ true);
+ if (!s.ok()) {
+ state_ = kCorrupt;
+ return;
+ }
+ }
+
+ pinnable_val_->PinSelf();
+ return;
+ }
+
+ std::string result;
+
+ {
+ const Status s = MergeHelper::TimedFullMergeWithEntity(
+ merge_operator_, user_key_, entity, merge_context_->GetOperands(),
+ &result, logger_, statistics_, clock_, /* update_num_ops_stats */ true);
+ if (!s.ok()) {
+ state_ = kCorrupt;
+ return;
+ }
+ }
+
+ {
+ assert(columns_);
+ const Status s = columns_->SetWideColumnValue(result);
+ if (!s.ok()) {
+ state_ = kCorrupt;
+ return;
+ }
+ }
+}
+
+bool GetContext::GetBlobValue(const Slice& blob_index,
+ PinnableSlice* blob_value) {
+ constexpr FilePrefetchBuffer* prefetch_buffer = nullptr;
+ constexpr uint64_t* bytes_read = nullptr;
+
+ Status status = blob_fetcher_->FetchBlob(
+ user_key_, blob_index, prefetch_buffer, blob_value, bytes_read);
+ if (!status.ok()) {
+ if (status.IsIncomplete()) {
+ // FIXME: this code is not covered by unit tests
+ MarkKeyMayExist();
+ return false;
+ }
+ state_ = kCorrupt;
+ return false;
+ }
+ *is_blob_index_ = false;
+ return true;
+}
+
+void GetContext::push_operand(const Slice& value, Cleanable* value_pinner) {
+ // TODO(yanqin) preserve timestamps information in merge_context
+ if (pinned_iters_mgr() && pinned_iters_mgr()->PinningEnabled() &&
+ value_pinner != nullptr) {
+ value_pinner->DelegateCleanupsTo(pinned_iters_mgr());
+ merge_context_->PushOperand(value, true /*value_pinned*/);
+ } else {
+ merge_context_->PushOperand(value, false);
+ }
+}
+
+void replayGetContextLog(const Slice& replay_log, const Slice& user_key,
+ GetContext* get_context, Cleanable* value_pinner) {
+#ifndef ROCKSDB_LITE
+ Slice s = replay_log;
+ while (s.size()) {
+ auto type = static_cast<ValueType>(*s.data());
+ s.remove_prefix(1);
+ Slice value;
+ bool ret = GetLengthPrefixedSlice(&s, &value);
+ assert(ret);
+ (void)ret;
+
+ bool dont_care __attribute__((__unused__));
+ // Since SequenceNumber is not stored and unknown, we will use
+ // kMaxSequenceNumber.
+ get_context->SaveValue(
+ ParsedInternalKey(user_key, kMaxSequenceNumber, type), value,
+ &dont_care, value_pinner);
+ }
+#else // ROCKSDB_LITE
+ (void)replay_log;
+ (void)user_key;
+ (void)get_context;
+ (void)value_pinner;
+ assert(false);
+#endif // ROCKSDB_LITE
+}
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/get_context.h b/src/rocksdb/table/get_context.h
new file mode 100644
index 000000000..dcc7ab8d6
--- /dev/null
+++ b/src/rocksdb/table/get_context.h
@@ -0,0 +1,231 @@
+// Copyright (c) 2011-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).
+
+#pragma once
+#include <string>
+
+#include "db/read_callback.h"
+#include "rocksdb/types.h"
+
+namespace ROCKSDB_NAMESPACE {
+class BlobFetcher;
+class Comparator;
+class Logger;
+class MergeContext;
+class MergeOperator;
+class PinnableWideColumns;
+class PinnedIteratorsManager;
+class Statistics;
+class SystemClock;
+struct ParsedInternalKey;
+
+// Data structure for accumulating statistics during a point lookup. At the
+// end of the point lookup, the corresponding ticker stats are updated. This
+// avoids the overhead of frequent ticker stats updates
+struct GetContextStats {
+ uint64_t num_cache_hit = 0;
+ uint64_t num_cache_index_hit = 0;
+ uint64_t num_cache_data_hit = 0;
+ uint64_t num_cache_filter_hit = 0;
+ uint64_t num_cache_compression_dict_hit = 0;
+ uint64_t num_cache_index_miss = 0;
+ uint64_t num_cache_filter_miss = 0;
+ uint64_t num_cache_data_miss = 0;
+ uint64_t num_cache_compression_dict_miss = 0;
+ uint64_t num_cache_bytes_read = 0;
+ uint64_t num_cache_miss = 0;
+ uint64_t num_cache_add = 0;
+ uint64_t num_cache_add_redundant = 0;
+ uint64_t num_cache_bytes_write = 0;
+ uint64_t num_cache_index_add = 0;
+ uint64_t num_cache_index_add_redundant = 0;
+ uint64_t num_cache_index_bytes_insert = 0;
+ uint64_t num_cache_data_add = 0;
+ uint64_t num_cache_data_add_redundant = 0;
+ uint64_t num_cache_data_bytes_insert = 0;
+ uint64_t num_cache_filter_add = 0;
+ uint64_t num_cache_filter_add_redundant = 0;
+ uint64_t num_cache_filter_bytes_insert = 0;
+ uint64_t num_cache_compression_dict_add = 0;
+ uint64_t num_cache_compression_dict_add_redundant = 0;
+ uint64_t num_cache_compression_dict_bytes_insert = 0;
+ // MultiGet stats.
+ uint64_t num_filter_read = 0;
+ uint64_t num_index_read = 0;
+ uint64_t num_sst_read = 0;
+};
+
+// A class to hold context about a point lookup, such as pointer to value
+// slice, key, merge context etc, as well as the current state of the
+// lookup. Any user using GetContext to track the lookup result must call
+// SaveValue() whenever the internal key is found. This can happen
+// repeatedly in case of merge operands. In case the key may exist with
+// high probability, but IO is required to confirm and the user doesn't allow
+// it, MarkKeyMayExist() must be called instead of SaveValue().
+class GetContext {
+ public:
+ // Current state of the point lookup. All except kNotFound and kMerge are
+ // terminal states
+ enum GetState {
+ kNotFound,
+ kFound,
+ kDeleted,
+ kCorrupt,
+ kMerge, // saver contains the current merge result (the operands)
+ kUnexpectedBlobIndex,
+ };
+ GetContextStats get_context_stats_;
+
+ // Constructor
+ // @param value Holds the value corresponding to user_key. If its nullptr
+ // then return all merge operands corresponding to user_key
+ // via merge_context
+ // @param value_found If non-nullptr, set to false if key may be present
+ // but we can't be certain because we cannot do IO
+ // @param max_covering_tombstone_seq Pointer to highest sequence number of
+ // range deletion covering the key. When an internal key
+ // is found with smaller sequence number, the lookup
+ // terminates
+ // @param seq If non-nullptr, the sequence number of the found key will be
+ // saved here
+ // @param callback Pointer to ReadCallback to perform additional checks
+ // for visibility of a key
+ // @param is_blob_index If non-nullptr, will be used to indicate if a found
+ // key is of type blob index
+ // @param do_merge True if value associated with user_key has to be returned
+ // and false if all the merge operands associated with user_key has to be
+ // returned. Id do_merge=false then all the merge operands are stored in
+ // merge_context and they are never merged. The value pointer is untouched.
+ GetContext(const Comparator* ucmp, const MergeOperator* merge_operator,
+ Logger* logger, Statistics* statistics, GetState init_state,
+ const Slice& user_key, PinnableSlice* value,
+ PinnableWideColumns* columns, bool* value_found,
+ MergeContext* merge_context, bool do_merge,
+ SequenceNumber* max_covering_tombstone_seq, SystemClock* clock,
+ SequenceNumber* seq = nullptr,
+ PinnedIteratorsManager* _pinned_iters_mgr = nullptr,
+ ReadCallback* callback = nullptr, bool* is_blob_index = nullptr,
+ uint64_t tracing_get_id = 0, BlobFetcher* blob_fetcher = nullptr);
+ GetContext(const Comparator* ucmp, const MergeOperator* merge_operator,
+ Logger* logger, Statistics* statistics, GetState init_state,
+ const Slice& user_key, PinnableSlice* value,
+ PinnableWideColumns* columns, std::string* timestamp,
+ bool* value_found, MergeContext* merge_context, bool do_merge,
+ SequenceNumber* max_covering_tombstone_seq, SystemClock* clock,
+ SequenceNumber* seq = nullptr,
+ PinnedIteratorsManager* _pinned_iters_mgr = nullptr,
+ ReadCallback* callback = nullptr, bool* is_blob_index = nullptr,
+ uint64_t tracing_get_id = 0, BlobFetcher* blob_fetcher = nullptr);
+
+ GetContext() = delete;
+
+ // This can be called to indicate that a key may be present, but cannot be
+ // confirmed due to IO not allowed
+ void MarkKeyMayExist();
+
+ // Records this key, value, and any meta-data (such as sequence number and
+ // state) into this GetContext.
+ //
+ // If the parsed_key matches the user key that we are looking for, sets
+ // matched to true.
+ //
+ // Returns True if more keys need to be read (due to merges) or
+ // False if the complete value has been found.
+ bool SaveValue(const ParsedInternalKey& parsed_key, const Slice& value,
+ bool* matched, Cleanable* value_pinner = nullptr);
+
+ // Simplified version of the previous function. Should only be used when we
+ // know that the operation is a Put.
+ void SaveValue(const Slice& value, SequenceNumber seq);
+
+ GetState State() const { return state_; }
+
+ SequenceNumber* max_covering_tombstone_seq() {
+ return max_covering_tombstone_seq_;
+ }
+
+ bool NeedTimestamp() { return timestamp_ != nullptr; }
+
+ void SetTimestampFromRangeTombstone(const Slice& timestamp) {
+ assert(timestamp_);
+ timestamp_->assign(timestamp.data(), timestamp.size());
+ ts_from_rangetombstone_ = true;
+ }
+
+ PinnedIteratorsManager* pinned_iters_mgr() { return pinned_iters_mgr_; }
+
+ // If a non-null string is passed, all the SaveValue calls will be
+ // logged into the string. The operations can then be replayed on
+ // another GetContext with replayGetContextLog.
+ void SetReplayLog(std::string* replay_log) { replay_log_ = replay_log; }
+
+ // Do we need to fetch the SequenceNumber for this key?
+ bool NeedToReadSequence() const { return (seq_ != nullptr); }
+
+ bool sample() const { return sample_; }
+
+ bool CheckCallback(SequenceNumber seq) {
+ if (callback_) {
+ return callback_->IsVisible(seq);
+ }
+ return true;
+ }
+
+ void ReportCounters();
+
+ bool has_callback() const { return callback_ != nullptr; }
+
+ uint64_t get_tracing_get_id() const { return tracing_get_id_; }
+
+ void push_operand(const Slice& value, Cleanable* value_pinner);
+
+ private:
+ void Merge(const Slice* value);
+ void MergeWithEntity(Slice entity);
+ bool GetBlobValue(const Slice& blob_index, PinnableSlice* blob_value);
+
+ const Comparator* ucmp_;
+ const MergeOperator* merge_operator_;
+ // the merge operations encountered;
+ Logger* logger_;
+ Statistics* statistics_;
+
+ GetState state_;
+ Slice user_key_;
+ PinnableSlice* pinnable_val_;
+ PinnableWideColumns* columns_;
+ std::string* timestamp_;
+ bool ts_from_rangetombstone_{false};
+ bool* value_found_; // Is value set correctly? Used by KeyMayExist
+ MergeContext* merge_context_;
+ SequenceNumber* max_covering_tombstone_seq_;
+ SystemClock* clock_;
+ // If a key is found, seq_ will be set to the SequenceNumber of most recent
+ // write to the key or kMaxSequenceNumber if unknown
+ SequenceNumber* seq_;
+ std::string* replay_log_;
+ // Used to temporarily pin blocks when state_ == GetContext::kMerge
+ PinnedIteratorsManager* pinned_iters_mgr_;
+ ReadCallback* callback_;
+ bool sample_;
+ // Value is true if it's called as part of DB Get API and false if it's
+ // called as part of DB GetMergeOperands API. When it's false merge operators
+ // are never merged.
+ bool do_merge_;
+ bool* is_blob_index_;
+ // Used for block cache tracing only. A tracing get id uniquely identifies a
+ // Get or a MultiGet.
+ const uint64_t tracing_get_id_;
+ BlobFetcher* blob_fetcher_;
+};
+
+// Call this to replay a log and bring the get_context up to date. The replay
+// log must have been created by another GetContext object, whose replay log
+// must have been set by calling GetContext::SetReplayLog().
+void replayGetContextLog(const Slice& replay_log, const Slice& user_key,
+ GetContext* get_context,
+ Cleanable* value_pinner = nullptr);
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/internal_iterator.h b/src/rocksdb/table/internal_iterator.h
new file mode 100644
index 000000000..945dec806
--- /dev/null
+++ b/src/rocksdb/table/internal_iterator.h
@@ -0,0 +1,226 @@
+// Copyright (c) 2011-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).
+//
+
+#pragma once
+
+#include <string>
+
+#include "db/dbformat.h"
+#include "file/readahead_file_info.h"
+#include "rocksdb/comparator.h"
+#include "rocksdb/iterator.h"
+#include "rocksdb/status.h"
+#include "table/format.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class PinnedIteratorsManager;
+
+enum class IterBoundCheck : char {
+ kUnknown = 0,
+ kOutOfBound,
+ kInbound,
+};
+
+struct IterateResult {
+ Slice key;
+ IterBoundCheck bound_check_result = IterBoundCheck::kUnknown;
+ // If false, PrepareValue() needs to be called before value().
+ bool value_prepared = true;
+};
+
+template <class TValue>
+class InternalIteratorBase : public Cleanable {
+ public:
+ InternalIteratorBase() {}
+
+ // No copying allowed
+ InternalIteratorBase(const InternalIteratorBase&) = delete;
+ InternalIteratorBase& operator=(const InternalIteratorBase&) = delete;
+
+ virtual ~InternalIteratorBase() {}
+
+ // An iterator is either positioned at a key/value pair, or
+ // not valid. This method returns true iff the iterator is valid.
+ // Always returns false if !status().ok().
+ virtual bool Valid() const = 0;
+
+ // Position at the first key in the source. The iterator is Valid()
+ // after this call iff the source is not empty.
+ virtual void SeekToFirst() = 0;
+
+ // Position at the last key in the source. The iterator is
+ // Valid() after this call iff the source is not empty.
+ virtual void SeekToLast() = 0;
+
+ // Position at the first key in the source that at or past target
+ // The iterator is Valid() after this call iff the source contains
+ // an entry that comes at or past target.
+ // All Seek*() methods clear any error status() that the iterator had prior to
+ // the call; after the seek, status() indicates only the error (if any) that
+ // happened during the seek, not any past errors.
+ // 'target' contains user timestamp if timestamp is enabled.
+ virtual void Seek(const Slice& target) = 0;
+
+ // Position at the first key in the source that at or before target
+ // The iterator is Valid() after this call iff the source contains
+ // an entry that comes at or before target.
+ virtual void SeekForPrev(const Slice& target) = 0;
+
+ // Moves to the next entry in the source. After this call, Valid() is
+ // true iff the iterator was not positioned at the last entry in the source.
+ // REQUIRES: Valid()
+ virtual void Next() = 0;
+
+ // Moves to the next entry in the source, and return result. Iterator
+ // implementation should override this method to help methods inline better,
+ // or when UpperBoundCheckResult() is non-trivial.
+ // REQUIRES: Valid()
+ virtual bool NextAndGetResult(IterateResult* result) {
+ Next();
+ bool is_valid = Valid();
+ if (is_valid) {
+ result->key = key();
+ // Default may_be_out_of_upper_bound to true to avoid unnecessary virtual
+ // call. If an implementation has non-trivial UpperBoundCheckResult(),
+ // it should also override NextAndGetResult().
+ result->bound_check_result = IterBoundCheck::kUnknown;
+ result->value_prepared = false;
+ assert(UpperBoundCheckResult() != IterBoundCheck::kOutOfBound);
+ }
+ return is_valid;
+ }
+
+ // Moves to the previous entry in the source. After this call, Valid() is
+ // true iff the iterator was not positioned at the first entry in source.
+ // REQUIRES: Valid()
+ virtual void Prev() = 0;
+
+ // Return the key for the current entry. The underlying storage for
+ // the returned slice is valid only until the next modification of
+ // the iterator.
+ // REQUIRES: Valid()
+ virtual Slice key() const = 0;
+
+ // Return user key for the current entry.
+ // REQUIRES: Valid()
+ virtual Slice user_key() const { return ExtractUserKey(key()); }
+
+ // Return the value for the current entry. The underlying storage for
+ // the returned slice is valid only until the next modification of
+ // the iterator.
+ // REQUIRES: Valid()
+ // REQUIRES: PrepareValue() has been called if needed (see PrepareValue()).
+ virtual TValue value() const = 0;
+
+ // If an error has occurred, return it. Else return an ok status.
+ // If non-blocking IO is requested and this operation cannot be
+ // satisfied without doing some IO, then this returns Status::Incomplete().
+ virtual Status status() const = 0;
+
+ // For some types of iterators, sometimes Seek()/Next()/SeekForPrev()/etc may
+ // load key but not value (to avoid the IO cost of reading the value from disk
+ // if it won't be not needed). This method loads the value in such situation.
+ //
+ // Needs to be called before value() at least once after each iterator
+ // movement (except if IterateResult::value_prepared = true), for iterators
+ // created with allow_unprepared_value = true.
+ //
+ // Returns false if an error occurred; in this case Valid() is also changed
+ // to false, and status() is changed to non-ok.
+ // REQUIRES: Valid()
+ virtual bool PrepareValue() { return true; }
+
+ // Keys return from this iterator can be smaller than iterate_lower_bound.
+ virtual bool MayBeOutOfLowerBound() { return true; }
+
+ // If the iterator has checked the key against iterate_upper_bound, returns
+ // the result here. The function can be used by user of the iterator to skip
+ // their own checks. If Valid() = true, IterBoundCheck::kUnknown is always
+ // a valid value. If Valid() = false, IterBoundCheck::kOutOfBound indicates
+ // that the iterator is filtered out by upper bound checks.
+ virtual IterBoundCheck UpperBoundCheckResult() {
+ return IterBoundCheck::kUnknown;
+ }
+
+ // Pass the PinnedIteratorsManager to the Iterator, most Iterators don't
+ // communicate with PinnedIteratorsManager so default implementation is no-op
+ // but for Iterators that need to communicate with PinnedIteratorsManager
+ // they will implement this function and use the passed pointer to communicate
+ // with PinnedIteratorsManager.
+ virtual void SetPinnedItersMgr(PinnedIteratorsManager* /*pinned_iters_mgr*/) {
+ }
+
+ // If true, this means that the Slice returned by key() is valid as long as
+ // PinnedIteratorsManager::ReleasePinnedData is not called and the
+ // Iterator is not deleted.
+ //
+ // IsKeyPinned() is guaranteed to always return true if
+ // - Iterator is created with ReadOptions::pin_data = true
+ // - DB tables were created with BlockBasedTableOptions::use_delta_encoding
+ // set to false.
+ virtual bool IsKeyPinned() const { return false; }
+
+ // If true, this means that the Slice returned by value() is valid as long as
+ // PinnedIteratorsManager::ReleasePinnedData is not called and the
+ // Iterator is not deleted.
+ // REQUIRES: Same as for value().
+ virtual bool IsValuePinned() const { return false; }
+
+ virtual Status GetProperty(std::string /*prop_name*/, std::string* /*prop*/) {
+ return Status::NotSupported("");
+ }
+
+ // When iterator moves from one file to another file at same level, new file's
+ // readahead state (details of last block read) is updated with previous
+ // file's readahead state. This way internal readahead_size of Prefetch Buffer
+ // doesn't start from scratch and can fall back to 8KB with no prefetch if
+ // reads are not sequential.
+ //
+ // Default implementation is no-op and its implemented by iterators.
+ virtual void GetReadaheadState(ReadaheadFileInfo* /*readahead_file_info*/) {}
+
+ // Default implementation is no-op and its implemented by iterators.
+ virtual void SetReadaheadState(ReadaheadFileInfo* /*readahead_file_info*/) {}
+
+ // When used under merging iterator, LevelIterator treats file boundaries
+ // as sentinel keys to prevent it from moving to next SST file before range
+ // tombstones in the current SST file are no longer needed. This method makes
+ // it cheap to check if the current key is a sentinel key. This should only be
+ // used by MergingIterator and LevelIterator for now.
+ virtual bool IsDeleteRangeSentinelKey() const { return false; }
+
+ protected:
+ void SeekForPrevImpl(const Slice& target, const CompareInterface* cmp) {
+ Seek(target);
+ if (!Valid()) {
+ SeekToLast();
+ }
+ while (Valid() && cmp->Compare(target, key()) < 0) {
+ Prev();
+ }
+ }
+
+ bool is_mutable_;
+};
+
+using InternalIterator = InternalIteratorBase<Slice>;
+
+// Return an empty iterator (yields nothing).
+template <class TValue = Slice>
+extern InternalIteratorBase<TValue>* NewEmptyInternalIterator();
+
+// Return an empty iterator with the specified status.
+template <class TValue = Slice>
+extern InternalIteratorBase<TValue>* NewErrorInternalIterator(
+ const Status& status);
+
+// Return an empty iterator with the specified status, allocated arena.
+template <class TValue = Slice>
+extern InternalIteratorBase<TValue>* NewErrorInternalIterator(
+ const Status& status, Arena* arena);
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/iter_heap.h b/src/rocksdb/table/iter_heap.h
new file mode 100644
index 000000000..6ad94be9b
--- /dev/null
+++ b/src/rocksdb/table/iter_heap.h
@@ -0,0 +1,44 @@
+// Copyright (c) 2011-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).
+//
+
+#pragma once
+
+#include "db/dbformat.h"
+#include "table/iterator_wrapper.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+// When used with std::priority_queue, this comparison functor puts the
+// iterator with the max/largest key on top.
+class MaxIteratorComparator {
+ public:
+ MaxIteratorComparator(const InternalKeyComparator* comparator)
+ : comparator_(comparator) {}
+
+ bool operator()(IteratorWrapper* a, IteratorWrapper* b) const {
+ return comparator_->Compare(a->key(), b->key()) < 0;
+ }
+
+ private:
+ const InternalKeyComparator* comparator_;
+};
+
+// When used with std::priority_queue, this comparison functor puts the
+// iterator with the min/smallest key on top.
+class MinIteratorComparator {
+ public:
+ MinIteratorComparator(const InternalKeyComparator* comparator)
+ : comparator_(comparator) {}
+
+ bool operator()(IteratorWrapper* a, IteratorWrapper* b) const {
+ return comparator_->Compare(a->key(), b->key()) > 0;
+ }
+
+ private:
+ const InternalKeyComparator* comparator_;
+};
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/iterator.cc b/src/rocksdb/table/iterator.cc
new file mode 100644
index 000000000..14e280a07
--- /dev/null
+++ b/src/rocksdb/table/iterator.cc
@@ -0,0 +1,130 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#include "rocksdb/iterator.h"
+
+#include "memory/arena.h"
+#include "table/internal_iterator.h"
+#include "table/iterator_wrapper.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+Status Iterator::GetProperty(std::string prop_name, std::string* prop) {
+ if (prop == nullptr) {
+ return Status::InvalidArgument("prop is nullptr");
+ }
+ if (prop_name == "rocksdb.iterator.is-key-pinned") {
+ *prop = "0";
+ return Status::OK();
+ }
+ return Status::InvalidArgument("Unidentified property.");
+}
+
+namespace {
+class EmptyIterator : public Iterator {
+ public:
+ explicit EmptyIterator(const Status& s) : status_(s) {}
+ bool Valid() const override { return false; }
+ void Seek(const Slice& /*target*/) override {}
+ void SeekForPrev(const Slice& /*target*/) override {}
+ void SeekToFirst() override {}
+ void SeekToLast() override {}
+ void Next() override { assert(false); }
+ void Prev() override { assert(false); }
+ Slice key() const override {
+ assert(false);
+ return Slice();
+ }
+ Slice value() const override {
+ assert(false);
+ return Slice();
+ }
+ Status status() const override { return status_; }
+
+ private:
+ Status status_;
+};
+
+template <class TValue = Slice>
+class EmptyInternalIterator : public InternalIteratorBase<TValue> {
+ public:
+ explicit EmptyInternalIterator(const Status& s) : status_(s) {}
+ bool Valid() const override { return false; }
+ void Seek(const Slice& /*target*/) override {}
+ void SeekForPrev(const Slice& /*target*/) override {}
+ void SeekToFirst() override {}
+ void SeekToLast() override {}
+ void Next() override { assert(false); }
+ void Prev() override { assert(false); }
+ Slice key() const override {
+ assert(false);
+ return Slice();
+ }
+ TValue value() const override {
+ assert(false);
+ return TValue();
+ }
+ Status status() const override { return status_; }
+
+ private:
+ Status status_;
+};
+} // namespace
+
+Iterator* NewEmptyIterator() { return new EmptyIterator(Status::OK()); }
+
+Iterator* NewErrorIterator(const Status& status) {
+ return new EmptyIterator(status);
+}
+
+template <class TValue>
+InternalIteratorBase<TValue>* NewErrorInternalIterator(const Status& status) {
+ return new EmptyInternalIterator<TValue>(status);
+}
+template InternalIteratorBase<IndexValue>* NewErrorInternalIterator(
+ const Status& status);
+template InternalIteratorBase<Slice>* NewErrorInternalIterator(
+ const Status& status);
+
+template <class TValue>
+InternalIteratorBase<TValue>* NewErrorInternalIterator(const Status& status,
+ Arena* arena) {
+ if (arena == nullptr) {
+ return NewErrorInternalIterator<TValue>(status);
+ } else {
+ auto mem = arena->AllocateAligned(sizeof(EmptyInternalIterator<TValue>));
+ return new (mem) EmptyInternalIterator<TValue>(status);
+ }
+}
+template InternalIteratorBase<IndexValue>* NewErrorInternalIterator(
+ const Status& status, Arena* arena);
+template InternalIteratorBase<Slice>* NewErrorInternalIterator(
+ const Status& status, Arena* arena);
+
+template <class TValue>
+InternalIteratorBase<TValue>* NewEmptyInternalIterator() {
+ return new EmptyInternalIterator<TValue>(Status::OK());
+}
+template InternalIteratorBase<IndexValue>* NewEmptyInternalIterator();
+template InternalIteratorBase<Slice>* NewEmptyInternalIterator();
+
+template <class TValue>
+InternalIteratorBase<TValue>* NewEmptyInternalIterator(Arena* arena) {
+ if (arena == nullptr) {
+ return NewEmptyInternalIterator<TValue>();
+ } else {
+ auto mem = arena->AllocateAligned(sizeof(EmptyInternalIterator<TValue>));
+ return new (mem) EmptyInternalIterator<TValue>(Status::OK());
+ }
+}
+template InternalIteratorBase<IndexValue>* NewEmptyInternalIterator(
+ Arena* arena);
+template InternalIteratorBase<Slice>* NewEmptyInternalIterator(Arena* arena);
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/iterator_wrapper.h b/src/rocksdb/table/iterator_wrapper.h
new file mode 100644
index 000000000..17abef4ac
--- /dev/null
+++ b/src/rocksdb/table/iterator_wrapper.h
@@ -0,0 +1,190 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#pragma once
+
+#include <set>
+
+#include "table/internal_iterator.h"
+#include "test_util/sync_point.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+// A internal wrapper class with an interface similar to Iterator that caches
+// the valid() and key() results for an underlying iterator.
+// This can help avoid virtual function calls and also gives better
+// cache locality.
+template <class TValue = Slice>
+class IteratorWrapperBase {
+ public:
+ IteratorWrapperBase() : iter_(nullptr), valid_(false) {}
+ explicit IteratorWrapperBase(InternalIteratorBase<TValue>* _iter)
+ : iter_(nullptr) {
+ Set(_iter);
+ }
+ ~IteratorWrapperBase() {}
+ InternalIteratorBase<TValue>* iter() const { return iter_; }
+
+ // Set the underlying Iterator to _iter and return
+ // previous underlying Iterator.
+ InternalIteratorBase<TValue>* Set(InternalIteratorBase<TValue>* _iter) {
+ InternalIteratorBase<TValue>* old_iter = iter_;
+
+ iter_ = _iter;
+ if (iter_ == nullptr) {
+ valid_ = false;
+ } else {
+ Update();
+ }
+ return old_iter;
+ }
+
+ void DeleteIter(bool is_arena_mode) {
+ if (iter_) {
+ if (!is_arena_mode) {
+ delete iter_;
+ } else {
+ iter_->~InternalIteratorBase<TValue>();
+ }
+ }
+ }
+
+ // Iterator interface methods
+ bool Valid() const { return valid_; }
+ Slice key() const {
+ assert(Valid());
+ return result_.key;
+ }
+ TValue value() const {
+ assert(Valid());
+ return iter_->value();
+ }
+ // Methods below require iter() != nullptr
+ Status status() const {
+ assert(iter_);
+ return iter_->status();
+ }
+ bool PrepareValue() {
+ assert(Valid());
+ if (result_.value_prepared) {
+ return true;
+ }
+ if (iter_->PrepareValue()) {
+ result_.value_prepared = true;
+ return true;
+ }
+
+ assert(!iter_->Valid());
+ valid_ = false;
+ return false;
+ }
+ void Next() {
+ assert(iter_);
+ valid_ = iter_->NextAndGetResult(&result_);
+ assert(!valid_ || iter_->status().ok());
+ }
+ bool NextAndGetResult(IterateResult* result) {
+ assert(iter_);
+ valid_ = iter_->NextAndGetResult(&result_);
+ *result = result_;
+ assert(!valid_ || iter_->status().ok());
+ return valid_;
+ }
+ void Prev() {
+ assert(iter_);
+ iter_->Prev();
+ Update();
+ }
+ void Seek(const Slice& k) {
+ assert(iter_);
+ iter_->Seek(k);
+ Update();
+ }
+ void SeekForPrev(const Slice& k) {
+ assert(iter_);
+ iter_->SeekForPrev(k);
+ Update();
+ }
+ void SeekToFirst() {
+ assert(iter_);
+ iter_->SeekToFirst();
+ Update();
+ }
+ void SeekToLast() {
+ assert(iter_);
+ iter_->SeekToLast();
+ Update();
+ }
+
+ bool MayBeOutOfLowerBound() {
+ assert(Valid());
+ return iter_->MayBeOutOfLowerBound();
+ }
+
+ IterBoundCheck UpperBoundCheckResult() {
+ assert(Valid());
+ return result_.bound_check_result;
+ }
+
+ void SetPinnedItersMgr(PinnedIteratorsManager* pinned_iters_mgr) {
+ assert(iter_);
+ iter_->SetPinnedItersMgr(pinned_iters_mgr);
+ }
+ bool IsKeyPinned() const {
+ assert(Valid());
+ return iter_->IsKeyPinned();
+ }
+ bool IsValuePinned() const {
+ assert(Valid());
+ return iter_->IsValuePinned();
+ }
+
+ bool IsValuePrepared() const { return result_.value_prepared; }
+
+ Slice user_key() const {
+ assert(Valid());
+ return iter_->user_key();
+ }
+
+ void UpdateReadaheadState(InternalIteratorBase<TValue>* old_iter) {
+ if (old_iter && iter_) {
+ ReadaheadFileInfo readahead_file_info;
+ old_iter->GetReadaheadState(&readahead_file_info);
+ iter_->SetReadaheadState(&readahead_file_info);
+ }
+ }
+
+ bool IsDeleteRangeSentinelKey() const {
+ return iter_->IsDeleteRangeSentinelKey();
+ }
+
+ private:
+ void Update() {
+ valid_ = iter_->Valid();
+ if (valid_) {
+ assert(iter_->status().ok());
+ result_.key = iter_->key();
+ result_.bound_check_result = IterBoundCheck::kUnknown;
+ result_.value_prepared = false;
+ }
+ }
+
+ InternalIteratorBase<TValue>* iter_;
+ IterateResult result_;
+ bool valid_;
+};
+
+using IteratorWrapper = IteratorWrapperBase<Slice>;
+
+class Arena;
+// Return an empty iterator (yields nothing) allocated from arena.
+template <class TValue = Slice>
+extern InternalIteratorBase<TValue>* NewEmptyInternalIterator(Arena* arena);
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/merger_test.cc b/src/rocksdb/table/merger_test.cc
new file mode 100644
index 000000000..71dc798e5
--- /dev/null
+++ b/src/rocksdb/table/merger_test.cc
@@ -0,0 +1,182 @@
+// Copyright (c) 2011-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 <string>
+#include <vector>
+
+#include "table/merging_iterator.h"
+#include "test_util/testharness.h"
+#include "test_util/testutil.h"
+#include "util/random.h"
+#include "util/vector_iterator.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class MergerTest : public testing::Test {
+ public:
+ MergerTest()
+ : icomp_(BytewiseComparator()),
+ rnd_(3),
+ merging_iterator_(nullptr),
+ single_iterator_(nullptr) {}
+ ~MergerTest() override = default;
+ std::vector<std::string> GenerateStrings(size_t len, int string_len) {
+ std::vector<std::string> ret;
+
+ for (size_t i = 0; i < len; ++i) {
+ InternalKey ik(rnd_.HumanReadableString(string_len), 0,
+ ValueType::kTypeValue);
+ ret.push_back(ik.Encode().ToString(false));
+ }
+ return ret;
+ }
+
+ void AssertEquivalence() {
+ auto a = merging_iterator_.get();
+ auto b = single_iterator_.get();
+ if (!a->Valid()) {
+ ASSERT_TRUE(!b->Valid());
+ } else {
+ ASSERT_TRUE(b->Valid());
+ ASSERT_EQ(b->key().ToString(), a->key().ToString());
+ ASSERT_EQ(b->value().ToString(), a->value().ToString());
+ }
+ }
+
+ void SeekToRandom() {
+ InternalKey ik(rnd_.HumanReadableString(5), 0, ValueType::kTypeValue);
+ Seek(ik.Encode().ToString(false));
+ }
+
+ void Seek(std::string target) {
+ merging_iterator_->Seek(target);
+ single_iterator_->Seek(target);
+ }
+
+ void SeekToFirst() {
+ merging_iterator_->SeekToFirst();
+ single_iterator_->SeekToFirst();
+ }
+
+ void SeekToLast() {
+ merging_iterator_->SeekToLast();
+ single_iterator_->SeekToLast();
+ }
+
+ void Next(int times) {
+ for (int i = 0; i < times && merging_iterator_->Valid(); ++i) {
+ AssertEquivalence();
+ merging_iterator_->Next();
+ single_iterator_->Next();
+ }
+ AssertEquivalence();
+ }
+
+ void Prev(int times) {
+ for (int i = 0; i < times && merging_iterator_->Valid(); ++i) {
+ AssertEquivalence();
+ merging_iterator_->Prev();
+ single_iterator_->Prev();
+ }
+ AssertEquivalence();
+ }
+
+ void NextAndPrev(int times) {
+ for (int i = 0; i < times && merging_iterator_->Valid(); ++i) {
+ AssertEquivalence();
+ if (rnd_.OneIn(2)) {
+ merging_iterator_->Prev();
+ single_iterator_->Prev();
+ } else {
+ merging_iterator_->Next();
+ single_iterator_->Next();
+ }
+ }
+ AssertEquivalence();
+ }
+
+ void Generate(size_t num_iterators, size_t strings_per_iterator,
+ int letters_per_string) {
+ std::vector<InternalIterator*> small_iterators;
+ for (size_t i = 0; i < num_iterators; ++i) {
+ auto strings = GenerateStrings(strings_per_iterator, letters_per_string);
+ small_iterators.push_back(new VectorIterator(strings, strings, &icomp_));
+ all_keys_.insert(all_keys_.end(), strings.begin(), strings.end());
+ }
+
+ merging_iterator_.reset(
+ NewMergingIterator(&icomp_, &small_iterators[0],
+ static_cast<int>(small_iterators.size())));
+ single_iterator_.reset(new VectorIterator(all_keys_, all_keys_, &icomp_));
+ }
+
+ InternalKeyComparator icomp_;
+ Random rnd_;
+ std::unique_ptr<InternalIterator> merging_iterator_;
+ std::unique_ptr<InternalIterator> single_iterator_;
+ std::vector<std::string> all_keys_;
+};
+
+TEST_F(MergerTest, SeekToRandomNextTest) {
+ Generate(1000, 50, 50);
+ for (int i = 0; i < 10; ++i) {
+ SeekToRandom();
+ AssertEquivalence();
+ Next(50000);
+ }
+}
+
+TEST_F(MergerTest, SeekToRandomNextSmallStringsTest) {
+ Generate(1000, 50, 2);
+ for (int i = 0; i < 10; ++i) {
+ SeekToRandom();
+ AssertEquivalence();
+ Next(50000);
+ }
+}
+
+TEST_F(MergerTest, SeekToRandomPrevTest) {
+ Generate(1000, 50, 50);
+ for (int i = 0; i < 10; ++i) {
+ SeekToRandom();
+ AssertEquivalence();
+ Prev(50000);
+ }
+}
+
+TEST_F(MergerTest, SeekToRandomRandomTest) {
+ Generate(200, 50, 50);
+ for (int i = 0; i < 3; ++i) {
+ SeekToRandom();
+ AssertEquivalence();
+ NextAndPrev(5000);
+ }
+}
+
+TEST_F(MergerTest, SeekToFirstTest) {
+ Generate(1000, 50, 50);
+ for (int i = 0; i < 10; ++i) {
+ SeekToFirst();
+ AssertEquivalence();
+ Next(50000);
+ }
+}
+
+TEST_F(MergerTest, SeekToLastTest) {
+ Generate(1000, 50, 50);
+ for (int i = 0; i < 10; ++i) {
+ SeekToLast();
+ AssertEquivalence();
+ Prev(50000);
+ }
+}
+
+} // namespace ROCKSDB_NAMESPACE
+
+int main(int argc, char** argv) {
+ ROCKSDB_NAMESPACE::port::InstallStackTraceHandler();
+ ::testing::InitGoogleTest(&argc, argv);
+ return RUN_ALL_TESTS();
+}
diff --git a/src/rocksdb/table/merging_iterator.cc b/src/rocksdb/table/merging_iterator.cc
new file mode 100644
index 000000000..beb35ea9a
--- /dev/null
+++ b/src/rocksdb/table/merging_iterator.cc
@@ -0,0 +1,1403 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#include "table/merging_iterator.h"
+
+#include "db/arena_wrapped_db_iter.h"
+#include "db/dbformat.h"
+#include "db/pinned_iterators_manager.h"
+#include "memory/arena.h"
+#include "monitoring/perf_context_imp.h"
+#include "rocksdb/comparator.h"
+#include "rocksdb/iterator.h"
+#include "rocksdb/options.h"
+#include "table/internal_iterator.h"
+#include "table/iter_heap.h"
+#include "table/iterator_wrapper.h"
+#include "test_util/sync_point.h"
+#include "util/autovector.h"
+#include "util/heap.h"
+#include "util/stop_watch.h"
+
+namespace ROCKSDB_NAMESPACE {
+// For merging iterator to process range tombstones, we treat the start and end
+// keys of a range tombstone as point keys and put them into the minHeap/maxHeap
+// used in merging iterator. Take minHeap for example, we are able to keep track
+// of currently "active" range tombstones (the ones whose start keys are popped
+// but end keys are still in the heap) in `active_`. This `active_` set of range
+// tombstones is then used to quickly determine whether the point key at heap
+// top is deleted (by heap property, the point key at heap top must be within
+// internal key range of active range tombstones).
+//
+// The HeapItem struct represents 3 types of elements in the minHeap/maxHeap:
+// point key and the start and end keys of a range tombstone.
+struct HeapItem {
+ HeapItem() = default;
+
+ enum Type { ITERATOR, DELETE_RANGE_START, DELETE_RANGE_END };
+ IteratorWrapper iter;
+ size_t level = 0;
+ std::string pinned_key;
+ // Will be overwritten before use, initialize here so compiler does not
+ // complain.
+ Type type = ITERATOR;
+
+ explicit HeapItem(size_t _level, InternalIteratorBase<Slice>* _iter)
+ : level(_level), type(Type::ITERATOR) {
+ iter.Set(_iter);
+ }
+
+ void SetTombstoneKey(ParsedInternalKey&& pik) {
+ pinned_key.clear();
+ // Range tombstone end key is exclusive. If a point internal key has the
+ // same user key and sequence number as the start or end key of a range
+ // tombstone, the order will be start < end key < internal key with the
+ // following op_type change. This is helpful to ensure keys popped from
+ // heap are in expected order since range tombstone start/end keys will
+ // be distinct from point internal keys. Strictly speaking, this is only
+ // needed for tombstone end points that are truncated in
+ // TruncatedRangeDelIterator since untruncated tombstone end points always
+ // have kMaxSequenceNumber and kTypeRangeDeletion (see
+ // TruncatedRangeDelIterator::start_key()/end_key()).
+ ParsedInternalKey p(pik.user_key, pik.sequence, kTypeMaxValid);
+ AppendInternalKey(&pinned_key, p);
+ }
+
+ Slice key() const {
+ if (type == Type::ITERATOR) {
+ return iter.key();
+ }
+ return pinned_key;
+ }
+
+ bool IsDeleteRangeSentinelKey() const {
+ if (type == Type::ITERATOR) {
+ return iter.IsDeleteRangeSentinelKey();
+ }
+ return false;
+ }
+};
+
+class MinHeapItemComparator {
+ public:
+ MinHeapItemComparator(const InternalKeyComparator* comparator)
+ : comparator_(comparator) {}
+ bool operator()(HeapItem* a, HeapItem* b) const {
+ return comparator_->Compare(a->key(), b->key()) > 0;
+ }
+
+ private:
+ const InternalKeyComparator* comparator_;
+};
+
+class MaxHeapItemComparator {
+ public:
+ MaxHeapItemComparator(const InternalKeyComparator* comparator)
+ : comparator_(comparator) {}
+ bool operator()(HeapItem* a, HeapItem* b) const {
+ return comparator_->Compare(a->key(), b->key()) < 0;
+ }
+
+ private:
+ const InternalKeyComparator* comparator_;
+};
+// Without anonymous namespace here, we fail the warning -Wmissing-prototypes
+namespace {
+using MergerMinIterHeap = BinaryHeap<HeapItem*, MinHeapItemComparator>;
+using MergerMaxIterHeap = BinaryHeap<HeapItem*, MaxHeapItemComparator>;
+} // namespace
+
+class MergingIterator : public InternalIterator {
+ public:
+ MergingIterator(const InternalKeyComparator* comparator,
+ InternalIterator** children, int n, bool is_arena_mode,
+ bool prefix_seek_mode,
+ const Slice* iterate_upper_bound = nullptr)
+ : is_arena_mode_(is_arena_mode),
+ prefix_seek_mode_(prefix_seek_mode),
+ direction_(kForward),
+ comparator_(comparator),
+ current_(nullptr),
+ minHeap_(comparator_),
+ pinned_iters_mgr_(nullptr),
+ iterate_upper_bound_(iterate_upper_bound) {
+ children_.resize(n);
+ for (int i = 0; i < n; i++) {
+ children_[i].level = i;
+ children_[i].iter.Set(children[i]);
+ }
+ }
+
+ void considerStatus(Status s) {
+ if (!s.ok() && status_.ok()) {
+ status_ = s;
+ }
+ }
+
+ virtual void AddIterator(InternalIterator* iter) {
+ children_.emplace_back(children_.size(), iter);
+ if (pinned_iters_mgr_) {
+ iter->SetPinnedItersMgr(pinned_iters_mgr_);
+ }
+ // Invalidate to ensure `Seek*()` is called to construct the heaps before
+ // use.
+ current_ = nullptr;
+ }
+
+ // Merging iterator can optionally process range tombstones: if a key is
+ // covered by a range tombstone, the merging iterator will not output it but
+ // skip it.
+ //
+ // Add the next range tombstone iterator to this merging iterator.
+ // There must be either no range tombstone iterator, or same number of
+ // range tombstone iterators as point iterators after all range tombstone
+ // iters are added. The i-th added range tombstone iterator and the i-th point
+ // iterator must point to the same sorted run.
+ // Merging iterator takes ownership of the range tombstone iterator and
+ // is responsible for freeing it. Note that during Iterator::Refresh()
+ // and when a level iterator moves to a different SST file, the range
+ // tombstone iterator could be updated. In that case, the merging iterator
+ // is only responsible to freeing the new range tombstone iterator
+ // that it has pointers to in range_tombstone_iters_.
+ void AddRangeTombstoneIterator(TruncatedRangeDelIterator* iter) {
+ range_tombstone_iters_.emplace_back(iter);
+ }
+
+ // Called by MergingIteratorBuilder when all point iterators and range
+ // tombstone iterators are added. Initializes HeapItems for range tombstone
+ // iterators so that no further allocation is needed for HeapItem.
+ void Finish() {
+ if (!range_tombstone_iters_.empty()) {
+ pinned_heap_item_.resize(range_tombstone_iters_.size());
+ for (size_t i = 0; i < range_tombstone_iters_.size(); ++i) {
+ pinned_heap_item_[i].level = i;
+ }
+ }
+ }
+
+ ~MergingIterator() override {
+ for (auto child : range_tombstone_iters_) {
+ delete child;
+ }
+
+ for (auto& child : children_) {
+ child.iter.DeleteIter(is_arena_mode_);
+ }
+ status_.PermitUncheckedError();
+ }
+
+ bool Valid() const override { return current_ != nullptr && status_.ok(); }
+
+ Status status() const override { return status_; }
+
+ // Add range_tombstone_iters_[level] into min heap.
+ // Updates active_ if the end key of a range tombstone is inserted.
+ // @param start_key specifies which end point of the range tombstone to add.
+ void InsertRangeTombstoneToMinHeap(size_t level, bool start_key = true,
+ bool replace_top = false) {
+ assert(!range_tombstone_iters_.empty() &&
+ range_tombstone_iters_[level]->Valid());
+ if (start_key) {
+ ParsedInternalKey pik = range_tombstone_iters_[level]->start_key();
+ // iterate_upper_bound does not have timestamp
+ if (iterate_upper_bound_ &&
+ comparator_->user_comparator()->CompareWithoutTimestamp(
+ pik.user_key, true /* a_has_ts */, *iterate_upper_bound_,
+ false /* b_has_ts */) >= 0) {
+ if (replace_top) {
+ // replace_top implies this range tombstone iterator is still in
+ // minHeap_ and at the top.
+ minHeap_.pop();
+ }
+ return;
+ }
+ pinned_heap_item_[level].SetTombstoneKey(std::move(pik));
+ pinned_heap_item_[level].type = HeapItem::DELETE_RANGE_START;
+ assert(active_.count(level) == 0);
+ } else {
+ // allow end key to go over upper bound (if present) since start key is
+ // before upper bound and the range tombstone could still cover a
+ // range before upper bound.
+ pinned_heap_item_[level].SetTombstoneKey(
+ range_tombstone_iters_[level]->end_key());
+ pinned_heap_item_[level].type = HeapItem::DELETE_RANGE_END;
+ active_.insert(level);
+ }
+ if (replace_top) {
+ minHeap_.replace_top(&pinned_heap_item_[level]);
+ } else {
+ minHeap_.push(&pinned_heap_item_[level]);
+ }
+ }
+
+ // Add range_tombstone_iters_[level] into max heap.
+ // Updates active_ if the start key of a range tombstone is inserted.
+ // @param end_key specifies which end point of the range tombstone to add.
+ void InsertRangeTombstoneToMaxHeap(size_t level, bool end_key = true,
+ bool replace_top = false) {
+ assert(!range_tombstone_iters_.empty() &&
+ range_tombstone_iters_[level]->Valid());
+ if (end_key) {
+ pinned_heap_item_[level].SetTombstoneKey(
+ range_tombstone_iters_[level]->end_key());
+ pinned_heap_item_[level].type = HeapItem::DELETE_RANGE_END;
+ assert(active_.count(level) == 0);
+ } else {
+ pinned_heap_item_[level].SetTombstoneKey(
+ range_tombstone_iters_[level]->start_key());
+ pinned_heap_item_[level].type = HeapItem::DELETE_RANGE_START;
+ active_.insert(level);
+ }
+ if (replace_top) {
+ maxHeap_->replace_top(&pinned_heap_item_[level]);
+ } else {
+ maxHeap_->push(&pinned_heap_item_[level]);
+ }
+ }
+
+ // Remove HeapItems from top of minHeap_ that are of type DELETE_RANGE_START
+ // until minHeap_ is empty or the top of the minHeap_ is not of type
+ // DELETE_RANGE_START. Each such item means a range tombstone becomes active,
+ // so `active_` is updated accordingly.
+ void PopDeleteRangeStart() {
+ while (!minHeap_.empty() &&
+ minHeap_.top()->type == HeapItem::DELETE_RANGE_START) {
+ TEST_SYNC_POINT_CALLBACK("MergeIterator::PopDeleteRangeStart", nullptr);
+ // insert end key of this range tombstone and updates active_
+ InsertRangeTombstoneToMinHeap(
+ minHeap_.top()->level, false /* start_key */, true /* replace_top */);
+ }
+ }
+
+ // Remove HeapItems from top of maxHeap_ that are of type DELETE_RANGE_END
+ // until maxHeap_ is empty or the top of the maxHeap_ is not of type
+ // DELETE_RANGE_END. Each such item means a range tombstone becomes active,
+ // so `active_` is updated accordingly.
+ void PopDeleteRangeEnd() {
+ while (!maxHeap_->empty() &&
+ maxHeap_->top()->type == HeapItem::DELETE_RANGE_END) {
+ // insert start key of this range tombstone and updates active_
+ InsertRangeTombstoneToMaxHeap(maxHeap_->top()->level, false /* end_key */,
+ true /* replace_top */);
+ }
+ }
+
+ void SeekToFirst() override {
+ ClearHeaps();
+ status_ = Status::OK();
+ for (auto& child : children_) {
+ child.iter.SeekToFirst();
+ AddToMinHeapOrCheckStatus(&child);
+ }
+
+ for (size_t i = 0; i < range_tombstone_iters_.size(); ++i) {
+ if (range_tombstone_iters_[i]) {
+ range_tombstone_iters_[i]->SeekToFirst();
+ if (range_tombstone_iters_[i]->Valid()) {
+ // It is possible to be invalid due to snapshots.
+ InsertRangeTombstoneToMinHeap(i);
+ }
+ }
+ }
+ FindNextVisibleKey();
+ direction_ = kForward;
+ current_ = CurrentForward();
+ }
+
+ void SeekToLast() override {
+ ClearHeaps();
+ InitMaxHeap();
+ status_ = Status::OK();
+ for (auto& child : children_) {
+ child.iter.SeekToLast();
+ AddToMaxHeapOrCheckStatus(&child);
+ }
+
+ for (size_t i = 0; i < range_tombstone_iters_.size(); ++i) {
+ if (range_tombstone_iters_[i]) {
+ range_tombstone_iters_[i]->SeekToLast();
+ if (range_tombstone_iters_[i]->Valid()) {
+ // It is possible to be invalid due to snapshots.
+ InsertRangeTombstoneToMaxHeap(i);
+ }
+ }
+ }
+ FindPrevVisibleKey();
+ direction_ = kReverse;
+ current_ = CurrentReverse();
+ }
+
+ // Position this merging iterator at the first key >= target (internal key).
+ // If range tombstones are present, keys covered by range tombstones are
+ // skipped, and this merging iter points to the first non-range-deleted key >=
+ // target after Seek(). If !Valid() and status().ok() then end of the iterator
+ // is reached.
+ //
+ // Internally, this involves positioning all child iterators at the first key
+ // >= target. If range tombstones are present, we apply a similar
+ // optimization, cascading seek, as in Pebble
+ // (https://github.com/cockroachdb/pebble). Specifically, if there is a range
+ // tombstone [start, end) that covers the target user key at level L, then
+ // this range tombstone must cover the range [target key, end) in all levels >
+ // L. So for all levels > L, we can pretend the target key is `end`. This
+ // optimization is applied at each level and hence the name "cascading seek".
+ // After a round of (cascading) seeks, the top of the heap is checked to see
+ // if it is covered by a range tombstone (see FindNextVisibleKey() for more
+ // detail), and advanced if so. The process is repeated until a
+ // non-range-deleted key is at the top of the heap, or heap becomes empty.
+ //
+ // As mentioned in comments above HeapItem, to make the checking of whether
+ // top of the heap is covered by some range tombstone efficient, we treat each
+ // range deletion [start, end) as two point keys and insert them into the same
+ // min/maxHeap_ where point iterators are. The set `active_` tracks the levels
+ // that have active range tombstones. If level L is in `active_`, and the
+ // point key at top of the heap is from level >= L, then the point key is
+ // within the internal key range of the range tombstone that
+ // range_tombstone_iters_[L] currently points to. For correctness reasoning,
+ // one invariant that Seek() (and every other public APIs Seek*(),
+ // Next/Prev()) guarantees is as follows. After Seek(), suppose `k` is the
+ // current key of level L's point iterator. Then for each range tombstone
+ // iterator at level <= L, it is at or before the first range tombstone with
+ // end key > `k`. This ensures that when level L's point iterator reaches top
+ // of the heap, `active_` is calculated correctly (it contains the covering
+ // range tombstone's level if there is one), since no range tombstone iterator
+ // was skipped beyond that point iterator's current key during Seek().
+ // Next()/Prev() maintains a stronger version of this invariant where all
+ // range tombstone iterators from level <= L are *at* the first range
+ // tombstone with end key > `k`.
+ void Seek(const Slice& target) override {
+ assert(range_tombstone_iters_.empty() ||
+ range_tombstone_iters_.size() == children_.size());
+ SeekImpl(target);
+ FindNextVisibleKey();
+
+ direction_ = kForward;
+ {
+ PERF_TIMER_GUARD(seek_min_heap_time);
+ current_ = CurrentForward();
+ }
+ }
+
+ void SeekForPrev(const Slice& target) override {
+ assert(range_tombstone_iters_.empty() ||
+ range_tombstone_iters_.size() == children_.size());
+ SeekForPrevImpl(target);
+ FindPrevVisibleKey();
+
+ direction_ = kReverse;
+ {
+ PERF_TIMER_GUARD(seek_max_heap_time);
+ current_ = CurrentReverse();
+ }
+ }
+
+ void Next() override {
+ assert(Valid());
+ // Ensure that all children are positioned after key().
+ // If we are moving in the forward direction, it is already
+ // true for all of the non-current children since current_ is
+ // the smallest child and key() == current_->key().
+ if (direction_ != kForward) {
+ // The loop advanced all non-current children to be > key() so current_
+ // should still be strictly the smallest key.
+ SwitchToForward();
+ }
+
+ // For the heap modifications below to be correct, current_ must be the
+ // current top of the heap.
+ assert(current_ == CurrentForward());
+ // as the current points to the current record. move the iterator forward.
+ current_->Next();
+ if (current_->Valid()) {
+ // current is still valid after the Next() call above. Call
+ // replace_top() to restore the heap property. When the same child
+ // iterator yields a sequence of keys, this is cheap.
+ assert(current_->status().ok());
+ minHeap_.replace_top(minHeap_.top());
+ } else {
+ // current stopped being valid, remove it from the heap.
+ considerStatus(current_->status());
+ minHeap_.pop();
+ }
+ FindNextVisibleKey();
+ current_ = CurrentForward();
+ }
+
+ bool NextAndGetResult(IterateResult* result) override {
+ Next();
+ bool is_valid = Valid();
+ if (is_valid) {
+ result->key = key();
+ result->bound_check_result = UpperBoundCheckResult();
+ result->value_prepared = current_->IsValuePrepared();
+ }
+ return is_valid;
+ }
+
+ void Prev() override {
+ assert(Valid());
+ // Ensure that all children are positioned before key().
+ // If we are moving in the reverse direction, it is already
+ // true for all of the non-current children since current_ is
+ // the largest child and key() == current_->key().
+ if (direction_ != kReverse) {
+ // Otherwise, retreat the non-current children. We retreat current_
+ // just after the if-block.
+ SwitchToBackward();
+ }
+
+ // For the heap modifications below to be correct, current_ must be the
+ // current top of the heap.
+ assert(current_ == CurrentReverse());
+ current_->Prev();
+ if (current_->Valid()) {
+ // current is still valid after the Prev() call above. Call
+ // replace_top() to restore the heap property. When the same child
+ // iterator yields a sequence of keys, this is cheap.
+ assert(current_->status().ok());
+ maxHeap_->replace_top(maxHeap_->top());
+ } else {
+ // current stopped being valid, remove it from the heap.
+ considerStatus(current_->status());
+ maxHeap_->pop();
+ }
+ FindPrevVisibleKey();
+ current_ = CurrentReverse();
+ }
+
+ Slice key() const override {
+ assert(Valid());
+ return current_->key();
+ }
+
+ Slice value() const override {
+ assert(Valid());
+ return current_->value();
+ }
+
+ bool PrepareValue() override {
+ assert(Valid());
+ if (current_->PrepareValue()) {
+ return true;
+ }
+
+ considerStatus(current_->status());
+ assert(!status_.ok());
+ return false;
+ }
+
+ // Here we simply relay MayBeOutOfLowerBound/MayBeOutOfUpperBound result
+ // from current child iterator. Potentially as long as one of child iterator
+ // report out of bound is not possible, we know current key is within bound.
+
+ bool MayBeOutOfLowerBound() override {
+ assert(Valid());
+ return current_->MayBeOutOfLowerBound();
+ }
+
+ IterBoundCheck UpperBoundCheckResult() override {
+ assert(Valid());
+ return current_->UpperBoundCheckResult();
+ }
+
+ void SetPinnedItersMgr(PinnedIteratorsManager* pinned_iters_mgr) override {
+ pinned_iters_mgr_ = pinned_iters_mgr;
+ for (auto& child : children_) {
+ child.iter.SetPinnedItersMgr(pinned_iters_mgr);
+ }
+ }
+
+ bool IsKeyPinned() const override {
+ assert(Valid());
+ return pinned_iters_mgr_ && pinned_iters_mgr_->PinningEnabled() &&
+ current_->IsKeyPinned();
+ }
+
+ bool IsValuePinned() const override {
+ assert(Valid());
+ return pinned_iters_mgr_ && pinned_iters_mgr_->PinningEnabled() &&
+ current_->IsValuePinned();
+ }
+
+ private:
+ friend class MergeIteratorBuilder;
+ // Clears heaps for both directions, used when changing direction or seeking
+ void ClearHeaps(bool clear_active = true);
+ // Ensures that maxHeap_ is initialized when starting to go in the reverse
+ // direction
+ void InitMaxHeap();
+
+ // Advance this merging iterator until the current key (top of min heap) is
+ // not covered by any range tombstone or that there is no more keys (heap is
+ // empty). After this call, if Valid(), current_ points to the next key that
+ // is not covered by any range tombstone.
+ void FindNextVisibleKey();
+ void FindPrevVisibleKey();
+
+ void SeekImpl(const Slice& target, size_t starting_level = 0,
+ bool range_tombstone_reseek = false);
+
+ // Seek to fist key <= target key (internal key) for
+ // children_[starting_level:].
+ void SeekForPrevImpl(const Slice& target, size_t starting_level = 0,
+ bool range_tombstone_reseek = false);
+
+ bool is_arena_mode_;
+ bool prefix_seek_mode_;
+ // Which direction is the iterator moving?
+ enum Direction : uint8_t { kForward, kReverse };
+ Direction direction_;
+ const InternalKeyComparator* comparator_;
+ // We could also use an autovector with a larger reserved size.
+ // HeapItem for all child point iterators.
+ std::vector<HeapItem> children_;
+ // HeapItem for range tombstone start and end keys. Each range tombstone
+ // iterator will have at most one side (start key or end key) in a heap
+ // at the same time, so this vector will be of size children_.size();
+ // pinned_heap_item_[i] corresponds to the start key and end key HeapItem
+ // for range_tombstone_iters_[i].
+ std::vector<HeapItem> pinned_heap_item_;
+ // range_tombstone_iters_[i] contains range tombstones in the sorted run that
+ // corresponds to children_[i]. range_tombstone_iters_.empty() means not
+ // handling range tombstones in merging iterator. range_tombstone_iters_[i] ==
+ // nullptr means the sorted run of children_[i] does not have range
+ // tombstones.
+ std::vector<TruncatedRangeDelIterator*> range_tombstone_iters_;
+
+ // Levels (indices into range_tombstone_iters_/children_ ) that currently have
+ // "active" range tombstones. See comments above Seek() for meaning of
+ // "active".
+ std::set<size_t> active_;
+
+ bool SkipNextDeleted();
+ bool SkipPrevDeleted();
+
+ // Cached pointer to child iterator with the current key, or nullptr if no
+ // child iterators are valid. This is the top of minHeap_ or maxHeap_
+ // depending on the direction.
+ IteratorWrapper* current_;
+ // If any of the children have non-ok status, this is one of them.
+ Status status_;
+ MergerMinIterHeap minHeap_;
+
+ // Max heap is used for reverse iteration, which is way less common than
+ // forward. Lazily initialize it to save memory.
+ std::unique_ptr<MergerMaxIterHeap> maxHeap_;
+ PinnedIteratorsManager* pinned_iters_mgr_;
+
+ // Used to bound range tombstones. For point keys, DBIter and SSTable iterator
+ // take care of boundary checking.
+ const Slice* iterate_upper_bound_;
+
+ // In forward direction, process a child that is not in the min heap.
+ // If valid, add to the min heap. Otherwise, check status.
+ void AddToMinHeapOrCheckStatus(HeapItem*);
+
+ // In backward direction, process a child that is not in the max heap.
+ // If valid, add to the min heap. Otherwise, check status.
+ void AddToMaxHeapOrCheckStatus(HeapItem*);
+
+ void SwitchToForward();
+
+ // Switch the direction from forward to backward without changing the
+ // position. Iterator should still be valid.
+ void SwitchToBackward();
+
+ IteratorWrapper* CurrentForward() const {
+ assert(direction_ == kForward);
+ assert(minHeap_.empty() || minHeap_.top()->type == HeapItem::ITERATOR);
+ return !minHeap_.empty() ? &minHeap_.top()->iter : nullptr;
+ }
+
+ IteratorWrapper* CurrentReverse() const {
+ assert(direction_ == kReverse);
+ assert(maxHeap_);
+ assert(maxHeap_->empty() || maxHeap_->top()->type == HeapItem::ITERATOR);
+ return !maxHeap_->empty() ? &maxHeap_->top()->iter : nullptr;
+ }
+};
+
+// Seek to fist key >= target key (internal key) for children_[starting_level:].
+// Cascading seek optimizations are applied if range tombstones are present (see
+// comment above Seek() for more).
+//
+// @param range_tombstone_reseek Whether target is some range tombstone
+// end, i.e., whether this SeekImpl() call is a part of a "cascading seek". This
+// is used only for recoding relevant perf_context.
+void MergingIterator::SeekImpl(const Slice& target, size_t starting_level,
+ bool range_tombstone_reseek) {
+ // active range tombstones before `starting_level` remain active
+ ClearHeaps(false /* clear_active */);
+ ParsedInternalKey pik;
+ if (!range_tombstone_iters_.empty()) {
+ // pik is only used in InsertRangeTombstoneToMinHeap().
+ ParseInternalKey(target, &pik, false).PermitUncheckedError();
+ }
+
+ // TODO: perhaps we could save some upheap cost by add all child iters first
+ // and then do a single heapify.
+ for (size_t level = 0; level < starting_level; ++level) {
+ PERF_TIMER_GUARD(seek_min_heap_time);
+ AddToMinHeapOrCheckStatus(&children_[level]);
+ }
+ if (!range_tombstone_iters_.empty()) {
+ // Add range tombstones from levels < starting_level. We can insert from
+ // pinned_heap_item_ for the following reasons:
+ // - pinned_heap_item_[level] is in minHeap_ iff
+ // range_tombstone_iters[level]->Valid().
+ // - If `level` is in active_, then range_tombstone_iters_[level]->Valid()
+ // and pinned_heap_item_[level] is of type RANGE_DELETION_END.
+ for (size_t level = 0; level < starting_level; ++level) {
+ if (range_tombstone_iters_[level] &&
+ range_tombstone_iters_[level]->Valid()) {
+ // use an iterator on active_ if performance becomes an issue here
+ if (active_.count(level) > 0) {
+ assert(pinned_heap_item_[level].type == HeapItem::DELETE_RANGE_END);
+ // if it was active, then start key must be within upper_bound,
+ // so we can add to minHeap_ directly.
+ minHeap_.push(&pinned_heap_item_[level]);
+ } else {
+ // this takes care of checking iterate_upper_bound, but with an extra
+ // key comparison if range_tombstone_iters_[level] was already out of
+ // bound. Consider using a new HeapItem type or some flag to remember
+ // boundary checking result.
+ InsertRangeTombstoneToMinHeap(level);
+ }
+ } else {
+ assert(!active_.count(level));
+ }
+ }
+ // levels >= starting_level will be reseeked below, so clearing their active
+ // state here.
+ active_.erase(active_.lower_bound(starting_level), active_.end());
+ }
+
+ status_ = Status::OK();
+ IterKey current_search_key;
+ current_search_key.SetInternalKey(target, false /* copy */);
+ // Seek target might change to some range tombstone end key, so
+ // we need to remember them for async requests.
+ // (level, target) pairs
+ autovector<std::pair<size_t, std::string>> prefetched_target;
+ for (auto level = starting_level; level < children_.size(); ++level) {
+ {
+ PERF_TIMER_GUARD(seek_child_seek_time);
+ children_[level].iter.Seek(current_search_key.GetInternalKey());
+ }
+
+ PERF_COUNTER_ADD(seek_child_seek_count, 1);
+
+ if (!range_tombstone_iters_.empty()) {
+ if (range_tombstone_reseek) {
+ // This seek is to some range tombstone end key.
+ // Should only happen when there are range tombstones.
+ PERF_COUNTER_ADD(internal_range_del_reseek_count, 1);
+ }
+ if (children_[level].iter.status().IsTryAgain()) {
+ prefetched_target.emplace_back(
+ level, current_search_key.GetInternalKey().ToString());
+ }
+ auto range_tombstone_iter = range_tombstone_iters_[level];
+ if (range_tombstone_iter) {
+ range_tombstone_iter->Seek(current_search_key.GetUserKey());
+ if (range_tombstone_iter->Valid()) {
+ // insert the range tombstone end that is closer to and >=
+ // current_search_key. Strictly speaking, since the Seek() call above
+ // is on user key, it is possible that range_tombstone_iter->end_key()
+ // < current_search_key. This can happen when range_tombstone_iter is
+ // truncated and range_tombstone_iter.largest_ has the same user key
+ // as current_search_key.GetUserKey() but with a larger sequence
+ // number than current_search_key. Correctness is not affected as this
+ // tombstone end key will be popped during FindNextVisibleKey().
+ InsertRangeTombstoneToMinHeap(
+ level, comparator_->Compare(range_tombstone_iter->start_key(),
+ pik) > 0 /* start_key */);
+ // current_search_key < end_key guaranteed by the Seek() and Valid()
+ // calls above. Only interested in user key coverage since older
+ // sorted runs must have smaller sequence numbers than this range
+ // tombstone.
+ //
+ // TODO: range_tombstone_iter->Seek() finds the max covering
+ // sequence number, can make it cheaper by not looking for max.
+ if (comparator_->user_comparator()->Compare(
+ range_tombstone_iter->start_key().user_key,
+ current_search_key.GetUserKey()) <= 0) {
+ // Since range_tombstone_iter->Valid(), seqno should be valid, so
+ // there is no need to check it.
+ range_tombstone_reseek = true;
+ // Current target user key is covered by this range tombstone.
+ // All older sorted runs will seek to range tombstone end key.
+ // Note that for prefix seek case, it is possible that the prefix
+ // is not the same as the original target, it should not affect
+ // correctness. Besides, in most cases, range tombstone start and
+ // end key should have the same prefix?
+ // If range_tombstone_iter->end_key() is truncated to its largest_
+ // boundary, the timestamp in user_key will not be max timestamp,
+ // but the timestamp of `range_tombstone_iter.largest_`. This should
+ // be fine here as current_search_key is used to Seek into lower
+ // levels.
+ current_search_key.SetInternalKey(
+ range_tombstone_iter->end_key().user_key, kMaxSequenceNumber);
+ }
+ }
+ }
+ }
+ // child.iter.status() is set to Status::TryAgain indicating asynchronous
+ // request for retrieval of data blocks has been submitted. So it should
+ // return at this point and Seek should be called again to retrieve the
+ // requested block and add the child to min heap.
+ if (children_[level].iter.status().IsTryAgain()) {
+ continue;
+ }
+ {
+ // Strictly, we timed slightly more than min heap operation,
+ // but these operations are very cheap.
+ PERF_TIMER_GUARD(seek_min_heap_time);
+ AddToMinHeapOrCheckStatus(&children_[level]);
+ }
+ }
+
+ if (range_tombstone_iters_.empty()) {
+ for (auto& child : children_) {
+ if (child.iter.status().IsTryAgain()) {
+ child.iter.Seek(target);
+ {
+ PERF_TIMER_GUARD(seek_min_heap_time);
+ AddToMinHeapOrCheckStatus(&child);
+ }
+ PERF_COUNTER_ADD(number_async_seek, 1);
+ }
+ }
+ } else {
+ for (auto& prefetch : prefetched_target) {
+ // (level, target) pairs
+ children_[prefetch.first].iter.Seek(prefetch.second);
+ {
+ PERF_TIMER_GUARD(seek_min_heap_time);
+ AddToMinHeapOrCheckStatus(&children_[prefetch.first]);
+ }
+ PERF_COUNTER_ADD(number_async_seek, 1);
+ }
+ }
+}
+
+// Returns true iff the current key (min heap top) should not be returned
+// to user (of the merging iterator). This can be because the current key
+// is deleted by some range tombstone, the current key is some fake file
+// boundary sentinel key, or the current key is an end point of a range
+// tombstone. Advance the iterator at heap top if needed. Heap order is restored
+// and `active_` is updated accordingly.
+// See FindNextVisibleKey() for more detail on internal implementation
+// of advancing child iters.
+//
+// REQUIRES:
+// - min heap is currently not empty, and iter is in kForward direction.
+// - minHeap_ top is not DELETE_RANGE_START (so that `active_` is current).
+bool MergingIterator::SkipNextDeleted() {
+ // 3 types of keys:
+ // - point key
+ // - file boundary sentinel keys
+ // - range deletion end key
+ auto current = minHeap_.top();
+ if (current->type == HeapItem::DELETE_RANGE_END) {
+ active_.erase(current->level);
+ assert(range_tombstone_iters_[current->level] &&
+ range_tombstone_iters_[current->level]->Valid());
+ range_tombstone_iters_[current->level]->Next();
+ if (range_tombstone_iters_[current->level]->Valid()) {
+ InsertRangeTombstoneToMinHeap(current->level, true /* start_key */,
+ true /* replace_top */);
+ } else {
+ minHeap_.pop();
+ }
+ return true /* current key deleted */;
+ }
+ if (current->iter.IsDeleteRangeSentinelKey()) {
+ // If the file boundary is defined by a range deletion, the range
+ // tombstone's end key must come before this sentinel key (see op_type in
+ // SetTombstoneKey()).
+ assert(ExtractValueType(current->iter.key()) != kTypeRangeDeletion ||
+ active_.count(current->level) == 0);
+ // LevelIterator enters a new SST file
+ current->iter.Next();
+ if (current->iter.Valid()) {
+ assert(current->iter.status().ok());
+ minHeap_.replace_top(current);
+ } else {
+ minHeap_.pop();
+ }
+ // Remove last SST file's range tombstone end key if there is one.
+ // This means file boundary is before range tombstone end key,
+ // which could happen when a range tombstone and a user key
+ // straddle two SST files. Note that in TruncatedRangeDelIterator
+ // constructor, parsed_largest.sequence is decremented 1 in this case.
+ if (!minHeap_.empty() && minHeap_.top()->level == current->level &&
+ minHeap_.top()->type == HeapItem::DELETE_RANGE_END) {
+ minHeap_.pop();
+ active_.erase(current->level);
+ }
+ if (range_tombstone_iters_[current->level] &&
+ range_tombstone_iters_[current->level]->Valid()) {
+ InsertRangeTombstoneToMinHeap(current->level);
+ }
+ return true /* current key deleted */;
+ }
+ assert(current->type == HeapItem::ITERATOR);
+ // Point key case: check active_ for range tombstone coverage.
+ ParsedInternalKey pik;
+ ParseInternalKey(current->iter.key(), &pik, false).PermitUncheckedError();
+ if (!active_.empty()) {
+ auto i = *active_.begin();
+ if (i < current->level) {
+ // range tombstone is from a newer level, definitely covers
+ assert(comparator_->Compare(range_tombstone_iters_[i]->start_key(),
+ pik) <= 0);
+ assert(comparator_->Compare(pik, range_tombstone_iters_[i]->end_key()) <
+ 0);
+ std::string target;
+ AppendInternalKey(&target, range_tombstone_iters_[i]->end_key());
+ SeekImpl(target, current->level, true);
+ return true /* current key deleted */;
+ } else if (i == current->level) {
+ // range tombstone is from the same level as current, check sequence
+ // number. By `active_` we know current key is between start key and end
+ // key.
+ assert(comparator_->Compare(range_tombstone_iters_[i]->start_key(),
+ pik) <= 0);
+ assert(comparator_->Compare(pik, range_tombstone_iters_[i]->end_key()) <
+ 0);
+ if (pik.sequence < range_tombstone_iters_[current->level]->seq()) {
+ // covered by range tombstone
+ current->iter.Next();
+ if (current->iter.Valid()) {
+ minHeap_.replace_top(current);
+ } else {
+ minHeap_.pop();
+ }
+ return true /* current key deleted */;
+ } else {
+ return false /* current key not deleted */;
+ }
+ } else {
+ return false /* current key not deleted */;
+ // range tombstone from an older sorted run with current key < end key.
+ // current key is not deleted and the older sorted run will have its range
+ // tombstone updated when the range tombstone's end key are popped from
+ // minHeap_.
+ }
+ }
+ // we can reach here only if active_ is empty
+ assert(active_.empty());
+ assert(minHeap_.top()->type == HeapItem::ITERATOR);
+ return false /* current key not deleted */;
+}
+
+void MergingIterator::SeekForPrevImpl(const Slice& target,
+ size_t starting_level,
+ bool range_tombstone_reseek) {
+ // active range tombstones before `starting_level` remain active
+ ClearHeaps(false /* clear_active */);
+ InitMaxHeap();
+ ParsedInternalKey pik;
+ if (!range_tombstone_iters_.empty()) {
+ ParseInternalKey(target, &pik, false).PermitUncheckedError();
+ }
+ for (size_t level = 0; level < starting_level; ++level) {
+ PERF_TIMER_GUARD(seek_max_heap_time);
+ AddToMaxHeapOrCheckStatus(&children_[level]);
+ }
+ if (!range_tombstone_iters_.empty()) {
+ // Add range tombstones before starting_level.
+ for (size_t level = 0; level < starting_level; ++level) {
+ if (range_tombstone_iters_[level] &&
+ range_tombstone_iters_[level]->Valid()) {
+ assert(static_cast<bool>(active_.count(level)) ==
+ (pinned_heap_item_[level].type == HeapItem::DELETE_RANGE_START));
+ maxHeap_->push(&pinned_heap_item_[level]);
+ } else {
+ assert(!active_.count(level));
+ }
+ }
+ // levels >= starting_level will be reseeked below,
+ active_.erase(active_.lower_bound(starting_level), active_.end());
+ }
+
+ status_ = Status::OK();
+ IterKey current_search_key;
+ current_search_key.SetInternalKey(target, false /* copy */);
+ // Seek target might change to some range tombstone end key, so
+ // we need to remember them for async requests.
+ // (level, target) pairs
+ autovector<std::pair<size_t, std::string>> prefetched_target;
+ for (auto level = starting_level; level < children_.size(); ++level) {
+ {
+ PERF_TIMER_GUARD(seek_child_seek_time);
+ children_[level].iter.SeekForPrev(current_search_key.GetInternalKey());
+ }
+
+ PERF_COUNTER_ADD(seek_child_seek_count, 1);
+
+ if (!range_tombstone_iters_.empty()) {
+ if (range_tombstone_reseek) {
+ // This seek is to some range tombstone end key.
+ // Should only happen when there are range tombstones.
+ PERF_COUNTER_ADD(internal_range_del_reseek_count, 1);
+ }
+ if (children_[level].iter.status().IsTryAgain()) {
+ prefetched_target.emplace_back(
+ level, current_search_key.GetInternalKey().ToString());
+ }
+ auto range_tombstone_iter = range_tombstone_iters_[level];
+ if (range_tombstone_iter) {
+ range_tombstone_iter->SeekForPrev(current_search_key.GetUserKey());
+ if (range_tombstone_iter->Valid()) {
+ InsertRangeTombstoneToMaxHeap(
+ level, comparator_->Compare(range_tombstone_iter->end_key(),
+ pik) <= 0 /* end_key */);
+ // start key <= current_search_key guaranteed by the Seek() call above
+ // Only interested in user key coverage since older sorted runs must
+ // have smaller sequence numbers than this tombstone.
+ if (comparator_->user_comparator()->Compare(
+ current_search_key.GetUserKey(),
+ range_tombstone_iter->end_key().user_key) < 0) {
+ range_tombstone_reseek = true;
+ current_search_key.SetInternalKey(
+ range_tombstone_iter->start_key().user_key, kMaxSequenceNumber,
+ kValueTypeForSeekForPrev);
+ }
+ }
+ }
+ }
+ // child.iter.status() is set to Status::TryAgain indicating asynchronous
+ // request for retrieval of data blocks has been submitted. So it should
+ // return at this point and Seek should be called again to retrieve the
+ // requested block and add the child to min heap.
+ if (children_[level].iter.status().IsTryAgain()) {
+ continue;
+ }
+ {
+ // Strictly, we timed slightly more than min heap operation,
+ // but these operations are very cheap.
+ PERF_TIMER_GUARD(seek_max_heap_time);
+ AddToMaxHeapOrCheckStatus(&children_[level]);
+ }
+ }
+
+ if (range_tombstone_iters_.empty()) {
+ for (auto& child : children_) {
+ if (child.iter.status().IsTryAgain()) {
+ child.iter.SeekForPrev(target);
+ {
+ PERF_TIMER_GUARD(seek_min_heap_time);
+ AddToMaxHeapOrCheckStatus(&child);
+ }
+ PERF_COUNTER_ADD(number_async_seek, 1);
+ }
+ }
+ } else {
+ for (auto& prefetch : prefetched_target) {
+ // (level, target) pairs
+ children_[prefetch.first].iter.SeekForPrev(prefetch.second);
+ {
+ PERF_TIMER_GUARD(seek_max_heap_time);
+ AddToMaxHeapOrCheckStatus(&children_[prefetch.first]);
+ }
+ PERF_COUNTER_ADD(number_async_seek, 1);
+ }
+ }
+}
+
+// See more in comments above SkipNextDeleted().
+// REQUIRES:
+// - max heap is currently not empty, and iter is in kReverse direction.
+// - maxHeap_ top is not DELETE_RANGE_END (so that `active_` is current).
+bool MergingIterator::SkipPrevDeleted() {
+ // 3 types of keys:
+ // - point key
+ // - file boundary sentinel keys
+ // - range deletion start key
+ auto current = maxHeap_->top();
+ if (current->type == HeapItem::DELETE_RANGE_START) {
+ active_.erase(current->level);
+ assert(range_tombstone_iters_[current->level] &&
+ range_tombstone_iters_[current->level]->Valid());
+ range_tombstone_iters_[current->level]->Prev();
+ if (range_tombstone_iters_[current->level]->Valid()) {
+ InsertRangeTombstoneToMaxHeap(current->level, true /* end_key */,
+ true /* replace_top */);
+ } else {
+ maxHeap_->pop();
+ }
+ return true /* current key deleted */;
+ }
+ if (current->iter.IsDeleteRangeSentinelKey()) {
+ // LevelIterator enters a new SST file
+ current->iter.Prev();
+ if (current->iter.Valid()) {
+ assert(current->iter.status().ok());
+ maxHeap_->replace_top(current);
+ } else {
+ maxHeap_->pop();
+ }
+ if (!maxHeap_->empty() && maxHeap_->top()->level == current->level &&
+ maxHeap_->top()->type == HeapItem::DELETE_RANGE_START) {
+ maxHeap_->pop();
+ active_.erase(current->level);
+ }
+ if (range_tombstone_iters_[current->level] &&
+ range_tombstone_iters_[current->level]->Valid()) {
+ InsertRangeTombstoneToMaxHeap(current->level);
+ }
+ return true /* current key deleted */;
+ }
+ assert(current->type == HeapItem::ITERATOR);
+ // Point key case: check active_ for range tombstone coverage.
+ ParsedInternalKey pik;
+ ParseInternalKey(current->iter.key(), &pik, false).PermitUncheckedError();
+ if (!active_.empty()) {
+ auto i = *active_.begin();
+ if (i < current->level) {
+ // range tombstone is from a newer level, definitely covers
+ assert(comparator_->Compare(range_tombstone_iters_[i]->start_key(),
+ pik) <= 0);
+ assert(comparator_->Compare(pik, range_tombstone_iters_[i]->end_key()) <
+ 0);
+ std::string target;
+ AppendInternalKey(&target, range_tombstone_iters_[i]->start_key());
+ // This is different from SkipNextDeleted() which does reseek at sorted
+ // runs >= level (instead of i+1 here). With min heap, if level L is at
+ // top of the heap, then levels <L all have internal keys > level L's
+ // current internal key, which means levels <L are already at a different
+ // user key. With max heap, if level L is at top of the heap, then levels
+ // <L all have internal keys smaller than level L's current internal key,
+ // which might still be the same user key.
+ SeekForPrevImpl(target, i + 1, true);
+ return true /* current key deleted */;
+ } else if (i == current->level) {
+ // By `active_` we know current key is between start key and end key.
+ assert(comparator_->Compare(range_tombstone_iters_[i]->start_key(),
+ pik) <= 0);
+ assert(comparator_->Compare(pik, range_tombstone_iters_[i]->end_key()) <
+ 0);
+ if (pik.sequence < range_tombstone_iters_[current->level]->seq()) {
+ current->iter.Prev();
+ if (current->iter.Valid()) {
+ maxHeap_->replace_top(current);
+ } else {
+ maxHeap_->pop();
+ }
+ return true /* current key deleted */;
+ } else {
+ return false /* current key not deleted */;
+ }
+ } else {
+ return false /* current key not deleted */;
+ }
+ }
+
+ assert(active_.empty());
+ assert(maxHeap_->top()->type == HeapItem::ITERATOR);
+ return false /* current key not deleted */;
+}
+
+void MergingIterator::AddToMinHeapOrCheckStatus(HeapItem* child) {
+ if (child->iter.Valid()) {
+ assert(child->iter.status().ok());
+ minHeap_.push(child);
+ } else {
+ considerStatus(child->iter.status());
+ }
+}
+
+void MergingIterator::AddToMaxHeapOrCheckStatus(HeapItem* child) {
+ if (child->iter.Valid()) {
+ assert(child->iter.status().ok());
+ maxHeap_->push(child);
+ } else {
+ considerStatus(child->iter.status());
+ }
+}
+
+// Advance all non current_ child to > current_.key().
+// We advance current_ after the this function call as it does not require
+// Seek().
+// Advance all range tombstones iters, including the one corresponding to
+// current_, to the first tombstone with end_key > current_.key().
+// TODO: potentially do cascading seek here too
+void MergingIterator::SwitchToForward() {
+ ClearHeaps();
+ Slice target = key();
+ for (auto& child : children_) {
+ if (&child.iter != current_) {
+ child.iter.Seek(target);
+ // child.iter.status() is set to Status::TryAgain indicating asynchronous
+ // request for retrieval of data blocks has been submitted. So it should
+ // return at this point and Seek should be called again to retrieve the
+ // requested block and add the child to min heap.
+ if (child.iter.status() == Status::TryAgain()) {
+ continue;
+ }
+ if (child.iter.Valid() && comparator_->Equal(target, child.key())) {
+ assert(child.iter.status().ok());
+ child.iter.Next();
+ }
+ }
+ AddToMinHeapOrCheckStatus(&child);
+ }
+
+ for (auto& child : children_) {
+ if (child.iter.status() == Status::TryAgain()) {
+ child.iter.Seek(target);
+ if (child.iter.Valid() && comparator_->Equal(target, child.key())) {
+ assert(child.iter.status().ok());
+ child.iter.Next();
+ }
+ AddToMinHeapOrCheckStatus(&child);
+ }
+ }
+
+ // Current range tombstone iter also needs to seek for the following case:
+ // Previous direction is backward, so range tombstone iter may point to a
+ // tombstone before current_. If there is no such tombstone, then the range
+ // tombstone iter is !Valid(). Need to reseek here to make it valid again.
+ if (!range_tombstone_iters_.empty()) {
+ ParsedInternalKey pik;
+ ParseInternalKey(target, &pik, false /* log_err_key */)
+ .PermitUncheckedError();
+ for (size_t i = 0; i < range_tombstone_iters_.size(); ++i) {
+ auto iter = range_tombstone_iters_[i];
+ if (iter) {
+ iter->Seek(pik.user_key);
+ // The while loop is needed as the Seek() call above is only for user
+ // key. We could have a range tombstone with end_key covering user_key,
+ // but still is smaller than target. This happens when the range
+ // tombstone is truncated at iter.largest_.
+ while (iter->Valid() &&
+ comparator_->Compare(iter->end_key(), pik) <= 0) {
+ iter->Next();
+ }
+ if (range_tombstone_iters_[i]->Valid()) {
+ InsertRangeTombstoneToMinHeap(
+ i, comparator_->Compare(range_tombstone_iters_[i]->start_key(),
+ pik) > 0 /* start_key */);
+ }
+ }
+ }
+ }
+
+ direction_ = kForward;
+ assert(current_ == CurrentForward());
+}
+
+// Advance all range tombstones iters, including the one corresponding to
+// current_, to the first tombstone with start_key <= current_.key().
+void MergingIterator::SwitchToBackward() {
+ ClearHeaps();
+ InitMaxHeap();
+ Slice target = key();
+ for (auto& child : children_) {
+ if (&child.iter != current_) {
+ child.iter.SeekForPrev(target);
+ TEST_SYNC_POINT_CALLBACK("MergeIterator::Prev:BeforePrev", &child);
+ if (child.iter.Valid() && comparator_->Equal(target, child.key())) {
+ assert(child.iter.status().ok());
+ child.iter.Prev();
+ }
+ }
+ AddToMaxHeapOrCheckStatus(&child);
+ }
+
+ ParsedInternalKey pik;
+ ParseInternalKey(target, &pik, false /* log_err_key */)
+ .PermitUncheckedError();
+ for (size_t i = 0; i < range_tombstone_iters_.size(); ++i) {
+ auto iter = range_tombstone_iters_[i];
+ if (iter) {
+ iter->SeekForPrev(pik.user_key);
+ // Since the SeekForPrev() call above is only for user key,
+ // we may end up with some range tombstone with start key having the
+ // same user key at current_, but with a smaller sequence number. This
+ // makes current_ not at maxHeap_ top for the CurrentReverse() call
+ // below. If there is a range tombstone start key with the same user
+ // key and the same sequence number as current_.key(), it will be fine as
+ // in InsertRangeTombstoneToMaxHeap() we change op_type to be the smallest
+ // op_type.
+ while (iter->Valid() &&
+ comparator_->Compare(iter->start_key(), pik) > 0) {
+ iter->Prev();
+ }
+ if (iter->Valid()) {
+ InsertRangeTombstoneToMaxHeap(
+ i, comparator_->Compare(range_tombstone_iters_[i]->end_key(),
+ pik) <= 0 /* end_key */);
+ }
+ }
+ }
+
+ direction_ = kReverse;
+ if (!prefix_seek_mode_) {
+ // Note that we don't do assert(current_ == CurrentReverse()) here
+ // because it is possible to have some keys larger than the seek-key
+ // inserted between Seek() and SeekToLast(), which makes current_ not
+ // equal to CurrentReverse().
+ current_ = CurrentReverse();
+ }
+ assert(current_ == CurrentReverse());
+}
+
+void MergingIterator::ClearHeaps(bool clear_active) {
+ minHeap_.clear();
+ if (maxHeap_) {
+ maxHeap_->clear();
+ }
+ if (clear_active) {
+ active_.clear();
+ }
+}
+
+void MergingIterator::InitMaxHeap() {
+ if (!maxHeap_) {
+ maxHeap_ = std::make_unique<MergerMaxIterHeap>(comparator_);
+ }
+}
+
+// Repeatedly check and remove heap top key if it is not a point key
+// that is not covered by range tombstones. SeekImpl() is called to seek to end
+// of a range tombstone if the heap top is a point key covered by some range
+// tombstone from a newer sorted run. If the covering tombstone is from current
+// key's level, then the current child iterator is simply advanced to its next
+// key without reseeking.
+inline void MergingIterator::FindNextVisibleKey() {
+ // When active_ is empty, we know heap top cannot be a range tombstone end
+ // key. It cannot be a range tombstone start key per PopDeleteRangeStart().
+ PopDeleteRangeStart();
+ while (!minHeap_.empty() &&
+ (!active_.empty() || minHeap_.top()->IsDeleteRangeSentinelKey()) &&
+ SkipNextDeleted()) {
+ PopDeleteRangeStart();
+ }
+}
+
+inline void MergingIterator::FindPrevVisibleKey() {
+ PopDeleteRangeEnd();
+ while (!maxHeap_->empty() &&
+ (!active_.empty() || maxHeap_->top()->IsDeleteRangeSentinelKey()) &&
+ SkipPrevDeleted()) {
+ PopDeleteRangeEnd();
+ }
+}
+
+InternalIterator* NewMergingIterator(const InternalKeyComparator* cmp,
+ InternalIterator** list, int n,
+ Arena* arena, bool prefix_seek_mode) {
+ assert(n >= 0);
+ if (n == 0) {
+ return NewEmptyInternalIterator<Slice>(arena);
+ } else if (n == 1) {
+ return list[0];
+ } else {
+ if (arena == nullptr) {
+ return new MergingIterator(cmp, list, n, false, prefix_seek_mode);
+ } else {
+ auto mem = arena->AllocateAligned(sizeof(MergingIterator));
+ return new (mem) MergingIterator(cmp, list, n, true, prefix_seek_mode);
+ }
+ }
+}
+
+MergeIteratorBuilder::MergeIteratorBuilder(
+ const InternalKeyComparator* comparator, Arena* a, bool prefix_seek_mode,
+ const Slice* iterate_upper_bound)
+ : first_iter(nullptr), use_merging_iter(false), arena(a) {
+ auto mem = arena->AllocateAligned(sizeof(MergingIterator));
+ merge_iter = new (mem) MergingIterator(comparator, nullptr, 0, true,
+ prefix_seek_mode, iterate_upper_bound);
+}
+
+MergeIteratorBuilder::~MergeIteratorBuilder() {
+ if (first_iter != nullptr) {
+ first_iter->~InternalIterator();
+ }
+ if (merge_iter != nullptr) {
+ merge_iter->~MergingIterator();
+ }
+}
+
+void MergeIteratorBuilder::AddIterator(InternalIterator* iter) {
+ if (!use_merging_iter && first_iter != nullptr) {
+ merge_iter->AddIterator(first_iter);
+ use_merging_iter = true;
+ first_iter = nullptr;
+ }
+ if (use_merging_iter) {
+ merge_iter->AddIterator(iter);
+ } else {
+ first_iter = iter;
+ }
+}
+
+void MergeIteratorBuilder::AddPointAndTombstoneIterator(
+ InternalIterator* point_iter, TruncatedRangeDelIterator* tombstone_iter,
+ TruncatedRangeDelIterator*** tombstone_iter_ptr) {
+ // tombstone_iter_ptr != nullptr means point_iter is a LevelIterator.
+ bool add_range_tombstone = tombstone_iter ||
+ !merge_iter->range_tombstone_iters_.empty() ||
+ tombstone_iter_ptr;
+ if (!use_merging_iter && (add_range_tombstone || first_iter)) {
+ use_merging_iter = true;
+ if (first_iter) {
+ merge_iter->AddIterator(first_iter);
+ first_iter = nullptr;
+ }
+ }
+ if (use_merging_iter) {
+ merge_iter->AddIterator(point_iter);
+ if (add_range_tombstone) {
+ // If there was a gap, fill in nullptr as empty range tombstone iterators.
+ while (merge_iter->range_tombstone_iters_.size() <
+ merge_iter->children_.size() - 1) {
+ merge_iter->AddRangeTombstoneIterator(nullptr);
+ }
+ merge_iter->AddRangeTombstoneIterator(tombstone_iter);
+ }
+
+ if (tombstone_iter_ptr) {
+ // This is needed instead of setting to &range_tombstone_iters_[i]
+ // directly here since the memory address of range_tombstone_iters_[i]
+ // might change during vector resizing.
+ range_del_iter_ptrs_.emplace_back(
+ merge_iter->range_tombstone_iters_.size() - 1, tombstone_iter_ptr);
+ }
+ } else {
+ first_iter = point_iter;
+ }
+}
+
+InternalIterator* MergeIteratorBuilder::Finish(ArenaWrappedDBIter* db_iter) {
+ InternalIterator* ret = nullptr;
+ if (!use_merging_iter) {
+ ret = first_iter;
+ first_iter = nullptr;
+ } else {
+ for (auto& p : range_del_iter_ptrs_) {
+ *(p.second) = &(merge_iter->range_tombstone_iters_[p.first]);
+ }
+ if (db_iter && !merge_iter->range_tombstone_iters_.empty()) {
+ // memtable is always the first level
+ db_iter->SetMemtableRangetombstoneIter(
+ &merge_iter->range_tombstone_iters_.front());
+ }
+ merge_iter->Finish();
+ ret = merge_iter;
+ merge_iter = nullptr;
+ }
+ return ret;
+}
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/merging_iterator.h b/src/rocksdb/table/merging_iterator.h
new file mode 100644
index 000000000..16fc0877e
--- /dev/null
+++ b/src/rocksdb/table/merging_iterator.h
@@ -0,0 +1,92 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#pragma once
+
+#include "db/range_del_aggregator.h"
+#include "rocksdb/slice.h"
+#include "rocksdb/types.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class Arena;
+class ArenaWrappedDBIter;
+class InternalKeyComparator;
+
+template <class TValue>
+class InternalIteratorBase;
+using InternalIterator = InternalIteratorBase<Slice>;
+
+// Return an iterator that provided the union of the data in
+// children[0,n-1]. Takes ownership of the child iterators and
+// will delete them when the result iterator is deleted.
+//
+// The result does no duplicate suppression. I.e., if a particular
+// key is present in K child iterators, it will be yielded K times.
+//
+// REQUIRES: n >= 0
+extern InternalIterator* NewMergingIterator(
+ const InternalKeyComparator* comparator, InternalIterator** children, int n,
+ Arena* arena = nullptr, bool prefix_seek_mode = false);
+
+class MergingIterator;
+
+// A builder class to build a merging iterator by adding iterators one by one.
+// User should call only one of AddIterator() or AddPointAndTombstoneIterator()
+// exclusively for the same builder.
+class MergeIteratorBuilder {
+ public:
+ // comparator: the comparator used in merging comparator
+ // arena: where the merging iterator needs to be allocated from.
+ explicit MergeIteratorBuilder(const InternalKeyComparator* comparator,
+ Arena* arena, bool prefix_seek_mode = false,
+ const Slice* iterate_upper_bound = nullptr);
+ ~MergeIteratorBuilder();
+
+ // Add iter to the merging iterator.
+ void AddIterator(InternalIterator* iter);
+
+ // Add a point key iterator and a range tombstone iterator.
+ // `tombstone_iter_ptr` should and only be set by LevelIterator.
+ // *tombstone_iter_ptr will be set to where the merging iterator stores
+ // `tombstone_iter` when MergeIteratorBuilder::Finish() is called. This is
+ // used by LevelIterator to update range tombstone iters when switching to a
+ // different SST file. If a single point iterator with a nullptr range
+ // tombstone iterator is provided, and the point iterator is not a level
+ // iterator, then this builder will return the point iterator directly,
+ // instead of creating a merging iterator on top of it. Internally, if all
+ // point iterators are not LevelIterator, then range tombstone iterator is
+ // only added to the merging iter if there is a non-null `tombstone_iter`.
+ void AddPointAndTombstoneIterator(
+ InternalIterator* point_iter, TruncatedRangeDelIterator* tombstone_iter,
+ TruncatedRangeDelIterator*** tombstone_iter_ptr = nullptr);
+
+ // Get arena used to build the merging iterator. It is called one a child
+ // iterator needs to be allocated.
+ Arena* GetArena() { return arena; }
+
+ // Return the result merging iterator.
+ // If db_iter is not nullptr, then db_iter->SetMemtableRangetombstoneIter()
+ // will be called with pointer to where the merging iterator
+ // stores the memtable range tombstone iterator.
+ // This is used for DB iterator to refresh memtable range tombstones.
+ InternalIterator* Finish(ArenaWrappedDBIter* db_iter = nullptr);
+
+ private:
+ MergingIterator* merge_iter;
+ InternalIterator* first_iter;
+ bool use_merging_iter;
+ Arena* arena;
+ // Used to set LevelIterator.range_tombstone_iter_.
+ // See AddRangeTombstoneIterator() implementation for more detail.
+ std::vector<std::pair<size_t, TruncatedRangeDelIterator***>>
+ range_del_iter_ptrs_;
+};
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/meta_blocks.cc b/src/rocksdb/table/meta_blocks.cc
new file mode 100644
index 000000000..78a62359d
--- /dev/null
+++ b/src/rocksdb/table/meta_blocks.cc
@@ -0,0 +1,553 @@
+// Copyright (c) 2011-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 "table/meta_blocks.h"
+
+#include <map>
+#include <string>
+
+#include "block_fetcher.h"
+#include "db/table_properties_collector.h"
+#include "file/random_access_file_reader.h"
+#include "logging/logging.h"
+#include "rocksdb/options.h"
+#include "rocksdb/table.h"
+#include "rocksdb/table_properties.h"
+#include "table/block_based/block.h"
+#include "table/block_based/reader_common.h"
+#include "table/format.h"
+#include "table/internal_iterator.h"
+#include "table/persistent_cache_helper.h"
+#include "table/sst_file_writer_collectors.h"
+#include "table/table_properties_internal.h"
+#include "test_util/sync_point.h"
+#include "util/coding.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+const std::string kPropertiesBlockName = "rocksdb.properties";
+// Old property block name for backward compatibility
+const std::string kPropertiesBlockOldName = "rocksdb.stats";
+const std::string kCompressionDictBlockName = "rocksdb.compression_dict";
+const std::string kRangeDelBlockName = "rocksdb.range_del";
+
+MetaIndexBuilder::MetaIndexBuilder()
+ : meta_index_block_(new BlockBuilder(1 /* restart interval */)) {}
+
+void MetaIndexBuilder::Add(const std::string& key, const BlockHandle& handle) {
+ std::string handle_encoding;
+ handle.EncodeTo(&handle_encoding);
+ meta_block_handles_.insert({key, handle_encoding});
+}
+
+Slice MetaIndexBuilder::Finish() {
+ for (const auto& metablock : meta_block_handles_) {
+ meta_index_block_->Add(metablock.first, metablock.second);
+ }
+ return meta_index_block_->Finish();
+}
+
+// Property block will be read sequentially and cached in a heap located
+// object, so there's no need for restart points. Thus we set the restart
+// interval to infinity to save space.
+PropertyBlockBuilder::PropertyBlockBuilder()
+ : properties_block_(new BlockBuilder(
+ std::numeric_limits<int32_t>::max() /* restart interval */)) {}
+
+void PropertyBlockBuilder::Add(const std::string& name,
+ const std::string& val) {
+ props_.insert({name, val});
+}
+
+void PropertyBlockBuilder::Add(const std::string& name, uint64_t val) {
+ assert(props_.find(name) == props_.end());
+
+ std::string dst;
+ PutVarint64(&dst, val);
+
+ Add(name, dst);
+}
+
+void PropertyBlockBuilder::Add(
+ const UserCollectedProperties& user_collected_properties) {
+ for (const auto& prop : user_collected_properties) {
+ Add(prop.first, prop.second);
+ }
+}
+
+void PropertyBlockBuilder::AddTableProperty(const TableProperties& props) {
+ TEST_SYNC_POINT_CALLBACK("PropertyBlockBuilder::AddTableProperty:Start",
+ const_cast<TableProperties*>(&props));
+
+ Add(TablePropertiesNames::kOriginalFileNumber, props.orig_file_number);
+ Add(TablePropertiesNames::kRawKeySize, props.raw_key_size);
+ Add(TablePropertiesNames::kRawValueSize, props.raw_value_size);
+ Add(TablePropertiesNames::kDataSize, props.data_size);
+ Add(TablePropertiesNames::kIndexSize, props.index_size);
+ if (props.index_partitions != 0) {
+ Add(TablePropertiesNames::kIndexPartitions, props.index_partitions);
+ Add(TablePropertiesNames::kTopLevelIndexSize, props.top_level_index_size);
+ }
+ Add(TablePropertiesNames::kIndexKeyIsUserKey, props.index_key_is_user_key);
+ Add(TablePropertiesNames::kIndexValueIsDeltaEncoded,
+ props.index_value_is_delta_encoded);
+ Add(TablePropertiesNames::kNumEntries, props.num_entries);
+ Add(TablePropertiesNames::kNumFilterEntries, props.num_filter_entries);
+ Add(TablePropertiesNames::kDeletedKeys, props.num_deletions);
+ Add(TablePropertiesNames::kMergeOperands, props.num_merge_operands);
+ Add(TablePropertiesNames::kNumRangeDeletions, props.num_range_deletions);
+ Add(TablePropertiesNames::kNumDataBlocks, props.num_data_blocks);
+ Add(TablePropertiesNames::kFilterSize, props.filter_size);
+ Add(TablePropertiesNames::kFormatVersion, props.format_version);
+ Add(TablePropertiesNames::kFixedKeyLen, props.fixed_key_len);
+ Add(TablePropertiesNames::kColumnFamilyId, props.column_family_id);
+ Add(TablePropertiesNames::kCreationTime, props.creation_time);
+ Add(TablePropertiesNames::kOldestKeyTime, props.oldest_key_time);
+ if (props.file_creation_time > 0) {
+ Add(TablePropertiesNames::kFileCreationTime, props.file_creation_time);
+ }
+ if (props.slow_compression_estimated_data_size > 0) {
+ Add(TablePropertiesNames::kSlowCompressionEstimatedDataSize,
+ props.slow_compression_estimated_data_size);
+ }
+ if (props.fast_compression_estimated_data_size > 0) {
+ Add(TablePropertiesNames::kFastCompressionEstimatedDataSize,
+ props.fast_compression_estimated_data_size);
+ }
+ if (!props.db_id.empty()) {
+ Add(TablePropertiesNames::kDbId, props.db_id);
+ }
+ if (!props.db_session_id.empty()) {
+ Add(TablePropertiesNames::kDbSessionId, props.db_session_id);
+ }
+ if (!props.db_host_id.empty()) {
+ Add(TablePropertiesNames::kDbHostId, props.db_host_id);
+ }
+
+ if (!props.filter_policy_name.empty()) {
+ Add(TablePropertiesNames::kFilterPolicy, props.filter_policy_name);
+ }
+ if (!props.comparator_name.empty()) {
+ Add(TablePropertiesNames::kComparator, props.comparator_name);
+ }
+
+ if (!props.merge_operator_name.empty()) {
+ Add(TablePropertiesNames::kMergeOperator, props.merge_operator_name);
+ }
+ if (!props.prefix_extractor_name.empty()) {
+ Add(TablePropertiesNames::kPrefixExtractorName,
+ props.prefix_extractor_name);
+ }
+ if (!props.property_collectors_names.empty()) {
+ Add(TablePropertiesNames::kPropertyCollectors,
+ props.property_collectors_names);
+ }
+ if (!props.column_family_name.empty()) {
+ Add(TablePropertiesNames::kColumnFamilyName, props.column_family_name);
+ }
+
+ if (!props.compression_name.empty()) {
+ Add(TablePropertiesNames::kCompression, props.compression_name);
+ }
+ if (!props.compression_options.empty()) {
+ Add(TablePropertiesNames::kCompressionOptions, props.compression_options);
+ }
+ if (!props.seqno_to_time_mapping.empty()) {
+ Add(TablePropertiesNames::kSequenceNumberTimeMapping,
+ props.seqno_to_time_mapping);
+ }
+}
+
+Slice PropertyBlockBuilder::Finish() {
+ for (const auto& prop : props_) {
+ properties_block_->Add(prop.first, prop.second);
+ }
+
+ return properties_block_->Finish();
+}
+
+void LogPropertiesCollectionError(Logger* info_log, const std::string& method,
+ const std::string& name) {
+ assert(method == "Add" || method == "Finish");
+
+ std::string msg =
+ "Encountered error when calling TablePropertiesCollector::" + method +
+ "() with collector name: " + name;
+ ROCKS_LOG_ERROR(info_log, "%s", msg.c_str());
+}
+
+bool NotifyCollectTableCollectorsOnAdd(
+ const Slice& key, const Slice& value, uint64_t file_size,
+ const std::vector<std::unique_ptr<IntTblPropCollector>>& collectors,
+ Logger* info_log) {
+ bool all_succeeded = true;
+ for (auto& collector : collectors) {
+ Status s = collector->InternalAdd(key, value, file_size);
+ all_succeeded = all_succeeded && s.ok();
+ if (!s.ok()) {
+ LogPropertiesCollectionError(info_log, "Add" /* method */,
+ collector->Name());
+ }
+ }
+ return all_succeeded;
+}
+
+void NotifyCollectTableCollectorsOnBlockAdd(
+ const std::vector<std::unique_ptr<IntTblPropCollector>>& collectors,
+ const uint64_t block_uncomp_bytes,
+ const uint64_t block_compressed_bytes_fast,
+ const uint64_t block_compressed_bytes_slow) {
+ for (auto& collector : collectors) {
+ collector->BlockAdd(block_uncomp_bytes, block_compressed_bytes_fast,
+ block_compressed_bytes_slow);
+ }
+}
+
+bool NotifyCollectTableCollectorsOnFinish(
+ const std::vector<std::unique_ptr<IntTblPropCollector>>& collectors,
+ Logger* info_log, PropertyBlockBuilder* builder) {
+ bool all_succeeded = true;
+ for (auto& collector : collectors) {
+ UserCollectedProperties user_collected_properties;
+ Status s = collector->Finish(&user_collected_properties);
+
+ all_succeeded = all_succeeded && s.ok();
+ if (!s.ok()) {
+ LogPropertiesCollectionError(info_log, "Finish" /* method */,
+ collector->Name());
+ } else {
+ builder->Add(user_collected_properties);
+ }
+ }
+
+ return all_succeeded;
+}
+
+// FIXME: should be a parameter for reading table properties to use persistent
+// cache?
+Status ReadTablePropertiesHelper(
+ const ReadOptions& ro, const BlockHandle& handle,
+ RandomAccessFileReader* file, FilePrefetchBuffer* prefetch_buffer,
+ const Footer& footer, const ImmutableOptions& ioptions,
+ std::unique_ptr<TableProperties>* table_properties,
+ MemoryAllocator* memory_allocator) {
+ assert(table_properties);
+
+ // If this is an external SST file ingested with write_global_seqno set to
+ // true, then we expect the checksum mismatch because checksum was written
+ // by SstFileWriter, but its global seqno in the properties block may have
+ // been changed during ingestion. For this reason, we initially read
+ // and process without checksum verification, then later try checksum
+ // verification so that if it fails, we can copy to a temporary buffer with
+ // global seqno set to its original value, i.e. 0, and attempt checksum
+ // verification again.
+ ReadOptions modified_ro = ro;
+ modified_ro.verify_checksums = false;
+ BlockContents block_contents;
+ BlockFetcher block_fetcher(file, prefetch_buffer, footer, modified_ro, handle,
+ &block_contents, ioptions, false /* decompress */,
+ false /*maybe_compressed*/, BlockType::kProperties,
+ UncompressionDict::GetEmptyDict(),
+ PersistentCacheOptions::kEmpty, memory_allocator);
+ Status s = block_fetcher.ReadBlockContents();
+ if (!s.ok()) {
+ return s;
+ }
+
+ // Unfortunately, Block::size() might not equal block_contents.data.size(),
+ // and Block hides block_contents
+ uint64_t block_size = block_contents.data.size();
+ Block properties_block(std::move(block_contents));
+ std::unique_ptr<MetaBlockIter> iter(properties_block.NewMetaIterator());
+
+ std::unique_ptr<TableProperties> new_table_properties{new TableProperties};
+ // All pre-defined properties of type uint64_t
+ std::unordered_map<std::string, uint64_t*> predefined_uint64_properties = {
+ {TablePropertiesNames::kOriginalFileNumber,
+ &new_table_properties->orig_file_number},
+ {TablePropertiesNames::kDataSize, &new_table_properties->data_size},
+ {TablePropertiesNames::kIndexSize, &new_table_properties->index_size},
+ {TablePropertiesNames::kIndexPartitions,
+ &new_table_properties->index_partitions},
+ {TablePropertiesNames::kTopLevelIndexSize,
+ &new_table_properties->top_level_index_size},
+ {TablePropertiesNames::kIndexKeyIsUserKey,
+ &new_table_properties->index_key_is_user_key},
+ {TablePropertiesNames::kIndexValueIsDeltaEncoded,
+ &new_table_properties->index_value_is_delta_encoded},
+ {TablePropertiesNames::kFilterSize, &new_table_properties->filter_size},
+ {TablePropertiesNames::kRawKeySize, &new_table_properties->raw_key_size},
+ {TablePropertiesNames::kRawValueSize,
+ &new_table_properties->raw_value_size},
+ {TablePropertiesNames::kNumDataBlocks,
+ &new_table_properties->num_data_blocks},
+ {TablePropertiesNames::kNumEntries, &new_table_properties->num_entries},
+ {TablePropertiesNames::kNumFilterEntries,
+ &new_table_properties->num_filter_entries},
+ {TablePropertiesNames::kDeletedKeys,
+ &new_table_properties->num_deletions},
+ {TablePropertiesNames::kMergeOperands,
+ &new_table_properties->num_merge_operands},
+ {TablePropertiesNames::kNumRangeDeletions,
+ &new_table_properties->num_range_deletions},
+ {TablePropertiesNames::kFormatVersion,
+ &new_table_properties->format_version},
+ {TablePropertiesNames::kFixedKeyLen,
+ &new_table_properties->fixed_key_len},
+ {TablePropertiesNames::kColumnFamilyId,
+ &new_table_properties->column_family_id},
+ {TablePropertiesNames::kCreationTime,
+ &new_table_properties->creation_time},
+ {TablePropertiesNames::kOldestKeyTime,
+ &new_table_properties->oldest_key_time},
+ {TablePropertiesNames::kFileCreationTime,
+ &new_table_properties->file_creation_time},
+ {TablePropertiesNames::kSlowCompressionEstimatedDataSize,
+ &new_table_properties->slow_compression_estimated_data_size},
+ {TablePropertiesNames::kFastCompressionEstimatedDataSize,
+ &new_table_properties->fast_compression_estimated_data_size},
+ };
+
+ std::string last_key;
+ for (iter->SeekToFirst(); iter->Valid(); iter->Next()) {
+ s = iter->status();
+ if (!s.ok()) {
+ break;
+ }
+
+ auto key = iter->key().ToString();
+ // properties block should be strictly sorted with no duplicate key.
+ if (!last_key.empty() &&
+ BytewiseComparator()->Compare(key, last_key) <= 0) {
+ s = Status::Corruption("properties unsorted");
+ break;
+ }
+ last_key = key;
+
+ auto raw_val = iter->value();
+ auto pos = predefined_uint64_properties.find(key);
+
+ if (key == ExternalSstFilePropertyNames::kGlobalSeqno) {
+ new_table_properties->external_sst_file_global_seqno_offset =
+ handle.offset() + iter->ValueOffset();
+ }
+
+ if (pos != predefined_uint64_properties.end()) {
+ if (key == TablePropertiesNames::kDeletedKeys ||
+ key == TablePropertiesNames::kMergeOperands) {
+ // Insert in user-collected properties for API backwards compatibility
+ new_table_properties->user_collected_properties.insert(
+ {key, raw_val.ToString()});
+ }
+ // handle predefined rocksdb properties
+ uint64_t val;
+ if (!GetVarint64(&raw_val, &val)) {
+ // skip malformed value
+ auto error_msg =
+ "Detect malformed value in properties meta-block:"
+ "\tkey: " +
+ key + "\tval: " + raw_val.ToString();
+ ROCKS_LOG_ERROR(ioptions.logger, "%s", error_msg.c_str());
+ continue;
+ }
+ *(pos->second) = val;
+ } else if (key == TablePropertiesNames::kDbId) {
+ new_table_properties->db_id = raw_val.ToString();
+ } else if (key == TablePropertiesNames::kDbSessionId) {
+ new_table_properties->db_session_id = raw_val.ToString();
+ } else if (key == TablePropertiesNames::kDbHostId) {
+ new_table_properties->db_host_id = raw_val.ToString();
+ } else if (key == TablePropertiesNames::kFilterPolicy) {
+ new_table_properties->filter_policy_name = raw_val.ToString();
+ } else if (key == TablePropertiesNames::kColumnFamilyName) {
+ new_table_properties->column_family_name = raw_val.ToString();
+ } else if (key == TablePropertiesNames::kComparator) {
+ new_table_properties->comparator_name = raw_val.ToString();
+ } else if (key == TablePropertiesNames::kMergeOperator) {
+ new_table_properties->merge_operator_name = raw_val.ToString();
+ } else if (key == TablePropertiesNames::kPrefixExtractorName) {
+ new_table_properties->prefix_extractor_name = raw_val.ToString();
+ } else if (key == TablePropertiesNames::kPropertyCollectors) {
+ new_table_properties->property_collectors_names = raw_val.ToString();
+ } else if (key == TablePropertiesNames::kCompression) {
+ new_table_properties->compression_name = raw_val.ToString();
+ } else if (key == TablePropertiesNames::kCompressionOptions) {
+ new_table_properties->compression_options = raw_val.ToString();
+ } else if (key == TablePropertiesNames::kSequenceNumberTimeMapping) {
+ new_table_properties->seqno_to_time_mapping = raw_val.ToString();
+ } else {
+ // handle user-collected properties
+ new_table_properties->user_collected_properties.insert(
+ {key, raw_val.ToString()});
+ }
+ }
+
+ // Modified version of BlockFetcher checksum verification
+ // (See write_global_seqno comment above)
+ if (s.ok() && footer.GetBlockTrailerSize() > 0) {
+ s = VerifyBlockChecksum(footer.checksum_type(), properties_block.data(),
+ block_size, file->file_name(), handle.offset());
+ if (s.IsCorruption()) {
+ if (new_table_properties->external_sst_file_global_seqno_offset != 0) {
+ std::string tmp_buf(properties_block.data(),
+ block_fetcher.GetBlockSizeWithTrailer());
+ uint64_t global_seqno_offset =
+ new_table_properties->external_sst_file_global_seqno_offset -
+ handle.offset();
+ EncodeFixed64(&tmp_buf[static_cast<size_t>(global_seqno_offset)], 0);
+ s = VerifyBlockChecksum(footer.checksum_type(), tmp_buf.data(),
+ block_size, file->file_name(), handle.offset());
+ }
+ }
+ }
+
+ if (s.ok()) {
+ *table_properties = std::move(new_table_properties);
+ }
+
+ return s;
+}
+
+Status ReadTableProperties(RandomAccessFileReader* file, uint64_t file_size,
+ uint64_t table_magic_number,
+ const ImmutableOptions& ioptions,
+ std::unique_ptr<TableProperties>* properties,
+ MemoryAllocator* memory_allocator,
+ FilePrefetchBuffer* prefetch_buffer) {
+ BlockHandle block_handle;
+ Footer footer;
+ Status s = FindMetaBlockInFile(file, file_size, table_magic_number, ioptions,
+ kPropertiesBlockName, &block_handle,
+ memory_allocator, prefetch_buffer, &footer);
+ if (!s.ok()) {
+ return s;
+ }
+
+ if (!block_handle.IsNull()) {
+ s = ReadTablePropertiesHelper(ReadOptions(), block_handle, file,
+ prefetch_buffer, footer, ioptions, properties,
+ memory_allocator);
+ } else {
+ s = Status::NotFound();
+ }
+ return s;
+}
+
+Status FindOptionalMetaBlock(InternalIterator* meta_index_iter,
+ const std::string& meta_block_name,
+ BlockHandle* block_handle) {
+ assert(block_handle != nullptr);
+ meta_index_iter->Seek(meta_block_name);
+ if (meta_index_iter->status().ok()) {
+ if (meta_index_iter->Valid() && meta_index_iter->key() == meta_block_name) {
+ Slice v = meta_index_iter->value();
+ return block_handle->DecodeFrom(&v);
+ } else if (meta_block_name == kPropertiesBlockName) {
+ // Have to try old name for compatibility
+ meta_index_iter->Seek(kPropertiesBlockOldName);
+ if (meta_index_iter->status().ok() && meta_index_iter->Valid() &&
+ meta_index_iter->key() == kPropertiesBlockOldName) {
+ Slice v = meta_index_iter->value();
+ return block_handle->DecodeFrom(&v);
+ }
+ }
+ }
+ // else
+ *block_handle = BlockHandle::NullBlockHandle();
+ return meta_index_iter->status();
+}
+
+Status FindMetaBlock(InternalIterator* meta_index_iter,
+ const std::string& meta_block_name,
+ BlockHandle* block_handle) {
+ Status s =
+ FindOptionalMetaBlock(meta_index_iter, meta_block_name, block_handle);
+ if (s.ok() && block_handle->IsNull()) {
+ return Status::Corruption("Cannot find the meta block", meta_block_name);
+ } else {
+ return s;
+ }
+}
+
+Status ReadMetaIndexBlockInFile(RandomAccessFileReader* file,
+ uint64_t file_size, uint64_t table_magic_number,
+ const ImmutableOptions& ioptions,
+ BlockContents* metaindex_contents,
+ MemoryAllocator* memory_allocator,
+ FilePrefetchBuffer* prefetch_buffer,
+ Footer* footer_out) {
+ Footer footer;
+ IOOptions opts;
+ auto s = ReadFooterFromFile(opts, file, prefetch_buffer, file_size, &footer,
+ table_magic_number);
+ if (!s.ok()) {
+ return s;
+ }
+ if (footer_out) {
+ *footer_out = footer;
+ }
+
+ auto metaindex_handle = footer.metaindex_handle();
+ return BlockFetcher(file, prefetch_buffer, footer, ReadOptions(),
+ metaindex_handle, metaindex_contents, ioptions,
+ false /* do decompression */, false /*maybe_compressed*/,
+ BlockType::kMetaIndex, UncompressionDict::GetEmptyDict(),
+ PersistentCacheOptions::kEmpty, memory_allocator)
+ .ReadBlockContents();
+}
+
+Status FindMetaBlockInFile(RandomAccessFileReader* file, uint64_t file_size,
+ uint64_t table_magic_number,
+ const ImmutableOptions& ioptions,
+ const std::string& meta_block_name,
+ BlockHandle* block_handle,
+ MemoryAllocator* memory_allocator,
+ FilePrefetchBuffer* prefetch_buffer,
+ Footer* footer_out) {
+ BlockContents metaindex_contents;
+ auto s = ReadMetaIndexBlockInFile(
+ file, file_size, table_magic_number, ioptions, &metaindex_contents,
+ memory_allocator, prefetch_buffer, footer_out);
+ if (!s.ok()) {
+ return s;
+ }
+ // meta blocks are never compressed. Need to add uncompress logic if we are to
+ // compress it.
+ Block metaindex_block(std::move(metaindex_contents));
+
+ std::unique_ptr<InternalIterator> meta_iter;
+ meta_iter.reset(metaindex_block.NewMetaIterator());
+
+ return FindMetaBlock(meta_iter.get(), meta_block_name, block_handle);
+}
+
+Status ReadMetaBlock(RandomAccessFileReader* file,
+ FilePrefetchBuffer* prefetch_buffer, uint64_t file_size,
+ uint64_t table_magic_number,
+ const ImmutableOptions& ioptions,
+ const std::string& meta_block_name, BlockType block_type,
+ BlockContents* contents,
+ MemoryAllocator* memory_allocator) {
+ // TableProperties requires special handling because of checksum issues.
+ // Call ReadTableProperties instead for that case.
+ assert(block_type != BlockType::kProperties);
+
+ BlockHandle block_handle;
+ Footer footer;
+ Status status = FindMetaBlockInFile(
+ file, file_size, table_magic_number, ioptions, meta_block_name,
+ &block_handle, memory_allocator, prefetch_buffer, &footer);
+ if (!status.ok()) {
+ return status;
+ }
+
+ return BlockFetcher(file, prefetch_buffer, footer, ReadOptions(),
+ block_handle, contents, ioptions, false /* decompress */,
+ false /*maybe_compressed*/, block_type,
+ UncompressionDict::GetEmptyDict(),
+ PersistentCacheOptions::kEmpty, memory_allocator)
+ .ReadBlockContents();
+}
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/meta_blocks.h b/src/rocksdb/table/meta_blocks.h
new file mode 100644
index 000000000..b867dd01d
--- /dev/null
+++ b/src/rocksdb/table/meta_blocks.h
@@ -0,0 +1,168 @@
+// Copyright (c) 2011-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).
+#pragma once
+
+#include <map>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "db/builder.h"
+#include "db/table_properties_collector.h"
+#include "rocksdb/comparator.h"
+#include "rocksdb/memory_allocator.h"
+#include "rocksdb/options.h"
+#include "rocksdb/slice.h"
+#include "table/block_based/block_builder.h"
+#include "table/block_based/block_type.h"
+#include "table/format.h"
+#include "util/kv_map.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class BlockBuilder;
+class BlockHandle;
+class Env;
+class Footer;
+class Logger;
+class RandomAccessFile;
+struct TableProperties;
+
+// Meta block names for metaindex
+extern const std::string kPropertiesBlockName;
+extern const std::string kPropertiesBlockOldName;
+extern const std::string kCompressionDictBlockName;
+extern const std::string kRangeDelBlockName;
+
+class MetaIndexBuilder {
+ public:
+ MetaIndexBuilder(const MetaIndexBuilder&) = delete;
+ MetaIndexBuilder& operator=(const MetaIndexBuilder&) = delete;
+
+ MetaIndexBuilder();
+ void Add(const std::string& key, const BlockHandle& handle);
+
+ // Write all the added key/value pairs to the block and return the contents
+ // of the block.
+ Slice Finish();
+
+ private:
+ // store the sorted key/handle of the metablocks.
+ stl_wrappers::KVMap meta_block_handles_;
+ std::unique_ptr<BlockBuilder> meta_index_block_;
+};
+
+class PropertyBlockBuilder {
+ public:
+ PropertyBlockBuilder(const PropertyBlockBuilder&) = delete;
+ PropertyBlockBuilder& operator=(const PropertyBlockBuilder&) = delete;
+
+ PropertyBlockBuilder();
+
+ void AddTableProperty(const TableProperties& props);
+ void Add(const std::string& key, uint64_t value);
+ void Add(const std::string& key, const std::string& value);
+ void Add(const UserCollectedProperties& user_collected_properties);
+
+ // Write all the added entries to the block and return the block contents
+ Slice Finish();
+
+ private:
+ std::unique_ptr<BlockBuilder> properties_block_;
+ stl_wrappers::KVMap props_;
+};
+
+// Were we encounter any error occurs during user-defined statistics collection,
+// we'll write the warning message to info log.
+void LogPropertiesCollectionError(Logger* info_log, const std::string& method,
+ const std::string& name);
+
+// Utility functions help table builder to trigger batch events for user
+// defined property collectors.
+// Return value indicates if there is any error occurred; if error occurred,
+// the warning message will be logged.
+// NotifyCollectTableCollectorsOnAdd() triggers the `Add` event for all
+// property collectors.
+bool NotifyCollectTableCollectorsOnAdd(
+ const Slice& key, const Slice& value, uint64_t file_size,
+ const std::vector<std::unique_ptr<IntTblPropCollector>>& collectors,
+ Logger* info_log);
+
+void NotifyCollectTableCollectorsOnBlockAdd(
+ const std::vector<std::unique_ptr<IntTblPropCollector>>& collectors,
+ uint64_t block_uncomp_bytes, uint64_t block_compressed_bytes_fast,
+ uint64_t block_compressed_bytes_slow);
+
+// NotifyCollectTableCollectorsOnFinish() triggers the `Finish` event for all
+// property collectors. The collected properties will be added to `builder`.
+bool NotifyCollectTableCollectorsOnFinish(
+ const std::vector<std::unique_ptr<IntTblPropCollector>>& collectors,
+ Logger* info_log, PropertyBlockBuilder* builder);
+
+// Read table properties from a file using known BlockHandle.
+// @returns a status to indicate if the operation succeeded. On success,
+// *table_properties will point to a heap-allocated TableProperties
+// object, otherwise value of `table_properties` will not be modified.
+Status ReadTablePropertiesHelper(
+ const ReadOptions& ro, const BlockHandle& handle,
+ RandomAccessFileReader* file, FilePrefetchBuffer* prefetch_buffer,
+ const Footer& footer, const ImmutableOptions& ioptions,
+ std::unique_ptr<TableProperties>* table_properties,
+ MemoryAllocator* memory_allocator = nullptr);
+
+// Read table properties from the properties block of a plain table.
+// @returns a status to indicate if the operation succeeded. On success,
+// *table_properties will point to a heap-allocated TableProperties
+// object, otherwise value of `table_properties` will not be modified.
+Status ReadTableProperties(RandomAccessFileReader* file, uint64_t file_size,
+ uint64_t table_magic_number,
+ const ImmutableOptions& ioptions,
+ std::unique_ptr<TableProperties>* properties,
+ MemoryAllocator* memory_allocator = nullptr,
+ FilePrefetchBuffer* prefetch_buffer = nullptr);
+
+// Find the meta block from the meta index block. Returns OK and
+// block_handle->IsNull() if not found.
+Status FindOptionalMetaBlock(InternalIterator* meta_index_iter,
+ const std::string& meta_block_name,
+ BlockHandle* block_handle);
+
+// Find the meta block from the meta index block. Returns Corruption if not
+// found.
+Status FindMetaBlock(InternalIterator* meta_index_iter,
+ const std::string& meta_block_name,
+ BlockHandle* block_handle);
+
+// Find the meta block
+Status FindMetaBlockInFile(RandomAccessFileReader* file, uint64_t file_size,
+ uint64_t table_magic_number,
+ const ImmutableOptions& ioptions,
+ const std::string& meta_block_name,
+ BlockHandle* block_handle,
+ MemoryAllocator* memory_allocator = nullptr,
+ FilePrefetchBuffer* prefetch_buffer = nullptr,
+ Footer* footer_out = nullptr);
+
+// Read meta block contents
+Status ReadMetaIndexBlockInFile(RandomAccessFileReader* file,
+ uint64_t file_size, uint64_t table_magic_number,
+ const ImmutableOptions& ioptions,
+ BlockContents* block_contents,
+ MemoryAllocator* memory_allocator = nullptr,
+ FilePrefetchBuffer* prefetch_buffer = nullptr,
+ Footer* footer_out = nullptr);
+
+// Read the specified meta block with name meta_block_name
+// from `file` and initialize `contents` with contents of this block.
+// Return Status::OK in case of success.
+Status ReadMetaBlock(RandomAccessFileReader* file,
+ FilePrefetchBuffer* prefetch_buffer, uint64_t file_size,
+ uint64_t table_magic_number,
+ const ImmutableOptions& ioptions,
+ const std::string& meta_block_name, BlockType block_type,
+ BlockContents* contents,
+ MemoryAllocator* memory_allocator = nullptr);
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/mock_table.cc b/src/rocksdb/table/mock_table.cc
new file mode 100644
index 000000000..130889eaa
--- /dev/null
+++ b/src/rocksdb/table/mock_table.cc
@@ -0,0 +1,344 @@
+// Copyright (c) 2011-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 "table/mock_table.h"
+
+#include "db/dbformat.h"
+#include "env/composite_env_wrapper.h"
+#include "file/random_access_file_reader.h"
+#include "port/port.h"
+#include "rocksdb/table_properties.h"
+#include "table/get_context.h"
+#include "util/coding.h"
+
+namespace ROCKSDB_NAMESPACE {
+namespace mock {
+
+KVVector MakeMockFile(std::initializer_list<KVPair> l) { return KVVector(l); }
+
+void SortKVVector(KVVector* kv_vector, const Comparator* ucmp) {
+ InternalKeyComparator icmp(ucmp);
+ std::sort(kv_vector->begin(), kv_vector->end(),
+ [icmp](KVPair a, KVPair b) -> bool {
+ return icmp.Compare(a.first, b.first) < 0;
+ });
+}
+
+class MockTableReader : public TableReader {
+ public:
+ explicit MockTableReader(const KVVector& table) : table_(table) {}
+
+ InternalIterator* NewIterator(const ReadOptions&,
+ const SliceTransform* prefix_extractor,
+ Arena* arena, bool skip_filters,
+ TableReaderCaller caller,
+ size_t compaction_readahead_size = 0,
+ bool allow_unprepared_value = false) override;
+
+ Status Get(const ReadOptions& readOptions, const Slice& key,
+ GetContext* get_context, const SliceTransform* prefix_extractor,
+ bool skip_filters = false) override;
+
+ uint64_t ApproximateOffsetOf(const Slice& /*key*/,
+ TableReaderCaller /*caller*/) override {
+ return 0;
+ }
+
+ uint64_t ApproximateSize(const Slice& /*start*/, const Slice& /*end*/,
+ TableReaderCaller /*caller*/) override {
+ return 0;
+ }
+
+ size_t ApproximateMemoryUsage() const override { return 0; }
+
+ void SetupForCompaction() override {}
+
+ std::shared_ptr<const TableProperties> GetTableProperties() const override;
+
+ ~MockTableReader() {}
+
+ private:
+ const KVVector& table_;
+};
+
+class MockTableIterator : public InternalIterator {
+ public:
+ explicit MockTableIterator(const KVVector& table) : table_(table) {
+ itr_ = table_.end();
+ }
+
+ bool Valid() const override { return itr_ != table_.end(); }
+
+ void SeekToFirst() override { itr_ = table_.begin(); }
+
+ void SeekToLast() override {
+ itr_ = table_.end();
+ --itr_;
+ }
+
+ void Seek(const Slice& target) override {
+ KVPair target_pair(target.ToString(), "");
+ InternalKeyComparator icmp(BytewiseComparator());
+ itr_ = std::lower_bound(table_.begin(), table_.end(), target_pair,
+ [icmp](KVPair a, KVPair b) -> bool {
+ return icmp.Compare(a.first, b.first) < 0;
+ });
+ }
+
+ void SeekForPrev(const Slice& target) override {
+ KVPair target_pair(target.ToString(), "");
+ InternalKeyComparator icmp(BytewiseComparator());
+ itr_ = std::upper_bound(table_.begin(), table_.end(), target_pair,
+ [icmp](KVPair a, KVPair b) -> bool {
+ return icmp.Compare(a.first, b.first) < 0;
+ });
+ Prev();
+ }
+
+ void Next() override { ++itr_; }
+
+ void Prev() override {
+ if (itr_ == table_.begin()) {
+ itr_ = table_.end();
+ } else {
+ --itr_;
+ }
+ }
+
+ Slice key() const override { return Slice(itr_->first); }
+
+ Slice value() const override { return Slice(itr_->second); }
+
+ Status status() const override { return Status::OK(); }
+
+ private:
+ const KVVector& table_;
+ KVVector::const_iterator itr_;
+};
+
+class MockTableBuilder : public TableBuilder {
+ public:
+ MockTableBuilder(uint32_t id, MockTableFileSystem* file_system,
+ MockTableFactory::MockCorruptionMode corrupt_mode =
+ MockTableFactory::kCorruptNone,
+ size_t key_value_size = 1)
+ : id_(id),
+ file_system_(file_system),
+ corrupt_mode_(corrupt_mode),
+ key_value_size_(key_value_size) {
+ table_ = MakeMockFile({});
+ }
+
+ // REQUIRES: Either Finish() or Abandon() has been called.
+ ~MockTableBuilder() {}
+
+ // Add key,value to the table being constructed.
+ // REQUIRES: key is after any previously added key according to comparator.
+ // REQUIRES: Finish(), Abandon() have not been called
+ void Add(const Slice& key, const Slice& value) override {
+ if (corrupt_mode_ == MockTableFactory::kCorruptValue) {
+ // Corrupt the value
+ table_.push_back({key.ToString(), value.ToString() + " "});
+ corrupt_mode_ = MockTableFactory::kCorruptNone;
+ } else if (corrupt_mode_ == MockTableFactory::kCorruptKey) {
+ table_.push_back({key.ToString() + " ", value.ToString()});
+ corrupt_mode_ = MockTableFactory::kCorruptNone;
+ } else if (corrupt_mode_ == MockTableFactory::kCorruptReorderKey) {
+ if (prev_key_.empty()) {
+ prev_key_ = key.ToString();
+ prev_value_ = value.ToString();
+ } else {
+ table_.push_back({key.ToString(), value.ToString()});
+ table_.push_back({prev_key_, prev_value_});
+ corrupt_mode_ = MockTableFactory::kCorruptNone;
+ }
+ } else {
+ table_.push_back({key.ToString(), value.ToString()});
+ }
+ }
+
+ // Return non-ok iff some error has been detected.
+ Status status() const override { return Status::OK(); }
+
+ // Return non-ok iff some error happens during IO.
+ IOStatus io_status() const override { return IOStatus::OK(); }
+
+ Status Finish() override {
+ MutexLock lock_guard(&file_system_->mutex);
+ file_system_->files.insert({id_, table_});
+ return Status::OK();
+ }
+
+ void Abandon() override {}
+
+ uint64_t NumEntries() const override { return table_.size(); }
+
+ uint64_t FileSize() const override { return table_.size() * key_value_size_; }
+
+ TableProperties GetTableProperties() const override {
+ return TableProperties();
+ }
+
+ // Get file checksum
+ std::string GetFileChecksum() const override { return kUnknownFileChecksum; }
+ // Get file checksum function name
+ const char* GetFileChecksumFuncName() const override {
+ return kUnknownFileChecksumFuncName;
+ }
+
+ private:
+ uint32_t id_;
+ std::string prev_key_;
+ std::string prev_value_;
+ MockTableFileSystem* file_system_;
+ int corrupt_mode_;
+ KVVector table_;
+ size_t key_value_size_;
+};
+
+InternalIterator* MockTableReader::NewIterator(
+ const ReadOptions&, const SliceTransform* /* prefix_extractor */,
+ Arena* /*arena*/, bool /*skip_filters*/, TableReaderCaller /*caller*/,
+ size_t /*compaction_readahead_size*/, bool /* allow_unprepared_value */) {
+ return new MockTableIterator(table_);
+}
+
+Status MockTableReader::Get(const ReadOptions&, const Slice& key,
+ GetContext* get_context,
+ const SliceTransform* /*prefix_extractor*/,
+ bool /*skip_filters*/) {
+ std::unique_ptr<MockTableIterator> iter(new MockTableIterator(table_));
+ for (iter->Seek(key); iter->Valid(); iter->Next()) {
+ ParsedInternalKey parsed_key;
+ Status pik_status =
+ ParseInternalKey(iter->key(), &parsed_key, true /* log_err_key */);
+ if (!pik_status.ok()) {
+ return pik_status;
+ }
+
+ bool dont_care __attribute__((__unused__));
+ if (!get_context->SaveValue(parsed_key, iter->value(), &dont_care)) {
+ break;
+ }
+ }
+ return Status::OK();
+}
+
+std::shared_ptr<const TableProperties> MockTableReader::GetTableProperties()
+ const {
+ return std::shared_ptr<const TableProperties>(new TableProperties());
+}
+
+MockTableFactory::MockTableFactory()
+ : next_id_(1), corrupt_mode_(MockTableFactory::kCorruptNone) {}
+
+Status MockTableFactory::NewTableReader(
+ const ReadOptions& /*ro*/,
+ const TableReaderOptions& /*table_reader_options*/,
+ std::unique_ptr<RandomAccessFileReader>&& file, uint64_t /*file_size*/,
+ std::unique_ptr<TableReader>* table_reader,
+ bool /*prefetch_index_and_filter_in_cache*/) const {
+ uint32_t id;
+ Status s = GetIDFromFile(file.get(), &id);
+ if (!s.ok()) {
+ return s;
+ }
+
+ MutexLock lock_guard(&file_system_.mutex);
+
+ auto it = file_system_.files.find(id);
+ if (it == file_system_.files.end()) {
+ return Status::IOError("Mock file not found");
+ }
+
+ table_reader->reset(new MockTableReader(it->second));
+
+ return Status::OK();
+}
+
+TableBuilder* MockTableFactory::NewTableBuilder(
+ const TableBuilderOptions& /*table_builder_options*/,
+ WritableFileWriter* file) const {
+ uint32_t id;
+ Status s = GetAndWriteNextID(file, &id);
+ assert(s.ok());
+
+ return new MockTableBuilder(id, &file_system_, corrupt_mode_,
+ key_value_size_);
+}
+
+Status MockTableFactory::CreateMockTable(Env* env, const std::string& fname,
+ KVVector file_contents) {
+ std::unique_ptr<WritableFileWriter> file_writer;
+ Status s = WritableFileWriter::Create(env->GetFileSystem(), fname,
+ FileOptions(), &file_writer, nullptr);
+ if (!s.ok()) {
+ return s;
+ }
+ uint32_t id;
+ s = GetAndWriteNextID(file_writer.get(), &id);
+ if (s.ok()) {
+ file_system_.files.insert({id, std::move(file_contents)});
+ }
+ return s;
+}
+
+Status MockTableFactory::GetAndWriteNextID(WritableFileWriter* file,
+ uint32_t* next_id) const {
+ *next_id = next_id_.fetch_add(1);
+ char buf[4];
+ EncodeFixed32(buf, *next_id);
+ return file->Append(Slice(buf, 4));
+}
+
+Status MockTableFactory::GetIDFromFile(RandomAccessFileReader* file,
+ uint32_t* id) const {
+ char buf[4];
+ Slice result;
+ Status s = file->Read(IOOptions(), 0, 4, &result, buf, nullptr,
+ Env::IO_TOTAL /* rate_limiter_priority */);
+ assert(result.size() == 4);
+ *id = DecodeFixed32(buf);
+ return s;
+}
+
+void MockTableFactory::AssertSingleFile(const KVVector& file_contents) {
+ ASSERT_EQ(file_system_.files.size(), 1U);
+ ASSERT_EQ(file_contents, file_system_.files.begin()->second);
+}
+
+void MockTableFactory::AssertLatestFiles(
+ const std::vector<KVVector>& files_contents) {
+ ASSERT_GE(file_system_.files.size(), files_contents.size());
+ auto it = file_system_.files.rbegin();
+ for (auto expect = files_contents.rbegin(); expect != files_contents.rend();
+ expect++, it++) {
+ ASSERT_TRUE(it != file_system_.files.rend());
+ if (*expect != it->second) {
+ std::cout << "Wrong content! Content of file, expect:" << std::endl;
+ for (const auto& kv : *expect) {
+ ParsedInternalKey ikey;
+ std::string key, value;
+ std::tie(key, value) = kv;
+ ASSERT_OK(ParseInternalKey(Slice(key), &ikey, true /* log_err_key */));
+ std::cout << ikey.DebugString(true, false) << " -> " << value
+ << std::endl;
+ }
+ std::cout << "actual:" << std::endl;
+ for (const auto& kv : it->second) {
+ ParsedInternalKey ikey;
+ std::string key, value;
+ std::tie(key, value) = kv;
+ ASSERT_OK(ParseInternalKey(Slice(key), &ikey, true /* log_err_key */));
+ std::cout << ikey.DebugString(true, false) << " -> " << value
+ << std::endl;
+ }
+ FAIL();
+ }
+ }
+}
+
+} // namespace mock
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/mock_table.h b/src/rocksdb/table/mock_table.h
new file mode 100644
index 000000000..e4850d060
--- /dev/null
+++ b/src/rocksdb/table/mock_table.h
@@ -0,0 +1,94 @@
+// Copyright (c) 2011-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).
+#pragma once
+
+#include <algorithm>
+#include <atomic>
+#include <map>
+#include <memory>
+#include <set>
+#include <string>
+#include <utility>
+
+#include "db/version_edit.h"
+#include "port/port.h"
+#include "rocksdb/comparator.h"
+#include "rocksdb/io_status.h"
+#include "rocksdb/table.h"
+#include "table/internal_iterator.h"
+#include "table/table_builder.h"
+#include "table/table_reader.h"
+#include "test_util/testharness.h"
+#include "test_util/testutil.h"
+#include "util/kv_map.h"
+#include "util/mutexlock.h"
+
+namespace ROCKSDB_NAMESPACE {
+namespace mock {
+using KVPair = std::pair<std::string, std::string>;
+using KVVector = std::vector<KVPair>;
+
+KVVector MakeMockFile(std::initializer_list<KVPair> l = {});
+void SortKVVector(KVVector* kv_vector,
+ const Comparator* ucmp = BytewiseComparator());
+
+struct MockTableFileSystem {
+ port::Mutex mutex;
+ std::map<uint32_t, KVVector> files;
+};
+
+class MockTableFactory : public TableFactory {
+ public:
+ enum MockCorruptionMode {
+ kCorruptNone,
+ kCorruptKey,
+ kCorruptValue,
+ kCorruptReorderKey,
+ };
+
+ MockTableFactory();
+ static const char* kClassName() { return "MockTable"; }
+ const char* Name() const override { return kClassName(); }
+ using TableFactory::NewTableReader;
+ Status NewTableReader(
+ const ReadOptions& ro, const TableReaderOptions& table_reader_options,
+ std::unique_ptr<RandomAccessFileReader>&& file, uint64_t file_size,
+ std::unique_ptr<TableReader>* table_reader,
+ bool prefetch_index_and_filter_in_cache = true) const override;
+ TableBuilder* NewTableBuilder(
+ const TableBuilderOptions& table_builder_options,
+ WritableFileWriter* file) const override;
+
+ // This function will directly create mock table instead of going through
+ // MockTableBuilder. file_contents has to have a format of <internal_key,
+ // value>. Those key-value pairs will then be inserted into the mock table.
+ Status CreateMockTable(Env* env, const std::string& fname,
+ KVVector file_contents);
+
+ virtual std::string GetPrintableOptions() const override {
+ return std::string();
+ }
+
+ void SetCorruptionMode(MockCorruptionMode mode) { corrupt_mode_ = mode; }
+
+ void SetKeyValueSize(size_t size) { key_value_size_ = size; }
+ // This function will assert that only a single file exists and that the
+ // contents are equal to file_contents
+ void AssertSingleFile(const KVVector& file_contents);
+ void AssertLatestFiles(const std::vector<KVVector>& files_contents);
+
+ private:
+ Status GetAndWriteNextID(WritableFileWriter* file, uint32_t* id) const;
+ Status GetIDFromFile(RandomAccessFileReader* file, uint32_t* id) const;
+
+ mutable MockTableFileSystem file_system_;
+ mutable std::atomic<uint32_t> next_id_;
+ MockCorruptionMode corrupt_mode_;
+
+ size_t key_value_size_ = 1;
+};
+
+} // namespace mock
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/multiget_context.h b/src/rocksdb/table/multiget_context.h
new file mode 100644
index 000000000..76027a952
--- /dev/null
+++ b/src/rocksdb/table/multiget_context.h
@@ -0,0 +1,402 @@
+// Copyright (c) 2011-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).
+
+#pragma once
+#include <algorithm>
+#include <array>
+#include <string>
+
+#include "db/dbformat.h"
+#include "db/lookup_key.h"
+#include "db/merge_context.h"
+#include "rocksdb/env.h"
+#include "rocksdb/statistics.h"
+#include "rocksdb/types.h"
+#include "util/async_file_reader.h"
+#include "util/autovector.h"
+#include "util/math.h"
+#include "util/single_thread_executor.h"
+
+namespace ROCKSDB_NAMESPACE {
+class GetContext;
+
+struct KeyContext {
+ const Slice* key;
+ LookupKey* lkey;
+ Slice ukey_with_ts;
+ Slice ukey_without_ts;
+ Slice ikey;
+ ColumnFamilyHandle* column_family;
+ Status* s;
+ MergeContext merge_context;
+ SequenceNumber max_covering_tombstone_seq;
+ bool key_exists;
+ bool is_blob_index;
+ void* cb_arg;
+ PinnableSlice* value;
+ std::string* timestamp;
+ GetContext* get_context;
+
+ KeyContext(ColumnFamilyHandle* col_family, const Slice& user_key,
+ PinnableSlice* val, std::string* ts, Status* stat)
+ : key(&user_key),
+ lkey(nullptr),
+ column_family(col_family),
+ s(stat),
+ max_covering_tombstone_seq(0),
+ key_exists(false),
+ is_blob_index(false),
+ cb_arg(nullptr),
+ value(val),
+ timestamp(ts),
+ get_context(nullptr) {}
+
+ KeyContext() = default;
+};
+
+// The MultiGetContext class is a container for the sorted list of keys that
+// we need to lookup in a batch. Its main purpose is to make batch execution
+// easier by allowing various stages of the MultiGet lookups to operate on
+// subsets of keys, potentially non-contiguous. In order to accomplish this,
+// it defines the following classes -
+//
+// MultiGetContext::Range
+// MultiGetContext::Range::Iterator
+// MultiGetContext::Range::IteratorWrapper
+//
+// Here is an example of how this can be used -
+//
+// {
+// MultiGetContext ctx(...);
+// MultiGetContext::Range range = ctx.GetMultiGetRange();
+//
+// // Iterate to determine some subset of the keys
+// MultiGetContext::Range::Iterator start = range.begin();
+// MultiGetContext::Range::Iterator end = ...;
+//
+// // Make a new range with a subset of keys
+// MultiGetContext::Range subrange(range, start, end);
+//
+// // Define an auxillary vector, if needed, to hold additional data for
+// // each key
+// std::array<Foo, MultiGetContext::MAX_BATCH_SIZE> aux;
+//
+// // Iterate over the subrange and the auxillary vector simultaneously
+// MultiGetContext::Range::Iterator iter = subrange.begin();
+// for (; iter != subrange.end(); ++iter) {
+// KeyContext& key = *iter;
+// Foo& aux_key = aux_iter[iter.index()];
+// ...
+// }
+// }
+class MultiGetContext {
+ public:
+ // Limit the number of keys in a batch to this number. Benchmarks show that
+ // there is negligible benefit for batches exceeding this. Keeping this < 32
+ // simplifies iteration, as well as reduces the amount of stack allocations
+ // that need to be performed
+ static const int MAX_BATCH_SIZE = 32;
+
+ // A bitmask of at least MAX_BATCH_SIZE - 1 bits, so that
+ // Mask{1} << MAX_BATCH_SIZE is well defined
+ using Mask = uint64_t;
+ static_assert(MAX_BATCH_SIZE < sizeof(Mask) * 8);
+
+ MultiGetContext(autovector<KeyContext*, MAX_BATCH_SIZE>* sorted_keys,
+ size_t begin, size_t num_keys, SequenceNumber snapshot,
+ const ReadOptions& read_opts, FileSystem* fs,
+ Statistics* stats)
+ : num_keys_(num_keys),
+ value_mask_(0),
+ value_size_(0),
+ lookup_key_ptr_(reinterpret_cast<LookupKey*>(lookup_key_stack_buf))
+#if USE_COROUTINES
+ ,
+ reader_(fs, stats),
+ executor_(reader_)
+#endif // USE_COROUTINES
+ {
+ (void)fs;
+ (void)stats;
+ assert(num_keys <= MAX_BATCH_SIZE);
+ if (num_keys > MAX_LOOKUP_KEYS_ON_STACK) {
+ lookup_key_heap_buf.reset(new char[sizeof(LookupKey) * num_keys]);
+ lookup_key_ptr_ = reinterpret_cast<LookupKey*>(lookup_key_heap_buf.get());
+ }
+
+ for (size_t iter = 0; iter != num_keys_; ++iter) {
+ // autovector may not be contiguous storage, so make a copy
+ sorted_keys_[iter] = (*sorted_keys)[begin + iter];
+ sorted_keys_[iter]->lkey = new (&lookup_key_ptr_[iter])
+ LookupKey(*sorted_keys_[iter]->key, snapshot, read_opts.timestamp);
+ sorted_keys_[iter]->ukey_with_ts = sorted_keys_[iter]->lkey->user_key();
+ sorted_keys_[iter]->ukey_without_ts = StripTimestampFromUserKey(
+ sorted_keys_[iter]->lkey->user_key(),
+ read_opts.timestamp == nullptr ? 0 : read_opts.timestamp->size());
+ sorted_keys_[iter]->ikey = sorted_keys_[iter]->lkey->internal_key();
+ sorted_keys_[iter]->timestamp = (*sorted_keys)[begin + iter]->timestamp;
+ sorted_keys_[iter]->get_context =
+ (*sorted_keys)[begin + iter]->get_context;
+ }
+ }
+
+ ~MultiGetContext() {
+ for (size_t i = 0; i < num_keys_; ++i) {
+ lookup_key_ptr_[i].~LookupKey();
+ }
+ }
+
+#if USE_COROUTINES
+ SingleThreadExecutor& executor() { return executor_; }
+
+ AsyncFileReader& reader() { return reader_; }
+#endif // USE_COROUTINES
+
+ private:
+ static const int MAX_LOOKUP_KEYS_ON_STACK = 16;
+ alignas(
+ alignof(LookupKey)) char lookup_key_stack_buf[sizeof(LookupKey) *
+ MAX_LOOKUP_KEYS_ON_STACK];
+ std::array<KeyContext*, MAX_BATCH_SIZE> sorted_keys_;
+ size_t num_keys_;
+ Mask value_mask_;
+ uint64_t value_size_;
+ std::unique_ptr<char[]> lookup_key_heap_buf;
+ LookupKey* lookup_key_ptr_;
+#if USE_COROUTINES
+ AsyncFileReader reader_;
+ SingleThreadExecutor executor_;
+#endif // USE_COROUTINES
+
+ public:
+ // MultiGetContext::Range - Specifies a range of keys, by start and end index,
+ // from the parent MultiGetContext. Each range contains a bit vector that
+ // indicates whether the corresponding keys need to be processed or skipped.
+ // A Range object can be copy constructed, and the new object inherits the
+ // original Range's bit vector. This is useful for progressively skipping
+ // keys as the lookup goes through various stages. For example, when looking
+ // up keys in the same SST file, a Range is created excluding keys not
+ // belonging to that file. A new Range is then copy constructed and individual
+ // keys are skipped based on bloom filter lookup.
+ class Range {
+ public:
+ // MultiGetContext::Range::Iterator - A forward iterator that iterates over
+ // non-skippable keys in a Range, as well as keys whose final value has been
+ // found. The latter is tracked by MultiGetContext::value_mask_
+ class Iterator {
+ public:
+ // -- iterator traits
+ using self_type = Iterator;
+ using value_type = KeyContext;
+ using reference = KeyContext&;
+ using pointer = KeyContext*;
+ using difference_type = int;
+ using iterator_category = std::forward_iterator_tag;
+
+ Iterator(const Range* range, size_t idx)
+ : range_(range), ctx_(range->ctx_), index_(idx) {
+ while (index_ < range_->end_ &&
+ (Mask{1} << index_) &
+ (range_->ctx_->value_mask_ | range_->skip_mask_ |
+ range_->invalid_mask_))
+ index_++;
+ }
+
+ Iterator(const Iterator&) = default;
+
+ Iterator(const Iterator& other, const Range* range)
+ : range_(range), ctx_(other.ctx_), index_(other.index_) {
+ assert(range->ctx_ == other.ctx_);
+ }
+ Iterator& operator=(const Iterator&) = default;
+
+ Iterator& operator++() {
+ while (++index_ < range_->end_ &&
+ (Mask{1} << index_) &
+ (range_->ctx_->value_mask_ | range_->skip_mask_ |
+ range_->invalid_mask_))
+ ;
+ return *this;
+ }
+
+ bool operator==(Iterator other) const {
+ assert(range_->ctx_ == other.range_->ctx_);
+ return index_ == other.index_;
+ }
+
+ bool operator!=(Iterator other) const {
+ assert(range_->ctx_ == other.range_->ctx_);
+ return index_ != other.index_;
+ }
+
+ KeyContext& operator*() {
+ assert(index_ < range_->end_ && index_ >= range_->start_);
+ return *(ctx_->sorted_keys_[index_]);
+ }
+
+ KeyContext* operator->() {
+ assert(index_ < range_->end_ && index_ >= range_->start_);
+ return ctx_->sorted_keys_[index_];
+ }
+
+ size_t index() { return index_; }
+
+ private:
+ friend Range;
+ const Range* range_;
+ const MultiGetContext* ctx_;
+ size_t index_;
+ };
+
+ Range(const Range& mget_range, const Iterator& first,
+ const Iterator& last) {
+ ctx_ = mget_range.ctx_;
+ if (first == last) {
+ // This means create an empty range based on mget_range. So just
+ // set start_ and and_ to the same value
+ start_ = mget_range.start_;
+ end_ = start_;
+ } else {
+ start_ = first.index_;
+ end_ = last.index_;
+ }
+ skip_mask_ = mget_range.skip_mask_;
+ invalid_mask_ = mget_range.invalid_mask_;
+ assert(start_ < 64);
+ assert(end_ < 64);
+ }
+
+ Range() = default;
+
+ Iterator begin() const { return Iterator(this, start_); }
+
+ Iterator end() const { return Iterator(this, end_); }
+
+ bool empty() const { return RemainingMask() == 0; }
+
+ void SkipIndex(size_t index) { skip_mask_ |= Mask{1} << index; }
+
+ void SkipKey(const Iterator& iter) { SkipIndex(iter.index_); }
+
+ bool IsKeySkipped(const Iterator& iter) const {
+ return skip_mask_ & (Mask{1} << iter.index_);
+ }
+
+ // Update the value_mask_ in MultiGetContext so its
+ // immediately reflected in all the Range Iterators
+ void MarkKeyDone(Iterator& iter) {
+ ctx_->value_mask_ |= (Mask{1} << iter.index_);
+ }
+
+ bool CheckKeyDone(Iterator& iter) const {
+ return ctx_->value_mask_ & (Mask{1} << iter.index_);
+ }
+
+ uint64_t KeysLeft() const { return BitsSetToOne(RemainingMask()); }
+
+ void AddSkipsFrom(const Range& other) {
+ assert(ctx_ == other.ctx_);
+ skip_mask_ |= other.skip_mask_;
+ }
+
+ uint64_t GetValueSize() { return ctx_->value_size_; }
+
+ void AddValueSize(uint64_t value_size) { ctx_->value_size_ += value_size; }
+
+ MultiGetContext* context() const { return ctx_; }
+
+ Range Suffix(const Range& other) const {
+ size_t other_last = other.FindLastRemaining();
+ size_t my_last = FindLastRemaining();
+
+ if (my_last > other_last) {
+ return Range(*this, Iterator(this, other_last),
+ Iterator(this, my_last));
+ } else {
+ return Range(*this, begin(), begin());
+ }
+ }
+
+ // The += operator expands the number of keys in this range. The expansion
+ // is always to the right, i.e start of the additional range >= end of
+ // current range. There should be no overlap. Any skipped keys in rhs are
+ // marked as invalid in the invalid_mask_.
+ Range& operator+=(const Range& rhs) {
+ assert(rhs.start_ >= end_);
+ // Check for non-overlapping ranges and adjust invalid_mask_ accordingly
+ if (end_ < rhs.start_) {
+ invalid_mask_ |= RangeMask(end_, rhs.start_);
+ skip_mask_ |= RangeMask(end_, rhs.start_);
+ }
+ start_ = std::min<size_t>(start_, rhs.start_);
+ end_ = std::max<size_t>(end_, rhs.end_);
+ skip_mask_ |= rhs.skip_mask_ & RangeMask(rhs.start_, rhs.end_);
+ invalid_mask_ |= (rhs.invalid_mask_ | rhs.skip_mask_) &
+ RangeMask(rhs.start_, rhs.end_);
+ assert(start_ < 64);
+ assert(end_ < 64);
+ return *this;
+ }
+
+ // The -= operator removes keys from this range. The removed keys should
+ // come from a range completely overlapping the current range. The removed
+ // keys are marked invalid in the invalid_mask_.
+ Range& operator-=(const Range& rhs) {
+ assert(start_ <= rhs.start_ && end_ >= rhs.end_);
+ skip_mask_ |= (~rhs.skip_mask_ | rhs.invalid_mask_) &
+ RangeMask(rhs.start_, rhs.end_);
+ invalid_mask_ |= (~rhs.skip_mask_ | rhs.invalid_mask_) &
+ RangeMask(rhs.start_, rhs.end_);
+ return *this;
+ }
+
+ // Return a complement of the current range
+ Range operator~() {
+ Range res = *this;
+ res.skip_mask_ = ~skip_mask_ & RangeMask(start_, end_);
+ return res;
+ }
+
+ private:
+ friend MultiGetContext;
+ MultiGetContext* ctx_;
+ size_t start_;
+ size_t end_;
+ Mask skip_mask_;
+ Mask invalid_mask_;
+
+ Range(MultiGetContext* ctx, size_t num_keys)
+ : ctx_(ctx),
+ start_(0),
+ end_(num_keys),
+ skip_mask_(0),
+ invalid_mask_(0) {
+ assert(num_keys < 64);
+ }
+
+ static Mask RangeMask(size_t start, size_t end) {
+ return (((Mask{1} << (end - start)) - 1) << start);
+ }
+
+ Mask RemainingMask() const {
+ return (((Mask{1} << end_) - 1) & ~((Mask{1} << start_) - 1) &
+ ~(ctx_->value_mask_ | skip_mask_));
+ }
+
+ size_t FindLastRemaining() const {
+ Mask mask = RemainingMask();
+ size_t index = (mask >>= start_) ? start_ : 0;
+ while (mask >>= 1) {
+ index++;
+ }
+ return index;
+ }
+ };
+
+ // Return the initial range that encompasses all the keys in the batch
+ Range GetMultiGetRange() { return Range(this, num_keys_); }
+};
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/persistent_cache_helper.cc b/src/rocksdb/table/persistent_cache_helper.cc
new file mode 100644
index 000000000..eece8100e
--- /dev/null
+++ b/src/rocksdb/table/persistent_cache_helper.cc
@@ -0,0 +1,111 @@
+// Copyright (c) 2011-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 "table/persistent_cache_helper.h"
+
+#include "table/block_based/block_based_table_reader.h"
+#include "table/format.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+const PersistentCacheOptions PersistentCacheOptions::kEmpty;
+
+void PersistentCacheHelper::InsertSerialized(
+ const PersistentCacheOptions& cache_options, const BlockHandle& handle,
+ const char* data, const size_t size) {
+ assert(cache_options.persistent_cache);
+ assert(cache_options.persistent_cache->IsCompressed());
+
+ CacheKey key =
+ BlockBasedTable::GetCacheKey(cache_options.base_cache_key, handle);
+
+ cache_options.persistent_cache->Insert(key.AsSlice(), data, size)
+ .PermitUncheckedError();
+}
+
+void PersistentCacheHelper::InsertUncompressed(
+ const PersistentCacheOptions& cache_options, const BlockHandle& handle,
+ const BlockContents& contents) {
+ assert(cache_options.persistent_cache);
+ assert(!cache_options.persistent_cache->IsCompressed());
+ // Precondition:
+ // (1) content is cacheable
+ // (2) content is not compressed
+
+ CacheKey key =
+ BlockBasedTable::GetCacheKey(cache_options.base_cache_key, handle);
+
+ cache_options.persistent_cache
+ ->Insert(key.AsSlice(), contents.data.data(), contents.data.size())
+ .PermitUncheckedError();
+ ;
+}
+
+Status PersistentCacheHelper::LookupSerialized(
+ const PersistentCacheOptions& cache_options, const BlockHandle& handle,
+ std::unique_ptr<char[]>* out_data, const size_t expected_data_size) {
+#ifdef NDEBUG
+ (void)expected_data_size;
+#endif
+ assert(cache_options.persistent_cache);
+ assert(cache_options.persistent_cache->IsCompressed());
+
+ CacheKey key =
+ BlockBasedTable::GetCacheKey(cache_options.base_cache_key, handle);
+
+ size_t size;
+ Status s =
+ cache_options.persistent_cache->Lookup(key.AsSlice(), out_data, &size);
+ if (!s.ok()) {
+ // cache miss
+ RecordTick(cache_options.statistics, PERSISTENT_CACHE_MISS);
+ return s;
+ }
+
+ // cache hit
+ // Block-based table is assumed
+ assert(expected_data_size ==
+ handle.size() + BlockBasedTable::kBlockTrailerSize);
+ assert(size == expected_data_size);
+ RecordTick(cache_options.statistics, PERSISTENT_CACHE_HIT);
+ return Status::OK();
+}
+
+Status PersistentCacheHelper::LookupUncompressed(
+ const PersistentCacheOptions& cache_options, const BlockHandle& handle,
+ BlockContents* contents) {
+ assert(cache_options.persistent_cache);
+ assert(!cache_options.persistent_cache->IsCompressed());
+ if (!contents) {
+ // We shouldn't lookup in the cache. Either
+ // (1) Nowhere to store
+ return Status::NotFound();
+ }
+
+ CacheKey key =
+ BlockBasedTable::GetCacheKey(cache_options.base_cache_key, handle);
+
+ std::unique_ptr<char[]> data;
+ size_t size;
+ Status s =
+ cache_options.persistent_cache->Lookup(key.AsSlice(), &data, &size);
+ if (!s.ok()) {
+ // cache miss
+ RecordTick(cache_options.statistics, PERSISTENT_CACHE_MISS);
+ return s;
+ }
+
+ // please note we are potentially comparing compressed data size with
+ // uncompressed data size
+ assert(handle.size() <= size);
+
+ // update stats
+ RecordTick(cache_options.statistics, PERSISTENT_CACHE_HIT);
+ // construct result and return
+ *contents = BlockContents(std::move(data), size);
+ return Status::OK();
+}
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/persistent_cache_helper.h b/src/rocksdb/table/persistent_cache_helper.h
new file mode 100644
index 000000000..ece339aee
--- /dev/null
+++ b/src/rocksdb/table/persistent_cache_helper.h
@@ -0,0 +1,46 @@
+// Copyright (c) 2011-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).
+#pragma once
+
+#include <string>
+
+#include "monitoring/statistics.h"
+#include "table/format.h"
+#include "table/persistent_cache_options.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+struct BlockContents;
+
+// PersistentCacheHelper
+//
+// Encapsulates some of the helper logic for read and writing from the cache
+class PersistentCacheHelper {
+ public:
+ // Insert block into cache of serialized blocks. Size includes block trailer
+ // (if applicable).
+ static void InsertSerialized(const PersistentCacheOptions& cache_options,
+ const BlockHandle& handle, const char* data,
+ const size_t size);
+
+ // Insert block into cache of uncompressed blocks. No block trailer.
+ static void InsertUncompressed(const PersistentCacheOptions& cache_options,
+ const BlockHandle& handle,
+ const BlockContents& contents);
+
+ // Lookup block from cache of serialized blocks. Size includes block trailer
+ // (if applicable).
+ static Status LookupSerialized(const PersistentCacheOptions& cache_options,
+ const BlockHandle& handle,
+ std::unique_ptr<char[]>* out_data,
+ const size_t expected_data_size);
+
+ // Lookup block from uncompressed cache. No block trailer.
+ static Status LookupUncompressed(const PersistentCacheOptions& cache_options,
+ const BlockHandle& handle,
+ BlockContents* contents);
+};
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/persistent_cache_options.h b/src/rocksdb/table/persistent_cache_options.h
new file mode 100644
index 000000000..b543ab3a3
--- /dev/null
+++ b/src/rocksdb/table/persistent_cache_options.h
@@ -0,0 +1,34 @@
+// Copyright (c) 2011-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).
+#pragma once
+
+#include <string>
+
+#include "cache/cache_key.h"
+#include "monitoring/statistics.h"
+#include "rocksdb/persistent_cache.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+// PersistentCacheOptions
+//
+// This describe the caching behavior for page cache
+// This is used to pass the context for caching and the cache handle
+struct PersistentCacheOptions {
+ PersistentCacheOptions() {}
+ explicit PersistentCacheOptions(
+ const std::shared_ptr<PersistentCache>& _persistent_cache,
+ const OffsetableCacheKey& _base_cache_key, Statistics* const _statistics)
+ : persistent_cache(_persistent_cache),
+ base_cache_key(_base_cache_key),
+ statistics(_statistics) {}
+ std::shared_ptr<PersistentCache> persistent_cache;
+ OffsetableCacheKey base_cache_key;
+ Statistics* statistics = nullptr;
+
+ static const PersistentCacheOptions kEmpty;
+};
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/plain/plain_table_bloom.cc b/src/rocksdb/table/plain/plain_table_bloom.cc
new file mode 100644
index 000000000..21441f616
--- /dev/null
+++ b/src/rocksdb/table/plain/plain_table_bloom.cc
@@ -0,0 +1,78 @@
+// Copyright (c) 2011-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 "table/plain/plain_table_bloom.h"
+
+#include <algorithm>
+#include <string>
+
+#include "memory/allocator.h"
+#include "util/dynamic_bloom.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+namespace {
+
+uint32_t GetTotalBitsForLocality(uint32_t total_bits) {
+ uint32_t num_blocks =
+ (total_bits + CACHE_LINE_SIZE * 8 - 1) / (CACHE_LINE_SIZE * 8);
+
+ // Make num_blocks an odd number to make sure more bits are involved
+ // when determining which block.
+ if (num_blocks % 2 == 0) {
+ num_blocks++;
+ }
+
+ return num_blocks * (CACHE_LINE_SIZE * 8);
+}
+} // namespace
+
+PlainTableBloomV1::PlainTableBloomV1(uint32_t num_probes)
+ : kTotalBits(0), kNumBlocks(0), kNumProbes(num_probes), data_(nullptr) {}
+
+void PlainTableBloomV1::SetRawData(char* raw_data, uint32_t total_bits,
+ uint32_t num_blocks) {
+ data_ = raw_data;
+ kTotalBits = total_bits;
+ kNumBlocks = num_blocks;
+}
+
+void PlainTableBloomV1::SetTotalBits(Allocator* allocator, uint32_t total_bits,
+ uint32_t locality,
+ size_t huge_page_tlb_size,
+ Logger* logger) {
+ kTotalBits = (locality > 0) ? GetTotalBitsForLocality(total_bits)
+ : (total_bits + 7) / 8 * 8;
+ kNumBlocks = (locality > 0) ? (kTotalBits / (CACHE_LINE_SIZE * 8)) : 0;
+
+ assert(kNumBlocks > 0 || kTotalBits > 0);
+ assert(kNumProbes > 0);
+
+ uint32_t sz = kTotalBits / 8;
+ if (kNumBlocks > 0) {
+ sz += CACHE_LINE_SIZE - 1;
+ }
+ assert(allocator);
+
+ char* raw = allocator->AllocateAligned(sz, huge_page_tlb_size, logger);
+ memset(raw, 0, sz);
+ auto cache_line_offset = reinterpret_cast<uintptr_t>(raw) % CACHE_LINE_SIZE;
+ if (kNumBlocks > 0 && cache_line_offset > 0) {
+ raw += CACHE_LINE_SIZE - cache_line_offset;
+ }
+ data_ = raw;
+}
+
+void BloomBlockBuilder::AddKeysHashes(
+ const std::vector<uint32_t>& keys_hashes) {
+ for (auto hash : keys_hashes) {
+ bloom_.AddHash(hash);
+ }
+}
+
+Slice BloomBlockBuilder::Finish() { return bloom_.GetRawData(); }
+
+const std::string BloomBlockBuilder::kBloomBlock = "kBloomBlock";
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/plain/plain_table_bloom.h b/src/rocksdb/table/plain/plain_table_bloom.h
new file mode 100644
index 000000000..460e7ec39
--- /dev/null
+++ b/src/rocksdb/table/plain/plain_table_bloom.h
@@ -0,0 +1,132 @@
+// Copyright (c) 2011-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).
+#pragma once
+
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "port/port.h"
+#include "rocksdb/slice.h"
+#include "util/bloom_impl.h"
+#include "util/hash.h"
+#include "util/math.h"
+
+namespace ROCKSDB_NAMESPACE {
+class Slice;
+class Allocator;
+class Logger;
+
+// A legacy Bloom filter implementation used by Plain Table db format, for
+// schema backward compatibility. Not for use in new filter applications.
+class PlainTableBloomV1 {
+ public:
+ // allocator: pass allocator to bloom filter, hence trace the usage of memory
+ // total_bits: fixed total bits for the bloom
+ // num_probes: number of hash probes for a single key
+ // locality: If positive, optimize for cache line locality, 0 otherwise.
+ // hash_func: customized hash function
+ // huge_page_tlb_size: if >0, try to allocate bloom bytes from huge page TLB
+ // within this page size. Need to reserve huge pages for
+ // it to be allocated, like:
+ // sysctl -w vm.nr_hugepages=20
+ // See linux doc Documentation/vm/hugetlbpage.txt
+ explicit PlainTableBloomV1(uint32_t num_probes = 6);
+ void SetTotalBits(Allocator* allocator, uint32_t total_bits,
+ uint32_t locality, size_t huge_page_tlb_size,
+ Logger* logger);
+
+ ~PlainTableBloomV1() {}
+
+ // Assuming single threaded access to this function.
+ void AddHash(uint32_t hash);
+
+ // Multithreaded access to this function is OK
+ bool MayContainHash(uint32_t hash) const;
+
+ void Prefetch(uint32_t hash);
+
+ uint32_t GetNumBlocks() const { return kNumBlocks; }
+
+ Slice GetRawData() const { return Slice(data_, GetTotalBits() / 8); }
+
+ void SetRawData(char* raw_data, uint32_t total_bits, uint32_t num_blocks = 0);
+
+ uint32_t GetTotalBits() const { return kTotalBits; }
+
+ bool IsInitialized() const { return kNumBlocks > 0 || kTotalBits > 0; }
+
+ private:
+ uint32_t kTotalBits;
+ uint32_t kNumBlocks;
+ const uint32_t kNumProbes;
+
+ char* data_;
+
+ static constexpr int LOG2_CACHE_LINE_SIZE =
+ ConstexprFloorLog2(CACHE_LINE_SIZE);
+};
+
+#if defined(_MSC_VER)
+#pragma warning(push)
+// local variable is initialized but not referenced
+#pragma warning(disable : 4189)
+#endif
+inline void PlainTableBloomV1::Prefetch(uint32_t h) {
+ if (kNumBlocks != 0) {
+ uint32_t ignored;
+ LegacyLocalityBloomImpl</*ExtraRotates*/ true>::PrepareHashMayMatch(
+ h, kNumBlocks, data_, &ignored, LOG2_CACHE_LINE_SIZE);
+ }
+}
+#if defined(_MSC_VER)
+#pragma warning(pop)
+#endif
+
+inline bool PlainTableBloomV1::MayContainHash(uint32_t h) const {
+ assert(IsInitialized());
+ if (kNumBlocks != 0) {
+ return LegacyLocalityBloomImpl<true>::HashMayMatch(
+ h, kNumBlocks, kNumProbes, data_, LOG2_CACHE_LINE_SIZE);
+ } else {
+ return LegacyNoLocalityBloomImpl::HashMayMatch(h, kTotalBits, kNumProbes,
+ data_);
+ }
+}
+
+inline void PlainTableBloomV1::AddHash(uint32_t h) {
+ assert(IsInitialized());
+ if (kNumBlocks != 0) {
+ LegacyLocalityBloomImpl<true>::AddHash(h, kNumBlocks, kNumProbes, data_,
+ LOG2_CACHE_LINE_SIZE);
+ } else {
+ LegacyNoLocalityBloomImpl::AddHash(h, kTotalBits, kNumProbes, data_);
+ }
+}
+
+class BloomBlockBuilder {
+ public:
+ static const std::string kBloomBlock;
+
+ explicit BloomBlockBuilder(uint32_t num_probes = 6) : bloom_(num_probes) {}
+
+ void SetTotalBits(Allocator* allocator, uint32_t total_bits,
+ uint32_t locality, size_t huge_page_tlb_size,
+ Logger* logger) {
+ bloom_.SetTotalBits(allocator, total_bits, locality, huge_page_tlb_size,
+ logger);
+ }
+
+ uint32_t GetNumBlocks() const { return bloom_.GetNumBlocks(); }
+
+ void AddKeysHashes(const std::vector<uint32_t>& keys_hashes);
+
+ Slice Finish();
+
+ private:
+ PlainTableBloomV1 bloom_;
+};
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/plain/plain_table_builder.cc b/src/rocksdb/table/plain/plain_table_builder.cc
new file mode 100644
index 000000000..04723955c
--- /dev/null
+++ b/src/rocksdb/table/plain/plain_table_builder.cc
@@ -0,0 +1,337 @@
+// Copyright (c) 2011-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).
+
+#ifndef ROCKSDB_LITE
+#include "table/plain/plain_table_builder.h"
+
+#include <assert.h>
+
+#include <limits>
+#include <map>
+#include <string>
+
+#include "db/dbformat.h"
+#include "file/writable_file_writer.h"
+#include "logging/logging.h"
+#include "rocksdb/comparator.h"
+#include "rocksdb/env.h"
+#include "rocksdb/filter_policy.h"
+#include "rocksdb/options.h"
+#include "rocksdb/table.h"
+#include "table/block_based/block_builder.h"
+#include "table/format.h"
+#include "table/meta_blocks.h"
+#include "table/plain/plain_table_bloom.h"
+#include "table/plain/plain_table_factory.h"
+#include "table/plain/plain_table_index.h"
+#include "util/coding.h"
+#include "util/crc32c.h"
+#include "util/stop_watch.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+namespace {
+
+// a utility that helps writing block content to the file
+// @offset will advance if @block_contents was successfully written.
+// @block_handle the block handle this particular block.
+IOStatus WriteBlock(const Slice& block_contents, WritableFileWriter* file,
+ uint64_t* offset, BlockHandle* block_handle) {
+ block_handle->set_offset(*offset);
+ block_handle->set_size(block_contents.size());
+ IOStatus io_s = file->Append(block_contents);
+
+ if (io_s.ok()) {
+ *offset += block_contents.size();
+ }
+ return io_s;
+}
+
+} // namespace
+
+// kPlainTableMagicNumber was picked by running
+// echo rocksdb.table.plain | sha1sum
+// and taking the leading 64 bits.
+extern const uint64_t kPlainTableMagicNumber = 0x8242229663bf9564ull;
+extern const uint64_t kLegacyPlainTableMagicNumber = 0x4f3418eb7a8f13b8ull;
+
+PlainTableBuilder::PlainTableBuilder(
+ const ImmutableOptions& ioptions, const MutableCFOptions& moptions,
+ const IntTblPropCollectorFactories* int_tbl_prop_collector_factories,
+ uint32_t column_family_id, int level_at_creation, WritableFileWriter* file,
+ uint32_t user_key_len, EncodingType encoding_type, size_t index_sparseness,
+ uint32_t bloom_bits_per_key, const std::string& column_family_name,
+ uint32_t num_probes, size_t huge_page_tlb_size, double hash_table_ratio,
+ bool store_index_in_file, const std::string& db_id,
+ const std::string& db_session_id, uint64_t file_number)
+ : ioptions_(ioptions),
+ moptions_(moptions),
+ bloom_block_(num_probes),
+ file_(file),
+ bloom_bits_per_key_(bloom_bits_per_key),
+ huge_page_tlb_size_(huge_page_tlb_size),
+ encoder_(encoding_type, user_key_len, moptions.prefix_extractor.get(),
+ index_sparseness),
+ store_index_in_file_(store_index_in_file),
+ prefix_extractor_(moptions.prefix_extractor.get()) {
+ // Build index block and save it in the file if hash_table_ratio > 0
+ if (store_index_in_file_) {
+ assert(hash_table_ratio > 0 || IsTotalOrderMode());
+ index_builder_.reset(new PlainTableIndexBuilder(
+ &arena_, ioptions, moptions.prefix_extractor.get(), index_sparseness,
+ hash_table_ratio, huge_page_tlb_size_));
+ properties_
+ .user_collected_properties[PlainTablePropertyNames::kBloomVersion] =
+ "1"; // For future use
+ }
+
+ properties_.fixed_key_len = user_key_len;
+
+ // for plain table, we put all the data in a big chuck.
+ properties_.num_data_blocks = 1;
+ // Fill it later if store_index_in_file_ == true
+ properties_.index_size = 0;
+ properties_.filter_size = 0;
+ // To support roll-back to previous version, now still use version 0 for
+ // plain encoding.
+ properties_.format_version = (encoding_type == kPlain) ? 0 : 1;
+ properties_.column_family_id = column_family_id;
+ properties_.column_family_name = column_family_name;
+ properties_.db_id = db_id;
+ properties_.db_session_id = db_session_id;
+ properties_.db_host_id = ioptions.db_host_id;
+ if (!ReifyDbHostIdProperty(ioptions_.env, &properties_.db_host_id).ok()) {
+ ROCKS_LOG_INFO(ioptions_.logger, "db_host_id property will not be set");
+ }
+ properties_.orig_file_number = file_number;
+ properties_.prefix_extractor_name =
+ moptions_.prefix_extractor != nullptr
+ ? moptions_.prefix_extractor->AsString()
+ : "nullptr";
+
+ std::string val;
+ PutFixed32(&val, static_cast<uint32_t>(encoder_.GetEncodingType()));
+ properties_
+ .user_collected_properties[PlainTablePropertyNames::kEncodingType] = val;
+
+ assert(int_tbl_prop_collector_factories);
+ for (auto& factory : *int_tbl_prop_collector_factories) {
+ assert(factory);
+
+ table_properties_collectors_.emplace_back(
+ factory->CreateIntTblPropCollector(column_family_id,
+ level_at_creation));
+ }
+}
+
+PlainTableBuilder::~PlainTableBuilder() {
+ // They are supposed to have been passed to users through Finish()
+ // if the file succeeds.
+ status_.PermitUncheckedError();
+ io_status_.PermitUncheckedError();
+}
+
+void PlainTableBuilder::Add(const Slice& key, const Slice& value) {
+ // temp buffer for metadata bytes between key and value.
+ char meta_bytes_buf[6];
+ size_t meta_bytes_buf_size = 0;
+
+ ParsedInternalKey internal_key;
+ if (!ParseInternalKey(key, &internal_key, false /* log_err_key */)
+ .ok()) { // TODO
+ assert(false);
+ return;
+ }
+ if (internal_key.type == kTypeRangeDeletion) {
+ status_ = Status::NotSupported("Range deletion unsupported");
+ return;
+ }
+
+ // Store key hash
+ if (store_index_in_file_) {
+ if (moptions_.prefix_extractor == nullptr) {
+ keys_or_prefixes_hashes_.push_back(GetSliceHash(internal_key.user_key));
+ } else {
+ Slice prefix =
+ moptions_.prefix_extractor->Transform(internal_key.user_key);
+ keys_or_prefixes_hashes_.push_back(GetSliceHash(prefix));
+ }
+ }
+
+ // Write value
+ assert(offset_ <= std::numeric_limits<uint32_t>::max());
+ auto prev_offset = static_cast<uint32_t>(offset_);
+ // Write out the key
+ io_status_ = encoder_.AppendKey(key, file_, &offset_, meta_bytes_buf,
+ &meta_bytes_buf_size);
+ if (SaveIndexInFile()) {
+ index_builder_->AddKeyPrefix(GetPrefix(internal_key), prev_offset);
+ }
+
+ // Write value length
+ uint32_t value_size = static_cast<uint32_t>(value.size());
+ if (io_status_.ok()) {
+ char* end_ptr =
+ EncodeVarint32(meta_bytes_buf + meta_bytes_buf_size, value_size);
+ assert(end_ptr <= meta_bytes_buf + sizeof(meta_bytes_buf));
+ meta_bytes_buf_size = end_ptr - meta_bytes_buf;
+ io_status_ = file_->Append(Slice(meta_bytes_buf, meta_bytes_buf_size));
+ }
+
+ // Write value
+ if (io_status_.ok()) {
+ io_status_ = file_->Append(value);
+ offset_ += value_size + meta_bytes_buf_size;
+ }
+
+ if (io_status_.ok()) {
+ properties_.num_entries++;
+ properties_.raw_key_size += key.size();
+ properties_.raw_value_size += value.size();
+ if (internal_key.type == kTypeDeletion ||
+ internal_key.type == kTypeSingleDeletion) {
+ properties_.num_deletions++;
+ } else if (internal_key.type == kTypeMerge) {
+ properties_.num_merge_operands++;
+ }
+ }
+
+ // notify property collectors
+ NotifyCollectTableCollectorsOnAdd(
+ key, value, offset_, table_properties_collectors_, ioptions_.logger);
+ status_ = io_status_;
+}
+
+Status PlainTableBuilder::Finish() {
+ assert(!closed_);
+ closed_ = true;
+
+ properties_.data_size = offset_;
+
+ // Write the following blocks
+ // 1. [meta block: bloom] - optional
+ // 2. [meta block: index] - optional
+ // 3. [meta block: properties]
+ // 4. [metaindex block]
+ // 5. [footer]
+
+ MetaIndexBuilder meta_index_builer;
+
+ if (store_index_in_file_ && (properties_.num_entries > 0)) {
+ assert(properties_.num_entries <= std::numeric_limits<uint32_t>::max());
+ BlockHandle bloom_block_handle;
+ if (bloom_bits_per_key_ > 0) {
+ bloom_block_.SetTotalBits(
+ &arena_,
+ static_cast<uint32_t>(properties_.num_entries) * bloom_bits_per_key_,
+ ioptions_.bloom_locality, huge_page_tlb_size_, ioptions_.logger);
+
+ PutVarint32(&properties_.user_collected_properties
+ [PlainTablePropertyNames::kNumBloomBlocks],
+ bloom_block_.GetNumBlocks());
+
+ bloom_block_.AddKeysHashes(keys_or_prefixes_hashes_);
+
+ Slice bloom_finish_result = bloom_block_.Finish();
+
+ properties_.filter_size = bloom_finish_result.size();
+ io_status_ =
+ WriteBlock(bloom_finish_result, file_, &offset_, &bloom_block_handle);
+
+ if (!io_status_.ok()) {
+ status_ = io_status_;
+ return status_;
+ }
+ meta_index_builer.Add(BloomBlockBuilder::kBloomBlock, bloom_block_handle);
+ }
+ BlockHandle index_block_handle;
+ Slice index_finish_result = index_builder_->Finish();
+
+ properties_.index_size = index_finish_result.size();
+ io_status_ =
+ WriteBlock(index_finish_result, file_, &offset_, &index_block_handle);
+
+ if (!io_status_.ok()) {
+ status_ = io_status_;
+ return status_;
+ }
+
+ meta_index_builer.Add(PlainTableIndexBuilder::kPlainTableIndexBlock,
+ index_block_handle);
+ }
+
+ // Calculate bloom block size and index block size
+ PropertyBlockBuilder property_block_builder;
+ // -- Add basic properties
+ property_block_builder.AddTableProperty(properties_);
+
+ property_block_builder.Add(properties_.user_collected_properties);
+
+ // -- Add user collected properties
+ NotifyCollectTableCollectorsOnFinish(
+ table_properties_collectors_, ioptions_.logger, &property_block_builder);
+
+ // -- Write property block
+ BlockHandle property_block_handle;
+ IOStatus s = WriteBlock(property_block_builder.Finish(), file_, &offset_,
+ &property_block_handle);
+ if (!s.ok()) {
+ return static_cast<Status>(s);
+ }
+ meta_index_builer.Add(kPropertiesBlockName, property_block_handle);
+
+ // -- write metaindex block
+ BlockHandle metaindex_block_handle;
+ io_status_ = WriteBlock(meta_index_builer.Finish(), file_, &offset_,
+ &metaindex_block_handle);
+ if (!io_status_.ok()) {
+ status_ = io_status_;
+ return status_;
+ }
+
+ // Write Footer
+ // no need to write out new footer if we're using default checksum
+ FooterBuilder footer;
+ footer.Build(kPlainTableMagicNumber, /* format_version */ 0, offset_,
+ kNoChecksum, metaindex_block_handle);
+ io_status_ = file_->Append(footer.GetSlice());
+ if (io_status_.ok()) {
+ offset_ += footer.GetSlice().size();
+ }
+ status_ = io_status_;
+ return status_;
+}
+
+void PlainTableBuilder::Abandon() { closed_ = true; }
+
+uint64_t PlainTableBuilder::NumEntries() const {
+ return properties_.num_entries;
+}
+
+uint64_t PlainTableBuilder::FileSize() const { return offset_; }
+
+std::string PlainTableBuilder::GetFileChecksum() const {
+ if (file_ != nullptr) {
+ return file_->GetFileChecksum();
+ } else {
+ return kUnknownFileChecksum;
+ }
+}
+
+const char* PlainTableBuilder::GetFileChecksumFuncName() const {
+ if (file_ != nullptr) {
+ return file_->GetFileChecksumFuncName();
+ } else {
+ return kUnknownFileChecksumFuncName;
+ }
+}
+void PlainTableBuilder::SetSeqnoTimeTableProperties(const std::string& string,
+ uint64_t uint_64) {
+ // TODO: storing seqno to time mapping is not yet support for plain table.
+ TableBuilder::SetSeqnoTimeTableProperties(string, uint_64);
+}
+
+} // namespace ROCKSDB_NAMESPACE
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/table/plain/plain_table_builder.h b/src/rocksdb/table/plain/plain_table_builder.h
new file mode 100644
index 000000000..445491c2a
--- /dev/null
+++ b/src/rocksdb/table/plain/plain_table_builder.h
@@ -0,0 +1,154 @@
+// Copyright (c) 2011-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).
+
+#pragma once
+
+#ifndef ROCKSDB_LITE
+#include <stdint.h>
+
+#include <string>
+#include <vector>
+
+#include "db/version_edit.h"
+#include "rocksdb/options.h"
+#include "rocksdb/status.h"
+#include "rocksdb/table.h"
+#include "rocksdb/table_properties.h"
+#include "table/plain/plain_table_bloom.h"
+#include "table/plain/plain_table_index.h"
+#include "table/plain/plain_table_key_coding.h"
+#include "table/table_builder.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class BlockBuilder;
+class BlockHandle;
+class WritableFile;
+class TableBuilder;
+
+// The builder class of PlainTable. For description of PlainTable format
+// See comments of class PlainTableFactory, where instances of
+// PlainTableReader are created.
+class PlainTableBuilder : public TableBuilder {
+ public:
+ // Create a builder that will store the contents of the table it is
+ // building in *file. Does not close the file. It is up to the
+ // caller to close the file after calling Finish(). The output file
+ // will be part of level specified by 'level'. A value of -1 means
+ // that the caller does not know which level the output file will reside.
+ PlainTableBuilder(
+ const ImmutableOptions& ioptions, const MutableCFOptions& moptions,
+ const IntTblPropCollectorFactories* int_tbl_prop_collector_factories,
+ uint32_t column_family_id, int level_at_creation,
+ WritableFileWriter* file, uint32_t user_key_size,
+ EncodingType encoding_type, size_t index_sparseness,
+ uint32_t bloom_bits_per_key, const std::string& column_family_name,
+ uint32_t num_probes = 6, size_t huge_page_tlb_size = 0,
+ double hash_table_ratio = 0, bool store_index_in_file = false,
+ const std::string& db_id = "", const std::string& db_session_id = "",
+ uint64_t file_number = 0);
+ // No copying allowed
+ PlainTableBuilder(const PlainTableBuilder&) = delete;
+ void operator=(const PlainTableBuilder&) = delete;
+
+ // REQUIRES: Either Finish() or Abandon() has been called.
+ ~PlainTableBuilder();
+
+ // Add key,value to the table being constructed.
+ // REQUIRES: key is after any previously added key according to comparator.
+ // REQUIRES: Finish(), Abandon() have not been called
+ void Add(const Slice& key, const Slice& value) override;
+
+ // Return non-ok iff some error has been detected.
+ Status status() const override { return status_; }
+
+ // Return non-ok iff some error happens during IO.
+ IOStatus io_status() const override { return io_status_; }
+
+ // Finish building the table. Stops using the file passed to the
+ // constructor after this function returns.
+ // REQUIRES: Finish(), Abandon() have not been called
+ Status Finish() override;
+
+ // Indicate that the contents of this builder should be abandoned. Stops
+ // using the file passed to the constructor after this function returns.
+ // If the caller is not going to call Finish(), it must call Abandon()
+ // before destroying this builder.
+ // REQUIRES: Finish(), Abandon() have not been called
+ void Abandon() override;
+
+ // Number of calls to Add() so far.
+ uint64_t NumEntries() const override;
+
+ // Size of the file generated so far. If invoked after a successful
+ // Finish() call, returns the size of the final generated file.
+ uint64_t FileSize() const override;
+
+ TableProperties GetTableProperties() const override { return properties_; }
+
+ bool SaveIndexInFile() const { return store_index_in_file_; }
+
+ // Get file checksum
+ std::string GetFileChecksum() const override;
+
+ // Get file checksum function name
+ const char* GetFileChecksumFuncName() const override;
+
+ void SetSeqnoTimeTableProperties(const std::string& string,
+ uint64_t uint_64) override;
+
+ private:
+ Arena arena_;
+ const ImmutableOptions& ioptions_;
+ const MutableCFOptions& moptions_;
+ std::vector<std::unique_ptr<IntTblPropCollector>>
+ table_properties_collectors_;
+
+ BloomBlockBuilder bloom_block_;
+ std::unique_ptr<PlainTableIndexBuilder> index_builder_;
+
+ WritableFileWriter* file_;
+ uint64_t offset_ = 0;
+ uint32_t bloom_bits_per_key_;
+ size_t huge_page_tlb_size_;
+ Status status_;
+ IOStatus io_status_;
+ TableProperties properties_;
+ PlainTableKeyEncoder encoder_;
+
+ bool store_index_in_file_;
+
+ std::vector<uint32_t> keys_or_prefixes_hashes_;
+ bool closed_ = false; // Either Finish() or Abandon() has been called.
+
+ const SliceTransform* prefix_extractor_;
+
+ Slice GetPrefix(const Slice& target) const {
+ assert(target.size() >= 8); // target is internal key
+ return GetPrefixFromUserKey(ExtractUserKey(target));
+ }
+
+ Slice GetPrefix(const ParsedInternalKey& target) const {
+ return GetPrefixFromUserKey(target.user_key);
+ }
+
+ Slice GetPrefixFromUserKey(const Slice& user_key) const {
+ if (!IsTotalOrderMode()) {
+ return prefix_extractor_->Transform(user_key);
+ } else {
+ // Use empty slice as prefix if prefix_extractor is not set.
+ // In that case,
+ // it falls back to pure binary search and
+ // total iterator seek is supported.
+ return Slice();
+ }
+ }
+
+ bool IsTotalOrderMode() const { return (prefix_extractor_ == nullptr); }
+};
+
+} // namespace ROCKSDB_NAMESPACE
+
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/table/plain/plain_table_factory.cc b/src/rocksdb/table/plain/plain_table_factory.cc
new file mode 100644
index 000000000..dfe5241a5
--- /dev/null
+++ b/src/rocksdb/table/plain/plain_table_factory.cc
@@ -0,0 +1,350 @@
+// Copyright (c) Facebook, Inc. and its affiliates. All Rights Reserved.
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#include "table/plain/plain_table_factory.h"
+
+#include <stdint.h>
+
+#include <memory>
+
+#include "db/dbformat.h"
+#include "port/port.h"
+#include "rocksdb/convenience.h"
+#include "rocksdb/utilities/customizable_util.h"
+#include "rocksdb/utilities/object_registry.h"
+#include "rocksdb/utilities/options_type.h"
+#include "table/plain/plain_table_builder.h"
+#include "table/plain/plain_table_reader.h"
+#include "util/string_util.h"
+
+namespace ROCKSDB_NAMESPACE {
+#ifndef ROCKSDB_LITE
+static std::unordered_map<std::string, OptionTypeInfo> plain_table_type_info = {
+ {"user_key_len",
+ {offsetof(struct PlainTableOptions, user_key_len), OptionType::kUInt32T,
+ OptionVerificationType::kNormal, OptionTypeFlags::kNone}},
+ {"bloom_bits_per_key",
+ {offsetof(struct PlainTableOptions, bloom_bits_per_key), OptionType::kInt,
+ OptionVerificationType::kNormal, OptionTypeFlags::kNone}},
+ {"hash_table_ratio",
+ {offsetof(struct PlainTableOptions, hash_table_ratio), OptionType::kDouble,
+ OptionVerificationType::kNormal, OptionTypeFlags::kNone}},
+ {"index_sparseness",
+ {offsetof(struct PlainTableOptions, index_sparseness), OptionType::kSizeT,
+ OptionVerificationType::kNormal, OptionTypeFlags::kNone}},
+ {"huge_page_tlb_size",
+ {offsetof(struct PlainTableOptions, huge_page_tlb_size),
+ OptionType::kSizeT, OptionVerificationType::kNormal,
+ OptionTypeFlags::kNone}},
+ {"encoding_type",
+ {offsetof(struct PlainTableOptions, encoding_type),
+ OptionType::kEncodingType, OptionVerificationType::kNormal,
+ OptionTypeFlags::kNone}},
+ {"full_scan_mode",
+ {offsetof(struct PlainTableOptions, full_scan_mode), OptionType::kBoolean,
+ OptionVerificationType::kNormal, OptionTypeFlags::kNone}},
+ {"store_index_in_file",
+ {offsetof(struct PlainTableOptions, store_index_in_file),
+ OptionType::kBoolean, OptionVerificationType::kNormal,
+ OptionTypeFlags::kNone}},
+};
+
+PlainTableFactory::PlainTableFactory(const PlainTableOptions& options)
+ : table_options_(options) {
+ RegisterOptions(&table_options_, &plain_table_type_info);
+}
+
+Status PlainTableFactory::NewTableReader(
+ const ReadOptions& /*ro*/, const TableReaderOptions& table_reader_options,
+ std::unique_ptr<RandomAccessFileReader>&& file, uint64_t file_size,
+ std::unique_ptr<TableReader>* table,
+ bool /*prefetch_index_and_filter_in_cache*/) const {
+ return PlainTableReader::Open(
+ table_reader_options.ioptions, table_reader_options.env_options,
+ table_reader_options.internal_comparator, std::move(file), file_size,
+ table, table_options_.bloom_bits_per_key, table_options_.hash_table_ratio,
+ table_options_.index_sparseness, table_options_.huge_page_tlb_size,
+ table_options_.full_scan_mode, table_reader_options.immortal,
+ table_reader_options.prefix_extractor.get());
+}
+
+TableBuilder* PlainTableFactory::NewTableBuilder(
+ const TableBuilderOptions& table_builder_options,
+ WritableFileWriter* file) const {
+ // Ignore the skip_filters flag. PlainTable format is optimized for small
+ // in-memory dbs. The skip_filters optimization is not useful for plain
+ // tables
+ //
+ return new PlainTableBuilder(
+ table_builder_options.ioptions, table_builder_options.moptions,
+ table_builder_options.int_tbl_prop_collector_factories,
+ table_builder_options.column_family_id,
+ table_builder_options.level_at_creation, file,
+ table_options_.user_key_len, table_options_.encoding_type,
+ table_options_.index_sparseness, table_options_.bloom_bits_per_key,
+ table_builder_options.column_family_name, 6,
+ table_options_.huge_page_tlb_size, table_options_.hash_table_ratio,
+ table_options_.store_index_in_file, table_builder_options.db_id,
+ table_builder_options.db_session_id, table_builder_options.cur_file_num);
+}
+
+std::string PlainTableFactory::GetPrintableOptions() const {
+ std::string ret;
+ ret.reserve(20000);
+ const int kBufferSize = 200;
+ char buffer[kBufferSize];
+
+ snprintf(buffer, kBufferSize, " user_key_len: %u\n",
+ table_options_.user_key_len);
+ ret.append(buffer);
+ snprintf(buffer, kBufferSize, " bloom_bits_per_key: %d\n",
+ table_options_.bloom_bits_per_key);
+ ret.append(buffer);
+ snprintf(buffer, kBufferSize, " hash_table_ratio: %lf\n",
+ table_options_.hash_table_ratio);
+ ret.append(buffer);
+ snprintf(buffer, kBufferSize, " index_sparseness: %" ROCKSDB_PRIszt "\n",
+ table_options_.index_sparseness);
+ ret.append(buffer);
+ snprintf(buffer, kBufferSize, " huge_page_tlb_size: %" ROCKSDB_PRIszt "\n",
+ table_options_.huge_page_tlb_size);
+ ret.append(buffer);
+ snprintf(buffer, kBufferSize, " encoding_type: %d\n",
+ table_options_.encoding_type);
+ ret.append(buffer);
+ snprintf(buffer, kBufferSize, " full_scan_mode: %d\n",
+ table_options_.full_scan_mode);
+ ret.append(buffer);
+ snprintf(buffer, kBufferSize, " store_index_in_file: %d\n",
+ table_options_.store_index_in_file);
+ ret.append(buffer);
+ return ret;
+}
+
+Status GetPlainTableOptionsFromString(const PlainTableOptions& table_options,
+ const std::string& opts_str,
+ PlainTableOptions* new_table_options) {
+ ConfigOptions config_options;
+ config_options.input_strings_escaped = false;
+ config_options.ignore_unknown_options = false;
+ config_options.invoke_prepare_options = false;
+ return GetPlainTableOptionsFromString(config_options, table_options, opts_str,
+ new_table_options);
+}
+
+Status GetPlainTableOptionsFromString(const ConfigOptions& config_options,
+ const PlainTableOptions& table_options,
+ const std::string& opts_str,
+ PlainTableOptions* new_table_options) {
+ std::unordered_map<std::string, std::string> opts_map;
+ Status s = StringToMap(opts_str, &opts_map);
+ if (!s.ok()) {
+ return s;
+ }
+
+ s = GetPlainTableOptionsFromMap(config_options, table_options, opts_map,
+ new_table_options);
+ // Translate any errors (NotFound, NotSupported, to InvalidArgument
+ if (s.ok() || s.IsInvalidArgument()) {
+ return s;
+ } else {
+ return Status::InvalidArgument(s.getState());
+ }
+}
+#endif // ROCKSDB_LITE
+
+#ifndef ROCKSDB_LITE
+static int RegisterBuiltinMemTableRepFactory(ObjectLibrary& library,
+ const std::string& /*arg*/) {
+ // The MemTableRepFactory built-in classes will be either a class
+ // (VectorRepFactory) or a nickname (vector), followed optionally by ":#",
+ // where # is the "size" of the factory.
+ auto AsPattern = [](const std::string& name, const std::string& alt) {
+ auto pattern = ObjectLibrary::PatternEntry(name, true);
+ pattern.AnotherName(alt);
+ pattern.AddNumber(":");
+ return pattern;
+ };
+ library.AddFactory<MemTableRepFactory>(
+ AsPattern(VectorRepFactory::kClassName(), VectorRepFactory::kNickName()),
+ [](const std::string& uri, std::unique_ptr<MemTableRepFactory>* guard,
+ std::string* /*errmsg*/) {
+ auto colon = uri.find(":");
+ if (colon != std::string::npos) {
+ size_t count = ParseSizeT(uri.substr(colon + 1));
+ guard->reset(new VectorRepFactory(count));
+ } else {
+ guard->reset(new VectorRepFactory());
+ }
+ return guard->get();
+ });
+ library.AddFactory<MemTableRepFactory>(
+ AsPattern(SkipListFactory::kClassName(), SkipListFactory::kNickName()),
+ [](const std::string& uri, std::unique_ptr<MemTableRepFactory>* guard,
+ std::string* /*errmsg*/) {
+ auto colon = uri.find(":");
+ if (colon != std::string::npos) {
+ size_t lookahead = ParseSizeT(uri.substr(colon + 1));
+ guard->reset(new SkipListFactory(lookahead));
+ } else {
+ guard->reset(new SkipListFactory());
+ }
+ return guard->get();
+ });
+ library.AddFactory<MemTableRepFactory>(
+ AsPattern("HashLinkListRepFactory", "hash_linkedlist"),
+ [](const std::string& uri, std::unique_ptr<MemTableRepFactory>* guard,
+ std::string* /*errmsg*/) {
+ // Expecting format: hash_linkedlist:<hash_bucket_count>
+ auto colon = uri.find(":");
+ if (colon != std::string::npos) {
+ size_t hash_bucket_count = ParseSizeT(uri.substr(colon + 1));
+ guard->reset(NewHashLinkListRepFactory(hash_bucket_count));
+ } else {
+ guard->reset(NewHashLinkListRepFactory());
+ }
+ return guard->get();
+ });
+ library.AddFactory<MemTableRepFactory>(
+ AsPattern("HashSkipListRepFactory", "prefix_hash"),
+ [](const std::string& uri, std::unique_ptr<MemTableRepFactory>* guard,
+ std::string* /*errmsg*/) {
+ // Expecting format: prefix_hash:<hash_bucket_count>
+ auto colon = uri.find(":");
+ if (colon != std::string::npos) {
+ size_t hash_bucket_count = ParseSizeT(uri.substr(colon + 1));
+ guard->reset(NewHashSkipListRepFactory(hash_bucket_count));
+ } else {
+ guard->reset(NewHashSkipListRepFactory());
+ }
+ return guard->get();
+ });
+ library.AddFactory<MemTableRepFactory>(
+ "cuckoo",
+ [](const std::string& /*uri*/,
+ std::unique_ptr<MemTableRepFactory>* /*guard*/, std::string* errmsg) {
+ *errmsg = "cuckoo hash memtable is not supported anymore.";
+ return nullptr;
+ });
+
+ size_t num_types;
+ return static_cast<int>(library.GetFactoryCount(&num_types));
+}
+#endif // ROCKSDB_LITE
+
+Status GetMemTableRepFactoryFromString(
+ const std::string& opts_str, std::unique_ptr<MemTableRepFactory>* result) {
+ ConfigOptions config_options;
+ config_options.ignore_unsupported_options = false;
+ config_options.ignore_unknown_options = false;
+ return MemTableRepFactory::CreateFromString(config_options, opts_str, result);
+}
+
+Status MemTableRepFactory::CreateFromString(
+ const ConfigOptions& config_options, const std::string& value,
+ std::unique_ptr<MemTableRepFactory>* result) {
+#ifndef ROCKSDB_LITE
+ static std::once_flag once;
+ std::call_once(once, [&]() {
+ RegisterBuiltinMemTableRepFactory(*(ObjectLibrary::Default().get()), "");
+ });
+#endif // ROCKSDB_LITE
+ std::string id;
+ std::unordered_map<std::string, std::string> opt_map;
+ Status status = Customizable::GetOptionsMap(config_options, result->get(),
+ value, &id, &opt_map);
+ if (!status.ok()) { // GetOptionsMap failed
+ return status;
+ } else if (value.empty()) {
+ // No Id and no options. Clear the object
+ result->reset();
+ return Status::OK();
+ } else if (id.empty()) { // We have no Id but have options. Not good
+ return Status::NotSupported("Cannot reset object ", id);
+ } else {
+#ifndef ROCKSDB_LITE
+ status = NewUniqueObject<MemTableRepFactory>(config_options, id, opt_map,
+ result);
+#else
+ // To make it possible to configure the memtables in LITE mode, the ID
+ // is of the form <name>:<size>, where name is the name of the class and
+ // <size> is the length of the object (e.g. skip_list:10).
+ std::vector<std::string> opts_list = StringSplit(id, ':');
+ if (opts_list.empty() || opts_list.size() > 2 || !opt_map.empty()) {
+ status = Status::InvalidArgument("Can't parse memtable_factory option ",
+ value);
+ } else if (opts_list[0] == SkipListFactory::kNickName() ||
+ opts_list[0] == SkipListFactory::kClassName()) {
+ // Expecting format
+ // skip_list:<lookahead>
+ if (opts_list.size() == 2) {
+ size_t lookahead = ParseSizeT(opts_list[1]);
+ result->reset(new SkipListFactory(lookahead));
+ } else {
+ result->reset(new SkipListFactory());
+ }
+ } else if (!config_options.ignore_unsupported_options) {
+ status = Status::NotSupported("Cannot load object in LITE mode ", id);
+ }
+#endif // ROCKSDB_LITE
+ }
+ return status;
+}
+
+Status MemTableRepFactory::CreateFromString(
+ const ConfigOptions& config_options, const std::string& value,
+ std::shared_ptr<MemTableRepFactory>* result) {
+ std::unique_ptr<MemTableRepFactory> factory;
+ Status s = CreateFromString(config_options, value, &factory);
+ if (factory && s.ok()) {
+ result->reset(factory.release());
+ }
+ return s;
+}
+
+#ifndef ROCKSDB_LITE
+Status GetPlainTableOptionsFromMap(
+ const PlainTableOptions& table_options,
+ const std::unordered_map<std::string, std::string>& opts_map,
+ PlainTableOptions* new_table_options, bool input_strings_escaped,
+ bool ignore_unknown_options) {
+ ConfigOptions config_options;
+ config_options.input_strings_escaped = input_strings_escaped;
+ config_options.ignore_unknown_options = ignore_unknown_options;
+ return GetPlainTableOptionsFromMap(config_options, table_options, opts_map,
+ new_table_options);
+}
+
+Status GetPlainTableOptionsFromMap(
+ const ConfigOptions& config_options, const PlainTableOptions& table_options,
+ const std::unordered_map<std::string, std::string>& opts_map,
+ PlainTableOptions* new_table_options) {
+ assert(new_table_options);
+ PlainTableFactory ptf(table_options);
+ Status s = ptf.ConfigureFromMap(config_options, opts_map);
+ if (s.ok()) {
+ *new_table_options = *(ptf.GetOptions<PlainTableOptions>());
+ } else {
+ // Restore "new_options" to the default "base_options".
+ *new_table_options = table_options;
+ }
+ return s;
+}
+
+extern TableFactory* NewPlainTableFactory(const PlainTableOptions& options) {
+ return new PlainTableFactory(options);
+}
+
+const std::string PlainTablePropertyNames::kEncodingType =
+ "rocksdb.plain.table.encoding.type";
+
+const std::string PlainTablePropertyNames::kBloomVersion =
+ "rocksdb.plain.table.bloom.version";
+
+const std::string PlainTablePropertyNames::kNumBloomBlocks =
+ "rocksdb.plain.table.bloom.numblocks";
+
+#endif // ROCKSDB_LITE
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/plain/plain_table_factory.h b/src/rocksdb/table/plain/plain_table_factory.h
new file mode 100644
index 000000000..ce60b9d19
--- /dev/null
+++ b/src/rocksdb/table/plain/plain_table_factory.h
@@ -0,0 +1,182 @@
+// Copyright (c) Facebook, Inc. and its affiliates. All Rights Reserved.
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#pragma once
+
+#ifndef ROCKSDB_LITE
+#include <stdint.h>
+
+#include <memory>
+#include <string>
+
+#include "rocksdb/table.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+struct EnvOptions;
+
+class Status;
+class RandomAccessFile;
+class WritableFile;
+class Table;
+class TableBuilder;
+
+// PlainTableFactory is the entrance function to the PlainTable format of
+// SST files. It returns instances PlainTableBuilder as the builder
+// class and PlainTableReader as the reader class, where the format is
+// actually implemented.
+//
+// The PlainTable is designed for memory-mapped file systems, e.g. tmpfs.
+// Data is not organized in blocks, which allows fast access. Because of
+// following downsides
+// 1. Data compression is not supported.
+// 2. Data is not checksumed.
+// it is not recommended to use this format on other type of file systems.
+//
+// PlainTable requires fixed length key, configured as a constructor
+// parameter of the factory class. Output file format:
+// +-------------+-----------------+
+// | version | user_key_length |
+// +------------++------------+-----------------+ <= key1 offset
+// | encoded key1 | value_size | |
+// +------------+-------------+-------------+ |
+// | value1 |
+// | |
+// +--------------------------+-------------+---+ <= key2 offset
+// | encoded key2 | value_size | |
+// +------------+-------------+-------------+ |
+// | value2 |
+// | |
+// | ...... |
+// +-----------------+--------------------------+
+//
+// When the key encoding type is kPlain. Key part is encoded as:
+// +------------+--------------------+
+// | [key_size] | internal key |
+// +------------+--------------------+
+// for the case of user_key_len = kPlainTableVariableLength case,
+// and simply:
+// +----------------------+
+// | internal key |
+// +----------------------+
+// for user_key_len != kPlainTableVariableLength case.
+//
+// If key encoding type is kPrefix. Keys are encoding in this format.
+// There are three ways to encode a key:
+// (1) Full Key
+// +---------------+---------------+-------------------+
+// | Full Key Flag | Full Key Size | Full Internal Key |
+// +---------------+---------------+-------------------+
+// which simply encodes a full key
+//
+// (2) A key shared the same prefix as the previous key, which is encoded as
+// format of (1).
+// +-------------+-------------+-------------+-------------+------------+
+// | Prefix Flag | Prefix Size | Suffix Flag | Suffix Size | Key Suffix |
+// +-------------+-------------+-------------+-------------+------------+
+// where key is the suffix part of the key, including the internal bytes.
+// the actual key will be constructed by concatenating prefix part of the
+// previous key, with the suffix part of the key here, with sizes given here.
+//
+// (3) A key shared the same prefix as the previous key, which is encoded as
+// the format of (2).
+// +-----------------+-----------------+------------------------+
+// | Key Suffix Flag | Key Suffix Size | Suffix of Internal Key |
+// +-----------------+-----------------+------------------------+
+// The key will be constructed by concatenating previous key's prefix (which is
+// also a prefix which the last key encoded in the format of (1)) and the
+// key given here.
+//
+// For example, we for following keys (prefix and suffix are separated by
+// spaces):
+// 0000 0001
+// 0000 00021
+// 0000 0002
+// 00011 00
+// 0002 0001
+// Will be encoded like this:
+// FK 8 00000001
+// PF 4 SF 5 00021
+// SF 4 0002
+// FK 7 0001100
+// FK 8 00020001
+// (where FK means full key flag, PF means prefix flag and SF means suffix flag)
+//
+// All those "key flag + key size" shown above are in this format:
+// The 8 bits of the first byte:
+// +----+----+----+----+----+----+----+----+
+// | Type | Size |
+// +----+----+----+----+----+----+----+----+
+// Type indicates: full key, prefix, or suffix.
+// The last 6 bits are for size. If the size bits are not all 1, it means the
+// size of the key. Otherwise, varint32 is read after this byte. This varint
+// value + 0x3F (the value of all 1) will be the key size.
+//
+// For example, full key with length 16 will be encoded as (binary):
+// 00 010000
+// (00 means full key)
+// and a prefix with 100 bytes will be encoded as:
+// 01 111111 00100101
+// (63) (37)
+// (01 means key suffix)
+//
+// All the internal keys above (including kPlain and kPrefix) are encoded in
+// this format:
+// There are two types:
+// (1) normal internal key format
+// +----------- ...... -------------+----+---+---+---+---+---+---+---+
+// | user key |type| sequence ID |
+// +----------- ..... --------------+----+---+---+---+---+---+---+---+
+// (2) Special case for keys whose sequence ID is 0 and is value type
+// +----------- ...... -------------+----+
+// | user key |0x80|
+// +----------- ..... --------------+----+
+// To save 7 bytes for the special case where sequence ID = 0.
+//
+//
+class PlainTableFactory : public TableFactory {
+ public:
+ ~PlainTableFactory() {}
+ // user_key_len is the length of the user key. If it is set to be
+ // kPlainTableVariableLength, then it means variable length. Otherwise, all
+ // the keys need to have the fix length of this value. bloom_bits_per_key is
+ // number of bits used for bloom filer per key. hash_table_ratio is
+ // the desired utilization of the hash table used for prefix hashing.
+ // hash_table_ratio = number of prefixes / #buckets in the hash table
+ // hash_table_ratio = 0 means skip hash table but only replying on binary
+ // search.
+ // index_sparseness determines index interval for keys
+ // inside the same prefix. It will be the maximum number of linear search
+ // required after hash and binary search.
+ // index_sparseness = 0 means index for every key.
+ // huge_page_tlb_size determines whether to allocate hash indexes from huge
+ // page TLB and the page size if allocating from there. See comments of
+ // Arena::AllocateAligned() for details.
+ explicit PlainTableFactory(
+ const PlainTableOptions& _table_options = PlainTableOptions());
+
+ // Method to allow CheckedCast to work for this class
+ static const char* kClassName() { return kPlainTableName(); }
+ const char* Name() const override { return kPlainTableName(); }
+ using TableFactory::NewTableReader;
+ Status NewTableReader(const ReadOptions& ro,
+ const TableReaderOptions& table_reader_options,
+ std::unique_ptr<RandomAccessFileReader>&& file,
+ uint64_t file_size, std::unique_ptr<TableReader>* table,
+ bool prefetch_index_and_filter_in_cache) const override;
+
+ TableBuilder* NewTableBuilder(
+ const TableBuilderOptions& table_builder_options,
+ WritableFileWriter* file) const override;
+
+ std::string GetPrintableOptions() const override;
+ static const char kValueTypeSeqId0 = char(~0);
+
+ private:
+ PlainTableOptions table_options_;
+};
+
+} // namespace ROCKSDB_NAMESPACE
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/table/plain/plain_table_index.cc b/src/rocksdb/table/plain/plain_table_index.cc
new file mode 100644
index 000000000..b7e07cfb2
--- /dev/null
+++ b/src/rocksdb/table/plain/plain_table_index.cc
@@ -0,0 +1,213 @@
+// Copyright (c) 2011-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).
+
+#ifndef ROCKSDB_LITE
+#include "table/plain/plain_table_index.h"
+
+#include <cinttypes>
+
+#include "logging/logging.h"
+#include "util/coding.h"
+#include "util/hash.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+namespace {
+inline uint32_t GetBucketIdFromHash(uint32_t hash, uint32_t num_buckets) {
+ assert(num_buckets > 0);
+ return hash % num_buckets;
+}
+} // namespace
+
+Status PlainTableIndex::InitFromRawData(Slice data) {
+ if (!GetVarint32(&data, &index_size_)) {
+ return Status::Corruption("Couldn't read the index size!");
+ }
+ assert(index_size_ > 0);
+ if (!GetVarint32(&data, &num_prefixes_)) {
+ return Status::Corruption("Couldn't read the index size!");
+ }
+ sub_index_size_ =
+ static_cast<uint32_t>(data.size()) - index_size_ * kOffsetLen;
+
+ char* index_data_begin = const_cast<char*>(data.data());
+ index_ = reinterpret_cast<uint32_t*>(index_data_begin);
+ sub_index_ = reinterpret_cast<char*>(index_ + index_size_);
+ return Status::OK();
+}
+
+PlainTableIndex::IndexSearchResult PlainTableIndex::GetOffset(
+ uint32_t prefix_hash, uint32_t* bucket_value) const {
+ int bucket = GetBucketIdFromHash(prefix_hash, index_size_);
+ GetUnaligned(index_ + bucket, bucket_value);
+ if ((*bucket_value & kSubIndexMask) == kSubIndexMask) {
+ *bucket_value ^= kSubIndexMask;
+ return kSubindex;
+ }
+ if (*bucket_value >= kMaxFileSize) {
+ return kNoPrefixForBucket;
+ } else {
+ // point directly to the file
+ return kDirectToFile;
+ }
+}
+
+void PlainTableIndexBuilder::IndexRecordList::AddRecord(uint32_t hash,
+ uint32_t offset) {
+ if (num_records_in_current_group_ == kNumRecordsPerGroup) {
+ current_group_ = AllocateNewGroup();
+ num_records_in_current_group_ = 0;
+ }
+ auto& new_record = current_group_[num_records_in_current_group_++];
+ new_record.hash = hash;
+ new_record.offset = offset;
+ new_record.next = nullptr;
+}
+
+void PlainTableIndexBuilder::AddKeyPrefix(Slice key_prefix_slice,
+ uint32_t key_offset) {
+ if (is_first_record_ || prev_key_prefix_ != key_prefix_slice.ToString()) {
+ ++num_prefixes_;
+ if (!is_first_record_) {
+ keys_per_prefix_hist_.Add(num_keys_per_prefix_);
+ }
+ num_keys_per_prefix_ = 0;
+ prev_key_prefix_ = key_prefix_slice.ToString();
+ prev_key_prefix_hash_ = GetSliceHash(key_prefix_slice);
+ due_index_ = true;
+ }
+
+ if (due_index_) {
+ // Add an index key for every kIndexIntervalForSamePrefixKeys keys
+ record_list_.AddRecord(prev_key_prefix_hash_, key_offset);
+ due_index_ = false;
+ }
+
+ num_keys_per_prefix_++;
+ if (index_sparseness_ == 0 || num_keys_per_prefix_ % index_sparseness_ == 0) {
+ due_index_ = true;
+ }
+ is_first_record_ = false;
+}
+
+Slice PlainTableIndexBuilder::Finish() {
+ AllocateIndex();
+ std::vector<IndexRecord*> hash_to_offsets(index_size_, nullptr);
+ std::vector<uint32_t> entries_per_bucket(index_size_, 0);
+ BucketizeIndexes(&hash_to_offsets, &entries_per_bucket);
+
+ keys_per_prefix_hist_.Add(num_keys_per_prefix_);
+ ROCKS_LOG_INFO(ioptions_.logger, "Number of Keys per prefix Histogram: %s",
+ keys_per_prefix_hist_.ToString().c_str());
+
+ // From the temp data structure, populate indexes.
+ return FillIndexes(hash_to_offsets, entries_per_bucket);
+}
+
+void PlainTableIndexBuilder::AllocateIndex() {
+ if (prefix_extractor_ == nullptr || hash_table_ratio_ <= 0) {
+ // Fall back to pure binary search if the user fails to specify a prefix
+ // extractor.
+ index_size_ = 1;
+ } else {
+ double hash_table_size_multipier = 1.0 / hash_table_ratio_;
+ index_size_ =
+ static_cast<uint32_t>(num_prefixes_ * hash_table_size_multipier) + 1;
+ assert(index_size_ > 0);
+ }
+}
+
+void PlainTableIndexBuilder::BucketizeIndexes(
+ std::vector<IndexRecord*>* hash_to_offsets,
+ std::vector<uint32_t>* entries_per_bucket) {
+ bool first = true;
+ uint32_t prev_hash = 0;
+ size_t num_records = record_list_.GetNumRecords();
+ for (size_t i = 0; i < num_records; i++) {
+ IndexRecord* index_record = record_list_.At(i);
+ uint32_t cur_hash = index_record->hash;
+ if (first || prev_hash != cur_hash) {
+ prev_hash = cur_hash;
+ first = false;
+ }
+ uint32_t bucket = GetBucketIdFromHash(cur_hash, index_size_);
+ IndexRecord* prev_bucket_head = (*hash_to_offsets)[bucket];
+ index_record->next = prev_bucket_head;
+ (*hash_to_offsets)[bucket] = index_record;
+ (*entries_per_bucket)[bucket]++;
+ }
+
+ sub_index_size_ = 0;
+ for (auto entry_count : *entries_per_bucket) {
+ if (entry_count <= 1) {
+ continue;
+ }
+ // Only buckets with more than 1 entry will have subindex.
+ sub_index_size_ += VarintLength(entry_count);
+ // total bytes needed to store these entries' in-file offsets.
+ sub_index_size_ += entry_count * PlainTableIndex::kOffsetLen;
+ }
+}
+
+Slice PlainTableIndexBuilder::FillIndexes(
+ const std::vector<IndexRecord*>& hash_to_offsets,
+ const std::vector<uint32_t>& entries_per_bucket) {
+ ROCKS_LOG_DEBUG(ioptions_.logger,
+ "Reserving %" PRIu32 " bytes for plain table's sub_index",
+ sub_index_size_);
+ auto total_allocate_size = GetTotalSize();
+ char* allocated = arena_->AllocateAligned(
+ total_allocate_size, huge_page_tlb_size_, ioptions_.logger);
+
+ auto temp_ptr = EncodeVarint32(allocated, index_size_);
+ uint32_t* index =
+ reinterpret_cast<uint32_t*>(EncodeVarint32(temp_ptr, num_prefixes_));
+ char* sub_index = reinterpret_cast<char*>(index + index_size_);
+
+ uint32_t sub_index_offset = 0;
+ for (uint32_t i = 0; i < index_size_; i++) {
+ uint32_t num_keys_for_bucket = entries_per_bucket[i];
+ switch (num_keys_for_bucket) {
+ case 0:
+ // No key for bucket
+ PutUnaligned(index + i, (uint32_t)PlainTableIndex::kMaxFileSize);
+ break;
+ case 1:
+ // point directly to the file offset
+ PutUnaligned(index + i, hash_to_offsets[i]->offset);
+ break;
+ default:
+ // point to second level indexes.
+ PutUnaligned(index + i,
+ sub_index_offset | PlainTableIndex::kSubIndexMask);
+ char* prev_ptr = &sub_index[sub_index_offset];
+ char* cur_ptr = EncodeVarint32(prev_ptr, num_keys_for_bucket);
+ sub_index_offset += static_cast<uint32_t>(cur_ptr - prev_ptr);
+ char* sub_index_pos = &sub_index[sub_index_offset];
+ IndexRecord* record = hash_to_offsets[i];
+ int j;
+ for (j = num_keys_for_bucket - 1; j >= 0 && record;
+ j--, record = record->next) {
+ EncodeFixed32(sub_index_pos + j * sizeof(uint32_t), record->offset);
+ }
+ assert(j == -1 && record == nullptr);
+ sub_index_offset += PlainTableIndex::kOffsetLen * num_keys_for_bucket;
+ assert(sub_index_offset <= sub_index_size_);
+ break;
+ }
+ }
+ assert(sub_index_offset == sub_index_size_);
+
+ ROCKS_LOG_DEBUG(ioptions_.logger,
+ "hash table size: %" PRIu32 ", suffix_map length %" PRIu32,
+ index_size_, sub_index_size_);
+ return Slice(allocated, GetTotalSize());
+}
+
+const std::string PlainTableIndexBuilder::kPlainTableIndexBlock =
+ "PlainTableIndexBlock";
+} // namespace ROCKSDB_NAMESPACE
+
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/table/plain/plain_table_index.h b/src/rocksdb/table/plain/plain_table_index.h
new file mode 100644
index 000000000..9f5f0eeff
--- /dev/null
+++ b/src/rocksdb/table/plain/plain_table_index.h
@@ -0,0 +1,248 @@
+// Copyright (c) 2011-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).
+
+#pragma once
+
+#ifndef ROCKSDB_LITE
+
+#include <string>
+#include <vector>
+
+#include "memory/arena.h"
+#include "monitoring/histogram.h"
+#include "options/cf_options.h"
+#include "rocksdb/options.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+// The file contains two classes PlainTableIndex and PlainTableIndexBuilder
+// The two classes implement the index format of PlainTable.
+// For description of PlainTable format, see comments of class
+// PlainTableFactory
+//
+//
+// PlainTableIndex contains buckets size of index_size_, each is a
+// 32-bit integer. The lower 31 bits contain an offset value (explained below)
+// and the first bit of the integer indicates type of the offset.
+//
+// +--------------+------------------------------------------------------+
+// | Flag (1 bit) | Offset to binary search buffer or file (31 bits) +
+// +--------------+------------------------------------------------------+
+//
+// Explanation for the "flag bit":
+//
+// 0 indicates that the bucket contains only one prefix (no conflict when
+// hashing this prefix), whose first row starts from this offset of the
+// file.
+// 1 indicates that the bucket contains more than one prefixes, or there
+// are too many rows for one prefix so we need a binary search for it. In
+// this case, the offset indicates the offset of sub_index_ holding the
+// binary search indexes of keys for those rows. Those binary search indexes
+// are organized in this way:
+//
+// The first 4 bytes, indicate how many indexes (N) are stored after it. After
+// it, there are N 32-bit integers, each points of an offset of the file,
+// which
+// points to starting of a row. Those offsets need to be guaranteed to be in
+// ascending order so the keys they are pointing to are also in ascending
+// order
+// to make sure we can use them to do binary searches. Below is visual
+// presentation of a bucket.
+//
+// <begin>
+// number_of_records: varint32
+// record 1 file offset: fixedint32
+// record 2 file offset: fixedint32
+// ....
+// record N file offset: fixedint32
+// <end>
+
+// The class loads the index block from a PlainTable SST file, and executes
+// the index lookup.
+// The class is used by PlainTableReader class.
+class PlainTableIndex {
+ public:
+ enum IndexSearchResult {
+ kNoPrefixForBucket = 0,
+ kDirectToFile = 1,
+ kSubindex = 2
+ };
+
+ explicit PlainTableIndex(Slice data) { InitFromRawData(data); }
+
+ PlainTableIndex()
+ : index_size_(0),
+ sub_index_size_(0),
+ num_prefixes_(0),
+ index_(nullptr),
+ sub_index_(nullptr) {}
+
+ // The function that executes the lookup the hash table.
+ // The hash key is `prefix_hash`. The function fills the hash bucket
+ // content in `bucket_value`, which is up to the caller to interpret.
+ IndexSearchResult GetOffset(uint32_t prefix_hash,
+ uint32_t* bucket_value) const;
+
+ // Initialize data from `index_data`, which points to raw data for
+ // index stored in the SST file.
+ Status InitFromRawData(Slice index_data);
+
+ // Decode the sub index for specific hash bucket.
+ // The `offset` is the value returned as `bucket_value` by GetOffset()
+ // and is only valid when the return value is `kSubindex`.
+ // The return value is the pointer to the starting address of the
+ // sub-index. `upper_bound` is filled with the value indicating how many
+ // entries the sub-index has.
+ const char* GetSubIndexBasePtrAndUpperBound(uint32_t offset,
+ uint32_t* upper_bound) const {
+ const char* index_ptr = &sub_index_[offset];
+ return GetVarint32Ptr(index_ptr, index_ptr + 4, upper_bound);
+ }
+
+ uint32_t GetIndexSize() const { return index_size_; }
+
+ uint32_t GetSubIndexSize() const { return sub_index_size_; }
+
+ uint32_t GetNumPrefixes() const { return num_prefixes_; }
+
+ static const uint64_t kMaxFileSize = (1u << 31) - 1;
+ static const uint32_t kSubIndexMask = 0x80000000;
+ static const size_t kOffsetLen = sizeof(uint32_t);
+
+ private:
+ uint32_t index_size_;
+ uint32_t sub_index_size_;
+ uint32_t num_prefixes_;
+
+ uint32_t* index_;
+ char* sub_index_;
+};
+
+// PlainTableIndexBuilder is used to create plain table index.
+// After calling Finish(), it returns Slice, which is usually
+// used either to initialize PlainTableIndex or
+// to save index to sst file.
+// For more details about the index, please refer to:
+// https://github.com/facebook/rocksdb/wiki/PlainTable-Format
+// #wiki-in-memory-index-format
+// The class is used by PlainTableBuilder class.
+class PlainTableIndexBuilder {
+ public:
+ PlainTableIndexBuilder(Arena* arena, const ImmutableOptions& ioptions,
+ const SliceTransform* prefix_extractor,
+ size_t index_sparseness, double hash_table_ratio,
+ size_t huge_page_tlb_size)
+ : arena_(arena),
+ ioptions_(ioptions),
+ record_list_(kRecordsPerGroup),
+ is_first_record_(true),
+ due_index_(false),
+ num_prefixes_(0),
+ num_keys_per_prefix_(0),
+ prev_key_prefix_hash_(0),
+ index_sparseness_(index_sparseness),
+ index_size_(0),
+ sub_index_size_(0),
+ prefix_extractor_(prefix_extractor),
+ hash_table_ratio_(hash_table_ratio),
+ huge_page_tlb_size_(huge_page_tlb_size) {}
+
+ void AddKeyPrefix(Slice key_prefix_slice, uint32_t key_offset);
+
+ Slice Finish();
+
+ uint32_t GetTotalSize() const {
+ return VarintLength(index_size_) + VarintLength(num_prefixes_) +
+ PlainTableIndex::kOffsetLen * index_size_ + sub_index_size_;
+ }
+
+ static const std::string kPlainTableIndexBlock;
+
+ private:
+ struct IndexRecord {
+ uint32_t hash; // hash of the prefix
+ uint32_t offset; // offset of a row
+ IndexRecord* next;
+ };
+
+ // Helper class to track all the index records
+ class IndexRecordList {
+ public:
+ explicit IndexRecordList(size_t num_records_per_group)
+ : kNumRecordsPerGroup(num_records_per_group),
+ current_group_(nullptr),
+ num_records_in_current_group_(num_records_per_group) {}
+
+ ~IndexRecordList() {
+ for (size_t i = 0; i < groups_.size(); i++) {
+ delete[] groups_[i];
+ }
+ }
+
+ void AddRecord(uint32_t hash, uint32_t offset);
+
+ size_t GetNumRecords() const {
+ return (groups_.size() - 1) * kNumRecordsPerGroup +
+ num_records_in_current_group_;
+ }
+ IndexRecord* At(size_t index) {
+ return &(
+ groups_[index / kNumRecordsPerGroup][index % kNumRecordsPerGroup]);
+ }
+
+ private:
+ IndexRecord* AllocateNewGroup() {
+ IndexRecord* result = new IndexRecord[kNumRecordsPerGroup];
+ groups_.push_back(result);
+ return result;
+ }
+
+ // Each group in `groups_` contains fix-sized records (determined by
+ // kNumRecordsPerGroup). Which can help us minimize the cost if resizing
+ // occurs.
+ const size_t kNumRecordsPerGroup;
+ IndexRecord* current_group_;
+ // List of arrays allocated
+ std::vector<IndexRecord*> groups_;
+ size_t num_records_in_current_group_;
+ };
+
+ void AllocateIndex();
+
+ // Internal helper function to bucket index record list to hash buckets.
+ void BucketizeIndexes(std::vector<IndexRecord*>* hash_to_offsets,
+ std::vector<uint32_t>* entries_per_bucket);
+
+ // Internal helper class to fill the indexes and bloom filters to internal
+ // data structures.
+ Slice FillIndexes(const std::vector<IndexRecord*>& hash_to_offsets,
+ const std::vector<uint32_t>& entries_per_bucket);
+
+ Arena* arena_;
+ const ImmutableOptions ioptions_;
+ HistogramImpl keys_per_prefix_hist_;
+ IndexRecordList record_list_;
+ bool is_first_record_;
+ bool due_index_;
+ uint32_t num_prefixes_;
+ uint32_t num_keys_per_prefix_;
+
+ uint32_t prev_key_prefix_hash_;
+ size_t index_sparseness_;
+ uint32_t index_size_;
+ uint32_t sub_index_size_;
+
+ const SliceTransform* prefix_extractor_;
+ double hash_table_ratio_;
+ size_t huge_page_tlb_size_;
+
+ std::string prev_key_prefix_;
+
+ static const size_t kRecordsPerGroup = 256;
+};
+
+} // namespace ROCKSDB_NAMESPACE
+
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/table/plain/plain_table_key_coding.cc b/src/rocksdb/table/plain/plain_table_key_coding.cc
new file mode 100644
index 000000000..800d8d76f
--- /dev/null
+++ b/src/rocksdb/table/plain/plain_table_key_coding.cc
@@ -0,0 +1,509 @@
+// Copyright (c) 2011-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).
+
+#ifndef ROCKSDB_LITE
+#include "table/plain/plain_table_key_coding.h"
+
+#include <algorithm>
+#include <string>
+
+#include "db/dbformat.h"
+#include "file/writable_file_writer.h"
+#include "table/plain/plain_table_factory.h"
+#include "table/plain/plain_table_reader.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+enum PlainTableEntryType : unsigned char {
+ kFullKey = 0,
+ kPrefixFromPreviousKey = 1,
+ kKeySuffix = 2,
+};
+
+namespace {
+
+// Control byte:
+// First two bits indicate type of entry
+// Other bytes are inlined sizes. If all bits are 1 (0x03F), overflow bytes
+// are used. key_size-0x3F will be encoded as a variint32 after this bytes.
+
+const unsigned char kSizeInlineLimit = 0x3F;
+
+// Return 0 for error
+size_t EncodeSize(PlainTableEntryType type, uint32_t key_size,
+ char* out_buffer) {
+ out_buffer[0] = type << 6;
+
+ if (key_size < static_cast<uint32_t>(kSizeInlineLimit)) {
+ // size inlined
+ out_buffer[0] |= static_cast<char>(key_size);
+ return 1;
+ } else {
+ out_buffer[0] |= kSizeInlineLimit;
+ char* ptr = EncodeVarint32(out_buffer + 1, key_size - kSizeInlineLimit);
+ return ptr - out_buffer;
+ }
+}
+} // namespace
+
+// Fill bytes_read with number of bytes read.
+inline Status PlainTableKeyDecoder::DecodeSize(uint32_t start_offset,
+ PlainTableEntryType* entry_type,
+ uint32_t* key_size,
+ uint32_t* bytes_read) {
+ Slice next_byte_slice;
+ bool success = file_reader_.Read(start_offset, 1, &next_byte_slice);
+ if (!success) {
+ return file_reader_.status();
+ }
+ *entry_type = static_cast<PlainTableEntryType>(
+ (static_cast<unsigned char>(next_byte_slice[0]) & ~kSizeInlineLimit) >>
+ 6);
+ char inline_key_size = next_byte_slice[0] & kSizeInlineLimit;
+ if (inline_key_size < kSizeInlineLimit) {
+ *key_size = inline_key_size;
+ *bytes_read = 1;
+ return Status::OK();
+ } else {
+ uint32_t extra_size;
+ uint32_t tmp_bytes_read;
+ success = file_reader_.ReadVarint32(start_offset + 1, &extra_size,
+ &tmp_bytes_read);
+ if (!success) {
+ return file_reader_.status();
+ }
+ assert(tmp_bytes_read > 0);
+ *key_size = kSizeInlineLimit + extra_size;
+ *bytes_read = tmp_bytes_read + 1;
+ return Status::OK();
+ }
+}
+
+IOStatus PlainTableKeyEncoder::AppendKey(const Slice& key,
+ WritableFileWriter* file,
+ uint64_t* offset, char* meta_bytes_buf,
+ size_t* meta_bytes_buf_size) {
+ ParsedInternalKey parsed_key;
+ Status pik_status =
+ ParseInternalKey(key, &parsed_key, false /* log_err_key */); // TODO
+ if (!pik_status.ok()) {
+ return IOStatus::Corruption(pik_status.getState());
+ }
+
+ Slice key_to_write = key; // Portion of internal key to write out.
+
+ uint32_t user_key_size = static_cast<uint32_t>(key.size() - 8);
+ if (encoding_type_ == kPlain) {
+ if (fixed_user_key_len_ == kPlainTableVariableLength) {
+ // Write key length
+ char key_size_buf[5]; // tmp buffer for key size as varint32
+ char* ptr = EncodeVarint32(key_size_buf, user_key_size);
+ assert(ptr <= key_size_buf + sizeof(key_size_buf));
+ auto len = ptr - key_size_buf;
+ IOStatus io_s = file->Append(Slice(key_size_buf, len));
+ if (!io_s.ok()) {
+ return io_s;
+ }
+ *offset += len;
+ }
+ } else {
+ assert(encoding_type_ == kPrefix);
+ char size_bytes[12];
+ size_t size_bytes_pos = 0;
+
+ Slice prefix =
+ prefix_extractor_->Transform(Slice(key.data(), user_key_size));
+ if (key_count_for_prefix_ == 0 || prefix != pre_prefix_.GetUserKey() ||
+ key_count_for_prefix_ % index_sparseness_ == 0) {
+ key_count_for_prefix_ = 1;
+ pre_prefix_.SetUserKey(prefix);
+ size_bytes_pos += EncodeSize(kFullKey, user_key_size, size_bytes);
+ IOStatus io_s = file->Append(Slice(size_bytes, size_bytes_pos));
+ if (!io_s.ok()) {
+ return io_s;
+ }
+ *offset += size_bytes_pos;
+ } else {
+ key_count_for_prefix_++;
+ if (key_count_for_prefix_ == 2) {
+ // For second key within a prefix, need to encode prefix length
+ size_bytes_pos +=
+ EncodeSize(kPrefixFromPreviousKey,
+ static_cast<uint32_t>(pre_prefix_.GetUserKey().size()),
+ size_bytes + size_bytes_pos);
+ }
+ uint32_t prefix_len =
+ static_cast<uint32_t>(pre_prefix_.GetUserKey().size());
+ size_bytes_pos += EncodeSize(kKeySuffix, user_key_size - prefix_len,
+ size_bytes + size_bytes_pos);
+ IOStatus io_s = file->Append(Slice(size_bytes, size_bytes_pos));
+ if (!io_s.ok()) {
+ return io_s;
+ }
+ *offset += size_bytes_pos;
+ key_to_write = Slice(key.data() + prefix_len, key.size() - prefix_len);
+ }
+ }
+
+ // Encode full key
+ // For value size as varint32 (up to 5 bytes).
+ // If the row is of value type with seqId 0, flush the special flag together
+ // in this buffer to safe one file append call, which takes 1 byte.
+ if (parsed_key.sequence == 0 && parsed_key.type == kTypeValue) {
+ IOStatus io_s =
+ file->Append(Slice(key_to_write.data(), key_to_write.size() - 8));
+ if (!io_s.ok()) {
+ return io_s;
+ }
+ *offset += key_to_write.size() - 8;
+ meta_bytes_buf[*meta_bytes_buf_size] = PlainTableFactory::kValueTypeSeqId0;
+ *meta_bytes_buf_size += 1;
+ } else {
+ IOStatus io_s = file->Append(key_to_write);
+ if (!io_s.ok()) {
+ return io_s;
+ }
+ *offset += key_to_write.size();
+ }
+
+ return IOStatus::OK();
+}
+
+Slice PlainTableFileReader::GetFromBuffer(Buffer* buffer, uint32_t file_offset,
+ uint32_t len) {
+ assert(file_offset + len <= file_info_->data_end_offset);
+ return Slice(buffer->buf.get() + (file_offset - buffer->buf_start_offset),
+ len);
+}
+
+bool PlainTableFileReader::ReadNonMmap(uint32_t file_offset, uint32_t len,
+ Slice* out) {
+ const uint32_t kPrefetchSize = 256u;
+
+ // Try to read from buffers.
+ for (uint32_t i = 0; i < num_buf_; i++) {
+ Buffer* buffer = buffers_[num_buf_ - 1 - i].get();
+ if (file_offset >= buffer->buf_start_offset &&
+ file_offset + len <= buffer->buf_start_offset + buffer->buf_len) {
+ *out = GetFromBuffer(buffer, file_offset, len);
+ return true;
+ }
+ }
+
+ Buffer* new_buffer;
+ // Data needed is not in any of the buffer. Allocate a new buffer.
+ if (num_buf_ < buffers_.size()) {
+ // Add a new buffer
+ new_buffer = new Buffer();
+ buffers_[num_buf_++].reset(new_buffer);
+ } else {
+ // Now simply replace the last buffer. Can improve the placement policy
+ // if needed.
+ new_buffer = buffers_[num_buf_ - 1].get();
+ }
+
+ assert(file_offset + len <= file_info_->data_end_offset);
+ uint32_t size_to_read = std::min(file_info_->data_end_offset - file_offset,
+ std::max(kPrefetchSize, len));
+ if (size_to_read > new_buffer->buf_capacity) {
+ new_buffer->buf.reset(new char[size_to_read]);
+ new_buffer->buf_capacity = size_to_read;
+ new_buffer->buf_len = 0;
+ }
+ Slice read_result;
+ // TODO: rate limit plain table reads.
+ Status s =
+ file_info_->file->Read(IOOptions(), file_offset, size_to_read,
+ &read_result, new_buffer->buf.get(), nullptr,
+ Env::IO_TOTAL /* rate_limiter_priority */);
+ if (!s.ok()) {
+ status_ = s;
+ return false;
+ }
+ new_buffer->buf_start_offset = file_offset;
+ new_buffer->buf_len = size_to_read;
+ *out = GetFromBuffer(new_buffer, file_offset, len);
+ return true;
+}
+
+inline bool PlainTableFileReader::ReadVarint32(uint32_t offset, uint32_t* out,
+ uint32_t* bytes_read) {
+ if (file_info_->is_mmap_mode) {
+ const char* start = file_info_->file_data.data() + offset;
+ const char* limit =
+ file_info_->file_data.data() + file_info_->data_end_offset;
+ const char* key_ptr = GetVarint32Ptr(start, limit, out);
+ assert(key_ptr != nullptr);
+ *bytes_read = static_cast<uint32_t>(key_ptr - start);
+ return true;
+ } else {
+ return ReadVarint32NonMmap(offset, out, bytes_read);
+ }
+}
+
+bool PlainTableFileReader::ReadVarint32NonMmap(uint32_t offset, uint32_t* out,
+ uint32_t* bytes_read) {
+ const char* start;
+ const char* limit;
+ const uint32_t kMaxVarInt32Size = 6u;
+ uint32_t bytes_to_read =
+ std::min(file_info_->data_end_offset - offset, kMaxVarInt32Size);
+ Slice bytes;
+ if (!Read(offset, bytes_to_read, &bytes)) {
+ return false;
+ }
+ start = bytes.data();
+ limit = bytes.data() + bytes.size();
+
+ const char* key_ptr = GetVarint32Ptr(start, limit, out);
+ *bytes_read =
+ (key_ptr != nullptr) ? static_cast<uint32_t>(key_ptr - start) : 0;
+ return true;
+}
+
+Status PlainTableKeyDecoder::ReadInternalKey(
+ uint32_t file_offset, uint32_t user_key_size, ParsedInternalKey* parsed_key,
+ uint32_t* bytes_read, bool* internal_key_valid, Slice* internal_key) {
+ Slice tmp_slice;
+ bool success = file_reader_.Read(file_offset, user_key_size + 1, &tmp_slice);
+ if (!success) {
+ return file_reader_.status();
+ }
+ if (tmp_slice[user_key_size] == PlainTableFactory::kValueTypeSeqId0) {
+ // Special encoding for the row with seqID=0
+ parsed_key->user_key = Slice(tmp_slice.data(), user_key_size);
+ parsed_key->sequence = 0;
+ parsed_key->type = kTypeValue;
+ *bytes_read += user_key_size + 1;
+ *internal_key_valid = false;
+ } else {
+ success = file_reader_.Read(file_offset, user_key_size + 8, internal_key);
+ if (!success) {
+ return file_reader_.status();
+ }
+ *internal_key_valid = true;
+ Status pik_status = ParseInternalKey(*internal_key, parsed_key,
+ false /* log_err_key */); // TODO
+ if (!pik_status.ok()) {
+ return Status::Corruption(
+ Slice("Corrupted key found during next key read. "),
+ pik_status.getState());
+ }
+ *bytes_read += user_key_size + 8;
+ }
+ return Status::OK();
+}
+
+Status PlainTableKeyDecoder::NextPlainEncodingKey(uint32_t start_offset,
+ ParsedInternalKey* parsed_key,
+ Slice* internal_key,
+ uint32_t* bytes_read,
+ bool* /*seekable*/) {
+ uint32_t user_key_size = 0;
+ Status s;
+ if (fixed_user_key_len_ != kPlainTableVariableLength) {
+ user_key_size = fixed_user_key_len_;
+ } else {
+ uint32_t tmp_size = 0;
+ uint32_t tmp_read;
+ bool success =
+ file_reader_.ReadVarint32(start_offset, &tmp_size, &tmp_read);
+ if (!success) {
+ return file_reader_.status();
+ }
+ assert(tmp_read > 0);
+ user_key_size = tmp_size;
+ *bytes_read = tmp_read;
+ }
+ // dummy initial value to avoid compiler complain
+ bool decoded_internal_key_valid = true;
+ Slice decoded_internal_key;
+ s = ReadInternalKey(start_offset + *bytes_read, user_key_size, parsed_key,
+ bytes_read, &decoded_internal_key_valid,
+ &decoded_internal_key);
+ if (!s.ok()) {
+ return s;
+ }
+ if (!file_reader_.file_info()->is_mmap_mode) {
+ cur_key_.SetInternalKey(*parsed_key);
+ parsed_key->user_key =
+ Slice(cur_key_.GetInternalKey().data(), user_key_size);
+ if (internal_key != nullptr) {
+ *internal_key = cur_key_.GetInternalKey();
+ }
+ } else if (internal_key != nullptr) {
+ if (decoded_internal_key_valid) {
+ *internal_key = decoded_internal_key;
+ } else {
+ // Need to copy out the internal key
+ cur_key_.SetInternalKey(*parsed_key);
+ *internal_key = cur_key_.GetInternalKey();
+ }
+ }
+ return Status::OK();
+}
+
+Status PlainTableKeyDecoder::NextPrefixEncodingKey(
+ uint32_t start_offset, ParsedInternalKey* parsed_key, Slice* internal_key,
+ uint32_t* bytes_read, bool* seekable) {
+ PlainTableEntryType entry_type;
+
+ bool expect_suffix = false;
+ Status s;
+ do {
+ uint32_t size = 0;
+ // dummy initial value to avoid compiler complain
+ bool decoded_internal_key_valid = true;
+ uint32_t my_bytes_read = 0;
+ s = DecodeSize(start_offset + *bytes_read, &entry_type, &size,
+ &my_bytes_read);
+ if (!s.ok()) {
+ return s;
+ }
+ if (my_bytes_read == 0) {
+ return Status::Corruption("Unexpected EOF when reading size of the key");
+ }
+ *bytes_read += my_bytes_read;
+
+ switch (entry_type) {
+ case kFullKey: {
+ expect_suffix = false;
+ Slice decoded_internal_key;
+ s = ReadInternalKey(start_offset + *bytes_read, size, parsed_key,
+ bytes_read, &decoded_internal_key_valid,
+ &decoded_internal_key);
+ if (!s.ok()) {
+ return s;
+ }
+ if (!file_reader_.file_info()->is_mmap_mode ||
+ (internal_key != nullptr && !decoded_internal_key_valid)) {
+ // In non-mmap mode, always need to make a copy of keys returned to
+ // users, because after reading value for the key, the key might
+ // be invalid.
+ cur_key_.SetInternalKey(*parsed_key);
+ saved_user_key_ = cur_key_.GetUserKey();
+ if (!file_reader_.file_info()->is_mmap_mode) {
+ parsed_key->user_key =
+ Slice(cur_key_.GetInternalKey().data(), size);
+ }
+ if (internal_key != nullptr) {
+ *internal_key = cur_key_.GetInternalKey();
+ }
+ } else {
+ if (internal_key != nullptr) {
+ *internal_key = decoded_internal_key;
+ }
+ saved_user_key_ = parsed_key->user_key;
+ }
+ break;
+ }
+ case kPrefixFromPreviousKey: {
+ if (seekable != nullptr) {
+ *seekable = false;
+ }
+ prefix_len_ = size;
+ assert(prefix_extractor_ == nullptr ||
+ prefix_extractor_->Transform(saved_user_key_).size() ==
+ prefix_len_);
+ // Need read another size flag for suffix
+ expect_suffix = true;
+ break;
+ }
+ case kKeySuffix: {
+ expect_suffix = false;
+ if (seekable != nullptr) {
+ *seekable = false;
+ }
+
+ Slice tmp_slice;
+ s = ReadInternalKey(start_offset + *bytes_read, size, parsed_key,
+ bytes_read, &decoded_internal_key_valid,
+ &tmp_slice);
+ if (!s.ok()) {
+ return s;
+ }
+ if (!file_reader_.file_info()->is_mmap_mode) {
+ // In non-mmap mode, we need to make a copy of keys returned to
+ // users, because after reading value for the key, the key might
+ // be invalid.
+ // saved_user_key_ points to cur_key_. We are making a copy of
+ // the prefix part to another string, and construct the current
+ // key from the prefix part and the suffix part back to cur_key_.
+ std::string tmp =
+ Slice(saved_user_key_.data(), prefix_len_).ToString();
+ cur_key_.Reserve(prefix_len_ + size);
+ cur_key_.SetInternalKey(tmp, *parsed_key);
+ parsed_key->user_key =
+ Slice(cur_key_.GetInternalKey().data(), prefix_len_ + size);
+ saved_user_key_ = cur_key_.GetUserKey();
+ } else {
+ cur_key_.Reserve(prefix_len_ + size);
+ cur_key_.SetInternalKey(Slice(saved_user_key_.data(), prefix_len_),
+ *parsed_key);
+ }
+ parsed_key->user_key = cur_key_.GetUserKey();
+ if (internal_key != nullptr) {
+ *internal_key = cur_key_.GetInternalKey();
+ }
+ break;
+ }
+ default:
+ return Status::Corruption("Un-identified size flag.");
+ }
+ } while (expect_suffix); // Another round if suffix is expected.
+ return Status::OK();
+}
+
+Status PlainTableKeyDecoder::NextKey(uint32_t start_offset,
+ ParsedInternalKey* parsed_key,
+ Slice* internal_key, Slice* value,
+ uint32_t* bytes_read, bool* seekable) {
+ assert(value != nullptr);
+ Status s = NextKeyNoValue(start_offset, parsed_key, internal_key, bytes_read,
+ seekable);
+ if (s.ok()) {
+ assert(bytes_read != nullptr);
+ uint32_t value_size;
+ uint32_t value_size_bytes;
+ bool success = file_reader_.ReadVarint32(start_offset + *bytes_read,
+ &value_size, &value_size_bytes);
+ if (!success) {
+ return file_reader_.status();
+ }
+ if (value_size_bytes == 0) {
+ return Status::Corruption(
+ "Unexpected EOF when reading the next value's size.");
+ }
+ *bytes_read += value_size_bytes;
+ success = file_reader_.Read(start_offset + *bytes_read, value_size, value);
+ if (!success) {
+ return file_reader_.status();
+ }
+ *bytes_read += value_size;
+ }
+ return s;
+}
+
+Status PlainTableKeyDecoder::NextKeyNoValue(uint32_t start_offset,
+ ParsedInternalKey* parsed_key,
+ Slice* internal_key,
+ uint32_t* bytes_read,
+ bool* seekable) {
+ *bytes_read = 0;
+ if (seekable != nullptr) {
+ *seekable = true;
+ }
+ if (encoding_type_ == kPlain) {
+ return NextPlainEncodingKey(start_offset, parsed_key, internal_key,
+ bytes_read, seekable);
+ } else {
+ assert(encoding_type_ == kPrefix);
+ return NextPrefixEncodingKey(start_offset, parsed_key, internal_key,
+ bytes_read, seekable);
+ }
+}
+
+} // namespace ROCKSDB_NAMESPACE
+#endif // ROCKSDB_LIT
diff --git a/src/rocksdb/table/plain/plain_table_key_coding.h b/src/rocksdb/table/plain/plain_table_key_coding.h
new file mode 100644
index 000000000..9cda7df32
--- /dev/null
+++ b/src/rocksdb/table/plain/plain_table_key_coding.h
@@ -0,0 +1,201 @@
+// Copyright (c) 2011-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).
+
+#pragma once
+
+#ifndef ROCKSDB_LITE
+
+#include <array>
+
+#include "rocksdb/slice.h"
+#include "table/plain/plain_table_reader.h"
+
+// The file contains three helper classes of PlainTable format,
+// PlainTableKeyEncoder, PlainTableKeyDecoder and PlainTableFileReader.
+// These classes issue the lowest level of operations of PlainTable.
+// Actual data format of the key is documented in comments of class
+// PlainTableFactory.
+namespace ROCKSDB_NAMESPACE {
+
+class WritableFile;
+struct ParsedInternalKey;
+struct PlainTableReaderFileInfo;
+enum PlainTableEntryType : unsigned char;
+
+// Helper class for PlainTable format to write out a key to an output file
+// The class is used in PlainTableBuilder.
+class PlainTableKeyEncoder {
+ public:
+ explicit PlainTableKeyEncoder(EncodingType encoding_type,
+ uint32_t user_key_len,
+ const SliceTransform* prefix_extractor,
+ size_t index_sparseness)
+ : encoding_type_((prefix_extractor != nullptr) ? encoding_type : kPlain),
+ fixed_user_key_len_(user_key_len),
+ prefix_extractor_(prefix_extractor),
+ index_sparseness_((index_sparseness > 1) ? index_sparseness : 1),
+ key_count_for_prefix_(0) {}
+ // key: the key to write out, in the format of internal key.
+ // file: the output file to write out
+ // offset: offset in the file. Needs to be updated after appending bytes
+ // for the key
+ // meta_bytes_buf: buffer for extra meta bytes
+ // meta_bytes_buf_size: offset to append extra meta bytes. Will be updated
+ // if meta_bytes_buf is updated.
+ IOStatus AppendKey(const Slice& key, WritableFileWriter* file,
+ uint64_t* offset, char* meta_bytes_buf,
+ size_t* meta_bytes_buf_size);
+
+ // Return actual encoding type to be picked
+ EncodingType GetEncodingType() { return encoding_type_; }
+
+ private:
+ EncodingType encoding_type_;
+ uint32_t fixed_user_key_len_;
+ const SliceTransform* prefix_extractor_;
+ const size_t index_sparseness_;
+ size_t key_count_for_prefix_;
+ IterKey pre_prefix_;
+};
+
+// The class does raw file reads for PlainTableReader.
+// It hides whether it is a mmap-read, or a non-mmap read.
+// The class is implemented in a way to favor the performance of mmap case.
+// The class is used by PlainTableReader.
+class PlainTableFileReader {
+ public:
+ explicit PlainTableFileReader(const PlainTableReaderFileInfo* _file_info)
+ : file_info_(_file_info), num_buf_(0) {}
+
+ ~PlainTableFileReader() {
+ // Should fix.
+ status_.PermitUncheckedError();
+ }
+
+ // In mmaped mode, the results point to mmaped area of the file, which
+ // means it is always valid before closing the file.
+ // In non-mmap mode, the results point to an internal buffer. If the caller
+ // makes another read call, the results may not be valid. So callers should
+ // make a copy when needed.
+ // In order to save read calls to files, we keep two internal buffers:
+ // the first read and the most recent read. This is efficient because it
+ // columns these two common use cases:
+ // (1) hash index only identify one location, we read the key to verify
+ // the location, and read key and value if it is the right location.
+ // (2) after hash index checking, we identify two locations (because of
+ // hash bucket conflicts), we binary search the two location to see
+ // which one is what we need and start to read from the location.
+ // These two most common use cases will be covered by the two buffers
+ // so that we don't need to re-read the same location.
+ // Currently we keep a fixed size buffer. If a read doesn't exactly fit
+ // the buffer, we replace the second buffer with the location user reads.
+ //
+ // If return false, status code is stored in status_.
+ bool Read(uint32_t file_offset, uint32_t len, Slice* out) {
+ if (file_info_->is_mmap_mode) {
+ assert(file_offset + len <= file_info_->data_end_offset);
+ *out = Slice(file_info_->file_data.data() + file_offset, len);
+ return true;
+ } else {
+ return ReadNonMmap(file_offset, len, out);
+ }
+ }
+
+ // If return false, status code is stored in status_.
+ bool ReadNonMmap(uint32_t file_offset, uint32_t len, Slice* output);
+
+ // *bytes_read = 0 means eof. false means failure and status is saved
+ // in status_. Not directly returning Status to save copying status
+ // object to map previous performance of mmap mode.
+ inline bool ReadVarint32(uint32_t offset, uint32_t* output,
+ uint32_t* bytes_read);
+
+ bool ReadVarint32NonMmap(uint32_t offset, uint32_t* output,
+ uint32_t* bytes_read);
+
+ Status status() const { return status_; }
+
+ const PlainTableReaderFileInfo* file_info() { return file_info_; }
+
+ private:
+ const PlainTableReaderFileInfo* file_info_;
+
+ struct Buffer {
+ Buffer() : buf_start_offset(0), buf_len(0), buf_capacity(0) {}
+ std::unique_ptr<char[]> buf;
+ uint32_t buf_start_offset;
+ uint32_t buf_len;
+ uint32_t buf_capacity;
+ };
+
+ // Keep buffers for two recent reads.
+ std::array<std::unique_ptr<Buffer>, 2> buffers_;
+ uint32_t num_buf_;
+ Status status_;
+
+ Slice GetFromBuffer(Buffer* buf, uint32_t file_offset, uint32_t len);
+};
+
+// A helper class to decode keys from input buffer
+// The class is used by PlainTableBuilder.
+class PlainTableKeyDecoder {
+ public:
+ explicit PlainTableKeyDecoder(const PlainTableReaderFileInfo* file_info,
+ EncodingType encoding_type,
+ uint32_t user_key_len,
+ const SliceTransform* prefix_extractor)
+ : file_reader_(file_info),
+ encoding_type_(encoding_type),
+ prefix_len_(0),
+ fixed_user_key_len_(user_key_len),
+ prefix_extractor_(prefix_extractor),
+ in_prefix_(false) {}
+
+ // Find the next key.
+ // start: char array where the key starts.
+ // limit: boundary of the char array
+ // parsed_key: the output of the result key
+ // internal_key: if not null, fill with the output of the result key in
+ // un-parsed format
+ // bytes_read: how many bytes read from start. Output
+ // seekable: whether key can be read from this place. Used when building
+ // indexes. Output.
+ Status NextKey(uint32_t start_offset, ParsedInternalKey* parsed_key,
+ Slice* internal_key, Slice* value, uint32_t* bytes_read,
+ bool* seekable = nullptr);
+
+ Status NextKeyNoValue(uint32_t start_offset, ParsedInternalKey* parsed_key,
+ Slice* internal_key, uint32_t* bytes_read,
+ bool* seekable = nullptr);
+
+ PlainTableFileReader file_reader_;
+ EncodingType encoding_type_;
+ uint32_t prefix_len_;
+ uint32_t fixed_user_key_len_;
+ Slice saved_user_key_;
+ IterKey cur_key_;
+ const SliceTransform* prefix_extractor_;
+ bool in_prefix_;
+
+ private:
+ Status NextPlainEncodingKey(uint32_t start_offset,
+ ParsedInternalKey* parsed_key,
+ Slice* internal_key, uint32_t* bytes_read,
+ bool* seekable = nullptr);
+ Status NextPrefixEncodingKey(uint32_t start_offset,
+ ParsedInternalKey* parsed_key,
+ Slice* internal_key, uint32_t* bytes_read,
+ bool* seekable = nullptr);
+ Status ReadInternalKey(uint32_t file_offset, uint32_t user_key_size,
+ ParsedInternalKey* parsed_key, uint32_t* bytes_read,
+ bool* internal_key_valid, Slice* internal_key);
+ inline Status DecodeSize(uint32_t start_offset,
+ PlainTableEntryType* entry_type, uint32_t* key_size,
+ uint32_t* bytes_read);
+};
+
+} // namespace ROCKSDB_NAMESPACE
+
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/table/plain/plain_table_reader.cc b/src/rocksdb/table/plain/plain_table_reader.cc
new file mode 100644
index 000000000..6ce3d0ab9
--- /dev/null
+++ b/src/rocksdb/table/plain/plain_table_reader.cc
@@ -0,0 +1,765 @@
+// Copyright (c) Facebook, Inc. and its affiliates. All Rights Reserved.
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#ifndef ROCKSDB_LITE
+
+#include "table/plain/plain_table_reader.h"
+
+#include <string>
+#include <vector>
+
+#include "db/dbformat.h"
+#include "memory/arena.h"
+#include "monitoring/histogram.h"
+#include "monitoring/perf_context_imp.h"
+#include "rocksdb/cache.h"
+#include "rocksdb/comparator.h"
+#include "rocksdb/env.h"
+#include "rocksdb/filter_policy.h"
+#include "rocksdb/options.h"
+#include "rocksdb/statistics.h"
+#include "table/block_based/block.h"
+#include "table/block_based/filter_block.h"
+#include "table/format.h"
+#include "table/get_context.h"
+#include "table/internal_iterator.h"
+#include "table/meta_blocks.h"
+#include "table/plain/plain_table_bloom.h"
+#include "table/plain/plain_table_factory.h"
+#include "table/plain/plain_table_key_coding.h"
+#include "table/two_level_iterator.h"
+#include "util/coding.h"
+#include "util/dynamic_bloom.h"
+#include "util/hash.h"
+#include "util/stop_watch.h"
+#include "util/string_util.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+namespace {
+
+// Safely getting a uint32_t element from a char array, where, starting from
+// `base`, every 4 bytes are considered as an fixed 32 bit integer.
+inline uint32_t GetFixed32Element(const char* base, size_t offset) {
+ return DecodeFixed32(base + offset * sizeof(uint32_t));
+}
+} // namespace
+
+// Iterator to iterate IndexedTable
+class PlainTableIterator : public InternalIterator {
+ public:
+ explicit PlainTableIterator(PlainTableReader* table, bool use_prefix_seek);
+ // No copying allowed
+ PlainTableIterator(const PlainTableIterator&) = delete;
+ void operator=(const Iterator&) = delete;
+
+ ~PlainTableIterator() override;
+
+ bool Valid() const override;
+
+ void SeekToFirst() override;
+
+ void SeekToLast() override;
+
+ void Seek(const Slice& target) override;
+
+ void SeekForPrev(const Slice& target) override;
+
+ void Next() override;
+
+ void Prev() override;
+
+ Slice key() const override;
+
+ Slice value() const override;
+
+ Status status() const override;
+
+ private:
+ PlainTableReader* table_;
+ PlainTableKeyDecoder decoder_;
+ bool use_prefix_seek_;
+ uint32_t offset_;
+ uint32_t next_offset_;
+ Slice key_;
+ Slice value_;
+ Status status_;
+};
+
+extern const uint64_t kPlainTableMagicNumber;
+PlainTableReader::PlainTableReader(
+ const ImmutableOptions& ioptions,
+ std::unique_ptr<RandomAccessFileReader>&& file,
+ const EnvOptions& storage_options, const InternalKeyComparator& icomparator,
+ EncodingType encoding_type, uint64_t file_size,
+ const TableProperties* table_properties,
+ const SliceTransform* prefix_extractor)
+ : internal_comparator_(icomparator),
+ encoding_type_(encoding_type),
+ full_scan_mode_(false),
+ user_key_len_(static_cast<uint32_t>(table_properties->fixed_key_len)),
+ prefix_extractor_(prefix_extractor),
+ enable_bloom_(false),
+ bloom_(6),
+ file_info_(std::move(file), storage_options,
+ static_cast<uint32_t>(table_properties->data_size)),
+ ioptions_(ioptions),
+ file_size_(file_size),
+ table_properties_(nullptr) {}
+
+PlainTableReader::~PlainTableReader() {
+ // Should fix?
+ status_.PermitUncheckedError();
+}
+
+Status PlainTableReader::Open(
+ const ImmutableOptions& ioptions, const EnvOptions& env_options,
+ const InternalKeyComparator& internal_comparator,
+ std::unique_ptr<RandomAccessFileReader>&& file, uint64_t file_size,
+ std::unique_ptr<TableReader>* table_reader, const int bloom_bits_per_key,
+ double hash_table_ratio, size_t index_sparseness, size_t huge_page_tlb_size,
+ bool full_scan_mode, const bool immortal_table,
+ const SliceTransform* prefix_extractor) {
+ if (file_size > PlainTableIndex::kMaxFileSize) {
+ return Status::NotSupported("File is too large for PlainTableReader!");
+ }
+
+ std::unique_ptr<TableProperties> props;
+ auto s = ReadTableProperties(file.get(), file_size, kPlainTableMagicNumber,
+ ioptions, &props);
+ if (!s.ok()) {
+ return s;
+ }
+
+ assert(hash_table_ratio >= 0.0);
+ auto& user_props = props->user_collected_properties;
+ auto prefix_extractor_in_file = props->prefix_extractor_name;
+
+ if (!full_scan_mode &&
+ !prefix_extractor_in_file.empty() /* old version sst file*/
+ && prefix_extractor_in_file != "nullptr") {
+ if (!prefix_extractor) {
+ return Status::InvalidArgument(
+ "Prefix extractor is missing when opening a PlainTable built "
+ "using a prefix extractor");
+ } else if (prefix_extractor_in_file != prefix_extractor->AsString()) {
+ return Status::InvalidArgument(
+ "Prefix extractor given doesn't match the one used to build "
+ "PlainTable");
+ }
+ }
+
+ EncodingType encoding_type = kPlain;
+ auto encoding_type_prop =
+ user_props.find(PlainTablePropertyNames::kEncodingType);
+ if (encoding_type_prop != user_props.end()) {
+ encoding_type = static_cast<EncodingType>(
+ DecodeFixed32(encoding_type_prop->second.c_str()));
+ }
+
+ std::unique_ptr<PlainTableReader> new_reader(new PlainTableReader(
+ ioptions, std::move(file), env_options, internal_comparator,
+ encoding_type, file_size, props.get(), prefix_extractor));
+
+ s = new_reader->MmapDataIfNeeded();
+ if (!s.ok()) {
+ return s;
+ }
+
+ if (!full_scan_mode) {
+ s = new_reader->PopulateIndex(props.get(), bloom_bits_per_key,
+ hash_table_ratio, index_sparseness,
+ huge_page_tlb_size);
+ if (!s.ok()) {
+ return s;
+ }
+ } else {
+ // Flag to indicate it is a full scan mode so that none of the indexes
+ // can be used.
+ new_reader->full_scan_mode_ = true;
+ }
+ // PopulateIndex can add to the props, so don't store them until now
+ new_reader->table_properties_ = std::move(props);
+
+ if (immortal_table && new_reader->file_info_.is_mmap_mode) {
+ new_reader->dummy_cleanable_.reset(new Cleanable());
+ }
+
+ *table_reader = std::move(new_reader);
+ return s;
+}
+
+void PlainTableReader::SetupForCompaction() {}
+
+InternalIterator* PlainTableReader::NewIterator(
+ const ReadOptions& options, const SliceTransform* /* prefix_extractor */,
+ Arena* arena, bool /*skip_filters*/, TableReaderCaller /*caller*/,
+ size_t /*compaction_readahead_size*/, bool /* allow_unprepared_value */) {
+ // Not necessarily used here, but make sure this has been initialized
+ assert(table_properties_);
+
+ // Auto prefix mode is not implemented in PlainTable.
+ bool use_prefix_seek = !IsTotalOrderMode() && !options.total_order_seek &&
+ !options.auto_prefix_mode;
+ if (arena == nullptr) {
+ return new PlainTableIterator(this, use_prefix_seek);
+ } else {
+ auto mem = arena->AllocateAligned(sizeof(PlainTableIterator));
+ return new (mem) PlainTableIterator(this, use_prefix_seek);
+ }
+}
+
+Status PlainTableReader::PopulateIndexRecordList(
+ PlainTableIndexBuilder* index_builder,
+ std::vector<uint32_t>* prefix_hashes) {
+ Slice prev_key_prefix_slice;
+ std::string prev_key_prefix_buf;
+ uint32_t pos = data_start_offset_;
+
+ bool is_first_record = true;
+ Slice key_prefix_slice;
+ PlainTableKeyDecoder decoder(&file_info_, encoding_type_, user_key_len_,
+ prefix_extractor_);
+ while (pos < file_info_.data_end_offset) {
+ uint32_t key_offset = pos;
+ ParsedInternalKey key;
+ Slice value_slice;
+ bool seekable = false;
+ Status s = Next(&decoder, &pos, &key, nullptr, &value_slice, &seekable);
+ if (!s.ok()) {
+ return s;
+ }
+
+ key_prefix_slice = GetPrefix(key);
+ if (enable_bloom_) {
+ bloom_.AddHash(GetSliceHash(key.user_key));
+ } else {
+ if (is_first_record || prev_key_prefix_slice != key_prefix_slice) {
+ if (!is_first_record) {
+ prefix_hashes->push_back(GetSliceHash(prev_key_prefix_slice));
+ }
+ if (file_info_.is_mmap_mode) {
+ prev_key_prefix_slice = key_prefix_slice;
+ } else {
+ prev_key_prefix_buf = key_prefix_slice.ToString();
+ prev_key_prefix_slice = prev_key_prefix_buf;
+ }
+ }
+ }
+
+ index_builder->AddKeyPrefix(GetPrefix(key), key_offset);
+
+ if (!seekable && is_first_record) {
+ return Status::Corruption("Key for a prefix is not seekable");
+ }
+
+ is_first_record = false;
+ }
+
+ prefix_hashes->push_back(GetSliceHash(key_prefix_slice));
+ auto s = index_.InitFromRawData(index_builder->Finish());
+ return s;
+}
+
+void PlainTableReader::AllocateBloom(int bloom_bits_per_key, int num_keys,
+ size_t huge_page_tlb_size) {
+ uint32_t bloom_total_bits = num_keys * bloom_bits_per_key;
+ if (bloom_total_bits > 0) {
+ enable_bloom_ = true;
+ bloom_.SetTotalBits(&arena_, bloom_total_bits, ioptions_.bloom_locality,
+ huge_page_tlb_size, ioptions_.logger);
+ }
+}
+
+void PlainTableReader::FillBloom(const std::vector<uint32_t>& prefix_hashes) {
+ assert(bloom_.IsInitialized());
+ for (const auto prefix_hash : prefix_hashes) {
+ bloom_.AddHash(prefix_hash);
+ }
+}
+
+Status PlainTableReader::MmapDataIfNeeded() {
+ if (file_info_.is_mmap_mode) {
+ // Get mmapped memory.
+ return file_info_.file->Read(
+ IOOptions(), 0, static_cast<size_t>(file_size_), &file_info_.file_data,
+ nullptr, nullptr, Env::IO_TOTAL /* rate_limiter_priority */);
+ }
+ return Status::OK();
+}
+
+Status PlainTableReader::PopulateIndex(TableProperties* props,
+ int bloom_bits_per_key,
+ double hash_table_ratio,
+ size_t index_sparseness,
+ size_t huge_page_tlb_size) {
+ assert(props != nullptr);
+
+ BlockContents index_block_contents;
+ Status s = ReadMetaBlock(file_info_.file.get(), nullptr /* prefetch_buffer */,
+ file_size_, kPlainTableMagicNumber, ioptions_,
+ PlainTableIndexBuilder::kPlainTableIndexBlock,
+ BlockType::kIndex, &index_block_contents);
+
+ bool index_in_file = s.ok();
+
+ BlockContents bloom_block_contents;
+ bool bloom_in_file = false;
+ // We only need to read the bloom block if index block is in file.
+ if (index_in_file) {
+ s = ReadMetaBlock(file_info_.file.get(), nullptr /* prefetch_buffer */,
+ file_size_, kPlainTableMagicNumber, ioptions_,
+ BloomBlockBuilder::kBloomBlock, BlockType::kFilter,
+ &bloom_block_contents);
+ bloom_in_file = s.ok() && bloom_block_contents.data.size() > 0;
+ }
+
+ Slice* bloom_block;
+ if (bloom_in_file) {
+ // If bloom_block_contents.allocation is not empty (which will be the case
+ // for non-mmap mode), it holds the alloated memory for the bloom block.
+ // It needs to be kept alive to keep `bloom_block` valid.
+ bloom_block_alloc_ = std::move(bloom_block_contents.allocation);
+ bloom_block = &bloom_block_contents.data;
+ } else {
+ bloom_block = nullptr;
+ }
+
+ Slice* index_block;
+ if (index_in_file) {
+ // If index_block_contents.allocation is not empty (which will be the case
+ // for non-mmap mode), it holds the alloated memory for the index block.
+ // It needs to be kept alive to keep `index_block` valid.
+ index_block_alloc_ = std::move(index_block_contents.allocation);
+ index_block = &index_block_contents.data;
+ } else {
+ index_block = nullptr;
+ }
+
+ if ((prefix_extractor_ == nullptr) && (hash_table_ratio != 0)) {
+ // moptions.prefix_extractor is requried for a hash-based look-up.
+ return Status::NotSupported(
+ "PlainTable requires a prefix extractor enable prefix hash mode.");
+ }
+
+ // First, read the whole file, for every kIndexIntervalForSamePrefixKeys rows
+ // for a prefix (starting from the first one), generate a record of (hash,
+ // offset) and append it to IndexRecordList, which is a data structure created
+ // to store them.
+
+ if (!index_in_file) {
+ // Allocate bloom filter here for total order mode.
+ if (IsTotalOrderMode()) {
+ AllocateBloom(bloom_bits_per_key,
+ static_cast<uint32_t>(props->num_entries),
+ huge_page_tlb_size);
+ }
+ } else if (bloom_in_file) {
+ enable_bloom_ = true;
+ auto num_blocks_property = props->user_collected_properties.find(
+ PlainTablePropertyNames::kNumBloomBlocks);
+
+ uint32_t num_blocks = 0;
+ if (num_blocks_property != props->user_collected_properties.end()) {
+ Slice temp_slice(num_blocks_property->second);
+ if (!GetVarint32(&temp_slice, &num_blocks)) {
+ num_blocks = 0;
+ }
+ }
+ // cast away const qualifier, because bloom_ won't be changed
+ bloom_.SetRawData(const_cast<char*>(bloom_block->data()),
+ static_cast<uint32_t>(bloom_block->size()) * 8,
+ num_blocks);
+ } else {
+ // Index in file but no bloom in file. Disable bloom filter in this case.
+ enable_bloom_ = false;
+ bloom_bits_per_key = 0;
+ }
+
+ PlainTableIndexBuilder index_builder(&arena_, ioptions_, prefix_extractor_,
+ index_sparseness, hash_table_ratio,
+ huge_page_tlb_size);
+
+ std::vector<uint32_t> prefix_hashes;
+ if (!index_in_file) {
+ // Populates _bloom if enabled (total order mode)
+ s = PopulateIndexRecordList(&index_builder, &prefix_hashes);
+ if (!s.ok()) {
+ return s;
+ }
+ } else {
+ s = index_.InitFromRawData(*index_block);
+ if (!s.ok()) {
+ return s;
+ }
+ }
+
+ if (!index_in_file) {
+ if (!IsTotalOrderMode()) {
+ // Calculated bloom filter size and allocate memory for
+ // bloom filter based on the number of prefixes, then fill it.
+ AllocateBloom(bloom_bits_per_key, index_.GetNumPrefixes(),
+ huge_page_tlb_size);
+ if (enable_bloom_) {
+ FillBloom(prefix_hashes);
+ }
+ }
+ }
+
+ // Fill two table properties.
+ if (!index_in_file) {
+ props->user_collected_properties["plain_table_hash_table_size"] =
+ std::to_string(index_.GetIndexSize() * PlainTableIndex::kOffsetLen);
+ props->user_collected_properties["plain_table_sub_index_size"] =
+ std::to_string(index_.GetSubIndexSize());
+ } else {
+ props->user_collected_properties["plain_table_hash_table_size"] =
+ std::to_string(0);
+ props->user_collected_properties["plain_table_sub_index_size"] =
+ std::to_string(0);
+ }
+
+ return Status::OK();
+}
+
+Status PlainTableReader::GetOffset(PlainTableKeyDecoder* decoder,
+ const Slice& target, const Slice& prefix,
+ uint32_t prefix_hash, bool& prefix_matched,
+ uint32_t* offset) const {
+ prefix_matched = false;
+ uint32_t prefix_index_offset;
+ auto res = index_.GetOffset(prefix_hash, &prefix_index_offset);
+ if (res == PlainTableIndex::kNoPrefixForBucket) {
+ *offset = file_info_.data_end_offset;
+ return Status::OK();
+ } else if (res == PlainTableIndex::kDirectToFile) {
+ *offset = prefix_index_offset;
+ return Status::OK();
+ }
+
+ // point to sub-index, need to do a binary search
+ uint32_t upper_bound = 0;
+ const char* base_ptr =
+ index_.GetSubIndexBasePtrAndUpperBound(prefix_index_offset, &upper_bound);
+ uint32_t low = 0;
+ uint32_t high = upper_bound;
+ ParsedInternalKey mid_key;
+ ParsedInternalKey parsed_target;
+ Status s = ParseInternalKey(target, &parsed_target,
+ false /* log_err_key */); // TODO
+ if (!s.ok()) return s;
+
+ // The key is between [low, high). Do a binary search between it.
+ while (high - low > 1) {
+ uint32_t mid = (high + low) / 2;
+ uint32_t file_offset = GetFixed32Element(base_ptr, mid);
+ uint32_t tmp;
+ s = decoder->NextKeyNoValue(file_offset, &mid_key, nullptr, &tmp);
+ if (!s.ok()) {
+ return s;
+ }
+ int cmp_result = internal_comparator_.Compare(mid_key, parsed_target);
+ if (cmp_result < 0) {
+ low = mid;
+ } else {
+ if (cmp_result == 0) {
+ // Happen to have found the exact key or target is smaller than the
+ // first key after base_offset.
+ prefix_matched = true;
+ *offset = file_offset;
+ return Status::OK();
+ } else {
+ high = mid;
+ }
+ }
+ }
+ // Both of the key at the position low or low+1 could share the same
+ // prefix as target. We need to rule out one of them to avoid to go
+ // to the wrong prefix.
+ ParsedInternalKey low_key;
+ uint32_t tmp;
+ uint32_t low_key_offset = GetFixed32Element(base_ptr, low);
+ s = decoder->NextKeyNoValue(low_key_offset, &low_key, nullptr, &tmp);
+ if (!s.ok()) {
+ return s;
+ }
+
+ if (GetPrefix(low_key) == prefix) {
+ prefix_matched = true;
+ *offset = low_key_offset;
+ } else if (low + 1 < upper_bound) {
+ // There is possible a next prefix, return it
+ prefix_matched = false;
+ *offset = GetFixed32Element(base_ptr, low + 1);
+ } else {
+ // target is larger than a key of the last prefix in this bucket
+ // but with a different prefix. Key does not exist.
+ *offset = file_info_.data_end_offset;
+ }
+ return Status::OK();
+}
+
+bool PlainTableReader::MatchBloom(uint32_t hash) const {
+ if (!enable_bloom_) {
+ return true;
+ }
+
+ if (bloom_.MayContainHash(hash)) {
+ PERF_COUNTER_ADD(bloom_sst_hit_count, 1);
+ return true;
+ } else {
+ PERF_COUNTER_ADD(bloom_sst_miss_count, 1);
+ return false;
+ }
+}
+
+Status PlainTableReader::Next(PlainTableKeyDecoder* decoder, uint32_t* offset,
+ ParsedInternalKey* parsed_key,
+ Slice* internal_key, Slice* value,
+ bool* seekable) const {
+ if (*offset == file_info_.data_end_offset) {
+ *offset = file_info_.data_end_offset;
+ return Status::OK();
+ }
+
+ if (*offset > file_info_.data_end_offset) {
+ return Status::Corruption("Offset is out of file size");
+ }
+
+ uint32_t bytes_read;
+ Status s = decoder->NextKey(*offset, parsed_key, internal_key, value,
+ &bytes_read, seekable);
+ if (!s.ok()) {
+ return s;
+ }
+ *offset = *offset + bytes_read;
+ return Status::OK();
+}
+
+void PlainTableReader::Prepare(const Slice& target) {
+ if (enable_bloom_) {
+ uint32_t prefix_hash = GetSliceHash(GetPrefix(target));
+ bloom_.Prefetch(prefix_hash);
+ }
+}
+
+Status PlainTableReader::Get(const ReadOptions& /*ro*/, const Slice& target,
+ GetContext* get_context,
+ const SliceTransform* /* prefix_extractor */,
+ bool /*skip_filters*/) {
+ // Check bloom filter first.
+ Slice prefix_slice;
+ uint32_t prefix_hash;
+ if (IsTotalOrderMode()) {
+ if (full_scan_mode_) {
+ status_ =
+ Status::InvalidArgument("Get() is not allowed in full scan mode.");
+ }
+ // Match whole user key for bloom filter check.
+ if (!MatchBloom(GetSliceHash(ExtractUserKey(target)))) {
+ return Status::OK();
+ }
+ // in total order mode, there is only one bucket 0, and we always use empty
+ // prefix.
+ prefix_slice = Slice();
+ prefix_hash = 0;
+ } else {
+ prefix_slice = GetPrefix(target);
+ prefix_hash = GetSliceHash(prefix_slice);
+ if (!MatchBloom(prefix_hash)) {
+ return Status::OK();
+ }
+ }
+ uint32_t offset;
+ bool prefix_match;
+ PlainTableKeyDecoder decoder(&file_info_, encoding_type_, user_key_len_,
+ prefix_extractor_);
+ Status s = GetOffset(&decoder, target, prefix_slice, prefix_hash,
+ prefix_match, &offset);
+
+ if (!s.ok()) {
+ return s;
+ }
+ ParsedInternalKey found_key;
+ ParsedInternalKey parsed_target;
+ s = ParseInternalKey(target, &parsed_target,
+ false /* log_err_key */); // TODO
+ if (!s.ok()) return s;
+
+ Slice found_value;
+ while (offset < file_info_.data_end_offset) {
+ s = Next(&decoder, &offset, &found_key, nullptr, &found_value);
+ if (!s.ok()) {
+ return s;
+ }
+ if (!prefix_match) {
+ // Need to verify prefix for the first key found if it is not yet
+ // checked.
+ if (GetPrefix(found_key) != prefix_slice) {
+ return Status::OK();
+ }
+ prefix_match = true;
+ }
+ // TODO(ljin): since we know the key comparison result here,
+ // can we enable the fast path?
+ if (internal_comparator_.Compare(found_key, parsed_target) >= 0) {
+ bool dont_care __attribute__((__unused__));
+ if (!get_context->SaveValue(found_key, found_value, &dont_care,
+ dummy_cleanable_.get())) {
+ break;
+ }
+ }
+ }
+ return Status::OK();
+}
+
+uint64_t PlainTableReader::ApproximateOffsetOf(const Slice& /*key*/,
+ TableReaderCaller /*caller*/) {
+ return 0;
+}
+
+uint64_t PlainTableReader::ApproximateSize(const Slice& /*start*/,
+ const Slice& /*end*/,
+ TableReaderCaller /*caller*/) {
+ return 0;
+}
+
+PlainTableIterator::PlainTableIterator(PlainTableReader* table,
+ bool use_prefix_seek)
+ : table_(table),
+ decoder_(&table_->file_info_, table_->encoding_type_,
+ table_->user_key_len_, table_->prefix_extractor_),
+ use_prefix_seek_(use_prefix_seek) {
+ next_offset_ = offset_ = table_->file_info_.data_end_offset;
+}
+
+PlainTableIterator::~PlainTableIterator() {}
+
+bool PlainTableIterator::Valid() const {
+ return offset_ < table_->file_info_.data_end_offset &&
+ offset_ >= table_->data_start_offset_;
+}
+
+void PlainTableIterator::SeekToFirst() {
+ status_ = Status::OK();
+ next_offset_ = table_->data_start_offset_;
+ if (next_offset_ >= table_->file_info_.data_end_offset) {
+ next_offset_ = offset_ = table_->file_info_.data_end_offset;
+ } else {
+ Next();
+ }
+}
+
+void PlainTableIterator::SeekToLast() {
+ assert(false);
+ status_ = Status::NotSupported("SeekToLast() is not supported in PlainTable");
+ next_offset_ = offset_ = table_->file_info_.data_end_offset;
+}
+
+void PlainTableIterator::Seek(const Slice& target) {
+ if (use_prefix_seek_ != !table_->IsTotalOrderMode()) {
+ // This check is done here instead of NewIterator() to permit creating an
+ // iterator with total_order_seek = true even if we won't be able to Seek()
+ // it. This is needed for compaction: it creates iterator with
+ // total_order_seek = true but usually never does Seek() on it,
+ // only SeekToFirst().
+ status_ = Status::InvalidArgument(
+ "total_order_seek not implemented for PlainTable.");
+ offset_ = next_offset_ = table_->file_info_.data_end_offset;
+ return;
+ }
+
+ // If the user doesn't set prefix seek option and we are not able to do a
+ // total Seek(). assert failure.
+ if (table_->IsTotalOrderMode()) {
+ if (table_->full_scan_mode_) {
+ status_ =
+ Status::InvalidArgument("Seek() is not allowed in full scan mode.");
+ offset_ = next_offset_ = table_->file_info_.data_end_offset;
+ return;
+ } else if (table_->GetIndexSize() > 1) {
+ assert(false);
+ status_ = Status::NotSupported(
+ "PlainTable cannot issue non-prefix seek unless in total order "
+ "mode.");
+ offset_ = next_offset_ = table_->file_info_.data_end_offset;
+ return;
+ }
+ }
+
+ Slice prefix_slice = table_->GetPrefix(target);
+ uint32_t prefix_hash = 0;
+ // Bloom filter is ignored in total-order mode.
+ if (!table_->IsTotalOrderMode()) {
+ prefix_hash = GetSliceHash(prefix_slice);
+ if (!table_->MatchBloom(prefix_hash)) {
+ status_ = Status::OK();
+ offset_ = next_offset_ = table_->file_info_.data_end_offset;
+ return;
+ }
+ }
+ bool prefix_match;
+ status_ = table_->GetOffset(&decoder_, target, prefix_slice, prefix_hash,
+ prefix_match, &next_offset_);
+ if (!status_.ok()) {
+ offset_ = next_offset_ = table_->file_info_.data_end_offset;
+ return;
+ }
+
+ if (next_offset_ < table_->file_info_.data_end_offset) {
+ for (Next(); status_.ok() && Valid(); Next()) {
+ if (!prefix_match) {
+ // Need to verify the first key's prefix
+ if (table_->GetPrefix(key()) != prefix_slice) {
+ offset_ = next_offset_ = table_->file_info_.data_end_offset;
+ break;
+ }
+ prefix_match = true;
+ }
+ if (table_->internal_comparator_.Compare(key(), target) >= 0) {
+ break;
+ }
+ }
+ } else {
+ offset_ = table_->file_info_.data_end_offset;
+ }
+}
+
+void PlainTableIterator::SeekForPrev(const Slice& /*target*/) {
+ assert(false);
+ status_ =
+ Status::NotSupported("SeekForPrev() is not supported in PlainTable");
+ offset_ = next_offset_ = table_->file_info_.data_end_offset;
+}
+
+void PlainTableIterator::Next() {
+ offset_ = next_offset_;
+ if (offset_ < table_->file_info_.data_end_offset) {
+ Slice tmp_slice;
+ ParsedInternalKey parsed_key;
+ status_ =
+ table_->Next(&decoder_, &next_offset_, &parsed_key, &key_, &value_);
+ if (!status_.ok()) {
+ offset_ = next_offset_ = table_->file_info_.data_end_offset;
+ }
+ }
+}
+
+void PlainTableIterator::Prev() { assert(false); }
+
+Slice PlainTableIterator::key() const {
+ assert(Valid());
+ return key_;
+}
+
+Slice PlainTableIterator::value() const {
+ assert(Valid());
+ return value_;
+}
+
+Status PlainTableIterator::status() const { return status_; }
+
+} // namespace ROCKSDB_NAMESPACE
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/table/plain/plain_table_reader.h b/src/rocksdb/table/plain/plain_table_reader.h
new file mode 100644
index 000000000..62bda693a
--- /dev/null
+++ b/src/rocksdb/table/plain/plain_table_reader.h
@@ -0,0 +1,244 @@
+// Copyright (c) Facebook, Inc. and its affiliates. All Rights Reserved.
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#pragma once
+
+#ifndef ROCKSDB_LITE
+#include <stdint.h>
+
+#include <memory>
+#include <string>
+#include <unordered_map>
+#include <vector>
+
+#include "file/random_access_file_reader.h"
+#include "memory/arena.h"
+#include "rocksdb/env.h"
+#include "rocksdb/iterator.h"
+#include "rocksdb/slice_transform.h"
+#include "rocksdb/table.h"
+#include "rocksdb/table_properties.h"
+#include "table/plain/plain_table_bloom.h"
+#include "table/plain/plain_table_factory.h"
+#include "table/plain/plain_table_index.h"
+#include "table/table_reader.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class Block;
+struct BlockContents;
+class BlockHandle;
+class Footer;
+struct Options;
+class RandomAccessFile;
+struct ReadOptions;
+class TableCache;
+class TableReader;
+class InternalKeyComparator;
+class PlainTableKeyDecoder;
+class GetContext;
+
+extern const uint32_t kPlainTableVariableLength;
+
+struct PlainTableReaderFileInfo {
+ bool is_mmap_mode;
+ Slice file_data;
+ uint32_t data_end_offset;
+ std::unique_ptr<RandomAccessFileReader> file;
+
+ PlainTableReaderFileInfo(std::unique_ptr<RandomAccessFileReader>&& _file,
+ const EnvOptions& storage_options,
+ uint32_t _data_size_offset)
+ : is_mmap_mode(storage_options.use_mmap_reads),
+ data_end_offset(_data_size_offset),
+ file(std::move(_file)) {}
+};
+
+// The reader class of PlainTable. For description of PlainTable format
+// See comments of class PlainTableFactory, where instances of
+// PlainTableReader are created.
+class PlainTableReader : public TableReader {
+ public:
+ // Based on following output file format shown in plain_table_factory.h
+ // When opening the output file, PlainTableReader creates a hash table
+ // from key prefixes to offset of the output file. PlainTable will decide
+ // whether it points to the data offset of the first key with the key prefix
+ // or the offset of it. If there are too many keys share this prefix, it will
+ // create a binary search-able index from the suffix to offset on disk.
+ static Status Open(const ImmutableOptions& ioptions,
+ const EnvOptions& env_options,
+ const InternalKeyComparator& internal_comparator,
+ std::unique_ptr<RandomAccessFileReader>&& file,
+ uint64_t file_size, std::unique_ptr<TableReader>* table,
+ const int bloom_bits_per_key, double hash_table_ratio,
+ size_t index_sparseness, size_t huge_page_tlb_size,
+ bool full_scan_mode, const bool immortal_table = false,
+ const SliceTransform* prefix_extractor = nullptr);
+
+ // Returns new iterator over table contents
+ // compaction_readahead_size: its value will only be used if for_compaction =
+ // true
+ InternalIterator* NewIterator(const ReadOptions&,
+ const SliceTransform* prefix_extractor,
+ Arena* arena, bool skip_filters,
+ TableReaderCaller caller,
+ size_t compaction_readahead_size = 0,
+ bool allow_unprepared_value = false) override;
+
+ void Prepare(const Slice& target) override;
+
+ Status Get(const ReadOptions& readOptions, const Slice& key,
+ GetContext* get_context, const SliceTransform* prefix_extractor,
+ bool skip_filters = false) override;
+
+ uint64_t ApproximateOffsetOf(const Slice& key,
+ TableReaderCaller caller) override;
+
+ uint64_t ApproximateSize(const Slice& start, const Slice& end,
+ TableReaderCaller caller) override;
+
+ uint32_t GetIndexSize() const { return index_.GetIndexSize(); }
+ void SetupForCompaction() override;
+
+ std::shared_ptr<const TableProperties> GetTableProperties() const override {
+ return table_properties_;
+ }
+
+ virtual size_t ApproximateMemoryUsage() const override {
+ return arena_.MemoryAllocatedBytes();
+ }
+
+ PlainTableReader(const ImmutableOptions& ioptions,
+ std::unique_ptr<RandomAccessFileReader>&& file,
+ const EnvOptions& env_options,
+ const InternalKeyComparator& internal_comparator,
+ EncodingType encoding_type, uint64_t file_size,
+ const TableProperties* table_properties,
+ const SliceTransform* prefix_extractor);
+ virtual ~PlainTableReader();
+
+ protected:
+ // Check bloom filter to see whether it might contain this prefix.
+ // The hash of the prefix is given, since it can be reused for index lookup
+ // too.
+ virtual bool MatchBloom(uint32_t hash) const;
+
+ // PopulateIndex() builds index of keys. It must be called before any query
+ // to the table.
+ //
+ // props: the table properties object that need to be stored. Ownership of
+ // the object will be passed.
+ //
+
+ Status PopulateIndex(TableProperties* props, int bloom_bits_per_key,
+ double hash_table_ratio, size_t index_sparseness,
+ size_t huge_page_tlb_size);
+
+ Status MmapDataIfNeeded();
+
+ private:
+ const InternalKeyComparator internal_comparator_;
+ EncodingType encoding_type_;
+ // represents plain table's current status.
+ Status status_;
+
+ PlainTableIndex index_;
+ bool full_scan_mode_;
+
+ // data_start_offset_ and data_end_offset_ defines the range of the
+ // sst file that stores data.
+ const uint32_t data_start_offset_ = 0;
+ const uint32_t user_key_len_;
+ const SliceTransform* prefix_extractor_;
+
+ static const size_t kNumInternalBytes = 8;
+
+ // Bloom filter is used to rule out non-existent key
+ bool enable_bloom_;
+ PlainTableBloomV1 bloom_;
+ PlainTableReaderFileInfo file_info_;
+ Arena arena_;
+ CacheAllocationPtr index_block_alloc_;
+ CacheAllocationPtr bloom_block_alloc_;
+
+ const ImmutableOptions& ioptions_;
+ std::unique_ptr<Cleanable> dummy_cleanable_;
+ uint64_t file_size_;
+
+ protected: // for testing
+ std::shared_ptr<const TableProperties> table_properties_;
+
+ private:
+ bool IsFixedLength() const {
+ return user_key_len_ != kPlainTableVariableLength;
+ }
+
+ size_t GetFixedInternalKeyLength() const {
+ return user_key_len_ + kNumInternalBytes;
+ }
+
+ Slice GetPrefix(const Slice& target) const {
+ assert(target.size() >= 8); // target is internal key
+ return GetPrefixFromUserKey(ExtractUserKey(target));
+ }
+
+ Slice GetPrefix(const ParsedInternalKey& target) const {
+ return GetPrefixFromUserKey(target.user_key);
+ }
+
+ Slice GetPrefixFromUserKey(const Slice& user_key) const {
+ if (!IsTotalOrderMode()) {
+ return prefix_extractor_->Transform(user_key);
+ } else {
+ // Use empty slice as prefix if prefix_extractor is not set.
+ // In that case,
+ // it falls back to pure binary search and
+ // total iterator seek is supported.
+ return Slice();
+ }
+ }
+
+ friend class TableCache;
+ friend class PlainTableIterator;
+
+ // Internal helper function to generate an IndexRecordList object from all
+ // the rows, which contains index records as a list.
+ // If bloom_ is not null, all the keys' full-key hash will be added to the
+ // bloom filter.
+ Status PopulateIndexRecordList(PlainTableIndexBuilder* index_builder,
+ std::vector<uint32_t>* prefix_hashes);
+
+ // Internal helper function to allocate memory for bloom filter
+ void AllocateBloom(int bloom_bits_per_key, int num_prefixes,
+ size_t huge_page_tlb_size);
+
+ void FillBloom(const std::vector<uint32_t>& prefix_hashes);
+
+ // Read the key and value at `offset` to parameters for keys, the and
+ // `seekable`.
+ // On success, `offset` will be updated as the offset for the next key.
+ // `parsed_key` will be key in parsed format.
+ // if `internal_key` is not empty, it will be filled with key with slice
+ // format.
+ // if `seekable` is not null, it will return whether we can directly read
+ // data using this offset.
+ Status Next(PlainTableKeyDecoder* decoder, uint32_t* offset,
+ ParsedInternalKey* parsed_key, Slice* internal_key, Slice* value,
+ bool* seekable = nullptr) const;
+ // 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.
+ Status GetOffset(PlainTableKeyDecoder* decoder, const Slice& target,
+ const Slice& prefix, uint32_t prefix_hash,
+ bool& prefix_matched, uint32_t* offset) const;
+
+ bool IsTotalOrderMode() const { return (prefix_extractor_ == nullptr); }
+
+ // No copying allowed
+ explicit PlainTableReader(const TableReader&) = delete;
+ void operator=(const TableReader&) = delete;
+};
+} // namespace ROCKSDB_NAMESPACE
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/table/scoped_arena_iterator.h b/src/rocksdb/table/scoped_arena_iterator.h
new file mode 100644
index 000000000..2b8824d95
--- /dev/null
+++ b/src/rocksdb/table/scoped_arena_iterator.h
@@ -0,0 +1,57 @@
+// Copyright (c) 2011-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).
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+#pragma once
+
+#include "port/port.h"
+#include "table/internal_iterator.h"
+
+namespace ROCKSDB_NAMESPACE {
+class ScopedArenaIterator {
+ void reset(InternalIterator* iter) noexcept {
+ if (iter_ != nullptr) {
+ iter_->~InternalIterator();
+ }
+ iter_ = iter;
+ }
+
+ public:
+ explicit ScopedArenaIterator(InternalIterator* iter = nullptr)
+ : iter_(iter) {}
+
+ ScopedArenaIterator(const ScopedArenaIterator&) = delete;
+ ScopedArenaIterator& operator=(const ScopedArenaIterator&) = delete;
+
+ ScopedArenaIterator(ScopedArenaIterator&& o) noexcept {
+ iter_ = o.iter_;
+ o.iter_ = nullptr;
+ }
+
+ ScopedArenaIterator& operator=(ScopedArenaIterator&& o) noexcept {
+ reset(o.iter_);
+ o.iter_ = nullptr;
+ return *this;
+ }
+
+ InternalIterator* operator->() { return iter_; }
+ InternalIterator* get() { return iter_; }
+
+ void set(InternalIterator* iter) { reset(iter); }
+
+ InternalIterator* release() {
+ assert(iter_ != nullptr);
+ auto* res = iter_;
+ iter_ = nullptr;
+ return res;
+ }
+
+ ~ScopedArenaIterator() { reset(nullptr); }
+
+ private:
+ InternalIterator* iter_;
+};
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/sst_file_dumper.cc b/src/rocksdb/table/sst_file_dumper.cc
new file mode 100644
index 000000000..122f0995a
--- /dev/null
+++ b/src/rocksdb/table/sst_file_dumper.cc
@@ -0,0 +1,519 @@
+// Copyright (c) 2011-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).
+//
+#ifndef ROCKSDB_LITE
+
+#include "table/sst_file_dumper.h"
+
+#include <chrono>
+#include <cinttypes>
+#include <iostream>
+#include <map>
+#include <memory>
+#include <sstream>
+#include <vector>
+
+#include "db/blob/blob_index.h"
+#include "db/memtable.h"
+#include "db/write_batch_internal.h"
+#include "options/cf_options.h"
+#include "port/port.h"
+#include "rocksdb/db.h"
+#include "rocksdb/env.h"
+#include "rocksdb/iterator.h"
+#include "rocksdb/slice_transform.h"
+#include "rocksdb/status.h"
+#include "rocksdb/table_properties.h"
+#include "rocksdb/utilities/ldb_cmd.h"
+#include "table/block_based/block.h"
+#include "table/block_based/block_based_table_builder.h"
+#include "table/block_based/block_based_table_factory.h"
+#include "table/block_based/block_builder.h"
+#include "table/format.h"
+#include "table/meta_blocks.h"
+#include "table/plain/plain_table_factory.h"
+#include "table/table_reader.h"
+#include "util/compression.h"
+#include "util/random.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+SstFileDumper::SstFileDumper(const Options& options,
+ const std::string& file_path,
+ Temperature file_temp, size_t readahead_size,
+ bool verify_checksum, bool output_hex,
+ bool decode_blob_index, const EnvOptions& soptions,
+ bool silent)
+ : file_name_(file_path),
+ read_num_(0),
+ file_temp_(file_temp),
+ output_hex_(output_hex),
+ decode_blob_index_(decode_blob_index),
+ soptions_(soptions),
+ silent_(silent),
+ options_(options),
+ ioptions_(options_),
+ moptions_(ColumnFamilyOptions(options_)),
+ read_options_(verify_checksum, false),
+ internal_comparator_(BytewiseComparator()) {
+ read_options_.readahead_size = readahead_size;
+ if (!silent_) {
+ fprintf(stdout, "Process %s\n", file_path.c_str());
+ }
+ init_result_ = GetTableReader(file_name_);
+}
+
+extern const uint64_t kBlockBasedTableMagicNumber;
+extern const uint64_t kLegacyBlockBasedTableMagicNumber;
+extern const uint64_t kPlainTableMagicNumber;
+extern const uint64_t kLegacyPlainTableMagicNumber;
+
+const char* testFileName = "test_file_name";
+
+Status SstFileDumper::GetTableReader(const std::string& file_path) {
+ // Warning about 'magic_number' being uninitialized shows up only in UBsan
+ // builds. Though access is guarded by 's.ok()' checks, fix the issue to
+ // avoid any warnings.
+ uint64_t magic_number = Footer::kNullTableMagicNumber;
+
+ // read table magic number
+ Footer footer;
+
+ const auto& fs = options_.env->GetFileSystem();
+ std::unique_ptr<FSRandomAccessFile> file;
+ uint64_t file_size = 0;
+ FileOptions fopts = soptions_;
+ fopts.temperature = file_temp_;
+ Status s = fs->NewRandomAccessFile(file_path, fopts, &file, nullptr);
+ if (s.ok()) {
+ s = fs->GetFileSize(file_path, IOOptions(), &file_size, nullptr);
+ }
+
+ // check empty file
+ // if true, skip further processing of this file
+ if (file_size == 0) {
+ return Status::Aborted(file_path, "Empty file");
+ }
+
+ file_.reset(new RandomAccessFileReader(std::move(file), file_path));
+
+ FilePrefetchBuffer prefetch_buffer(
+ 0 /* readahead_size */, 0 /* max_readahead_size */, true /* enable */,
+ false /* track_min_offset */);
+ if (s.ok()) {
+ const uint64_t kSstDumpTailPrefetchSize = 512 * 1024;
+ uint64_t prefetch_size = (file_size > kSstDumpTailPrefetchSize)
+ ? kSstDumpTailPrefetchSize
+ : file_size;
+ uint64_t prefetch_off = file_size - prefetch_size;
+ IOOptions opts;
+ s = prefetch_buffer.Prefetch(opts, file_.get(), prefetch_off,
+ static_cast<size_t>(prefetch_size),
+ Env::IO_TOTAL /* rate_limiter_priority */);
+
+ s = ReadFooterFromFile(opts, file_.get(), &prefetch_buffer, file_size,
+ &footer);
+ }
+ if (s.ok()) {
+ magic_number = footer.table_magic_number();
+ }
+
+ if (s.ok()) {
+ if (magic_number == kPlainTableMagicNumber ||
+ magic_number == kLegacyPlainTableMagicNumber) {
+ soptions_.use_mmap_reads = true;
+
+ fs->NewRandomAccessFile(file_path, fopts, &file, nullptr);
+ file_.reset(new RandomAccessFileReader(std::move(file), file_path));
+ }
+
+ // For old sst format, ReadTableProperties might fail but file can be read
+ if (ReadTableProperties(magic_number, file_.get(), file_size,
+ (magic_number == kBlockBasedTableMagicNumber)
+ ? &prefetch_buffer
+ : nullptr)
+ .ok()) {
+ s = SetTableOptionsByMagicNumber(magic_number);
+ if (s.ok()) {
+ if (table_properties_ && !table_properties_->comparator_name.empty()) {
+ ConfigOptions config_options;
+ const Comparator* user_comparator = nullptr;
+ s = Comparator::CreateFromString(config_options,
+ table_properties_->comparator_name,
+ &user_comparator);
+ if (s.ok()) {
+ assert(user_comparator);
+ internal_comparator_ = InternalKeyComparator(user_comparator);
+ }
+ }
+ }
+ } else {
+ s = SetOldTableOptions();
+ }
+ options_.comparator = internal_comparator_.user_comparator();
+ }
+
+ if (s.ok()) {
+ s = NewTableReader(ioptions_, soptions_, internal_comparator_, file_size,
+ &table_reader_);
+ }
+ return s;
+}
+
+Status SstFileDumper::NewTableReader(
+ const ImmutableOptions& /*ioptions*/, const EnvOptions& /*soptions*/,
+ const InternalKeyComparator& /*internal_comparator*/, uint64_t file_size,
+ std::unique_ptr<TableReader>* /*table_reader*/) {
+ auto t_opt =
+ TableReaderOptions(ioptions_, moptions_.prefix_extractor, soptions_,
+ internal_comparator_, false /* skip_filters */,
+ false /* imortal */, true /* force_direct_prefetch */);
+ // Allow open file with global sequence number for backward compatibility.
+ t_opt.largest_seqno = kMaxSequenceNumber;
+
+ // We need to turn off pre-fetching of index and filter nodes for
+ // BlockBasedTable
+ if (options_.table_factory->IsInstanceOf(
+ TableFactory::kBlockBasedTableName())) {
+ return options_.table_factory->NewTableReader(t_opt, std::move(file_),
+ file_size, &table_reader_,
+ /*enable_prefetch=*/false);
+ }
+
+ // For all other factory implementation
+ return options_.table_factory->NewTableReader(t_opt, std::move(file_),
+ file_size, &table_reader_);
+}
+
+Status SstFileDumper::VerifyChecksum() {
+ // We could pass specific readahead setting into read options if needed.
+ return table_reader_->VerifyChecksum(read_options_,
+ TableReaderCaller::kSSTDumpTool);
+}
+
+Status SstFileDumper::DumpTable(const std::string& out_filename) {
+ std::unique_ptr<WritableFile> out_file;
+ Env* env = options_.env;
+ Status s = env->NewWritableFile(out_filename, &out_file, soptions_);
+ if (s.ok()) {
+ s = table_reader_->DumpTable(out_file.get());
+ }
+ if (!s.ok()) {
+ // close the file before return error, ignore the close error if there's any
+ out_file->Close().PermitUncheckedError();
+ return s;
+ }
+ return out_file->Close();
+}
+
+Status SstFileDumper::CalculateCompressedTableSize(
+ const TableBuilderOptions& tb_options, size_t block_size,
+ uint64_t* num_data_blocks, uint64_t* compressed_table_size) {
+ std::unique_ptr<Env> env(NewMemEnv(options_.env));
+ std::unique_ptr<WritableFileWriter> dest_writer;
+ Status s =
+ WritableFileWriter::Create(env->GetFileSystem(), testFileName,
+ FileOptions(soptions_), &dest_writer, nullptr);
+ if (!s.ok()) {
+ return s;
+ }
+ BlockBasedTableOptions table_options;
+ table_options.block_size = block_size;
+ BlockBasedTableFactory block_based_tf(table_options);
+ std::unique_ptr<TableBuilder> table_builder;
+ table_builder.reset(
+ block_based_tf.NewTableBuilder(tb_options, dest_writer.get()));
+ std::unique_ptr<InternalIterator> iter(table_reader_->NewIterator(
+ read_options_, moptions_.prefix_extractor.get(), /*arena=*/nullptr,
+ /*skip_filters=*/false, TableReaderCaller::kSSTDumpTool));
+ for (iter->SeekToFirst(); iter->Valid(); iter->Next()) {
+ table_builder->Add(iter->key(), iter->value());
+ }
+ s = iter->status();
+ if (!s.ok()) {
+ return s;
+ }
+ s = table_builder->Finish();
+ if (!s.ok()) {
+ return s;
+ }
+ *compressed_table_size = table_builder->FileSize();
+ assert(num_data_blocks != nullptr);
+ *num_data_blocks = table_builder->GetTableProperties().num_data_blocks;
+ return env->DeleteFile(testFileName);
+}
+
+Status SstFileDumper::ShowAllCompressionSizes(
+ size_t block_size,
+ const std::vector<std::pair<CompressionType, const char*>>&
+ compression_types,
+ int32_t compress_level_from, int32_t compress_level_to,
+ uint32_t max_dict_bytes, uint32_t zstd_max_train_bytes,
+ uint64_t max_dict_buffer_bytes, bool use_zstd_dict_trainer) {
+ fprintf(stdout, "Block Size: %" ROCKSDB_PRIszt "\n", block_size);
+ for (auto& i : compression_types) {
+ if (CompressionTypeSupported(i.first)) {
+ fprintf(stdout, "Compression: %-24s\n", i.second);
+ CompressionOptions compress_opt;
+ compress_opt.max_dict_bytes = max_dict_bytes;
+ compress_opt.zstd_max_train_bytes = zstd_max_train_bytes;
+ compress_opt.max_dict_buffer_bytes = max_dict_buffer_bytes;
+ compress_opt.use_zstd_dict_trainer = use_zstd_dict_trainer;
+ for (int32_t j = compress_level_from; j <= compress_level_to; j++) {
+ fprintf(stdout, "Compression level: %d", j);
+ compress_opt.level = j;
+ Status s = ShowCompressionSize(block_size, i.first, compress_opt);
+ if (!s.ok()) {
+ return s;
+ }
+ }
+ } else {
+ fprintf(stdout, "Unsupported compression type: %s.\n", i.second);
+ }
+ }
+ return Status::OK();
+}
+
+Status SstFileDumper::ShowCompressionSize(
+ size_t block_size, CompressionType compress_type,
+ const CompressionOptions& compress_opt) {
+ Options opts;
+ opts.statistics = ROCKSDB_NAMESPACE::CreateDBStatistics();
+ opts.statistics->set_stats_level(StatsLevel::kAll);
+ const ImmutableOptions imoptions(opts);
+ const ColumnFamilyOptions cfo(opts);
+ const MutableCFOptions moptions(cfo);
+ ROCKSDB_NAMESPACE::InternalKeyComparator ikc(opts.comparator);
+ IntTblPropCollectorFactories block_based_table_factories;
+
+ std::string column_family_name;
+ int unknown_level = -1;
+ TableBuilderOptions tb_opts(
+ imoptions, moptions, ikc, &block_based_table_factories, compress_type,
+ compress_opt,
+ TablePropertiesCollectorFactory::Context::kUnknownColumnFamily,
+ column_family_name, unknown_level);
+ uint64_t num_data_blocks = 0;
+ std::chrono::steady_clock::time_point start =
+ std::chrono::steady_clock::now();
+ uint64_t file_size;
+ Status s = CalculateCompressedTableSize(tb_opts, block_size, &num_data_blocks,
+ &file_size);
+ if (!s.ok()) {
+ return s;
+ }
+
+ std::chrono::steady_clock::time_point end = std::chrono::steady_clock::now();
+ fprintf(stdout, " Size: %10" PRIu64, file_size);
+ fprintf(stdout, " Blocks: %6" PRIu64, num_data_blocks);
+ fprintf(stdout, " Time Taken: %10s microsecs",
+ std::to_string(
+ std::chrono::duration_cast<std::chrono::microseconds>(end - start)
+ .count())
+ .c_str());
+ const uint64_t compressed_blocks =
+ opts.statistics->getAndResetTickerCount(NUMBER_BLOCK_COMPRESSED);
+ const uint64_t not_compressed_blocks =
+ opts.statistics->getAndResetTickerCount(NUMBER_BLOCK_NOT_COMPRESSED);
+ // When the option enable_index_compression is true,
+ // NUMBER_BLOCK_COMPRESSED is incremented for index block(s).
+ if ((compressed_blocks + not_compressed_blocks) > num_data_blocks) {
+ num_data_blocks = compressed_blocks + not_compressed_blocks;
+ }
+
+ const uint64_t ratio_not_compressed_blocks =
+ (num_data_blocks - compressed_blocks) - not_compressed_blocks;
+ const double compressed_pcnt =
+ (0 == num_data_blocks) ? 0.0
+ : ((static_cast<double>(compressed_blocks) /
+ static_cast<double>(num_data_blocks)) *
+ 100.0);
+ const double ratio_not_compressed_pcnt =
+ (0 == num_data_blocks)
+ ? 0.0
+ : ((static_cast<double>(ratio_not_compressed_blocks) /
+ static_cast<double>(num_data_blocks)) *
+ 100.0);
+ const double not_compressed_pcnt =
+ (0 == num_data_blocks) ? 0.0
+ : ((static_cast<double>(not_compressed_blocks) /
+ static_cast<double>(num_data_blocks)) *
+ 100.0);
+ fprintf(stdout, " Compressed: %6" PRIu64 " (%5.1f%%)", compressed_blocks,
+ compressed_pcnt);
+ fprintf(stdout, " Not compressed (ratio): %6" PRIu64 " (%5.1f%%)",
+ ratio_not_compressed_blocks, ratio_not_compressed_pcnt);
+ fprintf(stdout, " Not compressed (abort): %6" PRIu64 " (%5.1f%%)\n",
+ not_compressed_blocks, not_compressed_pcnt);
+ return Status::OK();
+}
+
+// Reads TableProperties prior to opening table reader in order to set up
+// options.
+Status SstFileDumper::ReadTableProperties(uint64_t table_magic_number,
+ RandomAccessFileReader* file,
+ uint64_t file_size,
+ FilePrefetchBuffer* prefetch_buffer) {
+ Status s = ROCKSDB_NAMESPACE::ReadTableProperties(
+ file, file_size, table_magic_number, ioptions_, &table_properties_,
+ /* memory_allocator= */ nullptr, prefetch_buffer);
+ if (!s.ok()) {
+ if (!silent_) {
+ fprintf(stdout, "Not able to read table properties\n");
+ }
+ }
+ return s;
+}
+
+Status SstFileDumper::SetTableOptionsByMagicNumber(
+ uint64_t table_magic_number) {
+ assert(table_properties_);
+ if (table_magic_number == kBlockBasedTableMagicNumber ||
+ table_magic_number == kLegacyBlockBasedTableMagicNumber) {
+ BlockBasedTableFactory* bbtf = new BlockBasedTableFactory();
+ // To force tail prefetching, we fake reporting two useful reads of 512KB
+ // from the tail.
+ // It needs at least two data points to warm up the stats.
+ bbtf->tail_prefetch_stats()->RecordEffectiveSize(512 * 1024);
+ bbtf->tail_prefetch_stats()->RecordEffectiveSize(512 * 1024);
+
+ options_.table_factory.reset(bbtf);
+ if (!silent_) {
+ fprintf(stdout, "Sst file format: block-based\n");
+ }
+
+ auto& props = table_properties_->user_collected_properties;
+ auto pos = props.find(BlockBasedTablePropertyNames::kIndexType);
+ if (pos != props.end()) {
+ auto index_type_on_file = static_cast<BlockBasedTableOptions::IndexType>(
+ DecodeFixed32(pos->second.c_str()));
+ if (index_type_on_file ==
+ BlockBasedTableOptions::IndexType::kHashSearch) {
+ options_.prefix_extractor.reset(NewNoopTransform());
+ }
+ }
+ } else if (table_magic_number == kPlainTableMagicNumber ||
+ table_magic_number == kLegacyPlainTableMagicNumber) {
+ options_.allow_mmap_reads = true;
+
+ PlainTableOptions plain_table_options;
+ plain_table_options.user_key_len = kPlainTableVariableLength;
+ plain_table_options.bloom_bits_per_key = 0;
+ plain_table_options.hash_table_ratio = 0;
+ plain_table_options.index_sparseness = 1;
+ plain_table_options.huge_page_tlb_size = 0;
+ plain_table_options.encoding_type = kPlain;
+ plain_table_options.full_scan_mode = true;
+
+ options_.table_factory.reset(NewPlainTableFactory(plain_table_options));
+ if (!silent_) {
+ fprintf(stdout, "Sst file format: plain table\n");
+ }
+ } else {
+ char error_msg_buffer[80];
+ snprintf(error_msg_buffer, sizeof(error_msg_buffer) - 1,
+ "Unsupported table magic number --- %lx",
+ (long)table_magic_number);
+ return Status::InvalidArgument(error_msg_buffer);
+ }
+
+ return Status::OK();
+}
+
+Status SstFileDumper::SetOldTableOptions() {
+ assert(table_properties_ == nullptr);
+ options_.table_factory = std::make_shared<BlockBasedTableFactory>();
+ if (!silent_) {
+ fprintf(stdout, "Sst file format: block-based(old version)\n");
+ }
+
+ return Status::OK();
+}
+
+Status SstFileDumper::ReadSequential(bool print_kv, uint64_t read_num,
+ bool has_from, const std::string& from_key,
+ bool has_to, const std::string& to_key,
+ bool use_from_as_prefix) {
+ if (!table_reader_) {
+ return init_result_;
+ }
+
+ InternalIterator* iter = table_reader_->NewIterator(
+ read_options_, moptions_.prefix_extractor.get(),
+ /*arena=*/nullptr, /*skip_filters=*/false,
+ TableReaderCaller::kSSTDumpTool);
+ uint64_t i = 0;
+ if (has_from) {
+ InternalKey ikey;
+ ikey.SetMinPossibleForUserKey(from_key);
+ iter->Seek(ikey.Encode());
+ } else {
+ iter->SeekToFirst();
+ }
+ for (; iter->Valid(); iter->Next()) {
+ Slice key = iter->key();
+ Slice value = iter->value();
+ ++i;
+ if (read_num > 0 && i > read_num) break;
+
+ ParsedInternalKey ikey;
+ Status pik_status = ParseInternalKey(key, &ikey, true /* log_err_key */);
+ if (!pik_status.ok()) {
+ std::cerr << pik_status.getState() << "\n";
+ continue;
+ }
+
+ // the key returned is not prefixed with out 'from' key
+ if (use_from_as_prefix && !ikey.user_key.starts_with(from_key)) {
+ break;
+ }
+
+ // If end marker was specified, we stop before it
+ if (has_to && BytewiseComparator()->Compare(ikey.user_key, to_key) >= 0) {
+ break;
+ }
+
+ if (print_kv) {
+ if (!decode_blob_index_ || ikey.type != kTypeBlobIndex) {
+ fprintf(stdout, "%s => %s\n",
+ ikey.DebugString(true, output_hex_).c_str(),
+ value.ToString(output_hex_).c_str());
+ } else {
+ BlobIndex blob_index;
+
+ const Status s = blob_index.DecodeFrom(value);
+ if (!s.ok()) {
+ fprintf(stderr, "%s => error decoding blob index\n",
+ ikey.DebugString(true, output_hex_).c_str());
+ continue;
+ }
+
+ fprintf(stdout, "%s => %s\n",
+ ikey.DebugString(true, output_hex_).c_str(),
+ blob_index.DebugString(output_hex_).c_str());
+ }
+ }
+ }
+
+ read_num_ += i;
+
+ Status ret = iter->status();
+ delete iter;
+ return ret;
+}
+
+// Provides TableProperties to API user
+Status SstFileDumper::ReadTableProperties(
+ std::shared_ptr<const TableProperties>* table_properties) {
+ if (!table_reader_) {
+ return init_result_;
+ }
+
+ *table_properties = table_reader_->GetTableProperties();
+ return init_result_;
+}
+} // namespace ROCKSDB_NAMESPACE
+
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/table/sst_file_dumper.h b/src/rocksdb/table/sst_file_dumper.h
new file mode 100644
index 000000000..7be876390
--- /dev/null
+++ b/src/rocksdb/table/sst_file_dumper.h
@@ -0,0 +1,101 @@
+// Copyright (c) 2011-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).
+#pragma once
+#ifndef ROCKSDB_LITE
+
+#include <memory>
+#include <string>
+
+#include "db/dbformat.h"
+#include "file/writable_file_writer.h"
+#include "options/cf_options.h"
+#include "rocksdb/advanced_options.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class SstFileDumper {
+ public:
+ explicit SstFileDumper(const Options& options, const std::string& file_name,
+ Temperature file_temp, size_t readahead_size,
+ bool verify_checksum, bool output_hex,
+ bool decode_blob_index,
+ const EnvOptions& soptions = EnvOptions(),
+ bool silent = false);
+
+ Status ReadSequential(bool print_kv, uint64_t read_num, bool has_from,
+ const std::string& from_key, bool has_to,
+ const std::string& to_key,
+ bool use_from_as_prefix = false);
+
+ Status ReadTableProperties(
+ std::shared_ptr<const TableProperties>* table_properties);
+ uint64_t GetReadNumber() { return read_num_; }
+ TableProperties* GetInitTableProperties() { return table_properties_.get(); }
+
+ Status VerifyChecksum();
+ Status DumpTable(const std::string& out_filename);
+ Status getStatus() { return init_result_; }
+
+ Status ShowAllCompressionSizes(
+ size_t block_size,
+ const std::vector<std::pair<CompressionType, const char*>>&
+ compression_types,
+ int32_t compress_level_from, int32_t compress_level_to,
+ uint32_t max_dict_bytes, uint32_t zstd_max_train_bytes,
+ uint64_t max_dict_buffer_bytes, bool use_zstd_dict_trainer);
+
+ Status ShowCompressionSize(size_t block_size, CompressionType compress_type,
+ const CompressionOptions& compress_opt);
+
+ private:
+ // Get the TableReader implementation for the sst file
+ Status GetTableReader(const std::string& file_path);
+ Status ReadTableProperties(uint64_t table_magic_number,
+ RandomAccessFileReader* file, uint64_t file_size,
+ FilePrefetchBuffer* prefetch_buffer);
+
+ Status CalculateCompressedTableSize(const TableBuilderOptions& tb_options,
+ size_t block_size,
+ uint64_t* num_data_blocks,
+ uint64_t* compressed_table_size);
+
+ Status SetTableOptionsByMagicNumber(uint64_t table_magic_number);
+ Status SetOldTableOptions();
+
+ // Helper function to call the factory with settings specific to the
+ // factory implementation
+ Status NewTableReader(const ImmutableOptions& ioptions,
+ const EnvOptions& soptions,
+ const InternalKeyComparator& internal_comparator,
+ uint64_t file_size,
+ std::unique_ptr<TableReader>* table_reader);
+
+ std::string file_name_;
+ uint64_t read_num_;
+ Temperature file_temp_;
+ bool output_hex_;
+ bool decode_blob_index_;
+ EnvOptions soptions_;
+ // less verbose in stdout/stderr
+ bool silent_;
+
+ // options_ and internal_comparator_ will also be used in
+ // ReadSequential internally (specifically, seek-related operations)
+ Options options_;
+
+ Status init_result_;
+ std::unique_ptr<TableReader> table_reader_;
+ std::unique_ptr<RandomAccessFileReader> file_;
+
+ const ImmutableOptions ioptions_;
+ const MutableCFOptions moptions_;
+ ReadOptions read_options_;
+ InternalKeyComparator internal_comparator_;
+ std::unique_ptr<TableProperties> table_properties_;
+};
+
+} // namespace ROCKSDB_NAMESPACE
+
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/table/sst_file_reader.cc b/src/rocksdb/table/sst_file_reader.cc
new file mode 100644
index 000000000..48f1be0be
--- /dev/null
+++ b/src/rocksdb/table/sst_file_reader.cc
@@ -0,0 +1,101 @@
+// Copyright (c) 2011-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).
+
+#ifndef ROCKSDB_LITE
+
+#include "rocksdb/sst_file_reader.h"
+
+#include "db/arena_wrapped_db_iter.h"
+#include "db/db_iter.h"
+#include "db/dbformat.h"
+#include "file/random_access_file_reader.h"
+#include "options/cf_options.h"
+#include "rocksdb/env.h"
+#include "rocksdb/file_system.h"
+#include "table/get_context.h"
+#include "table/table_builder.h"
+#include "table/table_reader.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+struct SstFileReader::Rep {
+ Options options;
+ EnvOptions soptions;
+ ImmutableOptions ioptions;
+ MutableCFOptions moptions;
+
+ std::unique_ptr<TableReader> table_reader;
+
+ Rep(const Options& opts)
+ : options(opts),
+ soptions(options),
+ ioptions(options),
+ moptions(ColumnFamilyOptions(options)) {}
+};
+
+SstFileReader::SstFileReader(const Options& options) : rep_(new Rep(options)) {}
+
+SstFileReader::~SstFileReader() {}
+
+Status SstFileReader::Open(const std::string& file_path) {
+ auto r = rep_.get();
+ Status s;
+ uint64_t file_size = 0;
+ std::unique_ptr<FSRandomAccessFile> file;
+ std::unique_ptr<RandomAccessFileReader> file_reader;
+ FileOptions fopts(r->soptions);
+ const auto& fs = r->options.env->GetFileSystem();
+
+ s = fs->GetFileSize(file_path, fopts.io_options, &file_size, nullptr);
+ if (s.ok()) {
+ s = fs->NewRandomAccessFile(file_path, fopts, &file, nullptr);
+ }
+ if (s.ok()) {
+ file_reader.reset(new RandomAccessFileReader(std::move(file), file_path));
+ }
+ if (s.ok()) {
+ TableReaderOptions t_opt(r->ioptions, r->moptions.prefix_extractor,
+ r->soptions, r->ioptions.internal_comparator);
+ // Allow open file with global sequence number for backward compatibility.
+ t_opt.largest_seqno = kMaxSequenceNumber;
+ s = r->options.table_factory->NewTableReader(t_opt, std::move(file_reader),
+ file_size, &r->table_reader);
+ }
+ return s;
+}
+
+Iterator* SstFileReader::NewIterator(const ReadOptions& roptions) {
+ auto r = rep_.get();
+ auto sequence = roptions.snapshot != nullptr
+ ? roptions.snapshot->GetSequenceNumber()
+ : kMaxSequenceNumber;
+ ArenaWrappedDBIter* res = new ArenaWrappedDBIter();
+ res->Init(r->options.env, roptions, r->ioptions, r->moptions,
+ nullptr /* version */, sequence,
+ r->moptions.max_sequential_skip_in_iterations,
+ 0 /* version_number */, nullptr /* read_callback */,
+ nullptr /* db_impl */, nullptr /* cfd */,
+ true /* expose_blob_index */, false /* allow_refresh */);
+ auto internal_iter = r->table_reader->NewIterator(
+ res->GetReadOptions(), r->moptions.prefix_extractor.get(),
+ res->GetArena(), false /* skip_filters */,
+ TableReaderCaller::kSSTFileReader);
+ res->SetIterUnderDBIter(internal_iter);
+ return res;
+}
+
+std::shared_ptr<const TableProperties> SstFileReader::GetTableProperties()
+ const {
+ return rep_->table_reader->GetTableProperties();
+}
+
+Status SstFileReader::VerifyChecksum(const ReadOptions& read_options) {
+ return rep_->table_reader->VerifyChecksum(read_options,
+ TableReaderCaller::kSSTFileReader);
+}
+
+} // namespace ROCKSDB_NAMESPACE
+
+#endif // !ROCKSDB_LITE
diff --git a/src/rocksdb/table/sst_file_reader_test.cc b/src/rocksdb/table/sst_file_reader_test.cc
new file mode 100644
index 000000000..4837d223b
--- /dev/null
+++ b/src/rocksdb/table/sst_file_reader_test.cc
@@ -0,0 +1,434 @@
+// Copyright (c) 2011-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).
+
+#ifndef ROCKSDB_LITE
+
+#include "rocksdb/sst_file_reader.h"
+
+#include <cinttypes>
+
+#include "port/stack_trace.h"
+#include "rocksdb/convenience.h"
+#include "rocksdb/db.h"
+#include "rocksdb/sst_file_writer.h"
+#include "table/sst_file_writer_collectors.h"
+#include "test_util/testharness.h"
+#include "test_util/testutil.h"
+#include "utilities/merge_operators.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+std::string EncodeAsString(uint64_t v) {
+ char buf[16];
+ snprintf(buf, sizeof(buf), "%08" PRIu64, v);
+ return std::string(buf);
+}
+
+std::string EncodeAsUint64(uint64_t v) {
+ std::string dst;
+ PutFixed64(&dst, v);
+ return dst;
+}
+
+class SstFileReaderTest : public testing::Test {
+ public:
+ SstFileReaderTest() {
+ options_.merge_operator = MergeOperators::CreateUInt64AddOperator();
+ sst_name_ = test::PerThreadDBPath("sst_file");
+
+ Env* base_env = Env::Default();
+ EXPECT_OK(
+ test::CreateEnvFromSystem(ConfigOptions(), &base_env, &env_guard_));
+ EXPECT_NE(nullptr, base_env);
+ env_ = base_env;
+ options_.env = env_;
+ }
+
+ ~SstFileReaderTest() {
+ Status s = env_->DeleteFile(sst_name_);
+ EXPECT_OK(s);
+ }
+
+ void CreateFile(const std::string& file_name,
+ const std::vector<std::string>& keys) {
+ SstFileWriter writer(soptions_, options_);
+ ASSERT_OK(writer.Open(file_name));
+ for (size_t i = 0; i + 2 < keys.size(); i += 3) {
+ ASSERT_OK(writer.Put(keys[i], keys[i]));
+ ASSERT_OK(writer.Merge(keys[i + 1], EncodeAsUint64(i + 1)));
+ ASSERT_OK(writer.Delete(keys[i + 2]));
+ }
+ ASSERT_OK(writer.Finish());
+ }
+
+ void CheckFile(const std::string& file_name,
+ const std::vector<std::string>& keys,
+ bool check_global_seqno = false) {
+ ReadOptions ropts;
+ SstFileReader reader(options_);
+ ASSERT_OK(reader.Open(file_name));
+ ASSERT_OK(reader.VerifyChecksum());
+ std::unique_ptr<Iterator> iter(reader.NewIterator(ropts));
+ iter->SeekToFirst();
+ for (size_t i = 0; i + 2 < keys.size(); i += 3) {
+ ASSERT_TRUE(iter->Valid());
+ ASSERT_EQ(iter->key().compare(keys[i]), 0);
+ ASSERT_EQ(iter->value().compare(keys[i]), 0);
+ iter->Next();
+ ASSERT_TRUE(iter->Valid());
+ ASSERT_EQ(iter->key().compare(keys[i + 1]), 0);
+ ASSERT_EQ(iter->value().compare(EncodeAsUint64(i + 1)), 0);
+ iter->Next();
+ }
+ ASSERT_FALSE(iter->Valid());
+ if (check_global_seqno) {
+ auto properties = reader.GetTableProperties();
+ ASSERT_TRUE(properties);
+ std::string hostname;
+ ASSERT_OK(env_->GetHostNameString(&hostname));
+ ASSERT_EQ(properties->db_host_id, hostname);
+ auto& user_properties = properties->user_collected_properties;
+ ASSERT_TRUE(
+ user_properties.count(ExternalSstFilePropertyNames::kGlobalSeqno));
+ }
+ }
+
+ void CreateFileAndCheck(const std::vector<std::string>& keys) {
+ CreateFile(sst_name_, keys);
+ CheckFile(sst_name_, keys);
+ }
+
+ protected:
+ Options options_;
+ EnvOptions soptions_;
+ std::string sst_name_;
+ std::shared_ptr<Env> env_guard_;
+ Env* env_;
+};
+
+const uint64_t kNumKeys = 100;
+
+TEST_F(SstFileReaderTest, Basic) {
+ std::vector<std::string> keys;
+ for (uint64_t i = 0; i < kNumKeys; i++) {
+ keys.emplace_back(EncodeAsString(i));
+ }
+ CreateFileAndCheck(keys);
+}
+
+TEST_F(SstFileReaderTest, Uint64Comparator) {
+ options_.comparator = test::Uint64Comparator();
+ std::vector<std::string> keys;
+ for (uint64_t i = 0; i < kNumKeys; i++) {
+ keys.emplace_back(EncodeAsUint64(i));
+ }
+ CreateFileAndCheck(keys);
+}
+
+TEST_F(SstFileReaderTest, ReadOptionsOutOfScope) {
+ // Repro a bug where the SstFileReader depended on its configured ReadOptions
+ // outliving it.
+ options_.comparator = test::Uint64Comparator();
+ std::vector<std::string> keys;
+ for (uint64_t i = 0; i < kNumKeys; i++) {
+ keys.emplace_back(EncodeAsUint64(i));
+ }
+ CreateFile(sst_name_, keys);
+
+ SstFileReader reader(options_);
+ ASSERT_OK(reader.Open(sst_name_));
+ std::unique_ptr<Iterator> iter;
+ {
+ // Make sure ReadOptions go out of scope ASAP so we know the iterator
+ // operations do not depend on it.
+ ReadOptions ropts;
+ iter.reset(reader.NewIterator(ropts));
+ }
+ iter->SeekToFirst();
+ while (iter->Valid()) {
+ iter->Next();
+ }
+}
+
+TEST_F(SstFileReaderTest, ReadFileWithGlobalSeqno) {
+ std::vector<std::string> keys;
+ for (uint64_t i = 0; i < kNumKeys; i++) {
+ keys.emplace_back(EncodeAsString(i));
+ }
+ // Generate a SST file.
+ CreateFile(sst_name_, keys);
+
+ // Ingest the file into a db, to assign it a global sequence number.
+ Options options;
+ options.create_if_missing = true;
+ std::string db_name = test::PerThreadDBPath("test_db");
+ DB* db;
+ ASSERT_OK(DB::Open(options, db_name, &db));
+ // Bump sequence number.
+ ASSERT_OK(db->Put(WriteOptions(), keys[0], "foo"));
+ ASSERT_OK(db->Flush(FlushOptions()));
+ // Ingest the file.
+ IngestExternalFileOptions ingest_options;
+ ingest_options.write_global_seqno = true;
+ ASSERT_OK(db->IngestExternalFile({sst_name_}, ingest_options));
+ std::vector<std::string> live_files;
+ uint64_t manifest_file_size = 0;
+ ASSERT_OK(db->GetLiveFiles(live_files, &manifest_file_size));
+ // Get the ingested file.
+ std::string ingested_file;
+ for (auto& live_file : live_files) {
+ if (live_file.substr(live_file.size() - 4, std::string::npos) == ".sst") {
+ if (ingested_file.empty() || ingested_file < live_file) {
+ ingested_file = live_file;
+ }
+ }
+ }
+ ASSERT_FALSE(ingested_file.empty());
+ delete db;
+
+ // Verify the file can be open and read by SstFileReader.
+ CheckFile(db_name + ingested_file, keys, true /* check_global_seqno */);
+
+ // Cleanup.
+ ASSERT_OK(DestroyDB(db_name, options));
+}
+
+TEST_F(SstFileReaderTest, TimestampSizeMismatch) {
+ SstFileWriter writer(soptions_, options_);
+
+ ASSERT_OK(writer.Open(sst_name_));
+
+ // Comparator is not timestamp-aware; calls to APIs taking timestamps should
+ // fail.
+ ASSERT_NOK(writer.Put("key", EncodeAsUint64(100), "value"));
+ ASSERT_NOK(writer.Delete("another_key", EncodeAsUint64(200)));
+}
+
+class SstFileReaderTimestampTest : public testing::Test {
+ public:
+ SstFileReaderTimestampTest() {
+ Env* env = Env::Default();
+ EXPECT_OK(test::CreateEnvFromSystem(ConfigOptions(), &env, &env_guard_));
+ EXPECT_NE(nullptr, env);
+
+ options_.env = env;
+
+ options_.comparator = test::BytewiseComparatorWithU64TsWrapper();
+
+ sst_name_ = test::PerThreadDBPath("sst_file_ts");
+ }
+
+ ~SstFileReaderTimestampTest() {
+ EXPECT_OK(options_.env->DeleteFile(sst_name_));
+ }
+
+ struct KeyValueDesc {
+ KeyValueDesc(std::string k, std::string ts, std::string v)
+ : key(std::move(k)), timestamp(std::move(ts)), value(std::move(v)) {}
+
+ std::string key;
+ std::string timestamp;
+ std::string value;
+ };
+
+ struct InputKeyValueDesc : public KeyValueDesc {
+ InputKeyValueDesc(std::string k, std::string ts, std::string v, bool is_del,
+ bool use_contig_buf)
+ : KeyValueDesc(std::move(k), std::move(ts), std::move(v)),
+ is_delete(is_del),
+ use_contiguous_buffer(use_contig_buf) {}
+
+ bool is_delete = false;
+ bool use_contiguous_buffer = false;
+ };
+
+ struct OutputKeyValueDesc : public KeyValueDesc {
+ OutputKeyValueDesc(std::string k, std::string ts, std::string v)
+ : KeyValueDesc(std::move(k), std::string(ts), std::string(v)) {}
+ };
+
+ void CreateFile(const std::vector<InputKeyValueDesc>& descs) {
+ SstFileWriter writer(soptions_, options_);
+
+ ASSERT_OK(writer.Open(sst_name_));
+
+ for (const auto& desc : descs) {
+ if (desc.is_delete) {
+ if (desc.use_contiguous_buffer) {
+ std::string key_with_ts(desc.key + desc.timestamp);
+ ASSERT_OK(writer.Delete(Slice(key_with_ts.data(), desc.key.size()),
+ Slice(key_with_ts.data() + desc.key.size(),
+ desc.timestamp.size())));
+ } else {
+ ASSERT_OK(writer.Delete(desc.key, desc.timestamp));
+ }
+ } else {
+ if (desc.use_contiguous_buffer) {
+ std::string key_with_ts(desc.key + desc.timestamp);
+ ASSERT_OK(writer.Put(Slice(key_with_ts.data(), desc.key.size()),
+ Slice(key_with_ts.data() + desc.key.size(),
+ desc.timestamp.size()),
+ desc.value));
+ } else {
+ ASSERT_OK(writer.Put(desc.key, desc.timestamp, desc.value));
+ }
+ }
+ }
+
+ ASSERT_OK(writer.Finish());
+ }
+
+ void CheckFile(const std::string& timestamp,
+ const std::vector<OutputKeyValueDesc>& descs) {
+ SstFileReader reader(options_);
+
+ ASSERT_OK(reader.Open(sst_name_));
+ ASSERT_OK(reader.VerifyChecksum());
+
+ Slice ts_slice(timestamp);
+
+ ReadOptions read_options;
+ read_options.timestamp = &ts_slice;
+
+ std::unique_ptr<Iterator> iter(reader.NewIterator(read_options));
+ iter->SeekToFirst();
+
+ for (const auto& desc : descs) {
+ ASSERT_TRUE(iter->Valid());
+ ASSERT_EQ(iter->key(), desc.key);
+ ASSERT_EQ(iter->timestamp(), desc.timestamp);
+ ASSERT_EQ(iter->value(), desc.value);
+
+ iter->Next();
+ }
+
+ ASSERT_FALSE(iter->Valid());
+ }
+
+ protected:
+ std::shared_ptr<Env> env_guard_;
+ Options options_;
+ EnvOptions soptions_;
+ std::string sst_name_;
+};
+
+TEST_F(SstFileReaderTimestampTest, Basic) {
+ std::vector<InputKeyValueDesc> input_descs;
+
+ for (uint64_t k = 0; k < kNumKeys; k += 4) {
+ // A Put with key k, timestamp k that gets overwritten by a subsequent Put
+ // with timestamp (k + 1). Note that the comparator uses descending order
+ // for the timestamp part, so we add the later Put first.
+ input_descs.emplace_back(
+ /* key */ EncodeAsString(k), /* timestamp */ EncodeAsUint64(k + 1),
+ /* value */ EncodeAsString(k * 2), /* is_delete */ false,
+ /* use_contiguous_buffer */ false);
+ input_descs.emplace_back(
+ /* key */ EncodeAsString(k), /* timestamp */ EncodeAsUint64(k),
+ /* value */ EncodeAsString(k * 3), /* is_delete */ false,
+ /* use_contiguous_buffer */ true);
+
+ // A Put with key (k + 2), timestamp (k + 2) that gets cancelled out by a
+ // Delete with timestamp (k + 3). Note that the comparator uses descending
+ // order for the timestamp part, so we add the Delete first.
+ input_descs.emplace_back(/* key */ EncodeAsString(k + 2),
+ /* timestamp */ EncodeAsUint64(k + 3),
+ /* value */ std::string(), /* is_delete */ true,
+ /* use_contiguous_buffer */ (k % 8) == 0);
+ input_descs.emplace_back(
+ /* key */ EncodeAsString(k + 2), /* timestamp */ EncodeAsUint64(k + 2),
+ /* value */ EncodeAsString(k * 5), /* is_delete */ false,
+ /* use_contiguous_buffer */ (k % 8) != 0);
+ }
+
+ CreateFile(input_descs);
+
+ // Note: below, we check the results as of each timestamp in the range,
+ // updating the expected result as needed.
+ std::vector<OutputKeyValueDesc> output_descs;
+
+ for (uint64_t ts = 0; ts < kNumKeys; ++ts) {
+ const uint64_t k = ts - (ts % 4);
+
+ switch (ts % 4) {
+ case 0: // Initial Put for key k
+ output_descs.emplace_back(/* key */ EncodeAsString(k),
+ /* timestamp */ EncodeAsUint64(ts),
+ /* value */ EncodeAsString(k * 3));
+ break;
+
+ case 1: // Second Put for key k
+ assert(output_descs.back().key == EncodeAsString(k));
+ assert(output_descs.back().timestamp == EncodeAsUint64(ts - 1));
+ assert(output_descs.back().value == EncodeAsString(k * 3));
+ output_descs.back().timestamp = EncodeAsUint64(ts);
+ output_descs.back().value = EncodeAsString(k * 2);
+ break;
+
+ case 2: // Put for key (k + 2)
+ output_descs.emplace_back(/* key */ EncodeAsString(k + 2),
+ /* timestamp */ EncodeAsUint64(ts),
+ /* value */ EncodeAsString(k * 5));
+ break;
+
+ case 3: // Delete for key (k + 2)
+ assert(output_descs.back().key == EncodeAsString(k + 2));
+ assert(output_descs.back().timestamp == EncodeAsUint64(ts - 1));
+ assert(output_descs.back().value == EncodeAsString(k * 5));
+ output_descs.pop_back();
+ break;
+ }
+
+ CheckFile(EncodeAsUint64(ts), output_descs);
+ }
+}
+
+TEST_F(SstFileReaderTimestampTest, TimestampsOutOfOrder) {
+ SstFileWriter writer(soptions_, options_);
+
+ ASSERT_OK(writer.Open(sst_name_));
+
+ // Note: KVs that have the same user key disregarding timestamps should be in
+ // descending order of timestamps.
+ ASSERT_OK(writer.Put("key", EncodeAsUint64(1), "value1"));
+ ASSERT_NOK(writer.Put("key", EncodeAsUint64(2), "value2"));
+}
+
+TEST_F(SstFileReaderTimestampTest, TimestampSizeMismatch) {
+ SstFileWriter writer(soptions_, options_);
+
+ ASSERT_OK(writer.Open(sst_name_));
+
+ // Comparator expects 64-bit timestamps; timestamps with other sizes as well
+ // as calls to the timestamp-less APIs should be rejected.
+ ASSERT_NOK(writer.Put("key", "not_an_actual_64_bit_timestamp", "value"));
+ ASSERT_NOK(writer.Delete("another_key", "timestamp_of_unexpected_size"));
+
+ ASSERT_NOK(writer.Put("key_without_timestamp", "value"));
+ ASSERT_NOK(writer.Merge("another_key_missing_a_timestamp", "merge_operand"));
+ ASSERT_NOK(writer.Delete("yet_another_key_still_no_timestamp"));
+ ASSERT_NOK(writer.DeleteRange("begin_key_timestamp_absent",
+ "end_key_with_a_complete_lack_of_timestamps"));
+}
+
+} // namespace ROCKSDB_NAMESPACE
+
+int main(int argc, char** argv) {
+ ROCKSDB_NAMESPACE::port::InstallStackTraceHandler();
+ ::testing::InitGoogleTest(&argc, argv);
+ RegisterCustomObjects(argc, argv);
+ return RUN_ALL_TESTS();
+}
+
+#else
+#include <stdio.h>
+
+int main(int /*argc*/, char** /*argv*/) {
+ fprintf(stderr,
+ "SKIPPED as SstFileReader is not supported in ROCKSDB_LITE\n");
+ return 0;
+}
+
+#endif // ROCKSDB_LITE
diff --git a/src/rocksdb/table/sst_file_writer.cc b/src/rocksdb/table/sst_file_writer.cc
new file mode 100644
index 000000000..16d11efd3
--- /dev/null
+++ b/src/rocksdb/table/sst_file_writer.cc
@@ -0,0 +1,427 @@
+// Copyright (c) 2011-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 "rocksdb/sst_file_writer.h"
+
+#include <vector>
+
+#include "db/db_impl/db_impl.h"
+#include "db/dbformat.h"
+#include "file/writable_file_writer.h"
+#include "rocksdb/file_system.h"
+#include "rocksdb/table.h"
+#include "table/block_based/block_based_table_builder.h"
+#include "table/sst_file_writer_collectors.h"
+#include "test_util/sync_point.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+const std::string ExternalSstFilePropertyNames::kVersion =
+ "rocksdb.external_sst_file.version";
+const std::string ExternalSstFilePropertyNames::kGlobalSeqno =
+ "rocksdb.external_sst_file.global_seqno";
+
+#ifndef ROCKSDB_LITE
+
+const size_t kFadviseTrigger = 1024 * 1024; // 1MB
+
+struct SstFileWriter::Rep {
+ Rep(const EnvOptions& _env_options, const Options& options,
+ Env::IOPriority _io_priority, const Comparator* _user_comparator,
+ ColumnFamilyHandle* _cfh, bool _invalidate_page_cache, bool _skip_filters,
+ std::string _db_session_id)
+ : env_options(_env_options),
+ ioptions(options),
+ mutable_cf_options(options),
+ io_priority(_io_priority),
+ internal_comparator(_user_comparator),
+ cfh(_cfh),
+ invalidate_page_cache(_invalidate_page_cache),
+ skip_filters(_skip_filters),
+ db_session_id(_db_session_id) {}
+
+ std::unique_ptr<WritableFileWriter> file_writer;
+ std::unique_ptr<TableBuilder> builder;
+ EnvOptions env_options;
+ ImmutableOptions ioptions;
+ MutableCFOptions mutable_cf_options;
+ Env::IOPriority io_priority;
+ InternalKeyComparator internal_comparator;
+ ExternalSstFileInfo file_info;
+ InternalKey ikey;
+ std::string column_family_name;
+ ColumnFamilyHandle* cfh;
+ // If true, We will give the OS a hint that this file pages is not needed
+ // every time we write 1MB to the file.
+ bool invalidate_page_cache;
+ // The size of the file during the last time we called Fadvise to remove
+ // cached pages from page cache.
+ uint64_t last_fadvise_size = 0;
+ bool skip_filters;
+ std::string db_session_id;
+ uint64_t next_file_number = 1;
+
+ Status AddImpl(const Slice& user_key, const Slice& value,
+ ValueType value_type) {
+ if (!builder) {
+ return Status::InvalidArgument("File is not opened");
+ }
+
+ if (file_info.num_entries == 0) {
+ file_info.smallest_key.assign(user_key.data(), user_key.size());
+ } else {
+ if (internal_comparator.user_comparator()->Compare(
+ user_key, file_info.largest_key) <= 0) {
+ // Make sure that keys are added in order
+ return Status::InvalidArgument(
+ "Keys must be added in strict ascending order.");
+ }
+ }
+
+ assert(value_type == kTypeValue || value_type == kTypeMerge ||
+ value_type == kTypeDeletion ||
+ value_type == kTypeDeletionWithTimestamp);
+
+ constexpr SequenceNumber sequence_number = 0;
+
+ ikey.Set(user_key, sequence_number, value_type);
+
+ builder->Add(ikey.Encode(), value);
+
+ // update file info
+ file_info.num_entries++;
+ file_info.largest_key.assign(user_key.data(), user_key.size());
+ file_info.file_size = builder->FileSize();
+
+ InvalidatePageCache(false /* closing */).PermitUncheckedError();
+ return Status::OK();
+ }
+
+ Status Add(const Slice& user_key, const Slice& value, ValueType value_type) {
+ if (internal_comparator.user_comparator()->timestamp_size() != 0) {
+ return Status::InvalidArgument("Timestamp size mismatch");
+ }
+
+ return AddImpl(user_key, value, value_type);
+ }
+
+ Status Add(const Slice& user_key, const Slice& timestamp, const Slice& value,
+ ValueType value_type) {
+ const size_t timestamp_size = timestamp.size();
+
+ if (internal_comparator.user_comparator()->timestamp_size() !=
+ timestamp_size) {
+ return Status::InvalidArgument("Timestamp size mismatch");
+ }
+
+ const size_t user_key_size = user_key.size();
+
+ if (user_key.data() + user_key_size == timestamp.data()) {
+ Slice user_key_with_ts(user_key.data(), user_key_size + timestamp_size);
+ return AddImpl(user_key_with_ts, value, value_type);
+ }
+
+ std::string user_key_with_ts;
+ user_key_with_ts.reserve(user_key_size + timestamp_size);
+ user_key_with_ts.append(user_key.data(), user_key_size);
+ user_key_with_ts.append(timestamp.data(), timestamp_size);
+
+ return AddImpl(user_key_with_ts, value, value_type);
+ }
+
+ Status DeleteRangeImpl(const Slice& begin_key, const Slice& end_key) {
+ if (!builder) {
+ return Status::InvalidArgument("File is not opened");
+ }
+ RangeTombstone tombstone(begin_key, end_key, 0 /* Sequence Number */);
+ if (file_info.num_range_del_entries == 0) {
+ file_info.smallest_range_del_key.assign(tombstone.start_key_.data(),
+ tombstone.start_key_.size());
+ file_info.largest_range_del_key.assign(tombstone.end_key_.data(),
+ tombstone.end_key_.size());
+ } else {
+ if (internal_comparator.user_comparator()->Compare(
+ tombstone.start_key_, file_info.smallest_range_del_key) < 0) {
+ file_info.smallest_range_del_key.assign(tombstone.start_key_.data(),
+ tombstone.start_key_.size());
+ }
+ if (internal_comparator.user_comparator()->Compare(
+ tombstone.end_key_, file_info.largest_range_del_key) > 0) {
+ file_info.largest_range_del_key.assign(tombstone.end_key_.data(),
+ tombstone.end_key_.size());
+ }
+ }
+
+ auto ikey_and_end_key = tombstone.Serialize();
+ builder->Add(ikey_and_end_key.first.Encode(), ikey_and_end_key.second);
+
+ // update file info
+ file_info.num_range_del_entries++;
+ file_info.file_size = builder->FileSize();
+
+ InvalidatePageCache(false /* closing */).PermitUncheckedError();
+ return Status::OK();
+ }
+
+ Status DeleteRange(const Slice& begin_key, const Slice& end_key) {
+ if (internal_comparator.user_comparator()->timestamp_size() != 0) {
+ return Status::InvalidArgument("Timestamp size mismatch");
+ }
+ return DeleteRangeImpl(begin_key, end_key);
+ }
+
+ // begin_key and end_key should be users keys without timestamp.
+ Status DeleteRange(const Slice& begin_key, const Slice& end_key,
+ const Slice& timestamp) {
+ const size_t timestamp_size = timestamp.size();
+
+ if (internal_comparator.user_comparator()->timestamp_size() !=
+ timestamp_size) {
+ return Status::InvalidArgument("Timestamp size mismatch");
+ }
+
+ const size_t begin_key_size = begin_key.size();
+ const size_t end_key_size = end_key.size();
+ if (begin_key.data() + begin_key_size == timestamp.data() ||
+ end_key.data() + begin_key_size == timestamp.data()) {
+ assert(memcmp(begin_key.data() + begin_key_size,
+ end_key.data() + end_key_size, timestamp_size) == 0);
+ Slice begin_key_with_ts(begin_key.data(),
+ begin_key_size + timestamp_size);
+ Slice end_key_with_ts(end_key.data(), end_key.size() + timestamp_size);
+ return DeleteRangeImpl(begin_key_with_ts, end_key_with_ts);
+ }
+ std::string begin_key_with_ts;
+ begin_key_with_ts.reserve(begin_key_size + timestamp_size);
+ begin_key_with_ts.append(begin_key.data(), begin_key_size);
+ begin_key_with_ts.append(timestamp.data(), timestamp_size);
+ std::string end_key_with_ts;
+ end_key_with_ts.reserve(end_key_size + timestamp_size);
+ end_key_with_ts.append(end_key.data(), end_key_size);
+ end_key_with_ts.append(timestamp.data(), timestamp_size);
+ return DeleteRangeImpl(begin_key_with_ts, end_key_with_ts);
+ }
+
+ Status InvalidatePageCache(bool closing) {
+ Status s = Status::OK();
+ if (invalidate_page_cache == false) {
+ // Fadvise disabled
+ return s;
+ }
+ uint64_t bytes_since_last_fadvise = builder->FileSize() - last_fadvise_size;
+ if (bytes_since_last_fadvise > kFadviseTrigger || closing) {
+ TEST_SYNC_POINT_CALLBACK("SstFileWriter::Rep::InvalidatePageCache",
+ &(bytes_since_last_fadvise));
+ // Tell the OS that we don't need this file in page cache
+ s = file_writer->InvalidateCache(0, 0);
+ if (s.IsNotSupported()) {
+ // NotSupported is fine as it could be a file type that doesn't use page
+ // cache.
+ s = Status::OK();
+ }
+ last_fadvise_size = builder->FileSize();
+ }
+ return s;
+ }
+};
+
+SstFileWriter::SstFileWriter(const EnvOptions& env_options,
+ const Options& options,
+ const Comparator* user_comparator,
+ ColumnFamilyHandle* column_family,
+ bool invalidate_page_cache,
+ Env::IOPriority io_priority, bool skip_filters)
+ : rep_(new Rep(env_options, options, io_priority, user_comparator,
+ column_family, invalidate_page_cache, skip_filters,
+ DBImpl::GenerateDbSessionId(options.env))) {
+ // SstFileWriter is used to create sst files that can be added to database
+ // later. Therefore, no real db_id and db_session_id are associated with it.
+ // Here we mimic the way db_session_id behaves by getting a db_session_id
+ // for each SstFileWriter, and (later below) assign unique file numbers
+ // in the table properties. The db_id is set to be "SST Writer" for clarity.
+
+ rep_->file_info.file_size = 0;
+}
+
+SstFileWriter::~SstFileWriter() {
+ if (rep_->builder) {
+ // User did not call Finish() or Finish() failed, we need to
+ // abandon the builder.
+ rep_->builder->Abandon();
+ }
+}
+
+Status SstFileWriter::Open(const std::string& file_path) {
+ Rep* r = rep_.get();
+ Status s;
+ std::unique_ptr<FSWritableFile> sst_file;
+ FileOptions cur_file_opts(r->env_options);
+ s = r->ioptions.env->GetFileSystem()->NewWritableFile(
+ file_path, cur_file_opts, &sst_file, nullptr);
+ if (!s.ok()) {
+ return s;
+ }
+
+ sst_file->SetIOPriority(r->io_priority);
+
+ CompressionType compression_type;
+ CompressionOptions compression_opts;
+ if (r->mutable_cf_options.bottommost_compression !=
+ kDisableCompressionOption) {
+ compression_type = r->mutable_cf_options.bottommost_compression;
+ if (r->mutable_cf_options.bottommost_compression_opts.enabled) {
+ compression_opts = r->mutable_cf_options.bottommost_compression_opts;
+ } else {
+ compression_opts = r->mutable_cf_options.compression_opts;
+ }
+ } else if (!r->mutable_cf_options.compression_per_level.empty()) {
+ // Use the compression of the last level if we have per level compression
+ compression_type = *(r->mutable_cf_options.compression_per_level.rbegin());
+ compression_opts = r->mutable_cf_options.compression_opts;
+ } else {
+ compression_type = r->mutable_cf_options.compression;
+ compression_opts = r->mutable_cf_options.compression_opts;
+ }
+
+ IntTblPropCollectorFactories int_tbl_prop_collector_factories;
+
+ // SstFileWriter properties collector to add SstFileWriter version.
+ int_tbl_prop_collector_factories.emplace_back(
+ new SstFileWriterPropertiesCollectorFactory(2 /* version */,
+ 0 /* global_seqno*/));
+
+ // User collector factories
+ auto user_collector_factories =
+ r->ioptions.table_properties_collector_factories;
+ for (size_t i = 0; i < user_collector_factories.size(); i++) {
+ int_tbl_prop_collector_factories.emplace_back(
+ new UserKeyTablePropertiesCollectorFactory(
+ user_collector_factories[i]));
+ }
+ int unknown_level = -1;
+ uint32_t cf_id;
+
+ if (r->cfh != nullptr) {
+ // user explicitly specified that this file will be ingested into cfh,
+ // we can persist this information in the file.
+ cf_id = r->cfh->GetID();
+ r->column_family_name = r->cfh->GetName();
+ } else {
+ r->column_family_name = "";
+ cf_id = TablePropertiesCollectorFactory::Context::kUnknownColumnFamily;
+ }
+
+ // TODO: it would be better to set oldest_key_time to be used for getting the
+ // approximate time of ingested keys.
+ TableBuilderOptions table_builder_options(
+ r->ioptions, r->mutable_cf_options, r->internal_comparator,
+ &int_tbl_prop_collector_factories, compression_type, compression_opts,
+ cf_id, r->column_family_name, unknown_level, false /* is_bottommost */,
+ TableFileCreationReason::kMisc, 0 /* oldest_key_time */,
+ 0 /* file_creation_time */, "SST Writer" /* db_id */, r->db_session_id,
+ 0 /* target_file_size */, r->next_file_number);
+ // External SST files used to each get a unique session id. Now for
+ // slightly better uniqueness probability in constructing cache keys, we
+ // assign fake file numbers to each file (into table properties) and keep
+ // the same session id for the life of the SstFileWriter.
+ r->next_file_number++;
+ // XXX: when we can remove skip_filters from the SstFileWriter public API
+ // we can remove it from TableBuilderOptions.
+ table_builder_options.skip_filters = r->skip_filters;
+ FileTypeSet tmp_set = r->ioptions.checksum_handoff_file_types;
+ r->file_writer.reset(new WritableFileWriter(
+ std::move(sst_file), file_path, r->env_options, r->ioptions.clock,
+ nullptr /* io_tracer */, nullptr /* stats */, r->ioptions.listeners,
+ r->ioptions.file_checksum_gen_factory.get(),
+ tmp_set.Contains(FileType::kTableFile), false));
+
+ // TODO(tec) : If table_factory is using compressed block cache, we will
+ // be adding the external sst file blocks into it, which is wasteful.
+ r->builder.reset(r->ioptions.table_factory->NewTableBuilder(
+ table_builder_options, r->file_writer.get()));
+
+ r->file_info = ExternalSstFileInfo();
+ r->file_info.file_path = file_path;
+ r->file_info.version = 2;
+ return s;
+}
+
+Status SstFileWriter::Add(const Slice& user_key, const Slice& value) {
+ return rep_->Add(user_key, value, ValueType::kTypeValue);
+}
+
+Status SstFileWriter::Put(const Slice& user_key, const Slice& value) {
+ return rep_->Add(user_key, value, ValueType::kTypeValue);
+}
+
+Status SstFileWriter::Put(const Slice& user_key, const Slice& timestamp,
+ const Slice& value) {
+ return rep_->Add(user_key, timestamp, value, ValueType::kTypeValue);
+}
+
+Status SstFileWriter::Merge(const Slice& user_key, const Slice& value) {
+ return rep_->Add(user_key, value, ValueType::kTypeMerge);
+}
+
+Status SstFileWriter::Delete(const Slice& user_key) {
+ return rep_->Add(user_key, Slice(), ValueType::kTypeDeletion);
+}
+
+Status SstFileWriter::Delete(const Slice& user_key, const Slice& timestamp) {
+ return rep_->Add(user_key, timestamp, Slice(),
+ ValueType::kTypeDeletionWithTimestamp);
+}
+
+Status SstFileWriter::DeleteRange(const Slice& begin_key,
+ const Slice& end_key) {
+ return rep_->DeleteRange(begin_key, end_key);
+}
+
+Status SstFileWriter::DeleteRange(const Slice& begin_key, const Slice& end_key,
+ const Slice& timestamp) {
+ return rep_->DeleteRange(begin_key, end_key, timestamp);
+}
+
+Status SstFileWriter::Finish(ExternalSstFileInfo* file_info) {
+ Rep* r = rep_.get();
+ if (!r->builder) {
+ return Status::InvalidArgument("File is not opened");
+ }
+ if (r->file_info.num_entries == 0 &&
+ r->file_info.num_range_del_entries == 0) {
+ return Status::InvalidArgument("Cannot create sst file with no entries");
+ }
+
+ Status s = r->builder->Finish();
+ r->file_info.file_size = r->builder->FileSize();
+
+ if (s.ok()) {
+ s = r->file_writer->Sync(r->ioptions.use_fsync);
+ r->InvalidatePageCache(true /* closing */).PermitUncheckedError();
+ if (s.ok()) {
+ s = r->file_writer->Close();
+ }
+ }
+ if (s.ok()) {
+ r->file_info.file_checksum = r->file_writer->GetFileChecksum();
+ r->file_info.file_checksum_func_name =
+ r->file_writer->GetFileChecksumFuncName();
+ }
+ if (!s.ok()) {
+ r->ioptions.env->DeleteFile(r->file_info.file_path);
+ }
+
+ if (file_info != nullptr) {
+ *file_info = r->file_info;
+ }
+
+ r->builder.reset();
+ return s;
+}
+
+uint64_t SstFileWriter::FileSize() { return rep_->file_info.file_size; }
+#endif // !ROCKSDB_LITE
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/sst_file_writer_collectors.h b/src/rocksdb/table/sst_file_writer_collectors.h
new file mode 100644
index 000000000..486315fb5
--- /dev/null
+++ b/src/rocksdb/table/sst_file_writer_collectors.h
@@ -0,0 +1,95 @@
+// Copyright (c) 2011-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).
+
+#pragma once
+#include <string>
+
+#include "db/table_properties_collector.h"
+#include "rocksdb/types.h"
+#include "util/coding.h"
+#include "util/string_util.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+// Table Properties that are specific to tables created by SstFileWriter.
+struct ExternalSstFilePropertyNames {
+ // value of this property is a fixed uint32 number.
+ static const std::string kVersion;
+ // value of this property is a fixed uint64 number.
+ static const std::string kGlobalSeqno;
+};
+
+// PropertiesCollector used to add properties specific to tables
+// generated by SstFileWriter
+class SstFileWriterPropertiesCollector : public IntTblPropCollector {
+ public:
+ explicit SstFileWriterPropertiesCollector(int32_t version,
+ SequenceNumber global_seqno)
+ : version_(version), global_seqno_(global_seqno) {}
+
+ virtual Status InternalAdd(const Slice& /*key*/, const Slice& /*value*/,
+ uint64_t /*file_size*/) override {
+ // Intentionally left blank. Have no interest in collecting stats for
+ // individual key/value pairs.
+ return Status::OK();
+ }
+
+ virtual void BlockAdd(uint64_t /* block_uncomp_bytes */,
+ uint64_t /* block_compressed_bytes_fast */,
+ uint64_t /* block_compressed_bytes_slow */) override {
+ // Intentionally left blank. No interest in collecting stats for
+ // blocks.
+ return;
+ }
+
+ virtual Status Finish(UserCollectedProperties* properties) override {
+ // File version
+ std::string version_val;
+ PutFixed32(&version_val, static_cast<uint32_t>(version_));
+ properties->insert({ExternalSstFilePropertyNames::kVersion, version_val});
+
+ // Global Sequence number
+ std::string seqno_val;
+ PutFixed64(&seqno_val, static_cast<uint64_t>(global_seqno_));
+ properties->insert({ExternalSstFilePropertyNames::kGlobalSeqno, seqno_val});
+
+ return Status::OK();
+ }
+
+ virtual const char* Name() const override {
+ return "SstFileWriterPropertiesCollector";
+ }
+
+ virtual UserCollectedProperties GetReadableProperties() const override {
+ return {{ExternalSstFilePropertyNames::kVersion, std::to_string(version_)}};
+ }
+
+ private:
+ int32_t version_;
+ SequenceNumber global_seqno_;
+};
+
+class SstFileWriterPropertiesCollectorFactory
+ : public IntTblPropCollectorFactory {
+ public:
+ explicit SstFileWriterPropertiesCollectorFactory(int32_t version,
+ SequenceNumber global_seqno)
+ : version_(version), global_seqno_(global_seqno) {}
+
+ virtual IntTblPropCollector* CreateIntTblPropCollector(
+ uint32_t /*column_family_id*/, int /* level_at_creation */) override {
+ return new SstFileWriterPropertiesCollector(version_, global_seqno_);
+ }
+
+ virtual const char* Name() const override {
+ return "SstFileWriterPropertiesCollector";
+ }
+
+ private:
+ int32_t version_;
+ SequenceNumber global_seqno_;
+};
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/table_builder.h b/src/rocksdb/table/table_builder.h
new file mode 100644
index 000000000..1790f33b1
--- /dev/null
+++ b/src/rocksdb/table/table_builder.h
@@ -0,0 +1,219 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#pragma once
+
+#include <stdint.h>
+
+#include <string>
+#include <utility>
+#include <vector>
+
+#include "db/dbformat.h"
+#include "db/seqno_to_time_mapping.h"
+#include "db/table_properties_collector.h"
+#include "file/writable_file_writer.h"
+#include "options/cf_options.h"
+#include "rocksdb/options.h"
+#include "rocksdb/table_properties.h"
+#include "table/unique_id_impl.h"
+#include "trace_replay/block_cache_tracer.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class Slice;
+class Status;
+
+struct TableReaderOptions {
+ // @param skip_filters Disables loading/accessing the filter block
+ TableReaderOptions(
+ const ImmutableOptions& _ioptions,
+ const std::shared_ptr<const SliceTransform>& _prefix_extractor,
+ const EnvOptions& _env_options,
+ const InternalKeyComparator& _internal_comparator,
+ bool _skip_filters = false, bool _immortal = false,
+ bool _force_direct_prefetch = false, int _level = -1,
+ BlockCacheTracer* const _block_cache_tracer = nullptr,
+ size_t _max_file_size_for_l0_meta_pin = 0,
+ const std::string& _cur_db_session_id = "", uint64_t _cur_file_num = 0,
+ UniqueId64x2 _unique_id = {}, SequenceNumber _largest_seqno = 0)
+ : ioptions(_ioptions),
+ prefix_extractor(_prefix_extractor),
+ env_options(_env_options),
+ internal_comparator(_internal_comparator),
+ skip_filters(_skip_filters),
+ immortal(_immortal),
+ force_direct_prefetch(_force_direct_prefetch),
+ level(_level),
+ largest_seqno(_largest_seqno),
+ block_cache_tracer(_block_cache_tracer),
+ max_file_size_for_l0_meta_pin(_max_file_size_for_l0_meta_pin),
+ cur_db_session_id(_cur_db_session_id),
+ cur_file_num(_cur_file_num),
+ unique_id(_unique_id) {}
+
+ const ImmutableOptions& ioptions;
+ const std::shared_ptr<const SliceTransform>& prefix_extractor;
+ const EnvOptions& env_options;
+ const InternalKeyComparator& internal_comparator;
+ // This is only used for BlockBasedTable (reader)
+ bool skip_filters;
+ // Whether the table will be valid as long as the DB is open
+ bool immortal;
+ // When data prefetching is needed, even if direct I/O is off, read data to
+ // fetch into RocksDB's buffer, rather than relying
+ // RandomAccessFile::Prefetch().
+ bool force_direct_prefetch;
+ // What level this table/file is on, -1 for "not set, don't know." Used
+ // for level-specific statistics.
+ int level;
+ // largest seqno in the table (or 0 means unknown???)
+ SequenceNumber largest_seqno;
+ BlockCacheTracer* const block_cache_tracer;
+ // Largest L0 file size whose meta-blocks may be pinned (can be zero when
+ // unknown).
+ const size_t max_file_size_for_l0_meta_pin;
+
+ std::string cur_db_session_id;
+
+ uint64_t cur_file_num;
+
+ // Known unique_id or {}, kNullUniqueId64x2 means unknown
+ UniqueId64x2 unique_id;
+};
+
+struct TableBuilderOptions {
+ TableBuilderOptions(
+ const ImmutableOptions& _ioptions, const MutableCFOptions& _moptions,
+ const InternalKeyComparator& _internal_comparator,
+ const IntTblPropCollectorFactories* _int_tbl_prop_collector_factories,
+ CompressionType _compression_type,
+ const CompressionOptions& _compression_opts, uint32_t _column_family_id,
+ const std::string& _column_family_name, int _level,
+ bool _is_bottommost = false,
+ TableFileCreationReason _reason = TableFileCreationReason::kMisc,
+ const int64_t _oldest_key_time = 0,
+ const uint64_t _file_creation_time = 0, const std::string& _db_id = "",
+ const std::string& _db_session_id = "",
+ const uint64_t _target_file_size = 0, const uint64_t _cur_file_num = 0)
+ : ioptions(_ioptions),
+ moptions(_moptions),
+ internal_comparator(_internal_comparator),
+ int_tbl_prop_collector_factories(_int_tbl_prop_collector_factories),
+ compression_type(_compression_type),
+ compression_opts(_compression_opts),
+ column_family_id(_column_family_id),
+ column_family_name(_column_family_name),
+ oldest_key_time(_oldest_key_time),
+ target_file_size(_target_file_size),
+ file_creation_time(_file_creation_time),
+ db_id(_db_id),
+ db_session_id(_db_session_id),
+ level_at_creation(_level),
+ is_bottommost(_is_bottommost),
+ reason(_reason),
+ cur_file_num(_cur_file_num) {}
+
+ const ImmutableOptions& ioptions;
+ const MutableCFOptions& moptions;
+ const InternalKeyComparator& internal_comparator;
+ const IntTblPropCollectorFactories* int_tbl_prop_collector_factories;
+ const CompressionType compression_type;
+ const CompressionOptions& compression_opts;
+ const uint32_t column_family_id;
+ const std::string& column_family_name;
+ const int64_t oldest_key_time;
+ const uint64_t target_file_size;
+ const uint64_t file_creation_time;
+ const std::string db_id;
+ const std::string db_session_id;
+ // BEGIN for FilterBuildingContext
+ const int level_at_creation;
+ const bool is_bottommost;
+ const TableFileCreationReason reason;
+ // END for FilterBuildingContext
+
+ // XXX: only used by BlockBasedTableBuilder for SstFileWriter. If you
+ // want to skip filters, that should be (for example) null filter_policy
+ // in the table options of the ioptions.table_factory
+ bool skip_filters = false;
+ const uint64_t cur_file_num;
+};
+
+// TableBuilder provides the interface used to build a Table
+// (an immutable and sorted map from keys to values).
+//
+// Multiple threads can invoke const methods on a TableBuilder without
+// external synchronization, but if any of the threads may call a
+// non-const method, all threads accessing the same TableBuilder must use
+// external synchronization.
+class TableBuilder {
+ public:
+ // REQUIRES: Either Finish() or Abandon() has been called.
+ virtual ~TableBuilder() {}
+
+ // Add key,value to the table being constructed.
+ // REQUIRES: key is after any previously added key according to comparator.
+ // REQUIRES: Finish(), Abandon() have not been called
+ virtual void Add(const Slice& key, const Slice& value) = 0;
+
+ // Return non-ok iff some error has been detected.
+ virtual Status status() const = 0;
+
+ // Return non-ok iff some error happens during IO.
+ virtual IOStatus io_status() const = 0;
+
+ // Finish building the table.
+ // REQUIRES: Finish(), Abandon() have not been called
+ virtual Status Finish() = 0;
+
+ // Indicate that the contents of this builder should be abandoned.
+ // If the caller is not going to call Finish(), it must call Abandon()
+ // before destroying this builder.
+ // REQUIRES: Finish(), Abandon() have not been called
+ virtual void Abandon() = 0;
+
+ // Number of calls to Add() so far.
+ virtual uint64_t NumEntries() const = 0;
+
+ // Whether the output file is completely empty. It has neither entries
+ // or tombstones.
+ virtual bool IsEmpty() const {
+ return NumEntries() == 0 && GetTableProperties().num_range_deletions == 0;
+ }
+
+ // Size of the file generated so far. If invoked after a successful
+ // Finish() call, returns the size of the final generated file.
+ virtual uint64_t FileSize() const = 0;
+
+ // Estimated size of the file generated so far. This is used when
+ // FileSize() cannot estimate final SST size, e.g. parallel compression
+ // is enabled.
+ virtual uint64_t EstimatedFileSize() const { return FileSize(); }
+
+ // If the user defined table properties collector suggest the file to
+ // be further compacted.
+ virtual bool NeedCompact() const { return false; }
+
+ // Returns table properties
+ virtual TableProperties GetTableProperties() const = 0;
+
+ // Return file checksum
+ virtual std::string GetFileChecksum() const = 0;
+
+ // Return file checksum function name
+ virtual const char* GetFileChecksumFuncName() const = 0;
+
+ // Set the sequence number to time mapping
+ virtual void SetSeqnoTimeTableProperties(
+ const std::string& /*encoded_seqno_to_time_mapping*/,
+ uint64_t /*oldest_ancestor_time*/){};
+};
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/table_factory.cc b/src/rocksdb/table/table_factory.cc
new file mode 100644
index 000000000..fc5c5ccde
--- /dev/null
+++ b/src/rocksdb/table/table_factory.cc
@@ -0,0 +1,65 @@
+// Copyright (c) Facebook, Inc. and its affiliates. All Rights Reserved.
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#include <mutex>
+
+#include "rocksdb/convenience.h"
+#include "rocksdb/table.h"
+#include "rocksdb/utilities/customizable_util.h"
+#include "rocksdb/utilities/object_registry.h"
+#include "table/block_based/block_based_table_factory.h"
+#include "table/cuckoo/cuckoo_table_factory.h"
+#include "table/plain/plain_table_factory.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+static void RegisterTableFactories(const std::string& /*arg*/) {
+#ifndef ROCKSDB_LITE
+ static std::once_flag loaded;
+ std::call_once(loaded, []() {
+ auto library = ObjectLibrary::Default();
+ library->AddFactory<TableFactory>(
+ TableFactory::kBlockBasedTableName(),
+ [](const std::string& /*uri*/, std::unique_ptr<TableFactory>* guard,
+ std::string* /* errmsg */) {
+ guard->reset(new BlockBasedTableFactory());
+ return guard->get();
+ });
+ library->AddFactory<TableFactory>(
+ TableFactory::kPlainTableName(),
+ [](const std::string& /*uri*/, std::unique_ptr<TableFactory>* guard,
+ std::string* /* errmsg */) {
+ guard->reset(new PlainTableFactory());
+ return guard->get();
+ });
+ library->AddFactory<TableFactory>(
+ TableFactory::kCuckooTableName(),
+ [](const std::string& /*uri*/, std::unique_ptr<TableFactory>* guard,
+ std::string* /* errmsg */) {
+ guard->reset(new CuckooTableFactory());
+ return guard->get();
+ });
+ });
+#endif // ROCKSDB_LITE
+}
+
+static bool LoadFactory(const std::string& name,
+ std::shared_ptr<TableFactory>* factory) {
+ if (name == TableFactory::kBlockBasedTableName()) {
+ factory->reset(new BlockBasedTableFactory());
+ return true;
+ } else {
+ return false;
+ }
+}
+
+Status TableFactory::CreateFromString(const ConfigOptions& config_options,
+ const std::string& value,
+ std::shared_ptr<TableFactory>* factory) {
+ RegisterTableFactories("");
+ return LoadSharedObject<TableFactory>(config_options, value, LoadFactory,
+ factory);
+}
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/table_properties.cc b/src/rocksdb/table/table_properties.cc
new file mode 100644
index 000000000..b382281f8
--- /dev/null
+++ b/src/rocksdb/table/table_properties.cc
@@ -0,0 +1,349 @@
+// Copyright (c) 2011-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 "rocksdb/table_properties.h"
+
+#include "db/seqno_to_time_mapping.h"
+#include "port/malloc.h"
+#include "port/port.h"
+#include "rocksdb/env.h"
+#include "rocksdb/unique_id.h"
+#include "table/table_properties_internal.h"
+#include "table/unique_id_impl.h"
+#include "util/random.h"
+#include "util/string_util.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+const uint32_t TablePropertiesCollectorFactory::Context::kUnknownColumnFamily =
+ std::numeric_limits<int32_t>::max();
+
+namespace {
+void AppendProperty(std::string& props, const std::string& key,
+ const std::string& value, const std::string& prop_delim,
+ const std::string& kv_delim) {
+ props.append(key);
+ props.append(kv_delim);
+ props.append(value);
+ props.append(prop_delim);
+}
+
+template <class TValue>
+void AppendProperty(std::string& props, const std::string& key,
+ const TValue& value, const std::string& prop_delim,
+ const std::string& kv_delim) {
+ AppendProperty(props, key, std::to_string(value), prop_delim, kv_delim);
+}
+} // namespace
+
+std::string TableProperties::ToString(const std::string& prop_delim,
+ const std::string& kv_delim) const {
+ std::string result;
+ result.reserve(1024);
+
+ // Basic Info
+ AppendProperty(result, "# data blocks", num_data_blocks, prop_delim,
+ kv_delim);
+ AppendProperty(result, "# entries", num_entries, prop_delim, kv_delim);
+ AppendProperty(result, "# deletions", num_deletions, prop_delim, kv_delim);
+ AppendProperty(result, "# merge operands", num_merge_operands, prop_delim,
+ kv_delim);
+ AppendProperty(result, "# range deletions", num_range_deletions, prop_delim,
+ kv_delim);
+
+ AppendProperty(result, "raw key size", raw_key_size, prop_delim, kv_delim);
+ AppendProperty(result, "raw average key size",
+ num_entries != 0 ? 1.0 * raw_key_size / num_entries : 0.0,
+ prop_delim, kv_delim);
+ AppendProperty(result, "raw value size", raw_value_size, prop_delim,
+ kv_delim);
+ AppendProperty(result, "raw average value size",
+ num_entries != 0 ? 1.0 * raw_value_size / num_entries : 0.0,
+ prop_delim, kv_delim);
+
+ AppendProperty(result, "data block size", data_size, prop_delim, kv_delim);
+ char index_block_size_str[80];
+ snprintf(index_block_size_str, sizeof(index_block_size_str),
+ "index block size (user-key? %d, delta-value? %d)",
+ static_cast<int>(index_key_is_user_key),
+ static_cast<int>(index_value_is_delta_encoded));
+ AppendProperty(result, index_block_size_str, index_size, prop_delim,
+ kv_delim);
+ if (index_partitions != 0) {
+ AppendProperty(result, "# index partitions", index_partitions, prop_delim,
+ kv_delim);
+ AppendProperty(result, "top-level index size", top_level_index_size,
+ prop_delim, kv_delim);
+ }
+ AppendProperty(result, "filter block size", filter_size, prop_delim,
+ kv_delim);
+ AppendProperty(result, "# entries for filter", num_filter_entries, prop_delim,
+ kv_delim);
+ AppendProperty(result, "(estimated) table size",
+ data_size + index_size + filter_size, prop_delim, kv_delim);
+
+ AppendProperty(
+ result, "filter policy name",
+ filter_policy_name.empty() ? std::string("N/A") : filter_policy_name,
+ prop_delim, kv_delim);
+
+ AppendProperty(result, "prefix extractor name",
+ prefix_extractor_name.empty() ? std::string("N/A")
+ : prefix_extractor_name,
+ prop_delim, kv_delim);
+
+ AppendProperty(result, "column family ID",
+ column_family_id ==
+ ROCKSDB_NAMESPACE::TablePropertiesCollectorFactory::
+ Context::kUnknownColumnFamily
+ ? std::string("N/A")
+ : std::to_string(column_family_id),
+ prop_delim, kv_delim);
+ AppendProperty(
+ result, "column family name",
+ column_family_name.empty() ? std::string("N/A") : column_family_name,
+ prop_delim, kv_delim);
+
+ AppendProperty(result, "comparator name",
+ comparator_name.empty() ? std::string("N/A") : comparator_name,
+ prop_delim, kv_delim);
+
+ AppendProperty(
+ result, "merge operator name",
+ merge_operator_name.empty() ? std::string("N/A") : merge_operator_name,
+ prop_delim, kv_delim);
+
+ AppendProperty(result, "property collectors names",
+ property_collectors_names.empty() ? std::string("N/A")
+ : property_collectors_names,
+ prop_delim, kv_delim);
+
+ AppendProperty(
+ result, "SST file compression algo",
+ compression_name.empty() ? std::string("N/A") : compression_name,
+ prop_delim, kv_delim);
+
+ AppendProperty(
+ result, "SST file compression options",
+ compression_options.empty() ? std::string("N/A") : compression_options,
+ prop_delim, kv_delim);
+
+ AppendProperty(result, "creation time", creation_time, prop_delim, kv_delim);
+
+ AppendProperty(result, "time stamp of earliest key", oldest_key_time,
+ prop_delim, kv_delim);
+
+ AppendProperty(result, "file creation time", file_creation_time, prop_delim,
+ kv_delim);
+
+ AppendProperty(result, "slow compression estimated data size",
+ slow_compression_estimated_data_size, prop_delim, kv_delim);
+ AppendProperty(result, "fast compression estimated data size",
+ fast_compression_estimated_data_size, prop_delim, kv_delim);
+
+ // DB identity and DB session ID
+ AppendProperty(result, "DB identity", db_id, prop_delim, kv_delim);
+ AppendProperty(result, "DB session identity", db_session_id, prop_delim,
+ kv_delim);
+ AppendProperty(result, "DB host id", db_host_id, prop_delim, kv_delim);
+ AppendProperty(result, "original file number", orig_file_number, prop_delim,
+ kv_delim);
+
+ // Unique ID, when available
+ std::string id;
+ Status s = GetUniqueIdFromTableProperties(*this, &id);
+ AppendProperty(result, "unique ID",
+ s.ok() ? UniqueIdToHumanString(id) : "N/A", prop_delim,
+ kv_delim);
+
+ SeqnoToTimeMapping seq_time_mapping;
+ s = seq_time_mapping.Add(seqno_to_time_mapping);
+ AppendProperty(result, "Sequence number to time mapping",
+ s.ok() ? seq_time_mapping.ToHumanString() : "N/A", prop_delim,
+ kv_delim);
+
+ return result;
+}
+
+void TableProperties::Add(const TableProperties& tp) {
+ data_size += tp.data_size;
+ index_size += tp.index_size;
+ index_partitions += tp.index_partitions;
+ top_level_index_size += tp.top_level_index_size;
+ index_key_is_user_key += tp.index_key_is_user_key;
+ index_value_is_delta_encoded += tp.index_value_is_delta_encoded;
+ filter_size += tp.filter_size;
+ raw_key_size += tp.raw_key_size;
+ raw_value_size += tp.raw_value_size;
+ num_data_blocks += tp.num_data_blocks;
+ num_entries += tp.num_entries;
+ num_filter_entries += tp.num_filter_entries;
+ num_deletions += tp.num_deletions;
+ num_merge_operands += tp.num_merge_operands;
+ num_range_deletions += tp.num_range_deletions;
+ slow_compression_estimated_data_size +=
+ tp.slow_compression_estimated_data_size;
+ fast_compression_estimated_data_size +=
+ tp.fast_compression_estimated_data_size;
+}
+
+std::map<std::string, uint64_t>
+TableProperties::GetAggregatablePropertiesAsMap() const {
+ std::map<std::string, uint64_t> rv;
+ rv["data_size"] = data_size;
+ rv["index_size"] = index_size;
+ rv["index_partitions"] = index_partitions;
+ rv["top_level_index_size"] = top_level_index_size;
+ rv["filter_size"] = filter_size;
+ rv["raw_key_size"] = raw_key_size;
+ rv["raw_value_size"] = raw_value_size;
+ rv["num_data_blocks"] = num_data_blocks;
+ rv["num_entries"] = num_entries;
+ rv["num_filter_entries"] = num_filter_entries;
+ rv["num_deletions"] = num_deletions;
+ rv["num_merge_operands"] = num_merge_operands;
+ rv["num_range_deletions"] = num_range_deletions;
+ rv["slow_compression_estimated_data_size"] =
+ slow_compression_estimated_data_size;
+ rv["fast_compression_estimated_data_size"] =
+ fast_compression_estimated_data_size;
+ return rv;
+}
+
+// WARNING: manual update to this function is needed
+// whenever a new string property is added to TableProperties
+// to reduce approximation error.
+//
+// TODO: eliminate the need of manually updating this function
+// for new string properties
+std::size_t TableProperties::ApproximateMemoryUsage() const {
+ std::size_t usage = 0;
+#ifdef ROCKSDB_MALLOC_USABLE_SIZE
+ usage += malloc_usable_size((void*)this);
+#else
+ usage += sizeof(*this);
+#endif // ROCKSDB_MALLOC_USABLE_SIZE
+
+ std::size_t string_props_mem_usage =
+ db_id.size() + db_session_id.size() + db_host_id.size() +
+ column_family_name.size() + filter_policy_name.size() +
+ comparator_name.size() + merge_operator_name.size() +
+ prefix_extractor_name.size() + property_collectors_names.size() +
+ compression_name.size() + compression_options.size();
+ usage += string_props_mem_usage;
+
+ for (auto iter = user_collected_properties.begin();
+ iter != user_collected_properties.end(); ++iter) {
+ usage += (iter->first.size() + iter->second.size());
+ }
+
+ return usage;
+}
+
+const std::string TablePropertiesNames::kDbId = "rocksdb.creating.db.identity";
+const std::string TablePropertiesNames::kDbSessionId =
+ "rocksdb.creating.session.identity";
+const std::string TablePropertiesNames::kDbHostId =
+ "rocksdb.creating.host.identity";
+const std::string TablePropertiesNames::kOriginalFileNumber =
+ "rocksdb.original.file.number";
+const std::string TablePropertiesNames::kDataSize = "rocksdb.data.size";
+const std::string TablePropertiesNames::kIndexSize = "rocksdb.index.size";
+const std::string TablePropertiesNames::kIndexPartitions =
+ "rocksdb.index.partitions";
+const std::string TablePropertiesNames::kTopLevelIndexSize =
+ "rocksdb.top-level.index.size";
+const std::string TablePropertiesNames::kIndexKeyIsUserKey =
+ "rocksdb.index.key.is.user.key";
+const std::string TablePropertiesNames::kIndexValueIsDeltaEncoded =
+ "rocksdb.index.value.is.delta.encoded";
+const std::string TablePropertiesNames::kFilterSize = "rocksdb.filter.size";
+const std::string TablePropertiesNames::kRawKeySize = "rocksdb.raw.key.size";
+const std::string TablePropertiesNames::kRawValueSize =
+ "rocksdb.raw.value.size";
+const std::string TablePropertiesNames::kNumDataBlocks =
+ "rocksdb.num.data.blocks";
+const std::string TablePropertiesNames::kNumEntries = "rocksdb.num.entries";
+const std::string TablePropertiesNames::kNumFilterEntries =
+ "rocksdb.num.filter_entries";
+const std::string TablePropertiesNames::kDeletedKeys = "rocksdb.deleted.keys";
+const std::string TablePropertiesNames::kMergeOperands =
+ "rocksdb.merge.operands";
+const std::string TablePropertiesNames::kNumRangeDeletions =
+ "rocksdb.num.range-deletions";
+const std::string TablePropertiesNames::kFilterPolicy = "rocksdb.filter.policy";
+const std::string TablePropertiesNames::kFormatVersion =
+ "rocksdb.format.version";
+const std::string TablePropertiesNames::kFixedKeyLen =
+ "rocksdb.fixed.key.length";
+const std::string TablePropertiesNames::kColumnFamilyId =
+ "rocksdb.column.family.id";
+const std::string TablePropertiesNames::kColumnFamilyName =
+ "rocksdb.column.family.name";
+const std::string TablePropertiesNames::kComparator = "rocksdb.comparator";
+const std::string TablePropertiesNames::kMergeOperator =
+ "rocksdb.merge.operator";
+const std::string TablePropertiesNames::kPrefixExtractorName =
+ "rocksdb.prefix.extractor.name";
+const std::string TablePropertiesNames::kPropertyCollectors =
+ "rocksdb.property.collectors";
+const std::string TablePropertiesNames::kCompression = "rocksdb.compression";
+const std::string TablePropertiesNames::kCompressionOptions =
+ "rocksdb.compression_options";
+const std::string TablePropertiesNames::kCreationTime = "rocksdb.creation.time";
+const std::string TablePropertiesNames::kOldestKeyTime =
+ "rocksdb.oldest.key.time";
+const std::string TablePropertiesNames::kFileCreationTime =
+ "rocksdb.file.creation.time";
+const std::string TablePropertiesNames::kSlowCompressionEstimatedDataSize =
+ "rocksdb.sample_for_compression.slow.data.size";
+const std::string TablePropertiesNames::kFastCompressionEstimatedDataSize =
+ "rocksdb.sample_for_compression.fast.data.size";
+const std::string TablePropertiesNames::kSequenceNumberTimeMapping =
+ "rocksdb.seqno.time.map";
+
+#ifndef NDEBUG
+// WARNING: TEST_SetRandomTableProperties assumes the following layout of
+// TableProperties
+//
+// struct TableProperties {
+// int64_t orig_file_number = 0;
+// ...
+// ... int64_t properties only
+// ...
+// std::string db_id;
+// ...
+// ... std::string properties only
+// ...
+// std::string compression_options;
+// UserCollectedProperties user_collected_properties;
+// ...
+// ... Other extra properties: non-int64_t/non-std::string properties only
+// ...
+// }
+void TEST_SetRandomTableProperties(TableProperties* props) {
+ Random* r = Random::GetTLSInstance();
+ uint64_t* pu = &props->orig_file_number;
+ assert(static_cast<void*>(pu) == static_cast<void*>(props));
+ std::string* ps = &props->db_id;
+ const uint64_t* const pu_end = reinterpret_cast<const uint64_t*>(ps);
+ // Use the last string property's address instead of
+ // the first extra property (e.g `user_collected_properties`)'s address
+ // in the for-loop to avoid advancing pointer to pointing to
+ // potential non-zero padding bytes between these two addresses due to
+ // user_collected_properties's alignment requirement
+ const std::string* const ps_end_inclusive = &props->compression_options;
+
+ for (; pu < pu_end; ++pu) {
+ *pu = r->Next64();
+ }
+ assert(static_cast<void*>(pu) == static_cast<void*>(ps));
+ for (; ps <= ps_end_inclusive; ++ps) {
+ *ps = r->RandomBinaryString(13);
+ }
+}
+#endif
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/table_properties_internal.h b/src/rocksdb/table/table_properties_internal.h
new file mode 100644
index 000000000..5c2a0cb9a
--- /dev/null
+++ b/src/rocksdb/table/table_properties_internal.h
@@ -0,0 +1,14 @@
+// Copyright (c) 2011-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).
+
+#pragma once
+
+#include "rocksdb/table_properties.h"
+
+namespace ROCKSDB_NAMESPACE {
+#ifndef NDEBUG
+void TEST_SetRandomTableProperties(TableProperties* props);
+#endif
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/table_reader.h b/src/rocksdb/table/table_reader.h
new file mode 100644
index 000000000..391072eec
--- /dev/null
+++ b/src/rocksdb/table/table_reader.h
@@ -0,0 +1,184 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#pragma once
+#include <memory>
+
+#include "db/range_tombstone_fragmenter.h"
+#if USE_COROUTINES
+#include "folly/experimental/coro/Coroutine.h"
+#include "folly/experimental/coro/Task.h"
+#endif
+#include "rocksdb/slice_transform.h"
+#include "rocksdb/table_reader_caller.h"
+#include "table/get_context.h"
+#include "table/internal_iterator.h"
+#include "table/multiget_context.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+class Iterator;
+struct ParsedInternalKey;
+class Slice;
+class Arena;
+struct ReadOptions;
+struct TableProperties;
+class GetContext;
+class MultiGetContext;
+
+// A Table (also referred to as SST) is a sorted map from strings to strings.
+// Tables are immutable and persistent. A Table may be safely accessed from
+// multiple threads without external synchronization. Table readers are used
+// for reading various types of table formats supported by rocksdb including
+// BlockBasedTable, PlainTable and CuckooTable format.
+class TableReader {
+ public:
+ virtual ~TableReader() {}
+
+ // Returns a new iterator over the table contents.
+ // The result of NewIterator() is initially invalid (caller must
+ // call one of the Seek methods on the iterator before using it).
+ //
+ // read_options: Must outlive the returned iterator.
+ // arena: If not null, the arena needs to be used to allocate the Iterator.
+ // When destroying the iterator, the caller will not call "delete"
+ // but Iterator::~Iterator() directly. The destructor needs to destroy
+ // all the states but those allocated in arena.
+ // skip_filters: disables checking the bloom filters even if they exist. This
+ // option is effective only for block-based table format.
+ // compaction_readahead_size: its value will only be used if caller =
+ // kCompaction
+ virtual InternalIterator* NewIterator(
+ const ReadOptions& read_options, const SliceTransform* prefix_extractor,
+ Arena* arena, bool skip_filters, TableReaderCaller caller,
+ size_t compaction_readahead_size = 0,
+ bool allow_unprepared_value = false) = 0;
+
+ virtual FragmentedRangeTombstoneIterator* NewRangeTombstoneIterator(
+ const ReadOptions& /*read_options*/) {
+ return nullptr;
+ }
+
+ // 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
+ // bytes, and so includes effects like compression of the underlying data.
+ // E.g., the approximate offset of the last key in the table will
+ // be close to the file length.
+ // TODO(peterd): Since this function is only used for approximate size
+ // from beginning of file, reduce code duplication by removing this
+ // function and letting ApproximateSize take optional start and end, so
+ // that absolute start and end can be specified and optimized without
+ // key / index work.
+ virtual uint64_t ApproximateOffsetOf(const Slice& key,
+ TableReaderCaller caller) = 0;
+
+ // Given start and end keys, return the approximate data size in the file
+ // between the keys. The returned value is in terms of file bytes, and so
+ // includes effects like compression of the underlying data and applicable
+ // portions of metadata including filters and indexes. Nullptr for start or
+ // end (or both) indicates absolute start or end of the table.
+ virtual uint64_t ApproximateSize(const Slice& start, const Slice& end,
+ TableReaderCaller caller) = 0;
+
+ struct Anchor {
+ Anchor(const Slice& _user_key, size_t _range_size)
+ : user_key(_user_key.ToStringView()), range_size(_range_size) {}
+ std::string user_key;
+ size_t range_size;
+ };
+
+ // Now try to return approximately 128 anchor keys.
+ // The last one tends to be the largest key.
+ virtual Status ApproximateKeyAnchors(const ReadOptions& /*read_options*/,
+ std::vector<Anchor>& /*anchors*/) {
+ return Status::NotSupported("ApproximateKeyAnchors() not supported.");
+ }
+
+ // Set up the table for Compaction. Might change some parameters with
+ // posix_fadvise
+ virtual void SetupForCompaction() = 0;
+
+ virtual std::shared_ptr<const TableProperties> GetTableProperties() const = 0;
+
+ // Prepare work that can be done before the real Get()
+ virtual void Prepare(const Slice& /*target*/) {}
+
+ // Report an approximation of how much memory has been used.
+ virtual size_t ApproximateMemoryUsage() const = 0;
+
+ // Calls get_context->SaveValue() repeatedly, starting with
+ // the entry found after a call to Seek(key), until it returns false.
+ // May not make such a call if filter policy says that key is not present.
+ //
+ // get_context->MarkKeyMayExist needs to be called when it is configured to be
+ // memory only and the key is not found in the block cache.
+ //
+ // readOptions is the options for the read
+ // key is the key to search for
+ // skip_filters: disables checking the bloom filters even if they exist. This
+ // option is effective only for block-based table format.
+ virtual Status Get(const ReadOptions& readOptions, const Slice& key,
+ GetContext* get_context,
+ const SliceTransform* prefix_extractor,
+ bool skip_filters = false) = 0;
+
+ // Use bloom filters in the table file, if present, to filter out keys. The
+ // mget_range will be updated to skip keys that get a negative result from
+ // the filter lookup.
+ virtual Status MultiGetFilter(const ReadOptions& /*readOptions*/,
+ const SliceTransform* /*prefix_extractor*/,
+ MultiGetContext::Range* /*mget_range*/) {
+ return Status::NotSupported();
+ }
+
+ virtual void MultiGet(const ReadOptions& readOptions,
+ const MultiGetContext::Range* mget_range,
+ const SliceTransform* prefix_extractor,
+ bool skip_filters = false) {
+ for (auto iter = mget_range->begin(); iter != mget_range->end(); ++iter) {
+ *iter->s = Get(readOptions, iter->ikey, iter->get_context,
+ prefix_extractor, skip_filters);
+ }
+ }
+
+#if USE_COROUTINES
+ virtual folly::coro::Task<void> MultiGetCoroutine(
+ const ReadOptions& readOptions, const MultiGetContext::Range* mget_range,
+ const SliceTransform* prefix_extractor, bool skip_filters = false) {
+ MultiGet(readOptions, mget_range, prefix_extractor, skip_filters);
+ co_return;
+ }
+#endif // USE_COROUTINES
+
+ // Prefetch data corresponding to a give range of keys
+ // Typically this functionality is required for table implementations that
+ // persists the data on a non volatile storage medium like disk/SSD
+ virtual Status Prefetch(const Slice* begin = nullptr,
+ const Slice* end = nullptr) {
+ (void)begin;
+ (void)end;
+ // Default implementation is NOOP.
+ // The child class should implement functionality when applicable
+ return Status::OK();
+ }
+
+ // convert db file to a human readable form
+ virtual Status DumpTable(WritableFile* /*out_file*/) {
+ return Status::NotSupported("DumpTable() not supported");
+ }
+
+ // check whether there is corruption in this db file
+ virtual Status VerifyChecksum(const ReadOptions& /*read_options*/,
+ TableReaderCaller /*caller*/) {
+ return Status::NotSupported("VerifyChecksum() not supported");
+ }
+};
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/table_reader_bench.cc b/src/rocksdb/table/table_reader_bench.cc
new file mode 100644
index 000000000..b13caf68d
--- /dev/null
+++ b/src/rocksdb/table/table_reader_bench.cc
@@ -0,0 +1,349 @@
+// Copyright (c) 2011-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).
+
+#ifndef GFLAGS
+#include <cstdio>
+int main() {
+ fprintf(stderr, "Please install gflags to run rocksdb tools\n");
+ return 1;
+}
+#else
+
+#include "db/db_impl/db_impl.h"
+#include "db/dbformat.h"
+#include "file/random_access_file_reader.h"
+#include "monitoring/histogram.h"
+#include "rocksdb/db.h"
+#include "rocksdb/file_system.h"
+#include "rocksdb/slice_transform.h"
+#include "rocksdb/system_clock.h"
+#include "rocksdb/table.h"
+#include "table/block_based/block_based_table_factory.h"
+#include "table/get_context.h"
+#include "table/internal_iterator.h"
+#include "table/plain/plain_table_factory.h"
+#include "table/table_builder.h"
+#include "test_util/testharness.h"
+#include "test_util/testutil.h"
+#include "util/gflags_compat.h"
+
+using GFLAGS_NAMESPACE::ParseCommandLineFlags;
+using GFLAGS_NAMESPACE::SetUsageMessage;
+
+namespace ROCKSDB_NAMESPACE {
+
+namespace {
+// Make a key that i determines the first 4 characters and j determines the
+// last 4 characters.
+static std::string MakeKey(int i, int j, bool through_db) {
+ char buf[100];
+ snprintf(buf, sizeof(buf), "%04d__key___%04d", i, j);
+ if (through_db) {
+ return std::string(buf);
+ }
+ // If we directly query table, which operates on internal keys
+ // instead of user keys, we need to add 8 bytes of internal
+ // information (row type etc) to user key to make an internal
+ // key.
+ InternalKey key(std::string(buf), 0, ValueType::kTypeValue);
+ return key.Encode().ToString();
+}
+
+uint64_t Now(SystemClock* clock, bool measured_by_nanosecond) {
+ return measured_by_nanosecond ? clock->NowNanos() : clock->NowMicros();
+}
+} // namespace
+
+// A very simple benchmark that.
+// Create a table with roughly numKey1 * numKey2 keys,
+// where there are numKey1 prefixes of the key, each has numKey2 number of
+// distinguished key, differing in the suffix part.
+// If if_query_empty_keys = false, query the existing keys numKey1 * numKey2
+// times randomly.
+// If if_query_empty_keys = true, query numKey1 * numKey2 random empty keys.
+// Print out the total time.
+// If through_db=true, a full DB will be created and queries will be against
+// it. Otherwise, operations will be directly through table level.
+//
+// If for_terator=true, instead of just query one key each time, it queries
+// a range sharing the same prefix.
+namespace {
+void TableReaderBenchmark(Options& opts, EnvOptions& env_options,
+ ReadOptions& read_options, int num_keys1,
+ int num_keys2, int num_iter, int /*prefix_len*/,
+ bool if_query_empty_keys, bool for_iterator,
+ bool through_db, bool measured_by_nanosecond) {
+ ROCKSDB_NAMESPACE::InternalKeyComparator ikc(opts.comparator);
+
+ std::string file_name =
+ test::PerThreadDBPath("rocksdb_table_reader_benchmark");
+ std::string dbname = test::PerThreadDBPath("rocksdb_table_reader_bench_db");
+ WriteOptions wo;
+ Env* env = Env::Default();
+ auto* clock = env->GetSystemClock().get();
+ TableBuilder* tb = nullptr;
+ DB* db = nullptr;
+ Status s;
+ const ImmutableOptions ioptions(opts);
+ const ColumnFamilyOptions cfo(opts);
+ const MutableCFOptions moptions(cfo);
+ std::unique_ptr<WritableFileWriter> file_writer;
+ if (!through_db) {
+ ASSERT_OK(WritableFileWriter::Create(env->GetFileSystem(), file_name,
+ FileOptions(env_options), &file_writer,
+ nullptr));
+
+ IntTblPropCollectorFactories int_tbl_prop_collector_factories;
+
+ int unknown_level = -1;
+ tb = opts.table_factory->NewTableBuilder(
+ TableBuilderOptions(
+ ioptions, moptions, ikc, &int_tbl_prop_collector_factories,
+ CompressionType::kNoCompression, CompressionOptions(),
+ 0 /* column_family_id */, kDefaultColumnFamilyName, unknown_level),
+ file_writer.get());
+ } else {
+ s = DB::Open(opts, dbname, &db);
+ ASSERT_OK(s);
+ ASSERT_TRUE(db != nullptr);
+ }
+ // Populate slightly more than 1M keys
+ for (int i = 0; i < num_keys1; i++) {
+ for (int j = 0; j < num_keys2; j++) {
+ std::string key = MakeKey(i * 2, j, through_db);
+ if (!through_db) {
+ tb->Add(key, key);
+ } else {
+ db->Put(wo, key, key);
+ }
+ }
+ }
+ if (!through_db) {
+ tb->Finish();
+ file_writer->Close();
+ } else {
+ db->Flush(FlushOptions());
+ }
+
+ std::unique_ptr<TableReader> table_reader;
+ if (!through_db) {
+ const auto& fs = env->GetFileSystem();
+ FileOptions fopts(env_options);
+
+ std::unique_ptr<FSRandomAccessFile> raf;
+ s = fs->NewRandomAccessFile(file_name, fopts, &raf, nullptr);
+ if (!s.ok()) {
+ fprintf(stderr, "Create File Error: %s\n", s.ToString().c_str());
+ exit(1);
+ }
+ uint64_t file_size;
+ fs->GetFileSize(file_name, fopts.io_options, &file_size, nullptr);
+ std::unique_ptr<RandomAccessFileReader> file_reader(
+ new RandomAccessFileReader(std::move(raf), file_name));
+ s = opts.table_factory->NewTableReader(
+ TableReaderOptions(ioptions, moptions.prefix_extractor, env_options,
+ ikc),
+ std::move(file_reader), file_size, &table_reader);
+ if (!s.ok()) {
+ fprintf(stderr, "Open Table Error: %s\n", s.ToString().c_str());
+ exit(1);
+ }
+ }
+
+ Random rnd(301);
+ std::string result;
+ HistogramImpl hist;
+
+ for (int it = 0; it < num_iter; it++) {
+ for (int i = 0; i < num_keys1; i++) {
+ for (int j = 0; j < num_keys2; j++) {
+ int r1 = rnd.Uniform(num_keys1) * 2;
+ int r2 = rnd.Uniform(num_keys2);
+ if (if_query_empty_keys) {
+ r1++;
+ r2 = num_keys2 * 2 - r2;
+ }
+
+ if (!for_iterator) {
+ // Query one existing key;
+ std::string key = MakeKey(r1, r2, through_db);
+ uint64_t start_time = Now(clock, measured_by_nanosecond);
+ if (!through_db) {
+ PinnableSlice value;
+ MergeContext merge_context;
+ SequenceNumber max_covering_tombstone_seq = 0;
+ GetContext get_context(
+ ioptions.user_comparator, ioptions.merge_operator.get(),
+ ioptions.logger, ioptions.stats, GetContext::kNotFound,
+ Slice(key), &value, /*columns=*/nullptr, /*timestamp=*/nullptr,
+ &merge_context, true, &max_covering_tombstone_seq, clock);
+ s = table_reader->Get(read_options, key, &get_context, nullptr);
+ } else {
+ s = db->Get(read_options, key, &result);
+ }
+ hist.Add(Now(clock, measured_by_nanosecond) - start_time);
+ } else {
+ int r2_len;
+ if (if_query_empty_keys) {
+ r2_len = 0;
+ } else {
+ r2_len = rnd.Uniform(num_keys2) + 1;
+ if (r2_len + r2 > num_keys2) {
+ r2_len = num_keys2 - r2;
+ }
+ }
+ std::string start_key = MakeKey(r1, r2, through_db);
+ std::string end_key = MakeKey(r1, r2 + r2_len, through_db);
+ uint64_t total_time = 0;
+ uint64_t start_time = Now(clock, measured_by_nanosecond);
+ Iterator* iter = nullptr;
+ InternalIterator* iiter = nullptr;
+ if (!through_db) {
+ iiter = table_reader->NewIterator(
+ read_options, /*prefix_extractor=*/nullptr, /*arena=*/nullptr,
+ /*skip_filters=*/false, TableReaderCaller::kUncategorized);
+ } else {
+ iter = db->NewIterator(read_options);
+ }
+ int count = 0;
+ for (through_db ? iter->Seek(start_key) : iiter->Seek(start_key);
+ through_db ? iter->Valid() : iiter->Valid();
+ through_db ? iter->Next() : iiter->Next()) {
+ if (if_query_empty_keys) {
+ break;
+ }
+ // verify key;
+ total_time += Now(clock, measured_by_nanosecond) - start_time;
+ assert(Slice(MakeKey(r1, r2 + count, through_db)) ==
+ (through_db ? iter->key() : iiter->key()));
+ start_time = Now(clock, measured_by_nanosecond);
+ if (++count >= r2_len) {
+ break;
+ }
+ }
+ if (count != r2_len) {
+ fprintf(stderr,
+ "Iterator cannot iterate expected number of entries. "
+ "Expected %d but got %d\n",
+ r2_len, count);
+ assert(false);
+ }
+ delete iter;
+ total_time += Now(clock, measured_by_nanosecond) - start_time;
+ hist.Add(total_time);
+ }
+ }
+ }
+ }
+
+ fprintf(
+ stderr,
+ "==================================================="
+ "====================================================\n"
+ "InMemoryTableSimpleBenchmark: %20s num_key1: %5d "
+ "num_key2: %5d %10s\n"
+ "==================================================="
+ "===================================================="
+ "\nHistogram (unit: %s): \n%s",
+ opts.table_factory->Name(), num_keys1, num_keys2,
+ for_iterator ? "iterator" : (if_query_empty_keys ? "empty" : "non_empty"),
+ measured_by_nanosecond ? "nanosecond" : "microsecond",
+ hist.ToString().c_str());
+ if (!through_db) {
+ env->DeleteFile(file_name);
+ } else {
+ delete db;
+ db = nullptr;
+ DestroyDB(dbname, opts);
+ }
+}
+} // namespace
+} // namespace ROCKSDB_NAMESPACE
+
+DEFINE_bool(query_empty, false,
+ "query non-existing keys instead of existing ones.");
+DEFINE_int32(num_keys1, 4096, "number of distinguish prefix of keys");
+DEFINE_int32(num_keys2, 512, "number of distinguish keys for each prefix");
+DEFINE_int32(iter, 3, "query non-existing keys instead of existing ones");
+DEFINE_int32(prefix_len, 16, "Prefix length used for iterators and indexes");
+DEFINE_bool(iterator, false, "For test iterator");
+DEFINE_bool(through_db, false,
+ "If enable, a DB instance will be created and the query will be "
+ "against DB. Otherwise, will be directly against a table reader.");
+DEFINE_bool(mmap_read, true, "Whether use mmap read");
+DEFINE_string(table_factory, "block_based",
+ "Table factory to use: `block_based` (default), `plain_table` or "
+ "`cuckoo_hash`.");
+DEFINE_string(time_unit, "microsecond",
+ "The time unit used for measuring performance. User can specify "
+ "`microsecond` (default) or `nanosecond`");
+
+int main(int argc, char** argv) {
+ SetUsageMessage(std::string("\nUSAGE:\n") + std::string(argv[0]) +
+ " [OPTIONS]...");
+ ParseCommandLineFlags(&argc, &argv, true);
+
+ std::shared_ptr<ROCKSDB_NAMESPACE::TableFactory> tf;
+ ROCKSDB_NAMESPACE::Options options;
+ if (FLAGS_prefix_len < 16) {
+ options.prefix_extractor.reset(
+ ROCKSDB_NAMESPACE::NewFixedPrefixTransform(FLAGS_prefix_len));
+ }
+ ROCKSDB_NAMESPACE::ReadOptions ro;
+ ROCKSDB_NAMESPACE::EnvOptions env_options;
+ options.create_if_missing = true;
+ options.compression = ROCKSDB_NAMESPACE::CompressionType::kNoCompression;
+
+ if (FLAGS_table_factory == "cuckoo_hash") {
+#ifndef ROCKSDB_LITE
+ options.allow_mmap_reads = FLAGS_mmap_read;
+ env_options.use_mmap_reads = FLAGS_mmap_read;
+ ROCKSDB_NAMESPACE::CuckooTableOptions table_options;
+ table_options.hash_table_ratio = 0.75;
+ tf.reset(ROCKSDB_NAMESPACE::NewCuckooTableFactory(table_options));
+#else
+ fprintf(stderr, "Plain table is not supported in lite mode\n");
+ exit(1);
+#endif // ROCKSDB_LITE
+ } else if (FLAGS_table_factory == "plain_table") {
+#ifndef ROCKSDB_LITE
+ options.allow_mmap_reads = FLAGS_mmap_read;
+ env_options.use_mmap_reads = FLAGS_mmap_read;
+
+ ROCKSDB_NAMESPACE::PlainTableOptions plain_table_options;
+ plain_table_options.user_key_len = 16;
+ plain_table_options.bloom_bits_per_key = (FLAGS_prefix_len == 16) ? 0 : 8;
+ plain_table_options.hash_table_ratio = 0.75;
+
+ tf.reset(new ROCKSDB_NAMESPACE::PlainTableFactory(plain_table_options));
+ options.prefix_extractor.reset(
+ ROCKSDB_NAMESPACE::NewFixedPrefixTransform(FLAGS_prefix_len));
+#else
+ fprintf(stderr, "Cuckoo table is not supported in lite mode\n");
+ exit(1);
+#endif // ROCKSDB_LITE
+ } else if (FLAGS_table_factory == "block_based") {
+ tf.reset(new ROCKSDB_NAMESPACE::BlockBasedTableFactory());
+ } else {
+ fprintf(stderr, "Invalid table type %s\n", FLAGS_table_factory.c_str());
+ }
+
+ if (tf) {
+ // if user provides invalid options, just fall back to microsecond.
+ bool measured_by_nanosecond = FLAGS_time_unit == "nanosecond";
+
+ options.table_factory = tf;
+ ROCKSDB_NAMESPACE::TableReaderBenchmark(
+ options, env_options, ro, FLAGS_num_keys1, FLAGS_num_keys2, FLAGS_iter,
+ FLAGS_prefix_len, FLAGS_query_empty, FLAGS_iterator, FLAGS_through_db,
+ measured_by_nanosecond);
+ } else {
+ return 1;
+ }
+
+ return 0;
+}
+
+#endif // GFLAGS
diff --git a/src/rocksdb/table/table_test.cc b/src/rocksdb/table/table_test.cc
new file mode 100644
index 000000000..af9c177e8
--- /dev/null
+++ b/src/rocksdb/table/table_test.cc
@@ -0,0 +1,5596 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#include "rocksdb/table.h"
+
+#include <gtest/gtest.h>
+#include <stddef.h>
+#include <stdio.h>
+
+#include <algorithm>
+#include <iostream>
+#include <map>
+#include <memory>
+#include <string>
+#include <unordered_set>
+#include <vector>
+
+#include "cache/lru_cache.h"
+#include "db/db_test_util.h"
+#include "db/dbformat.h"
+#include "db/memtable.h"
+#include "db/write_batch_internal.h"
+#include "memtable/stl_wrappers.h"
+#include "monitoring/statistics.h"
+#include "options/options_helper.h"
+#include "port/port.h"
+#include "port/stack_trace.h"
+#include "rocksdb/cache.h"
+#include "rocksdb/compression_type.h"
+#include "rocksdb/convenience.h"
+#include "rocksdb/db.h"
+#include "rocksdb/env.h"
+#include "rocksdb/file_checksum.h"
+#include "rocksdb/file_system.h"
+#include "rocksdb/filter_policy.h"
+#include "rocksdb/iterator.h"
+#include "rocksdb/memtablerep.h"
+#include "rocksdb/options.h"
+#include "rocksdb/perf_context.h"
+#include "rocksdb/slice_transform.h"
+#include "rocksdb/statistics.h"
+#include "rocksdb/table_properties.h"
+#include "rocksdb/trace_record.h"
+#include "rocksdb/unique_id.h"
+#include "rocksdb/write_buffer_manager.h"
+#include "table/block_based/block.h"
+#include "table/block_based/block_based_table_builder.h"
+#include "table/block_based/block_based_table_factory.h"
+#include "table/block_based/block_based_table_reader.h"
+#include "table/block_based/block_builder.h"
+#include "table/block_based/filter_policy_internal.h"
+#include "table/block_based/flush_block_policy.h"
+#include "table/block_fetcher.h"
+#include "table/format.h"
+#include "table/get_context.h"
+#include "table/internal_iterator.h"
+#include "table/meta_blocks.h"
+#include "table/plain/plain_table_factory.h"
+#include "table/scoped_arena_iterator.h"
+#include "table/sst_file_writer_collectors.h"
+#include "table/unique_id_impl.h"
+#include "test_util/sync_point.h"
+#include "test_util/testharness.h"
+#include "test_util/testutil.h"
+#include "util/coding_lean.h"
+#include "util/compression.h"
+#include "util/file_checksum_helper.h"
+#include "util/random.h"
+#include "util/string_util.h"
+#include "utilities/memory_allocators.h"
+#include "utilities/merge_operators.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+extern const uint64_t kLegacyBlockBasedTableMagicNumber;
+extern const uint64_t kLegacyPlainTableMagicNumber;
+extern const uint64_t kBlockBasedTableMagicNumber;
+extern const uint64_t kPlainTableMagicNumber;
+
+namespace {
+
+const std::string kDummyValue(10000, 'o');
+
+// DummyPropertiesCollector used to test BlockBasedTableProperties
+class DummyPropertiesCollector : public TablePropertiesCollector {
+ public:
+ const char* Name() const override { return "DummyPropertiesCollector"; }
+
+ Status Finish(UserCollectedProperties* /*properties*/) override {
+ return Status::OK();
+ }
+
+ Status Add(const Slice& /*user_key*/, const Slice& /*value*/) override {
+ return Status::OK();
+ }
+
+ UserCollectedProperties GetReadableProperties() const override {
+ return UserCollectedProperties{};
+ }
+};
+
+class DummyPropertiesCollectorFactory1
+ : public TablePropertiesCollectorFactory {
+ public:
+ TablePropertiesCollector* CreateTablePropertiesCollector(
+ TablePropertiesCollectorFactory::Context /*context*/) override {
+ return new DummyPropertiesCollector();
+ }
+ const char* Name() const override {
+ return "DummyPropertiesCollectorFactory1";
+ }
+};
+
+class DummyPropertiesCollectorFactory2
+ : public TablePropertiesCollectorFactory {
+ public:
+ TablePropertiesCollector* CreateTablePropertiesCollector(
+ TablePropertiesCollectorFactory::Context /*context*/) override {
+ return new DummyPropertiesCollector();
+ }
+ const char* Name() const override {
+ return "DummyPropertiesCollectorFactory2";
+ }
+};
+
+// Return reverse of "key".
+// Used to test non-lexicographic comparators.
+std::string Reverse(const Slice& key) {
+ auto rev = key.ToString();
+ std::reverse(rev.begin(), rev.end());
+ return rev;
+}
+
+class ReverseKeyComparator : public Comparator {
+ public:
+ const char* Name() const override {
+ return "rocksdb.ReverseBytewiseComparator";
+ }
+
+ int Compare(const Slice& a, const Slice& b) const override {
+ return BytewiseComparator()->Compare(Reverse(a), Reverse(b));
+ }
+
+ void FindShortestSeparator(std::string* start,
+ const Slice& limit) const override {
+ std::string s = Reverse(*start);
+ std::string l = Reverse(limit);
+ BytewiseComparator()->FindShortestSeparator(&s, l);
+ *start = Reverse(s);
+ }
+
+ void FindShortSuccessor(std::string* key) const override {
+ std::string s = Reverse(*key);
+ BytewiseComparator()->FindShortSuccessor(&s);
+ *key = Reverse(s);
+ }
+};
+
+ReverseKeyComparator reverse_key_comparator;
+
+void Increment(const Comparator* cmp, std::string* key) {
+ if (cmp == BytewiseComparator()) {
+ key->push_back('\0');
+ } else {
+ assert(cmp == &reverse_key_comparator);
+ std::string rev = Reverse(*key);
+ rev.push_back('\0');
+ *key = Reverse(rev);
+ }
+}
+
+const auto kUnknownColumnFamily =
+ TablePropertiesCollectorFactory::Context::kUnknownColumnFamily;
+
+} // namespace
+
+// Helper class for tests to unify the interface between
+// BlockBuilder/TableBuilder and Block/Table.
+class Constructor {
+ public:
+ explicit Constructor(const Comparator* cmp)
+ : data_(stl_wrappers::LessOfComparator(cmp)) {}
+ virtual ~Constructor() {}
+
+ void Add(const std::string& key, const Slice& value) {
+ data_[key] = value.ToString();
+ }
+
+ // Finish constructing the data structure with all the keys that have
+ // been added so far. Returns the keys in sorted order in "*keys"
+ // and stores the key/value pairs in "*kvmap"
+ void Finish(const Options& options, const ImmutableOptions& ioptions,
+ const MutableCFOptions& moptions,
+ const BlockBasedTableOptions& table_options,
+ const InternalKeyComparator& internal_comparator,
+ std::vector<std::string>* keys, stl_wrappers::KVMap* kvmap) {
+ last_internal_comparator_ = &internal_comparator;
+ *kvmap = data_;
+ keys->clear();
+ for (const auto& kv : data_) {
+ keys->push_back(kv.first);
+ }
+ data_.clear();
+ Status s = FinishImpl(options, ioptions, moptions, table_options,
+ internal_comparator, *kvmap);
+ ASSERT_TRUE(s.ok()) << s.ToString();
+ }
+
+ // Construct the data structure from the data in "data"
+ virtual Status FinishImpl(const Options& options,
+ const ImmutableOptions& ioptions,
+ const MutableCFOptions& moptions,
+ const BlockBasedTableOptions& table_options,
+ const InternalKeyComparator& internal_comparator,
+ const stl_wrappers::KVMap& data) = 0;
+
+ virtual InternalIterator* NewIterator(
+ const SliceTransform* prefix_extractor = nullptr) const = 0;
+
+ virtual const stl_wrappers::KVMap& data() { return data_; }
+
+ virtual bool IsArenaMode() const { return false; }
+
+ virtual DB* db() const { return nullptr; } // Overridden in DBConstructor
+
+ virtual bool AnywayDeleteIterator() const { return false; }
+
+ protected:
+ const InternalKeyComparator* last_internal_comparator_;
+
+ private:
+ stl_wrappers::KVMap data_;
+};
+
+// A helper class that converts internal format keys into user keys
+class KeyConvertingIterator : public InternalIterator {
+ public:
+ explicit KeyConvertingIterator(InternalIterator* iter,
+ bool arena_mode = false)
+ : iter_(iter), arena_mode_(arena_mode) {}
+ ~KeyConvertingIterator() override {
+ if (arena_mode_) {
+ iter_->~InternalIterator();
+ } else {
+ delete iter_;
+ }
+ }
+ bool Valid() const override { return iter_->Valid() && status_.ok(); }
+ void Seek(const Slice& target) override {
+ ParsedInternalKey ikey(target, kMaxSequenceNumber, kTypeValue);
+ std::string encoded;
+ AppendInternalKey(&encoded, ikey);
+ iter_->Seek(encoded);
+ }
+ void SeekForPrev(const Slice& target) override {
+ ParsedInternalKey ikey(target, kMaxSequenceNumber, kTypeValue);
+ std::string encoded;
+ AppendInternalKey(&encoded, ikey);
+ iter_->SeekForPrev(encoded);
+ }
+ void SeekToFirst() override { iter_->SeekToFirst(); }
+ void SeekToLast() override { iter_->SeekToLast(); }
+ void Next() override { iter_->Next(); }
+ void Prev() override { iter_->Prev(); }
+ IterBoundCheck UpperBoundCheckResult() override {
+ return iter_->UpperBoundCheckResult();
+ }
+
+ Slice key() const override {
+ assert(Valid());
+ ParsedInternalKey parsed_key;
+ Status pik_status =
+ ParseInternalKey(iter_->key(), &parsed_key, true /* log_err_key */);
+ if (!pik_status.ok()) {
+ status_ = pik_status;
+ return Slice(status_.getState());
+ }
+ return parsed_key.user_key;
+ }
+
+ Slice value() const override { return iter_->value(); }
+ Status status() const override {
+ return status_.ok() ? iter_->status() : status_;
+ }
+
+ private:
+ mutable Status status_;
+ InternalIterator* iter_;
+ bool arena_mode_;
+
+ // No copying allowed
+ KeyConvertingIterator(const KeyConvertingIterator&);
+ void operator=(const KeyConvertingIterator&);
+};
+
+// `BlockConstructor` APIs always accept/return user keys.
+class BlockConstructor : public Constructor {
+ public:
+ explicit BlockConstructor(const Comparator* cmp)
+ : Constructor(cmp), comparator_(cmp), block_(nullptr) {}
+ ~BlockConstructor() override { delete block_; }
+ Status FinishImpl(const Options& /*options*/,
+ const ImmutableOptions& /*ioptions*/,
+ const MutableCFOptions& /*moptions*/,
+ const BlockBasedTableOptions& table_options,
+ const InternalKeyComparator& /*internal_comparator*/,
+ const stl_wrappers::KVMap& kv_map) override {
+ delete block_;
+ block_ = nullptr;
+ BlockBuilder builder(table_options.block_restart_interval);
+
+ for (const auto& kv : kv_map) {
+ // `DataBlockIter` assumes it reads only internal keys. `BlockConstructor`
+ // clients provide user keys, so we need to convert to internal key format
+ // before writing the data block.
+ ParsedInternalKey ikey(kv.first, kMaxSequenceNumber, kTypeValue);
+ std::string encoded;
+ AppendInternalKey(&encoded, ikey);
+ builder.Add(encoded, kv.second);
+ }
+ // Open the block
+ data_ = builder.Finish().ToString();
+ BlockContents contents;
+ contents.data = data_;
+ block_ = new Block(std::move(contents));
+ return Status::OK();
+ }
+ InternalIterator* NewIterator(
+ const SliceTransform* /*prefix_extractor*/) const override {
+ // `DataBlockIter` returns the internal keys it reads.
+ // `KeyConvertingIterator` converts them to user keys before they are
+ // exposed to the `BlockConstructor` clients.
+ return new KeyConvertingIterator(
+ block_->NewDataIterator(comparator_, kDisableGlobalSequenceNumber));
+ }
+
+ private:
+ const Comparator* comparator_;
+ std::string data_;
+ Block* block_;
+
+ BlockConstructor();
+};
+
+class TableConstructor : public Constructor {
+ public:
+ explicit TableConstructor(const Comparator* cmp,
+ bool convert_to_internal_key = false,
+ int level = -1, SequenceNumber largest_seqno = 0)
+ : Constructor(cmp),
+ largest_seqno_(largest_seqno),
+ convert_to_internal_key_(convert_to_internal_key),
+ level_(level) {
+ env_ = ROCKSDB_NAMESPACE::Env::Default();
+ }
+ ~TableConstructor() override { Reset(); }
+
+ Status FinishImpl(const Options& options, const ImmutableOptions& ioptions,
+ const MutableCFOptions& moptions,
+ const BlockBasedTableOptions& /*table_options*/,
+ const InternalKeyComparator& internal_comparator,
+ const stl_wrappers::KVMap& kv_map) override {
+ Reset();
+ soptions.use_mmap_reads = ioptions.allow_mmap_reads;
+ std::unique_ptr<FSWritableFile> sink(new test::StringSink());
+ file_writer_.reset(new WritableFileWriter(
+ std::move(sink), "" /* don't care */, FileOptions()));
+ std::unique_ptr<TableBuilder> builder;
+ IntTblPropCollectorFactories int_tbl_prop_collector_factories;
+
+ if (largest_seqno_ != 0) {
+ // Pretend that it's an external file written by SstFileWriter.
+ int_tbl_prop_collector_factories.emplace_back(
+ new SstFileWriterPropertiesCollectorFactory(2 /* version */,
+ 0 /* global_seqno*/));
+ }
+
+ std::string column_family_name;
+ builder.reset(ioptions.table_factory->NewTableBuilder(
+ TableBuilderOptions(ioptions, moptions, internal_comparator,
+ &int_tbl_prop_collector_factories,
+ options.compression, options.compression_opts,
+ kUnknownColumnFamily, column_family_name, level_),
+ file_writer_.get()));
+
+ for (const auto& kv : kv_map) {
+ if (convert_to_internal_key_) {
+ ParsedInternalKey ikey(kv.first, kMaxSequenceNumber, kTypeValue);
+ std::string encoded;
+ AppendInternalKey(&encoded, ikey);
+ builder->Add(encoded, kv.second);
+ } else {
+ builder->Add(kv.first, kv.second);
+ }
+ EXPECT_OK(builder->status());
+ }
+ Status s = builder->Finish();
+ EXPECT_OK(file_writer_->Flush());
+ EXPECT_TRUE(s.ok()) << s.ToString();
+
+ EXPECT_EQ(TEST_GetSink()->contents().size(), builder->FileSize());
+
+ // Open the table
+ file_num_ = cur_file_num_++;
+
+ return Reopen(ioptions, moptions);
+ }
+
+ InternalIterator* NewIterator(
+ const SliceTransform* prefix_extractor) const override {
+ InternalIterator* iter = table_reader_->NewIterator(
+ read_options_, prefix_extractor, /*arena=*/nullptr,
+ /*skip_filters=*/false, TableReaderCaller::kUncategorized);
+ if (convert_to_internal_key_) {
+ return new KeyConvertingIterator(iter);
+ } else {
+ return iter;
+ }
+ }
+
+ 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, TableReaderCaller::kUncategorized);
+ }
+ return table_reader_->ApproximateOffsetOf(
+ key, TableReaderCaller::kUncategorized);
+ }
+
+ virtual Status Reopen(const ImmutableOptions& ioptions,
+ const MutableCFOptions& moptions) {
+ std::unique_ptr<FSRandomAccessFile> source(new test::StringSource(
+ TEST_GetSink()->contents(), file_num_, ioptions.allow_mmap_reads));
+
+ file_reader_.reset(new RandomAccessFileReader(std::move(source), "test"));
+ return ioptions.table_factory->NewTableReader(
+ TableReaderOptions(ioptions, moptions.prefix_extractor, soptions,
+ *last_internal_comparator_, /*skip_filters*/ false,
+ /*immortal*/ false, false, level_,
+ &block_cache_tracer_, moptions.write_buffer_size, "",
+ file_num_, kNullUniqueId64x2, largest_seqno_),
+ std::move(file_reader_), TEST_GetSink()->contents().size(),
+ &table_reader_);
+ }
+
+ virtual TableReader* GetTableReader() { return table_reader_.get(); }
+
+ bool AnywayDeleteIterator() const override {
+ return convert_to_internal_key_;
+ }
+
+ void ResetTableReader() { table_reader_.reset(); }
+
+ bool ConvertToInternalKey() { return convert_to_internal_key_; }
+
+ test::StringSink* TEST_GetSink() {
+ return static_cast<test::StringSink*>(file_writer_->writable_file());
+ }
+
+ BlockCacheTracer block_cache_tracer_;
+
+ private:
+ void Reset() {
+ file_num_ = 0;
+ table_reader_.reset();
+ file_writer_.reset();
+ file_reader_.reset();
+ }
+
+ const ReadOptions read_options_;
+ uint64_t file_num_;
+ std::unique_ptr<WritableFileWriter> file_writer_;
+ std::unique_ptr<RandomAccessFileReader> file_reader_;
+ std::unique_ptr<TableReader> table_reader_;
+ SequenceNumber largest_seqno_;
+ bool convert_to_internal_key_;
+ int level_;
+
+ TableConstructor();
+
+ static uint64_t cur_file_num_;
+ EnvOptions soptions;
+ Env* env_;
+};
+uint64_t TableConstructor::cur_file_num_ = 1;
+
+class MemTableConstructor : public Constructor {
+ public:
+ explicit MemTableConstructor(const Comparator* cmp, WriteBufferManager* wb)
+ : Constructor(cmp),
+ internal_comparator_(cmp),
+ write_buffer_manager_(wb),
+ table_factory_(new SkipListFactory) {
+ options_.memtable_factory = table_factory_;
+ ImmutableOptions ioptions(options_);
+ memtable_ =
+ new MemTable(internal_comparator_, ioptions, MutableCFOptions(options_),
+ wb, kMaxSequenceNumber, 0 /* column_family_id */);
+ memtable_->Ref();
+ }
+ ~MemTableConstructor() override { delete memtable_->Unref(); }
+ Status FinishImpl(const Options&, const ImmutableOptions& ioptions,
+ const MutableCFOptions& /*moptions*/,
+ const BlockBasedTableOptions& /*table_options*/,
+ const InternalKeyComparator& /*internal_comparator*/,
+ const stl_wrappers::KVMap& kv_map) override {
+ delete memtable_->Unref();
+ ImmutableOptions mem_ioptions(ioptions);
+ memtable_ = new MemTable(internal_comparator_, mem_ioptions,
+ MutableCFOptions(options_), write_buffer_manager_,
+ kMaxSequenceNumber, 0 /* column_family_id */);
+ memtable_->Ref();
+ int seq = 1;
+ for (const auto& kv : kv_map) {
+ Status s = memtable_->Add(seq, kTypeValue, kv.first, kv.second,
+ nullptr /* kv_prot_info */);
+ if (!s.ok()) {
+ return s;
+ }
+ seq++;
+ }
+ return Status::OK();
+ }
+ InternalIterator* NewIterator(
+ const SliceTransform* /*prefix_extractor*/) const override {
+ return new KeyConvertingIterator(
+ memtable_->NewIterator(ReadOptions(), &arena_), true);
+ }
+
+ bool AnywayDeleteIterator() const override { return true; }
+
+ bool IsArenaMode() const override { return true; }
+
+ private:
+ mutable Arena arena_;
+ InternalKeyComparator internal_comparator_;
+ Options options_;
+ WriteBufferManager* write_buffer_manager_;
+ MemTable* memtable_;
+ std::shared_ptr<SkipListFactory> table_factory_;
+};
+
+class InternalIteratorFromIterator : public InternalIterator {
+ public:
+ explicit InternalIteratorFromIterator(Iterator* it) : it_(it) {}
+ bool Valid() const override { return it_->Valid(); }
+ void Seek(const Slice& target) override { it_->Seek(target); }
+ void SeekForPrev(const Slice& target) override { it_->SeekForPrev(target); }
+ void SeekToFirst() override { it_->SeekToFirst(); }
+ void SeekToLast() override { it_->SeekToLast(); }
+ void Next() override { it_->Next(); }
+ void Prev() override { it_->Prev(); }
+ Slice key() const override { return it_->key(); }
+ Slice value() const override { return it_->value(); }
+ Status status() const override { return it_->status(); }
+
+ private:
+ std::unique_ptr<Iterator> it_;
+};
+
+class DBConstructor : public Constructor {
+ public:
+ explicit DBConstructor(const Comparator* cmp)
+ : Constructor(cmp), comparator_(cmp) {
+ db_ = nullptr;
+ NewDB();
+ }
+ ~DBConstructor() override { delete db_; }
+ Status FinishImpl(const Options& /*options*/,
+ const ImmutableOptions& /*ioptions*/,
+ const MutableCFOptions& /*moptions*/,
+ const BlockBasedTableOptions& /*table_options*/,
+ const InternalKeyComparator& /*internal_comparator*/,
+ const stl_wrappers::KVMap& kv_map) override {
+ delete db_;
+ db_ = nullptr;
+ NewDB();
+ for (const auto& kv : kv_map) {
+ WriteBatch batch;
+ EXPECT_OK(batch.Put(kv.first, kv.second));
+ EXPECT_TRUE(db_->Write(WriteOptions(), &batch).ok());
+ }
+ return Status::OK();
+ }
+
+ InternalIterator* NewIterator(
+ const SliceTransform* /*prefix_extractor*/) const override {
+ return new InternalIteratorFromIterator(db_->NewIterator(ReadOptions()));
+ }
+
+ DB* db() const override { return db_; }
+
+ private:
+ void NewDB() {
+ std::string name = test::PerThreadDBPath("table_testdb");
+
+ Options options;
+ options.comparator = comparator_;
+ Status status = DestroyDB(name, options);
+ ASSERT_TRUE(status.ok()) << status.ToString();
+
+ options.create_if_missing = true;
+ options.error_if_exists = true;
+ options.write_buffer_size = 10000; // Something small to force merging
+ status = DB::Open(options, name, &db_);
+ ASSERT_TRUE(status.ok()) << status.ToString();
+ }
+
+ const Comparator* comparator_;
+ DB* db_;
+};
+
+enum TestType {
+ BLOCK_BASED_TABLE_TEST,
+#ifndef ROCKSDB_LITE
+ PLAIN_TABLE_SEMI_FIXED_PREFIX,
+ PLAIN_TABLE_FULL_STR_PREFIX,
+ PLAIN_TABLE_TOTAL_ORDER,
+#endif // !ROCKSDB_LITE
+ BLOCK_TEST,
+ MEMTABLE_TEST,
+ DB_TEST
+};
+
+struct TestArgs {
+ TestType type;
+ bool reverse_compare;
+ int restart_interval;
+ CompressionType compression;
+ uint32_t compression_parallel_threads;
+ uint32_t format_version;
+ bool use_mmap;
+};
+
+std::ostream& operator<<(std::ostream& os, const TestArgs& args) {
+ os << "type: " << args.type << " reverse_compare: " << args.reverse_compare
+ << " restart_interval: " << args.restart_interval
+ << " compression: " << args.compression
+ << " compression_parallel_threads: " << args.compression_parallel_threads
+ << " format_version: " << args.format_version
+ << " use_mmap: " << args.use_mmap;
+
+ return os;
+}
+
+static std::vector<TestArgs> GenerateArgList() {
+ std::vector<TestArgs> test_args;
+ std::vector<TestType> test_types = {BLOCK_BASED_TABLE_TEST,
+#ifndef ROCKSDB_LITE
+ PLAIN_TABLE_SEMI_FIXED_PREFIX,
+ PLAIN_TABLE_FULL_STR_PREFIX,
+ PLAIN_TABLE_TOTAL_ORDER,
+#endif // !ROCKSDB_LITE
+ BLOCK_TEST,
+ MEMTABLE_TEST,
+ DB_TEST};
+ std::vector<bool> reverse_compare_types = {false, true};
+ std::vector<int> restart_intervals = {16, 1, 1024};
+ std::vector<uint32_t> compression_parallel_threads = {1, 4};
+
+ // Only add compression if it is supported
+ std::vector<std::pair<CompressionType, bool>> compression_types;
+ compression_types.emplace_back(kNoCompression, false);
+ if (Snappy_Supported()) {
+ compression_types.emplace_back(kSnappyCompression, false);
+ }
+ if (Zlib_Supported()) {
+ compression_types.emplace_back(kZlibCompression, false);
+ compression_types.emplace_back(kZlibCompression, true);
+ }
+ if (BZip2_Supported()) {
+ compression_types.emplace_back(kBZip2Compression, false);
+ compression_types.emplace_back(kBZip2Compression, true);
+ }
+ if (LZ4_Supported()) {
+ compression_types.emplace_back(kLZ4Compression, false);
+ compression_types.emplace_back(kLZ4Compression, true);
+ compression_types.emplace_back(kLZ4HCCompression, false);
+ compression_types.emplace_back(kLZ4HCCompression, true);
+ }
+ if (XPRESS_Supported()) {
+ compression_types.emplace_back(kXpressCompression, false);
+ compression_types.emplace_back(kXpressCompression, true);
+ }
+ if (ZSTD_Supported()) {
+ compression_types.emplace_back(kZSTD, false);
+ compression_types.emplace_back(kZSTD, true);
+ }
+
+ for (auto test_type : test_types) {
+ for (auto reverse_compare : reverse_compare_types) {
+#ifndef ROCKSDB_LITE
+ if (test_type == PLAIN_TABLE_SEMI_FIXED_PREFIX ||
+ test_type == PLAIN_TABLE_FULL_STR_PREFIX ||
+ test_type == PLAIN_TABLE_TOTAL_ORDER) {
+ // Plain table doesn't use restart index or compression.
+ TestArgs one_arg;
+ one_arg.type = test_type;
+ one_arg.reverse_compare = reverse_compare;
+ one_arg.restart_interval = restart_intervals[0];
+ one_arg.compression = compression_types[0].first;
+ one_arg.compression_parallel_threads = 1;
+ one_arg.format_version = 0;
+ one_arg.use_mmap = true;
+ test_args.push_back(one_arg);
+ one_arg.use_mmap = false;
+ test_args.push_back(one_arg);
+ continue;
+ }
+#endif // !ROCKSDB_LITE
+
+ for (auto restart_interval : restart_intervals) {
+ for (auto compression_type : compression_types) {
+ for (auto num_threads : compression_parallel_threads) {
+ TestArgs one_arg;
+ one_arg.type = test_type;
+ one_arg.reverse_compare = reverse_compare;
+ one_arg.restart_interval = restart_interval;
+ one_arg.compression = compression_type.first;
+ one_arg.compression_parallel_threads = num_threads;
+ one_arg.format_version = compression_type.second ? 2 : 1;
+ one_arg.use_mmap = false;
+ test_args.push_back(one_arg);
+ }
+ }
+ }
+ }
+ }
+ return test_args;
+}
+
+// In order to make all tests run for plain table format, including
+// those operating on empty keys, create a new prefix transformer which
+// return fixed prefix if the slice is not shorter than the prefix length,
+// and the full slice if it is shorter.
+class FixedOrLessPrefixTransform : public SliceTransform {
+ private:
+ const size_t prefix_len_;
+
+ public:
+ explicit FixedOrLessPrefixTransform(size_t prefix_len)
+ : prefix_len_(prefix_len) {}
+
+ const char* Name() const override { return "rocksdb.FixedPrefix"; }
+
+ Slice Transform(const Slice& src) const override {
+ assert(InDomain(src));
+ if (src.size() < prefix_len_) {
+ return src;
+ }
+ return Slice(src.data(), prefix_len_);
+ }
+
+ bool InDomain(const Slice& /*src*/) const override { return true; }
+
+ bool InRange(const Slice& dst) const override {
+ return (dst.size() <= prefix_len_);
+ }
+ bool FullLengthEnabled(size_t* /*len*/) const override { return false; }
+};
+
+class HarnessTest : public testing::Test {
+ public:
+ explicit HarnessTest(const TestArgs& args)
+ : args_(args),
+ ioptions_(options_),
+ moptions_(options_),
+ write_buffer_(options_.db_write_buffer_size),
+ support_prev_(true),
+ only_support_prefix_seek_(false) {
+ options_.compression = args_.compression;
+ options_.compression_opts.parallel_threads =
+ args_.compression_parallel_threads;
+ // Use shorter block size for tests to exercise block boundary
+ // conditions more.
+ if (args_.reverse_compare) {
+ options_.comparator = &reverse_key_comparator;
+ }
+
+ internal_comparator_.reset(
+ new test::PlainInternalKeyComparator(options_.comparator));
+
+ options_.allow_mmap_reads = args_.use_mmap;
+ switch (args_.type) {
+ case BLOCK_BASED_TABLE_TEST:
+ table_options_.flush_block_policy_factory.reset(
+ new FlushBlockBySizePolicyFactory());
+ table_options_.block_size = 256;
+ table_options_.block_restart_interval = args_.restart_interval;
+ table_options_.index_block_restart_interval = args_.restart_interval;
+ table_options_.format_version = args_.format_version;
+ options_.table_factory.reset(
+ new BlockBasedTableFactory(table_options_));
+ constructor_.reset(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
+ case PLAIN_TABLE_SEMI_FIXED_PREFIX:
+ support_prev_ = false;
+ only_support_prefix_seek_ = true;
+ options_.prefix_extractor.reset(new FixedOrLessPrefixTransform(2));
+ options_.table_factory.reset(NewPlainTableFactory());
+ constructor_.reset(new TableConstructor(
+ options_.comparator, true /* convert_to_internal_key_ */));
+ internal_comparator_.reset(
+ new InternalKeyComparator(options_.comparator));
+ break;
+ case PLAIN_TABLE_FULL_STR_PREFIX:
+ support_prev_ = false;
+ only_support_prefix_seek_ = true;
+ options_.prefix_extractor.reset(NewNoopTransform());
+ options_.table_factory.reset(NewPlainTableFactory());
+ constructor_.reset(new TableConstructor(
+ options_.comparator, true /* convert_to_internal_key_ */));
+ internal_comparator_.reset(
+ new InternalKeyComparator(options_.comparator));
+ break;
+ case PLAIN_TABLE_TOTAL_ORDER:
+ support_prev_ = false;
+ only_support_prefix_seek_ = false;
+ options_.prefix_extractor = nullptr;
+
+ {
+ PlainTableOptions plain_table_options;
+ plain_table_options.user_key_len = kPlainTableVariableLength;
+ plain_table_options.bloom_bits_per_key = 0;
+ plain_table_options.hash_table_ratio = 0;
+
+ options_.table_factory.reset(
+ NewPlainTableFactory(plain_table_options));
+ }
+ constructor_.reset(new TableConstructor(
+ options_.comparator, true /* convert_to_internal_key_ */));
+ internal_comparator_.reset(
+ new InternalKeyComparator(options_.comparator));
+ break;
+#endif // !ROCKSDB_LITE
+ case BLOCK_TEST:
+ table_options_.block_size = 256;
+ options_.table_factory.reset(
+ new BlockBasedTableFactory(table_options_));
+ constructor_.reset(new BlockConstructor(options_.comparator));
+ break;
+ case MEMTABLE_TEST:
+ table_options_.block_size = 256;
+ options_.table_factory.reset(
+ new BlockBasedTableFactory(table_options_));
+ constructor_.reset(
+ new MemTableConstructor(options_.comparator, &write_buffer_));
+ break;
+ case DB_TEST:
+ table_options_.block_size = 256;
+ options_.table_factory.reset(
+ new BlockBasedTableFactory(table_options_));
+ constructor_.reset(new DBConstructor(options_.comparator));
+ break;
+ }
+ ioptions_ = ImmutableOptions(options_);
+ moptions_ = MutableCFOptions(options_);
+ }
+
+ void Add(const std::string& key, const std::string& value) {
+ constructor_->Add(key, value);
+ }
+
+ void Test(Random* rnd) {
+ std::vector<std::string> keys;
+ stl_wrappers::KVMap data;
+ constructor_->Finish(options_, ioptions_, moptions_, table_options_,
+ *internal_comparator_, &keys, &data);
+
+ TestForwardScan(keys, data);
+ if (support_prev_) {
+ TestBackwardScan(keys, data);
+ }
+ TestRandomAccess(rnd, keys, data);
+ }
+
+ void TestForwardScan(const std::vector<std::string>& /*keys*/,
+ const stl_wrappers::KVMap& data) {
+ InternalIterator* iter = constructor_->NewIterator();
+ ASSERT_TRUE(!iter->Valid());
+ iter->SeekToFirst();
+ ASSERT_OK(iter->status());
+ for (stl_wrappers::KVMap::const_iterator model_iter = data.begin();
+ model_iter != data.end(); ++model_iter) {
+ ASSERT_EQ(ToString(data, model_iter), ToString(iter));
+ iter->Next();
+ ASSERT_OK(iter->status());
+ }
+ ASSERT_TRUE(!iter->Valid());
+ ASSERT_OK(iter->status());
+ if (constructor_->IsArenaMode() && !constructor_->AnywayDeleteIterator()) {
+ iter->~InternalIterator();
+ } else {
+ delete iter;
+ }
+ }
+
+ void TestBackwardScan(const std::vector<std::string>& /*keys*/,
+ const stl_wrappers::KVMap& data) {
+ InternalIterator* iter = constructor_->NewIterator();
+ ASSERT_TRUE(!iter->Valid());
+ iter->SeekToLast();
+ ASSERT_OK(iter->status());
+ for (stl_wrappers::KVMap::const_reverse_iterator model_iter = data.rbegin();
+ model_iter != data.rend(); ++model_iter) {
+ ASSERT_EQ(ToString(data, model_iter), ToString(iter));
+ iter->Prev();
+ ASSERT_OK(iter->status());
+ }
+ ASSERT_TRUE(!iter->Valid());
+ ASSERT_OK(iter->status());
+ if (constructor_->IsArenaMode() && !constructor_->AnywayDeleteIterator()) {
+ iter->~InternalIterator();
+ } else {
+ delete iter;
+ }
+ }
+
+ void TestRandomAccess(Random* rnd, const std::vector<std::string>& keys,
+ const stl_wrappers::KVMap& data) {
+ static const bool kVerbose = false;
+ InternalIterator* iter = constructor_->NewIterator();
+ ASSERT_TRUE(!iter->Valid());
+ stl_wrappers::KVMap::const_iterator model_iter = data.begin();
+ if (kVerbose) fprintf(stderr, "---\n");
+ for (int i = 0; i < 200; i++) {
+ const int toss = rnd->Uniform(support_prev_ ? 5 : 3);
+ switch (toss) {
+ case 0: {
+ if (iter->Valid()) {
+ if (kVerbose) fprintf(stderr, "Next\n");
+ iter->Next();
+ ASSERT_OK(iter->status());
+ ++model_iter;
+ ASSERT_EQ(ToString(data, model_iter), ToString(iter));
+ }
+ break;
+ }
+
+ case 1: {
+ if (kVerbose) fprintf(stderr, "SeekToFirst\n");
+ iter->SeekToFirst();
+ ASSERT_OK(iter->status());
+ model_iter = data.begin();
+ ASSERT_EQ(ToString(data, model_iter), ToString(iter));
+ break;
+ }
+
+ case 2: {
+ std::string key = PickRandomKey(rnd, keys);
+ model_iter = data.lower_bound(key);
+ if (kVerbose)
+ fprintf(stderr, "Seek '%s'\n", EscapeString(key).c_str());
+ iter->Seek(Slice(key));
+ ASSERT_OK(iter->status());
+ ASSERT_EQ(ToString(data, model_iter), ToString(iter));
+ break;
+ }
+
+ case 3: {
+ if (iter->Valid()) {
+ if (kVerbose) fprintf(stderr, "Prev\n");
+ iter->Prev();
+ ASSERT_OK(iter->status());
+ if (model_iter == data.begin()) {
+ model_iter = data.end(); // Wrap around to invalid value
+ } else {
+ --model_iter;
+ }
+ ASSERT_EQ(ToString(data, model_iter), ToString(iter));
+ }
+ break;
+ }
+
+ case 4: {
+ if (kVerbose) fprintf(stderr, "SeekToLast\n");
+ iter->SeekToLast();
+ ASSERT_OK(iter->status());
+ if (keys.empty()) {
+ model_iter = data.end();
+ } else {
+ std::string last = data.rbegin()->first;
+ model_iter = data.lower_bound(last);
+ }
+ ASSERT_EQ(ToString(data, model_iter), ToString(iter));
+ break;
+ }
+ }
+ }
+ if (constructor_->IsArenaMode() && !constructor_->AnywayDeleteIterator()) {
+ iter->~InternalIterator();
+ } else {
+ delete iter;
+ }
+ }
+
+ std::string ToString(const stl_wrappers::KVMap& data,
+ const stl_wrappers::KVMap::const_iterator& it) {
+ if (it == data.end()) {
+ return "END";
+ } else {
+ return "'" + it->first + "->" + it->second + "'";
+ }
+ }
+
+ std::string ToString(const stl_wrappers::KVMap& data,
+ const stl_wrappers::KVMap::const_reverse_iterator& it) {
+ if (it == data.rend()) {
+ return "END";
+ } else {
+ return "'" + it->first + "->" + it->second + "'";
+ }
+ }
+
+ std::string ToString(const InternalIterator* it) {
+ if (!it->Valid()) {
+ return "END";
+ } else {
+ return "'" + it->key().ToString() + "->" + it->value().ToString() + "'";
+ }
+ }
+
+ std::string PickRandomKey(Random* rnd, const std::vector<std::string>& keys) {
+ if (keys.empty()) {
+ return "foo";
+ } else {
+ const int index = rnd->Uniform(static_cast<int>(keys.size()));
+ std::string result = keys[index];
+ switch (rnd->Uniform(support_prev_ ? 3 : 1)) {
+ case 0:
+ // Return an existing key
+ break;
+ case 1: {
+ // Attempt to return something smaller than an existing key
+ if (result.size() > 0 && result[result.size() - 1] > '\0' &&
+ (!only_support_prefix_seek_ ||
+ options_.prefix_extractor->Transform(result).size() <
+ result.size())) {
+ result[result.size() - 1]--;
+ }
+ break;
+ }
+ case 2: {
+ // Return something larger than an existing key
+ Increment(options_.comparator, &result);
+ break;
+ }
+ }
+ return result;
+ }
+ }
+
+ // Returns nullptr if not running against a DB
+ DB* db() const { return constructor_->db(); }
+
+ private:
+ TestArgs args_;
+ Options options_;
+ ImmutableOptions ioptions_;
+ MutableCFOptions moptions_;
+ BlockBasedTableOptions table_options_;
+ std::unique_ptr<Constructor> constructor_;
+ WriteBufferManager write_buffer_;
+ bool support_prev_;
+ bool only_support_prefix_seek_;
+ std::shared_ptr<InternalKeyComparator> internal_comparator_;
+};
+
+class ParameterizedHarnessTest : public HarnessTest,
+ public testing::WithParamInterface<TestArgs> {
+ public:
+ ParameterizedHarnessTest() : HarnessTest(GetParam()) {}
+};
+
+INSTANTIATE_TEST_CASE_P(TableTest, ParameterizedHarnessTest,
+ ::testing::ValuesIn(GenerateArgList()));
+
+class DBHarnessTest : public HarnessTest {
+ public:
+ DBHarnessTest()
+ : HarnessTest(TestArgs{DB_TEST, /* reverse_compare */ false,
+ /* restart_interval */ 16, kNoCompression,
+ /* compression_parallel_threads */ 1,
+ /* format_version */ 0, /* use_mmap */ false}) {}
+};
+
+static bool Between(uint64_t val, uint64_t low, uint64_t high) {
+ bool result = (val >= low) && (val <= high);
+ if (!result) {
+ fprintf(stderr, "Value %llu is not in range [%llu, %llu]\n",
+ (unsigned long long)(val), (unsigned long long)(low),
+ (unsigned long long)(high));
+ }
+ return result;
+}
+
+// Tests against all kinds of tables
+class TableTest : public testing::Test {
+ public:
+ const InternalKeyComparator& GetPlainInternalComparator(
+ const Comparator* comp) {
+ if (!plain_internal_comparator) {
+ plain_internal_comparator.reset(
+ new test::PlainInternalKeyComparator(comp));
+ }
+ return *plain_internal_comparator;
+ }
+ void IndexTest(BlockBasedTableOptions table_options);
+
+ private:
+ std::unique_ptr<InternalKeyComparator> plain_internal_comparator;
+};
+
+class GeneralTableTest : public TableTest {};
+class BlockBasedTableTestBase : public TableTest {};
+class BlockBasedTableTest
+ : public BlockBasedTableTestBase,
+ virtual public ::testing::WithParamInterface<uint32_t> {
+ public:
+ BlockBasedTableTest() : format_(GetParam()) {
+ env_ = ROCKSDB_NAMESPACE::Env::Default();
+ }
+
+ BlockBasedTableOptions GetBlockBasedTableOptions() {
+ BlockBasedTableOptions options;
+ options.format_version = format_;
+ return options;
+ }
+
+ void SetupTracingTest(TableConstructor* c) {
+ test_path_ = test::PerThreadDBPath("block_based_table_tracing_test");
+ EXPECT_OK(env_->CreateDir(test_path_));
+ trace_file_path_ = test_path_ + "/block_cache_trace_file";
+
+ BlockCacheTraceWriterOptions trace_writer_opt;
+ BlockCacheTraceOptions trace_opt;
+ std::unique_ptr<TraceWriter> trace_writer;
+ EXPECT_OK(NewFileTraceWriter(env_, EnvOptions(), trace_file_path_,
+ &trace_writer));
+ std::unique_ptr<BlockCacheTraceWriter> block_cache_trace_writer =
+ NewBlockCacheTraceWriter(env_->GetSystemClock().get(), trace_writer_opt,
+ std::move(trace_writer));
+ ASSERT_NE(block_cache_trace_writer, nullptr);
+ // Always return Status::OK().
+ assert(c->block_cache_tracer_
+ .StartTrace(trace_opt, std::move(block_cache_trace_writer))
+ .ok());
+
+ {
+ std::string user_key = "k01";
+ InternalKey internal_key(user_key, 0, kTypeValue);
+ std::string encoded_key = internal_key.Encode().ToString();
+ c->Add(encoded_key, kDummyValue);
+ }
+ {
+ std::string user_key = "k02";
+ InternalKey internal_key(user_key, 0, kTypeValue);
+ std::string encoded_key = internal_key.Encode().ToString();
+ c->Add(encoded_key, kDummyValue);
+ }
+ }
+
+ void VerifyBlockAccessTrace(
+ TableConstructor* c,
+ const std::vector<BlockCacheTraceRecord>& expected_records) {
+ c->block_cache_tracer_.EndTrace();
+
+ {
+ std::unique_ptr<TraceReader> trace_reader;
+ Status s = NewFileTraceReader(env_, EnvOptions(), trace_file_path_,
+ &trace_reader);
+ EXPECT_OK(s);
+ BlockCacheTraceReader reader(std::move(trace_reader));
+ BlockCacheTraceHeader header;
+ EXPECT_OK(reader.ReadHeader(&header));
+ uint32_t index = 0;
+ while (s.ok()) {
+ BlockCacheTraceRecord access;
+ s = reader.ReadAccess(&access);
+ if (!s.ok()) {
+ break;
+ }
+ ASSERT_LT(index, expected_records.size());
+ EXPECT_NE("", access.block_key);
+ EXPECT_EQ(access.block_type, expected_records[index].block_type);
+ EXPECT_GT(access.block_size, 0);
+ EXPECT_EQ(access.caller, expected_records[index].caller);
+ EXPECT_EQ(access.no_insert, expected_records[index].no_insert);
+ EXPECT_EQ(access.is_cache_hit, expected_records[index].is_cache_hit);
+ // Get
+ if (access.caller == TableReaderCaller::kUserGet) {
+ EXPECT_EQ(access.referenced_key,
+ expected_records[index].referenced_key);
+ EXPECT_EQ(access.get_id, expected_records[index].get_id);
+ EXPECT_EQ(access.get_from_user_specified_snapshot,
+ expected_records[index].get_from_user_specified_snapshot);
+ if (access.block_type == TraceType::kBlockTraceDataBlock) {
+ EXPECT_GT(access.referenced_data_size, 0);
+ EXPECT_GT(access.num_keys_in_block, 0);
+ EXPECT_EQ(access.referenced_key_exist_in_block,
+ expected_records[index].referenced_key_exist_in_block);
+ }
+ } else {
+ EXPECT_EQ(access.referenced_key, "");
+ EXPECT_EQ(access.get_id, 0);
+ EXPECT_FALSE(access.get_from_user_specified_snapshot);
+ EXPECT_EQ(access.referenced_data_size, 0);
+ EXPECT_EQ(access.num_keys_in_block, 0);
+ EXPECT_FALSE(access.referenced_key_exist_in_block);
+ }
+ index++;
+ }
+ EXPECT_EQ(index, expected_records.size());
+ }
+ EXPECT_OK(env_->DeleteFile(trace_file_path_));
+ EXPECT_OK(env_->DeleteDir(test_path_));
+ }
+
+ protected:
+ uint64_t IndexUncompressedHelper(bool indexCompress);
+
+ private:
+ uint32_t format_;
+ Env* env_;
+ std::string trace_file_path_;
+ std::string test_path_;
+};
+class PlainTableTest : public TableTest {};
+class TablePropertyTest : public testing::Test {};
+class BBTTailPrefetchTest : public TableTest {};
+
+// The helper class to test the file checksum
+class FileChecksumTestHelper {
+ public:
+ FileChecksumTestHelper(bool convert_to_internal_key = false)
+ : convert_to_internal_key_(convert_to_internal_key) {}
+ ~FileChecksumTestHelper() {}
+
+ void CreateWritableFile() {
+ sink_ = new test::StringSink();
+ std::unique_ptr<FSWritableFile> holder(sink_);
+ file_writer_.reset(new WritableFileWriter(
+ std::move(holder), "" /* don't care */, FileOptions()));
+ }
+
+ void SetFileChecksumGenerator(FileChecksumGenerator* checksum_generator) {
+ if (file_writer_ != nullptr) {
+ file_writer_->TEST_SetFileChecksumGenerator(checksum_generator);
+ } else {
+ delete checksum_generator;
+ }
+ }
+
+ WritableFileWriter* GetFileWriter() { return file_writer_.get(); }
+
+ Status ResetTableBuilder(std::unique_ptr<TableBuilder>&& builder) {
+ assert(builder != nullptr);
+ table_builder_ = std::move(builder);
+ return Status::OK();
+ }
+
+ void AddKVtoKVMap(int num_entries) {
+ Random rnd(test::RandomSeed());
+ for (int i = 0; i < num_entries; i++) {
+ std::string v = rnd.RandomString(100);
+ kv_map_[test::RandomKey(&rnd, 20)] = v;
+ }
+ }
+
+ Status WriteKVAndFlushTable() {
+ for (const auto& kv : kv_map_) {
+ if (convert_to_internal_key_) {
+ ParsedInternalKey ikey(kv.first, kMaxSequenceNumber, kTypeValue);
+ std::string encoded;
+ AppendInternalKey(&encoded, ikey);
+ table_builder_->Add(encoded, kv.second);
+ } else {
+ table_builder_->Add(kv.first, kv.second);
+ }
+ EXPECT_TRUE(table_builder_->status().ok());
+ }
+ Status s = table_builder_->Finish();
+ EXPECT_OK(file_writer_->Flush());
+ EXPECT_OK(s);
+
+ EXPECT_EQ(sink_->contents().size(), table_builder_->FileSize());
+ return s;
+ }
+
+ std::string GetFileChecksum() {
+ EXPECT_OK(file_writer_->Close());
+ return table_builder_->GetFileChecksum();
+ }
+
+ const char* GetFileChecksumFuncName() {
+ return table_builder_->GetFileChecksumFuncName();
+ }
+
+ Status CalculateFileChecksum(FileChecksumGenerator* file_checksum_generator,
+ std::string* checksum) {
+ assert(file_checksum_generator != nullptr);
+ cur_file_num_ = checksum_file_num_++;
+ test::StringSink* ss_rw =
+ static_cast<test::StringSink*>(file_writer_->writable_file());
+ std::unique_ptr<FSRandomAccessFile> source(
+ new test::StringSource(ss_rw->contents()));
+ file_reader_.reset(new RandomAccessFileReader(std::move(source), "test"));
+
+ std::unique_ptr<char[]> scratch(new char[2048]);
+ Slice result;
+ uint64_t offset = 0;
+ Status s;
+ s = file_reader_->Read(IOOptions(), offset, 2048, &result, scratch.get(),
+ nullptr, Env::IO_TOTAL /* rate_limiter_priority */);
+ if (!s.ok()) {
+ return s;
+ }
+ while (result.size() != 0) {
+ file_checksum_generator->Update(scratch.get(), result.size());
+ offset += static_cast<uint64_t>(result.size());
+ s = file_reader_->Read(IOOptions(), offset, 2048, &result, scratch.get(),
+ nullptr,
+ Env::IO_TOTAL /* rate_limiter_priority */);
+ if (!s.ok()) {
+ return s;
+ }
+ }
+ EXPECT_EQ(offset, static_cast<uint64_t>(table_builder_->FileSize()));
+ file_checksum_generator->Finalize();
+ *checksum = file_checksum_generator->GetChecksum();
+ return Status::OK();
+ }
+
+ private:
+ bool convert_to_internal_key_;
+ uint64_t cur_file_num_;
+ std::unique_ptr<WritableFileWriter> file_writer_;
+ std::unique_ptr<RandomAccessFileReader> file_reader_;
+ std::unique_ptr<TableBuilder> table_builder_;
+ stl_wrappers::KVMap kv_map_;
+ test::StringSink* sink_ = nullptr;
+
+ static uint64_t checksum_file_num_;
+};
+
+uint64_t FileChecksumTestHelper::checksum_file_num_ = 1;
+
+INSTANTIATE_TEST_CASE_P(FormatVersions, BlockBasedTableTest,
+ testing::ValuesIn(test::kFooterFormatVersionsToTest));
+
+// This test serves as the living tutorial for the prefix scan of user collected
+// properties.
+TEST_F(TablePropertyTest, PrefixScanTest) {
+ UserCollectedProperties props{
+ {"num.111.1", "1"}, {"num.111.2", "2"}, {"num.111.3", "3"},
+ {"num.333.1", "1"}, {"num.333.2", "2"}, {"num.333.3", "3"},
+ {"num.555.1", "1"}, {"num.555.2", "2"}, {"num.555.3", "3"},
+ };
+
+ // prefixes that exist
+ for (const std::string prefix : {"num.111", "num.333", "num.555"}) {
+ int num = 0;
+ for (auto pos = props.lower_bound(prefix);
+ pos != props.end() &&
+ pos->first.compare(0, prefix.size(), prefix) == 0;
+ ++pos) {
+ ++num;
+ auto key = prefix + "." + std::to_string(num);
+ ASSERT_EQ(key, pos->first);
+ ASSERT_EQ(std::to_string(num), pos->second);
+ }
+ ASSERT_EQ(3, num);
+ }
+
+ // prefixes that don't exist
+ for (const std::string prefix :
+ {"num.000", "num.222", "num.444", "num.666"}) {
+ auto pos = props.lower_bound(prefix);
+ ASSERT_TRUE(pos == props.end() ||
+ pos->first.compare(0, prefix.size(), prefix) != 0);
+ }
+}
+
+namespace {
+struct TestIds {
+ UniqueId64x3 internal_id;
+ UniqueId64x3 external_id;
+};
+
+inline bool operator==(const TestIds& lhs, const TestIds& rhs) {
+ return lhs.internal_id == rhs.internal_id &&
+ lhs.external_id == rhs.external_id;
+}
+
+std::ostream& operator<<(std::ostream& os, const TestIds& ids) {
+ return os << std::hex << "{{{ 0x" << ids.internal_id[0] << "U, 0x"
+ << ids.internal_id[1] << "U, 0x" << ids.internal_id[2]
+ << "U }}, {{ 0x" << ids.external_id[0] << "U, 0x"
+ << ids.external_id[1] << "U, 0x" << ids.external_id[2] << "U }}}";
+}
+
+TestIds GetUniqueId(TableProperties* tp, std::unordered_set<uint64_t>* seen,
+ const std::string& db_id, const std::string& db_session_id,
+ uint64_t file_number) {
+ // First test session id logic
+ if (db_session_id.size() == 20) {
+ uint64_t upper;
+ uint64_t lower;
+ EXPECT_OK(DecodeSessionId(db_session_id, &upper, &lower));
+ EXPECT_EQ(EncodeSessionId(upper, lower), db_session_id);
+ }
+
+ // Get external using public API
+ tp->db_id = db_id;
+ tp->db_session_id = db_session_id;
+ tp->orig_file_number = file_number;
+ TestIds t;
+ {
+ std::string euid;
+ EXPECT_OK(GetExtendedUniqueIdFromTableProperties(*tp, &euid));
+ EXPECT_EQ(euid.size(), 24U);
+ t.external_id[0] = DecodeFixed64(&euid[0]);
+ t.external_id[1] = DecodeFixed64(&euid[8]);
+ t.external_id[2] = DecodeFixed64(&euid[16]);
+
+ std::string uid;
+ EXPECT_OK(GetUniqueIdFromTableProperties(*tp, &uid));
+ EXPECT_EQ(uid.size(), 16U);
+ EXPECT_EQ(uid, euid.substr(0, 16));
+ EXPECT_EQ(t.external_id[0], DecodeFixed64(&uid[0]));
+ EXPECT_EQ(t.external_id[1], DecodeFixed64(&uid[8]));
+ }
+ // All these should be effectively random
+ EXPECT_TRUE(seen->insert(t.external_id[0]).second);
+ EXPECT_TRUE(seen->insert(t.external_id[1]).second);
+ EXPECT_TRUE(seen->insert(t.external_id[2]).second);
+
+ // Get internal with internal API
+ EXPECT_OK(GetSstInternalUniqueId(db_id, db_session_id, file_number,
+ &t.internal_id));
+ EXPECT_NE(t.internal_id, kNullUniqueId64x3);
+
+ // Verify relationship
+ UniqueId64x3 tmp = t.internal_id;
+ InternalUniqueIdToExternal(&tmp);
+ EXPECT_EQ(tmp, t.external_id);
+ ExternalUniqueIdToInternal(&tmp);
+ EXPECT_EQ(tmp, t.internal_id);
+
+ // And 128-bit internal version
+ UniqueId64x2 tmp2{};
+ EXPECT_OK(GetSstInternalUniqueId(db_id, db_session_id, file_number, &tmp2));
+ EXPECT_NE(tmp2, kNullUniqueId64x2);
+
+ EXPECT_EQ(tmp2[0], t.internal_id[0]);
+ EXPECT_EQ(tmp2[1], t.internal_id[1]);
+ InternalUniqueIdToExternal(&tmp2);
+ EXPECT_EQ(tmp2[0], t.external_id[0]);
+ EXPECT_EQ(tmp2[1], t.external_id[1]);
+ ExternalUniqueIdToInternal(&tmp2);
+ EXPECT_EQ(tmp2[0], t.internal_id[0]);
+ EXPECT_EQ(tmp2[1], t.internal_id[1]);
+
+ return t;
+}
+} // namespace
+
+TEST_F(TablePropertyTest, UniqueIdsSchemaAndQuality) {
+ // To ensure the computation only depends on the expected entries, we set
+ // the rest randomly
+ TableProperties tp;
+ TEST_SetRandomTableProperties(&tp);
+
+ // DB id is normally RFC-4122
+ const std::string db_id1 = "7265b6eb-4e42-4aec-86a4-0dc5e73a228d";
+ // Allow other forms of DB id
+ const std::string db_id2 = "1728000184588763620";
+ const std::string db_id3 = "x";
+
+ // DB session id is normally 20 chars in base-36, but 13 to 24 chars
+ // is ok, roughly 64 to 128 bits.
+ const std::string ses_id1 = "ABCDEFGHIJ0123456789";
+ // Same trailing 13 digits
+ const std::string ses_id2 = "HIJ0123456789";
+ const std::string ses_id3 = "0123ABCDEFGHIJ0123456789";
+ // Different trailing 12 digits
+ const std::string ses_id4 = "ABCDEFGH888888888888";
+ // And change length
+ const std::string ses_id5 = "ABCDEFGHIJ012";
+ const std::string ses_id6 = "ABCDEFGHIJ0123456789ABCD";
+
+ using T = TestIds;
+ std::unordered_set<uint64_t> seen;
+ // Establish a stable schema for the unique IDs. These values must not
+ // change for existing table files.
+ // (Note: parens needed for macro parsing, extra braces needed for some
+ // compilers.)
+ EXPECT_EQ(
+ GetUniqueId(&tp, &seen, db_id1, ses_id1, 1),
+ T({{{0x61d7dcf415d9cf19U, 0x160d77aae90757fdU, 0x907f41dfd90724ffU}},
+ {{0xf0bd230365df7464U, 0xca089303f3648eb4U, 0x4b44f7e7324b2817U}}}));
+ // Only change internal_id[1] with file number
+ EXPECT_EQ(
+ GetUniqueId(&tp, &seen, db_id1, ses_id1, 2),
+ T({{{0x61d7dcf415d9cf19U, 0x160d77aae90757feU, 0x907f41dfd90724ffU}},
+ {{0xf13fdf7adcfebb6dU, 0x97cd2226cc033ea2U, 0x198c438182091f0eU}}}));
+ EXPECT_EQ(
+ GetUniqueId(&tp, &seen, db_id1, ses_id1, 123456789),
+ T({{{0x61d7dcf415d9cf19U, 0x160d77aaee5c9ae9U, 0x907f41dfd90724ffU}},
+ {{0x81fbcebe1ac6c4f0U, 0x6b14a64cfdc0f1c4U, 0x7d8fb6eaf18edbb3U}}}));
+ // Change internal_id[1] and internal_id[2] with db_id
+ EXPECT_EQ(
+ GetUniqueId(&tp, &seen, db_id2, ses_id1, 1),
+ T({{{0x61d7dcf415d9cf19U, 0xf89c471f572f0d25U, 0x1f0f2a5eb0e6257eU}},
+ {{0x7f1d01d453616991U, 0x32ddf2afec804ab2U, 0xd10a1ee2f0c7d9c1U}}}));
+ EXPECT_EQ(
+ GetUniqueId(&tp, &seen, db_id3, ses_id1, 1),
+ T({{{0x61d7dcf415d9cf19U, 0xfed297a8154a57d0U, 0x8b931b9cdebd9e8U}},
+ {{0x62b2f43183f6894bU, 0x897ff2b460eefad1U, 0xf4ec189fb2d15e04U}}}));
+ // Keeping same last 13 digits of ses_id keeps same internal_id[0]
+ EXPECT_EQ(
+ GetUniqueId(&tp, &seen, db_id1, ses_id2, 1),
+ T({{{0x61d7dcf415d9cf19U, 0x5f6cc4fa2d528c8U, 0x7b70845d5bfb5446U}},
+ {{0x96d1c83ffcc94266U, 0x82663eac0ec6e14aU, 0x94a88b49678b77f6U}}}));
+ EXPECT_EQ(
+ GetUniqueId(&tp, &seen, db_id1, ses_id3, 1),
+ T({{{0x61d7dcf415d9cf19U, 0xfc7232879db37ea2U, 0xc0378d74ea4c89cdU}},
+ {{0xdf2ef57e98776905U, 0xda5b31c987da833bU, 0x79c1b4bd0a9e760dU}}}));
+ // Changing last 12 digits of ses_id only changes internal_id[0]
+ // (vs. db_id1, ses_id1, 1)
+ EXPECT_EQ(
+ GetUniqueId(&tp, &seen, db_id1, ses_id4, 1),
+ T({{{0x4f07cc0d003a83a8U, 0x160d77aae90757fdU, 0x907f41dfd90724ffU}},
+ {{0xbcf85336a9f71f04U, 0x4f2949e2f3adb60dU, 0x9ca0def976abfa10U}}}));
+ // ses_id can change everything.
+ EXPECT_EQ(
+ GetUniqueId(&tp, &seen, db_id1, ses_id5, 1),
+ T({{{0x94b8768e43f87ce6U, 0xc2559653ac4e7c93U, 0xde6dff6bbb1223U}},
+ {{0x5a9537af681817fbU, 0x1afcd1fecaead5eaU, 0x767077ad9ebe0008U}}}));
+ EXPECT_EQ(
+ GetUniqueId(&tp, &seen, db_id1, ses_id6, 1),
+ T({{{0x43cfb0ffa3b710edU, 0x263c580426406a1bU, 0xfacc91379a80d29dU}},
+ {{0xfa90547d84cb1cdbU, 0x2afe99c641992d4aU, 0x205b7f7b60e51cc2U}}}));
+
+ // Now verify more thoroughly that any small change in inputs completely
+ // changes external unique id.
+ // (Relying on 'seen' checks etc. in GetUniqueId)
+ std::string db_id = "00000000-0000-0000-0000-000000000000";
+ std::string ses_id = "000000000000000000000000";
+ uint64_t file_num = 1;
+ // change db_id
+ for (size_t i = 0; i < db_id.size(); ++i) {
+ if (db_id[i] == '-') {
+ continue;
+ }
+ for (char alt : std::string("123456789abcdef")) {
+ db_id[i] = alt;
+ GetUniqueId(&tp, &seen, db_id, ses_id, file_num);
+ }
+ db_id[i] = '0';
+ }
+ // change ses_id
+ for (size_t i = 0; i < ses_id.size(); ++i) {
+ for (char alt : std::string("123456789ABCDEFGHIJKLMNOPQRSTUVWXYZ")) {
+ ses_id[i] = alt;
+ GetUniqueId(&tp, &seen, db_id, ses_id, file_num);
+ }
+ ses_id[i] = '0';
+ }
+ // change file_num
+ for (int i = 1; i < 64; ++i) {
+ GetUniqueId(&tp, &seen, db_id, ses_id, file_num << i);
+ }
+
+ // Verify that "all zeros" in first 128 bits is equivalent for internal and
+ // external IDs. This way, as long as we avoid "all zeros" in internal IDs,
+ // we avoid it in external IDs.
+ {
+ UniqueId64x3 id1{{0, 0, Random::GetTLSInstance()->Next64()}};
+ UniqueId64x3 id2 = id1;
+ InternalUniqueIdToExternal(&id1);
+ EXPECT_EQ(id1, id2);
+ ExternalUniqueIdToInternal(&id2);
+ EXPECT_EQ(id1, id2);
+ }
+}
+
+namespace {
+void SetGoodTableProperties(TableProperties* tp) {
+ // To ensure the computation only depends on the expected entries, we set
+ // the rest randomly
+ TEST_SetRandomTableProperties(tp);
+ tp->db_id = "7265b6eb-4e42-4aec-86a4-0dc5e73a228d";
+ tp->db_session_id = "ABCDEFGHIJ0123456789";
+ tp->orig_file_number = 1;
+}
+} // namespace
+
+TEST_F(TablePropertyTest, UniqueIdHumanStrings) {
+ TableProperties tp;
+ SetGoodTableProperties(&tp);
+
+ std::string tmp;
+ EXPECT_OK(GetExtendedUniqueIdFromTableProperties(tp, &tmp));
+ EXPECT_EQ(tmp,
+ (std::string{{'\x64', '\x74', '\xdf', '\x65', '\x03', '\x23',
+ '\xbd', '\xf0', '\xb4', '\x8e', '\x64', '\xf3',
+ '\x03', '\x93', '\x08', '\xca', '\x17', '\x28',
+ '\x4b', '\x32', '\xe7', '\xf7', '\x44', '\x4b'}}));
+ EXPECT_EQ(UniqueIdToHumanString(tmp),
+ "6474DF650323BDF0-B48E64F3039308CA-17284B32E7F7444B");
+
+ EXPECT_OK(GetUniqueIdFromTableProperties(tp, &tmp));
+ EXPECT_EQ(UniqueIdToHumanString(tmp), "6474DF650323BDF0-B48E64F3039308CA");
+
+ // including zero padding
+ tmp = std::string(24U, '\0');
+ tmp[15] = '\x12';
+ tmp[23] = '\xAB';
+ EXPECT_EQ(UniqueIdToHumanString(tmp),
+ "0000000000000000-0000000000000012-00000000000000AB");
+
+ // And shortened
+ tmp = std::string(20U, '\0');
+ tmp[5] = '\x12';
+ tmp[10] = '\xAB';
+ tmp[17] = '\xEF';
+ EXPECT_EQ(UniqueIdToHumanString(tmp),
+ "0000000000120000-0000AB0000000000-00EF0000");
+
+ tmp.resize(16);
+ EXPECT_EQ(UniqueIdToHumanString(tmp), "0000000000120000-0000AB0000000000");
+
+ tmp.resize(11);
+ EXPECT_EQ(UniqueIdToHumanString(tmp), "0000000000120000-0000AB");
+
+ tmp.resize(6);
+ EXPECT_EQ(UniqueIdToHumanString(tmp), "000000000012");
+
+ // Also internal IDs to human string
+ UniqueId64x3 euid = {12345, 678, 9};
+ EXPECT_EQ(InternalUniqueIdToHumanString(&euid), "{12345,678,9}");
+
+ UniqueId64x2 uid = {1234, 567890};
+ EXPECT_EQ(InternalUniqueIdToHumanString(&uid), "{1234,567890}");
+}
+
+TEST_F(TablePropertyTest, UniqueIdsFailure) {
+ TableProperties tp;
+ std::string tmp;
+
+ // Missing DB id
+ SetGoodTableProperties(&tp);
+ tp.db_id = "";
+ EXPECT_TRUE(GetUniqueIdFromTableProperties(tp, &tmp).IsNotSupported());
+ EXPECT_TRUE(
+ GetExtendedUniqueIdFromTableProperties(tp, &tmp).IsNotSupported());
+
+ // Missing session id
+ SetGoodTableProperties(&tp);
+ tp.db_session_id = "";
+ EXPECT_TRUE(GetUniqueIdFromTableProperties(tp, &tmp).IsNotSupported());
+ EXPECT_TRUE(
+ GetExtendedUniqueIdFromTableProperties(tp, &tmp).IsNotSupported());
+
+ // Missing file number
+ SetGoodTableProperties(&tp);
+ tp.orig_file_number = 0;
+ EXPECT_TRUE(GetUniqueIdFromTableProperties(tp, &tmp).IsNotSupported());
+ EXPECT_TRUE(
+ GetExtendedUniqueIdFromTableProperties(tp, &tmp).IsNotSupported());
+}
+
+// This test include all the basic checks except those for index size and block
+// size, which will be conducted in separated unit tests.
+TEST_P(BlockBasedTableTest, BasicBlockBasedTableProperties) {
+ TableConstructor c(BytewiseComparator(), true /* convert_to_internal_key_ */);
+
+ c.Add("a1", "val1");
+ c.Add("b2", "val2");
+ c.Add("c3", "val3");
+ c.Add("d4", "val4");
+ c.Add("e5", "val5");
+ c.Add("f6", "val6");
+ 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;
+ Options options;
+ options.compression = kNoCompression;
+ options.statistics = CreateDBStatistics();
+ options.statistics->set_stats_level(StatsLevel::kAll);
+ BlockBasedTableOptions table_options = GetBlockBasedTableOptions();
+ table_options.block_restart_interval = 1;
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+
+ ImmutableOptions ioptions(options);
+ MutableCFOptions moptions(options);
+ c.Finish(options, ioptions, moptions, table_options,
+ GetPlainInternalComparator(options.comparator), &keys, &kvmap);
+ ASSERT_EQ(options.statistics->getTickerCount(NUMBER_BLOCK_NOT_COMPRESSED), 0);
+
+ auto& props = *c.GetTableReader()->GetTableProperties();
+ ASSERT_EQ(kvmap.size(), props.num_entries);
+
+ auto raw_key_size = kvmap.size() * 2ul;
+ auto raw_value_size = kvmap.size() * 4ul;
+
+ 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
+
+ // Verify data size.
+ BlockBuilder block_builder(1);
+ for (const auto& item : kvmap) {
+ block_builder.Add(item.first, item.second);
+ }
+ Slice content = block_builder.Finish();
+ ASSERT_EQ(content.size() + BlockBasedTable::kBlockTrailerSize +
+ diff_internal_user_bytes,
+ props.data_size);
+ c.ResetTableReader();
+}
+
+#ifdef SNAPPY
+uint64_t BlockBasedTableTest::IndexUncompressedHelper(bool compressed) {
+ TableConstructor c(BytewiseComparator(), true /* convert_to_internal_key_ */);
+ constexpr size_t kNumKeys = 10000;
+
+ for (size_t k = 0; k < kNumKeys; ++k) {
+ c.Add("key" + std::to_string(k), "val" + std::to_string(k));
+ }
+
+ std::vector<std::string> keys;
+ stl_wrappers::KVMap kvmap;
+ Options options;
+ options.compression = kSnappyCompression;
+ options.statistics = CreateDBStatistics();
+ options.statistics->set_stats_level(StatsLevel::kAll);
+ BlockBasedTableOptions table_options = GetBlockBasedTableOptions();
+ table_options.block_restart_interval = 1;
+ table_options.enable_index_compression = compressed;
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+
+ ImmutableOptions ioptions(options);
+ MutableCFOptions moptions(options);
+ c.Finish(options, ioptions, moptions, table_options,
+ GetPlainInternalComparator(options.comparator), &keys, &kvmap);
+ c.ResetTableReader();
+ return options.statistics->getTickerCount(NUMBER_BLOCK_COMPRESSED);
+}
+TEST_P(BlockBasedTableTest, IndexUncompressed) {
+ uint64_t tbl1_compressed_cnt = IndexUncompressedHelper(true);
+ uint64_t tbl2_compressed_cnt = IndexUncompressedHelper(false);
+ // tbl1_compressed_cnt should include 1 index block
+ EXPECT_EQ(tbl2_compressed_cnt + 1, tbl1_compressed_cnt);
+}
+#endif // SNAPPY
+
+TEST_P(BlockBasedTableTest, BlockBasedTableProperties2) {
+ TableConstructor c(&reverse_key_comparator);
+ std::vector<std::string> keys;
+ stl_wrappers::KVMap kvmap;
+
+ {
+ Options options;
+ options.compression = CompressionType::kNoCompression;
+ BlockBasedTableOptions table_options = GetBlockBasedTableOptions();
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+
+ const ImmutableOptions ioptions(options);
+ const MutableCFOptions moptions(options);
+ c.Finish(options, ioptions, moptions, table_options,
+ GetPlainInternalComparator(options.comparator), &keys, &kvmap);
+
+ auto& props = *c.GetTableReader()->GetTableProperties();
+
+ // Default comparator
+ ASSERT_EQ("leveldb.BytewiseComparator", props.comparator_name);
+ // No merge operator
+ ASSERT_EQ("nullptr", props.merge_operator_name);
+ // No prefix extractor
+ ASSERT_EQ("nullptr", props.prefix_extractor_name);
+ // No property collectors
+ ASSERT_EQ("[]", props.property_collectors_names);
+ // No filter policy is used
+ ASSERT_EQ("", props.filter_policy_name);
+ // Compression type == that set:
+ ASSERT_EQ("NoCompression", props.compression_name);
+ c.ResetTableReader();
+ }
+
+ {
+ Options options;
+ BlockBasedTableOptions table_options = GetBlockBasedTableOptions();
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+ options.comparator = &reverse_key_comparator;
+ options.merge_operator = MergeOperators::CreateUInt64AddOperator();
+ options.prefix_extractor.reset(NewNoopTransform());
+ options.table_properties_collector_factories.emplace_back(
+ new DummyPropertiesCollectorFactory1());
+ options.table_properties_collector_factories.emplace_back(
+ new DummyPropertiesCollectorFactory2());
+
+ const ImmutableOptions ioptions(options);
+ const MutableCFOptions moptions(options);
+ c.Finish(options, ioptions, moptions, table_options,
+ GetPlainInternalComparator(options.comparator), &keys, &kvmap);
+
+ auto& props = *c.GetTableReader()->GetTableProperties();
+
+ ASSERT_EQ("rocksdb.ReverseBytewiseComparator", props.comparator_name);
+ ASSERT_EQ("UInt64AddOperator", props.merge_operator_name);
+ ASSERT_EQ("rocksdb.Noop", props.prefix_extractor_name);
+ ASSERT_EQ(
+ "[DummyPropertiesCollectorFactory1,DummyPropertiesCollectorFactory2]",
+ props.property_collectors_names);
+ ASSERT_EQ("", props.filter_policy_name); // no filter policy is used
+ c.ResetTableReader();
+ }
+}
+
+TEST_P(BlockBasedTableTest, RangeDelBlock) {
+ TableConstructor c(BytewiseComparator());
+ std::vector<std::string> keys = {"1pika", "2chu"};
+ std::vector<std::string> vals = {"p", "c"};
+
+ std::vector<RangeTombstone> expected_tombstones = {
+ {"1pika", "2chu", 0},
+ {"2chu", "c", 1},
+ {"2chu", "c", 0},
+ {"c", "p", 0},
+ };
+
+ 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 = GetBlockBasedTableOptions();
+ table_options.block_restart_interval = 1;
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+
+ const ImmutableOptions ioptions(options);
+ const MutableCFOptions moptions(options);
+ std::unique_ptr<InternalKeyComparator> internal_cmp(
+ new InternalKeyComparator(options.comparator));
+ c.Finish(options, ioptions, moptions, table_options, *internal_cmp,
+ &sorted_keys, &kvmap);
+
+ for (int j = 0; j < 2; ++j) {
+ std::unique_ptr<InternalIterator> iter(
+ c.GetTableReader()->NewRangeTombstoneIterator(ReadOptions()));
+ if (j > 0) {
+ // For second iteration, delete the table reader object and verify the
+ // iterator can still access its metablock's range tombstones.
+ c.ResetTableReader();
+ }
+ ASSERT_FALSE(iter->Valid());
+ iter->SeekToFirst();
+ ASSERT_TRUE(iter->Valid());
+ for (size_t i = 0; i < expected_tombstones.size(); i++) {
+ ASSERT_TRUE(iter->Valid());
+ ParsedInternalKey parsed_key;
+ ASSERT_OK(
+ ParseInternalKey(iter->key(), &parsed_key, true /* log_err_key */));
+ RangeTombstone t(parsed_key, iter->value());
+ const auto& expected_t = expected_tombstones[i];
+ ASSERT_EQ(t.start_key_, expected_t.start_key_);
+ ASSERT_EQ(t.end_key_, expected_t.end_key_);
+ ASSERT_EQ(t.seq_, expected_t.seq_);
+ iter->Next();
+ }
+ ASSERT_TRUE(!iter->Valid());
+ }
+}
+
+TEST_P(BlockBasedTableTest, FilterPolicyNameProperties) {
+ TableConstructor c(BytewiseComparator(), true /* convert_to_internal_key_ */);
+ c.Add("a1", "val1");
+ std::vector<std::string> keys;
+ stl_wrappers::KVMap kvmap;
+ BlockBasedTableOptions table_options = GetBlockBasedTableOptions();
+ table_options.filter_policy.reset(NewBloomFilterPolicy(10));
+ Options options;
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+
+ const ImmutableOptions ioptions(options);
+ const MutableCFOptions moptions(options);
+ c.Finish(options, ioptions, moptions, table_options,
+ GetPlainInternalComparator(options.comparator), &keys, &kvmap);
+ auto& props = *c.GetTableReader()->GetTableProperties();
+ ASSERT_EQ(table_options.filter_policy->Name(), props.filter_policy_name);
+ c.ResetTableReader();
+}
+
+//
+// BlockBasedTableTest::PrefetchTest
+//
+void AssertKeysInCache(BlockBasedTable* table_reader,
+ const std::vector<std::string>& keys_in_cache,
+ 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 char* key_begin,
+ const char* key_end,
+ const std::vector<std::string>& keys_in_cache,
+ const std::vector<std::string>& keys_not_in_cache,
+ const Status expected_status = Status::OK()) {
+ // reset the cache and reopen the table
+ table_options->block_cache = NewLRUCache(16 * 1024 * 1024, 4);
+ opt->table_factory.reset(NewBlockBasedTableFactory(*table_options));
+ const ImmutableOptions ioptions2(*opt);
+ const MutableCFOptions moptions(*opt);
+ ASSERT_OK(c->Reopen(ioptions2, moptions));
+
+ // prefetch
+ auto* table_reader = dynamic_cast<BlockBasedTable*>(c->GetTableReader());
+ Status s;
+ std::unique_ptr<Slice> begin, end;
+ std::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,
+ c->ConvertToInternalKey());
+ c->ResetTableReader();
+}
+
+TEST_P(BlockBasedTableTest, PrefetchTest) {
+ // The purpose of this test is to test the prefetching operation built into
+ // BlockBasedTable.
+ Options opt;
+ std::unique_ptr<InternalKeyComparator> ikc;
+ ikc.reset(new test::PlainInternalKeyComparator(opt.comparator));
+ opt.compression = kNoCompression;
+ BlockBasedTableOptions table_options = GetBlockBasedTableOptions();
+ table_options.block_size = 1024;
+ // big enough so we don't ever lose cached values.
+ table_options.block_cache = NewLRUCache(16 * 1024 * 1024, 4);
+ opt.table_factory.reset(NewBlockBasedTableFactory(table_options));
+
+ TableConstructor c(BytewiseComparator(), true /* convert_to_internal_key_ */);
+ c.Add("k01", "hello");
+ c.Add("k02", "hello2");
+ c.Add("k03", std::string(10000, 'x'));
+ c.Add("k04", std::string(200000, 'x'));
+ c.Add("k05", std::string(300000, 'x'));
+ c.Add("k06", "hello3");
+ c.Add("k07", std::string(100000, 'x'));
+ std::vector<std::string> keys;
+ stl_wrappers::KVMap kvmap;
+ const ImmutableOptions ioptions(opt);
+ const MutableCFOptions moptions(opt);
+ c.Finish(opt, ioptions, moptions, table_options, *ikc, &keys, &kvmap);
+ c.ResetTableReader();
+
+ // We get the following data spread :
+ //
+ // Data block Index
+ // ========================
+ // [ k01 k02 k03 ] k03
+ // [ k04 ] k04
+ // [ k05 ] k05
+ // [ k06 k07 ] k07
+
+ // Simple
+ 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, "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, "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, 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, "k06", "k00", {}, {},
+ Status::InvalidArgument(Slice("k06 "), Slice("k07")));
+ c.ResetTableReader();
+}
+
+TEST_P(BlockBasedTableTest, TotalOrderSeekOnHashIndex) {
+ BlockBasedTableOptions table_options = GetBlockBasedTableOptions();
+ for (int i = 0; i <= 4; ++i) {
+ Options options;
+ // Make each key/value an individual block
+ table_options.block_size = 64;
+ switch (i) {
+ case 0:
+ // Binary search index
+ table_options.index_type = BlockBasedTableOptions::kBinarySearch;
+ options.table_factory.reset(new BlockBasedTableFactory(table_options));
+ break;
+ case 1:
+ // Hash search index
+ table_options.index_type = BlockBasedTableOptions::kHashSearch;
+ options.table_factory.reset(new BlockBasedTableFactory(table_options));
+ options.prefix_extractor.reset(NewFixedPrefixTransform(4));
+ break;
+ case 2:
+ // Hash search index with filter policy
+ table_options.index_type = BlockBasedTableOptions::kHashSearch;
+ table_options.filter_policy.reset(NewBloomFilterPolicy(10));
+ options.table_factory.reset(new BlockBasedTableFactory(table_options));
+ options.prefix_extractor.reset(NewFixedPrefixTransform(4));
+ break;
+ case 3:
+ // Two-level index
+ table_options.index_type = BlockBasedTableOptions::kTwoLevelIndexSearch;
+ options.table_factory.reset(new BlockBasedTableFactory(table_options));
+ break;
+ case 4:
+ // Binary search with first key
+ table_options.index_type =
+ BlockBasedTableOptions::kBinarySearchWithFirstKey;
+ options.table_factory.reset(new BlockBasedTableFactory(table_options));
+ break;
+ }
+
+ 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));
+ c.Add("bbbb1", std::string('a', 56));
+ c.Add("baaa1", std::string('a', 56));
+ c.Add("abbb1", std::string('a', 56));
+ c.Add("cccc2", std::string('a', 56));
+ std::vector<std::string> keys;
+ stl_wrappers::KVMap kvmap;
+ const ImmutableOptions ioptions(options);
+ const MutableCFOptions moptions(options);
+ c.Finish(options, ioptions, moptions, table_options,
+ GetPlainInternalComparator(options.comparator), &keys, &kvmap);
+ auto props = c.GetTableReader()->GetTableProperties();
+ ASSERT_EQ(7u, props->num_data_blocks);
+ auto* reader = c.GetTableReader();
+ ReadOptions ro;
+ ro.total_order_seek = true;
+ std::unique_ptr<InternalIterator> iter(reader->NewIterator(
+ ro, moptions.prefix_extractor.get(), /*arena=*/nullptr,
+ /*skip_filters=*/false, TableReaderCaller::kUncategorized));
+
+ iter->Seek(InternalKey("b", 0, kTypeValue).Encode());
+ ASSERT_OK(iter->status());
+ ASSERT_TRUE(iter->Valid());
+ ASSERT_EQ("baaa1", ExtractUserKey(iter->key()).ToString());
+ iter->Next();
+ ASSERT_OK(iter->status());
+ ASSERT_TRUE(iter->Valid());
+ ASSERT_EQ("bbaa1", ExtractUserKey(iter->key()).ToString());
+
+ iter->Seek(InternalKey("bb", 0, kTypeValue).Encode());
+ ASSERT_OK(iter->status());
+ ASSERT_TRUE(iter->Valid());
+ ASSERT_EQ("bbaa1", ExtractUserKey(iter->key()).ToString());
+ iter->Next();
+ ASSERT_OK(iter->status());
+ ASSERT_TRUE(iter->Valid());
+ ASSERT_EQ("bbbb1", ExtractUserKey(iter->key()).ToString());
+
+ iter->Seek(InternalKey("bbb", 0, kTypeValue).Encode());
+ ASSERT_OK(iter->status());
+ ASSERT_TRUE(iter->Valid());
+ ASSERT_EQ("bbbb1", ExtractUserKey(iter->key()).ToString());
+ iter->Next();
+ ASSERT_OK(iter->status());
+ ASSERT_TRUE(iter->Valid());
+ ASSERT_EQ("cccc1", ExtractUserKey(iter->key()).ToString());
+ }
+}
+
+TEST_P(BlockBasedTableTest, NoopTransformSeek) {
+ BlockBasedTableOptions table_options = GetBlockBasedTableOptions();
+ table_options.filter_policy.reset(NewBloomFilterPolicy(10));
+
+ Options options;
+ options.comparator = BytewiseComparator();
+ options.table_factory.reset(new BlockBasedTableFactory(table_options));
+ options.prefix_extractor.reset(NewNoopTransform());
+
+ TableConstructor c(options.comparator);
+ // To tickle the PrefixMayMatch bug it is important that the
+ // user-key is a single byte so that the index key exactly matches
+ // the user-key.
+ InternalKey key("a", 1, kTypeValue);
+ c.Add(key.Encode().ToString(), "b");
+ std::vector<std::string> keys;
+ stl_wrappers::KVMap kvmap;
+ const ImmutableOptions ioptions(options);
+ const MutableCFOptions moptions(options);
+ const InternalKeyComparator internal_comparator(options.comparator);
+ c.Finish(options, ioptions, moptions, table_options, internal_comparator,
+ &keys, &kvmap);
+
+ auto* reader = c.GetTableReader();
+ for (int i = 0; i < 2; ++i) {
+ ReadOptions ro;
+ ro.total_order_seek = (i == 0);
+ std::unique_ptr<InternalIterator> iter(reader->NewIterator(
+ ro, moptions.prefix_extractor.get(), /*arena=*/nullptr,
+ /*skip_filters=*/false, TableReaderCaller::kUncategorized));
+
+ iter->Seek(key.Encode());
+ ASSERT_OK(iter->status());
+ ASSERT_TRUE(iter->Valid());
+ ASSERT_EQ("a", ExtractUserKey(iter->key()).ToString());
+ }
+}
+
+TEST_P(BlockBasedTableTest, SkipPrefixBloomFilter) {
+ // if DB is opened with a prefix extractor of a different name,
+ // prefix bloom is skipped when read the file
+ BlockBasedTableOptions table_options = GetBlockBasedTableOptions();
+ table_options.filter_policy.reset(NewBloomFilterPolicy(2));
+ table_options.whole_key_filtering = false;
+
+ Options options;
+ options.comparator = BytewiseComparator();
+ options.table_factory.reset(new BlockBasedTableFactory(table_options));
+ options.prefix_extractor.reset(NewFixedPrefixTransform(1));
+
+ TableConstructor c(options.comparator);
+ InternalKey key("abcdefghijk", 1, kTypeValue);
+ c.Add(key.Encode().ToString(), "test");
+ std::vector<std::string> keys;
+ stl_wrappers::KVMap kvmap;
+ const ImmutableOptions ioptions(options);
+ const MutableCFOptions moptions(options);
+ const InternalKeyComparator internal_comparator(options.comparator);
+ c.Finish(options, ioptions, moptions, table_options, internal_comparator,
+ &keys, &kvmap);
+ // TODO(Zhongyi): update test to use MutableCFOptions
+ options.prefix_extractor.reset(NewFixedPrefixTransform(9));
+ const ImmutableOptions new_ioptions(options);
+ const MutableCFOptions new_moptions(options);
+ ASSERT_OK(c.Reopen(new_ioptions, new_moptions));
+ auto reader = c.GetTableReader();
+ ReadOptions read_options;
+ std::unique_ptr<InternalIterator> db_iter(reader->NewIterator(
+ read_options, new_moptions.prefix_extractor.get(), /*arena=*/nullptr,
+ /*skip_filters=*/false, TableReaderCaller::kUncategorized));
+
+ // Test point lookup
+ // only one kv
+ for (auto& kv : kvmap) {
+ db_iter->Seek(kv.first);
+ ASSERT_TRUE(db_iter->Valid());
+ ASSERT_OK(db_iter->status());
+ ASSERT_EQ(db_iter->key(), kv.first);
+ ASSERT_EQ(db_iter->value(), kv.second);
+ }
+}
+
+TEST_P(BlockBasedTableTest, BadChecksumType) {
+ BlockBasedTableOptions table_options = GetBlockBasedTableOptions();
+
+ Options options;
+ options.comparator = BytewiseComparator();
+ options.table_factory.reset(new BlockBasedTableFactory(table_options));
+
+ TableConstructor c(options.comparator);
+ InternalKey key("abc", 1, kTypeValue);
+ c.Add(key.Encode().ToString(), "test");
+ std::vector<std::string> keys;
+ stl_wrappers::KVMap kvmap;
+ const ImmutableOptions ioptions(options);
+ const MutableCFOptions moptions(options);
+ const InternalKeyComparator internal_comparator(options.comparator);
+ c.Finish(options, ioptions, moptions, table_options, internal_comparator,
+ &keys, &kvmap);
+
+ // Corrupt checksum type (123 is invalid)
+ auto& sink = *c.TEST_GetSink();
+ size_t len = sink.contents_.size();
+ ASSERT_EQ(sink.contents_[len - Footer::kNewVersionsEncodedLength],
+ table_options.checksum);
+ sink.contents_[len - Footer::kNewVersionsEncodedLength] = char{123};
+
+ // (Re-)Open table file with bad checksum type
+ const ImmutableOptions new_ioptions(options);
+ const MutableCFOptions new_moptions(options);
+ Status s = c.Reopen(new_ioptions, new_moptions);
+ ASSERT_NOK(s);
+ ASSERT_EQ(s.ToString(),
+ "Corruption: Corrupt or unsupported checksum type: 123");
+}
+
+namespace {
+std::string ChecksumAsString(const std::string& data,
+ ChecksumType checksum_type) {
+ uint32_t v = ComputeBuiltinChecksum(checksum_type, data.data(), data.size());
+
+ // Verify consistency with other function
+ if (data.size() >= 1) {
+ EXPECT_EQ(v, ComputeBuiltinChecksumWithLastByte(
+ checksum_type, data.data(), data.size() - 1, data.back()));
+ }
+ // Little endian as in file
+ std::array<char, 4> raw_bytes;
+ EncodeFixed32(raw_bytes.data(), v);
+ return Slice(raw_bytes.data(), raw_bytes.size()).ToString(/*hex*/ true);
+}
+
+std::string ChecksumAsString(std::string* data, char new_last_byte,
+ ChecksumType checksum_type) {
+ data->back() = new_last_byte;
+ return ChecksumAsString(*data, checksum_type);
+}
+} // namespace
+
+// Make sure that checksum values don't change in later versions, even if
+// consistent within current version.
+TEST_P(BlockBasedTableTest, ChecksumSchemas) {
+ std::string b0 = "x";
+ std::string b1 = "This is a short block!x";
+ std::string b2;
+ for (int i = 0; i < 100; ++i) {
+ b2.append("This is a long block!");
+ }
+ b2.append("x");
+ // Trailing 'x' will be replaced by compression type
+
+ std::string empty;
+
+ char ct1 = kNoCompression;
+ char ct2 = kSnappyCompression;
+ char ct3 = kZSTD;
+
+ // Note: first byte of trailer is compression type, last 4 are checksum
+
+ for (ChecksumType t : GetSupportedChecksums()) {
+ switch (t) {
+ case kNoChecksum:
+ EXPECT_EQ(ChecksumAsString(empty, t), "00000000");
+ EXPECT_EQ(ChecksumAsString(&b0, ct1, t), "00000000");
+ EXPECT_EQ(ChecksumAsString(&b0, ct2, t), "00000000");
+ EXPECT_EQ(ChecksumAsString(&b0, ct3, t), "00000000");
+ EXPECT_EQ(ChecksumAsString(&b1, ct1, t), "00000000");
+ EXPECT_EQ(ChecksumAsString(&b1, ct2, t), "00000000");
+ EXPECT_EQ(ChecksumAsString(&b1, ct3, t), "00000000");
+ EXPECT_EQ(ChecksumAsString(&b2, ct1, t), "00000000");
+ EXPECT_EQ(ChecksumAsString(&b2, ct2, t), "00000000");
+ EXPECT_EQ(ChecksumAsString(&b2, ct3, t), "00000000");
+ break;
+ case kCRC32c:
+ EXPECT_EQ(ChecksumAsString(empty, t), "D8EA82A2");
+ EXPECT_EQ(ChecksumAsString(&b0, ct1, t), "D28F2549");
+ EXPECT_EQ(ChecksumAsString(&b0, ct2, t), "052B2843");
+ EXPECT_EQ(ChecksumAsString(&b0, ct3, t), "46F8F711");
+ EXPECT_EQ(ChecksumAsString(&b1, ct1, t), "583F0355");
+ EXPECT_EQ(ChecksumAsString(&b1, ct2, t), "2F9B0A57");
+ EXPECT_EQ(ChecksumAsString(&b1, ct3, t), "ECE7DA1D");
+ EXPECT_EQ(ChecksumAsString(&b2, ct1, t), "943EF0AB");
+ EXPECT_EQ(ChecksumAsString(&b2, ct2, t), "43A2EDB1");
+ EXPECT_EQ(ChecksumAsString(&b2, ct3, t), "00E53D63");
+ break;
+ case kxxHash:
+ EXPECT_EQ(ChecksumAsString(empty, t), "055DCC02");
+ EXPECT_EQ(ChecksumAsString(&b0, ct1, t), "3EB065CF");
+ EXPECT_EQ(ChecksumAsString(&b0, ct2, t), "31F79238");
+ EXPECT_EQ(ChecksumAsString(&b0, ct3, t), "320D2E00");
+ EXPECT_EQ(ChecksumAsString(&b1, ct1, t), "4A2E5FB0");
+ EXPECT_EQ(ChecksumAsString(&b1, ct2, t), "0BD9F652");
+ EXPECT_EQ(ChecksumAsString(&b1, ct3, t), "B4107E50");
+ EXPECT_EQ(ChecksumAsString(&b2, ct1, t), "20F4D4BA");
+ EXPECT_EQ(ChecksumAsString(&b2, ct2, t), "8F1A1F99");
+ EXPECT_EQ(ChecksumAsString(&b2, ct3, t), "A191A338");
+ break;
+ case kxxHash64:
+ EXPECT_EQ(ChecksumAsString(empty, t), "99E9D851");
+ EXPECT_EQ(ChecksumAsString(&b0, ct1, t), "682705DB");
+ EXPECT_EQ(ChecksumAsString(&b0, ct2, t), "30E7211B");
+ EXPECT_EQ(ChecksumAsString(&b0, ct3, t), "B7BB58E8");
+ EXPECT_EQ(ChecksumAsString(&b1, ct1, t), "B74655EF");
+ EXPECT_EQ(ChecksumAsString(&b1, ct2, t), "B6C8BBBE");
+ EXPECT_EQ(ChecksumAsString(&b1, ct3, t), "AED9E3B4");
+ EXPECT_EQ(ChecksumAsString(&b2, ct1, t), "0D4999FE");
+ EXPECT_EQ(ChecksumAsString(&b2, ct2, t), "F5932423");
+ EXPECT_EQ(ChecksumAsString(&b2, ct3, t), "6B31BAB1");
+ break;
+ case kXXH3:
+ EXPECT_EQ(ChecksumAsString(empty, t), "00000000");
+ EXPECT_EQ(ChecksumAsString(&b0, ct1, t), "C294D338");
+ EXPECT_EQ(ChecksumAsString(&b0, ct2, t), "1B174353");
+ EXPECT_EQ(ChecksumAsString(&b0, ct3, t), "2D0E20C8");
+ EXPECT_EQ(ChecksumAsString(&b1, ct1, t), "B37FB5E6");
+ EXPECT_EQ(ChecksumAsString(&b1, ct2, t), "6AFC258D");
+ EXPECT_EQ(ChecksumAsString(&b1, ct3, t), "5CE54616");
+ EXPECT_EQ(ChecksumAsString(&b2, ct1, t), "FA2D482E");
+ EXPECT_EQ(ChecksumAsString(&b2, ct2, t), "23AED845");
+ EXPECT_EQ(ChecksumAsString(&b2, ct3, t), "15B7BBDE");
+ break;
+ default:
+ // Force this test to be updated on new ChecksumTypes
+ assert(false);
+ break;
+ }
+ }
+}
+
+void AddInternalKey(TableConstructor* c, const std::string& prefix,
+ std::string value = "v", int /*suffix_len*/ = 800) {
+ static Random rnd(1023);
+ InternalKey k(prefix + rnd.RandomString(800), 0, kTypeValue);
+ c->Add(k.Encode().ToString(), value);
+}
+
+void TableTest::IndexTest(BlockBasedTableOptions table_options) {
+ TableConstructor c(BytewiseComparator());
+
+ // keys with prefix length 3, make sure the key/value is big enough to fill
+ // one block
+ AddInternalKey(&c, "0015");
+ AddInternalKey(&c, "0035");
+
+ AddInternalKey(&c, "0054");
+ AddInternalKey(&c, "0055");
+
+ AddInternalKey(&c, "0056");
+ AddInternalKey(&c, "0057");
+
+ AddInternalKey(&c, "0058");
+ AddInternalKey(&c, "0075");
+
+ AddInternalKey(&c, "0076");
+ AddInternalKey(&c, "0095");
+
+ std::vector<std::string> keys;
+ stl_wrappers::KVMap kvmap;
+ Options options;
+ options.prefix_extractor.reset(NewFixedPrefixTransform(3));
+ table_options.block_size = 1700;
+ table_options.block_cache = NewLRUCache(1024, 4);
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+
+ std::unique_ptr<InternalKeyComparator> comparator(
+ new InternalKeyComparator(BytewiseComparator()));
+ const ImmutableOptions ioptions(options);
+ const MutableCFOptions moptions(options);
+ c.Finish(options, ioptions, moptions, table_options, *comparator, &keys,
+ &kvmap);
+ auto reader = c.GetTableReader();
+
+ auto props = reader->GetTableProperties();
+ ASSERT_EQ(5u, props->num_data_blocks);
+
+ // TODO(Zhongyi): update test to use MutableCFOptions
+ ReadOptions read_options;
+ std::unique_ptr<InternalIterator> index_iter(reader->NewIterator(
+ read_options, moptions.prefix_extractor.get(), /*arena=*/nullptr,
+ /*skip_filters=*/false, TableReaderCaller::kUncategorized));
+
+ // -- Find keys do not exist, but have common prefix.
+ std::vector<std::string> prefixes = {"001", "003", "005", "007", "009"};
+ std::vector<std::string> lower_bound = {
+ keys[0], keys[1], keys[2], keys[7], keys[9],
+ };
+
+ // find the lower bound of the prefix
+ for (size_t i = 0; i < prefixes.size(); ++i) {
+ index_iter->Seek(InternalKey(prefixes[i], 0, kTypeValue).Encode());
+ ASSERT_OK(index_iter->status());
+ ASSERT_TRUE(index_iter->Valid());
+
+ // seek the first element in the block
+ ASSERT_EQ(lower_bound[i], index_iter->key().ToString());
+ ASSERT_EQ("v", index_iter->value().ToString());
+ }
+
+ // find the upper bound of prefixes
+ std::vector<std::string> upper_bound = {
+ keys[1],
+ keys[2],
+ keys[7],
+ keys[9],
+ };
+
+ // find existing keys
+ for (const auto& item : kvmap) {
+ auto ukey = ExtractUserKey(item.first).ToString();
+ index_iter->Seek(ukey);
+
+ // ASSERT_OK(regular_iter->status());
+ ASSERT_OK(index_iter->status());
+
+ // ASSERT_TRUE(regular_iter->Valid());
+ ASSERT_TRUE(index_iter->Valid());
+
+ ASSERT_EQ(item.first, index_iter->key().ToString());
+ ASSERT_EQ(item.second, index_iter->value().ToString());
+ }
+
+ for (size_t i = 0; i < prefixes.size(); ++i) {
+ // the key is greater than any existing keys.
+ auto key = prefixes[i] + "9";
+ index_iter->Seek(InternalKey(key, 0, kTypeValue).Encode());
+
+ ASSERT_TRUE(index_iter->status().ok() || index_iter->status().IsNotFound());
+ ASSERT_TRUE(!index_iter->status().IsNotFound() || !index_iter->Valid());
+ if (i == prefixes.size() - 1) {
+ // last key
+ ASSERT_TRUE(!index_iter->Valid());
+ } else {
+ ASSERT_TRUE(index_iter->Valid());
+ // seek the first element in the block
+ ASSERT_EQ(upper_bound[i], index_iter->key().ToString());
+ ASSERT_EQ("v", index_iter->value().ToString());
+ }
+ }
+
+ // find keys with prefix that don't match any of the existing prefixes.
+ std::vector<std::string> non_exist_prefixes = {"002", "004", "006", "008"};
+ for (const auto& prefix : non_exist_prefixes) {
+ index_iter->Seek(InternalKey(prefix, 0, kTypeValue).Encode());
+ // regular_iter->Seek(prefix);
+
+ ASSERT_OK(index_iter->status());
+ // Seek to non-existing prefixes should yield either invalid, or a
+ // key with prefix greater than the target.
+ if (index_iter->Valid()) {
+ Slice ukey = ExtractUserKey(index_iter->key());
+ Slice ukey_prefix = options.prefix_extractor->Transform(ukey);
+ ASSERT_TRUE(BytewiseComparator()->Compare(prefix, ukey_prefix) < 0);
+ }
+ }
+ for (const auto& prefix : non_exist_prefixes) {
+ index_iter->SeekForPrev(InternalKey(prefix, 0, kTypeValue).Encode());
+ // regular_iter->Seek(prefix);
+
+ ASSERT_OK(index_iter->status());
+ // Seek to non-existing prefixes should yield either invalid, or a
+ // key with prefix greater than the target.
+ if (index_iter->Valid()) {
+ Slice ukey = ExtractUserKey(index_iter->key());
+ Slice ukey_prefix = options.prefix_extractor->Transform(ukey);
+ ASSERT_TRUE(BytewiseComparator()->Compare(prefix, ukey_prefix) > 0);
+ }
+ }
+
+ {
+ // Test reseek case. It should impact partitioned index more.
+ ReadOptions ro;
+ ro.total_order_seek = true;
+ std::unique_ptr<InternalIterator> index_iter2(reader->NewIterator(
+ ro, moptions.prefix_extractor.get(), /*arena=*/nullptr,
+ /*skip_filters=*/false, TableReaderCaller::kUncategorized));
+
+ // Things to cover in partitioned index:
+ // 1. Both of Seek() and SeekToLast() has optimization to prevent
+ // rereek leaf index block if it remains to the same one, and
+ // they reuse the same variable.
+ // 2. When Next() or Prev() is called, the block moves, so the
+ // optimization should kick in only with the current one.
+ index_iter2->Seek(InternalKey("0055", 0, kTypeValue).Encode());
+ ASSERT_TRUE(index_iter2->Valid());
+ ASSERT_EQ("0055", index_iter2->key().ToString().substr(0, 4));
+
+ index_iter2->SeekToLast();
+ ASSERT_TRUE(index_iter2->Valid());
+ ASSERT_EQ("0095", index_iter2->key().ToString().substr(0, 4));
+
+ index_iter2->Seek(InternalKey("0055", 0, kTypeValue).Encode());
+ ASSERT_TRUE(index_iter2->Valid());
+ ASSERT_EQ("0055", index_iter2->key().ToString().substr(0, 4));
+
+ index_iter2->SeekToLast();
+ ASSERT_TRUE(index_iter2->Valid());
+ ASSERT_EQ("0095", index_iter2->key().ToString().substr(0, 4));
+ index_iter2->Prev();
+ ASSERT_TRUE(index_iter2->Valid());
+ index_iter2->Prev();
+ ASSERT_TRUE(index_iter2->Valid());
+ ASSERT_EQ("0075", index_iter2->key().ToString().substr(0, 4));
+
+ index_iter2->Seek(InternalKey("0095", 0, kTypeValue).Encode());
+ ASSERT_TRUE(index_iter2->Valid());
+ ASSERT_EQ("0095", index_iter2->key().ToString().substr(0, 4));
+ index_iter2->Prev();
+ ASSERT_TRUE(index_iter2->Valid());
+ index_iter2->Prev();
+ ASSERT_TRUE(index_iter2->Valid());
+ ASSERT_EQ("0075", index_iter2->key().ToString().substr(0, 4));
+
+ index_iter2->SeekToLast();
+ ASSERT_TRUE(index_iter2->Valid());
+ ASSERT_EQ("0095", index_iter2->key().ToString().substr(0, 4));
+
+ index_iter2->Seek(InternalKey("0095", 0, kTypeValue).Encode());
+ ASSERT_TRUE(index_iter2->Valid());
+ ASSERT_EQ("0095", index_iter2->key().ToString().substr(0, 4));
+
+ index_iter2->Prev();
+ ASSERT_TRUE(index_iter2->Valid());
+ index_iter2->Prev();
+ ASSERT_TRUE(index_iter2->Valid());
+ ASSERT_EQ("0075", index_iter2->key().ToString().substr(0, 4));
+
+ index_iter2->Seek(InternalKey("0075", 0, kTypeValue).Encode());
+ ASSERT_TRUE(index_iter2->Valid());
+ ASSERT_EQ("0075", index_iter2->key().ToString().substr(0, 4));
+
+ index_iter2->Next();
+ ASSERT_TRUE(index_iter2->Valid());
+ index_iter2->Next();
+ ASSERT_TRUE(index_iter2->Valid());
+ ASSERT_EQ("0095", index_iter2->key().ToString().substr(0, 4));
+
+ index_iter2->SeekToLast();
+ ASSERT_TRUE(index_iter2->Valid());
+ ASSERT_EQ("0095", index_iter2->key().ToString().substr(0, 4));
+ }
+
+ c.ResetTableReader();
+}
+
+TEST_P(BlockBasedTableTest, BinaryIndexTest) {
+ BlockBasedTableOptions table_options = GetBlockBasedTableOptions();
+ table_options.index_type = BlockBasedTableOptions::kBinarySearch;
+ IndexTest(table_options);
+}
+
+TEST_P(BlockBasedTableTest, HashIndexTest) {
+ BlockBasedTableOptions table_options = GetBlockBasedTableOptions();
+ table_options.index_type = BlockBasedTableOptions::kHashSearch;
+ IndexTest(table_options);
+}
+
+TEST_P(BlockBasedTableTest, PartitionIndexTest) {
+ const int max_index_keys = 5;
+ const int est_max_index_key_value_size = 32;
+ const int est_max_index_size = max_index_keys * est_max_index_key_value_size;
+ for (int i = 1; i <= est_max_index_size + 1; i++) {
+ BlockBasedTableOptions table_options = GetBlockBasedTableOptions();
+ table_options.index_type = BlockBasedTableOptions::kTwoLevelIndexSearch;
+ table_options.metadata_block_size = i;
+ IndexTest(table_options);
+ }
+}
+
+TEST_P(BlockBasedTableTest, IndexSeekOptimizationIncomplete) {
+ std::unique_ptr<InternalKeyComparator> comparator(
+ new InternalKeyComparator(BytewiseComparator()));
+ BlockBasedTableOptions table_options = GetBlockBasedTableOptions();
+ Options options;
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+ const ImmutableOptions ioptions(options);
+ const MutableCFOptions moptions(options);
+
+ TableConstructor c(BytewiseComparator());
+ AddInternalKey(&c, "pika");
+
+ std::vector<std::string> keys;
+ stl_wrappers::KVMap kvmap;
+ c.Finish(options, ioptions, moptions, table_options, *comparator, &keys,
+ &kvmap);
+ ASSERT_EQ(1, keys.size());
+
+ auto reader = c.GetTableReader();
+ ReadOptions ropt;
+ ropt.read_tier = ReadTier::kBlockCacheTier;
+ std::unique_ptr<InternalIterator> iter(reader->NewIterator(
+ ropt, /*prefix_extractor=*/nullptr, /*arena=*/nullptr,
+ /*skip_filters=*/false, TableReaderCaller::kUncategorized));
+
+ auto ikey = [](Slice user_key) {
+ return InternalKey(user_key, 0, kTypeValue).Encode().ToString();
+ };
+
+ iter->Seek(ikey("pika"));
+ ASSERT_FALSE(iter->Valid());
+ ASSERT_TRUE(iter->status().IsIncomplete());
+
+ // This used to crash at some point.
+ iter->Seek(ikey("pika"));
+ ASSERT_FALSE(iter->Valid());
+ ASSERT_TRUE(iter->status().IsIncomplete());
+}
+
+TEST_P(BlockBasedTableTest, BinaryIndexWithFirstKey1) {
+ BlockBasedTableOptions table_options = GetBlockBasedTableOptions();
+ table_options.index_type = BlockBasedTableOptions::kBinarySearchWithFirstKey;
+ IndexTest(table_options);
+}
+
+class CustomFlushBlockPolicy : public FlushBlockPolicyFactory,
+ public FlushBlockPolicy {
+ public:
+ explicit CustomFlushBlockPolicy(std::vector<int> keys_per_block)
+ : keys_per_block_(keys_per_block) {}
+
+ const char* Name() const override { return "CustomFlushBlockPolicy"; }
+
+ FlushBlockPolicy* NewFlushBlockPolicy(const BlockBasedTableOptions&,
+ const BlockBuilder&) const override {
+ return new CustomFlushBlockPolicy(keys_per_block_);
+ }
+
+ bool Update(const Slice&, const Slice&) override {
+ if (keys_in_current_block_ >= keys_per_block_.at(current_block_idx_)) {
+ ++current_block_idx_;
+ keys_in_current_block_ = 1;
+ return true;
+ }
+
+ ++keys_in_current_block_;
+ return false;
+ }
+
+ std::vector<int> keys_per_block_;
+
+ int current_block_idx_ = 0;
+ int keys_in_current_block_ = 0;
+};
+
+TEST_P(BlockBasedTableTest, BinaryIndexWithFirstKey2) {
+ for (int use_first_key = 0; use_first_key < 2; ++use_first_key) {
+ SCOPED_TRACE("use_first_key = " + std::to_string(use_first_key));
+ BlockBasedTableOptions table_options = GetBlockBasedTableOptions();
+ table_options.index_type =
+ use_first_key ? BlockBasedTableOptions::kBinarySearchWithFirstKey
+ : BlockBasedTableOptions::kBinarySearch;
+ table_options.block_cache = NewLRUCache(10000); // fits all blocks
+ table_options.index_shortening =
+ BlockBasedTableOptions::IndexShorteningMode::kNoShortening;
+ table_options.flush_block_policy_factory =
+ std::make_shared<CustomFlushBlockPolicy>(std::vector<int>{2, 1, 3, 2});
+ Options options;
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+ options.statistics = CreateDBStatistics();
+ Statistics* stats = options.statistics.get();
+ std::unique_ptr<InternalKeyComparator> comparator(
+ new InternalKeyComparator(BytewiseComparator()));
+ const ImmutableOptions ioptions(options);
+ const MutableCFOptions moptions(options);
+
+ TableConstructor c(BytewiseComparator());
+
+ // Block 0.
+ AddInternalKey(&c, "aaaa", "v0");
+ AddInternalKey(&c, "aaac", "v1");
+
+ // Block 1.
+ AddInternalKey(&c, "aaca", "v2");
+
+ // Block 2.
+ AddInternalKey(&c, "caaa", "v3");
+ AddInternalKey(&c, "caac", "v4");
+ AddInternalKey(&c, "caae", "v5");
+
+ // Block 3.
+ AddInternalKey(&c, "ccaa", "v6");
+ AddInternalKey(&c, "ccac", "v7");
+
+ // Write the file.
+ std::vector<std::string> keys;
+ stl_wrappers::KVMap kvmap;
+ c.Finish(options, ioptions, moptions, table_options, *comparator, &keys,
+ &kvmap);
+ ASSERT_EQ(8, keys.size());
+
+ auto reader = c.GetTableReader();
+ auto props = reader->GetTableProperties();
+ ASSERT_EQ(4u, props->num_data_blocks);
+ ReadOptions read_options;
+ std::unique_ptr<InternalIterator> iter(reader->NewIterator(
+ read_options, /*prefix_extractor=*/nullptr, /*arena=*/nullptr,
+ /*skip_filters=*/false, TableReaderCaller::kUncategorized,
+ /*compaction_readahead_size=*/0, /*allow_unprepared_value=*/true));
+
+ // Shouldn't have read data blocks before iterator is seeked.
+ EXPECT_EQ(0, stats->getTickerCount(BLOCK_CACHE_DATA_MISS));
+ EXPECT_EQ(0, stats->getTickerCount(BLOCK_CACHE_DATA_HIT));
+
+ auto ikey = [](Slice user_key) {
+ return InternalKey(user_key, 0, kTypeValue).Encode().ToString();
+ };
+
+ // Seek to a key between blocks. If index contains first key, we shouldn't
+ // read any data blocks until value is requested.
+ iter->Seek(ikey("aaba"));
+ ASSERT_TRUE(iter->Valid());
+ EXPECT_EQ(keys[2], iter->key().ToString());
+ EXPECT_EQ(use_first_key ? 0 : 1,
+ stats->getTickerCount(BLOCK_CACHE_DATA_MISS));
+ ASSERT_TRUE(iter->PrepareValue());
+ EXPECT_EQ("v2", iter->value().ToString());
+ EXPECT_EQ(1, stats->getTickerCount(BLOCK_CACHE_DATA_MISS));
+ EXPECT_EQ(0, stats->getTickerCount(BLOCK_CACHE_DATA_HIT));
+
+ // Seek to the middle of a block. The block should be read right away.
+ iter->Seek(ikey("caab"));
+ ASSERT_TRUE(iter->Valid());
+ EXPECT_EQ(keys[4], iter->key().ToString());
+ EXPECT_EQ(2, stats->getTickerCount(BLOCK_CACHE_DATA_MISS));
+ EXPECT_EQ(0, stats->getTickerCount(BLOCK_CACHE_DATA_HIT));
+ ASSERT_TRUE(iter->PrepareValue());
+ EXPECT_EQ("v4", iter->value().ToString());
+ EXPECT_EQ(0, stats->getTickerCount(BLOCK_CACHE_DATA_HIT));
+
+ // Seek to just before the same block and don't access value.
+ // The iterator should keep pinning the block contents.
+ iter->Seek(ikey("baaa"));
+ ASSERT_TRUE(iter->Valid());
+ EXPECT_EQ(keys[3], iter->key().ToString());
+ EXPECT_EQ(0, stats->getTickerCount(BLOCK_CACHE_DATA_HIT));
+
+ // Seek to the same block again to check that the block is still pinned.
+ iter->Seek(ikey("caae"));
+ ASSERT_TRUE(iter->Valid());
+ EXPECT_EQ(keys[5], iter->key().ToString());
+ EXPECT_EQ(0, stats->getTickerCount(BLOCK_CACHE_DATA_HIT));
+ ASSERT_TRUE(iter->PrepareValue());
+ EXPECT_EQ("v5", iter->value().ToString());
+ EXPECT_EQ(2, stats->getTickerCount(BLOCK_CACHE_DATA_MISS));
+ EXPECT_EQ(0, stats->getTickerCount(BLOCK_CACHE_DATA_HIT));
+
+ // Step forward and fall through to the next block. Don't access value.
+ iter->Next();
+ ASSERT_TRUE(iter->Valid());
+ EXPECT_EQ(keys[6], iter->key().ToString());
+ EXPECT_EQ(use_first_key ? 2 : 3,
+ stats->getTickerCount(BLOCK_CACHE_DATA_MISS));
+ EXPECT_EQ(0, stats->getTickerCount(BLOCK_CACHE_DATA_HIT));
+
+ // Step forward again. Block should be read.
+ iter->Next();
+ ASSERT_TRUE(iter->Valid());
+ EXPECT_EQ(keys[7], iter->key().ToString());
+ EXPECT_EQ(3, stats->getTickerCount(BLOCK_CACHE_DATA_MISS));
+ ASSERT_TRUE(iter->PrepareValue());
+ EXPECT_EQ("v7", iter->value().ToString());
+ EXPECT_EQ(0, stats->getTickerCount(BLOCK_CACHE_DATA_HIT));
+
+ // Step forward and reach the end.
+ iter->Next();
+ EXPECT_FALSE(iter->Valid());
+ EXPECT_EQ(3, stats->getTickerCount(BLOCK_CACHE_DATA_MISS));
+ EXPECT_EQ(0, stats->getTickerCount(BLOCK_CACHE_DATA_HIT));
+
+ // Seek to a single-key block and step forward without accessing value.
+ iter->Seek(ikey("aaca"));
+ ASSERT_TRUE(iter->Valid());
+ EXPECT_EQ(keys[2], iter->key().ToString());
+ EXPECT_EQ(use_first_key ? 0 : 1,
+ stats->getTickerCount(BLOCK_CACHE_DATA_HIT));
+
+ iter->Next();
+ ASSERT_TRUE(iter->Valid());
+ EXPECT_EQ(keys[3], iter->key().ToString());
+ EXPECT_EQ(use_first_key ? 1 : 2,
+ stats->getTickerCount(BLOCK_CACHE_DATA_HIT));
+ ASSERT_TRUE(iter->PrepareValue());
+ EXPECT_EQ("v3", iter->value().ToString());
+ EXPECT_EQ(2, stats->getTickerCount(BLOCK_CACHE_DATA_HIT));
+ EXPECT_EQ(3, stats->getTickerCount(BLOCK_CACHE_DATA_MISS));
+
+ // Seek between blocks and step back without accessing value.
+ iter->Seek(ikey("aaca"));
+ ASSERT_TRUE(iter->Valid());
+ EXPECT_EQ(keys[2], iter->key().ToString());
+ EXPECT_EQ(use_first_key ? 2 : 3,
+ stats->getTickerCount(BLOCK_CACHE_DATA_HIT));
+ EXPECT_EQ(3, stats->getTickerCount(BLOCK_CACHE_DATA_MISS));
+
+ iter->Prev();
+ ASSERT_TRUE(iter->Valid());
+ EXPECT_EQ(keys[1], iter->key().ToString());
+ EXPECT_EQ(use_first_key ? 2 : 3,
+ stats->getTickerCount(BLOCK_CACHE_DATA_HIT));
+ // All blocks are in cache now, there'll be no more misses ever.
+ EXPECT_EQ(4, stats->getTickerCount(BLOCK_CACHE_DATA_MISS));
+ ASSERT_TRUE(iter->PrepareValue());
+ EXPECT_EQ("v1", iter->value().ToString());
+
+ // Next into the next block again.
+ iter->Next();
+ ASSERT_TRUE(iter->Valid());
+ EXPECT_EQ(keys[2], iter->key().ToString());
+ EXPECT_EQ(use_first_key ? 2 : 4,
+ stats->getTickerCount(BLOCK_CACHE_DATA_HIT));
+
+ // Seek to first and step back without accessing value.
+ iter->SeekToFirst();
+ ASSERT_TRUE(iter->Valid());
+ EXPECT_EQ(keys[0], iter->key().ToString());
+ EXPECT_EQ(use_first_key ? 2 : 5,
+ stats->getTickerCount(BLOCK_CACHE_DATA_HIT));
+
+ iter->Prev();
+ EXPECT_FALSE(iter->Valid());
+ EXPECT_EQ(use_first_key ? 2 : 5,
+ stats->getTickerCount(BLOCK_CACHE_DATA_HIT));
+
+ // Do some SeekForPrev() and SeekToLast() just to cover all methods.
+ iter->SeekForPrev(ikey("caad"));
+ ASSERT_TRUE(iter->Valid());
+ EXPECT_EQ(keys[4], iter->key().ToString());
+ EXPECT_EQ(use_first_key ? 3 : 6,
+ stats->getTickerCount(BLOCK_CACHE_DATA_HIT));
+ ASSERT_TRUE(iter->PrepareValue());
+ EXPECT_EQ("v4", iter->value().ToString());
+ EXPECT_EQ(use_first_key ? 3 : 6,
+ stats->getTickerCount(BLOCK_CACHE_DATA_HIT));
+
+ iter->SeekToLast();
+ ASSERT_TRUE(iter->Valid());
+ EXPECT_EQ(keys[7], iter->key().ToString());
+ EXPECT_EQ(use_first_key ? 4 : 7,
+ stats->getTickerCount(BLOCK_CACHE_DATA_HIT));
+ ASSERT_TRUE(iter->PrepareValue());
+ EXPECT_EQ("v7", iter->value().ToString());
+ EXPECT_EQ(use_first_key ? 4 : 7,
+ stats->getTickerCount(BLOCK_CACHE_DATA_HIT));
+
+ EXPECT_EQ(4, stats->getTickerCount(BLOCK_CACHE_DATA_MISS));
+
+ c.ResetTableReader();
+ }
+}
+
+TEST_P(BlockBasedTableTest, BinaryIndexWithFirstKeyGlobalSeqno) {
+ BlockBasedTableOptions table_options = GetBlockBasedTableOptions();
+ table_options.index_type = BlockBasedTableOptions::kBinarySearchWithFirstKey;
+ table_options.block_cache = NewLRUCache(10000);
+ Options options;
+ options.statistics = CreateDBStatistics();
+ Statistics* stats = options.statistics.get();
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+ std::unique_ptr<InternalKeyComparator> comparator(
+ new InternalKeyComparator(BytewiseComparator()));
+ const ImmutableOptions ioptions(options);
+ const MutableCFOptions moptions(options);
+
+ TableConstructor c(BytewiseComparator(), /* convert_to_internal_key */ false,
+ /* level */ -1, /* largest_seqno */ 42);
+
+ c.Add(InternalKey("b", 0, kTypeValue).Encode().ToString(), "x");
+ c.Add(InternalKey("c", 0, kTypeValue).Encode().ToString(), "y");
+
+ std::vector<std::string> keys;
+ stl_wrappers::KVMap kvmap;
+ c.Finish(options, ioptions, moptions, table_options, *comparator, &keys,
+ &kvmap);
+ ASSERT_EQ(2, keys.size());
+
+ auto reader = c.GetTableReader();
+ auto props = reader->GetTableProperties();
+ ASSERT_EQ(1u, props->num_data_blocks);
+ ReadOptions read_options;
+ std::unique_ptr<InternalIterator> iter(reader->NewIterator(
+ read_options, /*prefix_extractor=*/nullptr, /*arena=*/nullptr,
+ /*skip_filters=*/false, TableReaderCaller::kUncategorized,
+ /*compaction_readahead_size=*/0, /*allow_unprepared_value=*/true));
+
+ iter->Seek(InternalKey("a", 0, kTypeValue).Encode().ToString());
+ ASSERT_TRUE(iter->Valid());
+ EXPECT_EQ(InternalKey("b", 42, kTypeValue).Encode().ToString(),
+ iter->key().ToString());
+ EXPECT_NE(keys[0], iter->key().ToString());
+ // Key should have been served from index, without reading data blocks.
+ EXPECT_EQ(0, stats->getTickerCount(BLOCK_CACHE_DATA_MISS));
+
+ ASSERT_TRUE(iter->PrepareValue());
+ EXPECT_EQ("x", iter->value().ToString());
+ EXPECT_EQ(1, stats->getTickerCount(BLOCK_CACHE_DATA_MISS));
+ EXPECT_EQ(0, stats->getTickerCount(BLOCK_CACHE_DATA_HIT));
+ EXPECT_EQ(InternalKey("b", 42, kTypeValue).Encode().ToString(),
+ iter->key().ToString());
+
+ c.ResetTableReader();
+}
+
+// It's very hard to figure out the index block size of a block accurately.
+// To make sure we get the index size, we just make sure as key number
+// grows, the filter block size also grows.
+TEST_P(BlockBasedTableTest, IndexSizeStat) {
+ uint64_t last_index_size = 0;
+
+ // we need to use random keys since the pure human readable texts
+ // may be well compressed, resulting insignifcant change of index
+ // block size.
+ Random rnd(test::RandomSeed());
+ std::vector<std::string> keys;
+
+ for (int i = 0; i < 100; ++i) {
+ keys.push_back(rnd.RandomString(10000));
+ }
+
+ // 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(),
+ true /* convert_to_internal_key_ */);
+ for (size_t j = 0; j < i; ++j) {
+ c.Add(keys[j], "val");
+ }
+
+ std::vector<std::string> ks;
+ stl_wrappers::KVMap kvmap;
+ Options options;
+ options.compression = kNoCompression;
+ BlockBasedTableOptions table_options = GetBlockBasedTableOptions();
+ table_options.block_restart_interval = 1;
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+
+ const ImmutableOptions ioptions(options);
+ const MutableCFOptions moptions(options);
+ c.Finish(options, ioptions, moptions, table_options,
+ GetPlainInternalComparator(options.comparator), &ks, &kvmap);
+ auto index_size = c.GetTableReader()->GetTableProperties()->index_size;
+ ASSERT_GT(index_size, last_index_size);
+ last_index_size = index_size;
+ c.ResetTableReader();
+ }
+}
+
+TEST_P(BlockBasedTableTest, NumBlockStat) {
+ Random rnd(test::RandomSeed());
+ TableConstructor c(BytewiseComparator(), true /* convert_to_internal_key_ */);
+ Options options;
+ options.compression = kNoCompression;
+ BlockBasedTableOptions table_options = GetBlockBasedTableOptions();
+ table_options.block_restart_interval = 1;
+ table_options.block_size = 1000;
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+
+ for (int i = 0; i < 10; ++i) {
+ // the key/val are slightly smaller than block size, so that each block
+ // holds roughly one key/value pair.
+ c.Add(rnd.RandomString(900), "val");
+ }
+
+ std::vector<std::string> ks;
+ stl_wrappers::KVMap kvmap;
+ const ImmutableOptions ioptions(options);
+ const MutableCFOptions moptions(options);
+ c.Finish(options, ioptions, moptions, table_options,
+ GetPlainInternalComparator(options.comparator), &ks, &kvmap);
+ ASSERT_EQ(kvmap.size(),
+ c.GetTableReader()->GetTableProperties()->num_data_blocks);
+ c.ResetTableReader();
+}
+
+TEST_P(BlockBasedTableTest, TracingGetTest) {
+ TableConstructor c(BytewiseComparator());
+ Options options;
+ BlockBasedTableOptions table_options = GetBlockBasedTableOptions();
+ options.create_if_missing = true;
+ table_options.block_cache = NewLRUCache(1024 * 1024, 0);
+ table_options.cache_index_and_filter_blocks = true;
+ table_options.filter_policy.reset(NewBloomFilterPolicy(10));
+ options.table_factory.reset(new BlockBasedTableFactory(table_options));
+ SetupTracingTest(&c);
+ std::vector<std::string> keys;
+ stl_wrappers::KVMap kvmap;
+ ImmutableOptions ioptions(options);
+ MutableCFOptions moptions(options);
+ c.Finish(options, ioptions, moptions, table_options,
+ GetPlainInternalComparator(options.comparator), &keys, &kvmap);
+ std::string user_key = "k01";
+ InternalKey internal_key(user_key, 0, kTypeValue);
+ std::string encoded_key = internal_key.Encode().ToString();
+ for (uint32_t i = 1; i <= 2; i++) {
+ PinnableSlice value;
+ GetContext get_context(options.comparator, nullptr, nullptr, nullptr,
+ GetContext::kNotFound, user_key, &value, nullptr,
+ nullptr, nullptr, true, nullptr, nullptr, nullptr,
+ nullptr, nullptr, nullptr, /*tracing_get_id=*/i);
+ get_perf_context()->Reset();
+ ASSERT_OK(c.GetTableReader()->Get(ReadOptions(), encoded_key, &get_context,
+ moptions.prefix_extractor.get()));
+ ASSERT_EQ(get_context.State(), GetContext::kFound);
+ ASSERT_EQ(value.ToString(), kDummyValue);
+ }
+
+ // Verify traces.
+ std::vector<BlockCacheTraceRecord> expected_records;
+ // The first two records should be prefetching index and filter blocks.
+ BlockCacheTraceRecord record;
+ record.block_type = TraceType::kBlockTraceIndexBlock;
+ record.caller = TableReaderCaller::kPrefetch;
+ record.is_cache_hit = false;
+ record.no_insert = false;
+ expected_records.push_back(record);
+ record.block_type = TraceType::kBlockTraceFilterBlock;
+ expected_records.push_back(record);
+ // Then we should have three records for one index, one filter, and one data
+ // block access.
+ record.get_id = 1;
+ record.block_type = TraceType::kBlockTraceFilterBlock;
+ record.caller = TableReaderCaller::kUserGet;
+ record.get_from_user_specified_snapshot = false;
+ record.referenced_key = encoded_key;
+ record.referenced_key_exist_in_block = true;
+ record.is_cache_hit = true;
+ expected_records.push_back(record);
+ record.block_type = TraceType::kBlockTraceIndexBlock;
+ expected_records.push_back(record);
+ record.is_cache_hit = false;
+ record.block_type = TraceType::kBlockTraceDataBlock;
+ expected_records.push_back(record);
+ // The second get should all observe cache hits.
+ record.is_cache_hit = true;
+ record.get_id = 2;
+ record.block_type = TraceType::kBlockTraceFilterBlock;
+ record.caller = TableReaderCaller::kUserGet;
+ record.get_from_user_specified_snapshot = false;
+ record.referenced_key = encoded_key;
+ expected_records.push_back(record);
+ record.block_type = TraceType::kBlockTraceIndexBlock;
+ expected_records.push_back(record);
+ record.block_type = TraceType::kBlockTraceDataBlock;
+ expected_records.push_back(record);
+ VerifyBlockAccessTrace(&c, expected_records);
+ c.ResetTableReader();
+}
+
+TEST_P(BlockBasedTableTest, TracingApproximateOffsetOfTest) {
+ TableConstructor c(BytewiseComparator());
+ Options options;
+ BlockBasedTableOptions table_options = GetBlockBasedTableOptions();
+ options.create_if_missing = true;
+ table_options.block_cache = NewLRUCache(1024 * 1024, 0);
+ table_options.cache_index_and_filter_blocks = true;
+ table_options.filter_policy.reset(NewBloomFilterPolicy(10, true));
+ options.table_factory.reset(new BlockBasedTableFactory(table_options));
+ SetupTracingTest(&c);
+ std::vector<std::string> keys;
+ stl_wrappers::KVMap kvmap;
+ ImmutableOptions ioptions(options);
+ MutableCFOptions moptions(options);
+ c.Finish(options, ioptions, moptions, table_options,
+ GetPlainInternalComparator(options.comparator), &keys, &kvmap);
+ for (uint32_t i = 1; i <= 2; i++) {
+ std::string user_key = "k01";
+ InternalKey internal_key(user_key, 0, kTypeValue);
+ std::string encoded_key = internal_key.Encode().ToString();
+ c.GetTableReader()->ApproximateOffsetOf(
+ encoded_key, TableReaderCaller::kUserApproximateSize);
+ }
+ // Verify traces.
+ std::vector<BlockCacheTraceRecord> expected_records;
+ // The first two records should be prefetching index and filter blocks.
+ BlockCacheTraceRecord record;
+ record.block_type = TraceType::kBlockTraceIndexBlock;
+ record.caller = TableReaderCaller::kPrefetch;
+ record.is_cache_hit = false;
+ record.no_insert = false;
+ expected_records.push_back(record);
+ record.block_type = TraceType::kBlockTraceFilterBlock;
+ expected_records.push_back(record);
+ // Then we should have two records for only index blocks.
+ record.block_type = TraceType::kBlockTraceIndexBlock;
+ record.caller = TableReaderCaller::kUserApproximateSize;
+ record.is_cache_hit = true;
+ expected_records.push_back(record);
+ expected_records.push_back(record);
+ VerifyBlockAccessTrace(&c, expected_records);
+ c.ResetTableReader();
+}
+
+TEST_P(BlockBasedTableTest, TracingIterator) {
+ TableConstructor c(BytewiseComparator());
+ Options options;
+ BlockBasedTableOptions table_options = GetBlockBasedTableOptions();
+ options.create_if_missing = true;
+ table_options.block_cache = NewLRUCache(1024 * 1024, 0);
+ table_options.cache_index_and_filter_blocks = true;
+ table_options.filter_policy.reset(NewBloomFilterPolicy(10, true));
+ options.table_factory.reset(new BlockBasedTableFactory(table_options));
+ SetupTracingTest(&c);
+ std::vector<std::string> keys;
+ stl_wrappers::KVMap kvmap;
+ ImmutableOptions ioptions(options);
+ MutableCFOptions moptions(options);
+ c.Finish(options, ioptions, moptions, table_options,
+ GetPlainInternalComparator(options.comparator), &keys, &kvmap);
+
+ for (uint32_t i = 1; i <= 2; i++) {
+ ReadOptions read_options;
+ std::unique_ptr<InternalIterator> iter(c.GetTableReader()->NewIterator(
+ read_options, moptions.prefix_extractor.get(), /*arena=*/nullptr,
+ /*skip_filters=*/false, TableReaderCaller::kUserIterator));
+ iter->SeekToFirst();
+ while (iter->Valid()) {
+ iter->key();
+ iter->value();
+ iter->Next();
+ }
+ ASSERT_OK(iter->status());
+ iter.reset();
+ }
+
+ // Verify traces.
+ std::vector<BlockCacheTraceRecord> expected_records;
+ // The first two records should be prefetching index and filter blocks.
+ BlockCacheTraceRecord record;
+ record.block_type = TraceType::kBlockTraceIndexBlock;
+ record.caller = TableReaderCaller::kPrefetch;
+ record.is_cache_hit = false;
+ record.no_insert = false;
+ expected_records.push_back(record);
+ record.block_type = TraceType::kBlockTraceFilterBlock;
+ expected_records.push_back(record);
+ // Then we should have three records for index and two data block access.
+ record.block_type = TraceType::kBlockTraceIndexBlock;
+ record.caller = TableReaderCaller::kUserIterator;
+ record.is_cache_hit = true;
+ expected_records.push_back(record);
+ record.block_type = TraceType::kBlockTraceDataBlock;
+ record.is_cache_hit = false;
+ expected_records.push_back(record);
+ expected_records.push_back(record);
+ // When we iterate this file for the second time, we should observe all cache
+ // hits.
+ record.block_type = TraceType::kBlockTraceIndexBlock;
+ record.is_cache_hit = true;
+ expected_records.push_back(record);
+ record.block_type = TraceType::kBlockTraceDataBlock;
+ expected_records.push_back(record);
+ expected_records.push_back(record);
+ VerifyBlockAccessTrace(&c, expected_records);
+ c.ResetTableReader();
+}
+
+// A simple tool that takes the snapshot of block cache statistics.
+class BlockCachePropertiesSnapshot {
+ public:
+ explicit BlockCachePropertiesSnapshot(Statistics* statistics) {
+ block_cache_miss = statistics->getTickerCount(BLOCK_CACHE_MISS);
+ block_cache_hit = statistics->getTickerCount(BLOCK_CACHE_HIT);
+ index_block_cache_miss = statistics->getTickerCount(BLOCK_CACHE_INDEX_MISS);
+ index_block_cache_hit = statistics->getTickerCount(BLOCK_CACHE_INDEX_HIT);
+ data_block_cache_miss = statistics->getTickerCount(BLOCK_CACHE_DATA_MISS);
+ data_block_cache_hit = statistics->getTickerCount(BLOCK_CACHE_DATA_HIT);
+ filter_block_cache_miss =
+ statistics->getTickerCount(BLOCK_CACHE_FILTER_MISS);
+ filter_block_cache_hit = statistics->getTickerCount(BLOCK_CACHE_FILTER_HIT);
+ block_cache_bytes_read = statistics->getTickerCount(BLOCK_CACHE_BYTES_READ);
+ block_cache_bytes_write =
+ statistics->getTickerCount(BLOCK_CACHE_BYTES_WRITE);
+ }
+
+ void AssertIndexBlockStat(int64_t expected_index_block_cache_miss,
+ int64_t expected_index_block_cache_hit) {
+ ASSERT_EQ(expected_index_block_cache_miss, index_block_cache_miss);
+ ASSERT_EQ(expected_index_block_cache_hit, index_block_cache_hit);
+ }
+
+ void AssertFilterBlockStat(int64_t expected_filter_block_cache_miss,
+ int64_t expected_filter_block_cache_hit) {
+ ASSERT_EQ(expected_filter_block_cache_miss, filter_block_cache_miss);
+ ASSERT_EQ(expected_filter_block_cache_hit, filter_block_cache_hit);
+ }
+
+ // Check if the fetched props matches the expected ones.
+ // TODO(kailiu) Use this only when you disabled filter policy!
+ void AssertEqual(int64_t expected_index_block_cache_miss,
+ int64_t expected_index_block_cache_hit,
+ int64_t expected_data_block_cache_miss,
+ int64_t expected_data_block_cache_hit) const {
+ ASSERT_EQ(expected_index_block_cache_miss, index_block_cache_miss);
+ ASSERT_EQ(expected_index_block_cache_hit, index_block_cache_hit);
+ ASSERT_EQ(expected_data_block_cache_miss, data_block_cache_miss);
+ ASSERT_EQ(expected_data_block_cache_hit, data_block_cache_hit);
+ ASSERT_EQ(expected_index_block_cache_miss + expected_data_block_cache_miss,
+ block_cache_miss);
+ ASSERT_EQ(expected_index_block_cache_hit + expected_data_block_cache_hit,
+ block_cache_hit);
+ }
+
+ int64_t GetCacheBytesRead() { return block_cache_bytes_read; }
+
+ int64_t GetCacheBytesWrite() { return block_cache_bytes_write; }
+
+ private:
+ int64_t block_cache_miss = 0;
+ int64_t block_cache_hit = 0;
+ int64_t index_block_cache_miss = 0;
+ int64_t index_block_cache_hit = 0;
+ int64_t data_block_cache_miss = 0;
+ int64_t data_block_cache_hit = 0;
+ int64_t filter_block_cache_miss = 0;
+ int64_t filter_block_cache_hit = 0;
+ int64_t block_cache_bytes_read = 0;
+ int64_t block_cache_bytes_write = 0;
+};
+
+// Make sure, by default, index/filter blocks were pre-loaded (meaning we won't
+// use block cache to store them).
+TEST_P(BlockBasedTableTest, BlockCacheDisabledTest) {
+ Options options;
+ options.create_if_missing = true;
+ options.statistics = CreateDBStatistics();
+ BlockBasedTableOptions table_options = GetBlockBasedTableOptions();
+ table_options.block_cache = NewLRUCache(1024, 4);
+ table_options.filter_policy.reset(NewBloomFilterPolicy(10));
+ options.table_factory.reset(new BlockBasedTableFactory(table_options));
+ std::vector<std::string> keys;
+ stl_wrappers::KVMap kvmap;
+
+ TableConstructor c(BytewiseComparator(), true /* convert_to_internal_key_ */);
+ c.Add("key", "value");
+ const ImmutableOptions ioptions(options);
+ const MutableCFOptions moptions(options);
+ c.Finish(options, ioptions, moptions, table_options,
+ GetPlainInternalComparator(options.comparator), &keys, &kvmap);
+
+ // preloading filter/index blocks is enabled.
+ auto reader = dynamic_cast<BlockBasedTable*>(c.GetTableReader());
+ ASSERT_FALSE(reader->TEST_FilterBlockInCache());
+ ASSERT_FALSE(reader->TEST_IndexBlockInCache());
+
+ {
+ // nothing happens in the beginning
+ BlockCachePropertiesSnapshot props(options.statistics.get());
+ props.AssertIndexBlockStat(0, 0);
+ props.AssertFilterBlockStat(0, 0);
+ }
+
+ {
+ GetContext get_context(options.comparator, nullptr, nullptr, nullptr,
+ GetContext::kNotFound, Slice(), nullptr, nullptr,
+ nullptr, nullptr, true, nullptr, nullptr);
+ // a hack that just to trigger BlockBasedTable::GetFilter.
+ ASSERT_OK(reader->Get(ReadOptions(), "non-exist-key", &get_context,
+ moptions.prefix_extractor.get()));
+ BlockCachePropertiesSnapshot props(options.statistics.get());
+ props.AssertIndexBlockStat(0, 0);
+ props.AssertFilterBlockStat(0, 0);
+ }
+}
+
+// Due to the difficulities of the intersaction between statistics, this test
+// only tests the case when "index block is put to block cache"
+TEST_P(BlockBasedTableTest, FilterBlockInBlockCache) {
+ // -- Table construction
+ Options options;
+ options.create_if_missing = true;
+ options.statistics = CreateDBStatistics();
+
+ // Enable the cache for index/filter blocks
+ BlockBasedTableOptions table_options = GetBlockBasedTableOptions();
+ LRUCacheOptions co;
+ co.capacity = 2048;
+ co.num_shard_bits = 2;
+ co.metadata_charge_policy = kDontChargeCacheMetadata;
+ table_options.block_cache = NewLRUCache(co);
+ table_options.cache_index_and_filter_blocks = true;
+ options.table_factory.reset(new BlockBasedTableFactory(table_options));
+ std::vector<std::string> keys;
+ stl_wrappers::KVMap kvmap;
+
+ TableConstructor c(BytewiseComparator(), true /* convert_to_internal_key_ */);
+ c.Add("key", "value");
+ const ImmutableOptions ioptions(options);
+ const MutableCFOptions moptions(options);
+ c.Finish(options, ioptions, moptions, table_options,
+ GetPlainInternalComparator(options.comparator), &keys, &kvmap);
+ // preloading filter/index blocks is prohibited.
+ auto* reader = dynamic_cast<BlockBasedTable*>(c.GetTableReader());
+ ASSERT_FALSE(reader->TEST_FilterBlockInCache());
+ ASSERT_TRUE(reader->TEST_IndexBlockInCache());
+
+ // -- PART 1: Open with regular block cache.
+ // Since block_cache is disabled, no cache activities will be involved.
+ std::unique_ptr<InternalIterator> iter;
+
+ int64_t last_cache_bytes_read = 0;
+ // At first, no block will be accessed.
+ {
+ BlockCachePropertiesSnapshot props(options.statistics.get());
+ // index will be added to block cache.
+ props.AssertEqual(1, // index block miss
+ 0, 0, 0);
+ ASSERT_EQ(props.GetCacheBytesRead(), 0);
+ ASSERT_EQ(props.GetCacheBytesWrite(),
+ static_cast<int64_t>(table_options.block_cache->GetUsage()));
+ last_cache_bytes_read = props.GetCacheBytesRead();
+ }
+
+ // Only index block will be accessed
+ {
+ iter.reset(c.NewIterator(moptions.prefix_extractor.get()));
+ BlockCachePropertiesSnapshot props(options.statistics.get());
+ // NOTE: to help better highlight the "detla" of each ticker, I use
+ // <last_value> + <added_value> to indicate the increment of changed
+ // value; other numbers remain the same.
+ props.AssertEqual(1, 0 + 1, // index block hit
+ 0, 0);
+ // Cache hit, bytes read from cache should increase
+ ASSERT_GT(props.GetCacheBytesRead(), last_cache_bytes_read);
+ ASSERT_EQ(props.GetCacheBytesWrite(),
+ static_cast<int64_t>(table_options.block_cache->GetUsage()));
+ last_cache_bytes_read = props.GetCacheBytesRead();
+ }
+
+ // Only data block will be accessed
+ {
+ iter->SeekToFirst();
+ ASSERT_OK(iter->status());
+ BlockCachePropertiesSnapshot props(options.statistics.get());
+ props.AssertEqual(1, 1, 0 + 1, // data block miss
+ 0);
+ // Cache miss, Bytes read from cache should not change
+ ASSERT_EQ(props.GetCacheBytesRead(), last_cache_bytes_read);
+ ASSERT_EQ(props.GetCacheBytesWrite(),
+ static_cast<int64_t>(table_options.block_cache->GetUsage()));
+ last_cache_bytes_read = props.GetCacheBytesRead();
+ }
+
+ // Data block will be in cache
+ {
+ iter.reset(c.NewIterator(moptions.prefix_extractor.get()));
+ iter->SeekToFirst();
+ ASSERT_OK(iter->status());
+ BlockCachePropertiesSnapshot props(options.statistics.get());
+ props.AssertEqual(1, 1 + 1, /* index block hit */
+ 1, 0 + 1 /* data block hit */);
+ // Cache hit, bytes read from cache should increase
+ ASSERT_GT(props.GetCacheBytesRead(), last_cache_bytes_read);
+ ASSERT_EQ(props.GetCacheBytesWrite(),
+ static_cast<int64_t>(table_options.block_cache->GetUsage()));
+ }
+ // release the iterator so that the block cache can reset correctly.
+ iter.reset();
+
+ c.ResetTableReader();
+
+ // -- PART 2: Open with very small block cache
+ // In this test, no block will ever get hit since the block cache is
+ // too small to fit even one entry.
+ table_options.block_cache = NewLRUCache(1, 4);
+ options.statistics = CreateDBStatistics();
+ options.table_factory.reset(new BlockBasedTableFactory(table_options));
+ const ImmutableOptions ioptions2(options);
+ const MutableCFOptions moptions2(options);
+ ASSERT_OK(c.Reopen(ioptions2, moptions2));
+ {
+ BlockCachePropertiesSnapshot props(options.statistics.get());
+ props.AssertEqual(1, // index block miss
+ 0, 0, 0);
+ // Cache miss, Bytes read from cache should not change
+ ASSERT_EQ(props.GetCacheBytesRead(), 0);
+ }
+
+ {
+ // Both index and data block get accessed.
+ // It first cache index block then data block. But since the cache size
+ // is only 1, index block will be purged after data block is inserted.
+ iter.reset(c.NewIterator(moptions2.prefix_extractor.get()));
+ BlockCachePropertiesSnapshot props(options.statistics.get());
+ props.AssertEqual(1 + 1, // index block miss
+ 0, 0, // data block miss
+ 0);
+ // Cache hit, bytes read from cache should increase
+ ASSERT_EQ(props.GetCacheBytesRead(), 0);
+ }
+
+ {
+ // SeekToFirst() accesses data block. With similar reason, we expect data
+ // block's cache miss.
+ iter->SeekToFirst();
+ ASSERT_OK(iter->status());
+ BlockCachePropertiesSnapshot props(options.statistics.get());
+ props.AssertEqual(2, 0, 0 + 1, // data block miss
+ 0);
+ // Cache miss, Bytes read from cache should not change
+ ASSERT_EQ(props.GetCacheBytesRead(), 0);
+ }
+ iter.reset();
+ c.ResetTableReader();
+
+ // -- PART 3: Open table with bloom filter enabled but not in SST file
+ table_options.block_cache = NewLRUCache(4096, 4);
+ table_options.cache_index_and_filter_blocks = false;
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+
+ TableConstructor c3(BytewiseComparator());
+ std::string user_key = "k01";
+ InternalKey internal_key(user_key, 0, kTypeValue);
+ c3.Add(internal_key.Encode().ToString(), "hello");
+ ImmutableOptions ioptions3(options);
+ MutableCFOptions moptions3(options);
+ // Generate table without filter policy
+ c3.Finish(options, ioptions3, moptions3, table_options,
+ GetPlainInternalComparator(options.comparator), &keys, &kvmap);
+ c3.ResetTableReader();
+
+ // Open table with filter policy
+ table_options.filter_policy.reset(NewBloomFilterPolicy(1));
+ options.table_factory.reset(new BlockBasedTableFactory(table_options));
+ options.statistics = CreateDBStatistics();
+ ImmutableOptions ioptions4(options);
+ MutableCFOptions moptions4(options);
+ ASSERT_OK(c3.Reopen(ioptions4, moptions4));
+ reader = dynamic_cast<BlockBasedTable*>(c3.GetTableReader());
+ ASSERT_FALSE(reader->TEST_FilterBlockInCache());
+ PinnableSlice value;
+ GetContext get_context(options.comparator, nullptr, nullptr, nullptr,
+ GetContext::kNotFound, user_key, &value, nullptr,
+ nullptr, nullptr, true, nullptr, nullptr);
+ ASSERT_OK(reader->Get(ReadOptions(), internal_key.Encode(), &get_context,
+ moptions4.prefix_extractor.get()));
+ ASSERT_STREQ(value.data(), "hello");
+ BlockCachePropertiesSnapshot props(options.statistics.get());
+ props.AssertFilterBlockStat(0, 0);
+ c3.ResetTableReader();
+}
+
+void ValidateBlockSizeDeviation(int value, int expected) {
+ BlockBasedTableOptions table_options;
+ table_options.block_size_deviation = value;
+ BlockBasedTableFactory* factory = new BlockBasedTableFactory(table_options);
+
+ const BlockBasedTableOptions* normalized_table_options =
+ factory->GetOptions<BlockBasedTableOptions>();
+ ASSERT_EQ(normalized_table_options->block_size_deviation, expected);
+
+ delete factory;
+}
+
+void ValidateBlockRestartInterval(int value, int expected) {
+ BlockBasedTableOptions table_options;
+ table_options.block_restart_interval = value;
+ BlockBasedTableFactory* factory = new BlockBasedTableFactory(table_options);
+
+ const BlockBasedTableOptions* normalized_table_options =
+ factory->GetOptions<BlockBasedTableOptions>();
+ ASSERT_EQ(normalized_table_options->block_restart_interval, expected);
+
+ delete factory;
+}
+
+TEST_P(BlockBasedTableTest, InvalidOptions) {
+ // invalid values for block_size_deviation (<0 or >100) are silently set to 0
+ ValidateBlockSizeDeviation(-10, 0);
+ ValidateBlockSizeDeviation(-1, 0);
+ ValidateBlockSizeDeviation(0, 0);
+ ValidateBlockSizeDeviation(1, 1);
+ ValidateBlockSizeDeviation(99, 99);
+ ValidateBlockSizeDeviation(100, 100);
+ ValidateBlockSizeDeviation(101, 0);
+ ValidateBlockSizeDeviation(1000, 0);
+
+ // invalid values for block_restart_interval (<1) are silently set to 1
+ ValidateBlockRestartInterval(-10, 1);
+ ValidateBlockRestartInterval(-1, 1);
+ ValidateBlockRestartInterval(0, 1);
+ ValidateBlockRestartInterval(1, 1);
+ ValidateBlockRestartInterval(2, 2);
+ ValidateBlockRestartInterval(1000, 1000);
+}
+
+TEST_P(BlockBasedTableTest, BlockReadCountTest) {
+ // bloom_filter_type = 1 -- full filter using use_block_based_builder=false
+ // bloom_filter_type = 2 -- full filter using use_block_based_builder=true
+ // because of API change to hide block-based filter
+ for (int bloom_filter_type = 1; bloom_filter_type <= 2; ++bloom_filter_type) {
+ for (int index_and_filter_in_cache = 0; index_and_filter_in_cache < 2;
+ ++index_and_filter_in_cache) {
+ Options options;
+ options.create_if_missing = true;
+
+ BlockBasedTableOptions table_options = GetBlockBasedTableOptions();
+ table_options.block_cache = NewLRUCache(1, 0);
+ table_options.cache_index_and_filter_blocks = index_and_filter_in_cache;
+ table_options.filter_policy.reset(
+ NewBloomFilterPolicy(10, bloom_filter_type == 2));
+ options.table_factory.reset(new BlockBasedTableFactory(table_options));
+ std::vector<std::string> keys;
+ stl_wrappers::KVMap kvmap;
+
+ TableConstructor c(BytewiseComparator());
+ std::string user_key = "k04";
+ InternalKey internal_key(user_key, 0, kTypeValue);
+ std::string encoded_key = internal_key.Encode().ToString();
+ c.Add(encoded_key, "hello");
+ ImmutableOptions ioptions(options);
+ MutableCFOptions moptions(options);
+ // Generate table with filter policy
+ c.Finish(options, ioptions, moptions, table_options,
+ GetPlainInternalComparator(options.comparator), &keys, &kvmap);
+ auto reader = c.GetTableReader();
+ PinnableSlice value;
+ {
+ GetContext get_context(options.comparator, nullptr, nullptr, nullptr,
+ GetContext::kNotFound, user_key, &value, nullptr,
+ nullptr, nullptr, true, nullptr, nullptr);
+ get_perf_context()->Reset();
+ ASSERT_OK(reader->Get(ReadOptions(), encoded_key, &get_context,
+ moptions.prefix_extractor.get()));
+ if (index_and_filter_in_cache) {
+ // data, index and filter block
+ ASSERT_EQ(get_perf_context()->block_read_count, 3);
+ ASSERT_EQ(get_perf_context()->index_block_read_count, 1);
+ ASSERT_EQ(get_perf_context()->filter_block_read_count, 1);
+ } else {
+ // just the data block
+ ASSERT_EQ(get_perf_context()->block_read_count, 1);
+ }
+ ASSERT_EQ(get_context.State(), GetContext::kFound);
+ ASSERT_STREQ(value.data(), "hello");
+ }
+
+ // Get non-existing key
+ user_key = "does-not-exist";
+ internal_key = InternalKey(user_key, 0, kTypeValue);
+ encoded_key = internal_key.Encode().ToString();
+
+ value.Reset();
+ {
+ GetContext get_context(options.comparator, nullptr, nullptr, nullptr,
+ GetContext::kNotFound, user_key, &value, nullptr,
+ nullptr, nullptr, true, nullptr, nullptr);
+ get_perf_context()->Reset();
+ ASSERT_OK(reader->Get(ReadOptions(), encoded_key, &get_context,
+ moptions.prefix_extractor.get()));
+ ASSERT_EQ(get_context.State(), GetContext::kNotFound);
+ }
+
+ if (index_and_filter_in_cache) {
+ if (bloom_filter_type == 0) {
+ // with block-based, we read index and then the filter
+ ASSERT_EQ(get_perf_context()->block_read_count, 2);
+ ASSERT_EQ(get_perf_context()->index_block_read_count, 1);
+ ASSERT_EQ(get_perf_context()->filter_block_read_count, 1);
+ } else {
+ // with full-filter, we read filter first and then we stop
+ ASSERT_EQ(get_perf_context()->block_read_count, 1);
+ ASSERT_EQ(get_perf_context()->filter_block_read_count, 1);
+ }
+ } else {
+ // filter is already in memory and it figures out that the key doesn't
+ // exist
+ ASSERT_EQ(get_perf_context()->block_read_count, 0);
+ }
+ }
+ }
+}
+
+TEST_P(BlockBasedTableTest, BlockCacheLeak) {
+ // Check that when we reopen a table we don't lose access to blocks already
+ // in the cache. This test checks whether the Table actually makes use of the
+ // unique ID from the file.
+
+ Options opt;
+ std::unique_ptr<InternalKeyComparator> ikc;
+ ikc.reset(new test::PlainInternalKeyComparator(opt.comparator));
+ opt.compression = kNoCompression;
+ BlockBasedTableOptions table_options = GetBlockBasedTableOptions();
+ table_options.block_size = 1024;
+ // big enough so we don't ever lose cached values.
+ table_options.block_cache = NewLRUCache(16 * 1024 * 1024, 4);
+ opt.table_factory.reset(NewBlockBasedTableFactory(table_options));
+
+ TableConstructor c(BytewiseComparator(), true /* convert_to_internal_key_ */);
+ c.Add("k01", "hello");
+ c.Add("k02", "hello2");
+ c.Add("k03", std::string(10000, 'x'));
+ c.Add("k04", std::string(200000, 'x'));
+ c.Add("k05", std::string(300000, 'x'));
+ c.Add("k06", "hello3");
+ c.Add("k07", std::string(100000, 'x'));
+ std::vector<std::string> keys;
+ stl_wrappers::KVMap kvmap;
+ const ImmutableOptions ioptions(opt);
+ const MutableCFOptions moptions(opt);
+ c.Finish(opt, ioptions, moptions, table_options, *ikc, &keys, &kvmap);
+
+ std::unique_ptr<InternalIterator> iter(
+ c.NewIterator(moptions.prefix_extractor.get()));
+ iter->SeekToFirst();
+ while (iter->Valid()) {
+ iter->key();
+ iter->value();
+ iter->Next();
+ }
+ ASSERT_OK(iter->status());
+ iter.reset();
+
+ const ImmutableOptions ioptions1(opt);
+ const MutableCFOptions moptions1(opt);
+ ASSERT_OK(c.Reopen(ioptions1, moptions1));
+ auto table_reader = dynamic_cast<BlockBasedTable*>(c.GetTableReader());
+ for (const std::string& key : keys) {
+ InternalKey ikey(key, kMaxSequenceNumber, kTypeValue);
+ ASSERT_TRUE(table_reader->TEST_KeyInCache(ReadOptions(), ikey.Encode()));
+ }
+ c.ResetTableReader();
+
+ // rerun with different block cache
+ table_options.block_cache = NewLRUCache(16 * 1024 * 1024, 4);
+ opt.table_factory.reset(NewBlockBasedTableFactory(table_options));
+ const ImmutableOptions ioptions2(opt);
+ const MutableCFOptions moptions2(opt);
+ ASSERT_OK(c.Reopen(ioptions2, moptions2));
+ table_reader = dynamic_cast<BlockBasedTable*>(c.GetTableReader());
+ for (const std::string& key : keys) {
+ InternalKey ikey(key, kMaxSequenceNumber, kTypeValue);
+ ASSERT_TRUE(!table_reader->TEST_KeyInCache(ReadOptions(), ikey.Encode()));
+ }
+ c.ResetTableReader();
+}
+
+TEST_P(BlockBasedTableTest, MemoryAllocator) {
+ auto default_memory_allocator = std::make_shared<DefaultMemoryAllocator>();
+ auto custom_memory_allocator =
+ std::make_shared<CountedMemoryAllocator>(default_memory_allocator);
+ {
+ Options opt;
+ std::unique_ptr<InternalKeyComparator> ikc;
+ ikc.reset(new test::PlainInternalKeyComparator(opt.comparator));
+ opt.compression = kNoCompression;
+ BlockBasedTableOptions table_options;
+ table_options.block_size = 1024;
+ LRUCacheOptions lruOptions;
+ lruOptions.memory_allocator = custom_memory_allocator;
+ lruOptions.capacity = 16 * 1024 * 1024;
+ lruOptions.num_shard_bits = 4;
+ table_options.block_cache = NewLRUCache(std::move(lruOptions));
+ opt.table_factory.reset(NewBlockBasedTableFactory(table_options));
+
+ TableConstructor c(BytewiseComparator(),
+ true /* convert_to_internal_key_ */);
+ c.Add("k01", "hello");
+ c.Add("k02", "hello2");
+ c.Add("k03", std::string(10000, 'x'));
+ c.Add("k04", std::string(200000, 'x'));
+ c.Add("k05", std::string(300000, 'x'));
+ c.Add("k06", "hello3");
+ c.Add("k07", std::string(100000, 'x'));
+ std::vector<std::string> keys;
+ stl_wrappers::KVMap kvmap;
+ const ImmutableOptions ioptions(opt);
+ const MutableCFOptions moptions(opt);
+ c.Finish(opt, ioptions, moptions, table_options, *ikc, &keys, &kvmap);
+
+ std::unique_ptr<InternalIterator> iter(
+ c.NewIterator(moptions.prefix_extractor.get()));
+ iter->SeekToFirst();
+ while (iter->Valid()) {
+ iter->key();
+ iter->value();
+ iter->Next();
+ }
+ ASSERT_OK(iter->status());
+ }
+
+ // out of scope, block cache should have been deleted, all allocations
+ // deallocated
+ EXPECT_EQ(custom_memory_allocator->GetNumAllocations(),
+ custom_memory_allocator->GetNumDeallocations());
+ // make sure that allocations actually happened through the cache allocator
+ EXPECT_GT(custom_memory_allocator->GetNumAllocations(), 0);
+}
+
+// Test the file checksum of block based table
+TEST_P(BlockBasedTableTest, NoFileChecksum) {
+ Options options;
+ ImmutableOptions ioptions(options);
+ MutableCFOptions moptions(options);
+ BlockBasedTableOptions table_options = GetBlockBasedTableOptions();
+ std::unique_ptr<InternalKeyComparator> comparator(
+ new InternalKeyComparator(BytewiseComparator()));
+ int level = 0;
+ IntTblPropCollectorFactories int_tbl_prop_collector_factories;
+ std::string column_family_name;
+
+ FileChecksumTestHelper f(true);
+ f.CreateWritableFile();
+ std::unique_ptr<TableBuilder> builder;
+ builder.reset(ioptions.table_factory->NewTableBuilder(
+ TableBuilderOptions(ioptions, moptions, *comparator,
+ &int_tbl_prop_collector_factories,
+ options.compression, options.compression_opts,
+ kUnknownColumnFamily, column_family_name, level),
+ f.GetFileWriter()));
+ ASSERT_OK(f.ResetTableBuilder(std::move(builder)));
+ f.AddKVtoKVMap(1000);
+ ASSERT_OK(f.WriteKVAndFlushTable());
+ ASSERT_STREQ(f.GetFileChecksumFuncName(), kUnknownFileChecksumFuncName);
+ ASSERT_STREQ(f.GetFileChecksum().c_str(), kUnknownFileChecksum);
+}
+
+TEST_P(BlockBasedTableTest, Crc32cFileChecksum) {
+ FileChecksumGenCrc32cFactory* file_checksum_gen_factory =
+ new FileChecksumGenCrc32cFactory();
+ Options options;
+ options.file_checksum_gen_factory.reset(file_checksum_gen_factory);
+ ImmutableOptions ioptions(options);
+ MutableCFOptions moptions(options);
+ BlockBasedTableOptions table_options = GetBlockBasedTableOptions();
+ std::unique_ptr<InternalKeyComparator> comparator(
+ new InternalKeyComparator(BytewiseComparator()));
+ int level = 0;
+ IntTblPropCollectorFactories int_tbl_prop_collector_factories;
+ std::string column_family_name;
+
+ FileChecksumGenContext gen_context;
+ gen_context.file_name = "db/tmp";
+ std::unique_ptr<FileChecksumGenerator> checksum_crc32c_gen1 =
+ options.file_checksum_gen_factory->CreateFileChecksumGenerator(
+ gen_context);
+ FileChecksumTestHelper f(true);
+ f.CreateWritableFile();
+ f.SetFileChecksumGenerator(checksum_crc32c_gen1.release());
+ std::unique_ptr<TableBuilder> builder;
+ builder.reset(ioptions.table_factory->NewTableBuilder(
+ TableBuilderOptions(ioptions, moptions, *comparator,
+ &int_tbl_prop_collector_factories,
+ options.compression, options.compression_opts,
+ kUnknownColumnFamily, column_family_name, level),
+ f.GetFileWriter()));
+ ASSERT_OK(f.ResetTableBuilder(std::move(builder)));
+ f.AddKVtoKVMap(1000);
+ ASSERT_OK(f.WriteKVAndFlushTable());
+ ASSERT_STREQ(f.GetFileChecksumFuncName(), "FileChecksumCrc32c");
+
+ std::unique_ptr<FileChecksumGenerator> checksum_crc32c_gen2 =
+ options.file_checksum_gen_factory->CreateFileChecksumGenerator(
+ gen_context);
+ std::string checksum;
+ ASSERT_OK(f.CalculateFileChecksum(checksum_crc32c_gen2.get(), &checksum));
+ ASSERT_STREQ(f.GetFileChecksum().c_str(), checksum.c_str());
+
+ // Unit test the generator itself for schema stability
+ std::unique_ptr<FileChecksumGenerator> checksum_crc32c_gen3 =
+ options.file_checksum_gen_factory->CreateFileChecksumGenerator(
+ gen_context);
+ const char data[] = "here is some data";
+ checksum_crc32c_gen3->Update(data, sizeof(data));
+ checksum_crc32c_gen3->Finalize();
+ checksum = checksum_crc32c_gen3->GetChecksum();
+ ASSERT_STREQ(checksum.c_str(), "\345\245\277\110");
+}
+
+// Plain table is not supported in ROCKSDB_LITE
+#ifndef ROCKSDB_LITE
+TEST_F(PlainTableTest, BasicPlainTableProperties) {
+ PlainTableOptions plain_table_options;
+ plain_table_options.user_key_len = 8;
+ plain_table_options.bloom_bits_per_key = 8;
+ plain_table_options.hash_table_ratio = 0;
+
+ PlainTableFactory factory(plain_table_options);
+ std::unique_ptr<FSWritableFile> sink(new test::StringSink());
+ std::unique_ptr<WritableFileWriter> file_writer(new WritableFileWriter(
+ std::move(sink), "" /* don't care */, FileOptions()));
+ Options options;
+ const ImmutableOptions ioptions(options);
+ const MutableCFOptions moptions(options);
+ InternalKeyComparator ikc(options.comparator);
+ IntTblPropCollectorFactories int_tbl_prop_collector_factories;
+ std::string column_family_name;
+ int unknown_level = -1;
+ std::unique_ptr<TableBuilder> builder(factory.NewTableBuilder(
+ TableBuilderOptions(ioptions, moptions, ikc,
+ &int_tbl_prop_collector_factories, kNoCompression,
+ CompressionOptions(), kUnknownColumnFamily,
+ column_family_name, unknown_level),
+ file_writer.get()));
+
+ for (char c = 'a'; c <= 'z'; ++c) {
+ std::string key(8, c);
+ key.append("\1 "); // PlainTable expects internal key structure
+ std::string value(28, c + 42);
+ builder->Add(key, value);
+ }
+ ASSERT_OK(builder->Finish());
+ ASSERT_OK(file_writer->Flush());
+
+ test::StringSink* ss =
+ static_cast<test::StringSink*>(file_writer->writable_file());
+ std::unique_ptr<FSRandomAccessFile> source(
+ new test::StringSource(ss->contents(), 72242, true));
+ std::unique_ptr<RandomAccessFileReader> file_reader(
+ new RandomAccessFileReader(std::move(source), "test"));
+
+ std::unique_ptr<TableProperties> props;
+ auto s = ReadTableProperties(file_reader.get(), ss->contents().size(),
+ kPlainTableMagicNumber, ioptions, &props);
+ ASSERT_OK(s);
+
+ ASSERT_EQ(0ul, props->index_size);
+ ASSERT_EQ(0ul, props->filter_size);
+ ASSERT_EQ(16ul * 26, props->raw_key_size);
+ ASSERT_EQ(28ul * 26, props->raw_value_size);
+ ASSERT_EQ(26ul, props->num_entries);
+ ASSERT_EQ(1ul, props->num_data_blocks);
+}
+
+TEST_F(PlainTableTest, NoFileChecksum) {
+ PlainTableOptions plain_table_options;
+ plain_table_options.user_key_len = 20;
+ plain_table_options.bloom_bits_per_key = 8;
+ plain_table_options.hash_table_ratio = 0;
+ PlainTableFactory factory(plain_table_options);
+
+ Options options;
+ const ImmutableOptions ioptions(options);
+ const MutableCFOptions moptions(options);
+ InternalKeyComparator ikc(options.comparator);
+ IntTblPropCollectorFactories int_tbl_prop_collector_factories;
+ std::string column_family_name;
+ int unknown_level = -1;
+ FileChecksumTestHelper f(true);
+ f.CreateWritableFile();
+
+ std::unique_ptr<TableBuilder> builder(factory.NewTableBuilder(
+ TableBuilderOptions(ioptions, moptions, ikc,
+ &int_tbl_prop_collector_factories, kNoCompression,
+ CompressionOptions(), kUnknownColumnFamily,
+ column_family_name, unknown_level),
+ f.GetFileWriter()));
+ ASSERT_OK(f.ResetTableBuilder(std::move(builder)));
+ f.AddKVtoKVMap(1000);
+ ASSERT_OK(f.WriteKVAndFlushTable());
+ ASSERT_STREQ(f.GetFileChecksumFuncName(), kUnknownFileChecksumFuncName);
+ EXPECT_EQ(f.GetFileChecksum(), kUnknownFileChecksum);
+}
+
+TEST_F(PlainTableTest, Crc32cFileChecksum) {
+ PlainTableOptions plain_table_options;
+ plain_table_options.user_key_len = 20;
+ plain_table_options.bloom_bits_per_key = 8;
+ plain_table_options.hash_table_ratio = 0;
+ PlainTableFactory factory(plain_table_options);
+
+ FileChecksumGenCrc32cFactory* file_checksum_gen_factory =
+ new FileChecksumGenCrc32cFactory();
+ Options options;
+ options.file_checksum_gen_factory.reset(file_checksum_gen_factory);
+ const ImmutableOptions ioptions(options);
+ const MutableCFOptions moptions(options);
+ InternalKeyComparator ikc(options.comparator);
+ IntTblPropCollectorFactories int_tbl_prop_collector_factories;
+ std::string column_family_name;
+ int unknown_level = -1;
+
+ FileChecksumGenContext gen_context;
+ gen_context.file_name = "db/tmp";
+ std::unique_ptr<FileChecksumGenerator> checksum_crc32c_gen1 =
+ options.file_checksum_gen_factory->CreateFileChecksumGenerator(
+ gen_context);
+ FileChecksumTestHelper f(true);
+ f.CreateWritableFile();
+ f.SetFileChecksumGenerator(checksum_crc32c_gen1.release());
+
+ std::unique_ptr<TableBuilder> builder(factory.NewTableBuilder(
+ TableBuilderOptions(ioptions, moptions, ikc,
+ &int_tbl_prop_collector_factories, kNoCompression,
+ CompressionOptions(), kUnknownColumnFamily,
+ column_family_name, unknown_level),
+ f.GetFileWriter()));
+ ASSERT_OK(f.ResetTableBuilder(std::move(builder)));
+ f.AddKVtoKVMap(1000);
+ ASSERT_OK(f.WriteKVAndFlushTable());
+ ASSERT_STREQ(f.GetFileChecksumFuncName(), "FileChecksumCrc32c");
+
+ std::unique_ptr<FileChecksumGenerator> checksum_crc32c_gen2 =
+ options.file_checksum_gen_factory->CreateFileChecksumGenerator(
+ gen_context);
+ std::string checksum;
+ ASSERT_OK(f.CalculateFileChecksum(checksum_crc32c_gen2.get(), &checksum));
+ EXPECT_STREQ(f.GetFileChecksum().c_str(), checksum.c_str());
+}
+
+#endif // !ROCKSDB_LITE
+
+TEST_F(GeneralTableTest, ApproximateOffsetOfPlain) {
+ TableConstructor c(BytewiseComparator(), true /* convert_to_internal_key_ */);
+ c.Add("k01", "hello");
+ c.Add("k02", "hello2");
+ c.Add("k03", std::string(10000, 'x'));
+ c.Add("k04", std::string(200000, 'x'));
+ c.Add("k05", std::string(300000, 'x'));
+ c.Add("k06", "hello3");
+ c.Add("k07", std::string(100000, 'x'));
+ std::vector<std::string> keys;
+ stl_wrappers::KVMap kvmap;
+ Options options;
+ options.db_host_id = "";
+ test::PlainInternalKeyComparator internal_comparator(options.comparator);
+ options.compression = kNoCompression;
+ BlockBasedTableOptions table_options;
+ table_options.block_size = 1024;
+ const ImmutableOptions ioptions(options);
+ const MutableCFOptions moptions(options);
+ c.Finish(options, ioptions, moptions, table_options, internal_comparator,
+ &keys, &kvmap);
+
+ ASSERT_TRUE(Between(c.ApproximateOffsetOf("abc"), 0, 0));
+ ASSERT_TRUE(Between(c.ApproximateOffsetOf("k01"), 0, 0));
+ ASSERT_TRUE(Between(c.ApproximateOffsetOf("k01a"), 0, 0));
+ ASSERT_TRUE(Between(c.ApproximateOffsetOf("k02"), 0, 0));
+ ASSERT_TRUE(Between(c.ApproximateOffsetOf("k03"), 0, 0));
+ ASSERT_TRUE(Between(c.ApproximateOffsetOf("k04"), 10000, 11000));
+ // 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));
+ ASSERT_TRUE(Between(c.ApproximateOffsetOf("xyz"), 610000, 612000));
+ c.ResetTableReader();
+}
+
+static void DoCompressionTest(CompressionType comp) {
+ Random rnd(301);
+ 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));
+ c.Add("k03", "hello3");
+ c.Add("k04", test::CompressibleString(&rnd, 0.25, 10000, &tmp));
+ std::vector<std::string> keys;
+ stl_wrappers::KVMap kvmap;
+ Options options;
+ test::PlainInternalKeyComparator ikc(options.comparator);
+ options.compression = comp;
+ BlockBasedTableOptions table_options;
+ table_options.block_size = 1024;
+ const ImmutableOptions ioptions(options);
+ const MutableCFOptions moptions(options);
+ c.Finish(options, ioptions, moptions, table_options, ikc, &keys, &kvmap);
+
+ ASSERT_TRUE(Between(c.ApproximateOffsetOf("abc"), 0, 0));
+ ASSERT_TRUE(Between(c.ApproximateOffsetOf("k01"), 0, 0));
+ ASSERT_TRUE(Between(c.ApproximateOffsetOf("k02"), 0, 0));
+ ASSERT_TRUE(Between(c.ApproximateOffsetOf("k03"), 2000, 3525));
+ ASSERT_TRUE(Between(c.ApproximateOffsetOf("k04"), 2000, 3525));
+ ASSERT_TRUE(Between(c.ApproximateOffsetOf("xyz"), 4000, 7075));
+ c.ResetTableReader();
+}
+
+TEST_F(GeneralTableTest, ApproximateOffsetOfCompressed) {
+ std::vector<CompressionType> compression_state;
+ if (!Snappy_Supported()) {
+ fprintf(stderr, "skipping snappy compression tests\n");
+ } else {
+ compression_state.push_back(kSnappyCompression);
+ }
+
+ if (!Zlib_Supported()) {
+ fprintf(stderr, "skipping zlib compression tests\n");
+ } else {
+ compression_state.push_back(kZlibCompression);
+ }
+
+ // TODO(kailiu) DoCompressionTest() doesn't work with BZip2.
+ /*
+ if (!BZip2_Supported()) {
+ fprintf(stderr, "skipping bzip2 compression tests\n");
+ } else {
+ compression_state.push_back(kBZip2Compression);
+ }
+ */
+
+ if (!LZ4_Supported()) {
+ fprintf(stderr, "skipping lz4 and lz4hc compression tests\n");
+ } else {
+ compression_state.push_back(kLZ4Compression);
+ compression_state.push_back(kLZ4HCCompression);
+ }
+
+ if (!XPRESS_Supported()) {
+ fprintf(stderr, "skipping xpress and xpress compression tests\n");
+ } else {
+ compression_state.push_back(kXpressCompression);
+ }
+
+ for (auto state : compression_state) {
+ DoCompressionTest(state);
+ }
+}
+
+TEST_F(GeneralTableTest, ApproximateKeyAnchors) {
+ Random rnd(301);
+ TableConstructor c(BytewiseComparator(), true /* convert_to_internal_key_ */);
+ std::string tmp;
+ for (int i = 1000; i < 9000; i++) {
+ c.Add(std::to_string(i), rnd.RandomString(2000));
+ }
+ std::vector<std::string> keys;
+ stl_wrappers::KVMap kvmap;
+ Options options;
+ InternalKeyComparator ikc(options.comparator);
+ options.compression = kNoCompression;
+ BlockBasedTableOptions table_options;
+ table_options.block_size = 4096;
+ const ImmutableOptions ioptions(options);
+ const MutableCFOptions moptions(options);
+ c.Finish(options, ioptions, moptions, table_options, ikc, &keys, &kvmap);
+
+ std::vector<TableReader::Anchor> anchors;
+ ASSERT_OK(c.GetTableReader()->ApproximateKeyAnchors(ReadOptions(), anchors));
+ // The target is 128 anchors. But in reality it can be slightly more or fewer.
+ ASSERT_GT(anchors.size(), 120);
+ ASSERT_LT(anchors.size(), 140);
+
+ // We have around 8000 keys. With 128 anchors, in average 62.5 keys per
+ // anchor. Here we take a rough range and estimate the distance between
+ // anchors is between 50 and 100.
+ // Total data size is about 18,000,000, so each anchor range is about
+ // 140,625. We also take a rough range.
+ int prev_num = 1000;
+ // Non-last anchor
+ for (size_t i = 0; i + 1 < anchors.size(); i++) {
+ auto& anchor = anchors[i];
+ ASSERT_GT(anchor.range_size, 100000);
+ ASSERT_LT(anchor.range_size, 200000);
+
+ // Key might be shortened, so fill 0 in the end if it is the case.
+ std::string key_cpy = anchor.user_key;
+ key_cpy.append(4 - key_cpy.size(), '0');
+ int num = std::stoi(key_cpy);
+ ASSERT_GT(num - prev_num, 50);
+ ASSERT_LT(num - prev_num, 100);
+ prev_num = num;
+ }
+
+ ASSERT_EQ("8999", anchors.back().user_key);
+ ASSERT_LT(anchors.back().range_size, 200000);
+
+ c.ResetTableReader();
+}
+
+#if !defined(ROCKSDB_VALGRIND_RUN) || defined(ROCKSDB_FULL_VALGRIND_RUN)
+TEST_P(ParameterizedHarnessTest, RandomizedHarnessTest) {
+ Random rnd(test::RandomSeed() + 5);
+ for (int num_entries = 0; num_entries < 2000;
+ num_entries += (num_entries < 50 ? 1 : 200)) {
+ for (int e = 0; e < num_entries; e++) {
+ Add(test::RandomKey(&rnd, rnd.Skewed(4)),
+ rnd.RandomString(rnd.Skewed(5)));
+ }
+ Test(&rnd);
+ }
+}
+
+#ifndef ROCKSDB_LITE
+TEST_F(DBHarnessTest, RandomizedLongDB) {
+ Random rnd(test::RandomSeed());
+ int num_entries = 100000;
+ for (int e = 0; e < num_entries; e++) {
+ std::string v;
+ Add(test::RandomKey(&rnd, rnd.Skewed(4)), rnd.RandomString(rnd.Skewed(5)));
+ }
+ Test(&rnd);
+
+ // We must have created enough data to force merging
+ int files = 0;
+ for (int level = 0; level < db()->NumberLevels(); level++) {
+ std::string value;
+ char name[100];
+ snprintf(name, sizeof(name), "rocksdb.num-files-at-level%d", level);
+ ASSERT_TRUE(db()->GetProperty(name, &value));
+ files += atoi(value.c_str());
+ }
+ ASSERT_GT(files, 0);
+}
+#endif // ROCKSDB_LITE
+#endif // !defined(ROCKSDB_VALGRIND_RUN) || defined(ROCKSDB_FULL_VALGRIND_RUN)
+
+class MemTableTest : public testing::Test {
+ public:
+ MemTableTest() {
+ InternalKeyComparator cmp(BytewiseComparator());
+ auto table_factory = std::make_shared<SkipListFactory>();
+ options_.memtable_factory = table_factory;
+ ImmutableOptions ioptions(options_);
+ wb_ = new WriteBufferManager(options_.db_write_buffer_size);
+ memtable_ = new MemTable(cmp, ioptions, MutableCFOptions(options_), wb_,
+ kMaxSequenceNumber, 0 /* column_family_id */);
+ memtable_->Ref();
+ }
+
+ ~MemTableTest() {
+ delete memtable_->Unref();
+ delete wb_;
+ }
+
+ MemTable* GetMemTable() { return memtable_; }
+
+ private:
+ MemTable* memtable_;
+ Options options_;
+ WriteBufferManager* wb_;
+};
+
+TEST_F(MemTableTest, Simple) {
+ WriteBatch batch;
+ WriteBatchInternal::SetSequence(&batch, 100);
+ ASSERT_OK(batch.Put(std::string("k1"), std::string("v1")));
+ ASSERT_OK(batch.Put(std::string("k2"), std::string("v2")));
+ ASSERT_OK(batch.Put(std::string("k3"), std::string("v3")));
+ ASSERT_OK(batch.Put(std::string("largekey"), std::string("vlarge")));
+ ASSERT_OK(batch.DeleteRange(std::string("chi"), std::string("xigua")));
+ ASSERT_OK(batch.DeleteRange(std::string("begin"), std::string("end")));
+ ColumnFamilyMemTablesDefault cf_mems_default(GetMemTable());
+ ASSERT_TRUE(
+ WriteBatchInternal::InsertInto(&batch, &cf_mems_default, nullptr, nullptr)
+ .ok());
+
+ for (int i = 0; i < 2; ++i) {
+ Arena arena;
+ ScopedArenaIterator arena_iter_guard;
+ std::unique_ptr<InternalIterator> iter_guard;
+ InternalIterator* iter;
+ if (i == 0) {
+ iter = GetMemTable()->NewIterator(ReadOptions(), &arena);
+ arena_iter_guard.set(iter);
+ } else {
+ iter = GetMemTable()->NewRangeTombstoneIterator(
+ ReadOptions(), kMaxSequenceNumber /* read_seq */,
+ false /* immutable_memtable */);
+ iter_guard.reset(iter);
+ }
+ if (iter == nullptr) {
+ continue;
+ }
+ iter->SeekToFirst();
+ while (iter->Valid()) {
+ fprintf(stderr, "key: '%s' -> '%s'\n", iter->key().ToString().c_str(),
+ iter->value().ToString().c_str());
+ iter->Next();
+ }
+ }
+}
+
+// Test the empty key
+TEST_P(ParameterizedHarnessTest, SimpleEmptyKey) {
+ Random rnd(test::RandomSeed() + 1);
+ Add("", "v");
+ Test(&rnd);
+}
+
+TEST_P(ParameterizedHarnessTest, SimpleSingle) {
+ Random rnd(test::RandomSeed() + 2);
+ Add("abc", "v");
+ Test(&rnd);
+}
+
+TEST_P(ParameterizedHarnessTest, SimpleMulti) {
+ Random rnd(test::RandomSeed() + 3);
+ Add("abc", "v");
+ Add("abcd", "v");
+ Add("ac", "v2");
+ Test(&rnd);
+}
+
+TEST_P(ParameterizedHarnessTest, SimpleSpecialKey) {
+ Random rnd(test::RandomSeed() + 4);
+ Add("\xff\xff", "v3");
+ Test(&rnd);
+}
+
+TEST(TableTest, FooterTests) {
+ Random* r = Random::GetTLSInstance();
+ uint64_t data_size = (uint64_t{1} << r->Uniform(40)) + r->Uniform(100);
+ uint64_t index_size = r->Uniform(1000000000);
+ uint64_t metaindex_size = r->Uniform(1000000);
+ // 5 == block trailer size
+ BlockHandle index(data_size + 5, index_size);
+ BlockHandle meta_index(data_size + index_size + 2 * 5, metaindex_size);
+ uint64_t footer_offset = data_size + metaindex_size + index_size + 3 * 5;
+ {
+ // legacy block based
+ FooterBuilder footer;
+ footer.Build(kBlockBasedTableMagicNumber, /* format_version */ 0,
+ footer_offset, kCRC32c, meta_index, index);
+ Footer decoded_footer;
+ ASSERT_OK(decoded_footer.DecodeFrom(footer.GetSlice(), footer_offset));
+ ASSERT_EQ(decoded_footer.table_magic_number(), kBlockBasedTableMagicNumber);
+ ASSERT_EQ(decoded_footer.checksum_type(), kCRC32c);
+ ASSERT_EQ(decoded_footer.metaindex_handle().offset(), meta_index.offset());
+ ASSERT_EQ(decoded_footer.metaindex_handle().size(), meta_index.size());
+ ASSERT_EQ(decoded_footer.index_handle().offset(), index.offset());
+ ASSERT_EQ(decoded_footer.index_handle().size(), index.size());
+ ASSERT_EQ(decoded_footer.format_version(), 0U);
+ ASSERT_EQ(decoded_footer.GetBlockTrailerSize(), 5U);
+ // Ensure serialized with legacy magic
+ ASSERT_EQ(
+ DecodeFixed64(footer.GetSlice().data() + footer.GetSlice().size() - 8),
+ kLegacyBlockBasedTableMagicNumber);
+ }
+ // block based, various checksums, various versions
+ for (auto t : GetSupportedChecksums()) {
+ for (uint32_t fv = 1; IsSupportedFormatVersion(fv); ++fv) {
+ FooterBuilder footer;
+ footer.Build(kBlockBasedTableMagicNumber, fv, footer_offset, t,
+ meta_index, index);
+ Footer decoded_footer;
+ ASSERT_OK(decoded_footer.DecodeFrom(footer.GetSlice(), footer_offset));
+ ASSERT_EQ(decoded_footer.table_magic_number(),
+ kBlockBasedTableMagicNumber);
+ ASSERT_EQ(decoded_footer.checksum_type(), t);
+ ASSERT_EQ(decoded_footer.metaindex_handle().offset(),
+ meta_index.offset());
+ ASSERT_EQ(decoded_footer.metaindex_handle().size(), meta_index.size());
+ ASSERT_EQ(decoded_footer.index_handle().offset(), index.offset());
+ ASSERT_EQ(decoded_footer.index_handle().size(), index.size());
+ ASSERT_EQ(decoded_footer.format_version(), fv);
+ ASSERT_EQ(decoded_footer.GetBlockTrailerSize(), 5U);
+ }
+ }
+// Plain table is not supported in ROCKSDB_LITE
+#ifndef ROCKSDB_LITE
+ {
+ // legacy plain table
+ FooterBuilder footer;
+ footer.Build(kPlainTableMagicNumber, /* format_version */ 0, footer_offset,
+ kNoChecksum, meta_index);
+ Footer decoded_footer;
+ ASSERT_OK(decoded_footer.DecodeFrom(footer.GetSlice(), footer_offset));
+ ASSERT_EQ(decoded_footer.table_magic_number(), kPlainTableMagicNumber);
+ ASSERT_EQ(decoded_footer.checksum_type(), kCRC32c);
+ ASSERT_EQ(decoded_footer.metaindex_handle().offset(), meta_index.offset());
+ ASSERT_EQ(decoded_footer.metaindex_handle().size(), meta_index.size());
+ ASSERT_EQ(decoded_footer.index_handle().offset(), 0U);
+ ASSERT_EQ(decoded_footer.index_handle().size(), 0U);
+ ASSERT_EQ(decoded_footer.format_version(), 0U);
+ ASSERT_EQ(decoded_footer.GetBlockTrailerSize(), 0U);
+ // Ensure serialized with legacy magic
+ ASSERT_EQ(
+ DecodeFixed64(footer.GetSlice().data() + footer.GetSlice().size() - 8),
+ kLegacyPlainTableMagicNumber);
+ }
+ {
+ // xxhash plain table (not currently used)
+ FooterBuilder footer;
+ footer.Build(kPlainTableMagicNumber, /* format_version */ 1, footer_offset,
+ kxxHash, meta_index);
+ Footer decoded_footer;
+ ASSERT_OK(decoded_footer.DecodeFrom(footer.GetSlice(), footer_offset));
+ ASSERT_EQ(decoded_footer.table_magic_number(), kPlainTableMagicNumber);
+ ASSERT_EQ(decoded_footer.checksum_type(), kxxHash);
+ ASSERT_EQ(decoded_footer.metaindex_handle().offset(), meta_index.offset());
+ ASSERT_EQ(decoded_footer.metaindex_handle().size(), meta_index.size());
+ ASSERT_EQ(decoded_footer.index_handle().offset(), 0U);
+ ASSERT_EQ(decoded_footer.index_handle().size(), 0U);
+ ASSERT_EQ(decoded_footer.format_version(), 1U);
+ ASSERT_EQ(decoded_footer.GetBlockTrailerSize(), 0U);
+ }
+#endif // !ROCKSDB_LITE
+}
+
+class IndexBlockRestartIntervalTest
+ : public TableTest,
+ public ::testing::WithParamInterface<std::pair<int, bool>> {
+ public:
+ static std::vector<std::pair<int, bool>> GetRestartValues() {
+ return {{-1, false}, {0, false}, {1, false}, {8, false},
+ {16, false}, {32, false}, {-1, true}, {0, true},
+ {1, true}, {8, true}, {16, true}, {32, true}};
+ }
+};
+
+INSTANTIATE_TEST_CASE_P(
+ IndexBlockRestartIntervalTest, IndexBlockRestartIntervalTest,
+ ::testing::ValuesIn(IndexBlockRestartIntervalTest::GetRestartValues()));
+
+TEST_P(IndexBlockRestartIntervalTest, IndexBlockRestartInterval) {
+ const int kKeysInTable = 10000;
+ const int kKeySize = 100;
+ const int kValSize = 500;
+
+ const int index_block_restart_interval = std::get<0>(GetParam());
+ const bool value_delta_encoding = std::get<1>(GetParam());
+
+ Options options;
+ BlockBasedTableOptions table_options;
+ table_options.block_size = 64; // small block size to get big index block
+ table_options.index_block_restart_interval = index_block_restart_interval;
+ if (value_delta_encoding) {
+ table_options.format_version = 4;
+ } else {
+ table_options.format_version = 3;
+ }
+ options.table_factory.reset(new BlockBasedTableFactory(table_options));
+
+ TableConstructor c(BytewiseComparator());
+ static Random rnd(301);
+ for (int i = 0; i < kKeysInTable; i++) {
+ InternalKey k(rnd.RandomString(kKeySize), 0, kTypeValue);
+ c.Add(k.Encode().ToString(), rnd.RandomString(kValSize));
+ }
+
+ std::vector<std::string> keys;
+ stl_wrappers::KVMap kvmap;
+ std::unique_ptr<InternalKeyComparator> comparator(
+ new InternalKeyComparator(BytewiseComparator()));
+ const ImmutableOptions ioptions(options);
+ const MutableCFOptions moptions(options);
+ c.Finish(options, ioptions, moptions, table_options, *comparator, &keys,
+ &kvmap);
+ auto reader = c.GetTableReader();
+
+ ReadOptions read_options;
+ std::unique_ptr<InternalIterator> db_iter(reader->NewIterator(
+ read_options, moptions.prefix_extractor.get(), /*arena=*/nullptr,
+ /*skip_filters=*/false, TableReaderCaller::kUncategorized));
+
+ // Test point lookup
+ for (auto& kv : kvmap) {
+ db_iter->Seek(kv.first);
+
+ ASSERT_TRUE(db_iter->Valid());
+ ASSERT_OK(db_iter->status());
+ ASSERT_EQ(db_iter->key(), kv.first);
+ ASSERT_EQ(db_iter->value(), kv.second);
+ }
+
+ // Test iterating
+ auto kv_iter = kvmap.begin();
+ for (db_iter->SeekToFirst(); db_iter->Valid(); db_iter->Next()) {
+ ASSERT_EQ(db_iter->key(), kv_iter->first);
+ ASSERT_EQ(db_iter->value(), kv_iter->second);
+ kv_iter++;
+ }
+ ASSERT_EQ(kv_iter, kvmap.end());
+ c.ResetTableReader();
+}
+
+class PrefixTest : public testing::Test {
+ public:
+ PrefixTest() : testing::Test() {}
+ ~PrefixTest() override {}
+};
+
+namespace {
+// A simple PrefixExtractor that only works for test PrefixAndWholeKeyTest
+class TestPrefixExtractor : public ROCKSDB_NAMESPACE::SliceTransform {
+ public:
+ ~TestPrefixExtractor() override{};
+ const char* Name() const override { return "TestPrefixExtractor"; }
+
+ ROCKSDB_NAMESPACE::Slice Transform(
+ const ROCKSDB_NAMESPACE::Slice& src) const override {
+ assert(IsValid(src));
+ return ROCKSDB_NAMESPACE::Slice(src.data(), 3);
+ }
+
+ bool InDomain(const ROCKSDB_NAMESPACE::Slice& src) const override {
+ return IsValid(src);
+ }
+
+ bool InRange(const ROCKSDB_NAMESPACE::Slice& /*dst*/) const override {
+ return true;
+ }
+
+ bool IsValid(const ROCKSDB_NAMESPACE::Slice& src) const {
+ if (src.size() != 4) {
+ return false;
+ }
+ if (src[0] != '[') {
+ return false;
+ }
+ if (src[1] < '0' || src[1] > '9') {
+ return false;
+ }
+ if (src[2] != ']') {
+ return false;
+ }
+ if (src[3] < '0' || src[3] > '9') {
+ return false;
+ }
+ return true;
+ }
+};
+} // namespace
+
+TEST_F(PrefixTest, PrefixAndWholeKeyTest) {
+ ROCKSDB_NAMESPACE::Options options;
+ options.compaction_style = ROCKSDB_NAMESPACE::kCompactionStyleUniversal;
+ options.num_levels = 20;
+ options.create_if_missing = true;
+ options.optimize_filters_for_hits = false;
+ options.target_file_size_base = 268435456;
+ options.prefix_extractor = std::make_shared<TestPrefixExtractor>();
+ ROCKSDB_NAMESPACE::BlockBasedTableOptions bbto;
+ bbto.filter_policy.reset(ROCKSDB_NAMESPACE::NewBloomFilterPolicy(10));
+ bbto.block_size = 262144;
+ bbto.whole_key_filtering = true;
+
+ const std::string kDBPath = test::PerThreadDBPath("table_prefix_test");
+ options.table_factory.reset(NewBlockBasedTableFactory(bbto));
+ ASSERT_OK(DestroyDB(kDBPath, options));
+ ROCKSDB_NAMESPACE::DB* db;
+ ASSERT_OK(ROCKSDB_NAMESPACE::DB::Open(options, kDBPath, &db));
+
+ // Create a bunch of keys with 10 filters.
+ for (int i = 0; i < 10; i++) {
+ std::string prefix = "[" + std::to_string(i) + "]";
+ for (int j = 0; j < 10; j++) {
+ std::string key = prefix + std::to_string(j);
+ ASSERT_OK(db->Put(ROCKSDB_NAMESPACE::WriteOptions(), key, "1"));
+ }
+ }
+
+ // Trigger compaction.
+ ASSERT_OK(db->CompactRange(CompactRangeOptions(), nullptr, nullptr));
+ delete db;
+ // In the second round, turn whole_key_filtering off and expect
+ // rocksdb still works.
+}
+
+/*
+ * Disable TableWithGlobalSeqno since RocksDB does not store global_seqno in
+ * the SST file any more. Instead, RocksDB deduces global_seqno from the
+ * MANIFEST while reading from an SST. Therefore, it's not possible to test the
+ * functionality of global_seqno in a single, isolated unit test without the
+ * involvement of Version, VersionSet, etc.
+ */
+TEST_P(BlockBasedTableTest, DISABLED_TableWithGlobalSeqno) {
+ BlockBasedTableOptions bbto = GetBlockBasedTableOptions();
+ test::StringSink* sink = new test::StringSink();
+ std::unique_ptr<FSWritableFile> holder(sink);
+ std::unique_ptr<WritableFileWriter> file_writer(new WritableFileWriter(
+ std::move(holder), "" /* don't care */, FileOptions()));
+ Options options;
+ options.table_factory.reset(NewBlockBasedTableFactory(bbto));
+ const ImmutableOptions ioptions(options);
+ const MutableCFOptions moptions(options);
+ InternalKeyComparator ikc(options.comparator);
+ IntTblPropCollectorFactories int_tbl_prop_collector_factories;
+ int_tbl_prop_collector_factories.emplace_back(
+ new SstFileWriterPropertiesCollectorFactory(2 /* version */,
+ 0 /* global_seqno*/));
+ std::string column_family_name;
+ std::unique_ptr<TableBuilder> builder(options.table_factory->NewTableBuilder(
+ TableBuilderOptions(ioptions, moptions, ikc,
+ &int_tbl_prop_collector_factories, kNoCompression,
+ CompressionOptions(), kUnknownColumnFamily,
+ column_family_name, -1),
+ file_writer.get()));
+
+ for (char c = 'a'; c <= 'z'; ++c) {
+ std::string key(8, c);
+ std::string value = key;
+ InternalKey ik(key, 0, kTypeValue);
+
+ builder->Add(ik.Encode(), value);
+ }
+ ASSERT_OK(builder->Finish());
+ ASSERT_OK(file_writer->Flush());
+
+ test::RandomRWStringSink ss_rw(sink);
+ uint32_t version;
+ uint64_t global_seqno;
+ uint64_t global_seqno_offset;
+
+ // Helper function to get version, global_seqno, global_seqno_offset
+ std::function<void()> GetVersionAndGlobalSeqno = [&]() {
+ std::unique_ptr<FSRandomAccessFile> source(
+ new test::StringSource(ss_rw.contents(), 73342, true));
+ std::unique_ptr<RandomAccessFileReader> file_reader(
+ new RandomAccessFileReader(std::move(source), ""));
+
+ std::unique_ptr<TableProperties> props;
+ ASSERT_OK(ReadTableProperties(file_reader.get(), ss_rw.contents().size(),
+ kBlockBasedTableMagicNumber, ioptions,
+ &props));
+
+ UserCollectedProperties user_props = props->user_collected_properties;
+ version = DecodeFixed32(
+ user_props[ExternalSstFilePropertyNames::kVersion].c_str());
+ global_seqno = DecodeFixed64(
+ user_props[ExternalSstFilePropertyNames::kGlobalSeqno].c_str());
+ global_seqno_offset = props->external_sst_file_global_seqno_offset;
+ };
+
+ // Helper function to update the value of the global seqno in the file
+ std::function<void(uint64_t)> SetGlobalSeqno = [&](uint64_t val) {
+ std::string new_global_seqno;
+ PutFixed64(&new_global_seqno, val);
+
+ ASSERT_OK(ss_rw.Write(global_seqno_offset, new_global_seqno, IOOptions(),
+ nullptr));
+ };
+
+ // Helper function to get the contents of the table InternalIterator
+ std::unique_ptr<TableReader> table_reader;
+ const ReadOptions read_options;
+ std::function<InternalIterator*()> GetTableInternalIter = [&]() {
+ std::unique_ptr<FSRandomAccessFile> source(
+ new test::StringSource(ss_rw.contents(), 73342, true));
+ std::unique_ptr<RandomAccessFileReader> file_reader(
+ new RandomAccessFileReader(std::move(source), ""));
+
+ options.table_factory->NewTableReader(
+ TableReaderOptions(ioptions, moptions.prefix_extractor, EnvOptions(),
+ ikc),
+ std::move(file_reader), ss_rw.contents().size(), &table_reader);
+
+ return table_reader->NewIterator(
+ read_options, moptions.prefix_extractor.get(), /*arena=*/nullptr,
+ /*skip_filters=*/false, TableReaderCaller::kUncategorized);
+ };
+
+ GetVersionAndGlobalSeqno();
+ ASSERT_EQ(2u, version);
+ ASSERT_EQ(0u, global_seqno);
+
+ InternalIterator* iter = GetTableInternalIter();
+ char current_c = 'a';
+ for (iter->SeekToFirst(); iter->Valid(); iter->Next()) {
+ ParsedInternalKey pik;
+ ASSERT_OK(ParseInternalKey(iter->key(), &pik, true /* log_err_key */));
+
+ ASSERT_EQ(pik.type, ValueType::kTypeValue);
+ ASSERT_EQ(pik.sequence, 0);
+ ASSERT_EQ(pik.user_key, iter->value());
+ ASSERT_EQ(pik.user_key.ToString(), std::string(8, current_c));
+ current_c++;
+ }
+ ASSERT_EQ(current_c, 'z' + 1);
+ delete iter;
+
+ // Update global sequence number to 10
+ SetGlobalSeqno(10);
+ GetVersionAndGlobalSeqno();
+ ASSERT_EQ(2u, version);
+ ASSERT_EQ(10u, global_seqno);
+
+ iter = GetTableInternalIter();
+ current_c = 'a';
+ for (iter->SeekToFirst(); iter->Valid(); iter->Next()) {
+ ParsedInternalKey pik;
+ ASSERT_OK(ParseInternalKey(iter->key(), &pik, true /* log_err_key */));
+
+ ASSERT_EQ(pik.type, ValueType::kTypeValue);
+ ASSERT_EQ(pik.sequence, 10);
+ ASSERT_EQ(pik.user_key, iter->value());
+ ASSERT_EQ(pik.user_key.ToString(), std::string(8, current_c));
+ current_c++;
+ }
+ ASSERT_EQ(current_c, 'z' + 1);
+
+ // Verify Seek
+ for (char c = 'a'; c <= 'z'; c++) {
+ std::string k = std::string(8, c);
+ InternalKey ik(k, 10, kValueTypeForSeek);
+ iter->Seek(ik.Encode());
+ ASSERT_TRUE(iter->Valid());
+
+ ParsedInternalKey pik;
+ ASSERT_OK(ParseInternalKey(iter->key(), &pik, true /* log_err_key */));
+
+ ASSERT_EQ(pik.type, ValueType::kTypeValue);
+ ASSERT_EQ(pik.sequence, 10);
+ ASSERT_EQ(pik.user_key.ToString(), k);
+ ASSERT_EQ(iter->value().ToString(), k);
+ }
+ delete iter;
+
+ // Update global sequence number to 3
+ SetGlobalSeqno(3);
+ GetVersionAndGlobalSeqno();
+ ASSERT_EQ(2u, version);
+ ASSERT_EQ(3u, global_seqno);
+
+ iter = GetTableInternalIter();
+ current_c = 'a';
+ for (iter->SeekToFirst(); iter->Valid(); iter->Next()) {
+ ParsedInternalKey pik;
+ ASSERT_OK(ParseInternalKey(iter->key(), &pik, true /* log_err_key */));
+
+ ASSERT_EQ(pik.type, ValueType::kTypeValue);
+ ASSERT_EQ(pik.sequence, 3);
+ ASSERT_EQ(pik.user_key, iter->value());
+ ASSERT_EQ(pik.user_key.ToString(), std::string(8, current_c));
+ current_c++;
+ }
+ ASSERT_EQ(current_c, 'z' + 1);
+
+ // Verify Seek
+ for (char c = 'a'; c <= 'z'; c++) {
+ std::string k = std::string(8, c);
+ // seqno=4 is less than 3 so we still should get our key
+ InternalKey ik(k, 4, kValueTypeForSeek);
+ iter->Seek(ik.Encode());
+ ASSERT_TRUE(iter->Valid());
+
+ ParsedInternalKey pik;
+ ASSERT_OK(ParseInternalKey(iter->key(), &pik, true /* log_err_key */));
+
+ ASSERT_EQ(pik.type, ValueType::kTypeValue);
+ ASSERT_EQ(pik.sequence, 3);
+ ASSERT_EQ(pik.user_key.ToString(), k);
+ ASSERT_EQ(iter->value().ToString(), k);
+ }
+
+ delete iter;
+}
+
+TEST_P(BlockBasedTableTest, BlockAlignTest) {
+ BlockBasedTableOptions bbto = GetBlockBasedTableOptions();
+ bbto.block_align = true;
+ test::StringSink* sink = new test::StringSink();
+ std::unique_ptr<FSWritableFile> holder(sink);
+ std::unique_ptr<WritableFileWriter> file_writer(new WritableFileWriter(
+ std::move(holder), "" /* don't care */, FileOptions()));
+ Options options;
+ options.compression = kNoCompression;
+ options.table_factory.reset(NewBlockBasedTableFactory(bbto));
+ const ImmutableOptions ioptions(options);
+ const MutableCFOptions moptions(options);
+ InternalKeyComparator ikc(options.comparator);
+ IntTblPropCollectorFactories int_tbl_prop_collector_factories;
+ std::string column_family_name;
+ std::unique_ptr<TableBuilder> builder(options.table_factory->NewTableBuilder(
+ TableBuilderOptions(ioptions, moptions, ikc,
+ &int_tbl_prop_collector_factories, kNoCompression,
+ CompressionOptions(), kUnknownColumnFamily,
+ column_family_name, -1),
+ file_writer.get()));
+
+ for (int i = 1; i <= 10000; ++i) {
+ std::ostringstream ostr;
+ ostr << std::setfill('0') << std::setw(5) << i;
+ std::string key = ostr.str();
+ std::string value = "val";
+ InternalKey ik(key, 0, kTypeValue);
+
+ builder->Add(ik.Encode(), value);
+ }
+ ASSERT_OK(builder->Finish());
+ ASSERT_OK(file_writer->Flush());
+
+ std::unique_ptr<FSRandomAccessFile> source(
+ new test::StringSource(sink->contents(), 73342, false));
+ std::unique_ptr<RandomAccessFileReader> file_reader(
+ new RandomAccessFileReader(std::move(source), "test"));
+ // Helper function to get version, global_seqno, global_seqno_offset
+ std::function<void()> VerifyBlockAlignment = [&]() {
+ std::unique_ptr<TableProperties> props;
+ ASSERT_OK(ReadTableProperties(file_reader.get(), sink->contents().size(),
+ kBlockBasedTableMagicNumber, ioptions,
+ &props));
+
+ uint64_t data_block_size = props->data_size / props->num_data_blocks;
+ ASSERT_EQ(data_block_size, 4096);
+ ASSERT_EQ(props->data_size, data_block_size * props->num_data_blocks);
+ };
+
+ VerifyBlockAlignment();
+
+ // The below block of code verifies that we can read back the keys. Set
+ // block_align to false when creating the reader to ensure we can flip between
+ // the two modes without any issues
+ std::unique_ptr<TableReader> table_reader;
+ bbto.block_align = false;
+ Options options2;
+ options2.table_factory.reset(NewBlockBasedTableFactory(bbto));
+ ImmutableOptions ioptions2(options2);
+ const MutableCFOptions moptions2(options2);
+
+ ASSERT_OK(ioptions.table_factory->NewTableReader(
+ TableReaderOptions(ioptions2, moptions2.prefix_extractor, EnvOptions(),
+ GetPlainInternalComparator(options2.comparator)),
+ std::move(file_reader), sink->contents().size(), &table_reader));
+
+ ReadOptions read_options;
+ std::unique_ptr<InternalIterator> db_iter(table_reader->NewIterator(
+ read_options, moptions2.prefix_extractor.get(), /*arena=*/nullptr,
+ /*skip_filters=*/false, TableReaderCaller::kUncategorized));
+
+ int expected_key = 1;
+ for (db_iter->SeekToFirst(); db_iter->Valid(); db_iter->Next()) {
+ std::ostringstream ostr;
+ ostr << std::setfill('0') << std::setw(5) << expected_key++;
+ std::string key = ostr.str();
+ std::string value = "val";
+
+ ASSERT_OK(db_iter->status());
+ ASSERT_EQ(ExtractUserKey(db_iter->key()).ToString(), key);
+ ASSERT_EQ(db_iter->value().ToString(), value);
+ }
+ expected_key--;
+ ASSERT_EQ(expected_key, 10000);
+ table_reader.reset();
+}
+
+TEST_P(BlockBasedTableTest, PropertiesBlockRestartPointTest) {
+ BlockBasedTableOptions bbto = GetBlockBasedTableOptions();
+ bbto.block_align = true;
+ test::StringSink* sink = new test::StringSink();
+ std::unique_ptr<FSWritableFile> holder(sink);
+ std::unique_ptr<WritableFileWriter> file_writer(new WritableFileWriter(
+ std::move(holder), "" /* don't care */, FileOptions()));
+
+ Options options;
+ options.compression = kNoCompression;
+ options.table_factory.reset(NewBlockBasedTableFactory(bbto));
+
+ const ImmutableOptions ioptions(options);
+ const MutableCFOptions moptions(options);
+ InternalKeyComparator ikc(options.comparator);
+ IntTblPropCollectorFactories int_tbl_prop_collector_factories;
+ std::string column_family_name;
+
+ std::unique_ptr<TableBuilder> builder(options.table_factory->NewTableBuilder(
+ TableBuilderOptions(ioptions, moptions, ikc,
+ &int_tbl_prop_collector_factories, kNoCompression,
+ CompressionOptions(), kUnknownColumnFamily,
+ column_family_name, -1),
+ file_writer.get()));
+
+ for (int i = 1; i <= 10000; ++i) {
+ std::ostringstream ostr;
+ ostr << std::setfill('0') << std::setw(5) << i;
+ std::string key = ostr.str();
+ std::string value = "val";
+ InternalKey ik(key, 0, kTypeValue);
+
+ builder->Add(ik.Encode(), value);
+ }
+ ASSERT_OK(builder->Finish());
+ ASSERT_OK(file_writer->Flush());
+
+ std::unique_ptr<FSRandomAccessFile> source(
+ new test::StringSource(sink->contents(), 73342, true));
+ std::unique_ptr<RandomAccessFileReader> file_reader(
+ new RandomAccessFileReader(std::move(source), "test"));
+
+ {
+ RandomAccessFileReader* file = file_reader.get();
+ uint64_t file_size = sink->contents().size();
+
+ Footer footer;
+ IOOptions opts;
+ ASSERT_OK(ReadFooterFromFile(opts, file, nullptr /* prefetch_buffer */,
+ file_size, &footer,
+ kBlockBasedTableMagicNumber));
+
+ auto BlockFetchHelper = [&](const BlockHandle& handle, BlockType block_type,
+ BlockContents* contents) {
+ ReadOptions read_options;
+ read_options.verify_checksums = false;
+ PersistentCacheOptions cache_options;
+
+ BlockFetcher block_fetcher(
+ file, nullptr /* prefetch_buffer */, footer, read_options, handle,
+ contents, ioptions, false /* decompress */,
+ false /*maybe_compressed*/, block_type,
+ UncompressionDict::GetEmptyDict(), cache_options);
+
+ ASSERT_OK(block_fetcher.ReadBlockContents());
+ };
+
+ // -- Read metaindex block
+ auto metaindex_handle = footer.metaindex_handle();
+ BlockContents metaindex_contents;
+
+ BlockFetchHelper(metaindex_handle, BlockType::kMetaIndex,
+ &metaindex_contents);
+ Block metaindex_block(std::move(metaindex_contents));
+
+ std::unique_ptr<InternalIterator> meta_iter(metaindex_block.NewDataIterator(
+ BytewiseComparator(), kDisableGlobalSequenceNumber));
+
+ // -- Read properties block
+ BlockHandle properties_handle;
+ ASSERT_OK(FindOptionalMetaBlock(meta_iter.get(), kPropertiesBlockName,
+ &properties_handle));
+ ASSERT_FALSE(properties_handle.IsNull());
+ BlockContents properties_contents;
+ BlockFetchHelper(properties_handle, BlockType::kProperties,
+ &properties_contents);
+ Block properties_block(std::move(properties_contents));
+
+ ASSERT_EQ(properties_block.NumRestarts(), 1u);
+ }
+}
+
+TEST_P(BlockBasedTableTest, PropertiesMetaBlockLast) {
+ // The properties meta-block should come at the end since we always need to
+ // read it when opening a file, unlike index/filter/other meta-blocks, which
+ // are sometimes read depending on the user's configuration. This ordering
+ // allows us to do a small readahead on the end of the file to read properties
+ // and meta-index blocks with one I/O.
+ TableConstructor c(BytewiseComparator(), true /* convert_to_internal_key_ */);
+ c.Add("a1", "val1");
+ c.Add("b2", "val2");
+ c.Add("c3", "val3");
+ c.Add("d4", "val4");
+ c.Add("e5", "val5");
+ c.Add("f6", "val6");
+ c.Add("g7", "val7");
+ c.Add("h8", "val8");
+ c.Add("j9", "val9");
+
+ // write an SST file
+ Options options;
+ BlockBasedTableOptions table_options = GetBlockBasedTableOptions();
+ table_options.filter_policy.reset(NewBloomFilterPolicy(
+ 8 /* bits_per_key */, false /* use_block_based_filter */));
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+ ImmutableOptions ioptions(options);
+ MutableCFOptions moptions(options);
+ std::vector<std::string> keys;
+ stl_wrappers::KVMap kvmap;
+ c.Finish(options, ioptions, moptions, table_options,
+ GetPlainInternalComparator(options.comparator), &keys, &kvmap);
+
+ // get file reader
+ test::StringSink* table_sink = c.TEST_GetSink();
+ std::unique_ptr<FSRandomAccessFile> source(new test::StringSource(
+ table_sink->contents(), 0 /* unique_id */, false /* allow_mmap_reads */));
+
+ std::unique_ptr<RandomAccessFileReader> table_reader(
+ new RandomAccessFileReader(std::move(source), "test"));
+ size_t table_size = table_sink->contents().size();
+
+ // read footer
+ Footer footer;
+ IOOptions opts;
+ ASSERT_OK(ReadFooterFromFile(opts, table_reader.get(),
+ nullptr /* prefetch_buffer */, table_size,
+ &footer, kBlockBasedTableMagicNumber));
+
+ // read metaindex
+ auto metaindex_handle = footer.metaindex_handle();
+ BlockContents metaindex_contents;
+ PersistentCacheOptions pcache_opts;
+ BlockFetcher block_fetcher(
+ table_reader.get(), nullptr /* prefetch_buffer */, footer, ReadOptions(),
+ metaindex_handle, &metaindex_contents, ioptions, false /* decompress */,
+ false /*maybe_compressed*/, BlockType::kMetaIndex,
+ UncompressionDict::GetEmptyDict(), pcache_opts,
+ nullptr /*memory_allocator*/);
+ ASSERT_OK(block_fetcher.ReadBlockContents());
+ Block metaindex_block(std::move(metaindex_contents));
+
+ // verify properties block comes last
+ std::unique_ptr<InternalIterator> metaindex_iter{
+ metaindex_block.NewMetaIterator()};
+ uint64_t max_offset = 0;
+ std::string key_at_max_offset;
+ for (metaindex_iter->SeekToFirst(); metaindex_iter->Valid();
+ metaindex_iter->Next()) {
+ BlockHandle handle;
+ Slice value = metaindex_iter->value();
+ ASSERT_OK(handle.DecodeFrom(&value));
+ if (handle.offset() > max_offset) {
+ max_offset = handle.offset();
+ key_at_max_offset = metaindex_iter->key().ToString();
+ }
+ }
+ ASSERT_EQ(kPropertiesBlockName, key_at_max_offset);
+ // index handle is stored in footer rather than metaindex block, so need
+ // separate logic to verify it comes before properties block.
+ ASSERT_GT(max_offset, footer.index_handle().offset());
+ c.ResetTableReader();
+}
+
+TEST_P(BlockBasedTableTest, SeekMetaBlocks) {
+ TableConstructor c(BytewiseComparator(), true /* convert_to_internal_key_ */);
+ c.Add("foo_a1", "val1");
+ c.Add("foo_b2", "val2");
+ c.Add("foo_c3", "val3");
+ c.Add("foo_d4", "val4");
+ c.Add("foo_e5", "val5");
+ c.Add("foo_f6", "val6");
+ c.Add("foo_g7", "val7");
+ c.Add("foo_h8", "val8");
+ c.Add("foo_j9", "val9");
+
+ // write an SST file
+ Options options;
+ BlockBasedTableOptions table_options = GetBlockBasedTableOptions();
+ table_options.index_type = BlockBasedTableOptions::kHashSearch;
+ table_options.filter_policy.reset(NewBloomFilterPolicy(
+ 8 /* bits_per_key */, false /* use_block_based_filter */));
+ options.prefix_extractor.reset(NewFixedPrefixTransform(4));
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+ ImmutableOptions ioptions(options);
+ MutableCFOptions moptions(options);
+ std::vector<std::string> keys;
+ stl_wrappers::KVMap kvmap;
+ c.Finish(options, ioptions, moptions, table_options,
+ GetPlainInternalComparator(options.comparator), &keys, &kvmap);
+
+ // get file reader
+ test::StringSink* table_sink = c.TEST_GetSink();
+ std::unique_ptr<FSRandomAccessFile> source(new test::StringSource(
+ table_sink->contents(), 0 /* unique_id */, false /* allow_mmap_reads */));
+
+ std::unique_ptr<RandomAccessFileReader> table_reader(
+ new RandomAccessFileReader(std::move(source), "test"));
+ size_t table_size = table_sink->contents().size();
+
+ // read footer
+ Footer footer;
+ IOOptions opts;
+ ASSERT_OK(ReadFooterFromFile(opts, table_reader.get(),
+ nullptr /* prefetch_buffer */, table_size,
+ &footer, kBlockBasedTableMagicNumber));
+
+ // read metaindex
+ auto metaindex_handle = footer.metaindex_handle();
+ BlockContents metaindex_contents;
+ PersistentCacheOptions pcache_opts;
+ BlockFetcher block_fetcher(
+ table_reader.get(), nullptr /* prefetch_buffer */, footer, ReadOptions(),
+ metaindex_handle, &metaindex_contents, ioptions, false /* decompress */,
+ false /*maybe_compressed*/, BlockType::kMetaIndex,
+ UncompressionDict::GetEmptyDict(), pcache_opts,
+ nullptr /*memory_allocator*/);
+ ASSERT_OK(block_fetcher.ReadBlockContents());
+ Block metaindex_block(std::move(metaindex_contents));
+
+ // verify properties block comes last
+ std::unique_ptr<MetaBlockIter> metaindex_iter(
+ metaindex_block.NewMetaIterator());
+ bool has_hash_prefixes = false;
+ bool has_hash_metadata = false;
+ for (metaindex_iter->SeekToFirst(); metaindex_iter->Valid();
+ metaindex_iter->Next()) {
+ if (metaindex_iter->key().ToString() == kHashIndexPrefixesBlock) {
+ has_hash_prefixes = true;
+ } else if (metaindex_iter->key().ToString() ==
+ kHashIndexPrefixesMetadataBlock) {
+ has_hash_metadata = true;
+ }
+ }
+ if (has_hash_metadata) {
+ metaindex_iter->Seek(kHashIndexPrefixesMetadataBlock);
+ ASSERT_TRUE(metaindex_iter->Valid());
+ ASSERT_EQ(kHashIndexPrefixesMetadataBlock,
+ metaindex_iter->key().ToString());
+ }
+ if (has_hash_prefixes) {
+ metaindex_iter->Seek(kHashIndexPrefixesBlock);
+ ASSERT_TRUE(metaindex_iter->Valid());
+ ASSERT_EQ(kHashIndexPrefixesBlock, metaindex_iter->key().ToString());
+ }
+ c.ResetTableReader();
+}
+
+TEST_P(BlockBasedTableTest, BadOptions) {
+ ROCKSDB_NAMESPACE::Options options;
+ options.compression = kNoCompression;
+ BlockBasedTableOptions bbto = GetBlockBasedTableOptions();
+ bbto.block_size = 4000;
+ bbto.block_align = true;
+
+ const std::string kDBPath =
+ test::PerThreadDBPath("block_based_table_bad_options_test");
+ options.table_factory.reset(NewBlockBasedTableFactory(bbto));
+ ASSERT_OK(DestroyDB(kDBPath, options));
+ ROCKSDB_NAMESPACE::DB* db;
+ ASSERT_NOK(ROCKSDB_NAMESPACE::DB::Open(options, kDBPath, &db));
+
+ bbto.block_size = 4096;
+ options.compression = kSnappyCompression;
+ options.table_factory.reset(NewBlockBasedTableFactory(bbto));
+ ASSERT_NOK(ROCKSDB_NAMESPACE::DB::Open(options, kDBPath, &db));
+}
+
+TEST_F(BBTTailPrefetchTest, TestTailPrefetchStats) {
+ TailPrefetchStats tpstats;
+ ASSERT_EQ(0, tpstats.GetSuggestedPrefetchSize());
+ tpstats.RecordEffectiveSize(size_t{1000});
+ tpstats.RecordEffectiveSize(size_t{1005});
+ tpstats.RecordEffectiveSize(size_t{1002});
+ ASSERT_EQ(1005, tpstats.GetSuggestedPrefetchSize());
+
+ // One single super large value shouldn't influence much
+ tpstats.RecordEffectiveSize(size_t{1002000});
+ tpstats.RecordEffectiveSize(size_t{999});
+ ASSERT_LE(1005, tpstats.GetSuggestedPrefetchSize());
+ ASSERT_GT(1200, tpstats.GetSuggestedPrefetchSize());
+
+ // Only history of 32 is kept
+ for (int i = 0; i < 32; i++) {
+ tpstats.RecordEffectiveSize(size_t{100});
+ }
+ ASSERT_EQ(100, tpstats.GetSuggestedPrefetchSize());
+
+ // 16 large values and 16 small values. The result should be closer
+ // to the small value as the algorithm.
+ for (int i = 0; i < 16; i++) {
+ tpstats.RecordEffectiveSize(size_t{1000});
+ }
+ tpstats.RecordEffectiveSize(size_t{10});
+ tpstats.RecordEffectiveSize(size_t{20});
+ for (int i = 0; i < 6; i++) {
+ tpstats.RecordEffectiveSize(size_t{100});
+ }
+ ASSERT_LE(80, tpstats.GetSuggestedPrefetchSize());
+ ASSERT_GT(200, tpstats.GetSuggestedPrefetchSize());
+}
+
+TEST_F(BBTTailPrefetchTest, FilePrefetchBufferMinOffset) {
+ TailPrefetchStats tpstats;
+ FilePrefetchBuffer buffer(0 /* readahead_size */, 0 /* max_readahead_size */,
+ false /* enable */, true /* track_min_offset */);
+ IOOptions opts;
+ buffer.TryReadFromCache(opts, nullptr /* reader */, 500 /* offset */,
+ 10 /* n */, nullptr /* result */,
+ nullptr /* status */,
+ Env::IO_TOTAL /* rate_limiter_priority */);
+ buffer.TryReadFromCache(opts, nullptr /* reader */, 480 /* offset */,
+ 10 /* n */, nullptr /* result */,
+ nullptr /* status */,
+ Env::IO_TOTAL /* rate_limiter_priority */);
+ buffer.TryReadFromCache(opts, nullptr /* reader */, 490 /* offset */,
+ 10 /* n */, nullptr /* result */,
+ nullptr /* status */,
+ Env::IO_TOTAL /* rate_limiter_priority */);
+ ASSERT_EQ(480, buffer.min_offset_read());
+}
+
+TEST_P(BlockBasedTableTest, DataBlockHashIndex) {
+ const int kNumKeys = 500;
+ const int kKeySize = 8;
+ const int kValSize = 40;
+
+ BlockBasedTableOptions table_options = GetBlockBasedTableOptions();
+ table_options.data_block_index_type =
+ BlockBasedTableOptions::kDataBlockBinaryAndHash;
+
+ Options options;
+ options.comparator = BytewiseComparator();
+
+ options.table_factory.reset(new BlockBasedTableFactory(table_options));
+
+ TableConstructor c(options.comparator);
+
+ static Random rnd(1048);
+ for (int i = 0; i < kNumKeys; i++) {
+ // padding one "0" to mark existent keys.
+ std::string random_key(rnd.RandomString(kKeySize - 1) + "1");
+ InternalKey k(random_key, 0, kTypeValue);
+ c.Add(k.Encode().ToString(), rnd.RandomString(kValSize));
+ }
+
+ std::vector<std::string> keys;
+ stl_wrappers::KVMap kvmap;
+ const ImmutableOptions ioptions(options);
+ const MutableCFOptions moptions(options);
+ const InternalKeyComparator internal_comparator(options.comparator);
+ c.Finish(options, ioptions, moptions, table_options, internal_comparator,
+ &keys, &kvmap);
+
+ auto reader = c.GetTableReader();
+
+ std::unique_ptr<InternalIterator> seek_iter;
+ ReadOptions read_options;
+ seek_iter.reset(reader->NewIterator(
+ read_options, moptions.prefix_extractor.get(), /*arena=*/nullptr,
+ /*skip_filters=*/false, TableReaderCaller::kUncategorized));
+ for (int i = 0; i < 2; ++i) {
+ ReadOptions ro;
+ // for every kv, we seek using two method: Get() and Seek()
+ // Get() will use the SuffixIndexHash in Block. For non-existent key it
+ // will invalidate the iterator
+ // Seek() will use the default BinarySeek() in Block. So for non-existent
+ // key it will land at the closest key that is large than target.
+
+ // Search for existent keys
+ for (auto& kv : kvmap) {
+ if (i == 0) {
+ // Search using Seek()
+ seek_iter->Seek(kv.first);
+ ASSERT_OK(seek_iter->status());
+ ASSERT_TRUE(seek_iter->Valid());
+ ASSERT_EQ(seek_iter->key(), kv.first);
+ ASSERT_EQ(seek_iter->value(), kv.second);
+ } else {
+ // Search using Get()
+ PinnableSlice value;
+ std::string user_key = ExtractUserKey(kv.first).ToString();
+ GetContext get_context(options.comparator, nullptr, nullptr, nullptr,
+ GetContext::kNotFound, user_key, &value, nullptr,
+ nullptr, nullptr, true, nullptr, nullptr);
+ ASSERT_OK(reader->Get(ro, kv.first, &get_context,
+ moptions.prefix_extractor.get()));
+ ASSERT_EQ(get_context.State(), GetContext::kFound);
+ ASSERT_EQ(value, Slice(kv.second));
+ value.Reset();
+ }
+ }
+
+ // Search for non-existent keys
+ for (auto& kv : kvmap) {
+ std::string user_key = ExtractUserKey(kv.first).ToString();
+ user_key.back() = '0'; // make it non-existent key
+ InternalKey internal_key(user_key, 0, kTypeValue);
+ std::string encoded_key = internal_key.Encode().ToString();
+ if (i == 0) { // Search using Seek()
+ seek_iter->Seek(encoded_key);
+ ASSERT_OK(seek_iter->status());
+ if (seek_iter->Valid()) {
+ ASSERT_TRUE(BytewiseComparator()->Compare(
+ user_key, ExtractUserKey(seek_iter->key())) < 0);
+ }
+ } else { // Search using Get()
+ PinnableSlice value;
+ GetContext get_context(options.comparator, nullptr, nullptr, nullptr,
+ GetContext::kNotFound, user_key, &value, nullptr,
+ nullptr, nullptr, true, nullptr, nullptr);
+ ASSERT_OK(reader->Get(ro, encoded_key, &get_context,
+ moptions.prefix_extractor.get()));
+ ASSERT_EQ(get_context.State(), GetContext::kNotFound);
+ value.Reset();
+ }
+ }
+ }
+}
+
+// BlockBasedTableIterator should invalidate itself and return
+// OutOfBound()=true immediately after Seek(), to allow LevelIterator
+// filter out corresponding level.
+TEST_P(BlockBasedTableTest, OutOfBoundOnSeek) {
+ TableConstructor c(BytewiseComparator(), true /*convert_to_internal_key*/);
+ c.Add("foo", "v1");
+ std::vector<std::string> keys;
+ stl_wrappers::KVMap kvmap;
+ Options options;
+ BlockBasedTableOptions table_opt(GetBlockBasedTableOptions());
+ options.table_factory.reset(NewBlockBasedTableFactory(table_opt));
+ const ImmutableOptions ioptions(options);
+ const MutableCFOptions moptions(options);
+ c.Finish(options, ioptions, moptions, table_opt,
+ GetPlainInternalComparator(BytewiseComparator()), &keys, &kvmap);
+ auto* reader = c.GetTableReader();
+ ReadOptions read_opt;
+ std::string upper_bound = "bar";
+ Slice upper_bound_slice(upper_bound);
+ read_opt.iterate_upper_bound = &upper_bound_slice;
+ std::unique_ptr<InternalIterator> iter;
+ iter.reset(new KeyConvertingIterator(reader->NewIterator(
+ read_opt, /*prefix_extractor=*/nullptr, /*arena=*/nullptr,
+ /*skip_filters=*/false, TableReaderCaller::kUncategorized)));
+ iter->SeekToFirst();
+ ASSERT_FALSE(iter->Valid());
+ ASSERT_OK(iter->status());
+ ASSERT_TRUE(iter->UpperBoundCheckResult() == IterBoundCheck::kOutOfBound);
+ iter.reset(new KeyConvertingIterator(reader->NewIterator(
+ read_opt, /*prefix_extractor=*/nullptr, /*arena=*/nullptr,
+ /*skip_filters=*/false, TableReaderCaller::kUncategorized)));
+ iter->Seek("foo");
+ ASSERT_FALSE(iter->Valid());
+ ASSERT_OK(iter->status());
+ ASSERT_TRUE(iter->UpperBoundCheckResult() == IterBoundCheck::kOutOfBound);
+}
+
+// BlockBasedTableIterator should invalidate itself and return
+// OutOfBound()=true after Next(), if it finds current index key is no smaller
+// than upper bound, unless it is pointing to the last data block.
+TEST_P(BlockBasedTableTest, OutOfBoundOnNext) {
+ TableConstructor c(BytewiseComparator(), true /*convert_to_internal_key*/);
+ c.Add("bar", "v");
+ c.Add("foo", "v");
+ std::vector<std::string> keys;
+ stl_wrappers::KVMap kvmap;
+ Options options;
+ BlockBasedTableOptions table_opt(GetBlockBasedTableOptions());
+ table_opt.flush_block_policy_factory =
+ std::make_shared<FlushBlockEveryKeyPolicyFactory>();
+ options.table_factory.reset(NewBlockBasedTableFactory(table_opt));
+ const ImmutableOptions ioptions(options);
+ const MutableCFOptions moptions(options);
+ c.Finish(options, ioptions, moptions, table_opt,
+ GetPlainInternalComparator(BytewiseComparator()), &keys, &kvmap);
+ auto* reader = c.GetTableReader();
+ ReadOptions read_opt;
+ std::string ub1 = "bar_after";
+ Slice ub_slice1(ub1);
+ read_opt.iterate_upper_bound = &ub_slice1;
+ std::unique_ptr<InternalIterator> iter;
+ iter.reset(new KeyConvertingIterator(reader->NewIterator(
+ read_opt, /*prefix_extractor=*/nullptr, /*arena=*/nullptr,
+ /*skip_filters=*/false, TableReaderCaller::kUncategorized)));
+ iter->Seek("bar");
+ ASSERT_TRUE(iter->Valid());
+ ASSERT_EQ("bar", iter->key());
+ iter->Next();
+ ASSERT_FALSE(iter->Valid());
+ ASSERT_TRUE(iter->UpperBoundCheckResult() == IterBoundCheck::kOutOfBound);
+ std::string ub2 = "foo_after";
+ Slice ub_slice2(ub2);
+ read_opt.iterate_upper_bound = &ub_slice2;
+ iter.reset(new KeyConvertingIterator(reader->NewIterator(
+ read_opt, /*prefix_extractor=*/nullptr, /*arena=*/nullptr,
+ /*skip_filters=*/false, TableReaderCaller::kUncategorized)));
+ iter->Seek("foo");
+ ASSERT_TRUE(iter->Valid());
+ ASSERT_EQ("foo", iter->key());
+ iter->Next();
+ ASSERT_FALSE(iter->Valid());
+ ASSERT_FALSE(iter->UpperBoundCheckResult() == IterBoundCheck::kOutOfBound);
+}
+
+class ChargeCompressionDictionaryBuildingBufferTest
+ : public BlockBasedTableTestBase {};
+TEST_F(ChargeCompressionDictionaryBuildingBufferTest, Basic) {
+ constexpr std::size_t kSizeDummyEntry = 256 * 1024;
+ constexpr std::size_t kMetaDataChargeOverhead = 10000;
+ constexpr std::size_t kCacheCapacity = 8 * 1024 * 1024;
+ constexpr std::size_t kMaxDictBytes = 1024;
+ constexpr std::size_t kMaxDictBufferBytes = 1024;
+
+ for (CacheEntryRoleOptions::Decision
+ charge_compression_dictionary_building_buffer :
+ {CacheEntryRoleOptions::Decision::kEnabled,
+ CacheEntryRoleOptions::Decision::kDisabled}) {
+ BlockBasedTableOptions table_options;
+ LRUCacheOptions lo;
+ lo.capacity = kCacheCapacity;
+ lo.num_shard_bits = 0; // 2^0 shard
+ lo.strict_capacity_limit = true;
+ std::shared_ptr<Cache> cache(NewLRUCache(lo));
+ table_options.block_cache = cache;
+ table_options.flush_block_policy_factory =
+ std::make_shared<FlushBlockEveryKeyPolicyFactory>();
+ table_options.cache_usage_options.options_overrides.insert(
+ {CacheEntryRole::kCompressionDictionaryBuildingBuffer,
+ {/*.charged = */ charge_compression_dictionary_building_buffer}});
+ Options options;
+ options.compression = kSnappyCompression;
+ options.compression_opts.max_dict_bytes = kMaxDictBytes;
+ options.compression_opts.max_dict_buffer_bytes = kMaxDictBufferBytes;
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+
+ test::StringSink* sink = new test::StringSink();
+ std::unique_ptr<FSWritableFile> holder(sink);
+ std::unique_ptr<WritableFileWriter> file_writer(new WritableFileWriter(
+ std::move(holder), "test_file_name", FileOptions()));
+
+ ImmutableOptions ioptions(options);
+ MutableCFOptions moptions(options);
+ InternalKeyComparator ikc(options.comparator);
+ IntTblPropCollectorFactories int_tbl_prop_collector_factories;
+
+ std::unique_ptr<TableBuilder> builder(
+ options.table_factory->NewTableBuilder(
+ TableBuilderOptions(
+ ioptions, moptions, ikc, &int_tbl_prop_collector_factories,
+ kSnappyCompression, options.compression_opts,
+ kUnknownColumnFamily, "test_cf", -1 /* level */),
+ file_writer.get()));
+
+ std::string key1 = "key1";
+ std::string value1 = "val1";
+ InternalKey ik1(key1, 0 /* sequnce number */, kTypeValue);
+ // Adding the first key won't trigger a flush by FlushBlockEveryKeyPolicy
+ // therefore won't trigger any data block's buffering
+ builder->Add(ik1.Encode(), value1);
+ ASSERT_EQ(cache->GetPinnedUsage(), 0 * kSizeDummyEntry);
+
+ std::string key2 = "key2";
+ std::string value2 = "val2";
+ InternalKey ik2(key2, 1 /* sequnce number */, kTypeValue);
+ // Adding the second key will trigger a flush of the last data block (the
+ // one containing key1 and value1) by FlushBlockEveryKeyPolicy and hence
+ // trigger buffering of that data block.
+ builder->Add(ik2.Encode(), value2);
+ // Cache charging will increase for last buffered data block (the one
+ // containing key1 and value1) since the buffer limit is not exceeded after
+ // that buffering and the cache will not be full after this reservation
+ if (charge_compression_dictionary_building_buffer ==
+ CacheEntryRoleOptions::Decision::kEnabled) {
+ EXPECT_GE(cache->GetPinnedUsage(), 1 * kSizeDummyEntry);
+ EXPECT_LT(cache->GetPinnedUsage(),
+ 1 * kSizeDummyEntry + kMetaDataChargeOverhead);
+ } else {
+ EXPECT_EQ(cache->GetPinnedUsage(), 0 * kSizeDummyEntry);
+ }
+
+ ASSERT_OK(builder->Finish());
+ EXPECT_EQ(cache->GetPinnedUsage(), 0 * kSizeDummyEntry);
+ }
+}
+
+TEST_F(ChargeCompressionDictionaryBuildingBufferTest,
+ BasicWithBufferLimitExceed) {
+ constexpr std::size_t kSizeDummyEntry = 256 * 1024;
+ constexpr std::size_t kMetaDataChargeOverhead = 10000;
+ constexpr std::size_t kCacheCapacity = 8 * 1024 * 1024;
+ constexpr std::size_t kMaxDictBytes = 1024;
+ constexpr std::size_t kMaxDictBufferBytes = 2 * kSizeDummyEntry;
+
+ // `CacheEntryRoleOptions::charged` is enabled by default for
+ // CacheEntryRole::kCompressionDictionaryBuildingBuffer
+ BlockBasedTableOptions table_options;
+ LRUCacheOptions lo;
+ lo.capacity = kCacheCapacity;
+ lo.num_shard_bits = 0; // 2^0 shard
+ lo.strict_capacity_limit = true;
+ std::shared_ptr<Cache> cache(NewLRUCache(lo));
+ table_options.block_cache = cache;
+ table_options.flush_block_policy_factory =
+ std::make_shared<FlushBlockEveryKeyPolicyFactory>();
+
+ Options options;
+ options.compression = kSnappyCompression;
+ options.compression_opts.max_dict_bytes = kMaxDictBytes;
+ options.compression_opts.max_dict_buffer_bytes = kMaxDictBufferBytes;
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+
+ test::StringSink* sink = new test::StringSink();
+ std::unique_ptr<FSWritableFile> holder(sink);
+ std::unique_ptr<WritableFileWriter> file_writer(new WritableFileWriter(
+ std::move(holder), "test_file_name", FileOptions()));
+
+ ImmutableOptions ioptions(options);
+ MutableCFOptions moptions(options);
+ InternalKeyComparator ikc(options.comparator);
+ IntTblPropCollectorFactories int_tbl_prop_collector_factories;
+
+ std::unique_ptr<TableBuilder> builder(options.table_factory->NewTableBuilder(
+ TableBuilderOptions(ioptions, moptions, ikc,
+ &int_tbl_prop_collector_factories, kSnappyCompression,
+ options.compression_opts, kUnknownColumnFamily,
+ "test_cf", -1 /* level */),
+ file_writer.get()));
+
+ std::string key1 = "key1";
+ std::string value1(kSizeDummyEntry, '0');
+ InternalKey ik1(key1, 0 /* sequnce number */, kTypeValue);
+ // Adding the first key won't trigger a flush by FlushBlockEveryKeyPolicy
+ // therefore won't trigger any data block's buffering
+ builder->Add(ik1.Encode(), value1);
+ ASSERT_EQ(cache->GetPinnedUsage(), 0 * kSizeDummyEntry);
+
+ std::string key2 = "key2";
+ std::string value2(kSizeDummyEntry, '0');
+ InternalKey ik2(key2, 1 /* sequnce number */, kTypeValue);
+ // Adding the second key will trigger a flush of the last data block (the one
+ // containing key1 and value1) by FlushBlockEveryKeyPolicy and hence trigger
+ // buffering of the last data block.
+ builder->Add(ik2.Encode(), value2);
+ // Cache charging will increase for last buffered data block (the one
+ // containing key1 and value1) since the buffer limit is not exceeded after
+ // the buffering and the cache will not be full after this reservation
+ EXPECT_GE(cache->GetPinnedUsage(), 2 * kSizeDummyEntry);
+ EXPECT_LT(cache->GetPinnedUsage(),
+ 2 * kSizeDummyEntry + kMetaDataChargeOverhead);
+
+ std::string key3 = "key3";
+ std::string value3 = "val3";
+ InternalKey ik3(key3, 2 /* sequnce number */, kTypeValue);
+ // Adding the third key will trigger a flush of the last data block (the one
+ // containing key2 and value2) by FlushBlockEveryKeyPolicy and hence trigger
+ // buffering of the last data block.
+ builder->Add(ik3.Encode(), value3);
+ // Cache charging will decrease since the buffer limit is now exceeded
+ // after the last buffering and EnterUnbuffered() is triggered
+ EXPECT_EQ(cache->GetPinnedUsage(), 0 * kSizeDummyEntry);
+
+ ASSERT_OK(builder->Finish());
+ EXPECT_EQ(cache->GetPinnedUsage(), 0 * kSizeDummyEntry);
+}
+
+TEST_F(ChargeCompressionDictionaryBuildingBufferTest, BasicWithCacheFull) {
+ constexpr std::size_t kSizeDummyEntry = 256 * 1024;
+ constexpr std::size_t kMetaDataChargeOverhead = 10000;
+ // A small kCacheCapacity is chosen so that increase cache charging for
+ // buffering two data blocks, each containing key1/value1, key2/a big
+ // value2, will cause cache full
+ constexpr std::size_t kCacheCapacity =
+ 1 * kSizeDummyEntry + kSizeDummyEntry / 2;
+ constexpr std::size_t kMaxDictBytes = 1024;
+ // A big kMaxDictBufferBytes is chosen so that adding a big key value pair
+ // (key2, value2) won't exceed the buffer limit
+ constexpr std::size_t kMaxDictBufferBytes = 1024 * 1024 * 1024;
+
+ // `CacheEntryRoleOptions::charged` is enabled by default for
+ // CacheEntryRole::kCompressionDictionaryBuildingBuffer
+ BlockBasedTableOptions table_options;
+ LRUCacheOptions lo;
+ lo.capacity = kCacheCapacity;
+ lo.num_shard_bits = 0; // 2^0 shard
+ lo.strict_capacity_limit = true;
+ std::shared_ptr<Cache> cache(NewLRUCache(lo));
+ table_options.block_cache = cache;
+ table_options.flush_block_policy_factory =
+ std::make_shared<FlushBlockEveryKeyPolicyFactory>();
+
+ Options options;
+ options.compression = kSnappyCompression;
+ options.compression_opts.max_dict_bytes = kMaxDictBytes;
+ options.compression_opts.max_dict_buffer_bytes = kMaxDictBufferBytes;
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+
+ test::StringSink* sink = new test::StringSink();
+ std::unique_ptr<FSWritableFile> holder(sink);
+ std::unique_ptr<WritableFileWriter> file_writer(new WritableFileWriter(
+ std::move(holder), "test_file_name", FileOptions()));
+
+ ImmutableOptions ioptions(options);
+ MutableCFOptions moptions(options);
+ InternalKeyComparator ikc(options.comparator);
+ IntTblPropCollectorFactories int_tbl_prop_collector_factories;
+
+ std::unique_ptr<TableBuilder> builder(options.table_factory->NewTableBuilder(
+ TableBuilderOptions(ioptions, moptions, ikc,
+ &int_tbl_prop_collector_factories, kSnappyCompression,
+ options.compression_opts, kUnknownColumnFamily,
+ "test_cf", -1 /* level */),
+ file_writer.get()));
+
+ std::string key1 = "key1";
+ std::string value1 = "val1";
+ InternalKey ik1(key1, 0 /* sequnce number */, kTypeValue);
+ // Adding the first key won't trigger a flush by FlushBlockEveryKeyPolicy
+ // therefore won't trigger any data block's buffering
+ builder->Add(ik1.Encode(), value1);
+ ASSERT_EQ(cache->GetPinnedUsage(), 0 * kSizeDummyEntry);
+
+ std::string key2 = "key2";
+ std::string value2(kSizeDummyEntry, '0');
+ InternalKey ik2(key2, 1 /* sequnce number */, kTypeValue);
+ // Adding the second key will trigger a flush of the last data block (the one
+ // containing key1 and value1) by FlushBlockEveryKeyPolicy and hence trigger
+ // buffering of the last data block.
+ builder->Add(ik2.Encode(), value2);
+ // Cache charging will increase for the last buffered data block (the one
+ // containing key1 and value1) since the buffer limit is not exceeded after
+ // the buffering and the cache will not be full after this reservation
+ EXPECT_GE(cache->GetPinnedUsage(), 1 * kSizeDummyEntry);
+ EXPECT_LT(cache->GetPinnedUsage(),
+ 1 * kSizeDummyEntry + kMetaDataChargeOverhead);
+
+ std::string key3 = "key3";
+ std::string value3 = "value3";
+ InternalKey ik3(key3, 2 /* sequnce number */, kTypeValue);
+ // Adding the third key will trigger a flush of the last data block (the one
+ // containing key2 and value2) by FlushBlockEveryKeyPolicy and hence trigger
+ // buffering of the last data block.
+ builder->Add(ik3.Encode(), value3);
+ // Cache charging will decrease since the cache is now full after
+ // increasing reservation for the last buffered block and EnterUnbuffered() is
+ // triggered
+ EXPECT_EQ(cache->GetPinnedUsage(), 0 * kSizeDummyEntry);
+
+ ASSERT_OK(builder->Finish());
+ EXPECT_EQ(cache->GetPinnedUsage(), 0 * kSizeDummyEntry);
+}
+
+class CacheUsageOptionsOverridesTest : public DBTestBase {
+ public:
+ CacheUsageOptionsOverridesTest()
+ : DBTestBase("cache_usage_options_overrides_test",
+ /*env_do_fsync=*/false) {}
+};
+
+TEST_F(CacheUsageOptionsOverridesTest, SanitizeAndValidateOptions) {
+ // To test `cache_usage_options.options_overrides` is sanitized
+ // where `cache_usage_options.options` is used when there is no entry in
+ // `cache_usage_options.options_overrides`
+ Options options;
+ options.create_if_missing = true;
+ BlockBasedTableOptions table_options = BlockBasedTableOptions();
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+ Destroy(options);
+ Status s = TryReopen(options);
+ EXPECT_TRUE(s.ok());
+ const auto* sanitized_table_options =
+ options.table_factory->GetOptions<BlockBasedTableOptions>();
+ const auto sanitized_options_overrides =
+ sanitized_table_options->cache_usage_options.options_overrides;
+ EXPECT_EQ(sanitized_options_overrides.size(), kNumCacheEntryRoles);
+ for (auto options_overrides_iter = sanitized_options_overrides.cbegin();
+ options_overrides_iter != sanitized_options_overrides.cend();
+ ++options_overrides_iter) {
+ CacheEntryRoleOptions role_options = options_overrides_iter->second;
+ CacheEntryRoleOptions default_options =
+ sanitized_table_options->cache_usage_options.options;
+ EXPECT_TRUE(role_options == default_options);
+ }
+ Destroy(options);
+
+ // To test option validation on unsupported CacheEntryRole
+ table_options = BlockBasedTableOptions();
+ table_options.cache_usage_options.options_overrides.insert(
+ {CacheEntryRole::kDataBlock,
+ {/*.charged = */ CacheEntryRoleOptions::Decision::kDisabled}});
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+ Destroy(options);
+ s = TryReopen(options);
+ EXPECT_TRUE(s.IsNotSupported());
+ EXPECT_TRUE(
+ s.ToString().find("Enable/Disable CacheEntryRoleOptions::charged") !=
+ std::string::npos);
+ EXPECT_TRUE(
+ s.ToString().find(kCacheEntryRoleToCamelString[static_cast<uint32_t>(
+ CacheEntryRole::kDataBlock)]) != std::string::npos);
+ Destroy(options);
+
+ // To test option validation on existence of block cache
+ table_options = BlockBasedTableOptions();
+ table_options.no_block_cache = true;
+ table_options.cache_usage_options.options_overrides.insert(
+ {CacheEntryRole::kFilterConstruction,
+ {/*.charged = */ CacheEntryRoleOptions::Decision::kEnabled}});
+ options.table_factory.reset(NewBlockBasedTableFactory(table_options));
+ Destroy(options);
+ s = TryReopen(options);
+ EXPECT_TRUE(s.IsInvalidArgument());
+ EXPECT_TRUE(s.ToString().find("Enable CacheEntryRoleOptions::charged") !=
+ std::string::npos);
+ EXPECT_TRUE(
+ s.ToString().find(kCacheEntryRoleToCamelString[static_cast<std::size_t>(
+ CacheEntryRole::kFilterConstruction)]) != std::string::npos);
+ EXPECT_TRUE(s.ToString().find("block cache is disabled") !=
+ std::string::npos);
+ Destroy(options);
+}
+} // namespace ROCKSDB_NAMESPACE
+
+int main(int argc, char** argv) {
+ ROCKSDB_NAMESPACE::port::InstallStackTraceHandler();
+ ::testing::InitGoogleTest(&argc, argv);
+ return RUN_ALL_TESTS();
+}
diff --git a/src/rocksdb/table/two_level_iterator.cc b/src/rocksdb/table/two_level_iterator.cc
new file mode 100644
index 000000000..4b6634e5c
--- /dev/null
+++ b/src/rocksdb/table/two_level_iterator.cc
@@ -0,0 +1,220 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#include "table/two_level_iterator.h"
+
+#include "db/pinned_iterators_manager.h"
+#include "memory/arena.h"
+#include "rocksdb/options.h"
+#include "rocksdb/table.h"
+#include "table/block_based/block.h"
+#include "table/format.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+namespace {
+
+class TwoLevelIndexIterator : public InternalIteratorBase<IndexValue> {
+ public:
+ explicit TwoLevelIndexIterator(
+ TwoLevelIteratorState* state,
+ InternalIteratorBase<IndexValue>* first_level_iter);
+
+ ~TwoLevelIndexIterator() override {
+ first_level_iter_.DeleteIter(false /* is_arena_mode */);
+ second_level_iter_.DeleteIter(false /* is_arena_mode */);
+ delete state_;
+ }
+
+ void Seek(const Slice& target) override;
+ void SeekForPrev(const Slice& target) override;
+ void SeekToFirst() override;
+ void SeekToLast() override;
+ void Next() override;
+ void Prev() override;
+
+ bool Valid() const override { return second_level_iter_.Valid(); }
+ Slice key() const override {
+ assert(Valid());
+ return second_level_iter_.key();
+ }
+ Slice user_key() const override {
+ assert(Valid());
+ return second_level_iter_.user_key();
+ }
+ IndexValue value() const override {
+ assert(Valid());
+ return second_level_iter_.value();
+ }
+ Status status() const override {
+ if (!first_level_iter_.status().ok()) {
+ assert(second_level_iter_.iter() == nullptr);
+ return first_level_iter_.status();
+ } else if (second_level_iter_.iter() != nullptr &&
+ !second_level_iter_.status().ok()) {
+ return second_level_iter_.status();
+ } else {
+ return status_;
+ }
+ }
+ void SetPinnedItersMgr(
+ PinnedIteratorsManager* /*pinned_iters_mgr*/) override {}
+ bool IsKeyPinned() const override { return false; }
+ bool IsValuePinned() const override { return false; }
+
+ private:
+ void SaveError(const Status& s) {
+ if (status_.ok() && !s.ok()) status_ = s;
+ }
+ void SkipEmptyDataBlocksForward();
+ void SkipEmptyDataBlocksBackward();
+ void SetSecondLevelIterator(InternalIteratorBase<IndexValue>* iter);
+ void InitDataBlock();
+
+ TwoLevelIteratorState* state_;
+ IteratorWrapperBase<IndexValue> first_level_iter_;
+ IteratorWrapperBase<IndexValue> second_level_iter_; // May be nullptr
+ Status status_;
+ // If second_level_iter is non-nullptr, then "data_block_handle_" holds the
+ // "index_value" passed to block_function_ to create the second_level_iter.
+ BlockHandle data_block_handle_;
+};
+
+TwoLevelIndexIterator::TwoLevelIndexIterator(
+ TwoLevelIteratorState* state,
+ InternalIteratorBase<IndexValue>* first_level_iter)
+ : state_(state), first_level_iter_(first_level_iter) {}
+
+void TwoLevelIndexIterator::Seek(const Slice& target) {
+ first_level_iter_.Seek(target);
+
+ InitDataBlock();
+ if (second_level_iter_.iter() != nullptr) {
+ second_level_iter_.Seek(target);
+ }
+ SkipEmptyDataBlocksForward();
+}
+
+void TwoLevelIndexIterator::SeekForPrev(const Slice& target) {
+ first_level_iter_.Seek(target);
+ InitDataBlock();
+ if (second_level_iter_.iter() != nullptr) {
+ second_level_iter_.SeekForPrev(target);
+ }
+ if (!Valid()) {
+ if (!first_level_iter_.Valid() && first_level_iter_.status().ok()) {
+ first_level_iter_.SeekToLast();
+ InitDataBlock();
+ if (second_level_iter_.iter() != nullptr) {
+ second_level_iter_.SeekForPrev(target);
+ }
+ }
+ SkipEmptyDataBlocksBackward();
+ }
+}
+
+void TwoLevelIndexIterator::SeekToFirst() {
+ first_level_iter_.SeekToFirst();
+ InitDataBlock();
+ if (second_level_iter_.iter() != nullptr) {
+ second_level_iter_.SeekToFirst();
+ }
+ SkipEmptyDataBlocksForward();
+}
+
+void TwoLevelIndexIterator::SeekToLast() {
+ first_level_iter_.SeekToLast();
+ InitDataBlock();
+ if (second_level_iter_.iter() != nullptr) {
+ second_level_iter_.SeekToLast();
+ }
+ SkipEmptyDataBlocksBackward();
+}
+
+void TwoLevelIndexIterator::Next() {
+ assert(Valid());
+ second_level_iter_.Next();
+ SkipEmptyDataBlocksForward();
+}
+
+void TwoLevelIndexIterator::Prev() {
+ assert(Valid());
+ second_level_iter_.Prev();
+ SkipEmptyDataBlocksBackward();
+}
+
+void TwoLevelIndexIterator::SkipEmptyDataBlocksForward() {
+ while (second_level_iter_.iter() == nullptr ||
+ (!second_level_iter_.Valid() && second_level_iter_.status().ok())) {
+ // Move to next block
+ if (!first_level_iter_.Valid()) {
+ SetSecondLevelIterator(nullptr);
+ return;
+ }
+ first_level_iter_.Next();
+ InitDataBlock();
+ if (second_level_iter_.iter() != nullptr) {
+ second_level_iter_.SeekToFirst();
+ }
+ }
+}
+
+void TwoLevelIndexIterator::SkipEmptyDataBlocksBackward() {
+ while (second_level_iter_.iter() == nullptr ||
+ (!second_level_iter_.Valid() && second_level_iter_.status().ok())) {
+ // Move to next block
+ if (!first_level_iter_.Valid()) {
+ SetSecondLevelIterator(nullptr);
+ return;
+ }
+ first_level_iter_.Prev();
+ InitDataBlock();
+ if (second_level_iter_.iter() != nullptr) {
+ second_level_iter_.SeekToLast();
+ }
+ }
+}
+
+void TwoLevelIndexIterator::SetSecondLevelIterator(
+ InternalIteratorBase<IndexValue>* iter) {
+ InternalIteratorBase<IndexValue>* old_iter = second_level_iter_.Set(iter);
+ delete old_iter;
+}
+
+void TwoLevelIndexIterator::InitDataBlock() {
+ if (!first_level_iter_.Valid()) {
+ SetSecondLevelIterator(nullptr);
+ } else {
+ BlockHandle handle = first_level_iter_.value().handle;
+ if (second_level_iter_.iter() != nullptr &&
+ !second_level_iter_.status().IsIncomplete() &&
+ handle.offset() == data_block_handle_.offset()) {
+ // second_level_iter is already constructed with this iterator, so
+ // no need to change anything
+ } else {
+ InternalIteratorBase<IndexValue>* iter =
+ state_->NewSecondaryIterator(handle);
+ data_block_handle_ = handle;
+ SetSecondLevelIterator(iter);
+ if (iter == nullptr) {
+ status_ = Status::Corruption("Missing block for partition " +
+ handle.ToString());
+ }
+ }
+ }
+}
+
+} // namespace
+
+InternalIteratorBase<IndexValue>* NewTwoLevelIterator(
+ TwoLevelIteratorState* state,
+ InternalIteratorBase<IndexValue>* first_level_iter) {
+ return new TwoLevelIndexIterator(state, first_level_iter);
+}
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/two_level_iterator.h b/src/rocksdb/table/two_level_iterator.h
new file mode 100644
index 000000000..1fed93417
--- /dev/null
+++ b/src/rocksdb/table/two_level_iterator.h
@@ -0,0 +1,43 @@
+// Copyright (c) 2011-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).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors.
+
+#pragma once
+#include "rocksdb/env.h"
+#include "rocksdb/iterator.h"
+#include "table/iterator_wrapper.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+struct ReadOptions;
+class InternalKeyComparator;
+
+// TwoLevelIteratorState expects iterators are not created using the arena
+struct TwoLevelIteratorState {
+ TwoLevelIteratorState() {}
+
+ virtual ~TwoLevelIteratorState() {}
+ virtual InternalIteratorBase<IndexValue>* NewSecondaryIterator(
+ const BlockHandle& handle) = 0;
+};
+
+// Return a new two level iterator. A two-level iterator contains an
+// index iterator whose values point to a sequence of blocks where
+// each block is itself a sequence of key,value pairs. The returned
+// two-level iterator yields the concatenation of all key/value pairs
+// in the sequence of blocks. Takes ownership of "index_iter" and
+// will delete it when no longer needed.
+//
+// Uses a supplied function to convert an index_iter value into
+// an iterator over the contents of the corresponding block.
+// Note: this function expects first_level_iter was not created using the arena
+extern InternalIteratorBase<IndexValue>* NewTwoLevelIterator(
+ TwoLevelIteratorState* state,
+ InternalIteratorBase<IndexValue>* first_level_iter);
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/unique_id.cc b/src/rocksdb/table/unique_id.cc
new file mode 100644
index 000000000..fcdd75650
--- /dev/null
+++ b/src/rocksdb/table/unique_id.cc
@@ -0,0 +1,223 @@
+// Copyright (c) Facebook, Inc. and its affiliates. 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 <cstdint>
+
+#include "table/unique_id_impl.h"
+#include "util/coding_lean.h"
+#include "util/hash.h"
+#include "util/string_util.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+std::string EncodeSessionId(uint64_t upper, uint64_t lower) {
+ std::string db_session_id(20U, '\0');
+ char *buf = &db_session_id[0];
+ // Preserving `lower` is slightly tricky. 36^12 is slightly more than
+ // 62 bits, so we use 12 chars plus the bottom two bits of one more.
+ // (A tiny fraction of 20 digit strings go unused.)
+ uint64_t a = (upper << 2) | (lower >> 62);
+ uint64_t b = lower & (UINT64_MAX >> 2);
+ PutBaseChars<36>(&buf, 8, a, /*uppercase*/ true);
+ PutBaseChars<36>(&buf, 12, b, /*uppercase*/ true);
+ assert(buf == &db_session_id.back() + 1);
+ return db_session_id;
+}
+
+Status DecodeSessionId(const std::string &db_session_id, uint64_t *upper,
+ uint64_t *lower) {
+ const size_t len = db_session_id.size();
+ if (len == 0) {
+ return Status::NotSupported("Missing db_session_id");
+ }
+ // Anything from 13 to 24 chars is reasonable. We don't have to limit to
+ // exactly 20.
+ if (len < 13) {
+ return Status::NotSupported("Too short db_session_id");
+ }
+ if (len > 24) {
+ return Status::NotSupported("Too long db_session_id");
+ }
+ uint64_t a = 0, b = 0;
+ const char *buf = &db_session_id.front();
+ bool success = ParseBaseChars<36>(&buf, len - 12U, &a);
+ if (!success) {
+ return Status::NotSupported("Bad digit in db_session_id");
+ }
+ success = ParseBaseChars<36>(&buf, 12U, &b);
+ if (!success) {
+ return Status::NotSupported("Bad digit in db_session_id");
+ }
+ assert(buf == &db_session_id.back() + 1);
+ *upper = a >> 2;
+ *lower = (b & (UINT64_MAX >> 2)) | (a << 62);
+ return Status::OK();
+}
+
+Status GetSstInternalUniqueId(const std::string &db_id,
+ const std::string &db_session_id,
+ uint64_t file_number, UniqueIdPtr out,
+ bool force) {
+ if (!force) {
+ if (db_id.empty()) {
+ return Status::NotSupported("Missing db_id");
+ }
+ if (file_number == 0) {
+ return Status::NotSupported("Missing or bad file number");
+ }
+ if (db_session_id.empty()) {
+ return Status::NotSupported("Missing db_session_id");
+ }
+ }
+ uint64_t session_upper = 0; // Assignment to appease clang-analyze
+ uint64_t session_lower = 0; // Assignment to appease clang-analyze
+ {
+ Status s = DecodeSessionId(db_session_id, &session_upper, &session_lower);
+ if (!s.ok()) {
+ if (!force) {
+ return s;
+ } else {
+ // A reasonable fallback in case malformed
+ Hash2x64(db_session_id.data(), db_session_id.size(), &session_upper,
+ &session_lower);
+ if (session_lower == 0) {
+ session_lower = session_upper | 1;
+ }
+ }
+ }
+ }
+
+ // Exactly preserve session lower to ensure that session ids generated
+ // during the same process lifetime are guaranteed unique.
+ // DBImpl also guarantees (in recent versions) that this is not zero,
+ // so that we can guarantee unique ID is never all zeros. (Can't assert
+ // that here because of testing and old versions.)
+ // We put this first in anticipation of matching a small-ish set of cache
+ // key prefixes to cover entries relevant to any DB.
+ out.ptr[0] = session_lower;
+
+ // Hash the session upper (~39 bits entropy) and DB id (120+ bits entropy)
+ // for very high global uniqueness entropy.
+ // (It is possible that many DBs descended from one common DB id are copied
+ // around and proliferate, in which case session id is critical, but it is
+ // more common for different DBs to have different DB ids.)
+ uint64_t db_a, db_b;
+ Hash2x64(db_id.data(), db_id.size(), session_upper, &db_a, &db_b);
+
+ // Xor in file number for guaranteed uniqueness by file number for a given
+ // session and DB id. (Xor slightly better than + here. See
+ // https://github.com/pdillinger/unique_id )
+ out.ptr[1] = db_a ^ file_number;
+
+ // Extra (optional) global uniqueness
+ if (out.extended) {
+ out.ptr[2] = db_b;
+ }
+
+ return Status::OK();
+}
+
+namespace {
+// For InternalUniqueIdToExternal / ExternalUniqueIdToInternal we want all
+// zeros in first 128 bits to map to itself, so that excluding zero in
+// internal IDs (session_lower != 0 above) does the same for external IDs.
+// These values are meaningless except for making that work.
+constexpr uint64_t kHiOffsetForZero = 17391078804906429400U;
+constexpr uint64_t kLoOffsetForZero = 6417269962128484497U;
+} // namespace
+
+void InternalUniqueIdToExternal(UniqueIdPtr in_out) {
+ uint64_t hi, lo;
+ BijectiveHash2x64(in_out.ptr[1] + kHiOffsetForZero,
+ in_out.ptr[0] + kLoOffsetForZero, &hi, &lo);
+ in_out.ptr[0] = lo;
+ in_out.ptr[1] = hi;
+ if (in_out.extended) {
+ in_out.ptr[2] += lo + hi;
+ }
+}
+
+void ExternalUniqueIdToInternal(UniqueIdPtr in_out) {
+ uint64_t lo = in_out.ptr[0];
+ uint64_t hi = in_out.ptr[1];
+ if (in_out.extended) {
+ in_out.ptr[2] -= lo + hi;
+ }
+ BijectiveUnhash2x64(hi, lo, &hi, &lo);
+ in_out.ptr[0] = lo - kLoOffsetForZero;
+ in_out.ptr[1] = hi - kHiOffsetForZero;
+}
+
+std::string EncodeUniqueIdBytes(UniqueIdPtr in) {
+ std::string ret(in.extended ? 24U : 16U, '\0');
+ EncodeFixed64(&ret[0], in.ptr[0]);
+ EncodeFixed64(&ret[8], in.ptr[1]);
+ if (in.extended) {
+ EncodeFixed64(&ret[16], in.ptr[2]);
+ }
+ return ret;
+}
+
+Status DecodeUniqueIdBytes(const std::string &unique_id, UniqueIdPtr out) {
+ if (unique_id.size() != (out.extended ? 24 : 16)) {
+ return Status::NotSupported("Not a valid unique_id");
+ }
+ const char *buf = &unique_id.front();
+ out.ptr[0] = DecodeFixed64(&buf[0]);
+ out.ptr[1] = DecodeFixed64(&buf[8]);
+ if (out.extended) {
+ out.ptr[2] = DecodeFixed64(&buf[16]);
+ }
+ return Status::OK();
+}
+
+template <typename ID>
+Status GetUniqueIdFromTablePropertiesHelper(const TableProperties &props,
+ std::string *out_id) {
+ ID tmp{};
+ Status s = GetSstInternalUniqueId(props.db_id, props.db_session_id,
+ props.orig_file_number, &tmp);
+ if (s.ok()) {
+ InternalUniqueIdToExternal(&tmp);
+ *out_id = EncodeUniqueIdBytes(&tmp);
+ } else {
+ out_id->clear();
+ }
+ return s;
+}
+
+Status GetExtendedUniqueIdFromTableProperties(const TableProperties &props,
+ std::string *out_id) {
+ return GetUniqueIdFromTablePropertiesHelper<UniqueId64x3>(props, out_id);
+}
+
+Status GetUniqueIdFromTableProperties(const TableProperties &props,
+ std::string *out_id) {
+ return GetUniqueIdFromTablePropertiesHelper<UniqueId64x2>(props, out_id);
+}
+
+std::string UniqueIdToHumanString(const std::string &id) {
+ // Not so efficient, but that's OK
+ std::string str = Slice(id).ToString(/*hex*/ true);
+ for (size_t i = 16; i < str.size(); i += 17) {
+ str.insert(i, "-");
+ }
+ return str;
+}
+
+std::string InternalUniqueIdToHumanString(UniqueIdPtr in) {
+ std::string str = "{";
+ str += std::to_string(in.ptr[0]);
+ str += ",";
+ str += std::to_string(in.ptr[1]);
+ if (in.extended) {
+ str += ",";
+ str += std::to_string(in.ptr[2]);
+ }
+ str += "}";
+ return str;
+}
+
+} // namespace ROCKSDB_NAMESPACE
diff --git a/src/rocksdb/table/unique_id_impl.h b/src/rocksdb/table/unique_id_impl.h
new file mode 100644
index 000000000..6e3dc62c7
--- /dev/null
+++ b/src/rocksdb/table/unique_id_impl.h
@@ -0,0 +1,93 @@
+// Copyright (c) Facebook, Inc. and its affiliates. 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).
+
+#pragma once
+
+#include <array>
+
+#include "rocksdb/unique_id.h"
+
+namespace ROCKSDB_NAMESPACE {
+
+// Standard size unique ID, good enough for almost all practical purposes
+using UniqueId64x2 = std::array<uint64_t, 2>;
+
+// Value never used as an actual unique ID so can be used for "null"
+constexpr UniqueId64x2 kNullUniqueId64x2 = {};
+
+// Extended size unique ID, for extra certainty of uniqueness among SST files
+// spanning many hosts over a long time (rarely if ever needed)
+using UniqueId64x3 = std::array<uint64_t, 3>;
+
+// Value never used as an actual unique ID so can be used for "null"
+constexpr UniqueId64x3 kNullUniqueId64x3 = {};
+
+// Dynamic pointer wrapper for one of the two above
+struct UniqueIdPtr {
+ uint64_t *ptr = nullptr;
+ bool extended = false;
+
+ /*implicit*/ UniqueIdPtr(UniqueId64x2 *id) {
+ ptr = (*id).data();
+ extended = false;
+ }
+ /*implicit*/ UniqueIdPtr(UniqueId64x3 *id) {
+ ptr = (*id).data();
+ extended = true;
+ }
+};
+
+// Helper for GetUniqueIdFromTableProperties. This function can also be used
+// for temporary ids for files without sufficient information in table
+// properties. The internal unique id is more structured than the public
+// unique id, so can be manipulated in more ways but very carefully.
+// These must be long term stable to ensure GetUniqueIdFromTableProperties
+// is long term stable.
+Status GetSstInternalUniqueId(const std::string &db_id,
+ const std::string &db_session_id,
+ uint64_t file_number, UniqueIdPtr out,
+ bool force = false);
+
+// Helper for GetUniqueIdFromTableProperties. External unique ids go through
+// this extra hashing layer so that prefixes of the unique id have predictable
+// "full" entropy. This hashing layer is 1-to-1 on the first 128 bits and on
+// the full 192 bits.
+// This transformation must be long term stable to ensure
+// GetUniqueIdFromTableProperties is long term stable.
+void InternalUniqueIdToExternal(UniqueIdPtr in_out);
+
+// Reverse of InternalUniqueIdToExternal mostly for testing purposes
+// (demonstrably 1-to-1 on the first 128 bits and on the full 192 bits).
+void ExternalUniqueIdToInternal(UniqueIdPtr in_out);
+
+// Convert numerical format to byte format for public API
+std::string EncodeUniqueIdBytes(UniqueIdPtr in);
+
+// Reverse of EncodeUniqueIdBytes.
+Status DecodeUniqueIdBytes(const std::string &unique_id, UniqueIdPtr out);
+
+// For presenting internal IDs for debugging purposes. Visually distinct from
+// UniqueIdToHumanString for external IDs.
+std::string InternalUniqueIdToHumanString(UniqueIdPtr in);
+
+// Reformat a random value down to our "DB session id" format,
+// which is intended to be compact and friendly for use in file names.
+// `lower` is fully preserved and data is lost from `upper`.
+//
+// Detail: Encoded into 20 chars in base-36 ([0-9A-Z]), which is ~103 bits of
+// entropy, which is enough to expect no collisions across a billion servers
+// each opening DBs a million times (~2^50). Benefits vs. RFC-4122 unique id:
+// * Save ~ dozen bytes per SST file
+// * Shorter shared backup file names (some platforms have low limits)
+// * Visually distinct from DB id format (usually RFC-4122)
+std::string EncodeSessionId(uint64_t upper, uint64_t lower);
+
+// Reverse of EncodeSessionId. Returns NotSupported on error rather than
+// Corruption because non-standard session IDs should be allowed with degraded
+// functionality.
+Status DecodeSessionId(const std::string &db_session_id, uint64_t *upper,
+ uint64_t *lower);
+
+} // namespace ROCKSDB_NAMESPACE