First phase API clean up

Summary:
Addressed all the issues in https://reviews.facebook.net/D15447.
Now most table-related modules are hidden from user land.

Test Plan: make check

Reviewers: sdong, haobo, dhruba

CC: leveldb

Differential Revision: https://reviews.facebook.net/D15525
main
kailiu 11 years ago
parent 4e0298f23c
commit 4f6cb17bdb
  1. 11
      db/builder.cc
  2. 3
      db/builder.h
  3. 6
      db/db_impl.cc
  4. 2
      db/db_test.cc
  5. 2
      db/plain_table_db_test.cc
  6. 22
      db/simple_table_db_test.cc
  7. 7
      db/table_cache.cc
  8. 8
      db/table_cache.h
  9. 11
      db/table_properties_collector_test.cc
  10. 3
      db/version_set.cc
  11. 212
      include/rocksdb/table.h
  12. 18
      include/rocksdb/table_properties.h
  13. 2
      table/block_based_table_builder.cc
  14. 2
      table/block_based_table_builder.h
  15. 9
      table/block_based_table_factory.cc
  16. 18
      table/block_based_table_factory.h
  17. 31
      table/block_based_table_options.h
  18. 3
      table/block_based_table_reader.cc
  19. 11
      table/block_based_table_reader.h
  20. 2
      table/meta_blocks.cc
  21. 2
      table/plain_table_builder.h
  22. 21
      table/plain_table_factory.cc
  23. 48
      table/plain_table_factory.h
  24. 16
      table/plain_table_reader.cc
  25. 22
      table/plain_table_reader.h
  26. 55
      table/table_builder.h
  27. 82
      table/table_factory.h
  28. 70
      table/table_reader.h
  29. 6
      table/table_reader_bench.cc
  30. 22
      table/table_test.cc
  31. 5
      tools/sst_dump.cc
  32. 3
      util/options.cc

@ -9,26 +9,26 @@
#include "db/builder.h" #include "db/builder.h"
#include "db/filename.h"
#include "db/dbformat.h" #include "db/dbformat.h"
#include "db/filename.h"
#include "db/merge_helper.h" #include "db/merge_helper.h"
#include "db/table_cache.h" #include "db/table_cache.h"
#include "db/version_edit.h" #include "db/version_edit.h"
#include "rocksdb/db.h" #include "rocksdb/db.h"
#include "rocksdb/table.h"
#include "rocksdb/env.h" #include "rocksdb/env.h"
#include "rocksdb/iterator.h" #include "rocksdb/iterator.h"
#include "rocksdb/options.h" #include "rocksdb/options.h"
#include "table/block_based_table_builder.h" #include "table/block_based_table_builder.h"
#include "table/table_factory.h"
#include "util/stop_watch.h" #include "util/stop_watch.h"
namespace rocksdb { namespace rocksdb {
class TableFactory; class TableFactory;
TableBuilder* GetTableBuilder(const Options& options, WritableFile* file, TableBuilder* NewTableBuilder(const Options& options, WritableFile* file,
CompressionType compression_type) { CompressionType compression_type) {
return options.table_factory->GetTableBuilder(options, file, return options.table_factory->NewTableBuilder(options, file,
compression_type); compression_type);
} }
@ -64,8 +64,7 @@ Status BuildTable(const std::string& dbname,
return s; return s;
} }
TableBuilder* builder = GetTableBuilder(options, file.get(), TableBuilder* builder = NewTableBuilder(options, file.get(), compression);
compression);
// the first key is the smallest key // the first key is the smallest key
Slice key = iter->key(); Slice key = iter->key();

@ -24,8 +24,7 @@ class VersionEdit;
class TableBuilder; class TableBuilder;
class WritableFile; class WritableFile;
extern TableBuilder* NewTableBuilder(const Options& options, WritableFile* file,
extern TableBuilder* GetTableBuilder(const Options& options, WritableFile* file,
CompressionType compression_type); CompressionType compression_type);
// Build a Table file from the contents of *iter. The generated file // Build a Table file from the contents of *iter. The generated file

@ -48,9 +48,10 @@
#include "table/block.h" #include "table/block.h"
#include "table/block_based_table_factory.h" #include "table/block_based_table_factory.h"
#include "table/merger.h" #include "table/merger.h"
#include "table/table_builder.h"
#include "table/two_level_iterator.h" #include "table/two_level_iterator.h"
#include "util/autovector.h"
#include "util/auto_roll_logger.h" #include "util/auto_roll_logger.h"
#include "util/autovector.h"
#include "util/build_version.h" #include "util/build_version.h"
#include "util/coding.h" #include "util/coding.h"
#include "util/hash_skiplist_rep.h" #include "util/hash_skiplist_rep.h"
@ -58,7 +59,6 @@
#include "util/mutexlock.h" #include "util/mutexlock.h"
#include "util/perf_context_imp.h" #include "util/perf_context_imp.h"
#include "util/stop_watch.h" #include "util/stop_watch.h"
#include "util/autovector.h"
namespace rocksdb { namespace rocksdb {
@ -2138,7 +2138,7 @@ Status DBImpl::OpenCompactionOutputFile(CompactionState* compact) {
compact->compaction->enable_compression()); compact->compaction->enable_compression());
compact->builder.reset( compact->builder.reset(
GetTableBuilder(options_, compact->outfile.get(), compression_type)); NewTableBuilder(options_, compact->outfile.get(), compression_type));
} }
LogFlush(options_.info_log); LogFlush(options_.info_log);
return s; return s;

@ -21,7 +21,7 @@
#include "rocksdb/env.h" #include "rocksdb/env.h"
#include "rocksdb/filter_policy.h" #include "rocksdb/filter_policy.h"
#include "rocksdb/perf_context.h" #include "rocksdb/perf_context.h"
#include "rocksdb/plain_table_factory.h" #include "table/plain_table_factory.h"
#include "rocksdb/slice.h" #include "rocksdb/slice.h"
#include "rocksdb/slice_transform.h" #include "rocksdb/slice_transform.h"
#include "rocksdb/table.h" #include "rocksdb/table.h"

@ -20,9 +20,9 @@
#include "rocksdb/db.h" #include "rocksdb/db.h"
#include "rocksdb/env.h" #include "rocksdb/env.h"
#include "rocksdb/filter_policy.h" #include "rocksdb/filter_policy.h"
#include "rocksdb/plain_table_factory.h"
#include "rocksdb/slice_transform.h" #include "rocksdb/slice_transform.h"
#include "rocksdb/table.h" #include "rocksdb/table.h"
#include "table/plain_table_factory.h"
#include "util/hash.h" #include "util/hash.h"
#include "util/logging.h" #include "util/logging.h"
#include "util/mutexlock.h" #include "util/mutexlock.h"

@ -22,6 +22,8 @@
#include "rocksdb/compaction_filter.h" #include "rocksdb/compaction_filter.h"
#include "rocksdb/env.h" #include "rocksdb/env.h"
#include "rocksdb/table.h" #include "rocksdb/table.h"
#include "rocksdb/table_properties.h"
#include "table/table_builder.h"
#include "util/hash.h" #include "util/hash.h"
#include "util/logging.h" #include "util/logging.h"
#include "util/mutexlock.h" #include "util/mutexlock.h"
@ -92,8 +94,6 @@ public:
uint64_t ApproximateOffsetOf(const Slice& key) override; uint64_t ApproximateOffsetOf(const Slice& key) override;
bool TEST_KeyInCache(const ReadOptions& options, const Slice& key) override;
void SetupForCompaction() override; void SetupForCompaction() override;
TableProperties& GetTableProperties() override; TableProperties& GetTableProperties() override;
@ -296,11 +296,6 @@ Status SimpleTableReader::Get(
return s; return s;
} }
bool SimpleTableReader::TEST_KeyInCache(const ReadOptions& options,
const Slice& key) {
return false;
}
uint64_t SimpleTableReader::ApproximateOffsetOf(const Slice& key) { uint64_t SimpleTableReader::ApproximateOffsetOf(const Slice& key) {
return 0; return 0;
} }
@ -541,25 +536,24 @@ public:
const char* Name() const override { const char* Name() const override {
return "SimpleTable"; return "SimpleTable";
} }
Status GetTableReader(const Options& options, const EnvOptions& soptions, Status NewTableReader(const Options& options, const EnvOptions& soptions,
unique_ptr<RandomAccessFile> && file, unique_ptr<RandomAccessFile>&& file, uint64_t file_size,
uint64_t file_size,
unique_ptr<TableReader>* table_reader) const; unique_ptr<TableReader>* table_reader) const;
TableBuilder* GetTableBuilder(const Options& options, WritableFile* file, TableBuilder* NewTableBuilder(const Options& options, WritableFile* file,
CompressionType compression_type) const; CompressionType compression_type) const;
}; };
Status SimpleTableFactory::GetTableReader( Status SimpleTableFactory::NewTableReader(
const Options& options, const EnvOptions& soptions, const Options& options, const EnvOptions& soptions,
unique_ptr<RandomAccessFile> && file, uint64_t file_size, unique_ptr<RandomAccessFile>&& file, uint64_t file_size,
unique_ptr<TableReader>* table_reader) const { unique_ptr<TableReader>* table_reader) const {
return SimpleTableReader::Open(options, soptions, std::move(file), file_size, return SimpleTableReader::Open(options, soptions, std::move(file), file_size,
table_reader); table_reader);
} }
TableBuilder* SimpleTableFactory::GetTableBuilder( TableBuilder* SimpleTableFactory::NewTableBuilder(
const Options& options, WritableFile* file, const Options& options, WritableFile* file,
CompressionType compression_type) const { CompressionType compression_type) const {
return new SimpleTableBuilder(options, file, compression_type); return new SimpleTableBuilder(options, file, compression_type);

@ -13,7 +13,7 @@
#include "db/version_edit.h" #include "db/version_edit.h"
#include "rocksdb/statistics.h" #include "rocksdb/statistics.h"
#include "rocksdb/table.h" #include "table/table_reader.h"
#include "util/coding.h" #include "util/coding.h"
#include "util/stop_watch.h" #include "util/stop_watch.h"
@ -83,9 +83,8 @@ Status TableCache::FindTable(const EnvOptions& toptions,
file->Hint(RandomAccessFile::RANDOM); file->Hint(RandomAccessFile::RANDOM);
} }
StopWatch sw(env_, options_->statistics.get(), TABLE_OPEN_IO_MICROS); StopWatch sw(env_, options_->statistics.get(), TABLE_OPEN_IO_MICROS);
s = options_->table_factory->GetTableReader(*options_, toptions, s = options_->table_factory->NewTableReader(
std::move(file), file_size, *options_, toptions, std::move(file), file_size, &table_reader);
&table_reader);
} }
if (!s.ok()) { if (!s.ok()) {

@ -12,11 +12,13 @@
#pragma once #pragma once
#include <string> #include <string>
#include <stdint.h> #include <stdint.h>
#include "db/dbformat.h" #include "db/dbformat.h"
#include "rocksdb/env.h"
#include "rocksdb/cache.h"
#include "port/port.h" #include "port/port.h"
#include "rocksdb/table.h" #include "rocksdb/cache.h"
#include "rocksdb/env.h"
#include "table/table_factory.h"
#include "table/table_reader.h"
namespace rocksdb { namespace rocksdb {

@ -7,14 +7,14 @@
#include <memory> #include <memory>
#include <string> #include <string>
#include "db/dbformat.h"
#include "db/db_impl.h" #include "db/db_impl.h"
#include "db/dbformat.h"
#include "db/table_properties_collector.h" #include "db/table_properties_collector.h"
#include "rocksdb/table_properties.h"
#include "rocksdb/table.h" #include "rocksdb/table.h"
#include "rocksdb/plain_table_factory.h"
#include "table/block_based_table_factory.h" #include "table/block_based_table_factory.h"
#include "table/meta_blocks.h" #include "table/meta_blocks.h"
#include "table/plain_table_factory.h"
#include "table/table_builder.h"
#include "util/coding.h" #include "util/coding.h"
#include "util/testharness.h" #include "util/testharness.h"
#include "util/testutil.h" #include "util/testutil.h"
@ -88,9 +88,8 @@ void MakeBuilder(
std::unique_ptr<FakeWritableFile>* writable, std::unique_ptr<FakeWritableFile>* writable,
std::unique_ptr<TableBuilder>* builder) { std::unique_ptr<TableBuilder>* builder) {
writable->reset(new FakeWritableFile); writable->reset(new FakeWritableFile);
builder->reset( builder->reset(options.table_factory->NewTableBuilder(
options.table_factory->GetTableBuilder(options, writable->get(), options, writable->get(), options.compression));
options.compression));
} }
// Collects keys that starts with "A" in a table. // Collects keys that starts with "A" in a table.

@ -12,6 +12,7 @@
#include <algorithm> #include <algorithm>
#include <climits> #include <climits>
#include <stdio.h> #include <stdio.h>
#include "db/filename.h" #include "db/filename.h"
#include "db/log_reader.h" #include "db/log_reader.h"
#include "db/log_writer.h" #include "db/log_writer.h"
@ -21,7 +22,7 @@
#include "db/compaction.h" #include "db/compaction.h"
#include "rocksdb/env.h" #include "rocksdb/env.h"
#include "rocksdb/merge_operator.h" #include "rocksdb/merge_operator.h"
#include "rocksdb/table.h" #include "table/table_reader.h"
#include "table/merger.h" #include "table/merger.h"
#include "table/two_level_iterator.h" #include "table/two_level_iterator.h"
#include "util/coding.h" #include "util/coding.h"

@ -1,180 +1,72 @@
// Copyright (c) 2013, Facebook, Inc. All rights reserved.
// This source code is licensed under the BSD-style license found in the
// LICENSE file in the root directory of this source tree. An additional grant
// of patent rights can be found in the PATENTS file in the same directory.
//
// Copyright (c) 2011 The LevelDB Authors. 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 // 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. // found in the LICENSE file. See the AUTHORS file for names of contributors.
//
// Currently we support two types of tables: plain table and block-based table.
// 1. Block-based table: this is the default table type that we inherited from
// LevelDB, which was designed for storing data in hard disk or flash
// device.
// 2. Plain table: it is one of RocksDB's SST file format optimized
// for low query latency on pure-memory or really low-latency media.
//
// A tutorial of rocksdb table formats is available here:
// https://github.com/facebook/rocksdb/wiki/A-Tutorial-of-RocksDB-SST-formats
//
// Example code is also available
// https://github.com/facebook/rocksdb/wiki/A-Tutorial-of-RocksDB-SST-formats#wiki-examples
#pragma once #pragma once
#include <memory> #include <memory>
#include <stdint.h> #include <string>
#include <unordered_map>
#include "rocksdb/env.h" #include "rocksdb/env.h"
#include "rocksdb/iterator.h" #include "rocksdb/iterator.h"
#include "rocksdb/table_properties.h"
#include "rocksdb/options.h" #include "rocksdb/options.h"
#include "rocksdb/status.h"
namespace rocksdb { namespace rocksdb {
struct Options; class TableFactory;
class RandomAccessFile;
struct ReadOptions;
class TableCache;
class WritableFile;
using std::unique_ptr;
// 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;
// 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;
// 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;
};
// A Table is a sorted map from strings to strings. Tables are // -- Block-based Table
// immutable and persistent. A Table may be safely accessed from class FlushBlockPolicyFactory;
// multiple threads without external synchronization.
class TableReader {
public:
virtual ~TableReader() {}
// Determine whether there is a chance that the current table file // For advanced user only
// contains the key a key starting with iternal_prefix. The specific struct BlockBasedTableOptions {
// table implementation can use bloom filter and/or other heuristic // @flush_block_policy_factory creates the instances of flush block policy.
// to filter out this table as a whole. // which provides a configurable way to determine when to flush a block in
virtual bool PrefixMayMatch(const Slice& internal_prefix) = 0; // the block based tables. If not set, table builder will use the default
// block flush policy, which cut blocks by block size (please refer to
// `FlushBlockBySizePolicy`).
std::shared_ptr<FlushBlockPolicyFactory> flush_block_policy_factory;
// Returns a new iterator over the table contents. // TODO(kailiu) Temporarily disable this feature by making the default value
// The result of NewIterator() is initially invalid (caller must // to be false.
// call one of the Seek methods on the iterator before using it).
virtual Iterator* NewIterator(const ReadOptions&) = 0;
// 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.
virtual uint64_t ApproximateOffsetOf(const Slice& key) = 0;
// Returns true if the block for the specified key is in cache.
// REQUIRES: key is in this table.
virtual bool TEST_KeyInCache(const ReadOptions& options,
const Slice& key) = 0;
// Set up the table for Compaction. Might change some parameters with
// posix_fadvise
virtual void SetupForCompaction() = 0;
virtual TableProperties& GetTableProperties() = 0;
// Calls (*result_handler)(handle_context, ...) repeatedly, starting with
// the entry found after a call to Seek(key), until result_handler returns
// false, where k is the actual internal key for a row found and v as the
// value of the key. didIO is true if I/O is involved in the operation. May
// not make such a call if filter policy says that key is not present.
//
// mark_key_may_exist_handler needs to be called when it is configured to be
// memory only and the key is not found in the block cache, with
// the parameter to be handle_context.
// //
// readOptions is the options for the read // Indicating if we'd put index/filter blocks to the block cache.
// key is the key to search for // If not specified, each "table reader" object will pre-load index/filter
virtual Status Get( // block during table initialization.
const ReadOptions& readOptions, bool cache_index_and_filter_blocks = false;
const Slice& key,
void* handle_context,
bool (*result_handler)(void* handle_context, const Slice& k,
const Slice& v, bool didIO),
void (*mark_key_may_exist_handler)(void* handle_context) = nullptr) = 0;
}; };
// A base class for table factories // Create default block based table factory.
class TableFactory { extern TableFactory* NewBlockBasedTableFactory(
public: const BlockBasedTableOptions& table_options = BlockBasedTableOptions());
virtual ~TableFactory() {}
// -- Plain Table
// The type of the table. // @user_key_len: plain table has optimization for fix-sized keys, which can be
// // specified via user_key_len. Alternatively, you can pass
// The client of this package should switch to a new name whenever // `kPlainTableVariableLength` if your keys have variable
// the table format implementation changes. // lengths.
// // @bloom_bits_per_key: the number of bits used for bloom filer per key. You may
// Names starting with "rocksdb." are reserved and should not be used // disable it by passing a zero.
// by any clients of this package. // @hash_table_ratio: the desired utilization of the hash table used for prefix
virtual const char* Name() const = 0; // hashing. hash_table_ratio = number of prefixes / #buckets
// in the hash table
// Returns a Table object table that can fetch data from file specified const uint32_t kPlainTableVariableLength = 0;
// in parameter file. It's the caller's responsibility to make sure extern TableFactory* NewPlainTableFactory(
// file is in the correct format. uint32_t user_key_len = kPlainTableVariableLength,
// int bloom_bits_per_key = 10, double hash_table_ratio = 0.75);
// GetTableReader() is called in two places:
// (1) TableCache::FindTable() calls the function when table cache miss
// and cache the table object returned.
// (1) SstFileReader (for SST Dump) opens the table and dump the table
// contents using the interator of the table.
// options and soptions are options. options is the general options.
// Multiple configured can be accessed from there, including and not
// limited to block cache and key comparators.
// file is a file handler to handle the file for the table
// file_size is the physical file size of the file
// table_reader is the output table reader
virtual Status GetTableReader(
const Options& options, const EnvOptions& soptions,
unique_ptr<RandomAccessFile> && file, uint64_t file_size,
unique_ptr<TableReader>* table_reader) const = 0;
// Return a table builder to write to a file for this table type.
//
// It is called in several places:
// (1) When flushing memtable to a level-0 output file, it creates a table
// builder (In DBImpl::WriteLevel0Table(), by calling BuildTable())
// (2) During compaction, it gets the builder for writing compaction output
// files in DBImpl::OpenCompactionOutputFile().
// (3) When recovering from transaction logs, it creates a table builder to
// write to a level-0 output file (In DBImpl::WriteLevel0TableForRecovery,
// by calling BuildTable())
// (4) When running Repairer, it creates a table builder to convert logs to
// SST files (In Repairer::ConvertLogToTable() by calling BuildTable())
//
// options is the general options. Multiple configured can be acceseed from
// there, including and not limited to compression options.
// file is a handle of a writable file. It is the caller's responsibility to
// keep the file open and close the file after closing the table builder.
// compression_type is the compression type to use in this table.
virtual TableBuilder* GetTableBuilder(
const Options& options, WritableFile* file,
CompressionType compression_type) const = 0;
};
} // namespace rocksdb } // namespace rocksdb

@ -1,23 +1,20 @@
// Copyright (c) 2013, Facebook, Inc. All rights reserved. // Copyright (c) 2011 The LevelDB Authors. All rights reserved.
// This source code is licensed under the BSD-style license found in the // Use of this source code is governed by a BSD-style license that can be
// LICENSE file in the root directory of this source tree. An additional grant // found in the LICENSE file. See the AUTHORS file for names of contributors.
// of patent rights can be found in the PATENTS file in the same directory.
#pragma once #pragma once
#include <string> #include <string>
#include <unordered_map> #include <unordered_map>
#include "rocksdb/status.h" #include "rocksdb/status.h"
namespace rocksdb { namespace rocksdb {
// -- Table Properties
// Other than basic table properties, each table may also have the user // Other than basic table properties, each table may also have the user
// collected properties. // collected properties.
// The value of the user-collected properties are encoded as raw bytes -- // The value of the user-collected properties are encoded as raw bytes --
// users have to interprete these values by themselves. // users have to interprete these values by themselves.
typedef typedef std::unordered_map<std::string, std::string> UserCollectedProperties;
std::unordered_map<std::string, std::string>
UserCollectedProperties;
// TableProperties contains a bunch of read-only properties of its associated // TableProperties contains a bunch of read-only properties of its associated
// table. // table.
@ -51,8 +48,7 @@ struct TableProperties {
// convert this object to a human readable form // convert this object to a human readable form
// @prop_delim: delimiter for each property. // @prop_delim: delimiter for each property.
std::string ToString( std::string ToString(const std::string& prop_delim = "; ",
const std::string& prop_delim = "; ",
const std::string& kv_delim = "=") const; const std::string& kv_delim = "=") const;
}; };
@ -77,7 +73,7 @@ extern const std::string kPropertiesBlock;
// of callback functions that will be invoked during table building. // of callback functions that will be invoked during table building.
class TablePropertiesCollector { class TablePropertiesCollector {
public: public:
virtual ~TablePropertiesCollector() { } virtual ~TablePropertiesCollector() {}
// Add() will be called when a new key/value pair is inserted into the table. // Add() will be called when a new key/value pair is inserted into the table.
// @params key the original key that is inserted into the table. // @params key the original key that is inserted into the table.

@ -17,7 +17,7 @@
#include "rocksdb/flush_block_policy.h" #include "rocksdb/flush_block_policy.h"
#include "rocksdb/cache.h" #include "rocksdb/cache.h"
#include "rocksdb/comparator.h" #include "rocksdb/comparator.h"
#include "rocksdb/table.h" #include "table/table_builder.h"
#include "rocksdb/env.h" #include "rocksdb/env.h"
#include "rocksdb/filter_policy.h" #include "rocksdb/filter_policy.h"
#include "rocksdb/options.h" #include "rocksdb/options.h"

@ -12,7 +12,7 @@
#include "rocksdb/flush_block_policy.h" #include "rocksdb/flush_block_policy.h"
#include "rocksdb/options.h" #include "rocksdb/options.h"
#include "rocksdb/status.h" #include "rocksdb/status.h"
#include "rocksdb/table.h" #include "table/table_builder.h"
namespace rocksdb { namespace rocksdb {

@ -18,7 +18,7 @@
namespace rocksdb { namespace rocksdb {
Status BlockBasedTableFactory::GetTableReader( Status BlockBasedTableFactory::NewTableReader(
const Options& options, const EnvOptions& soptions, const Options& options, const EnvOptions& soptions,
unique_ptr<RandomAccessFile>&& file, uint64_t file_size, unique_ptr<RandomAccessFile>&& file, uint64_t file_size,
unique_ptr<TableReader>* table_reader) const { unique_ptr<TableReader>* table_reader) const {
@ -26,7 +26,7 @@ Status BlockBasedTableFactory::GetTableReader(
std::move(file), file_size, table_reader); std::move(file), file_size, table_reader);
} }
TableBuilder* BlockBasedTableFactory::GetTableBuilder( TableBuilder* BlockBasedTableFactory::NewTableBuilder(
const Options& options, WritableFile* file, const Options& options, WritableFile* file,
CompressionType compression_type) const { CompressionType compression_type) const {
auto flush_block_policy_factory = auto flush_block_policy_factory =
@ -63,4 +63,9 @@ TableBuilder* BlockBasedTableFactory::GetTableBuilder(
return table_builder; return table_builder;
} }
TableFactory* NewBlockBasedTableFactory(
const BlockBasedTableOptions& table_options) {
return new BlockBasedTableFactory(table_options);
}
} // namespace rocksdb } // namespace rocksdb

@ -14,7 +14,7 @@
#include "rocksdb/flush_block_policy.h" #include "rocksdb/flush_block_policy.h"
#include "rocksdb/options.h" #include "rocksdb/options.h"
#include "rocksdb/table.h" #include "rocksdb/table.h"
#include "table/block_based_table_options.h" #include "table/table_factory.h"
namespace rocksdb { namespace rocksdb {
@ -22,29 +22,23 @@ struct Options;
struct EnvOptions; struct EnvOptions;
using std::unique_ptr; using std::unique_ptr;
class Status;
class RandomAccessFile;
class WritableFile;
class Table;
class TableBuilder;
class BlockBasedTable;
class BlockBasedTableBuilder; class BlockBasedTableBuilder;
class BlockBasedTableFactory: public TableFactory { class BlockBasedTableFactory : public TableFactory {
public: public:
BlockBasedTableFactory() : BlockBasedTableFactory(BlockBasedTableOptions()) {} explicit BlockBasedTableFactory(
explicit BlockBasedTableFactory(const BlockBasedTableOptions& table_options) const BlockBasedTableOptions& table_options = BlockBasedTableOptions())
: table_options_(table_options) {} : table_options_(table_options) {}
~BlockBasedTableFactory() {} ~BlockBasedTableFactory() {}
const char* Name() const override { return "BlockBasedTable"; } const char* Name() const override { return "BlockBasedTable"; }
Status GetTableReader(const Options& options, const EnvOptions& soptions, Status NewTableReader(const Options& options, const EnvOptions& soptions,
unique_ptr<RandomAccessFile>&& file, uint64_t file_size, unique_ptr<RandomAccessFile>&& file, uint64_t file_size,
unique_ptr<TableReader>* table_reader) const override; unique_ptr<TableReader>* table_reader) const override;
TableBuilder* GetTableBuilder(const Options& options, WritableFile* file, TableBuilder* NewTableBuilder(const Options& options, WritableFile* file,
CompressionType compression_type) CompressionType compression_type)
const override; const override;

@ -1,31 +0,0 @@
// Copyright (c) 2013, Facebook, Inc. All rights reserved.
// This source code is licensed under the BSD-style license found in the
// LICENSE file in the root directory of this source tree. An additional grant
// of patent rights can be found in the PATENTS file in the same directory.
#pragma once
#include <memory>
namespace rocksdb {
class FlushBlockPolicyFactory;
struct BlockBasedTableOptions {
// @flush_block_policy_factory creates the instances of flush block policy.
// which provides a configurable way to determine when to flush a block in
// the block based tables. If not set, table builder will use the default
// block flush policy, which cut blocks by block size (please refer to
// `FlushBlockBySizePolicy`).
std::shared_ptr<FlushBlockPolicyFactory> flush_block_policy_factory;
// TODO(kailiu) Temporarily disable this feature by making the default value
// to be false. Also in master branch, this file is non-public so no user
// will be able to change the value of `cache_index_and_filter_blocks`.
//
// Indicating if we'd put index/filter blocks to the block cache.
// If not specified, each "table reader" object will pre-load index/filter
// block during table initialization.
bool cache_index_and_filter_blocks = false;
};
} // namespace rocksdb

@ -27,7 +27,6 @@
#include "util/coding.h" #include "util/coding.h"
#include "util/perf_context_imp.h" #include "util/perf_context_imp.h"
#include "util/stop_watch.h" #include "util/stop_watch.h"
#include "table/block_based_table_options.h"
namespace rocksdb { namespace rocksdb {
@ -338,7 +337,7 @@ void BlockBasedTable::SetupForCompaction() {
compaction_optimized_ = true; compaction_optimized_ = true;
} }
TableProperties& BlockBasedTable::GetTableProperties() { const TableProperties& BlockBasedTable::GetTableProperties() {
return rep_->table_properties; return rep_->table_properties;
} }

@ -14,8 +14,7 @@
#include "rocksdb/env.h" #include "rocksdb/env.h"
#include "rocksdb/iterator.h" #include "rocksdb/iterator.h"
#include "rocksdb/statistics.h" #include "rocksdb/statistics.h"
#include "rocksdb/table_properties.h" #include "table/table_reader.h"
#include "rocksdb/table.h"
#include "util/coding.h" #include "util/coding.h"
namespace rocksdb { namespace rocksdb {
@ -62,9 +61,7 @@ class BlockBasedTable : public TableReader {
// call one of the Seek methods on the iterator before using it). // call one of the Seek methods on the iterator before using it).
Iterator* NewIterator(const ReadOptions&) override; Iterator* NewIterator(const ReadOptions&) override;
Status Get( Status Get(const ReadOptions& readOptions, const Slice& key,
const ReadOptions& readOptions,
const Slice& key,
void* handle_context, void* handle_context,
bool (*result_handler)(void* handle_context, const Slice& k, bool (*result_handler)(void* handle_context, const Slice& k,
const Slice& v, bool didIO), const Slice& v, bool didIO),
@ -81,13 +78,13 @@ class BlockBasedTable : public TableReader {
// Returns true if the block for the specified key is in cache. // Returns true if the block for the specified key is in cache.
// REQUIRES: key is in this table. // REQUIRES: key is in this table.
bool TEST_KeyInCache(const ReadOptions& options, const Slice& key) override; bool TEST_KeyInCache(const ReadOptions& options, const Slice& key);
// Set up the table for Compaction. Might change some parameters with // Set up the table for Compaction. Might change some parameters with
// posix_fadvise // posix_fadvise
void SetupForCompaction() override; void SetupForCompaction() override;
TableProperties& GetTableProperties() override; const TableProperties& GetTableProperties() override;
~BlockBasedTable(); ~BlockBasedTable();

@ -7,7 +7,7 @@
#include <map> #include <map>
#include "rocksdb/table_properties.h" #include "rocksdb/table.h"
#include "table/block.h" #include "table/block.h"
#include "table/format.h" #include "table/format.h"
#include "util/coding.h" #include "util/coding.h"

@ -9,7 +9,7 @@
#include <stdint.h> #include <stdint.h>
#include "rocksdb/options.h" #include "rocksdb/options.h"
#include "rocksdb/status.h" #include "rocksdb/status.h"
#include "rocksdb/table.h" #include "table/table_builder.h"
#include "rocksdb/table_properties.h" #include "rocksdb/table_properties.h"
namespace rocksdb { namespace rocksdb {

@ -2,7 +2,7 @@
// Use of this source code is governed by a BSD-style license that can be // 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. // found in the LICENSE file. See the AUTHORS file for names of contributors.
#include "rocksdb/plain_table_factory.h" #include "table/plain_table_factory.h"
#include <memory> #include <memory>
#include <stdint.h> #include <stdint.h>
@ -12,19 +12,26 @@
namespace rocksdb { namespace rocksdb {
Status PlainTableFactory::GetTableReader(const Options& options, Status PlainTableFactory::NewTableReader(const Options& options,
const EnvOptions& soptions, const EnvOptions& soptions,
unique_ptr<RandomAccessFile> && file, unique_ptr<RandomAccessFile>&& file,
uint64_t file_size, uint64_t file_size,
unique_ptr<TableReader>* table) unique_ptr<TableReader>* table) const {
const {
return PlainTableReader::Open(options, soptions, std::move(file), file_size, return PlainTableReader::Open(options, soptions, std::move(file), file_size,
table, bloom_num_bits_, hash_table_ratio_); table, bloom_bits_per_key_, hash_table_ratio_);
} }
TableBuilder* PlainTableFactory::GetTableBuilder( TableBuilder* PlainTableFactory::NewTableBuilder(
const Options& options, WritableFile* file, const Options& options, WritableFile* file,
CompressionType compression_type) const { CompressionType compression_type) const {
return new PlainTableBuilder(options, file, user_key_len_); return new PlainTableBuilder(options, file, user_key_len_);
} }
extern TableFactory* NewPlainTableFactory(uint32_t user_key_len,
int bloom_bits_per_key,
double hash_table_ratio) {
return new PlainTableFactory(user_key_len, bloom_bits_per_key,
hash_table_ratio);
}
} // namespace rocksdb } // namespace rocksdb

@ -8,6 +8,7 @@
#include "rocksdb/options.h" #include "rocksdb/options.h"
#include "rocksdb/table.h" #include "rocksdb/table.h"
#include "table/table_factory.h"
namespace rocksdb { namespace rocksdb {
@ -37,40 +38,35 @@ class TableBuilder;
// | | // | |
// | ...... | // | ...... |
// +-----------------+--------------------------+ // +-----------------+--------------------------+
// If user_key_length = kVariableLength, it means the key is variable length, // If user_key_length = kPlainTableVariableLength, it means the key is variable
// there will be an extra field for key size encoded before every key. // length, there will be an extra field for key size encoded before every key.
class PlainTableFactory: public TableFactory { class PlainTableFactory : public TableFactory {
public: public:
~PlainTableFactory() { ~PlainTableFactory() {}
}
// user_key_size is the length of the user key. If it is set to be // user_key_size is the length of the user key. If it is set to be
// kVariableLength, then it means variable length. Otherwise, all the // kPlainTableVariableLength, then it means variable length. Otherwise, all
// keys need to have the fix length of this value. bloom_num_bits is // 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 // number of bits used for bloom filer per key. hash_table_ratio is
// the desired utilization of the hash table used for prefix hashing. // 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 = number of prefixes / #buckets in the hash table
explicit PlainTableFactory(uint32_t user_key_len = kVariableLength, explicit PlainTableFactory(uint32_t user_key_len = kPlainTableVariableLength,
int bloom_num_bits = 0, int bloom_bits_per_key = 0,
double hash_table_ratio = 0.75) : double hash_table_ratio = 0.75)
user_key_len_(user_key_len), bloom_num_bits_(bloom_num_bits), : user_key_len_(user_key_len),
hash_table_ratio_(hash_table_ratio) { bloom_bits_per_key_(bloom_bits_per_key),
} hash_table_ratio_(hash_table_ratio) {}
const char* Name() const override { const char* Name() const override { return "PlainTable"; }
return "PlainTable"; Status NewTableReader(const Options& options, const EnvOptions& soptions,
} unique_ptr<RandomAccessFile>&& file, uint64_t file_size,
Status GetTableReader(const Options& options, const EnvOptions& soptions,
unique_ptr<RandomAccessFile> && file,
uint64_t file_size,
unique_ptr<TableReader>* table) const override; unique_ptr<TableReader>* table) const override;
TableBuilder* GetTableBuilder(const Options& options, WritableFile* file, TableBuilder* NewTableBuilder(const Options& options, WritableFile* file,
CompressionType compression_type) const CompressionType compression_type)
override; const override;
static const uint32_t kVariableLength = 0; private:
private:
uint32_t user_key_len_; uint32_t user_key_len_;
int bloom_num_bits_; int bloom_bits_per_key_;
double hash_table_ratio_; double hash_table_ratio_;
}; };

@ -15,13 +15,13 @@
#include "rocksdb/filter_policy.h" #include "rocksdb/filter_policy.h"
#include "rocksdb/options.h" #include "rocksdb/options.h"
#include "rocksdb/statistics.h" #include "rocksdb/statistics.h"
#include "rocksdb/plain_table_factory.h"
#include "table/block.h" #include "table/block.h"
#include "table/filter_block.h" #include "table/filter_block.h"
#include "table/format.h" #include "table/format.h"
#include "table/meta_blocks.h" #include "table/meta_blocks.h"
#include "table/two_level_iterator.h" #include "table/two_level_iterator.h"
#include "table/plain_table_factory.h"
#include "util/coding.h" #include "util/coding.h"
#include "util/dynamic_bloom.h" #include "util/dynamic_bloom.h"
@ -103,10 +103,10 @@ PlainTableReader::~PlainTableReader() {
Status PlainTableReader::Open(const Options& options, Status PlainTableReader::Open(const Options& options,
const EnvOptions& soptions, const EnvOptions& soptions,
unique_ptr<RandomAccessFile> && file, unique_ptr<RandomAccessFile>&& file,
uint64_t file_size, uint64_t file_size,
unique_ptr<TableReader>* table_reader, unique_ptr<TableReader>* table_reader,
const int bloom_num_bits, const int bloom_bits_per_key,
double hash_table_ratio) { double hash_table_ratio) {
assert(options.allow_mmap_reads); assert(options.allow_mmap_reads);
@ -122,8 +122,9 @@ Status PlainTableReader::Open(const Options& options,
return s; return s;
} }
std::unique_ptr<PlainTableReader> new_reader(new PlainTableReader( std::unique_ptr<PlainTableReader> new_reader(
soptions, file_size, bloom_num_bits, hash_table_ratio, table_properties)); new PlainTableReader(soptions, file_size, bloom_bits_per_key,
hash_table_ratio, table_properties));
new_reader->file_ = std::move(file); new_reader->file_ = std::move(file);
new_reader->options_ = options; new_reader->options_ = options;
@ -556,11 +557,6 @@ Status PlainTableReader::Get(
return Status::OK(); return Status::OK();
} }
bool PlainTableReader::TEST_KeyInCache(const ReadOptions& options,
const Slice& key) {
return false;
}
uint64_t PlainTableReader::ApproximateOffsetOf(const Slice& key) { uint64_t PlainTableReader::ApproximateOffsetOf(const Slice& key) {
return 0; return 0;
} }

@ -7,11 +7,14 @@
#include <memory> #include <memory>
#include <vector> #include <vector>
#include <stdint.h> #include <stdint.h>
#include "rocksdb/env.h" #include "rocksdb/env.h"
#include "rocksdb/iterator.h" #include "rocksdb/iterator.h"
#include "rocksdb/table.h"
#include "rocksdb/slice_transform.h" #include "rocksdb/slice_transform.h"
#include "rocksdb/plain_table_factory.h" #include "rocksdb/table.h"
#include "rocksdb/table_properties.h"
#include "table/table_reader.h"
#include "table/plain_table_factory.h"
namespace rocksdb { namespace rocksdb {
@ -27,6 +30,7 @@ class DynamicBloom;
using std::unique_ptr; using std::unique_ptr;
using std::unordered_map; using std::unordered_map;
extern const uint32_t kPlainTableVariableLength;
// Based on following output file format shown in plain_table_factory.h // Based on following output file format shown in plain_table_factory.h
// When opening the output file, IndexedTableReader creates a hash table // When opening the output file, IndexedTableReader creates a hash table
@ -40,8 +44,8 @@ class PlainTableReader: public TableReader {
public: public:
static Status Open(const Options& options, const EnvOptions& soptions, static Status Open(const Options& options, const EnvOptions& soptions,
unique_ptr<RandomAccessFile>&& file, uint64_t file_size, unique_ptr<RandomAccessFile>&& file, uint64_t file_size,
unique_ptr<TableReader>* table, const int bloom_num_bits, unique_ptr<TableReader>* table,
double hash_table_ratio); const int bloom_bits_per_key, double hash_table_ratio);
bool PrefixMayMatch(const Slice& internal_prefix); bool PrefixMayMatch(const Slice& internal_prefix);
@ -54,16 +58,12 @@ class PlainTableReader: public TableReader {
uint64_t ApproximateOffsetOf(const Slice& key); uint64_t ApproximateOffsetOf(const Slice& key);
bool TEST_KeyInCache(const ReadOptions& options, const Slice& key);
void SetupForCompaction(); void SetupForCompaction();
TableProperties& GetTableProperties() { const TableProperties& GetTableProperties() { return table_properties_; }
return table_properties_;
}
PlainTableReader(const EnvOptions& storage_options, uint64_t file_size, PlainTableReader(const EnvOptions& storage_options, uint64_t file_size,
int bloom_num_bits, double hash_table_ratio, int bloom_bits_per_key, double hash_table_ratio,
const TableProperties& table_properties); const TableProperties& table_properties);
~PlainTableReader(); ~PlainTableReader();
@ -104,7 +104,7 @@ class PlainTableReader: public TableReader {
static const size_t kIndexIntervalForSamePrefixKeys = 16; static const size_t kIndexIntervalForSamePrefixKeys = 16;
bool IsFixedLength() const { bool IsFixedLength() const {
return user_key_len_ != PlainTableFactory::kVariableLength; return user_key_len_ != kPlainTableVariableLength;
} }
size_t GetFixedInternalKeyLength() const { size_t GetFixedInternalKeyLength() const {

@ -0,0 +1,55 @@
// Copyright (c) 2013, Facebook, Inc. All rights reserved.
// This source code is licensed under the BSD-style license found in the
// LICENSE file in the root directory of this source tree. An additional grant
// of patent rights can be found in the PATENTS file in the same directory.
//
// 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
namespace rocksdb {
class Slice;
class Status;
// 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;
// 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;
// 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;
};
} // namespace rocksdb

@ -0,0 +1,82 @@
// Copyright (c) 2013, Facebook, Inc. All rights reserved.
// This source code is licensed under the BSD-style license found in the
// LICENSE file in the root directory of this source tree. An additional grant
// of patent rights can be found in the PATENTS file in the same directory.
//
// 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 "rocksdb/status.h"
namespace rocksdb {
using std::unique_ptr;
class RandomAccessFile;
class TableBuilder;
class TableReader;
class WritableFile;
struct EnvOptions;
struct Options;
// A base class for table factories
class TableFactory {
public:
virtual ~TableFactory() {}
// The type of the table.
//
// The client of this package should switch to a new name whenever
// the table format implementation changes.
//
// Names starting with "rocksdb." are reserved and should not be used
// by any clients of this package.
virtual const char* Name() const = 0;
// Returns a Table object table that can fetch data from file specified
// in parameter file. It's the caller's responsibility to make sure
// file is in the correct format.
//
// NewTableReader() is called in two places:
// (1) TableCache::FindTable() calls the function when table cache miss
// and cache the table object returned.
// (1) SstFileReader (for SST Dump) opens the table and dump the table
// contents using the interator of the table.
// options and soptions are options. options is the general options.
// Multiple configured can be accessed from there, including and not
// limited to block cache and key comparators.
// file is a file handler to handle the file for the table
// file_size is the physical file size of the file
// table_reader is the output table reader
virtual Status NewTableReader(
const Options& options, const EnvOptions& soptions,
unique_ptr<RandomAccessFile>&& file, uint64_t file_size,
unique_ptr<TableReader>* table_reader) const = 0;
// Return a table builder to write to a file for this table type.
//
// It is called in several places:
// (1) When flushing memtable to a level-0 output file, it creates a table
// builder (In DBImpl::WriteLevel0Table(), by calling BuildTable())
// (2) During compaction, it gets the builder for writing compaction output
// files in DBImpl::OpenCompactionOutputFile().
// (3) When recovering from transaction logs, it creates a table builder to
// write to a level-0 output file (In DBImpl::WriteLevel0TableForRecovery,
// by calling BuildTable())
// (4) When running Repairer, it creates a table builder to convert logs to
// SST files (In Repairer::ConvertLogToTable() by calling BuildTable())
//
// options is the general options. Multiple configured can be acceseed from
// there, including and not limited to compression options.
// file is a handle of a writable file. It is the caller's responsibility to
// keep the file open and close the file after closing the table builder.
// compression_type is the compression type to use in this table.
virtual TableBuilder* NewTableBuilder(
const Options& options, WritableFile* file,
CompressionType compression_type) const = 0;
};
} // namespace rocksdb

@ -0,0 +1,70 @@
// Copyright (c) 2013, Facebook, Inc. All rights reserved.
// This source code is licensed under the BSD-style license found in the
// LICENSE file in the root directory of this source tree. An additional grant
// of patent rights can be found in the PATENTS file in the same directory.
//
// 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
namespace rocksdb {
class Iterator;
class Slice;
struct ReadOptions;
struct TableProperties;
// A Table 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.
class TableReader {
public:
virtual ~TableReader() {}
// Determine whether there is a chance that the current table file
// contains the key a key starting with iternal_prefix. The specific
// table implementation can use bloom filter and/or other heuristic
// to filter out this table as a whole.
virtual bool PrefixMayMatch(const Slice& internal_prefix) = 0;
// 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).
virtual Iterator* NewIterator(const ReadOptions&) = 0;
// 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.
virtual uint64_t ApproximateOffsetOf(const Slice& key) = 0;
// Set up the table for Compaction. Might change some parameters with
// posix_fadvise
virtual void SetupForCompaction() = 0;
virtual const TableProperties& GetTableProperties() = 0;
// Calls (*result_handler)(handle_context, ...) repeatedly, starting with
// the entry found after a call to Seek(key), until result_handler returns
// false, where k is the actual internal key for a row found and v as the
// value of the key. didIO is true if I/O is involved in the operation. May
// not make such a call if filter policy says that key is not present.
//
// mark_key_may_exist_handler needs to be called when it is configured to be
// memory only and the key is not found in the block cache, with
// the parameter to be handle_context.
//
// readOptions is the options for the read
// key is the key to search for
virtual Status Get(
const ReadOptions& readOptions, const Slice& key, void* handle_context,
bool (*result_handler)(void* handle_context, const Slice& k,
const Slice& v, bool didIO),
void (*mark_key_may_exist_handler)(void* handle_context) = nullptr) = 0;
};
} // namespace rocksdb

@ -12,7 +12,7 @@
#include "db/dbformat.h" #include "db/dbformat.h"
#include "port/atomic_pointer.h" #include "port/atomic_pointer.h"
#include "table/block_based_table_factory.h" #include "table/block_based_table_factory.h"
#include "rocksdb/plain_table_factory.h" #include "table/plain_table_factory.h"
#include "util/histogram.h" #include "util/histogram.h"
#include "util/testharness.h" #include "util/testharness.h"
#include "util/testutil.h" #include "util/testutil.h"
@ -71,7 +71,7 @@ void TableReaderBenchmark(Options& opts, EnvOptions& env_options,
Status s; Status s;
if (!through_db) { if (!through_db) {
env->NewWritableFile(file_name, &file, env_options); env->NewWritableFile(file_name, &file, env_options);
tb = opts.table_factory->GetTableBuilder(opts, file.get(), tb = opts.table_factory->NewTableBuilder(opts, file.get(),
CompressionType::kNoCompression); CompressionType::kNoCompression);
} else { } else {
s = DB::Open(opts, dbname, &db); s = DB::Open(opts, dbname, &db);
@ -102,7 +102,7 @@ void TableReaderBenchmark(Options& opts, EnvOptions& env_options,
Status s = env->NewRandomAccessFile(file_name, &raf, env_options); Status s = env->NewRandomAccessFile(file_name, &raf, env_options);
uint64_t file_size; uint64_t file_size;
env->GetFileSize(file_name, &file_size); env->GetFileSize(file_name, &file_size);
s = opts.table_factory->GetTableReader(opts, env_options, std::move(raf), s = opts.table_factory->NewTableReader(opts, env_options, std::move(raf),
file_size, &table_reader); file_size, &table_reader);
} }

@ -20,19 +20,18 @@
#include "rocksdb/cache.h" #include "rocksdb/cache.h"
#include "rocksdb/db.h" #include "rocksdb/db.h"
#include "rocksdb/plain_table_factory.h"
#include "rocksdb/env.h" #include "rocksdb/env.h"
#include "rocksdb/iterator.h" #include "rocksdb/iterator.h"
#include "rocksdb/slice_transform.h" #include "rocksdb/slice_transform.h"
#include "rocksdb/memtablerep.h" #include "rocksdb/memtablerep.h"
#include "table/meta_blocks.h" #include "table/block.h"
#include "rocksdb/plain_table_factory.h"
#include "table/block_based_table_builder.h" #include "table/block_based_table_builder.h"
#include "table/block_based_table_factory.h" #include "table/block_based_table_factory.h"
#include "table/block_based_table_reader.h" #include "table/block_based_table_reader.h"
#include "table/block_builder.h" #include "table/block_builder.h"
#include "table/block.h"
#include "table/format.h" #include "table/format.h"
#include "table/meta_blocks.h"
#include "table/plain_table_factory.h"
#include "util/random.h" #include "util/random.h"
#include "util/testharness.h" #include "util/testharness.h"
@ -303,8 +302,7 @@ class TableConstructor: public Constructor {
Reset(); Reset();
sink_.reset(new StringSink()); sink_.reset(new StringSink());
unique_ptr<TableBuilder> builder; unique_ptr<TableBuilder> builder;
builder.reset( builder.reset(options.table_factory->NewTableBuilder(options, sink_.get(),
options.table_factory->GetTableBuilder(options, sink_.get(),
options.compression)); options.compression));
for (KVMap::const_iterator it = data.begin(); for (KVMap::const_iterator it = data.begin();
@ -329,7 +327,7 @@ class TableConstructor: public Constructor {
uniq_id_ = cur_uniq_id_++; uniq_id_ = cur_uniq_id_++;
source_.reset(new StringSource(sink_->contents(), uniq_id_, source_.reset(new StringSource(sink_->contents(), uniq_id_,
options.allow_mmap_reads)); options.allow_mmap_reads));
return options.table_factory->GetTableReader( return options.table_factory->NewTableReader(
options, soptions, std::move(source_), sink_->contents().size(), options, soptions, std::move(source_), sink_->contents().size(),
&table_reader_); &table_reader_);
} }
@ -351,9 +349,8 @@ class TableConstructor: public Constructor {
source_.reset( source_.reset(
new StringSource(sink_->contents(), uniq_id_, new StringSource(sink_->contents(), uniq_id_,
options.allow_mmap_reads)); options.allow_mmap_reads));
return options.table_factory->GetTableReader(options, soptions, return options.table_factory->NewTableReader(
std::move(source_), options, soptions, std::move(source_), sink_->contents().size(),
sink_->contents().size(),
&table_reader_); &table_reader_);
} }
@ -1210,8 +1207,9 @@ TEST(BlockBasedTableTest, BlockCacheLeak) {
ASSERT_OK(iter->status()); ASSERT_OK(iter->status());
ASSERT_OK(c.Reopen(opt)); ASSERT_OK(c.Reopen(opt));
auto table_reader = dynamic_cast<BlockBasedTable*>(c.table_reader());
for (const std::string& key : keys) { for (const std::string& key : keys) {
ASSERT_TRUE(c.table_reader()->TEST_KeyInCache(ReadOptions(), key)); ASSERT_TRUE(table_reader->TEST_KeyInCache(ReadOptions(), key));
} }
} }
@ -1220,7 +1218,7 @@ TEST(PlainTableTest, BasicPlainTableProperties) {
PlainTableFactory factory(8, 8, 0); PlainTableFactory factory(8, 8, 0);
StringSink sink; StringSink sink;
std::unique_ptr<TableBuilder> builder( std::unique_ptr<TableBuilder> builder(
factory.GetTableBuilder(Options(), &sink, kNoCompression)); factory.NewTableBuilder(Options(), &sink, kNoCompression));
for (char c = 'a'; c <= 'z'; ++c) { for (char c = 'a'; c <= 'z'; ++c) {
std::string key(16, c); std::string key(16, c);

@ -80,9 +80,8 @@ Status SstFileReader::NewTableReader(const std::string& file_path) {
uint64_t file_size; uint64_t file_size;
table_options_.env->GetFileSize(file_path, &file_size); table_options_.env->GetFileSize(file_path, &file_size);
unique_ptr<TableFactory> table_factory; unique_ptr<TableFactory> table_factory;
s = table_options_.table_factory->GetTableReader(table_options_, soptions_, s = table_options_.table_factory->NewTableReader(
std::move(file_), file_size, table_options_, soptions_, std::move(file_), file_size, &table_reader_);
&table_reader_);
return s; return s;
} }

@ -16,10 +16,11 @@
#include "rocksdb/comparator.h" #include "rocksdb/comparator.h"
#include "rocksdb/env.h" #include "rocksdb/env.h"
#include "rocksdb/filter_policy.h" #include "rocksdb/filter_policy.h"
#include "rocksdb/merge_operator.h"
#include "rocksdb/memtablerep.h" #include "rocksdb/memtablerep.h"
#include "rocksdb/merge_operator.h"
#include "rocksdb/slice.h" #include "rocksdb/slice.h"
#include "rocksdb/slice_transform.h" #include "rocksdb/slice_transform.h"
#include "rocksdb/table.h"
#include "rocksdb/table_properties.h" #include "rocksdb/table_properties.h"
#include "table/block_based_table_factory.h" #include "table/block_based_table_factory.h"

Loading…
Cancel
Save