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/utilities/checkpoint/checkpoint_impl.cc

588 lines
22 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).
//
// Copyright (c) 2012 Facebook.
// Use of this source code is governed by a BSD-style license that can be
// found in the LICENSE file.
#ifndef ROCKSDB_LITE
#include "utilities/checkpoint/checkpoint_impl.h"
#include <algorithm>
#include <cinttypes>
#include <string>
#include <vector>
#include "db/wal_manager.h"
#include "file/file_util.h"
#include "file/filename.h"
#include "port/port.h"
#include "rocksdb/db.h"
#include "rocksdb/env.h"
Export Import sst files (#5495) Summary: Refresh of the earlier change here - https://github.com/facebook/rocksdb/issues/5135 This is a review request for code change needed for - https://github.com/facebook/rocksdb/issues/3469 "Add support for taking snapshot of a column family and creating column family from a given CF snapshot" We have an implementation for this that we have been testing internally. We have two new APIs that together provide this functionality. (1) ExportColumnFamily() - This API is modelled after CreateCheckpoint() as below. // Exports all live SST files of a specified Column Family onto export_dir, // returning SST files information in metadata. // - SST files will be created as hard links when the directory specified // is in the same partition as the db directory, copied otherwise. // - export_dir should not already exist and will be created by this API. // - Always triggers a flush. virtual Status ExportColumnFamily(ColumnFamilyHandle* handle, const std::string& export_dir, ExportImportFilesMetaData** metadata); Internally, the API will DisableFileDeletions(), GetColumnFamilyMetaData(), Parse through metadata, creating links/copies of all the sst files, EnableFileDeletions() and complete the call by returning the list of file metadata. (2) CreateColumnFamilyWithImport() - This API is modeled after IngestExternalFile(), but invoked only during a CF creation as below. // CreateColumnFamilyWithImport() will create a new column family with // column_family_name and import external SST files specified in metadata into // this column family. // (1) External SST files can be created using SstFileWriter. // (2) External SST files can be exported from a particular column family in // an existing DB. // Option in import_options specifies whether the external files are copied or // moved (default is copy). When option specifies copy, managing files at // external_file_path is caller's responsibility. When option specifies a // move, the call ensures that the specified files at external_file_path are // deleted on successful return and files are not modified on any error // return. // On error return, column family handle returned will be nullptr. // ColumnFamily will be present on successful return and will not be present // on error return. ColumnFamily may be present on any crash during this call. virtual Status CreateColumnFamilyWithImport( const ColumnFamilyOptions& options, const std::string& column_family_name, const ImportColumnFamilyOptions& import_options, const ExportImportFilesMetaData& metadata, ColumnFamilyHandle** handle); Internally, this API creates a new CF, parses all the sst files and adds it to the specified column family, at the same level and with same sequence number as in the metadata. Also performs safety checks with respect to overlaps between the sst files being imported. If incoming sequence number is higher than current local sequence number, local sequence number is updated to reflect this. Note, as the sst files is are being moved across Column Families, Column Family name in sst file will no longer match the actual column family on destination DB. The API does not modify Column Family name or id in the sst files being imported. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5495 Differential Revision: D16018881 fbshipit-source-id: 9ae2251025d5916d35a9fc4ea4d6707f6be16ff9
5 years ago
#include "rocksdb/metadata.h"
#include "rocksdb/transaction_log.h"
#include "rocksdb/utilities/checkpoint.h"
#include "test_util/sync_point.h"
#include "util/cast_util.h"
#include "util/file_checksum_helper.h"
namespace ROCKSDB_NAMESPACE {
Status Checkpoint::Create(DB* db, Checkpoint** checkpoint_ptr) {
*checkpoint_ptr = new CheckpointImpl(db);
return Status::OK();
}
Status Checkpoint::CreateCheckpoint(const std::string& /*checkpoint_dir*/,
uint64_t /*log_size_for_flush*/,
uint64_t* /*sequence_number_ptr*/) {
return Status::NotSupported("");
}
void CheckpointImpl::CleanStagingDirectory(
const std::string& full_private_path, Logger* info_log) {
std::vector<std::string> subchildren;
Status s = db_->GetEnv()->FileExists(full_private_path);
if (s.IsNotFound()) {
return;
}
ROCKS_LOG_INFO(info_log, "File exists %s -- %s",
full_private_path.c_str(), s.ToString().c_str());
s = db_->GetEnv()->GetChildren(full_private_path, &subchildren);
if (s.ok()) {
for (auto& subchild : subchildren) {
std::string subchild_path = full_private_path + "/" + subchild;
s = db_->GetEnv()->DeleteFile(subchild_path);
ROCKS_LOG_INFO(info_log, "Delete file %s -- %s", subchild_path.c_str(),
s.ToString().c_str());
}
}
// finally delete the private dir
s = db_->GetEnv()->DeleteDir(full_private_path);
ROCKS_LOG_INFO(info_log, "Delete dir %s -- %s",
full_private_path.c_str(), s.ToString().c_str());
}
Export Import sst files (#5495) Summary: Refresh of the earlier change here - https://github.com/facebook/rocksdb/issues/5135 This is a review request for code change needed for - https://github.com/facebook/rocksdb/issues/3469 "Add support for taking snapshot of a column family and creating column family from a given CF snapshot" We have an implementation for this that we have been testing internally. We have two new APIs that together provide this functionality. (1) ExportColumnFamily() - This API is modelled after CreateCheckpoint() as below. // Exports all live SST files of a specified Column Family onto export_dir, // returning SST files information in metadata. // - SST files will be created as hard links when the directory specified // is in the same partition as the db directory, copied otherwise. // - export_dir should not already exist and will be created by this API. // - Always triggers a flush. virtual Status ExportColumnFamily(ColumnFamilyHandle* handle, const std::string& export_dir, ExportImportFilesMetaData** metadata); Internally, the API will DisableFileDeletions(), GetColumnFamilyMetaData(), Parse through metadata, creating links/copies of all the sst files, EnableFileDeletions() and complete the call by returning the list of file metadata. (2) CreateColumnFamilyWithImport() - This API is modeled after IngestExternalFile(), but invoked only during a CF creation as below. // CreateColumnFamilyWithImport() will create a new column family with // column_family_name and import external SST files specified in metadata into // this column family. // (1) External SST files can be created using SstFileWriter. // (2) External SST files can be exported from a particular column family in // an existing DB. // Option in import_options specifies whether the external files are copied or // moved (default is copy). When option specifies copy, managing files at // external_file_path is caller's responsibility. When option specifies a // move, the call ensures that the specified files at external_file_path are // deleted on successful return and files are not modified on any error // return. // On error return, column family handle returned will be nullptr. // ColumnFamily will be present on successful return and will not be present // on error return. ColumnFamily may be present on any crash during this call. virtual Status CreateColumnFamilyWithImport( const ColumnFamilyOptions& options, const std::string& column_family_name, const ImportColumnFamilyOptions& import_options, const ExportImportFilesMetaData& metadata, ColumnFamilyHandle** handle); Internally, this API creates a new CF, parses all the sst files and adds it to the specified column family, at the same level and with same sequence number as in the metadata. Also performs safety checks with respect to overlaps between the sst files being imported. If incoming sequence number is higher than current local sequence number, local sequence number is updated to reflect this. Note, as the sst files is are being moved across Column Families, Column Family name in sst file will no longer match the actual column family on destination DB. The API does not modify Column Family name or id in the sst files being imported. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5495 Differential Revision: D16018881 fbshipit-source-id: 9ae2251025d5916d35a9fc4ea4d6707f6be16ff9
5 years ago
Status Checkpoint::ExportColumnFamily(
ColumnFamilyHandle* /*handle*/, const std::string& /*export_dir*/,
ExportImportFilesMetaData** /*metadata*/) {
return Status::NotSupported("");
}
// Builds an openable snapshot of RocksDB
Status CheckpointImpl::CreateCheckpoint(const std::string& checkpoint_dir,
uint64_t log_size_for_flush,
uint64_t* sequence_number_ptr) {
DBOptions db_options = db_->GetDBOptions();
Status s = db_->GetEnv()->FileExists(checkpoint_dir);
if (s.ok()) {
return Status::InvalidArgument("Directory exists");
} else if (!s.IsNotFound()) {
assert(s.IsIOError());
return s;
}
ROCKS_LOG_INFO(
db_options.info_log,
"Started the snapshot process -- creating snapshot in directory %s",
checkpoint_dir.c_str());
size_t final_nonslash_idx = checkpoint_dir.find_last_not_of('/');
if (final_nonslash_idx == std::string::npos) {
// npos means it's only slashes or empty. Non-empty means it's the root
// directory, but it shouldn't be because we verified above the directory
// doesn't exist.
assert(checkpoint_dir.empty());
return Status::InvalidArgument("invalid checkpoint directory name");
}
std::string full_private_path =
checkpoint_dir.substr(0, final_nonslash_idx + 1) + ".tmp";
ROCKS_LOG_INFO(
db_options.info_log,
"Snapshot process -- using temporary directory %s",
full_private_path.c_str());
CleanStagingDirectory(full_private_path, db_options.info_log.get());
// create snapshot directory
s = db_->GetEnv()->CreateDir(full_private_path);
uint64_t sequence_number = 0;
if (s.ok()) {
// enable file deletions
s = db_->DisableFileDeletions();
const bool disabled_file_deletions = s.ok();
if (s.ok() || s.IsNotSupported()) {
s = CreateCustomCheckpoint(
db_options,
[&](const std::string& src_dirname, const std::string& fname,
FileType) {
ROCKS_LOG_INFO(db_options.info_log, "Hard Linking %s",
fname.c_str());
return db_->GetFileSystem()->LinkFile(src_dirname + fname,
full_private_path + fname,
IOOptions(), nullptr);
} /* link_file_cb */,
[&](const std::string& src_dirname, const std::string& fname,
uint64_t size_limit_bytes, FileType,
const std::string& /* checksum_func_name */,
const std::string& /* checksum_val */) {
ROCKS_LOG_INFO(db_options.info_log, "Copying %s", fname.c_str());
return CopyFile(db_->GetFileSystem(), src_dirname + fname,
full_private_path + fname, size_limit_bytes,
db_options.use_fsync);
} /* copy_file_cb */,
[&](const std::string& fname, const std::string& contents, FileType) {
ROCKS_LOG_INFO(db_options.info_log, "Creating %s", fname.c_str());
return CreateFile(db_->GetFileSystem(), full_private_path + fname,
contents, db_options.use_fsync);
} /* create_file_cb */,
&sequence_number, log_size_for_flush);
// we copied all the files, enable file deletions
if (disabled_file_deletions) {
Status ss = db_->EnableFileDeletions(false);
assert(ss.ok());
ss.PermitUncheckedError();
}
}
}
if (s.ok()) {
// move tmp private backup to real snapshot directory
s = db_->GetEnv()->RenameFile(full_private_path, checkpoint_dir);
}
if (s.ok()) {
std::unique_ptr<Directory> checkpoint_directory;
s = db_->GetEnv()->NewDirectory(checkpoint_dir, &checkpoint_directory);
if (s.ok() && checkpoint_directory != nullptr) {
s = checkpoint_directory->Fsync();
}
}
if (s.ok()) {
if (sequence_number_ptr != nullptr) {
*sequence_number_ptr = sequence_number;
}
// here we know that we succeeded and installed the new snapshot
ROCKS_LOG_INFO(db_options.info_log, "Snapshot DONE. All is good");
ROCKS_LOG_INFO(db_options.info_log, "Snapshot sequence number: %" PRIu64,
sequence_number);
} else {
// clean all the files we might have created
ROCKS_LOG_INFO(db_options.info_log, "Snapshot failed -- %s",
s.ToString().c_str());
CleanStagingDirectory(full_private_path, db_options.info_log.get());
}
return s;
}
Status CheckpointImpl::CreateCustomCheckpoint(
const DBOptions& db_options,
std::function<Status(const std::string& src_dirname,
const std::string& src_fname, FileType type)>
link_file_cb,
std::function<Status(
const std::string& src_dirname, const std::string& src_fname,
uint64_t size_limit_bytes, FileType type,
const std::string& checksum_func_name, const std::string& checksum_val)>
copy_file_cb,
std::function<Status(const std::string& fname, const std::string& contents,
FileType type)>
create_file_cb,
uint64_t* sequence_number, uint64_t log_size_for_flush,
bool get_live_table_checksum) {
Status s;
std::vector<std::string> live_files;
uint64_t manifest_file_size = 0;
uint64_t min_log_num = port::kMaxUint64;
*sequence_number = db_->GetLatestSequenceNumber();
bool same_fs = true;
VectorLogPtr live_wal_files;
bool flush_memtable = true;
if (!db_options.allow_2pc) {
if (log_size_for_flush == port::kMaxUint64) {
flush_memtable = false;
} else if (log_size_for_flush > 0) {
// If out standing log files are small, we skip the flush.
s = db_->GetSortedWalFiles(live_wal_files);
if (!s.ok()) {
return s;
}
// Don't flush column families if total log size is smaller than
// log_size_for_flush. We copy the log files instead.
// We may be able to cover 2PC case too.
uint64_t total_wal_size = 0;
for (auto& wal : live_wal_files) {
total_wal_size += wal->SizeFileBytes();
}
if (total_wal_size < log_size_for_flush) {
flush_memtable = false;
}
live_wal_files.clear();
}
}
// this will return live_files prefixed with "/"
s = db_->GetLiveFiles(live_files, &manifest_file_size, flush_memtable);
Skip WALs according to MinLogNumberToKeep when creating checkpoint (#7789) Summary: In a stress test failure, we observe that a WAL is skipped when creating checkpoint, although its log number >= MinLogNumberToKeep(). This might happen in the following case: 1. when creating the checkpoint, there are 2 column families: CF0 and CF1, and there are 2 WALs: 1, 2; 2. CF0's log number is 1, CF0's active memtable is empty, CF1's log number is 2, CF1's active memtable is not empty, WAL 2 is not empty, the sequence number points to WAL 2; 2. the checkpoint process flushes CF0, since CF0' active memtable is empty, there is no need to SwitchMemtable, thus no new WAL will be created, so CF0's log number is now 2, concurrently, some data is written to CF0 and WAL 2; 3. the checkpoint process flushes CF1, WAL 3 is created and CF1's log number is now 3, CF0's log number is still 2 because CF0 is not empty and WAL 2 contains its unflushed data concurrently written in step 2; 4. the checkpoint process determines that WAL 1 and 2 are no longer needed according to [live_wal_files[i]->StartSequence() >= *sequence_number](https://github.com/facebook/rocksdb/blob/master/utilities/checkpoint/checkpoint_impl.cc#L388), so it skips linking them to the checkpoint directory; 5. but according to `MinLogNumberToKeep()`, WAL 2 still needs to be kept because CF0's log number is 2. If the checkpoint is reopened in read-only mode, and only read from the snapshot with the initial sequence number, then there will be no data loss or data inconsistency. But if the checkpoint is reopened and read from the most recent sequence number, suppose in step 3, there are also data concurrently written to CF1 and WAL 3, then the most recent sequence number refers to the latest entry in WAL 3, so the data written in step 2 should also be visible, but since WAL 2 is discarded, those data are lost. When tracking WAL in MANIFEST is enabled, when reopening the checkpoint, since WAL 2 is still tracked in MANIFEST as alive, but it's missing from the checkpoint directory, a corruption will be reported. This PR makes the checkpoint process to only skip a WAL if its log number < `MinLogNumberToKeep`. Pull Request resolved: https://github.com/facebook/rocksdb/pull/7789 Test Plan: watch existing tests to pass. Reviewed By: ajkr Differential Revision: D25662346 Pulled By: cheng-chang fbshipit-source-id: 136471095baa01886cf44809455cf855f24857a0
4 years ago
if (!db_->GetIntProperty(DB::Properties::kMinLogNumberToKeep, &min_log_num)) {
return Status::InvalidArgument("cannot get the min log number to keep.");
}
// Between GetLiveFiles and getting min_log_num, flush might happen
// concurrently, so new WAL deletions might be tracked in MANIFEST. If we do
// not get the new MANIFEST size, the deleted WALs might not be reflected in
// the checkpoint's MANIFEST.
//
// If we get min_log_num before the above GetLiveFiles, then there might
// be too many unnecessary WALs to be included in the checkpoint.
//
// Ideally, min_log_num should be got together with manifest_file_size in
// GetLiveFiles atomically. But that needs changes to GetLiveFiles' signature
// which is a public API.
s = db_->GetLiveFiles(live_files, &manifest_file_size, flush_memtable);
TEST_SYNC_POINT("CheckpointImpl::CreateCheckpoint:SavedLiveFiles1");
TEST_SYNC_POINT("CheckpointImpl::CreateCheckpoint:SavedLiveFiles2");
if (s.ok()) {
s = db_->FlushWAL(false /* sync */);
}
TEST_SYNC_POINT("CheckpointImpl::CreateCustomCheckpoint:AfterGetLive1");
TEST_SYNC_POINT("CheckpointImpl::CreateCustomCheckpoint:AfterGetLive2");
// if we have more than one column family, we need to also get WAL files
if (s.ok()) {
s = db_->GetSortedWalFiles(live_wal_files);
}
if (!s.ok()) {
return s;
}
size_t wal_size = live_wal_files.size();
// process live files, non-table files first
std::string manifest_fname, current_fname;
// record table files for processing next
std::vector<std::pair<std::string, uint64_t>> live_table_files;
for (auto& live_file : live_files) {
if (!s.ok()) {
break;
}
uint64_t number;
FileType type;
bool ok = ParseFileName(live_file, &number, &type);
if (!ok) {
s = Status::Corruption("Can't parse file name. This is very bad");
break;
}
// we should only get sst, options, manifest and current files here
assert(type == kTableFile || type == kDescriptorFile ||
type == kCurrentFile || type == kOptionsFile);
assert(live_file.size() > 0 && live_file[0] == '/');
if (type == kCurrentFile) {
// We will craft the current file manually to ensure it's consistent with
// the manifest number. This is necessary because current's file contents
// can change during checkpoint creation.
current_fname = live_file;
continue;
} else if (type == kDescriptorFile) {
manifest_fname = live_file;
}
if (type != kTableFile) {
// copy non-table files here
// * if it's kDescriptorFile, limit the size to manifest_file_size
s = copy_file_cb(db_->GetName(), live_file,
(type == kDescriptorFile) ? manifest_file_size : 0, type,
kUnknownFileChecksumFuncName, kUnknownFileChecksum);
} else {
// process table files below
live_table_files.push_back(make_pair(live_file, number));
}
}
// get checksum info for table files
// get table file checksums if get_live_table_checksum is true
std::unique_ptr<FileChecksumList> checksum_list;
if (s.ok() && get_live_table_checksum) {
checksum_list.reset(NewFileChecksumList());
// should succeed even without checksum info present, else manifest
// is corrupt
s = GetFileChecksumsFromManifest(db_->GetEnv(),
db_->GetName() + manifest_fname,
manifest_file_size, checksum_list.get());
}
// copy/hard link live table files
for (auto& ltf : live_table_files) {
if (!s.ok()) {
break;
}
std::string& src_fname = ltf.first;
uint64_t number = ltf.second;
// rules:
// * for kTableFile, attempt hard link instead of copy.
// * but can't hard link across filesystems.
if (same_fs) {
s = link_file_cb(db_->GetName(), src_fname, kTableFile);
if (s.IsNotSupported()) {
same_fs = false;
s = Status::OK();
}
}
if (!same_fs) {
std::string checksum_name = kUnknownFileChecksumFuncName;
std::string checksum_value = kUnknownFileChecksum;
// we ignore the checksums either they are not required or we failed to
// obtain the checksum lsit for old table files that have no file
// checksums
if (get_live_table_checksum) {
// find checksum info for table files
Status search = checksum_list->SearchOneFileChecksum(
number, &checksum_value, &checksum_name);
// could be a legacy file lacking checksum info. overall OK if
// not found
if (!search.ok()) {
assert(checksum_name == kUnknownFileChecksumFuncName);
assert(checksum_value == kUnknownFileChecksum);
}
}
s = copy_file_cb(db_->GetName(), src_fname, 0, kTableFile, checksum_name,
checksum_value);
}
}
if (s.ok() && !current_fname.empty() && !manifest_fname.empty()) {
s = create_file_cb(current_fname, manifest_fname.substr(1) + "\n",
kCurrentFile);
}
ROCKS_LOG_INFO(db_options.info_log, "Number of log files %" ROCKSDB_PRIszt,
live_wal_files.size());
// Link WAL files. Copy exact size of last one because it is the only one
// that has changes after the last flush.
for (size_t i = 0; s.ok() && i < wal_size; ++i) {
if ((live_wal_files[i]->Type() == kAliveLogFile) &&
(!flush_memtable ||
live_wal_files[i]->LogNumber() >= min_log_num)) {
if (i + 1 == wal_size) {
s = copy_file_cb(db_options.wal_dir, live_wal_files[i]->PathName(),
live_wal_files[i]->SizeFileBytes(), kWalFile,
kUnknownFileChecksumFuncName, kUnknownFileChecksum);
break;
}
if (same_fs) {
// we only care about live log files
s = link_file_cb(db_options.wal_dir, live_wal_files[i]->PathName(),
kWalFile);
if (s.IsNotSupported()) {
same_fs = false;
s = Status::OK();
}
}
if (!same_fs) {
s = copy_file_cb(db_options.wal_dir, live_wal_files[i]->PathName(), 0,
kWalFile, kUnknownFileChecksumFuncName,
kUnknownFileChecksum);
}
}
}
return s;
}
Export Import sst files (#5495) Summary: Refresh of the earlier change here - https://github.com/facebook/rocksdb/issues/5135 This is a review request for code change needed for - https://github.com/facebook/rocksdb/issues/3469 "Add support for taking snapshot of a column family and creating column family from a given CF snapshot" We have an implementation for this that we have been testing internally. We have two new APIs that together provide this functionality. (1) ExportColumnFamily() - This API is modelled after CreateCheckpoint() as below. // Exports all live SST files of a specified Column Family onto export_dir, // returning SST files information in metadata. // - SST files will be created as hard links when the directory specified // is in the same partition as the db directory, copied otherwise. // - export_dir should not already exist and will be created by this API. // - Always triggers a flush. virtual Status ExportColumnFamily(ColumnFamilyHandle* handle, const std::string& export_dir, ExportImportFilesMetaData** metadata); Internally, the API will DisableFileDeletions(), GetColumnFamilyMetaData(), Parse through metadata, creating links/copies of all the sst files, EnableFileDeletions() and complete the call by returning the list of file metadata. (2) CreateColumnFamilyWithImport() - This API is modeled after IngestExternalFile(), but invoked only during a CF creation as below. // CreateColumnFamilyWithImport() will create a new column family with // column_family_name and import external SST files specified in metadata into // this column family. // (1) External SST files can be created using SstFileWriter. // (2) External SST files can be exported from a particular column family in // an existing DB. // Option in import_options specifies whether the external files are copied or // moved (default is copy). When option specifies copy, managing files at // external_file_path is caller's responsibility. When option specifies a // move, the call ensures that the specified files at external_file_path are // deleted on successful return and files are not modified on any error // return. // On error return, column family handle returned will be nullptr. // ColumnFamily will be present on successful return and will not be present // on error return. ColumnFamily may be present on any crash during this call. virtual Status CreateColumnFamilyWithImport( const ColumnFamilyOptions& options, const std::string& column_family_name, const ImportColumnFamilyOptions& import_options, const ExportImportFilesMetaData& metadata, ColumnFamilyHandle** handle); Internally, this API creates a new CF, parses all the sst files and adds it to the specified column family, at the same level and with same sequence number as in the metadata. Also performs safety checks with respect to overlaps between the sst files being imported. If incoming sequence number is higher than current local sequence number, local sequence number is updated to reflect this. Note, as the sst files is are being moved across Column Families, Column Family name in sst file will no longer match the actual column family on destination DB. The API does not modify Column Family name or id in the sst files being imported. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5495 Differential Revision: D16018881 fbshipit-source-id: 9ae2251025d5916d35a9fc4ea4d6707f6be16ff9
5 years ago
// Exports all live SST files of a specified Column Family onto export_dir,
// returning SST files information in metadata.
Status CheckpointImpl::ExportColumnFamily(
ColumnFamilyHandle* handle, const std::string& export_dir,
ExportImportFilesMetaData** metadata) {
auto cfh = static_cast_with_check<ColumnFamilyHandleImpl>(handle);
Export Import sst files (#5495) Summary: Refresh of the earlier change here - https://github.com/facebook/rocksdb/issues/5135 This is a review request for code change needed for - https://github.com/facebook/rocksdb/issues/3469 "Add support for taking snapshot of a column family and creating column family from a given CF snapshot" We have an implementation for this that we have been testing internally. We have two new APIs that together provide this functionality. (1) ExportColumnFamily() - This API is modelled after CreateCheckpoint() as below. // Exports all live SST files of a specified Column Family onto export_dir, // returning SST files information in metadata. // - SST files will be created as hard links when the directory specified // is in the same partition as the db directory, copied otherwise. // - export_dir should not already exist and will be created by this API. // - Always triggers a flush. virtual Status ExportColumnFamily(ColumnFamilyHandle* handle, const std::string& export_dir, ExportImportFilesMetaData** metadata); Internally, the API will DisableFileDeletions(), GetColumnFamilyMetaData(), Parse through metadata, creating links/copies of all the sst files, EnableFileDeletions() and complete the call by returning the list of file metadata. (2) CreateColumnFamilyWithImport() - This API is modeled after IngestExternalFile(), but invoked only during a CF creation as below. // CreateColumnFamilyWithImport() will create a new column family with // column_family_name and import external SST files specified in metadata into // this column family. // (1) External SST files can be created using SstFileWriter. // (2) External SST files can be exported from a particular column family in // an existing DB. // Option in import_options specifies whether the external files are copied or // moved (default is copy). When option specifies copy, managing files at // external_file_path is caller's responsibility. When option specifies a // move, the call ensures that the specified files at external_file_path are // deleted on successful return and files are not modified on any error // return. // On error return, column family handle returned will be nullptr. // ColumnFamily will be present on successful return and will not be present // on error return. ColumnFamily may be present on any crash during this call. virtual Status CreateColumnFamilyWithImport( const ColumnFamilyOptions& options, const std::string& column_family_name, const ImportColumnFamilyOptions& import_options, const ExportImportFilesMetaData& metadata, ColumnFamilyHandle** handle); Internally, this API creates a new CF, parses all the sst files and adds it to the specified column family, at the same level and with same sequence number as in the metadata. Also performs safety checks with respect to overlaps between the sst files being imported. If incoming sequence number is higher than current local sequence number, local sequence number is updated to reflect this. Note, as the sst files is are being moved across Column Families, Column Family name in sst file will no longer match the actual column family on destination DB. The API does not modify Column Family name or id in the sst files being imported. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5495 Differential Revision: D16018881 fbshipit-source-id: 9ae2251025d5916d35a9fc4ea4d6707f6be16ff9
5 years ago
const auto cf_name = cfh->GetName();
const auto db_options = db_->GetDBOptions();
assert(metadata != nullptr);
assert(*metadata == nullptr);
auto s = db_->GetEnv()->FileExists(export_dir);
if (s.ok()) {
return Status::InvalidArgument("Specified export_dir exists");
} else if (!s.IsNotFound()) {
assert(s.IsIOError());
return s;
}
const auto final_nonslash_idx = export_dir.find_last_not_of('/');
if (final_nonslash_idx == std::string::npos) {
return Status::InvalidArgument("Specified export_dir invalid");
}
ROCKS_LOG_INFO(db_options.info_log,
"[%s] export column family onto export directory %s",
cf_name.c_str(), export_dir.c_str());
// Create a temporary export directory.
const auto tmp_export_dir =
export_dir.substr(0, final_nonslash_idx + 1) + ".tmp";
s = db_->GetEnv()->CreateDir(tmp_export_dir);
if (s.ok()) {
s = db_->Flush(ROCKSDB_NAMESPACE::FlushOptions(), handle);
Export Import sst files (#5495) Summary: Refresh of the earlier change here - https://github.com/facebook/rocksdb/issues/5135 This is a review request for code change needed for - https://github.com/facebook/rocksdb/issues/3469 "Add support for taking snapshot of a column family and creating column family from a given CF snapshot" We have an implementation for this that we have been testing internally. We have two new APIs that together provide this functionality. (1) ExportColumnFamily() - This API is modelled after CreateCheckpoint() as below. // Exports all live SST files of a specified Column Family onto export_dir, // returning SST files information in metadata. // - SST files will be created as hard links when the directory specified // is in the same partition as the db directory, copied otherwise. // - export_dir should not already exist and will be created by this API. // - Always triggers a flush. virtual Status ExportColumnFamily(ColumnFamilyHandle* handle, const std::string& export_dir, ExportImportFilesMetaData** metadata); Internally, the API will DisableFileDeletions(), GetColumnFamilyMetaData(), Parse through metadata, creating links/copies of all the sst files, EnableFileDeletions() and complete the call by returning the list of file metadata. (2) CreateColumnFamilyWithImport() - This API is modeled after IngestExternalFile(), but invoked only during a CF creation as below. // CreateColumnFamilyWithImport() will create a new column family with // column_family_name and import external SST files specified in metadata into // this column family. // (1) External SST files can be created using SstFileWriter. // (2) External SST files can be exported from a particular column family in // an existing DB. // Option in import_options specifies whether the external files are copied or // moved (default is copy). When option specifies copy, managing files at // external_file_path is caller's responsibility. When option specifies a // move, the call ensures that the specified files at external_file_path are // deleted on successful return and files are not modified on any error // return. // On error return, column family handle returned will be nullptr. // ColumnFamily will be present on successful return and will not be present // on error return. ColumnFamily may be present on any crash during this call. virtual Status CreateColumnFamilyWithImport( const ColumnFamilyOptions& options, const std::string& column_family_name, const ImportColumnFamilyOptions& import_options, const ExportImportFilesMetaData& metadata, ColumnFamilyHandle** handle); Internally, this API creates a new CF, parses all the sst files and adds it to the specified column family, at the same level and with same sequence number as in the metadata. Also performs safety checks with respect to overlaps between the sst files being imported. If incoming sequence number is higher than current local sequence number, local sequence number is updated to reflect this. Note, as the sst files is are being moved across Column Families, Column Family name in sst file will no longer match the actual column family on destination DB. The API does not modify Column Family name or id in the sst files being imported. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5495 Differential Revision: D16018881 fbshipit-source-id: 9ae2251025d5916d35a9fc4ea4d6707f6be16ff9
5 years ago
}
ColumnFamilyMetaData db_metadata;
if (s.ok()) {
// Export live sst files with file deletions disabled.
s = db_->DisableFileDeletions();
if (s.ok()) {
db_->GetColumnFamilyMetaData(handle, &db_metadata);
s = ExportFilesInMetaData(
db_options, db_metadata,
[&](const std::string& src_dirname, const std::string& fname) {
ROCKS_LOG_INFO(db_options.info_log, "[%s] HardLinking %s",
cf_name.c_str(), fname.c_str());
return db_->GetEnv()->LinkFile(src_dirname + fname,
tmp_export_dir + fname);
} /*link_file_cb*/,
[&](const std::string& src_dirname, const std::string& fname) {
ROCKS_LOG_INFO(db_options.info_log, "[%s] Copying %s",
cf_name.c_str(), fname.c_str());
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 CopyFile(db_->GetFileSystem(), src_dirname + fname,
Export Import sst files (#5495) Summary: Refresh of the earlier change here - https://github.com/facebook/rocksdb/issues/5135 This is a review request for code change needed for - https://github.com/facebook/rocksdb/issues/3469 "Add support for taking snapshot of a column family and creating column family from a given CF snapshot" We have an implementation for this that we have been testing internally. We have two new APIs that together provide this functionality. (1) ExportColumnFamily() - This API is modelled after CreateCheckpoint() as below. // Exports all live SST files of a specified Column Family onto export_dir, // returning SST files information in metadata. // - SST files will be created as hard links when the directory specified // is in the same partition as the db directory, copied otherwise. // - export_dir should not already exist and will be created by this API. // - Always triggers a flush. virtual Status ExportColumnFamily(ColumnFamilyHandle* handle, const std::string& export_dir, ExportImportFilesMetaData** metadata); Internally, the API will DisableFileDeletions(), GetColumnFamilyMetaData(), Parse through metadata, creating links/copies of all the sst files, EnableFileDeletions() and complete the call by returning the list of file metadata. (2) CreateColumnFamilyWithImport() - This API is modeled after IngestExternalFile(), but invoked only during a CF creation as below. // CreateColumnFamilyWithImport() will create a new column family with // column_family_name and import external SST files specified in metadata into // this column family. // (1) External SST files can be created using SstFileWriter. // (2) External SST files can be exported from a particular column family in // an existing DB. // Option in import_options specifies whether the external files are copied or // moved (default is copy). When option specifies copy, managing files at // external_file_path is caller's responsibility. When option specifies a // move, the call ensures that the specified files at external_file_path are // deleted on successful return and files are not modified on any error // return. // On error return, column family handle returned will be nullptr. // ColumnFamily will be present on successful return and will not be present // on error return. ColumnFamily may be present on any crash during this call. virtual Status CreateColumnFamilyWithImport( const ColumnFamilyOptions& options, const std::string& column_family_name, const ImportColumnFamilyOptions& import_options, const ExportImportFilesMetaData& metadata, ColumnFamilyHandle** handle); Internally, this API creates a new CF, parses all the sst files and adds it to the specified column family, at the same level and with same sequence number as in the metadata. Also performs safety checks with respect to overlaps between the sst files being imported. If incoming sequence number is higher than current local sequence number, local sequence number is updated to reflect this. Note, as the sst files is are being moved across Column Families, Column Family name in sst file will no longer match the actual column family on destination DB. The API does not modify Column Family name or id in the sst files being imported. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5495 Differential Revision: D16018881 fbshipit-source-id: 9ae2251025d5916d35a9fc4ea4d6707f6be16ff9
5 years ago
tmp_export_dir + fname, 0, db_options.use_fsync);
} /*copy_file_cb*/);
const auto enable_status = db_->EnableFileDeletions(false /*force*/);
if (s.ok()) {
s = enable_status;
}
}
}
auto moved_to_user_specified_dir = false;
if (s.ok()) {
// Move temporary export directory to the actual export directory.
s = db_->GetEnv()->RenameFile(tmp_export_dir, export_dir);
}
if (s.ok()) {
// Fsync export directory.
moved_to_user_specified_dir = true;
std::unique_ptr<Directory> dir_ptr;
s = db_->GetEnv()->NewDirectory(export_dir, &dir_ptr);
if (s.ok()) {
assert(dir_ptr != nullptr);
s = dir_ptr->Fsync();
}
}
if (s.ok()) {
// Export of files succeeded. Fill in the metadata information.
auto result_metadata = new ExportImportFilesMetaData();
result_metadata->db_comparator_name = handle->GetComparator()->Name();
for (const auto& level_metadata : db_metadata.levels) {
for (const auto& file_metadata : level_metadata.files) {
LiveFileMetaData live_file_metadata;
live_file_metadata.size = file_metadata.size;
live_file_metadata.name = std::move(file_metadata.name);
live_file_metadata.file_number = file_metadata.file_number;
Export Import sst files (#5495) Summary: Refresh of the earlier change here - https://github.com/facebook/rocksdb/issues/5135 This is a review request for code change needed for - https://github.com/facebook/rocksdb/issues/3469 "Add support for taking snapshot of a column family and creating column family from a given CF snapshot" We have an implementation for this that we have been testing internally. We have two new APIs that together provide this functionality. (1) ExportColumnFamily() - This API is modelled after CreateCheckpoint() as below. // Exports all live SST files of a specified Column Family onto export_dir, // returning SST files information in metadata. // - SST files will be created as hard links when the directory specified // is in the same partition as the db directory, copied otherwise. // - export_dir should not already exist and will be created by this API. // - Always triggers a flush. virtual Status ExportColumnFamily(ColumnFamilyHandle* handle, const std::string& export_dir, ExportImportFilesMetaData** metadata); Internally, the API will DisableFileDeletions(), GetColumnFamilyMetaData(), Parse through metadata, creating links/copies of all the sst files, EnableFileDeletions() and complete the call by returning the list of file metadata. (2) CreateColumnFamilyWithImport() - This API is modeled after IngestExternalFile(), but invoked only during a CF creation as below. // CreateColumnFamilyWithImport() will create a new column family with // column_family_name and import external SST files specified in metadata into // this column family. // (1) External SST files can be created using SstFileWriter. // (2) External SST files can be exported from a particular column family in // an existing DB. // Option in import_options specifies whether the external files are copied or // moved (default is copy). When option specifies copy, managing files at // external_file_path is caller's responsibility. When option specifies a // move, the call ensures that the specified files at external_file_path are // deleted on successful return and files are not modified on any error // return. // On error return, column family handle returned will be nullptr. // ColumnFamily will be present on successful return and will not be present // on error return. ColumnFamily may be present on any crash during this call. virtual Status CreateColumnFamilyWithImport( const ColumnFamilyOptions& options, const std::string& column_family_name, const ImportColumnFamilyOptions& import_options, const ExportImportFilesMetaData& metadata, ColumnFamilyHandle** handle); Internally, this API creates a new CF, parses all the sst files and adds it to the specified column family, at the same level and with same sequence number as in the metadata. Also performs safety checks with respect to overlaps between the sst files being imported. If incoming sequence number is higher than current local sequence number, local sequence number is updated to reflect this. Note, as the sst files is are being moved across Column Families, Column Family name in sst file will no longer match the actual column family on destination DB. The API does not modify Column Family name or id in the sst files being imported. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5495 Differential Revision: D16018881 fbshipit-source-id: 9ae2251025d5916d35a9fc4ea4d6707f6be16ff9
5 years ago
live_file_metadata.db_path = export_dir;
live_file_metadata.smallest_seqno = file_metadata.smallest_seqno;
live_file_metadata.largest_seqno = file_metadata.largest_seqno;
live_file_metadata.smallestkey = std::move(file_metadata.smallestkey);
live_file_metadata.largestkey = std::move(file_metadata.largestkey);
live_file_metadata.oldest_blob_file_number =
file_metadata.oldest_blob_file_number;
Export Import sst files (#5495) Summary: Refresh of the earlier change here - https://github.com/facebook/rocksdb/issues/5135 This is a review request for code change needed for - https://github.com/facebook/rocksdb/issues/3469 "Add support for taking snapshot of a column family and creating column family from a given CF snapshot" We have an implementation for this that we have been testing internally. We have two new APIs that together provide this functionality. (1) ExportColumnFamily() - This API is modelled after CreateCheckpoint() as below. // Exports all live SST files of a specified Column Family onto export_dir, // returning SST files information in metadata. // - SST files will be created as hard links when the directory specified // is in the same partition as the db directory, copied otherwise. // - export_dir should not already exist and will be created by this API. // - Always triggers a flush. virtual Status ExportColumnFamily(ColumnFamilyHandle* handle, const std::string& export_dir, ExportImportFilesMetaData** metadata); Internally, the API will DisableFileDeletions(), GetColumnFamilyMetaData(), Parse through metadata, creating links/copies of all the sst files, EnableFileDeletions() and complete the call by returning the list of file metadata. (2) CreateColumnFamilyWithImport() - This API is modeled after IngestExternalFile(), but invoked only during a CF creation as below. // CreateColumnFamilyWithImport() will create a new column family with // column_family_name and import external SST files specified in metadata into // this column family. // (1) External SST files can be created using SstFileWriter. // (2) External SST files can be exported from a particular column family in // an existing DB. // Option in import_options specifies whether the external files are copied or // moved (default is copy). When option specifies copy, managing files at // external_file_path is caller's responsibility. When option specifies a // move, the call ensures that the specified files at external_file_path are // deleted on successful return and files are not modified on any error // return. // On error return, column family handle returned will be nullptr. // ColumnFamily will be present on successful return and will not be present // on error return. ColumnFamily may be present on any crash during this call. virtual Status CreateColumnFamilyWithImport( const ColumnFamilyOptions& options, const std::string& column_family_name, const ImportColumnFamilyOptions& import_options, const ExportImportFilesMetaData& metadata, ColumnFamilyHandle** handle); Internally, this API creates a new CF, parses all the sst files and adds it to the specified column family, at the same level and with same sequence number as in the metadata. Also performs safety checks with respect to overlaps between the sst files being imported. If incoming sequence number is higher than current local sequence number, local sequence number is updated to reflect this. Note, as the sst files is are being moved across Column Families, Column Family name in sst file will no longer match the actual column family on destination DB. The API does not modify Column Family name or id in the sst files being imported. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5495 Differential Revision: D16018881 fbshipit-source-id: 9ae2251025d5916d35a9fc4ea4d6707f6be16ff9
5 years ago
live_file_metadata.level = level_metadata.level;
result_metadata->files.push_back(live_file_metadata);
}
*metadata = result_metadata;
}
ROCKS_LOG_INFO(db_options.info_log, "[%s] Export succeeded.",
cf_name.c_str());
} else {
// Failure: Clean up all the files/directories created.
ROCKS_LOG_INFO(db_options.info_log, "[%s] Export failed. %s",
cf_name.c_str(), s.ToString().c_str());
std::vector<std::string> subchildren;
const auto cleanup_dir =
moved_to_user_specified_dir ? export_dir : tmp_export_dir;
db_->GetEnv()->GetChildren(cleanup_dir, &subchildren);
for (const auto& subchild : subchildren) {
const auto subchild_path = cleanup_dir + "/" + subchild;
const auto status = db_->GetEnv()->DeleteFile(subchild_path);
if (!status.ok()) {
ROCKS_LOG_WARN(db_options.info_log, "Failed to cleanup file %s: %s",
subchild_path.c_str(), status.ToString().c_str());
}
}
const auto status = db_->GetEnv()->DeleteDir(cleanup_dir);
if (!status.ok()) {
ROCKS_LOG_WARN(db_options.info_log, "Failed to cleanup dir %s: %s",
cleanup_dir.c_str(), status.ToString().c_str());
}
}
return s;
}
Status CheckpointImpl::ExportFilesInMetaData(
const DBOptions& db_options, const ColumnFamilyMetaData& metadata,
std::function<Status(const std::string& src_dirname,
const std::string& src_fname)>
link_file_cb,
std::function<Status(const std::string& src_dirname,
const std::string& src_fname)>
copy_file_cb) {
Status s;
auto hardlink_file = true;
// Copy/hard link files in metadata.
size_t num_files = 0;
for (const auto& level_metadata : metadata.levels) {
for (const auto& file_metadata : level_metadata.files) {
uint64_t number;
FileType type;
const auto ok = ParseFileName(file_metadata.name, &number, &type);
if (!ok) {
s = Status::Corruption("Could not parse file name");
break;
}
// We should only get sst files here.
assert(type == kTableFile);
assert(file_metadata.size > 0 && file_metadata.name[0] == '/');
const auto src_fname = file_metadata.name;
++num_files;
if (hardlink_file) {
s = link_file_cb(db_->GetName(), src_fname);
if (num_files == 1 && s.IsNotSupported()) {
// Fallback to copy if link failed due to cross-device directories.
hardlink_file = false;
s = Status::OK();
}
}
if (!hardlink_file) {
s = copy_file_cb(db_->GetName(), src_fname);
}
if (!s.ok()) {
break;
}
}
}
ROCKS_LOG_INFO(db_options.info_log, "Number of table files %" ROCKSDB_PRIszt,
num_files);
return s;
}
} // namespace ROCKSDB_NAMESPACE
#endif // ROCKSDB_LITE