2013-10-16 23:59:46 +02:00
|
|
|
// Copyright (c) 2013, Facebook, Inc. All rights reserved.
|
|
|
|
// This source code is licensed under the BSD-style license found in the
|
|
|
|
// LICENSE file in the root directory of this source tree. An additional grant
|
|
|
|
// of patent rights can be found in the PATENTS file in the same directory.
|
|
|
|
//
|
2011-03-18 23:37:00 +01:00
|
|
|
// 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.
|
|
|
|
|
2013-10-29 01:54:09 +01:00
|
|
|
#include "table/block_based_table_builder.h"
|
2011-03-18 23:37:00 +01:00
|
|
|
|
|
|
|
#include <assert.h>
|
2013-11-20 01:29:42 +01:00
|
|
|
#include <inttypes.h>
|
2013-10-10 20:43:24 +02:00
|
|
|
#include <map>
|
2013-11-20 01:29:42 +01:00
|
|
|
#include <stdio.h>
|
2013-10-10 20:43:24 +02:00
|
|
|
|
2013-11-08 06:27:21 +01:00
|
|
|
#include "rocksdb/flush_block_policy.h"
|
2013-09-02 08:23:40 +02:00
|
|
|
#include "rocksdb/cache.h"
|
2013-08-23 17:38:13 +02:00
|
|
|
#include "rocksdb/comparator.h"
|
2013-10-29 01:54:09 +01:00
|
|
|
#include "rocksdb/table.h"
|
2013-08-23 17:38:13 +02:00
|
|
|
#include "rocksdb/env.h"
|
|
|
|
#include "rocksdb/filter_policy.h"
|
|
|
|
#include "rocksdb/options.h"
|
2013-10-30 18:52:33 +01:00
|
|
|
#include "table/block_based_table_reader.h"
|
2013-09-02 08:23:40 +02:00
|
|
|
#include "table/block.h"
|
2011-03-18 23:37:00 +01:00
|
|
|
#include "table/block_builder.h"
|
2012-04-17 17:36:46 +02:00
|
|
|
#include "table/filter_block.h"
|
2011-03-18 23:37:00 +01:00
|
|
|
#include "table/format.h"
|
|
|
|
#include "util/coding.h"
|
|
|
|
#include "util/crc32c.h"
|
2013-06-17 19:11:10 +02:00
|
|
|
#include "util/stop_watch.h"
|
2011-03-18 23:37:00 +01:00
|
|
|
|
2013-10-04 06:49:15 +02:00
|
|
|
namespace rocksdb {
|
2011-03-18 23:37:00 +01:00
|
|
|
|
2013-10-10 20:43:24 +02:00
|
|
|
namespace {
|
|
|
|
|
|
|
|
struct BytewiseLessThan {
|
2013-11-13 05:05:28 +01:00
|
|
|
bool operator()(const std::string& key1, const std::string& key2) const {
|
2013-10-10 20:43:24 +02:00
|
|
|
// smaller entries will be placed in front.
|
|
|
|
return comparator->Compare(key1, key2) <= 0;
|
|
|
|
}
|
|
|
|
const Comparator* comparator = BytewiseComparator();
|
|
|
|
};
|
|
|
|
|
|
|
|
// When writing to a block that requires entries to be sorted by
|
|
|
|
// `BytewiseComparator`, we can buffer the content to `BytewiseSortedMap`
|
|
|
|
// before writng to store.
|
|
|
|
typedef std::map<std::string, std::string, BytewiseLessThan> BytewiseSortedMap;
|
|
|
|
|
2013-11-20 01:29:42 +01:00
|
|
|
void AddProperties(BytewiseSortedMap& props, std::string name, uint64_t val) {
|
|
|
|
assert(props.find(name) == props.end());
|
2013-10-10 20:43:24 +02:00
|
|
|
|
|
|
|
std::string dst;
|
|
|
|
PutVarint64(&dst, val);
|
|
|
|
|
2013-11-20 01:29:42 +01:00
|
|
|
props.insert(
|
2013-10-10 20:43:24 +02:00
|
|
|
std::make_pair(name, dst)
|
|
|
|
);
|
|
|
|
}
|
|
|
|
|
|
|
|
static 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);
|
|
|
|
}
|
|
|
|
|
2013-10-16 20:50:50 +02:00
|
|
|
// Were we encounter any error occurs during user-defined statistics collection,
|
|
|
|
// we'll write the warning message to info log.
|
2013-11-20 01:29:42 +01:00
|
|
|
void LogPropertiesCollectionError(
|
2013-10-16 20:50:50 +02:00
|
|
|
Logger* info_log, const std::string& method, const std::string& name) {
|
|
|
|
assert(method == "Add" || method == "Finish");
|
|
|
|
|
|
|
|
std::string msg =
|
2013-11-20 01:29:42 +01:00
|
|
|
"[Warning] encountered error when calling TablePropertiesCollector::" +
|
2013-10-16 20:50:50 +02:00
|
|
|
method + "() with collector name: " + name;
|
2013-11-13 05:05:28 +01:00
|
|
|
Log(info_log, "%s", msg.c_str());
|
2013-10-16 20:50:50 +02:00
|
|
|
}
|
|
|
|
|
2013-10-10 20:43:24 +02:00
|
|
|
} // anonymous namespace
|
|
|
|
|
2013-10-29 01:54:09 +01:00
|
|
|
struct BlockBasedTableBuilder::Rep {
|
2011-03-18 23:37:00 +01:00
|
|
|
Options options;
|
|
|
|
WritableFile* file;
|
2013-10-10 20:43:24 +02:00
|
|
|
uint64_t offset = 0;
|
2011-03-18 23:37:00 +01:00
|
|
|
Status status;
|
|
|
|
BlockBuilder data_block;
|
|
|
|
BlockBuilder index_block;
|
|
|
|
std::string last_key;
|
2013-10-30 18:52:33 +01:00
|
|
|
CompressionType compression_type;
|
2013-11-20 01:29:42 +01:00
|
|
|
TableProperties props;
|
2013-10-10 20:43:24 +02:00
|
|
|
|
|
|
|
bool closed = false; // Either Finish() or Abandon() has been called.
|
2012-04-17 17:36:46 +02:00
|
|
|
FilterBlockBuilder* filter_block;
|
2013-09-02 08:23:40 +02:00
|
|
|
char compressed_cache_key_prefix[BlockBasedTable::kMaxCacheKeyPrefixSize];
|
|
|
|
size_t compressed_cache_key_prefix_size;
|
2011-03-18 23:37:00 +01:00
|
|
|
|
|
|
|
BlockHandle pending_handle; // Handle to add to index block
|
|
|
|
|
|
|
|
std::string compressed_output;
|
2013-11-08 06:27:21 +01:00
|
|
|
std::unique_ptr<FlushBlockPolicy> flush_block_policy;
|
2011-03-18 23:37:00 +01:00
|
|
|
|
2013-11-20 07:00:48 +01:00
|
|
|
Rep(const Options& opt,
|
|
|
|
WritableFile* f,
|
|
|
|
FlushBlockPolicyFactory* flush_block_policy_factory,
|
|
|
|
CompressionType compression_type)
|
2011-03-18 23:37:00 +01:00
|
|
|
: options(opt),
|
|
|
|
file(f),
|
2013-11-17 08:21:15 +01:00
|
|
|
data_block(options),
|
|
|
|
// To avoid linear scan, we make the block_restart_interval to be `1`
|
|
|
|
// in index block builder
|
|
|
|
index_block(1 /* block_restart_interval */, options.comparator),
|
2013-10-30 18:52:33 +01:00
|
|
|
compression_type(compression_type),
|
2013-03-01 03:04:58 +01:00
|
|
|
filter_block(opt.filter_policy == nullptr ? nullptr
|
2013-11-20 07:00:48 +01:00
|
|
|
: new FilterBlockBuilder(opt)),
|
|
|
|
flush_block_policy(
|
|
|
|
flush_block_policy_factory->NewFlushBlockPolicy(data_block)) {
|
2011-03-18 23:37:00 +01:00
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2013-11-20 07:00:48 +01:00
|
|
|
BlockBasedTableBuilder::BlockBasedTableBuilder(
|
|
|
|
const Options& options,
|
|
|
|
WritableFile* file,
|
|
|
|
FlushBlockPolicyFactory* flush_block_policy_factory,
|
|
|
|
CompressionType compression_type)
|
|
|
|
: rep_(new Rep(options,
|
|
|
|
file, flush_block_policy_factory, compression_type)) {
|
2013-03-01 03:04:58 +01:00
|
|
|
if (rep_->filter_block != nullptr) {
|
2012-04-17 17:36:46 +02:00
|
|
|
rep_->filter_block->StartBlock(0);
|
|
|
|
}
|
2013-09-02 08:23:40 +02:00
|
|
|
if (options.block_cache_compressed.get() != nullptr) {
|
|
|
|
BlockBasedTable::GenerateCachePrefix(options.block_cache_compressed, file,
|
|
|
|
&rep_->compressed_cache_key_prefix[0],
|
|
|
|
&rep_->compressed_cache_key_prefix_size);
|
|
|
|
}
|
2011-03-18 23:37:00 +01:00
|
|
|
}
|
|
|
|
|
2013-10-29 01:54:09 +01:00
|
|
|
BlockBasedTableBuilder::~BlockBasedTableBuilder() {
|
2011-03-18 23:37:00 +01:00
|
|
|
assert(rep_->closed); // Catch errors where caller forgot to call Finish()
|
2012-04-17 17:36:46 +02:00
|
|
|
delete rep_->filter_block;
|
2011-03-18 23:37:00 +01:00
|
|
|
delete rep_;
|
|
|
|
}
|
|
|
|
|
2013-10-29 01:54:09 +01:00
|
|
|
void BlockBasedTableBuilder::Add(const Slice& key, const Slice& value) {
|
2011-03-18 23:37:00 +01:00
|
|
|
Rep* r = rep_;
|
|
|
|
assert(!r->closed);
|
|
|
|
if (!ok()) return;
|
2013-11-20 01:29:42 +01:00
|
|
|
if (r->props.num_entries > 0) {
|
2011-03-18 23:37:00 +01:00
|
|
|
assert(r->options.comparator->Compare(key, Slice(r->last_key)) > 0);
|
|
|
|
}
|
|
|
|
|
2013-11-08 06:27:21 +01:00
|
|
|
auto should_flush = r->flush_block_policy->Update(key, value);
|
|
|
|
if (should_flush) {
|
|
|
|
assert(!r->data_block.empty());
|
2013-05-15 19:34:02 +02:00
|
|
|
Flush();
|
|
|
|
|
2013-11-08 06:27:21 +01:00
|
|
|
// Add item to index block.
|
|
|
|
// We do not emit the index entry for a block until we have seen the
|
|
|
|
// first key for the next data block. This allows us to use shorter
|
|
|
|
// keys in the index block. For example, consider a block boundary
|
|
|
|
// between the keys "the quick brown fox" and "the who". We can use
|
|
|
|
// "the r" as the key for the index block entry since it is >= all
|
|
|
|
// entries in the first block and < all entries in subsequent
|
|
|
|
// blocks.
|
|
|
|
if (ok()) {
|
|
|
|
r->options.comparator->FindShortestSeparator(&r->last_key, key);
|
|
|
|
std::string handle_encoding;
|
|
|
|
r->pending_handle.EncodeTo(&handle_encoding);
|
|
|
|
r->index_block.Add(r->last_key, Slice(handle_encoding));
|
|
|
|
}
|
2011-03-18 23:37:00 +01:00
|
|
|
}
|
|
|
|
|
2013-03-01 03:04:58 +01:00
|
|
|
if (r->filter_block != nullptr) {
|
2012-04-17 17:36:46 +02:00
|
|
|
r->filter_block->AddKey(key);
|
|
|
|
}
|
|
|
|
|
2011-03-18 23:37:00 +01:00
|
|
|
r->last_key.assign(key.data(), key.size());
|
|
|
|
r->data_block.Add(key, value);
|
2013-11-20 01:29:42 +01:00
|
|
|
r->props.num_entries++;
|
|
|
|
r->props.raw_key_size += key.size();
|
|
|
|
r->props.raw_value_size += value.size();
|
2013-10-16 20:50:50 +02:00
|
|
|
|
2013-11-20 01:29:42 +01:00
|
|
|
for (auto collector : r->options.table_properties_collectors) {
|
2013-10-16 20:50:50 +02:00
|
|
|
Status s = collector->Add(key, value);
|
|
|
|
if (!s.ok()) {
|
2013-11-20 01:29:42 +01:00
|
|
|
LogPropertiesCollectionError(
|
2013-10-16 20:50:50 +02:00
|
|
|
r->options.info_log.get(),
|
|
|
|
"Add", /* method */
|
|
|
|
collector->Name()
|
|
|
|
);
|
|
|
|
}
|
|
|
|
}
|
2011-03-18 23:37:00 +01:00
|
|
|
}
|
|
|
|
|
2013-10-29 01:54:09 +01:00
|
|
|
void BlockBasedTableBuilder::Flush() {
|
2011-03-18 23:37:00 +01:00
|
|
|
Rep* r = rep_;
|
|
|
|
assert(!r->closed);
|
|
|
|
if (!ok()) return;
|
|
|
|
if (r->data_block.empty()) return;
|
|
|
|
WriteBlock(&r->data_block, &r->pending_handle);
|
|
|
|
if (ok()) {
|
|
|
|
r->status = r->file->Flush();
|
|
|
|
}
|
2013-03-01 03:04:58 +01:00
|
|
|
if (r->filter_block != nullptr) {
|
2012-04-17 17:36:46 +02:00
|
|
|
r->filter_block->StartBlock(r->offset);
|
|
|
|
}
|
2013-11-20 01:29:42 +01:00
|
|
|
r->props.data_size = r->offset;
|
|
|
|
++r->props.num_data_blocks;
|
2012-06-28 08:41:33 +02:00
|
|
|
}
|
|
|
|
|
2013-10-29 01:54:09 +01:00
|
|
|
void BlockBasedTableBuilder::WriteBlock(BlockBuilder* block,
|
|
|
|
BlockHandle* handle) {
|
2011-03-18 23:37:00 +01:00
|
|
|
// File format contains a sequence of blocks where each block has:
|
|
|
|
// block_data: uint8[n]
|
|
|
|
// type: uint8
|
|
|
|
// crc: uint32
|
|
|
|
assert(ok());
|
|
|
|
Rep* r = rep_;
|
|
|
|
Slice raw = block->Finish();
|
|
|
|
|
|
|
|
Slice block_contents;
|
2012-06-29 04:26:43 +02:00
|
|
|
std::string* compressed = &r->compressed_output;
|
2013-10-30 18:52:33 +01:00
|
|
|
CompressionType type = r->compression_type;
|
2011-03-18 23:37:00 +01:00
|
|
|
switch (type) {
|
|
|
|
case kNoCompression:
|
|
|
|
block_contents = raw;
|
|
|
|
break;
|
|
|
|
|
2011-03-23 00:24:02 +01:00
|
|
|
case kSnappyCompression: {
|
|
|
|
std::string* compressed = &r->compressed_output;
|
2012-11-01 18:50:08 +01:00
|
|
|
if (port::Snappy_Compress(r->options.compression_opts, raw.data(),
|
|
|
|
raw.size(), compressed) &&
|
2012-06-28 08:41:33 +02:00
|
|
|
GoodCompressionRatio(compressed->size(), raw.size())) {
|
2011-03-23 00:24:02 +01:00
|
|
|
block_contents = *compressed;
|
|
|
|
} else {
|
2012-06-28 08:41:33 +02:00
|
|
|
// Snappy not supported, or not good compression ratio, so just
|
2011-03-23 00:24:02 +01:00
|
|
|
// store uncompressed form
|
2011-03-18 23:37:00 +01:00
|
|
|
block_contents = raw;
|
|
|
|
type = kNoCompression;
|
|
|
|
}
|
|
|
|
break;
|
|
|
|
}
|
2012-06-28 08:41:33 +02:00
|
|
|
case kZlibCompression:
|
2012-11-01 18:50:08 +01:00
|
|
|
if (port::Zlib_Compress(r->options.compression_opts, raw.data(),
|
|
|
|
raw.size(), compressed) &&
|
2012-06-28 08:41:33 +02:00
|
|
|
GoodCompressionRatio(compressed->size(), raw.size())) {
|
|
|
|
block_contents = *compressed;
|
|
|
|
} else {
|
|
|
|
// Zlib not supported, or not good compression ratio, so just
|
|
|
|
// store uncompressed form
|
|
|
|
block_contents = raw;
|
|
|
|
type = kNoCompression;
|
|
|
|
}
|
|
|
|
break;
|
2012-06-29 04:26:43 +02:00
|
|
|
case kBZip2Compression:
|
2012-11-01 18:50:08 +01:00
|
|
|
if (port::BZip2_Compress(r->options.compression_opts, raw.data(),
|
|
|
|
raw.size(), compressed) &&
|
2012-06-29 04:26:43 +02:00
|
|
|
GoodCompressionRatio(compressed->size(), raw.size())) {
|
|
|
|
block_contents = *compressed;
|
|
|
|
} else {
|
|
|
|
// BZip not supported, or not good compression ratio, so just
|
|
|
|
// store uncompressed form
|
|
|
|
block_contents = raw;
|
|
|
|
type = kNoCompression;
|
|
|
|
}
|
|
|
|
break;
|
2011-03-18 23:37:00 +01:00
|
|
|
}
|
2012-04-17 17:36:46 +02:00
|
|
|
WriteRawBlock(block_contents, type, handle);
|
|
|
|
r->compressed_output.clear();
|
|
|
|
block->Reset();
|
|
|
|
}
|
|
|
|
|
2013-10-29 01:54:09 +01:00
|
|
|
void BlockBasedTableBuilder::WriteRawBlock(const Slice& block_contents,
|
|
|
|
CompressionType type,
|
|
|
|
BlockHandle* handle) {
|
2012-04-17 17:36:46 +02:00
|
|
|
Rep* r = rep_;
|
2013-06-17 19:11:10 +02:00
|
|
|
StopWatch sw(r->options.env, r->options.statistics, WRITE_RAW_BLOCK_MICROS);
|
2011-03-18 23:37:00 +01:00
|
|
|
handle->set_offset(r->offset);
|
|
|
|
handle->set_size(block_contents.size());
|
|
|
|
r->status = r->file->Append(block_contents);
|
|
|
|
if (r->status.ok()) {
|
|
|
|
char trailer[kBlockTrailerSize];
|
|
|
|
trailer[0] = type;
|
|
|
|
uint32_t crc = crc32c::Value(block_contents.data(), block_contents.size());
|
|
|
|
crc = crc32c::Extend(crc, trailer, 1); // Extend crc to cover block type
|
|
|
|
EncodeFixed32(trailer+1, crc32c::Mask(crc));
|
|
|
|
r->status = r->file->Append(Slice(trailer, kBlockTrailerSize));
|
2013-09-02 08:23:40 +02:00
|
|
|
if (r->status.ok()) {
|
|
|
|
r->status = InsertBlockInCache(block_contents, type, handle);
|
|
|
|
}
|
2011-03-18 23:37:00 +01:00
|
|
|
if (r->status.ok()) {
|
|
|
|
r->offset += block_contents.size() + kBlockTrailerSize;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2013-10-29 01:54:09 +01:00
|
|
|
Status BlockBasedTableBuilder::status() const {
|
2011-03-18 23:37:00 +01:00
|
|
|
return rep_->status;
|
|
|
|
}
|
|
|
|
|
2013-09-02 08:23:40 +02:00
|
|
|
static void DeleteCachedBlock(const Slice& key, void* value) {
|
|
|
|
Block* block = reinterpret_cast<Block*>(value);
|
|
|
|
delete block;
|
|
|
|
}
|
|
|
|
|
|
|
|
//
|
|
|
|
// 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->options.block_cache_compressed.get();
|
|
|
|
|
|
|
|
if (type != kNoCompression && block_cache_compressed != nullptr) {
|
|
|
|
|
|
|
|
Cache::Handle* cache_handle = nullptr;
|
|
|
|
size_t size = block_contents.size();
|
|
|
|
|
|
|
|
char* ubuf = new char[size]; // make a new copy
|
|
|
|
memcpy(ubuf, block_contents.data(), size);
|
|
|
|
|
|
|
|
BlockContents results;
|
|
|
|
Slice sl(ubuf, size);
|
|
|
|
results.data = sl;
|
|
|
|
results.cachable = true; // XXX
|
|
|
|
results.heap_allocated = true;
|
|
|
|
results.compression_type = type;
|
|
|
|
|
|
|
|
Block* block = new Block(results);
|
|
|
|
|
|
|
|
// 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.
|
|
|
|
cache_handle = block_cache_compressed->Insert(key, block, block->size(),
|
|
|
|
&DeleteCachedBlock);
|
|
|
|
block_cache_compressed->Release(cache_handle);
|
|
|
|
|
|
|
|
// Invalidate OS cache.
|
|
|
|
r->file->InvalidateCache(r->offset, size);
|
|
|
|
}
|
|
|
|
return Status::OK();
|
|
|
|
}
|
|
|
|
|
2013-10-29 01:54:09 +01:00
|
|
|
Status BlockBasedTableBuilder::Finish() {
|
2011-03-18 23:37:00 +01:00
|
|
|
Rep* r = rep_;
|
2013-11-08 06:27:21 +01:00
|
|
|
bool empty_data_block = r->data_block.empty();
|
2011-03-18 23:37:00 +01:00
|
|
|
Flush();
|
|
|
|
assert(!r->closed);
|
|
|
|
r->closed = true;
|
2012-04-17 17:36:46 +02:00
|
|
|
|
2013-11-08 06:27:21 +01:00
|
|
|
BlockHandle filter_block_handle,
|
|
|
|
metaindex_block_handle,
|
|
|
|
index_block_handle;
|
2012-04-17 17:36:46 +02:00
|
|
|
|
|
|
|
// Write filter block
|
2013-03-01 03:04:58 +01:00
|
|
|
if (ok() && r->filter_block != nullptr) {
|
2013-11-20 01:29:42 +01:00
|
|
|
auto filter_contents = r->filter_block->Finish();
|
|
|
|
r->props.filter_size = filter_contents.size();
|
|
|
|
WriteRawBlock(filter_contents, kNoCompression, &filter_block_handle);
|
2012-04-17 17:36:46 +02:00
|
|
|
}
|
|
|
|
|
2013-11-20 01:29:42 +01:00
|
|
|
// To make sure properties block is able to keep the accurate size of index
|
2013-10-10 20:43:24 +02:00
|
|
|
// block, we will finish writing all index entries here and flush them
|
|
|
|
// to storage after metaindex block is written.
|
2013-11-08 06:27:21 +01:00
|
|
|
if (ok() && !empty_data_block) {
|
|
|
|
r->options.comparator->FindShortSuccessor(&r->last_key);
|
|
|
|
|
|
|
|
std::string handle_encoding;
|
|
|
|
r->pending_handle.EncodeTo(&handle_encoding);
|
|
|
|
r->index_block.Add(r->last_key, handle_encoding);
|
2013-10-10 20:43:24 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
// Write meta blocks and metaindex block with the following order.
|
|
|
|
// 1. [meta block: filter]
|
2013-11-20 01:29:42 +01:00
|
|
|
// 2. [meta block: properties]
|
2013-10-10 20:43:24 +02:00
|
|
|
// 3. [metaindex block]
|
2011-03-18 23:37:00 +01:00
|
|
|
if (ok()) {
|
2013-10-08 03:33:49 +02:00
|
|
|
// We use `BytewiseComparator` as the comparator for meta block.
|
|
|
|
BlockBuilder meta_index_block(
|
|
|
|
r->options.block_restart_interval,
|
|
|
|
BytewiseComparator()
|
|
|
|
);
|
2013-10-10 20:43:24 +02:00
|
|
|
// Key: meta block name
|
|
|
|
// Value: block handle to that meta block
|
|
|
|
BytewiseSortedMap meta_block_handles;
|
|
|
|
|
|
|
|
// Write filter block.
|
2013-03-01 03:04:58 +01:00
|
|
|
if (r->filter_block != nullptr) {
|
2013-10-10 20:43:24 +02:00
|
|
|
// Add mapping from "<filter_block_prefix>.Name" to location
|
|
|
|
// of filter data.
|
2013-10-29 01:54:09 +01:00
|
|
|
std::string key = BlockBasedTable::kFilterBlockPrefix;
|
2012-04-17 17:36:46 +02:00
|
|
|
key.append(r->options.filter_policy->Name());
|
|
|
|
std::string handle_encoding;
|
|
|
|
filter_block_handle.EncodeTo(&handle_encoding);
|
2013-10-10 20:43:24 +02:00
|
|
|
meta_block_handles.insert(
|
|
|
|
std::make_pair(key, handle_encoding)
|
|
|
|
);
|
|
|
|
}
|
|
|
|
|
2013-11-20 01:29:42 +01:00
|
|
|
// Write properties block.
|
2013-10-10 20:43:24 +02:00
|
|
|
{
|
2013-11-20 01:29:42 +01:00
|
|
|
BlockBuilder properties_block(
|
2013-10-10 20:43:24 +02:00
|
|
|
r->options.block_restart_interval,
|
|
|
|
BytewiseComparator()
|
|
|
|
);
|
|
|
|
|
2013-11-20 01:29:42 +01:00
|
|
|
BytewiseSortedMap properties;
|
|
|
|
|
|
|
|
// Add basic properties
|
|
|
|
AddProperties(
|
|
|
|
properties,
|
|
|
|
BlockBasedTablePropertiesNames::kRawKeySize,
|
|
|
|
r->props.raw_key_size
|
|
|
|
);
|
|
|
|
AddProperties(
|
|
|
|
properties,
|
|
|
|
BlockBasedTablePropertiesNames::kRawValueSize,
|
|
|
|
r->props.raw_value_size
|
|
|
|
);
|
|
|
|
AddProperties(
|
|
|
|
properties,
|
|
|
|
BlockBasedTablePropertiesNames::kDataSize,
|
|
|
|
r->props.data_size
|
2013-10-10 20:43:24 +02:00
|
|
|
);
|
2013-11-20 01:29:42 +01:00
|
|
|
r->props.index_size =
|
|
|
|
r->index_block.CurrentSizeEstimate() + kBlockTrailerSize;
|
|
|
|
AddProperties(
|
|
|
|
properties,
|
|
|
|
BlockBasedTablePropertiesNames::kIndexSize,
|
|
|
|
r->props.index_size
|
|
|
|
);
|
|
|
|
AddProperties(
|
|
|
|
properties,
|
|
|
|
BlockBasedTablePropertiesNames::kNumEntries,
|
|
|
|
r->props.num_entries
|
|
|
|
);
|
|
|
|
AddProperties(
|
|
|
|
properties,
|
|
|
|
BlockBasedTablePropertiesNames::kNumDataBlocks,
|
|
|
|
r->props.num_data_blocks);
|
2013-10-17 01:57:20 +02:00
|
|
|
if (r->filter_block != nullptr) {
|
2013-11-20 01:29:42 +01:00
|
|
|
properties.insert({
|
|
|
|
BlockBasedTablePropertiesNames::kFilterPolicy,
|
2013-10-17 01:57:20 +02:00
|
|
|
r->options.filter_policy->Name()
|
2013-11-20 01:29:42 +01:00
|
|
|
});
|
2013-10-17 01:57:20 +02:00
|
|
|
}
|
2013-11-20 01:29:42 +01:00
|
|
|
AddProperties(
|
|
|
|
properties,
|
|
|
|
BlockBasedTablePropertiesNames::kFilterSize,
|
|
|
|
r->props.filter_size
|
|
|
|
);
|
2013-10-10 20:43:24 +02:00
|
|
|
|
2013-11-20 01:29:42 +01:00
|
|
|
for (auto collector : r->options.table_properties_collectors) {
|
|
|
|
TableProperties::UserCollectedProperties user_collected_properties;
|
2013-10-16 20:50:50 +02:00
|
|
|
Status s =
|
2013-11-20 01:29:42 +01:00
|
|
|
collector->Finish(&user_collected_properties);
|
2013-10-16 20:50:50 +02:00
|
|
|
|
|
|
|
if (!s.ok()) {
|
2013-11-20 01:29:42 +01:00
|
|
|
LogPropertiesCollectionError(
|
2013-10-16 20:50:50 +02:00
|
|
|
r->options.info_log.get(),
|
|
|
|
"Finish", /* method */
|
|
|
|
collector->Name()
|
|
|
|
);
|
|
|
|
} else {
|
2013-11-20 01:29:42 +01:00
|
|
|
properties.insert(
|
|
|
|
user_collected_properties.begin(),
|
|
|
|
user_collected_properties.end()
|
2013-10-16 20:50:50 +02:00
|
|
|
);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2013-11-20 01:29:42 +01:00
|
|
|
for (const auto& stat : properties) {
|
|
|
|
properties_block.Add(stat.first, stat.second);
|
2013-10-10 20:43:24 +02:00
|
|
|
}
|
|
|
|
|
2013-11-20 01:29:42 +01:00
|
|
|
BlockHandle properties_block_handle;
|
|
|
|
WriteBlock(&properties_block, &properties_block_handle);
|
2013-10-10 20:43:24 +02:00
|
|
|
|
|
|
|
std::string handle_encoding;
|
2013-11-20 01:29:42 +01:00
|
|
|
properties_block_handle.EncodeTo(&handle_encoding);
|
2013-10-10 20:43:24 +02:00
|
|
|
meta_block_handles.insert(
|
2013-11-20 01:29:42 +01:00
|
|
|
{ BlockBasedTable::kPropertiesBlock, handle_encoding }
|
2013-10-10 20:43:24 +02:00
|
|
|
);
|
2013-11-20 01:29:42 +01:00
|
|
|
} // end of properties block writing
|
2013-10-10 20:43:24 +02:00
|
|
|
|
|
|
|
for (const auto& metablock : meta_block_handles) {
|
|
|
|
meta_index_block.Add(metablock.first, metablock.second);
|
2012-04-17 17:36:46 +02:00
|
|
|
}
|
|
|
|
|
2011-03-18 23:37:00 +01:00
|
|
|
WriteBlock(&meta_index_block, &metaindex_block_handle);
|
2013-10-10 20:43:24 +02:00
|
|
|
} // meta blocks and metaindex block.
|
2012-04-17 17:36:46 +02:00
|
|
|
|
|
|
|
// Write index block
|
2011-03-18 23:37:00 +01:00
|
|
|
if (ok()) {
|
|
|
|
WriteBlock(&r->index_block, &index_block_handle);
|
|
|
|
}
|
2012-04-17 17:36:46 +02:00
|
|
|
|
|
|
|
// Write footer
|
2011-03-18 23:37:00 +01:00
|
|
|
if (ok()) {
|
|
|
|
Footer footer;
|
|
|
|
footer.set_metaindex_handle(metaindex_block_handle);
|
|
|
|
footer.set_index_handle(index_block_handle);
|
|
|
|
std::string footer_encoding;
|
|
|
|
footer.EncodeTo(&footer_encoding);
|
|
|
|
r->status = r->file->Append(footer_encoding);
|
|
|
|
if (r->status.ok()) {
|
|
|
|
r->offset += footer_encoding.size();
|
|
|
|
}
|
|
|
|
}
|
2013-11-20 01:29:42 +01:00
|
|
|
|
|
|
|
// Print out the table stats
|
|
|
|
if (ok()) {
|
|
|
|
// user collected properties
|
|
|
|
std::string user_collected;
|
|
|
|
user_collected.reserve(1024);
|
|
|
|
for (auto collector : r->options.table_properties_collectors) {
|
|
|
|
for (const auto& prop : collector->GetReadableProperties()) {
|
|
|
|
user_collected.append(prop.first);
|
|
|
|
user_collected.append("=");
|
|
|
|
user_collected.append(prop.second);
|
|
|
|
user_collected.append("; ");
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
Log(
|
|
|
|
r->options.info_log,
|
|
|
|
"Table was constructed:\n"
|
2013-11-22 02:54:23 +01:00
|
|
|
" [basic properties]: %s\n"
|
|
|
|
" [user collected properties]: %s",
|
2013-11-20 01:29:42 +01:00
|
|
|
r->props.ToString().c_str(),
|
|
|
|
user_collected.c_str()
|
|
|
|
);
|
|
|
|
}
|
|
|
|
|
2011-03-18 23:37:00 +01:00
|
|
|
return r->status;
|
|
|
|
}
|
|
|
|
|
2013-10-29 01:54:09 +01:00
|
|
|
void BlockBasedTableBuilder::Abandon() {
|
2011-03-18 23:37:00 +01:00
|
|
|
Rep* r = rep_;
|
|
|
|
assert(!r->closed);
|
|
|
|
r->closed = true;
|
|
|
|
}
|
|
|
|
|
2013-10-29 01:54:09 +01:00
|
|
|
uint64_t BlockBasedTableBuilder::NumEntries() const {
|
2013-11-20 01:29:42 +01:00
|
|
|
return rep_->props.num_entries;
|
2011-03-18 23:37:00 +01:00
|
|
|
}
|
|
|
|
|
2013-10-29 01:54:09 +01:00
|
|
|
uint64_t BlockBasedTableBuilder::FileSize() const {
|
2011-03-18 23:37:00 +01:00
|
|
|
return rep_->offset;
|
|
|
|
}
|
|
|
|
|
2013-10-04 06:49:15 +02:00
|
|
|
} // namespace rocksdb
|