2016-02-10 00:12:00 +01:00
|
|
|
// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
|
2017-07-16 01:03:42 +02:00
|
|
|
// 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).
|
2014-11-01 00:31:25 +01:00
|
|
|
//
|
|
|
|
// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
|
|
|
|
// Use of this source code is governed by a BSD-style license that can be
|
|
|
|
// found in the LICENSE file. See the AUTHORS file for names of contributors.
|
|
|
|
#pragma once
|
|
|
|
|
|
|
|
#include <atomic>
|
|
|
|
#include <deque>
|
2015-09-02 22:58:22 +02:00
|
|
|
#include <functional>
|
2014-11-01 00:31:25 +01:00
|
|
|
#include <limits>
|
|
|
|
#include <set>
|
2015-09-02 22:58:22 +02:00
|
|
|
#include <string>
|
2014-11-01 00:31:25 +01:00
|
|
|
#include <utility>
|
|
|
|
#include <vector>
|
|
|
|
|
2015-09-02 22:58:22 +02:00
|
|
|
#include "db/column_family.h"
|
2019-05-31 20:52:59 +02:00
|
|
|
#include "db/compaction/compaction_iterator.h"
|
2014-11-01 00:31:25 +01:00
|
|
|
#include "db/dbformat.h"
|
2015-09-02 22:58:22 +02:00
|
|
|
#include "db/flush_scheduler.h"
|
|
|
|
#include "db/internal_stats.h"
|
|
|
|
#include "db/job_context.h"
|
2014-11-01 00:31:25 +01:00
|
|
|
#include "db/log_writer.h"
|
|
|
|
#include "db/memtable_list.h"
|
2018-12-18 02:26:56 +01:00
|
|
|
#include "db/range_del_aggregator.h"
|
2015-09-02 22:58:22 +02:00
|
|
|
#include "db/version_edit.h"
|
|
|
|
#include "db/write_controller.h"
|
|
|
|
#include "db/write_thread.h"
|
2019-06-01 02:19:43 +02:00
|
|
|
#include "logging/event_logger.h"
|
2018-05-04 01:35:46 +02:00
|
|
|
#include "options/cf_options.h"
|
2018-12-17 22:12:22 +01:00
|
|
|
#include "options/db_options.h"
|
2014-11-01 00:31:25 +01:00
|
|
|
#include "port/port.h"
|
2015-09-02 22:58:22 +02:00
|
|
|
#include "rocksdb/compaction_filter.h"
|
|
|
|
#include "rocksdb/compaction_job_stats.h"
|
2014-11-01 00:31:25 +01:00
|
|
|
#include "rocksdb/db.h"
|
|
|
|
#include "rocksdb/env.h"
|
|
|
|
#include "rocksdb/memtablerep.h"
|
|
|
|
#include "rocksdb/transaction_log.h"
|
2015-10-13 00:06:38 +02:00
|
|
|
#include "table/scoped_arena_iterator.h"
|
2014-11-01 00:31:25 +01:00
|
|
|
#include "util/autovector.h"
|
|
|
|
#include "util/stop_watch.h"
|
|
|
|
#include "util/thread_local.h"
|
|
|
|
|
2020-02-20 21:07:53 +01:00
|
|
|
namespace ROCKSDB_NAMESPACE {
|
2014-11-01 00:31:25 +01:00
|
|
|
|
2017-10-06 19:26:38 +02:00
|
|
|
class Arena;
|
2018-06-28 21:23:57 +02:00
|
|
|
class ErrorHandler;
|
2014-11-01 00:31:25 +01:00
|
|
|
class MemTable;
|
2017-10-06 19:26:38 +02:00
|
|
|
class SnapshotChecker;
|
2014-11-01 00:31:25 +01:00
|
|
|
class TableCache;
|
|
|
|
class Version;
|
|
|
|
class VersionEdit;
|
|
|
|
class VersionSet;
|
|
|
|
|
2019-05-24 01:26:07 +02:00
|
|
|
// CompactionJob is responsible for executing the compaction. Each (manual or
|
|
|
|
// automated) compaction corresponds to a CompactionJob object, and usually
|
|
|
|
// goes through the stages of `Prepare()`->`Run()`->`Install()`. CompactionJob
|
|
|
|
// will divide the compaction into subcompactions and execute them in parallel
|
|
|
|
// if needed.
|
2014-11-01 00:31:25 +01:00
|
|
|
class CompactionJob {
|
|
|
|
public:
|
2020-08-13 02:28:10 +02:00
|
|
|
CompactionJob(
|
|
|
|
int job_id, Compaction* compaction, const ImmutableDBOptions& db_options,
|
|
|
|
const FileOptions& file_options, VersionSet* versions,
|
|
|
|
const std::atomic<bool>* shutting_down,
|
|
|
|
const SequenceNumber preserve_deletes_seqnum, LogBuffer* log_buffer,
|
|
|
|
FSDirectory* db_directory, FSDirectory* output_directory,
|
|
|
|
Statistics* stats, InstrumentedMutex* db_mutex,
|
|
|
|
ErrorHandler* db_error_handler,
|
|
|
|
std::vector<SequenceNumber> existing_snapshots,
|
|
|
|
SequenceNumber earliest_write_conflict_snapshot,
|
|
|
|
const SnapshotChecker* snapshot_checker,
|
|
|
|
std::shared_ptr<Cache> table_cache, EventLogger* event_logger,
|
|
|
|
bool paranoid_file_checks, bool measure_io_stats,
|
|
|
|
const std::string& dbname, CompactionJobStats* compaction_job_stats,
|
|
|
|
Env::Priority thread_pri, const std::shared_ptr<IOTracer>& io_tracer,
|
2020-08-14 20:28:12 +02:00
|
|
|
const std::atomic<int>* manual_compaction_paused = nullptr,
|
2020-08-13 02:28:10 +02:00
|
|
|
const std::string& db_id = "", const std::string& db_session_id = "");
|
2014-11-01 00:31:25 +01:00
|
|
|
|
2015-03-13 18:45:40 +01:00
|
|
|
~CompactionJob();
|
2014-11-01 00:31:25 +01:00
|
|
|
|
|
|
|
// no copy/move
|
|
|
|
CompactionJob(CompactionJob&& job) = delete;
|
|
|
|
CompactionJob(const CompactionJob& job) = delete;
|
|
|
|
CompactionJob& operator=(const CompactionJob& job) = delete;
|
|
|
|
|
|
|
|
// REQUIRED: mutex held
|
2019-05-24 01:26:07 +02:00
|
|
|
// Prepare for the compaction by setting up boundaries for each subcompaction
|
2014-11-01 00:31:25 +01:00
|
|
|
void Prepare();
|
|
|
|
// REQUIRED mutex not held
|
2019-05-24 01:26:07 +02:00
|
|
|
// Launch threads for each subcompaction and wait for them to finish. After
|
|
|
|
// that, verify table is usable and finally do bookkeeping to unify
|
|
|
|
// subcompaction results
|
2014-11-01 00:31:25 +01:00
|
|
|
Status Run();
|
Parallelize L0-L1 Compaction: Restructure Compaction Job
Summary:
As of now compactions involving files from Level 0 and Level 1 are single
threaded because the files in L0, although sorted, are not range partitioned like
the other levels. This means that during L0-L1 compaction each file from L1
needs to be merged with potentially all the files from L0.
This attempt to parallelize the L0-L1 compaction assigns a thread and a
corresponding iterator to each L1 file that then considers only the key range
found in that L1 file and only the L0 files that have those keys (and only the
specific portion of those L0 files in which those keys are found). In this way
the overlap is minimized and potentially eliminated between different iterators
focusing on the same files.
The first step is to restructure the compaction logic to break L0-L1 compactions
into multiple, smaller, sequential compactions. Eventually each of these smaller
jobs will be run simultaneously. Areas to pay extra attention to are
# Correct aggregation of compaction job statistics across multiple threads
# Proper opening/closing of output files (make sure each thread's is unique)
# Keys that span multiple L1 files
# Skewed distributions of keys within L0 files
Test Plan: Make and run db_test (newer version has separate compaction tests) and compaction_job_stats_test
Reviewers: igor, noetzli, anthony, sdong, yhchiang
Reviewed By: yhchiang
Subscribers: MarkCallaghan, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D42699
2015-08-03 20:32:14 +02:00
|
|
|
|
2014-11-01 00:31:25 +01:00
|
|
|
// REQUIRED: mutex held
|
2019-05-24 01:26:07 +02:00
|
|
|
// Add compaction input/output to the current version
|
2016-02-18 00:20:23 +01:00
|
|
|
Status Install(const MutableCFOptions& mutable_cf_options);
|
2014-11-01 00:31:25 +01:00
|
|
|
|
Pass IOStatus to write path and set retryable IO Error as hard error in BG jobs (#6487)
Summary:
In the current code base, we use Status to get and store the returned status from the call. Specifically, for IO related functions, the current Status cannot reflect the IO Error details such as error scope, error retryable attribute, and others. With the implementation of https://github.com/facebook/rocksdb/issues/5761, we have the new Wrapper for IO, which returns IOStatus instead of Status. However, the IOStatus is purged at the lower level of write path and transferred to Status.
The first job of this PR is to pass the IOStatus to the write path (flush, WAL write, and Compaction). The second job is to identify the Retryable IO Error as HardError, and set the bg_error_ as HardError. In this case, the DB Instance becomes read only. User is informed of the Status and need to take actions to deal with it (e.g., call db->Resume()).
Pull Request resolved: https://github.com/facebook/rocksdb/pull/6487
Test Plan: Added the testing case to error_handler_fs_test. Pass make asan_check
Reviewed By: anand1976
Differential Revision: D20685017
Pulled By: zhichao-cao
fbshipit-source-id: ff85f042896243abcd6ef37877834e26f36b6eb0
2020-03-28 00:03:05 +01:00
|
|
|
// Return the IO status
|
|
|
|
IOStatus io_status() const { return io_status_; }
|
|
|
|
|
2014-11-01 00:31:25 +01:00
|
|
|
private:
|
2015-09-10 22:50:00 +02:00
|
|
|
struct SubcompactionState;
|
2015-08-18 20:06:23 +02:00
|
|
|
|
|
|
|
void AggregateStatistics();
|
2019-05-24 01:26:07 +02:00
|
|
|
|
|
|
|
// Generates a histogram representing potential divisions of key ranges from
|
|
|
|
// the input. It adds the starting and/or ending keys of certain input files
|
|
|
|
// to the working set and then finds the approximate size of data in between
|
|
|
|
// each consecutive pair of slices. Then it divides these ranges into
|
|
|
|
// consecutive groups such that each group has a similar size.
|
2015-09-10 22:50:00 +02:00
|
|
|
void GenSubcompactionBoundaries();
|
Parallelize L0-L1 Compaction: Restructure Compaction Job
Summary:
As of now compactions involving files from Level 0 and Level 1 are single
threaded because the files in L0, although sorted, are not range partitioned like
the other levels. This means that during L0-L1 compaction each file from L1
needs to be merged with potentially all the files from L0.
This attempt to parallelize the L0-L1 compaction assigns a thread and a
corresponding iterator to each L1 file that then considers only the key range
found in that L1 file and only the L0 files that have those keys (and only the
specific portion of those L0 files in which those keys are found). In this way
the overlap is minimized and potentially eliminated between different iterators
focusing on the same files.
The first step is to restructure the compaction logic to break L0-L1 compactions
into multiple, smaller, sequential compactions. Eventually each of these smaller
jobs will be run simultaneously. Areas to pay extra attention to are
# Correct aggregation of compaction job statistics across multiple threads
# Proper opening/closing of output files (make sure each thread's is unique)
# Keys that span multiple L1 files
# Skewed distributions of keys within L0 files
Test Plan: Make and run db_test (newer version has separate compaction tests) and compaction_job_stats_test
Reviewers: igor, noetzli, anthony, sdong, yhchiang
Reviewed By: yhchiang
Subscribers: MarkCallaghan, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D42699
2015-08-03 20:32:14 +02:00
|
|
|
|
2015-05-07 07:50:35 +02:00
|
|
|
// update the thread status for starting a compaction.
|
|
|
|
void ReportStartedCompaction(Compaction* compaction);
|
2014-11-01 00:31:25 +01:00
|
|
|
void AllocateCompactionOutputFileNumbers();
|
2015-07-17 18:59:11 +02:00
|
|
|
// Call compaction filter. Then iterate through input and compact the
|
|
|
|
// kv-pairs
|
2015-09-10 22:50:00 +02:00
|
|
|
void ProcessKeyValueCompaction(SubcompactionState* sub_compact);
|
2015-07-15 18:55:45 +02:00
|
|
|
|
2016-11-28 20:44:40 +01:00
|
|
|
Status FinishCompactionOutputFile(
|
|
|
|
const Status& input_status, SubcompactionState* sub_compact,
|
2018-12-18 02:26:56 +01:00
|
|
|
CompactionRangeDelAggregator* range_del_agg,
|
2016-11-28 20:44:40 +01:00
|
|
|
CompactionIterationStats* range_del_out_stats,
|
|
|
|
const Slice* next_table_min_key = nullptr);
|
2016-02-18 00:20:23 +01:00
|
|
|
Status InstallCompactionResults(const MutableCFOptions& mutable_cf_options);
|
2014-11-01 00:31:25 +01:00
|
|
|
void RecordCompactionIOStats();
|
2015-09-10 22:50:00 +02:00
|
|
|
Status OpenCompactionOutputFile(SubcompactionState* sub_compact);
|
2015-08-18 20:06:23 +02:00
|
|
|
void CleanupCompaction();
|
2015-06-03 02:07:16 +02:00
|
|
|
void UpdateCompactionJobStats(
|
|
|
|
const InternalStats::CompactionStats& stats) const;
|
2016-11-28 20:44:40 +01:00
|
|
|
void RecordDroppedKeys(const CompactionIterationStats& c_iter_stats,
|
2015-08-18 20:06:23 +02:00
|
|
|
CompactionJobStats* compaction_job_stats = nullptr);
|
2014-11-01 00:31:25 +01:00
|
|
|
|
2015-07-14 09:09:20 +02:00
|
|
|
void UpdateCompactionStats();
|
2015-06-18 08:40:34 +02:00
|
|
|
void UpdateCompactionInputStatsHelper(
|
|
|
|
int* num_files, uint64_t* bytes_read, int input_level);
|
|
|
|
|
2015-08-18 20:06:23 +02:00
|
|
|
void LogCompaction();
|
2015-07-14 09:09:20 +02:00
|
|
|
|
2015-02-12 18:54:48 +01:00
|
|
|
int job_id_;
|
|
|
|
|
2014-11-01 00:31:25 +01:00
|
|
|
// CompactionJob state
|
|
|
|
struct CompactionState;
|
|
|
|
CompactionState* compact_;
|
2015-06-03 02:07:16 +02:00
|
|
|
CompactionJobStats* compaction_job_stats_;
|
2014-11-01 00:31:25 +01:00
|
|
|
InternalStats::CompactionStats compaction_stats_;
|
|
|
|
|
|
|
|
// DBImpl state
|
2015-06-02 23:12:23 +02:00
|
|
|
const std::string& dbname_;
|
2020-06-17 19:55:42 +02:00
|
|
|
const std::string db_id_;
|
|
|
|
const std::string db_session_id_;
|
2016-09-24 01:34:04 +02:00
|
|
|
const ImmutableDBOptions& db_options_;
|
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
|
|
|
const FileOptions file_options_;
|
2016-05-17 22:11:56 +02:00
|
|
|
|
2014-11-01 00:31:25 +01:00
|
|
|
Env* env_;
|
2020-09-08 19:49:01 +02:00
|
|
|
std::shared_ptr<IOTracer> io_tracer_;
|
2020-08-13 02:28:10 +02:00
|
|
|
FileSystemPtr fs_;
|
2017-11-17 02:46:43 +01:00
|
|
|
// env_option optimized for compaction table reads
|
Introduce a new storage specific Env API (#5761)
Summary:
The current Env API encompasses both storage/file operations, as well as OS related operations. Most of the APIs return a Status, which does not have enough metadata about an error, such as whether its retry-able or not, scope (i.e fault domain) of the error etc., that may be required in order to properly handle a storage error. The file APIs also do not provide enough control over the IO SLA, such as timeout, prioritization, hinting about placement and redundancy etc.
This PR separates out the file/storage APIs from Env into a new FileSystem class. The APIs are updated to return an IOStatus with metadata about the error, as well as to take an IOOptions structure as input in order to allow more control over the IO.
The user can set both ```options.env``` and ```options.file_system``` to specify that RocksDB should use the former for OS related operations and the latter for storage operations. Internally, a ```CompositeEnvWrapper``` has been introduced that inherits from ```Env``` and redirects individual methods to either an ```Env``` implementation or the ```FileSystem``` as appropriate. When options are sanitized during ```DB::Open```, ```options.env``` is replaced with a newly allocated ```CompositeEnvWrapper``` instance if both env and file_system have been specified. This way, the rest of the RocksDB code can continue to function as before.
This PR also ports PosixEnv to the new API by splitting it into two - PosixEnv and PosixFileSystem. PosixEnv is defined as a sub-class of CompositeEnvWrapper, and threading/time functions are overridden with Posix specific implementations in order to avoid an extra level of indirection.
The ```CompositeEnvWrapper``` translates ```IOStatus``` return code to ```Status```, and sets the severity to ```kSoftError``` if the io_status is retryable. The error handling code in RocksDB can then recover the DB automatically.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5761
Differential Revision: D18868376
Pulled By: anand1976
fbshipit-source-id: 39efe18a162ea746fabac6360ff529baba48486f
2019-12-13 23:47:08 +01:00
|
|
|
FileOptions file_options_for_read_;
|
2014-11-01 00:31:25 +01:00
|
|
|
VersionSet* versions_;
|
2017-01-12 00:01:21 +01:00
|
|
|
const std::atomic<bool>* shutting_down_;
|
2020-08-14 20:28:12 +02:00
|
|
|
const std::atomic<int>* manual_compaction_paused_;
|
Added support for differential snapshots
Summary:
The motivation for this PR is to add to RocksDB support for differential (incremental) snapshots, as snapshot of the DB changes between two points in time (one can think of it as diff between to sequence numbers, or the diff D which can be thought of as an SST file or just set of KVs that can be applied to sequence number S1 to get the database to the state at sequence number S2).
This feature would be useful for various distributed storages layers built on top of RocksDB, as it should help reduce resources (time and network bandwidth) needed to recover and rebuilt DB instances as replicas in the context of distributed storages.
From the API standpoint that would like client app requesting iterator between (start seqnum) and current DB state, and reading the "diff".
This is a very draft PR for initial review in the discussion on the approach, i'm going to rework some parts and keep updating the PR.
For now, what's done here according to initial discussions:
Preserving deletes:
- We want to be able to optionally preserve recent deletes for some defined period of time, so that if a delete came in recently and might need to be included in the next incremental snapshot it would't get dropped by a compaction. This is done by adding new param to Options (preserve deletes flag) and new variable to DB Impl where we keep track of the sequence number after which we don't want to drop tombstones, even if they are otherwise eligible for deletion.
- I also added a new API call for clients to be able to advance this cutoff seqnum after which we drop deletes; i assume it's more flexible to let clients control this, since otherwise we'd need to keep some kind of timestamp < -- > seqnum mapping inside the DB, which sounds messy and painful to support. Clients could make use of it by periodically calling GetLatestSequenceNumber(), noting the timestamp, doing some calculation and figuring out by how much we need to advance the cutoff seqnum.
- Compaction codepath in compaction_iterator.cc has been modified to avoid dropping tombstones with seqnum > cutoff seqnum.
Iterator changes:
- couple params added to ReadOptions, to optionally allow client to request internal keys instead of user keys (so that client can get the latest value of a key, be it delete marker or a put), as well as min timestamp and min seqnum.
TableCache changes:
- I modified table_cache code to be able to quickly exclude SST files from iterators heep if creation_time on the file is less then iter_start_ts as passed in ReadOptions. That would help a lot in some DB settings (like reading very recent data only or using FIFO compactions), but not so much for universal compaction with more or less long iterator time span.
What's left:
- Still looking at how to best plug that inside DBIter codepath. So far it seems that FindNextUserKeyInternal only parses values as UserKeys, and iter->key() call generally returns user key. Can we add new API to DBIter as internal_key(), and modify this internal method to optionally set saved_key_ to point to the full internal key? I don't need to store actual seqnum there, but I do need to store type.
Closes https://github.com/facebook/rocksdb/pull/2999
Differential Revision: D6175602
Pulled By: mikhail-antonov
fbshipit-source-id: c779a6696ee2d574d86c69cec866a3ae095aa900
2017-11-02 02:43:29 +01:00
|
|
|
const SequenceNumber preserve_deletes_seqnum_;
|
2014-11-01 00:31:25 +01:00
|
|
|
LogBuffer* log_buffer_;
|
2020-03-03 01:14:00 +01:00
|
|
|
FSDirectory* db_directory_;
|
|
|
|
FSDirectory* output_directory_;
|
2014-11-01 00:31:25 +01:00
|
|
|
Statistics* stats_;
|
2016-02-18 00:20:23 +01:00
|
|
|
InstrumentedMutex* db_mutex_;
|
2018-06-28 21:23:57 +02:00
|
|
|
ErrorHandler* db_error_handler_;
|
2015-05-06 04:01:12 +02:00
|
|
|
// If there were two snapshots with seq numbers s1 and
|
|
|
|
// s2 and s1 < s2, and if we find two instances of a key k1 then lies
|
|
|
|
// entirely within s1 and s2, then the earlier version of k1 can be safely
|
|
|
|
// deleted because that version is not visible in any snapshot.
|
|
|
|
std::vector<SequenceNumber> existing_snapshots_;
|
2015-12-08 21:25:48 +01:00
|
|
|
|
|
|
|
// This is the earliest snapshot that could be used for write-conflict
|
|
|
|
// checking by a transaction. For any user-key newer than this snapshot, we
|
2015-12-10 17:54:48 +01:00
|
|
|
// should make sure not to remove evidence that a write occurred.
|
2015-12-08 21:25:48 +01:00
|
|
|
SequenceNumber earliest_write_conflict_snapshot_;
|
|
|
|
|
2017-10-06 19:26:38 +02:00
|
|
|
const SnapshotChecker* const snapshot_checker_;
|
|
|
|
|
2014-11-01 00:31:25 +01:00
|
|
|
std::shared_ptr<Cache> table_cache_;
|
|
|
|
|
Include bunch of more events into EventLogger
Summary:
Added these events:
* Recovery start, finish and also when recovery creates a file
* Trivial move
* Compaction start, finish and when compaction creates a file
* Flush start, finish
Also includes small fix to EventLogger
Also added option ROCKSDB_PRINT_EVENTS_TO_STDOUT which is useful when we debug things. I've spent far too much time chasing LOG files.
Still didn't get sst table properties in JSON. They are written very deeply into the stack. I'll address in separate diff.
TODO:
* Write specification. Let's first use this for a while and figure out what's good data to put here, too. After that we'll write spec
* Write tools that parse and analyze LOGs. This can be in python or go. Good intern task.
Test Plan: Ran db_bench with ROCKSDB_PRINT_EVENTS_TO_STDOUT. Here's the output: https://phabricator.fb.com/P19811976
Reviewers: sdong, yhchiang, rven, MarkCallaghan, kradhakrishnan, anthony
Reviewed By: anthony
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D37521
2015-04-28 00:20:02 +02:00
|
|
|
EventLogger* event_logger_;
|
2015-05-06 04:01:12 +02:00
|
|
|
|
2019-05-24 01:26:07 +02:00
|
|
|
// Is this compaction creating a file in the bottom most level?
|
2015-09-10 23:35:25 +02:00
|
|
|
bool bottommost_level_;
|
2015-05-06 04:01:12 +02:00
|
|
|
bool paranoid_file_checks_;
|
Add options.compaction_measure_io_stats to print write I/O stats in compactions
Summary:
Add options.compaction_measure_io_stats to print out / pass to listener accumulated time spent on write calls. Example outputs in info logs:
2015/08/12-16:27:59.463944 7fd428bff700 (Original Log Time 2015/08/12-16:27:59.463922) EVENT_LOG_v1 {"time_micros": 1439422079463897, "job": 6, "event": "compaction_finished", "output_level": 1, "num_output_files": 4, "total_output_size": 6900525, "num_input_records": 111483, "num_output_records": 106877, "file_write_nanos": 15663206, "file_range_sync_nanos": 649588, "file_fsync_nanos": 349614797, "file_prepare_write_nanos": 1505812, "lsm_state": [2, 4, 0, 0, 0, 0, 0]}
Add two more counters in iostats_context.
Also add a parameter of db_bench.
Test Plan: Add a unit test. Also manually verify LOG outputs in db_bench
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D44115
2015-08-13 02:24:45 +02:00
|
|
|
bool measure_io_stats_;
|
2015-09-10 22:50:00 +02:00
|
|
|
// Stores the Slices that designate the boundaries for each subcompaction
|
|
|
|
std::vector<Slice> boundaries_;
|
|
|
|
// Stores the approx size of keys covered in the range of each subcompaction
|
|
|
|
std::vector<uint64_t> sizes_;
|
2017-11-10 18:25:26 +01:00
|
|
|
Env::WriteLifeTimeHint write_hint_;
|
2019-03-20 01:24:09 +01:00
|
|
|
Env::Priority thread_pri_;
|
Pass IOStatus to write path and set retryable IO Error as hard error in BG jobs (#6487)
Summary:
In the current code base, we use Status to get and store the returned status from the call. Specifically, for IO related functions, the current Status cannot reflect the IO Error details such as error scope, error retryable attribute, and others. With the implementation of https://github.com/facebook/rocksdb/issues/5761, we have the new Wrapper for IO, which returns IOStatus instead of Status. However, the IOStatus is purged at the lower level of write path and transferred to Status.
The first job of this PR is to pass the IOStatus to the write path (flush, WAL write, and Compaction). The second job is to identify the Retryable IO Error as HardError, and set the bg_error_ as HardError. In this case, the DB Instance becomes read only. User is informed of the Status and need to take actions to deal with it (e.g., call db->Resume()).
Pull Request resolved: https://github.com/facebook/rocksdb/pull/6487
Test Plan: Added the testing case to error_handler_fs_test. Pass make asan_check
Reviewed By: anand1976
Differential Revision: D20685017
Pulled By: zhichao-cao
fbshipit-source-id: ff85f042896243abcd6ef37877834e26f36b6eb0
2020-03-28 00:03:05 +01:00
|
|
|
IOStatus io_status_;
|
2014-11-01 00:31:25 +01:00
|
|
|
};
|
|
|
|
|
2020-02-20 21:07:53 +01:00
|
|
|
} // namespace ROCKSDB_NAMESPACE
|