You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
rocksdb/db/transaction_log_impl.cc

299 lines
10 KiB

// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
// This source code is licensed under both the GPLv2 (found in the
// COPYING file in the root directory) and Apache 2.0 License
// (found in the LICENSE.Apache file in the root directory).
#ifndef ROCKSDB_LITE
#include "db/transaction_log_impl.h"
#include <cinttypes>
#include "db/write_batch_internal.h"
#include "file/sequence_file_reader.h"
#include "util/defer.h"
namespace ROCKSDB_NAMESPACE {
TransactionLogIteratorImpl::TransactionLogIteratorImpl(
const std::string& dir, const ImmutableDBOptions* options,
const TransactionLogIterator::ReadOptions& read_options,
const EnvOptions& soptions, const SequenceNumber seq,
std::unique_ptr<VectorLogPtr> files, VersionSet const* const versions,
const bool seq_per_batch, const std::shared_ptr<IOTracer>& io_tracer)
: dir_(dir),
options_(options),
read_options_(read_options),
soptions_(soptions),
starting_sequence_number_(seq),
files_(std::move(files)),
versions_(versions),
seq_per_batch_(seq_per_batch),
io_tracer_(io_tracer),
started_(false),
is_valid_(false),
current_file_index_(0),
current_batch_seq_(0),
current_last_seq_(0) {
assert(files_ != nullptr);
assert(versions_ != nullptr);
assert(!seq_per_batch_);
current_status_.PermitUncheckedError(); // Clear on start
reporter_.env = options_->env;
reporter_.info_log = options_->info_log.get();
SeekToStartSequence(); // Seek till starting sequence
}
Status TransactionLogIteratorImpl::OpenLogFile(
const LogFile* log_file,
std::unique_ptr<SequentialFileReader>* file_reader) {
FileSystemPtr fs(options_->fs, io_tracer_);
Introduce a new storage specific Env API (#5761) Summary: The current Env API encompasses both storage/file operations, as well as OS related operations. Most of the APIs return a Status, which does not have enough metadata about an error, such as whether its retry-able or not, scope (i.e fault domain) of the error etc., that may be required in order to properly handle a storage error. The file APIs also do not provide enough control over the IO SLA, such as timeout, prioritization, hinting about placement and redundancy etc. This PR separates out the file/storage APIs from Env into a new FileSystem class. The APIs are updated to return an IOStatus with metadata about the error, as well as to take an IOOptions structure as input in order to allow more control over the IO. The user can set both ```options.env``` and ```options.file_system``` to specify that RocksDB should use the former for OS related operations and the latter for storage operations. Internally, a ```CompositeEnvWrapper``` has been introduced that inherits from ```Env``` and redirects individual methods to either an ```Env``` implementation or the ```FileSystem``` as appropriate. When options are sanitized during ```DB::Open```, ```options.env``` is replaced with a newly allocated ```CompositeEnvWrapper``` instance if both env and file_system have been specified. This way, the rest of the RocksDB code can continue to function as before. This PR also ports PosixEnv to the new API by splitting it into two - PosixEnv and PosixFileSystem. PosixEnv is defined as a sub-class of CompositeEnvWrapper, and threading/time functions are overridden with Posix specific implementations in order to avoid an extra level of indirection. The ```CompositeEnvWrapper``` translates ```IOStatus``` return code to ```Status```, and sets the severity to ```kSoftError``` if the io_status is retryable. The error handling code in RocksDB can then recover the DB automatically. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5761 Differential Revision: D18868376 Pulled By: anand1976 fbshipit-source-id: 39efe18a162ea746fabac6360ff529baba48486f
5 years ago
std::unique_ptr<FSSequentialFile> file;
std::string fname;
Status s;
Introduce a new storage specific Env API (#5761) Summary: The current Env API encompasses both storage/file operations, as well as OS related operations. Most of the APIs return a Status, which does not have enough metadata about an error, such as whether its retry-able or not, scope (i.e fault domain) of the error etc., that may be required in order to properly handle a storage error. The file APIs also do not provide enough control over the IO SLA, such as timeout, prioritization, hinting about placement and redundancy etc. This PR separates out the file/storage APIs from Env into a new FileSystem class. The APIs are updated to return an IOStatus with metadata about the error, as well as to take an IOOptions structure as input in order to allow more control over the IO. The user can set both ```options.env``` and ```options.file_system``` to specify that RocksDB should use the former for OS related operations and the latter for storage operations. Internally, a ```CompositeEnvWrapper``` has been introduced that inherits from ```Env``` and redirects individual methods to either an ```Env``` implementation or the ```FileSystem``` as appropriate. When options are sanitized during ```DB::Open```, ```options.env``` is replaced with a newly allocated ```CompositeEnvWrapper``` instance if both env and file_system have been specified. This way, the rest of the RocksDB code can continue to function as before. This PR also ports PosixEnv to the new API by splitting it into two - PosixEnv and PosixFileSystem. PosixEnv is defined as a sub-class of CompositeEnvWrapper, and threading/time functions are overridden with Posix specific implementations in order to avoid an extra level of indirection. The ```CompositeEnvWrapper``` translates ```IOStatus``` return code to ```Status```, and sets the severity to ```kSoftError``` if the io_status is retryable. The error handling code in RocksDB can then recover the DB automatically. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5761 Differential Revision: D18868376 Pulled By: anand1976 fbshipit-source-id: 39efe18a162ea746fabac6360ff529baba48486f
5 years ago
EnvOptions optimized_env_options = fs->OptimizeForLogRead(soptions_);
if (log_file->Type() == kArchivedLogFile) {
fname = ArchivedLogFileName(dir_, log_file->LogNumber());
Introduce a new storage specific Env API (#5761) Summary: The current Env API encompasses both storage/file operations, as well as OS related operations. Most of the APIs return a Status, which does not have enough metadata about an error, such as whether its retry-able or not, scope (i.e fault domain) of the error etc., that may be required in order to properly handle a storage error. The file APIs also do not provide enough control over the IO SLA, such as timeout, prioritization, hinting about placement and redundancy etc. This PR separates out the file/storage APIs from Env into a new FileSystem class. The APIs are updated to return an IOStatus with metadata about the error, as well as to take an IOOptions structure as input in order to allow more control over the IO. The user can set both ```options.env``` and ```options.file_system``` to specify that RocksDB should use the former for OS related operations and the latter for storage operations. Internally, a ```CompositeEnvWrapper``` has been introduced that inherits from ```Env``` and redirects individual methods to either an ```Env``` implementation or the ```FileSystem``` as appropriate. When options are sanitized during ```DB::Open```, ```options.env``` is replaced with a newly allocated ```CompositeEnvWrapper``` instance if both env and file_system have been specified. This way, the rest of the RocksDB code can continue to function as before. This PR also ports PosixEnv to the new API by splitting it into two - PosixEnv and PosixFileSystem. PosixEnv is defined as a sub-class of CompositeEnvWrapper, and threading/time functions are overridden with Posix specific implementations in order to avoid an extra level of indirection. The ```CompositeEnvWrapper``` translates ```IOStatus``` return code to ```Status```, and sets the severity to ```kSoftError``` if the io_status is retryable. The error handling code in RocksDB can then recover the DB automatically. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5761 Differential Revision: D18868376 Pulled By: anand1976 fbshipit-source-id: 39efe18a162ea746fabac6360ff529baba48486f
5 years ago
s = fs->NewSequentialFile(fname, optimized_env_options, &file, nullptr);
} else {
fname = LogFileName(dir_, log_file->LogNumber());
Introduce a new storage specific Env API (#5761) Summary: The current Env API encompasses both storage/file operations, as well as OS related operations. Most of the APIs return a Status, which does not have enough metadata about an error, such as whether its retry-able or not, scope (i.e fault domain) of the error etc., that may be required in order to properly handle a storage error. The file APIs also do not provide enough control over the IO SLA, such as timeout, prioritization, hinting about placement and redundancy etc. This PR separates out the file/storage APIs from Env into a new FileSystem class. The APIs are updated to return an IOStatus with metadata about the error, as well as to take an IOOptions structure as input in order to allow more control over the IO. The user can set both ```options.env``` and ```options.file_system``` to specify that RocksDB should use the former for OS related operations and the latter for storage operations. Internally, a ```CompositeEnvWrapper``` has been introduced that inherits from ```Env``` and redirects individual methods to either an ```Env``` implementation or the ```FileSystem``` as appropriate. When options are sanitized during ```DB::Open```, ```options.env``` is replaced with a newly allocated ```CompositeEnvWrapper``` instance if both env and file_system have been specified. This way, the rest of the RocksDB code can continue to function as before. This PR also ports PosixEnv to the new API by splitting it into two - PosixEnv and PosixFileSystem. PosixEnv is defined as a sub-class of CompositeEnvWrapper, and threading/time functions are overridden with Posix specific implementations in order to avoid an extra level of indirection. The ```CompositeEnvWrapper``` translates ```IOStatus``` return code to ```Status```, and sets the severity to ```kSoftError``` if the io_status is retryable. The error handling code in RocksDB can then recover the DB automatically. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5761 Differential Revision: D18868376 Pulled By: anand1976 fbshipit-source-id: 39efe18a162ea746fabac6360ff529baba48486f
5 years ago
s = fs->NewSequentialFile(fname, optimized_env_options, &file, nullptr);
if (!s.ok()) {
// If cannot open file in DB directory.
// Try the archive dir, as it could have moved in the meanwhile.
fname = ArchivedLogFileName(dir_, log_file->LogNumber());
Introduce a new storage specific Env API (#5761) Summary: The current Env API encompasses both storage/file operations, as well as OS related operations. Most of the APIs return a Status, which does not have enough metadata about an error, such as whether its retry-able or not, scope (i.e fault domain) of the error etc., that may be required in order to properly handle a storage error. The file APIs also do not provide enough control over the IO SLA, such as timeout, prioritization, hinting about placement and redundancy etc. This PR separates out the file/storage APIs from Env into a new FileSystem class. The APIs are updated to return an IOStatus with metadata about the error, as well as to take an IOOptions structure as input in order to allow more control over the IO. The user can set both ```options.env``` and ```options.file_system``` to specify that RocksDB should use the former for OS related operations and the latter for storage operations. Internally, a ```CompositeEnvWrapper``` has been introduced that inherits from ```Env``` and redirects individual methods to either an ```Env``` implementation or the ```FileSystem``` as appropriate. When options are sanitized during ```DB::Open```, ```options.env``` is replaced with a newly allocated ```CompositeEnvWrapper``` instance if both env and file_system have been specified. This way, the rest of the RocksDB code can continue to function as before. This PR also ports PosixEnv to the new API by splitting it into two - PosixEnv and PosixFileSystem. PosixEnv is defined as a sub-class of CompositeEnvWrapper, and threading/time functions are overridden with Posix specific implementations in order to avoid an extra level of indirection. The ```CompositeEnvWrapper``` translates ```IOStatus``` return code to ```Status```, and sets the severity to ```kSoftError``` if the io_status is retryable. The error handling code in RocksDB can then recover the DB automatically. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5761 Differential Revision: D18868376 Pulled By: anand1976 fbshipit-source-id: 39efe18a162ea746fabac6360ff529baba48486f
5 years ago
s = fs->NewSequentialFile(fname, optimized_env_options,
&file, nullptr);
}
}
if (s.ok()) {
Support read rate-limiting in SequentialFileReader (#9973) Summary: Added rate limiter and read rate-limiting support to SequentialFileReader. I've updated call sites to SequentialFileReader::Read with appropriate IO priority (or left a TODO and specified IO_TOTAL for now). The PR is separated into four commits: the first one added the rate-limiting support, but with some fixes in the unit test since the number of request bytes from rate limiter in SequentialFileReader are not accurate (there is overcharge at EOF). The second commit fixed this by allowing SequentialFileReader to check file size and determine how many bytes are left in the file to read. The third commit added benchmark related code. The fourth commit moved the logic of using file size to avoid overcharging the rate limiter into backup engine (the main user of SequentialFileReader). Pull Request resolved: https://github.com/facebook/rocksdb/pull/9973 Test Plan: - `make check`, backup_engine_test covers usage of SequentialFileReader with rate limiter. - Run db_bench to check if rate limiting is throttling as expected: Verified that reads and writes are together throttled at 2MB/s, and at 0.2MB chunks that are 100ms apart. - Set up: `./db_bench --benchmarks=fillrandom -db=/dev/shm/test_rocksdb` - Benchmark: ``` strace -ttfe read,write ./db_bench --benchmarks=backup -db=/dev/shm/test_rocksdb --backup_rate_limit=2097152 --use_existing_db strace -ttfe read,write ./db_bench --benchmarks=restore -db=/dev/shm/test_rocksdb --restore_rate_limit=2097152 --use_existing_db ``` - db bench on backup and restore to ensure no performance regression. - backup (avg over 50 runs): pre-change: 1.90443e+06 micros/op; post-change: 1.8993e+06 micros/op (improve by 0.2%) - restore (avg over 50 runs): pre-change: 1.79105e+06 micros/op; post-change: 1.78192e+06 micros/op (improve by 0.5%) ``` # Set up ./db_bench --benchmarks=fillrandom -db=/tmp/test_rocksdb -num=10000000 # benchmark TEST_TMPDIR=/tmp/test_rocksdb NUM_RUN=50 for ((j=0;j<$NUM_RUN;j++)) do ./db_bench -db=$TEST_TMPDIR -num=10000000 -benchmarks=backup -use_existing_db | egrep 'backup' # Restore #./db_bench -db=$TEST_TMPDIR -num=10000000 -benchmarks=restore -use_existing_db done > rate_limit.txt && awk -v NUM_RUN=$NUM_RUN '{sum+=$3;sum_sqrt+=$3^2}END{print sum/NUM_RUN, sqrt(sum_sqrt/NUM_RUN-(sum/NUM_RUN)^2)}' rate_limit.txt >> rate_limit_2.txt ``` Reviewed By: hx235 Differential Revision: D36327418 Pulled By: cbi42 fbshipit-source-id: e75d4307cff815945482df5ba630c1e88d064691
2 years ago
file_reader->reset(new SequentialFileReader(std::move(file), fname,
io_tracer_, options_->listeners,
options_->rate_limiter.get()));
}
return s;
}
BatchResult TransactionLogIteratorImpl::GetBatch() {
assert(is_valid_); // cannot call in a non valid state.
BatchResult result;
result.sequence = current_batch_seq_;
result.writeBatchPtr = std::move(current_batch_);
return result;
}
Status TransactionLogIteratorImpl::status() { return current_status_; }
bool TransactionLogIteratorImpl::Valid() { return started_ && is_valid_; }
bool TransactionLogIteratorImpl::RestrictedRead(Slice* record) {
// Don't read if no more complete entries to read from logs
if (current_last_seq_ >= versions_->LastSequence()) {
return false;
}
return current_log_reader_->ReadRecord(record, &scratch_);
}
void TransactionLogIteratorImpl::SeekToStartSequence(uint64_t start_file_index,
bool strict) {
Slice record;
started_ = false;
is_valid_ = false;
// Check invariant of TransactionLogIterator when SeekToStartSequence()
// succeeds.
const Defer defer([this]() {
if (is_valid_) {
assert(current_status_.ok());
if (starting_sequence_number_ > current_batch_seq_) {
assert(current_batch_seq_ < current_last_seq_);
assert(current_last_seq_ >= starting_sequence_number_);
}
}
});
if (files_->size() <= start_file_index) {
return;
} else if (!current_status_.ok()) {
return;
}
Status s =
OpenLogReader(files_->at(static_cast<size_t>(start_file_index)).get());
if (!s.ok()) {
current_status_ = s;
reporter_.Info(current_status_.ToString().c_str());
return;
}
while (RestrictedRead(&record)) {
if (record.size() < WriteBatchInternal::kHeader) {
reporter_.Corruption(
record.size(), Status::Corruption("very small log record"));
continue;
}
UpdateCurrentWriteBatch(record);
if (current_last_seq_ >= starting_sequence_number_) {
if (strict && current_batch_seq_ != starting_sequence_number_) {
current_status_ = Status::Corruption(
"Gap in sequence number. Could not "
"seek to required sequence number");
reporter_.Info(current_status_.ToString().c_str());
return;
} else if (strict) {
reporter_.Info("Could seek required sequence number. Iterator will "
"continue.");
}
is_valid_ = true;
started_ = true; // set started_ as we could seek till starting sequence
return;
} else {
is_valid_ = false;
}
}
// Could not find start sequence in first file. Normally this must be the
// only file. Otherwise log the error and let the iterator return next entry
// If strict is set, we want to seek exactly till the start sequence and it
// should have been present in the file we scanned above
if (strict) {
current_status_ = Status::Corruption(
"Gap in sequence number. Could not "
"seek to required sequence number");
reporter_.Info(current_status_.ToString().c_str());
} else if (files_->size() != 1) {
current_status_ = Status::Corruption(
"Start sequence was not found, "
"skipping to the next available");
reporter_.Info(current_status_.ToString().c_str());
// Let NextImpl find the next available entry. started_ remains false
// because we don't want to check for gaps while moving to start sequence
NextImpl(true);
}
}
void TransactionLogIteratorImpl::Next() {
if (!current_status_.ok()) {
return;
}
return NextImpl(false);
}
void TransactionLogIteratorImpl::NextImpl(bool internal) {
Slice record;
is_valid_ = false;
if (!internal && !started_) {
// Runs every time until we can seek to the start sequence
SeekToStartSequence();
}
while(true) {
assert(current_log_reader_);
if (current_log_reader_->IsEOF()) {
current_log_reader_->UnmarkEOF();
}
while (RestrictedRead(&record)) {
if (record.size() < WriteBatchInternal::kHeader) {
reporter_.Corruption(
record.size(), Status::Corruption("very small log record"));
continue;
} else {
// started_ should be true if called by application
assert(internal || started_);
// started_ should be false if called internally
assert(!internal || !started_);
UpdateCurrentWriteBatch(record);
if (internal && !started_) {
started_ = true;
}
return;
}
}
// Open the next file
if (current_file_index_ < files_->size() - 1) {
++current_file_index_;
Status s = OpenLogReader(files_->at(current_file_index_).get());
if (!s.ok()) {
is_valid_ = false;
current_status_ = s;
return;
}
} else {
is_valid_ = false;
if (current_last_seq_ == versions_->LastSequence()) {
current_status_ = Status::OK();
} else {
const char* msg = "Create a new iterator to fetch the new tail.";
current_status_ = Status::TryAgain(msg);
}
return;
}
}
}
bool TransactionLogIteratorImpl::IsBatchExpected(
const WriteBatch* batch, const SequenceNumber expected_seq) {
assert(batch);
SequenceNumber batchSeq = WriteBatchInternal::Sequence(batch);
if (batchSeq != expected_seq) {
char buf[200];
snprintf(buf, sizeof(buf),
"Discontinuity in log records. Got seq=%" PRIu64
", Expected seq=%" PRIu64 ", Last flushed seq=%" PRIu64
".Log iterator will reseek the correct batch.",
batchSeq, expected_seq, versions_->LastSequence());
reporter_.Info(buf);
return false;
}
return true;
}
void TransactionLogIteratorImpl::UpdateCurrentWriteBatch(const Slice& record) {
std::unique_ptr<WriteBatch> batch(new WriteBatch());
Status s = WriteBatchInternal::SetContents(batch.get(), record);
s.PermitUncheckedError(); // TODO: What should we do with this error?
SequenceNumber expected_seq = current_last_seq_ + 1;
// If the iterator has started, then confirm that we get continuous batches
if (started_ && !IsBatchExpected(batch.get(), expected_seq)) {
// Seek to the batch having expected sequence number
if (expected_seq < files_->at(current_file_index_)->StartSequence()) {
// Expected batch must lie in the previous log file
// Avoid underflow.
if (current_file_index_ != 0) {
current_file_index_--;
}
}
starting_sequence_number_ = expected_seq;
// currentStatus_ will be set to Ok if reseek succeeds
// Note: this is still ok in seq_pre_batch_ && two_write_queuesp_ mode
// that allows gaps in the WAL since it will still skip over the gap.
current_status_ = Status::NotFound("Gap in sequence numbers");
// In seq_per_batch_ mode, gaps in the seq are possible so the strict mode
// should be disabled
return SeekToStartSequence(current_file_index_, !seq_per_batch_);
}
current_batch_seq_ = WriteBatchInternal::Sequence(batch.get());
assert(!seq_per_batch_);
current_last_seq_ =
current_batch_seq_ + WriteBatchInternal::Count(batch.get()) - 1;
// currentBatchSeq_ can only change here
assert(current_last_seq_ <= versions_->LastSequence());
current_batch_ = std::move(batch);
is_valid_ = true;
current_status_ = Status::OK();
}
Status TransactionLogIteratorImpl::OpenLogReader(const LogFile* log_file) {
std::unique_ptr<SequentialFileReader> file;
Status s = OpenLogFile(log_file, &file);
if (!s.ok()) {
return s;
}
assert(file);
current_log_reader_.reset(
new log::Reader(options_->info_log, std::move(file), &reporter_,
read_options_.verify_checksums_, log_file->LogNumber()));
return Status::OK();
}
} // namespace ROCKSDB_NAMESPACE
#endif // ROCKSDB_LITE