Summary: Changes in the diff API changes: - Introduce IngestExternalFile to replace AddFile (I think this make the API more clear) - Introduce IngestExternalFileOptions (This struct will encapsulate the options for ingesting the external file) - Deprecate AddFile() API Logic changes: - If our file overlap with the memtable we will flush the memtable - We will find the first level in the LSM tree that our file key range overlap with the keys in it - We will find the lowest level in the LSM tree above the the level we found in step 2 that our file can fit in and ingest our file in it - We will assign a global sequence number to our new file - Remove AddFile restrictions by using global sequence numbers Other changes: - Refactor all AddFile logic to be encapsulated in ExternalSstFileIngestionJob Test Plan: unit tests (still need to add more) addfile_stress (https://reviews.facebook.net/D65037) Reviewers: yiwu, andrewkr, lightmark, yhchiang, sdong Reviewed By: sdong Subscribers: jkedgar, hcz, andrewkr, dhruba Differential Revision: https://reviews.facebook.net/D65061main
parent
1d9dbef64e
commit
869ae5d786
@ -1,430 +0,0 @@ |
||||
// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
|
||||
// This source code is licensed under the BSD-style license found in the
|
||||
// LICENSE file in the root directory of this source tree. An additional grant
|
||||
// of patent rights can be found in the PATENTS file in the same directory.
|
||||
|
||||
#include "db/db_impl.h" |
||||
|
||||
#ifndef __STDC_FORMAT_MACROS |
||||
#define __STDC_FORMAT_MACROS |
||||
#endif |
||||
|
||||
#include <inttypes.h> |
||||
|
||||
#include "rocksdb/db.h" |
||||
#include "rocksdb/env.h" |
||||
#include "rocksdb/sst_file_writer.h" |
||||
|
||||
#include "db/builder.h" |
||||
#include "table/sst_file_writer_collectors.h" |
||||
#include "table/table_builder.h" |
||||
#include "util/file_reader_writer.h" |
||||
#include "util/file_util.h" |
||||
#include "util/sync_point.h" |
||||
|
||||
namespace rocksdb { |
||||
|
||||
#ifndef ROCKSDB_LITE |
||||
|
||||
Status DBImpl::ReadExternalSstFileInfo(ColumnFamilyHandle* column_family, |
||||
const std::string& file_path, |
||||
ExternalSstFileInfo* file_info) { |
||||
Status status; |
||||
auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family); |
||||
auto cfd = cfh->cfd(); |
||||
|
||||
file_info->file_path = file_path; |
||||
status = env_->GetFileSize(file_path, &file_info->file_size); |
||||
if (!status.ok()) { |
||||
return status; |
||||
} |
||||
|
||||
// Access the file using TableReader to extract
|
||||
// version, number of entries, smallest user key, largest user key
|
||||
std::unique_ptr<RandomAccessFile> sst_file; |
||||
status = env_->NewRandomAccessFile(file_path, &sst_file, env_options_); |
||||
if (!status.ok()) { |
||||
return status; |
||||
} |
||||
std::unique_ptr<RandomAccessFileReader> sst_file_reader; |
||||
sst_file_reader.reset(new RandomAccessFileReader(std::move(sst_file))); |
||||
|
||||
std::unique_ptr<TableReader> table_reader; |
||||
status = cfd->ioptions()->table_factory->NewTableReader( |
||||
TableReaderOptions(*cfd->ioptions(), env_options_, |
||||
cfd->internal_comparator()), |
||||
std::move(sst_file_reader), file_info->file_size, &table_reader); |
||||
if (!status.ok()) { |
||||
return status; |
||||
} |
||||
|
||||
// Get the external sst file version from table properties
|
||||
const UserCollectedProperties& user_collected_properties = |
||||
table_reader->GetTableProperties()->user_collected_properties; |
||||
UserCollectedProperties::const_iterator external_sst_file_version_iter = |
||||
user_collected_properties.find(ExternalSstFilePropertyNames::kVersion); |
||||
if (external_sst_file_version_iter == user_collected_properties.end()) { |
||||
return Status::InvalidArgument("Generated table version not found"); |
||||
} |
||||
|
||||
file_info->version = |
||||
DecodeFixed32(external_sst_file_version_iter->second.c_str()); |
||||
if (file_info->version == 2) { |
||||
// version 2 imply that we have global sequence number
|
||||
|
||||
// TODO(tec): Implement version 2 ingestion
|
||||
file_info->sequence_number = 0; |
||||
} else if (file_info->version == 1) { |
||||
// version 1 imply that all sequence numbers in table equal 0
|
||||
file_info->sequence_number = 0; |
||||
} else { |
||||
return Status::InvalidArgument("Generated table version is not supported"); |
||||
} |
||||
// Get number of entries in table
|
||||
file_info->num_entries = table_reader->GetTableProperties()->num_entries; |
||||
|
||||
ParsedInternalKey key; |
||||
std::unique_ptr<InternalIterator> iter( |
||||
table_reader->NewIterator(ReadOptions())); |
||||
|
||||
// Get first (smallest) key from file
|
||||
iter->SeekToFirst(); |
||||
if (!ParseInternalKey(iter->key(), &key)) { |
||||
return Status::Corruption("Generated table have corrupted keys"); |
||||
} |
||||
if (key.sequence != 0) { |
||||
return Status::Corruption("Generated table have non zero sequence number"); |
||||
} |
||||
file_info->smallest_key = key.user_key.ToString(); |
||||
|
||||
// Get last (largest) key from file
|
||||
iter->SeekToLast(); |
||||
if (!ParseInternalKey(iter->key(), &key)) { |
||||
return Status::Corruption("Generated table have corrupted keys"); |
||||
} |
||||
if (key.sequence != 0) { |
||||
return Status::Corruption("Generated table have non zero sequence number"); |
||||
} |
||||
file_info->largest_key = key.user_key.ToString(); |
||||
|
||||
return Status::OK(); |
||||
} |
||||
|
||||
Status DBImpl::AddFile(ColumnFamilyHandle* column_family, |
||||
const std::vector<std::string>& file_path_list, |
||||
bool move_file, bool skip_snapshot_check) { |
||||
Status status; |
||||
auto num_files = file_path_list.size(); |
||||
if (num_files == 0) { |
||||
return Status::InvalidArgument("The list of files is empty"); |
||||
} |
||||
|
||||
std::vector<ExternalSstFileInfo> file_info_list(num_files); |
||||
for (size_t i = 0; i < num_files; i++) { |
||||
status = ReadExternalSstFileInfo(column_family, file_path_list[i], |
||||
&file_info_list[i]); |
||||
if (!status.ok()) { |
||||
return status; |
||||
} |
||||
} |
||||
return AddFile(column_family, file_info_list, move_file, skip_snapshot_check); |
||||
} |
||||
|
||||
Status DBImpl::AddFile(ColumnFamilyHandle* column_family, |
||||
const std::vector<ExternalSstFileInfo>& file_info_list, |
||||
bool move_file, bool skip_snapshot_check) { |
||||
Status status; |
||||
auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family); |
||||
ColumnFamilyData* cfd = cfh->cfd(); |
||||
const Comparator* user_cmp = cfd->internal_comparator().user_comparator(); |
||||
|
||||
auto num_files = file_info_list.size(); |
||||
if (num_files == 0) { |
||||
return Status::InvalidArgument("The list of files is empty"); |
||||
} |
||||
|
||||
// Verify that passed files dont have overlapping ranges
|
||||
if (num_files > 1) { |
||||
std::vector<const ExternalSstFileInfo*> sorted_file_info_list(num_files); |
||||
for (size_t i = 0; i < num_files; i++) { |
||||
sorted_file_info_list[i] = &file_info_list[i]; |
||||
} |
||||
|
||||
std::sort(sorted_file_info_list.begin(), sorted_file_info_list.end(), |
||||
[&user_cmp, &file_info_list](const ExternalSstFileInfo* info1, |
||||
const ExternalSstFileInfo* info2) { |
||||
return user_cmp->Compare(info1->smallest_key, |
||||
info2->smallest_key) < 0; |
||||
}); |
||||
|
||||
for (size_t i = 0; i < num_files - 1; i++) { |
||||
if (user_cmp->Compare(sorted_file_info_list[i]->largest_key, |
||||
sorted_file_info_list[i + 1]->smallest_key) >= 0) { |
||||
return Status::NotSupported("Files have overlapping ranges"); |
||||
} |
||||
} |
||||
} |
||||
|
||||
std::vector<uint64_t> micro_list(num_files, 0); |
||||
std::vector<FileMetaData> meta_list(num_files); |
||||
for (size_t i = 0; i < num_files; i++) { |
||||
StopWatch sw(env_, nullptr, 0, µ_list[i], false); |
||||
if (file_info_list[i].num_entries == 0) { |
||||
return Status::InvalidArgument("File contain no entries"); |
||||
} |
||||
|
||||
if (file_info_list[i].version == 2) { |
||||
// version 2 imply that file have only Put Operations
|
||||
// with global Sequence Number
|
||||
|
||||
// TODO(tec): Implement changing file global sequence number
|
||||
} else if (file_info_list[i].version == 1) { |
||||
// version 1 imply that file have only Put Operations
|
||||
// with Sequence Number = 0
|
||||
} else { |
||||
// Unknown version !
|
||||
return Status::InvalidArgument( |
||||
"Generated table version is not supported"); |
||||
} |
||||
|
||||
meta_list[i].smallest = |
||||
InternalKey(file_info_list[i].smallest_key, |
||||
file_info_list[i].sequence_number, ValueType::kTypeValue); |
||||
meta_list[i].largest = |
||||
InternalKey(file_info_list[i].largest_key, |
||||
file_info_list[i].sequence_number, ValueType::kTypeValue); |
||||
if (!meta_list[i].smallest.Valid() || !meta_list[i].largest.Valid()) { |
||||
return Status::Corruption("Generated table have corrupted keys"); |
||||
} |
||||
meta_list[i].smallest_seqno = file_info_list[i].sequence_number; |
||||
meta_list[i].largest_seqno = file_info_list[i].sequence_number; |
||||
if (meta_list[i].smallest_seqno != 0 || meta_list[i].largest_seqno != 0) { |
||||
return Status::InvalidArgument( |
||||
"Non zero sequence numbers are not supported"); |
||||
} |
||||
} |
||||
|
||||
std::vector<std::list<uint64_t>::iterator> pending_outputs_inserted_elem_list( |
||||
num_files); |
||||
// Generate locations for the new tables
|
||||
{ |
||||
InstrumentedMutexLock l(&mutex_); |
||||
for (size_t i = 0; i < num_files; i++) { |
||||
StopWatch sw(env_, nullptr, 0, µ_list[i], false); |
||||
pending_outputs_inserted_elem_list[i] = |
||||
CaptureCurrentFileNumberInPendingOutputs(); |
||||
meta_list[i].fd = FileDescriptor(versions_->NewFileNumber(), 0, |
||||
file_info_list[i].file_size); |
||||
} |
||||
} |
||||
|
||||
// Copy/Move external files into DB
|
||||
std::vector<std::string> db_fname_list(num_files); |
||||
size_t j = 0; |
||||
for (; j < num_files; j++) { |
||||
StopWatch sw(env_, nullptr, 0, µ_list[j], false); |
||||
db_fname_list[j] = |
||||
TableFileName(immutable_db_options_.db_paths, |
||||
meta_list[j].fd.GetNumber(), meta_list[j].fd.GetPathId()); |
||||
if (move_file) { |
||||
status = env_->LinkFile(file_info_list[j].file_path, db_fname_list[j]); |
||||
if (status.IsNotSupported()) { |
||||
// Original file is on a different FS, use copy instead of hard linking
|
||||
status = |
||||
CopyFile(env_, file_info_list[j].file_path, db_fname_list[j], 0); |
||||
} |
||||
} else { |
||||
status = CopyFile(env_, file_info_list[j].file_path, db_fname_list[j], 0); |
||||
} |
||||
TEST_SYNC_POINT("DBImpl::AddFile:FileCopied"); |
||||
if (!status.ok()) { |
||||
for (size_t i = 0; i < j; i++) { |
||||
Status s = env_->DeleteFile(db_fname_list[i]); |
||||
if (!s.ok()) { |
||||
Log(InfoLogLevel::WARN_LEVEL, immutable_db_options_.info_log, |
||||
"AddFile() clean up for file %s failed : %s", |
||||
db_fname_list[i].c_str(), s.ToString().c_str()); |
||||
} |
||||
} |
||||
return status; |
||||
} |
||||
} |
||||
|
||||
{ |
||||
InstrumentedMutexLock l(&mutex_); |
||||
TEST_SYNC_POINT("DBImpl::AddFile:MutexLock"); |
||||
|
||||
const MutableCFOptions mutable_cf_options = |
||||
*cfd->GetLatestMutableCFOptions(); |
||||
|
||||
WriteThread::Writer w; |
||||
write_thread_.EnterUnbatched(&w, &mutex_); |
||||
|
||||
num_running_addfile_++; |
||||
|
||||
if (!skip_snapshot_check && !snapshots_.empty()) { |
||||
// Check that no snapshots are being held
|
||||
status = |
||||
Status::NotSupported("Cannot add a file while holding snapshots"); |
||||
} |
||||
|
||||
if (status.ok()) { |
||||
// Verify that added file key range dont overlap with any keys in DB
|
||||
SuperVersion* sv = cfd->GetSuperVersion()->Ref(); |
||||
Arena arena; |
||||
ReadOptions ro; |
||||
ro.total_order_seek = true; |
||||
ScopedArenaIterator iter(NewInternalIterator(ro, cfd, sv, &arena)); |
||||
|
||||
for (size_t i = 0; i < num_files; i++) { |
||||
StopWatch sw(env_, nullptr, 0, µ_list[i], false); |
||||
InternalKey range_start(file_info_list[i].smallest_key, |
||||
kMaxSequenceNumber, kValueTypeForSeek); |
||||
iter->Seek(range_start.Encode()); |
||||
status = iter->status(); |
||||
|
||||
if (status.ok() && iter->Valid()) { |
||||
ParsedInternalKey seek_result; |
||||
if (ParseInternalKey(iter->key(), &seek_result)) { |
||||
if (user_cmp->Compare(seek_result.user_key, |
||||
file_info_list[i].largest_key) <= 0) { |
||||
status = Status::NotSupported("Cannot add overlapping range"); |
||||
break; |
||||
} |
||||
} else { |
||||
status = Status::Corruption("DB have corrupted keys"); |
||||
break; |
||||
} |
||||
} |
||||
} |
||||
} |
||||
|
||||
// The levels that the files will be ingested into
|
||||
std::vector<int> target_level_list(num_files, 0); |
||||
if (status.ok()) { |
||||
VersionEdit edit; |
||||
edit.SetColumnFamily(cfd->GetID()); |
||||
for (size_t i = 0; i < num_files; i++) { |
||||
StopWatch sw(env_, nullptr, 0, µ_list[i], false); |
||||
// Add file to the lowest possible level
|
||||
target_level_list[i] = PickLevelForIngestedFile(cfd, file_info_list[i]); |
||||
edit.AddFile(target_level_list[i], meta_list[i].fd.GetNumber(), |
||||
meta_list[i].fd.GetPathId(), meta_list[i].fd.GetFileSize(), |
||||
meta_list[i].smallest, meta_list[i].largest, |
||||
meta_list[i].smallest_seqno, meta_list[i].largest_seqno, |
||||
meta_list[i].marked_for_compaction); |
||||
} |
||||
status = versions_->LogAndApply(cfd, mutable_cf_options, &edit, &mutex_, |
||||
directories_.GetDbDir()); |
||||
} |
||||
write_thread_.ExitUnbatched(&w); |
||||
|
||||
if (status.ok()) { |
||||
delete InstallSuperVersionAndScheduleWork(cfd, nullptr, |
||||
mutable_cf_options); |
||||
|
||||
// Update internal stats for new ingested files
|
||||
uint64_t total_keys = 0; |
||||
uint64_t total_l0_files = 0; |
||||
for (size_t i = 0; i < num_files; i++) { |
||||
InternalStats::CompactionStats stats(1); |
||||
stats.micros = micro_list[i]; |
||||
stats.bytes_written = meta_list[i].fd.GetFileSize(); |
||||
stats.num_output_files = 1; |
||||
cfd->internal_stats()->AddCompactionStats(target_level_list[i], stats); |
||||
cfd->internal_stats()->AddCFStats( |
||||
InternalStats::BYTES_INGESTED_ADD_FILE, |
||||
meta_list[i].fd.GetFileSize()); |
||||
total_keys += file_info_list[i].num_entries; |
||||
if (target_level_list[i] == 0) { |
||||
total_l0_files += 1; |
||||
} |
||||
Log(InfoLogLevel::INFO_LEVEL, immutable_db_options_.info_log, |
||||
"[AddFile] External SST file %s was ingested in L%d with path %s\n", |
||||
file_info_list[i].file_path.c_str(), target_level_list[i], |
||||
db_fname_list[i].c_str()); |
||||
} |
||||
cfd->internal_stats()->AddCFStats(InternalStats::INGESTED_NUM_KEYS_TOTAL, |
||||
total_keys); |
||||
cfd->internal_stats()->AddCFStats(InternalStats::INGESTED_NUM_FILES_TOTAL, |
||||
num_files); |
||||
cfd->internal_stats()->AddCFStats( |
||||
InternalStats::INGESTED_LEVEL0_NUM_FILES_TOTAL, total_l0_files); |
||||
} |
||||
|
||||
for (size_t i = 0; i < num_files; i++) { |
||||
ReleaseFileNumberFromPendingOutputs( |
||||
pending_outputs_inserted_elem_list[i]); |
||||
} |
||||
|
||||
num_running_addfile_--; |
||||
if (num_running_addfile_ == 0) { |
||||
bg_cv_.SignalAll(); |
||||
} |
||||
TEST_SYNC_POINT("DBImpl::AddFile:MutexUnlock"); |
||||
} // mutex_ is unlocked here;
|
||||
|
||||
if (!status.ok()) { |
||||
// We failed to add the files to the database
|
||||
for (size_t i = 0; i < num_files; i++) { |
||||
Status s = env_->DeleteFile(db_fname_list[i]); |
||||
if (!s.ok()) { |
||||
Log(InfoLogLevel::WARN_LEVEL, immutable_db_options_.info_log, |
||||
"AddFile() clean up for file %s failed : %s", |
||||
db_fname_list[i].c_str(), s.ToString().c_str()); |
||||
} |
||||
} |
||||
} else if (status.ok() && move_file) { |
||||
// The files were moved and added successfully, remove original file links
|
||||
for (size_t i = 0; i < num_files; i++) { |
||||
Status s = env_->DeleteFile(file_info_list[i].file_path); |
||||
if (!s.ok()) { |
||||
Log(InfoLogLevel::WARN_LEVEL, immutable_db_options_.info_log, |
||||
"%s was added to DB successfully but failed to remove original " |
||||
"file " |
||||
"link : %s", |
||||
file_info_list[i].file_path.c_str(), s.ToString().c_str()); |
||||
} |
||||
} |
||||
} |
||||
return status; |
||||
} |
||||
|
||||
// Finds the lowest level in the DB that the ingested file can be added to
|
||||
int DBImpl::PickLevelForIngestedFile(ColumnFamilyData* cfd, |
||||
const ExternalSstFileInfo& file_info) { |
||||
mutex_.AssertHeld(); |
||||
|
||||
int target_level = 0; |
||||
auto* vstorage = cfd->current()->storage_info(); |
||||
Slice file_smallest_user_key(file_info.smallest_key); |
||||
Slice file_largest_user_key(file_info.largest_key); |
||||
|
||||
for (int lvl = cfd->NumberLevels() - 1; lvl >= vstorage->base_level(); |
||||
lvl--) { |
||||
// Make sure that the file fits in Level `lvl` and dont overlap with
|
||||
// the output of any compaction running right now.
|
||||
if (vstorage->OverlapInLevel(lvl, &file_smallest_user_key, |
||||
&file_largest_user_key) == false && |
||||
cfd->RangeOverlapWithCompaction(file_smallest_user_key, |
||||
file_largest_user_key, lvl) == false) { |
||||
// Level lvl is the lowest level that dont have any files with key
|
||||
// range overlapping with our file key range and no compactions
|
||||
// planning to add overlapping files in it.
|
||||
target_level = lvl; |
||||
break; |
||||
} |
||||
} |
||||
|
||||
return target_level; |
||||
} |
||||
|
||||
void DBImpl::WaitForAddFile() { |
||||
mutex_.AssertHeld(); |
||||
while (num_running_addfile_ > 0) { |
||||
bg_cv_.Wait(); |
||||
} |
||||
} |
||||
#endif // ROCKSDB_LITE
|
||||
|
||||
} // namespace rocksdb
|
@ -0,0 +1,491 @@ |
||||
// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
|
||||
// This source code is licensed under the BSD-style license found in the
|
||||
// LICENSE file in the root directory of this source tree. An additional grant
|
||||
// of patent rights can be found in the PATENTS file in the same directory.
|
||||
|
||||
#include "db/external_sst_file_ingestion_job.h" |
||||
|
||||
#define __STDC_FORMAT_MACROS |
||||
|
||||
#include <inttypes.h> |
||||
#include <algorithm> |
||||
#include <string> |
||||
#include <vector> |
||||
|
||||
#include "db/version_edit.h" |
||||
#include "table/merger.h" |
||||
#include "table/scoped_arena_iterator.h" |
||||
#include "table/sst_file_writer_collectors.h" |
||||
#include "table/table_builder.h" |
||||
#include "util/file_reader_writer.h" |
||||
#include "util/file_util.h" |
||||
#include "util/stop_watch.h" |
||||
#include "util/sync_point.h" |
||||
|
||||
namespace rocksdb { |
||||
|
||||
Status ExternalSstFileIngestionJob::Prepare( |
||||
const std::vector<std::string>& external_files_paths) { |
||||
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); |
||||
if (!status.ok()) { |
||||
return status; |
||||
} |
||||
files_to_ingest_.push_back(file_to_ingest); |
||||
} |
||||
|
||||
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 dont 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 ucmp->Compare(info1->smallest_user_key, |
||||
info2->smallest_user_key) < 0; |
||||
}); |
||||
|
||||
for (size_t i = 0; i < num_files - 1; i++) { |
||||
if (ucmp->Compare(sorted_files[i]->largest_user_key, |
||||
sorted_files[i + 1]->smallest_user_key) >= 0) { |
||||
return Status::NotSupported("Files have overlapping ranges"); |
||||
} |
||||
} |
||||
} |
||||
|
||||
for (IngestedFileInfo& f : files_to_ingest_) { |
||||
if (f.num_entries == 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
|
||||
for (IngestedFileInfo& f : files_to_ingest_) { |
||||
f.fd = FileDescriptor(versions_->NewFileNumber(), 0, f.file_size); |
||||
|
||||
const std::string path_outside_db = f.external_file_path; |
||||
const std::string path_inside_db = |
||||
TableFileName(db_options_.db_paths, f.fd.GetNumber(), f.fd.GetPathId()); |
||||
|
||||
if (ingestion_options_.move_files) { |
||||
status = env_->LinkFile(path_outside_db, path_inside_db); |
||||
if (status.IsNotSupported()) { |
||||
// Original file is on a different FS, use copy instead of hard linking
|
||||
status = CopyFile(env_, path_outside_db, path_inside_db, 0); |
||||
} |
||||
} else { |
||||
status = CopyFile(env_, path_outside_db, path_inside_db, 0); |
||||
} |
||||
TEST_SYNC_POINT("DBImpl::AddFile:FileCopied"); |
||||
if (!status.ok()) { |
||||
break; |
||||
} |
||||
f.internal_file_path = path_inside_db; |
||||
} |
||||
|
||||
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 == "") { |
||||
break; |
||||
} |
||||
Status s = env_->DeleteFile(f.internal_file_path); |
||||
if (!s.ok()) { |
||||
Log(InfoLogLevel::WARN_LEVEL, 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 = cfd_->GetSuperVersion(); |
||||
Status status = |
||||
IngestedFilesOverlapWithMemtables(super_version, flush_needed); |
||||
|
||||
if (status.ok() && *flush_needed && |
||||
!ingestion_options_.allow_blocking_flush) { |
||||
status = Status::InvalidArgument("External file requires flush"); |
||||
} |
||||
return status; |
||||
} |
||||
|
||||
Status ExternalSstFileIngestionJob::Run() { |
||||
Status status; |
||||
#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); |
||||
assert(status.ok() && need_flush == false); |
||||
#endif |
||||
|
||||
bool consumed_seqno = false; |
||||
bool force_global_seqno = false; |
||||
const SequenceNumber last_seqno = versions_->LastSequence(); |
||||
if (ingestion_options_.snapshot_consistency && !db_snapshots_->empty()) { |
||||
// We need to assign a global sequence number to all the files even
|
||||
// if the dont overlap with any ranges since we have snapshots
|
||||
force_global_seqno = true; |
||||
} |
||||
|
||||
SuperVersion* super_version = cfd_->GetSuperVersion(); |
||||
edit_.SetColumnFamily(cfd_->GetID()); |
||||
// The levels that the files will be ingested into
|
||||
for (IngestedFileInfo& f : files_to_ingest_) { |
||||
bool overlap_with_db = false; |
||||
status = AssignLevelForIngestedFile(super_version, &f, &overlap_with_db); |
||||
if (!status.ok()) { |
||||
return status; |
||||
} |
||||
|
||||
if (overlap_with_db || force_global_seqno) { |
||||
status = AssignGlobalSeqnoForIngestedFile(&f, last_seqno + 1); |
||||
consumed_seqno = true; |
||||
} else { |
||||
status = AssignGlobalSeqnoForIngestedFile(&f, 0); |
||||
} |
||||
|
||||
if (!status.ok()) { |
||||
return status; |
||||
} |
||||
|
||||
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); |
||||
} |
||||
|
||||
if (consumed_seqno) { |
||||
versions_->SetLastSequence(last_seqno + 1); |
||||
} |
||||
|
||||
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_; |
||||
for (IngestedFileInfo& f : files_to_ingest_) { |
||||
InternalStats::CompactionStats stats(1); |
||||
stats.micros = total_time; |
||||
stats.bytes_written = f.fd.GetFileSize(); |
||||
stats.num_output_files = 1; |
||||
cfd_->internal_stats()->AddCompactionStats(f.picked_level, 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; |
||||
} |
||||
Log(InfoLogLevel::INFO_LEVEL, 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); |
||||
} |
||||
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_) { |
||||
Status s = env_->DeleteFile(f.internal_file_path); |
||||
if (!s.ok()) { |
||||
Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log, |
||||
"AddFile() clean up for file %s failed : %s", |
||||
f.internal_file_path.c_str(), s.ToString().c_str()); |
||||
} |
||||
} |
||||
} 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()) { |
||||
Log(InfoLogLevel::WARN_LEVEL, 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) { |
||||
file_to_ingest->external_file_path = external_file; |
||||
|
||||
// Get external file size
|
||||
Status status = env_->GetFileSize(external_file, &file_to_ingest->file_size); |
||||
if (!status.ok()) { |
||||
return status; |
||||
} |
||||
|
||||
// Create TableReader for external file
|
||||
std::unique_ptr<TableReader> table_reader; |
||||
std::unique_ptr<RandomAccessFile> sst_file; |
||||
std::unique_ptr<RandomAccessFileReader> sst_file_reader; |
||||
|
||||
status = env_->NewRandomAccessFile(external_file, &sst_file, env_options_); |
||||
if (!status.ok()) { |
||||
return status; |
||||
} |
||||
sst_file_reader.reset(new RandomAccessFileReader(std::move(sst_file))); |
||||
|
||||
status = cfd_->ioptions()->table_factory->NewTableReader( |
||||
TableReaderOptions(*cfd_->ioptions(), env_options_, |
||||
cfd_->internal_comparator()), |
||||
std::move(sst_file_reader), file_to_ingest->file_size, &table_reader); |
||||
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()); |
||||
file_to_ingest->global_seqno_offset = props->properties_offsets.at( |
||||
ExternalSstFilePropertyNames::kGlobalSeqno); |
||||
|
||||
if (file_to_ingest->global_seqno_offset == 0) { |
||||
return Status::Corruption("Was not able to find file global seqno field"); |
||||
} |
||||
} else { |
||||
return Status::InvalidArgument("external file version is not supported"); |
||||
} |
||||
// Get number of entries in table
|
||||
file_to_ingest->num_entries = props->num_entries; |
||||
|
||||
ParsedInternalKey key; |
||||
std::unique_ptr<InternalIterator> iter( |
||||
table_reader->NewIterator(ReadOptions())); |
||||
|
||||
// Get first (smallest) key from file
|
||||
iter->SeekToFirst(); |
||||
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_user_key = key.user_key.ToString(); |
||||
|
||||
// Get last (largest) key from file
|
||||
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_user_key = key.user_key.ToString(); |
||||
|
||||
return status; |
||||
} |
||||
|
||||
Status ExternalSstFileIngestionJob::IngestedFilesOverlapWithMemtables( |
||||
SuperVersion* sv, bool* overlap) { |
||||
// Create an InternalIterator over all memtables
|
||||
Arena arena; |
||||
ReadOptions ro; |
||||
ro.total_order_seek = true; |
||||
MergeIteratorBuilder merge_iter_builder(&cfd_->internal_comparator(), &arena); |
||||
merge_iter_builder.AddIterator(sv->mem->NewIterator(ro, &arena)); |
||||
sv->imm->AddIterators(ro, &merge_iter_builder); |
||||
ScopedArenaIterator memtable_iter(merge_iter_builder.Finish()); |
||||
|
||||
Status status; |
||||
*overlap = false; |
||||
for (IngestedFileInfo& f : files_to_ingest_) { |
||||
status = |
||||
IngestedFileOverlapWithIteratorRange(&f, memtable_iter.get(), overlap); |
||||
if (!status.ok() || *overlap == true) { |
||||
break; |
||||
} |
||||
} |
||||
|
||||
return status; |
||||
} |
||||
|
||||
Status ExternalSstFileIngestionJob::AssignLevelForIngestedFile( |
||||
SuperVersion* sv, IngestedFileInfo* file_to_ingest, bool* overlap_with_db) { |
||||
*overlap_with_db = false; |
||||
|
||||
Arena arena; |
||||
ReadOptions ro; |
||||
ro.total_order_seek = true; |
||||
|
||||
Status status; |
||||
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; |
||||
MergeIteratorBuilder merge_iter_builder(&cfd_->internal_comparator(), |
||||
&arena); |
||||
sv->current->AddIteratorsForLevel(ro, env_options_, &merge_iter_builder, |
||||
lvl); |
||||
ScopedArenaIterator level_iter(merge_iter_builder.Finish()); |
||||
|
||||
status = IngestedFileOverlapWithIteratorRange( |
||||
file_to_ingest, level_iter.get(), &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; |
||||
} |
||||
} |
||||
|
||||
// We dont 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; |
||||
} |
||||
} |
||||
file_to_ingest->picked_level = target_level; |
||||
return status; |
||||
} |
||||
|
||||
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 dont have a global seqno " |
||||
"field"); |
||||
} |
||||
|
||||
std::unique_ptr<RandomRWFile> rwfile; |
||||
Status status = env_->NewRandomRWFile(file_to_ingest->internal_file_path, |
||||
&rwfile, env_options_); |
||||
if (!status.ok()) { |
||||
return status; |
||||
} |
||||
|
||||
// Write the new seqno in the global sequence number field in the file
|
||||
std::string seqno_val; |
||||
PutFixed64(&seqno_val, seqno); |
||||
status = rwfile->Write(file_to_ingest->global_seqno_offset, seqno_val); |
||||
if (status.ok()) { |
||||
file_to_ingest->assigned_seqno = seqno; |
||||
} |
||||
return status; |
||||
} |
||||
|
||||
Status ExternalSstFileIngestionJob::IngestedFileOverlapWithIteratorRange( |
||||
const IngestedFileInfo* file_to_ingest, InternalIterator* iter, |
||||
bool* overlap) { |
||||
auto* vstorage = cfd_->current()->storage_info(); |
||||
auto* ucmp = vstorage->InternalComparator()->user_comparator(); |
||||
InternalKey range_start(file_to_ingest->smallest_user_key, kMaxSequenceNumber, |
||||
kValueTypeForSeek); |
||||
iter->Seek(range_start.Encode()); |
||||
if (!iter->status().ok()) { |
||||
return iter->status(); |
||||
} |
||||
|
||||
*overlap = false; |
||||
if (iter->Valid()) { |
||||
ParsedInternalKey seek_result; |
||||
if (!ParseInternalKey(iter->key(), &seek_result)) { |
||||
return Status::Corruption("DB have corrupted keys"); |
||||
} |
||||
|
||||
if (ucmp->Compare(seek_result.user_key, file_to_ingest->largest_user_key) <= |
||||
0) { |
||||
*overlap = true; |
||||
} |
||||
} |
||||
|
||||
return iter->status(); |
||||
} |
||||
|
||||
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_user_key); |
||||
Slice file_largest_user_key(file_to_ingest->largest_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; |
||||
} |
||||
|
||||
} // namespace rocksdb
|
@ -0,0 +1,143 @@ |
||||
// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
|
||||
// This source code is licensed under the BSD-style license found in the
|
||||
// LICENSE file in the root directory of this source tree. An additional grant
|
||||
// of patent rights can be found in the PATENTS file in the same directory.
|
||||
|
||||
#pragma once |
||||
#include <string> |
||||
#include <unordered_set> |
||||
#include <vector> |
||||
|
||||
#include "db/column_family.h" |
||||
#include "db/dbformat.h" |
||||
#include "db/internal_stats.h" |
||||
#include "db/snapshot_impl.h" |
||||
#include "rocksdb/db.h" |
||||
#include "rocksdb/env.h" |
||||
#include "rocksdb/sst_file_writer.h" |
||||
#include "util/autovector.h" |
||||
#include "util/db_options.h" |
||||
|
||||
namespace rocksdb { |
||||
|
||||
struct IngestedFileInfo { |
||||
// External file path
|
||||
std::string external_file_path; |
||||
// Smallest user key in external file
|
||||
std::string smallest_user_key; |
||||
// Largest user key in external file
|
||||
std::string largest_user_key; |
||||
// Sequence number for keys in external file
|
||||
SequenceNumber original_seqno; |
||||
// Offset of the global sequence number field in the file, will
|
||||
// be zero if version is 1 (global seqno is not supported)
|
||||
size_t global_seqno_offset; |
||||
// External file size
|
||||
uint64_t file_size; |
||||
// total number of keys in external file
|
||||
uint64_t num_entries; |
||||
// Version of external file
|
||||
int version; |
||||
|
||||
// FileDescriptor for the file inside the DB
|
||||
FileDescriptor fd; |
||||
// file path that we picked for file inside the DB
|
||||
std::string internal_file_path = ""; |
||||
// Global sequence number that we picked for the file inside the DB
|
||||
SequenceNumber assigned_seqno = 0; |
||||
// Level inside the DB we picked for the external file.
|
||||
int picked_level = 0; |
||||
|
||||
InternalKey smallest_internal_key() const { |
||||
return InternalKey(smallest_user_key, assigned_seqno, |
||||
ValueType::kTypeValue); |
||||
} |
||||
|
||||
InternalKey largest_internal_key() const { |
||||
return InternalKey(largest_user_key, assigned_seqno, ValueType::kTypeValue); |
||||
} |
||||
}; |
||||
|
||||
class ExternalSstFileIngestionJob { |
||||
public: |
||||
ExternalSstFileIngestionJob( |
||||
Env* env, VersionSet* versions, ColumnFamilyData* cfd, |
||||
const ImmutableDBOptions& db_options, const EnvOptions& env_options, |
||||
SnapshotList* db_snapshots, |
||||
const IngestExternalFileOptions& ingestion_options) |
||||
: env_(env), |
||||
versions_(versions), |
||||
cfd_(cfd), |
||||
db_options_(db_options), |
||||
env_options_(env_options), |
||||
db_snapshots_(db_snapshots), |
||||
ingestion_options_(ingestion_options), |
||||
job_start_time_(env_->NowMicros()) {} |
||||
|
||||
// Prepare the job by copying external files into the DB.
|
||||
Status Prepare(const std::vector<std::string>& external_files_paths); |
||||
|
||||
// Check if we need to flush the memtable before running the ingestion job
|
||||
// This will be true if the files we are ingesting are overlapping with any
|
||||
// key range in the memtable.
|
||||
// REQUIRES: Mutex held
|
||||
Status NeedsFlush(bool* flush_needed); |
||||
|
||||
// Will execute the ingestion job and prepare edit() to be applied.
|
||||
// REQUIRES: Mutex held
|
||||
Status Run(); |
||||
|
||||
// Update column family stats.
|
||||
// REQUIRES: Mutex held
|
||||
void UpdateStats(); |
||||
|
||||
// Cleanup after successfull/failed job
|
||||
void Cleanup(const Status& status); |
||||
|
||||
VersionEdit* edit() { return &edit_; } |
||||
|
||||
private: |
||||
// Open the external file and populate `file_to_ingest` with all the
|
||||
// external information we need to ingest this file.
|
||||
Status GetIngestedFileInfo(const std::string& external_file, |
||||
IngestedFileInfo* file_to_ingest); |
||||
|
||||
// Check if the files we are ingesting overlap with any memtable.
|
||||
// REQUIRES: Mutex held
|
||||
Status IngestedFilesOverlapWithMemtables(SuperVersion* sv, bool* overlap); |
||||
|
||||
// Assign `file_to_ingest` the lowest possible level that it can
|
||||
// be ingested to.
|
||||
// REQUIRES: Mutex held
|
||||
Status AssignLevelForIngestedFile(SuperVersion* sv, |
||||
IngestedFileInfo* file_to_ingest, |
||||
bool* overlap_with_db); |
||||
|
||||
// Set the file global sequence number to `seqno`
|
||||
Status AssignGlobalSeqnoForIngestedFile(IngestedFileInfo* file_to_ingest, |
||||
SequenceNumber seqno); |
||||
|
||||
// Check if `file_to_ingest` key range overlap with the range `iter` represent
|
||||
// REQUIRES: Mutex held
|
||||
Status IngestedFileOverlapWithIteratorRange( |
||||
const IngestedFileInfo* file_to_ingest, InternalIterator* iter, |
||||
bool* overlap); |
||||
|
||||
// Check if `file_to_ingest` can fit in level `level`
|
||||
// REQUIRES: Mutex held
|
||||
bool IngestedFileFitInLevel(const IngestedFileInfo* file_to_ingest, |
||||
int level); |
||||
|
||||
Env* env_; |
||||
VersionSet* versions_; |
||||
ColumnFamilyData* cfd_; |
||||
const ImmutableDBOptions& db_options_; |
||||
const EnvOptions& env_options_; |
||||
SnapshotList* db_snapshots_; |
||||
autovector<IngestedFileInfo> files_to_ingest_; |
||||
const IngestExternalFileOptions& ingestion_options_; |
||||
VersionEdit edit_; |
||||
uint64_t job_start_time_; |
||||
}; |
||||
|
||||
} // namespace rocksdb
|
Loading…
Reference in new issue