|
|
|
// Copyright (c) 2013, Facebook, Inc. All rights reserved.
|
|
|
|
// This source code is licensed under the BSD-style license found in the
|
|
|
|
// LICENSE file in the root directory of this source tree. An additional grant
|
|
|
|
// of patent rights can be found in the PATENTS file in the same directory.
|
|
|
|
//
|
|
|
|
// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
|
|
|
|
// Use of this source code is governed by a BSD-style license that can be
|
|
|
|
// found in the LICENSE file. See the AUTHORS file for names of contributors.
|
|
|
|
|
|
|
|
#include "db/compaction_job.h"
|
|
|
|
|
|
|
|
#ifndef __STDC_FORMAT_MACROS
|
|
|
|
#define __STDC_FORMAT_MACROS
|
|
|
|
#endif
|
|
|
|
|
|
|
|
#include <inttypes.h>
|
|
|
|
#include <algorithm>
|
|
|
|
#include <functional>
|
|
|
|
#include <vector>
|
|
|
|
#include <memory>
|
|
|
|
#include <list>
|
|
|
|
#include <set>
|
|
|
|
#include <thread>
|
|
|
|
#include <utility>
|
|
|
|
|
|
|
|
#include "db/builder.h"
|
|
|
|
#include "db/db_iter.h"
|
|
|
|
#include "db/dbformat.h"
|
|
|
|
#include "db/event_helpers.h"
|
|
|
|
#include "db/filename.h"
|
|
|
|
#include "db/log_reader.h"
|
|
|
|
#include "db/log_writer.h"
|
|
|
|
#include "db/memtable.h"
|
|
|
|
#include "db/memtable_list.h"
|
|
|
|
#include "db/merge_context.h"
|
|
|
|
#include "db/merge_helper.h"
|
|
|
|
#include "db/version_set.h"
|
|
|
|
#include "port/likely.h"
|
|
|
|
#include "port/port.h"
|
|
|
|
#include "rocksdb/db.h"
|
|
|
|
#include "rocksdb/env.h"
|
|
|
|
#include "rocksdb/statistics.h"
|
|
|
|
#include "rocksdb/status.h"
|
|
|
|
#include "rocksdb/table.h"
|
|
|
|
#include "table/block.h"
|
|
|
|
#include "table/block_based_table_factory.h"
|
|
|
|
#include "table/merger.h"
|
|
|
|
#include "table/table_builder.h"
|
|
|
|
#include "util/coding.h"
|
Move rate_limiter, write buffering, most perf context instrumentation and most random kill out of Env
Summary: We want to keep Env a think layer for better portability. Less platform dependent codes should be moved out of Env. In this patch, I create a wrapper of file readers and writers, and put rate limiting, write buffering, as well as most perf context instrumentation and random kill out of Env. It will make it easier to maintain multiple Env in the future.
Test Plan: Run all existing unit tests.
Reviewers: anthony, kradhakrishnan, IslamAbdelRahman, yhchiang, igor
Reviewed By: igor
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D42321
9 years ago
|
|
|
#include "util/file_reader_writer.h"
|
|
|
|
#include "util/iostats_context_imp.h"
|
|
|
|
#include "util/log_buffer.h"
|
|
|
|
#include "util/logging.h"
|
|
|
|
#include "util/mutexlock.h"
|
|
|
|
#include "util/perf_context_imp.h"
|
|
|
|
#include "util/stop_watch.h"
|
Include bunch of more events into EventLogger
Summary:
Added these events:
* Recovery start, finish and also when recovery creates a file
* Trivial move
* Compaction start, finish and when compaction creates a file
* Flush start, finish
Also includes small fix to EventLogger
Also added option ROCKSDB_PRINT_EVENTS_TO_STDOUT which is useful when we debug things. I've spent far too much time chasing LOG files.
Still didn't get sst table properties in JSON. They are written very deeply into the stack. I'll address in separate diff.
TODO:
* Write specification. Let's first use this for a while and figure out what's good data to put here, too. After that we'll write spec
* Write tools that parse and analyze LOGs. This can be in python or go. Good intern task.
Test Plan: Ran db_bench with ROCKSDB_PRINT_EVENTS_TO_STDOUT. Here's the output: https://phabricator.fb.com/P19811976
Reviewers: sdong, yhchiang, rven, MarkCallaghan, kradhakrishnan, anthony
Reviewed By: anthony
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D37521
10 years ago
|
|
|
#include "util/string_util.h"
|
|
|
|
#include "util/sync_point.h"
|
|
|
|
#include "util/thread_status_util.h"
|
|
|
|
|
|
|
|
namespace rocksdb {
|
|
|
|
|
|
|
|
// Maintains state for each sub-compaction
|
|
|
|
struct CompactionJob::SubcompactionState {
|
|
|
|
Compaction* compaction;
|
|
|
|
std::unique_ptr<CompactionIterator> c_iter;
|
|
|
|
|
|
|
|
// The boundaries of the key-range this compaction is interested in. No two
|
|
|
|
// subcompactions may have overlapping key-ranges.
|
|
|
|
// 'start' is inclusive, 'end' is exclusive, and nullptr means unbounded
|
|
|
|
Slice *start, *end;
|
|
|
|
|
|
|
|
// The return status of this subcompaction
|
|
|
|
Status status;
|
|
|
|
|
|
|
|
// Files produced by this subcompaction
|
|
|
|
struct Output {
|
|
|
|
FileMetaData meta;
|
|
|
|
bool finished;
|
|
|
|
};
|
|
|
|
|
|
|
|
// State kept for output being generated
|
|
|
|
std::vector<Output> outputs;
|
Move rate_limiter, write buffering, most perf context instrumentation and most random kill out of Env
Summary: We want to keep Env a think layer for better portability. Less platform dependent codes should be moved out of Env. In this patch, I create a wrapper of file readers and writers, and put rate limiting, write buffering, as well as most perf context instrumentation and random kill out of Env. It will make it easier to maintain multiple Env in the future.
Test Plan: Run all existing unit tests.
Reviewers: anthony, kradhakrishnan, IslamAbdelRahman, yhchiang, igor
Reviewed By: igor
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D42321
9 years ago
|
|
|
std::unique_ptr<WritableFileWriter> outfile;
|
|
|
|
std::unique_ptr<TableBuilder> builder;
|
|
|
|
Output* current_output() {
|
|
|
|
if (outputs.empty()) {
|
|
|
|
// This subcompaction's outptut could be empty if compaction was aborted
|
|
|
|
// before this subcompaction had a chance to generate any output files.
|
|
|
|
// When subcompactions are executed sequentially this is more likely and
|
|
|
|
// will be particulalry likely for the later subcompactions to be empty.
|
|
|
|
// Once they are run in parallel however it should be much rarer.
|
|
|
|
return nullptr;
|
|
|
|
} else {
|
|
|
|
return &outputs.back();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// State during the subcompaction
|
|
|
|
uint64_t total_bytes;
|
|
|
|
uint64_t num_input_records;
|
|
|
|
uint64_t num_output_records;
|
|
|
|
CompactionJobStats compaction_job_stats;
|
|
|
|
uint64_t approx_size;
|
|
|
|
|
|
|
|
SubcompactionState(Compaction* c, Slice* _start, Slice* _end,
|
|
|
|
uint64_t size = 0)
|
|
|
|
: compaction(c),
|
|
|
|
start(_start),
|
|
|
|
end(_end),
|
|
|
|
outfile(nullptr),
|
|
|
|
builder(nullptr),
|
|
|
|
total_bytes(0),
|
|
|
|
num_input_records(0),
|
|
|
|
num_output_records(0),
|
|
|
|
approx_size(size) {
|
|
|
|
assert(compaction != nullptr);
|
|
|
|
}
|
|
|
|
|
|
|
|
SubcompactionState(SubcompactionState&& o) { *this = std::move(o); }
|
|
|
|
|
|
|
|
SubcompactionState& operator=(SubcompactionState&& o) {
|
|
|
|
compaction = std::move(o.compaction);
|
|
|
|
start = std::move(o.start);
|
|
|
|
end = std::move(o.end);
|
|
|
|
status = std::move(o.status);
|
|
|
|
outputs = std::move(o.outputs);
|
|
|
|
outfile = std::move(o.outfile);
|
|
|
|
builder = std::move(o.builder);
|
|
|
|
total_bytes = std::move(o.total_bytes);
|
|
|
|
num_input_records = std::move(o.num_input_records);
|
|
|
|
num_output_records = std::move(o.num_output_records);
|
|
|
|
compaction_job_stats = std::move(o.compaction_job_stats);
|
|
|
|
approx_size = std::move(o.approx_size);
|
|
|
|
return *this;
|
|
|
|
}
|
|
|
|
|
|
|
|
// Because member unique_ptrs do not have these.
|
|
|
|
SubcompactionState(const SubcompactionState&) = delete;
|
|
|
|
|
|
|
|
SubcompactionState& operator=(const SubcompactionState&) = delete;
|
|
|
|
};
|
|
|
|
|
|
|
|
// Maintains state for the entire compaction
|
|
|
|
struct CompactionJob::CompactionState {
|
|
|
|
Compaction* const compaction;
|
|
|
|
|
|
|
|
// REQUIRED: subcompaction states are stored in order of increasing
|
|
|
|
// key-range
|
|
|
|
std::vector<CompactionJob::SubcompactionState> sub_compact_states;
|
|
|
|
Status status;
|
|
|
|
|
|
|
|
uint64_t total_bytes;
|
|
|
|
uint64_t num_input_records;
|
|
|
|
uint64_t num_output_records;
|
|
|
|
|
|
|
|
explicit CompactionState(Compaction* c)
|
|
|
|
: compaction(c),
|
|
|
|
total_bytes(0),
|
|
|
|
num_input_records(0),
|
|
|
|
num_output_records(0) {}
|
|
|
|
|
|
|
|
size_t NumOutputFiles() {
|
|
|
|
size_t total = 0;
|
|
|
|
for (auto& s : sub_compact_states) {
|
|
|
|
total += s.outputs.size();
|
|
|
|
}
|
|
|
|
return total;
|
|
|
|
}
|
|
|
|
|
|
|
|
Slice SmallestUserKey() {
|
|
|
|
for (const auto& sub_compact_state : sub_compact_states) {
|
|
|
|
if (!sub_compact_state.outputs.empty() &&
|
|
|
|
sub_compact_state.outputs[0].finished) {
|
|
|
|
return sub_compact_state.outputs[0].meta.smallest.user_key();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
// If there is no finished output, return an empty slice.
|
|
|
|
return Slice(nullptr, 0);
|
|
|
|
}
|
|
|
|
|
|
|
|
Slice LargestUserKey() {
|
|
|
|
for (auto it = sub_compact_states.rbegin(); it < sub_compact_states.rend();
|
|
|
|
++it) {
|
|
|
|
if (!it->outputs.empty() && it->current_output()->finished) {
|
|
|
|
assert(it->current_output() != nullptr);
|
|
|
|
return it->current_output()->meta.largest.user_key();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
// If there is no finished output, return an empty slice.
|
|
|
|
return Slice(nullptr, 0);
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
void CompactionJob::AggregateStatistics() {
|
|
|
|
for (SubcompactionState& sc : compact_->sub_compact_states) {
|
|
|
|
compact_->total_bytes += sc.total_bytes;
|
|
|
|
compact_->num_input_records += sc.num_input_records;
|
|
|
|
compact_->num_output_records += sc.num_output_records;
|
|
|
|
}
|
|
|
|
if (compaction_job_stats_) {
|
|
|
|
for (SubcompactionState& sc : compact_->sub_compact_states) {
|
|
|
|
compaction_job_stats_->Add(sc.compaction_job_stats);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
CompactionJob::CompactionJob(
|
|
|
|
int job_id, Compaction* compaction, const DBOptions& db_options,
|
|
|
|
const EnvOptions& env_options, VersionSet* versions,
|
|
|
|
std::atomic<bool>* shutting_down, LogBuffer* log_buffer,
|
|
|
|
Directory* db_directory, Directory* output_directory, Statistics* stats,
|
|
|
|
std::vector<SequenceNumber> existing_snapshots,
|
|
|
|
std::shared_ptr<Cache> table_cache, EventLogger* event_logger,
|
Add options.compaction_measure_io_stats to print write I/O stats in compactions
Summary:
Add options.compaction_measure_io_stats to print out / pass to listener accumulated time spent on write calls. Example outputs in info logs:
2015/08/12-16:27:59.463944 7fd428bff700 (Original Log Time 2015/08/12-16:27:59.463922) EVENT_LOG_v1 {"time_micros": 1439422079463897, "job": 6, "event": "compaction_finished", "output_level": 1, "num_output_files": 4, "total_output_size": 6900525, "num_input_records": 111483, "num_output_records": 106877, "file_write_nanos": 15663206, "file_range_sync_nanos": 649588, "file_fsync_nanos": 349614797, "file_prepare_write_nanos": 1505812, "lsm_state": [2, 4, 0, 0, 0, 0, 0]}
Add two more counters in iostats_context.
Also add a parameter of db_bench.
Test Plan: Add a unit test. Also manually verify LOG outputs in db_bench
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D44115
9 years ago
|
|
|
bool paranoid_file_checks, bool measure_io_stats, const std::string& dbname,
|
|
|
|
CompactionJobStats* compaction_job_stats)
|
|
|
|
: job_id_(job_id),
|
|
|
|
compact_(new CompactionState(compaction)),
|
|
|
|
compaction_job_stats_(compaction_job_stats),
|
|
|
|
compaction_stats_(1),
|
|
|
|
dbname_(dbname),
|
|
|
|
db_options_(db_options),
|
|
|
|
env_options_(env_options),
|
|
|
|
env_(db_options.env),
|
|
|
|
versions_(versions),
|
|
|
|
shutting_down_(shutting_down),
|
|
|
|
log_buffer_(log_buffer),
|
|
|
|
db_directory_(db_directory),
|
|
|
|
output_directory_(output_directory),
|
|
|
|
stats_(stats),
|
|
|
|
existing_snapshots_(std::move(existing_snapshots)),
|
|
|
|
table_cache_(std::move(table_cache)),
|
|
|
|
event_logger_(event_logger),
|
Add options.compaction_measure_io_stats to print write I/O stats in compactions
Summary:
Add options.compaction_measure_io_stats to print out / pass to listener accumulated time spent on write calls. Example outputs in info logs:
2015/08/12-16:27:59.463944 7fd428bff700 (Original Log Time 2015/08/12-16:27:59.463922) EVENT_LOG_v1 {"time_micros": 1439422079463897, "job": 6, "event": "compaction_finished", "output_level": 1, "num_output_files": 4, "total_output_size": 6900525, "num_input_records": 111483, "num_output_records": 106877, "file_write_nanos": 15663206, "file_range_sync_nanos": 649588, "file_fsync_nanos": 349614797, "file_prepare_write_nanos": 1505812, "lsm_state": [2, 4, 0, 0, 0, 0, 0]}
Add two more counters in iostats_context.
Also add a parameter of db_bench.
Test Plan: Add a unit test. Also manually verify LOG outputs in db_bench
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D44115
9 years ago
|
|
|
paranoid_file_checks_(paranoid_file_checks),
|
|
|
|
measure_io_stats_(measure_io_stats) {
|
|
|
|
assert(log_buffer_ != nullptr);
|
|
|
|
ThreadStatusUtil::SetColumnFamily(compact_->compaction->column_family_data());
|
|
|
|
ThreadStatusUtil::SetThreadOperation(ThreadStatus::OP_COMPACTION);
|
|
|
|
ReportStartedCompaction(compaction);
|
|
|
|
}
|
|
|
|
|
|
|
|
CompactionJob::~CompactionJob() {
|
|
|
|
assert(compact_ == nullptr);
|
|
|
|
ThreadStatusUtil::ResetThreadStatus();
|
|
|
|
}
|
|
|
|
|
|
|
|
void CompactionJob::ReportStartedCompaction(
|
|
|
|
Compaction* compaction) {
|
|
|
|
ThreadStatusUtil::SetColumnFamily(
|
|
|
|
compact_->compaction->column_family_data());
|
|
|
|
|
|
|
|
ThreadStatusUtil::SetThreadOperationProperty(
|
|
|
|
ThreadStatus::COMPACTION_JOB_ID,
|
|
|
|
job_id_);
|
|
|
|
|
|
|
|
ThreadStatusUtil::SetThreadOperationProperty(
|
|
|
|
ThreadStatus::COMPACTION_INPUT_OUTPUT_LEVEL,
|
|
|
|
(static_cast<uint64_t>(compact_->compaction->start_level()) << 32) +
|
|
|
|
compact_->compaction->output_level());
|
|
|
|
|
|
|
|
// In the current design, a CompactionJob is always created
|
|
|
|
// for non-trivial compaction.
|
|
|
|
assert(compaction->IsTrivialMove() == false ||
|
|
|
|
compaction->is_manual_compaction() == true);
|
|
|
|
|
|
|
|
ThreadStatusUtil::SetThreadOperationProperty(
|
|
|
|
ThreadStatus::COMPACTION_PROP_FLAGS,
|
|
|
|
compaction->is_manual_compaction() +
|
|
|
|
(compaction->deletion_compaction() << 1));
|
|
|
|
|
|
|
|
ThreadStatusUtil::SetThreadOperationProperty(
|
|
|
|
ThreadStatus::COMPACTION_TOTAL_INPUT_BYTES,
|
|
|
|
compaction->CalculateTotalInputSize());
|
|
|
|
|
|
|
|
IOSTATS_RESET(bytes_written);
|
|
|
|
IOSTATS_RESET(bytes_read);
|
|
|
|
ThreadStatusUtil::SetThreadOperationProperty(
|
|
|
|
ThreadStatus::COMPACTION_BYTES_WRITTEN, 0);
|
|
|
|
ThreadStatusUtil::SetThreadOperationProperty(
|
|
|
|
ThreadStatus::COMPACTION_BYTES_READ, 0);
|
|
|
|
|
|
|
|
// Set the thread operation after operation properties
|
|
|
|
// to ensure GetThreadList() can always show them all together.
|
|
|
|
ThreadStatusUtil::SetThreadOperation(
|
|
|
|
ThreadStatus::OP_COMPACTION);
|
|
|
|
|
|
|
|
if (compaction_job_stats_) {
|
|
|
|
compaction_job_stats_->is_manual_compaction =
|
|
|
|
compaction->is_manual_compaction();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
void CompactionJob::Prepare() {
|
|
|
|
AutoThreadOperationStageUpdater stage_updater(
|
|
|
|
ThreadStatus::STAGE_COMPACTION_PREPARE);
|
|
|
|
|
|
|
|
// Generate file_levels_ for compaction berfore making Iterator
|
|
|
|
auto* c = compact_->compaction;
|
|
|
|
assert(c->column_family_data() != nullptr);
|
|
|
|
assert(c->column_family_data()->current()->storage_info()
|
|
|
|
->NumLevelFiles(compact_->compaction->level()) > 0);
|
|
|
|
|
|
|
|
// Is this compaction producing files at the bottommost level?
|
|
|
|
bottommost_level_ = c->bottommost_level();
|
|
|
|
|
|
|
|
if (c->ShouldFormSubcompactions()) {
|
|
|
|
const uint64_t start_micros = env_->NowMicros();
|
|
|
|
GenSubcompactionBoundaries();
|
|
|
|
MeasureTime(stats_, SUBCOMPACTION_SETUP_TIME,
|
|
|
|
env_->NowMicros() - start_micros);
|
|
|
|
|
|
|
|
assert(sizes_.size() == boundaries_.size() + 1);
|
|
|
|
|
|
|
|
for (size_t i = 0; i <= boundaries_.size(); i++) {
|
|
|
|
Slice* start = i == 0 ? nullptr : &boundaries_[i - 1];
|
|
|
|
Slice* end = i == boundaries_.size() ? nullptr : &boundaries_[i];
|
|
|
|
compact_->sub_compact_states.emplace_back(c, start, end, sizes_[i]);
|
|
|
|
}
|
|
|
|
MeasureTime(stats_, NUM_SUBCOMPACTIONS_SCHEDULED,
|
|
|
|
compact_->sub_compact_states.size());
|
|
|
|
} else {
|
|
|
|
compact_->sub_compact_states.emplace_back(c, nullptr, nullptr);
|
|
|
|
}
|
Parallelize L0-L1 Compaction: Restructure Compaction Job
Summary:
As of now compactions involving files from Level 0 and Level 1 are single
threaded because the files in L0, although sorted, are not range partitioned like
the other levels. This means that during L0-L1 compaction each file from L1
needs to be merged with potentially all the files from L0.
This attempt to parallelize the L0-L1 compaction assigns a thread and a
corresponding iterator to each L1 file that then considers only the key range
found in that L1 file and only the L0 files that have those keys (and only the
specific portion of those L0 files in which those keys are found). In this way
the overlap is minimized and potentially eliminated between different iterators
focusing on the same files.
The first step is to restructure the compaction logic to break L0-L1 compactions
into multiple, smaller, sequential compactions. Eventually each of these smaller
jobs will be run simultaneously. Areas to pay extra attention to are
# Correct aggregation of compaction job statistics across multiple threads
# Proper opening/closing of output files (make sure each thread's is unique)
# Keys that span multiple L1 files
# Skewed distributions of keys within L0 files
Test Plan: Make and run db_test (newer version has separate compaction tests) and compaction_job_stats_test
Reviewers: igor, noetzli, anthony, sdong, yhchiang
Reviewed By: yhchiang
Subscribers: MarkCallaghan, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D42699
9 years ago
|
|
|
}
|
|
|
|
|
|
|
|
struct RangeWithSize {
|
|
|
|
Range range;
|
|
|
|
uint64_t size;
|
|
|
|
|
|
|
|
RangeWithSize(const Slice& a, const Slice& b, uint64_t s = 0)
|
|
|
|
: range(a, b), size(s) {}
|
|
|
|
};
|
|
|
|
|
|
|
|
bool SliceCompare(const Comparator* cmp, const Slice& a, const Slice& b) {
|
|
|
|
// Returns true if a < b
|
|
|
|
return cmp->Compare(ExtractUserKey(a), ExtractUserKey(b)) < 0;
|
|
|
|
}
|
|
|
|
|
|
|
|
// Generates a histogram representing potential divisions of key ranges from
|
|
|
|
// the input. It adds the starting and/or ending keys of certain input files
|
|
|
|
// to the working set and then finds the approximate size of data in between
|
|
|
|
// each consecutive pair of slices. Then it divides these ranges into
|
|
|
|
// consecutive groups such that each group has a similar size.
|
|
|
|
void CompactionJob::GenSubcompactionBoundaries() {
|
|
|
|
auto* c = compact_->compaction;
|
|
|
|
auto* cfd = c->column_family_data();
|
|
|
|
std::set<Slice, std::function<bool(const Slice& a, const Slice& b)> > bounds(
|
|
|
|
std::bind(&SliceCompare, cfd->user_comparator(), std::placeholders::_1,
|
|
|
|
std::placeholders::_2));
|
|
|
|
int start_lvl = c->start_level();
|
|
|
|
int out_lvl = c->output_level();
|
|
|
|
|
|
|
|
// Add the starting and/or ending key of certain input files as a potential
|
|
|
|
// boundary (because we're inserting into a set, it avoids duplicates)
|
|
|
|
for (size_t lvl_idx = 0; lvl_idx < c->num_input_levels(); lvl_idx++) {
|
|
|
|
int lvl = c->level(lvl_idx);
|
|
|
|
if (lvl >= start_lvl && lvl <= out_lvl) {
|
|
|
|
const LevelFilesBrief* flevel = c->input_levels(lvl_idx);
|
|
|
|
size_t num_files = flevel->num_files;
|
|
|
|
|
|
|
|
if (num_files == 0) {
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
|
|
|
|
if (lvl == 0) {
|
|
|
|
// For level 0 add the starting and ending key of each file since the
|
|
|
|
// files may have greatly differing key ranges (not range-partitioned)
|
|
|
|
for (size_t i = 0; i < num_files; i++) {
|
|
|
|
bounds.emplace(flevel->files[i].smallest_key);
|
|
|
|
bounds.emplace(flevel->files[i].largest_key);
|
|
|
|
}
|
|
|
|
} else {
|
|
|
|
// For all other levels add the smallest/largest key in the level to
|
|
|
|
// encompass the range covered by that level
|
|
|
|
bounds.emplace(flevel->files[0].smallest_key);
|
|
|
|
bounds.emplace(flevel->files[num_files - 1].largest_key);
|
|
|
|
if (lvl == out_lvl) {
|
|
|
|
// For the last level include the starting keys of all files since
|
|
|
|
// the last level is the largest and probably has the widest key
|
|
|
|
// range. Since it's range partitioned, the ending key of one file
|
|
|
|
// and the starting key of the next are very close (or identical).
|
|
|
|
for (size_t i = 1; i < num_files; i++) {
|
|
|
|
bounds.emplace(flevel->files[i].smallest_key);
|
|
|
|
}
|
Parallelize L0-L1 Compaction: Restructure Compaction Job
Summary:
As of now compactions involving files from Level 0 and Level 1 are single
threaded because the files in L0, although sorted, are not range partitioned like
the other levels. This means that during L0-L1 compaction each file from L1
needs to be merged with potentially all the files from L0.
This attempt to parallelize the L0-L1 compaction assigns a thread and a
corresponding iterator to each L1 file that then considers only the key range
found in that L1 file and only the L0 files that have those keys (and only the
specific portion of those L0 files in which those keys are found). In this way
the overlap is minimized and potentially eliminated between different iterators
focusing on the same files.
The first step is to restructure the compaction logic to break L0-L1 compactions
into multiple, smaller, sequential compactions. Eventually each of these smaller
jobs will be run simultaneously. Areas to pay extra attention to are
# Correct aggregation of compaction job statistics across multiple threads
# Proper opening/closing of output files (make sure each thread's is unique)
# Keys that span multiple L1 files
# Skewed distributions of keys within L0 files
Test Plan: Make and run db_test (newer version has separate compaction tests) and compaction_job_stats_test
Reviewers: igor, noetzli, anthony, sdong, yhchiang
Reviewed By: yhchiang
Subscribers: MarkCallaghan, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D42699
9 years ago
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// Combine consecutive pairs of boundaries into ranges with an approximate
|
|
|
|
// size of data covered by keys in that range
|
|
|
|
uint64_t sum = 0;
|
|
|
|
std::vector<RangeWithSize> ranges;
|
|
|
|
auto* v = cfd->current();
|
|
|
|
for (auto it = bounds.begin();;) {
|
|
|
|
const Slice a = *it;
|
|
|
|
it++;
|
|
|
|
|
|
|
|
if (it == bounds.end()) {
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
|
|
|
|
const Slice b = *it;
|
|
|
|
uint64_t size = versions_->ApproximateSize(v, a, b, start_lvl, out_lvl + 1);
|
|
|
|
ranges.emplace_back(a, b, size);
|
|
|
|
sum += size;
|
|
|
|
}
|
|
|
|
|
|
|
|
// Group the ranges into subcompactions
|
|
|
|
const double min_file_fill_percent = 4.0 / 5;
|
|
|
|
uint64_t max_output_files = std::ceil(
|
|
|
|
sum / min_file_fill_percent /
|
|
|
|
cfd->GetCurrentMutableCFOptions()->MaxFileSizeForLevel(out_lvl));
|
|
|
|
uint64_t subcompactions =
|
|
|
|
std::min({static_cast<uint64_t>(ranges.size()),
|
|
|
|
static_cast<uint64_t>(db_options_.max_subcompactions),
|
|
|
|
max_output_files});
|
|
|
|
|
|
|
|
double mean = sum * 1.0 / subcompactions;
|
|
|
|
|
|
|
|
if (subcompactions > 1) {
|
|
|
|
// Greedily add ranges to the subcompaction until the sum of the ranges'
|
|
|
|
// sizes becomes >= the expected mean size of a subcompaction
|
|
|
|
sum = 0;
|
|
|
|
for (size_t i = 0; i < ranges.size() - 1; i++) {
|
|
|
|
if (subcompactions == 1) {
|
|
|
|
// If there's only one left to schedule then it goes to the end so no
|
|
|
|
// need to put an end boundary
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
sum += ranges[i].size;
|
|
|
|
if (sum >= mean) {
|
|
|
|
boundaries_.emplace_back(ExtractUserKey(ranges[i].range.limit));
|
|
|
|
sizes_.emplace_back(sum);
|
|
|
|
subcompactions--;
|
|
|
|
sum = 0;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
sizes_.emplace_back(sum + ranges.back().size);
|
|
|
|
} else {
|
|
|
|
// Only one range so its size is the total sum of sizes computed above
|
|
|
|
sizes_.emplace_back(sum);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
Status CompactionJob::Run() {
|
|
|
|
AutoThreadOperationStageUpdater stage_updater(
|
|
|
|
ThreadStatus::STAGE_COMPACTION_RUN);
|
|
|
|
TEST_SYNC_POINT("CompactionJob::Run():Start");
|
|
|
|
log_buffer_->FlushBufferToLog();
|
|
|
|
LogCompaction();
|
Include bunch of more events into EventLogger
Summary:
Added these events:
* Recovery start, finish and also when recovery creates a file
* Trivial move
* Compaction start, finish and when compaction creates a file
* Flush start, finish
Also includes small fix to EventLogger
Also added option ROCKSDB_PRINT_EVENTS_TO_STDOUT which is useful when we debug things. I've spent far too much time chasing LOG files.
Still didn't get sst table properties in JSON. They are written very deeply into the stack. I'll address in separate diff.
TODO:
* Write specification. Let's first use this for a while and figure out what's good data to put here, too. After that we'll write spec
* Write tools that parse and analyze LOGs. This can be in python or go. Good intern task.
Test Plan: Ran db_bench with ROCKSDB_PRINT_EVENTS_TO_STDOUT. Here's the output: https://phabricator.fb.com/P19811976
Reviewers: sdong, yhchiang, rven, MarkCallaghan, kradhakrishnan, anthony
Reviewed By: anthony
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D37521
10 years ago
|
|
|
|
|
|
|
const size_t num_threads = compact_->sub_compact_states.size();
|
|
|
|
assert(num_threads > 0);
|
|
|
|
const uint64_t start_micros = env_->NowMicros();
|
|
|
|
|
|
|
|
// Launch a thread for each of subcompactions 1...num_threads-1
|
|
|
|
std::vector<std::thread> thread_pool;
|
|
|
|
thread_pool.reserve(num_threads - 1);
|
|
|
|
for (size_t i = 1; i < compact_->sub_compact_states.size(); i++) {
|
|
|
|
thread_pool.emplace_back(&CompactionJob::ProcessKeyValueCompaction, this,
|
|
|
|
&compact_->sub_compact_states[i]);
|
|
|
|
}
|
|
|
|
|
|
|
|
// Always schedule the first subcompaction (whether or not there are also
|
|
|
|
// others) in the current thread to be efficient with resources
|
|
|
|
ProcessKeyValueCompaction(&compact_->sub_compact_states[0]);
|
|
|
|
|
|
|
|
// Wait for all other threads (if there are any) to finish execution
|
|
|
|
for (auto& thread : thread_pool) {
|
|
|
|
thread.join();
|
|
|
|
}
|
|
|
|
|
|
|
|
if (output_directory_ && !db_options_.disableDataSync) {
|
|
|
|
output_directory_->Fsync();
|
|
|
|
}
|
|
|
|
|
|
|
|
compaction_stats_.micros = env_->NowMicros() - start_micros;
|
|
|
|
MeasureTime(stats_, COMPACTION_TIME, compaction_stats_.micros);
|
Parallelize L0-L1 Compaction: Restructure Compaction Job
Summary:
As of now compactions involving files from Level 0 and Level 1 are single
threaded because the files in L0, although sorted, are not range partitioned like
the other levels. This means that during L0-L1 compaction each file from L1
needs to be merged with potentially all the files from L0.
This attempt to parallelize the L0-L1 compaction assigns a thread and a
corresponding iterator to each L1 file that then considers only the key range
found in that L1 file and only the L0 files that have those keys (and only the
specific portion of those L0 files in which those keys are found). In this way
the overlap is minimized and potentially eliminated between different iterators
focusing on the same files.
The first step is to restructure the compaction logic to break L0-L1 compactions
into multiple, smaller, sequential compactions. Eventually each of these smaller
jobs will be run simultaneously. Areas to pay extra attention to are
# Correct aggregation of compaction job statistics across multiple threads
# Proper opening/closing of output files (make sure each thread's is unique)
# Keys that span multiple L1 files
# Skewed distributions of keys within L0 files
Test Plan: Make and run db_test (newer version has separate compaction tests) and compaction_job_stats_test
Reviewers: igor, noetzli, anthony, sdong, yhchiang
Reviewed By: yhchiang
Subscribers: MarkCallaghan, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D42699
9 years ago
|
|
|
|
|
|
|
// Check if any thread encountered an error during execution
|
|
|
|
Status status;
|
|
|
|
for (const auto& state : compact_->sub_compact_states) {
|
|
|
|
if (!state.status.ok()) {
|
|
|
|
status = state.status;
|
Parallelize L0-L1 Compaction: Restructure Compaction Job
Summary:
As of now compactions involving files from Level 0 and Level 1 are single
threaded because the files in L0, although sorted, are not range partitioned like
the other levels. This means that during L0-L1 compaction each file from L1
needs to be merged with potentially all the files from L0.
This attempt to parallelize the L0-L1 compaction assigns a thread and a
corresponding iterator to each L1 file that then considers only the key range
found in that L1 file and only the L0 files that have those keys (and only the
specific portion of those L0 files in which those keys are found). In this way
the overlap is minimized and potentially eliminated between different iterators
focusing on the same files.
The first step is to restructure the compaction logic to break L0-L1 compactions
into multiple, smaller, sequential compactions. Eventually each of these smaller
jobs will be run simultaneously. Areas to pay extra attention to are
# Correct aggregation of compaction job statistics across multiple threads
# Proper opening/closing of output files (make sure each thread's is unique)
# Keys that span multiple L1 files
# Skewed distributions of keys within L0 files
Test Plan: Make and run db_test (newer version has separate compaction tests) and compaction_job_stats_test
Reviewers: igor, noetzli, anthony, sdong, yhchiang
Reviewed By: yhchiang
Subscribers: MarkCallaghan, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D42699
9 years ago
|
|
|
break;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// Finish up all book-keeping to unify the subcompaction results
|
|
|
|
AggregateStatistics();
|
Parallelize L0-L1 Compaction: Restructure Compaction Job
Summary:
As of now compactions involving files from Level 0 and Level 1 are single
threaded because the files in L0, although sorted, are not range partitioned like
the other levels. This means that during L0-L1 compaction each file from L1
needs to be merged with potentially all the files from L0.
This attempt to parallelize the L0-L1 compaction assigns a thread and a
corresponding iterator to each L1 file that then considers only the key range
found in that L1 file and only the L0 files that have those keys (and only the
specific portion of those L0 files in which those keys are found). In this way
the overlap is minimized and potentially eliminated between different iterators
focusing on the same files.
The first step is to restructure the compaction logic to break L0-L1 compactions
into multiple, smaller, sequential compactions. Eventually each of these smaller
jobs will be run simultaneously. Areas to pay extra attention to are
# Correct aggregation of compaction job statistics across multiple threads
# Proper opening/closing of output files (make sure each thread's is unique)
# Keys that span multiple L1 files
# Skewed distributions of keys within L0 files
Test Plan: Make and run db_test (newer version has separate compaction tests) and compaction_job_stats_test
Reviewers: igor, noetzli, anthony, sdong, yhchiang
Reviewed By: yhchiang
Subscribers: MarkCallaghan, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D42699
9 years ago
|
|
|
UpdateCompactionStats();
|
|
|
|
RecordCompactionIOStats();
|
|
|
|
LogFlush(db_options_.info_log);
|
|
|
|
TEST_SYNC_POINT("CompactionJob::Run():End");
|
|
|
|
|
|
|
|
compact_->status = status;
|
Parallelize L0-L1 Compaction: Restructure Compaction Job
Summary:
As of now compactions involving files from Level 0 and Level 1 are single
threaded because the files in L0, although sorted, are not range partitioned like
the other levels. This means that during L0-L1 compaction each file from L1
needs to be merged with potentially all the files from L0.
This attempt to parallelize the L0-L1 compaction assigns a thread and a
corresponding iterator to each L1 file that then considers only the key range
found in that L1 file and only the L0 files that have those keys (and only the
specific portion of those L0 files in which those keys are found). In this way
the overlap is minimized and potentially eliminated between different iterators
focusing on the same files.
The first step is to restructure the compaction logic to break L0-L1 compactions
into multiple, smaller, sequential compactions. Eventually each of these smaller
jobs will be run simultaneously. Areas to pay extra attention to are
# Correct aggregation of compaction job statistics across multiple threads
# Proper opening/closing of output files (make sure each thread's is unique)
# Keys that span multiple L1 files
# Skewed distributions of keys within L0 files
Test Plan: Make and run db_test (newer version has separate compaction tests) and compaction_job_stats_test
Reviewers: igor, noetzli, anthony, sdong, yhchiang
Reviewed By: yhchiang
Subscribers: MarkCallaghan, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D42699
9 years ago
|
|
|
return status;
|
|
|
|
}
|
|
|
|
|
|
|
|
Status CompactionJob::Install(const MutableCFOptions& mutable_cf_options,
|
|
|
|
InstrumentedMutex* db_mutex) {
|
|
|
|
AutoThreadOperationStageUpdater stage_updater(
|
|
|
|
ThreadStatus::STAGE_COMPACTION_INSTALL);
|
|
|
|
db_mutex->AssertHeld();
|
|
|
|
Status status = compact_->status;
|
|
|
|
ColumnFamilyData* cfd = compact_->compaction->column_family_data();
|
|
|
|
cfd->internal_stats()->AddCompactionStats(
|
|
|
|
compact_->compaction->output_level(), compaction_stats_);
|
|
|
|
|
|
|
|
if (status.ok()) {
|
|
|
|
status = InstallCompactionResults(mutable_cf_options, db_mutex);
|
|
|
|
}
|
|
|
|
VersionStorageInfo::LevelSummaryStorage tmp;
|
Include bunch of more events into EventLogger
Summary:
Added these events:
* Recovery start, finish and also when recovery creates a file
* Trivial move
* Compaction start, finish and when compaction creates a file
* Flush start, finish
Also includes small fix to EventLogger
Also added option ROCKSDB_PRINT_EVENTS_TO_STDOUT which is useful when we debug things. I've spent far too much time chasing LOG files.
Still didn't get sst table properties in JSON. They are written very deeply into the stack. I'll address in separate diff.
TODO:
* Write specification. Let's first use this for a while and figure out what's good data to put here, too. After that we'll write spec
* Write tools that parse and analyze LOGs. This can be in python or go. Good intern task.
Test Plan: Ran db_bench with ROCKSDB_PRINT_EVENTS_TO_STDOUT. Here's the output: https://phabricator.fb.com/P19811976
Reviewers: sdong, yhchiang, rven, MarkCallaghan, kradhakrishnan, anthony
Reviewed By: anthony
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D37521
10 years ago
|
|
|
auto vstorage = cfd->current()->storage_info();
|
|
|
|
const auto& stats = compaction_stats_;
|
|
|
|
LogToBuffer(
|
|
|
|
log_buffer_,
|
|
|
|
"[%s] compacted to: %s, MB/sec: %.1f rd, %.1f wr, level %d, "
|
|
|
|
"files in(%d, %d) out(%d) "
|
|
|
|
"MB in(%.1f, %.1f) out(%.1f), read-write-amplify(%.1f) "
|
|
|
|
"write-amplify(%.1f) %s, records in: %d, records dropped: %d\n",
|
|
|
|
cfd->GetName().c_str(), vstorage->LevelSummary(&tmp),
|
|
|
|
(stats.bytes_read_non_output_levels + stats.bytes_read_output_level) /
|
|
|
|
static_cast<double>(stats.micros),
|
|
|
|
stats.bytes_written / static_cast<double>(stats.micros),
|
|
|
|
compact_->compaction->output_level(),
|
|
|
|
stats.num_input_files_in_non_output_levels,
|
|
|
|
stats.num_input_files_in_output_level,
|
|
|
|
stats.num_output_files,
|
|
|
|
stats.bytes_read_non_output_levels / 1048576.0,
|
|
|
|
stats.bytes_read_output_level / 1048576.0,
|
|
|
|
stats.bytes_written / 1048576.0,
|
|
|
|
(stats.bytes_written + stats.bytes_read_output_level +
|
|
|
|
stats.bytes_read_non_output_levels) /
|
|
|
|
static_cast<double>(stats.bytes_read_non_output_levels),
|
|
|
|
stats.bytes_written /
|
|
|
|
static_cast<double>(stats.bytes_read_non_output_levels),
|
|
|
|
status.ToString().c_str(), stats.num_input_records,
|
|
|
|
stats.num_dropped_records);
|
|
|
|
|
|
|
|
UpdateCompactionJobStats(stats);
|
|
|
|
|
Include bunch of more events into EventLogger
Summary:
Added these events:
* Recovery start, finish and also when recovery creates a file
* Trivial move
* Compaction start, finish and when compaction creates a file
* Flush start, finish
Also includes small fix to EventLogger
Also added option ROCKSDB_PRINT_EVENTS_TO_STDOUT which is useful when we debug things. I've spent far too much time chasing LOG files.
Still didn't get sst table properties in JSON. They are written very deeply into the stack. I'll address in separate diff.
TODO:
* Write specification. Let's first use this for a while and figure out what's good data to put here, too. After that we'll write spec
* Write tools that parse and analyze LOGs. This can be in python or go. Good intern task.
Test Plan: Ran db_bench with ROCKSDB_PRINT_EVENTS_TO_STDOUT. Here's the output: https://phabricator.fb.com/P19811976
Reviewers: sdong, yhchiang, rven, MarkCallaghan, kradhakrishnan, anthony
Reviewed By: anthony
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D37521
10 years ago
|
|
|
auto stream = event_logger_->LogToBuffer(log_buffer_);
|
|
|
|
stream << "job" << job_id_
|
|
|
|
<< "event" << "compaction_finished"
|
|
|
|
<< "compaction_time_micros" << compaction_stats_.micros
|
Include bunch of more events into EventLogger
Summary:
Added these events:
* Recovery start, finish and also when recovery creates a file
* Trivial move
* Compaction start, finish and when compaction creates a file
* Flush start, finish
Also includes small fix to EventLogger
Also added option ROCKSDB_PRINT_EVENTS_TO_STDOUT which is useful when we debug things. I've spent far too much time chasing LOG files.
Still didn't get sst table properties in JSON. They are written very deeply into the stack. I'll address in separate diff.
TODO:
* Write specification. Let's first use this for a while and figure out what's good data to put here, too. After that we'll write spec
* Write tools that parse and analyze LOGs. This can be in python or go. Good intern task.
Test Plan: Ran db_bench with ROCKSDB_PRINT_EVENTS_TO_STDOUT. Here's the output: https://phabricator.fb.com/P19811976
Reviewers: sdong, yhchiang, rven, MarkCallaghan, kradhakrishnan, anthony
Reviewed By: anthony
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D37521
10 years ago
|
|
|
<< "output_level" << compact_->compaction->output_level()
|
|
|
|
<< "num_output_files" << compact_->NumOutputFiles()
|
|
|
|
<< "total_output_size" << compact_->total_bytes
|
|
|
|
<< "num_input_records" << compact_->num_input_records
|
|
|
|
<< "num_output_records" << compact_->num_output_records
|
|
|
|
<< "num_subcompactions" << compact_->sub_compact_states.size();
|
Add options.compaction_measure_io_stats to print write I/O stats in compactions
Summary:
Add options.compaction_measure_io_stats to print out / pass to listener accumulated time spent on write calls. Example outputs in info logs:
2015/08/12-16:27:59.463944 7fd428bff700 (Original Log Time 2015/08/12-16:27:59.463922) EVENT_LOG_v1 {"time_micros": 1439422079463897, "job": 6, "event": "compaction_finished", "output_level": 1, "num_output_files": 4, "total_output_size": 6900525, "num_input_records": 111483, "num_output_records": 106877, "file_write_nanos": 15663206, "file_range_sync_nanos": 649588, "file_fsync_nanos": 349614797, "file_prepare_write_nanos": 1505812, "lsm_state": [2, 4, 0, 0, 0, 0, 0]}
Add two more counters in iostats_context.
Also add a parameter of db_bench.
Test Plan: Add a unit test. Also manually verify LOG outputs in db_bench
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D44115
9 years ago
|
|
|
|
|
|
|
if (measure_io_stats_ && compaction_job_stats_ != nullptr) {
|
|
|
|
stream << "file_write_nanos" << compaction_job_stats_->file_write_nanos;
|
|
|
|
stream << "file_range_sync_nanos"
|
|
|
|
<< compaction_job_stats_->file_range_sync_nanos;
|
|
|
|
stream << "file_fsync_nanos" << compaction_job_stats_->file_fsync_nanos;
|
|
|
|
stream << "file_prepare_write_nanos"
|
|
|
|
<< compaction_job_stats_->file_prepare_write_nanos;
|
|
|
|
}
|
|
|
|
|
Include bunch of more events into EventLogger
Summary:
Added these events:
* Recovery start, finish and also when recovery creates a file
* Trivial move
* Compaction start, finish and when compaction creates a file
* Flush start, finish
Also includes small fix to EventLogger
Also added option ROCKSDB_PRINT_EVENTS_TO_STDOUT which is useful when we debug things. I've spent far too much time chasing LOG files.
Still didn't get sst table properties in JSON. They are written very deeply into the stack. I'll address in separate diff.
TODO:
* Write specification. Let's first use this for a while and figure out what's good data to put here, too. After that we'll write spec
* Write tools that parse and analyze LOGs. This can be in python or go. Good intern task.
Test Plan: Ran db_bench with ROCKSDB_PRINT_EVENTS_TO_STDOUT. Here's the output: https://phabricator.fb.com/P19811976
Reviewers: sdong, yhchiang, rven, MarkCallaghan, kradhakrishnan, anthony
Reviewed By: anthony
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D37521
10 years ago
|
|
|
stream << "lsm_state";
|
|
|
|
stream.StartArray();
|
|
|
|
for (int level = 0; level < vstorage->num_levels(); ++level) {
|
|
|
|
stream << vstorage->NumLevelFiles(level);
|
|
|
|
}
|
|
|
|
stream.EndArray();
|
|
|
|
|
|
|
|
CleanupCompaction();
|
|
|
|
return status;
|
|
|
|
}
|
|
|
|
|
|
|
|
void CompactionJob::ProcessKeyValueCompaction(SubcompactionState* sub_compact) {
|
|
|
|
assert(sub_compact != nullptr);
|
|
|
|
std::unique_ptr<Iterator> input(
|
|
|
|
versions_->MakeInputIterator(sub_compact->compaction));
|
|
|
|
|
|
|
|
AutoThreadOperationStageUpdater stage_updater(
|
|
|
|
ThreadStatus::STAGE_COMPACTION_PROCESS_KV);
|
|
|
|
|
|
|
|
// I/O measurement variables
|
|
|
|
PerfLevel prev_perf_level = PerfLevel::kEnableTime;
|
|
|
|
const uint64_t kRecordStatsEvery = 1000;
|
|
|
|
uint64_t prev_write_nanos = 0;
|
|
|
|
uint64_t prev_fsync_nanos = 0;
|
|
|
|
uint64_t prev_range_sync_nanos = 0;
|
|
|
|
uint64_t prev_prepare_write_nanos = 0;
|
|
|
|
if (measure_io_stats_) {
|
|
|
|
prev_perf_level = GetPerfLevel();
|
|
|
|
SetPerfLevel(PerfLevel::kEnableTime);
|
|
|
|
prev_write_nanos = iostats_context.write_nanos;
|
|
|
|
prev_fsync_nanos = iostats_context.fsync_nanos;
|
|
|
|
prev_range_sync_nanos = iostats_context.range_sync_nanos;
|
|
|
|
prev_prepare_write_nanos = iostats_context.prepare_write_nanos;
|
|
|
|
}
|
|
|
|
|
|
|
|
ColumnFamilyData* cfd = sub_compact->compaction->column_family_data();
|
|
|
|
auto compaction_filter = cfd->ioptions()->compaction_filter;
|
|
|
|
std::unique_ptr<CompactionFilter> compaction_filter_from_factory = nullptr;
|
|
|
|
if (compaction_filter == nullptr) {
|
|
|
|
compaction_filter_from_factory =
|
|
|
|
sub_compact->compaction->CreateCompactionFilter();
|
|
|
|
compaction_filter = compaction_filter_from_factory.get();
|
|
|
|
}
|
Compaction filter on merge operands
Summary:
Since Andres' internship is over, I took over https://reviews.facebook.net/D42555 and rebased and simplified it a bit.
The behavior in this diff is a bit simpler than in D42555:
* only merge operators are passed through FilterMergeValue(). If fitler function returns true, the merge operator is ignored
* compaction filter is *not* called on: 1) results of merge operations and 2) base values that are getting merged with merge operands (the second case was also true in previous diff)
Do we also need a compaction filter to get called on merge results?
Test Plan: make && make check
Reviewers: lovro, tnovak, rven, yhchiang, sdong
Reviewed By: sdong
Subscribers: noetzli, kolmike, leveldb, dhruba, sdong
Differential Revision: https://reviews.facebook.net/D47847
9 years ago
|
|
|
MergeHelper merge(
|
|
|
|
env_, cfd->user_comparator(), cfd->ioptions()->merge_operator,
|
|
|
|
compaction_filter, db_options_.info_log.get(),
|
|
|
|
cfd->ioptions()->min_partial_merge_operands,
|
|
|
|
false /* internal key corruption is expected */,
|
|
|
|
existing_snapshots_.empty() ? 0 : existing_snapshots_.back(),
|
|
|
|
compact_->compaction->level(), db_options_.statistics.get());
|
|
|
|
|
|
|
|
TEST_SYNC_POINT("CompactionJob::Run():Inprogress");
|
|
|
|
|
|
|
|
Slice* start = sub_compact->start;
|
|
|
|
Slice* end = sub_compact->end;
|
Parallelize L0-L1 Compaction: Restructure Compaction Job
Summary:
As of now compactions involving files from Level 0 and Level 1 are single
threaded because the files in L0, although sorted, are not range partitioned like
the other levels. This means that during L0-L1 compaction each file from L1
needs to be merged with potentially all the files from L0.
This attempt to parallelize the L0-L1 compaction assigns a thread and a
corresponding iterator to each L1 file that then considers only the key range
found in that L1 file and only the L0 files that have those keys (and only the
specific portion of those L0 files in which those keys are found). In this way
the overlap is minimized and potentially eliminated between different iterators
focusing on the same files.
The first step is to restructure the compaction logic to break L0-L1 compactions
into multiple, smaller, sequential compactions. Eventually each of these smaller
jobs will be run simultaneously. Areas to pay extra attention to are
# Correct aggregation of compaction job statistics across multiple threads
# Proper opening/closing of output files (make sure each thread's is unique)
# Keys that span multiple L1 files
# Skewed distributions of keys within L0 files
Test Plan: Make and run db_test (newer version has separate compaction tests) and compaction_job_stats_test
Reviewers: igor, noetzli, anthony, sdong, yhchiang
Reviewed By: yhchiang
Subscribers: MarkCallaghan, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D42699
9 years ago
|
|
|
if (start != nullptr) {
|
|
|
|
IterKey start_iter;
|
|
|
|
start_iter.SetInternalKey(*start, kMaxSequenceNumber, kValueTypeForSeek);
|
|
|
|
input->Seek(start_iter.GetKey());
|
Parallelize L0-L1 Compaction: Restructure Compaction Job
Summary:
As of now compactions involving files from Level 0 and Level 1 are single
threaded because the files in L0, although sorted, are not range partitioned like
the other levels. This means that during L0-L1 compaction each file from L1
needs to be merged with potentially all the files from L0.
This attempt to parallelize the L0-L1 compaction assigns a thread and a
corresponding iterator to each L1 file that then considers only the key range
found in that L1 file and only the L0 files that have those keys (and only the
specific portion of those L0 files in which those keys are found). In this way
the overlap is minimized and potentially eliminated between different iterators
focusing on the same files.
The first step is to restructure the compaction logic to break L0-L1 compactions
into multiple, smaller, sequential compactions. Eventually each of these smaller
jobs will be run simultaneously. Areas to pay extra attention to are
# Correct aggregation of compaction job statistics across multiple threads
# Proper opening/closing of output files (make sure each thread's is unique)
# Keys that span multiple L1 files
# Skewed distributions of keys within L0 files
Test Plan: Make and run db_test (newer version has separate compaction tests) and compaction_job_stats_test
Reviewers: igor, noetzli, anthony, sdong, yhchiang
Reviewed By: yhchiang
Subscribers: MarkCallaghan, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D42699
9 years ago
|
|
|
} else {
|
|
|
|
input->SeekToFirst();
|
|
|
|
}
|
|
|
|
|
|
|
|
Status status;
|
|
|
|
sub_compact->c_iter.reset(new CompactionIterator(
|
|
|
|
input.get(), cfd->user_comparator(), &merge, versions_->LastSequence(),
|
Compaction filter on merge operands
Summary:
Since Andres' internship is over, I took over https://reviews.facebook.net/D42555 and rebased and simplified it a bit.
The behavior in this diff is a bit simpler than in D42555:
* only merge operators are passed through FilterMergeValue(). If fitler function returns true, the merge operator is ignored
* compaction filter is *not* called on: 1) results of merge operations and 2) base values that are getting merged with merge operands (the second case was also true in previous diff)
Do we also need a compaction filter to get called on merge results?
Test Plan: make && make check
Reviewers: lovro, tnovak, rven, yhchiang, sdong
Reviewed By: sdong
Subscribers: noetzli, kolmike, leveldb, dhruba, sdong
Differential Revision: https://reviews.facebook.net/D47847
9 years ago
|
|
|
&existing_snapshots_, env_, false, sub_compact->compaction,
|
|
|
|
compaction_filter));
|
|
|
|
auto c_iter = sub_compact->c_iter.get();
|
|
|
|
c_iter->SeekToFirst();
|
|
|
|
const auto& c_iter_stats = c_iter->iter_stats();
|
|
|
|
// TODO(noetzli): check whether we could check !shutting_down_->... only
|
|
|
|
// only occasionally (see diff D42687)
|
|
|
|
while (status.ok() && !shutting_down_->load(std::memory_order_acquire) &&
|
|
|
|
!cfd->IsDropped() && c_iter->Valid()) {
|
|
|
|
// Invariant: c_iter.status() is guaranteed to be OK if c_iter->Valid()
|
|
|
|
// returns true.
|
|
|
|
const Slice& key = c_iter->key();
|
|
|
|
const Slice& value = c_iter->value();
|
Parallelize L0-L1 Compaction: Restructure Compaction Job
Summary:
As of now compactions involving files from Level 0 and Level 1 are single
threaded because the files in L0, although sorted, are not range partitioned like
the other levels. This means that during L0-L1 compaction each file from L1
needs to be merged with potentially all the files from L0.
This attempt to parallelize the L0-L1 compaction assigns a thread and a
corresponding iterator to each L1 file that then considers only the key range
found in that L1 file and only the L0 files that have those keys (and only the
specific portion of those L0 files in which those keys are found). In this way
the overlap is minimized and potentially eliminated between different iterators
focusing on the same files.
The first step is to restructure the compaction logic to break L0-L1 compactions
into multiple, smaller, sequential compactions. Eventually each of these smaller
jobs will be run simultaneously. Areas to pay extra attention to are
# Correct aggregation of compaction job statistics across multiple threads
# Proper opening/closing of output files (make sure each thread's is unique)
# Keys that span multiple L1 files
# Skewed distributions of keys within L0 files
Test Plan: Make and run db_test (newer version has separate compaction tests) and compaction_job_stats_test
Reviewers: igor, noetzli, anthony, sdong, yhchiang
Reviewed By: yhchiang
Subscribers: MarkCallaghan, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D42699
9 years ago
|
|
|
|
|
|
|
// If an end key (exclusive) is specified, check if the current key is
|
|
|
|
// >= than it and exit if it is because the iterator is out of its range
|
Parallelize L0-L1 Compaction: Restructure Compaction Job
Summary:
As of now compactions involving files from Level 0 and Level 1 are single
threaded because the files in L0, although sorted, are not range partitioned like
the other levels. This means that during L0-L1 compaction each file from L1
needs to be merged with potentially all the files from L0.
This attempt to parallelize the L0-L1 compaction assigns a thread and a
corresponding iterator to each L1 file that then considers only the key range
found in that L1 file and only the L0 files that have those keys (and only the
specific portion of those L0 files in which those keys are found). In this way
the overlap is minimized and potentially eliminated between different iterators
focusing on the same files.
The first step is to restructure the compaction logic to break L0-L1 compactions
into multiple, smaller, sequential compactions. Eventually each of these smaller
jobs will be run simultaneously. Areas to pay extra attention to are
# Correct aggregation of compaction job statistics across multiple threads
# Proper opening/closing of output files (make sure each thread's is unique)
# Keys that span multiple L1 files
# Skewed distributions of keys within L0 files
Test Plan: Make and run db_test (newer version has separate compaction tests) and compaction_job_stats_test
Reviewers: igor, noetzli, anthony, sdong, yhchiang
Reviewed By: yhchiang
Subscribers: MarkCallaghan, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D42699
9 years ago
|
|
|
if (end != nullptr &&
|
|
|
|
cfd->user_comparator()->Compare(c_iter->user_key(), *end) >= 0) {
|
Parallelize L0-L1 Compaction: Restructure Compaction Job
Summary:
As of now compactions involving files from Level 0 and Level 1 are single
threaded because the files in L0, although sorted, are not range partitioned like
the other levels. This means that during L0-L1 compaction each file from L1
needs to be merged with potentially all the files from L0.
This attempt to parallelize the L0-L1 compaction assigns a thread and a
corresponding iterator to each L1 file that then considers only the key range
found in that L1 file and only the L0 files that have those keys (and only the
specific portion of those L0 files in which those keys are found). In this way
the overlap is minimized and potentially eliminated between different iterators
focusing on the same files.
The first step is to restructure the compaction logic to break L0-L1 compactions
into multiple, smaller, sequential compactions. Eventually each of these smaller
jobs will be run simultaneously. Areas to pay extra attention to are
# Correct aggregation of compaction job statistics across multiple threads
# Proper opening/closing of output files (make sure each thread's is unique)
# Keys that span multiple L1 files
# Skewed distributions of keys within L0 files
Test Plan: Make and run db_test (newer version has separate compaction tests) and compaction_job_stats_test
Reviewers: igor, noetzli, anthony, sdong, yhchiang
Reviewed By: yhchiang
Subscribers: MarkCallaghan, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D42699
9 years ago
|
|
|
break;
|
|
|
|
} else if (sub_compact->compaction->ShouldStopBefore(key) &&
|
|
|
|
sub_compact->builder != nullptr) {
|
|
|
|
status = FinishCompactionOutputFile(input->status(), sub_compact);
|
|
|
|
if (!status.ok()) {
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if (c_iter_stats.num_input_records % kRecordStatsEvery ==
|
|
|
|
kRecordStatsEvery - 1) {
|
|
|
|
RecordDroppedKeys(c_iter_stats, &sub_compact->compaction_job_stats);
|
|
|
|
c_iter->ResetRecordCounts();
|
|
|
|
RecordCompactionIOStats();
|
|
|
|
}
|
|
|
|
|
|
|
|
// Open output file if necessary
|
|
|
|
if (sub_compact->builder == nullptr) {
|
|
|
|
status = OpenCompactionOutputFile(sub_compact);
|
|
|
|
if (!status.ok()) {
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
assert(sub_compact->builder != nullptr);
|
|
|
|
assert(sub_compact->current_output() != nullptr);
|
|
|
|
sub_compact->builder->Add(key, value);
|
|
|
|
sub_compact->current_output()->meta.UpdateBoundaries(
|
|
|
|
key, c_iter->ikey().sequence);
|
|
|
|
sub_compact->num_output_records++;
|
|
|
|
|
|
|
|
// Close output file if it is big enough
|
|
|
|
// TODO(aekmekji): determine if file should be closed earlier than this
|
|
|
|
// during subcompactions (i.e. if output size, estimated by input size, is
|
|
|
|
// going to be 1.2MB and max_output_file_size = 1MB, prefer to have 0.6MB
|
|
|
|
// and 0.6MB instead of 1MB and 0.2MB)
|
|
|
|
if (sub_compact->builder->FileSize() >=
|
|
|
|
sub_compact->compaction->max_output_file_size()) {
|
|
|
|
status = FinishCompactionOutputFile(input->status(), sub_compact);
|
|
|
|
}
|
|
|
|
|
|
|
|
c_iter->Next();
|
|
|
|
}
|
|
|
|
|
|
|
|
sub_compact->num_input_records = c_iter_stats.num_input_records;
|
|
|
|
sub_compact->compaction_job_stats.num_input_deletion_records =
|
|
|
|
c_iter_stats.num_input_deletion_records;
|
|
|
|
sub_compact->compaction_job_stats.num_corrupt_keys =
|
|
|
|
c_iter_stats.num_input_corrupt_records;
|
|
|
|
sub_compact->compaction_job_stats.total_input_raw_key_bytes +=
|
|
|
|
c_iter_stats.total_input_raw_key_bytes;
|
|
|
|
sub_compact->compaction_job_stats.total_input_raw_value_bytes +=
|
|
|
|
c_iter_stats.total_input_raw_value_bytes;
|
|
|
|
|
|
|
|
RecordTick(stats_, FILTER_OPERATION_TOTAL_TIME,
|
|
|
|
c_iter_stats.total_filter_time);
|
|
|
|
RecordDroppedKeys(c_iter_stats, &sub_compact->compaction_job_stats);
|
|
|
|
RecordCompactionIOStats();
|
|
|
|
|
|
|
|
if (status.ok() &&
|
|
|
|
(shutting_down_->load(std::memory_order_acquire) || cfd->IsDropped())) {
|
|
|
|
status = Status::ShutdownInProgress(
|
|
|
|
"Database shutdown or Column family drop during compaction");
|
|
|
|
}
|
|
|
|
if (status.ok() && sub_compact->builder != nullptr) {
|
|
|
|
status = FinishCompactionOutputFile(input->status(), sub_compact);
|
|
|
|
}
|
|
|
|
if (status.ok()) {
|
|
|
|
status = input->status();
|
|
|
|
}
|
|
|
|
|
|
|
|
if (measure_io_stats_) {
|
|
|
|
sub_compact->compaction_job_stats.file_write_nanos +=
|
|
|
|
iostats_context.write_nanos - prev_write_nanos;
|
|
|
|
sub_compact->compaction_job_stats.file_fsync_nanos +=
|
|
|
|
iostats_context.fsync_nanos - prev_fsync_nanos;
|
|
|
|
sub_compact->compaction_job_stats.file_range_sync_nanos +=
|
|
|
|
iostats_context.range_sync_nanos - prev_range_sync_nanos;
|
|
|
|
sub_compact->compaction_job_stats.file_prepare_write_nanos +=
|
|
|
|
iostats_context.prepare_write_nanos - prev_prepare_write_nanos;
|
|
|
|
if (prev_perf_level != PerfLevel::kEnableTime) {
|
|
|
|
SetPerfLevel(prev_perf_level);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
sub_compact->c_iter.reset();
|
|
|
|
input.reset();
|
|
|
|
sub_compact->status = status;
|
|
|
|
}
|
|
|
|
|
|
|
|
void CompactionJob::RecordDroppedKeys(
|
|
|
|
const CompactionIteratorStats& c_iter_stats,
|
|
|
|
CompactionJobStats* compaction_job_stats) {
|
|
|
|
if (c_iter_stats.num_record_drop_user > 0) {
|
|
|
|
RecordTick(stats_, COMPACTION_KEY_DROP_USER,
|
|
|
|
c_iter_stats.num_record_drop_user);
|
|
|
|
}
|
|
|
|
if (c_iter_stats.num_record_drop_hidden > 0) {
|
|
|
|
RecordTick(stats_, COMPACTION_KEY_DROP_NEWER_ENTRY,
|
|
|
|
c_iter_stats.num_record_drop_hidden);
|
|
|
|
if (compaction_job_stats) {
|
|
|
|
compaction_job_stats->num_records_replaced +=
|
|
|
|
c_iter_stats.num_record_drop_hidden;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
if (c_iter_stats.num_record_drop_obsolete > 0) {
|
|
|
|
RecordTick(stats_, COMPACTION_KEY_DROP_OBSOLETE,
|
|
|
|
c_iter_stats.num_record_drop_obsolete);
|
|
|
|
if (compaction_job_stats) {
|
|
|
|
compaction_job_stats->num_expired_deletion_records +=
|
|
|
|
c_iter_stats.num_record_drop_obsolete;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
Status CompactionJob::FinishCompactionOutputFile(
|
|
|
|
const Status& input_status, SubcompactionState* sub_compact) {
|
|
|
|
AutoThreadOperationStageUpdater stage_updater(
|
|
|
|
ThreadStatus::STAGE_COMPACTION_SYNC_FILE);
|
|
|
|
assert(sub_compact != nullptr);
|
|
|
|
assert(sub_compact->outfile);
|
|
|
|
assert(sub_compact->builder != nullptr);
|
|
|
|
assert(sub_compact->current_output() != nullptr);
|
|
|
|
|
|
|
|
uint64_t output_number = sub_compact->current_output()->meta.fd.GetNumber();
|
|
|
|
assert(output_number != 0);
|
|
|
|
|
Add more table properties to EventLogger
Summary:
Example output:
{"time_micros": 1431463794310521, "job": 353, "event": "table_file_creation", "file_number": 387, "file_size": 86937, "table_info": {"data_size": "81801", "index_size": "9751", "filter_size": "0", "raw_key_size": "23448", "raw_average_key_size": "24.000000", "raw_value_size": "990571", "raw_average_value_size": "1013.890481", "num_data_blocks": "245", "num_entries": "977", "filter_policy_name": "", "kDeletedKeys": "0"}}
Also fixed a bug where BuildTable() in recovery was passing Env::IOHigh argument into paranoid_checks_file parameter.
Test Plan: make check + check out the output in the log
Reviewers: sdong, rven, yhchiang
Reviewed By: yhchiang
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D38343
10 years ago
|
|
|
TableProperties table_properties;
|
|
|
|
// Check for iterator errors
|
|
|
|
Status s = input_status;
|
|
|
|
auto meta = &sub_compact->current_output()->meta;
|
|
|
|
const uint64_t current_entries = sub_compact->builder->NumEntries();
|
|
|
|
meta->marked_for_compaction = sub_compact->builder->NeedCompact();
|
|
|
|
if (s.ok()) {
|
|
|
|
s = sub_compact->builder->Finish();
|
|
|
|
} else {
|
|
|
|
sub_compact->builder->Abandon();
|
|
|
|
}
|
|
|
|
const uint64_t current_bytes = sub_compact->builder->FileSize();
|
|
|
|
meta->fd.file_size = current_bytes;
|
|
|
|
sub_compact->current_output()->finished = true;
|
|
|
|
sub_compact->total_bytes += current_bytes;
|
|
|
|
|
|
|
|
// Finish and check for file errors
|
|
|
|
if (s.ok() && !db_options_.disableDataSync) {
|
Move rate_limiter, write buffering, most perf context instrumentation and most random kill out of Env
Summary: We want to keep Env a think layer for better portability. Less platform dependent codes should be moved out of Env. In this patch, I create a wrapper of file readers and writers, and put rate limiting, write buffering, as well as most perf context instrumentation and random kill out of Env. It will make it easier to maintain multiple Env in the future.
Test Plan: Run all existing unit tests.
Reviewers: anthony, kradhakrishnan, IslamAbdelRahman, yhchiang, igor
Reviewed By: igor
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D42321
9 years ago
|
|
|
StopWatch sw(env_, stats_, COMPACTION_OUTFILE_SYNC_MICROS);
|
|
|
|
s = sub_compact->outfile->Sync(db_options_.use_fsync);
|
|
|
|
}
|
|
|
|
if (s.ok()) {
|
|
|
|
s = sub_compact->outfile->Close();
|
|
|
|
}
|
|
|
|
sub_compact->outfile.reset();
|
|
|
|
|
|
|
|
if (s.ok() && current_entries > 0) {
|
|
|
|
// Verify that the table is usable
|
|
|
|
ColumnFamilyData* cfd = sub_compact->compaction->column_family_data();
|
|
|
|
Iterator* iter = cfd->table_cache()->NewIterator(
|
|
|
|
ReadOptions(), env_options_, cfd->internal_comparator(), meta->fd,
|
|
|
|
nullptr, cfd->internal_stats()->GetFileReadHist(
|
|
|
|
compact_->compaction->output_level()),
|
Measure file read latency histogram per level
Summary: In internal stats, remember read latency histogram, if statistics is enabled. It can be retrieved from DB::GetProperty() with "rocksdb.dbstats" property, if it is enabled.
Test Plan: Manually run db_bench and prints out "rocksdb.dbstats" by hand and make sure it prints out as expected
Reviewers: igor, IslamAbdelRahman, rven, kradhakrishnan, anthony, yhchiang
Reviewed By: yhchiang
Subscribers: MarkCallaghan, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D44193
9 years ago
|
|
|
false);
|
|
|
|
s = iter->status();
|
|
|
|
|
|
|
|
if (s.ok() && paranoid_file_checks_) {
|
|
|
|
for (iter->SeekToFirst(); iter->Valid(); iter->Next()) {}
|
|
|
|
s = iter->status();
|
|
|
|
}
|
|
|
|
|
|
|
|
delete iter;
|
|
|
|
if (s.ok()) {
|
|
|
|
TableFileCreationInfo info(sub_compact->builder->GetTableProperties());
|
|
|
|
info.db_name = dbname_;
|
|
|
|
info.cf_name = cfd->GetName();
|
|
|
|
info.file_path =
|
|
|
|
TableFileName(cfd->ioptions()->db_paths, meta->fd.GetNumber(),
|
|
|
|
meta->fd.GetPathId());
|
|
|
|
info.file_size = meta->fd.GetFileSize();
|
|
|
|
info.job_id = job_id_;
|
|
|
|
Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
|
|
|
|
"[%s] [JOB %d] Generated table #%" PRIu64 ": %" PRIu64
|
Print info message about files need compaction for debuging purpose
Summary:
When there are files marked for compaction after compactions, print extra messages to help debugging. Example:
2015/06/08-23:12:55.212855 7ff5013ff700 [default] [JOB 121] Generated table #75: 54 keys, 4807 bytes (need compaction)
2015/06/08-23:12:55.556194 7ff5013ff700 (Original Log Time 2015/06/08-23:12:55.556160) [default] compacted to: base level 1 max bytes base
10240 files[0 1 9 32 12 0 0 0] max score 0.96 (2 files need compaction), MB/sec: 0.0 rd, 0.1 wr, level 2, files in(1, 3) out(5) MB in(0.0,
0.0) out(0.0), read-write-amplify(11.3) write-amplify(5.7) OK, records in: 40, records dropped: 0
Test Plan:
Run test and see LOG files.
valgrind test DBTest.TablePropertiesNeedCompactTest
Reviewers: rven, yhchiang, kradhakrishnan, IslamAbdelRahman, igor
Reviewed By: igor
Subscribers: yoshinorim, maykov, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D39771
10 years ago
|
|
|
" keys, %" PRIu64 " bytes%s",
|
|
|
|
cfd->GetName().c_str(), job_id_, output_number, current_entries,
|
Print info message about files need compaction for debuging purpose
Summary:
When there are files marked for compaction after compactions, print extra messages to help debugging. Example:
2015/06/08-23:12:55.212855 7ff5013ff700 [default] [JOB 121] Generated table #75: 54 keys, 4807 bytes (need compaction)
2015/06/08-23:12:55.556194 7ff5013ff700 (Original Log Time 2015/06/08-23:12:55.556160) [default] compacted to: base level 1 max bytes base
10240 files[0 1 9 32 12 0 0 0] max score 0.96 (2 files need compaction), MB/sec: 0.0 rd, 0.1 wr, level 2, files in(1, 3) out(5) MB in(0.0,
0.0) out(0.0), read-write-amplify(11.3) write-amplify(5.7) OK, records in: 40, records dropped: 0
Test Plan:
Run test and see LOG files.
valgrind test DBTest.TablePropertiesNeedCompactTest
Reviewers: rven, yhchiang, kradhakrishnan, IslamAbdelRahman, igor
Reviewed By: igor
Subscribers: yoshinorim, maykov, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D39771
10 years ago
|
|
|
current_bytes,
|
|
|
|
meta->marked_for_compaction ? " (need compaction)" : "");
|
|
|
|
EventHelpers::LogAndNotifyTableFileCreation(
|
|
|
|
event_logger_, cfd->ioptions()->listeners, meta->fd, info);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
sub_compact->builder.reset();
|
|
|
|
return s;
|
|
|
|
}
|
|
|
|
|
|
|
|
Status CompactionJob::InstallCompactionResults(
|
|
|
|
const MutableCFOptions& mutable_cf_options, InstrumentedMutex* db_mutex) {
|
|
|
|
db_mutex->AssertHeld();
|
|
|
|
|
options.level_compaction_dynamic_level_bytes to allow RocksDB to pick size bases of levels dynamically.
Summary:
When having fixed max_bytes_for_level_base, the ratio of size of largest level and the second one can range from 0 to the multiplier. This makes LSM tree frequently irregular and unpredictable. It can also cause poor space amplification in some cases.
In this improvement (proposed by Igor Kabiljo), we introduce a parameter option.level_compaction_use_dynamic_max_bytes. When turning it on, RocksDB is free to pick a level base in the range of (options.max_bytes_for_level_base/options.max_bytes_for_level_multiplier, options.max_bytes_for_level_base] so that real level ratios are close to options.max_bytes_for_level_multiplier.
Test Plan: New unit tests and pass tests suites including valgrind.
Reviewers: MarkCallaghan, rven, yhchiang, igor, ikabiljo
Reviewed By: ikabiljo
Subscribers: yoshinorim, ikabiljo, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D31437
10 years ago
|
|
|
auto* compaction = compact_->compaction;
|
|
|
|
// paranoia: verify that the files that we started with
|
|
|
|
// still exist in the current version and in the same original level.
|
|
|
|
// This ensures that a concurrent compaction did not erroneously
|
|
|
|
// pick the same files to compact_.
|
options.level_compaction_dynamic_level_bytes to allow RocksDB to pick size bases of levels dynamically.
Summary:
When having fixed max_bytes_for_level_base, the ratio of size of largest level and the second one can range from 0 to the multiplier. This makes LSM tree frequently irregular and unpredictable. It can also cause poor space amplification in some cases.
In this improvement (proposed by Igor Kabiljo), we introduce a parameter option.level_compaction_use_dynamic_max_bytes. When turning it on, RocksDB is free to pick a level base in the range of (options.max_bytes_for_level_base/options.max_bytes_for_level_multiplier, options.max_bytes_for_level_base] so that real level ratios are close to options.max_bytes_for_level_multiplier.
Test Plan: New unit tests and pass tests suites including valgrind.
Reviewers: MarkCallaghan, rven, yhchiang, igor, ikabiljo
Reviewed By: ikabiljo
Subscribers: yoshinorim, ikabiljo, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D31437
10 years ago
|
|
|
if (!versions_->VerifyCompactionFileConsistency(compaction)) {
|
|
|
|
Compaction::InputLevelSummaryBuffer inputs_summary;
|
|
|
|
|
|
|
|
Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
|
options.level_compaction_dynamic_level_bytes to allow RocksDB to pick size bases of levels dynamically.
Summary:
When having fixed max_bytes_for_level_base, the ratio of size of largest level and the second one can range from 0 to the multiplier. This makes LSM tree frequently irregular and unpredictable. It can also cause poor space amplification in some cases.
In this improvement (proposed by Igor Kabiljo), we introduce a parameter option.level_compaction_use_dynamic_max_bytes. When turning it on, RocksDB is free to pick a level base in the range of (options.max_bytes_for_level_base/options.max_bytes_for_level_multiplier, options.max_bytes_for_level_base] so that real level ratios are close to options.max_bytes_for_level_multiplier.
Test Plan: New unit tests and pass tests suites including valgrind.
Reviewers: MarkCallaghan, rven, yhchiang, igor, ikabiljo
Reviewed By: ikabiljo
Subscribers: yoshinorim, ikabiljo, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D31437
10 years ago
|
|
|
"[%s] [JOB %d] Compaction %s aborted",
|
|
|
|
compaction->column_family_data()->GetName().c_str(), job_id_,
|
|
|
|
compaction->InputLevelSummary(&inputs_summary));
|
|
|
|
return Status::Corruption("Compaction input files inconsistent");
|
|
|
|
}
|
|
|
|
|
options.level_compaction_dynamic_level_bytes to allow RocksDB to pick size bases of levels dynamically.
Summary:
When having fixed max_bytes_for_level_base, the ratio of size of largest level and the second one can range from 0 to the multiplier. This makes LSM tree frequently irregular and unpredictable. It can also cause poor space amplification in some cases.
In this improvement (proposed by Igor Kabiljo), we introduce a parameter option.level_compaction_use_dynamic_max_bytes. When turning it on, RocksDB is free to pick a level base in the range of (options.max_bytes_for_level_base/options.max_bytes_for_level_multiplier, options.max_bytes_for_level_base] so that real level ratios are close to options.max_bytes_for_level_multiplier.
Test Plan: New unit tests and pass tests suites including valgrind.
Reviewers: MarkCallaghan, rven, yhchiang, igor, ikabiljo
Reviewed By: ikabiljo
Subscribers: yoshinorim, ikabiljo, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D31437
10 years ago
|
|
|
{
|
|
|
|
Compaction::InputLevelSummaryBuffer inputs_summary;
|
|
|
|
Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
|
|
|
|
"[%s] [JOB %d] Compacted %s => %" PRIu64 " bytes",
|
|
|
|
compaction->column_family_data()->GetName().c_str(), job_id_,
|
|
|
|
compaction->InputLevelSummary(&inputs_summary), compact_->total_bytes);
|
|
|
|
}
|
|
|
|
|
|
|
|
// Add compaction outputs
|
options.level_compaction_dynamic_level_bytes to allow RocksDB to pick size bases of levels dynamically.
Summary:
When having fixed max_bytes_for_level_base, the ratio of size of largest level and the second one can range from 0 to the multiplier. This makes LSM tree frequently irregular and unpredictable. It can also cause poor space amplification in some cases.
In this improvement (proposed by Igor Kabiljo), we introduce a parameter option.level_compaction_use_dynamic_max_bytes. When turning it on, RocksDB is free to pick a level base in the range of (options.max_bytes_for_level_base/options.max_bytes_for_level_multiplier, options.max_bytes_for_level_base] so that real level ratios are close to options.max_bytes_for_level_multiplier.
Test Plan: New unit tests and pass tests suites including valgrind.
Reviewers: MarkCallaghan, rven, yhchiang, igor, ikabiljo
Reviewed By: ikabiljo
Subscribers: yoshinorim, ikabiljo, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D31437
10 years ago
|
|
|
compaction->AddInputDeletions(compact_->compaction->edit());
|
|
|
|
|
|
|
|
for (const auto& sub_compact : compact_->sub_compact_states) {
|
|
|
|
for (const auto& out : sub_compact.outputs) {
|
|
|
|
compaction->edit()->AddFile(compaction->output_level(), out.meta);
|
|
|
|
}
|
options.level_compaction_dynamic_level_bytes to allow RocksDB to pick size bases of levels dynamically.
Summary:
When having fixed max_bytes_for_level_base, the ratio of size of largest level and the second one can range from 0 to the multiplier. This makes LSM tree frequently irregular and unpredictable. It can also cause poor space amplification in some cases.
In this improvement (proposed by Igor Kabiljo), we introduce a parameter option.level_compaction_use_dynamic_max_bytes. When turning it on, RocksDB is free to pick a level base in the range of (options.max_bytes_for_level_base/options.max_bytes_for_level_multiplier, options.max_bytes_for_level_base] so that real level ratios are close to options.max_bytes_for_level_multiplier.
Test Plan: New unit tests and pass tests suites including valgrind.
Reviewers: MarkCallaghan, rven, yhchiang, igor, ikabiljo
Reviewed By: ikabiljo
Subscribers: yoshinorim, ikabiljo, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D31437
10 years ago
|
|
|
}
|
|
|
|
return versions_->LogAndApply(compaction->column_family_data(),
|
|
|
|
mutable_cf_options, compaction->edit(),
|
options.level_compaction_dynamic_level_bytes to allow RocksDB to pick size bases of levels dynamically.
Summary:
When having fixed max_bytes_for_level_base, the ratio of size of largest level and the second one can range from 0 to the multiplier. This makes LSM tree frequently irregular and unpredictable. It can also cause poor space amplification in some cases.
In this improvement (proposed by Igor Kabiljo), we introduce a parameter option.level_compaction_use_dynamic_max_bytes. When turning it on, RocksDB is free to pick a level base in the range of (options.max_bytes_for_level_base/options.max_bytes_for_level_multiplier, options.max_bytes_for_level_base] so that real level ratios are close to options.max_bytes_for_level_multiplier.
Test Plan: New unit tests and pass tests suites including valgrind.
Reviewers: MarkCallaghan, rven, yhchiang, igor, ikabiljo
Reviewed By: ikabiljo
Subscribers: yoshinorim, ikabiljo, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D31437
10 years ago
|
|
|
db_mutex, db_directory_);
|
|
|
|
}
|
|
|
|
|
|
|
|
void CompactionJob::RecordCompactionIOStats() {
|
|
|
|
RecordTick(stats_, COMPACT_READ_BYTES, IOSTATS(bytes_read));
|
|
|
|
ThreadStatusUtil::IncreaseThreadOperationProperty(
|
|
|
|
ThreadStatus::COMPACTION_BYTES_READ, IOSTATS(bytes_read));
|
|
|
|
IOSTATS_RESET(bytes_read);
|
|
|
|
RecordTick(stats_, COMPACT_WRITE_BYTES, IOSTATS(bytes_written));
|
|
|
|
ThreadStatusUtil::IncreaseThreadOperationProperty(
|
|
|
|
ThreadStatus::COMPACTION_BYTES_WRITTEN, IOSTATS(bytes_written));
|
|
|
|
IOSTATS_RESET(bytes_written);
|
|
|
|
}
|
|
|
|
|
|
|
|
Status CompactionJob::OpenCompactionOutputFile(
|
|
|
|
SubcompactionState* sub_compact) {
|
|
|
|
assert(sub_compact != nullptr);
|
|
|
|
assert(sub_compact->builder == nullptr);
|
|
|
|
// no need to lock because VersionSet::next_file_number_ is atomic
|
|
|
|
uint64_t file_number = versions_->NewFileNumber();
|
|
|
|
// Make the output file
|
Move rate_limiter, write buffering, most perf context instrumentation and most random kill out of Env
Summary: We want to keep Env a think layer for better portability. Less platform dependent codes should be moved out of Env. In this patch, I create a wrapper of file readers and writers, and put rate limiting, write buffering, as well as most perf context instrumentation and random kill out of Env. It will make it easier to maintain multiple Env in the future.
Test Plan: Run all existing unit tests.
Reviewers: anthony, kradhakrishnan, IslamAbdelRahman, yhchiang, igor
Reviewed By: igor
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D42321
9 years ago
|
|
|
unique_ptr<WritableFile> writable_file;
|
|
|
|
std::string fname = TableFileName(db_options_.db_paths, file_number,
|
|
|
|
sub_compact->compaction->output_path_id());
|
Move rate_limiter, write buffering, most perf context instrumentation and most random kill out of Env
Summary: We want to keep Env a think layer for better portability. Less platform dependent codes should be moved out of Env. In this patch, I create a wrapper of file readers and writers, and put rate limiting, write buffering, as well as most perf context instrumentation and random kill out of Env. It will make it easier to maintain multiple Env in the future.
Test Plan: Run all existing unit tests.
Reviewers: anthony, kradhakrishnan, IslamAbdelRahman, yhchiang, igor
Reviewed By: igor
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D42321
9 years ago
|
|
|
Status s = env_->NewWritableFile(fname, &writable_file, env_options_);
|
|
|
|
if (!s.ok()) {
|
|
|
|
Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
|
|
|
|
"[%s] [JOB %d] OpenCompactionOutputFiles for table #%" PRIu64
|
|
|
|
" fails at NewWritableFile with status %s",
|
|
|
|
sub_compact->compaction->column_family_data()->GetName().c_str(),
|
|
|
|
job_id_, file_number, s.ToString().c_str());
|
|
|
|
LogFlush(db_options_.info_log);
|
|
|
|
return s;
|
|
|
|
}
|
|
|
|
SubcompactionState::Output out;
|
|
|
|
out.meta.fd =
|
|
|
|
FileDescriptor(file_number, sub_compact->compaction->output_path_id(), 0);
|
|
|
|
out.finished = false;
|
|
|
|
|
|
|
|
sub_compact->outputs.push_back(out);
|
Move rate_limiter, write buffering, most perf context instrumentation and most random kill out of Env
Summary: We want to keep Env a think layer for better portability. Less platform dependent codes should be moved out of Env. In this patch, I create a wrapper of file readers and writers, and put rate limiting, write buffering, as well as most perf context instrumentation and random kill out of Env. It will make it easier to maintain multiple Env in the future.
Test Plan: Run all existing unit tests.
Reviewers: anthony, kradhakrishnan, IslamAbdelRahman, yhchiang, igor
Reviewed By: igor
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D42321
9 years ago
|
|
|
writable_file->SetIOPriority(Env::IO_LOW);
|
|
|
|
writable_file->SetPreallocationBlockSize(static_cast<size_t>(
|
|
|
|
sub_compact->compaction->OutputFilePreallocationSize()));
|
|
|
|
sub_compact->outfile.reset(
|
Move rate_limiter, write buffering, most perf context instrumentation and most random kill out of Env
Summary: We want to keep Env a think layer for better portability. Less platform dependent codes should be moved out of Env. In this patch, I create a wrapper of file readers and writers, and put rate limiting, write buffering, as well as most perf context instrumentation and random kill out of Env. It will make it easier to maintain multiple Env in the future.
Test Plan: Run all existing unit tests.
Reviewers: anthony, kradhakrishnan, IslamAbdelRahman, yhchiang, igor
Reviewed By: igor
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D42321
9 years ago
|
|
|
new WritableFileWriter(std::move(writable_file), env_options_));
|
|
|
|
|
|
|
|
ColumnFamilyData* cfd = sub_compact->compaction->column_family_data();
|
|
|
|
// If the Column family flag is to only optimize filters for hits,
|
|
|
|
// we can skip creating filters if this is the bottommost_level where
|
|
|
|
// data is going to be found
|
|
|
|
bool skip_filters =
|
|
|
|
cfd->ioptions()->optimize_filters_for_hits && bottommost_level_;
|
|
|
|
sub_compact->builder.reset(NewTableBuilder(
|
A new call back to TablePropertiesCollector to allow users know the entry is add, delete or merge
Summary:
Currently users have no idea a key is add, delete or merge from TablePropertiesCollector call back. Add a new function to add it.
Also refactor the codes so that
(1) make table property collector and internal table property collector two separate data structures with the later one now exposed
(2) table builders only receive internal table properties
Test Plan: Add cases in table_properties_collector_test to cover both of old and new ways of using TablePropertiesCollector.
Reviewers: yhchiang, igor.sugak, rven, igor
Reviewed By: rven, igor
Subscribers: meyering, yoshinorim, maykov, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D35373
10 years ago
|
|
|
*cfd->ioptions(), cfd->internal_comparator(),
|
|
|
|
cfd->int_tbl_prop_collector_factories(), sub_compact->outfile.get(),
|
|
|
|
sub_compact->compaction->output_compression(),
|
|
|
|
cfd->ioptions()->compression_opts, skip_filters));
|
|
|
|
LogFlush(db_options_.info_log);
|
|
|
|
return s;
|
|
|
|
}
|
|
|
|
|
|
|
|
void CompactionJob::CleanupCompaction() {
|
|
|
|
for (SubcompactionState& sub_compact : compact_->sub_compact_states) {
|
|
|
|
const auto& sub_status = sub_compact.status;
|
|
|
|
|
|
|
|
if (sub_compact.builder != nullptr) {
|
|
|
|
// May happen if we get a shutdown call in the middle of compaction
|
|
|
|
sub_compact.builder->Abandon();
|
|
|
|
sub_compact.builder.reset();
|
|
|
|
} else {
|
|
|
|
assert(!sub_status.ok() || sub_compact.outfile == nullptr);
|
|
|
|
}
|
|
|
|
for (const auto& out : sub_compact.outputs) {
|
|
|
|
// If this file was inserted into the table cache then remove
|
|
|
|
// them here because this compaction was not committed.
|
|
|
|
if (!sub_status.ok()) {
|
|
|
|
TableCache::Evict(table_cache_.get(), out.meta.fd.GetNumber());
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
delete compact_;
|
|
|
|
compact_ = nullptr;
|
|
|
|
}
|
|
|
|
|
|
|
|
#ifndef ROCKSDB_LITE
|
|
|
|
namespace {
|
|
|
|
void CopyPrefix(
|
|
|
|
const Slice& src, size_t prefix_length, std::string* dst) {
|
|
|
|
assert(prefix_length > 0);
|
|
|
|
size_t length = src.size() > prefix_length ? prefix_length : src.size();
|
|
|
|
dst->assign(src.data(), length);
|
|
|
|
}
|
|
|
|
} // namespace
|
|
|
|
|
|
|
|
#endif // !ROCKSDB_LITE
|
|
|
|
|
|
|
|
void CompactionJob::UpdateCompactionStats() {
|
|
|
|
Compaction* compaction = compact_->compaction;
|
|
|
|
compaction_stats_.num_input_files_in_non_output_levels = 0;
|
|
|
|
compaction_stats_.num_input_files_in_output_level = 0;
|
|
|
|
for (int input_level = 0;
|
|
|
|
input_level < static_cast<int>(compaction->num_input_levels());
|
|
|
|
++input_level) {
|
|
|
|
if (compaction->start_level() + input_level
|
|
|
|
!= compaction->output_level()) {
|
|
|
|
UpdateCompactionInputStatsHelper(
|
|
|
|
&compaction_stats_.num_input_files_in_non_output_levels,
|
|
|
|
&compaction_stats_.bytes_read_non_output_levels,
|
|
|
|
input_level);
|
|
|
|
} else {
|
|
|
|
UpdateCompactionInputStatsHelper(
|
|
|
|
&compaction_stats_.num_input_files_in_output_level,
|
|
|
|
&compaction_stats_.bytes_read_output_level,
|
|
|
|
input_level);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
for (const auto& sub_compact : compact_->sub_compact_states) {
|
|
|
|
size_t num_output_files = sub_compact.outputs.size();
|
|
|
|
if (sub_compact.builder != nullptr) {
|
|
|
|
// An error occurred so ignore the last output.
|
|
|
|
assert(num_output_files > 0);
|
|
|
|
--num_output_files;
|
|
|
|
}
|
|
|
|
compaction_stats_.num_output_files += static_cast<int>(num_output_files);
|
|
|
|
|
|
|
|
for (const auto& out : sub_compact.outputs) {
|
|
|
|
compaction_stats_.bytes_written += out.meta.fd.file_size;
|
|
|
|
}
|
|
|
|
if (sub_compact.num_input_records > sub_compact.num_output_records) {
|
|
|
|
compaction_stats_.num_dropped_records +=
|
|
|
|
sub_compact.num_input_records - sub_compact.num_output_records;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
void CompactionJob::UpdateCompactionInputStatsHelper(
|
|
|
|
int* num_files, uint64_t* bytes_read, int input_level) {
|
|
|
|
const Compaction* compaction = compact_->compaction;
|
|
|
|
auto num_input_files = compaction->num_input_files(input_level);
|
|
|
|
*num_files += static_cast<int>(num_input_files);
|
|
|
|
|
|
|
|
for (size_t i = 0; i < num_input_files; ++i) {
|
|
|
|
const auto* file_meta = compaction->input(input_level, i);
|
|
|
|
*bytes_read += file_meta->fd.GetFileSize();
|
|
|
|
compaction_stats_.num_input_records +=
|
|
|
|
static_cast<uint64_t>(file_meta->num_entries);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
void CompactionJob::UpdateCompactionJobStats(
|
|
|
|
const InternalStats::CompactionStats& stats) const {
|
|
|
|
#ifndef ROCKSDB_LITE
|
|
|
|
if (compaction_job_stats_) {
|
|
|
|
compaction_job_stats_->elapsed_micros = stats.micros;
|
|
|
|
|
|
|
|
// input information
|
|
|
|
compaction_job_stats_->total_input_bytes =
|
|
|
|
stats.bytes_read_non_output_levels +
|
|
|
|
stats.bytes_read_output_level;
|
|
|
|
compaction_job_stats_->num_input_records =
|
|
|
|
compact_->num_input_records;
|
|
|
|
compaction_job_stats_->num_input_files =
|
|
|
|
stats.num_input_files_in_non_output_levels +
|
|
|
|
stats.num_input_files_in_output_level;
|
|
|
|
compaction_job_stats_->num_input_files_at_output_level =
|
|
|
|
stats.num_input_files_in_output_level;
|
|
|
|
|
|
|
|
// output information
|
|
|
|
compaction_job_stats_->total_output_bytes = stats.bytes_written;
|
|
|
|
compaction_job_stats_->num_output_records =
|
|
|
|
compact_->num_output_records;
|
|
|
|
compaction_job_stats_->num_output_files = stats.num_output_files;
|
|
|
|
|
|
|
|
if (compact_->NumOutputFiles() > 0U) {
|
|
|
|
CopyPrefix(
|
|
|
|
compact_->SmallestUserKey(),
|
|
|
|
CompactionJobStats::kMaxPrefixLength,
|
|
|
|
&compaction_job_stats_->smallest_output_key_prefix);
|
|
|
|
CopyPrefix(
|
|
|
|
compact_->LargestUserKey(),
|
|
|
|
CompactionJobStats::kMaxPrefixLength,
|
|
|
|
&compaction_job_stats_->largest_output_key_prefix);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
#endif // !ROCKSDB_LITE
|
|
|
|
}
|
|
|
|
|
|
|
|
void CompactionJob::LogCompaction() {
|
|
|
|
Compaction* compaction = compact_->compaction;
|
|
|
|
ColumnFamilyData* cfd = compaction->column_family_data();
|
|
|
|
|
|
|
|
// Let's check if anything will get logged. Don't prepare all the info if
|
|
|
|
// we're not logging
|
|
|
|
if (db_options_.info_log_level <= InfoLogLevel::INFO_LEVEL) {
|
|
|
|
Compaction::InputLevelSummaryBuffer inputs_summary;
|
|
|
|
Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
|
|
|
|
"[%s] [JOB %d] Compacting %s, score %.2f", cfd->GetName().c_str(),
|
|
|
|
job_id_, compaction->InputLevelSummary(&inputs_summary),
|
|
|
|
compaction->score());
|
|
|
|
char scratch[2345];
|
|
|
|
compaction->Summary(scratch, sizeof(scratch));
|
|
|
|
Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
|
|
|
|
"[%s] Compaction start summary: %s\n", cfd->GetName().c_str(), scratch);
|
|
|
|
// build event logger report
|
|
|
|
auto stream = event_logger_->Log();
|
|
|
|
stream << "job" << job_id_ << "event"
|
|
|
|
<< "compaction_started";
|
|
|
|
for (size_t i = 0; i < compaction->num_input_levels(); ++i) {
|
|
|
|
stream << ("files_L" + ToString(compaction->level(i)));
|
|
|
|
stream.StartArray();
|
|
|
|
for (auto f : *compaction->inputs(i)) {
|
|
|
|
stream << f->fd.GetNumber();
|
|
|
|
}
|
|
|
|
stream.EndArray();
|
|
|
|
}
|
|
|
|
stream << "score" << compaction->score() << "input_data_size"
|
|
|
|
<< compaction->CalculateTotalInputSize();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
} // namespace rocksdb
|