2016-02-10 00:12:00 +01:00
|
|
|
// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
|
2014-11-01 00:31:25 +01:00
|
|
|
// This source code is licensed under the BSD-style license found in the
|
|
|
|
// LICENSE file in the root directory of this source tree. An additional grant
|
|
|
|
// of patent rights can be found in the PATENTS file in the same directory.
|
|
|
|
//
|
|
|
|
// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
|
|
|
|
// Use of this source code is governed by a BSD-style license that can be
|
|
|
|
// found in the LICENSE file. See the AUTHORS file for names of contributors.
|
|
|
|
#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"
|
2015-09-10 23:35:25 +02:00
|
|
|
#include "db/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"
|
Compaction Support for Range Deletion
Summary:
This diff introduces RangeDelAggregator, which takes ownership of iterators
provided to it via AddTombstones(). The tombstones are organized in a two-level
map (snapshot stripe -> begin key -> tombstone). Tombstone creation avoids data
copy by holding Slices returned by the iterator, which remain valid thanks to pinning.
For compaction, we create a hierarchical range tombstone iterator with structure
matching the iterator over compaction input data. An aggregator based on that
iterator is used by CompactionIterator to determine which keys are covered by
range tombstones. In case of merge operand, the same aggregator is used by
MergeHelper. Upon finishing each file in the compaction, relevant range tombstones
are added to the output file's range tombstone metablock and file boundaries are
updated accordingly.
To check whether a key is covered by range tombstone, RangeDelAggregator::ShouldDelete()
considers tombstones in the key's snapshot stripe. When this function is used outside of
compaction, it also checks newer stripes, which can contain covering tombstones. Currently
the intra-stripe check involves a linear scan; however, in the future we plan to collapse ranges
within a stripe such that binary search can be used.
RangeDelAggregator::AddToBuilder() adds all range tombstones in the table's key-range
to a new table's range tombstone meta-block. Since range tombstones may fall in the gap
between files, we may need to extend some files' key-ranges. The strategy is (1) first file
extends as far left as possible and other files do not extend left, (2) all files extend right
until either the start of the next file or the end of the last range tombstone in the gap,
whichever comes first.
One other notable change is adding release/move semantics to ScopedArenaIterator
such that it can be used to transfer ownership of an arena-allocated iterator, similar to
how unique_ptr is used for malloc'd data.
Depends on D61473
Test Plan: compaction_iterator_test, mock_table, end-to-end tests in D63927
Reviewers: sdong, IslamAbdelRahman, wanning, yhchiang, lightmark
Reviewed By: lightmark
Subscribers: andrewkr, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D62205
2016-10-18 21:04:56 +02: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"
|
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"
|
2016-09-24 01:34:04 +02:00
|
|
|
#include "util/db_options.h"
|
Include bunch of more events into EventLogger
Summary:
Added these events:
* Recovery start, finish and also when recovery creates a file
* Trivial move
* Compaction start, finish and when compaction creates a file
* Flush start, finish
Also includes small fix to EventLogger
Also added option ROCKSDB_PRINT_EVENTS_TO_STDOUT which is useful when we debug things. I've spent far too much time chasing LOG files.
Still didn't get sst table properties in JSON. They are written very deeply into the stack. I'll address in separate diff.
TODO:
* Write specification. Let's first use this for a while and figure out what's good data to put here, too. After that we'll write spec
* Write tools that parse and analyze LOGs. This can be in python or go. Good intern task.
Test Plan: Ran db_bench with ROCKSDB_PRINT_EVENTS_TO_STDOUT. Here's the output: https://phabricator.fb.com/P19811976
Reviewers: sdong, yhchiang, rven, MarkCallaghan, kradhakrishnan, anthony
Reviewed By: anthony
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D37521
2015-04-28 00:20:02 +02:00
|
|
|
#include "util/event_logger.h"
|
2014-11-01 00:31:25 +01:00
|
|
|
#include "util/stop_watch.h"
|
|
|
|
#include "util/thread_local.h"
|
|
|
|
|
|
|
|
namespace rocksdb {
|
|
|
|
|
|
|
|
class MemTable;
|
|
|
|
class TableCache;
|
|
|
|
class Version;
|
|
|
|
class VersionEdit;
|
|
|
|
class VersionSet;
|
|
|
|
class Arena;
|
|
|
|
|
|
|
|
class CompactionJob {
|
|
|
|
public:
|
2016-09-24 01:34:04 +02:00
|
|
|
CompactionJob(int job_id, Compaction* compaction,
|
|
|
|
const ImmutableDBOptions& db_options,
|
2014-11-01 00:31:25 +01:00
|
|
|
const EnvOptions& env_options, VersionSet* versions,
|
2017-01-12 00:01:21 +01:00
|
|
|
const std::atomic<bool>* shutting_down, LogBuffer* log_buffer,
|
2015-01-26 22:59:38 +01:00
|
|
|
Directory* db_directory, Directory* output_directory,
|
2016-02-18 00:20:23 +01:00
|
|
|
Statistics* stats, InstrumentedMutex* db_mutex,
|
|
|
|
Status* db_bg_error,
|
2015-05-06 04:01:12 +02:00
|
|
|
std::vector<SequenceNumber> existing_snapshots,
|
2015-12-08 21:25:48 +01:00
|
|
|
SequenceNumber earliest_write_conflict_snapshot,
|
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
|
|
|
std::shared_ptr<Cache> table_cache, EventLogger* event_logger,
|
|
|
|
bool paranoid_file_checks, bool measure_io_stats,
|
2015-06-03 02:07:16 +02:00
|
|
|
const std::string& dbname,
|
|
|
|
CompactionJobStats* compaction_job_stats);
|
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
|
|
|
|
void Prepare();
|
|
|
|
// REQUIRED mutex not held
|
|
|
|
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
|
2016-02-18 00:20:23 +01:00
|
|
|
Status Install(const MutableCFOptions& mutable_cf_options);
|
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();
|
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,
|
|
|
|
RangeDelAggregator* range_del_agg,
|
|
|
|
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_;
|
2016-09-24 01:34:04 +02:00
|
|
|
const ImmutableDBOptions& db_options_;
|
2014-11-01 00:31:25 +01:00
|
|
|
const EnvOptions& env_options_;
|
2016-05-17 22:11:56 +02:00
|
|
|
|
2014-11-01 00:31:25 +01:00
|
|
|
Env* env_;
|
|
|
|
VersionSet* versions_;
|
2017-01-12 00:01:21 +01:00
|
|
|
const std::atomic<bool>* shutting_down_;
|
2014-11-01 00:31:25 +01:00
|
|
|
LogBuffer* log_buffer_;
|
|
|
|
Directory* db_directory_;
|
2015-01-26 22:59:38 +01:00
|
|
|
Directory* output_directory_;
|
2014-11-01 00:31:25 +01:00
|
|
|
Statistics* stats_;
|
2016-02-18 00:20:23 +01:00
|
|
|
InstrumentedMutex* db_mutex_;
|
|
|
|
Status* db_bg_error_;
|
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_;
|
|
|
|
|
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
|
|
|
|
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_;
|
2014-11-01 00:31:25 +01:00
|
|
|
};
|
|
|
|
|
|
|
|
} // namespace rocksdb
|