rocksdb/db/flush_job.cc

999 lines
39 KiB
C++
Raw Normal View History

// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
// This source code is licensed under both the GPLv2 (found in the
// COPYING file in the root directory) and Apache 2.0 License
// (found in the LICENSE.Apache file in the root directory).
//
// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
// Use of this source code is governed by a BSD-style license that can be
// found in the LICENSE file. See the AUTHORS file for names of contributors.
#include "db/flush_job.h"
#include <cinttypes>
#include <algorithm>
#include <vector>
#include "db/builder.h"
#include "db/db_iter.h"
#include "db/dbformat.h"
#include "db/event_helpers.h"
#include "db/log_reader.h"
#include "db/log_writer.h"
#include "db/memtable.h"
#include "db/memtable_list.h"
#include "db/merge_context.h"
#include "db/range_tombstone_fragmenter.h"
#include "db/version_set.h"
#include "file/file_util.h"
#include "file/filename.h"
#include "logging/event_logger.h"
#include "logging/log_buffer.h"
#include "logging/logging.h"
#include "monitoring/iostats_context_imp.h"
#include "monitoring/perf_context_imp.h"
#include "monitoring/thread_status_util.h"
#include "port/port.h"
#include "rocksdb/db.h"
#include "rocksdb/env.h"
#include "rocksdb/statistics.h"
#include "rocksdb/status.h"
#include "rocksdb/table.h"
#include "table/merging_iterator.h"
#include "table/table_builder.h"
#include "table/two_level_iterator.h"
#include "test_util/sync_point.h"
#include "util/coding.h"
#include "util/mutexlock.h"
#include "util/stop_watch.h"
namespace ROCKSDB_NAMESPACE {
const char* GetFlushReasonString (FlushReason flush_reason) {
switch (flush_reason) {
FlushReason improvement Summary: Right now flush reason "SuperVersion Change" covers a few different scenarios which is a bit vague. For example, the following db_bench job should trigger "Write Buffer Full" > $ TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=fillrandom -write_buffer_size=1048576 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304 $ grep 'flush_reason' /dev/shm/dbbench/LOG ... 2018/03/06-17:30:42.543638 7f2773b99700 EVENT_LOG_v1 {"time_micros": 1520386242543634, "job": 192, "event": "flush_started", "num_memtables": 1, "num_entries": 7006, "num_deletes": 0, "memory_usage": 1018024, "flush_reason": "SuperVersion Change"} 2018/03/06-17:30:42.569541 7f2773b99700 EVENT_LOG_v1 {"time_micros": 1520386242569536, "job": 193, "event": "flush_started", "num_memtables": 1, "num_entries": 7006, "num_deletes": 0, "memory_usage": 1018104, "flush_reason": "SuperVersion Change"} 2018/03/06-17:30:42.596396 7f2773b99700 EVENT_LOG_v1 {"time_micros": 1520386242596392, "job": 194, "event": "flush_started", "num_memtables": 1, "num_entries": 7008, "num_deletes": 0, "memory_usage": 1018048, "flush_reason": "SuperVersion Change"} 2018/03/06-17:30:42.622444 7f2773b99700 EVENT_LOG_v1 {"time_micros": 1520386242622440, "job": 195, "event": "flush_started", "num_memtables": 1, "num_entries": 7006, "num_deletes": 0, "memory_usage": 1018104, "flush_reason": "SuperVersion Change"} With the fix: > 2018/03/19-14:40:02.341451 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602341444, "job": 98, "event": "flush_started", "num_memtables": 1, "num_entries": 7009, "num_deletes": 0, "memory_usage": 1018008, "flush_reason": "Write Buffer Full"} 2018/03/19-14:40:02.379655 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602379642, "job": 100, "event": "flush_started", "num_memtables": 1, "num_entries": 7006, "num_deletes": 0, "memory_usage": 1018016, "flush_reason": "Write Buffer Full"} 2018/03/19-14:40:02.418479 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602418474, "job": 101, "event": "flush_started", "num_memtables": 1, "num_entries": 7009, "num_deletes": 0, "memory_usage": 1018104, "flush_reason": "Write Buffer Full"} 2018/03/19-14:40:02.455084 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602455079, "job": 102, "event": "flush_started", "num_memtables": 1, "num_entries": 7009, "num_deletes": 0, "memory_usage": 1018048, "flush_reason": "Write Buffer Full"} 2018/03/19-14:40:02.492293 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602492288, "job": 104, "event": "flush_started", "num_memtables": 1, "num_entries": 7007, "num_deletes": 0, "memory_usage": 1018056, "flush_reason": "Write Buffer Full"} 2018/03/19-14:40:02.528720 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602528715, "job": 105, "event": "flush_started", "num_memtables": 1, "num_entries": 7006, "num_deletes": 0, "memory_usage": 1018104, "flush_reason": "Write Buffer Full"} 2018/03/19-14:40:02.566255 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602566238, "job": 107, "event": "flush_started", "num_memtables": 1, "num_entries": 7009, "num_deletes": 0, "memory_usage": 1018112, "flush_reason": "Write Buffer Full"} Closes https://github.com/facebook/rocksdb/pull/3627 Differential Revision: D7328772 Pulled By: miasantreble fbshipit-source-id: 67c94065fbdd36930f09930aad0aaa6d2c152bb8
2018-03-23 02:34:04 +01:00
case FlushReason::kOthers:
return "Other Reasons";
case FlushReason::kGetLiveFiles:
return "Get Live Files";
case FlushReason::kShutDown:
return "Shut down";
case FlushReason::kExternalFileIngestion:
return "External File Ingestion";
case FlushReason::kManualCompaction:
return "Manual Compaction";
case FlushReason::kWriteBufferManager:
return "Write Buffer Manager";
case FlushReason::kWriteBufferFull:
return "Write Buffer Full";
case FlushReason::kTest:
return "Test";
FlushReason improvement Summary: Right now flush reason "SuperVersion Change" covers a few different scenarios which is a bit vague. For example, the following db_bench job should trigger "Write Buffer Full" > $ TEST_TMPDIR=/dev/shm ./db_bench -benchmarks=fillrandom -write_buffer_size=1048576 -target_file_size_base=1048576 -max_bytes_for_level_base=4194304 $ grep 'flush_reason' /dev/shm/dbbench/LOG ... 2018/03/06-17:30:42.543638 7f2773b99700 EVENT_LOG_v1 {"time_micros": 1520386242543634, "job": 192, "event": "flush_started", "num_memtables": 1, "num_entries": 7006, "num_deletes": 0, "memory_usage": 1018024, "flush_reason": "SuperVersion Change"} 2018/03/06-17:30:42.569541 7f2773b99700 EVENT_LOG_v1 {"time_micros": 1520386242569536, "job": 193, "event": "flush_started", "num_memtables": 1, "num_entries": 7006, "num_deletes": 0, "memory_usage": 1018104, "flush_reason": "SuperVersion Change"} 2018/03/06-17:30:42.596396 7f2773b99700 EVENT_LOG_v1 {"time_micros": 1520386242596392, "job": 194, "event": "flush_started", "num_memtables": 1, "num_entries": 7008, "num_deletes": 0, "memory_usage": 1018048, "flush_reason": "SuperVersion Change"} 2018/03/06-17:30:42.622444 7f2773b99700 EVENT_LOG_v1 {"time_micros": 1520386242622440, "job": 195, "event": "flush_started", "num_memtables": 1, "num_entries": 7006, "num_deletes": 0, "memory_usage": 1018104, "flush_reason": "SuperVersion Change"} With the fix: > 2018/03/19-14:40:02.341451 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602341444, "job": 98, "event": "flush_started", "num_memtables": 1, "num_entries": 7009, "num_deletes": 0, "memory_usage": 1018008, "flush_reason": "Write Buffer Full"} 2018/03/19-14:40:02.379655 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602379642, "job": 100, "event": "flush_started", "num_memtables": 1, "num_entries": 7006, "num_deletes": 0, "memory_usage": 1018016, "flush_reason": "Write Buffer Full"} 2018/03/19-14:40:02.418479 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602418474, "job": 101, "event": "flush_started", "num_memtables": 1, "num_entries": 7009, "num_deletes": 0, "memory_usage": 1018104, "flush_reason": "Write Buffer Full"} 2018/03/19-14:40:02.455084 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602455079, "job": 102, "event": "flush_started", "num_memtables": 1, "num_entries": 7009, "num_deletes": 0, "memory_usage": 1018048, "flush_reason": "Write Buffer Full"} 2018/03/19-14:40:02.492293 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602492288, "job": 104, "event": "flush_started", "num_memtables": 1, "num_entries": 7007, "num_deletes": 0, "memory_usage": 1018056, "flush_reason": "Write Buffer Full"} 2018/03/19-14:40:02.528720 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602528715, "job": 105, "event": "flush_started", "num_memtables": 1, "num_entries": 7006, "num_deletes": 0, "memory_usage": 1018104, "flush_reason": "Write Buffer Full"} 2018/03/19-14:40:02.566255 7f11dc257700 EVENT_LOG_v1 {"time_micros": 1521495602566238, "job": 107, "event": "flush_started", "num_memtables": 1, "num_entries": 7009, "num_deletes": 0, "memory_usage": 1018112, "flush_reason": "Write Buffer Full"} Closes https://github.com/facebook/rocksdb/pull/3627 Differential Revision: D7328772 Pulled By: miasantreble fbshipit-source-id: 67c94065fbdd36930f09930aad0aaa6d2c152bb8
2018-03-23 02:34:04 +01:00
case FlushReason::kDeleteFiles:
return "Delete Files";
case FlushReason::kAutoCompaction:
return "Auto Compaction";
case FlushReason::kManualFlush:
return "Manual Flush";
Auto recovery from out of space errors (#4164) Summary: This commit implements automatic recovery from a Status::NoSpace() error during background operations such as write callback, flush and compaction. The broad design is as follows - 1. Compaction errors are treated as soft errors and don't put the database in read-only mode. A compaction is delayed until enough free disk space is available to accomodate the compaction outputs, which is estimated based on the input size. This means that users can continue to write, and we rely on the WriteController to delay or stop writes if the compaction debt becomes too high due to persistent low disk space condition 2. Errors during write callback and flush are treated as hard errors, i.e the database is put in read-only mode and goes back to read-write only fater certain recovery actions are taken. 3. Both types of recovery rely on the SstFileManagerImpl to poll for sufficient disk space. We assume that there is a 1-1 mapping between an SFM and the underlying OS storage container. For cases where multiple DBs are hosted on a single storage container, the user is expected to allocate a single SFM instance and use the same one for all the DBs. If no SFM is specified by the user, DBImpl::Open() will allocate one, but this will be one per DB and each DB will recover independently. The recovery implemented by SFM is as follows - a) On the first occurance of an out of space error during compaction, subsequent compactions will be delayed until the disk free space check indicates enough available space. The required space is computed as the sum of input sizes. b) The free space check requirement will be removed once the amount of free space is greater than the size reserved by in progress compactions when the first error occured c) If the out of space error is a hard error, a background thread in SFM will poll for sufficient headroom before triggering the recovery of the database and putting it in write-only mode. The headroom is calculated as the sum of the write_buffer_size of all the DB instances associated with the SFM 4. EventListener callbacks will be called at the start and completion of automatic recovery. Users can disable the auto recov ery in the start callback, and later initiate it manually by calling DB::Resume() Todo: 1. More extensive testing 2. Add disk full condition to db_stress (follow-on PR) Pull Request resolved: https://github.com/facebook/rocksdb/pull/4164 Differential Revision: D9846378 Pulled By: anand1976 fbshipit-source-id: 80ea875dbd7f00205e19c82215ff6e37da10da4a
2018-09-15 22:36:19 +02:00
case FlushReason::kErrorRecovery:
return "Error Recovery";
case FlushReason::kWalFull:
return "WAL Full";
default:
return "Invalid";
}
}
FlushJob::FlushJob(
const std::string& dbname, ColumnFamilyData* cfd,
const ImmutableDBOptions& db_options,
const MutableCFOptions& mutable_cf_options, uint64_t max_memtable_id,
const FileOptions& file_options, VersionSet* versions,
InstrumentedMutex* db_mutex, std::atomic<bool>* shutting_down,
std::vector<SequenceNumber> existing_snapshots,
SequenceNumber earliest_write_conflict_snapshot,
SnapshotChecker* snapshot_checker, JobContext* job_context,
LogBuffer* log_buffer, FSDirectory* db_directory,
FSDirectory* output_file_directory, CompressionType output_compression,
Statistics* stats, EventLogger* event_logger, bool measure_io_stats,
const bool sync_output_directory, const bool write_manifest,
Env::Priority thread_pri, const std::shared_ptr<IOTracer>& io_tracer,
const std::string& db_id, const std::string& db_session_id,
std::string full_history_ts_low, BlobFileCompletionCallback* blob_callback)
: dbname_(dbname),
db_id_(db_id),
db_session_id_(db_session_id),
cfd_(cfd),
db_options_(db_options),
mutable_cf_options_(mutable_cf_options),
max_memtable_id_(max_memtable_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
2019-12-13 23:47:08 +01:00
file_options_(file_options),
versions_(versions),
db_mutex_(db_mutex),
shutting_down_(shutting_down),
existing_snapshots_(std::move(existing_snapshots)),
earliest_write_conflict_snapshot_(earliest_write_conflict_snapshot),
snapshot_checker_(snapshot_checker),
job_context_(job_context),
log_buffer_(log_buffer),
db_directory_(db_directory),
output_file_directory_(output_file_directory),
output_compression_(output_compression),
stats_(stats),
event_logger_(event_logger),
measure_io_stats_(measure_io_stats),
sync_output_directory_(sync_output_directory),
write_manifest_(write_manifest),
edit_(nullptr),
base_(nullptr),
pick_memtable_called(false),
thread_pri_(thread_pri),
io_tracer_(io_tracer),
clock_(db_options_.clock),
full_history_ts_low_(std::move(full_history_ts_low)),
blob_callback_(blob_callback) {
Allow GetThreadList() to report operation stage. Summary: Allow GetThreadList() to report operation stage. Test Plan: ./thread_list_test ./db_bench --benchmarks=fillrandom --num=100000 --threads=40 \ --max_background_compactions=10 --max_background_flushes=3 \ --thread_status_per_interval=1000 --key_size=16 --value_size=1000 \ --num_column_families=10 export ROCKSDB_TESTS=ThreadStatus ./db_test Sample output ThreadID ThreadType cfName Operation OP_StartTime ElapsedTime Stage State 140116265861184 Low Pri 140116270055488 Low Pri 140116274249792 High Pri column_family_name_000005 Flush 2015/03/10-14:58:11 0 us FlushJob::WriteLevel0Table 140116400078912 Low Pri column_family_name_000004 Compaction 2015/03/10-14:58:11 0 us CompactionJob::FinishCompactionOutputFile 140116358135872 Low Pri column_family_name_000006 Compaction 2015/03/10-14:58:10 1 us CompactionJob::FinishCompactionOutputFile 140116341358656 Low Pri 140116295221312 High Pri default Flush 2015/03/10-14:58:11 0 us FlushJob::WriteLevel0Table 140116324581440 Low Pri column_family_name_000009 Compaction 2015/03/10-14:58:11 0 us CompactionJob::ProcessKeyValueCompaction 140116278444096 Low Pri 140116299415616 Low Pri column_family_name_000008 Compaction 2015/03/10-14:58:11 0 us CompactionJob::FinishCompactionOutputFile 140116291027008 High Pri column_family_name_000001 Flush 2015/03/10-14:58:11 0 us FlushJob::WriteLevel0Table 140116286832704 Low Pri column_family_name_000002 Compaction 2015/03/10-14:58:11 0 us CompactionJob::FinishCompactionOutputFile 140116282638400 Low Pri Reviewers: rven, igor, sdong Reviewed By: sdong Subscribers: dhruba, leveldb Differential Revision: https://reviews.facebook.net/D34683
2015-03-13 18:45:40 +01:00
// Update the thread status to indicate flush.
ReportStartedFlush();
Allow GetThreadList() to report operation stage. Summary: Allow GetThreadList() to report operation stage. Test Plan: ./thread_list_test ./db_bench --benchmarks=fillrandom --num=100000 --threads=40 \ --max_background_compactions=10 --max_background_flushes=3 \ --thread_status_per_interval=1000 --key_size=16 --value_size=1000 \ --num_column_families=10 export ROCKSDB_TESTS=ThreadStatus ./db_test Sample output ThreadID ThreadType cfName Operation OP_StartTime ElapsedTime Stage State 140116265861184 Low Pri 140116270055488 Low Pri 140116274249792 High Pri column_family_name_000005 Flush 2015/03/10-14:58:11 0 us FlushJob::WriteLevel0Table 140116400078912 Low Pri column_family_name_000004 Compaction 2015/03/10-14:58:11 0 us CompactionJob::FinishCompactionOutputFile 140116358135872 Low Pri column_family_name_000006 Compaction 2015/03/10-14:58:10 1 us CompactionJob::FinishCompactionOutputFile 140116341358656 Low Pri 140116295221312 High Pri default Flush 2015/03/10-14:58:11 0 us FlushJob::WriteLevel0Table 140116324581440 Low Pri column_family_name_000009 Compaction 2015/03/10-14:58:11 0 us CompactionJob::ProcessKeyValueCompaction 140116278444096 Low Pri 140116299415616 Low Pri column_family_name_000008 Compaction 2015/03/10-14:58:11 0 us CompactionJob::FinishCompactionOutputFile 140116291027008 High Pri column_family_name_000001 Flush 2015/03/10-14:58:11 0 us FlushJob::WriteLevel0Table 140116286832704 Low Pri column_family_name_000002 Compaction 2015/03/10-14:58:11 0 us CompactionJob::FinishCompactionOutputFile 140116282638400 Low Pri Reviewers: rven, igor, sdong Reviewed By: sdong Subscribers: dhruba, leveldb Differential Revision: https://reviews.facebook.net/D34683
2015-03-13 18:45:40 +01:00
TEST_SYNC_POINT("FlushJob::FlushJob()");
}
FlushJob::~FlushJob() {
io_status_.PermitUncheckedError();
Allow GetThreadList() to report operation stage. Summary: Allow GetThreadList() to report operation stage. Test Plan: ./thread_list_test ./db_bench --benchmarks=fillrandom --num=100000 --threads=40 \ --max_background_compactions=10 --max_background_flushes=3 \ --thread_status_per_interval=1000 --key_size=16 --value_size=1000 \ --num_column_families=10 export ROCKSDB_TESTS=ThreadStatus ./db_test Sample output ThreadID ThreadType cfName Operation OP_StartTime ElapsedTime Stage State 140116265861184 Low Pri 140116270055488 Low Pri 140116274249792 High Pri column_family_name_000005 Flush 2015/03/10-14:58:11 0 us FlushJob::WriteLevel0Table 140116400078912 Low Pri column_family_name_000004 Compaction 2015/03/10-14:58:11 0 us CompactionJob::FinishCompactionOutputFile 140116358135872 Low Pri column_family_name_000006 Compaction 2015/03/10-14:58:10 1 us CompactionJob::FinishCompactionOutputFile 140116341358656 Low Pri 140116295221312 High Pri default Flush 2015/03/10-14:58:11 0 us FlushJob::WriteLevel0Table 140116324581440 Low Pri column_family_name_000009 Compaction 2015/03/10-14:58:11 0 us CompactionJob::ProcessKeyValueCompaction 140116278444096 Low Pri 140116299415616 Low Pri column_family_name_000008 Compaction 2015/03/10-14:58:11 0 us CompactionJob::FinishCompactionOutputFile 140116291027008 High Pri column_family_name_000001 Flush 2015/03/10-14:58:11 0 us FlushJob::WriteLevel0Table 140116286832704 Low Pri column_family_name_000002 Compaction 2015/03/10-14:58:11 0 us CompactionJob::FinishCompactionOutputFile 140116282638400 Low Pri Reviewers: rven, igor, sdong Reviewed By: sdong Subscribers: dhruba, leveldb Differential Revision: https://reviews.facebook.net/D34683
2015-03-13 18:45:40 +01:00
ThreadStatusUtil::ResetThreadStatus();
}
void FlushJob::ReportStartedFlush() {
ThreadStatusUtil::SetColumnFamily(cfd_, cfd_->ioptions()->env,
db_options_.enable_thread_tracking);
ThreadStatusUtil::SetThreadOperation(ThreadStatus::OP_FLUSH);
ThreadStatusUtil::SetThreadOperationProperty(
ThreadStatus::COMPACTION_JOB_ID,
job_context_->job_id);
IOSTATS_RESET(bytes_written);
}
void FlushJob::ReportFlushInputSize(const autovector<MemTable*>& mems) {
uint64_t input_size = 0;
for (auto* mem : mems) {
input_size += mem->ApproximateMemoryUsage();
}
ThreadStatusUtil::IncreaseThreadOperationProperty(
ThreadStatus::FLUSH_BYTES_MEMTABLES,
input_size);
}
void FlushJob::RecordFlushIOStats() {
RecordTick(stats_, FLUSH_WRITE_BYTES, IOSTATS(bytes_written));
ThreadStatusUtil::IncreaseThreadOperationProperty(
ThreadStatus::FLUSH_BYTES_WRITTEN, IOSTATS(bytes_written));
IOSTATS_RESET(bytes_written);
}
void FlushJob::PickMemTable() {
db_mutex_->AssertHeld();
assert(!pick_memtable_called);
pick_memtable_called = true;
// Save the contents of the earliest memtable as a new Table
cfd_->imm()->PickMemtablesToFlush(max_memtable_id_, &mems_);
if (mems_.empty()) {
return;
}
ReportFlushInputSize(mems_);
// entries mems are (implicitly) sorted in ascending order by their created
// time. We will use the first memtable's `edit` to keep the meta info for
// this flush.
MemTable* m = mems_[0];
edit_ = m->GetEdits();
edit_->SetPrevLogNumber(0);
// SetLogNumber(log_num) indicates logs with number smaller than log_num
// will no longer be picked up for recovery.
edit_->SetLogNumber(mems_.back()->GetNextLogNumber());
edit_->SetColumnFamily(cfd_->GetID());
// path 0 for level 0 file.
meta_.fd = FileDescriptor(versions_->NewFileNumber(), 0, 0);
Add simple heuristics for experimental mempurge. (#8583) Summary: Add `experimental_mempurge_policy` option flag and introduce two new `MemPurge` (Memtable Garbage Collection) policies: 'ALWAYS' and 'ALTERNATE'. Default value: ALTERNATE. `ALWAYS`: every flush will first go through a `MemPurge` process. If the output is too big to fit into a single memtable, then the mempurge is aborted and a regular flush process carries on. `ALWAYS` is designed for user that need to reduce the number of L0 SST file created to a strict minimum, and can afford a small dent in performance (possibly hits to CPU usage, read efficiency, and maximum burst write throughput). `ALTERNATE`: a flush is transformed into a `MemPurge` except if one of the memtables being flushed is the product of a previous `MemPurge`. `ALTERNATE` is a good tradeoff between reduction in number of L0 SST files created and performance. `ALTERNATE` perform particularly well for completely random garbage ratios, or garbage ratios anywhere in (0%,50%], and even higher when there is a wild variability in garbage ratios. This PR also includes support for `experimental_mempurge_policy` in `db_bench`. Testing was done locally by replacing all the `MemPurge` policies of the unit tests with `ALTERNATE`, as well as local testing with `db_crashtest.py` `whitebox` and `blackbox`. Overall, if an `ALWAYS` mempurge policy passes the tests, there is no reasons why an `ALTERNATE` policy would fail, and therefore the mempurge policy was set to `ALWAYS` for all mempurge unit tests. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8583 Reviewed By: pdillinger Differential Revision: D29888050 Pulled By: bjlemaire fbshipit-source-id: e2cf26646d66679f6f5fb29842624615610759c1
2021-07-26 20:55:27 +02:00
// If mempurge feature is activated, keep track of any potential
// memtables coming from a previous mempurge operation.
// Used for mempurge policy.
Memtable sampling for mempurge heuristic. (#8628) Summary: Changes the API of the MemPurge process: the `bool experimental_allow_mempurge` and `experimental_mempurge_policy` flags have been replaced by a `double experimental_mempurge_threshold` option. This change of API reflects another major change introduced in this PR: the MemPurgeDecider() function now works by sampling the memtables being flushed to estimate the overall amount of useful payload (payload minus the garbage), and then compare this useful payload estimate with the `double experimental_mempurge_threshold` value. Therefore, when the value of this flag is `0.0` (default value), mempurge is simply deactivated. On the other hand, a value of `DBL_MAX` would be equivalent to always going through a mempurge regardless of the garbage ratio estimate. At the moment, a `double experimental_mempurge_threshold` value else than 0.0 or `DBL_MAX` is opnly supported`with the `SkipList` memtable representation. Regarding the sampling, this PR includes the introduction of a `MemTable::UniqueRandomSample` function that collects (approximately) random entries from the memtable by using the new `SkipList::Iterator::RandomSeek()` under the hood, or by iterating through each memtable entry, depending on the target sample size and the total number of entries. The unit tests have been readapted to support this new API. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8628 Reviewed By: pdillinger Differential Revision: D30149315 Pulled By: bjlemaire fbshipit-source-id: 1feef5390c95db6f4480ab4434716533d3947f27
2021-08-11 03:07:48 +02:00
if (db_options_.experimental_mempurge_threshold > 0.0) {
Add simple heuristics for experimental mempurge. (#8583) Summary: Add `experimental_mempurge_policy` option flag and introduce two new `MemPurge` (Memtable Garbage Collection) policies: 'ALWAYS' and 'ALTERNATE'. Default value: ALTERNATE. `ALWAYS`: every flush will first go through a `MemPurge` process. If the output is too big to fit into a single memtable, then the mempurge is aborted and a regular flush process carries on. `ALWAYS` is designed for user that need to reduce the number of L0 SST file created to a strict minimum, and can afford a small dent in performance (possibly hits to CPU usage, read efficiency, and maximum burst write throughput). `ALTERNATE`: a flush is transformed into a `MemPurge` except if one of the memtables being flushed is the product of a previous `MemPurge`. `ALTERNATE` is a good tradeoff between reduction in number of L0 SST files created and performance. `ALTERNATE` perform particularly well for completely random garbage ratios, or garbage ratios anywhere in (0%,50%], and even higher when there is a wild variability in garbage ratios. This PR also includes support for `experimental_mempurge_policy` in `db_bench`. Testing was done locally by replacing all the `MemPurge` policies of the unit tests with `ALTERNATE`, as well as local testing with `db_crashtest.py` `whitebox` and `blackbox`. Overall, if an `ALWAYS` mempurge policy passes the tests, there is no reasons why an `ALTERNATE` policy would fail, and therefore the mempurge policy was set to `ALWAYS` for all mempurge unit tests. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8583 Reviewed By: pdillinger Differential Revision: D29888050 Pulled By: bjlemaire fbshipit-source-id: e2cf26646d66679f6f5fb29842624615610759c1
2021-07-26 20:55:27 +02:00
contains_mempurge_outcome_ = false;
for (MemTable* mt : mems_) {
if (cfd_->imm()->IsMemPurgeOutput(mt->GetID())) {
contains_mempurge_outcome_ = true;
break;
}
}
}
base_ = cfd_->current();
base_->Ref(); // it is likely that we do not need this reference
}
Skip deleted WALs during recovery Summary: This patch record min log number to keep to the manifest while flushing SST files to ignore them and any WAL older than them during recovery. This is to avoid scenarios when we have a gap between the WAL files are fed to the recovery procedure. The gap could happen by for example out-of-order WAL deletion. Such gap could cause problems in 2PC recovery where the prepared and commit entry are placed into two separate WAL and gap in the WALs could result into not processing the WAL with the commit entry and hence breaking the 2PC recovery logic. Before the commit, for 2PC case, we determined which log number to keep in FindObsoleteFiles(). We looked at the earliest logs with outstanding prepare entries, or prepare entries whose respective commit or abort are in memtable. With the commit, the same calculation is done while we apply the SST flush. Just before installing the flush file, we precompute the earliest log file to keep after the flush finishes using the same logic (but skipping the memtables just flushed), record this information to the manifest entry for this new flushed SST file. This pre-computed value is also remembered in memory, and will later be used to determine whether a log file can be deleted. This value is unlikely to change until next flush because the commit entry will stay in memtable. (In WritePrepared, we could have removed the older log files as soon as all prepared entries are committed. It's not yet done anyway. Even if we do it, the only thing we loss with this new approach is earlier log deletion between two flushes, which does not guarantee to happen anyway because the obsolete file clean-up function is only executed after flush or compaction) This min log number to keep is stored in the manifest using the safely-ignore customized field of AddFile entry, in order to guarantee that the DB generated using newer release can be opened by previous releases no older than 4.2. Closes https://github.com/facebook/rocksdb/pull/3765 Differential Revision: D7747618 Pulled By: siying fbshipit-source-id: d00c92105b4f83852e9754a1b70d6b64cb590729
2018-05-04 00:35:11 +02:00
Status FlushJob::Run(LogsWithPrepTracker* prep_tracker,
FileMetaData* file_meta) {
TEST_SYNC_POINT("FlushJob::Start");
db_mutex_->AssertHeld();
assert(pick_memtable_called);
Allow GetThreadList() to report operation stage. Summary: Allow GetThreadList() to report operation stage. Test Plan: ./thread_list_test ./db_bench --benchmarks=fillrandom --num=100000 --threads=40 \ --max_background_compactions=10 --max_background_flushes=3 \ --thread_status_per_interval=1000 --key_size=16 --value_size=1000 \ --num_column_families=10 export ROCKSDB_TESTS=ThreadStatus ./db_test Sample output ThreadID ThreadType cfName Operation OP_StartTime ElapsedTime Stage State 140116265861184 Low Pri 140116270055488 Low Pri 140116274249792 High Pri column_family_name_000005 Flush 2015/03/10-14:58:11 0 us FlushJob::WriteLevel0Table 140116400078912 Low Pri column_family_name_000004 Compaction 2015/03/10-14:58:11 0 us CompactionJob::FinishCompactionOutputFile 140116358135872 Low Pri column_family_name_000006 Compaction 2015/03/10-14:58:10 1 us CompactionJob::FinishCompactionOutputFile 140116341358656 Low Pri 140116295221312 High Pri default Flush 2015/03/10-14:58:11 0 us FlushJob::WriteLevel0Table 140116324581440 Low Pri column_family_name_000009 Compaction 2015/03/10-14:58:11 0 us CompactionJob::ProcessKeyValueCompaction 140116278444096 Low Pri 140116299415616 Low Pri column_family_name_000008 Compaction 2015/03/10-14:58:11 0 us CompactionJob::FinishCompactionOutputFile 140116291027008 High Pri column_family_name_000001 Flush 2015/03/10-14:58:11 0 us FlushJob::WriteLevel0Table 140116286832704 Low Pri column_family_name_000002 Compaction 2015/03/10-14:58:11 0 us CompactionJob::FinishCompactionOutputFile 140116282638400 Low Pri Reviewers: rven, igor, sdong Reviewed By: sdong Subscribers: dhruba, leveldb Differential Revision: https://reviews.facebook.net/D34683
2015-03-13 18:45:40 +01:00
AutoThreadOperationStageUpdater stage_run(
ThreadStatus::STAGE_FLUSH_RUN);
if (mems_.empty()) {
ROCKS_LOG_BUFFER(log_buffer_, "[%s] Nothing in memtable to flush",
cfd_->GetName().c_str());
return Status::OK();
}
// I/O measurement variables
PerfLevel prev_perf_level = PerfLevel::kEnableTime;
uint64_t prev_write_nanos = 0;
uint64_t prev_fsync_nanos = 0;
uint64_t prev_range_sync_nanos = 0;
uint64_t prev_prepare_write_nanos = 0;
uint64_t prev_cpu_write_nanos = 0;
uint64_t prev_cpu_read_nanos = 0;
if (measure_io_stats_) {
prev_perf_level = GetPerfLevel();
SetPerfLevel(PerfLevel::kEnableTime);
prev_write_nanos = IOSTATS(write_nanos);
prev_fsync_nanos = IOSTATS(fsync_nanos);
prev_range_sync_nanos = IOSTATS(range_sync_nanos);
prev_prepare_write_nanos = IOSTATS(prepare_write_nanos);
prev_cpu_write_nanos = IOSTATS(cpu_write_nanos);
prev_cpu_read_nanos = IOSTATS(cpu_read_nanos);
}
Make mempurge a background process (equivalent to in-memory compaction). (#8505) Summary: In https://github.com/facebook/rocksdb/issues/8454, I introduced a new process baptized `MemPurge` (memtable garbage collection). This new PR is built upon this past mempurge prototype. In this PR, I made the `mempurge` process a background task, which provides superior performance since the mempurge process does not cling on the db_mutex anymore, and addresses severe restrictions from the past iteration (including a scenario where the past mempurge was failling, when a memtable was mempurged but was still referred to by an iterator/snapshot/...). Now the mempurge process ressembles an in-memory compaction process: the stack of immutable memtables is filtered out, and the useful payload is used to populate an output memtable. If the output memtable is filled at more than 60% capacity (arbitrary heuristic) the mempurge process is aborted and a regular flush process takes place, else the output memtable is kept in the immutable memtable stack. Note that adding this output memtable to the `imm()` memtable stack does not trigger another flush process, so that the flush thread can go to sleep at the end of a successful mempurge. MemPurge is activated by making the `experimental_allow_mempurge` flag `true`. When activated, the `MemPurge` process will always happen when the flush reason is `kWriteBufferFull`. The 3 unit tests confirm that this process supports `Put`, `Get`, `Delete`, `DeleteRange` operators and is compatible with `Iterators` and `CompactionFilters`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8505 Reviewed By: pdillinger Differential Revision: D29619283 Pulled By: bjlemaire fbshipit-source-id: 8a99bee76b63a8211bff1a00e0ae32360aaece95
2021-07-10 02:16:00 +02:00
Status mempurge_s = Status::NotFound("No MemPurge.");
Memtable sampling for mempurge heuristic. (#8628) Summary: Changes the API of the MemPurge process: the `bool experimental_allow_mempurge` and `experimental_mempurge_policy` flags have been replaced by a `double experimental_mempurge_threshold` option. This change of API reflects another major change introduced in this PR: the MemPurgeDecider() function now works by sampling the memtables being flushed to estimate the overall amount of useful payload (payload minus the garbage), and then compare this useful payload estimate with the `double experimental_mempurge_threshold` value. Therefore, when the value of this flag is `0.0` (default value), mempurge is simply deactivated. On the other hand, a value of `DBL_MAX` would be equivalent to always going through a mempurge regardless of the garbage ratio estimate. At the moment, a `double experimental_mempurge_threshold` value else than 0.0 or `DBL_MAX` is opnly supported`with the `SkipList` memtable representation. Regarding the sampling, this PR includes the introduction of a `MemTable::UniqueRandomSample` function that collects (approximately) random entries from the memtable by using the new `SkipList::Iterator::RandomSeek()` under the hood, or by iterating through each memtable entry, depending on the target sample size and the total number of entries. The unit tests have been readapted to support this new API. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8628 Reviewed By: pdillinger Differential Revision: D30149315 Pulled By: bjlemaire fbshipit-source-id: 1feef5390c95db6f4480ab4434716533d3947f27
2021-08-11 03:07:48 +02:00
if ((db_options_.experimental_mempurge_threshold > 0.0) &&
Make mempurge a background process (equivalent to in-memory compaction). (#8505) Summary: In https://github.com/facebook/rocksdb/issues/8454, I introduced a new process baptized `MemPurge` (memtable garbage collection). This new PR is built upon this past mempurge prototype. In this PR, I made the `mempurge` process a background task, which provides superior performance since the mempurge process does not cling on the db_mutex anymore, and addresses severe restrictions from the past iteration (including a scenario where the past mempurge was failling, when a memtable was mempurged but was still referred to by an iterator/snapshot/...). Now the mempurge process ressembles an in-memory compaction process: the stack of immutable memtables is filtered out, and the useful payload is used to populate an output memtable. If the output memtable is filled at more than 60% capacity (arbitrary heuristic) the mempurge process is aborted and a regular flush process takes place, else the output memtable is kept in the immutable memtable stack. Note that adding this output memtable to the `imm()` memtable stack does not trigger another flush process, so that the flush thread can go to sleep at the end of a successful mempurge. MemPurge is activated by making the `experimental_allow_mempurge` flag `true`. When activated, the `MemPurge` process will always happen when the flush reason is `kWriteBufferFull`. The 3 unit tests confirm that this process supports `Put`, `Get`, `Delete`, `DeleteRange` operators and is compatible with `Iterators` and `CompactionFilters`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8505 Reviewed By: pdillinger Differential Revision: D29619283 Pulled By: bjlemaire fbshipit-source-id: 8a99bee76b63a8211bff1a00e0ae32360aaece95
2021-07-10 02:16:00 +02:00
(cfd_->GetFlushReason() == FlushReason::kWriteBufferFull) &&
Add simple heuristics for experimental mempurge. (#8583) Summary: Add `experimental_mempurge_policy` option flag and introduce two new `MemPurge` (Memtable Garbage Collection) policies: 'ALWAYS' and 'ALTERNATE'. Default value: ALTERNATE. `ALWAYS`: every flush will first go through a `MemPurge` process. If the output is too big to fit into a single memtable, then the mempurge is aborted and a regular flush process carries on. `ALWAYS` is designed for user that need to reduce the number of L0 SST file created to a strict minimum, and can afford a small dent in performance (possibly hits to CPU usage, read efficiency, and maximum burst write throughput). `ALTERNATE`: a flush is transformed into a `MemPurge` except if one of the memtables being flushed is the product of a previous `MemPurge`. `ALTERNATE` is a good tradeoff between reduction in number of L0 SST files created and performance. `ALTERNATE` perform particularly well for completely random garbage ratios, or garbage ratios anywhere in (0%,50%], and even higher when there is a wild variability in garbage ratios. This PR also includes support for `experimental_mempurge_policy` in `db_bench`. Testing was done locally by replacing all the `MemPurge` policies of the unit tests with `ALTERNATE`, as well as local testing with `db_crashtest.py` `whitebox` and `blackbox`. Overall, if an `ALWAYS` mempurge policy passes the tests, there is no reasons why an `ALTERNATE` policy would fail, and therefore the mempurge policy was set to `ALWAYS` for all mempurge unit tests. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8583 Reviewed By: pdillinger Differential Revision: D29888050 Pulled By: bjlemaire fbshipit-source-id: e2cf26646d66679f6f5fb29842624615610759c1
2021-07-26 20:55:27 +02:00
(!mems_.empty()) && MemPurgeDecider()) {
Make mempurge a background process (equivalent to in-memory compaction). (#8505) Summary: In https://github.com/facebook/rocksdb/issues/8454, I introduced a new process baptized `MemPurge` (memtable garbage collection). This new PR is built upon this past mempurge prototype. In this PR, I made the `mempurge` process a background task, which provides superior performance since the mempurge process does not cling on the db_mutex anymore, and addresses severe restrictions from the past iteration (including a scenario where the past mempurge was failling, when a memtable was mempurged but was still referred to by an iterator/snapshot/...). Now the mempurge process ressembles an in-memory compaction process: the stack of immutable memtables is filtered out, and the useful payload is used to populate an output memtable. If the output memtable is filled at more than 60% capacity (arbitrary heuristic) the mempurge process is aborted and a regular flush process takes place, else the output memtable is kept in the immutable memtable stack. Note that adding this output memtable to the `imm()` memtable stack does not trigger another flush process, so that the flush thread can go to sleep at the end of a successful mempurge. MemPurge is activated by making the `experimental_allow_mempurge` flag `true`. When activated, the `MemPurge` process will always happen when the flush reason is `kWriteBufferFull`. The 3 unit tests confirm that this process supports `Put`, `Get`, `Delete`, `DeleteRange` operators and is compatible with `Iterators` and `CompactionFilters`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8505 Reviewed By: pdillinger Differential Revision: D29619283 Pulled By: bjlemaire fbshipit-source-id: 8a99bee76b63a8211bff1a00e0ae32360aaece95
2021-07-10 02:16:00 +02:00
mempurge_s = MemPurge();
if (!mempurge_s.ok()) {
// Mempurge is typically aborted when the new_mem output memtable
// is filled at more than XX % capacity (currently: 60%).
if (mempurge_s.IsAborted()) {
ROCKS_LOG_INFO(db_options_.info_log, "Mempurge process aborted: %s\n",
mempurge_s.ToString().c_str());
} else {
// However the mempurge process can also fail for
// other reasons (eg: new_mem->Add() fails).
ROCKS_LOG_WARN(db_options_.info_log, "Mempurge process failed: %s\n",
mempurge_s.ToString().c_str());
}
Make mempurge a background process (equivalent to in-memory compaction). (#8505) Summary: In https://github.com/facebook/rocksdb/issues/8454, I introduced a new process baptized `MemPurge` (memtable garbage collection). This new PR is built upon this past mempurge prototype. In this PR, I made the `mempurge` process a background task, which provides superior performance since the mempurge process does not cling on the db_mutex anymore, and addresses severe restrictions from the past iteration (including a scenario where the past mempurge was failling, when a memtable was mempurged but was still referred to by an iterator/snapshot/...). Now the mempurge process ressembles an in-memory compaction process: the stack of immutable memtables is filtered out, and the useful payload is used to populate an output memtable. If the output memtable is filled at more than 60% capacity (arbitrary heuristic) the mempurge process is aborted and a regular flush process takes place, else the output memtable is kept in the immutable memtable stack. Note that adding this output memtable to the `imm()` memtable stack does not trigger another flush process, so that the flush thread can go to sleep at the end of a successful mempurge. MemPurge is activated by making the `experimental_allow_mempurge` flag `true`. When activated, the `MemPurge` process will always happen when the flush reason is `kWriteBufferFull`. The 3 unit tests confirm that this process supports `Put`, `Get`, `Delete`, `DeleteRange` operators and is compatible with `Iterators` and `CompactionFilters`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8505 Reviewed By: pdillinger Differential Revision: D29619283 Pulled By: bjlemaire fbshipit-source-id: 8a99bee76b63a8211bff1a00e0ae32360aaece95
2021-07-10 02:16:00 +02:00
}
}
Status s;
if (mempurge_s.ok()) {
base_->Unref();
s = Status::OK();
} else {
// This will release and re-acquire the mutex.
s = WriteLevel0Table();
}
if (s.ok() && cfd_->IsDropped()) {
s = Status::ColumnFamilyDropped("Column family dropped during compaction");
}
if ((s.ok() || s.IsColumnFamilyDropped()) &&
shutting_down_->load(std::memory_order_acquire)) {
s = Status::ShutdownInProgress("Database shutdown");
}
if (!s.ok()) {
cfd_->imm()->RollbackMemtableFlush(mems_, meta_.fd.GetNumber());
} else if (write_manifest_) {
LogAndApply() should fail if the column family has been dropped Summary: This patch finally fixes the ColumnFamilyTest.ReadDroppedColumnFamily test. The test has been failing very sporadically and it was hard to repro. However, I managed to write a new tests that reproes the failure deterministically. Here's what happens: 1. We start the flush for the column family 2. We check if the column family was dropped here: https://github.com/facebook/rocksdb/blob/a3fc49bfddcdb1ff29409aacd06c04df56c7a1d7/db/flush_job.cc#L149 3. This check goes through, ends up in InstallMemtableFlushResults() and it goes into LogAndApply() 4. At about this time, we start dropping the column family. Dropping the column family process gets to LogAndApply() at about the same time as LogAndApply() from flush process 5. Drop column family goes through LogAndApply() first, marking the column family as dropped. 6. Flush process gets woken up and gets a chance to write to the MANIFEST. However, this is where it gets stuck: https://github.com/facebook/rocksdb/blob/a3fc49bfddcdb1ff29409aacd06c04df56c7a1d7/db/version_set.cc#L1975 7. We see that the column family was dropped, so there is no need to write to the MANIFEST. We return OK. 8. Flush gets OK back from LogAndApply() and it deletes the memtable, thinking that the data is now safely persisted to sst file. The fix is pretty simple. Instead of OK, we return ShutdownInProgress. This is not really true, but we have been using this status code to also mean "this operation was canceled because the column family has been dropped". The fix is only one LOC. All other code is related to tests. I added a new test that reproes the failure. I also moved SleepingBackgroundTask to util/testutil.h (because I needed it in column_family_test for my new test). There's plenty of other places where we reimplement SleepingBackgroundTask, but I'll address that in a separate commit. Test Plan: 1. new test 2. make check 3. Make sure the ColumnFamilyTest.ReadDroppedColumnFamily doesn't fail on Travis: https://travis-ci.org/facebook/rocksdb/jobs/79952386 Reviewers: yhchiang, anthony, IslamAbdelRahman, kradhakrishnan, rven, sdong Reviewed By: sdong Subscribers: dhruba, leveldb Differential Revision: https://reviews.facebook.net/D46773
2015-09-15 20:28:44 +02:00
TEST_SYNC_POINT("FlushJob::InstallResults");
// Replace immutable memtable with the generated Table
IOStatus tmp_io_s;
s = cfd_->imm()->TryInstallMemtableFlushResults(
Skip deleted WALs during recovery Summary: This patch record min log number to keep to the manifest while flushing SST files to ignore them and any WAL older than them during recovery. This is to avoid scenarios when we have a gap between the WAL files are fed to the recovery procedure. The gap could happen by for example out-of-order WAL deletion. Such gap could cause problems in 2PC recovery where the prepared and commit entry are placed into two separate WAL and gap in the WALs could result into not processing the WAL with the commit entry and hence breaking the 2PC recovery logic. Before the commit, for 2PC case, we determined which log number to keep in FindObsoleteFiles(). We looked at the earliest logs with outstanding prepare entries, or prepare entries whose respective commit or abort are in memtable. With the commit, the same calculation is done while we apply the SST flush. Just before installing the flush file, we precompute the earliest log file to keep after the flush finishes using the same logic (but skipping the memtables just flushed), record this information to the manifest entry for this new flushed SST file. This pre-computed value is also remembered in memory, and will later be used to determine whether a log file can be deleted. This value is unlikely to change until next flush because the commit entry will stay in memtable. (In WritePrepared, we could have removed the older log files as soon as all prepared entries are committed. It's not yet done anyway. Even if we do it, the only thing we loss with this new approach is earlier log deletion between two flushes, which does not guarantee to happen anyway because the obsolete file clean-up function is only executed after flush or compaction) This min log number to keep is stored in the manifest using the safely-ignore customized field of AddFile entry, in order to guarantee that the DB generated using newer release can be opened by previous releases no older than 4.2. Closes https://github.com/facebook/rocksdb/pull/3765 Differential Revision: D7747618 Pulled By: siying fbshipit-source-id: d00c92105b4f83852e9754a1b70d6b64cb590729
2018-05-04 00:35:11 +02:00
cfd_, mutable_cf_options_, mems_, prep_tracker, versions_, db_mutex_,
meta_.fd.GetNumber(), &job_context_->memtables_to_free, db_directory_,
log_buffer_, &committed_flush_jobs_info_, &tmp_io_s,
!(mempurge_s.ok()) /* write_edit : true if no mempurge happened (or if aborted),
but 'false' if mempurge successful: no new min log number
or new level 0 file path to write to manifest. */);
if (!tmp_io_s.ok()) {
io_status_ = tmp_io_s;
}
}
if (s.ok() && file_meta != nullptr) {
*file_meta = meta_;
}
RecordFlushIOStats();
// When measure_io_stats_ is true, the default 512 bytes is not enough.
auto stream = event_logger_->LogToBuffer(log_buffer_, 1024);
stream << "job" << job_context_->job_id << "event"
<< "flush_finished";
stream << "output_compression"
<< CompressionTypeToString(output_compression_);
stream << "lsm_state";
stream.StartArray();
auto vstorage = cfd_->current()->storage_info();
for (int level = 0; level < vstorage->num_levels(); ++level) {
stream << vstorage->NumLevelFiles(level);
}
stream.EndArray();
const auto& blob_files = vstorage->GetBlobFiles();
if (!blob_files.empty()) {
stream << "blob_file_head" << blob_files.begin()->first;
stream << "blob_file_tail" << blob_files.rbegin()->first;
}
stream << "immutable_memtables" << cfd_->imm()->NumNotFlushed();
if (measure_io_stats_) {
if (prev_perf_level != PerfLevel::kEnableTime) {
SetPerfLevel(prev_perf_level);
}
stream << "file_write_nanos" << (IOSTATS(write_nanos) - prev_write_nanos);
stream << "file_range_sync_nanos"
<< (IOSTATS(range_sync_nanos) - prev_range_sync_nanos);
stream << "file_fsync_nanos" << (IOSTATS(fsync_nanos) - prev_fsync_nanos);
stream << "file_prepare_write_nanos"
<< (IOSTATS(prepare_write_nanos) - prev_prepare_write_nanos);
stream << "file_cpu_write_nanos"
<< (IOSTATS(cpu_write_nanos) - prev_cpu_write_nanos);
stream << "file_cpu_read_nanos"
<< (IOSTATS(cpu_read_nanos) - prev_cpu_read_nanos);
}
return s;
}
void FlushJob::Cancel() {
db_mutex_->AssertHeld();
assert(base_ != nullptr);
base_->Unref();
}
Make mempurge a background process (equivalent to in-memory compaction). (#8505) Summary: In https://github.com/facebook/rocksdb/issues/8454, I introduced a new process baptized `MemPurge` (memtable garbage collection). This new PR is built upon this past mempurge prototype. In this PR, I made the `mempurge` process a background task, which provides superior performance since the mempurge process does not cling on the db_mutex anymore, and addresses severe restrictions from the past iteration (including a scenario where the past mempurge was failling, when a memtable was mempurged but was still referred to by an iterator/snapshot/...). Now the mempurge process ressembles an in-memory compaction process: the stack of immutable memtables is filtered out, and the useful payload is used to populate an output memtable. If the output memtable is filled at more than 60% capacity (arbitrary heuristic) the mempurge process is aborted and a regular flush process takes place, else the output memtable is kept in the immutable memtable stack. Note that adding this output memtable to the `imm()` memtable stack does not trigger another flush process, so that the flush thread can go to sleep at the end of a successful mempurge. MemPurge is activated by making the `experimental_allow_mempurge` flag `true`. When activated, the `MemPurge` process will always happen when the flush reason is `kWriteBufferFull`. The 3 unit tests confirm that this process supports `Put`, `Get`, `Delete`, `DeleteRange` operators and is compatible with `Iterators` and `CompactionFilters`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8505 Reviewed By: pdillinger Differential Revision: D29619283 Pulled By: bjlemaire fbshipit-source-id: 8a99bee76b63a8211bff1a00e0ae32360aaece95
2021-07-10 02:16:00 +02:00
Status FlushJob::MemPurge() {
Status s;
db_mutex_->AssertHeld();
db_mutex_->Unlock();
assert(!mems_.empty());
Add simple heuristics for experimental mempurge. (#8583) Summary: Add `experimental_mempurge_policy` option flag and introduce two new `MemPurge` (Memtable Garbage Collection) policies: 'ALWAYS' and 'ALTERNATE'. Default value: ALTERNATE. `ALWAYS`: every flush will first go through a `MemPurge` process. If the output is too big to fit into a single memtable, then the mempurge is aborted and a regular flush process carries on. `ALWAYS` is designed for user that need to reduce the number of L0 SST file created to a strict minimum, and can afford a small dent in performance (possibly hits to CPU usage, read efficiency, and maximum burst write throughput). `ALTERNATE`: a flush is transformed into a `MemPurge` except if one of the memtables being flushed is the product of a previous `MemPurge`. `ALTERNATE` is a good tradeoff between reduction in number of L0 SST files created and performance. `ALTERNATE` perform particularly well for completely random garbage ratios, or garbage ratios anywhere in (0%,50%], and even higher when there is a wild variability in garbage ratios. This PR also includes support for `experimental_mempurge_policy` in `db_bench`. Testing was done locally by replacing all the `MemPurge` policies of the unit tests with `ALTERNATE`, as well as local testing with `db_crashtest.py` `whitebox` and `blackbox`. Overall, if an `ALWAYS` mempurge policy passes the tests, there is no reasons why an `ALTERNATE` policy would fail, and therefore the mempurge policy was set to `ALWAYS` for all mempurge unit tests. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8583 Reviewed By: pdillinger Differential Revision: D29888050 Pulled By: bjlemaire fbshipit-source-id: e2cf26646d66679f6f5fb29842624615610759c1
2021-07-26 20:55:27 +02:00
// Measure purging time.
const uint64_t start_micros = clock_->NowMicros();
const uint64_t start_cpu_micros = clock_->CPUNanos() / 1000;
Make mempurge a background process (equivalent to in-memory compaction). (#8505) Summary: In https://github.com/facebook/rocksdb/issues/8454, I introduced a new process baptized `MemPurge` (memtable garbage collection). This new PR is built upon this past mempurge prototype. In this PR, I made the `mempurge` process a background task, which provides superior performance since the mempurge process does not cling on the db_mutex anymore, and addresses severe restrictions from the past iteration (including a scenario where the past mempurge was failling, when a memtable was mempurged but was still referred to by an iterator/snapshot/...). Now the mempurge process ressembles an in-memory compaction process: the stack of immutable memtables is filtered out, and the useful payload is used to populate an output memtable. If the output memtable is filled at more than 60% capacity (arbitrary heuristic) the mempurge process is aborted and a regular flush process takes place, else the output memtable is kept in the immutable memtable stack. Note that adding this output memtable to the `imm()` memtable stack does not trigger another flush process, so that the flush thread can go to sleep at the end of a successful mempurge. MemPurge is activated by making the `experimental_allow_mempurge` flag `true`. When activated, the `MemPurge` process will always happen when the flush reason is `kWriteBufferFull`. The 3 unit tests confirm that this process supports `Put`, `Get`, `Delete`, `DeleteRange` operators and is compatible with `Iterators` and `CompactionFilters`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8505 Reviewed By: pdillinger Differential Revision: D29619283 Pulled By: bjlemaire fbshipit-source-id: 8a99bee76b63a8211bff1a00e0ae32360aaece95
2021-07-10 02:16:00 +02:00
MemTable* new_mem = nullptr;
Add simple heuristics for experimental mempurge. (#8583) Summary: Add `experimental_mempurge_policy` option flag and introduce two new `MemPurge` (Memtable Garbage Collection) policies: 'ALWAYS' and 'ALTERNATE'. Default value: ALTERNATE. `ALWAYS`: every flush will first go through a `MemPurge` process. If the output is too big to fit into a single memtable, then the mempurge is aborted and a regular flush process carries on. `ALWAYS` is designed for user that need to reduce the number of L0 SST file created to a strict minimum, and can afford a small dent in performance (possibly hits to CPU usage, read efficiency, and maximum burst write throughput). `ALTERNATE`: a flush is transformed into a `MemPurge` except if one of the memtables being flushed is the product of a previous `MemPurge`. `ALTERNATE` is a good tradeoff between reduction in number of L0 SST files created and performance. `ALTERNATE` perform particularly well for completely random garbage ratios, or garbage ratios anywhere in (0%,50%], and even higher when there is a wild variability in garbage ratios. This PR also includes support for `experimental_mempurge_policy` in `db_bench`. Testing was done locally by replacing all the `MemPurge` policies of the unit tests with `ALTERNATE`, as well as local testing with `db_crashtest.py` `whitebox` and `blackbox`. Overall, if an `ALWAYS` mempurge policy passes the tests, there is no reasons why an `ALTERNATE` policy would fail, and therefore the mempurge policy was set to `ALWAYS` for all mempurge unit tests. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8583 Reviewed By: pdillinger Differential Revision: D29888050 Pulled By: bjlemaire fbshipit-source-id: e2cf26646d66679f6f5fb29842624615610759c1
2021-07-26 20:55:27 +02:00
// For performance/log investigation purposes:
// look at how much useful payload we harvest in the new_mem.
// This value is then printed to the DB log.
double new_mem_capacity = 0.0;
Make mempurge a background process (equivalent to in-memory compaction). (#8505) Summary: In https://github.com/facebook/rocksdb/issues/8454, I introduced a new process baptized `MemPurge` (memtable garbage collection). This new PR is built upon this past mempurge prototype. In this PR, I made the `mempurge` process a background task, which provides superior performance since the mempurge process does not cling on the db_mutex anymore, and addresses severe restrictions from the past iteration (including a scenario where the past mempurge was failling, when a memtable was mempurged but was still referred to by an iterator/snapshot/...). Now the mempurge process ressembles an in-memory compaction process: the stack of immutable memtables is filtered out, and the useful payload is used to populate an output memtable. If the output memtable is filled at more than 60% capacity (arbitrary heuristic) the mempurge process is aborted and a regular flush process takes place, else the output memtable is kept in the immutable memtable stack. Note that adding this output memtable to the `imm()` memtable stack does not trigger another flush process, so that the flush thread can go to sleep at the end of a successful mempurge. MemPurge is activated by making the `experimental_allow_mempurge` flag `true`. When activated, the `MemPurge` process will always happen when the flush reason is `kWriteBufferFull`. The 3 unit tests confirm that this process supports `Put`, `Get`, `Delete`, `DeleteRange` operators and is compatible with `Iterators` and `CompactionFilters`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8505 Reviewed By: pdillinger Differential Revision: D29619283 Pulled By: bjlemaire fbshipit-source-id: 8a99bee76b63a8211bff1a00e0ae32360aaece95
2021-07-10 02:16:00 +02:00
// Create two iterators, one for the memtable data (contains
// info from puts + deletes), and one for the memtable
// Range Tombstones (from DeleteRanges).
ReadOptions ro;
ro.total_order_seek = true;
Arena arena;
std::vector<InternalIterator*> memtables;
std::vector<std::unique_ptr<FragmentedRangeTombstoneIterator>>
range_del_iters;
for (MemTable* m : mems_) {
memtables.push_back(m->NewIterator(ro, &arena));
auto* range_del_iter = m->NewRangeTombstoneIterator(ro, kMaxSequenceNumber);
if (range_del_iter != nullptr) {
range_del_iters.emplace_back(range_del_iter);
}
}
assert(!memtables.empty());
SequenceNumber first_seqno = kMaxSequenceNumber;
SequenceNumber earliest_seqno = kMaxSequenceNumber;
// Pick first and earliest seqno as min of all first_seqno
// and earliest_seqno of the mempurged memtables.
for (const auto& mem : mems_) {
first_seqno = mem->GetFirstSequenceNumber() < first_seqno
? mem->GetFirstSequenceNumber()
: first_seqno;
earliest_seqno = mem->GetEarliestSequenceNumber() < earliest_seqno
? mem->GetEarliestSequenceNumber()
: earliest_seqno;
}
Make mempurge a background process (equivalent to in-memory compaction). (#8505) Summary: In https://github.com/facebook/rocksdb/issues/8454, I introduced a new process baptized `MemPurge` (memtable garbage collection). This new PR is built upon this past mempurge prototype. In this PR, I made the `mempurge` process a background task, which provides superior performance since the mempurge process does not cling on the db_mutex anymore, and addresses severe restrictions from the past iteration (including a scenario where the past mempurge was failling, when a memtable was mempurged but was still referred to by an iterator/snapshot/...). Now the mempurge process ressembles an in-memory compaction process: the stack of immutable memtables is filtered out, and the useful payload is used to populate an output memtable. If the output memtable is filled at more than 60% capacity (arbitrary heuristic) the mempurge process is aborted and a regular flush process takes place, else the output memtable is kept in the immutable memtable stack. Note that adding this output memtable to the `imm()` memtable stack does not trigger another flush process, so that the flush thread can go to sleep at the end of a successful mempurge. MemPurge is activated by making the `experimental_allow_mempurge` flag `true`. When activated, the `MemPurge` process will always happen when the flush reason is `kWriteBufferFull`. The 3 unit tests confirm that this process supports `Put`, `Get`, `Delete`, `DeleteRange` operators and is compatible with `Iterators` and `CompactionFilters`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8505 Reviewed By: pdillinger Differential Revision: D29619283 Pulled By: bjlemaire fbshipit-source-id: 8a99bee76b63a8211bff1a00e0ae32360aaece95
2021-07-10 02:16:00 +02:00
ScopedArenaIterator iter(
NewMergingIterator(&(cfd_->internal_comparator()), memtables.data(),
static_cast<int>(memtables.size()), &arena));
auto* ioptions = cfd_->ioptions();
// Place iterator at the First (meaning most recent) key node.
iter->SeekToFirst();
std::unique_ptr<CompactionRangeDelAggregator> range_del_agg(
new CompactionRangeDelAggregator(&(cfd_->internal_comparator()),
existing_snapshots_));
for (auto& rd_iter : range_del_iters) {
range_del_agg->AddTombstones(std::move(rd_iter));
}
// If there is valid data in the memtable,
// or at least range tombstones, copy over the info
// to the new memtable.
if (iter->Valid() || !range_del_agg->IsEmpty()) {
Add simple heuristics for experimental mempurge. (#8583) Summary: Add `experimental_mempurge_policy` option flag and introduce two new `MemPurge` (Memtable Garbage Collection) policies: 'ALWAYS' and 'ALTERNATE'. Default value: ALTERNATE. `ALWAYS`: every flush will first go through a `MemPurge` process. If the output is too big to fit into a single memtable, then the mempurge is aborted and a regular flush process carries on. `ALWAYS` is designed for user that need to reduce the number of L0 SST file created to a strict minimum, and can afford a small dent in performance (possibly hits to CPU usage, read efficiency, and maximum burst write throughput). `ALTERNATE`: a flush is transformed into a `MemPurge` except if one of the memtables being flushed is the product of a previous `MemPurge`. `ALTERNATE` is a good tradeoff between reduction in number of L0 SST files created and performance. `ALTERNATE` perform particularly well for completely random garbage ratios, or garbage ratios anywhere in (0%,50%], and even higher when there is a wild variability in garbage ratios. This PR also includes support for `experimental_mempurge_policy` in `db_bench`. Testing was done locally by replacing all the `MemPurge` policies of the unit tests with `ALTERNATE`, as well as local testing with `db_crashtest.py` `whitebox` and `blackbox`. Overall, if an `ALWAYS` mempurge policy passes the tests, there is no reasons why an `ALTERNATE` policy would fail, and therefore the mempurge policy was set to `ALWAYS` for all mempurge unit tests. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8583 Reviewed By: pdillinger Differential Revision: D29888050 Pulled By: bjlemaire fbshipit-source-id: e2cf26646d66679f6f5fb29842624615610759c1
2021-07-26 20:55:27 +02:00
// MaxSize is the size of a memtable.
size_t maxSize = mutable_cf_options_.write_buffer_size;
Make mempurge a background process (equivalent to in-memory compaction). (#8505) Summary: In https://github.com/facebook/rocksdb/issues/8454, I introduced a new process baptized `MemPurge` (memtable garbage collection). This new PR is built upon this past mempurge prototype. In this PR, I made the `mempurge` process a background task, which provides superior performance since the mempurge process does not cling on the db_mutex anymore, and addresses severe restrictions from the past iteration (including a scenario where the past mempurge was failling, when a memtable was mempurged but was still referred to by an iterator/snapshot/...). Now the mempurge process ressembles an in-memory compaction process: the stack of immutable memtables is filtered out, and the useful payload is used to populate an output memtable. If the output memtable is filled at more than 60% capacity (arbitrary heuristic) the mempurge process is aborted and a regular flush process takes place, else the output memtable is kept in the immutable memtable stack. Note that adding this output memtable to the `imm()` memtable stack does not trigger another flush process, so that the flush thread can go to sleep at the end of a successful mempurge. MemPurge is activated by making the `experimental_allow_mempurge` flag `true`. When activated, the `MemPurge` process will always happen when the flush reason is `kWriteBufferFull`. The 3 unit tests confirm that this process supports `Put`, `Get`, `Delete`, `DeleteRange` operators and is compatible with `Iterators` and `CompactionFilters`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8505 Reviewed By: pdillinger Differential Revision: D29619283 Pulled By: bjlemaire fbshipit-source-id: 8a99bee76b63a8211bff1a00e0ae32360aaece95
2021-07-10 02:16:00 +02:00
std::unique_ptr<CompactionFilter> compaction_filter;
if (ioptions->compaction_filter_factory != nullptr &&
ioptions->compaction_filter_factory->ShouldFilterTableFileCreation(
TableFileCreationReason::kFlush)) {
CompactionFilter::Context ctx;
ctx.is_full_compaction = false;
ctx.is_manual_compaction = false;
ctx.column_family_id = cfd_->GetID();
ctx.reason = TableFileCreationReason::kFlush;
compaction_filter =
ioptions->compaction_filter_factory->CreateCompactionFilter(ctx);
if (compaction_filter != nullptr &&
!compaction_filter->IgnoreSnapshots()) {
s = Status::NotSupported(
"CompactionFilter::IgnoreSnapshots() = false is not supported "
"anymore.");
return s;
}
}
new_mem = new MemTable((cfd_->internal_comparator()), *(cfd_->ioptions()),
mutable_cf_options_, cfd_->write_buffer_mgr(),
earliest_seqno, cfd_->GetID());
Make mempurge a background process (equivalent to in-memory compaction). (#8505) Summary: In https://github.com/facebook/rocksdb/issues/8454, I introduced a new process baptized `MemPurge` (memtable garbage collection). This new PR is built upon this past mempurge prototype. In this PR, I made the `mempurge` process a background task, which provides superior performance since the mempurge process does not cling on the db_mutex anymore, and addresses severe restrictions from the past iteration (including a scenario where the past mempurge was failling, when a memtable was mempurged but was still referred to by an iterator/snapshot/...). Now the mempurge process ressembles an in-memory compaction process: the stack of immutable memtables is filtered out, and the useful payload is used to populate an output memtable. If the output memtable is filled at more than 60% capacity (arbitrary heuristic) the mempurge process is aborted and a regular flush process takes place, else the output memtable is kept in the immutable memtable stack. Note that adding this output memtable to the `imm()` memtable stack does not trigger another flush process, so that the flush thread can go to sleep at the end of a successful mempurge. MemPurge is activated by making the `experimental_allow_mempurge` flag `true`. When activated, the `MemPurge` process will always happen when the flush reason is `kWriteBufferFull`. The 3 unit tests confirm that this process supports `Put`, `Get`, `Delete`, `DeleteRange` operators and is compatible with `Iterators` and `CompactionFilters`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8505 Reviewed By: pdillinger Differential Revision: D29619283 Pulled By: bjlemaire fbshipit-source-id: 8a99bee76b63a8211bff1a00e0ae32360aaece95
2021-07-10 02:16:00 +02:00
assert(new_mem != nullptr);
Env* env = db_options_.env;
assert(env);
MergeHelper merge(
env, (cfd_->internal_comparator()).user_comparator(),
(ioptions->merge_operator).get(), compaction_filter.get(),
ioptions->logger, true /* internal key corruption is not ok */,
existing_snapshots_.empty() ? 0 : existing_snapshots_.back(),
snapshot_checker_);
CompactionIterator c_iter(
iter.get(), (cfd_->internal_comparator()).user_comparator(), &merge,
kMaxSequenceNumber, &existing_snapshots_,
earliest_write_conflict_snapshot_, snapshot_checker_, env,
ShouldReportDetailedTime(env, ioptions->stats),
true /* internal key corruption is not ok */, range_del_agg.get(),
nullptr, ioptions->allow_data_in_errors,
/*compaction=*/nullptr, compaction_filter.get(),
/*shutting_down=*/nullptr,
/*preserve_deletes_seqnum=*/0, /*manual_compaction_paused=*/nullptr,
/*manual_compaction_canceled=*/nullptr, ioptions->info_log,
&(cfd_->GetFullHistoryTsLow()));
// Set earliest sequence number in the new memtable
// to be equal to the earliest sequence number of the
// memtable being flushed (See later if there is a need
// to update this number!).
new_mem->SetEarliestSequenceNumber(earliest_seqno);
// Likewise for first seq number.
new_mem->SetFirstSequenceNumber(first_seqno);
SequenceNumber new_first_seqno = kMaxSequenceNumber;
c_iter.SeekToFirst();
// Key transfer
for (; c_iter.Valid(); c_iter.Next()) {
const ParsedInternalKey ikey = c_iter.ikey();
const Slice value = c_iter.value();
new_first_seqno =
ikey.sequence < new_first_seqno ? ikey.sequence : new_first_seqno;
// Should we update "OldestKeyTime" ???? -> timestamp appear
// to still be an "experimental" feature.
s = new_mem->Add(
ikey.sequence, ikey.type, ikey.user_key, value,
nullptr, // KV protection info set as nullptr since it
// should only be useful for the first add to
// the original memtable.
false, // : allow concurrent_memtable_writes_
// Not seen as necessary for now.
nullptr, // get_post_process_info(m) must be nullptr
// when concurrent_memtable_writes is switched off.
nullptr); // hint, only used when concurrent_memtable_writes_
// is switched on.
if (!s.ok()) {
break;
}
// If new_mem has size greater than maxSize,
// then rollback to regular flush operation,
// and destroy new_mem.
if (new_mem->ApproximateMemoryUsage() > maxSize) {
s = Status::Aborted("Mempurge filled more than one memtable.");
Add simple heuristics for experimental mempurge. (#8583) Summary: Add `experimental_mempurge_policy` option flag and introduce two new `MemPurge` (Memtable Garbage Collection) policies: 'ALWAYS' and 'ALTERNATE'. Default value: ALTERNATE. `ALWAYS`: every flush will first go through a `MemPurge` process. If the output is too big to fit into a single memtable, then the mempurge is aborted and a regular flush process carries on. `ALWAYS` is designed for user that need to reduce the number of L0 SST file created to a strict minimum, and can afford a small dent in performance (possibly hits to CPU usage, read efficiency, and maximum burst write throughput). `ALTERNATE`: a flush is transformed into a `MemPurge` except if one of the memtables being flushed is the product of a previous `MemPurge`. `ALTERNATE` is a good tradeoff between reduction in number of L0 SST files created and performance. `ALTERNATE` perform particularly well for completely random garbage ratios, or garbage ratios anywhere in (0%,50%], and even higher when there is a wild variability in garbage ratios. This PR also includes support for `experimental_mempurge_policy` in `db_bench`. Testing was done locally by replacing all the `MemPurge` policies of the unit tests with `ALTERNATE`, as well as local testing with `db_crashtest.py` `whitebox` and `blackbox`. Overall, if an `ALWAYS` mempurge policy passes the tests, there is no reasons why an `ALTERNATE` policy would fail, and therefore the mempurge policy was set to `ALWAYS` for all mempurge unit tests. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8583 Reviewed By: pdillinger Differential Revision: D29888050 Pulled By: bjlemaire fbshipit-source-id: e2cf26646d66679f6f5fb29842624615610759c1
2021-07-26 20:55:27 +02:00
new_mem_capacity = 1.0;
Make mempurge a background process (equivalent to in-memory compaction). (#8505) Summary: In https://github.com/facebook/rocksdb/issues/8454, I introduced a new process baptized `MemPurge` (memtable garbage collection). This new PR is built upon this past mempurge prototype. In this PR, I made the `mempurge` process a background task, which provides superior performance since the mempurge process does not cling on the db_mutex anymore, and addresses severe restrictions from the past iteration (including a scenario where the past mempurge was failling, when a memtable was mempurged but was still referred to by an iterator/snapshot/...). Now the mempurge process ressembles an in-memory compaction process: the stack of immutable memtables is filtered out, and the useful payload is used to populate an output memtable. If the output memtable is filled at more than 60% capacity (arbitrary heuristic) the mempurge process is aborted and a regular flush process takes place, else the output memtable is kept in the immutable memtable stack. Note that adding this output memtable to the `imm()` memtable stack does not trigger another flush process, so that the flush thread can go to sleep at the end of a successful mempurge. MemPurge is activated by making the `experimental_allow_mempurge` flag `true`. When activated, the `MemPurge` process will always happen when the flush reason is `kWriteBufferFull`. The 3 unit tests confirm that this process supports `Put`, `Get`, `Delete`, `DeleteRange` operators and is compatible with `Iterators` and `CompactionFilters`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8505 Reviewed By: pdillinger Differential Revision: D29619283 Pulled By: bjlemaire fbshipit-source-id: 8a99bee76b63a8211bff1a00e0ae32360aaece95
2021-07-10 02:16:00 +02:00
break;
}
}
// Check status and propagate
// potential error status from c_iter
if (!s.ok()) {
c_iter.status().PermitUncheckedError();
} else if (!c_iter.status().ok()) {
s = c_iter.status();
}
// Range tombstone transfer.
if (s.ok()) {
auto range_del_it = range_del_agg->NewIterator();
for (range_del_it->SeekToFirst(); range_del_it->Valid();
range_del_it->Next()) {
auto tombstone = range_del_it->Tombstone();
new_first_seqno =
tombstone.seq_ < new_first_seqno ? tombstone.seq_ : new_first_seqno;
s = new_mem->Add(
tombstone.seq_, // Sequence number
kTypeRangeDeletion, // KV type
tombstone.start_key_, // Key is start key.
tombstone.end_key_, // Value is end key.
nullptr, // KV protection info set as nullptr since it
// should only be useful for the first add to
// the original memtable.
false, // : allow concurrent_memtable_writes_
// Not seen as necessary for now.
nullptr, // get_post_process_info(m) must be nullptr
// when concurrent_memtable_writes is switched off.
nullptr); // hint, only used when concurrent_memtable_writes_
// is switched on.
if (!s.ok()) {
break;
}
// If new_mem has size greater than maxSize,
// then rollback to regular flush operation,
// and destroy new_mem.
if (new_mem->ApproximateMemoryUsage() > maxSize) {
s = Status::Aborted(Slice("Mempurge filled more than one memtable."));
Add simple heuristics for experimental mempurge. (#8583) Summary: Add `experimental_mempurge_policy` option flag and introduce two new `MemPurge` (Memtable Garbage Collection) policies: 'ALWAYS' and 'ALTERNATE'. Default value: ALTERNATE. `ALWAYS`: every flush will first go through a `MemPurge` process. If the output is too big to fit into a single memtable, then the mempurge is aborted and a regular flush process carries on. `ALWAYS` is designed for user that need to reduce the number of L0 SST file created to a strict minimum, and can afford a small dent in performance (possibly hits to CPU usage, read efficiency, and maximum burst write throughput). `ALTERNATE`: a flush is transformed into a `MemPurge` except if one of the memtables being flushed is the product of a previous `MemPurge`. `ALTERNATE` is a good tradeoff between reduction in number of L0 SST files created and performance. `ALTERNATE` perform particularly well for completely random garbage ratios, or garbage ratios anywhere in (0%,50%], and even higher when there is a wild variability in garbage ratios. This PR also includes support for `experimental_mempurge_policy` in `db_bench`. Testing was done locally by replacing all the `MemPurge` policies of the unit tests with `ALTERNATE`, as well as local testing with `db_crashtest.py` `whitebox` and `blackbox`. Overall, if an `ALWAYS` mempurge policy passes the tests, there is no reasons why an `ALTERNATE` policy would fail, and therefore the mempurge policy was set to `ALWAYS` for all mempurge unit tests. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8583 Reviewed By: pdillinger Differential Revision: D29888050 Pulled By: bjlemaire fbshipit-source-id: e2cf26646d66679f6f5fb29842624615610759c1
2021-07-26 20:55:27 +02:00
new_mem_capacity = 1.0;
Make mempurge a background process (equivalent to in-memory compaction). (#8505) Summary: In https://github.com/facebook/rocksdb/issues/8454, I introduced a new process baptized `MemPurge` (memtable garbage collection). This new PR is built upon this past mempurge prototype. In this PR, I made the `mempurge` process a background task, which provides superior performance since the mempurge process does not cling on the db_mutex anymore, and addresses severe restrictions from the past iteration (including a scenario where the past mempurge was failling, when a memtable was mempurged but was still referred to by an iterator/snapshot/...). Now the mempurge process ressembles an in-memory compaction process: the stack of immutable memtables is filtered out, and the useful payload is used to populate an output memtable. If the output memtable is filled at more than 60% capacity (arbitrary heuristic) the mempurge process is aborted and a regular flush process takes place, else the output memtable is kept in the immutable memtable stack. Note that adding this output memtable to the `imm()` memtable stack does not trigger another flush process, so that the flush thread can go to sleep at the end of a successful mempurge. MemPurge is activated by making the `experimental_allow_mempurge` flag `true`. When activated, the `MemPurge` process will always happen when the flush reason is `kWriteBufferFull`. The 3 unit tests confirm that this process supports `Put`, `Get`, `Delete`, `DeleteRange` operators and is compatible with `Iterators` and `CompactionFilters`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8505 Reviewed By: pdillinger Differential Revision: D29619283 Pulled By: bjlemaire fbshipit-source-id: 8a99bee76b63a8211bff1a00e0ae32360aaece95
2021-07-10 02:16:00 +02:00
break;
}
}
}
// If everything happened smoothly and new_mem contains valid data,
// decide if it is flushed to storage or kept in the imm()
// memtable list (memory).
if (s.ok() && (new_first_seqno != kMaxSequenceNumber)) {
// Rectify the first sequence number, which (unlike the earliest seq
// number) needs to be present in the new memtable.
new_mem->SetFirstSequenceNumber(new_first_seqno);
// The new_mem is added to the list of immutable memtables
Add simple heuristics for experimental mempurge. (#8583) Summary: Add `experimental_mempurge_policy` option flag and introduce two new `MemPurge` (Memtable Garbage Collection) policies: 'ALWAYS' and 'ALTERNATE'. Default value: ALTERNATE. `ALWAYS`: every flush will first go through a `MemPurge` process. If the output is too big to fit into a single memtable, then the mempurge is aborted and a regular flush process carries on. `ALWAYS` is designed for user that need to reduce the number of L0 SST file created to a strict minimum, and can afford a small dent in performance (possibly hits to CPU usage, read efficiency, and maximum burst write throughput). `ALTERNATE`: a flush is transformed into a `MemPurge` except if one of the memtables being flushed is the product of a previous `MemPurge`. `ALTERNATE` is a good tradeoff between reduction in number of L0 SST files created and performance. `ALTERNATE` perform particularly well for completely random garbage ratios, or garbage ratios anywhere in (0%,50%], and even higher when there is a wild variability in garbage ratios. This PR also includes support for `experimental_mempurge_policy` in `db_bench`. Testing was done locally by replacing all the `MemPurge` policies of the unit tests with `ALTERNATE`, as well as local testing with `db_crashtest.py` `whitebox` and `blackbox`. Overall, if an `ALWAYS` mempurge policy passes the tests, there is no reasons why an `ALTERNATE` policy would fail, and therefore the mempurge policy was set to `ALWAYS` for all mempurge unit tests. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8583 Reviewed By: pdillinger Differential Revision: D29888050 Pulled By: bjlemaire fbshipit-source-id: e2cf26646d66679f6f5fb29842624615610759c1
2021-07-26 20:55:27 +02:00
// only if it filled at less than 100% capacity and isn't flagged
// as in need of being flushed.
if (new_mem->ApproximateMemoryUsage() < maxSize &&
!(new_mem->ShouldFlushNow())) {
Make mempurge a background process (equivalent to in-memory compaction). (#8505) Summary: In https://github.com/facebook/rocksdb/issues/8454, I introduced a new process baptized `MemPurge` (memtable garbage collection). This new PR is built upon this past mempurge prototype. In this PR, I made the `mempurge` process a background task, which provides superior performance since the mempurge process does not cling on the db_mutex anymore, and addresses severe restrictions from the past iteration (including a scenario where the past mempurge was failling, when a memtable was mempurged but was still referred to by an iterator/snapshot/...). Now the mempurge process ressembles an in-memory compaction process: the stack of immutable memtables is filtered out, and the useful payload is used to populate an output memtable. If the output memtable is filled at more than 60% capacity (arbitrary heuristic) the mempurge process is aborted and a regular flush process takes place, else the output memtable is kept in the immutable memtable stack. Note that adding this output memtable to the `imm()` memtable stack does not trigger another flush process, so that the flush thread can go to sleep at the end of a successful mempurge. MemPurge is activated by making the `experimental_allow_mempurge` flag `true`. When activated, the `MemPurge` process will always happen when the flush reason is `kWriteBufferFull`. The 3 unit tests confirm that this process supports `Put`, `Get`, `Delete`, `DeleteRange` operators and is compatible with `Iterators` and `CompactionFilters`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8505 Reviewed By: pdillinger Differential Revision: D29619283 Pulled By: bjlemaire fbshipit-source-id: 8a99bee76b63a8211bff1a00e0ae32360aaece95
2021-07-10 02:16:00 +02:00
db_mutex_->Lock();
Add simple heuristics for experimental mempurge. (#8583) Summary: Add `experimental_mempurge_policy` option flag and introduce two new `MemPurge` (Memtable Garbage Collection) policies: 'ALWAYS' and 'ALTERNATE'. Default value: ALTERNATE. `ALWAYS`: every flush will first go through a `MemPurge` process. If the output is too big to fit into a single memtable, then the mempurge is aborted and a regular flush process carries on. `ALWAYS` is designed for user that need to reduce the number of L0 SST file created to a strict minimum, and can afford a small dent in performance (possibly hits to CPU usage, read efficiency, and maximum burst write throughput). `ALTERNATE`: a flush is transformed into a `MemPurge` except if one of the memtables being flushed is the product of a previous `MemPurge`. `ALTERNATE` is a good tradeoff between reduction in number of L0 SST files created and performance. `ALTERNATE` perform particularly well for completely random garbage ratios, or garbage ratios anywhere in (0%,50%], and even higher when there is a wild variability in garbage ratios. This PR also includes support for `experimental_mempurge_policy` in `db_bench`. Testing was done locally by replacing all the `MemPurge` policies of the unit tests with `ALTERNATE`, as well as local testing with `db_crashtest.py` `whitebox` and `blackbox`. Overall, if an `ALWAYS` mempurge policy passes the tests, there is no reasons why an `ALTERNATE` policy would fail, and therefore the mempurge policy was set to `ALWAYS` for all mempurge unit tests. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8583 Reviewed By: pdillinger Differential Revision: D29888050 Pulled By: bjlemaire fbshipit-source-id: e2cf26646d66679f6f5fb29842624615610759c1
2021-07-26 20:55:27 +02:00
uint64_t new_mem_id = mems_[0]->GetID();
// Copy lowest memtable ID
// House keeping work.
for (MemTable* mt : mems_) {
new_mem_id = mt->GetID() < new_mem_id ? mt->GetID() : new_mem_id;
// Note: if m is not a previous mempurge output memtable,
// nothing happens.
cfd_->imm()->RemoveMemPurgeOutputID(mt->GetID());
}
new_mem->SetID(new_mem_id);
cfd_->imm()->AddMemPurgeOutputID(new_mem_id);
Fix db stress crash mempurge (#8604) Summary: The db_stress crash was caused by a call to `IsFlushPending()` made by a stats function which triggered an `assert([false])`, which I didn't plan when I created the `trigger_flush` bool. It turns out that this bool variable is not useful: I created it because I thought the `imm_flush_needed` atomic bool would actually trigger a flush. It turns out that this bool is only checked in `IsFlushPending` - this is its only use - and a flush is triggered by either a background thread checking on the imm array, or by an explicit call to `SchedulePendingFlush` which creates a flush request, that is then added to a flush request queue. In this PR, I reverted the MemtableList::Add function to what it was before my changes. I tested the fix by running the exact command line that deterministically triggered the assert error (see below), which confirmed that this is where the error was coming from. I also run `db_crashtest.py whitebox` and `blackbox` for a couple hours locally before committing this PR. Experiment run: ```./db_stress --acquire_snapshot_one_in=0 --allow_concurrent_memtable_write=1 --avoid_flush_during_recovery=0 --avoid_unnecessary_blocking_io=1 --backup_max_size=104857600 --backup_one_in=100000 --batch_protection_bytes_per_key=0 --block_size=16384 --bloom_bits=76.90653425292307 --bottommost_compression_type=disable --cache_index_and_filter_blocks=1 --cache_size=1048576 --checkpoint_one_in=1000000 --checksum_type=kCRC32c --clear_column_family_one_in=0 --column_families=1 --compact_files_one_in=1000000 --compact_range_one_in=0 --compaction_ttl=2 --compression_max_dict_buffer_bytes=0 --compression_max_dict_bytes=0 --compression_parallel_threads=1 --compression_type=zstd --compression_zstd_max_train_bytes=0 --continuous_verification_interval=0 --db=/dev/shm/rocksdb/rocksdb_crashtest_blackbox --db_write_buffer_size=0 --delpercent=4 --delrangepercent=1 --destroy_db_initially=0 --enable_compaction_filter=1 --enable_pipelined_write=0 --expected_values_path=/dev/shm/rocksdb/rocksdb_crashtest_expected --experimental_allow_mempurge=1 --experimental_mempurge_policy=kAlternate --fail_if_options_file_error=1 --file_checksum_impl=none --flush_one_in=1000000 --format_version=2 --get_current_wal_file_one_in=0 --get_live_files_one_in=1000000 --get_property_one_in=1000000 --get_sorted_wal_files_one_in=0 --index_block_restart_interval=14 --index_type=0 --iterpercent=0 --key_len_percent_dist=1,30,69 --level_compaction_dynamic_level_bytes=False --long_running_snapshots=1 --mark_for_compaction_one_file_in=10 --max_background_compactions=1 --max_bytes_for_level_base=67108864 --max_key=100000000 --max_key_len=3 --max_manifest_file_size=1073741824 --max_write_batch_group_size_bytes=64 --max_write_buffer_number=3 --max_write_buffer_size_to_maintain=0 --memtablerep=skip_list --mmap_read=0 --mock_direct_io=True --nooverwritepercent=1 --open_files=-1 --open_metadata_write_fault_one_in=8 --open_read_fault_one_in=32 --open_write_fault_one_in=16 --ops_per_thread=100000000 --optimize_filters_for_memory=1 --paranoid_file_checks=0 --partition_filters=0 --partition_pinning=0 --pause_background_one_in=1000000 --periodic_compaction_seconds=1000 --prefix_size=-1 --prefixpercent=0 --progress_reports=0 --read_fault_one_in=0 --readpercent=60 --recycle_log_file_num=1 --reopen=20 --set_options_one_in=0 --snapshot_hold_ops=100000 --sst_file_manager_bytes_per_sec=104857600 --sst_file_manager_bytes_per_truncate=0 --subcompactions=3 --sync=1 --sync_fault_injection=False --target_file_size_base=16777216 --target_file_size_multiplier=1 --test_batches_snapshots=0 --top_level_index_pinning=1 --unpartitioned_pinning=3 --use_clock_cache=0 --use_direct_io_for_flush_and_compaction=1 --use_direct_reads=0 --use_full_merge_v1=1 --use_merge=0 --use_multiget=0 --use_ribbon_filter=1 --user_timestamp_size=0 --verify_checksum=1 --verify_checksum_one_in=1000000 --verify_db_one_in=100000 --write_buffer_size=33554432 --write_dbid_to_manifest=1 --writepercent=35``` Pull Request resolved: https://github.com/facebook/rocksdb/pull/8604 Reviewed By: pdillinger Differential Revision: D30047295 Pulled By: bjlemaire fbshipit-source-id: b9e379bfa3d6b9bd2b275725fb0bca4bd81a3dbe
2021-08-03 05:25:39 +02:00
// This addition will not trigger another flush, because
// we do not call SchedulePendingFlush().
cfd_->imm()->Add(new_mem, &job_context_->memtables_to_free);
Make mempurge a background process (equivalent to in-memory compaction). (#8505) Summary: In https://github.com/facebook/rocksdb/issues/8454, I introduced a new process baptized `MemPurge` (memtable garbage collection). This new PR is built upon this past mempurge prototype. In this PR, I made the `mempurge` process a background task, which provides superior performance since the mempurge process does not cling on the db_mutex anymore, and addresses severe restrictions from the past iteration (including a scenario where the past mempurge was failling, when a memtable was mempurged but was still referred to by an iterator/snapshot/...). Now the mempurge process ressembles an in-memory compaction process: the stack of immutable memtables is filtered out, and the useful payload is used to populate an output memtable. If the output memtable is filled at more than 60% capacity (arbitrary heuristic) the mempurge process is aborted and a regular flush process takes place, else the output memtable is kept in the immutable memtable stack. Note that adding this output memtable to the `imm()` memtable stack does not trigger another flush process, so that the flush thread can go to sleep at the end of a successful mempurge. MemPurge is activated by making the `experimental_allow_mempurge` flag `true`. When activated, the `MemPurge` process will always happen when the flush reason is `kWriteBufferFull`. The 3 unit tests confirm that this process supports `Put`, `Get`, `Delete`, `DeleteRange` operators and is compatible with `Iterators` and `CompactionFilters`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8505 Reviewed By: pdillinger Differential Revision: D29619283 Pulled By: bjlemaire fbshipit-source-id: 8a99bee76b63a8211bff1a00e0ae32360aaece95
2021-07-10 02:16:00 +02:00
new_mem->Ref();
db_mutex_->Unlock();
} else {
s = Status::Aborted(Slice("Mempurge filled more than one memtable."));
Add simple heuristics for experimental mempurge. (#8583) Summary: Add `experimental_mempurge_policy` option flag and introduce two new `MemPurge` (Memtable Garbage Collection) policies: 'ALWAYS' and 'ALTERNATE'. Default value: ALTERNATE. `ALWAYS`: every flush will first go through a `MemPurge` process. If the output is too big to fit into a single memtable, then the mempurge is aborted and a regular flush process carries on. `ALWAYS` is designed for user that need to reduce the number of L0 SST file created to a strict minimum, and can afford a small dent in performance (possibly hits to CPU usage, read efficiency, and maximum burst write throughput). `ALTERNATE`: a flush is transformed into a `MemPurge` except if one of the memtables being flushed is the product of a previous `MemPurge`. `ALTERNATE` is a good tradeoff between reduction in number of L0 SST files created and performance. `ALTERNATE` perform particularly well for completely random garbage ratios, or garbage ratios anywhere in (0%,50%], and even higher when there is a wild variability in garbage ratios. This PR also includes support for `experimental_mempurge_policy` in `db_bench`. Testing was done locally by replacing all the `MemPurge` policies of the unit tests with `ALTERNATE`, as well as local testing with `db_crashtest.py` `whitebox` and `blackbox`. Overall, if an `ALWAYS` mempurge policy passes the tests, there is no reasons why an `ALTERNATE` policy would fail, and therefore the mempurge policy was set to `ALWAYS` for all mempurge unit tests. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8583 Reviewed By: pdillinger Differential Revision: D29888050 Pulled By: bjlemaire fbshipit-source-id: e2cf26646d66679f6f5fb29842624615610759c1
2021-07-26 20:55:27 +02:00
new_mem_capacity = 1.0;
Make mempurge a background process (equivalent to in-memory compaction). (#8505) Summary: In https://github.com/facebook/rocksdb/issues/8454, I introduced a new process baptized `MemPurge` (memtable garbage collection). This new PR is built upon this past mempurge prototype. In this PR, I made the `mempurge` process a background task, which provides superior performance since the mempurge process does not cling on the db_mutex anymore, and addresses severe restrictions from the past iteration (including a scenario where the past mempurge was failling, when a memtable was mempurged but was still referred to by an iterator/snapshot/...). Now the mempurge process ressembles an in-memory compaction process: the stack of immutable memtables is filtered out, and the useful payload is used to populate an output memtable. If the output memtable is filled at more than 60% capacity (arbitrary heuristic) the mempurge process is aborted and a regular flush process takes place, else the output memtable is kept in the immutable memtable stack. Note that adding this output memtable to the `imm()` memtable stack does not trigger another flush process, so that the flush thread can go to sleep at the end of a successful mempurge. MemPurge is activated by making the `experimental_allow_mempurge` flag `true`. When activated, the `MemPurge` process will always happen when the flush reason is `kWriteBufferFull`. The 3 unit tests confirm that this process supports `Put`, `Get`, `Delete`, `DeleteRange` operators and is compatible with `Iterators` and `CompactionFilters`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8505 Reviewed By: pdillinger Differential Revision: D29619283 Pulled By: bjlemaire fbshipit-source-id: 8a99bee76b63a8211bff1a00e0ae32360aaece95
2021-07-10 02:16:00 +02:00
if (new_mem) {
job_context_->memtables_to_free.push_back(new_mem);
}
}
} else {
// In this case, the newly allocated new_mem is empty.
assert(new_mem != nullptr);
job_context_->memtables_to_free.push_back(new_mem);
}
}
// Reacquire the mutex for WriteLevel0 function.
db_mutex_->Lock();
// If mempurge successful, don't write input tables to level0,
// but write any full output table to level0.
if (s.ok()) {
TEST_SYNC_POINT("DBImpl::FlushJob:MemPurgeSuccessful");
} else {
TEST_SYNC_POINT("DBImpl::FlushJob:MemPurgeUnsuccessful");
}
Add simple heuristics for experimental mempurge. (#8583) Summary: Add `experimental_mempurge_policy` option flag and introduce two new `MemPurge` (Memtable Garbage Collection) policies: 'ALWAYS' and 'ALTERNATE'. Default value: ALTERNATE. `ALWAYS`: every flush will first go through a `MemPurge` process. If the output is too big to fit into a single memtable, then the mempurge is aborted and a regular flush process carries on. `ALWAYS` is designed for user that need to reduce the number of L0 SST file created to a strict minimum, and can afford a small dent in performance (possibly hits to CPU usage, read efficiency, and maximum burst write throughput). `ALTERNATE`: a flush is transformed into a `MemPurge` except if one of the memtables being flushed is the product of a previous `MemPurge`. `ALTERNATE` is a good tradeoff between reduction in number of L0 SST files created and performance. `ALTERNATE` perform particularly well for completely random garbage ratios, or garbage ratios anywhere in (0%,50%], and even higher when there is a wild variability in garbage ratios. This PR also includes support for `experimental_mempurge_policy` in `db_bench`. Testing was done locally by replacing all the `MemPurge` policies of the unit tests with `ALTERNATE`, as well as local testing with `db_crashtest.py` `whitebox` and `blackbox`. Overall, if an `ALWAYS` mempurge policy passes the tests, there is no reasons why an `ALTERNATE` policy would fail, and therefore the mempurge policy was set to `ALWAYS` for all mempurge unit tests. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8583 Reviewed By: pdillinger Differential Revision: D29888050 Pulled By: bjlemaire fbshipit-source-id: e2cf26646d66679f6f5fb29842624615610759c1
2021-07-26 20:55:27 +02:00
const uint64_t micros = clock_->NowMicros() - start_micros;
const uint64_t cpu_micros = clock_->CPUNanos() / 1000 - start_cpu_micros;
ROCKS_LOG_INFO(db_options_.info_log,
"[%s] [JOB %d] Mempurge lasted %" PRIu64
" microseconds, and %" PRIu64
" cpu "
"microseconds. Status is %s ok. Perc capacity: %f\n",
cfd_->GetName().c_str(), job_context_->job_id, micros,
cpu_micros, s.ok() ? "" : "not", new_mem_capacity);
Make mempurge a background process (equivalent to in-memory compaction). (#8505) Summary: In https://github.com/facebook/rocksdb/issues/8454, I introduced a new process baptized `MemPurge` (memtable garbage collection). This new PR is built upon this past mempurge prototype. In this PR, I made the `mempurge` process a background task, which provides superior performance since the mempurge process does not cling on the db_mutex anymore, and addresses severe restrictions from the past iteration (including a scenario where the past mempurge was failling, when a memtable was mempurged but was still referred to by an iterator/snapshot/...). Now the mempurge process ressembles an in-memory compaction process: the stack of immutable memtables is filtered out, and the useful payload is used to populate an output memtable. If the output memtable is filled at more than 60% capacity (arbitrary heuristic) the mempurge process is aborted and a regular flush process takes place, else the output memtable is kept in the immutable memtable stack. Note that adding this output memtable to the `imm()` memtable stack does not trigger another flush process, so that the flush thread can go to sleep at the end of a successful mempurge. MemPurge is activated by making the `experimental_allow_mempurge` flag `true`. When activated, the `MemPurge` process will always happen when the flush reason is `kWriteBufferFull`. The 3 unit tests confirm that this process supports `Put`, `Get`, `Delete`, `DeleteRange` operators and is compatible with `Iterators` and `CompactionFilters`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8505 Reviewed By: pdillinger Differential Revision: D29619283 Pulled By: bjlemaire fbshipit-source-id: 8a99bee76b63a8211bff1a00e0ae32360aaece95
2021-07-10 02:16:00 +02:00
return s;
}
Add simple heuristics for experimental mempurge. (#8583) Summary: Add `experimental_mempurge_policy` option flag and introduce two new `MemPurge` (Memtable Garbage Collection) policies: 'ALWAYS' and 'ALTERNATE'. Default value: ALTERNATE. `ALWAYS`: every flush will first go through a `MemPurge` process. If the output is too big to fit into a single memtable, then the mempurge is aborted and a regular flush process carries on. `ALWAYS` is designed for user that need to reduce the number of L0 SST file created to a strict minimum, and can afford a small dent in performance (possibly hits to CPU usage, read efficiency, and maximum burst write throughput). `ALTERNATE`: a flush is transformed into a `MemPurge` except if one of the memtables being flushed is the product of a previous `MemPurge`. `ALTERNATE` is a good tradeoff between reduction in number of L0 SST files created and performance. `ALTERNATE` perform particularly well for completely random garbage ratios, or garbage ratios anywhere in (0%,50%], and even higher when there is a wild variability in garbage ratios. This PR also includes support for `experimental_mempurge_policy` in `db_bench`. Testing was done locally by replacing all the `MemPurge` policies of the unit tests with `ALTERNATE`, as well as local testing with `db_crashtest.py` `whitebox` and `blackbox`. Overall, if an `ALWAYS` mempurge policy passes the tests, there is no reasons why an `ALTERNATE` policy would fail, and therefore the mempurge policy was set to `ALWAYS` for all mempurge unit tests. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8583 Reviewed By: pdillinger Differential Revision: D29888050 Pulled By: bjlemaire fbshipit-source-id: e2cf26646d66679f6f5fb29842624615610759c1
2021-07-26 20:55:27 +02:00
bool FlushJob::MemPurgeDecider() {
Memtable sampling for mempurge heuristic. (#8628) Summary: Changes the API of the MemPurge process: the `bool experimental_allow_mempurge` and `experimental_mempurge_policy` flags have been replaced by a `double experimental_mempurge_threshold` option. This change of API reflects another major change introduced in this PR: the MemPurgeDecider() function now works by sampling the memtables being flushed to estimate the overall amount of useful payload (payload minus the garbage), and then compare this useful payload estimate with the `double experimental_mempurge_threshold` value. Therefore, when the value of this flag is `0.0` (default value), mempurge is simply deactivated. On the other hand, a value of `DBL_MAX` would be equivalent to always going through a mempurge regardless of the garbage ratio estimate. At the moment, a `double experimental_mempurge_threshold` value else than 0.0 or `DBL_MAX` is opnly supported`with the `SkipList` memtable representation. Regarding the sampling, this PR includes the introduction of a `MemTable::UniqueRandomSample` function that collects (approximately) random entries from the memtable by using the new `SkipList::Iterator::RandomSeek()` under the hood, or by iterating through each memtable entry, depending on the target sample size and the total number of entries. The unit tests have been readapted to support this new API. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8628 Reviewed By: pdillinger Differential Revision: D30149315 Pulled By: bjlemaire fbshipit-source-id: 1feef5390c95db6f4480ab4434716533d3947f27
2021-08-11 03:07:48 +02:00
double threshold = db_options_.experimental_mempurge_threshold;
// Never trigger mempurge if threshold is not a strictly positive value.
if (!(threshold > 0.0)) {
return false;
}
if (threshold > (1.0 * mems_.size())) {
Add simple heuristics for experimental mempurge. (#8583) Summary: Add `experimental_mempurge_policy` option flag and introduce two new `MemPurge` (Memtable Garbage Collection) policies: 'ALWAYS' and 'ALTERNATE'. Default value: ALTERNATE. `ALWAYS`: every flush will first go through a `MemPurge` process. If the output is too big to fit into a single memtable, then the mempurge is aborted and a regular flush process carries on. `ALWAYS` is designed for user that need to reduce the number of L0 SST file created to a strict minimum, and can afford a small dent in performance (possibly hits to CPU usage, read efficiency, and maximum burst write throughput). `ALTERNATE`: a flush is transformed into a `MemPurge` except if one of the memtables being flushed is the product of a previous `MemPurge`. `ALTERNATE` is a good tradeoff between reduction in number of L0 SST files created and performance. `ALTERNATE` perform particularly well for completely random garbage ratios, or garbage ratios anywhere in (0%,50%], and even higher when there is a wild variability in garbage ratios. This PR also includes support for `experimental_mempurge_policy` in `db_bench`. Testing was done locally by replacing all the `MemPurge` policies of the unit tests with `ALTERNATE`, as well as local testing with `db_crashtest.py` `whitebox` and `blackbox`. Overall, if an `ALWAYS` mempurge policy passes the tests, there is no reasons why an `ALTERNATE` policy would fail, and therefore the mempurge policy was set to `ALWAYS` for all mempurge unit tests. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8583 Reviewed By: pdillinger Differential Revision: D29888050 Pulled By: bjlemaire fbshipit-source-id: e2cf26646d66679f6f5fb29842624615610759c1
2021-07-26 20:55:27 +02:00
return true;
}
Memtable sampling for mempurge heuristic. (#8628) Summary: Changes the API of the MemPurge process: the `bool experimental_allow_mempurge` and `experimental_mempurge_policy` flags have been replaced by a `double experimental_mempurge_threshold` option. This change of API reflects another major change introduced in this PR: the MemPurgeDecider() function now works by sampling the memtables being flushed to estimate the overall amount of useful payload (payload minus the garbage), and then compare this useful payload estimate with the `double experimental_mempurge_threshold` value. Therefore, when the value of this flag is `0.0` (default value), mempurge is simply deactivated. On the other hand, a value of `DBL_MAX` would be equivalent to always going through a mempurge regardless of the garbage ratio estimate. At the moment, a `double experimental_mempurge_threshold` value else than 0.0 or `DBL_MAX` is opnly supported`with the `SkipList` memtable representation. Regarding the sampling, this PR includes the introduction of a `MemTable::UniqueRandomSample` function that collects (approximately) random entries from the memtable by using the new `SkipList::Iterator::RandomSeek()` under the hood, or by iterating through each memtable entry, depending on the target sample size and the total number of entries. The unit tests have been readapted to support this new API. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8628 Reviewed By: pdillinger Differential Revision: D30149315 Pulled By: bjlemaire fbshipit-source-id: 1feef5390c95db6f4480ab4434716533d3947f27
2021-08-11 03:07:48 +02:00
// Payload and useful_payload (in bytes).
// The useful payload ratio of a given MemTable
// is estimated to be useful_payload/payload.
uint64_t payload = 0, useful_payload = 0;
// If estimated_useful_payload is > threshold,
// then flush to storage, else MemPurge.
double estimated_useful_payload = 0.0;
// Cochran formula for determining sample size.
// 95% confidence interval, 7% precision.
// n0 = (1.96*1.96)*0.25/(0.07*0.07) = 196.0
double n0 = 196.0;
ReadOptions ro;
ro.total_order_seek = true;
// Iterate over each memtable of the set.
for (MemTable* mt : mems_) {
// If the memtable is the output of a previous mempurge,
// its approximate useful payload ratio is already calculated.
if (cfd_->imm()->IsMemPurgeOutput(mt->GetID())) {
// We make the assumption that this memtable is already
// free of garbage (garbage underestimation).
estimated_useful_payload += mt->ApproximateMemoryUsage();
} else {
// Else sample from the table.
uint64_t nentries = mt->num_entries();
// Corrected Cochran formula for small populations
// (converges to n0 for large populations).
uint64_t target_sample_size =
static_cast<uint64_t>(ceil(n0 / (1.0 + (n0 / nentries))));
std::unordered_set<const char*> sentries = {};
// Populate sample entries set.
mt->UniqueRandomSample(target_sample_size, &sentries);
// Estimate the garbage ratio by comparing if
// each sample corresponds to a valid entry.
for (const char* ss : sentries) {
ParsedInternalKey res;
Slice entry_slice = GetLengthPrefixedSlice(ss);
Status parse_s =
ParseInternalKey(entry_slice, &res, true /*log_err_key*/);
if (!parse_s.ok()) {
ROCKS_LOG_WARN(db_options_.info_log,
"Memtable Decider: ParseInternalKey did not parse "
"entry_slice %s"
"successfully.",
entry_slice.data());
}
LookupKey lkey(res.user_key, kMaxSequenceNumber);
std::string vget;
Status s;
MergeContext merge_context;
SequenceNumber max_covering_tombstone_seq = 0, sqno = 0;
// Pick the oldest existing snapshot that is more recent
// than the sequence number of the sampled entry.
SequenceNumber min_seqno_snapshot = kMaxSequenceNumber;
SnapshotImpl min_snapshot;
for (SequenceNumber seq_num : existing_snapshots_) {
if (seq_num > res.sequence && seq_num < min_seqno_snapshot) {
min_seqno_snapshot = seq_num;
}
}
min_snapshot.number_ = min_seqno_snapshot;
ro.snapshot =
min_seqno_snapshot < kMaxSequenceNumber ? &min_snapshot : nullptr;
// Estimate if the sample entry is valid or not.
bool gres = mt->Get(lkey, &vget, nullptr, &s, &merge_context,
&max_covering_tombstone_seq, &sqno, ro);
if (!gres) {
ROCKS_LOG_WARN(
db_options_.info_log,
"Memtable Get returned false when Get(sampled entry). "
"Yet each sample entry should exist somewhere in the memtable, "
"unrelated to whether it has been deleted or not.");
}
payload += entry_slice.size();
// TODO(bjlemaire): evaluate typeMerge.
// This is where the sampled entry is estimated to be
// garbage or not. Note that this is a garbage *estimation*
// because we do not include certain items such as
// CompactionFitlers triggered at flush, or if the same delete
// has been inserted twice or more in the memtable.
if (res.type == kTypeValue && gres && s.ok() && sqno == res.sequence) {
useful_payload += entry_slice.size();
} else if (((res.type == kTypeDeletion) ||
(res.type == kTypeSingleDeletion)) &&
s.IsNotFound() && gres) {
useful_payload += entry_slice.size();
}
}
if (payload > 0) {
// We used the estimated useful payload ratio
// to evaluate how much of the total memtable is useful bytes.
estimated_useful_payload +=
(mt->ApproximateMemoryUsage()) * (useful_payload * 1.0 / payload);
ROCKS_LOG_INFO(
db_options_.info_log,
"Mempurge sampling - found garbage ratio from sampling: %f.\n",
(payload - useful_payload) * 1.0 / payload);
} else {
ROCKS_LOG_WARN(
db_options_.info_log,
"Mempurge kSampling policy: null payload measured, and collected "
"sample size is %zu\n.",
sentries.size());
}
}
}
// We convert the total number of useful paylaod bytes
// into the proportion of memtable necessary to store all these bytes.
// We compare this proportion with the threshold value.
return (estimated_useful_payload / mutable_cf_options_.write_buffer_size) <
threshold;
Add simple heuristics for experimental mempurge. (#8583) Summary: Add `experimental_mempurge_policy` option flag and introduce two new `MemPurge` (Memtable Garbage Collection) policies: 'ALWAYS' and 'ALTERNATE'. Default value: ALTERNATE. `ALWAYS`: every flush will first go through a `MemPurge` process. If the output is too big to fit into a single memtable, then the mempurge is aborted and a regular flush process carries on. `ALWAYS` is designed for user that need to reduce the number of L0 SST file created to a strict minimum, and can afford a small dent in performance (possibly hits to CPU usage, read efficiency, and maximum burst write throughput). `ALTERNATE`: a flush is transformed into a `MemPurge` except if one of the memtables being flushed is the product of a previous `MemPurge`. `ALTERNATE` is a good tradeoff between reduction in number of L0 SST files created and performance. `ALTERNATE` perform particularly well for completely random garbage ratios, or garbage ratios anywhere in (0%,50%], and even higher when there is a wild variability in garbage ratios. This PR also includes support for `experimental_mempurge_policy` in `db_bench`. Testing was done locally by replacing all the `MemPurge` policies of the unit tests with `ALTERNATE`, as well as local testing with `db_crashtest.py` `whitebox` and `blackbox`. Overall, if an `ALWAYS` mempurge policy passes the tests, there is no reasons why an `ALTERNATE` policy would fail, and therefore the mempurge policy was set to `ALWAYS` for all mempurge unit tests. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8583 Reviewed By: pdillinger Differential Revision: D29888050 Pulled By: bjlemaire fbshipit-source-id: e2cf26646d66679f6f5fb29842624615610759c1
2021-07-26 20:55:27 +02:00
}
Status FlushJob::WriteLevel0Table() {
Allow GetThreadList() to report operation stage. Summary: Allow GetThreadList() to report operation stage. Test Plan: ./thread_list_test ./db_bench --benchmarks=fillrandom --num=100000 --threads=40 \ --max_background_compactions=10 --max_background_flushes=3 \ --thread_status_per_interval=1000 --key_size=16 --value_size=1000 \ --num_column_families=10 export ROCKSDB_TESTS=ThreadStatus ./db_test Sample output ThreadID ThreadType cfName Operation OP_StartTime ElapsedTime Stage State 140116265861184 Low Pri 140116270055488 Low Pri 140116274249792 High Pri column_family_name_000005 Flush 2015/03/10-14:58:11 0 us FlushJob::WriteLevel0Table 140116400078912 Low Pri column_family_name_000004 Compaction 2015/03/10-14:58:11 0 us CompactionJob::FinishCompactionOutputFile 140116358135872 Low Pri column_family_name_000006 Compaction 2015/03/10-14:58:10 1 us CompactionJob::FinishCompactionOutputFile 140116341358656 Low Pri 140116295221312 High Pri default Flush 2015/03/10-14:58:11 0 us FlushJob::WriteLevel0Table 140116324581440 Low Pri column_family_name_000009 Compaction 2015/03/10-14:58:11 0 us CompactionJob::ProcessKeyValueCompaction 140116278444096 Low Pri 140116299415616 Low Pri column_family_name_000008 Compaction 2015/03/10-14:58:11 0 us CompactionJob::FinishCompactionOutputFile 140116291027008 High Pri column_family_name_000001 Flush 2015/03/10-14:58:11 0 us FlushJob::WriteLevel0Table 140116286832704 Low Pri column_family_name_000002 Compaction 2015/03/10-14:58:11 0 us CompactionJob::FinishCompactionOutputFile 140116282638400 Low Pri Reviewers: rven, igor, sdong Reviewed By: sdong Subscribers: dhruba, leveldb Differential Revision: https://reviews.facebook.net/D34683
2015-03-13 18:45:40 +01:00
AutoThreadOperationStageUpdater stage_updater(
ThreadStatus::STAGE_FLUSH_WRITE_L0);
db_mutex_->AssertHeld();
const uint64_t start_micros = clock_->NowMicros();
const uint64_t start_cpu_micros = clock_->CPUNanos() / 1000;
Status s;
std::vector<BlobFileAddition> blob_file_additions;
{
auto write_hint = cfd_->CalculateSSTWriteHint(0);
db_mutex_->Unlock();
if (log_buffer_) {
log_buffer_->FlushBufferToLog();
}
// memtables and range_del_iters store internal iterators over each data
// memtable and its associated range deletion memtable, respectively, at
// corresponding indexes.
std::vector<InternalIterator*> memtables;
std::vector<std::unique_ptr<FragmentedRangeTombstoneIterator>>
range_del_iters;
ReadOptions ro;
ro.total_order_seek = true;
Arena arena;
uint64_t total_num_entries = 0, total_num_deletes = 0;
uint64_t total_data_size = 0;
size_t total_memory_usage = 0;
for (MemTable* m : mems_) {
ROCKS_LOG_INFO(
db_options_.info_log,
"[%s] [JOB %d] Flushing memtable with next log file: %" PRIu64 "\n",
cfd_->GetName().c_str(), job_context_->job_id, m->GetNextLogNumber());
memtables.push_back(m->NewIterator(ro, &arena));
auto* range_del_iter =
m->NewRangeTombstoneIterator(ro, kMaxSequenceNumber);
if (range_del_iter != nullptr) {
range_del_iters.emplace_back(range_del_iter);
}
total_num_entries += m->num_entries();
total_num_deletes += m->num_deletes();
total_data_size += m->get_data_size();
total_memory_usage += m->ApproximateMemoryUsage();
}
event_logger_->Log() << "job" << job_context_->job_id << "event"
<< "flush_started"
<< "num_memtables" << mems_.size() << "num_entries"
<< total_num_entries << "num_deletes"
<< total_num_deletes << "total_data_size"
<< total_data_size << "memory_usage"
<< total_memory_usage << "flush_reason"
<< GetFlushReasonString(cfd_->GetFlushReason());
{
ScopedArenaIterator iter(
Make mempurge a background process (equivalent to in-memory compaction). (#8505) Summary: In https://github.com/facebook/rocksdb/issues/8454, I introduced a new process baptized `MemPurge` (memtable garbage collection). This new PR is built upon this past mempurge prototype. In this PR, I made the `mempurge` process a background task, which provides superior performance since the mempurge process does not cling on the db_mutex anymore, and addresses severe restrictions from the past iteration (including a scenario where the past mempurge was failling, when a memtable was mempurged but was still referred to by an iterator/snapshot/...). Now the mempurge process ressembles an in-memory compaction process: the stack of immutable memtables is filtered out, and the useful payload is used to populate an output memtable. If the output memtable is filled at more than 60% capacity (arbitrary heuristic) the mempurge process is aborted and a regular flush process takes place, else the output memtable is kept in the immutable memtable stack. Note that adding this output memtable to the `imm()` memtable stack does not trigger another flush process, so that the flush thread can go to sleep at the end of a successful mempurge. MemPurge is activated by making the `experimental_allow_mempurge` flag `true`. When activated, the `MemPurge` process will always happen when the flush reason is `kWriteBufferFull`. The 3 unit tests confirm that this process supports `Put`, `Get`, `Delete`, `DeleteRange` operators and is compatible with `Iterators` and `CompactionFilters`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8505 Reviewed By: pdillinger Differential Revision: D29619283 Pulled By: bjlemaire fbshipit-source-id: 8a99bee76b63a8211bff1a00e0ae32360aaece95
2021-07-10 02:16:00 +02:00
NewMergingIterator(&cfd_->internal_comparator(), memtables.data(),
static_cast<int>(memtables.size()), &arena));
ROCKS_LOG_INFO(db_options_.info_log,
"[%s] [JOB %d] Level-0 flush table #%" PRIu64 ": started",
cfd_->GetName().c_str(), job_context_->job_id,
meta_.fd.GetNumber());
TEST_SYNC_POINT_CALLBACK("FlushJob::WriteLevel0Table:output_compression",
&output_compression_);
int64_t _current_time = 0;
auto status = clock_->GetCurrentTime(&_current_time);
// Safe to proceed even if GetCurrentTime fails. So, log and proceed.
if (!status.ok()) {
ROCKS_LOG_WARN(
db_options_.info_log,
"Failed to get current time to populate creation_time property. "
"Status: %s",
status.ToString().c_str());
}
FIFO Compaction with TTL Summary: Introducing FIFO compactions with TTL. FIFO compaction is based on size only which makes it tricky to enable in production as use cases can have organic growth. A user requested an option to drop files based on the time of their creation instead of the total size. To address that request: - Added a new TTL option to FIFO compaction options. - Updated FIFO compaction score to take TTL into consideration. - Added a new table property, creation_time, to keep track of when the SST file is created. - Creation_time is set as below: - On Flush: Set to the time of flush. - On Compaction: Set to the max creation_time of all the files involved in the compaction. - On Repair and Recovery: Set to the time of repair/recovery. - Old files created prior to this code change will have a creation_time of 0. - FIFO compaction with TTL is enabled when ttl > 0. All files older than ttl will be deleted during compaction. i.e. `if (file.creation_time < (current_time - ttl)) then delete(file)`. This will enable cases where you might want to delete all files older than, say, 1 day. - FIFO compaction will fall back to the prior way of deleting files based on size if: - the creation_time of all files involved in compaction is 0. - the total size (of all SST files combined) does not drop below `compaction_options_fifo.max_table_files_size` even if the files older than ttl are deleted. This feature is not supported if max_open_files != -1 or with table formats other than Block-based. **Test Plan:** Added tests. **Benchmark results:** Base: FIFO with max size: 100MB :: ``` svemuri@dev15905 ~/rocksdb (fifo-compaction) $ TEST_TMPDIR=/dev/shm ./db_bench --benchmarks=readwhilewriting --num=5000000 --threads=16 --compaction_style=2 --fifo_compaction_max_table_files_size_mb=100 readwhilewriting : 1.924 micros/op 519858 ops/sec; 13.6 MB/s (1176277 of 5000000 found) ``` With TTL (a low one for testing) :: ``` svemuri@dev15905 ~/rocksdb (fifo-compaction) $ TEST_TMPDIR=/dev/shm ./db_bench --benchmarks=readwhilewriting --num=5000000 --threads=16 --compaction_style=2 --fifo_compaction_max_table_files_size_mb=100 --fifo_compaction_ttl=20 readwhilewriting : 1.902 micros/op 525817 ops/sec; 13.7 MB/s (1185057 of 5000000 found) ``` Example Log lines: ``` 2017/06/26-15:17:24.609249 7fd5a45ff700 (Original Log Time 2017/06/26-15:17:24.609177) [db/compaction_picker.cc:1471] [default] FIFO compaction: picking file 40 with creation time 1498515423 for deletion 2017/06/26-15:17:24.609255 7fd5a45ff700 (Original Log Time 2017/06/26-15:17:24.609234) [db/db_impl_compaction_flush.cc:1541] [default] Deleted 1 files ... 2017/06/26-15:17:25.553185 7fd5a61a5800 [DEBUG] [db/db_impl_files.cc:309] [JOB 0] Delete /dev/shm/dbbench/000040.sst type=2 #40 -- OK 2017/06/26-15:17:25.553205 7fd5a61a5800 EVENT_LOG_v1 {"time_micros": 1498515445553199, "job": 0, "event": "table_file_deletion", "file_number": 40} ``` SST Files remaining in the dbbench dir, after db_bench execution completed: ``` svemuri@dev15905 ~/rocksdb (fifo-compaction) $ ls -l /dev/shm//dbbench/*.sst -rw-r--r--. 1 svemuri users 30749887 Jun 26 15:17 /dev/shm//dbbench/000042.sst -rw-r--r--. 1 svemuri users 30768779 Jun 26 15:17 /dev/shm//dbbench/000044.sst -rw-r--r--. 1 svemuri users 30757481 Jun 26 15:17 /dev/shm//dbbench/000046.sst ``` Closes https://github.com/facebook/rocksdb/pull/2480 Differential Revision: D5305116 Pulled By: sagar0 fbshipit-source-id: 3e5cfcf5dd07ed2211b5b37492eb235b45139174
2017-06-28 02:02:20 +02:00
const uint64_t current_time = static_cast<uint64_t>(_current_time);
uint64_t oldest_key_time =
mems_.front()->ApproximateOldestKeyTime();
// It's not clear whether oldest_key_time is always available. In case
// it is not available, use current_time.
Fix+clean up handling of mock sleeps (#7101) Summary: We have a number of tests hanging on MacOS and windows due to mishandling of code for mock sleeps. In addition, the code was in terrible shape because the same variable (addon_time_) would sometimes refer to microseconds and sometimes to seconds. One test even assumed it was nanoseconds but was written to pass anyway. This has been cleaned up so that DB tests generally use a SpecialEnv function to mock sleep, for either some number of microseconds or seconds depending on the function called. But to call one of these, the test must first call SetMockSleep (precondition enforced with assertion), which also turns sleeps in RocksDB into mock sleeps. To also removes accounting for actual clock time, call SetTimeElapseOnlySleepOnReopen, which implies SetMockSleep (on DB re-open). This latter setting only works by applying on DB re-open, otherwise havoc can ensue if Env goes back in time with DB open. More specifics: Removed some unused test classes, and updated comments on the general problem. Fixed DBSSTTest.GetTotalSstFilesSize using a sync point callback instead of mock time. For this we have the only modification to production code, inserting a sync point callback in flush_job.cc, which is not a change to production behavior. Removed unnecessary resetting of mock times to 0 in many tests. RocksDB deals in relative time. Any behaviors relying on absolute date/time are likely a bug. (The above test DBSSTTest.GetTotalSstFilesSize was the only one clearly injecting a specific absolute time for actual testing convenience.) Just in case I misunderstood some test, I put this note in each replacement: // NOTE: Presumed unnecessary and removed: resetting mock time in env Strengthened some tests like MergeTestTime, MergeCompactionTimeTest, and FilterCompactionTimeTest in db_test.cc stats_history_test and blob_db_test are each their own beast, rather deeply dependent on MockTimeEnv. Each gets its own variant of a work-around for TimedWait in a mock time environment. (Reduces redundancy and inconsistency in stats_history_test.) Intended follow-up: Remove TimedWait from the public API of InstrumentedCondVar, and only make that accessible through Env by passing in an InstrumentedCondVar and a deadline. Then the Env implementations mocking time can fix this problem without using sync points. (Test infrastructure using sync points interferes with individual tests' control over sync points.) With that change, we can simplify/consolidate the scattered work-arounds. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7101 Test Plan: make check on Linux and MacOS Reviewed By: zhichao-cao Differential Revision: D23032815 Pulled By: pdillinger fbshipit-source-id: 7f33967ada8b83011fb54e8279365c008bd6610b
2020-08-11 21:39:49 +02:00
uint64_t oldest_ancester_time = std::min(current_time, oldest_key_time);
TEST_SYNC_POINT_CALLBACK(
"FlushJob::WriteLevel0Table:oldest_ancester_time",
&oldest_ancester_time);
meta_.oldest_ancester_time = oldest_ancester_time;
meta_.file_creation_time = current_time;
uint64_t creation_time = (cfd_->ioptions()->compaction_style ==
CompactionStyle::kCompactionStyleFIFO)
? current_time
: meta_.oldest_ancester_time;
uint64_t num_input_entries = 0;
Added memtable garbage statistics (#8411) Summary: **Summary**: 2 new statistics counters are added to RocksDB: `MEMTABLE_PAYLOAD_BYTES_AT_FLUSH` and `MEMTABLE_GARBAGE_BYTES_AT_FLUSH`. The former tracks how many raw bytes of useful data are present on the memtable at flush time, whereas the latter is tracks how many of these raw bytes are considered garbage, meaning that they ended up not being imported on the SSTables resulting from the flush operations. **Unit test**: run `make db_flush_test -j$(nproc); ./db_flush_test` to run the unit test. This executable includes 3 tests, that test support and correct stat calculations for workloads with inserts, deletes, and DeleteRanges. The parameters are set such that the workloads are performed on a single memtable, and a single SSTable is created as a result of the flush operation. The flush operation is manually called in the test file. The tests verify that the values of these 2 statistics counters introduced in this PR can be exactly predicted, showing that we have a full understanding of the underlying operations. **Performance testing**: `./db_bench -statistics -benchmarks=fillrandom -num=10000000` repeated 10 times. Timing done using "date" function in a bash script. _Results_: Original Rocksdb fork: mean 66.6 sec, std 1.18 sec. This feature branch: mean 67.4 sec, std 1.35 sec. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8411 Reviewed By: akankshamahajan15 Differential Revision: D29150629 Pulled By: bjlemaire fbshipit-source-id: 7b3c2e86d50c6aa34fa50fd134282eacb543a5b1
2021-06-18 13:56:43 +02:00
uint64_t memtable_payload_bytes = 0;
uint64_t memtable_garbage_bytes = 0;
IOStatus io_s;
const std::string* const full_history_ts_low =
(full_history_ts_low_.empty()) ? nullptr : &full_history_ts_low_;
TableBuilderOptions tboptions(
*cfd_->ioptions(), mutable_cf_options_, cfd_->internal_comparator(),
cfd_->int_tbl_prop_collector_factories(), output_compression_,
Add more LSM info to FilterBuildingContext (#8246) Summary: Add `num_levels`, `is_bottommost`, and table file creation `reason` to `FilterBuildingContext`, in anticipation of more powerful Bloom-like filter support. To support this, added `is_bottommost` and `reason` to `TableBuilderOptions`, which allowed removing `reason` parameter from `rocksdb::BuildTable`. I attempted to remove `skip_filters` from `TableBuilderOptions`, because filter construction decisions should arise from options, not one-off parameters. I could not completely remove it because the public API for SstFileWriter takes a `skip_filters` parameter, and translating this into an option change would mean awkwardly replacing the table_factory if it is BlockBasedTableFactory with new filter_policy=nullptr option. I marked this public skip_filters option as deprecated because of this oddity. (skip_filters on the read side probably makes sense.) At least `skip_filters` is now largely hidden for users of `TableBuilderOptions` and is no longer used for implementing the optimize_filters_for_hits option. Bringing the logic for that option closer to handling of FilterBuildingContext makes it more obvious that hese two are using the same notion of "bottommost." (Planned: configuration options for Bloom-like filters that generalize `optimize_filters_for_hits`) Recommended follow-up: Try to get away from "bottommost level" naming of things, which is inaccurate (see VersionStorageInfo::RangeMightExistAfterSortedRun), and move to "bottommost run" or just "bottommost." Pull Request resolved: https://github.com/facebook/rocksdb/pull/8246 Test Plan: extended an existing unit test to exercise and check various filter building contexts. Also, existing tests for optimize_filters_for_hits validate some of the "bottommost" handling, which is now closely connected to FilterBuildingContext::is_bottommost through TableBuilderOptions::is_bottommost Reviewed By: mrambacher Differential Revision: D28099346 Pulled By: pdillinger fbshipit-source-id: 2c1072e29c24d4ac404c761a7b7663292372600a
2021-04-30 22:49:24 +02:00
mutable_cf_options_.compression_opts, cfd_->GetID(), cfd_->GetName(),
0 /* level */, false /* is_bottommost */,
TableFileCreationReason::kFlush, creation_time, oldest_key_time,
current_time, db_id_, db_session_id_, 0 /* target_file_size */,
meta_.fd.GetNumber());
s = BuildTable(
dbname_, versions_, db_options_, tboptions, file_options_,
cfd_->table_cache(), iter.get(), std::move(range_del_iters), &meta_,
&blob_file_additions, existing_snapshots_,
earliest_write_conflict_snapshot_, snapshot_checker_,
mutable_cf_options_.paranoid_file_checks, cfd_->internal_stats(),
Add more LSM info to FilterBuildingContext (#8246) Summary: Add `num_levels`, `is_bottommost`, and table file creation `reason` to `FilterBuildingContext`, in anticipation of more powerful Bloom-like filter support. To support this, added `is_bottommost` and `reason` to `TableBuilderOptions`, which allowed removing `reason` parameter from `rocksdb::BuildTable`. I attempted to remove `skip_filters` from `TableBuilderOptions`, because filter construction decisions should arise from options, not one-off parameters. I could not completely remove it because the public API for SstFileWriter takes a `skip_filters` parameter, and translating this into an option change would mean awkwardly replacing the table_factory if it is BlockBasedTableFactory with new filter_policy=nullptr option. I marked this public skip_filters option as deprecated because of this oddity. (skip_filters on the read side probably makes sense.) At least `skip_filters` is now largely hidden for users of `TableBuilderOptions` and is no longer used for implementing the optimize_filters_for_hits option. Bringing the logic for that option closer to handling of FilterBuildingContext makes it more obvious that hese two are using the same notion of "bottommost." (Planned: configuration options for Bloom-like filters that generalize `optimize_filters_for_hits`) Recommended follow-up: Try to get away from "bottommost level" naming of things, which is inaccurate (see VersionStorageInfo::RangeMightExistAfterSortedRun), and move to "bottommost run" or just "bottommost." Pull Request resolved: https://github.com/facebook/rocksdb/pull/8246 Test Plan: extended an existing unit test to exercise and check various filter building contexts. Also, existing tests for optimize_filters_for_hits validate some of the "bottommost" handling, which is now closely connected to FilterBuildingContext::is_bottommost through TableBuilderOptions::is_bottommost Reviewed By: mrambacher Differential Revision: D28099346 Pulled By: pdillinger fbshipit-source-id: 2c1072e29c24d4ac404c761a7b7663292372600a
2021-04-30 22:49:24 +02:00
&io_s, io_tracer_, event_logger_, job_context_->job_id, Env::IO_HIGH,
&table_properties_, write_hint, full_history_ts_low, blob_callback_,
Added memtable garbage statistics (#8411) Summary: **Summary**: 2 new statistics counters are added to RocksDB: `MEMTABLE_PAYLOAD_BYTES_AT_FLUSH` and `MEMTABLE_GARBAGE_BYTES_AT_FLUSH`. The former tracks how many raw bytes of useful data are present on the memtable at flush time, whereas the latter is tracks how many of these raw bytes are considered garbage, meaning that they ended up not being imported on the SSTables resulting from the flush operations. **Unit test**: run `make db_flush_test -j$(nproc); ./db_flush_test` to run the unit test. This executable includes 3 tests, that test support and correct stat calculations for workloads with inserts, deletes, and DeleteRanges. The parameters are set such that the workloads are performed on a single memtable, and a single SSTable is created as a result of the flush operation. The flush operation is manually called in the test file. The tests verify that the values of these 2 statistics counters introduced in this PR can be exactly predicted, showing that we have a full understanding of the underlying operations. **Performance testing**: `./db_bench -statistics -benchmarks=fillrandom -num=10000000` repeated 10 times. Timing done using "date" function in a bash script. _Results_: Original Rocksdb fork: mean 66.6 sec, std 1.18 sec. This feature branch: mean 67.4 sec, std 1.35 sec. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8411 Reviewed By: akankshamahajan15 Differential Revision: D29150629 Pulled By: bjlemaire fbshipit-source-id: 7b3c2e86d50c6aa34fa50fd134282eacb543a5b1
2021-06-18 13:56:43 +02:00
&num_input_entries, &memtable_payload_bytes, &memtable_garbage_bytes);
if (!io_s.ok()) {
io_status_ = io_s;
}
if (num_input_entries != total_num_entries && s.ok()) {
std::string msg = "Expected " + ToString(total_num_entries) +
" entries in memtables, but read " +
ToString(num_input_entries);
ROCKS_LOG_WARN(db_options_.info_log, "[%s] [JOB %d] Level-0 flush %s",
cfd_->GetName().c_str(), job_context_->job_id,
msg.c_str());
if (db_options_.flush_verify_memtable_count) {
s = Status::Corruption(msg);
}
}
Added memtable garbage statistics (#8411) Summary: **Summary**: 2 new statistics counters are added to RocksDB: `MEMTABLE_PAYLOAD_BYTES_AT_FLUSH` and `MEMTABLE_GARBAGE_BYTES_AT_FLUSH`. The former tracks how many raw bytes of useful data are present on the memtable at flush time, whereas the latter is tracks how many of these raw bytes are considered garbage, meaning that they ended up not being imported on the SSTables resulting from the flush operations. **Unit test**: run `make db_flush_test -j$(nproc); ./db_flush_test` to run the unit test. This executable includes 3 tests, that test support and correct stat calculations for workloads with inserts, deletes, and DeleteRanges. The parameters are set such that the workloads are performed on a single memtable, and a single SSTable is created as a result of the flush operation. The flush operation is manually called in the test file. The tests verify that the values of these 2 statistics counters introduced in this PR can be exactly predicted, showing that we have a full understanding of the underlying operations. **Performance testing**: `./db_bench -statistics -benchmarks=fillrandom -num=10000000` repeated 10 times. Timing done using "date" function in a bash script. _Results_: Original Rocksdb fork: mean 66.6 sec, std 1.18 sec. This feature branch: mean 67.4 sec, std 1.35 sec. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8411 Reviewed By: akankshamahajan15 Differential Revision: D29150629 Pulled By: bjlemaire fbshipit-source-id: 7b3c2e86d50c6aa34fa50fd134282eacb543a5b1
2021-06-18 13:56:43 +02:00
if (tboptions.reason == TableFileCreationReason::kFlush) {
Memtable "MemPurge" prototype (#8454) Summary: Implement an experimental feature called "MemPurge", which consists in purging "garbage" bytes out of a memtable and reuse the memtable struct instead of making it immutable and eventually flushing its content to storage. The prototype is by default deactivated and is not intended for use. It is intended for correctness and validation testing. At the moment, the "MemPurge" feature can be switched on by using the `options.experimental_allow_mempurge` flag. For this early stage, when the allow_mempurge flag is set to `true`, all the flush operations will be rerouted to perform a MemPurge. This is a temporary design decision that will give us the time to explore meaningful heuristics to use MemPurge at the right time for relevant workloads . Moreover, the current MemPurge operation only supports `Puts`, `Deletes`, `DeleteRange` operations, and handles `Iterators` as well as `CompactionFilter`s that are invoked at flush time . Three unit tests are added to `db_flush_test.cc` to test if MemPurge works correctly (and checks that the previously mentioned operations are fully supported thoroughly tested). One noticeable design decision is the timing of the MemPurge operation in the memtable workflow: for this prototype, the mempurge happens when the memtable is switched (and usually made immutable). This is an inefficient process because it implies that the entirety of the MemPurge operation happens while holding the db_mutex. Future commits will make the MemPurge operation a background task (akin to the regular flush operation) and aim at drastically enhancing the performance of this operation. The MemPurge is also not fully "WAL-compatible" yet, but when the WAL is full, or when the regular MemPurge operation fails (or when the purged memtable still needs to be flushed), a regular flush operation takes place. Later commits will also correct these behaviors. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8454 Reviewed By: anand1976 Differential Revision: D29433971 Pulled By: bjlemaire fbshipit-source-id: 6af48213554e35048a7e03816955100a80a26dc5
2021-07-02 14:22:03 +02:00
TEST_SYNC_POINT("DBImpl::FlushJob:Flush");
Added memtable garbage statistics (#8411) Summary: **Summary**: 2 new statistics counters are added to RocksDB: `MEMTABLE_PAYLOAD_BYTES_AT_FLUSH` and `MEMTABLE_GARBAGE_BYTES_AT_FLUSH`. The former tracks how many raw bytes of useful data are present on the memtable at flush time, whereas the latter is tracks how many of these raw bytes are considered garbage, meaning that they ended up not being imported on the SSTables resulting from the flush operations. **Unit test**: run `make db_flush_test -j$(nproc); ./db_flush_test` to run the unit test. This executable includes 3 tests, that test support and correct stat calculations for workloads with inserts, deletes, and DeleteRanges. The parameters are set such that the workloads are performed on a single memtable, and a single SSTable is created as a result of the flush operation. The flush operation is manually called in the test file. The tests verify that the values of these 2 statistics counters introduced in this PR can be exactly predicted, showing that we have a full understanding of the underlying operations. **Performance testing**: `./db_bench -statistics -benchmarks=fillrandom -num=10000000` repeated 10 times. Timing done using "date" function in a bash script. _Results_: Original Rocksdb fork: mean 66.6 sec, std 1.18 sec. This feature branch: mean 67.4 sec, std 1.35 sec. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8411 Reviewed By: akankshamahajan15 Differential Revision: D29150629 Pulled By: bjlemaire fbshipit-source-id: 7b3c2e86d50c6aa34fa50fd134282eacb543a5b1
2021-06-18 13:56:43 +02:00
RecordTick(stats_, MEMTABLE_PAYLOAD_BYTES_AT_FLUSH,
memtable_payload_bytes);
RecordTick(stats_, MEMTABLE_GARBAGE_BYTES_AT_FLUSH,
memtable_garbage_bytes);
}
LogFlush(db_options_.info_log);
}
ROCKS_LOG_INFO(db_options_.info_log,
"[%s] [JOB %d] Level-0 flush table #%" PRIu64 ": %" PRIu64
" bytes %s"
"%s",
cfd_->GetName().c_str(), job_context_->job_id,
meta_.fd.GetNumber(), meta_.fd.GetFileSize(),
s.ToString().c_str(),
meta_.marked_for_compaction ? " (needs compaction)" : "");
if (s.ok() && output_file_directory_ != nullptr && sync_output_directory_) {
s = output_file_directory_->Fsync(IOOptions(), nullptr);
}
TEST_SYNC_POINT_CALLBACK("FlushJob::WriteLevel0Table", &mems_);
db_mutex_->Lock();
}
base_->Unref();
// Note that if file_size is zero, the file has been deleted and
// should not be added to the manifest.
const bool has_output = meta_.fd.GetFileSize() > 0;
if (s.ok() && has_output) {
Make mempurge a background process (equivalent to in-memory compaction). (#8505) Summary: In https://github.com/facebook/rocksdb/issues/8454, I introduced a new process baptized `MemPurge` (memtable garbage collection). This new PR is built upon this past mempurge prototype. In this PR, I made the `mempurge` process a background task, which provides superior performance since the mempurge process does not cling on the db_mutex anymore, and addresses severe restrictions from the past iteration (including a scenario where the past mempurge was failling, when a memtable was mempurged but was still referred to by an iterator/snapshot/...). Now the mempurge process ressembles an in-memory compaction process: the stack of immutable memtables is filtered out, and the useful payload is used to populate an output memtable. If the output memtable is filled at more than 60% capacity (arbitrary heuristic) the mempurge process is aborted and a regular flush process takes place, else the output memtable is kept in the immutable memtable stack. Note that adding this output memtable to the `imm()` memtable stack does not trigger another flush process, so that the flush thread can go to sleep at the end of a successful mempurge. MemPurge is activated by making the `experimental_allow_mempurge` flag `true`. When activated, the `MemPurge` process will always happen when the flush reason is `kWriteBufferFull`. The 3 unit tests confirm that this process supports `Put`, `Get`, `Delete`, `DeleteRange` operators and is compatible with `Iterators` and `CompactionFilters`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8505 Reviewed By: pdillinger Differential Revision: D29619283 Pulled By: bjlemaire fbshipit-source-id: 8a99bee76b63a8211bff1a00e0ae32360aaece95
2021-07-10 02:16:00 +02:00
TEST_SYNC_POINT("DBImpl::FlushJob:SSTFileCreated");
// if we have more than 1 background thread, then we cannot
// insert files directly into higher levels because some other
// threads could be concurrently producing compacted files for
// that key range.
// Add file to L0
edit_->AddFile(0 /* level */, meta_.fd.GetNumber(), meta_.fd.GetPathId(),
meta_.fd.GetFileSize(), meta_.smallest, meta_.largest,
meta_.fd.smallest_seqno, meta_.fd.largest_seqno,
meta_.marked_for_compaction, meta_.oldest_blob_file_number,
meta_.oldest_ancester_time, meta_.file_creation_time,
meta_.file_checksum, meta_.file_checksum_func_name);
edit_->SetBlobFileAdditions(std::move(blob_file_additions));
}
#ifndef ROCKSDB_LITE
// Piggyback FlushJobInfo on the first first flushed memtable.
mems_[0]->SetFlushJobInfo(GetFlushJobInfo());
#endif // !ROCKSDB_LITE
// Note that here we treat flush as level 0 compaction in internal stats
InternalStats::CompactionStats stats(CompactionReason::kFlush, 1);
Add simple heuristics for experimental mempurge. (#8583) Summary: Add `experimental_mempurge_policy` option flag and introduce two new `MemPurge` (Memtable Garbage Collection) policies: 'ALWAYS' and 'ALTERNATE'. Default value: ALTERNATE. `ALWAYS`: every flush will first go through a `MemPurge` process. If the output is too big to fit into a single memtable, then the mempurge is aborted and a regular flush process carries on. `ALWAYS` is designed for user that need to reduce the number of L0 SST file created to a strict minimum, and can afford a small dent in performance (possibly hits to CPU usage, read efficiency, and maximum burst write throughput). `ALTERNATE`: a flush is transformed into a `MemPurge` except if one of the memtables being flushed is the product of a previous `MemPurge`. `ALTERNATE` is a good tradeoff between reduction in number of L0 SST files created and performance. `ALTERNATE` perform particularly well for completely random garbage ratios, or garbage ratios anywhere in (0%,50%], and even higher when there is a wild variability in garbage ratios. This PR also includes support for `experimental_mempurge_policy` in `db_bench`. Testing was done locally by replacing all the `MemPurge` policies of the unit tests with `ALTERNATE`, as well as local testing with `db_crashtest.py` `whitebox` and `blackbox`. Overall, if an `ALWAYS` mempurge policy passes the tests, there is no reasons why an `ALTERNATE` policy would fail, and therefore the mempurge policy was set to `ALWAYS` for all mempurge unit tests. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8583 Reviewed By: pdillinger Differential Revision: D29888050 Pulled By: bjlemaire fbshipit-source-id: e2cf26646d66679f6f5fb29842624615610759c1
2021-07-26 20:55:27 +02:00
const uint64_t micros = clock_->NowMicros() - start_micros;
const uint64_t cpu_micros = clock_->CPUNanos() / 1000 - start_cpu_micros;
stats.micros = micros;
stats.cpu_micros = cpu_micros;
ROCKS_LOG_INFO(db_options_.info_log,
"[%s] [JOB %d] Flush lasted %" PRIu64
" microseconds, and %" PRIu64 " cpu microseconds.\n",
cfd_->GetName().c_str(), job_context_->job_id, micros,
cpu_micros);
if (has_output) {
stats.bytes_written = meta_.fd.GetFileSize();
stats.num_output_files = 1;
}
const auto& blobs = edit_->GetBlobFileAdditions();
for (const auto& blob : blobs) {
stats.bytes_written_blob += blob.GetTotalBlobBytes();
}
stats.num_output_files_blob = static_cast<int>(blobs.size());
Memtable sampling for mempurge heuristic. (#8628) Summary: Changes the API of the MemPurge process: the `bool experimental_allow_mempurge` and `experimental_mempurge_policy` flags have been replaced by a `double experimental_mempurge_threshold` option. This change of API reflects another major change introduced in this PR: the MemPurgeDecider() function now works by sampling the memtables being flushed to estimate the overall amount of useful payload (payload minus the garbage), and then compare this useful payload estimate with the `double experimental_mempurge_threshold` value. Therefore, when the value of this flag is `0.0` (default value), mempurge is simply deactivated. On the other hand, a value of `DBL_MAX` would be equivalent to always going through a mempurge regardless of the garbage ratio estimate. At the moment, a `double experimental_mempurge_threshold` value else than 0.0 or `DBL_MAX` is opnly supported`with the `SkipList` memtable representation. Regarding the sampling, this PR includes the introduction of a `MemTable::UniqueRandomSample` function that collects (approximately) random entries from the memtable by using the new `SkipList::Iterator::RandomSeek()` under the hood, or by iterating through each memtable entry, depending on the target sample size and the total number of entries. The unit tests have been readapted to support this new API. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8628 Reviewed By: pdillinger Differential Revision: D30149315 Pulled By: bjlemaire fbshipit-source-id: 1feef5390c95db6f4480ab4434716533d3947f27
2021-08-11 03:07:48 +02:00
if ((db_options_.experimental_mempurge_threshold > 0.0) && s.ok()) {
Add simple heuristics for experimental mempurge. (#8583) Summary: Add `experimental_mempurge_policy` option flag and introduce two new `MemPurge` (Memtable Garbage Collection) policies: 'ALWAYS' and 'ALTERNATE'. Default value: ALTERNATE. `ALWAYS`: every flush will first go through a `MemPurge` process. If the output is too big to fit into a single memtable, then the mempurge is aborted and a regular flush process carries on. `ALWAYS` is designed for user that need to reduce the number of L0 SST file created to a strict minimum, and can afford a small dent in performance (possibly hits to CPU usage, read efficiency, and maximum burst write throughput). `ALTERNATE`: a flush is transformed into a `MemPurge` except if one of the memtables being flushed is the product of a previous `MemPurge`. `ALTERNATE` is a good tradeoff between reduction in number of L0 SST files created and performance. `ALTERNATE` perform particularly well for completely random garbage ratios, or garbage ratios anywhere in (0%,50%], and even higher when there is a wild variability in garbage ratios. This PR also includes support for `experimental_mempurge_policy` in `db_bench`. Testing was done locally by replacing all the `MemPurge` policies of the unit tests with `ALTERNATE`, as well as local testing with `db_crashtest.py` `whitebox` and `blackbox`. Overall, if an `ALWAYS` mempurge policy passes the tests, there is no reasons why an `ALTERNATE` policy would fail, and therefore the mempurge policy was set to `ALWAYS` for all mempurge unit tests. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8583 Reviewed By: pdillinger Differential Revision: D29888050 Pulled By: bjlemaire fbshipit-source-id: e2cf26646d66679f6f5fb29842624615610759c1
2021-07-26 20:55:27 +02:00
// The db_mutex is held at this point.
for (MemTable* mt : mems_) {
// Note: if m is not a previous mempurge output memtable,
// nothing happens here.
cfd_->imm()->RemoveMemPurgeOutputID(mt->GetID());
}
}
RecordTimeToHistogram(stats_, FLUSH_TIME, stats.micros);
cfd_->internal_stats()->AddCompactionStats(0 /* level */, thread_pri_, stats);
cfd_->internal_stats()->AddCFStats(
InternalStats::BYTES_FLUSHED,
stats.bytes_written + stats.bytes_written_blob);
RecordFlushIOStats();
Add simple heuristics for experimental mempurge. (#8583) Summary: Add `experimental_mempurge_policy` option flag and introduce two new `MemPurge` (Memtable Garbage Collection) policies: 'ALWAYS' and 'ALTERNATE'. Default value: ALTERNATE. `ALWAYS`: every flush will first go through a `MemPurge` process. If the output is too big to fit into a single memtable, then the mempurge is aborted and a regular flush process carries on. `ALWAYS` is designed for user that need to reduce the number of L0 SST file created to a strict minimum, and can afford a small dent in performance (possibly hits to CPU usage, read efficiency, and maximum burst write throughput). `ALTERNATE`: a flush is transformed into a `MemPurge` except if one of the memtables being flushed is the product of a previous `MemPurge`. `ALTERNATE` is a good tradeoff between reduction in number of L0 SST files created and performance. `ALTERNATE` perform particularly well for completely random garbage ratios, or garbage ratios anywhere in (0%,50%], and even higher when there is a wild variability in garbage ratios. This PR also includes support for `experimental_mempurge_policy` in `db_bench`. Testing was done locally by replacing all the `MemPurge` policies of the unit tests with `ALTERNATE`, as well as local testing with `db_crashtest.py` `whitebox` and `blackbox`. Overall, if an `ALWAYS` mempurge policy passes the tests, there is no reasons why an `ALTERNATE` policy would fail, and therefore the mempurge policy was set to `ALWAYS` for all mempurge unit tests. Pull Request resolved: https://github.com/facebook/rocksdb/pull/8583 Reviewed By: pdillinger Differential Revision: D29888050 Pulled By: bjlemaire fbshipit-source-id: e2cf26646d66679f6f5fb29842624615610759c1
2021-07-26 20:55:27 +02:00
return s;
}
#ifndef ROCKSDB_LITE
std::unique_ptr<FlushJobInfo> FlushJob::GetFlushJobInfo() const {
db_mutex_->AssertHeld();
std::unique_ptr<FlushJobInfo> info(new FlushJobInfo{});
info->cf_id = cfd_->GetID();
info->cf_name = cfd_->GetName();
const uint64_t file_number = meta_.fd.GetNumber();
info->file_path =
MakeTableFileName(cfd_->ioptions()->cf_paths[0].path, file_number);
info->file_number = file_number;
info->oldest_blob_file_number = meta_.oldest_blob_file_number;
info->thread_id = db_options_.env->GetThreadID();
info->job_id = job_context_->job_id;
info->smallest_seqno = meta_.fd.smallest_seqno;
info->largest_seqno = meta_.fd.largest_seqno;
info->table_properties = table_properties_;
info->flush_reason = cfd_->GetFlushReason();
return info;
}
#endif // !ROCKSDB_LITE
} // namespace ROCKSDB_NAMESPACE