mirror of https://github.com/facebook/rocksdb.git
db_impl: recycle log files
If log recycling is enabled, put old WAL files on a recycle queue instead of deleting them. When we need a new log file, take a recycled file off the list if one is available. Signed-off-by: Sage Weil <sage@redhat.com>
This commit is contained in:
parent
d666225a0a
commit
666376150c
|
@ -605,7 +605,13 @@ void DBImpl::FindObsoleteFiles(JobContext* job_context, bool force,
|
|||
// find newly obsoleted log files
|
||||
while (alive_log_files_.begin()->number < min_log_number) {
|
||||
auto& earliest = *alive_log_files_.begin();
|
||||
job_context->log_delete_files.push_back(earliest.number);
|
||||
if (db_options_.recycle_log_file_num > log_recycle_files.size()) {
|
||||
Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
|
||||
"adding log %" PRIu64 " to recycle list\n", earliest.number);
|
||||
log_recycle_files.push_back(earliest.number);
|
||||
} else {
|
||||
job_context->log_delete_files.push_back(earliest.number);
|
||||
}
|
||||
total_log_size_ -= earliest.size;
|
||||
alive_log_files_.pop_front();
|
||||
// Current log should always stay alive since it can't have
|
||||
|
@ -4076,6 +4082,12 @@ Status DBImpl::SwitchMemtable(ColumnFamilyData* cfd, WriteContext* context) {
|
|||
// Do this without holding the dbmutex lock.
|
||||
assert(versions_->prev_log_number() == 0);
|
||||
bool creating_new_log = !log_empty_;
|
||||
uint64_t recycle_log_number = 0;
|
||||
if (creating_new_log && db_options_.recycle_log_file_num &&
|
||||
!log_recycle_files.empty()) {
|
||||
recycle_log_number = log_recycle_files.front();
|
||||
log_recycle_files.pop_front();
|
||||
}
|
||||
uint64_t new_log_number =
|
||||
creating_new_log ? versions_->NewFileNumber() : logfile_number_;
|
||||
SuperVersion* new_superversion = nullptr;
|
||||
|
@ -4086,14 +4098,23 @@ Status DBImpl::SwitchMemtable(ColumnFamilyData* cfd, WriteContext* context) {
|
|||
if (creating_new_log) {
|
||||
EnvOptions opt_env_opt =
|
||||
env_->OptimizeForLogWrite(env_options_, db_options_);
|
||||
s = NewWritableFile(env_,
|
||||
LogFileName(db_options_.wal_dir, new_log_number),
|
||||
&lfile, opt_env_opt);
|
||||
if (recycle_log_number) {
|
||||
Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
|
||||
"reusing log %" PRIu64 " from recycle list\n", recycle_log_number);
|
||||
s = env_->ReuseWritableFile(
|
||||
LogFileName(db_options_.wal_dir, new_log_number),
|
||||
LogFileName(db_options_.wal_dir, recycle_log_number), &lfile,
|
||||
opt_env_opt);
|
||||
} else {
|
||||
s = NewWritableFile(env_,
|
||||
LogFileName(db_options_.wal_dir, new_log_number),
|
||||
&lfile, opt_env_opt);
|
||||
}
|
||||
if (s.ok()) {
|
||||
// Our final size should be less than write_buffer_size
|
||||
// (compression, etc) but err on the side of caution.
|
||||
lfile->SetPreallocationBlockSize(
|
||||
1.1 * mutable_cf_options.write_buffer_size);
|
||||
lfile->SetPreallocationBlockSize(1.1 *
|
||||
mutable_cf_options.write_buffer_size);
|
||||
unique_ptr<WritableFileWriter> file_writer(
|
||||
new WritableFileWriter(std::move(lfile), opt_env_opt));
|
||||
new_log = new log::Writer(std::move(file_writer));
|
||||
|
|
|
@ -556,6 +556,8 @@ class DBImpl : public DB {
|
|||
// * whenever there is an error in background flush or compaction
|
||||
InstrumentedCondVar bg_cv_;
|
||||
uint64_t logfile_number_;
|
||||
std::deque<uint64_t>
|
||||
log_recycle_files; // a list of log files that we can recycle
|
||||
bool log_dir_synced_;
|
||||
bool log_empty_;
|
||||
ColumnFamilyHandleImpl* default_cf_handle_;
|
||||
|
|
Loading…
Reference in New Issue