|
|
|
// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
|
|
|
|
// This source code is licensed under both the GPLv2 (found in the
|
|
|
|
// COPYING file in the root directory) and Apache 2.0 License
|
|
|
|
// (found in the LICENSE.Apache file in the root directory).
|
|
|
|
//
|
|
|
|
// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
|
|
|
|
// Use of this source code is governed by a BSD-style license that can be
|
|
|
|
// found in the LICENSE file. See the AUTHORS file for names of contributors.
|
|
|
|
#pragma once
|
|
|
|
|
|
|
|
#include <atomic>
|
|
|
|
#include <deque>
|
|
|
|
#include <functional>
|
|
|
|
#include <limits>
|
|
|
|
#include <set>
|
|
|
|
#include <string>
|
|
|
|
#include <utility>
|
|
|
|
#include <vector>
|
|
|
|
|
|
|
|
#include "db/blob/blob_file_completion_callback.h"
|
|
|
|
#include "db/column_family.h"
|
|
|
|
#include "db/compaction/compaction_iterator.h"
|
|
|
|
#include "db/compaction/compaction_outputs.h"
|
|
|
|
#include "db/flush_scheduler.h"
|
|
|
|
#include "db/internal_stats.h"
|
|
|
|
#include "db/job_context.h"
|
|
|
|
#include "db/log_writer.h"
|
|
|
|
#include "db/memtable_list.h"
|
|
|
|
#include "db/range_del_aggregator.h"
|
|
|
|
#include "db/seqno_to_time_mapping.h"
|
|
|
|
#include "db/version_edit.h"
|
|
|
|
#include "db/write_controller.h"
|
|
|
|
#include "db/write_thread.h"
|
|
|
|
#include "logging/event_logger.h"
|
|
|
|
#include "options/cf_options.h"
|
|
|
|
#include "options/db_options.h"
|
|
|
|
#include "port/port.h"
|
|
|
|
#include "rocksdb/compaction_filter.h"
|
|
|
|
#include "rocksdb/compaction_job_stats.h"
|
|
|
|
#include "rocksdb/db.h"
|
|
|
|
#include "rocksdb/env.h"
|
|
|
|
#include "rocksdb/memtablerep.h"
|
|
|
|
#include "rocksdb/transaction_log.h"
|
|
|
|
#include "table/scoped_arena_iterator.h"
|
|
|
|
#include "util/autovector.h"
|
|
|
|
#include "util/stop_watch.h"
|
|
|
|
#include "util/thread_local.h"
|
|
|
|
|
|
|
|
namespace ROCKSDB_NAMESPACE {
|
|
|
|
|
|
|
|
class Arena;
|
|
|
|
class CompactionState;
|
|
|
|
class ErrorHandler;
|
|
|
|
class MemTable;
|
|
|
|
class SnapshotChecker;
|
|
|
|
class SystemClock;
|
|
|
|
class TableCache;
|
|
|
|
class Version;
|
|
|
|
class VersionEdit;
|
|
|
|
class VersionSet;
|
|
|
|
|
|
|
|
class SubcompactionState;
|
|
|
|
|
|
|
|
// CompactionJob is responsible for executing the compaction. Each (manual or
|
|
|
|
// automated) compaction corresponds to a CompactionJob object, and usually
|
|
|
|
// goes through the stages of `Prepare()`->`Run()`->`Install()`. CompactionJob
|
|
|
|
// will divide the compaction into subcompactions and execute them in parallel
|
|
|
|
// if needed.
|
|
|
|
//
|
|
|
|
// CompactionJob has 2 main stats:
|
|
|
|
// 1. CompactionJobStats compaction_job_stats_
|
|
|
|
// CompactionJobStats is a public data structure which is part of Compaction
|
|
|
|
// event listener that rocksdb share the job stats with the user.
|
|
|
|
// Internally it's an aggregation of all the compaction_job_stats from each
|
|
|
|
// `SubcompactionState`:
|
|
|
|
// +------------------------+
|
|
|
|
// | SubcompactionState |
|
|
|
|
// | |
|
|
|
|
// +--------->| compaction_job_stats |
|
|
|
|
// | | |
|
|
|
|
// | +------------------------+
|
|
|
|
// +------------------------+ |
|
|
|
|
// | CompactionJob | | +------------------------+
|
|
|
|
// | | | | SubcompactionState |
|
|
|
|
// | compaction_job_stats +-----+ | |
|
|
|
|
// | | +--------->| compaction_job_stats |
|
|
|
|
// | | | | |
|
|
|
|
// +------------------------+ | +------------------------+
|
|
|
|
// |
|
|
|
|
// | +------------------------+
|
|
|
|
// | | SubcompactionState |
|
|
|
|
// | | |
|
|
|
|
// +--------->+ compaction_job_stats |
|
|
|
|
// | | |
|
|
|
|
// | +------------------------+
|
|
|
|
// |
|
|
|
|
// | +------------------------+
|
|
|
|
// | | ... |
|
|
|
|
// +--------->+ |
|
|
|
|
// +------------------------+
|
|
|
|
//
|
|
|
|
// 2. CompactionStatsFull compaction_stats_
|
|
|
|
// `CompactionStatsFull` is an internal stats about the compaction, which
|
|
|
|
// is eventually sent to `ColumnFamilyData::internal_stats_` and used for
|
|
|
|
// logging and public metrics.
|
|
|
|
// Internally, it's an aggregation of stats_ from each `SubcompactionState`.
|
|
|
|
// It has 2 parts, normal stats about the main compaction information and
|
|
|
|
// the penultimate level output stats.
|
|
|
|
// `SubcompactionState` maintains the CompactionOutputs for normal output and
|
|
|
|
// the penultimate level output if exists, the per_level stats is
|
|
|
|
// stored with the outputs.
|
|
|
|
// +---------------------------+
|
|
|
|
// | SubcompactionState |
|
|
|
|
// | |
|
|
|
|
// | +----------------------+ |
|
|
|
|
// | | CompactionOutputs | |
|
|
|
|
// | | (normal output) | |
|
|
|
|
// +---->| stats_ | |
|
|
|
|
// | | +----------------------+ |
|
|
|
|
// | | |
|
|
|
|
// | | +----------------------+ |
|
|
|
|
// +--------------------------------+ | | | CompactionOutputs | |
|
|
|
|
// | CompactionJob | | | | (penultimate_level) | |
|
|
|
|
// | | +--------->| stats_ | |
|
|
|
|
// | compaction_stats_ | | | | +----------------------+ |
|
|
|
|
// | +-------------------------+ | | | | |
|
|
|
|
// | |stats (normal) |------|----+ +---------------------------+
|
|
|
|
// | +-------------------------+ | | |
|
|
|
|
// | | | |
|
|
|
|
// | +-------------------------+ | | | +---------------------------+
|
|
|
|
// | |penultimate_level_stats +------+ | | SubcompactionState |
|
|
|
|
// | +-------------------------+ | | | | |
|
|
|
|
// | | | | | +----------------------+ |
|
|
|
|
// | | | | | | CompactionOutputs | |
|
|
|
|
// +--------------------------------+ | | | | (normal output) | |
|
|
|
|
// | +---->| stats_ | |
|
|
|
|
// | | +----------------------+ |
|
|
|
|
// | | |
|
|
|
|
// | | +----------------------+ |
|
|
|
|
// | | | CompactionOutputs | |
|
|
|
|
// | | | (penultimate_level) | |
|
|
|
|
// +--------->| stats_ | |
|
|
|
|
// | +----------------------+ |
|
|
|
|
// | |
|
|
|
|
// +---------------------------+
|
|
|
|
|
|
|
|
class CompactionJob {
|
|
|
|
public:
|
|
|
|
CompactionJob(
|
|
|
|
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* db_directory, FSDirectory* output_directory,
|
|
|
|
FSDirectory* blob_output_directory, Statistics* stats,
|
|
|
|
InstrumentedMutex* db_mutex, ErrorHandler* db_error_handler,
|
|
|
|
std::vector<SequenceNumber> existing_snapshots,
|
|
|
|
SequenceNumber earliest_write_conflict_snapshot,
|
CompactionIterator sees consistent view of which keys are committed (#9830)
Summary:
**This PR does not affect the functionality of `DB` and write-committed transactions.**
`CompactionIterator` uses `KeyCommitted(seq)` to determine if a key in the database is committed.
As the name 'write-committed' implies, if write-committed policy is used, a key exists in the database only if
it is committed. In fact, the implementation of `KeyCommitted()` is as follows:
```
inline bool KeyCommitted(SequenceNumber seq) {
// For non-txn-db and write-committed, snapshot_checker_ is always nullptr.
return snapshot_checker_ == nullptr ||
snapshot_checker_->CheckInSnapshot(seq, kMaxSequence) == SnapshotCheckerResult::kInSnapshot;
}
```
With that being said, we focus on write-prepared/write-unprepared transactions.
A few notes:
- A key can exist in the db even if it's uncommitted. Therefore, we rely on `snapshot_checker_` to determine data visibility. We also require that all writes go through transaction API instead of the raw `WriteBatch` + `Write`, thus at most one uncommitted version of one user key can exist in the database.
- `CompactionIterator` outputs a key as long as the key is uncommitted.
Due to the above reasons, it is possible that `CompactionIterator` decides to output an uncommitted key without
doing further checks on the key (`NextFromInput()`). By the time the key is being prepared for output, the key becomes
committed because the `snapshot_checker_(seq, kMaxSequence)` becomes true in the implementation of `KeyCommitted()`.
Then `CompactionIterator` will try to zero its sequence number and hit assertion error if the key is a tombstone.
To fix this issue, we should make the `CompactionIterator` see a consistent view of the input keys. Note that
for write-prepared/write-unprepared, the background flush/compaction jobs already take a "job snapshot" before starting
processing keys. The job snapshot is released only after the entire flush/compaction finishes. We can use this snapshot
to determine whether a key is committed or not with minor change to `KeyCommitted()`.
```
inline bool KeyCommitted(SequenceNumber sequence) {
// For non-txn-db and write-committed, snapshot_checker_ is always nullptr.
return snapshot_checker_ == nullptr ||
snapshot_checker_->CheckInSnapshot(sequence, job_snapshot_) ==
SnapshotCheckerResult::kInSnapshot;
}
```
As a result, whether a key is committed or not will remain a constant throughout compaction, causing no trouble
for `CompactionIterator`s assertions.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/9830
Test Plan: make check
Reviewed By: ltamasi
Differential Revision: D35561162
Pulled By: riversand963
fbshipit-source-id: 0e00d200c195240341cfe6d34cbc86798b315b9f
3 years ago
|
|
|
const SnapshotChecker* snapshot_checker, JobContext* job_context,
|
|
|
|
std::shared_ptr<Cache> table_cache, EventLogger* event_logger,
|
|
|
|
bool paranoid_file_checks, bool measure_io_stats,
|
|
|
|
const std::string& dbname, CompactionJobStats* compaction_job_stats,
|
|
|
|
Env::Priority thread_pri, 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 full_history_ts_low = "", std::string trim_ts = "",
|
Support subcmpct using reserved resources for round-robin priority (#10341)
Summary:
Earlier implementation of round-robin priority can only pick one file at a time and disallows parallel compactions within the same level. In this PR, round-robin compaction policy will expand towards more input files with respecting some additional constraints, which are summarized as follows:
* Constraint 1: We can only pick consecutive files
- Constraint 1a: When a file is being compacted (or some input files are being compacted after expanding), we cannot choose it and have to stop choosing more files
- Constraint 1b: When we reach the last file (with the largest keys), we cannot choose more files (the next file will be the first one with small keys)
* Constraint 2: We should ensure the total compaction bytes (including the overlapped files from the next level) is no more than `mutable_cf_options_.max_compaction_bytes`
* Constraint 3: We try our best to pick as many files as possible so that the post-compaction level size can be just less than `MaxBytesForLevel(start_level_)`
* Constraint 4: If trivial move is allowed, we reuse the logic of `TryNonL0TrivialMove()` instead of expanding files with Constraint 3
More details can be found in `LevelCompactionBuilder::SetupOtherFilesWithRoundRobinExpansion()`.
The above optimization accelerates the process of moving the compaction cursor, in which the write-amp can be further reduced. While a large compaction may lead to high write stall, we break this large compaction into several subcompactions **regardless of** the `max_subcompactions` limit. The number of subcompactions for round-robin compaction priority is determined through the following steps:
* Step 1: Initialized against `max_output_file_limit`, the number of input files in the start level, and also the range size limit `ranges.size()`
* Step 2: Call `AcquireSubcompactionResources()`when max subcompactions is not sufficient, but we may or may not obtain desired resources, additional number of resources is stored in `extra_num_subcompaction_threads_reserved_`). Subcompaction limit is changed and update `num_planned_subcompactions` with `GetSubcompactionLimit()`
* Step 3: Call `ShrinkSubcompactionResources()` to ensure extra resources can be released (extra resources may exist for round-robin compaction when the number of actual number of subcompactions is less than the number of planned subcompactions)
More details can be found in `CompactionJob::AcquireSubcompactionResources()`,`CompactionJob::ShrinkSubcompactionResources()`, and `CompactionJob::ReleaseSubcompactionResources()`.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/10341
Test Plan: Add `CompactionPriMultipleFilesRoundRobin[1-3]` unit test in `compaction_picker_test.cc` and `RoundRobinSubcompactionsAgainstResources.SubcompactionsUsingResources/[0-4]`, `RoundRobinSubcompactionsAgainstPressureToken.PressureTokenTest/[0-1]` in `db_compaction_test.cc`
Reviewed By: ajkr, hx235
Differential Revision: D37792644
Pulled By: littlepig2013
fbshipit-source-id: 7fecb7c4ffd97b34bbf6e3b760b2c35a772a0657
2 years ago
|
|
|
BlobFileCompletionCallback* blob_callback = nullptr,
|
|
|
|
int* bg_compaction_scheduled = nullptr,
|
|
|
|
int* bg_bottom_compaction_scheduled = nullptr);
|
|
|
|
|
|
|
|
virtual ~CompactionJob();
|
|
|
|
|
|
|
|
// no copy/move
|
|
|
|
CompactionJob(CompactionJob&& job) = delete;
|
|
|
|
CompactionJob(const CompactionJob& job) = delete;
|
|
|
|
CompactionJob& operator=(const CompactionJob& job) = delete;
|
|
|
|
|
|
|
|
// REQUIRED: mutex held
|
|
|
|
// Prepare for the compaction by setting up boundaries for each subcompaction
|
|
|
|
void Prepare();
|
|
|
|
// REQUIRED mutex not held
|
|
|
|
// Launch threads for each subcompaction and wait for them to finish. After
|
|
|
|
// that, verify table is usable and finally do bookkeeping to unify
|
|
|
|
// subcompaction results
|
|
|
|
Status Run();
|
Parallelize L0-L1 Compaction: Restructure Compaction Job
Summary:
As of now compactions involving files from Level 0 and Level 1 are single
threaded because the files in L0, although sorted, are not range partitioned like
the other levels. This means that during L0-L1 compaction each file from L1
needs to be merged with potentially all the files from L0.
This attempt to parallelize the L0-L1 compaction assigns a thread and a
corresponding iterator to each L1 file that then considers only the key range
found in that L1 file and only the L0 files that have those keys (and only the
specific portion of those L0 files in which those keys are found). In this way
the overlap is minimized and potentially eliminated between different iterators
focusing on the same files.
The first step is to restructure the compaction logic to break L0-L1 compactions
into multiple, smaller, sequential compactions. Eventually each of these smaller
jobs will be run simultaneously. Areas to pay extra attention to are
# Correct aggregation of compaction job statistics across multiple threads
# Proper opening/closing of output files (make sure each thread's is unique)
# Keys that span multiple L1 files
# Skewed distributions of keys within L0 files
Test Plan: Make and run db_test (newer version has separate compaction tests) and compaction_job_stats_test
Reviewers: igor, noetzli, anthony, sdong, yhchiang
Reviewed By: yhchiang
Subscribers: MarkCallaghan, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D42699
9 years ago
|
|
|
|
|
|
|
// REQUIRED: mutex held
|
|
|
|
// Add compaction input/output to the current version
|
|
|
|
Status Install(const MutableCFOptions& mutable_cf_options);
|
|
|
|
|
Pass IOStatus to write path and set retryable IO Error as hard error in BG jobs (#6487)
Summary:
In the current code base, we use Status to get and store the returned status from the call. Specifically, for IO related functions, the current Status cannot reflect the IO Error details such as error scope, error retryable attribute, and others. With the implementation of https://github.com/facebook/rocksdb/issues/5761, we have the new Wrapper for IO, which returns IOStatus instead of Status. However, the IOStatus is purged at the lower level of write path and transferred to Status.
The first job of this PR is to pass the IOStatus to the write path (flush, WAL write, and Compaction). The second job is to identify the Retryable IO Error as HardError, and set the bg_error_ as HardError. In this case, the DB Instance becomes read only. User is informed of the Status and need to take actions to deal with it (e.g., call db->Resume()).
Pull Request resolved: https://github.com/facebook/rocksdb/pull/6487
Test Plan: Added the testing case to error_handler_fs_test. Pass make asan_check
Reviewed By: anand1976
Differential Revision: D20685017
Pulled By: zhichao-cao
fbshipit-source-id: ff85f042896243abcd6ef37877834e26f36b6eb0
5 years ago
|
|
|
// Return the IO status
|
|
|
|
IOStatus io_status() const { return io_status_; }
|
|
|
|
|
|
|
|
protected:
|
|
|
|
void UpdateCompactionStats();
|
|
|
|
void LogCompaction();
|
|
|
|
virtual void RecordCompactionIOStats();
|
|
|
|
void CleanupCompaction();
|
|
|
|
|
|
|
|
// Call compaction filter. Then iterate through input and compact the
|
|
|
|
// kv-pairs
|
|
|
|
void ProcessKeyValueCompaction(SubcompactionState* sub_compact);
|
|
|
|
|
|
|
|
CompactionState* compact_;
|
|
|
|
InternalStats::CompactionStatsFull compaction_stats_;
|
|
|
|
const ImmutableDBOptions& db_options_;
|
|
|
|
const MutableDBOptions mutable_db_options_copy_;
|
|
|
|
LogBuffer* log_buffer_;
|
|
|
|
FSDirectory* output_directory_;
|
|
|
|
Statistics* stats_;
|
|
|
|
// Is this compaction creating a file in the bottom most level?
|
|
|
|
bool bottommost_level_;
|
|
|
|
|
|
|
|
Env::WriteLifeTimeHint write_hint_;
|
|
|
|
|
|
|
|
IOStatus io_status_;
|
|
|
|
|
|
|
|
CompactionJobStats* compaction_job_stats_;
|
|
|
|
|
|
|
|
private:
|
Set Write rate limiter priority dynamically and pass it to FS (#9988)
Summary:
### Context:
Background compactions and flush generate large reads and writes, and can be long running, especially for universal compaction. In some cases, this can impact foreground reads and writes by users.
From the RocksDB perspective, there can be two kinds of rate limiters, the internal (native) one and the external one.
- The internal (native) rate limiter is introduced in [the wiki](https://github.com/facebook/rocksdb/wiki/Rate-Limiter). Currently, only IO_LOW and IO_HIGH are used and they are set statically.
- For the external rate limiter, in FSWritableFile functions, IOOptions is open for end users to set and get rate_limiter_priority for their own rate limiter. Currently, RocksDB doesn’t pass the rate_limiter_priority through IOOptions to the file system.
### Solution
During the User Read, Flush write, Compaction read/write, the WriteController is used to determine whether DB writes are stalled or slowed down. The rate limiter priority (Env::IOPriority) can be determined accordingly. We decided to always pass the priority in IOOptions. What the file system does with it should be a contract between the user and the file system. We would like to set the rate limiter priority at file level, since the Flush/Compaction job level may be too coarse with multiple files and block IO level is too granular.
**This PR is for the Write path.** The **Write:** dynamic priority for different state are listed as follows:
| State | Normal | Delayed | Stalled |
| ----- | ------ | ------- | ------- |
| Flush | IO_HIGH | IO_USER | IO_USER |
| Compaction | IO_LOW | IO_USER | IO_USER |
Flush and Compaction writes share the same call path through BlockBaseTableWriter, WritableFileWriter, and FSWritableFile. When a new FSWritableFile object is created, its io_priority_ can be set dynamically based on the state of the WriteController. In WritableFileWriter, before the call sites of FSWritableFile functions, WritableFileWriter::DecideRateLimiterPriority() determines the rate_limiter_priority. The options (IOOptions) argument of FSWritableFile functions will be updated with the rate_limiter_priority.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/9988
Test Plan: Add unit tests.
Reviewed By: anand1976
Differential Revision: D36395159
Pulled By: gitbw95
fbshipit-source-id: a7c82fc29759139a1a07ec46c37dbf7e753474cf
3 years ago
|
|
|
friend class CompactionJobTestBase;
|
|
|
|
|
|
|
|
// Generates a histogram representing potential divisions of key ranges from
|
|
|
|
// the input. It adds the starting and/or ending keys of certain input files
|
|
|
|
// to the working set and then finds the approximate size of data in between
|
|
|
|
// each consecutive pair of slices. Then it divides these ranges into
|
|
|
|
// consecutive groups such that each group has a similar size.
|
|
|
|
void GenSubcompactionBoundaries();
|
Parallelize L0-L1 Compaction: Restructure Compaction Job
Summary:
As of now compactions involving files from Level 0 and Level 1 are single
threaded because the files in L0, although sorted, are not range partitioned like
the other levels. This means that during L0-L1 compaction each file from L1
needs to be merged with potentially all the files from L0.
This attempt to parallelize the L0-L1 compaction assigns a thread and a
corresponding iterator to each L1 file that then considers only the key range
found in that L1 file and only the L0 files that have those keys (and only the
specific portion of those L0 files in which those keys are found). In this way
the overlap is minimized and potentially eliminated between different iterators
focusing on the same files.
The first step is to restructure the compaction logic to break L0-L1 compactions
into multiple, smaller, sequential compactions. Eventually each of these smaller
jobs will be run simultaneously. Areas to pay extra attention to are
# Correct aggregation of compaction job statistics across multiple threads
# Proper opening/closing of output files (make sure each thread's is unique)
# Keys that span multiple L1 files
# Skewed distributions of keys within L0 files
Test Plan: Make and run db_test (newer version has separate compaction tests) and compaction_job_stats_test
Reviewers: igor, noetzli, anthony, sdong, yhchiang
Reviewed By: yhchiang
Subscribers: MarkCallaghan, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D42699
9 years ago
|
|
|
|
Support subcmpct using reserved resources for round-robin priority (#10341)
Summary:
Earlier implementation of round-robin priority can only pick one file at a time and disallows parallel compactions within the same level. In this PR, round-robin compaction policy will expand towards more input files with respecting some additional constraints, which are summarized as follows:
* Constraint 1: We can only pick consecutive files
- Constraint 1a: When a file is being compacted (or some input files are being compacted after expanding), we cannot choose it and have to stop choosing more files
- Constraint 1b: When we reach the last file (with the largest keys), we cannot choose more files (the next file will be the first one with small keys)
* Constraint 2: We should ensure the total compaction bytes (including the overlapped files from the next level) is no more than `mutable_cf_options_.max_compaction_bytes`
* Constraint 3: We try our best to pick as many files as possible so that the post-compaction level size can be just less than `MaxBytesForLevel(start_level_)`
* Constraint 4: If trivial move is allowed, we reuse the logic of `TryNonL0TrivialMove()` instead of expanding files with Constraint 3
More details can be found in `LevelCompactionBuilder::SetupOtherFilesWithRoundRobinExpansion()`.
The above optimization accelerates the process of moving the compaction cursor, in which the write-amp can be further reduced. While a large compaction may lead to high write stall, we break this large compaction into several subcompactions **regardless of** the `max_subcompactions` limit. The number of subcompactions for round-robin compaction priority is determined through the following steps:
* Step 1: Initialized against `max_output_file_limit`, the number of input files in the start level, and also the range size limit `ranges.size()`
* Step 2: Call `AcquireSubcompactionResources()`when max subcompactions is not sufficient, but we may or may not obtain desired resources, additional number of resources is stored in `extra_num_subcompaction_threads_reserved_`). Subcompaction limit is changed and update `num_planned_subcompactions` with `GetSubcompactionLimit()`
* Step 3: Call `ShrinkSubcompactionResources()` to ensure extra resources can be released (extra resources may exist for round-robin compaction when the number of actual number of subcompactions is less than the number of planned subcompactions)
More details can be found in `CompactionJob::AcquireSubcompactionResources()`,`CompactionJob::ShrinkSubcompactionResources()`, and `CompactionJob::ReleaseSubcompactionResources()`.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/10341
Test Plan: Add `CompactionPriMultipleFilesRoundRobin[1-3]` unit test in `compaction_picker_test.cc` and `RoundRobinSubcompactionsAgainstResources.SubcompactionsUsingResources/[0-4]`, `RoundRobinSubcompactionsAgainstPressureToken.PressureTokenTest/[0-1]` in `db_compaction_test.cc`
Reviewed By: ajkr, hx235
Differential Revision: D37792644
Pulled By: littlepig2013
fbshipit-source-id: 7fecb7c4ffd97b34bbf6e3b760b2c35a772a0657
2 years ago
|
|
|
// Get the number of planned subcompactions based on max_subcompactions and
|
|
|
|
// extra reserved resources
|
|
|
|
uint64_t GetSubcompactionsLimit();
|
|
|
|
|
|
|
|
// Additional reserved threads are reserved and the number is stored in
|
|
|
|
// extra_num_subcompaction_threads_reserved__. For now, this happens only if
|
|
|
|
// the compaction priority is round-robin and max_subcompactions is not
|
|
|
|
// sufficient (extra resources may be needed)
|
|
|
|
void AcquireSubcompactionResources(int num_extra_required_subcompactions);
|
|
|
|
|
|
|
|
// Additional threads may be reserved during IncreaseSubcompactionResources()
|
|
|
|
// if num_actual_subcompactions is less than num_planned_subcompactions.
|
|
|
|
// Additional threads will be released and the bg_compaction_scheduled_ or
|
|
|
|
// bg_bottom_compaction_scheduled_ will be updated if they are used.
|
|
|
|
// DB Mutex lock is required.
|
|
|
|
void ShrinkSubcompactionResources(uint64_t num_extra_resources);
|
|
|
|
|
|
|
|
// Release all reserved threads and update the compaction limits.
|
|
|
|
void ReleaseSubcompactionResources();
|
|
|
|
|
|
|
|
CompactionServiceJobStatus ProcessKeyValueCompactionWithCompactionService(
|
|
|
|
SubcompactionState* sub_compact);
|
|
|
|
|
|
|
|
// update the thread status for starting a compaction.
|
|
|
|
void ReportStartedCompaction(Compaction* compaction);
|
|
|
|
|
|
|
|
Status FinishCompactionOutputFile(const Status& input_status,
|
|
|
|
SubcompactionState* sub_compact,
|
|
|
|
CompactionOutputs& outputs,
|
|
|
|
const Slice& next_table_min_key);
|
|
|
|
Status InstallCompactionResults(const MutableCFOptions& mutable_cf_options);
|
|
|
|
Status OpenCompactionOutputFile(SubcompactionState* sub_compact,
|
|
|
|
CompactionOutputs& outputs);
|
|
|
|
void UpdateCompactionJobStats(
|
|
|
|
const InternalStats::CompactionStats& stats) const;
|
|
|
|
void RecordDroppedKeys(const CompactionIterationStats& c_iter_stats,
|
|
|
|
CompactionJobStats* compaction_job_stats = nullptr);
|
|
|
|
|
|
|
|
void UpdateCompactionInputStatsHelper(int* num_files, uint64_t* bytes_read,
|
|
|
|
int input_level);
|
|
|
|
|
|
|
|
void NotifyOnSubcompactionBegin(SubcompactionState* sub_compact);
|
|
|
|
|
|
|
|
void NotifyOnSubcompactionCompleted(SubcompactionState* sub_compact);
|
|
|
|
|
|
|
|
uint32_t job_id_;
|
|
|
|
|
|
|
|
// DBImpl state
|
|
|
|
const std::string& dbname_;
|
|
|
|
const std::string db_id_;
|
|
|
|
const std::string db_session_id_;
|
Introduce a new storage specific Env API (#5761)
Summary:
The current Env API encompasses both storage/file operations, as well as OS related operations. Most of the APIs return a Status, which does not have enough metadata about an error, such as whether its retry-able or not, scope (i.e fault domain) of the error etc., that may be required in order to properly handle a storage error. The file APIs also do not provide enough control over the IO SLA, such as timeout, prioritization, hinting about placement and redundancy etc.
This PR separates out the file/storage APIs from Env into a new FileSystem class. The APIs are updated to return an IOStatus with metadata about the error, as well as to take an IOOptions structure as input in order to allow more control over the IO.
The user can set both ```options.env``` and ```options.file_system``` to specify that RocksDB should use the former for OS related operations and the latter for storage operations. Internally, a ```CompositeEnvWrapper``` has been introduced that inherits from ```Env``` and redirects individual methods to either an ```Env``` implementation or the ```FileSystem``` as appropriate. When options are sanitized during ```DB::Open```, ```options.env``` is replaced with a newly allocated ```CompositeEnvWrapper``` instance if both env and file_system have been specified. This way, the rest of the RocksDB code can continue to function as before.
This PR also ports PosixEnv to the new API by splitting it into two - PosixEnv and PosixFileSystem. PosixEnv is defined as a sub-class of CompositeEnvWrapper, and threading/time functions are overridden with Posix specific implementations in order to avoid an extra level of indirection.
The ```CompositeEnvWrapper``` translates ```IOStatus``` return code to ```Status```, and sets the severity to ```kSoftError``` if the io_status is retryable. The error handling code in RocksDB can then recover the DB automatically.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5761
Differential Revision: D18868376
Pulled By: anand1976
fbshipit-source-id: 39efe18a162ea746fabac6360ff529baba48486f
5 years ago
|
|
|
const FileOptions file_options_;
|
|
|
|
|
|
|
|
Env* env_;
|
|
|
|
std::shared_ptr<IOTracer> io_tracer_;
|
|
|
|
FileSystemPtr fs_;
|
|
|
|
// env_option optimized for compaction table reads
|
Introduce a new storage specific Env API (#5761)
Summary:
The current Env API encompasses both storage/file operations, as well as OS related operations. Most of the APIs return a Status, which does not have enough metadata about an error, such as whether its retry-able or not, scope (i.e fault domain) of the error etc., that may be required in order to properly handle a storage error. The file APIs also do not provide enough control over the IO SLA, such as timeout, prioritization, hinting about placement and redundancy etc.
This PR separates out the file/storage APIs from Env into a new FileSystem class. The APIs are updated to return an IOStatus with metadata about the error, as well as to take an IOOptions structure as input in order to allow more control over the IO.
The user can set both ```options.env``` and ```options.file_system``` to specify that RocksDB should use the former for OS related operations and the latter for storage operations. Internally, a ```CompositeEnvWrapper``` has been introduced that inherits from ```Env``` and redirects individual methods to either an ```Env``` implementation or the ```FileSystem``` as appropriate. When options are sanitized during ```DB::Open```, ```options.env``` is replaced with a newly allocated ```CompositeEnvWrapper``` instance if both env and file_system have been specified. This way, the rest of the RocksDB code can continue to function as before.
This PR also ports PosixEnv to the new API by splitting it into two - PosixEnv and PosixFileSystem. PosixEnv is defined as a sub-class of CompositeEnvWrapper, and threading/time functions are overridden with Posix specific implementations in order to avoid an extra level of indirection.
The ```CompositeEnvWrapper``` translates ```IOStatus``` return code to ```Status```, and sets the severity to ```kSoftError``` if the io_status is retryable. The error handling code in RocksDB can then recover the DB automatically.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5761
Differential Revision: D18868376
Pulled By: anand1976
fbshipit-source-id: 39efe18a162ea746fabac6360ff529baba48486f
5 years ago
|
|
|
FileOptions file_options_for_read_;
|
|
|
|
VersionSet* versions_;
|
|
|
|
const std::atomic<bool>* shutting_down_;
|
|
|
|
const std::atomic<bool>& manual_compaction_canceled_;
|
|
|
|
FSDirectory* db_directory_;
|
|
|
|
FSDirectory* blob_output_directory_;
|
|
|
|
InstrumentedMutex* db_mutex_;
|
|
|
|
ErrorHandler* db_error_handler_;
|
|
|
|
// If there were two snapshots with seq numbers s1 and
|
|
|
|
// s2 and s1 < s2, and if we find two instances of a key k1 then lies
|
|
|
|
// entirely within s1 and s2, then the earlier version of k1 can be safely
|
|
|
|
// deleted because that version is not visible in any snapshot.
|
|
|
|
std::vector<SequenceNumber> existing_snapshots_;
|
|
|
|
|
|
|
|
// This is the earliest snapshot that could be used for write-conflict
|
|
|
|
// checking by a transaction. For any user-key newer than this snapshot, we
|
|
|
|
// should make sure not to remove evidence that a write occurred.
|
|
|
|
SequenceNumber earliest_write_conflict_snapshot_;
|
|
|
|
|
|
|
|
const SnapshotChecker* const snapshot_checker_;
|
|
|
|
|
CompactionIterator sees consistent view of which keys are committed (#9830)
Summary:
**This PR does not affect the functionality of `DB` and write-committed transactions.**
`CompactionIterator` uses `KeyCommitted(seq)` to determine if a key in the database is committed.
As the name 'write-committed' implies, if write-committed policy is used, a key exists in the database only if
it is committed. In fact, the implementation of `KeyCommitted()` is as follows:
```
inline bool KeyCommitted(SequenceNumber seq) {
// For non-txn-db and write-committed, snapshot_checker_ is always nullptr.
return snapshot_checker_ == nullptr ||
snapshot_checker_->CheckInSnapshot(seq, kMaxSequence) == SnapshotCheckerResult::kInSnapshot;
}
```
With that being said, we focus on write-prepared/write-unprepared transactions.
A few notes:
- A key can exist in the db even if it's uncommitted. Therefore, we rely on `snapshot_checker_` to determine data visibility. We also require that all writes go through transaction API instead of the raw `WriteBatch` + `Write`, thus at most one uncommitted version of one user key can exist in the database.
- `CompactionIterator` outputs a key as long as the key is uncommitted.
Due to the above reasons, it is possible that `CompactionIterator` decides to output an uncommitted key without
doing further checks on the key (`NextFromInput()`). By the time the key is being prepared for output, the key becomes
committed because the `snapshot_checker_(seq, kMaxSequence)` becomes true in the implementation of `KeyCommitted()`.
Then `CompactionIterator` will try to zero its sequence number and hit assertion error if the key is a tombstone.
To fix this issue, we should make the `CompactionIterator` see a consistent view of the input keys. Note that
for write-prepared/write-unprepared, the background flush/compaction jobs already take a "job snapshot" before starting
processing keys. The job snapshot is released only after the entire flush/compaction finishes. We can use this snapshot
to determine whether a key is committed or not with minor change to `KeyCommitted()`.
```
inline bool KeyCommitted(SequenceNumber sequence) {
// For non-txn-db and write-committed, snapshot_checker_ is always nullptr.
return snapshot_checker_ == nullptr ||
snapshot_checker_->CheckInSnapshot(sequence, job_snapshot_) ==
SnapshotCheckerResult::kInSnapshot;
}
```
As a result, whether a key is committed or not will remain a constant throughout compaction, causing no trouble
for `CompactionIterator`s assertions.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/9830
Test Plan: make check
Reviewed By: ltamasi
Differential Revision: D35561162
Pulled By: riversand963
fbshipit-source-id: 0e00d200c195240341cfe6d34cbc86798b315b9f
3 years ago
|
|
|
JobContext* job_context_;
|
|
|
|
|
|
|
|
std::shared_ptr<Cache> table_cache_;
|
|
|
|
|
Include bunch of more events into EventLogger
Summary:
Added these events:
* Recovery start, finish and also when recovery creates a file
* Trivial move
* Compaction start, finish and when compaction creates a file
* Flush start, finish
Also includes small fix to EventLogger
Also added option ROCKSDB_PRINT_EVENTS_TO_STDOUT which is useful when we debug things. I've spent far too much time chasing LOG files.
Still didn't get sst table properties in JSON. They are written very deeply into the stack. I'll address in separate diff.
TODO:
* Write specification. Let's first use this for a while and figure out what's good data to put here, too. After that we'll write spec
* Write tools that parse and analyze LOGs. This can be in python or go. Good intern task.
Test Plan: Ran db_bench with ROCKSDB_PRINT_EVENTS_TO_STDOUT. Here's the output: https://phabricator.fb.com/P19811976
Reviewers: sdong, yhchiang, rven, MarkCallaghan, kradhakrishnan, anthony
Reviewed By: anthony
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D37521
10 years ago
|
|
|
EventLogger* event_logger_;
|
|
|
|
|
|
|
|
bool paranoid_file_checks_;
|
Add options.compaction_measure_io_stats to print write I/O stats in compactions
Summary:
Add options.compaction_measure_io_stats to print out / pass to listener accumulated time spent on write calls. Example outputs in info logs:
2015/08/12-16:27:59.463944 7fd428bff700 (Original Log Time 2015/08/12-16:27:59.463922) EVENT_LOG_v1 {"time_micros": 1439422079463897, "job": 6, "event": "compaction_finished", "output_level": 1, "num_output_files": 4, "total_output_size": 6900525, "num_input_records": 111483, "num_output_records": 106877, "file_write_nanos": 15663206, "file_range_sync_nanos": 649588, "file_fsync_nanos": 349614797, "file_prepare_write_nanos": 1505812, "lsm_state": [2, 4, 0, 0, 0, 0, 0]}
Add two more counters in iostats_context.
Also add a parameter of db_bench.
Test Plan: Add a unit test. Also manually verify LOG outputs in db_bench
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D44115
9 years ago
|
|
|
bool measure_io_stats_;
|
|
|
|
// Stores the Slices that designate the boundaries for each subcompaction
|
|
|
|
std::vector<std::string> boundaries_;
|
|
|
|
Env::Priority thread_pri_;
|
|
|
|
std::string full_history_ts_low_;
|
|
|
|
std::string trim_ts_;
|
|
|
|
BlobFileCompletionCallback* blob_callback_;
|
|
|
|
|
|
|
|
uint64_t GetCompactionId(SubcompactionState* sub_compact) const;
|
Support subcmpct using reserved resources for round-robin priority (#10341)
Summary:
Earlier implementation of round-robin priority can only pick one file at a time and disallows parallel compactions within the same level. In this PR, round-robin compaction policy will expand towards more input files with respecting some additional constraints, which are summarized as follows:
* Constraint 1: We can only pick consecutive files
- Constraint 1a: When a file is being compacted (or some input files are being compacted after expanding), we cannot choose it and have to stop choosing more files
- Constraint 1b: When we reach the last file (with the largest keys), we cannot choose more files (the next file will be the first one with small keys)
* Constraint 2: We should ensure the total compaction bytes (including the overlapped files from the next level) is no more than `mutable_cf_options_.max_compaction_bytes`
* Constraint 3: We try our best to pick as many files as possible so that the post-compaction level size can be just less than `MaxBytesForLevel(start_level_)`
* Constraint 4: If trivial move is allowed, we reuse the logic of `TryNonL0TrivialMove()` instead of expanding files with Constraint 3
More details can be found in `LevelCompactionBuilder::SetupOtherFilesWithRoundRobinExpansion()`.
The above optimization accelerates the process of moving the compaction cursor, in which the write-amp can be further reduced. While a large compaction may lead to high write stall, we break this large compaction into several subcompactions **regardless of** the `max_subcompactions` limit. The number of subcompactions for round-robin compaction priority is determined through the following steps:
* Step 1: Initialized against `max_output_file_limit`, the number of input files in the start level, and also the range size limit `ranges.size()`
* Step 2: Call `AcquireSubcompactionResources()`when max subcompactions is not sufficient, but we may or may not obtain desired resources, additional number of resources is stored in `extra_num_subcompaction_threads_reserved_`). Subcompaction limit is changed and update `num_planned_subcompactions` with `GetSubcompactionLimit()`
* Step 3: Call `ShrinkSubcompactionResources()` to ensure extra resources can be released (extra resources may exist for round-robin compaction when the number of actual number of subcompactions is less than the number of planned subcompactions)
More details can be found in `CompactionJob::AcquireSubcompactionResources()`,`CompactionJob::ShrinkSubcompactionResources()`, and `CompactionJob::ReleaseSubcompactionResources()`.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/10341
Test Plan: Add `CompactionPriMultipleFilesRoundRobin[1-3]` unit test in `compaction_picker_test.cc` and `RoundRobinSubcompactionsAgainstResources.SubcompactionsUsingResources/[0-4]`, `RoundRobinSubcompactionsAgainstPressureToken.PressureTokenTest/[0-1]` in `db_compaction_test.cc`
Reviewed By: ajkr, hx235
Differential Revision: D37792644
Pulled By: littlepig2013
fbshipit-source-id: 7fecb7c4ffd97b34bbf6e3b760b2c35a772a0657
2 years ago
|
|
|
// Stores the number of reserved threads in shared env_ for the number of
|
|
|
|
// extra subcompaction in kRoundRobin compaction priority
|
|
|
|
int extra_num_subcompaction_threads_reserved_;
|
|
|
|
|
|
|
|
// Stores the pointer to bg_compaction_scheduled_,
|
|
|
|
// bg_bottom_compaction_scheduled_ in DBImpl. Mutex is required when accessing
|
|
|
|
// or updating it.
|
|
|
|
int* bg_compaction_scheduled_;
|
|
|
|
int* bg_bottom_compaction_scheduled_;
|
|
|
|
|
|
|
|
// Stores the sequence number to time mapping gathered from all input files
|
|
|
|
// it also collects the smallest_seqno -> oldest_ancester_time from the SST.
|
|
|
|
SeqnoToTimeMapping seqno_time_mapping_;
|
|
|
|
|
|
|
|
// Minimal sequence number for preserving the time information. The time info
|
|
|
|
// older than this sequence number won't be preserved after the compaction and
|
|
|
|
// if it's bottommost compaction, the seq num will be zeroed out.
|
|
|
|
SequenceNumber preserve_time_min_seqno_ = kMaxSequenceNumber;
|
|
|
|
|
|
|
|
// Minimal sequence number to preclude the data from the last level. If the
|
|
|
|
// key has bigger (newer) sequence number than this, it will be precluded from
|
|
|
|
// the last level (output to penultimate level).
|
|
|
|
SequenceNumber preclude_last_level_min_seqno_ = kMaxSequenceNumber;
|
|
|
|
|
|
|
|
// Get table file name in where it's outputting to, which should also be in
|
|
|
|
// `output_directory_`.
|
|
|
|
virtual std::string GetTableFileName(uint64_t file_number);
|
Set Write rate limiter priority dynamically and pass it to FS (#9988)
Summary:
### Context:
Background compactions and flush generate large reads and writes, and can be long running, especially for universal compaction. In some cases, this can impact foreground reads and writes by users.
From the RocksDB perspective, there can be two kinds of rate limiters, the internal (native) one and the external one.
- The internal (native) rate limiter is introduced in [the wiki](https://github.com/facebook/rocksdb/wiki/Rate-Limiter). Currently, only IO_LOW and IO_HIGH are used and they are set statically.
- For the external rate limiter, in FSWritableFile functions, IOOptions is open for end users to set and get rate_limiter_priority for their own rate limiter. Currently, RocksDB doesn’t pass the rate_limiter_priority through IOOptions to the file system.
### Solution
During the User Read, Flush write, Compaction read/write, the WriteController is used to determine whether DB writes are stalled or slowed down. The rate limiter priority (Env::IOPriority) can be determined accordingly. We decided to always pass the priority in IOOptions. What the file system does with it should be a contract between the user and the file system. We would like to set the rate limiter priority at file level, since the Flush/Compaction job level may be too coarse with multiple files and block IO level is too granular.
**This PR is for the Write path.** The **Write:** dynamic priority for different state are listed as follows:
| State | Normal | Delayed | Stalled |
| ----- | ------ | ------- | ------- |
| Flush | IO_HIGH | IO_USER | IO_USER |
| Compaction | IO_LOW | IO_USER | IO_USER |
Flush and Compaction writes share the same call path through BlockBaseTableWriter, WritableFileWriter, and FSWritableFile. When a new FSWritableFile object is created, its io_priority_ can be set dynamically based on the state of the WriteController. In WritableFileWriter, before the call sites of FSWritableFile functions, WritableFileWriter::DecideRateLimiterPriority() determines the rate_limiter_priority. The options (IOOptions) argument of FSWritableFile functions will be updated with the rate_limiter_priority.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/9988
Test Plan: Add unit tests.
Reviewed By: anand1976
Differential Revision: D36395159
Pulled By: gitbw95
fbshipit-source-id: a7c82fc29759139a1a07ec46c37dbf7e753474cf
3 years ago
|
|
|
// The rate limiter priority (io_priority) is determined dynamically here.
|
|
|
|
// The Compaction Read and Write priorities are the same for different
|
|
|
|
// scenarios, such as write stalled.
|
|
|
|
Env::IOPriority GetRateLimiterPriority();
|
|
|
|
};
|
|
|
|
|
|
|
|
// CompactionServiceInput is used the pass compaction information between two
|
|
|
|
// db instances. It contains the information needed to do a compaction. It
|
|
|
|
// doesn't contain the LSM tree information, which is passed though MANIFEST
|
|
|
|
// file.
|
|
|
|
struct CompactionServiceInput {
|
|
|
|
ColumnFamilyDescriptor column_family;
|
|
|
|
|
|
|
|
DBOptions db_options;
|
|
|
|
|
|
|
|
std::vector<SequenceNumber> snapshots;
|
|
|
|
|
|
|
|
// SST files for compaction, it should already be expended to include all the
|
|
|
|
// files needed for this compaction, for both input level files and output
|
|
|
|
// level files.
|
|
|
|
std::vector<std::string> input_files;
|
|
|
|
int output_level;
|
|
|
|
|
|
|
|
// db_id is used to generate unique id of sst on the remote compactor
|
|
|
|
std::string db_id;
|
|
|
|
|
|
|
|
// information for subcompaction
|
|
|
|
bool has_begin = false;
|
|
|
|
std::string begin;
|
|
|
|
bool has_end = false;
|
|
|
|
std::string end;
|
|
|
|
|
|
|
|
// serialization interface to read and write the object
|
|
|
|
static Status Read(const std::string& data_str, CompactionServiceInput* obj);
|
|
|
|
Status Write(std::string* output);
|
|
|
|
|
|
|
|
// Initialize a dummy ColumnFamilyDescriptor
|
|
|
|
CompactionServiceInput() : column_family("", ColumnFamilyOptions()) {}
|
|
|
|
|
|
|
|
#ifndef NDEBUG
|
|
|
|
bool TEST_Equals(CompactionServiceInput* other);
|
|
|
|
bool TEST_Equals(CompactionServiceInput* other, std::string* mismatch);
|
|
|
|
#endif // NDEBUG
|
|
|
|
};
|
|
|
|
|
|
|
|
// CompactionServiceOutputFile is the metadata for the output SST file
|
|
|
|
struct CompactionServiceOutputFile {
|
|
|
|
std::string file_name;
|
|
|
|
SequenceNumber smallest_seqno;
|
|
|
|
SequenceNumber largest_seqno;
|
|
|
|
std::string smallest_internal_key;
|
|
|
|
std::string largest_internal_key;
|
|
|
|
uint64_t oldest_ancester_time;
|
|
|
|
uint64_t file_creation_time;
|
Sort L0 files by newly introduced epoch_num (#10922)
Summary:
**Context:**
Sorting L0 files by `largest_seqno` has at least two inconvenience:
- File ingestion and compaction involving ingested files can create files of overlapping seqno range with the existing files. `force_consistency_check=true` will catch such overlap seqno range even those harmless overlap.
- For example, consider the following sequence of events ("key@n" indicates key at seqno "n")
- insert k1@1 to memtable m1
- ingest file s1 with k2@2, ingest file s2 with k3@3
- insert k4@4 to m1
- compact files s1, s2 and result in new file s3 of seqno range [2, 3]
- flush m1 and result in new file s4 of seqno range [1, 4]. And `force_consistency_check=true` will think s4 and s3 has file reordering corruption that might cause retuning an old value of k1
- However such caught corruption is a false positive since s1, s2 will not have overlapped keys with k1 or whatever inserted into m1 before ingest file s1 by the requirement of file ingestion (otherwise the m1 will be flushed first before any of the file ingestion completes). Therefore there in fact isn't any file reordering corruption.
- Single delete can decrease a file's largest seqno and ordering by `largest_seqno` can introduce a wrong ordering hence file reordering corruption
- For example, consider the following sequence of events ("key@n" indicates key at seqno "n", Credit to ajkr for this example)
- an existing SST s1 contains only k1@1
- insert k1@2 to memtable m1
- ingest file s2 with k3@3, ingest file s3 with k4@4
- insert single delete k5@5 in m1
- flush m1 and result in new file s4 of seqno range [2, 5]
- compact s1, s2, s3 and result in new file s5 of seqno range [1, 4]
- compact s4 and result in new file s6 of seqno range [2] due to single delete
- By the last step, we have file ordering by largest seqno (">" means "newer") : s5 > s6 while s6 contains a newer version of the k1's value (i.e, k1@2) than s5, which is a real reordering corruption. While this can be caught by `force_consistency_check=true`, there isn't a good way to prevent this from happening if ordering by `largest_seqno`
Therefore, we are redesigning the sorting criteria of L0 files and avoid above inconvenience. Credit to ajkr , we now introduce `epoch_num` which describes the order of a file being flushed or ingested/imported (compaction output file will has the minimum `epoch_num` among input files'). This will avoid the above inconvenience in the following ways:
- In the first case above, there will no longer be overlap seqno range check in `force_consistency_check=true` but `epoch_number` ordering check. This will result in file ordering s1 < s2 < s4 (pre-compaction) and s3 < s4 (post-compaction) which won't trigger false positive corruption. See test class `DBCompactionTestL0FilesMisorderCorruption*` for more.
- In the second case above, this will result in file ordering s1 < s2 < s3 < s4 (pre-compacting s1, s2, s3), s5 < s4 (post-compacting s1, s2, s3), s5 < s6 (post-compacting s4), which are correct file ordering without causing any corruption.
**Summary:**
- Introduce `epoch_number` stored per `ColumnFamilyData` and sort CF's L0 files by their assigned `epoch_number` instead of `largest_seqno`.
- `epoch_number` is increased and assigned upon `VersionEdit::AddFile()` for flush (or similarly for WriteLevel0TableForRecovery) and file ingestion (except for allow_behind_true, which will always get assigned as the `kReservedEpochNumberForFileIngestedBehind`)
- Compaction output file is assigned with the minimum `epoch_number` among input files'
- Refit level: reuse refitted file's epoch_number
- Other paths needing `epoch_number` treatment:
- Import column families: reuse file's epoch_number if exists. If not, assign one based on `NewestFirstBySeqNo`
- Repair: reuse file's epoch_number if exists. If not, assign one based on `NewestFirstBySeqNo`.
- Assigning new epoch_number to a file and adding this file to LSM tree should be atomic. This is guaranteed by us assigning epoch_number right upon `VersionEdit::AddFile()` where this version edit will be apply to LSM tree shape right after by holding the db mutex (e.g, flush, file ingestion, import column family) or by there is only 1 ongoing edit per CF (e.g, WriteLevel0TableForRecovery, Repair).
- Assigning the minimum input epoch number to compaction output file won't misorder L0 files (even through later `Refit(target_level=0)`). It's due to for every key "k" in the input range, a legit compaction will cover a continuous epoch number range of that key. As long as we assign the key "k" the minimum input epoch number, it won't become newer or older than the versions of this key that aren't included in this compaction hence no misorder.
- Persist `epoch_number` of each file in manifest and recover `epoch_number` on db recovery
- Backward compatibility with old db without `epoch_number` support is guaranteed by assigning `epoch_number` to recovered files by `NewestFirstBySeqno` order. See `VersionStorageInfo::RecoverEpochNumbers()` for more
- Forward compatibility with manifest is guaranteed by flexibility of `NewFileCustomTag`
- Replace `force_consistent_check` on L0 with `epoch_number` and remove false positive check like case 1 with `largest_seqno` above
- Due to backward compatibility issue, we might encounter files with missing epoch number at the beginning of db recovery. We will still use old L0 sorting mechanism (`NewestFirstBySeqno`) to check/sort them till we infer their epoch number. See usages of `EpochNumberRequirement`.
- Remove fix https://github.com/facebook/rocksdb/pull/5958#issue-511150930 and their outdated tests to file reordering corruption because such fix can be replaced by this PR.
- Misc:
- update existing tests with `epoch_number` so make check will pass
- update https://github.com/facebook/rocksdb/pull/5958#issue-511150930 tests to verify corruption is fixed using `epoch_number` and cover universal/fifo compaction/CompactRange/CompactFile cases
- assert db_mutex is held for a few places before calling ColumnFamilyData::NewEpochNumber()
Pull Request resolved: https://github.com/facebook/rocksdb/pull/10922
Test Plan:
- `make check`
- New unit tests under `db/db_compaction_test.cc`, `db/db_test2.cc`, `db/version_builder_test.cc`, `db/repair_test.cc`
- Updated tests (i.e, `DBCompactionTestL0FilesMisorderCorruption*`) under https://github.com/facebook/rocksdb/pull/5958#issue-511150930
- [Ongoing] Compatibility test: manually run https://github.com/ajkr/rocksdb/commit/36a5686ec012f35a4371e409aa85c404ca1c210d (with file ingestion off for running the `.orig` binary to prevent this bug affecting upgrade/downgrade formality checking) for 1 hour on `simple black/white box`, `cf_consistency/txn/enable_ts with whitebox + test_best_efforts_recovery with blackbox`
- [Ongoing] normal db stress test
- [Ongoing] db stress test with aggressive value https://github.com/facebook/rocksdb/pull/10761
Reviewed By: ajkr
Differential Revision: D41063187
Pulled By: hx235
fbshipit-source-id: 826cb23455de7beaabe2d16c57682a82733a32a9
2 years ago
|
|
|
uint64_t epoch_number;
|
|
|
|
uint64_t paranoid_hash;
|
|
|
|
bool marked_for_compaction;
|
|
|
|
UniqueId64x2 unique_id;
|
|
|
|
|
|
|
|
CompactionServiceOutputFile() = default;
|
|
|
|
CompactionServiceOutputFile(
|
|
|
|
const std::string& name, SequenceNumber smallest, SequenceNumber largest,
|
|
|
|
std::string _smallest_internal_key, std::string _largest_internal_key,
|
|
|
|
uint64_t _oldest_ancester_time, uint64_t _file_creation_time,
|
Sort L0 files by newly introduced epoch_num (#10922)
Summary:
**Context:**
Sorting L0 files by `largest_seqno` has at least two inconvenience:
- File ingestion and compaction involving ingested files can create files of overlapping seqno range with the existing files. `force_consistency_check=true` will catch such overlap seqno range even those harmless overlap.
- For example, consider the following sequence of events ("key@n" indicates key at seqno "n")
- insert k1@1 to memtable m1
- ingest file s1 with k2@2, ingest file s2 with k3@3
- insert k4@4 to m1
- compact files s1, s2 and result in new file s3 of seqno range [2, 3]
- flush m1 and result in new file s4 of seqno range [1, 4]. And `force_consistency_check=true` will think s4 and s3 has file reordering corruption that might cause retuning an old value of k1
- However such caught corruption is a false positive since s1, s2 will not have overlapped keys with k1 or whatever inserted into m1 before ingest file s1 by the requirement of file ingestion (otherwise the m1 will be flushed first before any of the file ingestion completes). Therefore there in fact isn't any file reordering corruption.
- Single delete can decrease a file's largest seqno and ordering by `largest_seqno` can introduce a wrong ordering hence file reordering corruption
- For example, consider the following sequence of events ("key@n" indicates key at seqno "n", Credit to ajkr for this example)
- an existing SST s1 contains only k1@1
- insert k1@2 to memtable m1
- ingest file s2 with k3@3, ingest file s3 with k4@4
- insert single delete k5@5 in m1
- flush m1 and result in new file s4 of seqno range [2, 5]
- compact s1, s2, s3 and result in new file s5 of seqno range [1, 4]
- compact s4 and result in new file s6 of seqno range [2] due to single delete
- By the last step, we have file ordering by largest seqno (">" means "newer") : s5 > s6 while s6 contains a newer version of the k1's value (i.e, k1@2) than s5, which is a real reordering corruption. While this can be caught by `force_consistency_check=true`, there isn't a good way to prevent this from happening if ordering by `largest_seqno`
Therefore, we are redesigning the sorting criteria of L0 files and avoid above inconvenience. Credit to ajkr , we now introduce `epoch_num` which describes the order of a file being flushed or ingested/imported (compaction output file will has the minimum `epoch_num` among input files'). This will avoid the above inconvenience in the following ways:
- In the first case above, there will no longer be overlap seqno range check in `force_consistency_check=true` but `epoch_number` ordering check. This will result in file ordering s1 < s2 < s4 (pre-compaction) and s3 < s4 (post-compaction) which won't trigger false positive corruption. See test class `DBCompactionTestL0FilesMisorderCorruption*` for more.
- In the second case above, this will result in file ordering s1 < s2 < s3 < s4 (pre-compacting s1, s2, s3), s5 < s4 (post-compacting s1, s2, s3), s5 < s6 (post-compacting s4), which are correct file ordering without causing any corruption.
**Summary:**
- Introduce `epoch_number` stored per `ColumnFamilyData` and sort CF's L0 files by their assigned `epoch_number` instead of `largest_seqno`.
- `epoch_number` is increased and assigned upon `VersionEdit::AddFile()` for flush (or similarly for WriteLevel0TableForRecovery) and file ingestion (except for allow_behind_true, which will always get assigned as the `kReservedEpochNumberForFileIngestedBehind`)
- Compaction output file is assigned with the minimum `epoch_number` among input files'
- Refit level: reuse refitted file's epoch_number
- Other paths needing `epoch_number` treatment:
- Import column families: reuse file's epoch_number if exists. If not, assign one based on `NewestFirstBySeqNo`
- Repair: reuse file's epoch_number if exists. If not, assign one based on `NewestFirstBySeqNo`.
- Assigning new epoch_number to a file and adding this file to LSM tree should be atomic. This is guaranteed by us assigning epoch_number right upon `VersionEdit::AddFile()` where this version edit will be apply to LSM tree shape right after by holding the db mutex (e.g, flush, file ingestion, import column family) or by there is only 1 ongoing edit per CF (e.g, WriteLevel0TableForRecovery, Repair).
- Assigning the minimum input epoch number to compaction output file won't misorder L0 files (even through later `Refit(target_level=0)`). It's due to for every key "k" in the input range, a legit compaction will cover a continuous epoch number range of that key. As long as we assign the key "k" the minimum input epoch number, it won't become newer or older than the versions of this key that aren't included in this compaction hence no misorder.
- Persist `epoch_number` of each file in manifest and recover `epoch_number` on db recovery
- Backward compatibility with old db without `epoch_number` support is guaranteed by assigning `epoch_number` to recovered files by `NewestFirstBySeqno` order. See `VersionStorageInfo::RecoverEpochNumbers()` for more
- Forward compatibility with manifest is guaranteed by flexibility of `NewFileCustomTag`
- Replace `force_consistent_check` on L0 with `epoch_number` and remove false positive check like case 1 with `largest_seqno` above
- Due to backward compatibility issue, we might encounter files with missing epoch number at the beginning of db recovery. We will still use old L0 sorting mechanism (`NewestFirstBySeqno`) to check/sort them till we infer their epoch number. See usages of `EpochNumberRequirement`.
- Remove fix https://github.com/facebook/rocksdb/pull/5958#issue-511150930 and their outdated tests to file reordering corruption because such fix can be replaced by this PR.
- Misc:
- update existing tests with `epoch_number` so make check will pass
- update https://github.com/facebook/rocksdb/pull/5958#issue-511150930 tests to verify corruption is fixed using `epoch_number` and cover universal/fifo compaction/CompactRange/CompactFile cases
- assert db_mutex is held for a few places before calling ColumnFamilyData::NewEpochNumber()
Pull Request resolved: https://github.com/facebook/rocksdb/pull/10922
Test Plan:
- `make check`
- New unit tests under `db/db_compaction_test.cc`, `db/db_test2.cc`, `db/version_builder_test.cc`, `db/repair_test.cc`
- Updated tests (i.e, `DBCompactionTestL0FilesMisorderCorruption*`) under https://github.com/facebook/rocksdb/pull/5958#issue-511150930
- [Ongoing] Compatibility test: manually run https://github.com/ajkr/rocksdb/commit/36a5686ec012f35a4371e409aa85c404ca1c210d (with file ingestion off for running the `.orig` binary to prevent this bug affecting upgrade/downgrade formality checking) for 1 hour on `simple black/white box`, `cf_consistency/txn/enable_ts with whitebox + test_best_efforts_recovery with blackbox`
- [Ongoing] normal db stress test
- [Ongoing] db stress test with aggressive value https://github.com/facebook/rocksdb/pull/10761
Reviewed By: ajkr
Differential Revision: D41063187
Pulled By: hx235
fbshipit-source-id: 826cb23455de7beaabe2d16c57682a82733a32a9
2 years ago
|
|
|
uint64_t _epoch_number, uint64_t _paranoid_hash,
|
|
|
|
bool _marked_for_compaction, UniqueId64x2 _unique_id)
|
|
|
|
: file_name(name),
|
|
|
|
smallest_seqno(smallest),
|
|
|
|
largest_seqno(largest),
|
|
|
|
smallest_internal_key(std::move(_smallest_internal_key)),
|
|
|
|
largest_internal_key(std::move(_largest_internal_key)),
|
|
|
|
oldest_ancester_time(_oldest_ancester_time),
|
|
|
|
file_creation_time(_file_creation_time),
|
Sort L0 files by newly introduced epoch_num (#10922)
Summary:
**Context:**
Sorting L0 files by `largest_seqno` has at least two inconvenience:
- File ingestion and compaction involving ingested files can create files of overlapping seqno range with the existing files. `force_consistency_check=true` will catch such overlap seqno range even those harmless overlap.
- For example, consider the following sequence of events ("key@n" indicates key at seqno "n")
- insert k1@1 to memtable m1
- ingest file s1 with k2@2, ingest file s2 with k3@3
- insert k4@4 to m1
- compact files s1, s2 and result in new file s3 of seqno range [2, 3]
- flush m1 and result in new file s4 of seqno range [1, 4]. And `force_consistency_check=true` will think s4 and s3 has file reordering corruption that might cause retuning an old value of k1
- However such caught corruption is a false positive since s1, s2 will not have overlapped keys with k1 or whatever inserted into m1 before ingest file s1 by the requirement of file ingestion (otherwise the m1 will be flushed first before any of the file ingestion completes). Therefore there in fact isn't any file reordering corruption.
- Single delete can decrease a file's largest seqno and ordering by `largest_seqno` can introduce a wrong ordering hence file reordering corruption
- For example, consider the following sequence of events ("key@n" indicates key at seqno "n", Credit to ajkr for this example)
- an existing SST s1 contains only k1@1
- insert k1@2 to memtable m1
- ingest file s2 with k3@3, ingest file s3 with k4@4
- insert single delete k5@5 in m1
- flush m1 and result in new file s4 of seqno range [2, 5]
- compact s1, s2, s3 and result in new file s5 of seqno range [1, 4]
- compact s4 and result in new file s6 of seqno range [2] due to single delete
- By the last step, we have file ordering by largest seqno (">" means "newer") : s5 > s6 while s6 contains a newer version of the k1's value (i.e, k1@2) than s5, which is a real reordering corruption. While this can be caught by `force_consistency_check=true`, there isn't a good way to prevent this from happening if ordering by `largest_seqno`
Therefore, we are redesigning the sorting criteria of L0 files and avoid above inconvenience. Credit to ajkr , we now introduce `epoch_num` which describes the order of a file being flushed or ingested/imported (compaction output file will has the minimum `epoch_num` among input files'). This will avoid the above inconvenience in the following ways:
- In the first case above, there will no longer be overlap seqno range check in `force_consistency_check=true` but `epoch_number` ordering check. This will result in file ordering s1 < s2 < s4 (pre-compaction) and s3 < s4 (post-compaction) which won't trigger false positive corruption. See test class `DBCompactionTestL0FilesMisorderCorruption*` for more.
- In the second case above, this will result in file ordering s1 < s2 < s3 < s4 (pre-compacting s1, s2, s3), s5 < s4 (post-compacting s1, s2, s3), s5 < s6 (post-compacting s4), which are correct file ordering without causing any corruption.
**Summary:**
- Introduce `epoch_number` stored per `ColumnFamilyData` and sort CF's L0 files by their assigned `epoch_number` instead of `largest_seqno`.
- `epoch_number` is increased and assigned upon `VersionEdit::AddFile()` for flush (or similarly for WriteLevel0TableForRecovery) and file ingestion (except for allow_behind_true, which will always get assigned as the `kReservedEpochNumberForFileIngestedBehind`)
- Compaction output file is assigned with the minimum `epoch_number` among input files'
- Refit level: reuse refitted file's epoch_number
- Other paths needing `epoch_number` treatment:
- Import column families: reuse file's epoch_number if exists. If not, assign one based on `NewestFirstBySeqNo`
- Repair: reuse file's epoch_number if exists. If not, assign one based on `NewestFirstBySeqNo`.
- Assigning new epoch_number to a file and adding this file to LSM tree should be atomic. This is guaranteed by us assigning epoch_number right upon `VersionEdit::AddFile()` where this version edit will be apply to LSM tree shape right after by holding the db mutex (e.g, flush, file ingestion, import column family) or by there is only 1 ongoing edit per CF (e.g, WriteLevel0TableForRecovery, Repair).
- Assigning the minimum input epoch number to compaction output file won't misorder L0 files (even through later `Refit(target_level=0)`). It's due to for every key "k" in the input range, a legit compaction will cover a continuous epoch number range of that key. As long as we assign the key "k" the minimum input epoch number, it won't become newer or older than the versions of this key that aren't included in this compaction hence no misorder.
- Persist `epoch_number` of each file in manifest and recover `epoch_number` on db recovery
- Backward compatibility with old db without `epoch_number` support is guaranteed by assigning `epoch_number` to recovered files by `NewestFirstBySeqno` order. See `VersionStorageInfo::RecoverEpochNumbers()` for more
- Forward compatibility with manifest is guaranteed by flexibility of `NewFileCustomTag`
- Replace `force_consistent_check` on L0 with `epoch_number` and remove false positive check like case 1 with `largest_seqno` above
- Due to backward compatibility issue, we might encounter files with missing epoch number at the beginning of db recovery. We will still use old L0 sorting mechanism (`NewestFirstBySeqno`) to check/sort them till we infer their epoch number. See usages of `EpochNumberRequirement`.
- Remove fix https://github.com/facebook/rocksdb/pull/5958#issue-511150930 and their outdated tests to file reordering corruption because such fix can be replaced by this PR.
- Misc:
- update existing tests with `epoch_number` so make check will pass
- update https://github.com/facebook/rocksdb/pull/5958#issue-511150930 tests to verify corruption is fixed using `epoch_number` and cover universal/fifo compaction/CompactRange/CompactFile cases
- assert db_mutex is held for a few places before calling ColumnFamilyData::NewEpochNumber()
Pull Request resolved: https://github.com/facebook/rocksdb/pull/10922
Test Plan:
- `make check`
- New unit tests under `db/db_compaction_test.cc`, `db/db_test2.cc`, `db/version_builder_test.cc`, `db/repair_test.cc`
- Updated tests (i.e, `DBCompactionTestL0FilesMisorderCorruption*`) under https://github.com/facebook/rocksdb/pull/5958#issue-511150930
- [Ongoing] Compatibility test: manually run https://github.com/ajkr/rocksdb/commit/36a5686ec012f35a4371e409aa85c404ca1c210d (with file ingestion off for running the `.orig` binary to prevent this bug affecting upgrade/downgrade formality checking) for 1 hour on `simple black/white box`, `cf_consistency/txn/enable_ts with whitebox + test_best_efforts_recovery with blackbox`
- [Ongoing] normal db stress test
- [Ongoing] db stress test with aggressive value https://github.com/facebook/rocksdb/pull/10761
Reviewed By: ajkr
Differential Revision: D41063187
Pulled By: hx235
fbshipit-source-id: 826cb23455de7beaabe2d16c57682a82733a32a9
2 years ago
|
|
|
epoch_number(_epoch_number),
|
|
|
|
paranoid_hash(_paranoid_hash),
|
|
|
|
marked_for_compaction(_marked_for_compaction),
|
|
|
|
unique_id(std::move(_unique_id)) {}
|
|
|
|
};
|
|
|
|
|
|
|
|
// CompactionServiceResult contains the compaction result from a different db
|
|
|
|
// instance, with these information, the primary db instance with write
|
|
|
|
// permission is able to install the result to the DB.
|
|
|
|
struct CompactionServiceResult {
|
|
|
|
Status status;
|
|
|
|
std::vector<CompactionServiceOutputFile> output_files;
|
|
|
|
int output_level;
|
|
|
|
|
|
|
|
// location of the output files
|
|
|
|
std::string output_path;
|
|
|
|
|
|
|
|
// some statistics about the compaction
|
|
|
|
uint64_t num_output_records = 0;
|
|
|
|
uint64_t total_bytes = 0;
|
|
|
|
uint64_t bytes_read = 0;
|
|
|
|
uint64_t bytes_written = 0;
|
|
|
|
CompactionJobStats stats;
|
|
|
|
|
|
|
|
// serialization interface to read and write the object
|
|
|
|
static Status Read(const std::string& data_str, CompactionServiceResult* obj);
|
|
|
|
Status Write(std::string* output);
|
|
|
|
|
|
|
|
#ifndef NDEBUG
|
|
|
|
bool TEST_Equals(CompactionServiceResult* other);
|
|
|
|
bool TEST_Equals(CompactionServiceResult* other, std::string* mismatch);
|
|
|
|
#endif // NDEBUG
|
|
|
|
};
|
|
|
|
|
|
|
|
// CompactionServiceCompactionJob is an read-only compaction job, it takes
|
|
|
|
// input information from `compaction_service_input` and put result information
|
|
|
|
// in `compaction_service_result`, the SST files are generated to `output_path`.
|
|
|
|
class CompactionServiceCompactionJob : private CompactionJob {
|
|
|
|
public:
|
|
|
|
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);
|
|
|
|
|
|
|
|
// Run the compaction in current thread and return the result
|
|
|
|
Status Run();
|
|
|
|
|
|
|
|
void CleanupCompaction();
|
|
|
|
|
|
|
|
IOStatus io_status() const { return CompactionJob::io_status(); }
|
|
|
|
|
|
|
|
protected:
|
|
|
|
void RecordCompactionIOStats() override;
|
|
|
|
|
|
|
|
private:
|
|
|
|
// Get table file name in output_path
|
|
|
|
std::string GetTableFileName(uint64_t file_number) override;
|
|
|
|
// Specific the compaction output path, otherwise it uses default DB path
|
|
|
|
const std::string output_path_;
|
|
|
|
|
|
|
|
// Compaction job input
|
|
|
|
const CompactionServiceInput& compaction_input_;
|
|
|
|
|
|
|
|
// Compaction job result
|
|
|
|
CompactionServiceResult* compaction_result_;
|
|
|
|
};
|
|
|
|
|
|
|
|
} // namespace ROCKSDB_NAMESPACE
|