Tiered Compaction: per key placement support (#9964)
Summary: Support per_key_placement for last level compaction, which will be used for tiered compaction. * compaction iterator reports which level a key should output to; * compaction get the output level information and check if it's safe to output the data to penultimate level; * all compaction output files will be installed. * extra internal compaction stats added for penultimate level. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9964 Test Plan: * Unittest * db_bench, no significate difference: https://gist.github.com/jay-zhuang/3645f8fb97ec0ab47c10704bb39fd6e4 * microbench manual compaction no significate difference: https://gist.github.com/jay-zhuang/ba679b3e89e24992615ee9eef310e6dd * run the db_stress multiple times (not covering the new feature) looks good (internal: https://fburl.com/sandcastle/9w84pp2m) Reviewed By: ajkr Differential Revision: D36249494 Pulled By: jay-zhuang fbshipit-source-id: a96da57c8031c1df83e4a7a8567b657a112b80a3main
parent
7e1b417824
commit
6ce0b2ca34
File diff suppressed because it is too large
Load Diff
@ -0,0 +1,314 @@ |
|||||||
|
// Copyright (c) Meta Platforms, Inc. and affiliates.
|
||||||
|
//
|
||||||
|
// This source code is licensed under both the GPLv2 (found in the
|
||||||
|
// COPYING file in the root directory) and Apache 2.0 License
|
||||||
|
// (found in the LICENSE.Apache file in the root directory).
|
||||||
|
//
|
||||||
|
// 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/compaction_outputs.h" |
||||||
|
|
||||||
|
#include "db/builder.h" |
||||||
|
|
||||||
|
namespace ROCKSDB_NAMESPACE { |
||||||
|
|
||||||
|
void CompactionOutputs::NewBuilder(const TableBuilderOptions& tboptions) { |
||||||
|
builder_.reset(NewTableBuilder(tboptions, file_writer_.get())); |
||||||
|
} |
||||||
|
|
||||||
|
Status CompactionOutputs::Finish(const Status& intput_status) { |
||||||
|
FileMetaData* meta = GetMetaData(); |
||||||
|
assert(meta != nullptr); |
||||||
|
Status s = intput_status; |
||||||
|
if (s.ok()) { |
||||||
|
s = builder_->Finish(); |
||||||
|
} else { |
||||||
|
builder_->Abandon(); |
||||||
|
} |
||||||
|
Status io_s = builder_->io_status(); |
||||||
|
if (s.ok()) { |
||||||
|
s = io_s; |
||||||
|
} else { |
||||||
|
io_s.PermitUncheckedError(); |
||||||
|
} |
||||||
|
const uint64_t current_bytes = builder_->FileSize(); |
||||||
|
if (s.ok()) { |
||||||
|
meta->fd.file_size = current_bytes; |
||||||
|
meta->marked_for_compaction = builder_->NeedCompact(); |
||||||
|
} |
||||||
|
current_output().finished = true; |
||||||
|
stats_.bytes_written += current_bytes; |
||||||
|
stats_.num_output_files = outputs_.size(); |
||||||
|
|
||||||
|
return s; |
||||||
|
} |
||||||
|
|
||||||
|
IOStatus CompactionOutputs::WriterSyncClose(const Status& input_status, |
||||||
|
SystemClock* clock, |
||||||
|
Statistics* statistics, |
||||||
|
bool use_fsync) { |
||||||
|
IOStatus io_s; |
||||||
|
if (input_status.ok()) { |
||||||
|
StopWatch sw(clock, statistics, COMPACTION_OUTFILE_SYNC_MICROS); |
||||||
|
io_s = file_writer_->Sync(use_fsync); |
||||||
|
} |
||||||
|
if (input_status.ok() && io_s.ok()) { |
||||||
|
io_s = file_writer_->Close(); |
||||||
|
} |
||||||
|
|
||||||
|
if (input_status.ok() && io_s.ok()) { |
||||||
|
FileMetaData* meta = GetMetaData(); |
||||||
|
meta->file_checksum = file_writer_->GetFileChecksum(); |
||||||
|
meta->file_checksum_func_name = file_writer_->GetFileChecksumFuncName(); |
||||||
|
} |
||||||
|
|
||||||
|
file_writer_.reset(); |
||||||
|
|
||||||
|
return io_s; |
||||||
|
} |
||||||
|
|
||||||
|
Status CompactionOutputs::AddToOutput( |
||||||
|
const CompactionIterator& c_iter, |
||||||
|
const CompactionFileOpenFunc& open_file_func, |
||||||
|
const CompactionFileCloseFunc& close_file_func) { |
||||||
|
Status s; |
||||||
|
const Slice& key = c_iter.key(); |
||||||
|
|
||||||
|
if (!pending_close_ && c_iter.Valid() && partitioner_ && HasBuilder() && |
||||||
|
partitioner_->ShouldPartition( |
||||||
|
PartitionerRequest(last_key_for_partitioner_, c_iter.user_key(), |
||||||
|
current_output_file_size_)) == kRequired) { |
||||||
|
pending_close_ = true; |
||||||
|
} |
||||||
|
|
||||||
|
if (pending_close_) { |
||||||
|
s = close_file_func(*this, c_iter.InputStatus(), key); |
||||||
|
pending_close_ = false; |
||||||
|
} |
||||||
|
if (!s.ok()) { |
||||||
|
return s; |
||||||
|
} |
||||||
|
|
||||||
|
// Open output file if necessary
|
||||||
|
if (!HasBuilder()) { |
||||||
|
s = open_file_func(*this); |
||||||
|
} |
||||||
|
if (!s.ok()) { |
||||||
|
return s; |
||||||
|
} |
||||||
|
|
||||||
|
Output& curr = current_output(); |
||||||
|
assert(builder_ != nullptr); |
||||||
|
const Slice& value = c_iter.value(); |
||||||
|
s = curr.validator.Add(key, value); |
||||||
|
if (!s.ok()) { |
||||||
|
return s; |
||||||
|
} |
||||||
|
builder_->Add(key, value); |
||||||
|
|
||||||
|
stats_.num_output_records++; |
||||||
|
current_output_file_size_ = builder_->EstimatedFileSize(); |
||||||
|
|
||||||
|
if (blob_garbage_meter_) { |
||||||
|
s = blob_garbage_meter_->ProcessOutFlow(key, value); |
||||||
|
} |
||||||
|
|
||||||
|
if (!s.ok()) { |
||||||
|
return s; |
||||||
|
} |
||||||
|
|
||||||
|
const ParsedInternalKey& ikey = c_iter.ikey(); |
||||||
|
s = current_output().meta.UpdateBoundaries(key, value, ikey.sequence, |
||||||
|
ikey.type); |
||||||
|
|
||||||
|
// Close output file if it is big enough. Two possibilities determine it's
|
||||||
|
// time to close it: (1) the current key should be this file's last key, (2)
|
||||||
|
// the next key should not be in this file.
|
||||||
|
//
|
||||||
|
// 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 (compaction_->output_level() != 0 && |
||||||
|
current_output_file_size_ >= compaction_->max_output_file_size()) { |
||||||
|
pending_close_ = true; |
||||||
|
} |
||||||
|
|
||||||
|
if (partitioner_) { |
||||||
|
last_key_for_partitioner_.assign(c_iter.user_key().data_, |
||||||
|
c_iter.user_key().size_); |
||||||
|
} |
||||||
|
|
||||||
|
return s; |
||||||
|
} |
||||||
|
|
||||||
|
Status CompactionOutputs::AddRangeDels( |
||||||
|
const Slice* comp_start, const Slice* comp_end, |
||||||
|
CompactionIterationStats& range_del_out_stats, bool bottommost_level, |
||||||
|
const InternalKeyComparator& icmp, SequenceNumber earliest_snapshot, |
||||||
|
const Slice& next_table_min_key) { |
||||||
|
assert(HasRangeDel()); |
||||||
|
FileMetaData& meta = current_output().meta; |
||||||
|
const Comparator* ucmp = icmp.user_comparator(); |
||||||
|
|
||||||
|
Slice lower_bound_guard, upper_bound_guard; |
||||||
|
std::string smallest_user_key; |
||||||
|
const Slice *lower_bound, *upper_bound; |
||||||
|
bool lower_bound_from_sub_compact = false; |
||||||
|
|
||||||
|
size_t output_size = outputs_.size(); |
||||||
|
if (output_size == 1) { |
||||||
|
// For the first output table, include range tombstones before the min
|
||||||
|
// key but after the subcompaction boundary.
|
||||||
|
lower_bound = comp_start; |
||||||
|
lower_bound_from_sub_compact = true; |
||||||
|
} else if (meta.smallest.size() > 0) { |
||||||
|
// For subsequent output tables, only include range tombstones from min
|
||||||
|
// key onwards since the previous file was extended to contain range
|
||||||
|
// tombstones falling before min key.
|
||||||
|
smallest_user_key = meta.smallest.user_key().ToString(false /*hex*/); |
||||||
|
lower_bound_guard = Slice(smallest_user_key); |
||||||
|
lower_bound = &lower_bound_guard; |
||||||
|
} else { |
||||||
|
lower_bound = nullptr; |
||||||
|
} |
||||||
|
if (!next_table_min_key.empty()) { |
||||||
|
// This may be the last file in the subcompaction in some cases, so we
|
||||||
|
// need to compare the end key of subcompaction with the next file start
|
||||||
|
// key. When the end key is chosen by the subcompaction, we know that
|
||||||
|
// it must be the biggest key in output file. Therefore, it is safe to
|
||||||
|
// use the smaller key as the upper bound of the output file, to ensure
|
||||||
|
// that there is no overlapping between different output files.
|
||||||
|
upper_bound_guard = ExtractUserKey(next_table_min_key); |
||||||
|
if (comp_end != nullptr && |
||||||
|
ucmp->Compare(upper_bound_guard, *comp_end) >= 0) { |
||||||
|
upper_bound = comp_end; |
||||||
|
} else { |
||||||
|
upper_bound = &upper_bound_guard; |
||||||
|
} |
||||||
|
} else { |
||||||
|
// This is the last file in the subcompaction, so extend until the
|
||||||
|
// subcompaction ends.
|
||||||
|
upper_bound = comp_end; |
||||||
|
} |
||||||
|
bool has_overlapping_endpoints; |
||||||
|
if (upper_bound != nullptr && meta.largest.size() > 0) { |
||||||
|
has_overlapping_endpoints = |
||||||
|
ucmp->Compare(meta.largest.user_key(), *upper_bound) == 0; |
||||||
|
} else { |
||||||
|
has_overlapping_endpoints = false; |
||||||
|
} |
||||||
|
|
||||||
|
// The end key of the subcompaction must be bigger or equal to the upper
|
||||||
|
// bound. If the end of subcompaction is null or the upper bound is null,
|
||||||
|
// it means that this file is the last file in the compaction. So there
|
||||||
|
// will be no overlapping between this file and others.
|
||||||
|
assert(comp_end == nullptr || upper_bound == nullptr || |
||||||
|
ucmp->Compare(*upper_bound, *comp_end) <= 0); |
||||||
|
auto it = range_del_agg_->NewIterator(lower_bound, upper_bound, |
||||||
|
has_overlapping_endpoints); |
||||||
|
// Position the range tombstone output iterator. There may be tombstone
|
||||||
|
// fragments that are entirely out of range, so make sure that we do not
|
||||||
|
// include those.
|
||||||
|
if (lower_bound != nullptr) { |
||||||
|
it->Seek(*lower_bound); |
||||||
|
} else { |
||||||
|
it->SeekToFirst(); |
||||||
|
} |
||||||
|
for (; it->Valid(); it->Next()) { |
||||||
|
auto tombstone = it->Tombstone(); |
||||||
|
if (upper_bound != nullptr) { |
||||||
|
int cmp = ucmp->Compare(*upper_bound, tombstone.start_key_); |
||||||
|
if ((has_overlapping_endpoints && cmp < 0) || |
||||||
|
(!has_overlapping_endpoints && cmp <= 0)) { |
||||||
|
// Tombstones starting after upper_bound only need to be included in
|
||||||
|
// the next table. If the current SST ends before upper_bound, i.e.,
|
||||||
|
// `has_overlapping_endpoints == false`, we can also skip over range
|
||||||
|
// tombstones that start exactly at upper_bound. Such range
|
||||||
|
// tombstones will be included in the next file and are not relevant
|
||||||
|
// to the point keys or endpoints of the current file.
|
||||||
|
break; |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
if (bottommost_level && tombstone.seq_ <= earliest_snapshot) { |
||||||
|
// TODO(andrewkr): tombstones that span multiple output files are
|
||||||
|
// counted for each compaction output file, so lots of double
|
||||||
|
// counting.
|
||||||
|
range_del_out_stats.num_range_del_drop_obsolete++; |
||||||
|
range_del_out_stats.num_record_drop_obsolete++; |
||||||
|
continue; |
||||||
|
} |
||||||
|
|
||||||
|
auto kv = tombstone.Serialize(); |
||||||
|
assert(lower_bound == nullptr || |
||||||
|
ucmp->Compare(*lower_bound, kv.second) < 0); |
||||||
|
// Range tombstone is not supported by output validator yet.
|
||||||
|
builder_->Add(kv.first.Encode(), kv.second); |
||||||
|
InternalKey smallest_candidate = std::move(kv.first); |
||||||
|
if (lower_bound != nullptr && |
||||||
|
ucmp->Compare(smallest_candidate.user_key(), *lower_bound) <= 0) { |
||||||
|
// Pretend the smallest key has the same user key as lower_bound
|
||||||
|
// (the max key in the previous table or subcompaction) in order for
|
||||||
|
// files to appear key-space partitioned.
|
||||||
|
//
|
||||||
|
// When lower_bound is chosen by a subcompaction, we know that
|
||||||
|
// subcompactions over smaller keys cannot contain any keys at
|
||||||
|
// lower_bound. We also know that smaller subcompactions exist,
|
||||||
|
// because otherwise the subcompaction woud be unbounded on the left.
|
||||||
|
// As a result, we know that no other files on the output level will
|
||||||
|
// contain actual keys at lower_bound (an output file may have a
|
||||||
|
// largest key of lower_bound@kMaxSequenceNumber, but this only
|
||||||
|
// indicates a large range tombstone was truncated). Therefore, it is
|
||||||
|
// safe to use the tombstone's sequence number, to ensure that keys at
|
||||||
|
// lower_bound at lower levels are covered by truncated tombstones.
|
||||||
|
//
|
||||||
|
// If lower_bound was chosen by the smallest data key in the file,
|
||||||
|
// choose lowest seqnum so this file's smallest internal key comes
|
||||||
|
// after the previous file's largest. The fake seqnum is OK because
|
||||||
|
// the read path's file-picking code only considers user key.
|
||||||
|
smallest_candidate = InternalKey( |
||||||
|
*lower_bound, lower_bound_from_sub_compact ? tombstone.seq_ : 0, |
||||||
|
kTypeRangeDeletion); |
||||||
|
} |
||||||
|
InternalKey largest_candidate = tombstone.SerializeEndKey(); |
||||||
|
if (upper_bound != nullptr && |
||||||
|
ucmp->Compare(*upper_bound, largest_candidate.user_key()) <= 0) { |
||||||
|
// Pretend the largest key has the same user key as upper_bound (the
|
||||||
|
// min key in the following table or subcompaction) in order for files
|
||||||
|
// to appear key-space partitioned.
|
||||||
|
//
|
||||||
|
// Choose highest seqnum so this file's largest internal key comes
|
||||||
|
// before the next file's/subcompaction's smallest. The fake seqnum is
|
||||||
|
// OK because the read path's file-picking code only considers the
|
||||||
|
// user key portion.
|
||||||
|
//
|
||||||
|
// Note Seek() also creates InternalKey with (user_key,
|
||||||
|
// kMaxSequenceNumber), but with kTypeDeletion (0x7) instead of
|
||||||
|
// kTypeRangeDeletion (0xF), so the range tombstone comes before the
|
||||||
|
// Seek() key in InternalKey's ordering. So Seek() will look in the
|
||||||
|
// next file for the user key.
|
||||||
|
largest_candidate = |
||||||
|
InternalKey(*upper_bound, kMaxSequenceNumber, kTypeRangeDeletion); |
||||||
|
} |
||||||
|
#ifndef NDEBUG |
||||||
|
SequenceNumber smallest_ikey_seqnum = kMaxSequenceNumber; |
||||||
|
if (meta.smallest.size() > 0) { |
||||||
|
smallest_ikey_seqnum = GetInternalKeySeqno(meta.smallest.Encode()); |
||||||
|
} |
||||||
|
#endif |
||||||
|
meta.UpdateBoundariesForRange(smallest_candidate, largest_candidate, |
||||||
|
tombstone.seq_, icmp); |
||||||
|
// The smallest key in a file is used for range tombstone truncation, so
|
||||||
|
// it cannot have a seqnum of 0 (unless the smallest data key in a file
|
||||||
|
// has a seqnum of 0). Otherwise, the truncated tombstone may expose
|
||||||
|
// deleted keys at lower levels.
|
||||||
|
assert(smallest_ikey_seqnum == 0 || |
||||||
|
ExtractInternalKeyFooter(meta.smallest.Encode()) != |
||||||
|
PackSequenceAndType(0, kTypeRangeDeletion)); |
||||||
|
} |
||||||
|
return Status::OK(); |
||||||
|
} |
||||||
|
} // namespace ROCKSDB_NAMESPACE
|
@ -0,0 +1,328 @@ |
|||||||
|
// Copyright (c) Meta Platforms, Inc. and affiliates.
|
||||||
|
//
|
||||||
|
// This source code is licensed under both the GPLv2 (found in the
|
||||||
|
// COPYING file in the root directory) and Apache 2.0 License
|
||||||
|
// (found in the LICENSE.Apache file in the root directory).
|
||||||
|
//
|
||||||
|
// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
|
||||||
|
// Use of this source code is governed by a BSD-style license that can be
|
||||||
|
// found in the LICENSE file. See the AUTHORS file for names of contributors.
|
||||||
|
|
||||||
|
#pragma once |
||||||
|
|
||||||
|
#include "db/blob/blob_garbage_meter.h" |
||||||
|
#include "db/compaction/compaction.h" |
||||||
|
#include "db/compaction/compaction_iterator.h" |
||||||
|
#include "db/internal_stats.h" |
||||||
|
#include "db/output_validator.h" |
||||||
|
|
||||||
|
namespace ROCKSDB_NAMESPACE { |
||||||
|
|
||||||
|
class CompactionOutputs; |
||||||
|
using CompactionFileOpenFunc = std::function<Status(CompactionOutputs&)>; |
||||||
|
using CompactionFileCloseFunc = |
||||||
|
std::function<Status(CompactionOutputs&, const Status&, const Slice&)>; |
||||||
|
|
||||||
|
// Files produced by subcompaction, most of the functions are used by
|
||||||
|
// compaction_job Open/Close compaction file functions.
|
||||||
|
class CompactionOutputs { |
||||||
|
public: |
||||||
|
// compaction output file
|
||||||
|
struct Output { |
||||||
|
Output(FileMetaData&& _meta, const InternalKeyComparator& _icmp, |
||||||
|
bool _enable_order_check, bool _enable_hash, bool _finished, |
||||||
|
uint64_t precalculated_hash) |
||||||
|
: meta(std::move(_meta)), |
||||||
|
validator(_icmp, _enable_order_check, _enable_hash, |
||||||
|
precalculated_hash), |
||||||
|
finished(_finished) {} |
||||||
|
FileMetaData meta; |
||||||
|
OutputValidator validator; |
||||||
|
bool finished; |
||||||
|
std::shared_ptr<const TableProperties> table_properties; |
||||||
|
}; |
||||||
|
|
||||||
|
CompactionOutputs() = delete; |
||||||
|
|
||||||
|
explicit CompactionOutputs(const Compaction* compaction, |
||||||
|
const bool is_penultimate_level) |
||||||
|
: compaction_(compaction), is_penultimate_level_(is_penultimate_level) { |
||||||
|
partitioner_ = compaction->output_level() == 0 |
||||||
|
? nullptr |
||||||
|
: compaction->CreateSstPartitioner(); |
||||||
|
} |
||||||
|
|
||||||
|
// Add generated output to the list
|
||||||
|
void AddOutput(FileMetaData&& meta, const InternalKeyComparator& icmp, |
||||||
|
bool enable_order_check, bool enable_hash, |
||||||
|
bool finished = false, uint64_t precalculated_hash = 0) { |
||||||
|
outputs_.emplace_back(std::move(meta), icmp, enable_order_check, |
||||||
|
enable_hash, finished, precalculated_hash); |
||||||
|
} |
||||||
|
|
||||||
|
// Set new table builder for the current output
|
||||||
|
void NewBuilder(const TableBuilderOptions& tboptions); |
||||||
|
|
||||||
|
// Assign a new WritableFileWriter to the current output
|
||||||
|
void AssignFileWriter(WritableFileWriter* writer) { |
||||||
|
file_writer_.reset(writer); |
||||||
|
} |
||||||
|
|
||||||
|
// TODO: Remove it when remote compaction support tiered compaction
|
||||||
|
void SetTotalBytes(uint64_t bytes) { stats_.bytes_written += bytes; } |
||||||
|
void SetNumOutputRecords(uint64_t num) { stats_.num_output_records = num; } |
||||||
|
|
||||||
|
// TODO: Move the BlobDB builder into CompactionOutputs
|
||||||
|
const std::vector<BlobFileAddition>& GetBlobFileAdditions() const { |
||||||
|
if (is_penultimate_level_) { |
||||||
|
assert(blob_file_additions_.empty()); |
||||||
|
} |
||||||
|
return blob_file_additions_; |
||||||
|
} |
||||||
|
|
||||||
|
std::vector<BlobFileAddition>* GetBlobFileAdditionsPtr() { |
||||||
|
assert(!is_penultimate_level_); |
||||||
|
return &blob_file_additions_; |
||||||
|
} |
||||||
|
|
||||||
|
bool HasBlobFileAdditions() const { return !blob_file_additions_.empty(); } |
||||||
|
|
||||||
|
BlobGarbageMeter* CreateBlobGarbageMeter() { |
||||||
|
assert(!is_penultimate_level_); |
||||||
|
blob_garbage_meter_ = std::make_unique<BlobGarbageMeter>(); |
||||||
|
return blob_garbage_meter_.get(); |
||||||
|
} |
||||||
|
|
||||||
|
BlobGarbageMeter* GetBlobGarbageMeter() const { |
||||||
|
if (is_penultimate_level_) { |
||||||
|
// blobdb doesn't support per_key_placement yet
|
||||||
|
assert(blob_garbage_meter_ == nullptr); |
||||||
|
return nullptr; |
||||||
|
} |
||||||
|
return blob_garbage_meter_.get(); |
||||||
|
} |
||||||
|
|
||||||
|
void UpdateBlobStats() { |
||||||
|
assert(!is_penultimate_level_); |
||||||
|
stats_.num_output_files_blob = blob_file_additions_.size(); |
||||||
|
for (const auto& blob : blob_file_additions_) { |
||||||
|
stats_.bytes_written_blob += blob.GetTotalBlobBytes(); |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
// Finish the current output file
|
||||||
|
Status Finish(const Status& intput_status); |
||||||
|
|
||||||
|
// Update output table properties from table builder
|
||||||
|
void UpdateTableProperties() { |
||||||
|
current_output().table_properties = |
||||||
|
std::make_shared<TableProperties>(GetTableProperties()); |
||||||
|
} |
||||||
|
|
||||||
|
IOStatus WriterSyncClose(const Status& intput_status, SystemClock* clock, |
||||||
|
Statistics* statistics, bool use_fsync); |
||||||
|
|
||||||
|
TableProperties GetTableProperties() { |
||||||
|
return builder_->GetTableProperties(); |
||||||
|
} |
||||||
|
|
||||||
|
Slice SmallestUserKey() const { |
||||||
|
if (!outputs_.empty() && outputs_[0].finished) { |
||||||
|
return outputs_[0].meta.smallest.user_key(); |
||||||
|
} else { |
||||||
|
return Slice{nullptr, 0}; |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
Slice LargestUserKey() const { |
||||||
|
if (!outputs_.empty() && outputs_.back().finished) { |
||||||
|
return outputs_.back().meta.largest.user_key(); |
||||||
|
} else { |
||||||
|
return Slice{nullptr, 0}; |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
// In case the last output file is empty, which doesn't need to keep.
|
||||||
|
void RemoveLastEmptyOutput() { |
||||||
|
if (!outputs_.empty() && !outputs_.back().meta.fd.file_size) { |
||||||
|
// An error occurred, so ignore the last output.
|
||||||
|
outputs_.pop_back(); |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
// Remove the last output, for example the last output doesn't have data (no
|
||||||
|
// entry and no range-dels), but file_size might not be 0, as it has SST
|
||||||
|
// metadata.
|
||||||
|
void RemoveLastOutput() { |
||||||
|
assert(!outputs_.empty()); |
||||||
|
outputs_.pop_back(); |
||||||
|
} |
||||||
|
|
||||||
|
bool HasBuilder() const { return builder_ != nullptr; } |
||||||
|
|
||||||
|
FileMetaData* GetMetaData() { return ¤t_output().meta; } |
||||||
|
|
||||||
|
bool HasOutput() const { return !outputs_.empty(); } |
||||||
|
|
||||||
|
uint64_t NumEntries() const { return builder_->NumEntries(); } |
||||||
|
|
||||||
|
void ResetBuilder() { |
||||||
|
builder_.reset(); |
||||||
|
current_output_file_size_ = 0; |
||||||
|
} |
||||||
|
|
||||||
|
// Add range-dels from the aggregator to the current output file
|
||||||
|
Status AddRangeDels(const Slice* comp_start, const Slice* comp_end, |
||||||
|
CompactionIterationStats& range_del_out_stats, |
||||||
|
bool bottommost_level, const InternalKeyComparator& icmp, |
||||||
|
SequenceNumber earliest_snapshot, |
||||||
|
const Slice& next_table_min_key); |
||||||
|
|
||||||
|
// Is the current file is already pending for close
|
||||||
|
bool IsPendingClose() const { return pending_close_; } |
||||||
|
|
||||||
|
// Current file should close before adding a new key
|
||||||
|
void SetPendingClose() { pending_close_ = true; } |
||||||
|
|
||||||
|
// if the outputs have range delete, range delete is also data
|
||||||
|
bool HasRangeDel() const { |
||||||
|
return range_del_agg_ && !range_del_agg_->IsEmpty(); |
||||||
|
} |
||||||
|
|
||||||
|
private: |
||||||
|
friend class SubcompactionState; |
||||||
|
|
||||||
|
void Cleanup() { |
||||||
|
if (builder_ != nullptr) { |
||||||
|
// May happen if we get a shutdown call in the middle of compaction
|
||||||
|
builder_->Abandon(); |
||||||
|
builder_.reset(); |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
uint64_t GetCurrentOutputFileSize() const { |
||||||
|
return current_output_file_size_; |
||||||
|
} |
||||||
|
|
||||||
|
// Add curent key from compaction_iterator to the output file. If needed
|
||||||
|
// close and open new compaction output with the functions provided.
|
||||||
|
Status AddToOutput(const CompactionIterator& c_iter, |
||||||
|
const CompactionFileOpenFunc& open_file_func, |
||||||
|
const CompactionFileCloseFunc& close_file_func); |
||||||
|
|
||||||
|
// Close the current output. `open_file_func` is needed for creating new file
|
||||||
|
// for range-dels only output file.
|
||||||
|
Status CloseOutput(const Status& curr_status, |
||||||
|
const CompactionFileOpenFunc& open_file_func, |
||||||
|
const CompactionFileCloseFunc& close_file_func) { |
||||||
|
Status status = curr_status; |
||||||
|
// handle subcompaction containing only range deletions
|
||||||
|
if (status.ok() && !HasBuilder() && !HasOutput() && HasRangeDel()) { |
||||||
|
status = open_file_func(*this); |
||||||
|
} |
||||||
|
if (HasBuilder()) { |
||||||
|
const Slice empty_key{}; |
||||||
|
Status s = close_file_func(*this, status, empty_key); |
||||||
|
if (!s.ok() && status.ok()) { |
||||||
|
status = s; |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
return status; |
||||||
|
} |
||||||
|
|
||||||
|
// This subcompaction's output 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
|
||||||
|
// particularly likely for the later subcompactions to be empty. Once they are
|
||||||
|
// run in parallel however it should be much rarer.
|
||||||
|
// It's caller's responsibility to make sure it's not empty.
|
||||||
|
Output& current_output() { |
||||||
|
assert(!outputs_.empty()); |
||||||
|
return outputs_.back(); |
||||||
|
} |
||||||
|
|
||||||
|
// Assign the range_del_agg to the target output level. There's only one
|
||||||
|
// range-del-aggregator per compaction outputs, for
|
||||||
|
// output_to_penultimate_level compaction it is only assigned to the
|
||||||
|
// penultimate level.
|
||||||
|
void AssignRangeDelAggregator( |
||||||
|
std::unique_ptr<CompactionRangeDelAggregator>&& range_del_agg) { |
||||||
|
assert(range_del_agg_ == nullptr); |
||||||
|
range_del_agg_ = std::move(range_del_agg); |
||||||
|
} |
||||||
|
|
||||||
|
const Compaction* compaction_; |
||||||
|
|
||||||
|
// The current file is pending close, which needs to run `close_file_func()`
|
||||||
|
// first to add a new key.
|
||||||
|
bool pending_close_ = false; |
||||||
|
|
||||||
|
// current output builder and writer
|
||||||
|
std::unique_ptr<TableBuilder> builder_; |
||||||
|
std::unique_ptr<WritableFileWriter> file_writer_; |
||||||
|
uint64_t current_output_file_size_ = 0; |
||||||
|
|
||||||
|
// all the compaction outputs so far
|
||||||
|
std::vector<Output> outputs_; |
||||||
|
|
||||||
|
// BlobDB info
|
||||||
|
std::vector<BlobFileAddition> blob_file_additions_; |
||||||
|
std::unique_ptr<BlobGarbageMeter> blob_garbage_meter_; |
||||||
|
|
||||||
|
// Basic compaction output stats for this level's outputs
|
||||||
|
InternalStats::CompactionOutputsStats stats_; |
||||||
|
|
||||||
|
// indicate if this CompactionOutputs obj for penultimate_level, should always
|
||||||
|
// be false if per_key_placement feature is not enabled.
|
||||||
|
const bool is_penultimate_level_; |
||||||
|
std::unique_ptr<CompactionRangeDelAggregator> range_del_agg_ = nullptr; |
||||||
|
|
||||||
|
// partitioner information
|
||||||
|
std::string last_key_for_partitioner_; |
||||||
|
std::unique_ptr<SstPartitioner> partitioner_; |
||||||
|
}; |
||||||
|
|
||||||
|
// helper struct to concatenate the last level and penultimate level outputs
|
||||||
|
// which could be replaced by std::ranges::join_view() in c++20
|
||||||
|
struct OutputIterator { |
||||||
|
public: |
||||||
|
explicit OutputIterator(const std::vector<CompactionOutputs::Output>& a, |
||||||
|
const std::vector<CompactionOutputs::Output>& b) |
||||||
|
: a_(a), b_(b) { |
||||||
|
within_a = !a_.empty(); |
||||||
|
idx_ = 0; |
||||||
|
} |
||||||
|
|
||||||
|
OutputIterator begin() { return *this; } |
||||||
|
|
||||||
|
OutputIterator end() { return *this; } |
||||||
|
|
||||||
|
size_t size() { return a_.size() + b_.size(); } |
||||||
|
|
||||||
|
const CompactionOutputs::Output& operator*() const { |
||||||
|
return within_a ? a_[idx_] : b_[idx_]; |
||||||
|
} |
||||||
|
|
||||||
|
OutputIterator& operator++() { |
||||||
|
idx_++; |
||||||
|
if (within_a && idx_ >= a_.size()) { |
||||||
|
within_a = false; |
||||||
|
idx_ = 0; |
||||||
|
} |
||||||
|
assert(within_a || idx_ <= b_.size()); |
||||||
|
return *this; |
||||||
|
} |
||||||
|
|
||||||
|
bool operator!=(const OutputIterator& /*rhs*/) const { |
||||||
|
return within_a || idx_ < b_.size(); |
||||||
|
} |
||||||
|
|
||||||
|
private: |
||||||
|
const std::vector<CompactionOutputs::Output>& a_; |
||||||
|
const std::vector<CompactionOutputs::Output>& b_; |
||||||
|
bool within_a; |
||||||
|
size_t idx_; |
||||||
|
}; |
||||||
|
|
||||||
|
} // namespace ROCKSDB_NAMESPACE
|
@ -0,0 +1,825 @@ |
|||||||
|
// Copyright (c) Meta Platforms, Inc. and affiliates.
|
||||||
|
//
|
||||||
|
// This source code is licensed under both the GPLv2 (found in the
|
||||||
|
// COPYING file in the root directory) and Apache 2.0 License
|
||||||
|
// (found in the LICENSE.Apache file in the root directory).
|
||||||
|
//
|
||||||
|
// 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/compaction_job.h" |
||||||
|
#include "db/compaction/compaction_state.h" |
||||||
|
#include "logging/logging.h" |
||||||
|
#include "monitoring/iostats_context_imp.h" |
||||||
|
#include "monitoring/thread_status_util.h" |
||||||
|
#include "options/options_helper.h" |
||||||
|
#include "rocksdb/utilities/options_type.h" |
||||||
|
|
||||||
|
#ifndef ROCKSDB_LITE |
||||||
|
namespace ROCKSDB_NAMESPACE { |
||||||
|
class SubcompactionState; |
||||||
|
|
||||||
|
CompactionServiceJobStatus |
||||||
|
CompactionJob::ProcessKeyValueCompactionWithCompactionService( |
||||||
|
SubcompactionState* sub_compact) { |
||||||
|
assert(sub_compact); |
||||||
|
assert(sub_compact->compaction); |
||||||
|
assert(db_options_.compaction_service); |
||||||
|
|
||||||
|
const Compaction* compaction = sub_compact->compaction; |
||||||
|
CompactionServiceInput compaction_input; |
||||||
|
compaction_input.output_level = compaction->output_level(); |
||||||
|
compaction_input.db_id = db_id_; |
||||||
|
|
||||||
|
const std::vector<CompactionInputFiles>& inputs = |
||||||
|
*(compact_->compaction->inputs()); |
||||||
|
for (const auto& files_per_level : inputs) { |
||||||
|
for (const auto& file : files_per_level.files) { |
||||||
|
compaction_input.input_files.emplace_back( |
||||||
|
MakeTableFileName(file->fd.GetNumber())); |
||||||
|
} |
||||||
|
} |
||||||
|
compaction_input.column_family.name = |
||||||
|
compaction->column_family_data()->GetName(); |
||||||
|
compaction_input.column_family.options = |
||||||
|
compaction->column_family_data()->GetLatestCFOptions(); |
||||||
|
compaction_input.db_options = |
||||||
|
BuildDBOptions(db_options_, mutable_db_options_copy_); |
||||||
|
compaction_input.snapshots = existing_snapshots_; |
||||||
|
compaction_input.has_begin = sub_compact->start; |
||||||
|
compaction_input.begin = |
||||||
|
compaction_input.has_begin ? sub_compact->start->ToString() : ""; |
||||||
|
compaction_input.has_end = sub_compact->end; |
||||||
|
compaction_input.end = |
||||||
|
compaction_input.has_end ? sub_compact->end->ToString() : ""; |
||||||
|
|
||||||
|
std::string compaction_input_binary; |
||||||
|
Status s = compaction_input.Write(&compaction_input_binary); |
||||||
|
if (!s.ok()) { |
||||||
|
sub_compact->status = s; |
||||||
|
return CompactionServiceJobStatus::kFailure; |
||||||
|
} |
||||||
|
|
||||||
|
std::ostringstream input_files_oss; |
||||||
|
bool is_first_one = true; |
||||||
|
for (const auto& file : compaction_input.input_files) { |
||||||
|
input_files_oss << (is_first_one ? "" : ", ") << file; |
||||||
|
is_first_one = false; |
||||||
|
} |
||||||
|
|
||||||
|
ROCKS_LOG_INFO( |
||||||
|
db_options_.info_log, |
||||||
|
"[%s] [JOB %d] Starting remote compaction (output level: %d): %s", |
||||||
|
compaction_input.column_family.name.c_str(), job_id_, |
||||||
|
compaction_input.output_level, input_files_oss.str().c_str()); |
||||||
|
CompactionServiceJobInfo info(dbname_, db_id_, db_session_id_, |
||||||
|
GetCompactionId(sub_compact), thread_pri_); |
||||||
|
CompactionServiceJobStatus compaction_status = |
||||||
|
db_options_.compaction_service->StartV2(info, compaction_input_binary); |
||||||
|
switch (compaction_status) { |
||||||
|
case CompactionServiceJobStatus::kSuccess: |
||||||
|
break; |
||||||
|
case CompactionServiceJobStatus::kFailure: |
||||||
|
sub_compact->status = Status::Incomplete( |
||||||
|
"CompactionService failed to start compaction job."); |
||||||
|
ROCKS_LOG_WARN(db_options_.info_log, |
||||||
|
"[%s] [JOB %d] Remote compaction failed to start.", |
||||||
|
compaction_input.column_family.name.c_str(), job_id_); |
||||||
|
return compaction_status; |
||||||
|
case CompactionServiceJobStatus::kUseLocal: |
||||||
|
ROCKS_LOG_INFO( |
||||||
|
db_options_.info_log, |
||||||
|
"[%s] [JOB %d] Remote compaction fallback to local by API Start.", |
||||||
|
compaction_input.column_family.name.c_str(), job_id_); |
||||||
|
return compaction_status; |
||||||
|
default: |
||||||
|
assert(false); // unknown status
|
||||||
|
break; |
||||||
|
} |
||||||
|
|
||||||
|
ROCKS_LOG_INFO(db_options_.info_log, |
||||||
|
"[%s] [JOB %d] Waiting for remote compaction...", |
||||||
|
compaction_input.column_family.name.c_str(), job_id_); |
||||||
|
std::string compaction_result_binary; |
||||||
|
compaction_status = db_options_.compaction_service->WaitForCompleteV2( |
||||||
|
info, &compaction_result_binary); |
||||||
|
|
||||||
|
if (compaction_status == CompactionServiceJobStatus::kUseLocal) { |
||||||
|
ROCKS_LOG_INFO(db_options_.info_log, |
||||||
|
"[%s] [JOB %d] Remote compaction fallback to local by API " |
||||||
|
"WaitForComplete.", |
||||||
|
compaction_input.column_family.name.c_str(), job_id_); |
||||||
|
return compaction_status; |
||||||
|
} |
||||||
|
|
||||||
|
CompactionServiceResult compaction_result; |
||||||
|
s = CompactionServiceResult::Read(compaction_result_binary, |
||||||
|
&compaction_result); |
||||||
|
|
||||||
|
if (compaction_status == CompactionServiceJobStatus::kFailure) { |
||||||
|
if (s.ok()) { |
||||||
|
if (compaction_result.status.ok()) { |
||||||
|
sub_compact->status = Status::Incomplete( |
||||||
|
"CompactionService failed to run the compaction job (even though " |
||||||
|
"the internal status is okay)."); |
||||||
|
} else { |
||||||
|
// set the current sub compaction status with the status returned from
|
||||||
|
// remote
|
||||||
|
sub_compact->status = compaction_result.status; |
||||||
|
} |
||||||
|
} else { |
||||||
|
sub_compact->status = Status::Incomplete( |
||||||
|
"CompactionService failed to run the compaction job (and no valid " |
||||||
|
"result is returned)."); |
||||||
|
compaction_result.status.PermitUncheckedError(); |
||||||
|
} |
||||||
|
ROCKS_LOG_WARN(db_options_.info_log, |
||||||
|
"[%s] [JOB %d] Remote compaction failed.", |
||||||
|
compaction_input.column_family.name.c_str(), job_id_); |
||||||
|
return compaction_status; |
||||||
|
} |
||||||
|
|
||||||
|
if (!s.ok()) { |
||||||
|
sub_compact->status = s; |
||||||
|
compaction_result.status.PermitUncheckedError(); |
||||||
|
return CompactionServiceJobStatus::kFailure; |
||||||
|
} |
||||||
|
sub_compact->status = compaction_result.status; |
||||||
|
|
||||||
|
std::ostringstream output_files_oss; |
||||||
|
is_first_one = true; |
||||||
|
for (const auto& file : compaction_result.output_files) { |
||||||
|
output_files_oss << (is_first_one ? "" : ", ") << file.file_name; |
||||||
|
is_first_one = false; |
||||||
|
} |
||||||
|
|
||||||
|
ROCKS_LOG_INFO(db_options_.info_log, |
||||||
|
"[%s] [JOB %d] Receive remote compaction result, output path: " |
||||||
|
"%s, files: %s", |
||||||
|
compaction_input.column_family.name.c_str(), job_id_, |
||||||
|
compaction_result.output_path.c_str(), |
||||||
|
output_files_oss.str().c_str()); |
||||||
|
|
||||||
|
if (!s.ok()) { |
||||||
|
sub_compact->status = s; |
||||||
|
return CompactionServiceJobStatus::kFailure; |
||||||
|
} |
||||||
|
|
||||||
|
for (const auto& file : compaction_result.output_files) { |
||||||
|
uint64_t file_num = versions_->NewFileNumber(); |
||||||
|
auto src_file = compaction_result.output_path + "/" + file.file_name; |
||||||
|
auto tgt_file = TableFileName(compaction->immutable_options()->cf_paths, |
||||||
|
file_num, compaction->output_path_id()); |
||||||
|
s = fs_->RenameFile(src_file, tgt_file, IOOptions(), nullptr); |
||||||
|
if (!s.ok()) { |
||||||
|
sub_compact->status = s; |
||||||
|
return CompactionServiceJobStatus::kFailure; |
||||||
|
} |
||||||
|
|
||||||
|
FileMetaData meta; |
||||||
|
uint64_t file_size; |
||||||
|
s = fs_->GetFileSize(tgt_file, IOOptions(), &file_size, nullptr); |
||||||
|
if (!s.ok()) { |
||||||
|
sub_compact->status = s; |
||||||
|
return CompactionServiceJobStatus::kFailure; |
||||||
|
} |
||||||
|
meta.fd = FileDescriptor(file_num, compaction->output_path_id(), file_size, |
||||||
|
file.smallest_seqno, file.largest_seqno); |
||||||
|
meta.smallest.DecodeFrom(file.smallest_internal_key); |
||||||
|
meta.largest.DecodeFrom(file.largest_internal_key); |
||||||
|
meta.oldest_ancester_time = file.oldest_ancester_time; |
||||||
|
meta.file_creation_time = file.file_creation_time; |
||||||
|
meta.marked_for_compaction = file.marked_for_compaction; |
||||||
|
meta.unique_id = file.unique_id; |
||||||
|
|
||||||
|
auto cfd = compaction->column_family_data(); |
||||||
|
sub_compact->Current().AddOutput(std::move(meta), |
||||||
|
cfd->internal_comparator(), false, false, |
||||||
|
true, file.paranoid_hash); |
||||||
|
} |
||||||
|
sub_compact->compaction_job_stats = compaction_result.stats; |
||||||
|
sub_compact->Current().SetNumOutputRecords( |
||||||
|
compaction_result.num_output_records); |
||||||
|
sub_compact->Current().SetTotalBytes(compaction_result.total_bytes); |
||||||
|
RecordTick(stats_, REMOTE_COMPACT_READ_BYTES, compaction_result.bytes_read); |
||||||
|
RecordTick(stats_, REMOTE_COMPACT_WRITE_BYTES, |
||||||
|
compaction_result.bytes_written); |
||||||
|
return CompactionServiceJobStatus::kSuccess; |
||||||
|
} |
||||||
|
|
||||||
|
std::string CompactionServiceCompactionJob::GetTableFileName( |
||||||
|
uint64_t file_number) { |
||||||
|
return MakeTableFileName(output_path_, file_number); |
||||||
|
} |
||||||
|
|
||||||
|
void CompactionServiceCompactionJob::RecordCompactionIOStats() { |
||||||
|
compaction_result_->bytes_read += IOSTATS(bytes_read); |
||||||
|
compaction_result_->bytes_written += IOSTATS(bytes_written); |
||||||
|
CompactionJob::RecordCompactionIOStats(); |
||||||
|
} |
||||||
|
|
||||||
|
CompactionServiceCompactionJob::CompactionServiceCompactionJob( |
||||||
|
int job_id, Compaction* compaction, const ImmutableDBOptions& db_options, |
||||||
|
const MutableDBOptions& mutable_db_options, const FileOptions& file_options, |
||||||
|
VersionSet* versions, const std::atomic<bool>* shutting_down, |
||||||
|
LogBuffer* log_buffer, FSDirectory* output_directory, Statistics* stats, |
||||||
|
InstrumentedMutex* db_mutex, ErrorHandler* db_error_handler, |
||||||
|
std::vector<SequenceNumber> existing_snapshots, |
||||||
|
std::shared_ptr<Cache> table_cache, EventLogger* event_logger, |
||||||
|
const std::string& dbname, const std::shared_ptr<IOTracer>& io_tracer, |
||||||
|
const std::atomic<bool>& manual_compaction_canceled, |
||||||
|
const std::string& db_id, const std::string& db_session_id, |
||||||
|
std::string output_path, |
||||||
|
const CompactionServiceInput& compaction_service_input, |
||||||
|
CompactionServiceResult* compaction_service_result) |
||||||
|
: CompactionJob( |
||||||
|
job_id, compaction, db_options, mutable_db_options, file_options, |
||||||
|
versions, shutting_down, log_buffer, nullptr, output_directory, |
||||||
|
nullptr, stats, db_mutex, db_error_handler, |
||||||
|
std::move(existing_snapshots), kMaxSequenceNumber, nullptr, nullptr, |
||||||
|
std::move(table_cache), event_logger, |
||||||
|
compaction->mutable_cf_options()->paranoid_file_checks, |
||||||
|
compaction->mutable_cf_options()->report_bg_io_stats, dbname, |
||||||
|
&(compaction_service_result->stats), Env::Priority::USER, io_tracer, |
||||||
|
manual_compaction_canceled, db_id, db_session_id, |
||||||
|
compaction->column_family_data()->GetFullHistoryTsLow()), |
||||||
|
output_path_(std::move(output_path)), |
||||||
|
compaction_input_(compaction_service_input), |
||||||
|
compaction_result_(compaction_service_result) {} |
||||||
|
|
||||||
|
Status CompactionServiceCompactionJob::Run() { |
||||||
|
AutoThreadOperationStageUpdater stage_updater( |
||||||
|
ThreadStatus::STAGE_COMPACTION_RUN); |
||||||
|
|
||||||
|
auto* c = compact_->compaction; |
||||||
|
assert(c->column_family_data() != nullptr); |
||||||
|
assert(c->column_family_data()->current()->storage_info()->NumLevelFiles( |
||||||
|
compact_->compaction->level()) > 0); |
||||||
|
|
||||||
|
write_hint_ = |
||||||
|
c->column_family_data()->CalculateSSTWriteHint(c->output_level()); |
||||||
|
bottommost_level_ = c->bottommost_level(); |
||||||
|
|
||||||
|
Slice begin = compaction_input_.begin; |
||||||
|
Slice end = compaction_input_.end; |
||||||
|
compact_->sub_compact_states.emplace_back( |
||||||
|
c, compaction_input_.has_begin ? &begin : nullptr, |
||||||
|
compaction_input_.has_end ? &end : nullptr, /*sub_job_id*/ 0); |
||||||
|
|
||||||
|
log_buffer_->FlushBufferToLog(); |
||||||
|
LogCompaction(); |
||||||
|
const uint64_t start_micros = db_options_.clock->NowMicros(); |
||||||
|
// Pick the only sub-compaction we should have
|
||||||
|
assert(compact_->sub_compact_states.size() == 1); |
||||||
|
SubcompactionState* sub_compact = compact_->sub_compact_states.data(); |
||||||
|
|
||||||
|
ProcessKeyValueCompaction(sub_compact); |
||||||
|
|
||||||
|
compaction_stats_.stats.micros = |
||||||
|
db_options_.clock->NowMicros() - start_micros; |
||||||
|
compaction_stats_.stats.cpu_micros = |
||||||
|
sub_compact->compaction_job_stats.cpu_micros; |
||||||
|
|
||||||
|
RecordTimeToHistogram(stats_, COMPACTION_TIME, |
||||||
|
compaction_stats_.stats.micros); |
||||||
|
RecordTimeToHistogram(stats_, COMPACTION_CPU_TIME, |
||||||
|
compaction_stats_.stats.cpu_micros); |
||||||
|
|
||||||
|
Status status = sub_compact->status; |
||||||
|
IOStatus io_s = sub_compact->io_status; |
||||||
|
|
||||||
|
if (io_status_.ok()) { |
||||||
|
io_status_ = io_s; |
||||||
|
} |
||||||
|
|
||||||
|
if (status.ok()) { |
||||||
|
constexpr IODebugContext* dbg = nullptr; |
||||||
|
|
||||||
|
if (output_directory_) { |
||||||
|
io_s = output_directory_->FsyncWithDirOptions(IOOptions(), dbg, |
||||||
|
DirFsyncOptions()); |
||||||
|
} |
||||||
|
} |
||||||
|
if (io_status_.ok()) { |
||||||
|
io_status_ = io_s; |
||||||
|
} |
||||||
|
if (status.ok()) { |
||||||
|
status = io_s; |
||||||
|
} |
||||||
|
if (status.ok()) { |
||||||
|
// TODO: Add verify_table()
|
||||||
|
} |
||||||
|
|
||||||
|
// Finish up all book-keeping to unify the subcompaction results
|
||||||
|
compact_->AggregateCompactionStats(compaction_stats_, *compaction_job_stats_); |
||||||
|
UpdateCompactionStats(); |
||||||
|
RecordCompactionIOStats(); |
||||||
|
|
||||||
|
LogFlush(db_options_.info_log); |
||||||
|
compact_->status = status; |
||||||
|
compact_->status.PermitUncheckedError(); |
||||||
|
|
||||||
|
// Build compaction result
|
||||||
|
compaction_result_->output_level = compact_->compaction->output_level(); |
||||||
|
compaction_result_->output_path = output_path_; |
||||||
|
for (const auto& output_file : sub_compact->GetOutputs()) { |
||||||
|
auto& meta = output_file.meta; |
||||||
|
compaction_result_->output_files.emplace_back( |
||||||
|
MakeTableFileName(meta.fd.GetNumber()), meta.fd.smallest_seqno, |
||||||
|
meta.fd.largest_seqno, meta.smallest.Encode().ToString(), |
||||||
|
meta.largest.Encode().ToString(), meta.oldest_ancester_time, |
||||||
|
meta.file_creation_time, output_file.validator.GetHash(), |
||||||
|
meta.marked_for_compaction, meta.unique_id); |
||||||
|
} |
||||||
|
InternalStats::CompactionStatsFull compaction_stats; |
||||||
|
sub_compact->AggregateCompactionStats(compaction_stats); |
||||||
|
compaction_result_->num_output_records = |
||||||
|
compaction_stats.stats.num_output_records; |
||||||
|
compaction_result_->total_bytes = compaction_stats.TotalBytesWritten(); |
||||||
|
|
||||||
|
return status; |
||||||
|
} |
||||||
|
|
||||||
|
void CompactionServiceCompactionJob::CleanupCompaction() { |
||||||
|
CompactionJob::CleanupCompaction(); |
||||||
|
} |
||||||
|
|
||||||
|
// Internal binary format for the input and result data
|
||||||
|
enum BinaryFormatVersion : uint32_t { |
||||||
|
kOptionsString = 1, // Use string format similar to Option string format
|
||||||
|
}; |
||||||
|
|
||||||
|
static std::unordered_map<std::string, OptionTypeInfo> cfd_type_info = { |
||||||
|
{"name", |
||||||
|
{offsetof(struct ColumnFamilyDescriptor, name), OptionType::kEncodedString, |
||||||
|
OptionVerificationType::kNormal, OptionTypeFlags::kNone}}, |
||||||
|
{"options", |
||||||
|
{offsetof(struct ColumnFamilyDescriptor, options), |
||||||
|
OptionType::kConfigurable, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone, |
||||||
|
[](const ConfigOptions& opts, const std::string& /*name*/, |
||||||
|
const std::string& value, void* addr) { |
||||||
|
auto cf_options = static_cast<ColumnFamilyOptions*>(addr); |
||||||
|
return GetColumnFamilyOptionsFromString(opts, ColumnFamilyOptions(), |
||||||
|
value, cf_options); |
||||||
|
}, |
||||||
|
[](const ConfigOptions& opts, const std::string& /*name*/, |
||||||
|
const void* addr, std::string* value) { |
||||||
|
const auto cf_options = static_cast<const ColumnFamilyOptions*>(addr); |
||||||
|
std::string result; |
||||||
|
auto status = |
||||||
|
GetStringFromColumnFamilyOptions(opts, *cf_options, &result); |
||||||
|
*value = "{" + result + "}"; |
||||||
|
return status; |
||||||
|
}, |
||||||
|
[](const ConfigOptions& opts, const std::string& name, const void* addr1, |
||||||
|
const void* addr2, std::string* mismatch) { |
||||||
|
const auto this_one = static_cast<const ColumnFamilyOptions*>(addr1); |
||||||
|
const auto that_one = static_cast<const ColumnFamilyOptions*>(addr2); |
||||||
|
auto this_conf = CFOptionsAsConfigurable(*this_one); |
||||||
|
auto that_conf = CFOptionsAsConfigurable(*that_one); |
||||||
|
std::string mismatch_opt; |
||||||
|
bool result = |
||||||
|
this_conf->AreEquivalent(opts, that_conf.get(), &mismatch_opt); |
||||||
|
if (!result) { |
||||||
|
*mismatch = name + "." + mismatch_opt; |
||||||
|
} |
||||||
|
return result; |
||||||
|
}}}, |
||||||
|
}; |
||||||
|
|
||||||
|
static std::unordered_map<std::string, OptionTypeInfo> cs_input_type_info = { |
||||||
|
{"column_family", |
||||||
|
OptionTypeInfo::Struct( |
||||||
|
"column_family", &cfd_type_info, |
||||||
|
offsetof(struct CompactionServiceInput, column_family), |
||||||
|
OptionVerificationType::kNormal, OptionTypeFlags::kNone)}, |
||||||
|
{"db_options", |
||||||
|
{offsetof(struct CompactionServiceInput, db_options), |
||||||
|
OptionType::kConfigurable, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone, |
||||||
|
[](const ConfigOptions& opts, const std::string& /*name*/, |
||||||
|
const std::string& value, void* addr) { |
||||||
|
auto options = static_cast<DBOptions*>(addr); |
||||||
|
return GetDBOptionsFromString(opts, DBOptions(), value, options); |
||||||
|
}, |
||||||
|
[](const ConfigOptions& opts, const std::string& /*name*/, |
||||||
|
const void* addr, std::string* value) { |
||||||
|
const auto options = static_cast<const DBOptions*>(addr); |
||||||
|
std::string result; |
||||||
|
auto status = GetStringFromDBOptions(opts, *options, &result); |
||||||
|
*value = "{" + result + "}"; |
||||||
|
return status; |
||||||
|
}, |
||||||
|
[](const ConfigOptions& opts, const std::string& name, const void* addr1, |
||||||
|
const void* addr2, std::string* mismatch) { |
||||||
|
const auto this_one = static_cast<const DBOptions*>(addr1); |
||||||
|
const auto that_one = static_cast<const DBOptions*>(addr2); |
||||||
|
auto this_conf = DBOptionsAsConfigurable(*this_one); |
||||||
|
auto that_conf = DBOptionsAsConfigurable(*that_one); |
||||||
|
std::string mismatch_opt; |
||||||
|
bool result = |
||||||
|
this_conf->AreEquivalent(opts, that_conf.get(), &mismatch_opt); |
||||||
|
if (!result) { |
||||||
|
*mismatch = name + "." + mismatch_opt; |
||||||
|
} |
||||||
|
return result; |
||||||
|
}}}, |
||||||
|
{"snapshots", OptionTypeInfo::Vector<uint64_t>( |
||||||
|
offsetof(struct CompactionServiceInput, snapshots), |
||||||
|
OptionVerificationType::kNormal, OptionTypeFlags::kNone, |
||||||
|
{0, OptionType::kUInt64T})}, |
||||||
|
{"input_files", OptionTypeInfo::Vector<std::string>( |
||||||
|
offsetof(struct CompactionServiceInput, input_files), |
||||||
|
OptionVerificationType::kNormal, OptionTypeFlags::kNone, |
||||||
|
{0, OptionType::kEncodedString})}, |
||||||
|
{"output_level", |
||||||
|
{offsetof(struct CompactionServiceInput, output_level), OptionType::kInt, |
||||||
|
OptionVerificationType::kNormal, OptionTypeFlags::kNone}}, |
||||||
|
{"db_id", |
||||||
|
{offsetof(struct CompactionServiceInput, db_id), |
||||||
|
OptionType::kEncodedString}}, |
||||||
|
{"has_begin", |
||||||
|
{offsetof(struct CompactionServiceInput, has_begin), OptionType::kBoolean, |
||||||
|
OptionVerificationType::kNormal, OptionTypeFlags::kNone}}, |
||||||
|
{"begin", |
||||||
|
{offsetof(struct CompactionServiceInput, begin), |
||||||
|
OptionType::kEncodedString, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
{"has_end", |
||||||
|
{offsetof(struct CompactionServiceInput, has_end), OptionType::kBoolean, |
||||||
|
OptionVerificationType::kNormal, OptionTypeFlags::kNone}}, |
||||||
|
{"end", |
||||||
|
{offsetof(struct CompactionServiceInput, end), OptionType::kEncodedString, |
||||||
|
OptionVerificationType::kNormal, OptionTypeFlags::kNone}}, |
||||||
|
}; |
||||||
|
|
||||||
|
static std::unordered_map<std::string, OptionTypeInfo> |
||||||
|
cs_output_file_type_info = { |
||||||
|
{"file_name", |
||||||
|
{offsetof(struct CompactionServiceOutputFile, file_name), |
||||||
|
OptionType::kEncodedString, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
{"smallest_seqno", |
||||||
|
{offsetof(struct CompactionServiceOutputFile, smallest_seqno), |
||||||
|
OptionType::kUInt64T, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
{"largest_seqno", |
||||||
|
{offsetof(struct CompactionServiceOutputFile, largest_seqno), |
||||||
|
OptionType::kUInt64T, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
{"smallest_internal_key", |
||||||
|
{offsetof(struct CompactionServiceOutputFile, smallest_internal_key), |
||||||
|
OptionType::kEncodedString, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
{"largest_internal_key", |
||||||
|
{offsetof(struct CompactionServiceOutputFile, largest_internal_key), |
||||||
|
OptionType::kEncodedString, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
{"oldest_ancester_time", |
||||||
|
{offsetof(struct CompactionServiceOutputFile, oldest_ancester_time), |
||||||
|
OptionType::kUInt64T, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
{"file_creation_time", |
||||||
|
{offsetof(struct CompactionServiceOutputFile, file_creation_time), |
||||||
|
OptionType::kUInt64T, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
{"paranoid_hash", |
||||||
|
{offsetof(struct CompactionServiceOutputFile, paranoid_hash), |
||||||
|
OptionType::kUInt64T, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
{"marked_for_compaction", |
||||||
|
{offsetof(struct CompactionServiceOutputFile, marked_for_compaction), |
||||||
|
OptionType::kBoolean, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
{"unique_id", |
||||||
|
OptionTypeInfo::Array<uint64_t, 2>( |
||||||
|
offsetof(struct CompactionServiceOutputFile, unique_id), |
||||||
|
OptionVerificationType::kNormal, OptionTypeFlags::kNone, |
||||||
|
{0, OptionType::kUInt64T})}, |
||||||
|
}; |
||||||
|
|
||||||
|
static std::unordered_map<std::string, OptionTypeInfo> |
||||||
|
compaction_job_stats_type_info = { |
||||||
|
{"elapsed_micros", |
||||||
|
{offsetof(struct CompactionJobStats, elapsed_micros), |
||||||
|
OptionType::kUInt64T, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
{"cpu_micros", |
||||||
|
{offsetof(struct CompactionJobStats, cpu_micros), OptionType::kUInt64T, |
||||||
|
OptionVerificationType::kNormal, OptionTypeFlags::kNone}}, |
||||||
|
{"num_input_records", |
||||||
|
{offsetof(struct CompactionJobStats, num_input_records), |
||||||
|
OptionType::kUInt64T, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
{"num_blobs_read", |
||||||
|
{offsetof(struct CompactionJobStats, num_blobs_read), |
||||||
|
OptionType::kUInt64T, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
{"num_input_files", |
||||||
|
{offsetof(struct CompactionJobStats, num_input_files), |
||||||
|
OptionType::kSizeT, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
{"num_input_files_at_output_level", |
||||||
|
{offsetof(struct CompactionJobStats, num_input_files_at_output_level), |
||||||
|
OptionType::kSizeT, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
{"num_output_records", |
||||||
|
{offsetof(struct CompactionJobStats, num_output_records), |
||||||
|
OptionType::kUInt64T, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
{"num_output_files", |
||||||
|
{offsetof(struct CompactionJobStats, num_output_files), |
||||||
|
OptionType::kSizeT, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
{"num_output_files_blob", |
||||||
|
{offsetof(struct CompactionJobStats, num_output_files_blob), |
||||||
|
OptionType::kSizeT, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
{"is_full_compaction", |
||||||
|
{offsetof(struct CompactionJobStats, is_full_compaction), |
||||||
|
OptionType::kBoolean, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
{"is_manual_compaction", |
||||||
|
{offsetof(struct CompactionJobStats, is_manual_compaction), |
||||||
|
OptionType::kBoolean, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
{"total_input_bytes", |
||||||
|
{offsetof(struct CompactionJobStats, total_input_bytes), |
||||||
|
OptionType::kUInt64T, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
{"total_blob_bytes_read", |
||||||
|
{offsetof(struct CompactionJobStats, total_blob_bytes_read), |
||||||
|
OptionType::kUInt64T, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
{"total_output_bytes", |
||||||
|
{offsetof(struct CompactionJobStats, total_output_bytes), |
||||||
|
OptionType::kUInt64T, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
{"total_output_bytes_blob", |
||||||
|
{offsetof(struct CompactionJobStats, total_output_bytes_blob), |
||||||
|
OptionType::kUInt64T, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
{"num_records_replaced", |
||||||
|
{offsetof(struct CompactionJobStats, num_records_replaced), |
||||||
|
OptionType::kUInt64T, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
{"total_input_raw_key_bytes", |
||||||
|
{offsetof(struct CompactionJobStats, total_input_raw_key_bytes), |
||||||
|
OptionType::kUInt64T, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
{"total_input_raw_value_bytes", |
||||||
|
{offsetof(struct CompactionJobStats, total_input_raw_value_bytes), |
||||||
|
OptionType::kUInt64T, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
{"num_input_deletion_records", |
||||||
|
{offsetof(struct CompactionJobStats, num_input_deletion_records), |
||||||
|
OptionType::kUInt64T, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
{"num_expired_deletion_records", |
||||||
|
{offsetof(struct CompactionJobStats, num_expired_deletion_records), |
||||||
|
OptionType::kUInt64T, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
{"num_corrupt_keys", |
||||||
|
{offsetof(struct CompactionJobStats, num_corrupt_keys), |
||||||
|
OptionType::kUInt64T, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
{"file_write_nanos", |
||||||
|
{offsetof(struct CompactionJobStats, file_write_nanos), |
||||||
|
OptionType::kUInt64T, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
{"file_range_sync_nanos", |
||||||
|
{offsetof(struct CompactionJobStats, file_range_sync_nanos), |
||||||
|
OptionType::kUInt64T, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
{"file_fsync_nanos", |
||||||
|
{offsetof(struct CompactionJobStats, file_fsync_nanos), |
||||||
|
OptionType::kUInt64T, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
{"file_prepare_write_nanos", |
||||||
|
{offsetof(struct CompactionJobStats, file_prepare_write_nanos), |
||||||
|
OptionType::kUInt64T, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
{"smallest_output_key_prefix", |
||||||
|
{offsetof(struct CompactionJobStats, smallest_output_key_prefix), |
||||||
|
OptionType::kEncodedString, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
{"largest_output_key_prefix", |
||||||
|
{offsetof(struct CompactionJobStats, largest_output_key_prefix), |
||||||
|
OptionType::kEncodedString, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
{"num_single_del_fallthru", |
||||||
|
{offsetof(struct CompactionJobStats, num_single_del_fallthru), |
||||||
|
OptionType::kUInt64T, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
{"num_single_del_mismatch", |
||||||
|
{offsetof(struct CompactionJobStats, num_single_del_mismatch), |
||||||
|
OptionType::kUInt64T, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
}; |
||||||
|
|
||||||
|
namespace { |
||||||
|
// this is a helper struct to serialize and deserialize class Status, because
|
||||||
|
// Status's members are not public.
|
||||||
|
struct StatusSerializationAdapter { |
||||||
|
uint8_t code; |
||||||
|
uint8_t subcode; |
||||||
|
uint8_t severity; |
||||||
|
std::string message; |
||||||
|
|
||||||
|
StatusSerializationAdapter() = default; |
||||||
|
explicit StatusSerializationAdapter(const Status& s) { |
||||||
|
code = s.code(); |
||||||
|
subcode = s.subcode(); |
||||||
|
severity = s.severity(); |
||||||
|
auto msg = s.getState(); |
||||||
|
message = msg ? msg : ""; |
||||||
|
} |
||||||
|
|
||||||
|
Status GetStatus() const { |
||||||
|
return Status{static_cast<Status::Code>(code), |
||||||
|
static_cast<Status::SubCode>(subcode), |
||||||
|
static_cast<Status::Severity>(severity), message}; |
||||||
|
} |
||||||
|
}; |
||||||
|
} // namespace
|
||||||
|
|
||||||
|
static std::unordered_map<std::string, OptionTypeInfo> |
||||||
|
status_adapter_type_info = { |
||||||
|
{"code", |
||||||
|
{offsetof(struct StatusSerializationAdapter, code), |
||||||
|
OptionType::kUInt8T, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
{"subcode", |
||||||
|
{offsetof(struct StatusSerializationAdapter, subcode), |
||||||
|
OptionType::kUInt8T, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
{"severity", |
||||||
|
{offsetof(struct StatusSerializationAdapter, severity), |
||||||
|
OptionType::kUInt8T, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
{"message", |
||||||
|
{offsetof(struct StatusSerializationAdapter, message), |
||||||
|
OptionType::kEncodedString, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
}; |
||||||
|
|
||||||
|
static std::unordered_map<std::string, OptionTypeInfo> cs_result_type_info = { |
||||||
|
{"status", |
||||||
|
{offsetof(struct CompactionServiceResult, status), |
||||||
|
OptionType::kCustomizable, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone, |
||||||
|
[](const ConfigOptions& opts, const std::string& /*name*/, |
||||||
|
const std::string& value, void* addr) { |
||||||
|
auto status_obj = static_cast<Status*>(addr); |
||||||
|
StatusSerializationAdapter adapter; |
||||||
|
Status s = OptionTypeInfo::ParseType( |
||||||
|
opts, value, status_adapter_type_info, &adapter); |
||||||
|
*status_obj = adapter.GetStatus(); |
||||||
|
return s; |
||||||
|
}, |
||||||
|
[](const ConfigOptions& opts, const std::string& /*name*/, |
||||||
|
const void* addr, std::string* value) { |
||||||
|
const auto status_obj = static_cast<const Status*>(addr); |
||||||
|
StatusSerializationAdapter adapter(*status_obj); |
||||||
|
std::string result; |
||||||
|
Status s = OptionTypeInfo::SerializeType(opts, status_adapter_type_info, |
||||||
|
&adapter, &result); |
||||||
|
*value = "{" + result + "}"; |
||||||
|
return s; |
||||||
|
}, |
||||||
|
[](const ConfigOptions& opts, const std::string& /*name*/, |
||||||
|
const void* addr1, const void* addr2, std::string* mismatch) { |
||||||
|
const auto status1 = static_cast<const Status*>(addr1); |
||||||
|
const auto status2 = static_cast<const Status*>(addr2); |
||||||
|
|
||||||
|
StatusSerializationAdapter adatper1(*status1); |
||||||
|
StatusSerializationAdapter adapter2(*status2); |
||||||
|
return OptionTypeInfo::TypesAreEqual(opts, status_adapter_type_info, |
||||||
|
&adatper1, &adapter2, mismatch); |
||||||
|
}}}, |
||||||
|
{"output_files", |
||||||
|
OptionTypeInfo::Vector<CompactionServiceOutputFile>( |
||||||
|
offsetof(struct CompactionServiceResult, output_files), |
||||||
|
OptionVerificationType::kNormal, OptionTypeFlags::kNone, |
||||||
|
OptionTypeInfo::Struct("output_files", &cs_output_file_type_info, 0, |
||||||
|
OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone))}, |
||||||
|
{"output_level", |
||||||
|
{offsetof(struct CompactionServiceResult, output_level), OptionType::kInt, |
||||||
|
OptionVerificationType::kNormal, OptionTypeFlags::kNone}}, |
||||||
|
{"output_path", |
||||||
|
{offsetof(struct CompactionServiceResult, output_path), |
||||||
|
OptionType::kEncodedString, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
{"num_output_records", |
||||||
|
{offsetof(struct CompactionServiceResult, num_output_records), |
||||||
|
OptionType::kUInt64T, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
{"total_bytes", |
||||||
|
{offsetof(struct CompactionServiceResult, total_bytes), |
||||||
|
OptionType::kUInt64T, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
{"bytes_read", |
||||||
|
{offsetof(struct CompactionServiceResult, bytes_read), |
||||||
|
OptionType::kUInt64T, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
{"bytes_written", |
||||||
|
{offsetof(struct CompactionServiceResult, bytes_written), |
||||||
|
OptionType::kUInt64T, OptionVerificationType::kNormal, |
||||||
|
OptionTypeFlags::kNone}}, |
||||||
|
{"stats", OptionTypeInfo::Struct( |
||||||
|
"stats", &compaction_job_stats_type_info, |
||||||
|
offsetof(struct CompactionServiceResult, stats), |
||||||
|
OptionVerificationType::kNormal, OptionTypeFlags::kNone)}, |
||||||
|
}; |
||||||
|
|
||||||
|
Status CompactionServiceInput::Read(const std::string& data_str, |
||||||
|
CompactionServiceInput* obj) { |
||||||
|
if (data_str.size() <= sizeof(BinaryFormatVersion)) { |
||||||
|
return Status::InvalidArgument("Invalid CompactionServiceInput string"); |
||||||
|
} |
||||||
|
auto format_version = DecodeFixed32(data_str.data()); |
||||||
|
if (format_version == kOptionsString) { |
||||||
|
ConfigOptions cf; |
||||||
|
cf.invoke_prepare_options = false; |
||||||
|
cf.ignore_unknown_options = true; |
||||||
|
return OptionTypeInfo::ParseType( |
||||||
|
cf, data_str.substr(sizeof(BinaryFormatVersion)), cs_input_type_info, |
||||||
|
obj); |
||||||
|
} else { |
||||||
|
return Status::NotSupported( |
||||||
|
"Compaction Service Input data version not supported: " + |
||||||
|
std::to_string(format_version)); |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
Status CompactionServiceInput::Write(std::string* output) { |
||||||
|
char buf[sizeof(BinaryFormatVersion)]; |
||||||
|
EncodeFixed32(buf, kOptionsString); |
||||||
|
output->append(buf, sizeof(BinaryFormatVersion)); |
||||||
|
ConfigOptions cf; |
||||||
|
cf.invoke_prepare_options = false; |
||||||
|
return OptionTypeInfo::SerializeType(cf, cs_input_type_info, this, output); |
||||||
|
} |
||||||
|
|
||||||
|
Status CompactionServiceResult::Read(const std::string& data_str, |
||||||
|
CompactionServiceResult* obj) { |
||||||
|
if (data_str.size() <= sizeof(BinaryFormatVersion)) { |
||||||
|
return Status::InvalidArgument("Invalid CompactionServiceResult string"); |
||||||
|
} |
||||||
|
auto format_version = DecodeFixed32(data_str.data()); |
||||||
|
if (format_version == kOptionsString) { |
||||||
|
ConfigOptions cf; |
||||||
|
cf.invoke_prepare_options = false; |
||||||
|
cf.ignore_unknown_options = true; |
||||||
|
return OptionTypeInfo::ParseType( |
||||||
|
cf, data_str.substr(sizeof(BinaryFormatVersion)), cs_result_type_info, |
||||||
|
obj); |
||||||
|
} else { |
||||||
|
return Status::NotSupported( |
||||||
|
"Compaction Service Result data version not supported: " + |
||||||
|
std::to_string(format_version)); |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
Status CompactionServiceResult::Write(std::string* output) { |
||||||
|
char buf[sizeof(BinaryFormatVersion)]; |
||||||
|
EncodeFixed32(buf, kOptionsString); |
||||||
|
output->append(buf, sizeof(BinaryFormatVersion)); |
||||||
|
ConfigOptions cf; |
||||||
|
cf.invoke_prepare_options = false; |
||||||
|
return OptionTypeInfo::SerializeType(cf, cs_result_type_info, this, output); |
||||||
|
} |
||||||
|
|
||||||
|
#ifndef NDEBUG |
||||||
|
bool CompactionServiceResult::TEST_Equals(CompactionServiceResult* other) { |
||||||
|
std::string mismatch; |
||||||
|
return TEST_Equals(other, &mismatch); |
||||||
|
} |
||||||
|
|
||||||
|
bool CompactionServiceResult::TEST_Equals(CompactionServiceResult* other, |
||||||
|
std::string* mismatch) { |
||||||
|
ConfigOptions cf; |
||||||
|
cf.invoke_prepare_options = false; |
||||||
|
return OptionTypeInfo::TypesAreEqual(cf, cs_result_type_info, this, other, |
||||||
|
mismatch); |
||||||
|
} |
||||||
|
|
||||||
|
bool CompactionServiceInput::TEST_Equals(CompactionServiceInput* other) { |
||||||
|
std::string mismatch; |
||||||
|
return TEST_Equals(other, &mismatch); |
||||||
|
} |
||||||
|
|
||||||
|
bool CompactionServiceInput::TEST_Equals(CompactionServiceInput* other, |
||||||
|
std::string* mismatch) { |
||||||
|
ConfigOptions cf; |
||||||
|
cf.invoke_prepare_options = false; |
||||||
|
return OptionTypeInfo::TypesAreEqual(cf, cs_input_type_info, this, other, |
||||||
|
mismatch); |
||||||
|
} |
||||||
|
#endif // NDEBUG
|
||||||
|
} // namespace ROCKSDB_NAMESPACE
|
||||||
|
|
||||||
|
#endif // !ROCKSDB_LITE
|
@ -0,0 +1,46 @@ |
|||||||
|
// Copyright (c) Meta Platforms, Inc. and affiliates.
|
||||||
|
//
|
||||||
|
// This source code is licensed under both the GPLv2 (found in the
|
||||||
|
// COPYING file in the root directory) and Apache 2.0 License
|
||||||
|
// (found in the LICENSE.Apache file in the root directory).
|
||||||
|
//
|
||||||
|
// 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/compaction_state.h" |
||||||
|
|
||||||
|
namespace ROCKSDB_NAMESPACE { |
||||||
|
|
||||||
|
Slice CompactionState::SmallestUserKey() { |
||||||
|
for (const auto& sub_compact_state : sub_compact_states) { |
||||||
|
Slice smallest = sub_compact_state.SmallestUserKey(); |
||||||
|
if (!smallest.empty()) { |
||||||
|
return smallest; |
||||||
|
} |
||||||
|
} |
||||||
|
// If there is no finished output, return an empty slice.
|
||||||
|
return Slice{nullptr, 0}; |
||||||
|
} |
||||||
|
|
||||||
|
Slice CompactionState::LargestUserKey() { |
||||||
|
for (auto it = sub_compact_states.rbegin(); it < sub_compact_states.rend(); |
||||||
|
++it) { |
||||||
|
Slice largest = it->LargestUserKey(); |
||||||
|
if (!largest.empty()) { |
||||||
|
return largest; |
||||||
|
} |
||||||
|
} |
||||||
|
// If there is no finished output, return an empty slice.
|
||||||
|
return Slice{nullptr, 0}; |
||||||
|
} |
||||||
|
|
||||||
|
void CompactionState::AggregateCompactionStats( |
||||||
|
InternalStats::CompactionStatsFull& compaction_stats, |
||||||
|
CompactionJobStats& compaction_job_stats) { |
||||||
|
for (const auto& sc : sub_compact_states) { |
||||||
|
sc.AggregateCompactionStats(compaction_stats); |
||||||
|
compaction_job_stats.Add(sc.compaction_job_stats); |
||||||
|
} |
||||||
|
} |
||||||
|
} // namespace ROCKSDB_NAMESPACE
|
@ -0,0 +1,42 @@ |
|||||||
|
// Copyright (c) Meta Platforms, Inc. and affiliates.
|
||||||
|
//
|
||||||
|
// This source code is licensed under both the GPLv2 (found in the
|
||||||
|
// COPYING file in the root directory) and Apache 2.0 License
|
||||||
|
// (found in the LICENSE.Apache file in the root directory).
|
||||||
|
//
|
||||||
|
// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
|
||||||
|
// Use of this source code is governed by a BSD-style license that can be
|
||||||
|
// found in the LICENSE file. See the AUTHORS file for names of contributors.
|
||||||
|
|
||||||
|
#pragma once |
||||||
|
|
||||||
|
#include "db/compaction/compaction.h" |
||||||
|
#include "db/compaction/subcompaction_state.h" |
||||||
|
#include "db/internal_stats.h" |
||||||
|
|
||||||
|
// Data structures used for compaction_job and compaction_service_job which has
|
||||||
|
// the list of sub_compact_states and the aggregated information for the
|
||||||
|
// compaction.
|
||||||
|
namespace ROCKSDB_NAMESPACE { |
||||||
|
|
||||||
|
// Maintains state for the entire compaction
|
||||||
|
class CompactionState { |
||||||
|
public: |
||||||
|
Compaction* const compaction; |
||||||
|
|
||||||
|
// REQUIRED: subcompaction states are stored in order of increasing key-range
|
||||||
|
std::vector<SubcompactionState> sub_compact_states; |
||||||
|
Status status; |
||||||
|
|
||||||
|
void AggregateCompactionStats( |
||||||
|
InternalStats::CompactionStatsFull& compaction_stats, |
||||||
|
CompactionJobStats& compaction_job_stats); |
||||||
|
|
||||||
|
explicit CompactionState(Compaction* c) : compaction(c) {} |
||||||
|
|
||||||
|
Slice SmallestUserKey(); |
||||||
|
|
||||||
|
Slice LargestUserKey(); |
||||||
|
}; |
||||||
|
|
||||||
|
} // namespace ROCKSDB_NAMESPACE
|
@ -0,0 +1,223 @@ |
|||||||
|
// Copyright (c) Meta Platforms, Inc. and affiliates.
|
||||||
|
//
|
||||||
|
// This source code is licensed under both the GPLv2 (found in the
|
||||||
|
// COPYING file in the root directory) and Apache 2.0 License
|
||||||
|
// (found in the LICENSE.Apache file in the root directory).
|
||||||
|
//
|
||||||
|
// 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/subcompaction_state.h" |
||||||
|
|
||||||
|
#include "rocksdb/sst_partitioner.h" |
||||||
|
|
||||||
|
namespace ROCKSDB_NAMESPACE { |
||||||
|
void SubcompactionState::AggregateCompactionStats( |
||||||
|
InternalStats::CompactionStatsFull& compaction_stats) const { |
||||||
|
compaction_stats.stats.Add(compaction_outputs_.stats_); |
||||||
|
if (HasPenultimateLevelOutputs()) { |
||||||
|
compaction_stats.has_penultimate_level_output = true; |
||||||
|
compaction_stats.penultimate_level_stats.Add( |
||||||
|
penultimate_level_outputs_.stats_); |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
void SubcompactionState::FillFilesToCutForTtl() { |
||||||
|
if (compaction->immutable_options()->compaction_style != |
||||||
|
CompactionStyle::kCompactionStyleLevel || |
||||||
|
compaction->immutable_options()->compaction_pri != |
||||||
|
CompactionPri::kMinOverlappingRatio || |
||||||
|
compaction->mutable_cf_options()->ttl == 0 || |
||||||
|
compaction->num_input_levels() < 2 || compaction->bottommost_level()) { |
||||||
|
return; |
||||||
|
} |
||||||
|
|
||||||
|
// We define new file with the oldest ancestor time to be younger than 1/4
|
||||||
|
// TTL, and an old one to be older than 1/2 TTL time.
|
||||||
|
int64_t temp_current_time; |
||||||
|
auto get_time_status = compaction->immutable_options()->clock->GetCurrentTime( |
||||||
|
&temp_current_time); |
||||||
|
if (!get_time_status.ok()) { |
||||||
|
return; |
||||||
|
} |
||||||
|
auto current_time = static_cast<uint64_t>(temp_current_time); |
||||||
|
if (current_time < compaction->mutable_cf_options()->ttl) { |
||||||
|
return; |
||||||
|
} |
||||||
|
uint64_t old_age_thres = |
||||||
|
current_time - compaction->mutable_cf_options()->ttl / 2; |
||||||
|
|
||||||
|
const std::vector<FileMetaData*>& olevel = |
||||||
|
*(compaction->inputs(compaction->num_input_levels() - 1)); |
||||||
|
for (FileMetaData* file : olevel) { |
||||||
|
// Worth filtering out by start and end?
|
||||||
|
uint64_t oldest_ancester_time = file->TryGetOldestAncesterTime(); |
||||||
|
// We put old files if they are not too small to prevent a flood
|
||||||
|
// of small files.
|
||||||
|
if (oldest_ancester_time < old_age_thres && |
||||||
|
file->fd.GetFileSize() > |
||||||
|
compaction->mutable_cf_options()->target_file_size_base / 2) { |
||||||
|
files_to_cut_for_ttl_.push_back(file); |
||||||
|
} |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
OutputIterator SubcompactionState::GetOutputs() const { |
||||||
|
return OutputIterator(penultimate_level_outputs_.outputs_, |
||||||
|
compaction_outputs_.outputs_); |
||||||
|
} |
||||||
|
|
||||||
|
void SubcompactionState::Cleanup(Cache* cache) { |
||||||
|
penultimate_level_outputs_.Cleanup(); |
||||||
|
compaction_outputs_.Cleanup(); |
||||||
|
|
||||||
|
if (!status.ok()) { |
||||||
|
for (const auto& out : GetOutputs()) { |
||||||
|
// If this file was inserted into the table cache then remove
|
||||||
|
// them here because this compaction was not committed.
|
||||||
|
TableCache::Evict(cache, out.meta.fd.GetNumber()); |
||||||
|
} |
||||||
|
} |
||||||
|
// TODO: sub_compact.io_status is not checked like status. Not sure if thats
|
||||||
|
// intentional. So ignoring the io_status as of now.
|
||||||
|
io_status.PermitUncheckedError(); |
||||||
|
} |
||||||
|
|
||||||
|
Slice SubcompactionState::SmallestUserKey() const { |
||||||
|
if (has_penultimate_level_outputs_) { |
||||||
|
Slice a = compaction_outputs_.SmallestUserKey(); |
||||||
|
Slice b = penultimate_level_outputs_.SmallestUserKey(); |
||||||
|
if (a.empty()) { |
||||||
|
return b; |
||||||
|
} |
||||||
|
if (b.empty()) { |
||||||
|
return a; |
||||||
|
} |
||||||
|
const Comparator* user_cmp = |
||||||
|
compaction->column_family_data()->user_comparator(); |
||||||
|
if (user_cmp->Compare(a, b) > 0) { |
||||||
|
return b; |
||||||
|
} else { |
||||||
|
return a; |
||||||
|
} |
||||||
|
} else { |
||||||
|
return compaction_outputs_.SmallestUserKey(); |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
Slice SubcompactionState::LargestUserKey() const { |
||||||
|
if (has_penultimate_level_outputs_) { |
||||||
|
Slice a = compaction_outputs_.LargestUserKey(); |
||||||
|
Slice b = penultimate_level_outputs_.LargestUserKey(); |
||||||
|
if (a.empty()) { |
||||||
|
return b; |
||||||
|
} |
||||||
|
if (b.empty()) { |
||||||
|
return a; |
||||||
|
} |
||||||
|
const Comparator* user_cmp = |
||||||
|
compaction->column_family_data()->user_comparator(); |
||||||
|
if (user_cmp->Compare(a, b) < 0) { |
||||||
|
return b; |
||||||
|
} else { |
||||||
|
return a; |
||||||
|
} |
||||||
|
} else { |
||||||
|
return compaction_outputs_.LargestUserKey(); |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
bool SubcompactionState::ShouldStopBefore(const Slice& internal_key) { |
||||||
|
uint64_t curr_file_size = Current().GetCurrentOutputFileSize(); |
||||||
|
const InternalKeyComparator* icmp = |
||||||
|
&compaction->column_family_data()->internal_comparator(); |
||||||
|
|
||||||
|
// Invalid local_output_split_key indicates that we do not need to split
|
||||||
|
if (local_output_split_key_ != nullptr && !is_split_) { |
||||||
|
// Split occurs when the next key is larger than/equal to the cursor
|
||||||
|
if (icmp->Compare(internal_key, local_output_split_key_->Encode()) >= 0) { |
||||||
|
is_split_ = true; |
||||||
|
return true; |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
const std::vector<FileMetaData*>& grandparents = compaction->grandparents(); |
||||||
|
bool grandparant_file_switched = false; |
||||||
|
// Scan to find the earliest grandparent file that contains key.
|
||||||
|
while (grandparent_index_ < grandparents.size() && |
||||||
|
icmp->Compare(internal_key, |
||||||
|
grandparents[grandparent_index_]->largest.Encode()) > |
||||||
|
0) { |
||||||
|
if (seen_key_) { |
||||||
|
overlapped_bytes_ += grandparents[grandparent_index_]->fd.GetFileSize(); |
||||||
|
grandparant_file_switched = true; |
||||||
|
} |
||||||
|
assert(grandparent_index_ + 1 >= grandparents.size() || |
||||||
|
icmp->Compare( |
||||||
|
grandparents[grandparent_index_]->largest.Encode(), |
||||||
|
grandparents[grandparent_index_ + 1]->smallest.Encode()) <= 0); |
||||||
|
grandparent_index_++; |
||||||
|
} |
||||||
|
seen_key_ = true; |
||||||
|
|
||||||
|
if (grandparant_file_switched && |
||||||
|
overlapped_bytes_ + curr_file_size > compaction->max_compaction_bytes()) { |
||||||
|
// Too much overlap for current output; start new output
|
||||||
|
overlapped_bytes_ = 0; |
||||||
|
return true; |
||||||
|
} |
||||||
|
|
||||||
|
if (!files_to_cut_for_ttl_.empty()) { |
||||||
|
if (cur_files_to_cut_for_ttl_ != -1) { |
||||||
|
// Previous key is inside the range of a file
|
||||||
|
if (icmp->Compare(internal_key, |
||||||
|
files_to_cut_for_ttl_[cur_files_to_cut_for_ttl_] |
||||||
|
->largest.Encode()) > 0) { |
||||||
|
next_files_to_cut_for_ttl_ = cur_files_to_cut_for_ttl_ + 1; |
||||||
|
cur_files_to_cut_for_ttl_ = -1; |
||||||
|
return true; |
||||||
|
} |
||||||
|
} else { |
||||||
|
// Look for the key position
|
||||||
|
while (next_files_to_cut_for_ttl_ < |
||||||
|
static_cast<int>(files_to_cut_for_ttl_.size())) { |
||||||
|
if (icmp->Compare(internal_key, |
||||||
|
files_to_cut_for_ttl_[next_files_to_cut_for_ttl_] |
||||||
|
->smallest.Encode()) >= 0) { |
||||||
|
if (icmp->Compare(internal_key, |
||||||
|
files_to_cut_for_ttl_[next_files_to_cut_for_ttl_] |
||||||
|
->largest.Encode()) <= 0) { |
||||||
|
// With in the current file
|
||||||
|
cur_files_to_cut_for_ttl_ = next_files_to_cut_for_ttl_; |
||||||
|
return true; |
||||||
|
} |
||||||
|
// Beyond the current file
|
||||||
|
next_files_to_cut_for_ttl_++; |
||||||
|
} else { |
||||||
|
// Still fall into the gap
|
||||||
|
break; |
||||||
|
} |
||||||
|
} |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
return false; |
||||||
|
} |
||||||
|
|
||||||
|
Status SubcompactionState::AddToOutput( |
||||||
|
const CompactionIterator& iter, |
||||||
|
const CompactionFileOpenFunc& open_file_func, |
||||||
|
const CompactionFileCloseFunc& close_file_func) { |
||||||
|
// update target output first
|
||||||
|
is_current_penultimate_level_ = iter.output_to_penultimate_level(); |
||||||
|
current_outputs_ = is_current_penultimate_level_ ? &penultimate_level_outputs_ |
||||||
|
: &compaction_outputs_; |
||||||
|
if (is_current_penultimate_level_) { |
||||||
|
has_penultimate_level_outputs_ = true; |
||||||
|
} |
||||||
|
|
||||||
|
return Current().AddToOutput(iter, open_file_func, close_file_func); |
||||||
|
} |
||||||
|
|
||||||
|
} // namespace ROCKSDB_NAMESPACE
|
@ -0,0 +1,255 @@ |
|||||||
|
// Copyright (c) Meta Platforms, Inc. and affiliates.
|
||||||
|
//
|
||||||
|
// This source code is licensed under both the GPLv2 (found in the
|
||||||
|
// COPYING file in the root directory) and Apache 2.0 License
|
||||||
|
// (found in the LICENSE.Apache file in the root directory).
|
||||||
|
//
|
||||||
|
// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
|
||||||
|
// Use of this source code is governed by a BSD-style license that can be
|
||||||
|
// found in the LICENSE file. See the AUTHORS file for names of contributors.
|
||||||
|
|
||||||
|
#pragma once |
||||||
|
|
||||||
|
#include "db/blob/blob_file_addition.h" |
||||||
|
#include "db/blob/blob_garbage_meter.h" |
||||||
|
#include "db/compaction/compaction.h" |
||||||
|
#include "db/compaction/compaction_iterator.h" |
||||||
|
#include "db/compaction/compaction_outputs.h" |
||||||
|
#include "db/internal_stats.h" |
||||||
|
#include "db/output_validator.h" |
||||||
|
#include "db/range_del_aggregator.h" |
||||||
|
|
||||||
|
namespace ROCKSDB_NAMESPACE { |
||||||
|
|
||||||
|
// Maintains state and outputs for each sub-compaction
|
||||||
|
// It contains 2 `CompactionOutputs`:
|
||||||
|
// 1. one for the normal output files
|
||||||
|
// 2. another for the penultimate level outputs
|
||||||
|
// a `current` pointer maintains the current output group, when calling
|
||||||
|
// `AddToOutput()`, it checks the output of the current compaction_iterator key
|
||||||
|
// and point `current` to the target output group. By default, it just points to
|
||||||
|
// normal compaction_outputs, if the compaction_iterator key should be placed on
|
||||||
|
// the penultimate level, `current` is changed to point to
|
||||||
|
// `penultimate_level_outputs`.
|
||||||
|
// The later operations uses `Current()` to get the target group.
|
||||||
|
//
|
||||||
|
// +----------+ +-----------------------------+ +---------+
|
||||||
|
// | *current |--------> | compaction_outputs |----->| output |
|
||||||
|
// +----------+ +-----------------------------+ +---------+
|
||||||
|
// | | output |
|
||||||
|
// | +---------+
|
||||||
|
// | | ... |
|
||||||
|
// |
|
||||||
|
// | +-----------------------------+ +---------+
|
||||||
|
// +-------------> | penultimate_level_outputs |----->| output |
|
||||||
|
// +-----------------------------+ +---------+
|
||||||
|
// | ... |
|
||||||
|
|
||||||
|
class SubcompactionState { |
||||||
|
public: |
||||||
|
const Compaction* compaction; |
||||||
|
|
||||||
|
// The boundaries of the key-range this compaction is interested in. No two
|
||||||
|
// sub-compactions may have overlapping key-ranges.
|
||||||
|
// 'start' is inclusive, 'end' is exclusive, and nullptr means unbounded
|
||||||
|
const Slice *start, *end; |
||||||
|
|
||||||
|
// The return status of this sub-compaction
|
||||||
|
Status status; |
||||||
|
|
||||||
|
// The return IO Status of this sub-compaction
|
||||||
|
IOStatus io_status; |
||||||
|
|
||||||
|
// Notify on sub-compaction completion only if listener was notified on
|
||||||
|
// sub-compaction begin.
|
||||||
|
bool notify_on_subcompaction_completion = false; |
||||||
|
|
||||||
|
// compaction job stats for this sub-compaction
|
||||||
|
CompactionJobStats compaction_job_stats; |
||||||
|
|
||||||
|
// sub-compaction job id, which is used to identify different sub-compaction
|
||||||
|
// within the same compaction job.
|
||||||
|
const uint32_t sub_job_id; |
||||||
|
|
||||||
|
Slice SmallestUserKey() const; |
||||||
|
|
||||||
|
Slice LargestUserKey() const; |
||||||
|
|
||||||
|
// Get all outputs from the subcompaction. For per_key_placement compaction,
|
||||||
|
// it returns both the last level outputs and penultimate level outputs.
|
||||||
|
OutputIterator GetOutputs() const; |
||||||
|
|
||||||
|
// Assign range dels aggregator, for each range_del, it can only be assigned
|
||||||
|
// to one output level, for per_key_placement, it's going to be the
|
||||||
|
// penultimate level.
|
||||||
|
void AssignRangeDelAggregator( |
||||||
|
std::unique_ptr<CompactionRangeDelAggregator>&& range_del_agg) { |
||||||
|
if (compaction->SupportsPerKeyPlacement()) { |
||||||
|
penultimate_level_outputs_.AssignRangeDelAggregator( |
||||||
|
std::move(range_del_agg)); |
||||||
|
} else { |
||||||
|
compaction_outputs_.AssignRangeDelAggregator(std::move(range_del_agg)); |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
void RemoveLastEmptyOutput() { |
||||||
|
compaction_outputs_.RemoveLastEmptyOutput(); |
||||||
|
penultimate_level_outputs_.RemoveLastEmptyOutput(); |
||||||
|
} |
||||||
|
|
||||||
|
#ifndef ROCKSDB_LITE |
||||||
|
void BuildSubcompactionJobInfo( |
||||||
|
SubcompactionJobInfo& subcompaction_job_info) const { |
||||||
|
const Compaction* c = compaction; |
||||||
|
const ColumnFamilyData* cfd = c->column_family_data(); |
||||||
|
|
||||||
|
subcompaction_job_info.cf_id = cfd->GetID(); |
||||||
|
subcompaction_job_info.cf_name = cfd->GetName(); |
||||||
|
subcompaction_job_info.status = status; |
||||||
|
subcompaction_job_info.subcompaction_job_id = static_cast<int>(sub_job_id); |
||||||
|
subcompaction_job_info.base_input_level = c->start_level(); |
||||||
|
subcompaction_job_info.output_level = c->output_level(); |
||||||
|
subcompaction_job_info.stats = compaction_job_stats; |
||||||
|
} |
||||||
|
#endif // !ROCKSDB_LITE
|
||||||
|
|
||||||
|
SubcompactionState() = delete; |
||||||
|
SubcompactionState(const SubcompactionState&) = delete; |
||||||
|
SubcompactionState& operator=(const SubcompactionState&) = delete; |
||||||
|
|
||||||
|
SubcompactionState(Compaction* c, Slice* _start, Slice* _end, |
||||||
|
uint32_t _sub_job_id) |
||||||
|
: compaction(c), |
||||||
|
start(_start), |
||||||
|
end(_end), |
||||||
|
sub_job_id(_sub_job_id), |
||||||
|
compaction_outputs_(c, /*is_penultimate_level=*/false), |
||||||
|
penultimate_level_outputs_(c, /*is_penultimate_level=*/true) { |
||||||
|
assert(compaction != nullptr); |
||||||
|
const InternalKeyComparator* icmp = |
||||||
|
&compaction->column_family_data()->internal_comparator(); |
||||||
|
const InternalKey* output_split_key = compaction->GetOutputSplitKey(); |
||||||
|
// Invalid output_split_key indicates that we do not need to split
|
||||||
|
if (output_split_key != nullptr) { |
||||||
|
// We may only split the output when the cursor is in the range. Split
|
||||||
|
if ((end == nullptr || icmp->user_comparator()->Compare( |
||||||
|
ExtractUserKey(output_split_key->Encode()), |
||||||
|
ExtractUserKey(*end)) < 0) && |
||||||
|
(start == nullptr || icmp->user_comparator()->Compare( |
||||||
|
ExtractUserKey(output_split_key->Encode()), |
||||||
|
ExtractUserKey(*start)) > 0)) { |
||||||
|
local_output_split_key_ = output_split_key; |
||||||
|
} |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
SubcompactionState(SubcompactionState&& state) noexcept |
||||||
|
: compaction(state.compaction), |
||||||
|
start(state.start), |
||||||
|
end(state.end), |
||||||
|
status(std::move(state.status)), |
||||||
|
io_status(std::move(state.io_status)), |
||||||
|
notify_on_subcompaction_completion( |
||||||
|
state.notify_on_subcompaction_completion), |
||||||
|
compaction_job_stats(std::move(state.compaction_job_stats)), |
||||||
|
sub_job_id(state.sub_job_id), |
||||||
|
files_to_cut_for_ttl_(std::move(state.files_to_cut_for_ttl_)), |
||||||
|
cur_files_to_cut_for_ttl_(state.cur_files_to_cut_for_ttl_), |
||||||
|
next_files_to_cut_for_ttl_(state.next_files_to_cut_for_ttl_), |
||||||
|
grandparent_index_(state.grandparent_index_), |
||||||
|
overlapped_bytes_(state.overlapped_bytes_), |
||||||
|
seen_key_(state.seen_key_), |
||||||
|
compaction_outputs_(std::move(state.compaction_outputs_)), |
||||||
|
penultimate_level_outputs_(std::move(state.penultimate_level_outputs_)), |
||||||
|
is_current_penultimate_level_(state.is_current_penultimate_level_), |
||||||
|
has_penultimate_level_outputs_(state.has_penultimate_level_outputs_) { |
||||||
|
current_outputs_ = is_current_penultimate_level_ |
||||||
|
? &penultimate_level_outputs_ |
||||||
|
: &compaction_outputs_; |
||||||
|
} |
||||||
|
|
||||||
|
bool HasPenultimateLevelOutputs() const { |
||||||
|
return has_penultimate_level_outputs_ || |
||||||
|
penultimate_level_outputs_.HasRangeDel(); |
||||||
|
} |
||||||
|
|
||||||
|
void FillFilesToCutForTtl(); |
||||||
|
|
||||||
|
// Returns true iff we should stop building the current output
|
||||||
|
// before processing "internal_key".
|
||||||
|
bool ShouldStopBefore(const Slice& internal_key); |
||||||
|
|
||||||
|
bool IsCurrentPenultimateLevel() const { |
||||||
|
return is_current_penultimate_level_; |
||||||
|
} |
||||||
|
|
||||||
|
// Add all the new files from this compaction to version_edit
|
||||||
|
void AddOutputsEdit(VersionEdit* out_edit) const { |
||||||
|
for (const auto& file : penultimate_level_outputs_.outputs_) { |
||||||
|
out_edit->AddFile(compaction->GetPenultimateLevel(), file.meta); |
||||||
|
} |
||||||
|
for (const auto& file : compaction_outputs_.outputs_) { |
||||||
|
out_edit->AddFile(compaction->output_level(), file.meta); |
||||||
|
} |
||||||
|
} |
||||||
|
|
||||||
|
void Cleanup(Cache* cache); |
||||||
|
|
||||||
|
void AggregateCompactionStats( |
||||||
|
InternalStats::CompactionStatsFull& compaction_stats) const; |
||||||
|
|
||||||
|
CompactionOutputs& Current() const { |
||||||
|
assert(current_outputs_); |
||||||
|
return *current_outputs_; |
||||||
|
} |
||||||
|
|
||||||
|
// Add compaction_iterator key/value to the `Current` output group.
|
||||||
|
Status AddToOutput(const CompactionIterator& iter, |
||||||
|
const CompactionFileOpenFunc& open_file_func, |
||||||
|
const CompactionFileCloseFunc& close_file_func); |
||||||
|
|
||||||
|
// Close all compaction output files, both output_to_penultimate_level outputs
|
||||||
|
// and normal outputs.
|
||||||
|
Status CloseCompactionFiles(const Status& curr_status, |
||||||
|
const CompactionFileOpenFunc& open_file_func, |
||||||
|
const CompactionFileCloseFunc& close_file_func) { |
||||||
|
// Call FinishCompactionOutputFile() even if status is not ok: it needs to
|
||||||
|
// close the output file.
|
||||||
|
Status s = penultimate_level_outputs_.CloseOutput( |
||||||
|
curr_status, open_file_func, close_file_func); |
||||||
|
s = compaction_outputs_.CloseOutput(s, open_file_func, close_file_func); |
||||||
|
return s; |
||||||
|
} |
||||||
|
|
||||||
|
private: |
||||||
|
// Some identified files with old oldest ancester time and the range should be
|
||||||
|
// isolated out so that the output file(s) in that range can be merged down
|
||||||
|
// for TTL and clear the timestamps for the range.
|
||||||
|
std::vector<FileMetaData*> files_to_cut_for_ttl_; |
||||||
|
int cur_files_to_cut_for_ttl_ = -1; |
||||||
|
int next_files_to_cut_for_ttl_ = 0; |
||||||
|
|
||||||
|
// An index that used to speed up ShouldStopBefore().
|
||||||
|
size_t grandparent_index_ = 0; |
||||||
|
// The number of bytes overlapping between the current output and
|
||||||
|
// grandparent files used in ShouldStopBefore().
|
||||||
|
uint64_t overlapped_bytes_ = 0; |
||||||
|
// A flag determines whether the key has been seen in ShouldStopBefore()
|
||||||
|
bool seen_key_ = false; |
||||||
|
|
||||||
|
// A flag determines if this subcompaction has been split by the cursor
|
||||||
|
bool is_split_ = false; |
||||||
|
|
||||||
|
// We also maintain the output split key for each subcompaction to avoid
|
||||||
|
// repetitive comparison in ShouldStopBefore()
|
||||||
|
const InternalKey* local_output_split_key_ = nullptr; |
||||||
|
|
||||||
|
// State kept for output being generated
|
||||||
|
CompactionOutputs compaction_outputs_; |
||||||
|
CompactionOutputs penultimate_level_outputs_; |
||||||
|
CompactionOutputs* current_outputs_ = &compaction_outputs_; |
||||||
|
bool is_current_penultimate_level_ = false; |
||||||
|
bool has_penultimate_level_outputs_ = false; |
||||||
|
}; |
||||||
|
|
||||||
|
} // namespace ROCKSDB_NAMESPACE
|
File diff suppressed because it is too large
Load Diff
Loading…
Reference in new issue