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).
|
2013-10-16 23:59:46 +02:00
|
|
|
//
|
2011-03-18 23:37:00 +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.
|
2013-10-05 07:32:05 +02:00
|
|
|
#pragma once
|
2014-01-02 20:26:57 +01:00
|
|
|
|
2013-05-24 21:52:45 +02:00
|
|
|
#include <atomic>
|
2012-03-09 01:23:21 +01:00
|
|
|
#include <deque>
|
2016-04-18 20:11:51 +02:00
|
|
|
#include <functional>
|
2014-07-04 00:47:02 +02:00
|
|
|
#include <limits>
|
2014-11-07 20:50:34 +01:00
|
|
|
#include <list>
|
2016-11-12 05:45:47 +01:00
|
|
|
#include <map>
|
2015-09-02 22:58:22 +02:00
|
|
|
#include <set>
|
2014-02-26 19:03:34 +01:00
|
|
|
#include <string>
|
2015-09-02 22:58:22 +02:00
|
|
|
#include <utility>
|
|
|
|
#include <vector>
|
2013-12-31 03:33:57 +01:00
|
|
|
|
2014-01-24 23:30:28 +01:00
|
|
|
#include "db/column_family.h"
|
2019-05-31 20:52:59 +02:00
|
|
|
#include "db/compaction/compaction_job.h"
|
2015-09-02 22:58:22 +02:00
|
|
|
#include "db/dbformat.h"
|
2018-06-28 21:23:57 +02:00
|
|
|
#include "db/error_handler.h"
|
|
|
|
#include "db/event_helpers.h"
|
2018-08-23 19:04:10 +02:00
|
|
|
#include "db/external_sst_file_ingestion_job.h"
|
2015-06-02 23:12:23 +02:00
|
|
|
#include "db/flush_job.h"
|
2015-09-02 22:58:22 +02:00
|
|
|
#include "db/flush_scheduler.h"
|
Export Import sst files (#5495)
Summary:
Refresh of the earlier change here - https://github.com/facebook/rocksdb/issues/5135
This is a review request for code change needed for - https://github.com/facebook/rocksdb/issues/3469
"Add support for taking snapshot of a column family and creating column family from a given CF snapshot"
We have an implementation for this that we have been testing internally. We have two new APIs that together provide this functionality.
(1) ExportColumnFamily() - This API is modelled after CreateCheckpoint() as below.
// Exports all live SST files of a specified Column Family onto export_dir,
// returning SST files information in metadata.
// - SST files will be created as hard links when the directory specified
// is in the same partition as the db directory, copied otherwise.
// - export_dir should not already exist and will be created by this API.
// - Always triggers a flush.
virtual Status ExportColumnFamily(ColumnFamilyHandle* handle,
const std::string& export_dir,
ExportImportFilesMetaData** metadata);
Internally, the API will DisableFileDeletions(), GetColumnFamilyMetaData(), Parse through
metadata, creating links/copies of all the sst files, EnableFileDeletions() and complete the call by
returning the list of file metadata.
(2) CreateColumnFamilyWithImport() - This API is modeled after IngestExternalFile(), but invoked only during a CF creation as below.
// CreateColumnFamilyWithImport() will create a new column family with
// column_family_name and import external SST files specified in metadata into
// this column family.
// (1) External SST files can be created using SstFileWriter.
// (2) External SST files can be exported from a particular column family in
// an existing DB.
// Option in import_options specifies whether the external files are copied or
// moved (default is copy). When option specifies copy, managing files at
// external_file_path is caller's responsibility. When option specifies a
// move, the call ensures that the specified files at external_file_path are
// deleted on successful return and files are not modified on any error
// return.
// On error return, column family handle returned will be nullptr.
// ColumnFamily will be present on successful return and will not be present
// on error return. ColumnFamily may be present on any crash during this call.
virtual Status CreateColumnFamilyWithImport(
const ColumnFamilyOptions& options, const std::string& column_family_name,
const ImportColumnFamilyOptions& import_options,
const ExportImportFilesMetaData& metadata,
ColumnFamilyHandle** handle);
Internally, this API creates a new CF, parses all the sst files and adds it to the specified column family, at the same level and with same sequence number as in the metadata. Also performs safety checks with respect to overlaps between the sst files being imported.
If incoming sequence number is higher than current local sequence number, local sequence
number is updated to reflect this.
Note, as the sst files is are being moved across Column Families, Column Family name in sst file
will no longer match the actual column family on destination DB. The API does not modify Column
Family name or id in the sst files being imported.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5495
Differential Revision: D16018881
fbshipit-source-id: 9ae2251025d5916d35a9fc4ea4d6707f6be16ff9
2019-07-17 21:22:21 +02:00
|
|
|
#include "db/import_column_family_job.h"
|
2015-09-02 22:58:22 +02:00
|
|
|
#include "db/internal_stats.h"
|
2015-08-07 02:59:05 +02:00
|
|
|
#include "db/log_writer.h"
|
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
|
|
|
#include "db/logs_with_prep_tracker.h"
|
2019-06-01 00:21:36 +02:00
|
|
|
#include "db/memtable_list.h"
|
2017-12-01 08:39:56 +01:00
|
|
|
#include "db/pre_release_callback.h"
|
2018-12-18 02:26:56 +01:00
|
|
|
#include "db/range_del_aggregator.h"
|
2017-09-11 17:58:52 +02:00
|
|
|
#include "db/read_callback.h"
|
2017-10-06 19:26:38 +02:00
|
|
|
#include "db/snapshot_checker.h"
|
2015-08-07 02:59:05 +02:00
|
|
|
#include "db/snapshot_impl.h"
|
Refactor trimming logic for immutable memtables (#5022)
Summary:
MyRocks currently sets `max_write_buffer_number_to_maintain` in order to maintain enough history for transaction conflict checking. The effectiveness of this approach depends on the size of memtables. When memtables are small, it may not keep enough history; when memtables are large, this may consume too much memory.
We are proposing a new way to configure memtable list history: by limiting the memory usage of immutable memtables. The new option is `max_write_buffer_size_to_maintain` and it will take precedence over the old `max_write_buffer_number_to_maintain` if they are both set to non-zero values. The new option accounts for the total memory usage of flushed immutable memtables and mutable memtable. When the total usage exceeds the limit, RocksDB may start dropping immutable memtables (which is also called trimming history), starting from the oldest one.
The semantics of the old option actually works both as an upper bound and lower bound. History trimming will start if number of immutable memtables exceeds the limit, but it will never go below (limit-1) due to history trimming.
In order the mimic the behavior with the new option, history trimming will stop if dropping the next immutable memtable causes the total memory usage go below the size limit. For example, assuming the size limit is set to 64MB, and there are 3 immutable memtables with sizes of 20, 30, 30. Although the total memory usage is 80MB > 64MB, dropping the oldest memtable will reduce the memory usage to 60MB < 64MB, so in this case no memtable will be dropped.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5022
Differential Revision: D14394062
Pulled By: miasantreble
fbshipit-source-id: 60457a509c6af89d0993f988c9b5c2aa9e45f5c5
2019-08-23 22:54:09 +02:00
|
|
|
#include "db/trim_history_scheduler.h"
|
2013-11-12 20:53:26 +01:00
|
|
|
#include "db/version_edit.h"
|
2014-10-30 01:43:37 +01:00
|
|
|
#include "db/wal_manager.h"
|
2015-09-02 22:58:22 +02:00
|
|
|
#include "db/write_controller.h"
|
|
|
|
#include "db/write_thread.h"
|
2019-06-01 02:19:43 +02:00
|
|
|
#include "logging/event_logger.h"
|
2017-04-06 04:02:00 +02:00
|
|
|
#include "monitoring/instrumented_mutex.h"
|
|
|
|
#include "options/db_options.h"
|
2014-01-02 20:26:57 +01:00
|
|
|
#include "port/port.h"
|
2013-08-23 17:38:13 +02:00
|
|
|
#include "rocksdb/db.h"
|
|
|
|
#include "rocksdb/env.h"
|
|
|
|
#include "rocksdb/memtablerep.h"
|
2017-04-06 02:14:05 +02:00
|
|
|
#include "rocksdb/status.h"
|
2018-08-01 09:14:43 +02:00
|
|
|
#include "rocksdb/trace_reader_writer.h"
|
2013-08-23 17:38:13 +02:00
|
|
|
#include "rocksdb/transaction_log.h"
|
2016-06-21 03:01:03 +02:00
|
|
|
#include "rocksdb/write_buffer_manager.h"
|
2015-10-13 00:06:38 +02:00
|
|
|
#include "table/scoped_arena_iterator.h"
|
2014-01-14 23:49:31 +01:00
|
|
|
#include "util/autovector.h"
|
EventLogger
Summary:
Here's my proposal for making our LOGs easier to read by machines.
The idea is to dump all events as JSON objects. JSON is easy to read by humans, but more importantly, it's easy to read by machines. That way, we can parse this, load into SQLite/mongo and then query or visualize.
I started with table_create and table_delete events, but if everybody agrees, I'll continue by adding more events (flush/compaction/etc etc)
Test Plan:
Ran db_bench. Observed:
2015/01/15-14:13:25.788019 1105ef000 EVENT_LOG_v1 {"time_micros": 1421360005788015, "event": "table_file_creation", "file_number": 12, "file_size": 1909699}
2015/01/15-14:13:25.956500 110740000 EVENT_LOG_v1 {"time_micros": 1421360005956498, "event": "table_file_deletion", "file_number": 12}
Reviewers: yhchiang, rven, dhruba, MarkCallaghan, lgalanis, sdong
Reviewed By: sdong
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D31647
2015-03-13 18:15:54 +01:00
|
|
|
#include "util/hash.h"
|
move dump stats to a separate thread (#4382)
Summary:
Currently statistics are supposed to be dumped to info log at intervals of `options.stats_dump_period_sec`. However the implementation choice was to bind it with compaction thread, meaning if the database has been serving very light traffic, the stats may not get dumped at all.
We decided to separate stats dumping into a new timed thread using `TimerQueue`, which is already used in blob_db. This will allow us schedule new timed tasks with more deterministic behavior.
Tested with db_bench using `--stats_dump_period_sec=20` in command line:
> LOG:2018/09/17-14:07:45.575025 7fe99fbfe700 [WARN] [db/db_impl.cc:605] ------- DUMPING STATS -------
LOG:2018/09/17-14:08:05.643286 7fe99fbfe700 [WARN] [db/db_impl.cc:605] ------- DUMPING STATS -------
LOG:2018/09/17-14:08:25.691325 7fe99fbfe700 [WARN] [db/db_impl.cc:605] ------- DUMPING STATS -------
LOG:2018/09/17-14:08:45.740989 7fe99fbfe700 [WARN] [db/db_impl.cc:605] ------- DUMPING STATS -------
LOG content:
> 2018/09/17-14:07:45.575025 7fe99fbfe700 [WARN] [db/db_impl.cc:605] ------- DUMPING STATS -------
2018/09/17-14:07:45.575080 7fe99fbfe700 [WARN] [db/db_impl.cc:606]
** DB Stats **
Uptime(secs): 20.0 total, 20.0 interval
Cumulative writes: 4447K writes, 4447K keys, 4447K commit groups, 1.0 writes per commit group, ingest: 5.57 GB, 285.01 MB/s
Cumulative WAL: 4447K writes, 0 syncs, 4447638.00 writes per sync, written: 5.57 GB, 285.01 MB/s
Cumulative stall: 00:00:0.012 H:M:S, 0.1 percent
Interval writes: 4447K writes, 4447K keys, 4447K commit groups, 1.0 writes per commit group, ingest: 5700.71 MB, 285.01 MB/s
Interval WAL: 4447K writes, 0 syncs, 4447638.00 writes per sync, written: 5.57 MB, 285.01 MB/s
Interval stall: 00:00:0.012 H:M:S, 0.1 percent
** Compaction Stats [default] **
Level Files Size Score Read(GB) Rn(GB) Rnp1(GB) Write(GB) Wnew(GB) Moved(GB) W-Amp Rd(MB/s) Wr(MB/s) Comp(sec) Comp(cnt) Avg(sec) KeyIn KeyDrop
Pull Request resolved: https://github.com/facebook/rocksdb/pull/4382
Differential Revision: D9933051
Pulled By: miasantreble
fbshipit-source-id: 6d12bb1e4977674eea4bf2d2ac6d486b814bb2fa
2018-10-09 07:52:58 +02:00
|
|
|
#include "util/repeatable_thread.h"
|
2015-09-02 22:58:22 +02:00
|
|
|
#include "util/stop_watch.h"
|
|
|
|
#include "util/thread_local.h"
|
2012-08-15 00:20:36 +02:00
|
|
|
|
2020-02-20 21:07:53 +01:00
|
|
|
namespace ROCKSDB_NAMESPACE {
|
2011-03-18 23:37:00 +01:00
|
|
|
|
2017-10-06 19:26:38 +02:00
|
|
|
class Arena;
|
2017-10-03 18:08:07 +02:00
|
|
|
class ArenaWrappedDBIter;
|
2019-02-21 00:46:59 +01:00
|
|
|
class InMemoryStatsHistoryIterator;
|
2011-03-18 23:37:00 +01:00
|
|
|
class MemTable;
|
2019-06-18 00:17:43 +02:00
|
|
|
class PersistentStatsHistoryIterator;
|
2020-10-02 04:12:26 +02:00
|
|
|
class PeriodicWorkScheduler;
|
2020-08-15 05:11:35 +02:00
|
|
|
#ifndef NDEBUG
|
2020-10-02 04:12:26 +02:00
|
|
|
class PeriodicWorkTestScheduler;
|
2020-08-15 05:11:35 +02:00
|
|
|
#endif // !NDEBUG
|
2011-03-18 23:37:00 +01:00
|
|
|
class TableCache;
|
Concurrent task limiter for compaction thread control (#4332)
Summary:
The PR is targeting to resolve the issue of:
https://github.com/facebook/rocksdb/issues/3972#issue-330771918
We have a rocksdb created with leveled-compaction with multiple column families (CFs), some of CFs are using HDD to store big and less frequently accessed data and others are using SSD.
When there are continuously write traffics going on to all CFs, the compaction thread pool is mostly occupied by those slow HDD compactions, which blocks fully utilize SSD bandwidth.
Since atomic write and transaction is needed across CFs, so splitting it to multiple rocksdb instance is not an option for us.
With the compaction thread control, we got 30%+ HDD write throughput gain, and also a lot smooth SSD write since less write stall happening.
ConcurrentTaskLimiter can be shared with multi-CFs across rocksdb instances, so the feature does not only work for multi-CFs scenarios, but also for multi-rocksdbs scenarios, who need disk IO resource control per tenant.
The usage is straight forward:
e.g.:
//
// Enable compaction thread limiter thru ColumnFamilyOptions
//
std::shared_ptr<ConcurrentTaskLimiter> ctl(NewConcurrentTaskLimiter("foo_limiter", 4));
Options options;
ColumnFamilyOptions cf_opt(options);
cf_opt.compaction_thread_limiter = ctl;
...
//
// Compaction thread limiter can be tuned or disabled on-the-fly
//
ctl->SetMaxOutstandingTask(12); // enlarge to 12 tasks
...
ctl->ResetMaxOutstandingTask(); // disable (bypass) thread limiter
ctl->SetMaxOutstandingTask(-1); // Same as above
...
ctl->SetMaxOutstandingTask(0); // full throttle (0 task)
//
// Sharing compaction thread limiter among CFs (to resolve multiple storage perf issue)
//
std::shared_ptr<ConcurrentTaskLimiter> ctl_ssd(NewConcurrentTaskLimiter("ssd_limiter", 8));
std::shared_ptr<ConcurrentTaskLimiter> ctl_hdd(NewConcurrentTaskLimiter("hdd_limiter", 4));
Options options;
ColumnFamilyOptions cf_opt_ssd1(options);
ColumnFamilyOptions cf_opt_ssd2(options);
ColumnFamilyOptions cf_opt_hdd1(options);
ColumnFamilyOptions cf_opt_hdd2(options);
ColumnFamilyOptions cf_opt_hdd3(options);
// SSD CFs
cf_opt_ssd1.compaction_thread_limiter = ctl_ssd;
cf_opt_ssd2.compaction_thread_limiter = ctl_ssd;
// HDD CFs
cf_opt_hdd1.compaction_thread_limiter = ctl_hdd;
cf_opt_hdd2.compaction_thread_limiter = ctl_hdd;
cf_opt_hdd3.compaction_thread_limiter = ctl_hdd;
...
//
// The limiter is disabled by default (or set to nullptr explicitly)
//
Options options;
ColumnFamilyOptions cf_opt(options);
cf_opt.compaction_thread_limiter = nullptr;
Pull Request resolved: https://github.com/facebook/rocksdb/pull/4332
Differential Revision: D13226590
Pulled By: siying
fbshipit-source-id: 14307aec55b8bd59c8223d04aa6db3c03d1b0c1d
2018-12-13 22:16:04 +01:00
|
|
|
class TaskLimiterToken;
|
2011-03-18 23:37:00 +01:00
|
|
|
class Version;
|
|
|
|
class VersionEdit;
|
|
|
|
class VersionSet;
|
2015-05-29 23:36:35 +02:00
|
|
|
class WriteCallback;
|
2014-11-15 00:43:10 +01:00
|
|
|
struct JobContext;
|
2015-09-23 21:42:43 +02:00
|
|
|
struct ExternalSstFileInfo;
|
2016-06-02 20:57:31 +02:00
|
|
|
struct MemTableInfo;
|
2011-03-18 23:37:00 +01:00
|
|
|
|
2019-06-21 19:12:29 +02:00
|
|
|
// Class to maintain directories for all database paths other than main one.
|
|
|
|
class Directories {
|
|
|
|
public:
|
2020-03-03 01:14:00 +01:00
|
|
|
IOStatus SetDirectories(FileSystem* fs, const std::string& dbname,
|
|
|
|
const std::string& wal_dir,
|
|
|
|
const std::vector<DbPath>& data_paths);
|
2019-06-21 19:12:29 +02:00
|
|
|
|
2020-03-03 01:14:00 +01:00
|
|
|
FSDirectory* GetDataDir(size_t path_id) const {
|
2019-06-21 19:12:29 +02:00
|
|
|
assert(path_id < data_dirs_.size());
|
2020-03-03 01:14:00 +01:00
|
|
|
FSDirectory* ret_dir = data_dirs_[path_id].get();
|
2019-06-21 19:12:29 +02:00
|
|
|
if (ret_dir == nullptr) {
|
|
|
|
// Should use db_dir_
|
|
|
|
return db_dir_.get();
|
|
|
|
}
|
|
|
|
return ret_dir;
|
|
|
|
}
|
|
|
|
|
2020-03-03 01:14:00 +01:00
|
|
|
FSDirectory* GetWalDir() {
|
2019-06-21 19:12:29 +02:00
|
|
|
if (wal_dir_) {
|
|
|
|
return wal_dir_.get();
|
|
|
|
}
|
|
|
|
return db_dir_.get();
|
|
|
|
}
|
|
|
|
|
2020-03-03 01:14:00 +01:00
|
|
|
FSDirectory* GetDbDir() { return db_dir_.get(); }
|
2019-06-21 19:12:29 +02:00
|
|
|
|
|
|
|
private:
|
2020-03-03 01:14:00 +01:00
|
|
|
std::unique_ptr<FSDirectory> db_dir_;
|
|
|
|
std::vector<std::unique_ptr<FSDirectory>> data_dirs_;
|
|
|
|
std::unique_ptr<FSDirectory> wal_dir_;
|
2019-06-21 19:12:29 +02:00
|
|
|
};
|
|
|
|
|
2019-05-24 22:05:58 +02:00
|
|
|
// While DB is the public interface of RocksDB, and DBImpl is the actual
|
|
|
|
// class implementing it. It's the entrance of the core RocksdB engine.
|
|
|
|
// All other DB implementations, e.g. TransactionDB, BlobDB, etc, wrap a
|
|
|
|
// DBImpl internally.
|
|
|
|
// Other than functions implementing the DB interface, some public
|
|
|
|
// functions are there for other internal components to call. For
|
|
|
|
// example, TransactionDB directly calls DBImpl::WriteImpl() and
|
|
|
|
// BlobDB directly calls DBImpl::GetImpl(). Some other functions
|
|
|
|
// are for sub-components to call. For example, ColumnFamilyHandleImpl
|
|
|
|
// calls DBImpl::FindObsoleteFiles().
|
|
|
|
//
|
|
|
|
// Since it's a very large class, the definition of the functions is
|
|
|
|
// divided in several db_impl_*.cc files, besides db_impl.cc.
|
2011-03-18 23:37:00 +01:00
|
|
|
class DBImpl : public DB {
|
|
|
|
public:
|
2017-11-11 02:18:01 +01:00
|
|
|
DBImpl(const DBOptions& options, const std::string& dbname,
|
2018-06-29 03:46:39 +02:00
|
|
|
const bool seq_per_batch = false, const bool batch_per_txn = true);
|
2019-09-12 03:07:12 +02:00
|
|
|
// No copying allowed
|
|
|
|
DBImpl(const DBImpl&) = delete;
|
|
|
|
void operator=(const DBImpl&) = delete;
|
|
|
|
|
2011-03-18 23:37:00 +01:00
|
|
|
virtual ~DBImpl();
|
|
|
|
|
2019-05-24 22:05:58 +02:00
|
|
|
// ---- Implementations of the DB interface ----
|
|
|
|
|
2018-06-28 21:23:57 +02:00
|
|
|
using DB::Resume;
|
|
|
|
virtual Status Resume() override;
|
|
|
|
|
[RocksDB] [Column Family] Interface proposal
Summary:
<This diff is for Column Family branch>
Sharing some of the work I've done so far. This diff compiles and passes the tests.
The biggest change is in options.h - I broke down Options into two parts - DBOptions and ColumnFamilyOptions. DBOptions is DB-specific (env, create_if_missing, block_cache, etc.) and ColumnFamilyOptions is column family-specific (all compaction options, compresion options, etc.). Note that this does not break backwards compatibility at all.
Further, I created DBWithColumnFamily which inherits DB interface and adds new functions with column family support. Clients can transparently switch to DBWithColumnFamily and it will not break their backwards compatibility.
There are few methods worth checking out: ListColumnFamilies(), MultiNewIterator(), MultiGet() and GetSnapshot(). [GetSnapshot() returns the snapshot across all column families for now - I think that's what we agreed on]
Finally, I made small changes to WriteBatch so we are able to atomically insert data across column families.
Please provide feedback.
Test Plan: make check works, the code is backward compatible
Reviewers: dhruba, haobo, sdong, kailiu, emayanke
CC: leveldb
Differential Revision: https://reviews.facebook.net/D14445
2013-12-03 20:14:09 +01:00
|
|
|
using DB::Put;
|
|
|
|
virtual Status Put(const WriteOptions& options,
|
2014-02-11 02:04:44 +01:00
|
|
|
ColumnFamilyHandle* column_family, const Slice& key,
|
2015-02-26 20:28:41 +01:00
|
|
|
const Slice& value) override;
|
[RocksDB] [Column Family] Interface proposal
Summary:
<This diff is for Column Family branch>
Sharing some of the work I've done so far. This diff compiles and passes the tests.
The biggest change is in options.h - I broke down Options into two parts - DBOptions and ColumnFamilyOptions. DBOptions is DB-specific (env, create_if_missing, block_cache, etc.) and ColumnFamilyOptions is column family-specific (all compaction options, compresion options, etc.). Note that this does not break backwards compatibility at all.
Further, I created DBWithColumnFamily which inherits DB interface and adds new functions with column family support. Clients can transparently switch to DBWithColumnFamily and it will not break their backwards compatibility.
There are few methods worth checking out: ListColumnFamilies(), MultiNewIterator(), MultiGet() and GetSnapshot(). [GetSnapshot() returns the snapshot across all column families for now - I think that's what we agreed on]
Finally, I made small changes to WriteBatch so we are able to atomically insert data across column families.
Please provide feedback.
Test Plan: make check works, the code is backward compatible
Reviewers: dhruba, haobo, sdong, kailiu, emayanke
CC: leveldb
Differential Revision: https://reviews.facebook.net/D14445
2013-12-03 20:14:09 +01:00
|
|
|
using DB::Merge;
|
|
|
|
virtual Status Merge(const WriteOptions& options,
|
2014-02-11 02:04:44 +01:00
|
|
|
ColumnFamilyHandle* column_family, const Slice& key,
|
2015-02-26 20:28:41 +01:00
|
|
|
const Slice& value) override;
|
[RocksDB] [Column Family] Interface proposal
Summary:
<This diff is for Column Family branch>
Sharing some of the work I've done so far. This diff compiles and passes the tests.
The biggest change is in options.h - I broke down Options into two parts - DBOptions and ColumnFamilyOptions. DBOptions is DB-specific (env, create_if_missing, block_cache, etc.) and ColumnFamilyOptions is column family-specific (all compaction options, compresion options, etc.). Note that this does not break backwards compatibility at all.
Further, I created DBWithColumnFamily which inherits DB interface and adds new functions with column family support. Clients can transparently switch to DBWithColumnFamily and it will not break their backwards compatibility.
There are few methods worth checking out: ListColumnFamilies(), MultiNewIterator(), MultiGet() and GetSnapshot(). [GetSnapshot() returns the snapshot across all column families for now - I think that's what we agreed on]
Finally, I made small changes to WriteBatch so we are able to atomically insert data across column families.
Please provide feedback.
Test Plan: make check works, the code is backward compatible
Reviewers: dhruba, haobo, sdong, kailiu, emayanke
CC: leveldb
Differential Revision: https://reviews.facebook.net/D14445
2013-12-03 20:14:09 +01:00
|
|
|
using DB::Delete;
|
|
|
|
virtual Status Delete(const WriteOptions& options,
|
2015-02-26 20:28:41 +01:00
|
|
|
ColumnFamilyHandle* column_family,
|
|
|
|
const Slice& key) override;
|
Support for SingleDelete()
Summary:
This patch fixes #7460559. It introduces SingleDelete as a new database
operation. This operation can be used to delete keys that were never
overwritten (no put following another put of the same key). If an overwritten
key is single deleted the behavior is undefined. Single deletion of a
non-existent key has no effect but multiple consecutive single deletions are
not allowed (see limitations).
In contrast to the conventional Delete() operation, the deletion entry is
removed along with the value when the two are lined up in a compaction. Note:
The semantics are similar to @igor's prototype that allowed to have this
behavior on the granularity of a column family (
https://reviews.facebook.net/D42093 ). This new patch, however, is more
aggressive when it comes to removing tombstones: It removes the SingleDelete
together with the value whenever there is no snapshot between them while the
older patch only did this when the sequence number of the deletion was older
than the earliest snapshot.
Most of the complex additions are in the Compaction Iterator, all other changes
should be relatively straightforward. The patch also includes basic support for
single deletions in db_stress and db_bench.
Limitations:
- Not compatible with cuckoo hash tables
- Single deletions cannot be used in combination with merges and normal
deletions on the same key (other keys are not affected by this)
- Consecutive single deletions are currently not allowed (and older version of
this patch supported this so it could be resurrected if needed)
Test Plan: make all check
Reviewers: yhchiang, sdong, rven, anthony, yoshinorim, igor
Reviewed By: igor
Subscribers: maykov, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D43179
2015-09-17 20:42:56 +02:00
|
|
|
using DB::SingleDelete;
|
|
|
|
virtual Status SingleDelete(const WriteOptions& options,
|
|
|
|
ColumnFamilyHandle* column_family,
|
|
|
|
const Slice& key) override;
|
[RocksDB] [Column Family] Interface proposal
Summary:
<This diff is for Column Family branch>
Sharing some of the work I've done so far. This diff compiles and passes the tests.
The biggest change is in options.h - I broke down Options into two parts - DBOptions and ColumnFamilyOptions. DBOptions is DB-specific (env, create_if_missing, block_cache, etc.) and ColumnFamilyOptions is column family-specific (all compaction options, compresion options, etc.). Note that this does not break backwards compatibility at all.
Further, I created DBWithColumnFamily which inherits DB interface and adds new functions with column family support. Clients can transparently switch to DBWithColumnFamily and it will not break their backwards compatibility.
There are few methods worth checking out: ListColumnFamilies(), MultiNewIterator(), MultiGet() and GetSnapshot(). [GetSnapshot() returns the snapshot across all column families for now - I think that's what we agreed on]
Finally, I made small changes to WriteBatch so we are able to atomically insert data across column families.
Please provide feedback.
Test Plan: make check works, the code is backward compatible
Reviewers: dhruba, haobo, sdong, kailiu, emayanke
CC: leveldb
Differential Revision: https://reviews.facebook.net/D14445
2013-12-03 20:14:09 +01:00
|
|
|
using DB::Write;
|
2015-02-26 20:28:41 +01:00
|
|
|
virtual Status Write(const WriteOptions& options,
|
|
|
|
WriteBatch* updates) override;
|
2015-05-29 23:36:35 +02:00
|
|
|
|
[RocksDB] [Column Family] Interface proposal
Summary:
<This diff is for Column Family branch>
Sharing some of the work I've done so far. This diff compiles and passes the tests.
The biggest change is in options.h - I broke down Options into two parts - DBOptions and ColumnFamilyOptions. DBOptions is DB-specific (env, create_if_missing, block_cache, etc.) and ColumnFamilyOptions is column family-specific (all compaction options, compresion options, etc.). Note that this does not break backwards compatibility at all.
Further, I created DBWithColumnFamily which inherits DB interface and adds new functions with column family support. Clients can transparently switch to DBWithColumnFamily and it will not break their backwards compatibility.
There are few methods worth checking out: ListColumnFamilies(), MultiNewIterator(), MultiGet() and GetSnapshot(). [GetSnapshot() returns the snapshot across all column families for now - I think that's what we agreed on]
Finally, I made small changes to WriteBatch so we are able to atomically insert data across column families.
Please provide feedback.
Test Plan: make check works, the code is backward compatible
Reviewers: dhruba, haobo, sdong, kailiu, emayanke
CC: leveldb
Differential Revision: https://reviews.facebook.net/D14445
2013-12-03 20:14:09 +01:00
|
|
|
using DB::Get;
|
2011-03-18 23:37:00 +01:00
|
|
|
virtual Status Get(const ReadOptions& options,
|
2014-02-11 02:04:44 +01:00
|
|
|
ColumnFamilyHandle* column_family, const Slice& key,
|
2017-03-13 19:44:50 +01:00
|
|
|
PinnableSlice* value) override;
|
2020-03-03 00:58:32 +01:00
|
|
|
virtual Status Get(const ReadOptions& options,
|
|
|
|
ColumnFamilyHandle* column_family, const Slice& key,
|
|
|
|
PinnableSlice* value, std::string* timestamp) override;
|
2017-10-18 02:24:25 +02:00
|
|
|
|
New API to get all merge operands for a Key (#5604)
Summary:
This is a new API added to db.h to allow for fetching all merge operands associated with a Key. The main motivation for this API is to support use cases where doing a full online merge is not necessary as it is performance sensitive. Example use-cases:
1. Update subset of columns and read subset of columns -
Imagine a SQL Table, a row is encoded as a K/V pair (as it is done in MyRocks). If there are many columns and users only updated one of them, we can use merge operator to reduce write amplification. While users only read one or two columns in the read query, this feature can avoid a full merging of the whole row, and save some CPU.
2. Updating very few attributes in a value which is a JSON-like document -
Updating one attribute can be done efficiently using merge operator, while reading back one attribute can be done more efficiently if we don't need to do a full merge.
----------------------------------------------------------------------------------------------------
API :
Status GetMergeOperands(
const ReadOptions& options, ColumnFamilyHandle* column_family,
const Slice& key, PinnableSlice* merge_operands,
GetMergeOperandsOptions* get_merge_operands_options,
int* number_of_operands)
Example usage :
int size = 100;
int number_of_operands = 0;
std::vector<PinnableSlice> values(size);
GetMergeOperandsOptions merge_operands_info;
db_->GetMergeOperands(ReadOptions(), db_->DefaultColumnFamily(), "k1", values.data(), merge_operands_info, &number_of_operands);
Description :
Returns all the merge operands corresponding to the key. If the number of merge operands in DB is greater than merge_operands_options.expected_max_number_of_operands no merge operands are returned and status is Incomplete. Merge operands returned are in the order of insertion.
merge_operands-> Points to an array of at-least merge_operands_options.expected_max_number_of_operands and the caller is responsible for allocating it. If the status returned is Incomplete then number_of_operands will contain the total number of merge operands found in DB for key.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5604
Test Plan:
Added unit test and perf test in db_bench that can be run using the command:
./db_bench -benchmarks=getmergeoperands --merge_operator=sortlist
Differential Revision: D16657366
Pulled By: vjnadimpalli
fbshipit-source-id: 0faadd752351745224ee12d4ae9ef3cb529951bf
2019-08-06 23:22:34 +02:00
|
|
|
using DB::GetMergeOperands;
|
|
|
|
Status GetMergeOperands(const ReadOptions& options,
|
|
|
|
ColumnFamilyHandle* column_family, const Slice& key,
|
|
|
|
PinnableSlice* merge_operands,
|
|
|
|
GetMergeOperandsOptions* get_merge_operands_options,
|
|
|
|
int* number_of_operands) override {
|
|
|
|
GetImplOptions get_impl_options;
|
|
|
|
get_impl_options.column_family = column_family;
|
|
|
|
get_impl_options.merge_operands = merge_operands;
|
|
|
|
get_impl_options.get_merge_operands_options = get_merge_operands_options;
|
|
|
|
get_impl_options.number_of_operands = number_of_operands;
|
|
|
|
get_impl_options.get_value = false;
|
|
|
|
return GetImpl(options, key, get_impl_options);
|
|
|
|
}
|
|
|
|
|
[RocksDB] [Column Family] Interface proposal
Summary:
<This diff is for Column Family branch>
Sharing some of the work I've done so far. This diff compiles and passes the tests.
The biggest change is in options.h - I broke down Options into two parts - DBOptions and ColumnFamilyOptions. DBOptions is DB-specific (env, create_if_missing, block_cache, etc.) and ColumnFamilyOptions is column family-specific (all compaction options, compresion options, etc.). Note that this does not break backwards compatibility at all.
Further, I created DBWithColumnFamily which inherits DB interface and adds new functions with column family support. Clients can transparently switch to DBWithColumnFamily and it will not break their backwards compatibility.
There are few methods worth checking out: ListColumnFamilies(), MultiNewIterator(), MultiGet() and GetSnapshot(). [GetSnapshot() returns the snapshot across all column families for now - I think that's what we agreed on]
Finally, I made small changes to WriteBatch so we are able to atomically insert data across column families.
Please provide feedback.
Test Plan: make check works, the code is backward compatible
Reviewers: dhruba, haobo, sdong, kailiu, emayanke
CC: leveldb
Differential Revision: https://reviews.facebook.net/D14445
2013-12-03 20:14:09 +01:00
|
|
|
using DB::MultiGet;
|
|
|
|
virtual std::vector<Status> MultiGet(
|
|
|
|
const ReadOptions& options,
|
2014-02-11 02:04:44 +01:00
|
|
|
const std::vector<ColumnFamilyHandle*>& column_family,
|
2015-02-26 20:28:41 +01:00
|
|
|
const std::vector<Slice>& keys,
|
|
|
|
std::vector<std::string>* values) override;
|
2020-03-24 19:21:10 +01:00
|
|
|
virtual std::vector<Status> MultiGet(
|
|
|
|
const ReadOptions& options,
|
|
|
|
const std::vector<ColumnFamilyHandle*>& column_family,
|
|
|
|
const std::vector<Slice>& keys, std::vector<std::string>* values,
|
|
|
|
std::vector<std::string>* timestamps) override;
|
2013-07-06 03:49:18 +02:00
|
|
|
|
Introduce a new MultiGet batching implementation (#5011)
Summary:
This PR introduces a new MultiGet() API, with the underlying implementation grouping keys based on SST file and batching lookups in a file. The reason for the new API is twofold - the definition allows callers to allocate storage for status and values on stack instead of std::vector, as well as return values as PinnableSlices in order to avoid copying, and it keeps the original MultiGet() implementation intact while we experiment with batching.
Batching is useful when there is some spatial locality to the keys being queries, as well as larger batch sizes. The main benefits are due to -
1. Fewer function calls, especially to BlockBasedTableReader::MultiGet() and FullFilterBlockReader::KeysMayMatch()
2. Bloom filter cachelines can be prefetched, hiding the cache miss latency
The next step is to optimize the binary searches in the level_storage_info, index blocks and data blocks, since we could reduce the number of key comparisons if the keys are relatively close to each other. The batching optimizations also need to be extended to other formats, such as PlainTable and filter formats. This also needs to be added to db_stress.
Benchmark results from db_bench for various batch size/locality of reference combinations are given below. Locality was simulated by offsetting the keys in a batch by a stride length. Each SST file is about 8.6MB uncompressed and key/value size is 16/100 uncompressed. To focus on the cpu benefit of batching, the runs were single threaded and bound to the same cpu to eliminate interference from other system events. The results show a 10-25% improvement in micros/op from smaller to larger batch sizes (4 - 32).
Batch Sizes
1 | 2 | 4 | 8 | 16 | 32
Random pattern (Stride length 0)
4.158 | 4.109 | 4.026 | 4.05 | 4.1 | 4.074 - Get
4.438 | 4.302 | 4.165 | 4.122 | 4.096 | 4.075 - MultiGet (no batching)
4.461 | 4.256 | 4.277 | 4.11 | 4.182 | 4.14 - MultiGet (w/ batching)
Good locality (Stride length 16)
4.048 | 3.659 | 3.248 | 2.99 | 2.84 | 2.753
4.429 | 3.728 | 3.406 | 3.053 | 2.911 | 2.781
4.452 | 3.45 | 2.833 | 2.451 | 2.233 | 2.135
Good locality (Stride length 256)
4.066 | 3.786 | 3.581 | 3.447 | 3.415 | 3.232
4.406 | 4.005 | 3.644 | 3.49 | 3.381 | 3.268
4.393 | 3.649 | 3.186 | 2.882 | 2.676 | 2.62
Medium locality (Stride length 4096)
4.012 | 3.922 | 3.768 | 3.61 | 3.582 | 3.555
4.364 | 4.057 | 3.791 | 3.65 | 3.57 | 3.465
4.479 | 3.758 | 3.316 | 3.077 | 2.959 | 2.891
dbbench command used (on a DB with 4 levels, 12 million keys)-
TEST_TMPDIR=/dev/shm numactl -C 10 ./db_bench.tmp -use_existing_db=true -benchmarks="readseq,multireadrandom" -write_buffer_size=4194304 -target_file_size_base=4194304 -max_bytes_for_level_base=16777216 -num=12000000 -reads=12000000 -duration=90 -threads=1 -compression_type=none -cache_size=4194304000 -batch_size=32 -disable_auto_compactions=true -bloom_bits=10 -cache_index_and_filter_blocks=true -pin_l0_filter_and_index_blocks_in_cache=true -multiread_batched=true -multiread_stride=4
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5011
Differential Revision: D14348703
Pulled By: anand1976
fbshipit-source-id: 774406dab3776d979c809522a67bedac6c17f84b
2019-04-11 23:24:09 +02:00
|
|
|
// This MultiGet is a batched version, which may be faster than calling Get
|
|
|
|
// multiple times, especially if the keys have some spatial locality that
|
|
|
|
// enables them to be queried in the same SST files/set of files. The larger
|
|
|
|
// the batch size, the more scope for batching and performance improvement
|
|
|
|
// The values and statuses parameters are arrays with number of elements
|
|
|
|
// equal to keys.size(). This allows the storage for those to be alloacted
|
|
|
|
// by the caller on the stack for small batches
|
|
|
|
virtual void MultiGet(const ReadOptions& options,
|
|
|
|
ColumnFamilyHandle* column_family,
|
|
|
|
const size_t num_keys, const Slice* keys,
|
|
|
|
PinnableSlice* values, Status* statuses,
|
|
|
|
const bool sorted_input = false) override;
|
2020-03-24 19:21:10 +01:00
|
|
|
virtual void MultiGet(const ReadOptions& options,
|
|
|
|
ColumnFamilyHandle* column_family,
|
|
|
|
const size_t num_keys, const Slice* keys,
|
|
|
|
PinnableSlice* values, std::string* timestamps,
|
|
|
|
Status* statuses,
|
|
|
|
const bool sorted_input = false) override;
|
Introduce a new MultiGet batching implementation (#5011)
Summary:
This PR introduces a new MultiGet() API, with the underlying implementation grouping keys based on SST file and batching lookups in a file. The reason for the new API is twofold - the definition allows callers to allocate storage for status and values on stack instead of std::vector, as well as return values as PinnableSlices in order to avoid copying, and it keeps the original MultiGet() implementation intact while we experiment with batching.
Batching is useful when there is some spatial locality to the keys being queries, as well as larger batch sizes. The main benefits are due to -
1. Fewer function calls, especially to BlockBasedTableReader::MultiGet() and FullFilterBlockReader::KeysMayMatch()
2. Bloom filter cachelines can be prefetched, hiding the cache miss latency
The next step is to optimize the binary searches in the level_storage_info, index blocks and data blocks, since we could reduce the number of key comparisons if the keys are relatively close to each other. The batching optimizations also need to be extended to other formats, such as PlainTable and filter formats. This also needs to be added to db_stress.
Benchmark results from db_bench for various batch size/locality of reference combinations are given below. Locality was simulated by offsetting the keys in a batch by a stride length. Each SST file is about 8.6MB uncompressed and key/value size is 16/100 uncompressed. To focus on the cpu benefit of batching, the runs were single threaded and bound to the same cpu to eliminate interference from other system events. The results show a 10-25% improvement in micros/op from smaller to larger batch sizes (4 - 32).
Batch Sizes
1 | 2 | 4 | 8 | 16 | 32
Random pattern (Stride length 0)
4.158 | 4.109 | 4.026 | 4.05 | 4.1 | 4.074 - Get
4.438 | 4.302 | 4.165 | 4.122 | 4.096 | 4.075 - MultiGet (no batching)
4.461 | 4.256 | 4.277 | 4.11 | 4.182 | 4.14 - MultiGet (w/ batching)
Good locality (Stride length 16)
4.048 | 3.659 | 3.248 | 2.99 | 2.84 | 2.753
4.429 | 3.728 | 3.406 | 3.053 | 2.911 | 2.781
4.452 | 3.45 | 2.833 | 2.451 | 2.233 | 2.135
Good locality (Stride length 256)
4.066 | 3.786 | 3.581 | 3.447 | 3.415 | 3.232
4.406 | 4.005 | 3.644 | 3.49 | 3.381 | 3.268
4.393 | 3.649 | 3.186 | 2.882 | 2.676 | 2.62
Medium locality (Stride length 4096)
4.012 | 3.922 | 3.768 | 3.61 | 3.582 | 3.555
4.364 | 4.057 | 3.791 | 3.65 | 3.57 | 3.465
4.479 | 3.758 | 3.316 | 3.077 | 2.959 | 2.891
dbbench command used (on a DB with 4 levels, 12 million keys)-
TEST_TMPDIR=/dev/shm numactl -C 10 ./db_bench.tmp -use_existing_db=true -benchmarks="readseq,multireadrandom" -write_buffer_size=4194304 -target_file_size_base=4194304 -max_bytes_for_level_base=16777216 -num=12000000 -reads=12000000 -duration=90 -threads=1 -compression_type=none -cache_size=4194304000 -batch_size=32 -disable_auto_compactions=true -bloom_bits=10 -cache_index_and_filter_blocks=true -pin_l0_filter_and_index_blocks_in_cache=true -multiread_batched=true -multiread_stride=4
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5011
Differential Revision: D14348703
Pulled By: anand1976
fbshipit-source-id: 774406dab3776d979c809522a67bedac6c17f84b
2019-04-11 23:24:09 +02:00
|
|
|
|
2019-11-12 22:51:18 +01:00
|
|
|
virtual void MultiGet(const ReadOptions& options, const size_t num_keys,
|
|
|
|
ColumnFamilyHandle** column_families, const Slice* keys,
|
|
|
|
PinnableSlice* values, Status* statuses,
|
|
|
|
const bool sorted_input = false) override;
|
2020-03-24 19:21:10 +01:00
|
|
|
virtual void MultiGet(const ReadOptions& options, const size_t num_keys,
|
|
|
|
ColumnFamilyHandle** column_families, const Slice* keys,
|
|
|
|
PinnableSlice* values, std::string* timestamps,
|
|
|
|
Status* statuses,
|
|
|
|
const bool sorted_input = false) override;
|
2019-11-12 22:51:18 +01:00
|
|
|
|
|
|
|
virtual void MultiGetWithCallback(
|
Introduce a new MultiGet batching implementation (#5011)
Summary:
This PR introduces a new MultiGet() API, with the underlying implementation grouping keys based on SST file and batching lookups in a file. The reason for the new API is twofold - the definition allows callers to allocate storage for status and values on stack instead of std::vector, as well as return values as PinnableSlices in order to avoid copying, and it keeps the original MultiGet() implementation intact while we experiment with batching.
Batching is useful when there is some spatial locality to the keys being queries, as well as larger batch sizes. The main benefits are due to -
1. Fewer function calls, especially to BlockBasedTableReader::MultiGet() and FullFilterBlockReader::KeysMayMatch()
2. Bloom filter cachelines can be prefetched, hiding the cache miss latency
The next step is to optimize the binary searches in the level_storage_info, index blocks and data blocks, since we could reduce the number of key comparisons if the keys are relatively close to each other. The batching optimizations also need to be extended to other formats, such as PlainTable and filter formats. This also needs to be added to db_stress.
Benchmark results from db_bench for various batch size/locality of reference combinations are given below. Locality was simulated by offsetting the keys in a batch by a stride length. Each SST file is about 8.6MB uncompressed and key/value size is 16/100 uncompressed. To focus on the cpu benefit of batching, the runs were single threaded and bound to the same cpu to eliminate interference from other system events. The results show a 10-25% improvement in micros/op from smaller to larger batch sizes (4 - 32).
Batch Sizes
1 | 2 | 4 | 8 | 16 | 32
Random pattern (Stride length 0)
4.158 | 4.109 | 4.026 | 4.05 | 4.1 | 4.074 - Get
4.438 | 4.302 | 4.165 | 4.122 | 4.096 | 4.075 - MultiGet (no batching)
4.461 | 4.256 | 4.277 | 4.11 | 4.182 | 4.14 - MultiGet (w/ batching)
Good locality (Stride length 16)
4.048 | 3.659 | 3.248 | 2.99 | 2.84 | 2.753
4.429 | 3.728 | 3.406 | 3.053 | 2.911 | 2.781
4.452 | 3.45 | 2.833 | 2.451 | 2.233 | 2.135
Good locality (Stride length 256)
4.066 | 3.786 | 3.581 | 3.447 | 3.415 | 3.232
4.406 | 4.005 | 3.644 | 3.49 | 3.381 | 3.268
4.393 | 3.649 | 3.186 | 2.882 | 2.676 | 2.62
Medium locality (Stride length 4096)
4.012 | 3.922 | 3.768 | 3.61 | 3.582 | 3.555
4.364 | 4.057 | 3.791 | 3.65 | 3.57 | 3.465
4.479 | 3.758 | 3.316 | 3.077 | 2.959 | 2.891
dbbench command used (on a DB with 4 levels, 12 million keys)-
TEST_TMPDIR=/dev/shm numactl -C 10 ./db_bench.tmp -use_existing_db=true -benchmarks="readseq,multireadrandom" -write_buffer_size=4194304 -target_file_size_base=4194304 -max_bytes_for_level_base=16777216 -num=12000000 -reads=12000000 -duration=90 -threads=1 -compression_type=none -cache_size=4194304000 -batch_size=32 -disable_auto_compactions=true -bloom_bits=10 -cache_index_and_filter_blocks=true -pin_l0_filter_and_index_blocks_in_cache=true -multiread_batched=true -multiread_stride=4
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5011
Differential Revision: D14348703
Pulled By: anand1976
fbshipit-source-id: 774406dab3776d979c809522a67bedac6c17f84b
2019-04-11 23:24:09 +02:00
|
|
|
const ReadOptions& options, ColumnFamilyHandle* column_family,
|
2019-11-12 22:51:18 +01:00
|
|
|
ReadCallback* callback,
|
|
|
|
autovector<KeyContext*, MultiGetContext::MAX_BATCH_SIZE>* sorted_keys);
|
Introduce a new MultiGet batching implementation (#5011)
Summary:
This PR introduces a new MultiGet() API, with the underlying implementation grouping keys based on SST file and batching lookups in a file. The reason for the new API is twofold - the definition allows callers to allocate storage for status and values on stack instead of std::vector, as well as return values as PinnableSlices in order to avoid copying, and it keeps the original MultiGet() implementation intact while we experiment with batching.
Batching is useful when there is some spatial locality to the keys being queries, as well as larger batch sizes. The main benefits are due to -
1. Fewer function calls, especially to BlockBasedTableReader::MultiGet() and FullFilterBlockReader::KeysMayMatch()
2. Bloom filter cachelines can be prefetched, hiding the cache miss latency
The next step is to optimize the binary searches in the level_storage_info, index blocks and data blocks, since we could reduce the number of key comparisons if the keys are relatively close to each other. The batching optimizations also need to be extended to other formats, such as PlainTable and filter formats. This also needs to be added to db_stress.
Benchmark results from db_bench for various batch size/locality of reference combinations are given below. Locality was simulated by offsetting the keys in a batch by a stride length. Each SST file is about 8.6MB uncompressed and key/value size is 16/100 uncompressed. To focus on the cpu benefit of batching, the runs were single threaded and bound to the same cpu to eliminate interference from other system events. The results show a 10-25% improvement in micros/op from smaller to larger batch sizes (4 - 32).
Batch Sizes
1 | 2 | 4 | 8 | 16 | 32
Random pattern (Stride length 0)
4.158 | 4.109 | 4.026 | 4.05 | 4.1 | 4.074 - Get
4.438 | 4.302 | 4.165 | 4.122 | 4.096 | 4.075 - MultiGet (no batching)
4.461 | 4.256 | 4.277 | 4.11 | 4.182 | 4.14 - MultiGet (w/ batching)
Good locality (Stride length 16)
4.048 | 3.659 | 3.248 | 2.99 | 2.84 | 2.753
4.429 | 3.728 | 3.406 | 3.053 | 2.911 | 2.781
4.452 | 3.45 | 2.833 | 2.451 | 2.233 | 2.135
Good locality (Stride length 256)
4.066 | 3.786 | 3.581 | 3.447 | 3.415 | 3.232
4.406 | 4.005 | 3.644 | 3.49 | 3.381 | 3.268
4.393 | 3.649 | 3.186 | 2.882 | 2.676 | 2.62
Medium locality (Stride length 4096)
4.012 | 3.922 | 3.768 | 3.61 | 3.582 | 3.555
4.364 | 4.057 | 3.791 | 3.65 | 3.57 | 3.465
4.479 | 3.758 | 3.316 | 3.077 | 2.959 | 2.891
dbbench command used (on a DB with 4 levels, 12 million keys)-
TEST_TMPDIR=/dev/shm numactl -C 10 ./db_bench.tmp -use_existing_db=true -benchmarks="readseq,multireadrandom" -write_buffer_size=4194304 -target_file_size_base=4194304 -max_bytes_for_level_base=16777216 -num=12000000 -reads=12000000 -duration=90 -threads=1 -compression_type=none -cache_size=4194304000 -batch_size=32 -disable_auto_compactions=true -bloom_bits=10 -cache_index_and_filter_blocks=true -pin_l0_filter_and_index_blocks_in_cache=true -multiread_batched=true -multiread_stride=4
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5011
Differential Revision: D14348703
Pulled By: anand1976
fbshipit-source-id: 774406dab3776d979c809522a67bedac6c17f84b
2019-04-11 23:24:09 +02:00
|
|
|
|
2017-05-08 07:12:55 +02:00
|
|
|
virtual Status CreateColumnFamily(const ColumnFamilyOptions& cf_options,
|
2014-01-06 22:31:06 +01:00
|
|
|
const std::string& column_family,
|
2015-02-26 20:28:41 +01:00
|
|
|
ColumnFamilyHandle** handle) override;
|
2017-05-08 07:12:55 +02:00
|
|
|
virtual Status CreateColumnFamilies(
|
|
|
|
const ColumnFamilyOptions& cf_options,
|
|
|
|
const std::vector<std::string>& column_family_names,
|
|
|
|
std::vector<ColumnFamilyHandle*>* handles) override;
|
|
|
|
virtual Status CreateColumnFamilies(
|
|
|
|
const std::vector<ColumnFamilyDescriptor>& column_families,
|
|
|
|
std::vector<ColumnFamilyHandle*>* handles) override;
|
2015-02-26 20:28:41 +01:00
|
|
|
virtual Status DropColumnFamily(ColumnFamilyHandle* column_family) override;
|
2017-05-08 07:12:55 +02:00
|
|
|
virtual Status DropColumnFamilies(
|
|
|
|
const std::vector<ColumnFamilyHandle*>& column_families) override;
|
2014-01-02 18:08:12 +01:00
|
|
|
|
2013-07-26 21:57:01 +02:00
|
|
|
// Returns false if key doesn't exist in the database and true if it may.
|
|
|
|
// If value_found is not passed in as null, then return the value if found in
|
|
|
|
// memory. On return, if value was found, then value_found will be set to true
|
|
|
|
// , otherwise false.
|
[RocksDB] [Column Family] Interface proposal
Summary:
<This diff is for Column Family branch>
Sharing some of the work I've done so far. This diff compiles and passes the tests.
The biggest change is in options.h - I broke down Options into two parts - DBOptions and ColumnFamilyOptions. DBOptions is DB-specific (env, create_if_missing, block_cache, etc.) and ColumnFamilyOptions is column family-specific (all compaction options, compresion options, etc.). Note that this does not break backwards compatibility at all.
Further, I created DBWithColumnFamily which inherits DB interface and adds new functions with column family support. Clients can transparently switch to DBWithColumnFamily and it will not break their backwards compatibility.
There are few methods worth checking out: ListColumnFamilies(), MultiNewIterator(), MultiGet() and GetSnapshot(). [GetSnapshot() returns the snapshot across all column families for now - I think that's what we agreed on]
Finally, I made small changes to WriteBatch so we are able to atomically insert data across column families.
Please provide feedback.
Test Plan: make check works, the code is backward compatible
Reviewers: dhruba, haobo, sdong, kailiu, emayanke
CC: leveldb
Differential Revision: https://reviews.facebook.net/D14445
2013-12-03 20:14:09 +01:00
|
|
|
using DB::KeyMayExist;
|
2013-07-26 21:57:01 +02:00
|
|
|
virtual bool KeyMayExist(const ReadOptions& options,
|
2014-02-11 02:04:44 +01:00
|
|
|
ColumnFamilyHandle* column_family, const Slice& key,
|
2020-03-03 00:58:32 +01:00
|
|
|
std::string* value, std::string* timestamp,
|
2015-02-26 20:28:41 +01:00
|
|
|
bool* value_found = nullptr) override;
|
2017-10-03 18:08:07 +02:00
|
|
|
|
[RocksDB] [Column Family] Interface proposal
Summary:
<This diff is for Column Family branch>
Sharing some of the work I've done so far. This diff compiles and passes the tests.
The biggest change is in options.h - I broke down Options into two parts - DBOptions and ColumnFamilyOptions. DBOptions is DB-specific (env, create_if_missing, block_cache, etc.) and ColumnFamilyOptions is column family-specific (all compaction options, compresion options, etc.). Note that this does not break backwards compatibility at all.
Further, I created DBWithColumnFamily which inherits DB interface and adds new functions with column family support. Clients can transparently switch to DBWithColumnFamily and it will not break their backwards compatibility.
There are few methods worth checking out: ListColumnFamilies(), MultiNewIterator(), MultiGet() and GetSnapshot(). [GetSnapshot() returns the snapshot across all column families for now - I think that's what we agreed on]
Finally, I made small changes to WriteBatch so we are able to atomically insert data across column families.
Please provide feedback.
Test Plan: make check works, the code is backward compatible
Reviewers: dhruba, haobo, sdong, kailiu, emayanke
CC: leveldb
Differential Revision: https://reviews.facebook.net/D14445
2013-12-03 20:14:09 +01:00
|
|
|
using DB::NewIterator;
|
|
|
|
virtual Iterator* NewIterator(const ReadOptions& options,
|
2015-02-26 20:28:41 +01:00
|
|
|
ColumnFamilyHandle* column_family) override;
|
[RocksDB] [Column Family] Interface proposal
Summary:
<This diff is for Column Family branch>
Sharing some of the work I've done so far. This diff compiles and passes the tests.
The biggest change is in options.h - I broke down Options into two parts - DBOptions and ColumnFamilyOptions. DBOptions is DB-specific (env, create_if_missing, block_cache, etc.) and ColumnFamilyOptions is column family-specific (all compaction options, compresion options, etc.). Note that this does not break backwards compatibility at all.
Further, I created DBWithColumnFamily which inherits DB interface and adds new functions with column family support. Clients can transparently switch to DBWithColumnFamily and it will not break their backwards compatibility.
There are few methods worth checking out: ListColumnFamilies(), MultiNewIterator(), MultiGet() and GetSnapshot(). [GetSnapshot() returns the snapshot across all column families for now - I think that's what we agreed on]
Finally, I made small changes to WriteBatch so we are able to atomically insert data across column families.
Please provide feedback.
Test Plan: make check works, the code is backward compatible
Reviewers: dhruba, haobo, sdong, kailiu, emayanke
CC: leveldb
Differential Revision: https://reviews.facebook.net/D14445
2013-12-03 20:14:09 +01:00
|
|
|
virtual Status NewIterators(
|
|
|
|
const ReadOptions& options,
|
2014-03-08 01:12:34 +01:00
|
|
|
const std::vector<ColumnFamilyHandle*>& column_families,
|
2015-02-26 20:28:41 +01:00
|
|
|
std::vector<Iterator*>* iterators) override;
|
2017-10-03 18:08:07 +02:00
|
|
|
|
2015-02-26 20:28:41 +01:00
|
|
|
virtual const Snapshot* GetSnapshot() override;
|
|
|
|
virtual void ReleaseSnapshot(const Snapshot* snapshot) override;
|
[RocksDB] [Column Family] Interface proposal
Summary:
<This diff is for Column Family branch>
Sharing some of the work I've done so far. This diff compiles and passes the tests.
The biggest change is in options.h - I broke down Options into two parts - DBOptions and ColumnFamilyOptions. DBOptions is DB-specific (env, create_if_missing, block_cache, etc.) and ColumnFamilyOptions is column family-specific (all compaction options, compresion options, etc.). Note that this does not break backwards compatibility at all.
Further, I created DBWithColumnFamily which inherits DB interface and adds new functions with column family support. Clients can transparently switch to DBWithColumnFamily and it will not break their backwards compatibility.
There are few methods worth checking out: ListColumnFamilies(), MultiNewIterator(), MultiGet() and GetSnapshot(). [GetSnapshot() returns the snapshot across all column families for now - I think that's what we agreed on]
Finally, I made small changes to WriteBatch so we are able to atomically insert data across column families.
Please provide feedback.
Test Plan: make check works, the code is backward compatible
Reviewers: dhruba, haobo, sdong, kailiu, emayanke
CC: leveldb
Differential Revision: https://reviews.facebook.net/D14445
2013-12-03 20:14:09 +01:00
|
|
|
using DB::GetProperty;
|
2014-02-11 02:04:44 +01:00
|
|
|
virtual bool GetProperty(ColumnFamilyHandle* column_family,
|
2015-02-26 20:28:41 +01:00
|
|
|
const Slice& property, std::string* value) override;
|
2016-11-12 05:45:47 +01:00
|
|
|
using DB::GetMapProperty;
|
2017-08-31 00:20:48 +02:00
|
|
|
virtual bool GetMapProperty(
|
|
|
|
ColumnFamilyHandle* column_family, const Slice& property,
|
|
|
|
std::map<std::string, std::string>* value) override;
|
2014-07-29 00:28:53 +02:00
|
|
|
using DB::GetIntProperty;
|
|
|
|
virtual bool GetIntProperty(ColumnFamilyHandle* column_family,
|
|
|
|
const Slice& property, uint64_t* value) override;
|
2015-11-04 00:54:18 +01:00
|
|
|
using DB::GetAggregatedIntProperty;
|
|
|
|
virtual bool GetAggregatedIntProperty(const Slice& property,
|
|
|
|
uint64_t* aggregated_value) override;
|
[RocksDB] [Column Family] Interface proposal
Summary:
<This diff is for Column Family branch>
Sharing some of the work I've done so far. This diff compiles and passes the tests.
The biggest change is in options.h - I broke down Options into two parts - DBOptions and ColumnFamilyOptions. DBOptions is DB-specific (env, create_if_missing, block_cache, etc.) and ColumnFamilyOptions is column family-specific (all compaction options, compresion options, etc.). Note that this does not break backwards compatibility at all.
Further, I created DBWithColumnFamily which inherits DB interface and adds new functions with column family support. Clients can transparently switch to DBWithColumnFamily and it will not break their backwards compatibility.
There are few methods worth checking out: ListColumnFamilies(), MultiNewIterator(), MultiGet() and GetSnapshot(). [GetSnapshot() returns the snapshot across all column families for now - I think that's what we agreed on]
Finally, I made small changes to WriteBatch so we are able to atomically insert data across column families.
Please provide feedback.
Test Plan: make check works, the code is backward compatible
Reviewers: dhruba, haobo, sdong, kailiu, emayanke
CC: leveldb
Differential Revision: https://reviews.facebook.net/D14445
2013-12-03 20:14:09 +01:00
|
|
|
using DB::GetApproximateSizes;
|
2019-07-26 07:38:53 +02:00
|
|
|
virtual Status GetApproximateSizes(const SizeApproximationOptions& options,
|
|
|
|
ColumnFamilyHandle* column_family,
|
|
|
|
const Range* range, int n,
|
|
|
|
uint64_t* sizes) override;
|
2017-02-06 23:42:38 +01:00
|
|
|
using DB::GetApproximateMemTableStats;
|
|
|
|
virtual void GetApproximateMemTableStats(ColumnFamilyHandle* column_family,
|
|
|
|
const Range& range,
|
|
|
|
uint64_t* const count,
|
|
|
|
uint64_t* const size) override;
|
[RocksDB] [Column Family] Interface proposal
Summary:
<This diff is for Column Family branch>
Sharing some of the work I've done so far. This diff compiles and passes the tests.
The biggest change is in options.h - I broke down Options into two parts - DBOptions and ColumnFamilyOptions. DBOptions is DB-specific (env, create_if_missing, block_cache, etc.) and ColumnFamilyOptions is column family-specific (all compaction options, compresion options, etc.). Note that this does not break backwards compatibility at all.
Further, I created DBWithColumnFamily which inherits DB interface and adds new functions with column family support. Clients can transparently switch to DBWithColumnFamily and it will not break their backwards compatibility.
There are few methods worth checking out: ListColumnFamilies(), MultiNewIterator(), MultiGet() and GetSnapshot(). [GetSnapshot() returns the snapshot across all column families for now - I think that's what we agreed on]
Finally, I made small changes to WriteBatch so we are able to atomically insert data across column families.
Please provide feedback.
Test Plan: make check works, the code is backward compatible
Reviewers: dhruba, haobo, sdong, kailiu, emayanke
CC: leveldb
Differential Revision: https://reviews.facebook.net/D14445
2013-12-03 20:14:09 +01:00
|
|
|
using DB::CompactRange;
|
2015-06-17 23:36:14 +02:00
|
|
|
virtual Status CompactRange(const CompactRangeOptions& options,
|
|
|
|
ColumnFamilyHandle* column_family,
|
|
|
|
const Slice* begin, const Slice* end) override;
|
[RocksDB] [Column Family] Interface proposal
Summary:
<This diff is for Column Family branch>
Sharing some of the work I've done so far. This diff compiles and passes the tests.
The biggest change is in options.h - I broke down Options into two parts - DBOptions and ColumnFamilyOptions. DBOptions is DB-specific (env, create_if_missing, block_cache, etc.) and ColumnFamilyOptions is column family-specific (all compaction options, compresion options, etc.). Note that this does not break backwards compatibility at all.
Further, I created DBWithColumnFamily which inherits DB interface and adds new functions with column family support. Clients can transparently switch to DBWithColumnFamily and it will not break their backwards compatibility.
There are few methods worth checking out: ListColumnFamilies(), MultiNewIterator(), MultiGet() and GetSnapshot(). [GetSnapshot() returns the snapshot across all column families for now - I think that's what we agreed on]
Finally, I made small changes to WriteBatch so we are able to atomically insert data across column families.
Please provide feedback.
Test Plan: make check works, the code is backward compatible
Reviewers: dhruba, haobo, sdong, kailiu, emayanke
CC: leveldb
Differential Revision: https://reviews.facebook.net/D14445
2013-12-03 20:14:09 +01:00
|
|
|
|
CompactFiles, EventListener and GetDatabaseMetaData
Summary:
This diff adds three sets of APIs to RocksDB.
= GetColumnFamilyMetaData =
* This APIs allow users to obtain the current state of a RocksDB instance on one column family.
* See GetColumnFamilyMetaData in include/rocksdb/db.h
= EventListener =
* A virtual class that allows users to implement a set of
call-back functions which will be called when specific
events of a RocksDB instance happens.
* To register EventListener, simply insert an EventListener to ColumnFamilyOptions::listeners
= CompactFiles =
* CompactFiles API inputs a set of file numbers and an output level, and RocksDB
will try to compact those files into the specified level.
= Example =
* Example code can be found in example/compact_files_example.cc, which implements
a simple external compactor using EventListener, GetColumnFamilyMetaData, and
CompactFiles API.
Test Plan:
listener_test
compactor_test
example/compact_files_example
export ROCKSDB_TESTS=CompactFiles
db_test
export ROCKSDB_TESTS=MetaData
db_test
Reviewers: ljin, igor, rven, sdong
Reviewed By: sdong
Subscribers: MarkCallaghan, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D24705
2014-11-07 23:45:18 +01:00
|
|
|
using DB::CompactFiles;
|
2018-12-13 23:12:02 +01:00
|
|
|
virtual Status CompactFiles(
|
|
|
|
const CompactionOptions& compact_options,
|
|
|
|
ColumnFamilyHandle* column_family,
|
|
|
|
const std::vector<std::string>& input_file_names, const int output_level,
|
|
|
|
const int output_path_id = -1,
|
|
|
|
std::vector<std::string>* const output_file_names = nullptr,
|
|
|
|
CompactionJobInfo* compaction_job_info = nullptr) override;
|
CompactFiles, EventListener and GetDatabaseMetaData
Summary:
This diff adds three sets of APIs to RocksDB.
= GetColumnFamilyMetaData =
* This APIs allow users to obtain the current state of a RocksDB instance on one column family.
* See GetColumnFamilyMetaData in include/rocksdb/db.h
= EventListener =
* A virtual class that allows users to implement a set of
call-back functions which will be called when specific
events of a RocksDB instance happens.
* To register EventListener, simply insert an EventListener to ColumnFamilyOptions::listeners
= CompactFiles =
* CompactFiles API inputs a set of file numbers and an output level, and RocksDB
will try to compact those files into the specified level.
= Example =
* Example code can be found in example/compact_files_example.cc, which implements
a simple external compactor using EventListener, GetColumnFamilyMetaData, and
CompactFiles API.
Test Plan:
listener_test
compactor_test
example/compact_files_example
export ROCKSDB_TESTS=CompactFiles
db_test
export ROCKSDB_TESTS=MetaData
db_test
Reviewers: ljin, igor, rven, sdong
Reviewed By: sdong
Subscribers: MarkCallaghan, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D24705
2014-11-07 23:45:18 +01:00
|
|
|
|
2015-10-02 22:17:34 +02:00
|
|
|
virtual Status PauseBackgroundWork() override;
|
|
|
|
virtual Status ContinueBackgroundWork() override;
|
|
|
|
|
2015-11-20 10:07:24 +01:00
|
|
|
virtual Status EnableAutoCompaction(
|
|
|
|
const std::vector<ColumnFamilyHandle*>& column_family_handles) override;
|
|
|
|
|
2019-09-17 06:00:13 +02:00
|
|
|
virtual void EnableManualCompaction() override;
|
|
|
|
virtual void DisableManualCompaction() override;
|
|
|
|
|
2014-09-17 21:49:13 +02:00
|
|
|
using DB::SetOptions;
|
2015-02-26 20:28:41 +01:00
|
|
|
Status SetOptions(
|
|
|
|
ColumnFamilyHandle* column_family,
|
|
|
|
const std::unordered_map<std::string, std::string>& options_map) override;
|
2014-09-17 21:49:13 +02:00
|
|
|
|
2016-10-14 21:25:39 +02:00
|
|
|
virtual Status SetDBOptions(
|
|
|
|
const std::unordered_map<std::string, std::string>& options_map) override;
|
|
|
|
|
[RocksDB] [Column Family] Interface proposal
Summary:
<This diff is for Column Family branch>
Sharing some of the work I've done so far. This diff compiles and passes the tests.
The biggest change is in options.h - I broke down Options into two parts - DBOptions and ColumnFamilyOptions. DBOptions is DB-specific (env, create_if_missing, block_cache, etc.) and ColumnFamilyOptions is column family-specific (all compaction options, compresion options, etc.). Note that this does not break backwards compatibility at all.
Further, I created DBWithColumnFamily which inherits DB interface and adds new functions with column family support. Clients can transparently switch to DBWithColumnFamily and it will not break their backwards compatibility.
There are few methods worth checking out: ListColumnFamilies(), MultiNewIterator(), MultiGet() and GetSnapshot(). [GetSnapshot() returns the snapshot across all column families for now - I think that's what we agreed on]
Finally, I made small changes to WriteBatch so we are able to atomically insert data across column families.
Please provide feedback.
Test Plan: make check works, the code is backward compatible
Reviewers: dhruba, haobo, sdong, kailiu, emayanke
CC: leveldb
Differential Revision: https://reviews.facebook.net/D14445
2013-12-03 20:14:09 +01:00
|
|
|
using DB::NumberLevels;
|
2015-02-26 20:28:41 +01:00
|
|
|
virtual int NumberLevels(ColumnFamilyHandle* column_family) override;
|
[RocksDB] [Column Family] Interface proposal
Summary:
<This diff is for Column Family branch>
Sharing some of the work I've done so far. This diff compiles and passes the tests.
The biggest change is in options.h - I broke down Options into two parts - DBOptions and ColumnFamilyOptions. DBOptions is DB-specific (env, create_if_missing, block_cache, etc.) and ColumnFamilyOptions is column family-specific (all compaction options, compresion options, etc.). Note that this does not break backwards compatibility at all.
Further, I created DBWithColumnFamily which inherits DB interface and adds new functions with column family support. Clients can transparently switch to DBWithColumnFamily and it will not break their backwards compatibility.
There are few methods worth checking out: ListColumnFamilies(), MultiNewIterator(), MultiGet() and GetSnapshot(). [GetSnapshot() returns the snapshot across all column families for now - I think that's what we agreed on]
Finally, I made small changes to WriteBatch so we are able to atomically insert data across column families.
Please provide feedback.
Test Plan: make check works, the code is backward compatible
Reviewers: dhruba, haobo, sdong, kailiu, emayanke
CC: leveldb
Differential Revision: https://reviews.facebook.net/D14445
2013-12-03 20:14:09 +01:00
|
|
|
using DB::MaxMemCompactionLevel;
|
2015-02-26 20:28:41 +01:00
|
|
|
virtual int MaxMemCompactionLevel(ColumnFamilyHandle* column_family) override;
|
[RocksDB] [Column Family] Interface proposal
Summary:
<This diff is for Column Family branch>
Sharing some of the work I've done so far. This diff compiles and passes the tests.
The biggest change is in options.h - I broke down Options into two parts - DBOptions and ColumnFamilyOptions. DBOptions is DB-specific (env, create_if_missing, block_cache, etc.) and ColumnFamilyOptions is column family-specific (all compaction options, compresion options, etc.). Note that this does not break backwards compatibility at all.
Further, I created DBWithColumnFamily which inherits DB interface and adds new functions with column family support. Clients can transparently switch to DBWithColumnFamily and it will not break their backwards compatibility.
There are few methods worth checking out: ListColumnFamilies(), MultiNewIterator(), MultiGet() and GetSnapshot(). [GetSnapshot() returns the snapshot across all column families for now - I think that's what we agreed on]
Finally, I made small changes to WriteBatch so we are able to atomically insert data across column families.
Please provide feedback.
Test Plan: make check works, the code is backward compatible
Reviewers: dhruba, haobo, sdong, kailiu, emayanke
CC: leveldb
Differential Revision: https://reviews.facebook.net/D14445
2013-12-03 20:14:09 +01:00
|
|
|
using DB::Level0StopWriteTrigger;
|
2015-02-26 20:28:41 +01:00
|
|
|
virtual int Level0StopWriteTrigger(
|
|
|
|
ColumnFamilyHandle* column_family) override;
|
|
|
|
virtual const std::string& GetName() const override;
|
|
|
|
virtual Env* GetEnv() const override;
|
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
|
|
|
virtual FileSystem* GetFileSystem() const override;
|
[RocksDB] [Column Family] Interface proposal
Summary:
<This diff is for Column Family branch>
Sharing some of the work I've done so far. This diff compiles and passes the tests.
The biggest change is in options.h - I broke down Options into two parts - DBOptions and ColumnFamilyOptions. DBOptions is DB-specific (env, create_if_missing, block_cache, etc.) and ColumnFamilyOptions is column family-specific (all compaction options, compresion options, etc.). Note that this does not break backwards compatibility at all.
Further, I created DBWithColumnFamily which inherits DB interface and adds new functions with column family support. Clients can transparently switch to DBWithColumnFamily and it will not break their backwards compatibility.
There are few methods worth checking out: ListColumnFamilies(), MultiNewIterator(), MultiGet() and GetSnapshot(). [GetSnapshot() returns the snapshot across all column families for now - I think that's what we agreed on]
Finally, I made small changes to WriteBatch so we are able to atomically insert data across column families.
Please provide feedback.
Test Plan: make check works, the code is backward compatible
Reviewers: dhruba, haobo, sdong, kailiu, emayanke
CC: leveldb
Differential Revision: https://reviews.facebook.net/D14445
2013-12-03 20:14:09 +01:00
|
|
|
using DB::GetOptions;
|
2016-09-15 07:10:28 +02:00
|
|
|
virtual Options GetOptions(ColumnFamilyHandle* column_family) const override;
|
2015-05-11 23:51:51 +02:00
|
|
|
using DB::GetDBOptions;
|
2016-09-24 01:34:04 +02:00
|
|
|
virtual DBOptions GetDBOptions() const override;
|
[RocksDB] [Column Family] Interface proposal
Summary:
<This diff is for Column Family branch>
Sharing some of the work I've done so far. This diff compiles and passes the tests.
The biggest change is in options.h - I broke down Options into two parts - DBOptions and ColumnFamilyOptions. DBOptions is DB-specific (env, create_if_missing, block_cache, etc.) and ColumnFamilyOptions is column family-specific (all compaction options, compresion options, etc.). Note that this does not break backwards compatibility at all.
Further, I created DBWithColumnFamily which inherits DB interface and adds new functions with column family support. Clients can transparently switch to DBWithColumnFamily and it will not break their backwards compatibility.
There are few methods worth checking out: ListColumnFamilies(), MultiNewIterator(), MultiGet() and GetSnapshot(). [GetSnapshot() returns the snapshot across all column families for now - I think that's what we agreed on]
Finally, I made small changes to WriteBatch so we are able to atomically insert data across column families.
Please provide feedback.
Test Plan: make check works, the code is backward compatible
Reviewers: dhruba, haobo, sdong, kailiu, emayanke
CC: leveldb
Differential Revision: https://reviews.facebook.net/D14445
2013-12-03 20:14:09 +01:00
|
|
|
using DB::Flush;
|
|
|
|
virtual Status Flush(const FlushOptions& options,
|
2015-02-26 20:28:41 +01:00
|
|
|
ColumnFamilyHandle* column_family) override;
|
2018-10-27 00:06:44 +02:00
|
|
|
virtual Status Flush(
|
|
|
|
const FlushOptions& options,
|
|
|
|
const std::vector<ColumnFamilyHandle*>& column_families) override;
|
2017-06-24 23:06:43 +02:00
|
|
|
virtual Status FlushWAL(bool sync) override;
|
2019-04-06 15:36:42 +02:00
|
|
|
bool TEST_WALBufferIsEmpty(bool lock = true);
|
[wal changes 3/3] method in DB to sync WAL without blocking writers
Summary:
Subj. We really need this feature.
Previous diff D40899 has most of the changes to make this possible, this diff just adds the method.
Test Plan: `make check`, the new test fails without this diff; ran with ASAN, TSAN and valgrind.
Reviewers: igor, rven, IslamAbdelRahman, anthony, kradhakrishnan, tnovak, yhchiang, sdong
Reviewed By: sdong
Subscribers: MarkCallaghan, maykov, hermanlee4, yoshinorim, tnovak, dhruba
Differential Revision: https://reviews.facebook.net/D40905
2015-08-05 15:06:39 +02:00
|
|
|
virtual Status SyncWAL() override;
|
2019-04-06 15:36:42 +02:00
|
|
|
virtual Status LockWAL() override;
|
|
|
|
virtual Status UnlockWAL() override;
|
2014-04-15 22:39:26 +02:00
|
|
|
|
2015-02-26 20:28:41 +01:00
|
|
|
virtual SequenceNumber GetLatestSequenceNumber() const override;
|
2014-04-15 22:39:26 +02:00
|
|
|
|
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
|
|
|
virtual bool SetPreserveDeletesSequenceNumber(SequenceNumber seqnum) override;
|
|
|
|
|
2019-05-24 22:05:58 +02:00
|
|
|
virtual Status GetDbIdentity(std::string& identity) const override;
|
|
|
|
|
2019-09-03 17:50:47 +02:00
|
|
|
virtual Status GetDbIdentityFromIdentityFile(std::string* identity) const;
|
|
|
|
|
2020-06-15 19:45:03 +02:00
|
|
|
virtual Status GetDbSessionId(std::string& session_id) const override;
|
|
|
|
|
2019-05-24 22:05:58 +02:00
|
|
|
ColumnFamilyHandle* DefaultColumnFamily() const override;
|
|
|
|
|
2019-06-18 00:17:43 +02:00
|
|
|
ColumnFamilyHandle* PersistentStatsColumnFamily() const;
|
|
|
|
|
2019-05-24 22:05:58 +02:00
|
|
|
virtual Status Close() override;
|
|
|
|
|
First step towards handling MANIFEST write error (#6949)
Summary:
This PR provides preliminary support for handling IO error during MANIFEST write.
File write/sync is not guaranteed to be atomic. If we encounter an IOError while writing/syncing to the MANIFEST file, we cannot be sure about the state of the MANIFEST file. The version edits may or may not have reached the file. During cleanup, if we delete the newly-generated SST files referenced by the pending version edit(s), but the version edit(s) actually are persistent in the MANIFEST, then next recovery attempt will process the version edits(s) and then fail since the SST files have already been deleted.
One approach is to truncate the MANIFEST after write/sync error, so that it is safe to delete the SST files. However, file truncation may not be supported on certain file systems. Therefore, we take the following approach.
If an IOError is detected during MANIFEST write/sync, we disable file deletions for the faulty database. Depending on whether the IOError is retryable (set by underlying file system), either RocksDB or application can call `DB::Resume()`, or simply shutdown and restart. During `Resume()`, RocksDB will try to switch to a new MANIFEST and write all existing in-memory version storage in the new file. If this succeeds, then RocksDB may proceed. If all recovery is completed, then file deletions will be re-enabled.
Note that multiple threads can call `LogAndApply()` at the same time, though only one of them will be going through the process MANIFEST write, possibly batching the version edits of other threads. When the leading MANIFEST writer finishes, all of the MANIFEST writing threads in this batch will have the same IOError. They will all call `ErrorHandler::SetBGError()` in which file deletion will be disabled.
Possible future directions:
- Add an `ErrorContext` structure so that it is easier to pass more info to `ErrorHandler`. Currently, as in this example, a new `BackgroundErrorReason` has to be added.
Test plan (dev server):
make check
Pull Request resolved: https://github.com/facebook/rocksdb/pull/6949
Reviewed By: anand1976
Differential Revision: D22026020
Pulled By: riversand963
fbshipit-source-id: f3c68a2ef45d9b505d0d625c7c5e0c88495b91c8
2020-06-25 04:05:47 +02:00
|
|
|
virtual Status DisableFileDeletions() override;
|
|
|
|
|
|
|
|
virtual Status EnableFileDeletions(bool force) override;
|
|
|
|
|
|
|
|
virtual bool IsFileDeletionsEnabled() const;
|
|
|
|
|
2019-05-24 22:05:58 +02:00
|
|
|
Status GetStatsHistory(
|
|
|
|
uint64_t start_time, uint64_t end_time,
|
|
|
|
std::unique_ptr<StatsHistoryIterator>* stats_iterator) override;
|
|
|
|
|
2014-04-15 22:39:26 +02:00
|
|
|
#ifndef ROCKSDB_LITE
|
2017-04-19 01:30:51 +02:00
|
|
|
using DB::ResetStats;
|
|
|
|
virtual Status ResetStats() override;
|
2013-11-09 00:23:46 +01:00
|
|
|
// All the returned filenames start with "/"
|
2012-11-06 20:21:57 +01:00
|
|
|
virtual Status GetLiveFiles(std::vector<std::string>&,
|
2013-10-03 23:38:32 +02:00
|
|
|
uint64_t* manifest_file_size,
|
2015-02-26 20:28:41 +01:00
|
|
|
bool flush_memtable = true) override;
|
|
|
|
virtual Status GetSortedWalFiles(VectorLogPtr& files) override;
|
2019-09-18 01:43:07 +02:00
|
|
|
virtual Status GetCurrentWalFile(
|
|
|
|
std::unique_ptr<LogFile>* current_log_file) override;
|
2019-10-25 20:52:24 +02:00
|
|
|
virtual Status GetCreationTimeOfOldestFile(
|
|
|
|
uint64_t* creation_time) override;
|
2014-04-15 22:39:26 +02:00
|
|
|
|
2014-02-28 20:50:36 +01:00
|
|
|
virtual Status GetUpdatesSince(
|
2018-11-09 20:17:34 +01:00
|
|
|
SequenceNumber seq_number, std::unique_ptr<TransactionLogIterator>* iter,
|
|
|
|
const TransactionLogIterator::ReadOptions& read_options =
|
|
|
|
TransactionLogIterator::ReadOptions()) override;
|
2015-02-26 20:28:41 +01:00
|
|
|
virtual Status DeleteFile(std::string name) override;
|
2018-01-30 22:46:08 +01:00
|
|
|
Status DeleteFilesInRanges(ColumnFamilyHandle* column_family,
|
|
|
|
const RangePtr* ranges, size_t n,
|
|
|
|
bool include_end = true);
|
2013-08-22 23:32:53 +02:00
|
|
|
|
2015-02-26 20:28:41 +01:00
|
|
|
virtual void GetLiveFilesMetaData(
|
|
|
|
std::vector<LiveFileMetaData>* metadata) override;
|
CompactFiles, EventListener and GetDatabaseMetaData
Summary:
This diff adds three sets of APIs to RocksDB.
= GetColumnFamilyMetaData =
* This APIs allow users to obtain the current state of a RocksDB instance on one column family.
* See GetColumnFamilyMetaData in include/rocksdb/db.h
= EventListener =
* A virtual class that allows users to implement a set of
call-back functions which will be called when specific
events of a RocksDB instance happens.
* To register EventListener, simply insert an EventListener to ColumnFamilyOptions::listeners
= CompactFiles =
* CompactFiles API inputs a set of file numbers and an output level, and RocksDB
will try to compact those files into the specified level.
= Example =
* Example code can be found in example/compact_files_example.cc, which implements
a simple external compactor using EventListener, GetColumnFamilyMetaData, and
CompactFiles API.
Test Plan:
listener_test
compactor_test
example/compact_files_example
export ROCKSDB_TESTS=CompactFiles
db_test
export ROCKSDB_TESTS=MetaData
db_test
Reviewers: ljin, igor, rven, sdong
Reviewed By: sdong
Subscribers: MarkCallaghan, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D24705
2014-11-07 23:45:18 +01:00
|
|
|
|
2020-07-03 03:13:31 +02:00
|
|
|
virtual Status GetLiveFilesChecksumInfo(
|
|
|
|
FileChecksumList* checksum_list) override;
|
|
|
|
|
CompactFiles, EventListener and GetDatabaseMetaData
Summary:
This diff adds three sets of APIs to RocksDB.
= GetColumnFamilyMetaData =
* This APIs allow users to obtain the current state of a RocksDB instance on one column family.
* See GetColumnFamilyMetaData in include/rocksdb/db.h
= EventListener =
* A virtual class that allows users to implement a set of
call-back functions which will be called when specific
events of a RocksDB instance happens.
* To register EventListener, simply insert an EventListener to ColumnFamilyOptions::listeners
= CompactFiles =
* CompactFiles API inputs a set of file numbers and an output level, and RocksDB
will try to compact those files into the specified level.
= Example =
* Example code can be found in example/compact_files_example.cc, which implements
a simple external compactor using EventListener, GetColumnFamilyMetaData, and
CompactFiles API.
Test Plan:
listener_test
compactor_test
example/compact_files_example
export ROCKSDB_TESTS=CompactFiles
db_test
export ROCKSDB_TESTS=MetaData
db_test
Reviewers: ljin, igor, rven, sdong
Reviewed By: sdong
Subscribers: MarkCallaghan, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D24705
2014-11-07 23:45:18 +01:00
|
|
|
// Obtains the meta data of the specified column family of the DB.
|
|
|
|
// Status::NotFound() will be returned if the current DB does not have
|
|
|
|
// any column family match the specified name.
|
|
|
|
// TODO(yhchiang): output parameter is placed in the end in this codebase.
|
2019-03-28 00:13:08 +01:00
|
|
|
virtual void GetColumnFamilyMetaData(ColumnFamilyHandle* column_family,
|
|
|
|
ColumnFamilyMetaData* metadata) override;
|
CompactFiles, EventListener and GetDatabaseMetaData
Summary:
This diff adds three sets of APIs to RocksDB.
= GetColumnFamilyMetaData =
* This APIs allow users to obtain the current state of a RocksDB instance on one column family.
* See GetColumnFamilyMetaData in include/rocksdb/db.h
= EventListener =
* A virtual class that allows users to implement a set of
call-back functions which will be called when specific
events of a RocksDB instance happens.
* To register EventListener, simply insert an EventListener to ColumnFamilyOptions::listeners
= CompactFiles =
* CompactFiles API inputs a set of file numbers and an output level, and RocksDB
will try to compact those files into the specified level.
= Example =
* Example code can be found in example/compact_files_example.cc, which implements
a simple external compactor using EventListener, GetColumnFamilyMetaData, and
CompactFiles API.
Test Plan:
listener_test
compactor_test
example/compact_files_example
export ROCKSDB_TESTS=CompactFiles
db_test
export ROCKSDB_TESTS=MetaData
db_test
Reviewers: ljin, igor, rven, sdong
Reviewed By: sdong
Subscribers: MarkCallaghan, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D24705
2014-11-07 23:45:18 +01:00
|
|
|
|
Add experimental API MarkForCompaction()
Summary:
Some Mongo+Rocks datasets in Parse's environment are not doing compactions very frequently. During the quiet period (with no IO), we'd like to schedule compactions so that our reads become faster. Also, aggressively compacting during quiet periods helps when write bursts happen. In addition, we also want to compact files that are containing deleted key ranges (like old oplog keys).
All of this is currently not possible with CompactRange() because it's single-threaded and blocks all other compactions from happening. Running CompactRange() risks an issue of blocking writes because we generate too much Level 0 files before the compaction is over. Stopping writes is very dangerous because they hold transaction locks. We tried running manual compaction once on Mongo+Rocks and everything fell apart.
MarkForCompaction() solves all of those problems. This is very light-weight manual compaction. It is lower priority than automatic compactions, which means it shouldn't interfere with background process keeping the LSM tree clean. However, if no automatic compactions need to be run (or we have extra background threads available), we will start compacting files that are marked for compaction.
Test Plan: added a new unit test
Reviewers: yhchiang, rven, MarkCallaghan, sdong
Reviewed By: sdong
Subscribers: yoshinorim, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D37083
2015-04-18 01:44:45 +02:00
|
|
|
Status SuggestCompactRange(ColumnFamilyHandle* column_family,
|
2017-07-29 01:23:50 +02:00
|
|
|
const Slice* begin, const Slice* end) override;
|
Add experimental API MarkForCompaction()
Summary:
Some Mongo+Rocks datasets in Parse's environment are not doing compactions very frequently. During the quiet period (with no IO), we'd like to schedule compactions so that our reads become faster. Also, aggressively compacting during quiet periods helps when write bursts happen. In addition, we also want to compact files that are containing deleted key ranges (like old oplog keys).
All of this is currently not possible with CompactRange() because it's single-threaded and blocks all other compactions from happening. Running CompactRange() risks an issue of blocking writes because we generate too much Level 0 files before the compaction is over. Stopping writes is very dangerous because they hold transaction locks. We tried running manual compaction once on Mongo+Rocks and everything fell apart.
MarkForCompaction() solves all of those problems. This is very light-weight manual compaction. It is lower priority than automatic compactions, which means it shouldn't interfere with background process keeping the LSM tree clean. However, if no automatic compactions need to be run (or we have extra background threads available), we will start compacting files that are marked for compaction.
Test Plan: added a new unit test
Reviewers: yhchiang, rven, MarkCallaghan, sdong
Reviewed By: sdong
Subscribers: yoshinorim, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D37083
2015-04-18 01:44:45 +02:00
|
|
|
|
2017-07-29 01:23:50 +02:00
|
|
|
Status PromoteL0(ColumnFamilyHandle* column_family,
|
|
|
|
int target_level) override;
|
Implement DB::PromoteL0 method
Summary:
This diff implements a new `DB` method `PromoteL0` which moves all files in L0
to a given level skipping compaction, provided that the files have disjoint
ranges and all levels up to the target level are empty.
This method provides finer-grain control for trivial compactions, and it is
useful for bulk-loading pre-sorted keys. Compared to D34797, it does not change
the semantics of an existing operation, which can impact existing code.
PromoteL0 is designed to work well in combination with the proposed
`GetSstFileWriter`/`AddFile` interface, enabling to "design" the level structure
by populating one level at a time. Such fine-grained control can be very useful
for static or mostly-static databases.
Test Plan: `make check`
Reviewers: IslamAbdelRahman, philipp, MarkCallaghan, yhchiang, igor, sdong
Reviewed By: sdong
Subscribers: dhruba
Differential Revision: https://reviews.facebook.net/D37107
2015-04-23 21:10:36 +02:00
|
|
|
|
2019-05-24 22:05:58 +02:00
|
|
|
using DB::IngestExternalFile;
|
|
|
|
virtual Status IngestExternalFile(
|
|
|
|
ColumnFamilyHandle* column_family,
|
|
|
|
const std::vector<std::string>& external_files,
|
|
|
|
const IngestExternalFileOptions& ingestion_options) override;
|
|
|
|
|
|
|
|
using DB::IngestExternalFiles;
|
|
|
|
virtual Status IngestExternalFiles(
|
|
|
|
const std::vector<IngestExternalFileArg>& args) override;
|
|
|
|
|
Export Import sst files (#5495)
Summary:
Refresh of the earlier change here - https://github.com/facebook/rocksdb/issues/5135
This is a review request for code change needed for - https://github.com/facebook/rocksdb/issues/3469
"Add support for taking snapshot of a column family and creating column family from a given CF snapshot"
We have an implementation for this that we have been testing internally. We have two new APIs that together provide this functionality.
(1) ExportColumnFamily() - This API is modelled after CreateCheckpoint() as below.
// Exports all live SST files of a specified Column Family onto export_dir,
// returning SST files information in metadata.
// - SST files will be created as hard links when the directory specified
// is in the same partition as the db directory, copied otherwise.
// - export_dir should not already exist and will be created by this API.
// - Always triggers a flush.
virtual Status ExportColumnFamily(ColumnFamilyHandle* handle,
const std::string& export_dir,
ExportImportFilesMetaData** metadata);
Internally, the API will DisableFileDeletions(), GetColumnFamilyMetaData(), Parse through
metadata, creating links/copies of all the sst files, EnableFileDeletions() and complete the call by
returning the list of file metadata.
(2) CreateColumnFamilyWithImport() - This API is modeled after IngestExternalFile(), but invoked only during a CF creation as below.
// CreateColumnFamilyWithImport() will create a new column family with
// column_family_name and import external SST files specified in metadata into
// this column family.
// (1) External SST files can be created using SstFileWriter.
// (2) External SST files can be exported from a particular column family in
// an existing DB.
// Option in import_options specifies whether the external files are copied or
// moved (default is copy). When option specifies copy, managing files at
// external_file_path is caller's responsibility. When option specifies a
// move, the call ensures that the specified files at external_file_path are
// deleted on successful return and files are not modified on any error
// return.
// On error return, column family handle returned will be nullptr.
// ColumnFamily will be present on successful return and will not be present
// on error return. ColumnFamily may be present on any crash during this call.
virtual Status CreateColumnFamilyWithImport(
const ColumnFamilyOptions& options, const std::string& column_family_name,
const ImportColumnFamilyOptions& import_options,
const ExportImportFilesMetaData& metadata,
ColumnFamilyHandle** handle);
Internally, this API creates a new CF, parses all the sst files and adds it to the specified column family, at the same level and with same sequence number as in the metadata. Also performs safety checks with respect to overlaps between the sst files being imported.
If incoming sequence number is higher than current local sequence number, local sequence
number is updated to reflect this.
Note, as the sst files is are being moved across Column Families, Column Family name in sst file
will no longer match the actual column family on destination DB. The API does not modify Column
Family name or id in the sst files being imported.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5495
Differential Revision: D16018881
fbshipit-source-id: 9ae2251025d5916d35a9fc4ea4d6707f6be16ff9
2019-07-17 21:22:21 +02:00
|
|
|
using DB::CreateColumnFamilyWithImport;
|
|
|
|
virtual Status CreateColumnFamilyWithImport(
|
|
|
|
const ColumnFamilyOptions& options, const std::string& column_family_name,
|
|
|
|
const ImportColumnFamilyOptions& import_options,
|
|
|
|
const ExportImportFilesMetaData& metadata,
|
|
|
|
ColumnFamilyHandle** handle) override;
|
|
|
|
|
2020-11-04 05:33:45 +01:00
|
|
|
using DB::VerifyFileChecksums;
|
|
|
|
Status VerifyFileChecksums(const ReadOptions& read_options) override;
|
|
|
|
|
2019-08-17 01:40:09 +02:00
|
|
|
using DB::VerifyChecksum;
|
|
|
|
virtual Status VerifyChecksum(const ReadOptions& /*read_options*/) override;
|
2020-11-04 05:33:45 +01:00
|
|
|
// Verify the checksums of files in db. Currently only tables are checked.
|
|
|
|
//
|
|
|
|
// read_options: controls file I/O behavior, e.g. read ahead size while
|
|
|
|
// reading all the live table files.
|
|
|
|
//
|
|
|
|
// use_file_checksum: if false, verify the block checksums of all live table
|
|
|
|
// in db. Otherwise, obtain the file checksums and compare
|
|
|
|
// with the MANIFEST. Currently, file checksums are
|
|
|
|
// recomputed by reading all table files.
|
|
|
|
//
|
|
|
|
// Returns: OK if there is no file whose file or block checksum mismatches.
|
|
|
|
Status VerifyChecksumInternal(const ReadOptions& read_options,
|
|
|
|
bool use_file_checksum);
|
|
|
|
|
2021-02-23 07:07:59 +01:00
|
|
|
Status VerifyFullFileChecksum(const std::string& file_checksum_expected,
|
|
|
|
const std::string& func_name_expected,
|
|
|
|
const std::string& fpath,
|
|
|
|
const ReadOptions& read_options);
|
2019-05-24 22:05:58 +02:00
|
|
|
|
|
|
|
using DB::StartTrace;
|
|
|
|
virtual Status StartTrace(
|
|
|
|
const TraceOptions& options,
|
|
|
|
std::unique_ptr<TraceWriter>&& trace_writer) override;
|
|
|
|
|
|
|
|
using DB::EndTrace;
|
|
|
|
virtual Status EndTrace() override;
|
|
|
|
|
2019-06-14 00:39:52 +02:00
|
|
|
using DB::StartBlockCacheTrace;
|
|
|
|
Status StartBlockCacheTrace(
|
|
|
|
const TraceOptions& options,
|
|
|
|
std::unique_ptr<TraceWriter>&& trace_writer) override;
|
|
|
|
|
|
|
|
using DB::EndBlockCacheTrace;
|
|
|
|
Status EndBlockCacheTrace() override;
|
|
|
|
|
2020-08-05 03:40:19 +02:00
|
|
|
using DB::StartIOTrace;
|
|
|
|
Status StartIOTrace(Env* env, const TraceOptions& options,
|
|
|
|
std::unique_ptr<TraceWriter>&& trace_writer) override;
|
|
|
|
|
|
|
|
using DB::EndIOTrace;
|
|
|
|
Status EndIOTrace() override;
|
|
|
|
|
2019-05-24 22:05:58 +02:00
|
|
|
using DB::GetPropertiesOfAllTables;
|
|
|
|
virtual Status GetPropertiesOfAllTables(
|
|
|
|
ColumnFamilyHandle* column_family,
|
|
|
|
TablePropertiesCollection* props) override;
|
|
|
|
virtual Status GetPropertiesOfTablesInRange(
|
|
|
|
ColumnFamilyHandle* column_family, const Range* range, std::size_t n,
|
|
|
|
TablePropertiesCollection* props) override;
|
|
|
|
|
|
|
|
#endif // ROCKSDB_LITE
|
2019-05-31 06:29:44 +02:00
|
|
|
|
2019-05-24 22:05:58 +02:00
|
|
|
// ---- End of implementations of the DB interface ----
|
|
|
|
|
New API to get all merge operands for a Key (#5604)
Summary:
This is a new API added to db.h to allow for fetching all merge operands associated with a Key. The main motivation for this API is to support use cases where doing a full online merge is not necessary as it is performance sensitive. Example use-cases:
1. Update subset of columns and read subset of columns -
Imagine a SQL Table, a row is encoded as a K/V pair (as it is done in MyRocks). If there are many columns and users only updated one of them, we can use merge operator to reduce write amplification. While users only read one or two columns in the read query, this feature can avoid a full merging of the whole row, and save some CPU.
2. Updating very few attributes in a value which is a JSON-like document -
Updating one attribute can be done efficiently using merge operator, while reading back one attribute can be done more efficiently if we don't need to do a full merge.
----------------------------------------------------------------------------------------------------
API :
Status GetMergeOperands(
const ReadOptions& options, ColumnFamilyHandle* column_family,
const Slice& key, PinnableSlice* merge_operands,
GetMergeOperandsOptions* get_merge_operands_options,
int* number_of_operands)
Example usage :
int size = 100;
int number_of_operands = 0;
std::vector<PinnableSlice> values(size);
GetMergeOperandsOptions merge_operands_info;
db_->GetMergeOperands(ReadOptions(), db_->DefaultColumnFamily(), "k1", values.data(), merge_operands_info, &number_of_operands);
Description :
Returns all the merge operands corresponding to the key. If the number of merge operands in DB is greater than merge_operands_options.expected_max_number_of_operands no merge operands are returned and status is Incomplete. Merge operands returned are in the order of insertion.
merge_operands-> Points to an array of at-least merge_operands_options.expected_max_number_of_operands and the caller is responsible for allocating it. If the status returned is Incomplete then number_of_operands will contain the total number of merge operands found in DB for key.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5604
Test Plan:
Added unit test and perf test in db_bench that can be run using the command:
./db_bench -benchmarks=getmergeoperands --merge_operator=sortlist
Differential Revision: D16657366
Pulled By: vjnadimpalli
fbshipit-source-id: 0faadd752351745224ee12d4ae9ef3cb529951bf
2019-08-06 23:22:34 +02:00
|
|
|
struct GetImplOptions {
|
|
|
|
ColumnFamilyHandle* column_family = nullptr;
|
|
|
|
PinnableSlice* value = nullptr;
|
2020-03-03 00:58:32 +01:00
|
|
|
std::string* timestamp = nullptr;
|
New API to get all merge operands for a Key (#5604)
Summary:
This is a new API added to db.h to allow for fetching all merge operands associated with a Key. The main motivation for this API is to support use cases where doing a full online merge is not necessary as it is performance sensitive. Example use-cases:
1. Update subset of columns and read subset of columns -
Imagine a SQL Table, a row is encoded as a K/V pair (as it is done in MyRocks). If there are many columns and users only updated one of them, we can use merge operator to reduce write amplification. While users only read one or two columns in the read query, this feature can avoid a full merging of the whole row, and save some CPU.
2. Updating very few attributes in a value which is a JSON-like document -
Updating one attribute can be done efficiently using merge operator, while reading back one attribute can be done more efficiently if we don't need to do a full merge.
----------------------------------------------------------------------------------------------------
API :
Status GetMergeOperands(
const ReadOptions& options, ColumnFamilyHandle* column_family,
const Slice& key, PinnableSlice* merge_operands,
GetMergeOperandsOptions* get_merge_operands_options,
int* number_of_operands)
Example usage :
int size = 100;
int number_of_operands = 0;
std::vector<PinnableSlice> values(size);
GetMergeOperandsOptions merge_operands_info;
db_->GetMergeOperands(ReadOptions(), db_->DefaultColumnFamily(), "k1", values.data(), merge_operands_info, &number_of_operands);
Description :
Returns all the merge operands corresponding to the key. If the number of merge operands in DB is greater than merge_operands_options.expected_max_number_of_operands no merge operands are returned and status is Incomplete. Merge operands returned are in the order of insertion.
merge_operands-> Points to an array of at-least merge_operands_options.expected_max_number_of_operands and the caller is responsible for allocating it. If the status returned is Incomplete then number_of_operands will contain the total number of merge operands found in DB for key.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5604
Test Plan:
Added unit test and perf test in db_bench that can be run using the command:
./db_bench -benchmarks=getmergeoperands --merge_operator=sortlist
Differential Revision: D16657366
Pulled By: vjnadimpalli
fbshipit-source-id: 0faadd752351745224ee12d4ae9ef3cb529951bf
2019-08-06 23:22:34 +02:00
|
|
|
bool* value_found = nullptr;
|
|
|
|
ReadCallback* callback = nullptr;
|
|
|
|
bool* is_blob_index = nullptr;
|
|
|
|
// If true return value associated with key via value pointer else return
|
|
|
|
// all merge operands for key via merge_operands pointer
|
|
|
|
bool get_value = true;
|
|
|
|
// Pointer to an array of size
|
|
|
|
// get_merge_operands_options.expected_max_number_of_operands allocated by
|
|
|
|
// user
|
|
|
|
PinnableSlice* merge_operands = nullptr;
|
|
|
|
GetMergeOperandsOptions* get_merge_operands_options = nullptr;
|
|
|
|
int* number_of_operands = nullptr;
|
|
|
|
};
|
|
|
|
|
2019-05-24 22:05:58 +02:00
|
|
|
// Function that Get and KeyMayExist call with no_io true or false
|
|
|
|
// Note: 'value_found' from KeyMayExist propagates here
|
New API to get all merge operands for a Key (#5604)
Summary:
This is a new API added to db.h to allow for fetching all merge operands associated with a Key. The main motivation for this API is to support use cases where doing a full online merge is not necessary as it is performance sensitive. Example use-cases:
1. Update subset of columns and read subset of columns -
Imagine a SQL Table, a row is encoded as a K/V pair (as it is done in MyRocks). If there are many columns and users only updated one of them, we can use merge operator to reduce write amplification. While users only read one or two columns in the read query, this feature can avoid a full merging of the whole row, and save some CPU.
2. Updating very few attributes in a value which is a JSON-like document -
Updating one attribute can be done efficiently using merge operator, while reading back one attribute can be done more efficiently if we don't need to do a full merge.
----------------------------------------------------------------------------------------------------
API :
Status GetMergeOperands(
const ReadOptions& options, ColumnFamilyHandle* column_family,
const Slice& key, PinnableSlice* merge_operands,
GetMergeOperandsOptions* get_merge_operands_options,
int* number_of_operands)
Example usage :
int size = 100;
int number_of_operands = 0;
std::vector<PinnableSlice> values(size);
GetMergeOperandsOptions merge_operands_info;
db_->GetMergeOperands(ReadOptions(), db_->DefaultColumnFamily(), "k1", values.data(), merge_operands_info, &number_of_operands);
Description :
Returns all the merge operands corresponding to the key. If the number of merge operands in DB is greater than merge_operands_options.expected_max_number_of_operands no merge operands are returned and status is Incomplete. Merge operands returned are in the order of insertion.
merge_operands-> Points to an array of at-least merge_operands_options.expected_max_number_of_operands and the caller is responsible for allocating it. If the status returned is Incomplete then number_of_operands will contain the total number of merge operands found in DB for key.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5604
Test Plan:
Added unit test and perf test in db_bench that can be run using the command:
./db_bench -benchmarks=getmergeoperands --merge_operator=sortlist
Differential Revision: D16657366
Pulled By: vjnadimpalli
fbshipit-source-id: 0faadd752351745224ee12d4ae9ef3cb529951bf
2019-08-06 23:22:34 +02:00
|
|
|
// This function is also called by GetMergeOperands
|
|
|
|
// If get_impl_options.get_value = true get value associated with
|
|
|
|
// get_impl_options.key via get_impl_options.value
|
|
|
|
// If get_impl_options.get_value = false get merge operands associated with
|
|
|
|
// get_impl_options.key via get_impl_options.merge_operands
|
|
|
|
Status GetImpl(const ReadOptions& options, const Slice& key,
|
2020-03-03 00:58:32 +01:00
|
|
|
GetImplOptions& get_impl_options);
|
2019-05-24 22:05:58 +02:00
|
|
|
|
2020-06-18 19:15:16 +02:00
|
|
|
// If `snapshot` == kMaxSequenceNumber, set a recent one inside the file.
|
2019-05-24 22:05:58 +02:00
|
|
|
ArenaWrappedDBIter* NewIteratorImpl(const ReadOptions& options,
|
|
|
|
ColumnFamilyData* cfd,
|
|
|
|
SequenceNumber snapshot,
|
|
|
|
ReadCallback* read_callback,
|
2020-12-05 06:28:26 +01:00
|
|
|
bool expose_blob_index = false,
|
2019-05-24 22:05:58 +02:00
|
|
|
bool allow_refresh = true);
|
|
|
|
|
|
|
|
virtual SequenceNumber GetLastPublishedSequence() const {
|
|
|
|
if (last_seq_same_as_publish_seq_) {
|
|
|
|
return versions_->LastSequence();
|
|
|
|
} else {
|
|
|
|
return versions_->LastPublishedSequence();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// REQUIRES: joined the main write queue if two_write_queues is disabled, and
|
|
|
|
// the second write queue otherwise.
|
|
|
|
virtual void SetLastPublishedSequence(SequenceNumber seq);
|
|
|
|
// Returns LastSequence in last_seq_same_as_publish_seq_
|
|
|
|
// mode and LastAllocatedSequence otherwise. This is useful when visiblility
|
|
|
|
// depends also on data written to the WAL but not to the memtable.
|
|
|
|
SequenceNumber TEST_GetLastVisibleSequence() const;
|
|
|
|
|
2019-05-31 06:29:44 +02:00
|
|
|
#ifndef ROCKSDB_LITE
|
2015-05-29 23:36:35 +02:00
|
|
|
// Similar to Write() but will call the callback once on the single write
|
|
|
|
// thread to determine whether it is safe to perform the write.
|
|
|
|
virtual Status WriteWithCallback(const WriteOptions& write_options,
|
|
|
|
WriteBatch* my_batch,
|
|
|
|
WriteCallback* callback);
|
|
|
|
|
|
|
|
// Returns the sequence number that is guaranteed to be smaller than or equal
|
|
|
|
// to the sequence number of any key that could be inserted into the current
|
|
|
|
// memtables. It can then be assumed that any write with a larger(or equal)
|
|
|
|
// sequence number will be present in this memtable or a later memtable.
|
|
|
|
//
|
|
|
|
// If the earliest sequence number could not be determined,
|
|
|
|
// kMaxSequenceNumber will be returned.
|
|
|
|
//
|
|
|
|
// If include_history=true, will also search Memtables in MemTableList
|
|
|
|
// History.
|
|
|
|
SequenceNumber GetEarliestMemTableSequenceNumber(SuperVersion* sv,
|
|
|
|
bool include_history);
|
|
|
|
|
|
|
|
// For a given key, check to see if there are any records for this key
|
Use SST files for Transaction conflict detection
Summary:
Currently, transactions can fail even if there is no actual write conflict. This is due to relying on only the memtables to check for write-conflicts. Users have to tune memtable settings to try to avoid this, but it's hard to figure out exactly how to tune these settings.
With this diff, TransactionDB will use both memtables and SST files to determine if there are any write conflicts. This relies on the fact that BlockBasedTable stores sequence numbers for all writes that happen after any open snapshot. Also, D50295 is needed to prevent SingleDelete from disappearing writes (the TODOs in this test code will be fixed once the other diff is approved and merged).
Note that Optimistic transactions will still rely on tuning memtable settings as we do not want to read from SST while on the write thread. Also, memtable settings can still be used to reduce how often TransactionDB needs to read SST files.
Test Plan: unit tests, db bench
Reviewers: rven, yhchiang, kradhakrishnan, IslamAbdelRahman, sdong
Reviewed By: sdong
Subscribers: dhruba, leveldb, yoshinorim
Differential Revision: https://reviews.facebook.net/D50475
2015-10-16 01:37:15 +02:00
|
|
|
// in the memtables, including memtable history. If cache_only is false,
|
|
|
|
// SST files will also be checked.
|
|
|
|
//
|
|
|
|
// If a key is found, *found_record_for_key will be set to true and
|
2016-04-28 11:30:44 +02:00
|
|
|
// *seq will be set to the stored sequence number for the latest
|
Use SST files for Transaction conflict detection
Summary:
Currently, transactions can fail even if there is no actual write conflict. This is due to relying on only the memtables to check for write-conflicts. Users have to tune memtable settings to try to avoid this, but it's hard to figure out exactly how to tune these settings.
With this diff, TransactionDB will use both memtables and SST files to determine if there are any write conflicts. This relies on the fact that BlockBasedTable stores sequence numbers for all writes that happen after any open snapshot. Also, D50295 is needed to prevent SingleDelete from disappearing writes (the TODOs in this test code will be fixed once the other diff is approved and merged).
Note that Optimistic transactions will still rely on tuning memtable settings as we do not want to read from SST while on the write thread. Also, memtable settings can still be used to reduce how often TransactionDB needs to read SST files.
Test Plan: unit tests, db bench
Reviewers: rven, yhchiang, kradhakrishnan, IslamAbdelRahman, sdong
Reviewed By: sdong
Subscribers: dhruba, leveldb, yoshinorim
Differential Revision: https://reviews.facebook.net/D50475
2015-10-16 01:37:15 +02:00
|
|
|
// operation on this key or kMaxSequenceNumber if unknown.
|
|
|
|
// If no key is found, *found_record_for_key will be set to false.
|
|
|
|
//
|
|
|
|
// Note: If cache_only=false, it is possible for *seq to be set to 0 if
|
|
|
|
// the sequence number has been cleared from the record. If the caller is
|
|
|
|
// holding an active db snapshot, we know the missing sequence must be less
|
|
|
|
// than the snapshot's sequence number (sequence numbers are only cleared
|
|
|
|
// when there are no earlier active snapshots).
|
|
|
|
//
|
|
|
|
// If NotFound is returned and found_record_for_key is set to false, then no
|
|
|
|
// record for this key was found. If the caller is holding an active db
|
|
|
|
// snapshot, we know that no key could have existing after this snapshot
|
|
|
|
// (since we do not compact keys that have an earlier snapshot).
|
|
|
|
//
|
2019-06-11 20:42:19 +02:00
|
|
|
// Only records newer than or at `lower_bound_seq` are guaranteed to be
|
|
|
|
// returned. Memtables and files may not be checked if it only contains data
|
|
|
|
// older than `lower_bound_seq`.
|
|
|
|
//
|
Use SST files for Transaction conflict detection
Summary:
Currently, transactions can fail even if there is no actual write conflict. This is due to relying on only the memtables to check for write-conflicts. Users have to tune memtable settings to try to avoid this, but it's hard to figure out exactly how to tune these settings.
With this diff, TransactionDB will use both memtables and SST files to determine if there are any write conflicts. This relies on the fact that BlockBasedTable stores sequence numbers for all writes that happen after any open snapshot. Also, D50295 is needed to prevent SingleDelete from disappearing writes (the TODOs in this test code will be fixed once the other diff is approved and merged).
Note that Optimistic transactions will still rely on tuning memtable settings as we do not want to read from SST while on the write thread. Also, memtable settings can still be used to reduce how often TransactionDB needs to read SST files.
Test Plan: unit tests, db bench
Reviewers: rven, yhchiang, kradhakrishnan, IslamAbdelRahman, sdong
Reviewed By: sdong
Subscribers: dhruba, leveldb, yoshinorim
Differential Revision: https://reviews.facebook.net/D50475
2015-10-16 01:37:15 +02:00
|
|
|
// Returns OK or NotFound on success,
|
|
|
|
// other status on unexpected error.
|
2016-08-20 00:10:31 +02:00
|
|
|
// TODO(andrewkr): this API need to be aware of range deletion operations
|
Use SST files for Transaction conflict detection
Summary:
Currently, transactions can fail even if there is no actual write conflict. This is due to relying on only the memtables to check for write-conflicts. Users have to tune memtable settings to try to avoid this, but it's hard to figure out exactly how to tune these settings.
With this diff, TransactionDB will use both memtables and SST files to determine if there are any write conflicts. This relies on the fact that BlockBasedTable stores sequence numbers for all writes that happen after any open snapshot. Also, D50295 is needed to prevent SingleDelete from disappearing writes (the TODOs in this test code will be fixed once the other diff is approved and merged).
Note that Optimistic transactions will still rely on tuning memtable settings as we do not want to read from SST while on the write thread. Also, memtable settings can still be used to reduce how often TransactionDB needs to read SST files.
Test Plan: unit tests, db bench
Reviewers: rven, yhchiang, kradhakrishnan, IslamAbdelRahman, sdong
Reviewed By: sdong
Subscribers: dhruba, leveldb, yoshinorim
Differential Revision: https://reviews.facebook.net/D50475
2015-10-16 01:37:15 +02:00
|
|
|
Status GetLatestSequenceForKey(SuperVersion* sv, const Slice& key,
|
2019-06-11 20:42:19 +02:00
|
|
|
bool cache_only,
|
|
|
|
SequenceNumber lower_bound_seq,
|
|
|
|
SequenceNumber* seq,
|
2017-10-18 02:24:25 +02:00
|
|
|
bool* found_record_for_key,
|
|
|
|
bool* is_blob_index = nullptr);
|
2015-05-29 23:36:35 +02:00
|
|
|
|
2021-02-19 08:03:49 +01:00
|
|
|
Status TraceIteratorSeek(const uint32_t& cf_id, const Slice& key,
|
|
|
|
const Slice& lower_bound, const Slice upper_bound);
|
|
|
|
Status TraceIteratorSeekForPrev(const uint32_t& cf_id, const Slice& key,
|
|
|
|
const Slice& lower_bound,
|
|
|
|
const Slice upper_bound);
|
2014-04-15 22:39:26 +02:00
|
|
|
#endif // ROCKSDB_LITE
|
2012-11-26 22:56:45 +01:00
|
|
|
|
2015-12-08 21:25:48 +01:00
|
|
|
// Similar to GetSnapshot(), but also lets the db know that this snapshot
|
|
|
|
// will be used for transaction write-conflict checking. The DB can then
|
|
|
|
// make sure not to compact any keys that would prevent a write-conflict from
|
|
|
|
// being detected.
|
|
|
|
const Snapshot* GetSnapshotForWriteConflictBoundary();
|
|
|
|
|
2014-03-20 22:18:29 +01:00
|
|
|
// checks if all live files exist on file system and that their file sizes
|
|
|
|
// match to our in-memory records
|
|
|
|
virtual Status CheckConsistency();
|
|
|
|
|
2019-04-17 08:29:32 +02:00
|
|
|
// max_file_num_to_ignore allows bottom level compaction to filter out newly
|
|
|
|
// compacted SST files. Setting max_file_num_to_ignore to kMaxUint64 will
|
|
|
|
// disable the filtering
|
2014-02-01 01:45:20 +01:00
|
|
|
Status RunManualCompaction(ColumnFamilyData* cfd, int input_level,
|
2019-04-17 08:29:32 +02:00
|
|
|
int output_level,
|
|
|
|
const CompactRangeOptions& compact_range_options,
|
|
|
|
const Slice* begin, const Slice* end,
|
|
|
|
bool exclusive, bool disallow_trivial_move,
|
|
|
|
uint64_t max_file_num_to_ignore);
|
2014-01-15 01:19:09 +01:00
|
|
|
|
2015-10-13 02:22:37 +02:00
|
|
|
// Return an internal iterator over the current state of the database.
|
|
|
|
// The keys of this iterator are internal keys (see format.h).
|
|
|
|
// The returned iterator should be deleted when no longer needed.
|
Properly report IO errors when IndexType::kBinarySearchWithFirstKey is used (#6621)
Summary:
Context: Index type `kBinarySearchWithFirstKey` added the ability for sst file iterator to sometimes report a key from index without reading the corresponding data block. This is useful when sst blocks are cut at some meaningful boundaries (e.g. one block per key prefix), and many seeks land between blocks (e.g. for each prefix, the ranges of keys in different sst files are nearly disjoint, so a typical seek needs to read a data block from only one file even if all files have the prefix). But this added a new error condition, which rocksdb code was really not equipped to deal with: `InternalIterator::value()` may fail with an IO error or Status::Incomplete, but it's just a method returning a Slice, with no way to report error instead. Before this PR, this type of error wasn't handled at all (an empty slice was returned), and kBinarySearchWithFirstKey implementation was considered a prototype.
Now that we (LogDevice) have experimented with kBinarySearchWithFirstKey for a while and confirmed that it's really useful, this PR is adding the missing error handling.
It's a pretty inconvenient situation implementation-wise. The error needs to be reported from InternalIterator when trying to access value. But there are ~700 call sites of `InternalIterator::value()`, most of which either can't hit the error condition (because the iterator is reading from memtable or from index or something) or wouldn't benefit from the deferred loading of the value (e.g. compaction iterator that reads all values anyway). Adding error handling to all these call sites would needlessly bloat the code. So instead I made the deferred value loading optional: only the call sites that may use deferred loading have to call the new method `PrepareValue()` before calling `value()`. The feature is enabled with a new bool argument `allow_unprepared_value` to a bunch of methods that create iterators (it wouldn't make sense to put it in ReadOptions because it's completely internal to iterators, with virtually no user-visible effect). Lmk if you have better ideas.
Note that the deferred value loading only happens for *internal* iterators. The user-visible iterator (DBIter) always prepares the value before returning from Seek/Next/etc. We could go further and add an API to defer that value loading too, but that's most likely not useful for LogDevice, so it doesn't seem worth the complexity for now.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/6621
Test Plan: make -j5 check . Will also deploy to some logdevice test clusters and look at stats.
Reviewed By: siying
Differential Revision: D20786930
Pulled By: al13n321
fbshipit-source-id: 6da77d918bad3780522e918f17f4d5513d3e99ee
2020-04-16 02:37:23 +02:00
|
|
|
// If allow_unprepared_value is true, the returned iterator may defer reading
|
|
|
|
// the value and so will require PrepareValue() to be called before value();
|
|
|
|
// allow_unprepared_value = false is convenient when this optimization is not
|
|
|
|
// useful, e.g. when reading the whole column family.
|
2020-08-04 00:21:56 +02:00
|
|
|
// @param read_options Must outlive the returned iterator.
|
2015-10-13 00:06:38 +02:00
|
|
|
InternalIterator* NewInternalIterator(
|
2020-08-04 00:21:56 +02:00
|
|
|
const ReadOptions& read_options, Arena* arena,
|
|
|
|
RangeDelAggregator* range_del_agg, SequenceNumber sequence,
|
Properly report IO errors when IndexType::kBinarySearchWithFirstKey is used (#6621)
Summary:
Context: Index type `kBinarySearchWithFirstKey` added the ability for sst file iterator to sometimes report a key from index without reading the corresponding data block. This is useful when sst blocks are cut at some meaningful boundaries (e.g. one block per key prefix), and many seeks land between blocks (e.g. for each prefix, the ranges of keys in different sst files are nearly disjoint, so a typical seek needs to read a data block from only one file even if all files have the prefix). But this added a new error condition, which rocksdb code was really not equipped to deal with: `InternalIterator::value()` may fail with an IO error or Status::Incomplete, but it's just a method returning a Slice, with no way to report error instead. Before this PR, this type of error wasn't handled at all (an empty slice was returned), and kBinarySearchWithFirstKey implementation was considered a prototype.
Now that we (LogDevice) have experimented with kBinarySearchWithFirstKey for a while and confirmed that it's really useful, this PR is adding the missing error handling.
It's a pretty inconvenient situation implementation-wise. The error needs to be reported from InternalIterator when trying to access value. But there are ~700 call sites of `InternalIterator::value()`, most of which either can't hit the error condition (because the iterator is reading from memtable or from index or something) or wouldn't benefit from the deferred loading of the value (e.g. compaction iterator that reads all values anyway). Adding error handling to all these call sites would needlessly bloat the code. So instead I made the deferred value loading optional: only the call sites that may use deferred loading have to call the new method `PrepareValue()` before calling `value()`. The feature is enabled with a new bool argument `allow_unprepared_value` to a bunch of methods that create iterators (it wouldn't make sense to put it in ReadOptions because it's completely internal to iterators, with virtually no user-visible effect). Lmk if you have better ideas.
Note that the deferred value loading only happens for *internal* iterators. The user-visible iterator (DBIter) always prepares the value before returning from Seek/Next/etc. We could go further and add an API to defer that value loading too, but that's most likely not useful for LogDevice, so it doesn't seem worth the complexity for now.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/6621
Test Plan: make -j5 check . Will also deploy to some logdevice test clusters and look at stats.
Reviewed By: siying
Differential Revision: D20786930
Pulled By: al13n321
fbshipit-source-id: 6da77d918bad3780522e918f17f4d5513d3e99ee
2020-04-16 02:37:23 +02:00
|
|
|
ColumnFamilyHandle* column_family = nullptr,
|
|
|
|
bool allow_unprepared_value = false);
|
2015-10-13 02:22:37 +02:00
|
|
|
|
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
|
|
|
LogsWithPrepTracker* logs_with_prep_tracker() {
|
|
|
|
return &logs_with_prep_tracker_;
|
|
|
|
}
|
|
|
|
|
2017-05-24 20:25:38 +02:00
|
|
|
struct BGJobLimits {
|
|
|
|
int max_flushes;
|
|
|
|
int max_compactions;
|
|
|
|
};
|
|
|
|
// Returns maximum background flushes and compactions allowed to be scheduled
|
|
|
|
BGJobLimits GetBGJobLimits() const;
|
|
|
|
// Need a static version that can be called during SanitizeOptions().
|
|
|
|
static BGJobLimits GetBGJobLimits(int max_background_flushes,
|
|
|
|
int max_background_compactions,
|
|
|
|
int max_background_jobs,
|
|
|
|
bool parallelize_compactions);
|
Add options.base_background_compactions as a number of compaction threads for low compaction debt
Summary:
If options.base_background_compactions is given, we try to schedule number of compactions not existing this number, only when L0 files increase to certain number, or pending compaction bytes more than certain threshold, we schedule compactions based on options.max_background_compactions.
The watermarks are calculated based on slowdown thresholds.
Test Plan:
Add new test cases in column_family_test.
Adding more unit tests.
Reviewers: IslamAbdelRahman, yhchiang, kradhakrishnan, rven, anthony
Reviewed By: anthony
Subscribers: leveldb, dhruba, yoshinorim
Differential Revision: https://reviews.facebook.net/D53409
2016-01-28 20:56:16 +01:00
|
|
|
|
2016-08-04 02:42:06 +02:00
|
|
|
// move logs pending closing from job_context to the DB queue and
|
|
|
|
// schedule a purge
|
|
|
|
void ScheduleBgLogWriterClose(JobContext* job_context);
|
|
|
|
|
2016-12-28 20:53:29 +01:00
|
|
|
uint64_t MinLogNumberToKeep();
|
|
|
|
|
2018-11-06 04:28:21 +01:00
|
|
|
// Returns the lower bound file number for SSTs that won't be deleted, even if
|
|
|
|
// they're obsolete. This lower bound is used internally to prevent newly
|
|
|
|
// created flush/compaction output files from being deleted before they're
|
|
|
|
// installed. This technique avoids the need for tracking the exact numbers of
|
|
|
|
// files pending creation, although it prevents more files than necessary from
|
|
|
|
// being deleted.
|
|
|
|
uint64_t MinObsoleteSstNumberToKeep();
|
|
|
|
|
2013-11-15 03:03:57 +01:00
|
|
|
// Returns the list of live files in 'live' and the list
|
2013-12-31 03:33:57 +01:00
|
|
|
// of all files in the filesystem in 'candidate_files'.
|
2013-11-15 03:03:57 +01:00
|
|
|
// If force == false and the last call was less than
|
2014-09-05 20:48:17 +02:00
|
|
|
// db_options_.delete_obsolete_files_period_micros microseconds ago,
|
2014-10-28 19:54:33 +01:00
|
|
|
// it will not fill up the job_context
|
|
|
|
void FindObsoleteFiles(JobContext* job_context, bool force,
|
2013-11-15 03:03:57 +01:00
|
|
|
bool no_full_scan = false);
|
|
|
|
|
|
|
|
// Diffs the files listed in filenames and those that do not
|
2018-03-08 19:18:34 +01:00
|
|
|
// belong to live files are possibly removed. Also, removes all the
|
2013-11-15 03:03:57 +01:00
|
|
|
// files in sst_delete_files and log_delete_files.
|
|
|
|
// It is not necessary to hold the mutex when invoking this method.
|
2018-01-18 02:37:10 +01:00
|
|
|
// If FindObsoleteFiles() was run, we need to also run
|
|
|
|
// PurgeObsoleteFiles(), even if disable_delete_obsolete_files_ is true
|
2018-04-06 04:49:06 +02:00
|
|
|
void PurgeObsoleteFiles(JobContext& background_contet,
|
2016-06-22 03:41:23 +02:00
|
|
|
bool schedule_only = false);
|
|
|
|
|
2019-05-24 22:05:58 +02:00
|
|
|
// Schedule a background job to actually delete obsolete files.
|
2016-06-22 03:41:23 +02:00
|
|
|
void SchedulePurge();
|
2013-11-15 03:03:57 +01:00
|
|
|
|
2014-12-06 01:12:10 +01:00
|
|
|
const SnapshotList& snapshots() const { return snapshots_; }
|
|
|
|
|
2019-05-24 22:05:58 +02:00
|
|
|
// load list of snapshots to `snap_vector` that is no newer than `max_seq`
|
|
|
|
// in ascending order.
|
|
|
|
// `oldest_write_conflict_snapshot` is filled with the oldest snapshot
|
|
|
|
// which satisfies SnapshotImpl.is_write_conflict_boundary_ = true.
|
2019-05-04 02:26:20 +02:00
|
|
|
void LoadSnapshots(std::vector<SequenceNumber>* snap_vector,
|
|
|
|
SequenceNumber* oldest_write_conflict_snapshot,
|
|
|
|
const SequenceNumber& max_seq) const {
|
|
|
|
InstrumentedMutexLock l(mutex());
|
|
|
|
snapshots().GetAll(snap_vector, oldest_write_conflict_snapshot, max_seq);
|
|
|
|
}
|
|
|
|
|
2016-10-18 22:19:26 +02:00
|
|
|
const ImmutableDBOptions& immutable_db_options() const {
|
|
|
|
return immutable_db_options_;
|
|
|
|
}
|
|
|
|
|
2019-05-24 22:05:58 +02:00
|
|
|
// Cancel all background jobs, including flush, compaction, background
|
|
|
|
// purging, stats dumping threads, etc. If `wait` = true, wait for the
|
|
|
|
// running jobs to abort or finish before returning. Otherwise, only
|
|
|
|
// sends the signals.
|
2015-05-06 01:54:47 +02:00
|
|
|
void CancelAllBackgroundWork(bool wait);
|
2015-03-11 18:31:02 +01:00
|
|
|
|
2015-05-29 23:36:35 +02:00
|
|
|
// Find Super version and reference it. Based on options, it might return
|
|
|
|
// the thread local cached one.
|
|
|
|
// Call ReturnAndCleanupSuperVersion() when it is no longer needed.
|
|
|
|
SuperVersion* GetAndRefSuperVersion(ColumnFamilyData* cfd);
|
|
|
|
|
|
|
|
// Similar to the previous function but looks up based on a column family id.
|
|
|
|
// nullptr will be returned if this column family no longer exists.
|
|
|
|
// REQUIRED: this function should only be called on the write thread or if the
|
|
|
|
// mutex is held.
|
|
|
|
SuperVersion* GetAndRefSuperVersion(uint32_t column_family_id);
|
|
|
|
|
2017-10-11 23:48:28 +02:00
|
|
|
// Un-reference the super version and clean it up if it is the last reference.
|
|
|
|
void CleanupSuperVersion(SuperVersion* sv);
|
|
|
|
|
2015-05-29 23:36:35 +02:00
|
|
|
// Un-reference the super version and return it to thread local cache if
|
|
|
|
// needed. If it is the last reference of the super version. Clean it up
|
|
|
|
// after un-referencing it.
|
|
|
|
void ReturnAndCleanupSuperVersion(ColumnFamilyData* cfd, SuperVersion* sv);
|
|
|
|
|
|
|
|
// Similar to the previous function but looks up based on a column family id.
|
|
|
|
// nullptr will be returned if this column family no longer exists.
|
|
|
|
// REQUIRED: this function should only be called on the write thread.
|
|
|
|
void ReturnAndCleanupSuperVersion(uint32_t colun_family_id, SuperVersion* sv);
|
|
|
|
|
|
|
|
// REQUIRED: this function should only be called on the write thread or if the
|
|
|
|
// mutex is held. Return value only valid until next call to this function or
|
|
|
|
// mutex is released.
|
|
|
|
ColumnFamilyHandle* GetColumnFamilyHandle(uint32_t column_family_id);
|
Pessimistic Transactions
Summary:
Initial implementation of Pessimistic Transactions. This diff contains the api changes discussed in D38913. This diff is pretty large, so let me know if people would prefer to meet up to discuss it.
MyRocks folks: please take a look at the API in include/rocksdb/utilities/transaction[_db].h and let me know if you have any issues.
Also, you'll notice a couple of TODOs in the implementation of RollbackToSavePoint(). After chatting with Siying, I'm going to send out a separate diff for an alternate implementation of this feature that implements the rollback inside of WriteBatch/WriteBatchWithIndex. We can then decide which route is preferable.
Next, I'm planning on doing some perf testing and then integrating this diff into MongoRocks for further testing.
Test Plan: Unit tests, db_bench parallel testing.
Reviewers: igor, rven, sdong, yhchiang, yoshinorim
Reviewed By: sdong
Subscribers: hermanlee4, maykov, spetrunia, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D40869
2015-05-26 02:37:33 +02:00
|
|
|
|
2017-05-10 23:54:35 +02:00
|
|
|
// Same as above, should called without mutex held and not on write thread.
|
2018-10-08 23:22:06 +02:00
|
|
|
std::unique_ptr<ColumnFamilyHandle> GetColumnFamilyHandleUnlocked(
|
|
|
|
uint32_t column_family_id);
|
2017-05-10 23:54:35 +02:00
|
|
|
|
2015-10-17 09:16:36 +02:00
|
|
|
// Returns the number of currently running flushes.
|
|
|
|
// REQUIREMENT: mutex_ must be held when calling this function.
|
|
|
|
int num_running_flushes() {
|
|
|
|
mutex_.AssertHeld();
|
|
|
|
return num_running_flushes_;
|
|
|
|
}
|
|
|
|
|
|
|
|
// Returns the number of currently running compactions.
|
|
|
|
// REQUIREMENT: mutex_ must be held when calling this function.
|
|
|
|
int num_running_compactions() {
|
|
|
|
mutex_.AssertHeld();
|
|
|
|
return num_running_compactions_;
|
|
|
|
}
|
|
|
|
|
2017-03-29 20:42:56 +02:00
|
|
|
const WriteController& write_controller() { return write_controller_; }
|
|
|
|
|
2020-08-04 00:21:56 +02:00
|
|
|
// @param read_options Must outlive the returned iterator.
|
|
|
|
InternalIterator* NewInternalIterator(const ReadOptions& read_options,
|
|
|
|
ColumnFamilyData* cfd,
|
|
|
|
SuperVersion* super_version,
|
|
|
|
Arena* arena,
|
|
|
|
RangeDelAggregator* range_del_agg,
|
|
|
|
SequenceNumber sequence,
|
|
|
|
bool allow_unprepared_value);
|
2017-07-24 19:28:17 +02:00
|
|
|
|
2016-04-18 20:11:51 +02:00
|
|
|
// hollow transactions shell used for recovery.
|
|
|
|
// these will then be passed to TransactionDB so that
|
|
|
|
// locks can be reacquired before writing can resume.
|
|
|
|
struct RecoveredTransaction {
|
|
|
|
std::string name_;
|
2018-07-07 02:17:36 +02:00
|
|
|
bool unprepared_;
|
|
|
|
|
|
|
|
struct BatchInfo {
|
|
|
|
uint64_t log_number_;
|
|
|
|
// TODO(lth): For unprepared, the memory usage here can be big for
|
|
|
|
// unprepared transactions. This is only useful for rollbacks, and we
|
|
|
|
// can in theory just keep keyset for that.
|
|
|
|
WriteBatch* batch_;
|
|
|
|
// Number of sub-batches. A new sub-batch is created if txn attempts to
|
|
|
|
// insert a duplicate key,seq to memtable. This is currently used in
|
|
|
|
// WritePreparedTxn/WriteUnpreparedTxn.
|
|
|
|
size_t batch_cnt_;
|
|
|
|
};
|
|
|
|
|
|
|
|
// This maps the seq of the first key in the batch to BatchInfo, which
|
|
|
|
// contains WriteBatch and other information relevant to the batch.
|
|
|
|
//
|
|
|
|
// For WriteUnprepared, batches_ can have size greater than 1, but for
|
|
|
|
// other write policies, it must be of size 1.
|
|
|
|
std::map<SequenceNumber, BatchInfo> batches_;
|
|
|
|
|
2016-04-18 20:11:51 +02:00
|
|
|
explicit RecoveredTransaction(const uint64_t log, const std::string& name,
|
2018-03-05 19:48:29 +01:00
|
|
|
WriteBatch* batch, SequenceNumber seq,
|
2018-07-07 02:17:36 +02:00
|
|
|
size_t batch_cnt, bool unprepared)
|
|
|
|
: name_(name), unprepared_(unprepared) {
|
|
|
|
batches_[seq] = {log, batch, batch_cnt};
|
|
|
|
}
|
|
|
|
|
|
|
|
~RecoveredTransaction() {
|
|
|
|
for (auto& it : batches_) {
|
|
|
|
delete it.second.batch_;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
void AddBatch(SequenceNumber seq, uint64_t log_number, WriteBatch* batch,
|
|
|
|
size_t batch_cnt, bool unprepared) {
|
|
|
|
assert(batches_.count(seq) == 0);
|
|
|
|
batches_[seq] = {log_number, batch, batch_cnt};
|
|
|
|
// Prior state must be unprepared, since the prepare batch must be the
|
|
|
|
// last batch.
|
|
|
|
assert(unprepared_);
|
|
|
|
unprepared_ = unprepared;
|
|
|
|
}
|
2016-04-18 20:11:51 +02:00
|
|
|
};
|
|
|
|
|
2016-09-24 01:34:04 +02:00
|
|
|
bool allow_2pc() const { return immutable_db_options_.allow_2pc; }
|
2016-04-18 20:11:51 +02:00
|
|
|
|
2016-04-18 20:15:50 +02:00
|
|
|
std::unordered_map<std::string, RecoveredTransaction*>
|
|
|
|
recovered_transactions() {
|
|
|
|
return recovered_transactions_;
|
|
|
|
}
|
|
|
|
|
2016-04-18 20:11:51 +02:00
|
|
|
RecoveredTransaction* GetRecoveredTransaction(const std::string& name) {
|
|
|
|
auto it = recovered_transactions_.find(name);
|
|
|
|
if (it == recovered_transactions_.end()) {
|
|
|
|
return nullptr;
|
|
|
|
} else {
|
|
|
|
return it->second;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
void InsertRecoveredTransaction(const uint64_t log, const std::string& name,
|
2018-03-05 19:48:29 +01:00
|
|
|
WriteBatch* batch, SequenceNumber seq,
|
2018-07-07 02:17:36 +02:00
|
|
|
size_t batch_cnt, bool unprepared_batch) {
|
|
|
|
// For WriteUnpreparedTxn, InsertRecoveredTransaction is called multiple
|
|
|
|
// times for every unprepared batch encountered during recovery.
|
|
|
|
//
|
|
|
|
// If the transaction is prepared, then the last call to
|
|
|
|
// InsertRecoveredTransaction will have unprepared_batch = false.
|
|
|
|
auto rtxn = recovered_transactions_.find(name);
|
|
|
|
if (rtxn == recovered_transactions_.end()) {
|
|
|
|
recovered_transactions_[name] = new RecoveredTransaction(
|
|
|
|
log, name, batch, seq, batch_cnt, unprepared_batch);
|
|
|
|
} else {
|
|
|
|
rtxn->second->AddBatch(seq, log, batch, batch_cnt, unprepared_batch);
|
|
|
|
}
|
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
|
|
|
logs_with_prep_tracker_.MarkLogAsContainingPrepSection(log);
|
2016-04-18 20:11:51 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
void DeleteRecoveredTransaction(const std::string& name) {
|
|
|
|
auto it = recovered_transactions_.find(name);
|
|
|
|
assert(it != recovered_transactions_.end());
|
|
|
|
auto* trx = it->second;
|
|
|
|
recovered_transactions_.erase(it);
|
2018-07-07 02:17:36 +02:00
|
|
|
for (const auto& info : trx->batches_) {
|
|
|
|
logs_with_prep_tracker_.MarkLogAsHavingPrepSectionFlushed(
|
|
|
|
info.second.log_number_);
|
|
|
|
}
|
2016-04-18 20:11:51 +02:00
|
|
|
delete trx;
|
|
|
|
}
|
|
|
|
|
2016-05-13 19:37:46 +02:00
|
|
|
void DeleteAllRecoveredTransactions() {
|
|
|
|
for (auto it = recovered_transactions_.begin();
|
2019-05-15 22:14:18 +02:00
|
|
|
it != recovered_transactions_.end(); ++it) {
|
2016-05-13 19:37:46 +02:00
|
|
|
delete it->second;
|
|
|
|
}
|
|
|
|
recovered_transactions_.clear();
|
|
|
|
}
|
|
|
|
|
2016-08-04 02:42:06 +02:00
|
|
|
void AddToLogsToFreeQueue(log::Writer* log_writer) {
|
|
|
|
logs_to_free_queue_.push_back(log_writer);
|
|
|
|
}
|
2017-10-06 19:26:38 +02:00
|
|
|
|
2019-12-17 22:20:42 +01:00
|
|
|
void AddSuperVersionsToFreeQueue(SuperVersion* sv) {
|
|
|
|
superversions_to_free_queue_.push_back(sv);
|
|
|
|
}
|
|
|
|
|
2017-10-06 19:26:38 +02:00
|
|
|
void SetSnapshotChecker(SnapshotChecker* snapshot_checker);
|
|
|
|
|
2019-01-16 06:32:15 +01:00
|
|
|
// Fill JobContext with snapshot information needed by flush and compaction.
|
|
|
|
void GetSnapshotContext(JobContext* job_context,
|
|
|
|
std::vector<SequenceNumber>* snapshot_seqs,
|
|
|
|
SequenceNumber* earliest_write_conflict_snapshot,
|
|
|
|
SnapshotChecker** snapshot_checker);
|
|
|
|
|
2018-03-28 21:01:09 +02:00
|
|
|
// Not thread-safe.
|
|
|
|
void SetRecoverableStatePreReleaseCallback(PreReleaseCallback* callback);
|
|
|
|
|
2019-05-04 02:26:20 +02:00
|
|
|
InstrumentedMutex* mutex() const { return &mutex_; }
|
2016-04-18 20:11:51 +02:00
|
|
|
|
2019-05-24 22:05:58 +02:00
|
|
|
// Initialize a brand new DB. The DB directory is expected to be empty before
|
2020-07-10 22:39:47 +02:00
|
|
|
// calling it. Push new manifest file name into `new_filenames`.
|
|
|
|
Status NewDB(std::vector<std::string>* new_filenames);
|
2016-06-24 20:19:40 +02:00
|
|
|
|
2017-11-11 02:18:01 +01:00
|
|
|
// This is to be used only by internal rocksdb classes.
|
|
|
|
static Status Open(const DBOptions& db_options, const std::string& name,
|
|
|
|
const std::vector<ColumnFamilyDescriptor>& column_families,
|
|
|
|
std::vector<ColumnFamilyHandle*>* handles, DB** dbptr,
|
2018-06-29 03:46:39 +02:00
|
|
|
const bool seq_per_batch, const bool batch_per_txn);
|
2017-11-11 02:18:01 +01:00
|
|
|
|
2020-03-03 01:14:00 +01:00
|
|
|
static IOStatus CreateAndNewDirectory(
|
|
|
|
FileSystem* fs, const std::string& dirname,
|
|
|
|
std::unique_ptr<FSDirectory>* directory);
|
2018-04-06 04:49:06 +02:00
|
|
|
|
2019-02-21 00:46:59 +01:00
|
|
|
// find stats map from stats_history_ with smallest timestamp in
|
|
|
|
// the range of [start_time, end_time)
|
|
|
|
bool FindStatsByTime(uint64_t start_time, uint64_t end_time,
|
|
|
|
uint64_t* new_time,
|
|
|
|
std::map<std::string, uint64_t>* stats_map);
|
|
|
|
|
2019-08-16 01:59:42 +02:00
|
|
|
// Print information of all tombstones of all iterators to the std::string
|
|
|
|
// This is only used by ldb. The output might be capped. Tombstones
|
|
|
|
// printed out are not guaranteed to be in any order.
|
|
|
|
Status TablesRangeTombstoneSummary(ColumnFamilyHandle* column_family,
|
|
|
|
int max_entries_to_print,
|
|
|
|
std::string* out_str);
|
|
|
|
|
2019-05-24 22:05:58 +02:00
|
|
|
#ifndef NDEBUG
|
|
|
|
// Compact any files in the named level that overlap [*begin, *end]
|
|
|
|
Status TEST_CompactRange(int level, const Slice* begin, const Slice* end,
|
|
|
|
ColumnFamilyHandle* column_family = nullptr,
|
|
|
|
bool disallow_trivial_move = false);
|
|
|
|
|
2020-12-10 06:19:55 +01:00
|
|
|
Status TEST_SwitchWAL();
|
2019-05-24 22:05:58 +02:00
|
|
|
|
|
|
|
bool TEST_UnableToReleaseOldestLog() { return unable_to_release_oldest_log_; }
|
|
|
|
|
|
|
|
bool TEST_IsLogGettingFlushed() {
|
|
|
|
return alive_log_files_.begin()->getting_flushed;
|
|
|
|
}
|
|
|
|
|
|
|
|
Status TEST_SwitchMemtable(ColumnFamilyData* cfd = nullptr);
|
|
|
|
|
|
|
|
// Force current memtable contents to be flushed.
|
|
|
|
Status TEST_FlushMemTable(bool wait = true, bool allow_write_stall = false,
|
|
|
|
ColumnFamilyHandle* cfh = nullptr);
|
|
|
|
|
2019-07-01 23:04:10 +02:00
|
|
|
Status TEST_FlushMemTable(ColumnFamilyData* cfd,
|
|
|
|
const FlushOptions& flush_opts);
|
|
|
|
|
|
|
|
// Flush (multiple) ColumnFamilyData without using ColumnFamilyHandle. This
|
|
|
|
// is because in certain cases, we can flush column families, wait for the
|
|
|
|
// flush to complete, but delete the column family handle before the wait
|
|
|
|
// finishes. For example in CompactRange.
|
|
|
|
Status TEST_AtomicFlushMemTables(const autovector<ColumnFamilyData*>& cfds,
|
|
|
|
const FlushOptions& flush_opts);
|
|
|
|
|
2019-05-24 22:05:58 +02:00
|
|
|
// Wait for memtable compaction
|
|
|
|
Status TEST_WaitForFlushMemTable(ColumnFamilyHandle* column_family = nullptr);
|
|
|
|
|
|
|
|
// Wait for any compaction
|
|
|
|
// We add a bool parameter to wait for unscheduledCompactions_ == 0, but this
|
|
|
|
// is only for the special test of CancelledCompactions
|
|
|
|
Status TEST_WaitForCompact(bool waitUnscheduled = false);
|
|
|
|
|
2021-01-28 02:56:17 +01:00
|
|
|
// Get the background error status
|
|
|
|
Status TEST_GetBGError();
|
|
|
|
|
2019-05-24 22:05:58 +02:00
|
|
|
// Return the maximum overlapping data (in bytes) at next level for any
|
|
|
|
// file at a level >= 1.
|
|
|
|
int64_t TEST_MaxNextLevelOverlappingBytes(
|
|
|
|
ColumnFamilyHandle* column_family = nullptr);
|
|
|
|
|
|
|
|
// Return the current manifest file no.
|
|
|
|
uint64_t TEST_Current_Manifest_FileNo();
|
|
|
|
|
|
|
|
// Returns the number that'll be assigned to the next file that's created.
|
|
|
|
uint64_t TEST_Current_Next_FileNo();
|
|
|
|
|
|
|
|
// get total level0 file size. Only for testing.
|
|
|
|
uint64_t TEST_GetLevel0TotalSize();
|
|
|
|
|
|
|
|
void TEST_GetFilesMetaData(ColumnFamilyHandle* column_family,
|
|
|
|
std::vector<std::vector<FileMetaData>>* metadata);
|
|
|
|
|
|
|
|
void TEST_LockMutex();
|
|
|
|
|
|
|
|
void TEST_UnlockMutex();
|
|
|
|
|
|
|
|
// REQUIRES: mutex locked
|
|
|
|
void* TEST_BeginWrite();
|
|
|
|
|
|
|
|
// REQUIRES: mutex locked
|
|
|
|
// pass the pointer that you got from TEST_BeginWrite()
|
|
|
|
void TEST_EndWrite(void* w);
|
|
|
|
|
|
|
|
uint64_t TEST_MaxTotalInMemoryState() const {
|
|
|
|
return max_total_in_memory_state_;
|
|
|
|
}
|
|
|
|
|
|
|
|
size_t TEST_LogsToFreeSize();
|
|
|
|
|
|
|
|
uint64_t TEST_LogfileNumber();
|
|
|
|
|
|
|
|
uint64_t TEST_total_log_size() const { return total_log_size_; }
|
|
|
|
|
|
|
|
// Returns column family name to ImmutableCFOptions map.
|
|
|
|
Status TEST_GetAllImmutableCFOptions(
|
|
|
|
std::unordered_map<std::string, const ImmutableCFOptions*>* iopts_map);
|
|
|
|
|
|
|
|
// Return the lastest MutableCFOptions of a column family
|
|
|
|
Status TEST_GetLatestMutableCFOptions(ColumnFamilyHandle* column_family,
|
|
|
|
MutableCFOptions* mutable_cf_options);
|
|
|
|
|
|
|
|
Cache* TEST_table_cache() { return table_cache_.get(); }
|
|
|
|
|
|
|
|
WriteController& TEST_write_controler() { return write_controller_; }
|
|
|
|
|
|
|
|
uint64_t TEST_FindMinLogContainingOutstandingPrep();
|
|
|
|
uint64_t TEST_FindMinPrepLogReferencedByMemTable();
|
|
|
|
size_t TEST_PreparedSectionCompletedSize();
|
|
|
|
size_t TEST_LogsWithPrepSize();
|
|
|
|
|
|
|
|
int TEST_BGCompactionsAllowed() const;
|
|
|
|
int TEST_BGFlushesAllowed() const;
|
|
|
|
size_t TEST_GetWalPreallocateBlockSize(uint64_t write_buffer_size) const;
|
2020-08-15 05:11:35 +02:00
|
|
|
void TEST_WaitForStatsDumpRun(std::function<void()> callback) const;
|
2019-06-18 00:17:43 +02:00
|
|
|
size_t TEST_EstimateInMemoryStatsHistorySize() const;
|
2020-05-05 00:05:34 +02:00
|
|
|
|
|
|
|
VersionSet* TEST_GetVersionSet() const { return versions_.get(); }
|
2020-05-07 18:29:21 +02:00
|
|
|
|
2020-11-18 00:54:49 +01:00
|
|
|
uint64_t TEST_GetCurrentLogNumber() const {
|
|
|
|
InstrumentedMutexLock l(mutex());
|
|
|
|
assert(!logs_.empty());
|
|
|
|
return logs_.back().number;
|
|
|
|
}
|
|
|
|
|
2020-05-07 18:29:21 +02:00
|
|
|
const std::unordered_set<uint64_t>& TEST_GetFilesGrabbedForPurge() const {
|
|
|
|
return files_grabbed_for_purge_;
|
|
|
|
}
|
2020-08-15 05:11:35 +02:00
|
|
|
|
|
|
|
#ifndef ROCKSDB_LITE
|
2020-10-02 04:12:26 +02:00
|
|
|
PeriodicWorkTestScheduler* TEST_GetPeriodicWorkScheduler() const;
|
2020-08-15 05:11:35 +02:00
|
|
|
#endif // !ROCKSDB_LITE
|
|
|
|
|
2019-05-24 22:05:58 +02:00
|
|
|
#endif // NDEBUG
|
2019-05-31 06:29:44 +02:00
|
|
|
|
2020-08-15 05:11:35 +02:00
|
|
|
// persist stats to column family "_persistent_stats"
|
|
|
|
void PersistStats();
|
|
|
|
|
|
|
|
// dump rocksdb.stats to LOG
|
|
|
|
void DumpStats();
|
|
|
|
|
2020-10-02 04:12:26 +02:00
|
|
|
// flush LOG out of application buffer
|
|
|
|
void FlushInfoLog();
|
|
|
|
|
2012-12-18 22:05:39 +01:00
|
|
|
protected:
|
2012-11-06 04:18:49 +01:00
|
|
|
const std::string dbname_;
|
2019-09-03 17:50:47 +02:00
|
|
|
std::string db_id_;
|
2020-06-15 19:45:03 +02:00
|
|
|
// db_session_id_ is an identifier that gets reset
|
|
|
|
// every time the DB is opened
|
|
|
|
std::string db_session_id_;
|
2018-11-09 20:17:34 +01:00
|
|
|
std::unique_ptr<VersionSet> versions_;
|
2018-01-16 19:57:56 +01:00
|
|
|
// Flag to check whether we allocated and own the info log file
|
|
|
|
bool own_info_log_;
|
2016-10-14 21:25:39 +02:00
|
|
|
const DBOptions initial_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
|
|
|
Env* const env_;
|
2021-01-26 07:07:26 +01:00
|
|
|
std::shared_ptr<SystemClock> clock_;
|
2020-08-05 03:40:19 +02:00
|
|
|
std::shared_ptr<IOTracer> io_tracer_;
|
2016-09-24 01:34:04 +02:00
|
|
|
const ImmutableDBOptions immutable_db_options_;
|
2020-08-13 02:28:10 +02:00
|
|
|
FileSystemPtr fs_;
|
2016-09-24 01:34:04 +02:00
|
|
|
MutableDBOptions mutable_db_options_;
|
2014-07-28 21:05:36 +02:00
|
|
|
Statistics* stats_;
|
2016-04-18 20:11:51 +02:00
|
|
|
std::unordered_map<std::string, RecoveredTransaction*>
|
|
|
|
recovered_transactions_;
|
2018-08-01 09:14:43 +02:00
|
|
|
std::unique_ptr<Tracer> tracer_;
|
|
|
|
InstrumentedMutex trace_mutex_;
|
2019-06-14 00:39:52 +02:00
|
|
|
BlockCacheTracer block_cache_tracer_;
|
2012-11-06 04:18:49 +01:00
|
|
|
|
2019-03-27 00:41:31 +01:00
|
|
|
// State below is protected by mutex_
|
|
|
|
// With two_write_queues enabled, some of the variables that accessed during
|
|
|
|
// WriteToWAL need different synchronization: log_empty_, alive_log_files_,
|
|
|
|
// logs_, logfile_number_. Refer to the definition of each variable below for
|
|
|
|
// more description.
|
|
|
|
mutable InstrumentedMutex mutex_;
|
|
|
|
|
|
|
|
ColumnFamilyHandleImpl* default_cf_handle_;
|
|
|
|
InternalStats* default_cf_internal_stats_;
|
|
|
|
|
|
|
|
// only used for dynamically adjusting max_total_wal_size. it is a sum of
|
|
|
|
// [write_buffer_size * max_write_buffer_number] over all column families
|
|
|
|
uint64_t max_total_in_memory_state_;
|
|
|
|
// If true, we have only one (default) column family. We use this to optimize
|
|
|
|
// some code-paths
|
|
|
|
bool single_column_family_mode_;
|
|
|
|
|
|
|
|
// The options to access storage files
|
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_;
|
2019-03-27 00:41:31 +01:00
|
|
|
|
|
|
|
// Additonal options for compaction and flush
|
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_compaction_;
|
2019-03-27 00:41:31 +01:00
|
|
|
|
2019-04-24 21:05:29 +02:00
|
|
|
std::unique_ptr<ColumnFamilyMemTablesImpl> column_family_memtables_;
|
|
|
|
|
|
|
|
// Increase the sequence number after writing each batch, whether memtable is
|
|
|
|
// disabled for that or not. Otherwise the sequence number is increased after
|
|
|
|
// writing each key into memtable. This implies that when disable_memtable is
|
|
|
|
// set, the seq is not increased at all.
|
|
|
|
//
|
|
|
|
// Default: false
|
|
|
|
const bool seq_per_batch_;
|
|
|
|
// This determines during recovery whether we expect one writebatch per
|
|
|
|
// recovered transaction, or potentially multiple writebatches per
|
|
|
|
// transaction. For WriteUnprepared, this is set to false, since multiple
|
|
|
|
// batches can exist per transaction.
|
|
|
|
//
|
|
|
|
// Default: true
|
|
|
|
const bool batch_per_txn_;
|
|
|
|
|
2015-12-09 02:01:02 +01:00
|
|
|
// Except in DB::Open(), WriteOptionsFile can only be called when:
|
2017-05-08 07:12:55 +02:00
|
|
|
// Persist options to options file.
|
|
|
|
// If need_mutex_lock = false, the method will lock DB mutex.
|
|
|
|
// If need_enter_write_thread = false, the method will enter write thread.
|
|
|
|
Status WriteOptionsFile(bool need_mutex_lock, bool need_enter_write_thread);
|
2015-12-09 02:01:02 +01:00
|
|
|
|
2020-12-02 21:59:23 +01:00
|
|
|
Status CompactRangeInternal(const CompactRangeOptions& options,
|
|
|
|
ColumnFamilyHandle* column_family,
|
|
|
|
const Slice* begin, const Slice* end);
|
|
|
|
|
|
|
|
Status GetApproximateSizesInternal(const SizeApproximationOptions& options,
|
|
|
|
ColumnFamilyHandle* column_family,
|
|
|
|
const Range* range, int n,
|
|
|
|
uint64_t* sizes);
|
|
|
|
|
2015-12-09 02:01:02 +01:00
|
|
|
// The following two functions can only be called when:
|
|
|
|
// 1. WriteThread::Writer::EnterUnbatched() is used.
|
|
|
|
// 2. db_mutex is NOT held
|
2015-11-11 07:58:01 +01:00
|
|
|
Status RenameTempFileToOptionsFile(const std::string& file_name);
|
|
|
|
Status DeleteObsoleteOptionsFiles();
|
|
|
|
|
2017-04-18 21:00:36 +02:00
|
|
|
void NotifyOnFlushBegin(ColumnFamilyData* cfd, FileMetaData* file_meta,
|
|
|
|
const MutableCFOptions& mutable_cf_options,
|
2019-10-16 19:39:00 +02:00
|
|
|
int job_id);
|
2017-04-18 21:00:36 +02:00
|
|
|
|
2019-10-16 19:39:00 +02:00
|
|
|
void NotifyOnFlushCompleted(
|
|
|
|
ColumnFamilyData* cfd, const MutableCFOptions& mutable_cf_options,
|
|
|
|
std::list<std::unique_ptr<FlushJobInfo>>* flush_jobs_info);
|
CompactFiles, EventListener and GetDatabaseMetaData
Summary:
This diff adds three sets of APIs to RocksDB.
= GetColumnFamilyMetaData =
* This APIs allow users to obtain the current state of a RocksDB instance on one column family.
* See GetColumnFamilyMetaData in include/rocksdb/db.h
= EventListener =
* A virtual class that allows users to implement a set of
call-back functions which will be called when specific
events of a RocksDB instance happens.
* To register EventListener, simply insert an EventListener to ColumnFamilyOptions::listeners
= CompactFiles =
* CompactFiles API inputs a set of file numbers and an output level, and RocksDB
will try to compact those files into the specified level.
= Example =
* Example code can be found in example/compact_files_example.cc, which implements
a simple external compactor using EventListener, GetColumnFamilyMetaData, and
CompactFiles API.
Test Plan:
listener_test
compactor_test
example/compact_files_example
export ROCKSDB_TESTS=CompactFiles
db_test
export ROCKSDB_TESTS=MetaData
db_test
Reviewers: ljin, igor, rven, sdong
Reviewed By: sdong
Subscribers: MarkCallaghan, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D24705
2014-11-07 23:45:18 +01:00
|
|
|
|
2019-03-28 00:13:08 +01:00
|
|
|
void NotifyOnCompactionBegin(ColumnFamilyData* cfd, Compaction* c,
|
|
|
|
const Status& st,
|
|
|
|
const CompactionJobStats& job_stats, int job_id);
|
2018-10-11 02:30:22 +02:00
|
|
|
|
2019-03-28 00:13:08 +01:00
|
|
|
void NotifyOnCompactionCompleted(ColumnFamilyData* cfd, Compaction* c,
|
|
|
|
const Status& st,
|
2015-06-03 02:07:16 +02:00
|
|
|
const CompactionJobStats& job_stats,
|
2015-06-03 02:35:39 +02:00
|
|
|
int job_id);
|
2016-06-10 04:03:10 +02:00
|
|
|
void NotifyOnMemTableSealed(ColumnFamilyData* cfd,
|
2016-06-02 20:57:31 +02:00
|
|
|
const MemTableInfo& mem_table_info);
|
2015-01-27 23:44:02 +01:00
|
|
|
|
2017-02-28 20:05:08 +01:00
|
|
|
#ifndef ROCKSDB_LITE
|
2016-12-06 22:56:17 +01:00
|
|
|
void NotifyOnExternalFileIngested(
|
|
|
|
ColumnFamilyData* cfd, const ExternalSstFileIngestionJob& ingestion_job);
|
2017-02-28 20:05:08 +01:00
|
|
|
#endif // !ROCKSDB_LITE
|
2016-12-06 22:56:17 +01:00
|
|
|
|
2014-11-20 19:49:32 +01:00
|
|
|
void NewThreadStatusCfInfo(ColumnFamilyData* cfd) const;
|
|
|
|
|
|
|
|
void EraseThreadStatusCfInfo(ColumnFamilyData* cfd) const;
|
|
|
|
|
|
|
|
void EraseThreadStatusDbInfo() const;
|
|
|
|
|
2017-11-15 17:22:54 +01:00
|
|
|
// If disable_memtable is set the application logic must guarantee that the
|
2018-04-08 06:55:42 +02:00
|
|
|
// batch will still be skipped from memtable during the recovery. An excption
|
|
|
|
// to this is seq_per_batch_ mode, in which since each batch already takes one
|
|
|
|
// seq, it is ok for the batch to write to memtable during recovery as long as
|
|
|
|
// it only takes one sequence number: i.e., no duplicate keys.
|
|
|
|
// In WriteCommitted it is guarnateed since disable_memtable is used for
|
|
|
|
// prepare batch which will be written to memtable later during the commit,
|
|
|
|
// and in WritePrepared it is guaranteed since it will be used only for WAL
|
|
|
|
// markers which will never be written to memtable. If the commit marker is
|
|
|
|
// accompanied with CommitTimeWriteBatch that is not written to memtable as
|
|
|
|
// long as it has no duplicate keys, it does not violate the one-seq-per-batch
|
|
|
|
// policy.
|
|
|
|
// batch_cnt is expected to be non-zero in seq_per_batch mode and
|
|
|
|
// indicates the number of sub-patches. A sub-patch is a subset of the write
|
|
|
|
// batch that does not have duplicate keys.
|
2015-05-29 23:36:35 +02:00
|
|
|
Status WriteImpl(const WriteOptions& options, WriteBatch* updates,
|
2016-04-18 20:11:51 +02:00
|
|
|
WriteCallback* callback = nullptr,
|
|
|
|
uint64_t* log_used = nullptr, uint64_t log_ref = 0,
|
2017-12-01 08:39:56 +01:00
|
|
|
bool disable_memtable = false, uint64_t* seq_used = nullptr,
|
2018-02-06 03:32:54 +01:00
|
|
|
size_t batch_cnt = 0,
|
2017-12-01 08:39:56 +01:00
|
|
|
PreReleaseCallback* pre_release_callback = nullptr);
|
2016-04-18 20:11:51 +02:00
|
|
|
|
2017-05-19 23:24:23 +02:00
|
|
|
Status PipelinedWriteImpl(const WriteOptions& options, WriteBatch* updates,
|
|
|
|
WriteCallback* callback = nullptr,
|
|
|
|
uint64_t* log_used = nullptr, uint64_t log_ref = 0,
|
2017-08-17 01:49:11 +02:00
|
|
|
bool disable_memtable = false,
|
|
|
|
uint64_t* seq_used = nullptr);
|
2017-05-19 23:24:23 +02:00
|
|
|
|
2019-05-14 02:43:47 +02:00
|
|
|
// Write only to memtables without joining any write queue
|
|
|
|
Status UnorderedWriteMemtable(const WriteOptions& write_options,
|
|
|
|
WriteBatch* my_batch, WriteCallback* callback,
|
|
|
|
uint64_t log_ref, SequenceNumber seq,
|
|
|
|
const size_t sub_batch_cnt);
|
|
|
|
|
|
|
|
// Whether the batch requires to be assigned with an order
|
|
|
|
enum AssignOrder : bool { kDontAssignOrder, kDoAssignOrder };
|
|
|
|
// Whether it requires publishing last sequence or not
|
|
|
|
enum PublishLastSeq : bool { kDontPublishLastSeq, kDoPublishLastSeq };
|
|
|
|
|
|
|
|
// Join the write_thread to write the batch only to the WAL. It is the
|
|
|
|
// responsibility of the caller to also write the write batch to the memtable
|
|
|
|
// if it required.
|
|
|
|
//
|
|
|
|
// sub_batch_cnt is expected to be non-zero when assign_order = kDoAssignOrder
|
|
|
|
// indicating the number of sub-batches in my_batch. A sub-patch is a subset
|
|
|
|
// of the write batch that does not have duplicate keys. When seq_per_batch is
|
|
|
|
// not set, each key is a separate sub_batch. Otherwise each duplicate key
|
|
|
|
// marks start of a new sub-batch.
|
|
|
|
Status WriteImplWALOnly(
|
|
|
|
WriteThread* write_thread, const WriteOptions& options,
|
|
|
|
WriteBatch* updates, WriteCallback* callback, uint64_t* log_used,
|
|
|
|
const uint64_t log_ref, uint64_t* seq_used, const size_t sub_batch_cnt,
|
|
|
|
PreReleaseCallback* pre_release_callback, const AssignOrder assign_order,
|
|
|
|
const PublishLastSeq publish_last_seq, const bool disable_memtable);
|
2017-06-24 23:06:43 +02:00
|
|
|
|
2017-11-02 01:23:52 +01:00
|
|
|
// write cached_recoverable_state_ to memtable if it is not empty
|
|
|
|
// The writer must be the leader in write_thread_ and holding mutex_
|
|
|
|
Status WriteRecoverableState();
|
2015-05-29 23:36:35 +02:00
|
|
|
|
2018-02-23 22:50:02 +01:00
|
|
|
// Actual implementation of Close()
|
|
|
|
Status CloseImpl();
|
|
|
|
|
2019-03-27 00:41:31 +01:00
|
|
|
// Recover the descriptor from persistent storage. May do a significant
|
|
|
|
// amount of work to recover recently logged updates. Any changes to
|
|
|
|
// be made to the descriptor are added to *edit.
|
2020-01-29 20:39:11 +01:00
|
|
|
// recovered_seq is set to less than kMaxSequenceNumber if the log's tail is
|
|
|
|
// skipped.
|
2019-03-27 00:41:31 +01:00
|
|
|
virtual Status Recover(
|
|
|
|
const std::vector<ColumnFamilyDescriptor>& column_families,
|
2020-09-18 00:39:25 +02:00
|
|
|
bool read_only = false, bool error_if_wal_file_exists = false,
|
|
|
|
bool error_if_data_exists_in_wals = false,
|
2020-01-29 20:39:11 +01:00
|
|
|
uint64_t* recovered_seq = nullptr);
|
2019-03-27 00:41:31 +01:00
|
|
|
|
2019-12-03 02:43:37 +01:00
|
|
|
virtual bool OwnTablesAndLogs() const { return true; }
|
|
|
|
|
Fix a recovery corner case (#7621)
Summary:
Consider the following sequence of events:
1. Db flushed an SST with file number N, appended to MANIFEST, and tried to sync the MANIFEST.
2. Syncing MANIFEST failed and db crashed.
3. Db tried to recover with this MANIFEST. In the meantime, no entry about the newly-flushed SST was found in the MANIFEST. Therefore, RocksDB replayed WAL and tried to flush to an SST file reusing the same file number N. This failed because file system does not support overwrite. Then Db deleted this file.
4. Db crashed again.
5. Db tried to recover. When db read the MANIFEST, there was an entry referencing N.sst. This could happen probably because the append in step 1 finally reached the MANIFEST and became visible. Since N.sst had been deleted in step 3, recovery failed.
It is possible that N.sst created in step 1 is valid. Although step 3 would still fail since the MANIFEST was not synced properly in step 1 and 2, deleting N.sst would make it impossible for the db to recover even if the remaining part of MANIFEST was appended and visible after step 5.
After this PR, in step 3, immediately after recovering from MANIFEST, a new MANIFEST is created, then we find that N.sst is not referenced in the MANIFEST, so we delete it, and we'll not reuse N as file number. Then in step 5, since the new MANIFEST does not contain N.sst, the recovery failure situation in step 5 won't happen.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7621
Test Plan:
1. some tests are updated, because these tests assume that new MANIFEST is created after WAL recovery.
2. a new unit test is added in db_basic_test to simulate step 3.
Reviewed By: riversand963
Differential Revision: D24668144
Pulled By: cheng-chang
fbshipit-source-id: 90d7487fbad2bc3714f5ede46ea949895b15ae3b
2020-11-08 06:54:55 +01:00
|
|
|
// Set DB identity file, and write DB ID to manifest if necessary.
|
|
|
|
Status SetDBId();
|
|
|
|
|
2020-03-21 03:17:54 +01:00
|
|
|
// REQUIRES: db mutex held when calling this function, but the db mutex can
|
|
|
|
// be released and re-acquired. Db mutex will be held when the function
|
|
|
|
// returns.
|
Fix a recovery corner case (#7621)
Summary:
Consider the following sequence of events:
1. Db flushed an SST with file number N, appended to MANIFEST, and tried to sync the MANIFEST.
2. Syncing MANIFEST failed and db crashed.
3. Db tried to recover with this MANIFEST. In the meantime, no entry about the newly-flushed SST was found in the MANIFEST. Therefore, RocksDB replayed WAL and tried to flush to an SST file reusing the same file number N. This failed because file system does not support overwrite. Then Db deleted this file.
4. Db crashed again.
5. Db tried to recover. When db read the MANIFEST, there was an entry referencing N.sst. This could happen probably because the append in step 1 finally reached the MANIFEST and became visible. Since N.sst had been deleted in step 3, recovery failed.
It is possible that N.sst created in step 1 is valid. Although step 3 would still fail since the MANIFEST was not synced properly in step 1 and 2, deleting N.sst would make it impossible for the db to recover even if the remaining part of MANIFEST was appended and visible after step 5.
After this PR, in step 3, immediately after recovering from MANIFEST, a new MANIFEST is created, then we find that N.sst is not referenced in the MANIFEST, so we delete it, and we'll not reuse N as file number. Then in step 5, since the new MANIFEST does not contain N.sst, the recovery failure situation in step 5 won't happen.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7621
Test Plan:
1. some tests are updated, because these tests assume that new MANIFEST is created after WAL recovery.
2. a new unit test is added in db_basic_test to simulate step 3.
Reviewed By: riversand963
Differential Revision: D24668144
Pulled By: cheng-chang
fbshipit-source-id: 90d7487fbad2bc3714f5ede46ea949895b15ae3b
2020-11-08 06:54:55 +01:00
|
|
|
// After recovery, there may be SST files in db/cf paths that are
|
|
|
|
// not referenced in the MANIFEST (e.g.
|
|
|
|
// 1. It's best effort recovery;
|
|
|
|
// 2. The VersionEdits referencing the SST files are appended to
|
|
|
|
// MANIFEST, DB crashes when syncing the MANIFEST, the VersionEdits are
|
|
|
|
// still not synced to MANIFEST during recovery.)
|
|
|
|
// We delete these SST files. In the
|
2020-03-21 03:17:54 +01:00
|
|
|
// meantime, we find out the largest file number present in the paths, and
|
|
|
|
// bump up the version set's next_file_number_ to be 1 + largest_file_number.
|
Fix a recovery corner case (#7621)
Summary:
Consider the following sequence of events:
1. Db flushed an SST with file number N, appended to MANIFEST, and tried to sync the MANIFEST.
2. Syncing MANIFEST failed and db crashed.
3. Db tried to recover with this MANIFEST. In the meantime, no entry about the newly-flushed SST was found in the MANIFEST. Therefore, RocksDB replayed WAL and tried to flush to an SST file reusing the same file number N. This failed because file system does not support overwrite. Then Db deleted this file.
4. Db crashed again.
5. Db tried to recover. When db read the MANIFEST, there was an entry referencing N.sst. This could happen probably because the append in step 1 finally reached the MANIFEST and became visible. Since N.sst had been deleted in step 3, recovery failed.
It is possible that N.sst created in step 1 is valid. Although step 3 would still fail since the MANIFEST was not synced properly in step 1 and 2, deleting N.sst would make it impossible for the db to recover even if the remaining part of MANIFEST was appended and visible after step 5.
After this PR, in step 3, immediately after recovering from MANIFEST, a new MANIFEST is created, then we find that N.sst is not referenced in the MANIFEST, so we delete it, and we'll not reuse N as file number. Then in step 5, since the new MANIFEST does not contain N.sst, the recovery failure situation in step 5 won't happen.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/7621
Test Plan:
1. some tests are updated, because these tests assume that new MANIFEST is created after WAL recovery.
2. a new unit test is added in db_basic_test to simulate step 3.
Reviewed By: riversand963
Differential Revision: D24668144
Pulled By: cheng-chang
fbshipit-source-id: 90d7487fbad2bc3714f5ede46ea949895b15ae3b
2020-11-08 06:54:55 +01:00
|
|
|
Status DeleteUnreferencedSstFiles();
|
2020-03-21 03:17:54 +01:00
|
|
|
|
2020-06-15 19:45:03 +02:00
|
|
|
// SetDbSessionId() should be called in the constuctor DBImpl()
|
|
|
|
// to ensure that db_session_id_ gets updated every time the DB is opened
|
|
|
|
void SetDbSessionId();
|
|
|
|
|
2011-03-18 23:37:00 +01:00
|
|
|
private:
|
|
|
|
friend class DB;
|
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
|
|
|
friend class ErrorHandler;
|
2014-03-27 19:59:37 +01:00
|
|
|
friend class InternalStats;
|
2017-08-08 01:07:40 +02:00
|
|
|
friend class PessimisticTransaction;
|
2018-04-03 05:19:21 +02:00
|
|
|
friend class TransactionBaseImpl;
|
2017-08-08 01:07:40 +02:00
|
|
|
friend class WriteCommittedTxn;
|
|
|
|
friend class WritePreparedTxn;
|
2017-09-29 01:43:04 +02:00
|
|
|
friend class WritePreparedTxnDB;
|
2017-09-11 17:58:52 +02:00
|
|
|
friend class WriteBatchWithIndex;
|
2018-07-07 02:17:36 +02:00
|
|
|
friend class WriteUnpreparedTxnDB;
|
2018-07-24 09:09:18 +02:00
|
|
|
friend class WriteUnpreparedTxn;
|
|
|
|
|
2014-04-15 22:39:26 +02:00
|
|
|
#ifndef ROCKSDB_LITE
|
2014-05-30 23:31:55 +02:00
|
|
|
friend class ForwardIterator;
|
2014-04-15 22:39:26 +02:00
|
|
|
#endif
|
2014-03-09 06:12:13 +01:00
|
|
|
friend struct SuperVersion;
|
2014-09-25 20:14:01 +02:00
|
|
|
friend class CompactedDBImpl;
|
2018-03-27 01:16:59 +02:00
|
|
|
friend class DBTest_ConcurrentFlushWAL_Test;
|
2018-10-10 07:50:59 +02:00
|
|
|
friend class DBTest_MixedSlowdownOptionsStop_Test;
|
2019-03-26 03:14:04 +01:00
|
|
|
friend class DBCompactionTest_CompactBottomLevelFilesWithDeletions_Test;
|
2019-06-04 07:37:40 +02:00
|
|
|
friend class DBCompactionTest_CompactionDuringShutdown_Test;
|
2019-06-18 00:17:43 +02:00
|
|
|
friend class StatsHistoryTest_PersistentStatsCreateColumnFamilies_Test;
|
2015-05-29 23:36:35 +02:00
|
|
|
#ifndef NDEBUG
|
2017-10-03 18:08:07 +02:00
|
|
|
friend class DBTest2_ReadCallbackTest_Test;
|
2020-06-30 21:29:01 +02:00
|
|
|
friend class WriteCallbackPTest_WriteWithCallbackTest_Test;
|
2015-05-29 23:36:35 +02:00
|
|
|
friend class XFTransactionWriteHandler;
|
2017-10-03 18:08:07 +02:00
|
|
|
friend class DBBlobIndexTest;
|
2018-07-24 09:09:18 +02:00
|
|
|
friend class WriteUnpreparedTransactionTest_RecoveryTest_Test;
|
2015-05-29 23:36:35 +02:00
|
|
|
#endif
|
2019-05-31 06:29:44 +02:00
|
|
|
|
2012-03-09 01:23:21 +01:00
|
|
|
struct CompactionState;
|
2019-05-31 06:29:44 +02:00
|
|
|
struct PrepickedCompaction;
|
|
|
|
struct PurgeFileInfo;
|
2014-09-06 00:20:05 +02:00
|
|
|
|
2017-04-04 19:19:33 +02:00
|
|
|
struct WriteContext {
|
2017-10-06 03:00:38 +02:00
|
|
|
SuperVersionContext superversion_context;
|
2017-04-04 19:19:33 +02:00
|
|
|
autovector<MemTable*> memtables_to_free_;
|
|
|
|
|
2017-10-06 03:00:38 +02:00
|
|
|
explicit WriteContext(bool create_superversion = false)
|
|
|
|
: superversion_context(create_superversion) {}
|
|
|
|
|
2017-04-04 19:19:33 +02:00
|
|
|
~WriteContext() {
|
2017-10-06 03:00:38 +02:00
|
|
|
superversion_context.Clean();
|
2017-04-04 19:19:33 +02:00
|
|
|
for (auto& m : memtables_to_free_) {
|
|
|
|
delete m;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
};
|
2011-03-18 23:37:00 +01:00
|
|
|
|
2019-05-31 06:29:44 +02:00
|
|
|
struct LogFileNumberSize {
|
|
|
|
explicit LogFileNumberSize(uint64_t _number) : number(_number) {}
|
|
|
|
void AddSize(uint64_t new_size) { size += new_size; }
|
|
|
|
uint64_t number;
|
|
|
|
uint64_t size = 0;
|
|
|
|
bool getting_flushed = false;
|
|
|
|
};
|
|
|
|
|
|
|
|
struct LogWriterNumber {
|
|
|
|
// pass ownership of _writer
|
|
|
|
LogWriterNumber(uint64_t _number, log::Writer* _writer)
|
|
|
|
: number(_number), writer(_writer) {}
|
|
|
|
|
|
|
|
log::Writer* ReleaseWriter() {
|
|
|
|
auto* w = writer;
|
|
|
|
writer = nullptr;
|
|
|
|
return w;
|
|
|
|
}
|
|
|
|
Status ClearWriter() {
|
|
|
|
Status s = writer->WriteBuffer();
|
|
|
|
delete writer;
|
|
|
|
writer = nullptr;
|
|
|
|
return s;
|
|
|
|
}
|
|
|
|
|
|
|
|
uint64_t number;
|
|
|
|
// Visual Studio doesn't support deque's member to be noncopyable because
|
|
|
|
// of a std::unique_ptr as a member.
|
|
|
|
log::Writer* writer; // own
|
|
|
|
// true for some prefix of logs_
|
|
|
|
bool getting_synced = false;
|
|
|
|
};
|
|
|
|
|
|
|
|
// PurgeFileInfo is a structure to hold information of files to be deleted in
|
2019-12-18 05:07:21 +01:00
|
|
|
// purge_files_
|
2019-05-31 06:29:44 +02:00
|
|
|
struct PurgeFileInfo {
|
|
|
|
std::string fname;
|
|
|
|
std::string dir_to_sync;
|
|
|
|
FileType type;
|
|
|
|
uint64_t number;
|
|
|
|
int job_id;
|
|
|
|
PurgeFileInfo(std::string fn, std::string d, FileType t, uint64_t num,
|
|
|
|
int jid)
|
|
|
|
: fname(fn), dir_to_sync(d), type(t), number(num), job_id(jid) {}
|
|
|
|
};
|
|
|
|
|
|
|
|
// Argument required by background flush thread.
|
|
|
|
struct BGFlushArg {
|
|
|
|
BGFlushArg()
|
|
|
|
: cfd_(nullptr), max_memtable_id_(0), superversion_context_(nullptr) {}
|
|
|
|
BGFlushArg(ColumnFamilyData* cfd, uint64_t max_memtable_id,
|
|
|
|
SuperVersionContext* superversion_context)
|
|
|
|
: cfd_(cfd),
|
|
|
|
max_memtable_id_(max_memtable_id),
|
|
|
|
superversion_context_(superversion_context) {}
|
|
|
|
|
|
|
|
// Column family to flush.
|
|
|
|
ColumnFamilyData* cfd_;
|
|
|
|
// Maximum ID of memtable to flush. In this column family, memtables with
|
|
|
|
// IDs smaller than this value must be flushed before this flush completes.
|
|
|
|
uint64_t max_memtable_id_;
|
|
|
|
// Pointer to a SuperVersionContext object. After flush completes, RocksDB
|
|
|
|
// installs a new superversion for the column family. This operation
|
|
|
|
// requires a SuperVersionContext object (currently embedded in JobContext).
|
|
|
|
SuperVersionContext* superversion_context_;
|
|
|
|
};
|
|
|
|
|
|
|
|
// Argument passed to flush thread.
|
|
|
|
struct FlushThreadArg {
|
|
|
|
DBImpl* db_;
|
|
|
|
|
|
|
|
Env::Priority thread_pri_;
|
|
|
|
};
|
|
|
|
|
|
|
|
// Information for a manual compaction
|
|
|
|
struct ManualCompactionState {
|
|
|
|
ColumnFamilyData* cfd;
|
|
|
|
int input_level;
|
|
|
|
int output_level;
|
|
|
|
uint32_t output_path_id;
|
|
|
|
Status status;
|
|
|
|
bool done;
|
|
|
|
bool in_progress; // compaction request being processed?
|
|
|
|
bool incomplete; // only part of requested range compacted
|
|
|
|
bool exclusive; // current behavior of only one manual
|
|
|
|
bool disallow_trivial_move; // Force actual compaction to run
|
|
|
|
const InternalKey* begin; // nullptr means beginning of key range
|
|
|
|
const InternalKey* end; // nullptr means end of key range
|
|
|
|
InternalKey* manual_end; // how far we are compacting
|
|
|
|
InternalKey tmp_storage; // Used to keep track of compaction progress
|
|
|
|
InternalKey tmp_storage1; // Used to keep track of compaction progress
|
|
|
|
};
|
|
|
|
struct PrepickedCompaction {
|
|
|
|
// background compaction takes ownership of `compaction`.
|
|
|
|
Compaction* compaction;
|
|
|
|
// caller retains ownership of `manual_compaction_state` as it is reused
|
|
|
|
// across background compactions.
|
|
|
|
ManualCompactionState* manual_compaction_state; // nullptr if non-manual
|
|
|
|
// task limiter token is requested during compaction picking.
|
|
|
|
std::unique_ptr<TaskLimiterToken> task_token;
|
|
|
|
};
|
|
|
|
|
|
|
|
struct CompactionArg {
|
|
|
|
// caller retains ownership of `db`.
|
|
|
|
DBImpl* db;
|
|
|
|
// background compaction takes ownership of `prepicked_compaction`.
|
|
|
|
PrepickedCompaction* prepicked_compaction;
|
|
|
|
};
|
2016-06-22 03:41:23 +02:00
|
|
|
|
2019-06-18 00:17:43 +02:00
|
|
|
// Initialize the built-in column family for persistent stats. Depending on
|
|
|
|
// whether on-disk persistent stats have been enabled before, it may either
|
|
|
|
// create a new column family and column family handle or just a column family
|
|
|
|
// handle.
|
|
|
|
// Required: DB mutex held
|
|
|
|
Status InitPersistStatsColumnFamily();
|
|
|
|
|
|
|
|
// Persistent Stats column family has two format version key which are used
|
|
|
|
// for compatibility check. Write format version if it's created for the
|
|
|
|
// first time, read format version and check compatibility if recovering
|
|
|
|
// from disk. This function requires DB mutex held at entrance but may
|
|
|
|
// release and re-acquire DB mutex in the process.
|
|
|
|
// Required: DB mutex held
|
|
|
|
Status PersistentStatsProcessFormatVersion();
|
|
|
|
|
2020-09-18 05:22:35 +02:00
|
|
|
Status ResumeImpl(DBRecoverContext context);
|
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
|
|
|
|
2011-03-18 23:37:00 +01:00
|
|
|
void MaybeIgnoreError(Status* s) const;
|
|
|
|
|
2012-11-26 22:56:45 +01:00
|
|
|
const Status CreateArchivalDirectory();
|
|
|
|
|
2017-05-08 07:12:55 +02:00
|
|
|
Status CreateColumnFamilyImpl(const ColumnFamilyOptions& cf_options,
|
|
|
|
const std::string& cf_name,
|
|
|
|
ColumnFamilyHandle** handle);
|
|
|
|
|
|
|
|
Status DropColumnFamilyImpl(ColumnFamilyHandle* column_family);
|
|
|
|
|
2011-03-18 23:37:00 +01:00
|
|
|
// Delete any unneeded files and stale in-memory entries.
|
|
|
|
void DeleteObsoleteFiles();
|
2016-06-22 03:41:23 +02:00
|
|
|
// Delete obsolete files and log status and information of file deletion
|
2017-09-07 23:11:15 +02:00
|
|
|
void DeleteObsoleteFileImpl(int job_id, const std::string& fname,
|
2018-04-26 22:51:39 +02:00
|
|
|
const std::string& path_to_sync, FileType type,
|
|
|
|
uint64_t number);
|
2011-03-18 23:37:00 +01:00
|
|
|
|
2014-11-07 20:50:34 +01:00
|
|
|
// Background process needs to call
|
|
|
|
// auto x = CaptureCurrentFileNumberInPendingOutputs()
|
2016-03-04 03:25:07 +01:00
|
|
|
// auto file_num = versions_->NewFileNumber();
|
2014-11-07 20:50:34 +01:00
|
|
|
// <do something>
|
|
|
|
// ReleaseFileNumberFromPendingOutputs(x)
|
2016-03-04 03:25:07 +01:00
|
|
|
// This will protect any file with number `file_num` or greater from being
|
|
|
|
// deleted while <do something> is running.
|
2014-11-07 20:50:34 +01:00
|
|
|
// -----------
|
|
|
|
// This function will capture current file number and append it to
|
|
|
|
// pending_outputs_. This will prevent any background process to delete any
|
|
|
|
// file created after this point.
|
|
|
|
std::list<uint64_t>::iterator CaptureCurrentFileNumberInPendingOutputs();
|
|
|
|
// This function should be called with the result of
|
|
|
|
// CaptureCurrentFileNumberInPendingOutputs(). It then marks that any file
|
|
|
|
// created between the calls CaptureCurrentFileNumberInPendingOutputs() and
|
|
|
|
// ReleaseFileNumberFromPendingOutputs() can now be deleted (if it's not live
|
|
|
|
// and blocked by any other pending_outputs_ calls)
|
2019-10-08 23:18:48 +02:00
|
|
|
void ReleaseFileNumberFromPendingOutputs(
|
|
|
|
std::unique_ptr<std::list<uint64_t>::iterator>& v);
|
2014-11-07 20:50:34 +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
|
|
|
IOStatus SyncClosedLogs(JobContext* job_context);
|
2016-07-20 00:12:46 +02:00
|
|
|
|
2013-10-15 00:12:15 +02:00
|
|
|
// Flush the in-memory write buffer to storage. Switches to a new
|
2018-08-24 22:17:29 +02:00
|
|
|
// log-file/memtable and writes a new descriptor iff successful. Then
|
|
|
|
// installs a new super version for the column family.
|
2019-01-31 20:53:29 +01:00
|
|
|
Status FlushMemTableToOutputFile(
|
|
|
|
ColumnFamilyData* cfd, const MutableCFOptions& mutable_cf_options,
|
|
|
|
bool* madeProgress, JobContext* job_context,
|
|
|
|
SuperVersionContext* superversion_context,
|
|
|
|
std::vector<SequenceNumber>& snapshot_seqs,
|
|
|
|
SequenceNumber earliest_write_conflict_snapshot,
|
2019-03-20 01:24:09 +01:00
|
|
|
SnapshotChecker* snapshot_checker, LogBuffer* log_buffer,
|
|
|
|
Env::Priority thread_pri);
|
2011-03-18 23:37:00 +01:00
|
|
|
|
2018-08-24 22:17:29 +02:00
|
|
|
// Flush the memtables of (multiple) column families to multiple files on
|
|
|
|
// persistent storage.
|
|
|
|
Status FlushMemTablesToOutputFiles(
|
|
|
|
const autovector<BGFlushArg>& bg_flush_args, bool* made_progress,
|
2019-03-20 01:24:09 +01:00
|
|
|
JobContext* job_context, LogBuffer* log_buffer, Env::Priority thread_pri);
|
2018-08-24 22:17:29 +02:00
|
|
|
|
2018-10-16 04:59:20 +02:00
|
|
|
Status AtomicFlushMemTablesToOutputFiles(
|
|
|
|
const autovector<BGFlushArg>& bg_flush_args, bool* made_progress,
|
2019-03-20 01:24:09 +01:00
|
|
|
JobContext* job_context, LogBuffer* log_buffer, Env::Priority thread_pri);
|
2018-10-16 04:59:20 +02:00
|
|
|
|
2014-09-09 20:18:50 +02:00
|
|
|
// REQUIRES: log_numbers are sorted in ascending order
|
2020-01-29 20:39:11 +01:00
|
|
|
// corrupted_log_found is set to true if we recover from a corrupted log file.
|
2019-05-18 04:16:51 +02:00
|
|
|
Status RecoverLogFiles(const std::vector<uint64_t>& log_numbers,
|
2020-01-29 20:39:11 +01:00
|
|
|
SequenceNumber* next_sequence, bool read_only,
|
|
|
|
bool* corrupted_log_found);
|
2011-03-18 23:37:00 +01:00
|
|
|
|
2012-10-19 23:00:53 +02:00
|
|
|
// The following two methods are used to flush a memtable to
|
2016-03-25 03:39:13 +01:00
|
|
|
// storage. The first one is used at database RecoveryTime (when the
|
2012-10-19 23:00:53 +02:00
|
|
|
// database is opened) and is heavyweight because it holds the mutex
|
|
|
|
// for the entire period. The second method WriteLevel0Table supports
|
|
|
|
// concurrent flush memtables to storage.
|
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
|
|
|
Status WriteLevel0TableForRecovery(int job_id, ColumnFamilyData* cfd,
|
|
|
|
MemTable* mem, VersionEdit* edit);
|
2015-05-16 00:52:51 +02:00
|
|
|
|
2018-09-26 19:34:56 +02:00
|
|
|
// Restore alive_log_files_ and total_log_size_ after recovery.
|
|
|
|
// It needs to run only when there's no flush during recovery
|
|
|
|
// (e.g. avoid_flush_during_recovery=true). May also trigger flush
|
|
|
|
// in case total_log_size > max_total_wal_size.
|
|
|
|
Status RestoreAliveLogFiles(const std::vector<uint64_t>& log_numbers);
|
|
|
|
|
2015-05-16 00:52:51 +02:00
|
|
|
// num_bytes: for slowdown case, delay time is calculated based on
|
|
|
|
// `num_bytes` going through.
|
2016-11-22 02:56:19 +01:00
|
|
|
Status DelayWrite(uint64_t num_bytes, const WriteOptions& write_options);
|
Push- instead of pull-model for managing Write stalls
Summary:
Introducing WriteController, which is a source of truth about per-DB write delays. Let's define an DB epoch as a period where there are no flushes and compactions (i.e. new epoch is started when flush or compaction finishes). Each epoch can either:
* proceed with all writes without delay
* delay all writes by fixed time
* stop all writes
The three modes are recomputed at each epoch change (flush, compaction), rather than on every write (which is currently the case).
When we have a lot of column families, our current pull behavior adds a big overhead, since we need to loop over every column family for every write. With new push model, overhead on Write code-path is minimal.
This is just the start. Next step is to also take care of stalls introduced by slow memtable flushes. The final goal is to eliminate function MakeRoomForWrite(), which currently needs to be called for every column family by every write.
Test Plan: make check for now. I'll add some unit tests later. Also, perf test.
Reviewers: dhruba, yhchiang, MarkCallaghan, sdong, ljin
Reviewed By: ljin
Subscribers: leveldb
Differential Revision: https://reviews.facebook.net/D22791
2014-09-08 20:20:25 +02:00
|
|
|
|
2017-06-05 23:42:34 +02:00
|
|
|
Status ThrottleLowPriWritesIfNeeded(const WriteOptions& write_options,
|
|
|
|
WriteBatch* my_batch);
|
|
|
|
|
2020-02-18 22:52:09 +01:00
|
|
|
// REQUIRES: mutex locked and in write thread.
|
2014-09-11 03:46:09 +02:00
|
|
|
Status ScheduleFlushes(WriteContext* context);
|
2014-04-07 20:29:48 +02:00
|
|
|
|
2019-07-01 20:53:25 +02:00
|
|
|
void MaybeFlushStatsCF(autovector<ColumnFamilyData*>* cfds);
|
|
|
|
|
Refactor trimming logic for immutable memtables (#5022)
Summary:
MyRocks currently sets `max_write_buffer_number_to_maintain` in order to maintain enough history for transaction conflict checking. The effectiveness of this approach depends on the size of memtables. When memtables are small, it may not keep enough history; when memtables are large, this may consume too much memory.
We are proposing a new way to configure memtable list history: by limiting the memory usage of immutable memtables. The new option is `max_write_buffer_size_to_maintain` and it will take precedence over the old `max_write_buffer_number_to_maintain` if they are both set to non-zero values. The new option accounts for the total memory usage of flushed immutable memtables and mutable memtable. When the total usage exceeds the limit, RocksDB may start dropping immutable memtables (which is also called trimming history), starting from the oldest one.
The semantics of the old option actually works both as an upper bound and lower bound. History trimming will start if number of immutable memtables exceeds the limit, but it will never go below (limit-1) due to history trimming.
In order the mimic the behavior with the new option, history trimming will stop if dropping the next immutable memtable causes the total memory usage go below the size limit. For example, assuming the size limit is set to 64MB, and there are 3 immutable memtables with sizes of 20, 30, 30. Although the total memory usage is 80MB > 64MB, dropping the oldest memtable will reduce the memory usage to 60MB < 64MB, so in this case no memtable will be dropped.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5022
Differential Revision: D14394062
Pulled By: miasantreble
fbshipit-source-id: 60457a509c6af89d0993f988c9b5c2aa9e45f5c5
2019-08-23 22:54:09 +02:00
|
|
|
Status TrimMemtableHistory(WriteContext* context);
|
|
|
|
|
2018-08-24 22:17:29 +02:00
|
|
|
Status SwitchMemtable(ColumnFamilyData* cfd, WriteContext* context);
|
2014-08-12 07:10:32 +02:00
|
|
|
|
2018-10-27 00:06:44 +02:00
|
|
|
void SelectColumnFamiliesForAtomicFlush(autovector<ColumnFamilyData*>* cfds);
|
|
|
|
|
2012-07-06 20:42:09 +02:00
|
|
|
// Force current memtable contents to be flushed.
|
2016-10-21 02:05:32 +02:00
|
|
|
Status FlushMemTable(ColumnFamilyData* cfd, const FlushOptions& options,
|
2018-02-09 21:09:55 +01:00
|
|
|
FlushReason flush_reason, bool writes_stopped = false);
|
2012-07-06 20:42:09 +02:00
|
|
|
|
2018-10-27 00:06:44 +02:00
|
|
|
Status AtomicFlushMemTables(
|
|
|
|
const autovector<ColumnFamilyData*>& column_family_datas,
|
|
|
|
const FlushOptions& options, FlushReason flush_reason,
|
|
|
|
bool writes_stopped = false);
|
|
|
|
|
2018-08-29 20:58:13 +02:00
|
|
|
// Wait until flushing this column family won't stall writes
|
|
|
|
Status WaitUntilFlushWouldNotStallWrites(ColumnFamilyData* cfd,
|
|
|
|
bool* flush_needed);
|
|
|
|
|
2018-01-19 02:32:50 +01:00
|
|
|
// Wait for memtable flushed.
|
|
|
|
// If flush_memtable_id is non-null, wait until the memtable with the ID
|
|
|
|
// gets flush. Otherwise, wait until the column family don't have any
|
|
|
|
// memtable pending flush.
|
2018-10-27 00:06:44 +02:00
|
|
|
// resuming_from_bg_err indicates whether the caller is attempting to resume
|
|
|
|
// from background error.
|
2018-01-19 02:32:50 +01:00
|
|
|
Status WaitForFlushMemTable(ColumnFamilyData* cfd,
|
2018-10-27 00:06:44 +02:00
|
|
|
const uint64_t* flush_memtable_id = nullptr,
|
|
|
|
bool resuming_from_bg_err = false) {
|
|
|
|
return WaitForFlushMemTables({cfd}, {flush_memtable_id},
|
|
|
|
resuming_from_bg_err);
|
2018-08-24 22:17:29 +02:00
|
|
|
}
|
|
|
|
// Wait for memtables to be flushed for multiple column families.
|
|
|
|
Status WaitForFlushMemTables(
|
|
|
|
const autovector<ColumnFamilyData*>& cfds,
|
2018-10-27 00:06:44 +02:00
|
|
|
const autovector<const uint64_t*>& flush_memtable_ids,
|
|
|
|
bool resuming_from_bg_err);
|
|
|
|
|
2019-05-14 02:43:47 +02:00
|
|
|
inline void WaitForPendingWrites() {
|
2019-10-30 02:15:08 +01:00
|
|
|
mutex_.AssertHeld();
|
2019-12-12 23:05:48 +01:00
|
|
|
TEST_SYNC_POINT("DBImpl::WaitForPendingWrites:BeforeBlock");
|
2019-10-30 02:15:08 +01:00
|
|
|
// In case of pipelined write is enabled, wait for all pending memtable
|
|
|
|
// writers.
|
|
|
|
if (immutable_db_options_.enable_pipelined_write) {
|
|
|
|
// Memtable writers may call DB::Get in case max_successive_merges > 0,
|
|
|
|
// which may lock mutex. Unlocking mutex here to avoid deadlock.
|
|
|
|
mutex_.Unlock();
|
|
|
|
write_thread_.WaitForMemTableWriters();
|
|
|
|
mutex_.Lock();
|
|
|
|
}
|
|
|
|
|
2019-05-14 02:43:47 +02:00
|
|
|
if (!immutable_db_options_.unordered_write) {
|
|
|
|
// Then the writes are finished before the next write group starts
|
|
|
|
return;
|
|
|
|
}
|
2019-10-30 02:15:08 +01:00
|
|
|
|
2019-05-14 02:43:47 +02:00
|
|
|
// Wait for the ones who already wrote to the WAL to finish their
|
|
|
|
// memtable write.
|
|
|
|
if (pending_memtable_writes_.load() != 0) {
|
|
|
|
std::unique_lock<std::mutex> guard(switch_mutex_);
|
|
|
|
switch_cv_.wait(guard,
|
|
|
|
[&] { return pending_memtable_writes_.load() == 0; });
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-10-27 00:06:44 +02:00
|
|
|
// REQUIRES: mutex locked and in write thread.
|
|
|
|
void AssignAtomicFlushSeq(const autovector<ColumnFamilyData*>& cfds);
|
2012-07-06 20:42:09 +02:00
|
|
|
|
2020-02-18 22:52:09 +01:00
|
|
|
// REQUIRES: mutex locked and in write thread.
|
2017-09-28 02:37:08 +02:00
|
|
|
Status SwitchWAL(WriteContext* write_context);
|
2017-04-04 19:19:33 +02:00
|
|
|
|
2020-02-18 22:52:09 +01:00
|
|
|
// REQUIRES: mutex locked and in write thread.
|
2017-04-04 19:19:33 +02:00
|
|
|
Status HandleWriteBufferFull(WriteContext* write_context);
|
|
|
|
|
|
|
|
// REQUIRES: mutex locked
|
2017-05-19 23:24:23 +02:00
|
|
|
Status PreprocessWrite(const WriteOptions& write_options, bool* need_log_sync,
|
|
|
|
WriteContext* write_context);
|
2017-04-04 19:19:33 +02:00
|
|
|
|
2017-06-24 23:06:43 +02:00
|
|
|
WriteBatch* MergeBatch(const WriteThread::WriteGroup& write_group,
|
2017-11-02 01:23:52 +01:00
|
|
|
WriteBatch* tmp_batch, size_t* write_with_wal,
|
|
|
|
WriteBatch** to_be_cached_state);
|
2017-06-24 23:06:43 +02: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
|
|
|
IOStatus WriteToWAL(const WriteBatch& merged_batch, log::Writer* log_writer,
|
|
|
|
uint64_t* log_used, uint64_t* log_size);
|
2017-06-24 23:06:43 +02: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
|
|
|
IOStatus WriteToWAL(const WriteThread::WriteGroup& write_group,
|
|
|
|
log::Writer* log_writer, uint64_t* log_used,
|
|
|
|
bool need_log_sync, bool need_log_dir_sync,
|
|
|
|
SequenceNumber sequence);
|
2017-06-24 23:06:43 +02: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
|
|
|
IOStatus ConcurrentWriteToWAL(const WriteThread::WriteGroup& write_group,
|
|
|
|
uint64_t* log_used,
|
|
|
|
SequenceNumber* last_sequence, size_t seq_inc);
|
2017-04-04 19:19:33 +02:00
|
|
|
|
2020-08-07 20:27:14 +02:00
|
|
|
// Used by WriteImpl to update bg_error_ if paranoid check is enabled.
|
|
|
|
// Caller must hold mutex_.
|
|
|
|
void WriteStatusCheckOnLocked(const Status& status);
|
|
|
|
|
2017-05-19 23:24:23 +02:00
|
|
|
// Used by WriteImpl to update bg_error_ if paranoid check is enabled.
|
2018-03-22 23:56:52 +01:00
|
|
|
void WriteStatusCheck(const Status& status);
|
2017-05-19 23:24:23 +02: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
|
|
|
// Used by WriteImpl to update bg_error_ when IO error happens, e.g., write
|
|
|
|
// WAL, sync WAL fails, if paranoid check is enabled.
|
|
|
|
void IOStatusCheck(const IOStatus& status);
|
|
|
|
|
2017-05-19 23:24:23 +02:00
|
|
|
// Used by WriteImpl to update bg_error_ in case of memtable insert error.
|
|
|
|
void MemTableInsertStatusCheck(const Status& memtable_insert_status);
|
2017-04-13 23:46:25 +02:00
|
|
|
|
2014-11-13 22:45:33 +01:00
|
|
|
#ifndef ROCKSDB_LITE
|
2016-09-26 23:44:48 +02:00
|
|
|
|
2016-09-29 00:42:06 +02:00
|
|
|
Status CompactFilesImpl(const CompactionOptions& compact_options,
|
|
|
|
ColumnFamilyData* cfd, Version* version,
|
|
|
|
const std::vector<std::string>& input_file_names,
|
2018-03-15 19:46:16 +01:00
|
|
|
std::vector<std::string>* const output_file_names,
|
2016-09-29 00:42:06 +02:00
|
|
|
const int output_level, int output_path_id,
|
2018-12-13 23:12:02 +01:00
|
|
|
JobContext* job_context, LogBuffer* log_buffer,
|
|
|
|
CompactionJobInfo* compaction_job_info);
|
2016-09-29 00:42:06 +02:00
|
|
|
|
2016-10-21 02:05:32 +02:00
|
|
|
// Wait for current IngestExternalFile() calls to finish.
|
|
|
|
// REQUIRES: mutex_ held
|
|
|
|
void WaitForIngestFile();
|
2016-07-11 19:43:12 +02:00
|
|
|
|
2016-09-26 23:44:48 +02:00
|
|
|
#else
|
2016-10-21 02:05:32 +02:00
|
|
|
// IngestExternalFile is not supported in ROCKSDB_LITE so this function
|
2016-09-26 23:44:48 +02:00
|
|
|
// will be no-op
|
2016-10-21 02:05:32 +02:00
|
|
|
void WaitForIngestFile() {}
|
2014-11-13 22:45:33 +01:00
|
|
|
#endif // ROCKSDB_LITE
|
CompactFiles, EventListener and GetDatabaseMetaData
Summary:
This diff adds three sets of APIs to RocksDB.
= GetColumnFamilyMetaData =
* This APIs allow users to obtain the current state of a RocksDB instance on one column family.
* See GetColumnFamilyMetaData in include/rocksdb/db.h
= EventListener =
* A virtual class that allows users to implement a set of
call-back functions which will be called when specific
events of a RocksDB instance happens.
* To register EventListener, simply insert an EventListener to ColumnFamilyOptions::listeners
= CompactFiles =
* CompactFiles API inputs a set of file numbers and an output level, and RocksDB
will try to compact those files into the specified level.
= Example =
* Example code can be found in example/compact_files_example.cc, which implements
a simple external compactor using EventListener, GetColumnFamilyMetaData, and
CompactFiles API.
Test Plan:
listener_test
compactor_test
example/compact_files_example
export ROCKSDB_TESTS=CompactFiles
db_test
export ROCKSDB_TESTS=MetaData
db_test
Reviewers: ljin, igor, rven, sdong
Reviewed By: sdong
Subscribers: MarkCallaghan, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D24705
2014-11-07 23:45:18 +01:00
|
|
|
|
|
|
|
ColumnFamilyData* GetColumnFamilyDataByName(const std::string& cf_name);
|
|
|
|
|
2013-10-15 00:12:15 +02:00
|
|
|
void MaybeScheduleFlushOrCompaction();
|
2018-08-24 22:17:29 +02:00
|
|
|
|
|
|
|
// A flush request specifies the column families to flush as well as the
|
|
|
|
// largest memtable id to persist for each column family. Once all the
|
|
|
|
// memtables whose IDs are smaller than or equal to this per-column-family
|
|
|
|
// specified value, this flush request is considered to have completed its
|
|
|
|
// work of flushing this column family. After completing the work for all
|
|
|
|
// column families in this request, this flush is considered complete.
|
|
|
|
typedef std::vector<std::pair<ColumnFamilyData*, uint64_t>> FlushRequest;
|
|
|
|
|
2018-10-27 00:06:44 +02:00
|
|
|
void GenerateFlushRequest(const autovector<ColumnFamilyData*>& cfds,
|
|
|
|
FlushRequest* req);
|
|
|
|
|
2018-08-24 22:17:29 +02:00
|
|
|
void SchedulePendingFlush(const FlushRequest& req, FlushReason flush_reason);
|
|
|
|
|
Rewritten system for scheduling background work
Summary:
When scaling to higher number of column families, the worst bottleneck was MaybeScheduleFlushOrCompaction(), which did a for loop over all column families while holding a mutex. This patch addresses the issue.
The approach is similar to our earlier efforts: instead of a pull-model, where we do something for every column family, we can do a push-based model -- when we detect that column family is ready to be flushed/compacted, we add it to the flush_queue_/compaction_queue_. That way we don't need to loop over every column family in MaybeScheduleFlushOrCompaction.
Here are the performance results:
Command:
./db_bench --write_buffer_size=268435456 --db_write_buffer_size=268435456 --db=/fast-rocksdb-tmp/rocks_lots_of_cf --use_existing_db=0 --open_files=55000 --statistics=1 --histogram=1 --disable_data_sync=1 --max_write_buffer_number=2 --sync=0 --benchmarks=fillrandom --threads=16 --num_column_families=5000 --disable_wal=1 --max_background_flushes=16 --max_background_compactions=16 --level0_file_num_compaction_trigger=2 --level0_slowdown_writes_trigger=2 --level0_stop_writes_trigger=3 --hard_rate_limit=1 --num=33333333 --writes=33333333
Before the patch:
fillrandom : 26.950 micros/op 37105 ops/sec; 4.1 MB/s
After the patch:
fillrandom : 17.404 micros/op 57456 ops/sec; 6.4 MB/s
Next bottleneck is VersionSet::AddLiveFiles, which is painfully slow when we have a lot of files. This is coming in the next patch, but when I removed that code, here's what I got:
fillrandom : 7.590 micros/op 131758 ops/sec; 14.6 MB/s
Test Plan:
make check
two stress tests:
Big number of compactions and flushes:
./db_stress --threads=30 --ops_per_thread=20000000 --max_key=10000 --column_families=20 --clear_column_family_one_in=10000000 --verify_before_write=0 --reopen=15 --max_background_compactions=10 --max_background_flushes=10 --db=/fast-rocksdb-tmp/db_stress --prefixpercent=0 --iterpercent=0 --writepercent=75 --db_write_buffer_size=2000000
max_background_flushes=0, to verify that this case also works correctly
./db_stress --threads=30 --ops_per_thread=2000000 --max_key=10000 --column_families=20 --clear_column_family_one_in=10000000 --verify_before_write=0 --reopen=3 --max_background_compactions=3 --max_background_flushes=0 --db=/fast-rocksdb-tmp/db_stress --prefixpercent=0 --iterpercent=0 --writepercent=75 --db_write_buffer_size=2000000
Reviewers: ljin, rven, yhchiang, sdong
Reviewed By: sdong
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D30123
2014-12-19 20:38:12 +01:00
|
|
|
void SchedulePendingCompaction(ColumnFamilyData* cfd);
|
2018-04-26 22:51:39 +02:00
|
|
|
void SchedulePendingPurge(std::string fname, std::string dir_to_sync,
|
|
|
|
FileType type, uint64_t number, int job_id);
|
Running manual compactions in parallel with other automatic or manual compactions in restricted cases
Summary:
This diff provides a framework for doing manual
compactions in parallel with other compactions. We now have a deque of manual compactions. We also pass manual compactions as an argument from RunManualCompactions down to
BackgroundCompactions, so that RunManualCompactions can be reentrant.
Parallelism is controlled by the two routines
ConflictingManualCompaction to allow/disallow new parallel/manual
compactions based on already existing ManualCompactions. In this diff, by default manual compactions still have to run exclusive of other compactions. However, by setting the compaction option, exclusive_manual_compaction to false, it is possible to run other compactions in parallel with a manual compaction. However, we are still restricted to one manual compaction per column family at a time. All of these restrictions will be relaxed in future diffs.
I will be adding more tests later.
Test Plan: Rocksdb regression + new tests + valgrind
Reviewers: igor, anthony, IslamAbdelRahman, kradhakrishnan, yhchiang, sdong
Reviewed By: sdong
Subscribers: yoshinorim, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D47973
2015-12-14 20:20:34 +01:00
|
|
|
static void BGWorkCompaction(void* arg);
|
2017-08-04 00:36:28 +02:00
|
|
|
// Runs a pre-chosen universal compaction involving bottom level in a
|
|
|
|
// separate, bottom-pri thread pool.
|
|
|
|
static void BGWorkBottomCompaction(void* arg);
|
2019-03-20 01:24:09 +01:00
|
|
|
static void BGWorkFlush(void* arg);
|
2016-06-22 03:41:23 +02:00
|
|
|
static void BGWorkPurge(void* arg);
|
2019-03-20 01:24:09 +01:00
|
|
|
static void UnscheduleCompactionCallback(void* arg);
|
|
|
|
static void UnscheduleFlushCallback(void* arg);
|
2017-08-04 00:36:28 +02:00
|
|
|
void BackgroundCallCompaction(PrepickedCompaction* prepicked_compaction,
|
2019-03-20 01:24:09 +01:00
|
|
|
Env::Priority thread_pri);
|
|
|
|
void BackgroundCallFlush(Env::Priority thread_pri);
|
2016-06-22 03:41:23 +02:00
|
|
|
void BackgroundCallPurge();
|
2014-10-28 19:54:33 +01:00
|
|
|
Status BackgroundCompaction(bool* madeProgress, JobContext* job_context,
|
2017-08-04 00:36:28 +02:00
|
|
|
LogBuffer* log_buffer,
|
2019-03-20 01:24:09 +01:00
|
|
|
PrepickedCompaction* prepicked_compaction,
|
|
|
|
Env::Priority thread_pri);
|
2014-10-28 19:54:33 +01:00
|
|
|
Status BackgroundFlush(bool* madeProgress, JobContext* job_context,
|
2019-03-20 01:24:09 +01:00
|
|
|
LogBuffer* log_buffer, FlushReason* reason,
|
|
|
|
Env::Priority thread_pri);
|
2011-03-18 23:37:00 +01:00
|
|
|
|
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
|
|
|
bool EnoughRoomForCompaction(ColumnFamilyData* cfd,
|
|
|
|
const std::vector<CompactionInputFiles>& inputs,
|
2018-04-03 04:53:19 +02:00
|
|
|
bool* sfm_bookkeeping, LogBuffer* log_buffer);
|
|
|
|
|
Concurrent task limiter for compaction thread control (#4332)
Summary:
The PR is targeting to resolve the issue of:
https://github.com/facebook/rocksdb/issues/3972#issue-330771918
We have a rocksdb created with leveled-compaction with multiple column families (CFs), some of CFs are using HDD to store big and less frequently accessed data and others are using SSD.
When there are continuously write traffics going on to all CFs, the compaction thread pool is mostly occupied by those slow HDD compactions, which blocks fully utilize SSD bandwidth.
Since atomic write and transaction is needed across CFs, so splitting it to multiple rocksdb instance is not an option for us.
With the compaction thread control, we got 30%+ HDD write throughput gain, and also a lot smooth SSD write since less write stall happening.
ConcurrentTaskLimiter can be shared with multi-CFs across rocksdb instances, so the feature does not only work for multi-CFs scenarios, but also for multi-rocksdbs scenarios, who need disk IO resource control per tenant.
The usage is straight forward:
e.g.:
//
// Enable compaction thread limiter thru ColumnFamilyOptions
//
std::shared_ptr<ConcurrentTaskLimiter> ctl(NewConcurrentTaskLimiter("foo_limiter", 4));
Options options;
ColumnFamilyOptions cf_opt(options);
cf_opt.compaction_thread_limiter = ctl;
...
//
// Compaction thread limiter can be tuned or disabled on-the-fly
//
ctl->SetMaxOutstandingTask(12); // enlarge to 12 tasks
...
ctl->ResetMaxOutstandingTask(); // disable (bypass) thread limiter
ctl->SetMaxOutstandingTask(-1); // Same as above
...
ctl->SetMaxOutstandingTask(0); // full throttle (0 task)
//
// Sharing compaction thread limiter among CFs (to resolve multiple storage perf issue)
//
std::shared_ptr<ConcurrentTaskLimiter> ctl_ssd(NewConcurrentTaskLimiter("ssd_limiter", 8));
std::shared_ptr<ConcurrentTaskLimiter> ctl_hdd(NewConcurrentTaskLimiter("hdd_limiter", 4));
Options options;
ColumnFamilyOptions cf_opt_ssd1(options);
ColumnFamilyOptions cf_opt_ssd2(options);
ColumnFamilyOptions cf_opt_hdd1(options);
ColumnFamilyOptions cf_opt_hdd2(options);
ColumnFamilyOptions cf_opt_hdd3(options);
// SSD CFs
cf_opt_ssd1.compaction_thread_limiter = ctl_ssd;
cf_opt_ssd2.compaction_thread_limiter = ctl_ssd;
// HDD CFs
cf_opt_hdd1.compaction_thread_limiter = ctl_hdd;
cf_opt_hdd2.compaction_thread_limiter = ctl_hdd;
cf_opt_hdd3.compaction_thread_limiter = ctl_hdd;
...
//
// The limiter is disabled by default (or set to nullptr explicitly)
//
Options options;
ColumnFamilyOptions cf_opt(options);
cf_opt.compaction_thread_limiter = nullptr;
Pull Request resolved: https://github.com/facebook/rocksdb/pull/4332
Differential Revision: D13226590
Pulled By: siying
fbshipit-source-id: 14307aec55b8bd59c8223d04aa6db3c03d1b0c1d
2018-12-13 22:16:04 +01:00
|
|
|
// Request compaction tasks token from compaction thread limiter.
|
|
|
|
// It always succeeds if force = true or limiter is disable.
|
|
|
|
bool RequestCompactionToken(ColumnFamilyData* cfd, bool force,
|
|
|
|
std::unique_ptr<TaskLimiterToken>* token,
|
|
|
|
LogBuffer* log_buffer);
|
|
|
|
|
move dump stats to a separate thread (#4382)
Summary:
Currently statistics are supposed to be dumped to info log at intervals of `options.stats_dump_period_sec`. However the implementation choice was to bind it with compaction thread, meaning if the database has been serving very light traffic, the stats may not get dumped at all.
We decided to separate stats dumping into a new timed thread using `TimerQueue`, which is already used in blob_db. This will allow us schedule new timed tasks with more deterministic behavior.
Tested with db_bench using `--stats_dump_period_sec=20` in command line:
> LOG:2018/09/17-14:07:45.575025 7fe99fbfe700 [WARN] [db/db_impl.cc:605] ------- DUMPING STATS -------
LOG:2018/09/17-14:08:05.643286 7fe99fbfe700 [WARN] [db/db_impl.cc:605] ------- DUMPING STATS -------
LOG:2018/09/17-14:08:25.691325 7fe99fbfe700 [WARN] [db/db_impl.cc:605] ------- DUMPING STATS -------
LOG:2018/09/17-14:08:45.740989 7fe99fbfe700 [WARN] [db/db_impl.cc:605] ------- DUMPING STATS -------
LOG content:
> 2018/09/17-14:07:45.575025 7fe99fbfe700 [WARN] [db/db_impl.cc:605] ------- DUMPING STATS -------
2018/09/17-14:07:45.575080 7fe99fbfe700 [WARN] [db/db_impl.cc:606]
** DB Stats **
Uptime(secs): 20.0 total, 20.0 interval
Cumulative writes: 4447K writes, 4447K keys, 4447K commit groups, 1.0 writes per commit group, ingest: 5.57 GB, 285.01 MB/s
Cumulative WAL: 4447K writes, 0 syncs, 4447638.00 writes per sync, written: 5.57 GB, 285.01 MB/s
Cumulative stall: 00:00:0.012 H:M:S, 0.1 percent
Interval writes: 4447K writes, 4447K keys, 4447K commit groups, 1.0 writes per commit group, ingest: 5700.71 MB, 285.01 MB/s
Interval WAL: 4447K writes, 0 syncs, 4447638.00 writes per sync, written: 5.57 MB, 285.01 MB/s
Interval stall: 00:00:0.012 H:M:S, 0.1 percent
** Compaction Stats [default] **
Level Files Size Score Read(GB) Rn(GB) Rnp1(GB) Write(GB) Wnew(GB) Moved(GB) W-Amp Rd(MB/s) Wr(MB/s) Comp(sec) Comp(cnt) Avg(sec) KeyIn KeyDrop
Pull Request resolved: https://github.com/facebook/rocksdb/pull/4382
Differential Revision: D9933051
Pulled By: miasantreble
fbshipit-source-id: 6d12bb1e4977674eea4bf2d2ac6d486b814bb2fa
2018-10-09 07:52:58 +02:00
|
|
|
// Schedule background tasks
|
2020-10-02 04:12:26 +02:00
|
|
|
void StartPeriodicWorkScheduler();
|
move dump stats to a separate thread (#4382)
Summary:
Currently statistics are supposed to be dumped to info log at intervals of `options.stats_dump_period_sec`. However the implementation choice was to bind it with compaction thread, meaning if the database has been serving very light traffic, the stats may not get dumped at all.
We decided to separate stats dumping into a new timed thread using `TimerQueue`, which is already used in blob_db. This will allow us schedule new timed tasks with more deterministic behavior.
Tested with db_bench using `--stats_dump_period_sec=20` in command line:
> LOG:2018/09/17-14:07:45.575025 7fe99fbfe700 [WARN] [db/db_impl.cc:605] ------- DUMPING STATS -------
LOG:2018/09/17-14:08:05.643286 7fe99fbfe700 [WARN] [db/db_impl.cc:605] ------- DUMPING STATS -------
LOG:2018/09/17-14:08:25.691325 7fe99fbfe700 [WARN] [db/db_impl.cc:605] ------- DUMPING STATS -------
LOG:2018/09/17-14:08:45.740989 7fe99fbfe700 [WARN] [db/db_impl.cc:605] ------- DUMPING STATS -------
LOG content:
> 2018/09/17-14:07:45.575025 7fe99fbfe700 [WARN] [db/db_impl.cc:605] ------- DUMPING STATS -------
2018/09/17-14:07:45.575080 7fe99fbfe700 [WARN] [db/db_impl.cc:606]
** DB Stats **
Uptime(secs): 20.0 total, 20.0 interval
Cumulative writes: 4447K writes, 4447K keys, 4447K commit groups, 1.0 writes per commit group, ingest: 5.57 GB, 285.01 MB/s
Cumulative WAL: 4447K writes, 0 syncs, 4447638.00 writes per sync, written: 5.57 GB, 285.01 MB/s
Cumulative stall: 00:00:0.012 H:M:S, 0.1 percent
Interval writes: 4447K writes, 4447K keys, 4447K commit groups, 1.0 writes per commit group, ingest: 5700.71 MB, 285.01 MB/s
Interval WAL: 4447K writes, 0 syncs, 4447638.00 writes per sync, written: 5.57 MB, 285.01 MB/s
Interval stall: 00:00:0.012 H:M:S, 0.1 percent
** Compaction Stats [default] **
Level Files Size Score Read(GB) Rn(GB) Rnp1(GB) Write(GB) Wnew(GB) Moved(GB) W-Amp Rd(MB/s) Wr(MB/s) Comp(sec) Comp(cnt) Avg(sec) KeyIn KeyDrop
Pull Request resolved: https://github.com/facebook/rocksdb/pull/4382
Differential Revision: D9933051
Pulled By: miasantreble
fbshipit-source-id: 6d12bb1e4977674eea4bf2d2ac6d486b814bb2fa
2018-10-09 07:52:58 +02:00
|
|
|
|
2013-05-28 21:35:43 +02:00
|
|
|
void PrintStatistics();
|
|
|
|
|
2019-06-18 00:17:43 +02:00
|
|
|
size_t EstimateInMemoryStatsHistorySize() const;
|
2019-02-21 00:46:59 +01:00
|
|
|
|
2013-06-30 08:21:36 +02:00
|
|
|
// Return the minimum empty level that could hold the total data in the
|
|
|
|
// input level. Return the input level, if such level could not be found.
|
2014-10-02 01:19:16 +02:00
|
|
|
int FindMinimumEmptyLevelFitting(ColumnFamilyData* cfd,
|
2019-03-28 00:13:08 +01:00
|
|
|
const MutableCFOptions& mutable_cf_options,
|
|
|
|
int level);
|
2013-06-30 08:21:36 +02:00
|
|
|
|
2013-09-04 22:13:08 +02:00
|
|
|
// Move the files in the input level to the target level.
|
|
|
|
// If target_level < 0, automatically calculate the minimum level that could
|
|
|
|
// hold the data set.
|
2014-02-01 01:45:20 +01:00
|
|
|
Status ReFitLevel(ColumnFamilyData* cfd, int level, int target_level = -1);
|
2013-06-30 08:21:36 +02:00
|
|
|
|
Rewritten system for scheduling background work
Summary:
When scaling to higher number of column families, the worst bottleneck was MaybeScheduleFlushOrCompaction(), which did a for loop over all column families while holding a mutex. This patch addresses the issue.
The approach is similar to our earlier efforts: instead of a pull-model, where we do something for every column family, we can do a push-based model -- when we detect that column family is ready to be flushed/compacted, we add it to the flush_queue_/compaction_queue_. That way we don't need to loop over every column family in MaybeScheduleFlushOrCompaction.
Here are the performance results:
Command:
./db_bench --write_buffer_size=268435456 --db_write_buffer_size=268435456 --db=/fast-rocksdb-tmp/rocks_lots_of_cf --use_existing_db=0 --open_files=55000 --statistics=1 --histogram=1 --disable_data_sync=1 --max_write_buffer_number=2 --sync=0 --benchmarks=fillrandom --threads=16 --num_column_families=5000 --disable_wal=1 --max_background_flushes=16 --max_background_compactions=16 --level0_file_num_compaction_trigger=2 --level0_slowdown_writes_trigger=2 --level0_stop_writes_trigger=3 --hard_rate_limit=1 --num=33333333 --writes=33333333
Before the patch:
fillrandom : 26.950 micros/op 37105 ops/sec; 4.1 MB/s
After the patch:
fillrandom : 17.404 micros/op 57456 ops/sec; 6.4 MB/s
Next bottleneck is VersionSet::AddLiveFiles, which is painfully slow when we have a lot of files. This is coming in the next patch, but when I removed that code, here's what I got:
fillrandom : 7.590 micros/op 131758 ops/sec; 14.6 MB/s
Test Plan:
make check
two stress tests:
Big number of compactions and flushes:
./db_stress --threads=30 --ops_per_thread=20000000 --max_key=10000 --column_families=20 --clear_column_family_one_in=10000000 --verify_before_write=0 --reopen=15 --max_background_compactions=10 --max_background_flushes=10 --db=/fast-rocksdb-tmp/db_stress --prefixpercent=0 --iterpercent=0 --writepercent=75 --db_write_buffer_size=2000000
max_background_flushes=0, to verify that this case also works correctly
./db_stress --threads=30 --ops_per_thread=2000000 --max_key=10000 --column_families=20 --clear_column_family_one_in=10000000 --verify_before_write=0 --reopen=3 --max_background_compactions=3 --max_background_flushes=0 --db=/fast-rocksdb-tmp/db_stress --prefixpercent=0 --iterpercent=0 --writepercent=75 --db_write_buffer_size=2000000
Reviewers: ljin, rven, yhchiang, sdong
Reviewed By: sdong
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D30123
2014-12-19 20:38:12 +01:00
|
|
|
// helper functions for adding and removing from flush & compaction queues
|
|
|
|
void AddToCompactionQueue(ColumnFamilyData* cfd);
|
|
|
|
ColumnFamilyData* PopFirstFromCompactionQueue();
|
2018-08-24 22:17:29 +02:00
|
|
|
FlushRequest PopFirstFromFlushQueue();
|
Rewritten system for scheduling background work
Summary:
When scaling to higher number of column families, the worst bottleneck was MaybeScheduleFlushOrCompaction(), which did a for loop over all column families while holding a mutex. This patch addresses the issue.
The approach is similar to our earlier efforts: instead of a pull-model, where we do something for every column family, we can do a push-based model -- when we detect that column family is ready to be flushed/compacted, we add it to the flush_queue_/compaction_queue_. That way we don't need to loop over every column family in MaybeScheduleFlushOrCompaction.
Here are the performance results:
Command:
./db_bench --write_buffer_size=268435456 --db_write_buffer_size=268435456 --db=/fast-rocksdb-tmp/rocks_lots_of_cf --use_existing_db=0 --open_files=55000 --statistics=1 --histogram=1 --disable_data_sync=1 --max_write_buffer_number=2 --sync=0 --benchmarks=fillrandom --threads=16 --num_column_families=5000 --disable_wal=1 --max_background_flushes=16 --max_background_compactions=16 --level0_file_num_compaction_trigger=2 --level0_slowdown_writes_trigger=2 --level0_stop_writes_trigger=3 --hard_rate_limit=1 --num=33333333 --writes=33333333
Before the patch:
fillrandom : 26.950 micros/op 37105 ops/sec; 4.1 MB/s
After the patch:
fillrandom : 17.404 micros/op 57456 ops/sec; 6.4 MB/s
Next bottleneck is VersionSet::AddLiveFiles, which is painfully slow when we have a lot of files. This is coming in the next patch, but when I removed that code, here's what I got:
fillrandom : 7.590 micros/op 131758 ops/sec; 14.6 MB/s
Test Plan:
make check
two stress tests:
Big number of compactions and flushes:
./db_stress --threads=30 --ops_per_thread=20000000 --max_key=10000 --column_families=20 --clear_column_family_one_in=10000000 --verify_before_write=0 --reopen=15 --max_background_compactions=10 --max_background_flushes=10 --db=/fast-rocksdb-tmp/db_stress --prefixpercent=0 --iterpercent=0 --writepercent=75 --db_write_buffer_size=2000000
max_background_flushes=0, to verify that this case also works correctly
./db_stress --threads=30 --ops_per_thread=2000000 --max_key=10000 --column_families=20 --clear_column_family_one_in=10000000 --verify_before_write=0 --reopen=3 --max_background_compactions=3 --max_background_flushes=0 --db=/fast-rocksdb-tmp/db_stress --prefixpercent=0 --iterpercent=0 --writepercent=75 --db_write_buffer_size=2000000
Reviewers: ljin, rven, yhchiang, sdong
Reviewed By: sdong
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D30123
2014-12-19 20:38:12 +01:00
|
|
|
|
Concurrent task limiter for compaction thread control (#4332)
Summary:
The PR is targeting to resolve the issue of:
https://github.com/facebook/rocksdb/issues/3972#issue-330771918
We have a rocksdb created with leveled-compaction with multiple column families (CFs), some of CFs are using HDD to store big and less frequently accessed data and others are using SSD.
When there are continuously write traffics going on to all CFs, the compaction thread pool is mostly occupied by those slow HDD compactions, which blocks fully utilize SSD bandwidth.
Since atomic write and transaction is needed across CFs, so splitting it to multiple rocksdb instance is not an option for us.
With the compaction thread control, we got 30%+ HDD write throughput gain, and also a lot smooth SSD write since less write stall happening.
ConcurrentTaskLimiter can be shared with multi-CFs across rocksdb instances, so the feature does not only work for multi-CFs scenarios, but also for multi-rocksdbs scenarios, who need disk IO resource control per tenant.
The usage is straight forward:
e.g.:
//
// Enable compaction thread limiter thru ColumnFamilyOptions
//
std::shared_ptr<ConcurrentTaskLimiter> ctl(NewConcurrentTaskLimiter("foo_limiter", 4));
Options options;
ColumnFamilyOptions cf_opt(options);
cf_opt.compaction_thread_limiter = ctl;
...
//
// Compaction thread limiter can be tuned or disabled on-the-fly
//
ctl->SetMaxOutstandingTask(12); // enlarge to 12 tasks
...
ctl->ResetMaxOutstandingTask(); // disable (bypass) thread limiter
ctl->SetMaxOutstandingTask(-1); // Same as above
...
ctl->SetMaxOutstandingTask(0); // full throttle (0 task)
//
// Sharing compaction thread limiter among CFs (to resolve multiple storage perf issue)
//
std::shared_ptr<ConcurrentTaskLimiter> ctl_ssd(NewConcurrentTaskLimiter("ssd_limiter", 8));
std::shared_ptr<ConcurrentTaskLimiter> ctl_hdd(NewConcurrentTaskLimiter("hdd_limiter", 4));
Options options;
ColumnFamilyOptions cf_opt_ssd1(options);
ColumnFamilyOptions cf_opt_ssd2(options);
ColumnFamilyOptions cf_opt_hdd1(options);
ColumnFamilyOptions cf_opt_hdd2(options);
ColumnFamilyOptions cf_opt_hdd3(options);
// SSD CFs
cf_opt_ssd1.compaction_thread_limiter = ctl_ssd;
cf_opt_ssd2.compaction_thread_limiter = ctl_ssd;
// HDD CFs
cf_opt_hdd1.compaction_thread_limiter = ctl_hdd;
cf_opt_hdd2.compaction_thread_limiter = ctl_hdd;
cf_opt_hdd3.compaction_thread_limiter = ctl_hdd;
...
//
// The limiter is disabled by default (or set to nullptr explicitly)
//
Options options;
ColumnFamilyOptions cf_opt(options);
cf_opt.compaction_thread_limiter = nullptr;
Pull Request resolved: https://github.com/facebook/rocksdb/pull/4332
Differential Revision: D13226590
Pulled By: siying
fbshipit-source-id: 14307aec55b8bd59c8223d04aa6db3c03d1b0c1d
2018-12-13 22:16:04 +01:00
|
|
|
// Pick the first unthrottled compaction with task token from queue.
|
|
|
|
ColumnFamilyData* PickCompactionFromQueue(
|
|
|
|
std::unique_ptr<TaskLimiterToken>* token, LogBuffer* log_buffer);
|
|
|
|
|
[wal changes 3/3] method in DB to sync WAL without blocking writers
Summary:
Subj. We really need this feature.
Previous diff D40899 has most of the changes to make this possible, this diff just adds the method.
Test Plan: `make check`, the new test fails without this diff; ran with ASAN, TSAN and valgrind.
Reviewers: igor, rven, IslamAbdelRahman, anthony, kradhakrishnan, tnovak, yhchiang, sdong
Reviewed By: sdong
Subscribers: MarkCallaghan, maykov, hermanlee4, yoshinorim, tnovak, dhruba
Differential Revision: https://reviews.facebook.net/D40905
2015-08-05 15:06:39 +02:00
|
|
|
// helper function to call after some of the logs_ were synced
|
2020-11-07 01:30:44 +01:00
|
|
|
Status MarkLogsSynced(uint64_t up_to, bool synced_dir);
|
|
|
|
// WALs with log number up to up_to are not synced successfully.
|
|
|
|
void MarkLogsNotSynced(uint64_t up_to);
|
[wal changes 3/3] method in DB to sync WAL without blocking writers
Summary:
Subj. We really need this feature.
Previous diff D40899 has most of the changes to make this possible, this diff just adds the method.
Test Plan: `make check`, the new test fails without this diff; ran with ASAN, TSAN and valgrind.
Reviewers: igor, rven, IslamAbdelRahman, anthony, kradhakrishnan, tnovak, yhchiang, sdong
Reviewed By: sdong
Subscribers: MarkCallaghan, maykov, hermanlee4, yoshinorim, tnovak, dhruba
Differential Revision: https://reviews.facebook.net/D40905
2015-08-05 15:06:39 +02:00
|
|
|
|
2019-01-16 06:32:15 +01:00
|
|
|
SnapshotImpl* GetSnapshotImpl(bool is_write_conflict_boundary,
|
|
|
|
bool lock = true);
|
2015-12-08 21:25:48 +01:00
|
|
|
|
2016-11-15 07:45:16 +01:00
|
|
|
uint64_t GetMaxTotalWalSize() const;
|
|
|
|
|
2020-03-03 01:14:00 +01:00
|
|
|
FSDirectory* GetDataDir(ColumnFamilyData* cfd, size_t path_id) const;
|
2018-04-06 04:49:06 +02:00
|
|
|
|
2018-02-23 22:50:02 +01:00
|
|
|
Status CloseHelper();
|
2018-01-16 19:57:56 +01:00
|
|
|
|
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
|
|
|
void WaitForBackgroundWork();
|
|
|
|
|
2019-05-31 06:29:44 +02:00
|
|
|
// Background threads call this function, which is just a wrapper around
|
|
|
|
// the InstallSuperVersion() function. Background threads carry
|
|
|
|
// sv_context which can have new_superversion already
|
|
|
|
// allocated.
|
|
|
|
// All ColumnFamily state changes go through this function. Here we analyze
|
|
|
|
// the new state and we schedule background work if we detect that the new
|
|
|
|
// state needs flush or compaction.
|
|
|
|
void InstallSuperVersionAndScheduleWork(
|
|
|
|
ColumnFamilyData* cfd, SuperVersionContext* sv_context,
|
|
|
|
const MutableCFOptions& mutable_cf_options);
|
|
|
|
|
|
|
|
bool GetIntPropertyInternal(ColumnFamilyData* cfd,
|
|
|
|
const DBPropertyInfo& property_info,
|
|
|
|
bool is_locked, uint64_t* value);
|
|
|
|
bool GetPropertyHandleOptionsStatistics(std::string* value);
|
|
|
|
|
|
|
|
bool HasPendingManualCompaction();
|
|
|
|
bool HasExclusiveManualCompaction();
|
|
|
|
void AddManualCompaction(ManualCompactionState* m);
|
|
|
|
void RemoveManualCompaction(ManualCompactionState* m);
|
|
|
|
bool ShouldntRunManualCompaction(ManualCompactionState* m);
|
|
|
|
bool HaveManualCompaction(ColumnFamilyData* cfd);
|
|
|
|
bool MCOverlap(ManualCompactionState* m, ManualCompactionState* m1);
|
|
|
|
#ifndef ROCKSDB_LITE
|
|
|
|
void BuildCompactionJobInfo(const ColumnFamilyData* cfd, Compaction* c,
|
|
|
|
const Status& st,
|
|
|
|
const CompactionJobStats& compaction_job_stats,
|
|
|
|
const int job_id, const Version* current,
|
|
|
|
CompactionJobInfo* compaction_job_info) const;
|
|
|
|
// Reserve the next 'num' file numbers for to-be-ingested external SST files,
|
|
|
|
// and return the current file_number in 'next_file_number'.
|
|
|
|
// Write a version edit to the MANIFEST.
|
|
|
|
Status ReserveFileNumbersBeforeIngestion(
|
|
|
|
ColumnFamilyData* cfd, uint64_t num,
|
2019-10-08 23:18:48 +02:00
|
|
|
std::unique_ptr<std::list<uint64_t>::iterator>& pending_output_elem,
|
2019-05-31 06:29:44 +02:00
|
|
|
uint64_t* next_file_number);
|
|
|
|
#endif //! ROCKSDB_LITE
|
|
|
|
|
|
|
|
bool ShouldPurge(uint64_t file_number) const;
|
|
|
|
void MarkAsGrabbedForPurge(uint64_t file_number);
|
|
|
|
|
|
|
|
size_t GetWalPreallocateBlockSize(uint64_t write_buffer_size) const;
|
|
|
|
Env::WriteLifeTimeHint CalculateWALWriteHint() { return Env::WLTH_SHORT; }
|
|
|
|
|
2020-07-15 20:02:44 +02:00
|
|
|
IOStatus CreateWAL(uint64_t log_file_num, uint64_t recycle_log_number,
|
|
|
|
size_t preallocate_block_size, log::Writer** new_log);
|
2019-05-31 06:29:44 +02:00
|
|
|
|
2019-06-04 04:47:02 +02:00
|
|
|
// Validate self-consistency of DB options
|
|
|
|
static Status ValidateOptions(const DBOptions& db_options);
|
|
|
|
// Validate self-consistency of DB options and its consistency with cf options
|
|
|
|
static Status ValidateOptions(
|
|
|
|
const DBOptions& db_options,
|
|
|
|
const std::vector<ColumnFamilyDescriptor>& column_families);
|
|
|
|
|
2019-11-12 22:51:18 +01:00
|
|
|
// Utility function to do some debug validation and sort the given vector
|
|
|
|
// of MultiGet keys
|
|
|
|
void PrepareMultiGetKeys(
|
|
|
|
const size_t num_keys, bool sorted,
|
|
|
|
autovector<KeyContext*, MultiGetContext::MAX_BATCH_SIZE>* key_ptrs);
|
|
|
|
|
|
|
|
// A structure to hold the information required to process MultiGet of keys
|
|
|
|
// belonging to one column family. For a multi column family MultiGet, there
|
|
|
|
// will be a container of these objects.
|
|
|
|
struct MultiGetColumnFamilyData {
|
|
|
|
ColumnFamilyHandle* cf;
|
|
|
|
ColumnFamilyData* cfd;
|
|
|
|
|
|
|
|
// For the batched MultiGet which relies on sorted keys, start specifies
|
|
|
|
// the index of first key belonging to this column family in the sorted
|
|
|
|
// list.
|
|
|
|
size_t start;
|
|
|
|
|
|
|
|
// For the batched MultiGet case, num_keys specifies the number of keys
|
|
|
|
// belonging to this column family in the sorted list
|
|
|
|
size_t num_keys;
|
|
|
|
|
|
|
|
// SuperVersion for the column family obtained in a manner that ensures a
|
|
|
|
// consistent view across all column families in the DB
|
|
|
|
SuperVersion* super_version;
|
|
|
|
MultiGetColumnFamilyData(ColumnFamilyHandle* column_family,
|
|
|
|
SuperVersion* sv)
|
|
|
|
: cf(column_family),
|
|
|
|
cfd(static_cast<ColumnFamilyHandleImpl*>(cf)->cfd()),
|
|
|
|
start(0),
|
|
|
|
num_keys(0),
|
|
|
|
super_version(sv) {}
|
|
|
|
|
|
|
|
MultiGetColumnFamilyData(ColumnFamilyHandle* column_family, size_t first,
|
|
|
|
size_t count, SuperVersion* sv)
|
|
|
|
: cf(column_family),
|
|
|
|
cfd(static_cast<ColumnFamilyHandleImpl*>(cf)->cfd()),
|
|
|
|
start(first),
|
|
|
|
num_keys(count),
|
|
|
|
super_version(sv) {}
|
|
|
|
|
|
|
|
MultiGetColumnFamilyData() = default;
|
|
|
|
};
|
|
|
|
|
|
|
|
// A common function to obtain a consistent snapshot, which can be implicit
|
|
|
|
// if the user doesn't specify a snapshot in read_options, across
|
|
|
|
// multiple column families for MultiGet. It will attempt to get an implicit
|
|
|
|
// snapshot without acquiring the db_mutes, but will give up after a few
|
|
|
|
// tries and acquire the mutex if a memtable flush happens. The template
|
|
|
|
// allows both the batched and non-batched MultiGet to call this with
|
|
|
|
// either an std::unordered_map or autovector of column families.
|
|
|
|
//
|
|
|
|
// If callback is non-null, the callback is refreshed with the snapshot
|
|
|
|
// sequence number
|
|
|
|
//
|
|
|
|
// A return value of true indicates that the SuperVersions were obtained
|
|
|
|
// from the ColumnFamilyData, whereas false indicates they are thread
|
|
|
|
// local
|
|
|
|
template <class T>
|
|
|
|
bool MultiCFSnapshot(
|
|
|
|
const ReadOptions& read_options, ReadCallback* callback,
|
|
|
|
std::function<MultiGetColumnFamilyData*(typename T::iterator&)>&
|
|
|
|
iter_deref_func,
|
|
|
|
T* cf_list, SequenceNumber* snapshot);
|
|
|
|
|
|
|
|
// The actual implementation of the batching MultiGet. The caller is expected
|
|
|
|
// to have acquired the SuperVersion and pass in a snapshot sequence number
|
|
|
|
// in order to construct the LookupKeys. The start_key and num_keys specify
|
|
|
|
// the range of keys in the sorted_keys vector for a single column family.
|
2020-04-21 23:49:50 +02:00
|
|
|
Status MultiGetImpl(
|
2019-11-12 22:51:18 +01:00
|
|
|
const ReadOptions& read_options, size_t start_key, size_t num_keys,
|
|
|
|
autovector<KeyContext*, MultiGetContext::MAX_BATCH_SIZE>* sorted_keys,
|
2020-12-14 22:47:17 +01:00
|
|
|
SuperVersion* sv, SequenceNumber snap_seqnum, ReadCallback* callback);
|
2019-11-12 22:51:18 +01:00
|
|
|
|
First step towards handling MANIFEST write error (#6949)
Summary:
This PR provides preliminary support for handling IO error during MANIFEST write.
File write/sync is not guaranteed to be atomic. If we encounter an IOError while writing/syncing to the MANIFEST file, we cannot be sure about the state of the MANIFEST file. The version edits may or may not have reached the file. During cleanup, if we delete the newly-generated SST files referenced by the pending version edit(s), but the version edit(s) actually are persistent in the MANIFEST, then next recovery attempt will process the version edits(s) and then fail since the SST files have already been deleted.
One approach is to truncate the MANIFEST after write/sync error, so that it is safe to delete the SST files. However, file truncation may not be supported on certain file systems. Therefore, we take the following approach.
If an IOError is detected during MANIFEST write/sync, we disable file deletions for the faulty database. Depending on whether the IOError is retryable (set by underlying file system), either RocksDB or application can call `DB::Resume()`, or simply shutdown and restart. During `Resume()`, RocksDB will try to switch to a new MANIFEST and write all existing in-memory version storage in the new file. If this succeeds, then RocksDB may proceed. If all recovery is completed, then file deletions will be re-enabled.
Note that multiple threads can call `LogAndApply()` at the same time, though only one of them will be going through the process MANIFEST write, possibly batching the version edits of other threads. When the leading MANIFEST writer finishes, all of the MANIFEST writing threads in this batch will have the same IOError. They will all call `ErrorHandler::SetBGError()` in which file deletion will be disabled.
Possible future directions:
- Add an `ErrorContext` structure so that it is easier to pass more info to `ErrorHandler`. Currently, as in this example, a new `BackgroundErrorReason` has to be added.
Test plan (dev server):
make check
Pull Request resolved: https://github.com/facebook/rocksdb/pull/6949
Reviewed By: anand1976
Differential Revision: D22026020
Pulled By: riversand963
fbshipit-source-id: f3c68a2ef45d9b505d0d625c7c5e0c88495b91c8
2020-06-25 04:05:47 +02:00
|
|
|
Status DisableFileDeletionsWithLock();
|
|
|
|
|
2021-02-08 22:43:23 +01:00
|
|
|
Status IncreaseFullHistoryTsLow(ColumnFamilyData* cfd, std::string ts_low);
|
|
|
|
|
2011-03-18 23:37:00 +01:00
|
|
|
// table_cache_ provides its own synchronization
|
[CF] Rethink table cache
Summary:
Adapting table cache to column families is interesting. We want table cache to be global LRU, so if some column families are use not as often as others, we want them to be evicted from cache. However, current TableCache object also constructs tables on its own. If table is not found in the cache, TableCache automatically creates new table. We want each column family to be able to specify different table factory.
To solve the problem, we still have a single LRU, but we provide the LRUCache object to TableCache on construction. We have one TableCache per column family, but the underyling cache is shared by all TableCache objects.
This allows us to have a global LRU, but still be able to support different table factories for different column families. Also, in the future it will also be able to support different directories for different column families.
Test Plan: make check
Reviewers: dhruba, haobo, kailiu, sdong
CC: leveldb
Differential Revision: https://reviews.facebook.net/D15915
2014-02-05 18:07:55 +01:00
|
|
|
std::shared_ptr<Cache> table_cache_;
|
2011-03-18 23:37:00 +01:00
|
|
|
|
2013-02-16 00:28:24 +01:00
|
|
|
// Lock over the persistent DB state. Non-nullptr iff successfully acquired.
|
2011-03-18 23:37:00 +01:00
|
|
|
FileLock* db_lock_;
|
|
|
|
|
2019-02-21 00:46:59 +01:00
|
|
|
// In addition to mutex_, log_write_mutex_ protected writes to stats_history_
|
|
|
|
InstrumentedMutex stats_history_mutex_;
|
2017-06-30 18:30:03 +02:00
|
|
|
// In addition to mutex_, log_write_mutex_ protected writes to logs_ and
|
2017-11-11 02:18:01 +01:00
|
|
|
// logfile_number_. With two_write_queues it also protects alive_log_files_,
|
2017-06-30 18:30:03 +02:00
|
|
|
// and log_empty_. Refer to the definition of each variable below for more
|
|
|
|
// details.
|
2019-06-11 02:02:23 +02:00
|
|
|
// Note: to avoid dealock, if needed to acquire both log_write_mutex_ and
|
|
|
|
// mutex_, the order should be first mutex_ and then log_write_mutex_.
|
2017-06-24 23:06:43 +02:00
|
|
|
InstrumentedMutex log_write_mutex_;
|
2015-11-11 07:58:01 +01:00
|
|
|
|
2014-10-27 22:50:21 +01:00
|
|
|
std::atomic<bool> shutting_down_;
|
2020-08-14 20:28:12 +02:00
|
|
|
|
|
|
|
// If zero, manual compactions are allowed to proceed. If non-zero, manual
|
|
|
|
// compactions may still be running, but will quickly fail with
|
|
|
|
// `Status::Incomplete`. The value indicates how many threads have paused
|
|
|
|
// manual compactions. It is accessed in read mode outside the DB mutex in
|
|
|
|
// compaction code paths.
|
|
|
|
std::atomic<int> manual_compaction_paused_;
|
|
|
|
|
2014-06-03 02:23:55 +02:00
|
|
|
// This condition variable is signaled on these conditions:
|
|
|
|
// * whenever bg_compaction_scheduled_ goes down to 0
|
Running manual compactions in parallel with other automatic or manual compactions in restricted cases
Summary:
This diff provides a framework for doing manual
compactions in parallel with other compactions. We now have a deque of manual compactions. We also pass manual compactions as an argument from RunManualCompactions down to
BackgroundCompactions, so that RunManualCompactions can be reentrant.
Parallelism is controlled by the two routines
ConflictingManualCompaction to allow/disallow new parallel/manual
compactions based on already existing ManualCompactions. In this diff, by default manual compactions still have to run exclusive of other compactions. However, by setting the compaction option, exclusive_manual_compaction to false, it is possible to run other compactions in parallel with a manual compaction. However, we are still restricted to one manual compaction per column family at a time. All of these restrictions will be relaxed in future diffs.
I will be adding more tests later.
Test Plan: Rocksdb regression + new tests + valgrind
Reviewers: igor, anthony, IslamAbdelRahman, kradhakrishnan, yhchiang, sdong
Reviewed By: sdong
Subscribers: yoshinorim, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D47973
2015-12-14 20:20:34 +01:00
|
|
|
// * if AnyManualCompaction, whenever a compaction finishes, even if it hasn't
|
2014-06-03 02:23:55 +02:00
|
|
|
// made any progress
|
|
|
|
// * whenever a compaction made any progress
|
2016-06-22 03:41:23 +02:00
|
|
|
// * whenever bg_flush_scheduled_ or bg_purge_scheduled_ value decreases
|
|
|
|
// (i.e. whenever a flush is done, even if it didn't make any progress)
|
|
|
|
// * whenever there is an error in background purge, flush or compaction
|
2016-10-21 02:05:32 +02:00
|
|
|
// * whenever num_running_ingest_file_ goes to 0.
|
2018-01-18 02:37:10 +01:00
|
|
|
// * whenever pending_purge_obsolete_files_ goes to 0.
|
|
|
|
// * whenever disable_delete_obsolete_files_ goes to 0.
|
2018-02-13 00:34:39 +01:00
|
|
|
// * whenever SetOptions successfully updates options.
|
|
|
|
// * whenever a column family is dropped.
|
2015-02-05 06:39:45 +01:00
|
|
|
InstrumentedCondVar bg_cv_;
|
2017-06-30 18:30:03 +02:00
|
|
|
// Writes are protected by locking both mutex_ and log_write_mutex_, and reads
|
|
|
|
// must be under either mutex_ or log_write_mutex_. Since after ::Open,
|
|
|
|
// logfile_number_ is currently updated only in write_thread_, it can be read
|
|
|
|
// from the same write_thread_ without any locks.
|
2011-06-22 04:36:45 +02:00
|
|
|
uint64_t logfile_number_;
|
2015-10-01 15:12:04 +02:00
|
|
|
std::deque<uint64_t>
|
2018-01-09 21:51:10 +01:00
|
|
|
log_recycle_files_; // a list of log files that we can recycle
|
2015-01-27 00:48:59 +01:00
|
|
|
bool log_dir_synced_;
|
2017-11-11 02:18:01 +01:00
|
|
|
// Without two_write_queues, read and writes to log_empty_ are protected by
|
2017-06-30 18:30:03 +02:00
|
|
|
// mutex_. Since it is currently updated/read only in write_thread_, it can be
|
|
|
|
// accessed from the same write_thread_ without any locks. With
|
2017-11-11 02:18:01 +01:00
|
|
|
// two_write_queues writes, where it can be updated in different threads,
|
2017-06-30 18:30:03 +02:00
|
|
|
// read and writes are protected by log_write_mutex_ instead. This is to avoid
|
|
|
|
// expesnive mutex_ lock during WAL write, which update log_empty_.
|
2014-04-15 18:57:25 +02:00
|
|
|
bool log_empty_;
|
2019-03-27 00:41:31 +01:00
|
|
|
|
2019-06-18 00:17:43 +02:00
|
|
|
ColumnFamilyHandleImpl* persist_stats_cf_handle_;
|
|
|
|
|
|
|
|
bool persistent_stats_cfd_exists_ = true;
|
2015-08-06 05:51:27 +02:00
|
|
|
|
2017-11-11 02:18:01 +01:00
|
|
|
// Without two_write_queues, read and writes to alive_log_files_ are
|
2017-06-30 18:30:03 +02:00
|
|
|
// protected by mutex_. However since back() is never popped, and push_back()
|
|
|
|
// is done only from write_thread_, the same thread can access the item
|
2017-11-11 02:18:01 +01:00
|
|
|
// reffered by back() without mutex_. With two_write_queues_, writes
|
2017-06-30 18:30:03 +02:00
|
|
|
// are protected by locking both mutex_ and log_write_mutex_, and reads must
|
|
|
|
// be under either mutex_ or log_write_mutex_.
|
2014-04-30 20:33:40 +02:00
|
|
|
std::deque<LogFileNumberSize> alive_log_files_;
|
[wal changes 2/3] write with sync=true syncs previous unsynced wals to prevent illegal data loss
Summary:
I'll just copy internal task summary here:
"
This sequence will cause data loss in the middle after an sync write:
non-sync write key 1
flush triggered, not yet scheduled
sync write key 2
system crash
After rebooting, users might see key 2 but not key 1, which violates the API of sync write.
This can be reproduced using unit test FaultInjectionTest::DISABLED_WriteOptionSyncTest.
One way to fix it is for a sync write, if there is outstanding unsynced log files, we need to syc them too.
"
This diff should be considered together with the next diff D40905; in isolation this fix probably could be a little simpler.
Test Plan: `make check`; added a test for that (DBTest.SyncingPreviousLogs) before noticing FaultInjectionTest.WriteOptionSyncTest (keeping both since mine asserts a bit more); both tests fail without this diff; for D40905 stacked on top of this diff, ran tests with ASAN, TSAN and valgrind
Reviewers: rven, yhchiang, IslamAbdelRahman, anthony, kradhakrishnan, igor, sdong
Reviewed By: sdong
Subscribers: dhruba
Differential Revision: https://reviews.facebook.net/D40899
2015-07-22 12:28:08 +02:00
|
|
|
// Log files that aren't fully synced, and the current log file.
|
|
|
|
// Synchronization:
|
2017-06-30 18:30:03 +02:00
|
|
|
// - push_back() is done from write_thread_ with locked mutex_ and
|
|
|
|
// log_write_mutex_
|
|
|
|
// - pop_front() is done from any thread with locked mutex_ and
|
|
|
|
// log_write_mutex_
|
|
|
|
// - reads are done with either locked mutex_ or log_write_mutex_
|
[wal changes 2/3] write with sync=true syncs previous unsynced wals to prevent illegal data loss
Summary:
I'll just copy internal task summary here:
"
This sequence will cause data loss in the middle after an sync write:
non-sync write key 1
flush triggered, not yet scheduled
sync write key 2
system crash
After rebooting, users might see key 2 but not key 1, which violates the API of sync write.
This can be reproduced using unit test FaultInjectionTest::DISABLED_WriteOptionSyncTest.
One way to fix it is for a sync write, if there is outstanding unsynced log files, we need to syc them too.
"
This diff should be considered together with the next diff D40905; in isolation this fix probably could be a little simpler.
Test Plan: `make check`; added a test for that (DBTest.SyncingPreviousLogs) before noticing FaultInjectionTest.WriteOptionSyncTest (keeping both since mine asserts a bit more); both tests fail without this diff; for D40905 stacked on top of this diff, ran tests with ASAN, TSAN and valgrind
Reviewers: rven, yhchiang, IslamAbdelRahman, anthony, kradhakrishnan, igor, sdong
Reviewed By: sdong
Subscribers: dhruba
Differential Revision: https://reviews.facebook.net/D40899
2015-07-22 12:28:08 +02:00
|
|
|
// - back() and items with getting_synced=true are not popped,
|
2017-06-30 18:30:03 +02:00
|
|
|
// - The same thread that sets getting_synced=true will reset it.
|
|
|
|
// - it follows that the object referred by back() can be safely read from
|
|
|
|
// the write_thread_ without using mutex
|
|
|
|
// - it follows that the items with getting_synced=true can be safely read
|
|
|
|
// from the same thread that has set getting_synced=true
|
[wal changes 2/3] write with sync=true syncs previous unsynced wals to prevent illegal data loss
Summary:
I'll just copy internal task summary here:
"
This sequence will cause data loss in the middle after an sync write:
non-sync write key 1
flush triggered, not yet scheduled
sync write key 2
system crash
After rebooting, users might see key 2 but not key 1, which violates the API of sync write.
This can be reproduced using unit test FaultInjectionTest::DISABLED_WriteOptionSyncTest.
One way to fix it is for a sync write, if there is outstanding unsynced log files, we need to syc them too.
"
This diff should be considered together with the next diff D40905; in isolation this fix probably could be a little simpler.
Test Plan: `make check`; added a test for that (DBTest.SyncingPreviousLogs) before noticing FaultInjectionTest.WriteOptionSyncTest (keeping both since mine asserts a bit more); both tests fail without this diff; for D40905 stacked on top of this diff, ran tests with ASAN, TSAN and valgrind
Reviewers: rven, yhchiang, IslamAbdelRahman, anthony, kradhakrishnan, igor, sdong
Reviewed By: sdong
Subscribers: dhruba
Differential Revision: https://reviews.facebook.net/D40899
2015-07-22 12:28:08 +02:00
|
|
|
std::deque<LogWriterNumber> logs_;
|
|
|
|
// Signaled when getting_synced becomes false for some of the logs_.
|
|
|
|
InstrumentedCondVar log_sync_cv_;
|
2017-11-02 01:23:52 +01:00
|
|
|
// This is the app-level state that is written to the WAL but will be used
|
|
|
|
// only during recovery. Using this feature enables not writing the state to
|
|
|
|
// memtable on normal writes and hence improving the throughput. Each new
|
|
|
|
// write of the state will replace the previous state entirely even if the
|
|
|
|
// keys in the two consecuitive states do not overlap.
|
2017-11-11 02:18:01 +01:00
|
|
|
// It is protected by log_write_mutex_ when two_write_queues_ is enabled.
|
2017-11-02 01:23:52 +01:00
|
|
|
// Otherwise only the heaad of write_thread_ can access it.
|
|
|
|
WriteBatch cached_recoverable_state_;
|
|
|
|
std::atomic<bool> cached_recoverable_state_empty_ = {true};
|
2017-03-22 19:50:25 +01:00
|
|
|
std::atomic<uint64_t> total_log_size_;
|
2019-03-27 00:41:31 +01:00
|
|
|
|
2015-03-30 21:04:10 +02:00
|
|
|
// If this is non-empty, we need to delete these log files in background
|
|
|
|
// threads. Protected by db mutex.
|
|
|
|
autovector<log::Writer*> logs_to_free_;
|
2013-12-20 18:57:58 +01:00
|
|
|
|
2014-12-11 03:39:09 +01:00
|
|
|
bool is_snapshot_supported_;
|
|
|
|
|
2019-02-21 00:46:59 +01:00
|
|
|
std::map<uint64_t, std::map<std::string, uint64_t>> stats_history_;
|
|
|
|
|
|
|
|
std::map<std::string, uint64_t> stats_slice_;
|
|
|
|
|
|
|
|
bool stats_slice_initialized_ = false;
|
|
|
|
|
2015-01-26 22:59:38 +01:00
|
|
|
Directories directories_;
|
2014-01-27 20:02:21 +01:00
|
|
|
|
2016-06-21 03:01:03 +02:00
|
|
|
WriteBufferManager* write_buffer_manager_;
|
2014-12-02 21:09:20 +01:00
|
|
|
|
2014-09-13 01:23:58 +02:00
|
|
|
WriteThread write_thread_;
|
2013-03-28 23:19:28 +01:00
|
|
|
WriteBatch tmp_batch_;
|
2017-06-24 23:06:43 +02:00
|
|
|
// The write thread when the writers have no memtable write. This will be used
|
|
|
|
// in 2PC to batch the prepares separately from the serial commit.
|
|
|
|
WriteThread nonmem_write_thread_;
|
2012-03-09 01:23:21 +01:00
|
|
|
|
Push- instead of pull-model for managing Write stalls
Summary:
Introducing WriteController, which is a source of truth about per-DB write delays. Let's define an DB epoch as a period where there are no flushes and compactions (i.e. new epoch is started when flush or compaction finishes). Each epoch can either:
* proceed with all writes without delay
* delay all writes by fixed time
* stop all writes
The three modes are recomputed at each epoch change (flush, compaction), rather than on every write (which is currently the case).
When we have a lot of column families, our current pull behavior adds a big overhead, since we need to loop over every column family for every write. With new push model, overhead on Write code-path is minimal.
This is just the start. Next step is to also take care of stalls introduced by slow memtable flushes. The final goal is to eliminate function MakeRoomForWrite(), which currently needs to be called for every column family by every write.
Test Plan: make check for now. I'll add some unit tests later. Also, perf test.
Reviewers: dhruba, yhchiang, MarkCallaghan, sdong, ljin
Reviewed By: ljin
Subscribers: leveldb
Differential Revision: https://reviews.facebook.net/D22791
2014-09-08 20:20:25 +02:00
|
|
|
WriteController write_controller_;
|
2015-05-16 00:52:51 +02:00
|
|
|
|
|
|
|
// Size of the last batch group. In slowdown mode, next write needs to
|
|
|
|
// sleep if it uses up the quota.
|
2017-06-24 23:06:43 +02:00
|
|
|
// Note: This is to protect memtable and compaction. If the batch only writes
|
|
|
|
// to the WAL its size need not to be included in this.
|
2015-05-16 00:52:51 +02:00
|
|
|
uint64_t last_batch_group_size_;
|
|
|
|
|
2014-09-11 03:46:09 +02:00
|
|
|
FlushScheduler flush_scheduler_;
|
Push- instead of pull-model for managing Write stalls
Summary:
Introducing WriteController, which is a source of truth about per-DB write delays. Let's define an DB epoch as a period where there are no flushes and compactions (i.e. new epoch is started when flush or compaction finishes). Each epoch can either:
* proceed with all writes without delay
* delay all writes by fixed time
* stop all writes
The three modes are recomputed at each epoch change (flush, compaction), rather than on every write (which is currently the case).
When we have a lot of column families, our current pull behavior adds a big overhead, since we need to loop over every column family for every write. With new push model, overhead on Write code-path is minimal.
This is just the start. Next step is to also take care of stalls introduced by slow memtable flushes. The final goal is to eliminate function MakeRoomForWrite(), which currently needs to be called for every column family by every write.
Test Plan: make check for now. I'll add some unit tests later. Also, perf test.
Reviewers: dhruba, yhchiang, MarkCallaghan, sdong, ljin
Reviewed By: ljin
Subscribers: leveldb
Differential Revision: https://reviews.facebook.net/D22791
2014-09-08 20:20:25 +02:00
|
|
|
|
Refactor trimming logic for immutable memtables (#5022)
Summary:
MyRocks currently sets `max_write_buffer_number_to_maintain` in order to maintain enough history for transaction conflict checking. The effectiveness of this approach depends on the size of memtables. When memtables are small, it may not keep enough history; when memtables are large, this may consume too much memory.
We are proposing a new way to configure memtable list history: by limiting the memory usage of immutable memtables. The new option is `max_write_buffer_size_to_maintain` and it will take precedence over the old `max_write_buffer_number_to_maintain` if they are both set to non-zero values. The new option accounts for the total memory usage of flushed immutable memtables and mutable memtable. When the total usage exceeds the limit, RocksDB may start dropping immutable memtables (which is also called trimming history), starting from the oldest one.
The semantics of the old option actually works both as an upper bound and lower bound. History trimming will start if number of immutable memtables exceeds the limit, but it will never go below (limit-1) due to history trimming.
In order the mimic the behavior with the new option, history trimming will stop if dropping the next immutable memtable causes the total memory usage go below the size limit. For example, assuming the size limit is set to 64MB, and there are 3 immutable memtables with sizes of 20, 30, 30. Although the total memory usage is 80MB > 64MB, dropping the oldest memtable will reduce the memory usage to 60MB < 64MB, so in this case no memtable will be dropped.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5022
Differential Revision: D14394062
Pulled By: miasantreble
fbshipit-source-id: 60457a509c6af89d0993f988c9b5c2aa9e45f5c5
2019-08-23 22:54:09 +02:00
|
|
|
TrimHistoryScheduler trim_history_scheduler_;
|
|
|
|
|
2011-03-18 23:37:00 +01:00
|
|
|
SnapshotList snapshots_;
|
|
|
|
|
2014-11-07 20:50:34 +01:00
|
|
|
// For each background job, pending_outputs_ keeps the current file number at
|
|
|
|
// the time that background job started.
|
|
|
|
// FindObsoleteFiles()/PurgeObsoleteFiles() never deletes any file that has
|
|
|
|
// number bigger than any of the file number in pending_outputs_. Since file
|
|
|
|
// numbers grow monotonically, this also means that pending_outputs_ is always
|
|
|
|
// sorted. After a background job is done executing, its file number is
|
|
|
|
// deleted from pending_outputs_, which allows PurgeObsoleteFiles() to clean
|
|
|
|
// it up.
|
|
|
|
// State is protected with db mutex.
|
|
|
|
std::list<uint64_t> pending_outputs_;
|
2011-03-18 23:37:00 +01:00
|
|
|
|
Rewritten system for scheduling background work
Summary:
When scaling to higher number of column families, the worst bottleneck was MaybeScheduleFlushOrCompaction(), which did a for loop over all column families while holding a mutex. This patch addresses the issue.
The approach is similar to our earlier efforts: instead of a pull-model, where we do something for every column family, we can do a push-based model -- when we detect that column family is ready to be flushed/compacted, we add it to the flush_queue_/compaction_queue_. That way we don't need to loop over every column family in MaybeScheduleFlushOrCompaction.
Here are the performance results:
Command:
./db_bench --write_buffer_size=268435456 --db_write_buffer_size=268435456 --db=/fast-rocksdb-tmp/rocks_lots_of_cf --use_existing_db=0 --open_files=55000 --statistics=1 --histogram=1 --disable_data_sync=1 --max_write_buffer_number=2 --sync=0 --benchmarks=fillrandom --threads=16 --num_column_families=5000 --disable_wal=1 --max_background_flushes=16 --max_background_compactions=16 --level0_file_num_compaction_trigger=2 --level0_slowdown_writes_trigger=2 --level0_stop_writes_trigger=3 --hard_rate_limit=1 --num=33333333 --writes=33333333
Before the patch:
fillrandom : 26.950 micros/op 37105 ops/sec; 4.1 MB/s
After the patch:
fillrandom : 17.404 micros/op 57456 ops/sec; 6.4 MB/s
Next bottleneck is VersionSet::AddLiveFiles, which is painfully slow when we have a lot of files. This is coming in the next patch, but when I removed that code, here's what I got:
fillrandom : 7.590 micros/op 131758 ops/sec; 14.6 MB/s
Test Plan:
make check
two stress tests:
Big number of compactions and flushes:
./db_stress --threads=30 --ops_per_thread=20000000 --max_key=10000 --column_families=20 --clear_column_family_one_in=10000000 --verify_before_write=0 --reopen=15 --max_background_compactions=10 --max_background_flushes=10 --db=/fast-rocksdb-tmp/db_stress --prefixpercent=0 --iterpercent=0 --writepercent=75 --db_write_buffer_size=2000000
max_background_flushes=0, to verify that this case also works correctly
./db_stress --threads=30 --ops_per_thread=2000000 --max_key=10000 --column_families=20 --clear_column_family_one_in=10000000 --verify_before_write=0 --reopen=3 --max_background_compactions=3 --max_background_flushes=0 --db=/fast-rocksdb-tmp/db_stress --prefixpercent=0 --iterpercent=0 --writepercent=75 --db_write_buffer_size=2000000
Reviewers: ljin, rven, yhchiang, sdong
Reviewed By: sdong
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D30123
2014-12-19 20:38:12 +01:00
|
|
|
// flush_queue_ and compaction_queue_ hold column families that we need to
|
|
|
|
// flush and compact, respectively.
|
|
|
|
// A column family is inserted into flush_queue_ when it satisfies condition
|
|
|
|
// cfd->imm()->IsFlushPending()
|
|
|
|
// A column family is inserted into compaction_queue_ when it satisfied
|
|
|
|
// condition cfd->NeedsCompaction()
|
|
|
|
// Column families in this list are all Ref()-erenced
|
|
|
|
// TODO(icanadi) Provide some kind of ReferencedColumnFamily class that will
|
|
|
|
// do RAII on ColumnFamilyData
|
|
|
|
// Column families are in this queue when they need to be flushed or
|
|
|
|
// compacted. Consumers of these queues are flush and compaction threads. When
|
|
|
|
// column family is put on this queue, we increase unscheduled_flushes_ and
|
|
|
|
// unscheduled_compactions_. When these variables are bigger than zero, that
|
2019-01-07 22:50:26 +01:00
|
|
|
// means we need to schedule background threads for flush and compaction.
|
Rewritten system for scheduling background work
Summary:
When scaling to higher number of column families, the worst bottleneck was MaybeScheduleFlushOrCompaction(), which did a for loop over all column families while holding a mutex. This patch addresses the issue.
The approach is similar to our earlier efforts: instead of a pull-model, where we do something for every column family, we can do a push-based model -- when we detect that column family is ready to be flushed/compacted, we add it to the flush_queue_/compaction_queue_. That way we don't need to loop over every column family in MaybeScheduleFlushOrCompaction.
Here are the performance results:
Command:
./db_bench --write_buffer_size=268435456 --db_write_buffer_size=268435456 --db=/fast-rocksdb-tmp/rocks_lots_of_cf --use_existing_db=0 --open_files=55000 --statistics=1 --histogram=1 --disable_data_sync=1 --max_write_buffer_number=2 --sync=0 --benchmarks=fillrandom --threads=16 --num_column_families=5000 --disable_wal=1 --max_background_flushes=16 --max_background_compactions=16 --level0_file_num_compaction_trigger=2 --level0_slowdown_writes_trigger=2 --level0_stop_writes_trigger=3 --hard_rate_limit=1 --num=33333333 --writes=33333333
Before the patch:
fillrandom : 26.950 micros/op 37105 ops/sec; 4.1 MB/s
After the patch:
fillrandom : 17.404 micros/op 57456 ops/sec; 6.4 MB/s
Next bottleneck is VersionSet::AddLiveFiles, which is painfully slow when we have a lot of files. This is coming in the next patch, but when I removed that code, here's what I got:
fillrandom : 7.590 micros/op 131758 ops/sec; 14.6 MB/s
Test Plan:
make check
two stress tests:
Big number of compactions and flushes:
./db_stress --threads=30 --ops_per_thread=20000000 --max_key=10000 --column_families=20 --clear_column_family_one_in=10000000 --verify_before_write=0 --reopen=15 --max_background_compactions=10 --max_background_flushes=10 --db=/fast-rocksdb-tmp/db_stress --prefixpercent=0 --iterpercent=0 --writepercent=75 --db_write_buffer_size=2000000
max_background_flushes=0, to verify that this case also works correctly
./db_stress --threads=30 --ops_per_thread=2000000 --max_key=10000 --column_families=20 --clear_column_family_one_in=10000000 --verify_before_write=0 --reopen=3 --max_background_compactions=3 --max_background_flushes=0 --db=/fast-rocksdb-tmp/db_stress --prefixpercent=0 --iterpercent=0 --writepercent=75 --db_write_buffer_size=2000000
Reviewers: ljin, rven, yhchiang, sdong
Reviewed By: sdong
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D30123
2014-12-19 20:38:12 +01:00
|
|
|
// Once the background threads are scheduled, we decrease unscheduled_flushes_
|
|
|
|
// and unscheduled_compactions_. That way we keep track of number of
|
|
|
|
// compaction and flush threads we need to schedule. This scheduling is done
|
|
|
|
// in MaybeScheduleFlushOrCompaction()
|
|
|
|
// invariant(column family present in flush_queue_ <==>
|
|
|
|
// ColumnFamilyData::pending_flush_ == true)
|
2018-08-24 22:17:29 +02:00
|
|
|
std::deque<FlushRequest> flush_queue_;
|
Rewritten system for scheduling background work
Summary:
When scaling to higher number of column families, the worst bottleneck was MaybeScheduleFlushOrCompaction(), which did a for loop over all column families while holding a mutex. This patch addresses the issue.
The approach is similar to our earlier efforts: instead of a pull-model, where we do something for every column family, we can do a push-based model -- when we detect that column family is ready to be flushed/compacted, we add it to the flush_queue_/compaction_queue_. That way we don't need to loop over every column family in MaybeScheduleFlushOrCompaction.
Here are the performance results:
Command:
./db_bench --write_buffer_size=268435456 --db_write_buffer_size=268435456 --db=/fast-rocksdb-tmp/rocks_lots_of_cf --use_existing_db=0 --open_files=55000 --statistics=1 --histogram=1 --disable_data_sync=1 --max_write_buffer_number=2 --sync=0 --benchmarks=fillrandom --threads=16 --num_column_families=5000 --disable_wal=1 --max_background_flushes=16 --max_background_compactions=16 --level0_file_num_compaction_trigger=2 --level0_slowdown_writes_trigger=2 --level0_stop_writes_trigger=3 --hard_rate_limit=1 --num=33333333 --writes=33333333
Before the patch:
fillrandom : 26.950 micros/op 37105 ops/sec; 4.1 MB/s
After the patch:
fillrandom : 17.404 micros/op 57456 ops/sec; 6.4 MB/s
Next bottleneck is VersionSet::AddLiveFiles, which is painfully slow when we have a lot of files. This is coming in the next patch, but when I removed that code, here's what I got:
fillrandom : 7.590 micros/op 131758 ops/sec; 14.6 MB/s
Test Plan:
make check
two stress tests:
Big number of compactions and flushes:
./db_stress --threads=30 --ops_per_thread=20000000 --max_key=10000 --column_families=20 --clear_column_family_one_in=10000000 --verify_before_write=0 --reopen=15 --max_background_compactions=10 --max_background_flushes=10 --db=/fast-rocksdb-tmp/db_stress --prefixpercent=0 --iterpercent=0 --writepercent=75 --db_write_buffer_size=2000000
max_background_flushes=0, to verify that this case also works correctly
./db_stress --threads=30 --ops_per_thread=2000000 --max_key=10000 --column_families=20 --clear_column_family_one_in=10000000 --verify_before_write=0 --reopen=3 --max_background_compactions=3 --max_background_flushes=0 --db=/fast-rocksdb-tmp/db_stress --prefixpercent=0 --iterpercent=0 --writepercent=75 --db_write_buffer_size=2000000
Reviewers: ljin, rven, yhchiang, sdong
Reviewed By: sdong
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D30123
2014-12-19 20:38:12 +01:00
|
|
|
// invariant(column family present in compaction_queue_ <==>
|
|
|
|
// ColumnFamilyData::pending_compaction_ == true)
|
|
|
|
std::deque<ColumnFamilyData*> compaction_queue_;
|
2016-06-22 03:41:23 +02:00
|
|
|
|
2019-09-18 01:43:07 +02:00
|
|
|
// A map to store file numbers and filenames of the files to be purged
|
|
|
|
std::unordered_map<uint64_t, PurgeFileInfo> purge_files_;
|
2016-08-04 02:42:06 +02:00
|
|
|
|
2018-03-28 19:23:31 +02:00
|
|
|
// A vector to store the file numbers that have been assigned to certain
|
2020-05-07 18:29:21 +02:00
|
|
|
// JobContext. Current implementation tracks table and blob files only.
|
2019-09-18 01:43:07 +02:00
|
|
|
std::unordered_set<uint64_t> files_grabbed_for_purge_;
|
2018-03-28 19:23:31 +02:00
|
|
|
|
2016-08-04 02:42:06 +02:00
|
|
|
// A queue to store log writers to close
|
|
|
|
std::deque<log::Writer*> logs_to_free_queue_;
|
2019-12-17 22:20:42 +01:00
|
|
|
std::deque<SuperVersion*> superversions_to_free_queue_;
|
Rewritten system for scheduling background work
Summary:
When scaling to higher number of column families, the worst bottleneck was MaybeScheduleFlushOrCompaction(), which did a for loop over all column families while holding a mutex. This patch addresses the issue.
The approach is similar to our earlier efforts: instead of a pull-model, where we do something for every column family, we can do a push-based model -- when we detect that column family is ready to be flushed/compacted, we add it to the flush_queue_/compaction_queue_. That way we don't need to loop over every column family in MaybeScheduleFlushOrCompaction.
Here are the performance results:
Command:
./db_bench --write_buffer_size=268435456 --db_write_buffer_size=268435456 --db=/fast-rocksdb-tmp/rocks_lots_of_cf --use_existing_db=0 --open_files=55000 --statistics=1 --histogram=1 --disable_data_sync=1 --max_write_buffer_number=2 --sync=0 --benchmarks=fillrandom --threads=16 --num_column_families=5000 --disable_wal=1 --max_background_flushes=16 --max_background_compactions=16 --level0_file_num_compaction_trigger=2 --level0_slowdown_writes_trigger=2 --level0_stop_writes_trigger=3 --hard_rate_limit=1 --num=33333333 --writes=33333333
Before the patch:
fillrandom : 26.950 micros/op 37105 ops/sec; 4.1 MB/s
After the patch:
fillrandom : 17.404 micros/op 57456 ops/sec; 6.4 MB/s
Next bottleneck is VersionSet::AddLiveFiles, which is painfully slow when we have a lot of files. This is coming in the next patch, but when I removed that code, here's what I got:
fillrandom : 7.590 micros/op 131758 ops/sec; 14.6 MB/s
Test Plan:
make check
two stress tests:
Big number of compactions and flushes:
./db_stress --threads=30 --ops_per_thread=20000000 --max_key=10000 --column_families=20 --clear_column_family_one_in=10000000 --verify_before_write=0 --reopen=15 --max_background_compactions=10 --max_background_flushes=10 --db=/fast-rocksdb-tmp/db_stress --prefixpercent=0 --iterpercent=0 --writepercent=75 --db_write_buffer_size=2000000
max_background_flushes=0, to verify that this case also works correctly
./db_stress --threads=30 --ops_per_thread=2000000 --max_key=10000 --column_families=20 --clear_column_family_one_in=10000000 --verify_before_write=0 --reopen=3 --max_background_compactions=3 --max_background_flushes=0 --db=/fast-rocksdb-tmp/db_stress --prefixpercent=0 --iterpercent=0 --writepercent=75 --db_write_buffer_size=2000000
Reviewers: ljin, rven, yhchiang, sdong
Reviewed By: sdong
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D30123
2014-12-19 20:38:12 +01:00
|
|
|
int unscheduled_flushes_;
|
|
|
|
int unscheduled_compactions_;
|
Fix data race against logging data structure because of LogBuffer
Summary:
@igor pointed out that there is a potential data race because of the way we use the newly introduced LogBuffer. After "bg_compaction_scheduled_--" or "bg_flush_scheduled_--", they can both become 0. As soon as the lock is released after that, DBImpl's deconstructor can go ahead and deconstruct all the states inside DB, including the info_log object hold in a shared pointer of the options object it keeps. At that point it is not safe anymore to continue using the info logger to write the delayed logs.
With the patch, lock is released temporarily for log buffer to be flushed before "bg_compaction_scheduled_--" or "bg_flush_scheduled_--". In order to make sure we don't miss any pending flush or compaction, a new flag bg_schedule_needed_ is added, which is set to be true if there is a pending flush or compaction but not scheduled because of the max thread limit. If the flag is set to be true, the scheduling function will be called before compaction or flush thread finishes.
Thanks @igor for this finding!
Test Plan: make all check
Reviewers: haobo, igor
Reviewed By: haobo
CC: dhruba, ljin, yhchiang, igor, leveldb
Differential Revision: https://reviews.facebook.net/D16767
2014-03-11 18:36:30 +01:00
|
|
|
|
2017-08-04 00:36:28 +02:00
|
|
|
// count how many background compactions are running or have been scheduled in
|
|
|
|
// the BOTTOM pool
|
|
|
|
int bg_bottom_compaction_scheduled_;
|
|
|
|
|
Fix a deadlock in CompactRange()
Summary:
The way DBImpl::TEST_CompactRange() throttles down the number of bg compactions
can cause it to deadlock when CompactRange() is called concurrently from
multiple threads. Imagine a following scenario with only two threads
(max_background_compactions is 10 and bg_compaction_scheduled_ is initially 0):
1. Thread #1 increments bg_compaction_scheduled_ (to LargeNumber), sets
bg_compaction_scheduled_ to 9 (newvalue), schedules the compaction
(bg_compaction_scheduled_ is now 10) and waits for it to complete.
2. Thread #2 calls TEST_CompactRange(), increments bg_compaction_scheduled_
(now LargeNumber + 10) and waits on a cv for bg_compaction_scheduled_ to
drop to LargeNumber.
3. BG thread completes the first manual compaction, decrements
bg_compaction_scheduled_ and wakes up all threads waiting on bg_cv_.
Thread #1 runs, increments bg_compaction_scheduled_ by LargeNumber again
(now 2*LargeNumber + 9). Since that's more than LargeNumber + newvalue,
thread #2 also goes to sleep (waiting on bg_cv_), without resetting
bg_compaction_scheduled_.
This diff attempts to address the problem by introducing a new counter
bg_manual_only_ (when positive, MaybeScheduleFlushOrCompaction() will only
schedule manual compactions).
Test Plan:
I could pretty much consistently reproduce the deadlock with a program that
calls CompactRange(nullptr, nullptr) immediately after Write() from multiple
threads. This no longer happens with this patch.
Tests (make check) pass.
Reviewers: dhruba, igor, sdong, haobo
Reviewed By: igor
CC: leveldb
Differential Revision: https://reviews.facebook.net/D14799
2013-12-22 00:10:39 +01:00
|
|
|
// count how many background compactions are running or have been scheduled
|
2012-10-19 23:00:53 +02:00
|
|
|
int bg_compaction_scheduled_;
|
2011-03-18 23:37:00 +01:00
|
|
|
|
2015-10-17 09:16:36 +02:00
|
|
|
// stores the number of compactions are currently running
|
|
|
|
int num_running_compactions_;
|
|
|
|
|
2013-09-13 23:38:37 +02:00
|
|
|
// number of background memtable flush jobs, submitted to the HIGH pool
|
|
|
|
int bg_flush_scheduled_;
|
|
|
|
|
2015-10-17 09:16:36 +02:00
|
|
|
// stores the number of flushes are currently running
|
|
|
|
int num_running_flushes_;
|
|
|
|
|
2016-06-22 03:41:23 +02:00
|
|
|
// number of background obsolete file purge jobs, submitted to the HIGH pool
|
|
|
|
int bg_purge_scheduled_;
|
|
|
|
|
2017-08-04 00:36:28 +02:00
|
|
|
std::deque<ManualCompactionState*> manual_compaction_dequeue_;
|
Running manual compactions in parallel with other automatic or manual compactions in restricted cases
Summary:
This diff provides a framework for doing manual
compactions in parallel with other compactions. We now have a deque of manual compactions. We also pass manual compactions as an argument from RunManualCompactions down to
BackgroundCompactions, so that RunManualCompactions can be reentrant.
Parallelism is controlled by the two routines
ConflictingManualCompaction to allow/disallow new parallel/manual
compactions based on already existing ManualCompactions. In this diff, by default manual compactions still have to run exclusive of other compactions. However, by setting the compaction option, exclusive_manual_compaction to false, it is possible to run other compactions in parallel with a manual compaction. However, we are still restricted to one manual compaction per column family at a time. All of these restrictions will be relaxed in future diffs.
I will be adding more tests later.
Test Plan: Rocksdb regression + new tests + valgrind
Reviewers: igor, anthony, IslamAbdelRahman, kradhakrishnan, yhchiang, sdong
Reviewed By: sdong
Subscribers: yoshinorim, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D47973
2015-12-14 20:20:34 +01:00
|
|
|
|
2012-09-15 02:11:35 +02:00
|
|
|
// shall we disable deletion of obsolete files
|
2014-01-02 12:33:42 +01:00
|
|
|
// if 0 the deletion is enabled.
|
|
|
|
// if non-zero, files will not be getting deleted
|
|
|
|
// This enables two different threads to call
|
|
|
|
// EnableFileDeletions() and DisableFileDeletions()
|
|
|
|
// without any synchronization
|
|
|
|
int disable_delete_obsolete_files_;
|
2012-09-15 02:11:35 +02:00
|
|
|
|
2018-01-18 02:37:10 +01:00
|
|
|
// Number of times FindObsoleteFiles has found deletable files and the
|
|
|
|
// corresponding call to PurgeObsoleteFiles has not yet finished.
|
|
|
|
int pending_purge_obsolete_files_;
|
|
|
|
|
2019-05-14 02:43:47 +02:00
|
|
|
// last time when DeleteObsoleteFiles with full scan was executed. Originally
|
2016-12-05 23:09:35 +01:00
|
|
|
// initialized with startup time.
|
|
|
|
uint64_t delete_obsolete_files_last_run_;
|
2012-10-16 17:53:46 +02:00
|
|
|
|
2013-05-11 00:21:04 +02:00
|
|
|
// last time stats were dumped to LOG
|
2013-05-24 21:52:45 +02:00
|
|
|
std::atomic<uint64_t> last_stats_dump_time_microsec_;
|
2013-05-11 00:21:04 +02:00
|
|
|
|
2019-05-14 02:43:47 +02:00
|
|
|
// The thread that wants to switch memtable, can wait on this cv until the
|
|
|
|
// pending writes to memtable finishes.
|
|
|
|
std::condition_variable switch_cv_;
|
|
|
|
// The mutex used by switch_cv_. mutex_ should be acquired beforehand.
|
|
|
|
std::mutex switch_mutex_;
|
|
|
|
// Number of threads intending to write to memtable
|
|
|
|
std::atomic<size_t> pending_memtable_writes_ = {};
|
|
|
|
|
2015-02-12 18:54:48 +01:00
|
|
|
// Each flush or compaction gets its own job id. this counter makes sure
|
|
|
|
// they're unique
|
|
|
|
std::atomic<int> next_job_id_;
|
|
|
|
|
2016-02-09 20:20:22 +01:00
|
|
|
// A flag indicating whether the current rocksdb database has any
|
|
|
|
// data that is not yet persisted into either WAL or SST file.
|
|
|
|
// Used when disableWAL is true.
|
2017-02-14 03:44:21 +01:00
|
|
|
std::atomic<bool> has_unpersisted_data_;
|
2017-01-20 00:21:07 +01:00
|
|
|
|
|
|
|
// if an attempt was made to flush all column families that
|
2019-01-02 20:15:01 +01:00
|
|
|
// the oldest log depends on but uncommitted data in the oldest
|
2017-01-20 00:21:07 +01:00
|
|
|
// log prevents the log from being released.
|
|
|
|
// We must attempt to free the dependent memtables again
|
|
|
|
// at a later time after the transaction in the oldest
|
|
|
|
// log is fully commited.
|
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
|
|
|
bool unable_to_release_oldest_log_;
|
2017-01-20 00:21:07 +01:00
|
|
|
|
2012-08-18 01:06:05 +02:00
|
|
|
static const int KEEP_LOG_FILE_NUM = 1000;
|
2015-07-02 01:13:49 +02:00
|
|
|
// MSVC version 1800 still does not have constexpr for ::max()
|
2015-07-11 18:54:33 +02:00
|
|
|
static const uint64_t kNoTimeOut = port::kMaxUint64;
|
2015-07-02 01:13:49 +02:00
|
|
|
|
2012-09-06 02:44:13 +02:00
|
|
|
std::string db_absolute_path_;
|
2012-08-18 01:06:05 +02:00
|
|
|
|
Export Import sst files (#5495)
Summary:
Refresh of the earlier change here - https://github.com/facebook/rocksdb/issues/5135
This is a review request for code change needed for - https://github.com/facebook/rocksdb/issues/3469
"Add support for taking snapshot of a column family and creating column family from a given CF snapshot"
We have an implementation for this that we have been testing internally. We have two new APIs that together provide this functionality.
(1) ExportColumnFamily() - This API is modelled after CreateCheckpoint() as below.
// Exports all live SST files of a specified Column Family onto export_dir,
// returning SST files information in metadata.
// - SST files will be created as hard links when the directory specified
// is in the same partition as the db directory, copied otherwise.
// - export_dir should not already exist and will be created by this API.
// - Always triggers a flush.
virtual Status ExportColumnFamily(ColumnFamilyHandle* handle,
const std::string& export_dir,
ExportImportFilesMetaData** metadata);
Internally, the API will DisableFileDeletions(), GetColumnFamilyMetaData(), Parse through
metadata, creating links/copies of all the sst files, EnableFileDeletions() and complete the call by
returning the list of file metadata.
(2) CreateColumnFamilyWithImport() - This API is modeled after IngestExternalFile(), but invoked only during a CF creation as below.
// CreateColumnFamilyWithImport() will create a new column family with
// column_family_name and import external SST files specified in metadata into
// this column family.
// (1) External SST files can be created using SstFileWriter.
// (2) External SST files can be exported from a particular column family in
// an existing DB.
// Option in import_options specifies whether the external files are copied or
// moved (default is copy). When option specifies copy, managing files at
// external_file_path is caller's responsibility. When option specifies a
// move, the call ensures that the specified files at external_file_path are
// deleted on successful return and files are not modified on any error
// return.
// On error return, column family handle returned will be nullptr.
// ColumnFamily will be present on successful return and will not be present
// on error return. ColumnFamily may be present on any crash during this call.
virtual Status CreateColumnFamilyWithImport(
const ColumnFamilyOptions& options, const std::string& column_family_name,
const ImportColumnFamilyOptions& import_options,
const ExportImportFilesMetaData& metadata,
ColumnFamilyHandle** handle);
Internally, this API creates a new CF, parses all the sst files and adds it to the specified column family, at the same level and with same sequence number as in the metadata. Also performs safety checks with respect to overlaps between the sst files being imported.
If incoming sequence number is higher than current local sequence number, local sequence
number is updated to reflect this.
Note, as the sst files is are being moved across Column Families, Column Family name in sst file
will no longer match the actual column family on destination DB. The API does not modify Column
Family name or id in the sst files being imported.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/5495
Differential Revision: D16018881
fbshipit-source-id: 9ae2251025d5916d35a9fc4ea4d6707f6be16ff9
2019-07-17 21:22:21 +02:00
|
|
|
// Number of running IngestExternalFile() or CreateColumnFamilyWithImport()
|
|
|
|
// calls.
|
2016-09-26 23:44:48 +02:00
|
|
|
// REQUIRES: mutex held
|
2016-10-21 02:05:32 +02:00
|
|
|
int num_running_ingest_file_;
|
2016-09-26 23:44:48 +02:00
|
|
|
|
2014-10-30 01:43:37 +01:00
|
|
|
#ifndef ROCKSDB_LITE
|
|
|
|
WalManager wal_manager_;
|
|
|
|
#endif // ROCKSDB_LITE
|
|
|
|
|
EventLogger
Summary:
Here's my proposal for making our LOGs easier to read by machines.
The idea is to dump all events as JSON objects. JSON is easy to read by humans, but more importantly, it's easy to read by machines. That way, we can parse this, load into SQLite/mongo and then query or visualize.
I started with table_create and table_delete events, but if everybody agrees, I'll continue by adding more events (flush/compaction/etc etc)
Test Plan:
Ran db_bench. Observed:
2015/01/15-14:13:25.788019 1105ef000 EVENT_LOG_v1 {"time_micros": 1421360005788015, "event": "table_file_creation", "file_number": 12, "file_size": 1909699}
2015/01/15-14:13:25.956500 110740000 EVENT_LOG_v1 {"time_micros": 1421360005956498, "event": "table_file_deletion", "file_number": 12}
Reviewers: yhchiang, rven, dhruba, MarkCallaghan, lgalanis, sdong
Reviewed By: sdong
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D31647
2015-03-13 18:15:54 +01:00
|
|
|
// Unified interface for logging events
|
|
|
|
EventLogger event_logger_;
|
|
|
|
|
Fix a bug where flush does not happen when a manual compaction is running
Summary:
Currently, when rocksdb tries to run manual compaction to refit data into a level,
there's a ReFitLevel() process that requires no bg work is currently running.
When RocksDB plans to ReFitLevel(), it will do the following:
1. pause scheduling new bg work.
2. wait until all bg work finished
3. do the ReFitLevel()
4. unpause scheduling new bg work.
However, as it pause scheduling new bg work at step one and waiting for all bg work
finished in step 2, RocksDB will stop flushing until all bg work is done (which
could take a long time.)
This patch fix this issue by changing the way ReFitLevel() pause the background work:
1. pause scheduling compaction.
2. wait until all bg work finished.
3. pause scheduling flush
4. do ReFitLevel()
5. unpause both flush and compaction.
The major difference is that. We only pause scheduling compaction in step 1 and wait
for all bg work finished in step 2. This prevent flush being blocked for a long time.
Although there's a very rare case that ReFitLevel() might be in starvation in step 2,
but it's less likely the case as flush typically finish very fast.
Test Plan: existing test.
Reviewers: anthony, IslamAbdelRahman, kradhakrishnan, sdong
Reviewed By: sdong
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D55029
2016-03-04 23:24:52 +01:00
|
|
|
// A value of > 0 temporarily disables scheduling of background work
|
2015-10-02 22:17:34 +02:00
|
|
|
int bg_work_paused_;
|
2013-06-30 08:21:36 +02:00
|
|
|
|
Fix a bug where flush does not happen when a manual compaction is running
Summary:
Currently, when rocksdb tries to run manual compaction to refit data into a level,
there's a ReFitLevel() process that requires no bg work is currently running.
When RocksDB plans to ReFitLevel(), it will do the following:
1. pause scheduling new bg work.
2. wait until all bg work finished
3. do the ReFitLevel()
4. unpause scheduling new bg work.
However, as it pause scheduling new bg work at step one and waiting for all bg work
finished in step 2, RocksDB will stop flushing until all bg work is done (which
could take a long time.)
This patch fix this issue by changing the way ReFitLevel() pause the background work:
1. pause scheduling compaction.
2. wait until all bg work finished.
3. pause scheduling flush
4. do ReFitLevel()
5. unpause both flush and compaction.
The major difference is that. We only pause scheduling compaction in step 1 and wait
for all bg work finished in step 2. This prevent flush being blocked for a long time.
Although there's a very rare case that ReFitLevel() might be in starvation in step 2,
but it's less likely the case as flush typically finish very fast.
Test Plan: existing test.
Reviewers: anthony, IslamAbdelRahman, kradhakrishnan, sdong
Reviewed By: sdong
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D55029
2016-03-04 23:24:52 +01:00
|
|
|
// A value of > 0 temporarily disables scheduling of background compaction
|
|
|
|
int bg_compaction_paused_;
|
|
|
|
|
2013-06-30 08:21:36 +02:00
|
|
|
// Guard against multiple concurrent refitting
|
|
|
|
bool refitting_level_;
|
|
|
|
|
2014-02-27 20:38:55 +01:00
|
|
|
// Indicate DB was opened successfully
|
|
|
|
bool opened_successfully_;
|
|
|
|
|
2019-03-26 03:14:04 +01:00
|
|
|
// The min threshold to triggere bottommost compaction for removing
|
|
|
|
// garbages, among all column families.
|
|
|
|
SequenceNumber bottommost_files_mark_threshold_ = kMaxSequenceNumber;
|
|
|
|
|
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
|
|
|
LogsWithPrepTracker logs_with_prep_tracker_;
|
2016-04-18 20:11:51 +02:00
|
|
|
|
2017-10-06 19:26:38 +02:00
|
|
|
// Callback for compaction to check if a key is visible to a snapshot.
|
|
|
|
// REQUIRES: mutex held
|
|
|
|
std::unique_ptr<SnapshotChecker> snapshot_checker_;
|
|
|
|
|
2018-03-28 21:01:09 +02:00
|
|
|
// Callback for when the cached_recoverable_state_ is written to memtable
|
|
|
|
// Only to be set during initialization
|
|
|
|
std::unique_ptr<PreReleaseCallback> recoverable_state_pre_release_callback_;
|
|
|
|
|
2020-08-15 05:11:35 +02:00
|
|
|
#ifndef ROCKSDB_LITE
|
2020-10-02 04:12:26 +02:00
|
|
|
// Scheduler to run DumpStats(), PersistStats(), and FlushInfoLog().
|
|
|
|
// Currently, it always use a global instance from
|
|
|
|
// PeriodicWorkScheduler::Default(). Only in unittest, it can be overrided by
|
|
|
|
// PeriodicWorkTestScheduler.
|
|
|
|
PeriodicWorkScheduler* periodic_work_scheduler_;
|
2020-08-15 05:11:35 +02:00
|
|
|
#endif
|
2019-02-21 00:46:59 +01:00
|
|
|
|
2020-02-28 23:10:51 +01:00
|
|
|
// When set, we use a separate queue for writes that don't write to memtable.
|
2018-03-02 02:50:54 +01:00
|
|
|
// In 2PC these are the writes at Prepare phase.
|
2017-11-11 02:18:01 +01:00
|
|
|
const bool two_write_queues_;
|
2017-06-24 23:06:43 +02:00
|
|
|
const bool manual_wal_flush_;
|
2019-04-24 21:05:29 +02:00
|
|
|
|
2017-12-01 08:39:56 +01:00
|
|
|
// LastSequence also indicates last published sequence visibile to the
|
|
|
|
// readers. Otherwise LastPublishedSequence should be used.
|
|
|
|
const bool last_seq_same_as_publish_seq_;
|
2017-11-11 02:18:01 +01:00
|
|
|
// It indicates that a customized gc algorithm must be used for
|
|
|
|
// flush/compaction and if it is not provided vis SnapshotChecker, we should
|
|
|
|
// disable gc to be safe.
|
2017-10-18 18:09:31 +02:00
|
|
|
const bool use_custom_gc_;
|
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
|
|
|
// Flag to indicate that the DB instance shutdown has been initiated. This
|
|
|
|
// different from shutting_down_ atomic in that it is set at the beginning
|
|
|
|
// of shutdown sequence, specifically in order to prevent any background
|
|
|
|
// error recovery from going on in parallel. The latter, shutting_down_,
|
|
|
|
// is set a little later during the shutdown after scheduling memtable
|
|
|
|
// flushes
|
move dump stats to a separate thread (#4382)
Summary:
Currently statistics are supposed to be dumped to info log at intervals of `options.stats_dump_period_sec`. However the implementation choice was to bind it with compaction thread, meaning if the database has been serving very light traffic, the stats may not get dumped at all.
We decided to separate stats dumping into a new timed thread using `TimerQueue`, which is already used in blob_db. This will allow us schedule new timed tasks with more deterministic behavior.
Tested with db_bench using `--stats_dump_period_sec=20` in command line:
> LOG:2018/09/17-14:07:45.575025 7fe99fbfe700 [WARN] [db/db_impl.cc:605] ------- DUMPING STATS -------
LOG:2018/09/17-14:08:05.643286 7fe99fbfe700 [WARN] [db/db_impl.cc:605] ------- DUMPING STATS -------
LOG:2018/09/17-14:08:25.691325 7fe99fbfe700 [WARN] [db/db_impl.cc:605] ------- DUMPING STATS -------
LOG:2018/09/17-14:08:45.740989 7fe99fbfe700 [WARN] [db/db_impl.cc:605] ------- DUMPING STATS -------
LOG content:
> 2018/09/17-14:07:45.575025 7fe99fbfe700 [WARN] [db/db_impl.cc:605] ------- DUMPING STATS -------
2018/09/17-14:07:45.575080 7fe99fbfe700 [WARN] [db/db_impl.cc:606]
** DB Stats **
Uptime(secs): 20.0 total, 20.0 interval
Cumulative writes: 4447K writes, 4447K keys, 4447K commit groups, 1.0 writes per commit group, ingest: 5.57 GB, 285.01 MB/s
Cumulative WAL: 4447K writes, 0 syncs, 4447638.00 writes per sync, written: 5.57 GB, 285.01 MB/s
Cumulative stall: 00:00:0.012 H:M:S, 0.1 percent
Interval writes: 4447K writes, 4447K keys, 4447K commit groups, 1.0 writes per commit group, ingest: 5700.71 MB, 285.01 MB/s
Interval WAL: 4447K writes, 0 syncs, 4447638.00 writes per sync, written: 5.57 MB, 285.01 MB/s
Interval stall: 00:00:0.012 H:M:S, 0.1 percent
** Compaction Stats [default] **
Level Files Size Score Read(GB) Rn(GB) Rnp1(GB) Write(GB) Wnew(GB) Moved(GB) W-Amp Rd(MB/s) Wr(MB/s) Comp(sec) Comp(cnt) Avg(sec) KeyIn KeyDrop
Pull Request resolved: https://github.com/facebook/rocksdb/pull/4382
Differential Revision: D9933051
Pulled By: miasantreble
fbshipit-source-id: 6d12bb1e4977674eea4bf2d2ac6d486b814bb2fa
2018-10-09 07:52:58 +02:00
|
|
|
std::atomic<bool> shutdown_initiated_;
|
2018-09-17 22:08:13 +02:00
|
|
|
// Flag to indicate whether sst_file_manager object was allocated in
|
|
|
|
// DB::Open() or passed to us
|
|
|
|
bool own_sfm_;
|
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
|
|
|
|
|
|
|
// Clients must periodically call SetPreserveDeletesSequenceNumber()
|
|
|
|
// to advance this seqnum. Default value is 0 which means ALL deletes are
|
|
|
|
// preserved. Note that this has no effect if DBOptions.preserve_deletes
|
|
|
|
// is set to false.
|
|
|
|
std::atomic<SequenceNumber> preserve_deletes_seqnum_;
|
|
|
|
const bool preserve_deletes_;
|
2018-01-16 19:57:56 +01:00
|
|
|
|
|
|
|
// Flag to check whether Close() has been called on this DB
|
|
|
|
bool closed_;
|
2018-06-28 21:23:57 +02:00
|
|
|
|
|
|
|
ErrorHandler error_handler_;
|
2018-10-16 04:59:20 +02:00
|
|
|
|
2019-01-04 05:53:52 +01:00
|
|
|
// Conditional variable to coordinate installation of atomic flush results.
|
|
|
|
// With atomic flush, each bg thread installs the result of flushing multiple
|
|
|
|
// column families, and different threads can flush different column
|
|
|
|
// families. It's difficult to rely on one thread to perform batch
|
|
|
|
// installation for all threads. This is different from the non-atomic flush
|
|
|
|
// case.
|
|
|
|
// atomic_flush_install_cv_ makes sure that threads install atomic flush
|
|
|
|
// results sequentially. Flush results of memtables with lower IDs get
|
|
|
|
// installed to MANIFEST first.
|
|
|
|
InstrumentedCondVar atomic_flush_install_cv_;
|
2019-07-07 06:04:22 +02:00
|
|
|
|
|
|
|
bool wal_in_db_path_;
|
2011-03-18 23:37:00 +01:00
|
|
|
};
|
|
|
|
|
2019-03-28 00:13:08 +01:00
|
|
|
extern Options SanitizeOptions(const std::string& db, const Options& src);
|
2016-09-24 01:34:04 +02:00
|
|
|
|
2014-02-05 01:31:18 +01:00
|
|
|
extern DBOptions SanitizeOptions(const std::string& db, const DBOptions& src);
|
2013-10-30 18:52:33 +01:00
|
|
|
|
2017-04-06 02:14:05 +02:00
|
|
|
extern CompressionType GetCompressionFlush(
|
|
|
|
const ImmutableCFOptions& ioptions,
|
|
|
|
const MutableCFOptions& mutable_cf_options);
|
2017-05-24 20:25:38 +02:00
|
|
|
|
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
|
|
|
// Return the earliest log file to keep after the memtable flush is
|
|
|
|
// finalized.
|
|
|
|
// `cfd_to_flush` is the column family whose memtable (specified in
|
|
|
|
// `memtables_to_flush`) will be flushed and thus will not depend on any WAL
|
|
|
|
// file.
|
|
|
|
// The function is only applicable to 2pc mode.
|
2020-11-07 01:30:44 +01:00
|
|
|
extern uint64_t PrecomputeMinLogNumberToKeep2PC(
|
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
|
|
|
VersionSet* vset, const ColumnFamilyData& cfd_to_flush,
|
2020-11-07 01:30:44 +01:00
|
|
|
const autovector<VersionEdit*>& edit_list,
|
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
|
|
|
const autovector<MemTable*>& memtables_to_flush,
|
|
|
|
LogsWithPrepTracker* prep_tracker);
|
2020-12-04 04:21:08 +01:00
|
|
|
// For atomic flush.
|
|
|
|
extern uint64_t PrecomputeMinLogNumberToKeep2PC(
|
|
|
|
VersionSet* vset, const autovector<ColumnFamilyData*>& cfds_to_flush,
|
|
|
|
const autovector<autovector<VersionEdit*>>& edit_lists,
|
|
|
|
const autovector<const autovector<MemTable*>*>& memtables_to_flush,
|
|
|
|
LogsWithPrepTracker* prep_tracker);
|
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
|
|
|
|
2020-11-07 01:30:44 +01:00
|
|
|
// In non-2PC mode, WALs with log number < the returned number can be
|
|
|
|
// deleted after the cfd_to_flush column family is flushed successfully.
|
|
|
|
extern uint64_t PrecomputeMinLogNumberToKeepNon2PC(
|
|
|
|
VersionSet* vset, const ColumnFamilyData& cfd_to_flush,
|
|
|
|
const autovector<VersionEdit*>& edit_list);
|
2020-11-18 00:54:49 +01:00
|
|
|
// For atomic flush.
|
|
|
|
extern uint64_t PrecomputeMinLogNumberToKeepNon2PC(
|
|
|
|
VersionSet* vset, const autovector<ColumnFamilyData*>& cfds_to_flush,
|
|
|
|
const autovector<autovector<VersionEdit*>>& edit_lists);
|
2020-11-07 01:30:44 +01:00
|
|
|
|
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_to_flush` is the column family whose memtable will be flushed and thus
|
|
|
|
// will not depend on any WAL file. nullptr means no memtable is being flushed.
|
|
|
|
// The function is only applicable to 2pc mode.
|
|
|
|
extern uint64_t FindMinPrepLogReferencedByMemTable(
|
|
|
|
VersionSet* vset, const ColumnFamilyData* cfd_to_flush,
|
|
|
|
const autovector<MemTable*>& memtables_to_flush);
|
2020-12-04 04:21:08 +01:00
|
|
|
// For atomic flush.
|
|
|
|
extern uint64_t FindMinPrepLogReferencedByMemTable(
|
|
|
|
VersionSet* vset, const autovector<ColumnFamilyData*>& cfds_to_flush,
|
|
|
|
const autovector<const autovector<MemTable*>*>& memtables_to_flush);
|
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
|
|
|
|
2014-08-07 19:05:04 +02:00
|
|
|
// Fix user-supplied options to be reasonable
|
|
|
|
template <class T, class V>
|
|
|
|
static void ClipToRange(T* ptr, V minvalue, V maxvalue) {
|
|
|
|
if (static_cast<V>(*ptr) > maxvalue) *ptr = maxvalue;
|
|
|
|
if (static_cast<V>(*ptr) < minvalue) *ptr = minvalue;
|
|
|
|
}
|
|
|
|
|
2020-02-20 21:07:53 +01:00
|
|
|
} // namespace ROCKSDB_NAMESPACE
|