You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
rocksdb/table/block_based/block_based_table_builder.cc

1804 lines
68 KiB

// 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 <string>
#include <unordered_map>
#include <utility>
#include "db/dbformat.h"
#include "index_builder.h"
#include "port/lang.h"
#include "rocksdb/cache.h"
#include "rocksdb/comparator.h"
#include "rocksdb/env.h"
#include "rocksdb/flush_block_policy.h"
#include "rocksdb/merge_operator.h"
#include "rocksdb/table.h"
#include "table/block_based/block.h"
#include "table/block_based/block_based_filter_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/filter_block.h"
New Bloom filter implementation for full and partitioned filters (#6007) Summary: Adds an improved, replacement Bloom filter implementation (FastLocalBloom) for full and partitioned filters in the block-based table. This replacement is faster and more accurate, especially for high bits per key or millions of keys in a single filter. Speed The improved speed, at least on recent x86_64, comes from * Using fastrange instead of modulo (%) * Using our new hash function (XXH3 preview, added in a previous commit), which is much faster for large keys and only *slightly* slower on keys around 12 bytes if hashing the same size many thousands of times in a row. * Optimizing the Bloom filter queries with AVX2 SIMD operations. (Added AVX2 to the USE_SSE=1 build.) Careful design was required to support (a) SIMD-optimized queries, (b) compatible non-SIMD code that's simple and efficient, (c) flexible choice of number of probes, and (d) essentially maximized accuracy for a cache-local Bloom filter. Probes are made eight at a time, so any number of probes up to 8 is the same speed, then up to 16, etc. * Prefetching cache lines when building the filter. Although this optimization could be applied to the old structure as well, it seems to balance out the small added cost of accumulating 64 bit hashes for adding to the filter rather than 32 bit hashes. Here's nominal speed data from filter_bench (200MB in filters, about 10k keys each, 10 bits filter data / key, 6 probes, avg key size 24 bytes, includes hashing time) on Skylake DE (relatively low clock speed): $ ./filter_bench -quick -impl=2 -net_includes_hashing # New Bloom filter Build avg ns/key: 47.7135 Mixed inside/outside queries... Single filter net ns/op: 26.2825 Random filter net ns/op: 150.459 Average FP rate %: 0.954651 $ ./filter_bench -quick -impl=0 -net_includes_hashing # Old Bloom filter Build avg ns/key: 47.2245 Mixed inside/outside queries... Single filter net ns/op: 63.2978 Random filter net ns/op: 188.038 Average FP rate %: 1.13823 Similar build time but dramatically faster query times on hot data (63 ns to 26 ns), and somewhat faster on stale data (188 ns to 150 ns). Performance differences on batched and skewed query loads are between these extremes as expected. The only other interesting thing about speed is "inside" (query key was added to filter) vs. "outside" (query key was not added to filter) query times. The non-SIMD implementations are substantially slower when most queries are "outside" vs. "inside". This goes against what one might expect or would have observed years ago, as "outside" queries only need about two probes on average, due to short-circuiting, while "inside" always have num_probes (say 6). The problem is probably the nastily unpredictable branch. The SIMD implementation has few branches (very predictable) and has pretty consistent running time regardless of query outcome. Accuracy The generally improved accuracy (re: Issue https://github.com/facebook/rocksdb/issues/5857) comes from a better design for probing indices within a cache line (re: Issue https://github.com/facebook/rocksdb/issues/4120) and improved accuracy for millions of keys in a single filter from using a 64-bit hash function (XXH3p). Design details in code comments. Accuracy data (generalizes, except old impl gets worse with millions of keys): Memory bits per key: FP rate percent old impl -> FP rate percent new impl 6: 5.70953 -> 5.69888 8: 2.45766 -> 2.29709 10: 1.13977 -> 0.959254 12: 0.662498 -> 0.411593 16: 0.353023 -> 0.0873754 24: 0.261552 -> 0.0060971 50: 0.225453 -> ~0.00003 (less than 1 in a million queries are FP) Fixes https://github.com/facebook/rocksdb/issues/5857 Fixes https://github.com/facebook/rocksdb/issues/4120 Unlike the old implementation, this implementation has a fixed cache line size (64 bytes). At 10 bits per key, the accuracy of this new implementation is very close to the old implementation with 128-byte cache line size. If there's sufficient demand, this implementation could be generalized. Compatibility Although old releases would see the new structure as corrupt filter data and read the table as if there's no filter, we've decided only to enable the new Bloom filter with new format_version=5. This provides a smooth path for automatic adoption over time, with an option for early opt-in. Pull Request resolved: https://github.com/facebook/rocksdb/pull/6007 Test Plan: filter_bench has been used thoroughly to validate speed, accuracy, and correctness. Unit tests have been carefully updated to exercise new and old implementations, as well as the logic to select an implementation based on context (format_version). Differential Revision: D18294749 Pulled By: pdillinger fbshipit-source-id: d44c9db3696e4d0a17caaec47075b7755c262c5f
5 years ago
#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/table_builder.h"
#include "memory/memory_allocator.h"
#include "util/coding.h"
#include "util/compression.h"
#include "util/crc32c.h"
#include "util/stop_watch.h"
#include "util/string_util.h"
#include "util/work_queue.h"
#include "util/xxhash.h"
namespace ROCKSDB_NAMESPACE {
extern const std::string kHashIndexPrefixesBlock;
extern const std::string kHashIndexPrefixesMetadataBlock;
typedef BlockBasedTableOptions::IndexType IndexType;
// Without anonymous namespace here, we fail the warning -Wmissing-prototypes
namespace {
// 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;
Implement full filter for block based table. Summary: 1. Make filter_block.h a base class. Derive block_based_filter_block and full_filter_block. The previous one is the traditional filter block. The full_filter_block is newly added. It would generate a filter block that contain all the keys in SST file. 2. When querying a key, table would first check if full_filter is available. If not, it would go to the exact data block and check using block_based filter. 3. User could choose to use full_filter or tradional(block_based_filter). They would be stored in SST file with different meta index name. "filter.filter_policy" or "full_filter.filter_policy". Then, Table reader is able to know the fllter block type. 4. Some optimizations have been done for full_filter_block, thus it requires a different interface compared to the original one in filter_policy.h. 5. Actual implementation of filter bits coding/decoding is placed in util/bloom_impl.cc Benchmark: base commit 1d23b5c470844c1208301311f0889eca750431c0 Command: db_bench --db=/dev/shm/rocksdb --num_levels=6 --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --write_buffer_size=134217728 --max_write_buffer_number=2 --target_file_size_base=33554432 --max_bytes_for_level_base=1073741824 --verify_checksum=false --max_background_compactions=4 --use_plain_table=0 --memtablerep=prefix_hash --open_files=-1 --mmap_read=1 --mmap_write=0 --bloom_bits=10 --bloom_locality=1 --memtable_bloom_bits=500000 --compression_type=lz4 --num=393216000 --use_hash_search=1 --block_size=1024 --block_restart_interval=16 --use_existing_db=1 --threads=1 --benchmarks=readrandom —disable_auto_compactions=1 Read QPS increase for about 30% from 2230002 to 2991411. Test Plan: make all check valgrind db_test db_stress --use_block_based_filter = 0 ./auto_sanity_test.sh Reviewers: igor, yhchiang, ljin, sdong Reviewed By: sdong Subscribers: dhruba, leveldb Differential Revision: https://reviews.facebook.net/D20979
10 years ago
if (table_opt.filter_policy == nullptr) return nullptr;
FilterBitsBuilder* filter_bits_builder =
BloomFilterPolicy::GetBuilderFromContext(context);
Implement full filter for block based table. Summary: 1. Make filter_block.h a base class. Derive block_based_filter_block and full_filter_block. The previous one is the traditional filter block. The full_filter_block is newly added. It would generate a filter block that contain all the keys in SST file. 2. When querying a key, table would first check if full_filter is available. If not, it would go to the exact data block and check using block_based filter. 3. User could choose to use full_filter or tradional(block_based_filter). They would be stored in SST file with different meta index name. "filter.filter_policy" or "full_filter.filter_policy". Then, Table reader is able to know the fllter block type. 4. Some optimizations have been done for full_filter_block, thus it requires a different interface compared to the original one in filter_policy.h. 5. Actual implementation of filter bits coding/decoding is placed in util/bloom_impl.cc Benchmark: base commit 1d23b5c470844c1208301311f0889eca750431c0 Command: db_bench --db=/dev/shm/rocksdb --num_levels=6 --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --write_buffer_size=134217728 --max_write_buffer_number=2 --target_file_size_base=33554432 --max_bytes_for_level_base=1073741824 --verify_checksum=false --max_background_compactions=4 --use_plain_table=0 --memtablerep=prefix_hash --open_files=-1 --mmap_read=1 --mmap_write=0 --bloom_bits=10 --bloom_locality=1 --memtable_bloom_bits=500000 --compression_type=lz4 --num=393216000 --use_hash_search=1 --block_size=1024 --block_restart_interval=16 --use_existing_db=1 --threads=1 --benchmarks=readrandom —disable_auto_compactions=1 Read QPS increase for about 30% from 2230002 to 2991411. Test Plan: make all check valgrind db_test db_stress --use_block_based_filter = 0 ./auto_sanity_test.sh Reviewers: igor, yhchiang, ljin, sdong Reviewed By: sdong Subscribers: dhruba, leveldb Differential Revision: https://reviews.facebook.net/D20979
10 years ago
if (filter_bits_builder == nullptr) {
return new BlockBasedFilterBlockBuilder(mopt.prefix_extractor.get(),
table_opt);
Implement full filter for block based table. Summary: 1. Make filter_block.h a base class. Derive block_based_filter_block and full_filter_block. The previous one is the traditional filter block. The full_filter_block is newly added. It would generate a filter block that contain all the keys in SST file. 2. When querying a key, table would first check if full_filter is available. If not, it would go to the exact data block and check using block_based filter. 3. User could choose to use full_filter or tradional(block_based_filter). They would be stored in SST file with different meta index name. "filter.filter_policy" or "full_filter.filter_policy". Then, Table reader is able to know the fllter block type. 4. Some optimizations have been done for full_filter_block, thus it requires a different interface compared to the original one in filter_policy.h. 5. Actual implementation of filter bits coding/decoding is placed in util/bloom_impl.cc Benchmark: base commit 1d23b5c470844c1208301311f0889eca750431c0 Command: db_bench --db=/dev/shm/rocksdb --num_levels=6 --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --write_buffer_size=134217728 --max_write_buffer_number=2 --target_file_size_base=33554432 --max_bytes_for_level_base=1073741824 --verify_checksum=false --max_background_compactions=4 --use_plain_table=0 --memtablerep=prefix_hash --open_files=-1 --mmap_read=1 --mmap_write=0 --bloom_bits=10 --bloom_locality=1 --memtable_bloom_bits=500000 --compression_type=lz4 --num=393216000 --use_hash_search=1 --block_size=1024 --block_restart_interval=16 --use_existing_db=1 --threads=1 --benchmarks=readrandom —disable_auto_compactions=1 Read QPS increase for about 30% from 2230002 to 2991411. Test Plan: make all check valgrind db_test db_stress --use_block_based_filter = 0 ./auto_sanity_test.sh Reviewers: igor, yhchiang, ljin, sdong Reviewed By: sdong Subscribers: dhruba, leveldb Differential Revision: https://reviews.facebook.net/D20979
10 years ago
} 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, 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);
}
Implement full filter for block based table. Summary: 1. Make filter_block.h a base class. Derive block_based_filter_block and full_filter_block. The previous one is the traditional filter block. The full_filter_block is newly added. It would generate a filter block that contain all the keys in SST file. 2. When querying a key, table would first check if full_filter is available. If not, it would go to the exact data block and check using block_based filter. 3. User could choose to use full_filter or tradional(block_based_filter). They would be stored in SST file with different meta index name. "filter.filter_policy" or "full_filter.filter_policy". Then, Table reader is able to know the fllter block type. 4. Some optimizations have been done for full_filter_block, thus it requires a different interface compared to the original one in filter_policy.h. 5. Actual implementation of filter bits coding/decoding is placed in util/bloom_impl.cc Benchmark: base commit 1d23b5c470844c1208301311f0889eca750431c0 Command: db_bench --db=/dev/shm/rocksdb --num_levels=6 --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --write_buffer_size=134217728 --max_write_buffer_number=2 --target_file_size_base=33554432 --max_bytes_for_level_base=1073741824 --verify_checksum=false --max_background_compactions=4 --use_plain_table=0 --memtablerep=prefix_hash --open_files=-1 --mmap_read=1 --mmap_write=0 --bloom_bits=10 --bloom_locality=1 --memtable_bloom_bits=500000 --compression_type=lz4 --num=393216000 --use_hash_search=1 --block_size=1024 --block_restart_interval=16 --use_existing_db=1 --threads=1 --benchmarks=readrandom —disable_auto_compactions=1 Read QPS increase for about 30% from 2230002 to 2991411. Test Plan: make all check valgrind db_test db_stress --use_block_based_filter = 0 ./auto_sanity_test.sh Reviewers: igor, yhchiang, ljin, sdong Reviewed By: sdong Subscribers: dhruba, leveldb Differential Revision: https://reviews.facebook.net/D20979
10 years ago
}
}
bool GoodCompressionRatio(size_t compressed_size, size_t raw_size) {
// Check to see if compressed less than 12.5%
return compressed_size < raw_size - (raw_size / 8u);
}
bool CompressBlockInternal(const Slice& raw,
const CompressionInfo& compression_info,
uint32_t format_version,
std::string* compressed_output) {
bool ret;
// Will return compressed block contents if (1) the compression method is
// supported in this platform and (2) the compression rate is "good enough".
switch (compression_info.type()) {
case kSnappyCompression:
ret = Snappy_Compress(compression_info, raw.data(), raw.size(),
compressed_output);
break;
case kZlibCompression:
ret = Zlib_Compress(
compression_info,
GetCompressFormatForVersion(kZlibCompression, format_version),
raw.data(), raw.size(), compressed_output);
break;
case kBZip2Compression:
ret = BZip2_Compress(
compression_info,
GetCompressFormatForVersion(kBZip2Compression, format_version),
raw.data(), raw.size(), compressed_output);
break;
case kLZ4Compression:
ret = LZ4_Compress(
compression_info,
GetCompressFormatForVersion(kLZ4Compression, format_version),
raw.data(), raw.size(), compressed_output);
break;
case kLZ4HCCompression:
ret = LZ4HC_Compress(
compression_info,
GetCompressFormatForVersion(kLZ4HCCompression, format_version),
raw.data(), raw.size(), compressed_output);
break;
case kXpressCompression:
ret = XPRESS_Compress(raw.data(), raw.size(), compressed_output);
break;
case kZSTD:
case kZSTDNotFinalCompression:
ret = ZSTD_Compress(compression_info, raw.data(), raw.size(),
compressed_output);
break;
default:
// Do not recognize this compression type
ret = false;
}
TEST_SYNC_POINT_CALLBACK(
"BlockBasedTableBuilder::CompressBlockInternal:TamperWithReturnValue",
static_cast<void*>(&ret));
return ret;
}
} // namespace
// format_version is the block format as defined in include/rocksdb/table.h
Slice CompressBlock(const Slice& raw, 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) {
*type = info.type();
if (info.type() == kNoCompression && !info.SampleForCompression()) {
return raw;
}
// 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((int)info.SampleForCompression()) &&
sampled_output_fast && sampled_output_slow) {
// Sampling with a fast compression algorithm
if (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());
CompressBlockInternal(raw, info_tmp, format_version, sampled_output_fast);
}
// Sampling with a slow but high-compression algorithm
if (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());
CompressBlockInternal(raw, info_tmp, format_version, sampled_output_slow);
}
}
// Actually compress the data
if (*type != kNoCompression) {
if (CompressBlockInternal(raw, info, format_version, compressed_output) &&
GoodCompressionRatio(compressed_output->size(), raw.size())) {
return *compressed_output;
}
}
// Compression method is not supported, or not good
// compression ratio, so just fall back to uncompressed form.
*type = kNoCompression;
return raw;
}
// 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.
10 years ago
// 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 /* blockRawBytes */,
uint64_t /* blockCompressedBytesFast */,
uint64_t /* blockCompressedBytesSlow */) 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 ImmutableCFOptions ioptions;
const MutableCFOptions moptions;
const BlockBasedTableOptions table_options;
const InternalKeyComparator& internal_comparator;
WritableFileWriter* file;
std::atomic<uint64_t> offset;
// Synchronize status & io_status accesses across threads from main thread,
// compression thread and write thread in parallel compression.
std::mutex status_mutex;
size_t alignment;
BlockBuilder data_block;
Reduce scope of compression dictionary to single SST (#4952) Summary: Our previous approach was to train one compression dictionary per compaction, using the first output SST to train a dictionary, and then applying it on subsequent SSTs in the same compaction. While this was great for minimizing CPU/memory/I/O overhead, it did not achieve good compression ratios in practice. In our most promising potential use case, moderate reductions in a dictionary's scope make a major difference on compression ratio. So, this PR changes compression dictionary to be scoped per-SST. It accepts the tradeoff during table building to use more memory and CPU. Important changes include: - The `BlockBasedTableBuilder` has a new state when dictionary compression is in-use: `kBuffered`. In that state it accumulates uncompressed data in-memory whenever `Add` is called. - After accumulating target file size bytes or calling `BlockBasedTableBuilder::Finish`, a `BlockBasedTableBuilder` moves to the `kUnbuffered` state. The transition (`EnterUnbuffered()`) involves sampling the buffered data, training a dictionary, and compressing/writing out all buffered data. In the `kUnbuffered` state, a `BlockBasedTableBuilder` behaves the same as before -- blocks are compressed/written out as soon as they fill up. - Samples are now whole uncompressed data blocks, except the final sample may be a partial data block so we don't breach the user's configured `max_dict_bytes` or `zstd_max_train_bytes`. The dictionary trainer is supposed to work better when we pass it real units of compression. Previously we were passing 64-byte KV samples which was not realistic. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4952 Differential Revision: D13967980 Pulled By: ajkr fbshipit-source-id: 82bea6f7537e1529c7a1a4cdee84585f5949300f
6 years ago
// Buffers uncompressed data blocks and keys to replay later. Needed when
// compression dictionary is enabled so we can finalize the dictionary before
// compressing any data blocks.
// TODO(ajkr): ideally we don't buffer all keys and all uncompressed data
// blocks as it's redundant, but it's easier to implement for now.
std::vector<std::pair<std::string, std::vector<std::string>>>
data_block_and_keys_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;
CompressionOptions compression_opts;
Reduce scope of compression dictionary to single SST (#4952) Summary: Our previous approach was to train one compression dictionary per compaction, using the first output SST to train a dictionary, and then applying it on subsequent SSTs in the same compaction. While this was great for minimizing CPU/memory/I/O overhead, it did not achieve good compression ratios in practice. In our most promising potential use case, moderate reductions in a dictionary's scope make a major difference on compression ratio. So, this PR changes compression dictionary to be scoped per-SST. It accepts the tradeoff during table building to use more memory and CPU. Important changes include: - The `BlockBasedTableBuilder` has a new state when dictionary compression is in-use: `kBuffered`. In that state it accumulates uncompressed data in-memory whenever `Add` is called. - After accumulating target file size bytes or calling `BlockBasedTableBuilder::Finish`, a `BlockBasedTableBuilder` moves to the `kUnbuffered` state. The transition (`EnterUnbuffered()`) involves sampling the buffered data, training a dictionary, and compressing/writing out all buffered data. In the `kUnbuffered` state, a `BlockBasedTableBuilder` behaves the same as before -- blocks are compressed/written out as soon as they fill up. - Samples are now whole uncompressed data blocks, except the final sample may be a partial data block so we don't breach the user's configured `max_dict_bytes` or `zstd_max_train_bytes`. The dictionary trainer is supposed to work better when we pass it real units of compression. Previously we were passing 64-byte KV samples which was not realistic. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4952 Differential Revision: D13967980 Pulled By: ajkr fbshipit-source-id: 82bea6f7537e1529c7a1a4cdee84585f5949300f
6 years ago
std::unique_ptr<CompressionDict> compression_dict;
std::vector<std::unique_ptr<CompressionContext>> compression_ctxs;
std::vector<std::unique_ptr<UncompressionContext>> verify_ctxs;
Reduce scope of compression dictionary to single SST (#4952) Summary: Our previous approach was to train one compression dictionary per compaction, using the first output SST to train a dictionary, and then applying it on subsequent SSTs in the same compaction. While this was great for minimizing CPU/memory/I/O overhead, it did not achieve good compression ratios in practice. In our most promising potential use case, moderate reductions in a dictionary's scope make a major difference on compression ratio. So, this PR changes compression dictionary to be scoped per-SST. It accepts the tradeoff during table building to use more memory and CPU. Important changes include: - The `BlockBasedTableBuilder` has a new state when dictionary compression is in-use: `kBuffered`. In that state it accumulates uncompressed data in-memory whenever `Add` is called. - After accumulating target file size bytes or calling `BlockBasedTableBuilder::Finish`, a `BlockBasedTableBuilder` moves to the `kUnbuffered` state. The transition (`EnterUnbuffered()`) involves sampling the buffered data, training a dictionary, and compressing/writing out all buffered data. In the `kUnbuffered` state, a `BlockBasedTableBuilder` behaves the same as before -- blocks are compressed/written out as soon as they fill up. - Samples are now whole uncompressed data blocks, except the final sample may be a partial data block so we don't breach the user's configured `max_dict_bytes` or `zstd_max_train_bytes`. The dictionary trainer is supposed to work better when we pass it real units of compression. Previously we were passing 64-byte KV samples which was not realistic. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4952 Differential Revision: D13967980 Pulled By: ajkr fbshipit-source-id: 82bea6f7537e1529c7a1a4cdee84585f5949300f
6 years ago
std::unique_ptr<UncompressionDict> verify_dict;
size_t data_begin_offset = 0;
TableProperties props;
Reduce scope of compression dictionary to single SST (#4952) Summary: Our previous approach was to train one compression dictionary per compaction, using the first output SST to train a dictionary, and then applying it on subsequent SSTs in the same compaction. While this was great for minimizing CPU/memory/I/O overhead, it did not achieve good compression ratios in practice. In our most promising potential use case, moderate reductions in a dictionary's scope make a major difference on compression ratio. So, this PR changes compression dictionary to be scoped per-SST. It accepts the tradeoff during table building to use more memory and CPU. Important changes include: - The `BlockBasedTableBuilder` has a new state when dictionary compression is in-use: `kBuffered`. In that state it accumulates uncompressed data in-memory whenever `Add` is called. - After accumulating target file size bytes or calling `BlockBasedTableBuilder::Finish`, a `BlockBasedTableBuilder` moves to the `kUnbuffered` state. The transition (`EnterUnbuffered()`) involves sampling the buffered data, training a dictionary, and compressing/writing out all buffered data. In the `kUnbuffered` state, a `BlockBasedTableBuilder` behaves the same as before -- blocks are compressed/written out as soon as they fill up. - Samples are now whole uncompressed data blocks, except the final sample may be a partial data block so we don't breach the user's configured `max_dict_bytes` or `zstd_max_train_bytes`. The dictionary trainer is supposed to work better when we pass it real units of compression. Previously we were passing 64-byte KV samples which was not realistic. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4952 Differential Revision: D13967980 Pulled By: ajkr fbshipit-source-id: 82bea6f7537e1529c7a1a4cdee84585f5949300f
6 years ago
// 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;
const bool use_delta_encoding_for_index_values;
std::unique_ptr<FilterBlockBuilder> filter_builder;
char compressed_cache_key_prefix[BlockBasedTable::kMaxCacheKeyPrefixSize];
size_t compressed_cache_key_prefix_size;
BlockHandle pending_handle; // Handle to add to index block
std::string compressed_output;
std::unique_ptr<FlushBlockPolicy> flush_block_policy;
int level_at_creation;
uint32_t column_family_id;
const std::string& column_family_name;
FIFO Compaction with TTL Summary: Introducing FIFO compactions with TTL. FIFO compaction is based on size only which makes it tricky to enable in production as use cases can have organic growth. A user requested an option to drop files based on the time of their creation instead of the total size. To address that request: - Added a new TTL option to FIFO compaction options. - Updated FIFO compaction score to take TTL into consideration. - Added a new table property, creation_time, to keep track of when the SST file is created. - Creation_time is set as below: - On Flush: Set to the time of flush. - On Compaction: Set to the max creation_time of all the files involved in the compaction. - On Repair and Recovery: Set to the time of repair/recovery. - Old files created prior to this code change will have a creation_time of 0. - FIFO compaction with TTL is enabled when ttl > 0. All files older than ttl will be deleted during compaction. i.e. `if (file.creation_time < (current_time - ttl)) then delete(file)`. This will enable cases where you might want to delete all files older than, say, 1 day. - FIFO compaction will fall back to the prior way of deleting files based on size if: - the creation_time of all files involved in compaction is 0. - the total size (of all SST files combined) does not drop below `compaction_options_fifo.max_table_files_size` even if the files older than ttl are deleted. This feature is not supported if max_open_files != -1 or with table formats other than Block-based. **Test Plan:** Added tests. **Benchmark results:** Base: FIFO with max size: 100MB :: ``` svemuri@dev15905 ~/rocksdb (fifo-compaction) $ TEST_TMPDIR=/dev/shm ./db_bench --benchmarks=readwhilewriting --num=5000000 --threads=16 --compaction_style=2 --fifo_compaction_max_table_files_size_mb=100 readwhilewriting : 1.924 micros/op 519858 ops/sec; 13.6 MB/s (1176277 of 5000000 found) ``` With TTL (a low one for testing) :: ``` svemuri@dev15905 ~/rocksdb (fifo-compaction) $ TEST_TMPDIR=/dev/shm ./db_bench --benchmarks=readwhilewriting --num=5000000 --threads=16 --compaction_style=2 --fifo_compaction_max_table_files_size_mb=100 --fifo_compaction_ttl=20 readwhilewriting : 1.902 micros/op 525817 ops/sec; 13.7 MB/s (1185057 of 5000000 found) ``` Example Log lines: ``` 2017/06/26-15:17:24.609249 7fd5a45ff700 (Original Log Time 2017/06/26-15:17:24.609177) [db/compaction_picker.cc:1471] [default] FIFO compaction: picking file 40 with creation time 1498515423 for deletion 2017/06/26-15:17:24.609255 7fd5a45ff700 (Original Log Time 2017/06/26-15:17:24.609234) [db/db_impl_compaction_flush.cc:1541] [default] Deleted 1 files ... 2017/06/26-15:17:25.553185 7fd5a61a5800 [DEBUG] [db/db_impl_files.cc:309] [JOB 0] Delete /dev/shm/dbbench/000040.sst type=2 #40 -- OK 2017/06/26-15:17:25.553205 7fd5a61a5800 EVENT_LOG_v1 {"time_micros": 1498515445553199, "job": 0, "event": "table_file_deletion", "file_number": 40} ``` SST Files remaining in the dbbench dir, after db_bench execution completed: ``` svemuri@dev15905 ~/rocksdb (fifo-compaction) $ ls -l /dev/shm//dbbench/*.sst -rw-r--r--. 1 svemuri users 30749887 Jun 26 15:17 /dev/shm//dbbench/000042.sst -rw-r--r--. 1 svemuri users 30768779 Jun 26 15:17 /dev/shm//dbbench/000044.sst -rw-r--r--. 1 svemuri users 30757481 Jun 26 15:17 /dev/shm//dbbench/000046.sst ``` Closes https://github.com/facebook/rocksdb/pull/2480 Differential Revision: D5305116 Pulled By: sagar0 fbshipit-source-id: 3e5cfcf5dd07ed2211b5b37492eb235b45139174
7 years ago
uint64_t creation_time = 0;
uint64_t oldest_key_time = 0;
Reduce scope of compression dictionary to single SST (#4952) Summary: Our previous approach was to train one compression dictionary per compaction, using the first output SST to train a dictionary, and then applying it on subsequent SSTs in the same compaction. While this was great for minimizing CPU/memory/I/O overhead, it did not achieve good compression ratios in practice. In our most promising potential use case, moderate reductions in a dictionary's scope make a major difference on compression ratio. So, this PR changes compression dictionary to be scoped per-SST. It accepts the tradeoff during table building to use more memory and CPU. Important changes include: - The `BlockBasedTableBuilder` has a new state when dictionary compression is in-use: `kBuffered`. In that state it accumulates uncompressed data in-memory whenever `Add` is called. - After accumulating target file size bytes or calling `BlockBasedTableBuilder::Finish`, a `BlockBasedTableBuilder` moves to the `kUnbuffered` state. The transition (`EnterUnbuffered()`) involves sampling the buffered data, training a dictionary, and compressing/writing out all buffered data. In the `kUnbuffered` state, a `BlockBasedTableBuilder` behaves the same as before -- blocks are compressed/written out as soon as they fill up. - Samples are now whole uncompressed data blocks, except the final sample may be a partial data block so we don't breach the user's configured `max_dict_bytes` or `zstd_max_train_bytes`. The dictionary trainer is supposed to work better when we pass it real units of compression. Previously we were passing 64-byte KV samples which was not realistic. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4952 Differential Revision: D13967980 Pulled By: ajkr fbshipit-source-id: 82bea6f7537e1529c7a1a4cdee84585f5949300f
6 years ago
const uint64_t target_file_size;
Periodic Compactions (#5166) Summary: Introducing Periodic Compactions. This feature allows all the files in a CF to be periodically compacted. It could help in catching any corruptions that could creep into the DB proactively as every file is constantly getting re-compacted. And also, of course, it helps to cleanup data older than certain threshold. - Introduced a new option `periodic_compaction_time` to control how long a file can live without being compacted in a CF. - This works across all levels. - The files are put in the same level after going through the compaction. (Related files in the same level are picked up as `ExpandInputstoCleanCut` is used). - Compaction filters, if any, are invoked as usual. - A new table property, `file_creation_time`, is introduced to implement this feature. This property is set to the time at which the SST file was created (and that time is given by the underlying Env/OS). This feature can be enabled on its own, or in conjunction with `ttl`. It is possible to set a different time threshold for the bottom level when used in conjunction with ttl. Since `ttl` works only on 0 to last but one levels, you could set `ttl` to, say, 1 day, and `periodic_compaction_time` to, say, 7 days. Since `ttl < periodic_compaction_time` all files in last but one levels keep getting picked up based on ttl, and almost never based on periodic_compaction_time. The files in the bottom level get picked up for compaction based on `periodic_compaction_time`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5166 Differential Revision: D14884441 Pulled By: sagar0 fbshipit-source-id: 408426cbacb409c06386a98632dcf90bfa1bda47
6 years ago
uint64_t file_creation_time = 0;
// DB IDs
const std::string db_id;
const std::string db_session_id;
std::vector<std::unique_ptr<IntTblPropCollector>> table_properties_collectors;
TablePropertiesCollectorFactory Summary: This diff addresses task #4296714 and rethinks how users provide us with TablePropertiesCollectors as part of Options. Here's description of task #4296714: I'm debugging #4295529 and noticed that our count of user properties kDeletedKeys is wrong. We're sharing one single InternalKeyPropertiesCollector with all Table Builders. In LOG Files, we're outputting number of kDeletedKeys as connected with a single table, while it's actually the total count of deleted keys since creation of the DB. For example, this table has 3155 entries and 1391828 deleted keys. The problem with current approach that we call methods on a single TablePropertiesCollector for all the tables we create. Even worse, we could do it from multiple threads at the same time and TablePropertiesCollector has no way of knowing which table we're calling it for. Good part: Looks like nobody inside Facebook is using Options::table_properties_collectors. This means we should be able to painfully change the API. In this change, I introduce TablePropertiesCollectorFactory. For every table we create, we call `CreateTablePropertiesCollector`, which creates a TablePropertiesCollector for a single table. We then use it sequentially from a single thread, which means it doesn't have to be thread-safe. Test Plan: Added a test in table_properties_collector_test that fails on master (build two tables, assert that kDeletedKeys count is correct for the second one). Also, all other tests Reviewers: sdong, dhruba, haobo, kailiu Reviewed By: kailiu CC: leveldb Differential Revision: https://reviews.facebook.net/D18579
11 years ago
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); }
const IOStatus& GetIOStatus() {
if (compression_opts.parallel_threads > 1) {
std::lock_guard<std::mutex> lock(status_mutex);
return io_status;
} else {
return io_status;
}
}
const Status& GetStatus() {
if (compression_opts.parallel_threads > 1) {
std::lock_guard<std::mutex> lock(status_mutex);
return status;
} else {
return status;
}
}
void SyncStatusFromIOStatus() {
if (compression_opts.parallel_threads > 1) {
std::lock_guard<std::mutex> lock(status_mutex);
if (status.ok()) {
status = io_status;
}
} else if (status.ok()) {
status = io_status;
}
}
// Never erase an existing status that is not OK.
void SetStatus(Status s) {
if (!s.ok()) {
// 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);
if (status.ok()) {
status = s;
}
}
}
// Never erase an existing I/O status that is not OK.
void SetIOStatus(IOStatus ios) {
if (!ios.ok()) {
// 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);
if (io_status.ok()) {
io_status = ios;
}
}
}
Rep(const ImmutableCFOptions& _ioptions, const MutableCFOptions& _moptions,
const BlockBasedTableOptions& table_opt,
const InternalKeyComparator& icomparator,
const std::vector<std::unique_ptr<IntTblPropCollectorFactory>>*
int_tbl_prop_collector_factories,
uint32_t _column_family_id, WritableFileWriter* f,
const CompressionType _compression_type,
const uint64_t _sample_for_compression,
Reduce scope of compression dictionary to single SST (#4952) Summary: Our previous approach was to train one compression dictionary per compaction, using the first output SST to train a dictionary, and then applying it on subsequent SSTs in the same compaction. While this was great for minimizing CPU/memory/I/O overhead, it did not achieve good compression ratios in practice. In our most promising potential use case, moderate reductions in a dictionary's scope make a major difference on compression ratio. So, this PR changes compression dictionary to be scoped per-SST. It accepts the tradeoff during table building to use more memory and CPU. Important changes include: - The `BlockBasedTableBuilder` has a new state when dictionary compression is in-use: `kBuffered`. In that state it accumulates uncompressed data in-memory whenever `Add` is called. - After accumulating target file size bytes or calling `BlockBasedTableBuilder::Finish`, a `BlockBasedTableBuilder` moves to the `kUnbuffered` state. The transition (`EnterUnbuffered()`) involves sampling the buffered data, training a dictionary, and compressing/writing out all buffered data. In the `kUnbuffered` state, a `BlockBasedTableBuilder` behaves the same as before -- blocks are compressed/written out as soon as they fill up. - Samples are now whole uncompressed data blocks, except the final sample may be a partial data block so we don't breach the user's configured `max_dict_bytes` or `zstd_max_train_bytes`. The dictionary trainer is supposed to work better when we pass it real units of compression. Previously we were passing 64-byte KV samples which was not realistic. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4952 Differential Revision: D13967980 Pulled By: ajkr fbshipit-source-id: 82bea6f7537e1529c7a1a4cdee84585f5949300f
6 years ago
const CompressionOptions& _compression_opts, const bool skip_filters,
const int _level_at_creation, const std::string& _column_family_name,
const uint64_t _creation_time, const uint64_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)
: ioptions(_ioptions),
moptions(_moptions),
table_options(table_opt),
internal_comparator(icomparator),
file(f),
offset(0),
alignment(table_options.block_align
? std::min(table_options.block_size, kDefaultPageSize)
: 0),
Introduce ReadOptions::pin_data (support zero copy for keys) Summary: This patch update the Iterator API to introduce new functions that allow users to keep the Slices returned by key() valid as long as the Iterator is not deleted ReadOptions::pin_data : If true keep loaded blocks in memory as long as the iterator is not deleted Iterator::IsKeyPinned() : If true, this mean that the Slice returned by key() is valid as long as the iterator is not deleted Also add a new option BlockBasedTableOptions::use_delta_encoding to allow users to disable delta_encoding if needed. Benchmark results (using https://phabricator.fb.com/P20083553) ``` // $ du -h /home/tec/local/normal.4K.Snappy/db10077 // 6.1G /home/tec/local/normal.4K.Snappy/db10077 // $ du -h /home/tec/local/zero.8K.LZ4/db10077 // 6.4G /home/tec/local/zero.8K.LZ4/db10077 // Benchmarks for shard db10077 // _build/opt/rocks/benchmark/rocks_copy_benchmark \ // --normal_db_path="/home/tec/local/normal.4K.Snappy/db10077" \ // --zero_db_path="/home/tec/local/zero.8K.LZ4/db10077" // First run // ============================================================================ // rocks/benchmark/RocksCopyBenchmark.cpp relative time/iter iters/s // ============================================================================ // BM_StringCopy 1.73s 576.97m // BM_StringPiece 103.74% 1.67s 598.55m // ============================================================================ // Match rate : 1000000 / 1000000 // Second run // ============================================================================ // rocks/benchmark/RocksCopyBenchmark.cpp relative time/iter iters/s // ============================================================================ // BM_StringCopy 611.99ms 1.63 // BM_StringPiece 203.76% 300.35ms 3.33 // ============================================================================ // Match rate : 1000000 / 1000000 ``` Test Plan: Unit tests Reviewers: sdong, igor, anthony, yhchiang, rven Reviewed By: rven Subscribers: dhruba, lovro, adsharma Differential Revision: https://reviews.facebook.net/D48999
9 years ago
data_block(table_options.block_restart_interval,
table_options.use_delta_encoding,
false /* use_value_delta_encoding */,
icomparator.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(_moptions.prefix_extractor.get()),
compression_type(_compression_type),
sample_for_compression(_sample_for_compression),
compression_opts(_compression_opts),
Reduce scope of compression dictionary to single SST (#4952) Summary: Our previous approach was to train one compression dictionary per compaction, using the first output SST to train a dictionary, and then applying it on subsequent SSTs in the same compaction. While this was great for minimizing CPU/memory/I/O overhead, it did not achieve good compression ratios in practice. In our most promising potential use case, moderate reductions in a dictionary's scope make a major difference on compression ratio. So, this PR changes compression dictionary to be scoped per-SST. It accepts the tradeoff during table building to use more memory and CPU. Important changes include: - The `BlockBasedTableBuilder` has a new state when dictionary compression is in-use: `kBuffered`. In that state it accumulates uncompressed data in-memory whenever `Add` is called. - After accumulating target file size bytes or calling `BlockBasedTableBuilder::Finish`, a `BlockBasedTableBuilder` moves to the `kUnbuffered` state. The transition (`EnterUnbuffered()`) involves sampling the buffered data, training a dictionary, and compressing/writing out all buffered data. In the `kUnbuffered` state, a `BlockBasedTableBuilder` behaves the same as before -- blocks are compressed/written out as soon as they fill up. - Samples are now whole uncompressed data blocks, except the final sample may be a partial data block so we don't breach the user's configured `max_dict_bytes` or `zstd_max_train_bytes`. The dictionary trainer is supposed to work better when we pass it real units of compression. Previously we were passing 64-byte KV samples which was not realistic. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4952 Differential Revision: D13967980 Pulled By: ajkr fbshipit-source-id: 82bea6f7537e1529c7a1a4cdee84585f5949300f
6 years ago
compression_dict(),
compression_ctxs(_compression_opts.parallel_threads),
verify_ctxs(_compression_opts.parallel_threads),
Reduce scope of compression dictionary to single SST (#4952) Summary: Our previous approach was to train one compression dictionary per compaction, using the first output SST to train a dictionary, and then applying it on subsequent SSTs in the same compaction. While this was great for minimizing CPU/memory/I/O overhead, it did not achieve good compression ratios in practice. In our most promising potential use case, moderate reductions in a dictionary's scope make a major difference on compression ratio. So, this PR changes compression dictionary to be scoped per-SST. It accepts the tradeoff during table building to use more memory and CPU. Important changes include: - The `BlockBasedTableBuilder` has a new state when dictionary compression is in-use: `kBuffered`. In that state it accumulates uncompressed data in-memory whenever `Add` is called. - After accumulating target file size bytes or calling `BlockBasedTableBuilder::Finish`, a `BlockBasedTableBuilder` moves to the `kUnbuffered` state. The transition (`EnterUnbuffered()`) involves sampling the buffered data, training a dictionary, and compressing/writing out all buffered data. In the `kUnbuffered` state, a `BlockBasedTableBuilder` behaves the same as before -- blocks are compressed/written out as soon as they fill up. - Samples are now whole uncompressed data blocks, except the final sample may be a partial data block so we don't breach the user's configured `max_dict_bytes` or `zstd_max_train_bytes`. The dictionary trainer is supposed to work better when we pass it real units of compression. Previously we were passing 64-byte KV samples which was not realistic. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4952 Differential Revision: D13967980 Pulled By: ajkr fbshipit-source-id: 82bea6f7537e1529c7a1a4cdee84585f5949300f
6 years ago
verify_dict(),
state((_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),
table: Fix coverity issues Summary: table/block.cc: 420 } CID 1396127 (#1 of 1): Uninitialized scalar field (UNINIT_CTOR) 7. uninit_member: Non-static class member restart_offset_ is not initialized in this constructor nor in any functions that it calls. 421} table/block_based_table_builder.cc: CID 1418259 (#1 of 1): Uninitialized scalar field (UNINIT_CTOR) 7. uninit_member: Non-static class member compressed_cache_key_prefix_size is not initialized in this constructor nor in any functions that it calls. table/block_based_table_reader.h: 3. uninit_member: Non-static class member index_type is not initialized in this constructor nor in any functions that it calls. CID 1396147 (#1 of 1): Uninitialized scalar field (UNINIT_CTOR) 5. uninit_member: Non-static class member hash_index_allow_collision is not initialized in this constructor nor in any functions that it calls. 413 global_seqno(kDisableGlobalSequenceNumber) {} 414 table/cuckoo_table_reader.cc: 55 if (hash_funs == user_props.end()) { 56 status_ = Status::Corruption("Number of hash functions not found"); 5. uninit_member: Non-static class member is_last_level_ is not initialized in this constructor nor in any functions that it calls. 7. uninit_member: Non-static class member identity_as_first_hash_ is not initialized in this constructor nor in any functions that it calls. 9. uninit_member: Non-static class member use_module_hash_ is not initialized in this constructor nor in any functions that it calls. 11. uninit_member: Non-static class member num_hash_func_ is not initialized in this constructor nor in any functions that it calls. 13. uninit_member: Non-static class member key_length_ is not initialized in this constructor nor in any functions that it calls. 15. uninit_member: Non-static class member user_key_length_ is not initialized in this constructor nor in any functions that it calls. 17. uninit_member: Non-static class member value_length_ is not initialized in this constructor nor in any functions that it calls. 19. uninit_member: Non-static class member bucket_length_ is not initialized in this constructor nor in any functions that it calls. 21. uninit_member: Non-static class member cuckoo_block_size_ is not initialized in this constructor nor in any functions that it calls. 23. uninit_member: Non-static class member cuckoo_block_bytes_minus_one_ is not initialized in this constructor nor in any functions that it calls. CID 1322785 (#2 of 2): Uninitialized scalar field (UNINIT_CTOR) 25. uninit_member: Non-static class member table_size_ is not initialized in this constructor nor in any functions that it calls. 57 return; table/plain_table_index.h: 2. uninit_member: Non-static class member index_size_ is not initialized in this constructor nor in any functions that it calls. CID 1322801 (#1 of 1): Uninitialized scalar field (UNINIT_CTOR) 4. uninit_member: Non-static class member sub_index_size_ is not initialized in this constructor nor in any functions that it calls. 128 huge_page_tlb_size_(huge_page_tlb_size) {} 129 Closes https://github.com/facebook/rocksdb/pull/3113 Differential Revision: D6505719 Pulled By: yiwu-arbug fbshipit-source-id: 38f44d8f9dfefb4c2e25d83b8df25a5201c75618
7 years ago
compressed_cache_key_prefix_size(0),
flush_block_policy(
table_options.flush_block_policy_factory->NewFlushBlockPolicy(
table_options, data_block)),
level_at_creation(_level_at_creation),
column_family_id(_column_family_id),
FIFO Compaction with TTL Summary: Introducing FIFO compactions with TTL. FIFO compaction is based on size only which makes it tricky to enable in production as use cases can have organic growth. A user requested an option to drop files based on the time of their creation instead of the total size. To address that request: - Added a new TTL option to FIFO compaction options. - Updated FIFO compaction score to take TTL into consideration. - Added a new table property, creation_time, to keep track of when the SST file is created. - Creation_time is set as below: - On Flush: Set to the time of flush. - On Compaction: Set to the max creation_time of all the files involved in the compaction. - On Repair and Recovery: Set to the time of repair/recovery. - Old files created prior to this code change will have a creation_time of 0. - FIFO compaction with TTL is enabled when ttl > 0. All files older than ttl will be deleted during compaction. i.e. `if (file.creation_time < (current_time - ttl)) then delete(file)`. This will enable cases where you might want to delete all files older than, say, 1 day. - FIFO compaction will fall back to the prior way of deleting files based on size if: - the creation_time of all files involved in compaction is 0. - the total size (of all SST files combined) does not drop below `compaction_options_fifo.max_table_files_size` even if the files older than ttl are deleted. This feature is not supported if max_open_files != -1 or with table formats other than Block-based. **Test Plan:** Added tests. **Benchmark results:** Base: FIFO with max size: 100MB :: ``` svemuri@dev15905 ~/rocksdb (fifo-compaction) $ TEST_TMPDIR=/dev/shm ./db_bench --benchmarks=readwhilewriting --num=5000000 --threads=16 --compaction_style=2 --fifo_compaction_max_table_files_size_mb=100 readwhilewriting : 1.924 micros/op 519858 ops/sec; 13.6 MB/s (1176277 of 5000000 found) ``` With TTL (a low one for testing) :: ``` svemuri@dev15905 ~/rocksdb (fifo-compaction) $ TEST_TMPDIR=/dev/shm ./db_bench --benchmarks=readwhilewriting --num=5000000 --threads=16 --compaction_style=2 --fifo_compaction_max_table_files_size_mb=100 --fifo_compaction_ttl=20 readwhilewriting : 1.902 micros/op 525817 ops/sec; 13.7 MB/s (1185057 of 5000000 found) ``` Example Log lines: ``` 2017/06/26-15:17:24.609249 7fd5a45ff700 (Original Log Time 2017/06/26-15:17:24.609177) [db/compaction_picker.cc:1471] [default] FIFO compaction: picking file 40 with creation time 1498515423 for deletion 2017/06/26-15:17:24.609255 7fd5a45ff700 (Original Log Time 2017/06/26-15:17:24.609234) [db/db_impl_compaction_flush.cc:1541] [default] Deleted 1 files ... 2017/06/26-15:17:25.553185 7fd5a61a5800 [DEBUG] [db/db_impl_files.cc:309] [JOB 0] Delete /dev/shm/dbbench/000040.sst type=2 #40 -- OK 2017/06/26-15:17:25.553205 7fd5a61a5800 EVENT_LOG_v1 {"time_micros": 1498515445553199, "job": 0, "event": "table_file_deletion", "file_number": 40} ``` SST Files remaining in the dbbench dir, after db_bench execution completed: ``` svemuri@dev15905 ~/rocksdb (fifo-compaction) $ ls -l /dev/shm//dbbench/*.sst -rw-r--r--. 1 svemuri users 30749887 Jun 26 15:17 /dev/shm//dbbench/000042.sst -rw-r--r--. 1 svemuri users 30768779 Jun 26 15:17 /dev/shm//dbbench/000044.sst -rw-r--r--. 1 svemuri users 30757481 Jun 26 15:17 /dev/shm//dbbench/000046.sst ``` Closes https://github.com/facebook/rocksdb/pull/2480 Differential Revision: D5305116 Pulled By: sagar0 fbshipit-source-id: 3e5cfcf5dd07ed2211b5b37492eb235b45139174
7 years ago
column_family_name(_column_family_name),
creation_time(_creation_time),
Reduce scope of compression dictionary to single SST (#4952) Summary: Our previous approach was to train one compression dictionary per compaction, using the first output SST to train a dictionary, and then applying it on subsequent SSTs in the same compaction. While this was great for minimizing CPU/memory/I/O overhead, it did not achieve good compression ratios in practice. In our most promising potential use case, moderate reductions in a dictionary's scope make a major difference on compression ratio. So, this PR changes compression dictionary to be scoped per-SST. It accepts the tradeoff during table building to use more memory and CPU. Important changes include: - The `BlockBasedTableBuilder` has a new state when dictionary compression is in-use: `kBuffered`. In that state it accumulates uncompressed data in-memory whenever `Add` is called. - After accumulating target file size bytes or calling `BlockBasedTableBuilder::Finish`, a `BlockBasedTableBuilder` moves to the `kUnbuffered` state. The transition (`EnterUnbuffered()`) involves sampling the buffered data, training a dictionary, and compressing/writing out all buffered data. In the `kUnbuffered` state, a `BlockBasedTableBuilder` behaves the same as before -- blocks are compressed/written out as soon as they fill up. - Samples are now whole uncompressed data blocks, except the final sample may be a partial data block so we don't breach the user's configured `max_dict_bytes` or `zstd_max_train_bytes`. The dictionary trainer is supposed to work better when we pass it real units of compression. Previously we were passing 64-byte KV samples which was not realistic. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4952 Differential Revision: D13967980 Pulled By: ajkr fbshipit-source-id: 82bea6f7537e1529c7a1a4cdee84585f5949300f
6 years ago
oldest_key_time(_oldest_key_time),
Periodic Compactions (#5166) Summary: Introducing Periodic Compactions. This feature allows all the files in a CF to be periodically compacted. It could help in catching any corruptions that could creep into the DB proactively as every file is constantly getting re-compacted. And also, of course, it helps to cleanup data older than certain threshold. - Introduced a new option `periodic_compaction_time` to control how long a file can live without being compacted in a CF. - This works across all levels. - The files are put in the same level after going through the compaction. (Related files in the same level are picked up as `ExpandInputstoCleanCut` is used). - Compaction filters, if any, are invoked as usual. - A new table property, `file_creation_time`, is introduced to implement this feature. This property is set to the time at which the SST file was created (and that time is given by the underlying Env/OS). This feature can be enabled on its own, or in conjunction with `ttl`. It is possible to set a different time threshold for the bottom level when used in conjunction with ttl. Since `ttl` works only on 0 to last but one levels, you could set `ttl` to, say, 1 day, and `periodic_compaction_time` to, say, 7 days. Since `ttl < periodic_compaction_time` all files in last but one levels keep getting picked up based on ttl, and almost never based on periodic_compaction_time. The files in the bottom level get picked up for compaction based on `periodic_compaction_time`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5166 Differential Revision: D14884441 Pulled By: sagar0 fbshipit-source-id: 408426cbacb409c06386a98632dcf90bfa1bda47
6 years ago
target_file_size(_target_file_size),
file_creation_time(_file_creation_time),
db_id(_db_id),
db_session_id(_db_session_id) {
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 (skip_filters) {
filter_builder = nullptr;
} else {
FilterBuildingContext context(table_options);
context.column_family_name = column_family_name;
context.compaction_style = ioptions.compaction_style;
context.level_at_creation = level_at_creation;
context.info_log = ioptions.info_log;
filter_builder.reset(CreateFilterBlockBuilder(
ioptions, moptions, context, use_delta_encoding_for_index_values,
p_index_builder_));
}
for (auto& collector_factories : *int_tbl_prop_collector_factories) {
TablePropertiesCollectorFactory Summary: This diff addresses task #4296714 and rethinks how users provide us with TablePropertiesCollectors as part of Options. Here's description of task #4296714: I'm debugging #4295529 and noticed that our count of user properties kDeletedKeys is wrong. We're sharing one single InternalKeyPropertiesCollector with all Table Builders. In LOG Files, we're outputting number of kDeletedKeys as connected with a single table, while it's actually the total count of deleted keys since creation of the DB. For example, this table has 3155 entries and 1391828 deleted keys. The problem with current approach that we call methods on a single TablePropertiesCollector for all the tables we create. Even worse, we could do it from multiple threads at the same time and TablePropertiesCollector has no way of knowing which table we're calling it for. Good part: Looks like nobody inside Facebook is using Options::table_properties_collectors. This means we should be able to painfully change the API. In this change, I introduce TablePropertiesCollectorFactory. For every table we create, we call `CreateTablePropertiesCollector`, which creates a TablePropertiesCollector for a single table. We then use it sequentially from a single thread, which means it doesn't have to be thread-safe. Test Plan: Added a test in table_properties_collector_test that fails on master (build two tables, assert that kDeletedKeys count is correct for the second one). Also, all other tests Reviewers: sdong, dhruba, haobo, kailiu Reviewed By: kailiu CC: leveldb Differential Revision: https://reviews.facebook.net/D18579
11 years ago
table_properties_collectors.emplace_back(
collector_factories->CreateIntTblPropCollector(column_family_id));
TablePropertiesCollectorFactory Summary: This diff addresses task #4296714 and rethinks how users provide us with TablePropertiesCollectors as part of Options. Here's description of task #4296714: I'm debugging #4295529 and noticed that our count of user properties kDeletedKeys is wrong. We're sharing one single InternalKeyPropertiesCollector with all Table Builders. In LOG Files, we're outputting number of kDeletedKeys as connected with a single table, while it's actually the total count of deleted keys since creation of the DB. For example, this table has 3155 entries and 1391828 deleted keys. The problem with current approach that we call methods on a single TablePropertiesCollector for all the tables we create. Even worse, we could do it from multiple threads at the same time and TablePropertiesCollector has no way of knowing which table we're calling it for. Good part: Looks like nobody inside Facebook is using Options::table_properties_collectors. This means we should be able to painfully change the API. In this change, I introduce TablePropertiesCollectorFactory. For every table we create, we call `CreateTablePropertiesCollector`, which creates a TablePropertiesCollector for a single table. We then use it sequentially from a single thread, which means it doesn't have to be thread-safe. Test Plan: Added a test in table_properties_collector_test that fails on master (build two tables, assert that kDeletedKeys count is correct for the second one). Also, all other tests Reviewers: sdong, dhruba, haobo, kailiu Reviewed By: kailiu CC: leveldb Differential Revision: https://reviews.facebook.net/D18579
11 years ago
}
table_properties_collectors.emplace_back(
new BlockBasedTablePropertiesCollector(
table_options.index_type, table_options.whole_key_filtering,
_moptions.prefix_extractor != nullptr));
if (table_options.verify_compression) {
for (uint32_t i = 0; i < compression_opts.parallel_threads; i++) {
verify_ctxs[i].reset(new UncompressionContext(compression_type));
}
}
}
Rep(const Rep&) = delete;
Rep& operator=(const Rep&) = delete;
~Rep() {}
private:
Status status;
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.
typedef std::vector<BlockRep> BlockRepBuffer;
BlockRepBuffer block_rep_buf;
// Use a thread-safe queue for concurrent access from block
// building thread and writer thread.
typedef WorkQueue<BlockRep*> BlockRepPool;
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.
typedef WorkQueue<BlockRep*> CompressQueue;
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.
typedef WorkQueue<BlockRepSlot*> WriteQueue;
WriteQueue write_queue;
std::unique_ptr<port::Thread> write_thread;
// Raw bytes compressed so far.
uint64_t raw_bytes_compressed;
// Size of current block being appended.
uint64_t raw_bytes_curr_block;
// Raw bytes under compression and not appended yet.
std::atomic<uint64_t> raw_bytes_inflight;
// Number of blocks under compression and not appended yet.
std::atomic<uint64_t> blocks_inflight;
// Current compression ratio, maintained by BGWorkWriteRawBlock.
std::atomic<double> curr_compression_ratio;
// Estimated SST file size.
std::atomic<uint64_t> estimated_file_size;
// Wait for the completion of first block compression to get a
// non-zero compression ratio.
bool first_block;
std::condition_variable first_block_cond;
std::mutex first_block_mutex;
bool finished;
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),
raw_bytes_compressed(0),
raw_bytes_curr_block(0),
raw_bytes_inflight(0),
blocks_inflight(0),
curr_compression_ratio(0),
estimated_file_size(0),
first_block(true),
finished(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(); }
};
BlockBasedTableBuilder::BlockBasedTableBuilder(
const ImmutableCFOptions& ioptions, const MutableCFOptions& moptions,
const BlockBasedTableOptions& table_options,
const InternalKeyComparator& internal_comparator,
const std::vector<std::unique_ptr<IntTblPropCollectorFactory>>*
int_tbl_prop_collector_factories,
uint32_t column_family_id, WritableFileWriter* file,
const CompressionType compression_type,
const uint64_t sample_for_compression,
Reduce scope of compression dictionary to single SST (#4952) Summary: Our previous approach was to train one compression dictionary per compaction, using the first output SST to train a dictionary, and then applying it on subsequent SSTs in the same compaction. While this was great for minimizing CPU/memory/I/O overhead, it did not achieve good compression ratios in practice. In our most promising potential use case, moderate reductions in a dictionary's scope make a major difference on compression ratio. So, this PR changes compression dictionary to be scoped per-SST. It accepts the tradeoff during table building to use more memory and CPU. Important changes include: - The `BlockBasedTableBuilder` has a new state when dictionary compression is in-use: `kBuffered`. In that state it accumulates uncompressed data in-memory whenever `Add` is called. - After accumulating target file size bytes or calling `BlockBasedTableBuilder::Finish`, a `BlockBasedTableBuilder` moves to the `kUnbuffered` state. The transition (`EnterUnbuffered()`) involves sampling the buffered data, training a dictionary, and compressing/writing out all buffered data. In the `kUnbuffered` state, a `BlockBasedTableBuilder` behaves the same as before -- blocks are compressed/written out as soon as they fill up. - Samples are now whole uncompressed data blocks, except the final sample may be a partial data block so we don't breach the user's configured `max_dict_bytes` or `zstd_max_train_bytes`. The dictionary trainer is supposed to work better when we pass it real units of compression. Previously we were passing 64-byte KV samples which was not realistic. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4952 Differential Revision: D13967980 Pulled By: ajkr fbshipit-source-id: 82bea6f7537e1529c7a1a4cdee84585f5949300f
6 years ago
const CompressionOptions& compression_opts, const bool skip_filters,
const std::string& column_family_name, const int level_at_creation,
const uint64_t creation_time, const uint64_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) {
BlockBasedTableOptions sanitized_table_options(table_options);
if (sanitized_table_options.format_version == 0 &&
sanitized_table_options.checksum != kCRC32c) {
ROCKS_LOG_WARN(
ioptions.info_log,
"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(
ioptions, moptions, sanitized_table_options, internal_comparator,
int_tbl_prop_collector_factories, column_family_id, file,
compression_type, sample_for_compression, compression_opts, skip_filters,
level_at_creation, column_family_name, creation_time, oldest_key_time,
target_file_size, file_creation_time, db_id, db_session_id);
if (rep_->filter_builder != nullptr) {
rep_->filter_builder->StartBlock(0);
}
if (table_options.block_cache_compressed.get() != nullptr) {
BlockBasedTable::GenerateCachePrefix<Cache, FSWritableFile>(
table_options.block_cache_compressed.get(), file->writable_file(),
&rep_->compressed_cache_key_prefix[0],
&rep_->compressed_cache_key_prefix_size);
}
if (rep_->compression_opts.parallel_threads > 1) {
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] { BGWorkWriteRawBlock(); }));
}
}
BlockBasedTableBuilder::~BlockBasedTableBuilder() {
Reduce scope of compression dictionary to single SST (#4952) Summary: Our previous approach was to train one compression dictionary per compaction, using the first output SST to train a dictionary, and then applying it on subsequent SSTs in the same compaction. While this was great for minimizing CPU/memory/I/O overhead, it did not achieve good compression ratios in practice. In our most promising potential use case, moderate reductions in a dictionary's scope make a major difference on compression ratio. So, this PR changes compression dictionary to be scoped per-SST. It accepts the tradeoff during table building to use more memory and CPU. Important changes include: - The `BlockBasedTableBuilder` has a new state when dictionary compression is in-use: `kBuffered`. In that state it accumulates uncompressed data in-memory whenever `Add` is called. - After accumulating target file size bytes or calling `BlockBasedTableBuilder::Finish`, a `BlockBasedTableBuilder` moves to the `kUnbuffered` state. The transition (`EnterUnbuffered()`) involves sampling the buffered data, training a dictionary, and compressing/writing out all buffered data. In the `kUnbuffered` state, a `BlockBasedTableBuilder` behaves the same as before -- blocks are compressed/written out as soon as they fill up. - Samples are now whole uncompressed data blocks, except the final sample may be a partial data block so we don't breach the user's configured `max_dict_bytes` or `zstd_max_train_bytes`. The dictionary trainer is supposed to work better when we pass it real units of compression. Previously we were passing 64-byte KV samples which was not realistic. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4952 Differential Revision: D13967980 Pulled By: ajkr fbshipit-source-id: 82bea6f7537e1529c7a1a4cdee84585f5949300f
6 years ago
// 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_;
Reduce scope of compression dictionary to single SST (#4952) Summary: Our previous approach was to train one compression dictionary per compaction, using the first output SST to train a dictionary, and then applying it on subsequent SSTs in the same compaction. While this was great for minimizing CPU/memory/I/O overhead, it did not achieve good compression ratios in practice. In our most promising potential use case, moderate reductions in a dictionary's scope make a major difference on compression ratio. So, this PR changes compression dictionary to be scoped per-SST. It accepts the tradeoff during table building to use more memory and CPU. Important changes include: - The `BlockBasedTableBuilder` has a new state when dictionary compression is in-use: `kBuffered`. In that state it accumulates uncompressed data in-memory whenever `Add` is called. - After accumulating target file size bytes or calling `BlockBasedTableBuilder::Finish`, a `BlockBasedTableBuilder` moves to the `kUnbuffered` state. The transition (`EnterUnbuffered()`) involves sampling the buffered data, training a dictionary, and compressing/writing out all buffered data. In the `kUnbuffered` state, a `BlockBasedTableBuilder` behaves the same as before -- blocks are compressed/written out as soon as they fill up. - Samples are now whole uncompressed data blocks, except the final sample may be a partial data block so we don't breach the user's configured `max_dict_bytes` or `zstd_max_train_bytes`. The dictionary trainer is supposed to work better when we pass it real units of compression. Previously we were passing 64-byte KV samples which was not realistic. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4952 Differential Revision: D13967980 Pulled By: ajkr fbshipit-source-id: 82bea6f7537e1529c7a1a4cdee84585f5949300f
6 years ago
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();
Reduce scope of compression dictionary to single SST (#4952) Summary: Our previous approach was to train one compression dictionary per compaction, using the first output SST to train a dictionary, and then applying it on subsequent SSTs in the same compaction. While this was great for minimizing CPU/memory/I/O overhead, it did not achieve good compression ratios in practice. In our most promising potential use case, moderate reductions in a dictionary's scope make a major difference on compression ratio. So, this PR changes compression dictionary to be scoped per-SST. It accepts the tradeoff during table building to use more memory and CPU. Important changes include: - The `BlockBasedTableBuilder` has a new state when dictionary compression is in-use: `kBuffered`. In that state it accumulates uncompressed data in-memory whenever `Add` is called. - After accumulating target file size bytes or calling `BlockBasedTableBuilder::Finish`, a `BlockBasedTableBuilder` moves to the `kUnbuffered` state. The transition (`EnterUnbuffered()`) involves sampling the buffered data, training a dictionary, and compressing/writing out all buffered data. In the `kUnbuffered` state, a `BlockBasedTableBuilder` behaves the same as before -- blocks are compressed/written out as soon as they fill up. - Samples are now whole uncompressed data blocks, except the final sample may be a partial data block so we don't breach the user's configured `max_dict_bytes` or `zstd_max_train_bytes`. The dictionary trainer is supposed to work better when we pass it real units of compression. Previously we were passing 64-byte KV samples which was not realistic. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4952 Differential Revision: D13967980 Pulled By: ajkr fbshipit-source-id: 82bea6f7537e1529c7a1a4cdee84585f5949300f
6 years ago
if (r->state == Rep::State::kBuffered &&
r->data_begin_offset > r->target_file_size) {
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.
Reduce scope of compression dictionary to single SST (#4952) Summary: Our previous approach was to train one compression dictionary per compaction, using the first output SST to train a dictionary, and then applying it on subsequent SSTs in the same compaction. While this was great for minimizing CPU/memory/I/O overhead, it did not achieve good compression ratios in practice. In our most promising potential use case, moderate reductions in a dictionary's scope make a major difference on compression ratio. So, this PR changes compression dictionary to be scoped per-SST. It accepts the tradeoff during table building to use more memory and CPU. Important changes include: - The `BlockBasedTableBuilder` has a new state when dictionary compression is in-use: `kBuffered`. In that state it accumulates uncompressed data in-memory whenever `Add` is called. - After accumulating target file size bytes or calling `BlockBasedTableBuilder::Finish`, a `BlockBasedTableBuilder` moves to the `kUnbuffered` state. The transition (`EnterUnbuffered()`) involves sampling the buffered data, training a dictionary, and compressing/writing out all buffered data. In the `kUnbuffered` state, a `BlockBasedTableBuilder` behaves the same as before -- blocks are compressed/written out as soon as they fill up. - Samples are now whole uncompressed data blocks, except the final sample may be a partial data block so we don't breach the user's configured `max_dict_bytes` or `zstd_max_train_bytes`. The dictionary trainer is supposed to work better when we pass it real units of compression. Previously we were passing 64-byte KV samples which was not realistic. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4952 Differential Revision: D13967980 Pulled By: ajkr fbshipit-source-id: 82bea6f7537e1529c7a1a4cdee84585f5949300f
6 years ago
if (ok() && r->state == Rep::State::kUnbuffered) {
if (r->compression_opts.parallel_threads > 1) {
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->compression_opts.parallel_threads > 1) {
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->last_key.assign(key.data(), key.size());
r->data_block.Add(key, value);
Reduce scope of compression dictionary to single SST (#4952) Summary: Our previous approach was to train one compression dictionary per compaction, using the first output SST to train a dictionary, and then applying it on subsequent SSTs in the same compaction. While this was great for minimizing CPU/memory/I/O overhead, it did not achieve good compression ratios in practice. In our most promising potential use case, moderate reductions in a dictionary's scope make a major difference on compression ratio. So, this PR changes compression dictionary to be scoped per-SST. It accepts the tradeoff during table building to use more memory and CPU. Important changes include: - The `BlockBasedTableBuilder` has a new state when dictionary compression is in-use: `kBuffered`. In that state it accumulates uncompressed data in-memory whenever `Add` is called. - After accumulating target file size bytes or calling `BlockBasedTableBuilder::Finish`, a `BlockBasedTableBuilder` moves to the `kUnbuffered` state. The transition (`EnterUnbuffered()`) involves sampling the buffered data, training a dictionary, and compressing/writing out all buffered data. In the `kUnbuffered` state, a `BlockBasedTableBuilder` behaves the same as before -- blocks are compressed/written out as soon as they fill up. - Samples are now whole uncompressed data blocks, except the final sample may be a partial data block so we don't breach the user's configured `max_dict_bytes` or `zstd_max_train_bytes`. The dictionary trainer is supposed to work better when we pass it real units of compression. Previously we were passing 64-byte KV samples which was not realistic. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4952 Differential Revision: D13967980 Pulled By: ajkr fbshipit-source-id: 82bea6f7537e1529c7a1a4cdee84585f5949300f
6 years ago
if (r->state == Rep::State::kBuffered) {
// Buffer keys to be replayed during `Finish()` once compression
// dictionary has been finalized.
if (r->data_block_and_keys_buffers.empty() || should_flush) {
r->data_block_and_keys_buffers.emplace_back();
}
r->data_block_and_keys_buffers.back().second.emplace_back(key.ToString());
} else {
if (r->compression_opts.parallel_threads == 1) {
r->index_builder->OnKeyAdded(key);
}
Reduce scope of compression dictionary to single SST (#4952) Summary: Our previous approach was to train one compression dictionary per compaction, using the first output SST to train a dictionary, and then applying it on subsequent SSTs in the same compaction. While this was great for minimizing CPU/memory/I/O overhead, it did not achieve good compression ratios in practice. In our most promising potential use case, moderate reductions in a dictionary's scope make a major difference on compression ratio. So, this PR changes compression dictionary to be scoped per-SST. It accepts the tradeoff during table building to use more memory and CPU. Important changes include: - The `BlockBasedTableBuilder` has a new state when dictionary compression is in-use: `kBuffered`. In that state it accumulates uncompressed data in-memory whenever `Add` is called. - After accumulating target file size bytes or calling `BlockBasedTableBuilder::Finish`, a `BlockBasedTableBuilder` moves to the `kUnbuffered` state. The transition (`EnterUnbuffered()`) involves sampling the buffered data, training a dictionary, and compressing/writing out all buffered data. In the `kUnbuffered` state, a `BlockBasedTableBuilder` behaves the same as before -- blocks are compressed/written out as soon as they fill up. - Samples are now whole uncompressed data blocks, except the final sample may be a partial data block so we don't breach the user's configured `max_dict_bytes` or `zstd_max_train_bytes`. The dictionary trainer is supposed to work better when we pass it real units of compression. Previously we were passing 64-byte KV samples which was not realistic. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4952 Differential Revision: D13967980 Pulled By: ajkr fbshipit-source-id: 82bea6f7537e1529c7a1a4cdee84585f5949300f
6 years ago
}
// TODO offset passed in is not accurate for parallel compression case
NotifyCollectTableCollectorsOnAdd(key, value, r->get_offset(),
r->table_properties_collectors,
r->ioptions.info_log);
} 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.info_log);
} 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) {
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_;
Reduce scope of compression dictionary to single SST (#4952) Summary: Our previous approach was to train one compression dictionary per compaction, using the first output SST to train a dictionary, and then applying it on subsequent SSTs in the same compaction. While this was great for minimizing CPU/memory/I/O overhead, it did not achieve good compression ratios in practice. In our most promising potential use case, moderate reductions in a dictionary's scope make a major difference on compression ratio. So, this PR changes compression dictionary to be scoped per-SST. It accepts the tradeoff during table building to use more memory and CPU. Important changes include: - The `BlockBasedTableBuilder` has a new state when dictionary compression is in-use: `kBuffered`. In that state it accumulates uncompressed data in-memory whenever `Add` is called. - After accumulating target file size bytes or calling `BlockBasedTableBuilder::Finish`, a `BlockBasedTableBuilder` moves to the `kUnbuffered` state. The transition (`EnterUnbuffered()`) involves sampling the buffered data, training a dictionary, and compressing/writing out all buffered data. In the `kUnbuffered` state, a `BlockBasedTableBuilder` behaves the same as before -- blocks are compressed/written out as soon as they fill up. - Samples are now whole uncompressed data blocks, except the final sample may be a partial data block so we don't breach the user's configured `max_dict_bytes` or `zstd_max_train_bytes`. The dictionary trainer is supposed to work better when we pass it real units of compression. Previously we were passing 64-byte KV samples which was not realistic. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4952 Differential Revision: D13967980 Pulled By: ajkr fbshipit-source-id: 82bea6f7537e1529c7a1a4cdee84585f5949300f
6 years ago
assert(rep_->state != Rep::State::kClosed);
if (!ok()) return;
if (r->data_block.empty()) return;
if (r->compression_opts.parallel_threads > 1 &&
r->state == Rep::State::kUnbuffered) {
ParallelCompressionRep::BlockRep* block_rep = nullptr;
r->pc_rep->block_rep_pool.pop(block_rep);
assert(block_rep != nullptr);
r->data_block.Finish();
assert(block_rep->data);
r->data_block.SwapAndReset(*(block_rep->data));
block_rep->contents = *(block_rep->data);
block_rep->compression_type = r->compression_type;
std::swap(block_rep->keys, r->pc_rep->curr_block_keys);
r->pc_rep->curr_block_keys->Clear();
if (r->first_key_in_next_block == nullptr) {
block_rep->first_key_in_next_block.reset(nullptr);
} else {
block_rep->first_key_in_next_block->assign(
r->first_key_in_next_block->data(),
r->first_key_in_next_block->size());
}
uint64_t new_raw_bytes_inflight =
r->pc_rep->raw_bytes_inflight.fetch_add(block_rep->data->size(),
std::memory_order_relaxed) +
block_rep->data->size();
uint64_t new_blocks_inflight =
r->pc_rep->blocks_inflight.fetch_add(1, std::memory_order_relaxed) + 1;
r->pc_rep->estimated_file_size.store(
r->get_offset() +
static_cast<uint64_t>(static_cast<double>(new_raw_bytes_inflight) *
r->pc_rep->curr_compression_ratio.load(
std::memory_order_relaxed)) +
new_blocks_inflight * kBlockTrailerSize,
std::memory_order_relaxed);
// Read out first_block here to avoid data race with BGWorkWriteRawBlock
bool first_block = r->pc_rep->first_block;
assert(block_rep->status.ok());
if (!r->pc_rep->write_queue.push(block_rep->slot.get())) {
return;
}
if (!r->pc_rep->compress_queue.push(block_rep)) {
return;
}
if (first_block) {
std::unique_lock<std::mutex> lock(r->pc_rep->first_block_mutex);
r->pc_rep->first_block_cond.wait(lock,
[r] { return !r->pc_rep->first_block; });
}
} else {
WriteBlock(&r->data_block, &r->pending_handle, true /* is_data_block */);
}
}
void BlockBasedTableBuilder::WriteBlock(BlockBuilder* block,
BlockHandle* handle,
bool is_data_block) {
WriteBlock(block->Finish(), handle, is_data_block);
block->Reset();
}
void BlockBasedTableBuilder::WriteBlock(const Slice& raw_block_contents,
BlockHandle* handle,
bool is_data_block) {
Rep* r = rep_;
Slice block_contents;
CompressionType type;
if (r->state == Rep::State::kBuffered) {
assert(is_data_block);
assert(!r->data_block_and_keys_buffers.empty());
r->data_block_and_keys_buffers.back().first = raw_block_contents.ToString();
r->data_begin_offset += r->data_block_and_keys_buffers.back().first.size();
return;
}
Status compress_status;
CompressAndVerifyBlock(raw_block_contents, 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;
}
WriteRawBlock(block_contents, type, handle, is_data_block);
r->compressed_output.clear();
if (is_data_block) {
if (r->filter_builder != nullptr) {
r->filter_builder->StartBlock(r->get_offset());
}
r->props.data_size = r->get_offset();
++r->props.num_data_blocks;
}
}
void BlockBasedTableBuilder::BGWorkCompression(
CompressionContext& compression_ctx, UncompressionContext* verify_ctx) {
ParallelCompressionRep::BlockRep* block_rep;
while (rep_->pc_rep->compress_queue.pop(block_rep)) {
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& raw_block_contents, bool is_data_block,
CompressionContext& compression_ctx, UncompressionContext* verify_ctx_ptr,
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
assert(ok());
Rep* r = rep_;
*type = r->compression_type;
uint64_t sample_for_compression = r->sample_for_compression;
bool abort_compression = false;
StopWatchNano timer(
r->ioptions.env,
ShouldReportDetailedTime(r->ioptions.env, r->ioptions.statistics));
if (raw_block_contents.size() < kCompressionSizeLimit) {
Reduce scope of compression dictionary to single SST (#4952) Summary: Our previous approach was to train one compression dictionary per compaction, using the first output SST to train a dictionary, and then applying it on subsequent SSTs in the same compaction. While this was great for minimizing CPU/memory/I/O overhead, it did not achieve good compression ratios in practice. In our most promising potential use case, moderate reductions in a dictionary's scope make a major difference on compression ratio. So, this PR changes compression dictionary to be scoped per-SST. It accepts the tradeoff during table building to use more memory and CPU. Important changes include: - The `BlockBasedTableBuilder` has a new state when dictionary compression is in-use: `kBuffered`. In that state it accumulates uncompressed data in-memory whenever `Add` is called. - After accumulating target file size bytes or calling `BlockBasedTableBuilder::Finish`, a `BlockBasedTableBuilder` moves to the `kUnbuffered` state. The transition (`EnterUnbuffered()`) involves sampling the buffered data, training a dictionary, and compressing/writing out all buffered data. In the `kUnbuffered` state, a `BlockBasedTableBuilder` behaves the same as before -- blocks are compressed/written out as soon as they fill up. - Samples are now whole uncompressed data blocks, except the final sample may be a partial data block so we don't breach the user's configured `max_dict_bytes` or `zstd_max_train_bytes`. The dictionary trainer is supposed to work better when we pass it real units of compression. Previously we were passing 64-byte KV samples which was not realistic. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4952 Differential Revision: D13967980 Pulled By: ajkr fbshipit-source-id: 82bea6f7537e1529c7a1a4cdee84585f5949300f
6 years ago
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(
raw_block_contents, compression_info, type,
r->table_options.format_version, is_data_block /* do_sample */,
compressed_output, &sampled_output_fast, &sampled_output_slow);
// notify collectors on block add
NotifyCollectTableCollectorsOnBlockAdd(
r->table_properties_collectors, raw_block_contents.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
Reduce scope of compression dictionary to single SST (#4952) Summary: Our previous approach was to train one compression dictionary per compaction, using the first output SST to train a dictionary, and then applying it on subsequent SSTs in the same compaction. While this was great for minimizing CPU/memory/I/O overhead, it did not achieve good compression ratios in practice. In our most promising potential use case, moderate reductions in a dictionary's scope make a major difference on compression ratio. So, this PR changes compression dictionary to be scoped per-SST. It accepts the tradeoff during table building to use more memory and CPU. Important changes include: - The `BlockBasedTableBuilder` has a new state when dictionary compression is in-use: `kBuffered`. In that state it accumulates uncompressed data in-memory whenever `Add` is called. - After accumulating target file size bytes or calling `BlockBasedTableBuilder::Finish`, a `BlockBasedTableBuilder` moves to the `kUnbuffered` state. The transition (`EnterUnbuffered()`) involves sampling the buffered data, training a dictionary, and compressing/writing out all buffered data. In the `kUnbuffered` state, a `BlockBasedTableBuilder` behaves the same as before -- blocks are compressed/written out as soon as they fill up. - Samples are now whole uncompressed data blocks, except the final sample may be a partial data block so we don't breach the user's configured `max_dict_bytes` or `zstd_max_train_bytes`. The dictionary trainer is supposed to work better when we pass it real units of compression. Previously we were passing 64-byte KV samples which was not realistic. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4952 Differential Revision: D13967980 Pulled By: ajkr fbshipit-source-id: 82bea6f7537e1529c7a1a4cdee84585f5949300f
6 years ago
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_ptr, *verify_dict,
Reduce scope of compression dictionary to single SST (#4952) Summary: Our previous approach was to train one compression dictionary per compaction, using the first output SST to train a dictionary, and then applying it on subsequent SSTs in the same compaction. While this was great for minimizing CPU/memory/I/O overhead, it did not achieve good compression ratios in practice. In our most promising potential use case, moderate reductions in a dictionary's scope make a major difference on compression ratio. So, this PR changes compression dictionary to be scoped per-SST. It accepts the tradeoff during table building to use more memory and CPU. Important changes include: - The `BlockBasedTableBuilder` has a new state when dictionary compression is in-use: `kBuffered`. In that state it accumulates uncompressed data in-memory whenever `Add` is called. - After accumulating target file size bytes or calling `BlockBasedTableBuilder::Finish`, a `BlockBasedTableBuilder` moves to the `kUnbuffered` state. The transition (`EnterUnbuffered()`) involves sampling the buffered data, training a dictionary, and compressing/writing out all buffered data. In the `kUnbuffered` state, a `BlockBasedTableBuilder` behaves the same as before -- blocks are compressed/written out as soon as they fill up. - Samples are now whole uncompressed data blocks, except the final sample may be a partial data block so we don't breach the user's configured `max_dict_bytes` or `zstd_max_train_bytes`. The dictionary trainer is supposed to work better when we pass it real units of compression. Previously we were passing 64-byte KV samples which was not realistic. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4952 Differential Revision: D13967980 Pulled By: ajkr fbshipit-source-id: 82bea6f7537e1529c7a1a4cdee84585f5949300f
6 years ago
r->compression_type);
Status stat = UncompressBlockContentsForCompressionType(
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(raw_block_contents) == 0;
if (!compressed_ok) {
// The result of the compression was invalid. abort.
abort_compression = true;
ROCKS_LOG_ERROR(r->ioptions.info_log,
"Decompressed block did not match raw block");
*out_status =
Status::Corruption("Decompressed block did not match raw block");
}
} 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.
abort_compression = true;
}
// Abort compression if the block is too big, or did not pass
// verification.
if (abort_compression) {
RecordTick(r->ioptions.statistics, NUMBER_BLOCK_NOT_COMPRESSED);
*type = kNoCompression;
*block_contents = raw_block_contents;
} else if (*type != kNoCompression) {
if (ShouldReportDetailedTime(r->ioptions.env, r->ioptions.statistics)) {
RecordTimeToHistogram(r->ioptions.statistics, COMPRESSION_TIMES_NANOS,
timer.ElapsedNanos());
}
RecordInHistogram(r->ioptions.statistics, BYTES_COMPRESSED,
raw_block_contents.size());
RecordTick(r->ioptions.statistics, NUMBER_BLOCK_COMPRESSED);
} else if (*type != r->compression_type) {
RecordTick(r->ioptions.statistics, NUMBER_BLOCK_NOT_COMPRESSED);
}
}
void BlockBasedTableBuilder::WriteRawBlock(const Slice& block_contents,
CompressionType type,
BlockHandle* handle,
bool is_data_block) {
Rep* r = rep_;
Status s = Status::OK();
IOStatus io_s = IOStatus::OK();
StopWatch sw(r->ioptions.env, r->ioptions.statistics, WRITE_RAW_BLOCK_MICROS);
handle->set_offset(r->get_offset());
handle->set_size(block_contents.size());
assert(status().ok());
assert(io_status().ok());
io_s = r->file->Append(block_contents);
if (io_s.ok()) {
char trailer[kBlockTrailerSize];
trailer[0] = type;
uint32_t checksum = 0;
switch (r->table_options.checksum) {
case kNoChecksum:
break;
case kCRC32c: {
uint32_t crc =
crc32c::Value(block_contents.data(), block_contents.size());
// Extend to cover compression type
crc = crc32c::Extend(crc, trailer, 1);
checksum = crc32c::Mask(crc);
break;
}
case kxxHash: {
XXH32_state_t* const state = XXH32_createState();
XXH32_reset(state, 0);
XXH32_update(state, block_contents.data(), block_contents.size());
// Extend to cover compression type
XXH32_update(state, trailer, 1);
checksum = XXH32_digest(state);
XXH32_freeState(state);
break;
}
case kxxHash64: {
XXH64_state_t* const state = XXH64_createState();
XXH64_reset(state, 0);
XXH64_update(state, block_contents.data(), block_contents.size());
// Extend to cover compression type
XXH64_update(state, trailer, 1);
checksum = Lower32of64(XXH64_digest(state));
XXH64_freeState(state);
break;
}
default:
assert(false);
break;
}
EncodeFixed32(trailer + 1, checksum);
assert(io_s.ok());
TEST_SYNC_POINT_CALLBACK(
"BlockBasedTableBuilder::WriteRawBlock:TamperWithChecksum",
static_cast<char*>(trailer));
io_s = r->file->Append(Slice(trailer, kBlockTrailerSize));
if (io_s.ok()) {
s = InsertBlockInCache(block_contents, type, handle);
if (!s.ok()) {
r->SetStatus(s);
}
} else {
r->SetIOStatus(io_s);
}
if (s.ok() && io_s.ok()) {
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);
io_s = r->file->Pad(pad_bytes);
if (io_s.ok()) {
r->set_offset(r->get_offset() + pad_bytes);
} else {
r->SetIOStatus(io_s);
}
}
if (r->compression_opts.parallel_threads > 1) {
if (!r->pc_rep->finished) {
assert(r->pc_rep->raw_bytes_compressed +
r->pc_rep->raw_bytes_curr_block >
0);
r->pc_rep->curr_compression_ratio.store(
(r->pc_rep->curr_compression_ratio.load(
std::memory_order_relaxed) *
r->pc_rep->raw_bytes_compressed +
block_contents.size()) /
static_cast<double>(r->pc_rep->raw_bytes_compressed +
r->pc_rep->raw_bytes_curr_block),
std::memory_order_relaxed);
r->pc_rep->raw_bytes_compressed += r->pc_rep->raw_bytes_curr_block;
uint64_t new_raw_bytes_inflight =
r->pc_rep->raw_bytes_inflight.fetch_sub(
r->pc_rep->raw_bytes_curr_block, std::memory_order_relaxed) -
r->pc_rep->raw_bytes_curr_block;
uint64_t new_blocks_inflight = r->pc_rep->blocks_inflight.fetch_sub(
1, std::memory_order_relaxed) -
1;
assert(new_blocks_inflight < r->compression_opts.parallel_threads);
r->pc_rep->estimated_file_size.store(
r->get_offset() +
static_cast<uint64_t>(
static_cast<double>(new_raw_bytes_inflight) *
r->pc_rep->curr_compression_ratio.load(
std::memory_order_relaxed)) +
new_blocks_inflight * kBlockTrailerSize,
std::memory_order_relaxed);
} else {
r->pc_rep->estimated_file_size.store(r->get_offset(),
std::memory_order_relaxed);
}
}
}
} else {
r->SetIOStatus(io_s);
}
if (!io_s.ok() && s.ok()) {
r->SetStatus(io_s);
}
}
void BlockBasedTableBuilder::BGWorkWriteRawBlock() {
Rep* r = rep_;
ParallelCompressionRep::BlockRepSlot* slot;
ParallelCompressionRep::BlockRep* block_rep;
while (r->pc_rep->write_queue.pop(slot)) {
slot->Take(block_rep);
if (!block_rep->status.ok()) {
r->SetStatus(block_rep->status);
// Return block_rep to the pool so that blocked Flush() can finish
// if there is one, and Flush() will notice !ok() next time.
block_rep->status = Status::OK();
block_rep->compressed_data->clear();
r->pc_rep->block_rep_pool.push(block_rep);
// Unlock first block if necessary.
if (r->pc_rep->first_block) {
std::lock_guard<std::mutex> lock(r->pc_rep->first_block_mutex);
r->pc_rep->first_block = false;
r->pc_rep->first_block_cond.notify_one();
}
break;
}
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->raw_bytes_curr_block = block_rep->data->size();
WriteRawBlock(block_rep->compressed_contents, block_rep->compression_type,
&r->pending_handle, true /* is_data_block*/);
if (!ok()) {
break;
}
if (r->pc_rep->first_block) {
std::lock_guard<std::mutex> lock(r->pc_rep->first_block_mutex);
r->pc_rep->first_block = false;
r->pc_rep->first_block_cond.notify_one();
}
if (r->filter_builder != nullptr) {
r->filter_builder->StartBlock(r->get_offset());
}
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);
}
block_rep->compressed_data->clear();
r->pc_rep->block_rep_pool.push(block_rep);
}
}
Status BlockBasedTableBuilder::status() const { return rep_->GetStatus(); }
IOStatus BlockBasedTableBuilder::io_status() const {
return rep_->GetIOStatus();
}
static void DeleteCachedBlockContents(const Slice& /*key*/, void* value) {
BlockContents* bc = reinterpret_cast<BlockContents*>(value);
delete bc;
}
//
// Make a copy of the block contents and insert into compressed block cache
//
Status BlockBasedTableBuilder::InsertBlockInCache(const Slice& block_contents,
const CompressionType type,
const BlockHandle* handle) {
Rep* r = rep_;
Cache* block_cache_compressed = r->table_options.block_cache_compressed.get();
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->is_raw_block = true;
#endif // NDEBUG
// make cache key by appending the file offset to the cache prefix id
char* end = EncodeVarint64(
r->compressed_cache_key_prefix + r->compressed_cache_key_prefix_size,
handle->offset());
Slice key(r->compressed_cache_key_prefix,
static_cast<size_t>(end - r->compressed_cache_key_prefix));
// Insert into compressed block cache.
block_cache_compressed->Insert(
key, block_contents_to_cache,
block_contents_to_cache->ApproximateMemoryUsage(),
&DeleteCachedBlockContents);
// Invalidate OS cache.
r->file->InvalidateCache(static_cast<size_t>(r->get_offset()), size);
}
return Status::OK();
}
void BlockBasedTableBuilder::WriteFilterBlock(
MetaIndexBuilder* meta_index_builder) {
BlockHandle filter_block_handle;
bool empty_filter_block = (rep_->filter_builder == nullptr ||
rep_->filter_builder->NumAdded() == 0);
if (ok() && !empty_filter_block) {
Status s = Status::Incomplete();
while (ok() && s.IsIncomplete()) {
Slice filter_content =
rep_->filter_builder->Finish(filter_block_handle, &s);
assert(s.ok() || s.IsIncomplete());
rep_->props.filter_size += filter_content.size();
WriteRawBlock(filter_content, kNoCompression, &filter_block_handle);
}
}
if (ok() && !empty_filter_block) {
// Add mapping from "<filter_block_prefix>.Name" to location
// of filter data.
std::string key;
if (rep_->filter_builder->IsBlockBased()) {
key = BlockBasedTable::kFilterBlockPrefix;
} else {
key = rep_->table_options.partition_filters
? BlockBasedTable::kPartitionedFilterBlockPrefix
: BlockBasedTable::kFullFilterBlockPrefix;
}
key.append(rep_->table_options.filter_policy->Name());
meta_index_builder->Add(key, filter_block_handle);
}
}
void BlockBasedTableBuilder::WriteIndexBlock(
MetaIndexBuilder* meta_index_builder, BlockHandle* index_block_handle) {
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, false /* is_data_block */);
if (!ok()) {
break;
Implement full filter for block based table. Summary: 1. Make filter_block.h a base class. Derive block_based_filter_block and full_filter_block. The previous one is the traditional filter block. The full_filter_block is newly added. It would generate a filter block that contain all the keys in SST file. 2. When querying a key, table would first check if full_filter is available. If not, it would go to the exact data block and check using block_based filter. 3. User could choose to use full_filter or tradional(block_based_filter). They would be stored in SST file with different meta index name. "filter.filter_policy" or "full_filter.filter_policy". Then, Table reader is able to know the fllter block type. 4. Some optimizations have been done for full_filter_block, thus it requires a different interface compared to the original one in filter_policy.h. 5. Actual implementation of filter bits coding/decoding is placed in util/bloom_impl.cc Benchmark: base commit 1d23b5c470844c1208301311f0889eca750431c0 Command: db_bench --db=/dev/shm/rocksdb --num_levels=6 --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --write_buffer_size=134217728 --max_write_buffer_number=2 --target_file_size_base=33554432 --max_bytes_for_level_base=1073741824 --verify_checksum=false --max_background_compactions=4 --use_plain_table=0 --memtablerep=prefix_hash --open_files=-1 --mmap_read=1 --mmap_write=0 --bloom_bits=10 --bloom_locality=1 --memtable_bloom_bits=500000 --compression_type=lz4 --num=393216000 --use_hash_search=1 --block_size=1024 --block_restart_interval=16 --use_existing_db=1 --threads=1 --benchmarks=readrandom —disable_auto_compactions=1 Read QPS increase for about 30% from 2230002 to 2991411. Test Plan: make all check valgrind db_test db_stress --use_block_based_filter = 0 ./auto_sanity_test.sh Reviewers: igor, yhchiang, ljin, sdong Reviewed By: sdong Subscribers: dhruba, leveldb Differential Revision: https://reviews.facebook.net/D20979
10 years ago
}
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, false);
} else {
WriteRawBlock(index_blocks.index_block_contents, kNoCompression,
index_block_handle);
}
}
// If there are more index partitions, finish them and write them out
Status s = index_builder_status;
while (ok() && s.IsIncomplete()) {
s = rep_->index_builder->Finish(&index_blocks, *index_block_handle);
if (!s.ok() && !s.IsIncomplete()) {
rep_->SetStatus(s);
return;
}
if (rep_->table_options.enable_index_compression) {
WriteBlock(index_blocks.index_block_contents, index_block_handle, false);
} else {
WriteRawBlock(index_blocks.index_block_contents, kNoCompression,
index_block_handle);
}
// 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.column_family_id = rep_->column_family_id;
rep_->props.column_family_name = rep_->column_family_name;
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);
add compression options to table properties (#5081) Summary: Since we are planning to use dictionary compression and to use different compression level, it is quite useful to add compression options to TableProperties. For example, in MyRocks, if the feature is available, we can query from information_schema.rocksdb_sst_props to see if all sst files are converted to ZSTD dictionary compressions. Resolves https://github.com/facebook/rocksdb/issues/4992 With this PR, user can query table properties through `GetPropertiesOfAllTables` API and get compression options as std::string: `window_bits=-14; level=32767; strategy=0; max_dict_bytes=0; zstd_max_train_bytes=0; enabled=0;` or table_properties->ToString() will also contain it `# data blocks=1; # entries=13; # deletions=0; # merge operands=0; # range deletions=0; raw key size=143; raw average key size=11.000000; raw value size=39; raw average value size=3.000000; data block size=120; index block size (user-key? 0, delta-value? 0)=27; filter block size=0; (estimated) table size=147; filter policy name=N/A; prefix extractor name=nullptr; column family ID=0; column family name=default; comparator name=leveldb.BytewiseComparator; merge operator name=nullptr; property collectors names=[]; SST file compression algo=Snappy; SST file compression options=window_bits=-14; level=32767; strategy=0; max_dict_bytes=0; zstd_max_train_bytes=0; enabled=0; ; creation time=1552946632; time stamp of earliest key=1552946632;` Pull Request resolved: https://github.com/facebook/rocksdb/pull/5081 Differential Revision: D14716692 Pulled By: miasantreble fbshipit-source-id: 7d2f2cf84e052bff876e71b4212cfdebf5be32dd
6 years ago
rep_->props.compression_options =
CompressionOptionsToString(rep_->compression_opts);
rep_->props.prefix_extractor_name =
rep_->moptions.prefix_extractor != nullptr
? rep_->moptions.prefix_extractor->Name()
: "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;
rep_->props.creation_time = rep_->creation_time;
rep_->props.oldest_key_time = rep_->oldest_key_time;
Periodic Compactions (#5166) Summary: Introducing Periodic Compactions. This feature allows all the files in a CF to be periodically compacted. It could help in catching any corruptions that could creep into the DB proactively as every file is constantly getting re-compacted. And also, of course, it helps to cleanup data older than certain threshold. - Introduced a new option `periodic_compaction_time` to control how long a file can live without being compacted in a CF. - This works across all levels. - The files are put in the same level after going through the compaction. (Related files in the same level are picked up as `ExpandInputstoCleanCut` is used). - Compaction filters, if any, are invoked as usual. - A new table property, `file_creation_time`, is introduced to implement this feature. This property is set to the time at which the SST file was created (and that time is given by the underlying Env/OS). This feature can be enabled on its own, or in conjunction with `ttl`. It is possible to set a different time threshold for the bottom level when used in conjunction with ttl. Since `ttl` works only on 0 to last but one levels, you could set `ttl` to, say, 1 day, and `periodic_compaction_time` to, say, 7 days. Since `ttl < periodic_compaction_time` all files in last but one levels keep getting picked up based on ttl, and almost never based on periodic_compaction_time. The files in the bottom level get picked up for compaction based on `periodic_compaction_time`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5166 Differential Revision: D14884441 Pulled By: sagar0 fbshipit-source-id: 408426cbacb409c06386a98632dcf90bfa1bda47
6 years ago
rep_->props.file_creation_time = rep_->file_creation_time;
rep_->props.db_id = rep_->db_id;
rep_->props.db_session_id = rep_->db_session_id;
// Add basic properties
property_block_builder.AddTableProperty(rep_->props);
// Add use collected properties
NotifyCollectTableCollectorsOnFinish(rep_->table_properties_collectors,
rep_->ioptions.info_log,
&property_block_builder);
WriteRawBlock(property_block_builder.Finish(), kNoCompression,
&properties_block_handle);
}
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
meta_index_builder->Add(kPropertiesBlock, properties_block_handle);
}
}
void BlockBasedTableBuilder::WriteCompressionDictBlock(
MetaIndexBuilder* meta_index_builder) {
Reduce scope of compression dictionary to single SST (#4952) Summary: Our previous approach was to train one compression dictionary per compaction, using the first output SST to train a dictionary, and then applying it on subsequent SSTs in the same compaction. While this was great for minimizing CPU/memory/I/O overhead, it did not achieve good compression ratios in practice. In our most promising potential use case, moderate reductions in a dictionary's scope make a major difference on compression ratio. So, this PR changes compression dictionary to be scoped per-SST. It accepts the tradeoff during table building to use more memory and CPU. Important changes include: - The `BlockBasedTableBuilder` has a new state when dictionary compression is in-use: `kBuffered`. In that state it accumulates uncompressed data in-memory whenever `Add` is called. - After accumulating target file size bytes or calling `BlockBasedTableBuilder::Finish`, a `BlockBasedTableBuilder` moves to the `kUnbuffered` state. The transition (`EnterUnbuffered()`) involves sampling the buffered data, training a dictionary, and compressing/writing out all buffered data. In the `kUnbuffered` state, a `BlockBasedTableBuilder` behaves the same as before -- blocks are compressed/written out as soon as they fill up. - Samples are now whole uncompressed data blocks, except the final sample may be a partial data block so we don't breach the user's configured `max_dict_bytes` or `zstd_max_train_bytes`. The dictionary trainer is supposed to work better when we pass it real units of compression. Previously we were passing 64-byte KV samples which was not realistic. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4952 Differential Revision: D13967980 Pulled By: ajkr fbshipit-source-id: 82bea6f7537e1529c7a1a4cdee84585f5949300f
6 years ago
if (rep_->compression_dict != nullptr &&
rep_->compression_dict->GetRawDict().size()) {
BlockHandle compression_dict_block_handle;
if (ok()) {
Reduce scope of compression dictionary to single SST (#4952) Summary: Our previous approach was to train one compression dictionary per compaction, using the first output SST to train a dictionary, and then applying it on subsequent SSTs in the same compaction. While this was great for minimizing CPU/memory/I/O overhead, it did not achieve good compression ratios in practice. In our most promising potential use case, moderate reductions in a dictionary's scope make a major difference on compression ratio. So, this PR changes compression dictionary to be scoped per-SST. It accepts the tradeoff during table building to use more memory and CPU. Important changes include: - The `BlockBasedTableBuilder` has a new state when dictionary compression is in-use: `kBuffered`. In that state it accumulates uncompressed data in-memory whenever `Add` is called. - After accumulating target file size bytes or calling `BlockBasedTableBuilder::Finish`, a `BlockBasedTableBuilder` moves to the `kUnbuffered` state. The transition (`EnterUnbuffered()`) involves sampling the buffered data, training a dictionary, and compressing/writing out all buffered data. In the `kUnbuffered` state, a `BlockBasedTableBuilder` behaves the same as before -- blocks are compressed/written out as soon as they fill up. - Samples are now whole uncompressed data blocks, except the final sample may be a partial data block so we don't breach the user's configured `max_dict_bytes` or `zstd_max_train_bytes`. The dictionary trainer is supposed to work better when we pass it real units of compression. Previously we were passing 64-byte KV samples which was not realistic. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4952 Differential Revision: D13967980 Pulled By: ajkr fbshipit-source-id: 82bea6f7537e1529c7a1a4cdee84585f5949300f
6 years ago
WriteRawBlock(rep_->compression_dict->GetRawDict(), kNoCompression,
&compression_dict_block_handle);
Reduce scope of compression dictionary to single SST (#4952) Summary: Our previous approach was to train one compression dictionary per compaction, using the first output SST to train a dictionary, and then applying it on subsequent SSTs in the same compaction. While this was great for minimizing CPU/memory/I/O overhead, it did not achieve good compression ratios in practice. In our most promising potential use case, moderate reductions in a dictionary's scope make a major difference on compression ratio. So, this PR changes compression dictionary to be scoped per-SST. It accepts the tradeoff during table building to use more memory and CPU. Important changes include: - The `BlockBasedTableBuilder` has a new state when dictionary compression is in-use: `kBuffered`. In that state it accumulates uncompressed data in-memory whenever `Add` is called. - After accumulating target file size bytes or calling `BlockBasedTableBuilder::Finish`, a `BlockBasedTableBuilder` moves to the `kUnbuffered` state. The transition (`EnterUnbuffered()`) involves sampling the buffered data, training a dictionary, and compressing/writing out all buffered data. In the `kUnbuffered` state, a `BlockBasedTableBuilder` behaves the same as before -- blocks are compressed/written out as soon as they fill up. - Samples are now whole uncompressed data blocks, except the final sample may be a partial data block so we don't breach the user's configured `max_dict_bytes` or `zstd_max_train_bytes`. The dictionary trainer is supposed to work better when we pass it real units of compression. Previously we were passing 64-byte KV samples which was not realistic. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4952 Differential Revision: D13967980 Pulled By: ajkr fbshipit-source-id: 82bea6f7537e1529c7a1a4cdee84585f5949300f
6 years ago
#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(kCompressionDictBlock,
compression_dict_block_handle);
}
}
}
void BlockBasedTableBuilder::WriteRangeDelBlock(
MetaIndexBuilder* meta_index_builder) {
if (ok() && !rep_->range_del_block.empty()) {
BlockHandle range_del_block_handle;
WriteRawBlock(rep_->range_del_block.Finish(), kNoCompression,
&range_del_block_handle);
meta_index_builder->Add(kRangeDelBlock, range_del_block_handle);
}
}
void BlockBasedTableBuilder::WriteFooter(BlockHandle& metaindex_block_handle,
BlockHandle& index_block_handle) {
Rep* r = rep_;
// No need to write out new footer if we're using default checksum.
// We're writing legacy magic number because we want old versions of RocksDB
// be able to read files generated with new release (just in case if
// somebody wants to roll back after an upgrade)
// TODO(icanadi) at some point in the future, when we're absolutely sure
// nobody will roll back to RocksDB 2.x versions, retire the legacy magic
// number and always write new table files with new magic number
bool legacy = (r->table_options.format_version == 0);
// this is guaranteed by BlockBasedTableBuilder's constructor
assert(r->table_options.checksum == kCRC32c ||
r->table_options.format_version != 0);
Footer footer(
legacy ? kLegacyBlockBasedTableMagicNumber : kBlockBasedTableMagicNumber,
r->table_options.format_version);
footer.set_metaindex_handle(metaindex_block_handle);
footer.set_index_handle(index_block_handle);
footer.set_checksum(r->table_options.checksum);
std::string footer_encoding;
footer.EncodeTo(&footer_encoding);
assert(ok());
IOStatus ios = r->file->Append(footer_encoding);
r->SetIOStatus(ios);
if (ios.ok()) {
r->set_offset(r->get_offset() + footer_encoding.size());
}
r->SyncStatusFromIOStatus();
}
Reduce scope of compression dictionary to single SST (#4952) Summary: Our previous approach was to train one compression dictionary per compaction, using the first output SST to train a dictionary, and then applying it on subsequent SSTs in the same compaction. While this was great for minimizing CPU/memory/I/O overhead, it did not achieve good compression ratios in practice. In our most promising potential use case, moderate reductions in a dictionary's scope make a major difference on compression ratio. So, this PR changes compression dictionary to be scoped per-SST. It accepts the tradeoff during table building to use more memory and CPU. Important changes include: - The `BlockBasedTableBuilder` has a new state when dictionary compression is in-use: `kBuffered`. In that state it accumulates uncompressed data in-memory whenever `Add` is called. - After accumulating target file size bytes or calling `BlockBasedTableBuilder::Finish`, a `BlockBasedTableBuilder` moves to the `kUnbuffered` state. The transition (`EnterUnbuffered()`) involves sampling the buffered data, training a dictionary, and compressing/writing out all buffered data. In the `kUnbuffered` state, a `BlockBasedTableBuilder` behaves the same as before -- blocks are compressed/written out as soon as they fill up. - Samples are now whole uncompressed data blocks, except the final sample may be a partial data block so we don't breach the user's configured `max_dict_bytes` or `zstd_max_train_bytes`. The dictionary trainer is supposed to work better when we pass it real units of compression. Previously we were passing 64-byte KV samples which was not realistic. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4952 Differential Revision: D13967980 Pulled By: ajkr fbshipit-source-id: 82bea6f7537e1529c7a1a4cdee84585f5949300f
6 years ago
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;
Random64 generator{r->creation_time};
std::string compression_dict_samples;
std::vector<size_t> compression_dict_sample_lens;
if (!r->data_block_and_keys_buffers.empty()) {
while (compression_dict_samples.size() < kSampleBytes) {
size_t rand_idx =
static_cast<size_t>(
generator.Uniform(r->data_block_and_keys_buffers.size()));
Reduce scope of compression dictionary to single SST (#4952) Summary: Our previous approach was to train one compression dictionary per compaction, using the first output SST to train a dictionary, and then applying it on subsequent SSTs in the same compaction. While this was great for minimizing CPU/memory/I/O overhead, it did not achieve good compression ratios in practice. In our most promising potential use case, moderate reductions in a dictionary's scope make a major difference on compression ratio. So, this PR changes compression dictionary to be scoped per-SST. It accepts the tradeoff during table building to use more memory and CPU. Important changes include: - The `BlockBasedTableBuilder` has a new state when dictionary compression is in-use: `kBuffered`. In that state it accumulates uncompressed data in-memory whenever `Add` is called. - After accumulating target file size bytes or calling `BlockBasedTableBuilder::Finish`, a `BlockBasedTableBuilder` moves to the `kUnbuffered` state. The transition (`EnterUnbuffered()`) involves sampling the buffered data, training a dictionary, and compressing/writing out all buffered data. In the `kUnbuffered` state, a `BlockBasedTableBuilder` behaves the same as before -- blocks are compressed/written out as soon as they fill up. - Samples are now whole uncompressed data blocks, except the final sample may be a partial data block so we don't breach the user's configured `max_dict_bytes` or `zstd_max_train_bytes`. The dictionary trainer is supposed to work better when we pass it real units of compression. Previously we were passing 64-byte KV samples which was not realistic. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4952 Differential Revision: D13967980 Pulled By: ajkr fbshipit-source-id: 82bea6f7537e1529c7a1a4cdee84585f5949300f
6 years ago
size_t copy_len =
std::min(kSampleBytes - compression_dict_samples.size(),
r->data_block_and_keys_buffers[rand_idx].first.size());
compression_dict_samples.append(
r->data_block_and_keys_buffers[rand_idx].first, 0, copy_len);
compression_dict_sample_lens.emplace_back(copy_len);
}
}
// 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) {
dict = ZSTD_TrainDictionary(compression_dict_samples,
compression_dict_sample_lens,
r->compression_opts.max_dict_bytes);
} 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));
for (size_t i = 0; ok() && i < r->data_block_and_keys_buffers.size(); ++i) {
auto& data_block = r->data_block_and_keys_buffers[i].first;
Reduce scope of compression dictionary to single SST (#4952) Summary: Our previous approach was to train one compression dictionary per compaction, using the first output SST to train a dictionary, and then applying it on subsequent SSTs in the same compaction. While this was great for minimizing CPU/memory/I/O overhead, it did not achieve good compression ratios in practice. In our most promising potential use case, moderate reductions in a dictionary's scope make a major difference on compression ratio. So, this PR changes compression dictionary to be scoped per-SST. It accepts the tradeoff during table building to use more memory and CPU. Important changes include: - The `BlockBasedTableBuilder` has a new state when dictionary compression is in-use: `kBuffered`. In that state it accumulates uncompressed data in-memory whenever `Add` is called. - After accumulating target file size bytes or calling `BlockBasedTableBuilder::Finish`, a `BlockBasedTableBuilder` moves to the `kUnbuffered` state. The transition (`EnterUnbuffered()`) involves sampling the buffered data, training a dictionary, and compressing/writing out all buffered data. In the `kUnbuffered` state, a `BlockBasedTableBuilder` behaves the same as before -- blocks are compressed/written out as soon as they fill up. - Samples are now whole uncompressed data blocks, except the final sample may be a partial data block so we don't breach the user's configured `max_dict_bytes` or `zstd_max_train_bytes`. The dictionary trainer is supposed to work better when we pass it real units of compression. Previously we were passing 64-byte KV samples which was not realistic. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4952 Differential Revision: D13967980 Pulled By: ajkr fbshipit-source-id: 82bea6f7537e1529c7a1a4cdee84585f5949300f
6 years ago
auto& keys = r->data_block_and_keys_buffers[i].second;
assert(!data_block.empty());
assert(!keys.empty());
if (r->compression_opts.parallel_threads > 1) {
ParallelCompressionRep::BlockRep* block_rep;
r->pc_rep->block_rep_pool.pop(block_rep);
std::swap(*(block_rep->data), data_block);
block_rep->contents = *(block_rep->data);
block_rep->compression_type = r->compression_type;
block_rep->keys->SwapAssign(keys);
if (i + 1 < r->data_block_and_keys_buffers.size()) {
block_rep->first_key_in_next_block->assign(
r->data_block_and_keys_buffers[i + 1].second.front());
} else {
if (r->first_key_in_next_block == nullptr) {
block_rep->first_key_in_next_block.reset(nullptr);
} else {
block_rep->first_key_in_next_block->assign(
r->first_key_in_next_block->data(),
r->first_key_in_next_block->size());
}
}
uint64_t new_raw_bytes_inflight =
r->pc_rep->raw_bytes_inflight.fetch_add(block_rep->data->size(),
std::memory_order_relaxed) +
block_rep->data->size();
uint64_t new_blocks_inflight =
r->pc_rep->blocks_inflight.fetch_add(1, std::memory_order_relaxed) +
1;
r->pc_rep->estimated_file_size.store(
r->get_offset() +
static_cast<uint64_t>(
static_cast<double>(new_raw_bytes_inflight) *
r->pc_rep->curr_compression_ratio.load(
std::memory_order_relaxed)) +
new_blocks_inflight * kBlockTrailerSize,
std::memory_order_relaxed);
// Read out first_block here to avoid data race with BGWorkWriteRawBlock
bool first_block = r->pc_rep->first_block;
assert(block_rep->status.ok());
if (!r->pc_rep->write_queue.push(block_rep->slot.get())) {
return;
}
if (!r->pc_rep->compress_queue.push(block_rep)) {
return;
}
if (first_block) {
std::unique_lock<std::mutex> lock(r->pc_rep->first_block_mutex);
r->pc_rep->first_block_cond.wait(
lock, [r] { return !r->pc_rep->first_block; });
}
} else {
for (const auto& key : keys) {
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,
true /* is_data_block */);
if (ok() && i + 1 < r->data_block_and_keys_buffers.size()) {
Slice first_key_in_next_block =
r->data_block_and_keys_buffers[i + 1].second.front();
Slice* first_key_in_next_block_ptr = &first_key_in_next_block;
r->index_builder->AddIndexEntry(
&keys.back(), first_key_in_next_block_ptr, r->pending_handle);
Reduce scope of compression dictionary to single SST (#4952) Summary: Our previous approach was to train one compression dictionary per compaction, using the first output SST to train a dictionary, and then applying it on subsequent SSTs in the same compaction. While this was great for minimizing CPU/memory/I/O overhead, it did not achieve good compression ratios in practice. In our most promising potential use case, moderate reductions in a dictionary's scope make a major difference on compression ratio. So, this PR changes compression dictionary to be scoped per-SST. It accepts the tradeoff during table building to use more memory and CPU. Important changes include: - The `BlockBasedTableBuilder` has a new state when dictionary compression is in-use: `kBuffered`. In that state it accumulates uncompressed data in-memory whenever `Add` is called. - After accumulating target file size bytes or calling `BlockBasedTableBuilder::Finish`, a `BlockBasedTableBuilder` moves to the `kUnbuffered` state. The transition (`EnterUnbuffered()`) involves sampling the buffered data, training a dictionary, and compressing/writing out all buffered data. In the `kUnbuffered` state, a `BlockBasedTableBuilder` behaves the same as before -- blocks are compressed/written out as soon as they fill up. - Samples are now whole uncompressed data blocks, except the final sample may be a partial data block so we don't breach the user's configured `max_dict_bytes` or `zstd_max_train_bytes`. The dictionary trainer is supposed to work better when we pass it real units of compression. Previously we were passing 64-byte KV samples which was not realistic. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4952 Differential Revision: D13967980 Pulled By: ajkr fbshipit-source-id: 82bea6f7537e1529c7a1a4cdee84585f5949300f
6 years ago
}
}
}
r->data_block_and_keys_buffers.clear();
}
Status BlockBasedTableBuilder::Finish() {
Rep* r = rep_;
Reduce scope of compression dictionary to single SST (#4952) Summary: Our previous approach was to train one compression dictionary per compaction, using the first output SST to train a dictionary, and then applying it on subsequent SSTs in the same compaction. While this was great for minimizing CPU/memory/I/O overhead, it did not achieve good compression ratios in practice. In our most promising potential use case, moderate reductions in a dictionary's scope make a major difference on compression ratio. So, this PR changes compression dictionary to be scoped per-SST. It accepts the tradeoff during table building to use more memory and CPU. Important changes include: - The `BlockBasedTableBuilder` has a new state when dictionary compression is in-use: `kBuffered`. In that state it accumulates uncompressed data in-memory whenever `Add` is called. - After accumulating target file size bytes or calling `BlockBasedTableBuilder::Finish`, a `BlockBasedTableBuilder` moves to the `kUnbuffered` state. The transition (`EnterUnbuffered()`) involves sampling the buffered data, training a dictionary, and compressing/writing out all buffered data. In the `kUnbuffered` state, a `BlockBasedTableBuilder` behaves the same as before -- blocks are compressed/written out as soon as they fill up. - Samples are now whole uncompressed data blocks, except the final sample may be a partial data block so we don't breach the user's configured `max_dict_bytes` or `zstd_max_train_bytes`. The dictionary trainer is supposed to work better when we pass it real units of compression. Previously we were passing 64-byte KV samples which was not realistic. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4952 Differential Revision: D13967980 Pulled By: ajkr fbshipit-source-id: 82bea6f7537e1529c7a1a4cdee84585f5949300f
6 years ago
assert(r->state != Rep::State::kClosed);
bool empty_data_block = r->data_block.empty();
r->first_key_in_next_block = nullptr;
Flush();
Reduce scope of compression dictionary to single SST (#4952) Summary: Our previous approach was to train one compression dictionary per compaction, using the first output SST to train a dictionary, and then applying it on subsequent SSTs in the same compaction. While this was great for minimizing CPU/memory/I/O overhead, it did not achieve good compression ratios in practice. In our most promising potential use case, moderate reductions in a dictionary's scope make a major difference on compression ratio. So, this PR changes compression dictionary to be scoped per-SST. It accepts the tradeoff during table building to use more memory and CPU. Important changes include: - The `BlockBasedTableBuilder` has a new state when dictionary compression is in-use: `kBuffered`. In that state it accumulates uncompressed data in-memory whenever `Add` is called. - After accumulating target file size bytes or calling `BlockBasedTableBuilder::Finish`, a `BlockBasedTableBuilder` moves to the `kUnbuffered` state. The transition (`EnterUnbuffered()`) involves sampling the buffered data, training a dictionary, and compressing/writing out all buffered data. In the `kUnbuffered` state, a `BlockBasedTableBuilder` behaves the same as before -- blocks are compressed/written out as soon as they fill up. - Samples are now whole uncompressed data blocks, except the final sample may be a partial data block so we don't breach the user's configured `max_dict_bytes` or `zstd_max_train_bytes`. The dictionary trainer is supposed to work better when we pass it real units of compression. Previously we were passing 64-byte KV samples which was not realistic. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4952 Differential Revision: D13967980 Pulled By: ajkr fbshipit-source-id: 82bea6f7537e1529c7a1a4cdee84585f5949300f
6 years ago
if (r->state == Rep::State::kBuffered) {
EnterUnbuffered();
}
if (r->compression_opts.parallel_threads > 1) {
r->pc_rep->compress_queue.finish();
for (auto& thread : r->pc_rep->compress_thread_pool) {
thread.join();
}
r->pc_rep->write_queue.finish();
r->pc_rep->write_thread->join();
r->pc_rep->finished = true;
} 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
WriteRawBlock(meta_index_builder.Finish(), kNoCompression,
&metaindex_block_handle);
}
if (ok()) {
WriteFooter(metaindex_block_handle, index_block_handle);
}
Reduce scope of compression dictionary to single SST (#4952) Summary: Our previous approach was to train one compression dictionary per compaction, using the first output SST to train a dictionary, and then applying it on subsequent SSTs in the same compaction. While this was great for minimizing CPU/memory/I/O overhead, it did not achieve good compression ratios in practice. In our most promising potential use case, moderate reductions in a dictionary's scope make a major difference on compression ratio. So, this PR changes compression dictionary to be scoped per-SST. It accepts the tradeoff during table building to use more memory and CPU. Important changes include: - The `BlockBasedTableBuilder` has a new state when dictionary compression is in-use: `kBuffered`. In that state it accumulates uncompressed data in-memory whenever `Add` is called. - After accumulating target file size bytes or calling `BlockBasedTableBuilder::Finish`, a `BlockBasedTableBuilder` moves to the `kUnbuffered` state. The transition (`EnterUnbuffered()`) involves sampling the buffered data, training a dictionary, and compressing/writing out all buffered data. In the `kUnbuffered` state, a `BlockBasedTableBuilder` behaves the same as before -- blocks are compressed/written out as soon as they fill up. - Samples are now whole uncompressed data blocks, except the final sample may be a partial data block so we don't breach the user's configured `max_dict_bytes` or `zstd_max_train_bytes`. The dictionary trainer is supposed to work better when we pass it real units of compression. Previously we were passing 64-byte KV samples which was not realistic. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4952 Differential Revision: D13967980 Pulled By: ajkr fbshipit-source-id: 82bea6f7537e1529c7a1a4cdee84585f5949300f
6 years ago
r->state = Rep::State::kClosed;
return r->GetStatus();
}
void BlockBasedTableBuilder::Abandon() {
Reduce scope of compression dictionary to single SST (#4952) Summary: Our previous approach was to train one compression dictionary per compaction, using the first output SST to train a dictionary, and then applying it on subsequent SSTs in the same compaction. While this was great for minimizing CPU/memory/I/O overhead, it did not achieve good compression ratios in practice. In our most promising potential use case, moderate reductions in a dictionary's scope make a major difference on compression ratio. So, this PR changes compression dictionary to be scoped per-SST. It accepts the tradeoff during table building to use more memory and CPU. Important changes include: - The `BlockBasedTableBuilder` has a new state when dictionary compression is in-use: `kBuffered`. In that state it accumulates uncompressed data in-memory whenever `Add` is called. - After accumulating target file size bytes or calling `BlockBasedTableBuilder::Finish`, a `BlockBasedTableBuilder` moves to the `kUnbuffered` state. The transition (`EnterUnbuffered()`) involves sampling the buffered data, training a dictionary, and compressing/writing out all buffered data. In the `kUnbuffered` state, a `BlockBasedTableBuilder` behaves the same as before -- blocks are compressed/written out as soon as they fill up. - Samples are now whole uncompressed data blocks, except the final sample may be a partial data block so we don't breach the user's configured `max_dict_bytes` or `zstd_max_train_bytes`. The dictionary trainer is supposed to work better when we pass it real units of compression. Previously we were passing 64-byte KV samples which was not realistic. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4952 Differential Revision: D13967980 Pulled By: ajkr fbshipit-source-id: 82bea6f7537e1529c7a1a4cdee84585f5949300f
6 years ago
assert(rep_->state != Rep::State::kClosed);
if (rep_->compression_opts.parallel_threads > 1) {
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();
rep_->pc_rep->finished = true;
}
Reduce scope of compression dictionary to single SST (#4952) Summary: Our previous approach was to train one compression dictionary per compaction, using the first output SST to train a dictionary, and then applying it on subsequent SSTs in the same compaction. While this was great for minimizing CPU/memory/I/O overhead, it did not achieve good compression ratios in practice. In our most promising potential use case, moderate reductions in a dictionary's scope make a major difference on compression ratio. So, this PR changes compression dictionary to be scoped per-SST. It accepts the tradeoff during table building to use more memory and CPU. Important changes include: - The `BlockBasedTableBuilder` has a new state when dictionary compression is in-use: `kBuffered`. In that state it accumulates uncompressed data in-memory whenever `Add` is called. - After accumulating target file size bytes or calling `BlockBasedTableBuilder::Finish`, a `BlockBasedTableBuilder` moves to the `kUnbuffered` state. The transition (`EnterUnbuffered()`) involves sampling the buffered data, training a dictionary, and compressing/writing out all buffered data. In the `kUnbuffered` state, a `BlockBasedTableBuilder` behaves the same as before -- blocks are compressed/written out as soon as they fill up. - Samples are now whole uncompressed data blocks, except the final sample may be a partial data block so we don't breach the user's configured `max_dict_bytes` or `zstd_max_train_bytes`. The dictionary trainer is supposed to work better when we pass it real units of compression. Previously we were passing 64-byte KV samples which was not realistic. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4952 Differential Revision: D13967980 Pulled By: ajkr fbshipit-source-id: 82bea6f7537e1529c7a1a4cdee84585f5949300f
6 years ago
rep_->state = Rep::State::kClosed;
}
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;
}
Reduce scope of compression dictionary to single SST (#4952) Summary: Our previous approach was to train one compression dictionary per compaction, using the first output SST to train a dictionary, and then applying it on subsequent SSTs in the same compaction. While this was great for minimizing CPU/memory/I/O overhead, it did not achieve good compression ratios in practice. In our most promising potential use case, moderate reductions in a dictionary's scope make a major difference on compression ratio. So, this PR changes compression dictionary to be scoped per-SST. It accepts the tradeoff during table building to use more memory and CPU. Important changes include: - The `BlockBasedTableBuilder` has a new state when dictionary compression is in-use: `kBuffered`. In that state it accumulates uncompressed data in-memory whenever `Add` is called. - After accumulating target file size bytes or calling `BlockBasedTableBuilder::Finish`, a `BlockBasedTableBuilder` moves to the `kUnbuffered` state. The transition (`EnterUnbuffered()`) involves sampling the buffered data, training a dictionary, and compressing/writing out all buffered data. In the `kUnbuffered` state, a `BlockBasedTableBuilder` behaves the same as before -- blocks are compressed/written out as soon as they fill up. - Samples are now whole uncompressed data blocks, except the final sample may be a partial data block so we don't breach the user's configured `max_dict_bytes` or `zstd_max_train_bytes`. The dictionary trainer is supposed to work better when we pass it real units of compression. Previously we were passing 64-byte KV samples which was not realistic. Pull Request resolved: https://github.com/facebook/rocksdb/pull/4952 Differential Revision: D13967980 Pulled By: ajkr fbshipit-source-id: 82bea6f7537e1529c7a1a4cdee84585f5949300f
6 years ago
uint64_t BlockBasedTableBuilder::FileSize() const { return rep_->offset; }
uint64_t BlockBasedTableBuilder::EstimatedFileSize() const {
if (rep_->compression_opts.parallel_threads > 1) {
// Use compression ratio so far and inflight raw bytes to estimate
// final SST size.
return rep_->pc_rep->estimated_file_size.load(std::memory_order_relaxed);
} 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);
}
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;
}
}
const std::string BlockBasedTable::kFilterBlockPrefix = "filter.";
Implement full filter for block based table. Summary: 1. Make filter_block.h a base class. Derive block_based_filter_block and full_filter_block. The previous one is the traditional filter block. The full_filter_block is newly added. It would generate a filter block that contain all the keys in SST file. 2. When querying a key, table would first check if full_filter is available. If not, it would go to the exact data block and check using block_based filter. 3. User could choose to use full_filter or tradional(block_based_filter). They would be stored in SST file with different meta index name. "filter.filter_policy" or "full_filter.filter_policy". Then, Table reader is able to know the fllter block type. 4. Some optimizations have been done for full_filter_block, thus it requires a different interface compared to the original one in filter_policy.h. 5. Actual implementation of filter bits coding/decoding is placed in util/bloom_impl.cc Benchmark: base commit 1d23b5c470844c1208301311f0889eca750431c0 Command: db_bench --db=/dev/shm/rocksdb --num_levels=6 --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --write_buffer_size=134217728 --max_write_buffer_number=2 --target_file_size_base=33554432 --max_bytes_for_level_base=1073741824 --verify_checksum=false --max_background_compactions=4 --use_plain_table=0 --memtablerep=prefix_hash --open_files=-1 --mmap_read=1 --mmap_write=0 --bloom_bits=10 --bloom_locality=1 --memtable_bloom_bits=500000 --compression_type=lz4 --num=393216000 --use_hash_search=1 --block_size=1024 --block_restart_interval=16 --use_existing_db=1 --threads=1 --benchmarks=readrandom —disable_auto_compactions=1 Read QPS increase for about 30% from 2230002 to 2991411. Test Plan: make all check valgrind db_test db_stress --use_block_based_filter = 0 ./auto_sanity_test.sh Reviewers: igor, yhchiang, ljin, sdong Reviewed By: sdong Subscribers: dhruba, leveldb Differential Revision: https://reviews.facebook.net/D20979
10 years ago
const std::string BlockBasedTable::kFullFilterBlockPrefix = "fullfilter.";
const std::string BlockBasedTable::kPartitionedFilterBlockPrefix =
"partitionedfilter.";
} // namespace ROCKSDB_NAMESPACE