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/external_sst_file_ingestion...

732 lines
27 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/external_sst_file_ingestion_job.h"
#include <algorithm>
#include <cinttypes>
#include <string>
#include <unordered_set>
#include <vector>
#include "db/db_impl/db_impl.h"
#include "db/version_edit.h"
#include "file/file_util.h"
#include "file/random_access_file_reader.h"
#include "table/merging_iterator.h"
#include "table/scoped_arena_iterator.h"
#include "table/sst_file_writer_collectors.h"
#include "table/table_builder.h"
#include "test_util/sync_point.h"
#include "util/stop_watch.h"
namespace ROCKSDB_NAMESPACE {
Status ExternalSstFileIngestionJob::Prepare(
const std::vector<std::string>& external_files_paths,
uint64_t next_file_number, SuperVersion* sv) {
Status status;
// Read the information of files we are ingesting
for (const std::string& file_path : external_files_paths) {
IngestedFileInfo file_to_ingest;
status = GetIngestedFileInfo(file_path, &file_to_ingest, sv);
if (!status.ok()) {
return status;
}
files_to_ingest_.push_back(file_to_ingest);
}
for (const IngestedFileInfo& f : files_to_ingest_) {
if (f.cf_id !=
TablePropertiesCollectorFactory::Context::kUnknownColumnFamily &&
f.cf_id != cfd_->GetID()) {
return Status::InvalidArgument(
"External file column family id don't match");
}
}
const Comparator* ucmp = cfd_->internal_comparator().user_comparator();
auto num_files = files_to_ingest_.size();
if (num_files == 0) {
return Status::InvalidArgument("The list of files is empty");
} else if (num_files > 1) {
// Verify that passed files don't have overlapping ranges
autovector<const IngestedFileInfo*> sorted_files;
for (size_t i = 0; i < num_files; i++) {
sorted_files.push_back(&files_to_ingest_[i]);
}
std::sort(
sorted_files.begin(), sorted_files.end(),
[&ucmp](const IngestedFileInfo* info1, const IngestedFileInfo* info2) {
return sstableKeyCompare(ucmp, info1->smallest_internal_key,
info2->smallest_internal_key) < 0;
});
for (size_t i = 0; i < num_files - 1; i++) {
if (sstableKeyCompare(ucmp, sorted_files[i]->largest_internal_key,
sorted_files[i + 1]->smallest_internal_key) >= 0) {
files_overlap_ = true;
break;
}
}
}
if (ingestion_options_.ingest_behind && files_overlap_) {
return Status::NotSupported("Files have overlapping ranges");
}
for (IngestedFileInfo& f : files_to_ingest_) {
if (f.num_entries == 0 && f.num_range_deletions == 0) {
return Status::InvalidArgument("File contain no entries");
}
if (!f.smallest_internal_key.Valid() || !f.largest_internal_key.Valid()) {
return Status::Corruption("Generated table have corrupted keys");
}
}
// Copy/Move external files into DB
std::unordered_set<size_t> ingestion_path_ids;
for (IngestedFileInfo& f : files_to_ingest_) {
f.fd = FileDescriptor(next_file_number++, 0, f.file_size);
f.copy_file = false;
const std::string path_outside_db = f.external_file_path;
const std::string path_inside_db =
TableFileName(cfd_->ioptions()->cf_paths, f.fd.GetNumber(),
f.fd.GetPathId());
if (ingestion_options_.move_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
5 years ago
status =
fs_->LinkFile(path_outside_db, path_inside_db, IOOptions(), nullptr);
if (status.ok()) {
// It is unsafe to assume application had sync the file and file
// directory before ingest the file. For integrity of RocksDB we need
// to sync the file.
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<FSWritableFile> file_to_sync;
status = fs_->ReopenWritableFile(path_inside_db, env_options_,
&file_to_sync, nullptr);
if (status.ok()) {
TEST_SYNC_POINT(
"ExternalSstFileIngestionJob::BeforeSyncIngestedFile");
status = SyncIngestedFile(file_to_sync.get());
TEST_SYNC_POINT("ExternalSstFileIngestionJob::AfterSyncIngestedFile");
if (!status.ok()) {
ROCKS_LOG_WARN(db_options_.info_log,
"Failed to sync ingested file %s: %s",
path_inside_db.c_str(), status.ToString().c_str());
}
}
} else if (status.IsNotSupported() &&
ingestion_options_.failed_move_fall_back_to_copy) {
// Original file is on a different FS, use copy instead of hard linking.
f.copy_file = true;
}
} else {
f.copy_file = true;
}
if (f.copy_file) {
TEST_SYNC_POINT_CALLBACK("ExternalSstFileIngestionJob::Prepare:CopyFile",
nullptr);
// CopyFile also sync the new file.
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
status = CopyFile(fs_, path_outside_db, path_inside_db, 0,
db_options_.use_fsync);
}
TEST_SYNC_POINT("ExternalSstFileIngestionJob::Prepare:FileAdded");
if (!status.ok()) {
break;
}
f.internal_file_path = path_inside_db;
ingestion_path_ids.insert(f.fd.GetPathId());
}
TEST_SYNC_POINT("ExternalSstFileIngestionJob::BeforeSyncDir");
if (status.ok()) {
for (auto path_id : ingestion_path_ids) {
status = directories_->GetDataDir(path_id)->Fsync(IOOptions(), nullptr);
if (!status.ok()) {
ROCKS_LOG_WARN(db_options_.info_log,
"Failed to sync directory %" ROCKSDB_PRIszt
" while ingest file: %s",
path_id, status.ToString().c_str());
break;
}
}
}
TEST_SYNC_POINT("ExternalSstFileIngestionJob::AfterSyncDir");
// TODO: The following is duplicated with Cleanup().
if (!status.ok()) {
// We failed, remove all files that we copied into the db
for (IngestedFileInfo& f : files_to_ingest_) {
if (f.internal_file_path.empty()) {
continue;
}
Status s = env_->DeleteFile(f.internal_file_path);
if (!s.ok()) {
ROCKS_LOG_WARN(db_options_.info_log,
"AddFile() clean up for file %s failed : %s",
f.internal_file_path.c_str(), s.ToString().c_str());
}
}
}
return status;
}
Status ExternalSstFileIngestionJob::NeedsFlush(bool* flush_needed,
SuperVersion* super_version) {
autovector<Range> ranges;
for (const IngestedFileInfo& file_to_ingest : files_to_ingest_) {
ranges.emplace_back(file_to_ingest.smallest_internal_key.user_key(),
file_to_ingest.largest_internal_key.user_key());
}
Status status =
cfd_->RangesOverlapWithMemtables(ranges, super_version, flush_needed);
if (status.ok() && *flush_needed &&
!ingestion_options_.allow_blocking_flush) {
status = Status::InvalidArgument("External file requires flush");
}
return status;
}
Optimize for serial commits in 2PC Summary: Throughput: 46k tps in our sysbench settings (filling the details later) The idea is to have the simplest change that gives us a reasonable boost in 2PC throughput. Major design changes: 1. The WAL file internal buffer is not flushed after each write. Instead it is flushed before critical operations (WAL copy via fs) or when FlushWAL is called by MySQL. Flushing the WAL buffer is also protected via mutex_. 2. Use two sequence numbers: last seq, and last seq for write. Last seq is the last visible sequence number for reads. Last seq for write is the next sequence number that should be used to write to WAL/memtable. This allows to have a memtable write be in parallel to WAL writes. 3. BatchGroup is not used for writes. This means that we can have parallel writers which changes a major assumption in the code base. To accommodate for that i) allow only 1 WriteImpl that intends to write to memtable via mem_mutex_--which is fine since in 2PC almost all of the memtable writes come via group commit phase which is serial anyway, ii) make all the parts in the code base that assumed to be the only writer (via EnterUnbatched) to also acquire mem_mutex_, iii) stat updates are protected via a stat_mutex_. Note: the first commit has the approach figured out but is not clean. Submitting the PR anyway to get the early feedback on the approach. If we are ok with the approach I will go ahead with this updates: 0) Rebase with Yi's pipelining changes 1) Currently batching is disabled by default to make sure that it will be consistent with all unit tests. Will make this optional via a config. 2) A couple of unit tests are disabled. They need to be updated with the serial commit of 2PC taken into account. 3) Replacing BatchGroup with mem_mutex_ got a bit ugly as it requires releasing mutex_ beforehand (the same way EnterUnbatched does). This needs to be cleaned up. Closes https://github.com/facebook/rocksdb/pull/2345 Differential Revision: D5210732 Pulled By: maysamyabandeh fbshipit-source-id: 78653bd95a35cd1e831e555e0e57bdfd695355a4
7 years ago
// REQUIRES: we have become the only writer by entering both write_thread_ and
// nonmem_write_thread_
Status ExternalSstFileIngestionJob::Run() {
Status status;
SuperVersion* super_version = cfd_->GetSuperVersion();
#ifndef NDEBUG
// We should never run the job with a memtable that is overlapping
// with the files we are ingesting
bool need_flush = false;
status = NeedsFlush(&need_flush, super_version);
assert(status.ok() && need_flush == false);
#endif
bool force_global_seqno = false;
if (ingestion_options_.snapshot_consistency && !db_snapshots_->empty()) {
// We need to assign a global sequence number to all the files even
// if the don't overlap with any ranges since we have snapshots
force_global_seqno = true;
}
// It is safe to use this instead of LastAllocatedSequence since we are
Optimize for serial commits in 2PC Summary: Throughput: 46k tps in our sysbench settings (filling the details later) The idea is to have the simplest change that gives us a reasonable boost in 2PC throughput. Major design changes: 1. The WAL file internal buffer is not flushed after each write. Instead it is flushed before critical operations (WAL copy via fs) or when FlushWAL is called by MySQL. Flushing the WAL buffer is also protected via mutex_. 2. Use two sequence numbers: last seq, and last seq for write. Last seq is the last visible sequence number for reads. Last seq for write is the next sequence number that should be used to write to WAL/memtable. This allows to have a memtable write be in parallel to WAL writes. 3. BatchGroup is not used for writes. This means that we can have parallel writers which changes a major assumption in the code base. To accommodate for that i) allow only 1 WriteImpl that intends to write to memtable via mem_mutex_--which is fine since in 2PC almost all of the memtable writes come via group commit phase which is serial anyway, ii) make all the parts in the code base that assumed to be the only writer (via EnterUnbatched) to also acquire mem_mutex_, iii) stat updates are protected via a stat_mutex_. Note: the first commit has the approach figured out but is not clean. Submitting the PR anyway to get the early feedback on the approach. If we are ok with the approach I will go ahead with this updates: 0) Rebase with Yi's pipelining changes 1) Currently batching is disabled by default to make sure that it will be consistent with all unit tests. Will make this optional via a config. 2) A couple of unit tests are disabled. They need to be updated with the serial commit of 2PC taken into account. 3) Replacing BatchGroup with mem_mutex_ got a bit ugly as it requires releasing mutex_ beforehand (the same way EnterUnbatched does). This needs to be cleaned up. Closes https://github.com/facebook/rocksdb/pull/2345 Differential Revision: D5210732 Pulled By: maysamyabandeh fbshipit-source-id: 78653bd95a35cd1e831e555e0e57bdfd695355a4
7 years ago
// the only active writer, and hence they are equal
SequenceNumber last_seqno = versions_->LastSequence();
edit_.SetColumnFamily(cfd_->GetID());
// The levels that the files will be ingested into
for (IngestedFileInfo& f : files_to_ingest_) {
SequenceNumber assigned_seqno = 0;
if (ingestion_options_.ingest_behind) {
status = CheckLevelForIngestedBehindFile(&f);
} else {
status = AssignLevelAndSeqnoForIngestedFile(
super_version, force_global_seqno, cfd_->ioptions()->compaction_style,
last_seqno, &f, &assigned_seqno);
}
if (!status.ok()) {
return status;
}
status = AssignGlobalSeqnoForIngestedFile(&f, assigned_seqno);
TEST_SYNC_POINT_CALLBACK("ExternalSstFileIngestionJob::Run",
&assigned_seqno);
if (assigned_seqno > last_seqno) {
assert(assigned_seqno == last_seqno + 1);
last_seqno = assigned_seqno;
++consumed_seqno_count_;
}
if (!status.ok()) {
return status;
}
// We use the import time as the ancester time. This is the time the data
// is written to the database.
int64_t temp_current_time = 0;
uint64_t current_time = kUnknownFileCreationTime;
uint64_t oldest_ancester_time = kUnknownOldestAncesterTime;
if (env_->GetCurrentTime(&temp_current_time).ok()) {
current_time = oldest_ancester_time =
static_cast<uint64_t>(temp_current_time);
}
edit_.AddFile(
f.picked_level, f.fd.GetNumber(), f.fd.GetPathId(), f.fd.GetFileSize(),
f.smallest_internal_key, f.largest_internal_key, f.assigned_seqno,
f.assigned_seqno, false, kInvalidBlobFileNumber, oldest_ancester_time,
current_time, kUnknownFileChecksum, kUnknownFileChecksumFuncName);
}
return status;
}
void ExternalSstFileIngestionJob::UpdateStats() {
// Update internal stats for new ingested files
uint64_t total_keys = 0;
uint64_t total_l0_files = 0;
uint64_t total_time = env_->NowMicros() - job_start_time_;
EventLoggerStream stream = event_logger_->Log();
stream << "event"
<< "ingest_finished";
stream << "files_ingested";
stream.StartArray();
for (IngestedFileInfo& f : files_to_ingest_) {
InternalStats::CompactionStats stats(CompactionReason::kExternalSstIngestion, 1);
stats.micros = total_time;
// If actual copy occurred for this file, then we need to count the file
// size as the actual bytes written. If the file was linked, then we ignore
// the bytes written for file metadata.
// TODO (yanqin) maybe account for file metadata bytes for exact accuracy?
if (f.copy_file) {
stats.bytes_written = f.fd.GetFileSize();
} else {
stats.bytes_moved = f.fd.GetFileSize();
}
stats.num_output_files = 1;
cfd_->internal_stats()->AddCompactionStats(f.picked_level,
Env::Priority::USER, stats);
cfd_->internal_stats()->AddCFStats(InternalStats::BYTES_INGESTED_ADD_FILE,
f.fd.GetFileSize());
total_keys += f.num_entries;
if (f.picked_level == 0) {
total_l0_files += 1;
}
ROCKS_LOG_INFO(
db_options_.info_log,
"[AddFile] External SST file %s was ingested in L%d with path %s "
"(global_seqno=%" PRIu64 ")\n",
f.external_file_path.c_str(), f.picked_level,
f.internal_file_path.c_str(), f.assigned_seqno);
stream << "file" << f.internal_file_path << "level" << f.picked_level;
}
stream.EndArray();
stream << "lsm_state";
stream.StartArray();
auto vstorage = cfd_->current()->storage_info();
for (int level = 0; level < vstorage->num_levels(); ++level) {
stream << vstorage->NumLevelFiles(level);
}
stream.EndArray();
cfd_->internal_stats()->AddCFStats(InternalStats::INGESTED_NUM_KEYS_TOTAL,
total_keys);
cfd_->internal_stats()->AddCFStats(InternalStats::INGESTED_NUM_FILES_TOTAL,
files_to_ingest_.size());
cfd_->internal_stats()->AddCFStats(
InternalStats::INGESTED_LEVEL0_NUM_FILES_TOTAL, total_l0_files);
}
void ExternalSstFileIngestionJob::Cleanup(const Status& status) {
if (!status.ok()) {
// We failed to add the files to the database
// remove all the files we copied
for (IngestedFileInfo& f : files_to_ingest_) {
if (f.internal_file_path.empty()) {
continue;
}
Status s = env_->DeleteFile(f.internal_file_path);
if (!s.ok()) {
ROCKS_LOG_WARN(db_options_.info_log,
"AddFile() clean up for file %s failed : %s",
f.internal_file_path.c_str(), s.ToString().c_str());
}
}
consumed_seqno_count_ = 0;
files_overlap_ = false;
} else if (status.ok() && ingestion_options_.move_files) {
// The files were moved and added successfully, remove original file links
for (IngestedFileInfo& f : files_to_ingest_) {
Status s = env_->DeleteFile(f.external_file_path);
if (!s.ok()) {
ROCKS_LOG_WARN(
db_options_.info_log,
"%s was added to DB successfully but failed to remove original "
"file link : %s",
f.external_file_path.c_str(), s.ToString().c_str());
}
}
}
}
Status ExternalSstFileIngestionJob::GetIngestedFileInfo(
const std::string& external_file, IngestedFileInfo* file_to_ingest,
SuperVersion* sv) {
file_to_ingest->external_file_path = external_file;
// Get external file size
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
Status status = fs_->GetFileSize(external_file, IOOptions(),
&file_to_ingest->file_size, nullptr);
if (!status.ok()) {
return status;
}
// Create TableReader for external file
std::unique_ptr<TableReader> table_reader;
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<FSRandomAccessFile> sst_file;
std::unique_ptr<RandomAccessFileReader> sst_file_reader;
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
status = fs_->NewRandomAccessFile(external_file, env_options_,
&sst_file, nullptr);
if (!status.ok()) {
return status;
}
sst_file_reader.reset(new RandomAccessFileReader(std::move(sst_file),
external_file));
status = cfd_->ioptions()->table_factory->NewTableReader(
TableReaderOptions(*cfd_->ioptions(),
sv->mutable_cf_options.prefix_extractor.get(),
env_options_, cfd_->internal_comparator()),
std::move(sst_file_reader), file_to_ingest->file_size, &table_reader);
if (!status.ok()) {
return status;
}
if (ingestion_options_.verify_checksums_before_ingest) {
// If customized readahead size is needed, we can pass a user option
// all the way to here. Right now we just rely on the default readahead
// to keep things simple.
ReadOptions ro;
ro.readahead_size = ingestion_options_.verify_checksums_readahead_size;
status = table_reader->VerifyChecksum(
ro, TableReaderCaller::kExternalSSTIngestion);
}
if (!status.ok()) {
return status;
}
// Get the external file properties
auto props = table_reader->GetTableProperties();
const auto& uprops = props->user_collected_properties;
// Get table version
auto version_iter = uprops.find(ExternalSstFilePropertyNames::kVersion);
if (version_iter == uprops.end()) {
return Status::Corruption("External file version not found");
}
file_to_ingest->version = DecodeFixed32(version_iter->second.c_str());
auto seqno_iter = uprops.find(ExternalSstFilePropertyNames::kGlobalSeqno);
if (file_to_ingest->version == 2) {
// version 2 imply that we have global sequence number
if (seqno_iter == uprops.end()) {
return Status::Corruption(
"External file global sequence number not found");
}
// Set the global sequence number
file_to_ingest->original_seqno = DecodeFixed64(seqno_iter->second.c_str());
auto offsets_iter = props->properties_offsets.find(
ExternalSstFilePropertyNames::kGlobalSeqno);
if (offsets_iter == props->properties_offsets.end() ||
offsets_iter->second == 0) {
file_to_ingest->global_seqno_offset = 0;
return Status::Corruption("Was not able to find file global seqno field");
}
file_to_ingest->global_seqno_offset = static_cast<size_t>(offsets_iter->second);
} else if (file_to_ingest->version == 1) {
// SST file V1 should not have global seqno field
assert(seqno_iter == uprops.end());
file_to_ingest->original_seqno = 0;
if (ingestion_options_.allow_blocking_flush ||
ingestion_options_.allow_global_seqno) {
return Status::InvalidArgument(
"External SST file V1 does not support global seqno");
}
} else {
return Status::InvalidArgument("External file version is not supported");
}
// Get number of entries in table
file_to_ingest->num_entries = props->num_entries;
file_to_ingest->num_range_deletions = props->num_range_deletions;
ParsedInternalKey key;
ReadOptions ro;
// During reading the external file we can cache blocks that we read into
// the block cache, if we later change the global seqno of this file, we will
// have block in cache that will include keys with wrong seqno.
// We need to disable fill_cache so that we read from the file without
// updating the block cache.
ro.fill_cache = false;
std::unique_ptr<InternalIterator> iter(table_reader->NewIterator(
ro, sv->mutable_cf_options.prefix_extractor.get(), /*arena=*/nullptr,
/*skip_filters=*/false, TableReaderCaller::kExternalSSTIngestion));
std::unique_ptr<InternalIterator> range_del_iter(
table_reader->NewRangeTombstoneIterator(ro));
// Get first (smallest) and last (largest) key from file.
file_to_ingest->smallest_internal_key =
InternalKey("", 0, ValueType::kTypeValue);
file_to_ingest->largest_internal_key =
InternalKey("", 0, ValueType::kTypeValue);
bool bounds_set = false;
iter->SeekToFirst();
if (iter->Valid()) {
if (!ParseInternalKey(iter->key(), &key)) {
return Status::Corruption("external file have corrupted keys");
}
if (key.sequence != 0) {
return Status::Corruption("external file have non zero sequence number");
}
file_to_ingest->smallest_internal_key.SetFrom(key);
iter->SeekToLast();
if (!ParseInternalKey(iter->key(), &key)) {
return Status::Corruption("external file have corrupted keys");
}
if (key.sequence != 0) {
return Status::Corruption("external file have non zero sequence number");
}
file_to_ingest->largest_internal_key.SetFrom(key);
bounds_set = true;
}
// We may need to adjust these key bounds, depending on whether any range
// deletion tombstones extend past them.
const Comparator* ucmp = cfd_->internal_comparator().user_comparator();
if (range_del_iter != nullptr) {
for (range_del_iter->SeekToFirst(); range_del_iter->Valid();
range_del_iter->Next()) {
if (!ParseInternalKey(range_del_iter->key(), &key)) {
return Status::Corruption("external file have corrupted keys");
}
RangeTombstone tombstone(key, range_del_iter->value());
InternalKey start_key = tombstone.SerializeKey();
if (!bounds_set ||
sstableKeyCompare(ucmp, start_key,
file_to_ingest->smallest_internal_key) < 0) {
file_to_ingest->smallest_internal_key = start_key;
}
InternalKey end_key = tombstone.SerializeEndKey();
if (!bounds_set ||
sstableKeyCompare(ucmp, end_key,
file_to_ingest->largest_internal_key) > 0) {
file_to_ingest->largest_internal_key = end_key;
}
bounds_set = true;
}
}
file_to_ingest->cf_id = static_cast<uint32_t>(props->column_family_id);
file_to_ingest->table_properties = *props;
return status;
}
Status ExternalSstFileIngestionJob::AssignLevelAndSeqnoForIngestedFile(
SuperVersion* sv, bool force_global_seqno, CompactionStyle compaction_style,
SequenceNumber last_seqno, IngestedFileInfo* file_to_ingest,
SequenceNumber* assigned_seqno) {
Status status;
*assigned_seqno = 0;
if (force_global_seqno) {
*assigned_seqno = last_seqno + 1;
if (compaction_style == kCompactionStyleUniversal || files_overlap_) {
file_to_ingest->picked_level = 0;
return status;
}
}
bool overlap_with_db = false;
Arena arena;
ReadOptions ro;
ro.total_order_seek = true;
int target_level = 0;
auto* vstorage = cfd_->current()->storage_info();
for (int lvl = 0; lvl < cfd_->NumberLevels(); lvl++) {
if (lvl > 0 && lvl < vstorage->base_level()) {
continue;
}
if (vstorage->NumLevelFiles(lvl) > 0) {
bool overlap_with_level = false;
status = sv->current->OverlapWithLevelIterator(
ro, env_options_, file_to_ingest->smallest_internal_key.user_key(),
file_to_ingest->largest_internal_key.user_key(), lvl,
&overlap_with_level);
if (!status.ok()) {
return status;
}
if (overlap_with_level) {
// We must use L0 or any level higher than `lvl` to be able to overwrite
// the keys that we overlap with in this level, We also need to assign
// this file a seqno to overwrite the existing keys in level `lvl`
overlap_with_db = true;
break;
}
if (compaction_style == kCompactionStyleUniversal && lvl != 0) {
const std::vector<FileMetaData*>& level_files =
vstorage->LevelFiles(lvl);
const SequenceNumber level_largest_seqno =
(*max_element(level_files.begin(), level_files.end(),
[](FileMetaData* f1, FileMetaData* f2) {
return f1->fd.largest_seqno < f2->fd.largest_seqno;
}))
->fd.largest_seqno;
// should only assign seqno to current level's largest seqno when
// the file fits
if (level_largest_seqno != 0 &&
IngestedFileFitInLevel(file_to_ingest, lvl)) {
*assigned_seqno = level_largest_seqno;
} else {
continue;
}
}
} else if (compaction_style == kCompactionStyleUniversal) {
continue;
}
// We don't overlap with any keys in this level, but we still need to check
// if our file can fit in it
if (IngestedFileFitInLevel(file_to_ingest, lvl)) {
target_level = lvl;
}
}
// If files overlap, we have to ingest them at level 0 and assign the newest
// sequence number
if (files_overlap_) {
target_level = 0;
*assigned_seqno = last_seqno + 1;
}
TEST_SYNC_POINT_CALLBACK(
"ExternalSstFileIngestionJob::AssignLevelAndSeqnoForIngestedFile",
&overlap_with_db);
file_to_ingest->picked_level = target_level;
if (overlap_with_db && *assigned_seqno == 0) {
*assigned_seqno = last_seqno + 1;
}
return status;
}
Status ExternalSstFileIngestionJob::CheckLevelForIngestedBehindFile(
IngestedFileInfo* file_to_ingest) {
auto* vstorage = cfd_->current()->storage_info();
// first check if new files fit in the bottommost level
int bottom_lvl = cfd_->NumberLevels() - 1;
if(!IngestedFileFitInLevel(file_to_ingest, bottom_lvl)) {
return Status::InvalidArgument(
"Can't ingest_behind file as it doesn't fit "
"at the bottommost level!");
}
// second check if despite allow_ingest_behind=true we still have 0 seqnums
// at some upper level
for (int lvl = 0; lvl < cfd_->NumberLevels() - 1; lvl++) {
for (auto file : vstorage->LevelFiles(lvl)) {
if (file->fd.smallest_seqno == 0) {
return Status::InvalidArgument(
"Can't ingest_behind file as despite allow_ingest_behind=true "
"there are files with 0 seqno in database at upper levels!");
}
}
}
file_to_ingest->picked_level = bottom_lvl;
return Status::OK();
}
Status ExternalSstFileIngestionJob::AssignGlobalSeqnoForIngestedFile(
IngestedFileInfo* file_to_ingest, SequenceNumber seqno) {
if (file_to_ingest->original_seqno == seqno) {
// This file already have the correct global seqno
return Status::OK();
} else if (!ingestion_options_.allow_global_seqno) {
return Status::InvalidArgument("Global seqno is required, but disabled");
} else if (file_to_ingest->global_seqno_offset == 0) {
return Status::InvalidArgument(
"Trying to set global seqno for a file that don't have a global seqno "
"field");
}
if (ingestion_options_.write_global_seqno) {
// Determine if we can write global_seqno to a given offset of file.
// If the file system does not support random write, then we should not.
// Otherwise we should.
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<FSRandomRWFile> rwfile;
Status status =
fs_->NewRandomRWFile(file_to_ingest->internal_file_path, env_options_,
&rwfile, nullptr);
if (status.ok()) {
std::string seqno_val;
PutFixed64(&seqno_val, seqno);
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
status = rwfile->Write(file_to_ingest->global_seqno_offset, seqno_val,
IOOptions(), nullptr);
if (status.ok()) {
TEST_SYNC_POINT("ExternalSstFileIngestionJob::BeforeSyncGlobalSeqno");
status = SyncIngestedFile(rwfile.get());
TEST_SYNC_POINT("ExternalSstFileIngestionJob::AfterSyncGlobalSeqno");
if (!status.ok()) {
ROCKS_LOG_WARN(db_options_.info_log,
"Failed to sync ingested file %s after writing global "
"sequence number: %s",
file_to_ingest->internal_file_path.c_str(),
status.ToString().c_str());
}
}
if (!status.ok()) {
return status;
}
} else if (!status.IsNotSupported()) {
return status;
}
}
file_to_ingest->assigned_seqno = seqno;
return Status::OK();
}
bool ExternalSstFileIngestionJob::IngestedFileFitInLevel(
const IngestedFileInfo* file_to_ingest, int level) {
if (level == 0) {
// Files can always fit in L0
return true;
}
auto* vstorage = cfd_->current()->storage_info();
Slice file_smallest_user_key(
file_to_ingest->smallest_internal_key.user_key());
Slice file_largest_user_key(file_to_ingest->largest_internal_key.user_key());
if (vstorage->OverlapInLevel(level, &file_smallest_user_key,
&file_largest_user_key)) {
// File overlap with another files in this level, we cannot
// add it to this level
return false;
}
if (cfd_->RangeOverlapWithCompaction(file_smallest_user_key,
file_largest_user_key, level)) {
// File overlap with a running compaction output that will be stored
// in this level, we cannot add this file to this level
return false;
}
// File did not overlap with level files, our compaction output
return true;
}
template <typename TWritableFile>
Status ExternalSstFileIngestionJob::SyncIngestedFile(TWritableFile* file) {
assert(file != nullptr);
if (db_options_.use_fsync) {
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
return file->Fsync(IOOptions(), nullptr);
} else {
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
return file->Sync(IOOptions(), nullptr);
}
}
} // namespace ROCKSDB_NAMESPACE
#endif // !ROCKSDB_LITE