Add EnvTestWithParam::OptionsTest to the ASSERT_STATUS_CHECKED passes (#7283)

Summary:
This test uses database functionality and required more extensive work to get it to pass than the other tests.  The DB functionality required for this test now passes the check.

When it was unclear what the proper behavior was for unchecked status codes, a TODO was added.

Pull Request resolved: https://github.com/facebook/rocksdb/pull/7283

Reviewed By: akankshamahajan15

Differential Revision: D23251497

Pulled By: ajkr

fbshipit-source-id: 52b79629bdafa0a58de8ead1d1d66f141b331523
main
mrambacher 4 years ago committed by Facebook GitHub Bot
parent b288f0131b
commit e9befdebbf
  1. 37
      db/db_impl/db_impl.cc
  2. 6
      db/db_impl/db_impl_compaction_flush.cc
  3. 4
      db/db_impl/db_impl_files.cc
  4. 4
      db/db_impl/db_impl_open.cc
  5. 6
      db/error_handler.h
  6. 1
      db/version_set.cc
  7. 1
      db/write_batch.cc
  8. 2
      db/write_thread.h
  9. 15
      env/env_test.cc
  10. 12
      file/delete_scheduler.cc
  11. 6
      file/sst_file_manager_impl.cc
  12. 4
      include/rocksdb/env.h
  13. 15
      logging/auto_roll_logger.cc
  14. 2
      logging/posix_logger.h
  15. 3
      monitoring/stats_dump_scheduler.cc
  16. 1
      table/block_based/block.h
  17. 2
      table/block_based/block_based_table_builder.cc
  18. 2
      table/block_based/block_based_table_reader.h
  19. 2
      utilities/fault_injection_env.h
  20. 15
      utilities/fault_injection_fs.cc

@ -235,7 +235,8 @@ DBImpl::DBImpl(const DBOptions& options, const std::string& dbname,
// !batch_per_trx_ implies seq_per_batch_ because it is only unset for // !batch_per_trx_ implies seq_per_batch_ because it is only unset for
// WriteUnprepared, which should use seq_per_batch_. // WriteUnprepared, which should use seq_per_batch_.
assert(batch_per_txn_ || seq_per_batch_); assert(batch_per_txn_ || seq_per_batch_);
env_->GetAbsolutePath(dbname, &db_absolute_path_); // TODO: Check for an error here
env_->GetAbsolutePath(dbname, &db_absolute_path_).PermitUncheckedError();
// Reserve ten files or so for other uses and give the rest to TableCache. // Reserve ten files or so for other uses and give the rest to TableCache.
// Give a large number for setting of "infinite" open files. // Give a large number for setting of "infinite" open files.
@ -501,7 +502,7 @@ Status DBImpl::CloseHelper() {
env_->UnSchedule(this, Env::Priority::BOTTOM); env_->UnSchedule(this, Env::Priority::BOTTOM);
int compactions_unscheduled = env_->UnSchedule(this, Env::Priority::LOW); int compactions_unscheduled = env_->UnSchedule(this, Env::Priority::LOW);
int flushes_unscheduled = env_->UnSchedule(this, Env::Priority::HIGH); int flushes_unscheduled = env_->UnSchedule(this, Env::Priority::HIGH);
Status ret; Status ret = Status::OK();
mutex_.Lock(); mutex_.Lock();
bg_bottom_compaction_scheduled_ -= bottom_compactions_unscheduled; bg_bottom_compaction_scheduled_ -= bottom_compactions_unscheduled;
bg_compaction_scheduled_ -= compactions_unscheduled; bg_compaction_scheduled_ -= compactions_unscheduled;
@ -613,7 +614,8 @@ Status DBImpl::CloseHelper() {
versions_.reset(); versions_.reset();
mutex_.Unlock(); mutex_.Unlock();
if (db_lock_ != nullptr) { if (db_lock_ != nullptr) {
env_->UnlockFile(db_lock_); // TODO: Check for unlock error
env_->UnlockFile(db_lock_).PermitUncheckedError();
} }
ROCKS_LOG_INFO(immutable_db_options_.info_log, "Shutdown complete"); ROCKS_LOG_INFO(immutable_db_options_.info_log, "Shutdown complete");
@ -632,7 +634,7 @@ Status DBImpl::CloseHelper() {
if (immutable_db_options_.info_log && own_info_log_) { if (immutable_db_options_.info_log && own_info_log_) {
Status s = immutable_db_options_.info_log->Close(); Status s = immutable_db_options_.info_log->Close();
if (ret.ok()) { if (!s.ok() && ret.ok()) {
ret = s; ret = s;
} }
} }
@ -651,7 +653,7 @@ Status DBImpl::CloseImpl() { return CloseHelper(); }
DBImpl::~DBImpl() { DBImpl::~DBImpl() {
if (!closed_) { if (!closed_) {
closed_ = true; closed_ = true;
CloseHelper(); CloseHelper().PermitUncheckedError();
} }
} }
@ -3769,7 +3771,7 @@ Status DestroyDB(const std::string& dbname, const Options& options,
// log file and prevents cleanup and directory removal // log file and prevents cleanup and directory removal
soptions.info_log.reset(); soptions.info_log.reset();
// Ignore error in case directory does not exist // Ignore error in case directory does not exist
env->GetChildren(dbname, &filenames); env->GetChildren(dbname, &filenames).PermitUncheckedError();
FileLock* lock; FileLock* lock;
const std::string lockname = LockFileName(dbname); const std::string lockname = LockFileName(dbname);
@ -3791,7 +3793,7 @@ Status DestroyDB(const std::string& dbname, const Options& options,
} else { } else {
del = env->DeleteFile(path_to_delete); del = env->DeleteFile(path_to_delete);
} }
if (result.ok() && !del.ok()) { if (!del.ok() && result.ok()) {
result = del; result = del;
} }
} }
@ -3814,7 +3816,7 @@ Status DestroyDB(const std::string& dbname, const Options& options,
std::string table_path = path + "/" + fname; std::string table_path = path + "/" + fname;
Status del = DeleteDBFile(&soptions, table_path, dbname, Status del = DeleteDBFile(&soptions, table_path, dbname,
/*force_bg=*/false, /*force_fg=*/false); /*force_bg=*/false, /*force_fg=*/false);
if (result.ok() && !del.ok()) { if (!del.ok() && result.ok()) {
result = del; result = del;
} }
} }
@ -3842,12 +3844,13 @@ Status DestroyDB(const std::string& dbname, const Options& options,
Status del = Status del =
DeleteDBFile(&soptions, archivedir + "/" + file, archivedir, DeleteDBFile(&soptions, archivedir + "/" + file, archivedir,
/*force_bg=*/false, /*force_fg=*/!wal_in_db_path); /*force_bg=*/false, /*force_fg=*/!wal_in_db_path);
if (result.ok() && !del.ok()) { if (!del.ok() && result.ok()) {
result = del; result = del;
} }
} }
} }
env->DeleteDir(archivedir); // TODO: Should we check for errors here?
env->DeleteDir(archivedir).PermitUncheckedError();
} }
// Delete log files in the WAL dir // Delete log files in the WAL dir
@ -3858,7 +3861,7 @@ Status DestroyDB(const std::string& dbname, const Options& options,
DeleteDBFile(&soptions, LogFileName(soptions.wal_dir, number), DeleteDBFile(&soptions, LogFileName(soptions.wal_dir, number),
soptions.wal_dir, /*force_bg=*/false, soptions.wal_dir, /*force_bg=*/false,
/*force_fg=*/!wal_in_db_path); /*force_fg=*/!wal_in_db_path);
if (result.ok() && !del.ok()) { if (!del.ok() && result.ok()) {
result = del; result = del;
} }
} }
@ -3866,14 +3869,17 @@ Status DestroyDB(const std::string& dbname, const Options& options,
env->DeleteDir(soptions.wal_dir); env->DeleteDir(soptions.wal_dir);
} }
env->UnlockFile(lock); // Ignore error since state is already gone // Ignore error since state is already gone
env->DeleteFile(lockname); env->UnlockFile(lock).PermitUncheckedError();
env->DeleteFile(lockname).PermitUncheckedError();
// sst_file_manager holds a ref to the logger. Make sure the logger is // sst_file_manager holds a ref to the logger. Make sure the logger is
// gone before trying to remove the directory. // gone before trying to remove the directory.
soptions.sst_file_manager.reset(); soptions.sst_file_manager.reset();
env->DeleteDir(dbname); // Ignore error in case dir contains other files // Ignore error in case dir contains other files
env->DeleteDir(dbname).PermitUncheckedError();
;
} }
return result; return result;
} }
@ -4008,7 +4014,8 @@ Status DBImpl::RenameTempFileToOptionsFile(const std::string& file_name) {
} }
if (0 == disable_delete_obsolete_files_) { if (0 == disable_delete_obsolete_files_) {
DeleteObsoleteOptionsFiles(); // TODO: Should we check for errors here?
DeleteObsoleteOptionsFiles().PermitUncheckedError();
} }
return s; return s;
#else #else

@ -167,7 +167,7 @@ Status DBImpl::FlushMemTableToOutputFile(
#endif // ROCKSDB_LITE #endif // ROCKSDB_LITE
Status s; Status s;
IOStatus io_s; IOStatus io_s = IOStatus::OK();
if (logfile_number_ > 0 && if (logfile_number_ > 0 &&
versions_->GetColumnFamilySet()->NumberOfColumnFamilies() > 1) { versions_->GetColumnFamilySet()->NumberOfColumnFamilies() > 1) {
// If there are more than one column families, we need to make sure that // If there are more than one column families, we need to make sure that
@ -225,6 +225,10 @@ Status DBImpl::FlushMemTableToOutputFile(
Status new_bg_error = s; Status new_bg_error = s;
error_handler_.SetBGError(new_bg_error, BackgroundErrorReason::kFlush); error_handler_.SetBGError(new_bg_error, BackgroundErrorReason::kFlush);
} }
} else {
// If we got here, then we decided not to care about the i_os status (either
// from never needing it or ignoring the flush job status
io_s.PermitUncheckedError();
} }
if (s.ok()) { if (s.ok()) {
#ifndef ROCKSDB_LITE #ifndef ROCKSDB_LITE

@ -447,7 +447,8 @@ void DBImpl::PurgeObsoleteFiles(JobContext& state, bool schedule_only) {
// Close WALs before trying to delete them. // Close WALs before trying to delete them.
for (const auto w : state.logs_to_free) { for (const auto w : state.logs_to_free) {
// TODO: maybe check the return value of Close. // TODO: maybe check the return value of Close.
w->Close(); auto s = w->Close();
s.PermitUncheckedError();
} }
bool own_files = OwnTablesAndLogs(); bool own_files = OwnTablesAndLogs();
@ -566,7 +567,6 @@ void DBImpl::PurgeObsoleteFiles(JobContext& state, bool schedule_only) {
if (!own_files) { if (!own_files) {
continue; continue;
} }
Status file_deletion_status;
if (schedule_only) { if (schedule_only) {
InstrumentedMutexLock guard_lock(&mutex_); InstrumentedMutexLock guard_lock(&mutex_);
SchedulePendingPurge(fname, dir_to_sync, type, number, state.job_id); SchedulePendingPurge(fname, dir_to_sync, type, number, state.job_id);

@ -1684,7 +1684,9 @@ Status DBImpl::Open(const DBOptions& db_options, const std::string& dbname,
paths.erase(std::unique(paths.begin(), paths.end()), paths.end()); paths.erase(std::unique(paths.begin(), paths.end()), paths.end());
for (auto& path : paths) { for (auto& path : paths) {
std::vector<std::string> existing_files; std::vector<std::string> existing_files;
impl->immutable_db_options_.env->GetChildren(path, &existing_files); // TODO: Check for errors here?
impl->immutable_db_options_.env->GetChildren(path, &existing_files)
.PermitUncheckedError();
for (auto& file_name : existing_files) { for (auto& file_name : existing_files) {
uint64_t file_number; uint64_t file_number;
FileType file_type; FileType file_type;

@ -29,7 +29,11 @@ class ErrorHandler {
db_mutex_(db_mutex), db_mutex_(db_mutex),
auto_recovery_(false), auto_recovery_(false),
recovery_in_prog_(false) {} recovery_in_prog_(false) {}
~ErrorHandler() {} ~ErrorHandler() {
bg_error_.PermitUncheckedError();
recovery_error_.PermitUncheckedError();
recovery_io_error_.PermitUncheckedError();
}
void EnableAutoRecovery() { auto_recovery_ = true; } void EnableAutoRecovery() { auto_recovery_ = true; }

@ -3563,6 +3563,7 @@ struct VersionSet::ManifestWriter {
cfd(_cfd), cfd(_cfd),
mutable_cf_options(cf_options), mutable_cf_options(cf_options),
edit_list(e) {} edit_list(e) {}
~ManifestWriter() { status.PermitUncheckedError(); }
}; };
Status AtomicGroupReadBuffer::AddEdit(VersionEdit* edit) { Status AtomicGroupReadBuffer::AddEdit(VersionEdit* edit) {

@ -1428,6 +1428,7 @@ class MemTableInserter : public WriteBatch::Handler {
MaybeAdvanceSeq(batch_boundry); MaybeAdvanceSeq(batch_boundry);
return seek_status; return seek_status;
} }
seek_status.PermitUncheckedError(); // Ignore errors
Status ret_status; Status ret_status;
MemTable* mem = cf_mems_->GetMemTable(); MemTable* mem = cf_mems_->GetMemTable();

@ -179,6 +179,8 @@ class WriteThread {
StateMutex().~mutex(); StateMutex().~mutex();
StateCV().~condition_variable(); StateCV().~condition_variable();
} }
status.PermitUncheckedError();
callback_status.PermitUncheckedError();
} }
bool CheckCallback(DB* db) { bool CheckCallback(DB* db) {

15
env/env_test.cc vendored

@ -2111,8 +2111,6 @@ class EnvFSTestWithParam
std::string dbname2_; std::string dbname2_;
}; };
#ifndef ROCKSDB_ASSERT_STATUS_CHECKED // Database tests do not do well with
// this flag
TEST_P(EnvFSTestWithParam, OptionsTest) { TEST_P(EnvFSTestWithParam, OptionsTest) {
Options opts; Options opts;
opts.env = env_; opts.env = env_;
@ -2132,11 +2130,11 @@ TEST_P(EnvFSTestWithParam, OptionsTest) {
ASSERT_OK(s); ASSERT_OK(s);
WriteOptions wo; WriteOptions wo;
db->Put(wo, "a", "a"); ASSERT_OK(db->Put(wo, "a", "a"));
db->Flush(FlushOptions()); ASSERT_OK(db->Flush(FlushOptions()));
db->Put(wo, "b", "b"); ASSERT_OK(db->Put(wo, "b", "b"));
db->Flush(FlushOptions()); ASSERT_OK(db->Flush(FlushOptions()));
db->CompactRange(CompactRangeOptions(), nullptr, nullptr); ASSERT_OK(db->CompactRange(CompactRangeOptions(), nullptr, nullptr));
std::string val; std::string val;
ASSERT_OK(db->Get(ReadOptions(), "a", &val)); ASSERT_OK(db->Get(ReadOptions(), "a", &val));
@ -2144,14 +2142,13 @@ TEST_P(EnvFSTestWithParam, OptionsTest) {
ASSERT_OK(db->Get(ReadOptions(), "b", &val)); ASSERT_OK(db->Get(ReadOptions(), "b", &val));
ASSERT_EQ("b", val); ASSERT_EQ("b", val);
db->Close(); ASSERT_OK(db->Close());
delete db; delete db;
DestroyDB(dbname, opts); DestroyDB(dbname, opts);
dbname = dbname2_; dbname = dbname2_;
} }
} }
#endif // ROCKSDB_ASSERT_STATUS_CHECKED
// The parameters are as follows - // The parameters are as follows -
// 1. True means Options::env is non-null, false means null // 1. True means Options::env is non-null, false means null

@ -65,7 +65,7 @@ Status DeleteScheduler::DeleteFile(const std::string& file_path,
TEST_SYNC_POINT("DeleteScheduler::DeleteFile"); TEST_SYNC_POINT("DeleteScheduler::DeleteFile");
s = fs_->DeleteFile(file_path, IOOptions(), nullptr); s = fs_->DeleteFile(file_path, IOOptions(), nullptr);
if (s.ok()) { if (s.ok()) {
sst_file_manager_->OnDeleteFile(file_path); s = sst_file_manager_->OnDeleteFile(file_path);
ROCKS_LOG_INFO(info_log_, ROCKS_LOG_INFO(info_log_,
"Deleted file %s immediately, rate_bytes_per_sec %" PRIi64 "Deleted file %s immediately, rate_bytes_per_sec %" PRIi64
", total_trash_size %" PRIu64 " max_trash_db_ratio %lf", ", total_trash_size %" PRIu64 " max_trash_db_ratio %lf",
@ -88,7 +88,7 @@ Status DeleteScheduler::DeleteFile(const std::string& file_path,
file_path.c_str(), s.ToString().c_str()); file_path.c_str(), s.ToString().c_str());
s = fs_->DeleteFile(file_path, IOOptions(), nullptr); s = fs_->DeleteFile(file_path, IOOptions(), nullptr);
if (s.ok()) { if (s.ok()) {
sst_file_manager_->OnDeleteFile(file_path); s = sst_file_manager_->OnDeleteFile(file_path);
ROCKS_LOG_INFO(info_log_, "Deleted file %s immediately", ROCKS_LOG_INFO(info_log_, "Deleted file %s immediately",
trash_file.c_str()); trash_file.c_str());
InstrumentedMutexLock l(&mu_); InstrumentedMutexLock l(&mu_);
@ -146,7 +146,7 @@ Status DeleteScheduler::CleanupDirectory(Env* env, SstFileManagerImpl* sfm,
std::string trash_file = path + "/" + current_file; std::string trash_file = path + "/" + current_file;
if (sfm) { if (sfm) {
// We have an SstFileManager that will schedule the file delete // We have an SstFileManager that will schedule the file delete
sfm->OnAddFile(trash_file); s = sfm->OnAddFile(trash_file);
file_delete = sfm->ScheduleFileDeletion(trash_file, path); file_delete = sfm->ScheduleFileDeletion(trash_file, path);
} else { } else {
// Delete the file immediately // Delete the file immediately
@ -354,8 +354,10 @@ Status DeleteScheduler::DeleteTrashFile(const std::string& path_in_trash,
reinterpret_cast<void*>(const_cast<std::string*>(&dir_to_sync))); reinterpret_cast<void*>(const_cast<std::string*>(&dir_to_sync)));
} }
} }
*deleted_bytes = file_size; if (s.ok()) {
sst_file_manager_->OnDeleteFile(path_in_trash); *deleted_bytes = file_size;
s = sst_file_manager_->OnDeleteFile(path_in_trash);
}
} }
} }
if (!s.ok()) { if (!s.ok()) {

@ -43,6 +43,7 @@ SstFileManagerImpl::SstFileManagerImpl(Env* env, std::shared_ptr<FileSystem> fs,
SstFileManagerImpl::~SstFileManagerImpl() { SstFileManagerImpl::~SstFileManagerImpl() {
Close(); Close();
bg_err_.PermitUncheckedError();
} }
void SstFileManagerImpl::Close() { void SstFileManagerImpl::Close() {
@ -183,12 +184,13 @@ bool SstFileManagerImpl::EnoughRoomForCompaction(
// seen a NoSpace() error. This is tin order to contain a single potentially // seen a NoSpace() error. This is tin order to contain a single potentially
// misbehaving DB instance and prevent it from slowing down compactions of // misbehaving DB instance and prevent it from slowing down compactions of
// other DB instances // other DB instances
if (CheckFreeSpace() && bg_error == Status::NoSpace()) { if (bg_error == Status::NoSpace() && CheckFreeSpace()) {
auto fn = auto fn =
TableFileName(cfd->ioptions()->cf_paths, inputs[0][0]->fd.GetNumber(), TableFileName(cfd->ioptions()->cf_paths, inputs[0][0]->fd.GetNumber(),
inputs[0][0]->fd.GetPathId()); inputs[0][0]->fd.GetPathId());
uint64_t free_space = 0; uint64_t free_space = 0;
fs_->GetFreeSpace(fn, IOOptions(), &free_space, nullptr); Status s = fs_->GetFreeSpace(fn, IOOptions(), &free_space, nullptr);
s.PermitUncheckedError(); // TODO: Check the status
// needed_headroom is based on current size reserved by compactions, // needed_headroom is based on current size reserved by compactions,
// minus any files created by running compactions as they would count // minus any files created by running compactions as they would count
// against the reserved size. If user didn't specify any compaction // against the reserved size. If user didn't specify any compaction

@ -901,8 +901,10 @@ class WritableFile {
if (new_last_preallocated_block > last_preallocated_block_) { if (new_last_preallocated_block > last_preallocated_block_) {
size_t num_spanned_blocks = size_t num_spanned_blocks =
new_last_preallocated_block - last_preallocated_block_; new_last_preallocated_block - last_preallocated_block_;
// TODO: Don't ignore errors from allocate
Allocate(block_size * last_preallocated_block_, Allocate(block_size * last_preallocated_block_,
block_size * num_spanned_blocks); block_size * num_spanned_blocks)
.PermitUncheckedError();
last_preallocated_block_ = new_last_preallocated_block; last_preallocated_block_ = new_last_preallocated_block;
} }
} }

@ -257,11 +257,15 @@ Status CreateLoggerFromOptions(const std::string& dbname,
Env* env = options.env; Env* env = options.env;
std::string db_absolute_path; std::string db_absolute_path;
env->GetAbsolutePath(dbname, &db_absolute_path); Status s = env->GetAbsolutePath(dbname, &db_absolute_path);
if (!s.ok()) {
return s;
}
std::string fname = std::string fname =
InfoLogFileName(dbname, db_absolute_path, options.db_log_dir); InfoLogFileName(dbname, db_absolute_path, options.db_log_dir);
env->CreateDirIfMissing(dbname); // In case it does not exist env->CreateDirIfMissing(dbname)
.PermitUncheckedError(); // In case it does not exist
// Currently we only support roll by time-to-roll and log size // Currently we only support roll by time-to-roll and log size
#ifndef ROCKSDB_LITE #ifndef ROCKSDB_LITE
if (options.log_file_time_to_roll > 0 || options.max_log_file_size > 0) { if (options.log_file_time_to_roll > 0 || options.max_log_file_size > 0) {
@ -269,7 +273,7 @@ Status CreateLoggerFromOptions(const std::string& dbname,
env, dbname, options.db_log_dir, options.max_log_file_size, env, dbname, options.db_log_dir, options.max_log_file_size,
options.log_file_time_to_roll, options.keep_log_file_num, options.log_file_time_to_roll, options.keep_log_file_num,
options.info_log_level); options.info_log_level);
Status s = result->GetStatus(); s = result->GetStatus();
if (!s.ok()) { if (!s.ok()) {
delete result; delete result;
} else { } else {
@ -281,8 +285,9 @@ Status CreateLoggerFromOptions(const std::string& dbname,
// Open a log file in the same directory as the db // Open a log file in the same directory as the db
env->RenameFile(fname, env->RenameFile(fname,
OldInfoLogFileName(dbname, env->NowMicros(), db_absolute_path, OldInfoLogFileName(dbname, env->NowMicros(), db_absolute_path,
options.db_log_dir)); options.db_log_dir))
auto s = env->NewLogger(fname, logger); .PermitUncheckedError();
s = env->NewLogger(fname, logger);
if (logger->get() != nullptr) { if (logger->get() != nullptr) {
(*logger)->SetInfoLogLevel(options.info_log_level); (*logger)->SetInfoLogLevel(options.info_log_level);
} }

@ -68,7 +68,7 @@ class PosixLogger : public Logger {
virtual ~PosixLogger() { virtual ~PosixLogger() {
if (!closed_) { if (!closed_) {
closed_ = true; closed_ = true;
PosixCloseHelper(); PosixCloseHelper().PermitUncheckedError();
} }
} }
virtual void Flush() override { virtual void Flush() override {

@ -56,7 +56,8 @@ StatsDumpScheduler* StatsDumpScheduler::Default() {
std::string StatsDumpScheduler::GetTaskName(DBImpl* dbi, std::string StatsDumpScheduler::GetTaskName(DBImpl* dbi,
const std::string& func_name) { const std::string& func_name) {
std::string db_session_id; std::string db_session_id;
dbi->GetDbSessionId(db_session_id); // TODO: Should this error be ignored?
dbi->GetDbSessionId(db_session_id).PermitUncheckedError();
return db_session_id + ":" + func_name; return db_session_id + ":" + func_name;
} }

@ -332,6 +332,7 @@ class BlockIter : public InternalIteratorBase<TValue> {
// Assert that the BlockIter is never deleted while Pinning is Enabled. // Assert that the BlockIter is never deleted while Pinning is Enabled.
assert(!pinned_iters_mgr_ || assert(!pinned_iters_mgr_ ||
(pinned_iters_mgr_ && !pinned_iters_mgr_->PinningEnabled())); (pinned_iters_mgr_ && !pinned_iters_mgr_->PinningEnabled()));
status_.PermitUncheckedError();
} }
void SetPinnedItersMgr(PinnedIteratorsManager* pinned_iters_mgr) override { void SetPinnedItersMgr(PinnedIteratorsManager* pinned_iters_mgr) override {
pinned_iters_mgr_ = pinned_iters_mgr; pinned_iters_mgr_ = pinned_iters_mgr;

@ -1729,7 +1729,7 @@ TableProperties BlockBasedTableBuilder::GetTableProperties() const {
for (const auto& prop : collector->GetReadableProperties()) { for (const auto& prop : collector->GetReadableProperties()) {
ret.readable_properties.insert(prop); ret.readable_properties.insert(prop);
} }
collector->Finish(&ret.user_collected_properties); collector->Finish(&ret.user_collected_properties).PermitUncheckedError();
} }
return ret; return ret;
} }

@ -520,7 +520,7 @@ struct BlockBasedTable::Rep {
file_size(_file_size), file_size(_file_size),
level(_level), level(_level),
immortal_table(_immortal_table) {} immortal_table(_immortal_table) {}
~Rep() { status.PermitUncheckedError(); }
const ImmutableCFOptions& ioptions; const ImmutableCFOptions& ioptions;
const EnvOptions& env_options; const EnvOptions& env_options;
const BlockBasedTableOptions table_options; const BlockBasedTableOptions table_options;

@ -122,7 +122,7 @@ class FaultInjectionTestEnv : public EnvWrapper {
public: public:
explicit FaultInjectionTestEnv(Env* base) explicit FaultInjectionTestEnv(Env* base)
: EnvWrapper(base), filesystem_active_(true) {} : EnvWrapper(base), filesystem_active_(true) {}
virtual ~FaultInjectionTestEnv() {} virtual ~FaultInjectionTestEnv() { error_.PermitUncheckedError(); }
Status NewDirectory(const std::string& name, Status NewDirectory(const std::string& name,
std::unique_ptr<Directory>* result) override; std::unique_ptr<Directory>* result) override;

@ -86,7 +86,7 @@ TestFSWritableFile::TestFSWritableFile(const std::string& fname,
TestFSWritableFile::~TestFSWritableFile() { TestFSWritableFile::~TestFSWritableFile() {
if (writable_file_opened_) { if (writable_file_opened_) {
Close(IOOptions(), nullptr); Close(IOOptions(), nullptr).PermitUncheckedError();
} }
} }
@ -112,7 +112,8 @@ IOStatus TestFSWritableFile::Close(const IOOptions& options,
io_s = target_->Append(state_.buffer_, options, dbg); io_s = target_->Append(state_.buffer_, options, dbg);
if (io_s.ok()) { if (io_s.ok()) {
state_.buffer_.resize(0); state_.buffer_.resize(0);
target_->Sync(options, dbg); // Ignore sync errors
target_->Sync(options, dbg).PermitUncheckedError();
io_s = target_->Close(options, dbg); io_s = target_->Close(options, dbg);
} }
if (io_s.ok()) { if (io_s.ok()) {
@ -125,11 +126,10 @@ IOStatus TestFSWritableFile::Flush(const IOOptions&, IODebugContext*) {
if (!fs_->IsFilesystemActive()) { if (!fs_->IsFilesystemActive()) {
return fs_->GetError(); return fs_->GetError();
} }
IOStatus io_s; if (fs_->IsFilesystemActive()) {
if (io_s.ok() && fs_->IsFilesystemActive()) {
state_.pos_at_last_flush_ = state_.pos_; state_.pos_at_last_flush_ = state_.pos_;
} }
return io_s; return IOStatus::OK();
} }
IOStatus TestFSWritableFile::Sync(const IOOptions& options, IOStatus TestFSWritableFile::Sync(const IOOptions& options,
@ -139,7 +139,8 @@ IOStatus TestFSWritableFile::Sync(const IOOptions& options,
} }
IOStatus io_s = target_->Append(state_.buffer_, options, dbg); IOStatus io_s = target_->Append(state_.buffer_, options, dbg);
state_.buffer_.resize(0); state_.buffer_.resize(0);
target_->Sync(options, dbg); // Ignore sync errors
target_->Sync(options, dbg).PermitUncheckedError();
state_.pos_at_last_sync_ = state_.pos_; state_.pos_at_last_sync_ = state_.pos_;
fs_->WritableFileSynced(state_); fs_->WritableFileSynced(state_);
return io_s; return io_s;
@ -154,7 +155,7 @@ TestFSRandomRWFile::TestFSRandomRWFile(const std::string& /*fname*/,
TestFSRandomRWFile::~TestFSRandomRWFile() { TestFSRandomRWFile::~TestFSRandomRWFile() {
if (file_opened_) { if (file_opened_) {
Close(IOOptions(), nullptr); Close(IOOptions(), nullptr).PermitUncheckedError();
} }
} }

Loading…
Cancel
Save