Compare commits

...

22 Commits

Author SHA1 Message Date
sdong
7705123545 Reduce tail prefetching from 512KB to 4KB
Summary: https://github.com/facebook/rocksdb/pull/4159 has a merging conflict so it can't be easily merged. To minimize the risk of merging, we just simply change 512KB to 4KB. It should be OK for MyRocks.

Test Plan: Make all check
2018-07-20 16:03:31 -07:00
Siying Dong
b9637562cf Change default value of bytes_max_delete_chunk to 0 in NewSstFileManager() (#4092)
Summary:
Now by default, with NewSstFileManager, checkpoints may be corrupted. Disable this feature to avoid this issue.
Closes https://github.com/facebook/rocksdb/pull/4092

Differential Revision: D8729856

Pulled By: siying

fbshipit-source-id: 914c321d6eaf52d8c5981171322d85dd29088307
2018-07-03 20:00:34 -07:00
sdong
dc4ac5ffa6 Fix regression bug of Prev() with upper bound (#3989)
Summary:
A recent change pushed down the upper bound checking to child iterators. However, this causes the logic of following sequence wrong:
  Seek(key);
  if (!Valid()) SeekToLast();
Because !Valid() may be caused by upper bounds, rather than the end of the iterator. In this case SeekToLast() points to totally wrong places. This can cause wrong results, infinite loops, or segfault in some cases.
This sequence is called when changing direction from forward to backward. And this by itself also implicitly happen during reseeking optimization in Prev().

Fix this bug by using SeekForPrev() rather than this sequence, as what is already done in prefix extrator case.
Closes https://github.com/facebook/rocksdb/pull/3989

Differential Revision: D8385422

Pulled By: siying

fbshipit-source-id: 429e869990cfd2dc389421e0836fc496bed67bb4
2018-06-12 16:59:46 -07:00
Siying Dong
c7e015271f Sync parent directory after deleting a file in delete scheduler
Summary:
sync parent directory after deleting a file in delete scheduler. Otherwise, trim speed may not be as smooth as what we want.
Closes https://github.com/facebook/rocksdb/pull/3767

Differential Revision: D7760136

Pulled By: siying

fbshipit-source-id: ec131d53b61953f09c60d67e901e5eeb2716b05f
2018-05-08 17:01:51 -07:00
Phani Shekhar Mantripragada
88ca4197a8 Support for Column family specific paths.
Summary:
In this change, an option to set different paths for different column families is added.
This option is set via cf_paths setting of ColumnFamilyOptions. This option will work in a similar fashion to db_paths setting. Cf_paths is a vector of Dbpath values which contains a pair of the absolute path and target size. Multiple levels in a Column family can go to different paths if cf_paths has more than one path.
To maintain backward compatibility, if cf_paths is not specified for a column family, db_paths setting will be used. Note that, if db_paths setting is also not specified, RocksDB already has code to use db_name as the only path.

Changes :
1) A new member "cf_paths" is added to ImmutableCfOptions. This is set, based on cf_paths setting of ColumnFamilyOptions and db_paths setting of ImmutableDbOptions.  This member is used to identify the path information whenever files are accessed.
2) Validation checks are added for cf_paths setting based on existing checks for db_paths setting.
3) DestroyDB, PurgeObsoleteFiles etc. are edited to support multiple cf_paths.
4) Unit tests are added appropriately.
Closes https://github.com/facebook/rocksdb/pull/3102

Differential Revision: D6951697

Pulled By: ajkr

fbshipit-source-id: 60d2262862b0a8fd6605b09ccb0da32bb331787d
2018-05-08 17:00:57 -07:00
Zhongyi Xie
192fdd79e8 avoid double delete on dummy record insertion failure
Summary:
When the dummy record insertion fails, there is no need to explicitly delete the block as it will be registered for cleanup regardless.
Closes https://github.com/facebook/rocksdb/pull/3688

Differential Revision: D7537741

Pulled By: miasantreble

fbshipit-source-id: fcd3a3d3d382ee8e2c7ced0a4980e683d93a16d6
2018-05-08 14:25:07 -07:00
Zhongyi Xie
43f2a27daf fix memory leak in two_level_iterator
Summary:
this PR fixes a few failed contbuild:
1. ASAN memory leak in Block::NewIterator (table/block.cc:429). the proper destruction of first_level_iter_ and second_level_iter_ of two_level_iterator.cc is missing from the code after the refactoring in https://github.com/facebook/rocksdb/pull/3406
2. various unused param errors introduced by https://github.com/facebook/rocksdb/pull/3662
3. updated comment for `ForceReleaseCachedEntry` to emphasize the use of `force_erase` flag.
Closes https://github.com/facebook/rocksdb/pull/3718

Reviewed By: maysamyabandeh

Differential Revision: D7621192

Pulled By: miasantreble

fbshipit-source-id: 476c94264083a0730ded957c29de7807e4f5b146
2018-05-08 14:24:34 -07:00
Maysam Yabandeh
dcb7f3bdae Fix the bloom filter skipping empty prefixes
Summary:
bc0da4b512 optimized bloom filters by skipping duplicate entires when the whole key and prefixes are both added to the bloom. It however used empty string as the initial value of the last entry added to the bloom. This is incorrect since empty key/prefix are valid entires by themselves. This patch fixes that.
Closes https://github.com/facebook/rocksdb/pull/3776

Differential Revision: D7778803

Pulled By: maysamyabandeh

fbshipit-source-id: d5a065daebee17f9403cac51e9d5626aac87bfbc
2018-05-08 14:14:52 -07:00
Maysam Yabandeh
5753c17212 Skip duplicate bloom keys when whole_key and prefix are mixed
Summary:
Currently we rely on FilterBitsBuilder to skip the duplicate keys. It does that by comparing that hash of the key to the hash of the last added entry. This logic breaks however when we have whole_key_filtering mixed with prefix blooms as their addition to FilterBitsBuilder will be interleaved. The patch fixes that by comparing the last whole key and last prefix with the whole key and prefix of the new key respectively and skip the call to FilterBitsBuilder if it is a duplicate.
Closes https://github.com/facebook/rocksdb/pull/3764

Differential Revision: D7744413

Pulled By: maysamyabandeh

fbshipit-source-id: 15df73bbbafdfd754d4e1f42ea07f47b03bc5eb8
2018-05-08 14:14:38 -07:00
Maysam Yabandeh
52aab66424 WritePrepared Txn: enable TryAgain for duplicates at the end of the batch
Summary:
The WriteBatch::Iterate will try with a larger sequence number if the memtable reports a duplicate. This status is specified with TryAgain status. So far the assumption was that the last entry in the batch will never return TryAgain, which is correct when WAL is created via WritePrepared since it always appends a batch separator if a natural one does not exist. However when reading a WAL generated by WriteCommitted this batch separator might  not exist. Although WritePrepared is not supposed to be able to read the WAL generated by WriteCommitted we should avoid confusing scenarios in which the behavior becomes unpredictable. The path fixes that by allowing TryAgain even for the last entry of the write batch.
Closes https://github.com/facebook/rocksdb/pull/3747

Differential Revision: D7708391

Pulled By: maysamyabandeh

fbshipit-source-id: bfaddaa9b14a4cdaff6977f6f63c789a6ab1ee0d
2018-05-08 14:13:12 -07:00
Siying Dong
91c087997e Skip deleted WALs during recovery
Summary:
This patch record min log number to keep to the manifest while flushing SST files to ignore them and any WAL older than them during recovery. This is to avoid scenarios when we have a gap between the WAL files are fed to the recovery procedure. The gap could happen by for example out-of-order WAL deletion. Such gap could cause problems in 2PC recovery where the prepared and commit entry are placed into two separate WAL and gap in the WALs could result into not processing the WAL with the commit entry and hence breaking the 2PC recovery logic.

Before the commit, for 2PC case, we determined which log number to keep in FindObsoleteFiles(). We looked at the earliest logs with outstanding prepare entries, or prepare entries whose respective commit or abort are in memtable. With the commit, the same calculation is done while we apply the SST flush. Just before installing the flush file, we precompute the earliest log file to keep after the flush finishes using the same logic (but skipping the memtables just flushed), record this information to the manifest entry for this new flushed SST file. This pre-computed value is also remembered in memory, and will later be used to determine whether a log file can be deleted. This value is unlikely to change until next flush because the commit entry will stay in memtable. (In WritePrepared, we could have removed the older log files as soon as all prepared entries are committed. It's not yet done anyway. Even if we do it, the only thing we loss with this new approach is earlier log deletion between two flushes, which does not guarantee to happen anyway because the obsolete file clean-up function is only executed after flush or compaction)

This min log number to keep is stored in the manifest using the safely-ignore customized field of AddFile entry, in order to guarantee that the DB generated using newer release can be opened by previous releases no older than 4.2.
Closes https://github.com/facebook/rocksdb/pull/3765

Differential Revision: D7747618

Pulled By: siying

fbshipit-source-id: d00c92105b4f83852e9754a1b70d6b64cb590729
2018-05-08 14:10:18 -07:00
Maysam Yabandeh
e049d5a12a Revert "Skip deleted WALs during recovery"
This reverts commit 140c308f6f.
2018-04-23 12:19:16 -07:00
Sagar Vemuri
544978d935 Ttl-triggered and snapshot-release-triggered compactions should not be manual compactions
Summary:
Ttl-triggered and snapshot-release-triggered compactions should not be considered as manual compactions. This is a bug.
Closes https://github.com/facebook/rocksdb/pull/3678

Differential Revision: D7498151

Pulled By: sagar0

fbshipit-source-id: a2d5bed05268a4dc93d54ea97a9ae44b366df15d
2018-04-13 04:52:03 -07:00
Sagar Vemuri
ce3b2912d2 Level Compaction with TTL
Summary:
Level Compaction with TTL.

As of today, a file could exist in the LSM tree without going through the compaction process for a really long time if there are no updates to the data in the file's key range. For example, in certain use cases, the keys are not actually "deleted"; instead they are just set to empty values. There might not be any more writes to this "deleted" key range, and if so, such data could remain in the LSM for a really long time resulting in wasted space.

Introducing a TTL could solve this problem. Files (and, in turn, data) older than TTL will be scheduled for compaction when there is no other background work. This will make the data go through the regular compaction process and get rid of old unwanted data.
This also has the (good) side-effect of all the data in the non-bottommost level being newer than ttl, and all data in the bottommost level older than ttl. It could lead to more writes while reducing space.

This functionality can be controlled by the newly introduced column family option -- ttl.

TODO for later:
- Make ttl mutable
- Extend TTL to Universal compaction as well? (TTL is already supported in FIFO)
- Maybe deprecate CompactionOptionsFIFO.ttl in favor of this new ttl option.
Closes https://github.com/facebook/rocksdb/pull/3591

Differential Revision: D7275442

Pulled By: sagar0

fbshipit-source-id: dcba484717341200d419b0953dafcdf9eb2f0267
2018-04-13 04:51:17 -07:00
Maysam Yabandeh
74fc31ec92 WritePrepared Txn: rollback_merge_operands hack
Summary:
This is a hack as temporary fix of MyRocks with rollbacking  the merge operands. The way MyRocks uses merge operands is without protection of locks, which violates the assumption behind the rollback algorithm. They are ok with not being rolled back as it would just create a gap in the autoincrement column. The hack add an option to disable the rollback of merge operands by default and only enables it to let the unit test pass.
Closes https://github.com/facebook/rocksdb/pull/3711

Differential Revision: D7597177

Pulled By: maysamyabandeh

fbshipit-source-id: 544be0f666c7e7abb7f651ec8b23124e05056728
2018-04-12 14:01:54 -07:00
Maysam Yabandeh
88fe4ef1f1 WritePrepared Txn: fix smallest_prep atomicity issue
Summary:
We introduced smallest_prep optimization in this commit b225de7e10, which enables storing the smallest uncommitted sequence number along with the snapshot. This enables the readers that read from the snapshot to skip further checks and safely assumed the data is committed if its sequence number is less than smallest uncommitted when the snapshot was taken. The problem was that smallest uncommitted and the snapshot must be taken atomically, and the lack of atomicity had led to readers using a smallest uncommitted after the snapshot was taken and hence mistakenly skipping some data.
This patch fixes the problem by i) separating the process of removing of prepare entries from the AddCommitted function, ii) removing the prepare entires AFTER the committed sequence number is published, iii) getting smallest uncommitted (from the prepare list) BEFORE taking a snapshot. This guarantees that the smallest uncommitted that is accompanied with a snapshot is less than or equal of such number if it was obtained atomically.

Tested by running MySQLStyleTransactionTest/MySQLStyleTransactionTest.TransactionStressTest that was failing sporadically.
Closes https://github.com/facebook/rocksdb/pull/3703

Differential Revision: D7581934

Pulled By: maysamyabandeh

fbshipit-source-id: dc9d6f4fb477eba75d4d5927326905b548a96a32
2018-04-12 14:00:25 -07:00
Maysam Yabandeh
0f537bf14c Fix the memory leak with pinned partitioned filters
Summary:
The existing unit test did not set the level so the check for pinned partitioned filter/index being properly released from the block cache was not properly exercised as they only take effect in level 0. As a result a memory leak in pinned partitioned filters was hidden. The patch fix the test as well as the bug.
Closes https://github.com/facebook/rocksdb/pull/3692

Differential Revision: D7559763

Pulled By: maysamyabandeh

fbshipit-source-id: 55eff274945838af983c764a7d71e8daff092e4a
2018-04-12 14:00:04 -07:00
Maysam Yabandeh
011fc8be2d WritePrepared Txn: add stats
Summary:
Adding some stats that would be helpful to monitor if the DB has gone to unlikely stats that would hurt the performance. These are mostly when we end up needing to acquire a mutex.
Closes https://github.com/facebook/rocksdb/pull/3683

Differential Revision: D7529393

Pulled By: maysamyabandeh

fbshipit-source-id: f7d36279a8f39bd84d8ddbf64b5c97f670c5d6d9
2018-04-12 13:56:07 -07:00
Maysam Yabandeh
59b4f25d2d WritePrepared Txn: add write_committed option to dump_wal
Summary:
Currently dump_wal cannot print the prepared records from the WAL that is generated by WRITE_PREPARED write policy since the default reaction of the handler is to return NotSupported if markers of WRITE_PREPARED are encountered. This patch enables the admin to pass --write_committed=false option, which will be accordingly passed to the handler. Note that DBFileDumperCommand and DBDumperCommand are still not updated by this patch but firstly they are not urgent and secondly we need to revise this approach later when we also add WRITE_UNPREPARED markers so I leave it for future work.

Tested by running it on a WAL generated by WRITE_PREPARED:
$ ./ldb dump_wal --walfile=/dev/shm/dbbench/000003.log  | grep BEGIN_PREARE | head -1
1,2,70,0,BEGIN_PREARE
$ ./ldb dump_wal --walfile=/dev/shm/dbbench/000003.log --write_committed=false | grep BEGIN_PREARE | head -1
1,2,70,0,BEGIN_PREARE PUT(0) : 0x30303031313330313938 PUT(0) : 0x30303032353732313935 END_PREPARE(0x74786E31313535383434323738303738363938313335312D30)
Closes https://github.com/facebook/rocksdb/pull/3682

Differential Revision: D7522090

Pulled By: maysamyabandeh

fbshipit-source-id: a0332207261c61e18b2f9dfbe9feecd9a1339aca
2018-04-12 13:55:55 -07:00
Maysam Yabandeh
5b66eee2cb Stats for false positive rate of full filtesr
Summary:
Adds two stats to allow us measuring the false positive rate of full filters:
- The total count of positives: rocksdb.bloom.filter.full.positive
- The total count of true positives: rocksdb.bloom.filter.full.true.positive
Not the term "full" in the stat name to indicate that they are meaningful in full filters. block-based filters are to be deprecated soon and supporting it is not worth the the additional cost of if-then-else branches.

Closes #3680

Tested by:
$ ./db_bench -benchmarks=fillrandom  -db /dev/shm/rocksdb-tmpdb --num=1000000 -bloom_bits=10
$ ./db_bench -benchmarks="readwhilewriting"  -db /dev/shm/rocksdb-tmpdb --statistics -bloom_bits=10 --duration=60 --num=2000000 --use_existing_db 2>&1 > /tmp/full.log
$ grep filter.full /tmp/full.log
rocksdb.bloom.filter.full.positive COUNT : 3628593
rocksdb.bloom.filter.full.true.positive COUNT : 3536026
which gives the false positive rate of 2.5%
Closes https://github.com/facebook/rocksdb/pull/3681

Differential Revision: D7517570

Pulled By: maysamyabandeh

fbshipit-source-id: 630ab1a473afdce404916d297035b6318de4c052
2018-04-12 13:55:45 -07:00
Maysam Yabandeh
f2827845f9 WritePrepared Txn: smallest_prepare optimization
Summary:
The is an optimization to reduce lookup in the CommitCache when querying IsInSnapshot. The optimization takes the smallest uncommitted data at the time that the snapshot was taken and if the sequence number of the read data is lower than that number it assumes the data as committed.
To implement this optimization two changes are required: i) The AddPrepared function must be called sequentially to avoid out of order insertion in the PrepareHeap (otherwise the top of the heap does not indicate the smallest prepare in future too), ii) non-2PC transactions also call AddPrepared if they do not commit in one step.
Closes https://github.com/facebook/rocksdb/pull/3649

Differential Revision: D7388630

Pulled By: maysamyabandeh

fbshipit-source-id: b79506238c17467d590763582960d4d90181c600
2018-04-12 13:55:36 -07:00
Maysam Yabandeh
140c308f6f Skip deleted WALs during recovery
Summary:
This patch record the deleted WAL numbers in the manifest to ignore them and any WAL older than them during recovery. This is to avoid scenarios when we have a gap between the WAL files are fed to the recovery procedure. The gap could happen by for example out-of-order WAL deletion. Such gap could cause problems in 2PC recovery where the prepared and commit entry are placed into two separate WAL and gap in the WALs could result into not processing the WAL with the commit entry and hence breaking the 2PC recovery logic.
Closes https://github.com/facebook/rocksdb/pull/3488

Differential Revision: D6967893

Pulled By: maysamyabandeh

fbshipit-source-id: 13119feb155a08ab6d4909f437c7a750480dc8a1
2018-04-12 13:55:12 -07:00
144 changed files with 2499 additions and 818 deletions

View File

@ -455,6 +455,7 @@ set(SOURCES
db/flush_scheduler.cc db/flush_scheduler.cc
db/forward_iterator.cc db/forward_iterator.cc
db/internal_stats.cc db/internal_stats.cc
db/logs_with_prep_tracker.cc
db/log_reader.cc db/log_reader.cc
db/log_writer.cc db/log_writer.cc
db/malloc_stats.cc db/malloc_stats.cc

View File

@ -4,10 +4,18 @@
* Add a BlockBasedTableOption to align uncompressed data blocks on the smaller of block size or page size boundary, to reduce flash reads by avoiding reads spanning 4K pages. * Add a BlockBasedTableOption to align uncompressed data blocks on the smaller of block size or page size boundary, to reduce flash reads by avoiding reads spanning 4K pages.
### New Features ### New Features
* TransactionDBOptions::write_policy can be configured to enable WritePrepared 2PC transactions. Read more about them in the wiki.
* Introduce TTL for level compaction so that all files older than ttl go through the compaction process to get rid of old data.
* Add DB properties "rocksdb.block-cache-capacity", "rocksdb.block-cache-usage", "rocksdb.block-cache-pinned-usage" to show block cache usage.
* Add `Env::LowerThreadPoolCPUPriority(Priority)` method, which lowers the CPU priority of background (esp. compaction) threads to minimize interference with foreground tasks.
* Fsync parent directory after deleting a file in delete scheduler.
* Change default value of `bytes_max_delete_chunk` to 0 in NewSstFileManager() as it doesn't work well with checkpoints.
### Bug Fixes ### Bug Fixes
* Fsync after writing global seq number to the ingestion file in ExternalSstFileIngestionJob. * Fsync after writing global seq number to the ingestion file in ExternalSstFileIngestionJob.
* Fix WAL corruption caused by race condition between user write thread and FlushWAL when two_write_queue is not set. * Fix WAL corruption caused by race condition between user write thread and FlushWAL when two_write_queue is not set.
* Fix memory leak when pin_l0_filter_and_index_blocks_in_cache is used with partitioned filters
* Disable rollback of merge operands in WritePrepared transactions to work around an issue in MyRocks. It can be enabled back by setting TransactionDBOptions::rollback_merge_operands to true.
### Java API Changes ### Java API Changes
* Add `BlockBasedTableConfig.setBlockCache` to allow sharing a block cache across DB instances. * Add `BlockBasedTableConfig.setBlockCache` to allow sharing a block cache across DB instances.

11
TARGETS
View File

@ -99,6 +99,7 @@ cpp_library(
"db/internal_stats.cc", "db/internal_stats.cc",
"db/log_reader.cc", "db/log_reader.cc",
"db/log_writer.cc", "db/log_writer.cc",
"db/logs_with_prep_tracker.cc",
"db/malloc_stats.cc", "db/malloc_stats.cc",
"db/managed_iterator.cc", "db/managed_iterator.cc",
"db/memtable.cc", "db/memtable.cc",
@ -655,11 +656,6 @@ ROCKS_TESTS = [
"utilities/document/document_db_test.cc", "utilities/document/document_db_test.cc",
"serial", "serial",
], ],
[
"obsolete_files_test",
"db/obsolete_files_test.cc",
"serial",
],
[ [
"dynamic_bloom_test", "dynamic_bloom_test",
"util/dynamic_bloom_test.cc", "util/dynamic_bloom_test.cc",
@ -830,6 +826,11 @@ ROCKS_TESTS = [
"utilities/object_registry_test.cc", "utilities/object_registry_test.cc",
"serial", "serial",
], ],
[
"obsolete_files_test",
"db/obsolete_files_test.cc",
"serial",
],
[ [
"optimistic_transaction_test", "optimistic_transaction_test",
"utilities/transactions/optimistic_transaction_test.cc", "utilities/transactions/optimistic_transaction_test.cc",

View File

@ -92,7 +92,7 @@ Status BuildTable(
return s; return s;
} }
std::string fname = TableFileName(ioptions.db_paths, meta->fd.GetNumber(), std::string fname = TableFileName(ioptions.cf_paths, meta->fd.GetNumber(),
meta->fd.GetPathId()); meta->fd.GetPathId());
#ifndef ROCKSDB_LITE #ifndef ROCKSDB_LITE
EventHelpers::NotifyTableFileCreationStarted( EventHelpers::NotifyTableFileCreationStarted(

View File

@ -34,6 +34,7 @@
#include "table/merging_iterator.h" #include "table/merging_iterator.h"
#include "util/autovector.h" #include "util/autovector.h"
#include "util/compression.h" #include "util/compression.h"
#include "util/sst_file_manager_impl.h"
namespace rocksdb { namespace rocksdb {
@ -159,6 +160,28 @@ Status CheckConcurrentWritesSupported(const ColumnFamilyOptions& cf_options) {
return Status::OK(); return Status::OK();
} }
Status CheckCFPathsSupported(const DBOptions& db_options,
const ColumnFamilyOptions& cf_options) {
// More than one cf_paths are supported only in universal
// and level compaction styles. This function also checks the case
// in which cf_paths is not specified, which results in db_paths
// being used.
if ((cf_options.compaction_style != kCompactionStyleUniversal) &&
(cf_options.compaction_style != kCompactionStyleLevel)) {
if (cf_options.cf_paths.size() > 1) {
return Status::NotSupported(
"More than one CF paths are only supported in "
"universal and level compaction styles. ");
} else if (cf_options.cf_paths.empty() &&
db_options.db_paths.size() > 1) {
return Status::NotSupported(
"More than one DB paths are only supported in "
"universal and level compaction styles. ");
}
}
return Status::OK();
}
ColumnFamilyOptions SanitizeOptions(const ImmutableDBOptions& db_options, ColumnFamilyOptions SanitizeOptions(const ImmutableDBOptions& db_options,
const ColumnFamilyOptions& src) { const ColumnFamilyOptions& src) {
ColumnFamilyOptions result = src; ColumnFamilyOptions result = src;
@ -277,9 +300,24 @@ ColumnFamilyOptions SanitizeOptions(const ImmutableDBOptions& db_options,
result.hard_pending_compaction_bytes_limit; result.hard_pending_compaction_bytes_limit;
} }
#ifndef ROCKSDB_LITE
// When the DB is stopped, it's possible that there are some .trash files that
// were not deleted yet, when we open the DB we will find these .trash files
// and schedule them to be deleted (or delete immediately if SstFileManager
// was not used)
auto sfm = static_cast<SstFileManagerImpl*>(db_options.sst_file_manager.get());
for (size_t i = 0; i < result.cf_paths.size(); i++) {
DeleteScheduler::CleanupDirectory(db_options.env, sfm, result.cf_paths[i].path);
}
#endif
if (result.cf_paths.empty()) {
result.cf_paths = db_options.db_paths;
}
if (result.level_compaction_dynamic_level_bytes) { if (result.level_compaction_dynamic_level_bytes) {
if (result.compaction_style != kCompactionStyleLevel || if (result.compaction_style != kCompactionStyleLevel ||
db_options.db_paths.size() > 1U) { result.cf_paths.size() > 1U) {
// 1. level_compaction_dynamic_level_bytes only makes sense for // 1. level_compaction_dynamic_level_bytes only makes sense for
// level-based compaction. // level-based compaction.
// 2. we don't yet know how to make both of this feature and multiple // 2. we don't yet know how to make both of this feature and multiple
@ -519,7 +557,9 @@ uint64_t ColumnFamilyData::OldestLogToKeep() {
auto current_log = GetLogNumber(); auto current_log = GetLogNumber();
if (allow_2pc_) { if (allow_2pc_) {
auto imm_prep_log = imm()->GetMinLogContainingPrepSection(); autovector<MemTable*> empty_list;
auto imm_prep_log =
imm()->PrecomputeMinLogContainingPrepSection(empty_list);
auto mem_prep_log = mem()->GetMinLogContainingPrepSection(); auto mem_prep_log = mem()->GetMinLogContainingPrepSection();
if (imm_prep_log > 0 && imm_prep_log < current_log) { if (imm_prep_log > 0 && imm_prep_log < current_log) {
@ -1138,6 +1178,31 @@ Env::WriteLifeTimeHint ColumnFamilyData::CalculateSSTWriteHint(int level) {
static_cast<int>(Env::WLTH_MEDIUM)); static_cast<int>(Env::WLTH_MEDIUM));
} }
Status ColumnFamilyData::AddDirectories() {
Status s;
assert(data_dirs_.empty());
for (auto& p : ioptions_.cf_paths) {
std::unique_ptr<Directory> path_directory;
s = DBImpl::CreateAndNewDirectory(ioptions_.env, p.path, &path_directory);
if (!s.ok()) {
return s;
}
assert(path_directory != nullptr);
data_dirs_.emplace_back(path_directory.release());
}
assert(data_dirs_.size() == ioptions_.cf_paths.size());
return s;
}
Directory* ColumnFamilyData::GetDataDir(size_t path_id) const {
if (data_dirs_.empty()) {
return nullptr;
}
assert(path_id < data_dirs_.size());
return data_dirs_[path_id].get();
}
ColumnFamilySet::ColumnFamilySet(const std::string& dbname, ColumnFamilySet::ColumnFamilySet(const std::string& dbname,
const ImmutableDBOptions* db_options, const ImmutableDBOptions* db_options,
const EnvOptions& env_options, const EnvOptions& env_options,

View File

@ -139,6 +139,9 @@ extern Status CheckCompressionSupported(const ColumnFamilyOptions& cf_options);
extern Status CheckConcurrentWritesSupported( extern Status CheckConcurrentWritesSupported(
const ColumnFamilyOptions& cf_options); const ColumnFamilyOptions& cf_options);
extern Status CheckCFPathsSupported(const DBOptions& db_options,
const ColumnFamilyOptions& cf_options);
extern ColumnFamilyOptions SanitizeOptions(const ImmutableDBOptions& db_options, extern ColumnFamilyOptions SanitizeOptions(const ImmutableDBOptions& db_options,
const ColumnFamilyOptions& src); const ColumnFamilyOptions& src);
// Wrap user defined table proproties collector factories `from cf_options` // Wrap user defined table proproties collector factories `from cf_options`
@ -376,6 +379,10 @@ class ColumnFamilyData {
Env::WriteLifeTimeHint CalculateSSTWriteHint(int level); Env::WriteLifeTimeHint CalculateSSTWriteHint(int level);
Status AddDirectories();
Directory* GetDataDir(size_t path_id) const;
private: private:
friend class ColumnFamilySet; friend class ColumnFamilySet;
ColumnFamilyData(uint32_t id, const std::string& name, ColumnFamilyData(uint32_t id, const std::string& name,
@ -459,6 +466,9 @@ class ColumnFamilyData {
// Memtable id to track flush. // Memtable id to track flush.
std::atomic<uint64_t> last_memtable_id_; std::atomic<uint64_t> last_memtable_id_;
// Directories corresponding to cf_paths.
std::vector<std::unique_ptr<Directory>> data_dirs_;
}; };
// ColumnFamilySet has interesting thread-safety requirements // ColumnFamilySet has interesting thread-safety requirements

View File

@ -69,9 +69,15 @@ class ColumnFamilyTest : public testing::Test {
} }
~ColumnFamilyTest() { ~ColumnFamilyTest() {
std::vector<ColumnFamilyDescriptor> column_families;
for (auto h : handles_) {
ColumnFamilyDescriptor cfdescriptor;
h->GetDescriptor(&cfdescriptor);
column_families.push_back(cfdescriptor);
}
Close(); Close();
rocksdb::SyncPoint::GetInstance()->DisableProcessing(); rocksdb::SyncPoint::GetInstance()->DisableProcessing();
Destroy(); Destroy(column_families);
delete env_; delete env_;
} }
@ -236,9 +242,11 @@ class ColumnFamilyTest : public testing::Test {
#endif // !ROCKSDB_LITE #endif // !ROCKSDB_LITE
} }
void Destroy() { void Destroy(const std::vector<ColumnFamilyDescriptor>& column_families =
std::vector<ColumnFamilyDescriptor>()) {
Close(); Close();
ASSERT_OK(DestroyDB(dbname_, Options(db_options_, column_family_options_))); ASSERT_OK(DestroyDB(dbname_, Options(db_options_, column_family_options_),
column_families));
} }
void CreateColumnFamilies( void CreateColumnFamilies(
@ -383,6 +391,9 @@ class ColumnFamilyTest : public testing::Test {
void AssertFilesPerLevel(const std::string& value, int cf) { void AssertFilesPerLevel(const std::string& value, int cf) {
#ifndef ROCKSDB_LITE #ifndef ROCKSDB_LITE
ASSERT_EQ(value, FilesPerLevel(cf)); ASSERT_EQ(value, FilesPerLevel(cf));
#else
(void) value;
(void) cf;
#endif #endif
} }
@ -397,6 +408,8 @@ class ColumnFamilyTest : public testing::Test {
void AssertCountLiveFiles(int expected_value) { void AssertCountLiveFiles(int expected_value) {
#ifndef ROCKSDB_LITE #ifndef ROCKSDB_LITE
ASSERT_EQ(expected_value, CountLiveFiles()); ASSERT_EQ(expected_value, CountLiveFiles());
#else
(void) expected_value;
#endif #endif
} }
@ -445,6 +458,8 @@ class ColumnFamilyTest : public testing::Test {
void AssertCountLiveLogFiles(int value) { void AssertCountLiveLogFiles(int value) {
#ifndef ROCKSDB_LITE // GetSortedWalFiles is not supported #ifndef ROCKSDB_LITE // GetSortedWalFiles is not supported
ASSERT_EQ(value, CountLiveLogFiles()); ASSERT_EQ(value, CountLiveLogFiles());
#else
(void) value;
#endif // !ROCKSDB_LITE #endif // !ROCKSDB_LITE
} }
@ -483,6 +498,12 @@ class ColumnFamilyTest : public testing::Test {
ASSERT_OK(destfile->Close()); ASSERT_OK(destfile->Close());
} }
int GetSstFileCount(std::string path) {
std::vector<std::string> files;
DBTestBase::GetSstFiles(env_, path, &files);
return static_cast<int>(files.size());
}
std::vector<ColumnFamilyHandle*> handles_; std::vector<ColumnFamilyHandle*> handles_;
std::vector<std::string> names_; std::vector<std::string> names_;
std::set<std::string> keys_; std::set<std::string> keys_;
@ -3129,6 +3150,58 @@ TEST_F(ColumnFamilyTest, DISABLED_LogTruncationTest) {
// cleanup // cleanup
env_->DeleteDir(backup_logs); env_->DeleteDir(backup_logs);
} }
TEST_F(ColumnFamilyTest, DefaultCfPathsTest) {
Open();
// Leave cf_paths for one column families to be empty.
// Files should be generated according to db_paths for that
// column family.
ColumnFamilyOptions cf_opt1, cf_opt2;
cf_opt1.cf_paths.emplace_back(dbname_ + "_one_1",
std::numeric_limits<uint64_t>::max());
CreateColumnFamilies({"one", "two"}, {cf_opt1, cf_opt2});
Reopen({ColumnFamilyOptions(), cf_opt1, cf_opt2});
// Fill Column family 1.
PutRandomData(1, 100, 100);
Flush(1);
ASSERT_EQ(1, GetSstFileCount(cf_opt1.cf_paths[0].path));
ASSERT_EQ(0, GetSstFileCount(dbname_));
// Fill column family 2
PutRandomData(2, 100, 100);
Flush(2);
// SST from Column family 2 should be generated in
// db_paths which is dbname_ in this case.
ASSERT_EQ(1, GetSstFileCount(dbname_));
}
TEST_F(ColumnFamilyTest, MultipleCFPathsTest) {
Open();
// Configure Column family specific paths.
ColumnFamilyOptions cf_opt1, cf_opt2;
cf_opt1.cf_paths.emplace_back(dbname_ + "_one_1",
std::numeric_limits<uint64_t>::max());
cf_opt2.cf_paths.emplace_back(dbname_ + "_two_1",
std::numeric_limits<uint64_t>::max());
CreateColumnFamilies({"one", "two"}, {cf_opt1, cf_opt2});
Reopen({ColumnFamilyOptions(), cf_opt1, cf_opt2});
PutRandomData(1, 100, 100);
Flush(1);
// Check that files are generated in appropriate paths.
ASSERT_EQ(1, GetSstFileCount(cf_opt1.cf_paths[0].path));
ASSERT_EQ(0, GetSstFileCount(dbname_));
PutRandomData(2, 100, 100);
Flush(2);
ASSERT_EQ(1, GetSstFileCount(cf_opt2.cf_paths[0].path));
ASSERT_EQ(0, GetSstFileCount(dbname_));
}
} // namespace rocksdb } // namespace rocksdb
int main(int argc, char** argv) { int main(int argc, char** argv) {

View File

@ -318,7 +318,7 @@ int main(int argc, char** argv) {
#else #else
#include <stdio.h> #include <stdio.h>
int main(int argc, char** argv) { int main(int /*argc*/, char** /*argv*/) {
fprintf(stderr, fprintf(stderr,
"SKIPPED as DBImpl::CompactFiles is not supported in ROCKSDB_LITE\n"); "SKIPPED as DBImpl::CompactFiles is not supported in ROCKSDB_LITE\n");
return 0; return 0;

View File

@ -566,7 +566,9 @@ Status CompactionJob::Run() {
TablePropertiesCollection tp; TablePropertiesCollection tp;
for (const auto& state : compact_->sub_compact_states) { for (const auto& state : compact_->sub_compact_states) {
for (const auto& output : state.outputs) { for (const auto& output : state.outputs) {
auto fn = TableFileName(db_options_.db_paths, output.meta.fd.GetNumber(), auto fn = TableFileName(
state.compaction->immutable_cf_options()->cf_paths,
output.meta.fd.GetNumber(),
output.meta.fd.GetPathId()); output.meta.fd.GetPathId());
tp[fn] = output.table_properties; tp[fn] = output.table_properties;
} }
@ -1112,7 +1114,9 @@ Status CompactionJob::FinishCompactionOutputFile(
// This happens when the output level is bottom level, at the same time // This happens when the output level is bottom level, at the same time
// the sub_compact output nothing. // the sub_compact output nothing.
std::string fname = TableFileName( std::string fname = TableFileName(
db_options_.db_paths, meta->fd.GetNumber(), meta->fd.GetPathId()); sub_compact->compaction->immutable_cf_options()->cf_paths,
meta->fd.GetNumber(),
meta->fd.GetPathId());
env_->DeleteFile(fname); env_->DeleteFile(fname);
// Also need to remove the file from outputs, or it will be added to the // Also need to remove the file from outputs, or it will be added to the
@ -1165,7 +1169,9 @@ Status CompactionJob::FinishCompactionOutputFile(
std::string fname; std::string fname;
FileDescriptor output_fd; FileDescriptor output_fd;
if (meta != nullptr) { if (meta != nullptr) {
fname = TableFileName(db_options_.db_paths, meta->fd.GetNumber(), fname = TableFileName(
sub_compact->compaction->immutable_cf_options()->cf_paths,
meta->fd.GetNumber(),
meta->fd.GetPathId()); meta->fd.GetPathId());
output_fd = meta->fd; output_fd = meta->fd;
} else { } else {
@ -1180,7 +1186,9 @@ Status CompactionJob::FinishCompactionOutputFile(
auto sfm = auto sfm =
static_cast<SstFileManagerImpl*>(db_options_.sst_file_manager.get()); static_cast<SstFileManagerImpl*>(db_options_.sst_file_manager.get());
if (sfm && meta != nullptr && meta->fd.GetPathId() == 0) { if (sfm && meta != nullptr && meta->fd.GetPathId() == 0) {
auto fn = TableFileName(cfd->ioptions()->db_paths, meta->fd.GetNumber(), auto fn = TableFileName(
sub_compact->compaction->immutable_cf_options()->cf_paths,
meta->fd.GetNumber(),
meta->fd.GetPathId()); meta->fd.GetPathId());
sfm->OnAddFile(fn); sfm->OnAddFile(fn);
if (sfm->IsMaxAllowedSpaceReached()) { if (sfm->IsMaxAllowedSpaceReached()) {
@ -1266,7 +1274,9 @@ Status CompactionJob::OpenCompactionOutputFile(
assert(sub_compact->builder == nullptr); assert(sub_compact->builder == nullptr);
// no need to lock because VersionSet::next_file_number_ is atomic // no need to lock because VersionSet::next_file_number_ is atomic
uint64_t file_number = versions_->NewFileNumber(); uint64_t file_number = versions_->NewFileNumber();
std::string fname = TableFileName(db_options_.db_paths, file_number, std::string fname = TableFileName(
sub_compact->compaction->immutable_cf_options()->cf_paths,
file_number,
sub_compact->compaction->output_path_id()); sub_compact->compaction->output_path_id());
// Fire events. // Fire events.
ColumnFamilyData* cfd = sub_compact->compaction->column_family_data(); ColumnFamilyData* cfd = sub_compact->compaction->column_family_data();

View File

@ -1034,7 +1034,7 @@ int main(int argc, char** argv) {
#else #else
#include <stdio.h> #include <stdio.h>
int main(int argc, char** argv) { int main(int /*argc*/, char** /*argv*/) {
fprintf(stderr, "SKIPPED, not supported in ROCKSDB_LITE\n"); fprintf(stderr, "SKIPPED, not supported in ROCKSDB_LITE\n");
return 0; return 0;
} }
@ -1043,5 +1043,5 @@ int main(int argc, char** argv) {
#else #else
int main(int argc, char** argv) { return 0; } int main(int /*argc*/, char** /*argv*/) { return 0; }
#endif // !defined(IOS_CROSS_COMPILE) #endif // !defined(IOS_CROSS_COMPILE)

View File

@ -946,7 +946,7 @@ int main(int argc, char** argv) {
#else #else
#include <stdio.h> #include <stdio.h>
int main(int argc, char** argv) { int main(int /*argc*/, char** /*argv*/) {
fprintf(stderr, fprintf(stderr,
"SKIPPED as CompactionJobStats is not supported in ROCKSDB_LITE\n"); "SKIPPED as CompactionJobStats is not supported in ROCKSDB_LITE\n");
return 0; return 0;

View File

@ -616,7 +616,7 @@ Compaction* CompactionPicker::CompactRange(
} }
} }
} }
assert(output_path_id < static_cast<uint32_t>(ioptions_.db_paths.size())); assert(output_path_id < static_cast<uint32_t>(ioptions_.cf_paths.size()));
if (ExpandInputsToCleanCut(cf_name, vstorage, &inputs) == false) { if (ExpandInputsToCleanCut(cf_name, vstorage, &inputs) == false) {
// manual compaction is now multi-threaded, so it can // manual compaction is now multi-threaded, so it can
@ -941,6 +941,9 @@ void CompactionPicker::UnregisterCompaction(Compaction* c) {
bool LevelCompactionPicker::NeedsCompaction( bool LevelCompactionPicker::NeedsCompaction(
const VersionStorageInfo* vstorage) const { const VersionStorageInfo* vstorage) const {
if (!vstorage->ExpiredTtlFiles().empty()) {
return true;
}
if (!vstorage->BottommostFilesMarkedForCompaction().empty()) { if (!vstorage->BottommostFilesMarkedForCompaction().empty()) {
return true; return true;
} }
@ -1010,6 +1013,8 @@ class LevelCompactionBuilder {
// If there is any file marked for compaction, put put it into inputs. // If there is any file marked for compaction, put put it into inputs.
void PickFilesMarkedForCompaction(); void PickFilesMarkedForCompaction();
void PickExpiredTtlFiles();
const std::string& cf_name_; const std::string& cf_name_;
VersionStorageInfo* vstorage_; VersionStorageInfo* vstorage_;
CompactionPicker* compaction_picker_; CompactionPicker* compaction_picker_;
@ -1080,6 +1085,42 @@ void LevelCompactionBuilder::PickFilesMarkedForCompaction() {
start_level_inputs_.files.clear(); start_level_inputs_.files.clear();
} }
void LevelCompactionBuilder::PickExpiredTtlFiles() {
if (vstorage_->ExpiredTtlFiles().empty()) {
return;
}
auto continuation = [&](std::pair<int, FileMetaData*> level_file) {
// If it's being compacted it has nothing to do here.
// If this assert() fails that means that some function marked some
// files as being_compacted, but didn't call ComputeCompactionScore()
assert(!level_file.second->being_compacted);
start_level_ = level_file.first;
output_level_ =
(start_level_ == 0) ? vstorage_->base_level() : start_level_ + 1;
if ((start_level_ == vstorage_->num_non_empty_levels() - 1) ||
(start_level_ == 0 &&
!compaction_picker_->level0_compactions_in_progress()->empty())) {
return false;
}
start_level_inputs_.files = {level_file.second};
start_level_inputs_.level = start_level_;
return compaction_picker_->ExpandInputsToCleanCut(cf_name_, vstorage_,
&start_level_inputs_);
};
for (auto& level_file : vstorage_->ExpiredTtlFiles()) {
if (continuation(level_file)) {
// found the compaction!
return;
}
}
start_level_inputs_.files.clear();
}
void LevelCompactionBuilder::SetupInitialFiles() { void LevelCompactionBuilder::SetupInitialFiles() {
// Find the compactions by size on all levels. // Find the compactions by size on all levels.
bool skipped_l0_to_base = false; bool skipped_l0_to_base = false;
@ -1131,15 +1172,19 @@ void LevelCompactionBuilder::SetupInitialFiles() {
// if we didn't find a compaction, check if there are any files marked for // if we didn't find a compaction, check if there are any files marked for
// compaction // compaction
if (start_level_inputs_.empty()) { if (start_level_inputs_.empty()) {
is_manual_ = true;
parent_index_ = base_index_ = -1; parent_index_ = base_index_ = -1;
PickFilesMarkedForCompaction(); PickFilesMarkedForCompaction();
if (start_level_inputs_.empty()) { if (!start_level_inputs_.empty()) {
is_manual_ = true;
compaction_reason_ = CompactionReason::kFilesMarkedForCompaction;
return;
}
size_t i; size_t i;
for (i = 0; i < vstorage_->BottommostFilesMarkedForCompaction().size(); for (i = 0; i < vstorage_->BottommostFilesMarkedForCompaction().size();
++i) { ++i) {
auto& level_and_file = auto& level_and_file = vstorage_->BottommostFilesMarkedForCompaction()[i];
vstorage_->BottommostFilesMarkedForCompaction()[i];
assert(!level_and_file.second->being_compacted); assert(!level_and_file.second->being_compacted);
start_level_inputs_.level = output_level_ = start_level_ = start_level_inputs_.level = output_level_ = start_level_ =
level_and_file.first; level_and_file.first;
@ -1154,9 +1199,13 @@ void LevelCompactionBuilder::SetupInitialFiles() {
} else { } else {
assert(!start_level_inputs_.empty()); assert(!start_level_inputs_.empty());
compaction_reason_ = CompactionReason::kBottommostFiles; compaction_reason_ = CompactionReason::kBottommostFiles;
return;
} }
} else {
compaction_reason_ = CompactionReason::kFilesMarkedForCompaction; assert(start_level_inputs_.empty());
PickExpiredTtlFiles();
if (!start_level_inputs_.empty()) {
compaction_reason_ = CompactionReason::kTtl;
} }
} }
} }
@ -1287,10 +1336,10 @@ uint32_t LevelCompactionBuilder::GetPathId(
const ImmutableCFOptions& ioptions, const ImmutableCFOptions& ioptions,
const MutableCFOptions& mutable_cf_options, int level) { const MutableCFOptions& mutable_cf_options, int level) {
uint32_t p = 0; uint32_t p = 0;
assert(!ioptions.db_paths.empty()); assert(!ioptions.cf_paths.empty());
// size remaining in the most recent path // size remaining in the most recent path
uint64_t current_path_size = ioptions.db_paths[0].target_size; uint64_t current_path_size = ioptions.cf_paths[0].target_size;
uint64_t level_size; uint64_t level_size;
int cur_level = 0; int cur_level = 0;
@ -1300,7 +1349,7 @@ uint32_t LevelCompactionBuilder::GetPathId(
level_size = mutable_cf_options.max_bytes_for_level_base; level_size = mutable_cf_options.max_bytes_for_level_base;
// Last path is the fallback // Last path is the fallback
while (p < ioptions.db_paths.size() - 1) { while (p < ioptions.cf_paths.size() - 1) {
if (level_size <= current_path_size) { if (level_size <= current_path_size) {
if (cur_level == level) { if (cur_level == level) {
// Does desired level fit in this path? // Does desired level fit in this path?
@ -1327,7 +1376,7 @@ uint32_t LevelCompactionBuilder::GetPathId(
} }
} }
p++; p++;
current_path_size = ioptions.db_paths[p].target_size; current_path_size = ioptions.cf_paths[p].target_size;
} }
return p; return p;
} }

View File

@ -59,7 +59,7 @@ class CompactionPickerTest : public testing::Test {
vstorage_(nullptr) { vstorage_(nullptr) {
fifo_options_.max_table_files_size = 1; fifo_options_.max_table_files_size = 1;
mutable_cf_options_.RefreshDerivedOptions(ioptions_); mutable_cf_options_.RefreshDerivedOptions(ioptions_);
ioptions_.db_paths.emplace_back("dummy", ioptions_.cf_paths.emplace_back("dummy",
std::numeric_limits<uint64_t>::max()); std::numeric_limits<uint64_t>::max());
} }

View File

@ -406,9 +406,9 @@ uint32_t UniversalCompactionPicker::GetPathId(
file_size * file_size *
(100 - mutable_cf_options.compaction_options_universal.size_ratio) / 100; (100 - mutable_cf_options.compaction_options_universal.size_ratio) / 100;
uint32_t p = 0; uint32_t p = 0;
assert(!ioptions.db_paths.empty()); assert(!ioptions.cf_paths.empty());
for (; p < ioptions.db_paths.size() - 1; p++) { for (; p < ioptions.cf_paths.size() - 1; p++) {
uint64_t target_size = ioptions.db_paths[p].target_size; uint64_t target_size = ioptions.cf_paths[p].target_size;
if (target_size > file_size && if (target_size > file_size &&
accumulated_size + (target_size - file_size) > future_size) { accumulated_size + (target_size - file_size) > future_size) {
return p; return p;

View File

@ -510,7 +510,7 @@ int main(int argc, char** argv) {
#else #else
#include <stdio.h> #include <stdio.h>
int main(int argc, char** argv) { int main(int /*argc*/, char** /*argv*/) {
fprintf(stderr, "SKIPPED as RepairDB() is not supported in ROCKSDB_LITE\n"); fprintf(stderr, "SKIPPED as RepairDB() is not supported in ROCKSDB_LITE\n");
return 0; return 0;
} }

View File

@ -333,7 +333,7 @@ int main(int argc, char** argv) {
#else #else
#include <stdio.h> #include <stdio.h>
int main(int argc, char** argv) { int main(int /*argc*/, char** /*argv*/) {
fprintf(stderr, "SKIPPED as Cuckoo table is not supported in ROCKSDB_LITE\n"); fprintf(stderr, "SKIPPED as Cuckoo table is not supported in ROCKSDB_LITE\n");
return 0; return 0;
} }

View File

@ -305,6 +305,41 @@ TEST_F(DBBlockCacheTest, IndexAndFilterBlocksOfNewTableAddedToCache) {
TestGetTickerCount(options, BLOCK_CACHE_INDEX_HIT)); TestGetTickerCount(options, BLOCK_CACHE_INDEX_HIT));
} }
// With fill_cache = false, fills up the cache, then iterates over the entire
// db, verify dummy entries inserted in `BlockBasedTable::NewDataBlockIterator`
// does not cause heap-use-after-free errors in COMPILE_WITH_ASAN=1 runs
TEST_F(DBBlockCacheTest, FillCacheAndIterateDB) {
ReadOptions read_options;
read_options.fill_cache = false;
auto table_options = GetTableOptions();
auto options = GetOptions(table_options);
InitTable(options);
std::shared_ptr<Cache> cache = NewLRUCache(10, 0, true);
table_options.block_cache = cache;
options.table_factory.reset(new BlockBasedTableFactory(table_options));
Reopen(options);
ASSERT_OK(Put("key1", "val1"));
ASSERT_OK(Put("key2", "val2"));
ASSERT_OK(Flush());
ASSERT_OK(Put("key3", "val3"));
ASSERT_OK(Put("key4", "val4"));
ASSERT_OK(Flush());
ASSERT_OK(Put("key5", "val5"));
ASSERT_OK(Put("key6", "val6"));
ASSERT_OK(Flush());
Iterator* iter = nullptr;
iter = db_->NewIterator(read_options);
iter->Seek(ToString(0));
while (iter->Valid()) {
iter->Next();
}
delete iter;
iter = nullptr;
}
TEST_F(DBBlockCacheTest, IndexAndFilterBlocksStats) { TEST_F(DBBlockCacheTest, IndexAndFilterBlocksStats) {
Options options = CurrentOptions(); Options options = CurrentOptions();
options.create_if_missing = true; options.create_if_missing = true;

View File

@ -1983,6 +1983,125 @@ TEST_P(DBCompactionTestWithParam, LevelCompactionPathUse) {
Destroy(options); Destroy(options);
} }
TEST_P(DBCompactionTestWithParam, LevelCompactionCFPathUse) {
Options options = CurrentOptions();
options.db_paths.emplace_back(dbname_, 500 * 1024);
options.db_paths.emplace_back(dbname_ + "_2", 4 * 1024 * 1024);
options.db_paths.emplace_back(dbname_ + "_3", 1024 * 1024 * 1024);
options.memtable_factory.reset(
new SpecialSkipListFactory(KNumKeysByGenerateNewFile - 1));
options.compaction_style = kCompactionStyleLevel;
options.write_buffer_size = 110 << 10; // 110KB
options.arena_block_size = 4 << 10;
options.level0_file_num_compaction_trigger = 2;
options.num_levels = 4;
options.max_bytes_for_level_base = 400 * 1024;
options.max_subcompactions = max_subcompactions_;
std::vector<Options> option_vector;
option_vector.emplace_back(options);
ColumnFamilyOptions cf_opt1(options), cf_opt2(options);
// Configure CF1 specific paths.
cf_opt1.cf_paths.emplace_back(dbname_ + "cf1", 500 * 1024);
cf_opt1.cf_paths.emplace_back(dbname_ + "cf1_2", 4 * 1024 * 1024);
cf_opt1.cf_paths.emplace_back(dbname_ + "cf1_3", 1024 * 1024 * 1024);
option_vector.emplace_back(DBOptions(options), cf_opt1);
CreateColumnFamilies({"one"},option_vector[1]);
// Configura CF2 specific paths.
cf_opt2.cf_paths.emplace_back(dbname_ + "cf2", 500 * 1024);
cf_opt2.cf_paths.emplace_back(dbname_ + "cf2_2", 4 * 1024 * 1024);
cf_opt2.cf_paths.emplace_back(dbname_ + "cf2_3", 1024 * 1024 * 1024);
option_vector.emplace_back(DBOptions(options), cf_opt2);
CreateColumnFamilies({"two"},option_vector[2]);
ReopenWithColumnFamilies({"default", "one", "two"}, option_vector);
Random rnd(301);
int key_idx = 0;
int key_idx1 = 0;
int key_idx2 = 0;
auto generate_file = [&]() {
GenerateNewFile(0, &rnd, &key_idx);
GenerateNewFile(1, &rnd, &key_idx1);
GenerateNewFile(2, &rnd, &key_idx2);
};
auto check_sstfilecount = [&](int path_id, int expected) {
ASSERT_EQ(expected, GetSstFileCount(options.db_paths[path_id].path));
ASSERT_EQ(expected, GetSstFileCount(cf_opt1.cf_paths[path_id].path));
ASSERT_EQ(expected, GetSstFileCount(cf_opt2.cf_paths[path_id].path));
};
auto check_filesperlevel = [&](const std::string& expected) {
ASSERT_EQ(expected, FilesPerLevel(0));
ASSERT_EQ(expected, FilesPerLevel(1));
ASSERT_EQ(expected, FilesPerLevel(2));
};
auto check_getvalues = [&]() {
for (int i = 0; i < key_idx; i++) {
auto v = Get(0, Key(i));
ASSERT_NE(v, "NOT_FOUND");
ASSERT_TRUE(v.size() == 1 || v.size() == 990);
}
for (int i = 0; i < key_idx1; i++) {
auto v = Get(1, Key(i));
ASSERT_NE(v, "NOT_FOUND");
ASSERT_TRUE(v.size() == 1 || v.size() == 990);
}
for (int i = 0; i < key_idx2; i++) {
auto v = Get(2, Key(i));
ASSERT_NE(v, "NOT_FOUND");
ASSERT_TRUE(v.size() == 1 || v.size() == 990);
}
};
// Check that default column family uses db_paths.
// And Column family "one" uses cf_paths.
// First three 110KB files are not going to second path.
// After that, (100K, 200K)
for (int num = 0; num < 3; num++) {
generate_file();
}
// Another 110KB triggers a compaction to 400K file to fill up first path
generate_file();
check_sstfilecount(1, 3);
// (1, 4)
generate_file();
check_filesperlevel("1,4");
check_sstfilecount(1, 4);
check_sstfilecount(0, 1);
// (1, 4, 1)
generate_file();
check_filesperlevel("1,4,1");
check_sstfilecount(2, 1);
check_sstfilecount(1, 4);
check_sstfilecount(0, 1);
// (1, 4, 2)
generate_file();
check_filesperlevel("1,4,2");
check_sstfilecount(2, 2);
check_sstfilecount(1, 4);
check_sstfilecount(0, 1);
check_getvalues();
ReopenWithColumnFamilies({"default", "one", "two"}, option_vector);
check_getvalues();
Destroy(options, true);
}
TEST_P(DBCompactionTestWithParam, ConvertCompactionStyle) { TEST_P(DBCompactionTestWithParam, ConvertCompactionStyle) {
Random rnd(301); Random rnd(301);
int max_key_level_insert = 200; int max_key_level_insert = 200;
@ -3091,6 +3210,13 @@ TEST_F(DBCompactionTest, CompactBottomLevelFilesWithDeletions) {
// compactions should be triggered, which reduce the size of each bottom-level // compactions should be triggered, which reduce the size of each bottom-level
// file without changing file count. // file without changing file count.
db_->ReleaseSnapshot(snapshot); db_->ReleaseSnapshot(snapshot);
rocksdb::SyncPoint::GetInstance()->SetCallBack(
"LevelCompactionPicker::PickCompaction:Return", [&](void* arg) {
Compaction* compaction = reinterpret_cast<Compaction*>(arg);
ASSERT_TRUE(compaction->compaction_reason() ==
CompactionReason::kBottommostFiles);
});
rocksdb::SyncPoint::GetInstance()->EnableProcessing();
dbfull()->TEST_WaitForCompact(); dbfull()->TEST_WaitForCompact();
db_->GetLiveFilesMetaData(&post_release_metadata); db_->GetLiveFilesMetaData(&post_release_metadata);
ASSERT_EQ(pre_release_metadata.size(), post_release_metadata.size()); ASSERT_EQ(pre_release_metadata.size(), post_release_metadata.size());
@ -3104,6 +3230,66 @@ TEST_F(DBCompactionTest, CompactBottomLevelFilesWithDeletions) {
// deletion markers/deleted keys. // deletion markers/deleted keys.
ASSERT_LT(post_file.size, pre_file.size); ASSERT_LT(post_file.size, pre_file.size);
} }
rocksdb::SyncPoint::GetInstance()->DisableProcessing();
}
TEST_F(DBCompactionTest, LevelCompactExpiredTtlFiles) {
const int kNumKeysPerFile = 32;
const int kNumLevelFiles = 2;
const int kValueSize = 1024;
Options options = CurrentOptions();
options.compression = kNoCompression;
options.ttl = 24 * 60 * 60; // 24 hours
options.max_open_files = -1;
env_->time_elapse_only_sleep_ = false;
options.env = env_;
env_->addon_time_.store(0);
DestroyAndReopen(options);
Random rnd(301);
for (int i = 0; i < kNumLevelFiles; ++i) {
for (int j = 0; j < kNumKeysPerFile; ++j) {
ASSERT_OK(
Put(Key(i * kNumKeysPerFile + j), RandomString(&rnd, kValueSize)));
}
Flush();
}
Flush();
dbfull()->TEST_WaitForCompact();
MoveFilesToLevel(3);
ASSERT_EQ("0,0,0,2", FilesPerLevel());
for (int i = 0; i < kNumLevelFiles; ++i) {
for (int j = 0; j < kNumKeysPerFile; ++j) {
// Overwrite previous keys with smaller, but predictable, values.
ASSERT_OK(Delete(Key(i * kNumKeysPerFile + j)));
}
Flush();
}
dbfull()->TEST_WaitForCompact();
ASSERT_EQ("2,0,0,2", FilesPerLevel());
MoveFilesToLevel(1);
ASSERT_EQ("0,2,0,2", FilesPerLevel());
env_->addon_time_.fetch_add(36 * 60 * 60); // 36 hours
ASSERT_EQ("0,2,0,2", FilesPerLevel());
// Just do a siimple write + flush so that the Ttl expired files get
// compacted.
ASSERT_OK(Put("a", "1"));
Flush();
rocksdb::SyncPoint::GetInstance()->SetCallBack(
"LevelCompactionPicker::PickCompaction:Return", [&](void* arg) {
Compaction* compaction = reinterpret_cast<Compaction*>(arg);
ASSERT_TRUE(compaction->compaction_reason() == CompactionReason::kTtl);
});
rocksdb::SyncPoint::GetInstance()->EnableProcessing();
dbfull()->TEST_WaitForCompact();
// All non-L0 files are deleted, as they contained only deleted data.
ASSERT_EQ("1", FilesPerLevel());
rocksdb::SyncPoint::GetInstance()->DisableProcessing();
} }
TEST_F(DBCompactionTest, CompactRangeDelayedByL0FileCount) { TEST_F(DBCompactionTest, CompactRangeDelayedByL0FileCount) {
@ -3482,6 +3668,8 @@ int main(int argc, char** argv) {
::testing::InitGoogleTest(&argc, argv); ::testing::InitGoogleTest(&argc, argv);
return RUN_ALL_TESTS(); return RUN_ALL_TESTS();
#else #else
(void) argc;
(void) argv;
return 0; return 0;
#endif #endif
} }

View File

@ -501,6 +501,8 @@ int main(int argc, char** argv) {
::testing::InitGoogleTest(&argc, argv); ::testing::InitGoogleTest(&argc, argv);
return RUN_ALL_TESTS(); return RUN_ALL_TESTS();
#else #else
(void) argc;
(void) argv;
return 0; return 0;
#endif #endif
} }

View File

@ -72,19 +72,23 @@ TEST_F(DBFlushTest, SyncFail) {
auto* cfd = auto* cfd =
reinterpret_cast<ColumnFamilyHandleImpl*>(db_->DefaultColumnFamily()) reinterpret_cast<ColumnFamilyHandleImpl*>(db_->DefaultColumnFamily())
->cfd(); ->cfd();
int refs_before = cfd->current()->TEST_refs();
FlushOptions flush_options; FlushOptions flush_options;
flush_options.wait = false; flush_options.wait = false;
ASSERT_OK(dbfull()->Flush(flush_options)); ASSERT_OK(dbfull()->Flush(flush_options));
// Flush installs a new super-version. Get the ref count after that.
auto current_before = cfd->current();
int refs_before = cfd->current()->TEST_refs();
fault_injection_env->SetFilesystemActive(false); fault_injection_env->SetFilesystemActive(false);
TEST_SYNC_POINT("DBFlushTest::SyncFail:1"); TEST_SYNC_POINT("DBFlushTest::SyncFail:1");
TEST_SYNC_POINT("DBFlushTest::SyncFail:2"); TEST_SYNC_POINT("DBFlushTest::SyncFail:2");
fault_injection_env->SetFilesystemActive(true); fault_injection_env->SetFilesystemActive(true);
// Now the background job will do the flush; wait for it.
dbfull()->TEST_WaitForFlushMemTable(); dbfull()->TEST_WaitForFlushMemTable();
#ifndef ROCKSDB_LITE #ifndef ROCKSDB_LITE
ASSERT_EQ("", FilesPerLevel()); // flush failed. ASSERT_EQ("", FilesPerLevel()); // flush failed.
#endif // ROCKSDB_LITE #endif // ROCKSDB_LITE
// Flush job should release ref count to current version. // Backgroun flush job should release ref count to current version.
ASSERT_EQ(current_before, cfd->current());
ASSERT_EQ(refs_before, cfd->current()->TEST_refs()); ASSERT_EQ(refs_before, cfd->current()->TEST_refs());
Destroy(options); Destroy(options);
} }

View File

@ -183,7 +183,7 @@ DBImpl::DBImpl(const DBOptions& options, const std::string& dbname,
last_stats_dump_time_microsec_(0), last_stats_dump_time_microsec_(0),
next_job_id_(1), next_job_id_(1),
has_unpersisted_data_(false), has_unpersisted_data_(false),
unable_to_flush_oldest_log_(false), unable_to_release_oldest_log_(false),
env_options_(BuildDBOptions(immutable_db_options_, mutable_db_options_)), env_options_(BuildDBOptions(immutable_db_options_, mutable_db_options_)),
env_options_for_compaction_(env_->OptimizeForCompactionTableWrite( env_options_for_compaction_(env_->OptimizeForCompactionTableWrite(
env_options_, immutable_db_options_)), env_options_, immutable_db_options_)),
@ -514,7 +514,16 @@ void DBImpl::ScheduleBgLogWriterClose(JobContext* job_context) {
} }
} }
Directory* DBImpl::Directories::GetDataDir(size_t path_id) { Directory* DBImpl::GetDataDir(ColumnFamilyData* cfd, size_t path_id) const {
assert(cfd);
Directory* ret_dir = cfd->GetDataDir(path_id);
if (ret_dir == nullptr) {
return directories_.GetDataDir(path_id);
}
return ret_dir;
}
Directory* DBImpl::Directories::GetDataDir(size_t path_id) const {
assert(path_id < data_dirs_.size()); assert(path_id < data_dirs_.size());
Directory* ret_dir = data_dirs_[path_id].get(); Directory* ret_dir = data_dirs_[path_id].get();
if (ret_dir == nullptr) { if (ret_dir == nullptr) {
@ -858,14 +867,14 @@ void DBImpl::BackgroundCallPurge() {
if (!purge_queue_.empty()) { if (!purge_queue_.empty()) {
auto purge_file = purge_queue_.begin(); auto purge_file = purge_queue_.begin();
auto fname = purge_file->fname; auto fname = purge_file->fname;
auto dir_to_sync = purge_file->dir_to_sync;
auto type = purge_file->type; auto type = purge_file->type;
auto number = purge_file->number; auto number = purge_file->number;
auto path_id = purge_file->path_id;
auto job_id = purge_file->job_id; auto job_id = purge_file->job_id;
purge_queue_.pop_front(); purge_queue_.pop_front();
mutex_.Unlock(); mutex_.Unlock();
DeleteObsoleteFileImpl(job_id, fname, type, number, path_id); DeleteObsoleteFileImpl(job_id, fname, dir_to_sync, type, number);
mutex_.Lock(); mutex_.Lock();
} else { } else {
assert(!logs_to_free_queue_.empty()); assert(!logs_to_free_queue_.empty());
@ -1306,6 +1315,17 @@ Status DBImpl::CreateColumnFamilyImpl(const ColumnFamilyOptions& cf_options,
if (s.ok() && immutable_db_options_.allow_concurrent_memtable_write) { if (s.ok() && immutable_db_options_.allow_concurrent_memtable_write) {
s = CheckConcurrentWritesSupported(cf_options); s = CheckConcurrentWritesSupported(cf_options);
} }
if (s.ok()) {
s = CheckCFPathsSupported(initial_db_options_, cf_options);
}
if (s.ok()) {
for (auto& cf_path : cf_options.cf_paths) {
s = env_->CreateDirIfMissing(cf_path.path);
if (!s.ok()) {
break;
}
}
}
if (!s.ok()) { if (!s.ok()) {
return s; return s;
} }
@ -1337,6 +1357,12 @@ Status DBImpl::CreateColumnFamilyImpl(const ColumnFamilyOptions& cf_options,
&cf_options); &cf_options);
write_thread_.ExitUnbatched(&w); write_thread_.ExitUnbatched(&w);
} }
if (s.ok()) {
auto* cfd =
versions_->GetColumnFamilySet()->GetColumnFamily(column_family_name);
assert(cfd != nullptr);
s = cfd->AddDirectories();
}
if (s.ok()) { if (s.ok()) {
single_column_family_mode_ = false; single_column_family_mode_ = false;
auto* cfd = auto* cfd =
@ -1681,7 +1707,7 @@ const Snapshot* DBImpl::GetSnapshotForWriteConflictBoundary() {
} }
#endif // ROCKSDB_LITE #endif // ROCKSDB_LITE
const Snapshot* DBImpl::GetSnapshotImpl(bool is_write_conflict_boundary) { SnapshotImpl* DBImpl::GetSnapshotImpl(bool is_write_conflict_boundary) {
int64_t unix_time = 0; int64_t unix_time = 0;
env_->GetCurrentTime(&unix_time); // Ignore error env_->GetCurrentTime(&unix_time); // Ignore error
SnapshotImpl* s = new SnapshotImpl; SnapshotImpl* s = new SnapshotImpl;
@ -2391,7 +2417,8 @@ Status DB::ListColumnFamilies(const DBOptions& db_options,
Snapshot::~Snapshot() { Snapshot::~Snapshot() {
} }
Status DestroyDB(const std::string& dbname, const Options& options) { Status DestroyDB(const std::string& dbname, const Options& options,
const std::vector<ColumnFamilyDescriptor>& column_families) {
ImmutableDBOptions soptions(SanitizeOptions(dbname, options)); ImmutableDBOptions soptions(SanitizeOptions(dbname, options));
Env* env = soptions.env; Env* env = soptions.env;
std::vector<std::string> filenames; std::vector<std::string> filenames;
@ -2417,7 +2444,7 @@ Status DestroyDB(const std::string& dbname, const Options& options) {
if (type == kMetaDatabase) { if (type == kMetaDatabase) {
del = DestroyDB(path_to_delete, options); del = DestroyDB(path_to_delete, options);
} else if (type == kTableFile) { } else if (type == kTableFile) {
del = DeleteSSTFile(&soptions, path_to_delete, 0); del = DeleteSSTFile(&soptions, path_to_delete, dbname);
} else { } else {
del = env->DeleteFile(path_to_delete); del = env->DeleteFile(path_to_delete);
} }
@ -2427,15 +2454,32 @@ Status DestroyDB(const std::string& dbname, const Options& options) {
} }
} }
std::vector<std::string> paths;
for (size_t path_id = 0; path_id < options.db_paths.size(); path_id++) { for (size_t path_id = 0; path_id < options.db_paths.size(); path_id++) {
const auto& db_path = options.db_paths[path_id]; paths.emplace_back(options.db_paths[path_id].path);
env->GetChildren(db_path.path, &filenames); }
for (auto& cf : column_families) {
for (size_t path_id = 0; path_id < cf.options.cf_paths.size();
path_id++) {
paths.emplace_back(cf.options.cf_paths[path_id].path);
}
}
// Remove duplicate paths.
// Note that we compare only the actual paths but not path ids.
// This reason is that same path can appear at different path_ids
// for different column families.
std::sort(paths.begin(), paths.end());
paths.erase(std::unique(paths.begin(), paths.end()), paths.end());
for (auto& path : paths) {
env->GetChildren(path, &filenames);
for (size_t i = 0; i < filenames.size(); i++) { for (size_t i = 0; i < filenames.size(); i++) {
if (ParseFileName(filenames[i], &number, &type) && if (ParseFileName(filenames[i], &number, &type) &&
type == kTableFile) { // Lock file will be deleted at end type == kTableFile) { // Lock file will be deleted at end
std::string table_path = db_path.path + "/" + filenames[i]; std::string table_path = path + "/" + filenames[i];
Status del = DeleteSSTFile(&soptions, table_path, Status del = DeleteSSTFile(&soptions, table_path, path);
static_cast<uint32_t>(path_id));
if (result.ok() && !del.ok()) { if (result.ok() && !del.ok()) {
result = del; result = del;
} }
@ -2921,11 +2965,12 @@ Status DBImpl::VerifyChecksum() {
} }
for (auto& sv : sv_list) { for (auto& sv : sv_list) {
VersionStorageInfo* vstorage = sv->current->storage_info(); VersionStorageInfo* vstorage = sv->current->storage_info();
ColumnFamilyData* cfd = sv->current->cfd();
for (int i = 0; i < vstorage->num_non_empty_levels() && s.ok(); i++) { for (int i = 0; i < vstorage->num_non_empty_levels() && s.ok(); i++) {
for (size_t j = 0; j < vstorage->LevelFilesBrief(i).num_files && s.ok(); for (size_t j = 0; j < vstorage->LevelFilesBrief(i).num_files && s.ok();
j++) { j++) {
const auto& fd = vstorage->LevelFilesBrief(i).files[j].fd; const auto& fd = vstorage->LevelFilesBrief(i).files[j].fd;
std::string fname = TableFileName(immutable_db_options_.db_paths, std::string fname = TableFileName(cfd->ioptions()->cf_paths,
fd.GetNumber(), fd.GetPathId()); fd.GetNumber(), fd.GetPathId());
s = rocksdb::VerifySstFileChecksum(options, env_options, fname); s = rocksdb::VerifySstFileChecksum(options, env_options, fname);
} }
@ -2979,5 +3024,4 @@ void DBImpl::WaitForIngestFile() {
} }
#endif // ROCKSDB_LITE #endif // ROCKSDB_LITE
} // namespace rocksdb } // namespace rocksdb

View File

@ -27,6 +27,7 @@
#include "db/flush_scheduler.h" #include "db/flush_scheduler.h"
#include "db/internal_stats.h" #include "db/internal_stats.h"
#include "db/log_writer.h" #include "db/log_writer.h"
#include "db/logs_with_prep_tracker.h"
#include "db/pre_release_callback.h" #include "db/pre_release_callback.h"
#include "db/read_callback.h" #include "db/read_callback.h"
#include "db/snapshot_checker.h" #include "db/snapshot_checker.h"
@ -353,6 +354,10 @@ class DBImpl : public DB {
Arena* arena, RangeDelAggregator* range_del_agg, Arena* arena, RangeDelAggregator* range_del_agg,
ColumnFamilyHandle* column_family = nullptr); ColumnFamilyHandle* column_family = nullptr);
LogsWithPrepTracker* logs_with_prep_tracker() {
return &logs_with_prep_tracker_;
}
#ifndef NDEBUG #ifndef NDEBUG
// Extra methods (for testing) that are not in the public DB interface // Extra methods (for testing) that are not in the public DB interface
// Implemented in db_impl_debug.cc // Implemented in db_impl_debug.cc
@ -364,9 +369,7 @@ class DBImpl : public DB {
void TEST_SwitchWAL(); void TEST_SwitchWAL();
bool TEST_UnableToFlushOldestLog() { bool TEST_UnableToReleaseOldestLog() { return unable_to_release_oldest_log_; }
return unable_to_flush_oldest_log_;
}
bool TEST_IsLogGettingFlushed() { bool TEST_IsLogGettingFlushed() {
return alive_log_files_.begin()->getting_flushed; return alive_log_files_.begin()->getting_flushed;
@ -478,7 +481,7 @@ class DBImpl : public DB {
// It is not necessary to hold the mutex when invoking this method. // It is not necessary to hold the mutex when invoking this method.
// If FindObsoleteFiles() was run, we need to also run // If FindObsoleteFiles() was run, we need to also run
// PurgeObsoleteFiles(), even if disable_delete_obsolete_files_ is true // PurgeObsoleteFiles(), even if disable_delete_obsolete_files_ is true
void PurgeObsoleteFiles(const JobContext& background_contet, void PurgeObsoleteFiles(JobContext& background_contet,
bool schedule_only = false); bool schedule_only = false);
void SchedulePurge(); void SchedulePurge();
@ -593,7 +596,7 @@ class DBImpl : public DB {
size_t batch_cnt) { size_t batch_cnt) {
recovered_transactions_[name] = recovered_transactions_[name] =
new RecoveredTransaction(log, name, batch, seq, batch_cnt); new RecoveredTransaction(log, name, batch, seq, batch_cnt);
MarkLogAsContainingPrepSection(log); logs_with_prep_tracker_.MarkLogAsContainingPrepSection(log);
} }
void DeleteRecoveredTransaction(const std::string& name) { void DeleteRecoveredTransaction(const std::string& name) {
@ -601,7 +604,7 @@ class DBImpl : public DB {
assert(it != recovered_transactions_.end()); assert(it != recovered_transactions_.end());
auto* trx = it->second; auto* trx = it->second;
recovered_transactions_.erase(it); recovered_transactions_.erase(it);
MarkLogAsHavingPrepSectionFlushed(trx->log_number_); logs_with_prep_tracker_.MarkLogAsHavingPrepSectionFlushed(trx->log_number_);
delete trx; delete trx;
} }
@ -613,8 +616,6 @@ class DBImpl : public DB {
recovered_transactions_.clear(); recovered_transactions_.clear();
} }
void MarkLogAsHavingPrepSectionFlushed(uint64_t log);
void MarkLogAsContainingPrepSection(uint64_t log);
void AddToLogsToFreeQueue(log::Writer* log_writer) { void AddToLogsToFreeQueue(log::Writer* log_writer) {
logs_to_free_queue_.push_back(log_writer); logs_to_free_queue_.push_back(log_writer);
} }
@ -636,6 +637,9 @@ class DBImpl : public DB {
virtual Status Close() override; virtual Status Close() override;
static Status CreateAndNewDirectory(Env* env, const std::string& dirname,
std::unique_ptr<Directory>* directory);
protected: protected:
Env* const env_; Env* const env_;
const std::string dbname_; const std::string dbname_;
@ -688,14 +692,20 @@ class DBImpl : public DB {
void EraseThreadStatusDbInfo() const; void EraseThreadStatusDbInfo() const;
// If disable_memtable is set the application logic must guarantee that the // If disable_memtable is set the application logic must guarantee that the
// batch will still be skipped from memtable during the recovery. In // batch will still be skipped from memtable during the recovery. An excption
// WriteCommitted it is guarnateed since disable_memtable is used for prepare // to this is seq_per_batch_ mode, in which since each batch already takes one
// batch which will be written to memtable later during the commit, and in // seq, it is ok for the batch to write to memtable during recovery as long as
// WritePrepared it is guaranteed since it will be used only for WAL markers // it only takes one sequence number: i.e., no duplicate keys.
// which will never be written to memtable. // In WriteCommitted it is guarnateed since disable_memtable is used for
// batch_cnt is expected to be non-zero in seq_per_batch mode and indicates // prepare batch which will be written to memtable later during the commit,
// the number of sub-patches. A sub-patch is a subset of the write batch that // and in WritePrepared it is guaranteed since it will be used only for WAL
// does not have duplicate keys. // markers which will never be written to memtable. If the commit marker is
// accompanied with CommitTimeWriteBatch that is not written to memtable as
// long as it has no duplicate keys, it does not violate the one-seq-per-batch
// policy.
// batch_cnt is expected to be non-zero in seq_per_batch mode and
// indicates the number of sub-patches. A sub-patch is a subset of the write
// batch that does not have duplicate keys.
Status WriteImpl(const WriteOptions& options, WriteBatch* updates, Status WriteImpl(const WriteOptions& options, WriteBatch* updates,
WriteCallback* callback = nullptr, WriteCallback* callback = nullptr,
uint64_t* log_used = nullptr, uint64_t log_ref = 0, uint64_t* log_used = nullptr, uint64_t log_ref = 0,
@ -718,8 +728,6 @@ class DBImpl : public DB {
uint64_t* seq_used = nullptr, size_t batch_cnt = 0, uint64_t* seq_used = nullptr, size_t batch_cnt = 0,
PreReleaseCallback* pre_release_callback = nullptr); PreReleaseCallback* pre_release_callback = nullptr);
uint64_t FindMinLogContainingOutstandingPrep();
uint64_t FindMinPrepLogReferencedByMemTable();
// write cached_recoverable_state_ to memtable if it is not empty // write cached_recoverable_state_ to memtable if it is not empty
// The writer must be the leader in write_thread_ and holding mutex_ // The writer must be the leader in write_thread_ and holding mutex_
Status WriteRecoverableState(); Status WriteRecoverableState();
@ -731,6 +739,7 @@ class DBImpl : public DB {
friend class DB; friend class DB;
friend class InternalStats; friend class InternalStats;
friend class PessimisticTransaction; friend class PessimisticTransaction;
friend class TransactionBaseImpl;
friend class WriteCommittedTxn; friend class WriteCommittedTxn;
friend class WritePreparedTxn; friend class WritePreparedTxn;
friend class WritePreparedTxnDB; friend class WritePreparedTxnDB;
@ -788,7 +797,8 @@ class DBImpl : public DB {
void DeleteObsoleteFiles(); void DeleteObsoleteFiles();
// Delete obsolete files and log status and information of file deletion // Delete obsolete files and log status and information of file deletion
void DeleteObsoleteFileImpl(int job_id, const std::string& fname, void DeleteObsoleteFileImpl(int job_id, const std::string& fname,
FileType type, uint64_t number, uint32_t path_id); const std::string& path_to_sync, FileType type,
uint64_t number);
// Background process needs to call // Background process needs to call
// auto x = CaptureCurrentFileNumberInPendingOutputs() // auto x = CaptureCurrentFileNumberInPendingOutputs()
@ -909,8 +919,8 @@ class DBImpl : public DB {
void MaybeScheduleFlushOrCompaction(); void MaybeScheduleFlushOrCompaction();
void SchedulePendingFlush(ColumnFamilyData* cfd, FlushReason flush_reason); void SchedulePendingFlush(ColumnFamilyData* cfd, FlushReason flush_reason);
void SchedulePendingCompaction(ColumnFamilyData* cfd); void SchedulePendingCompaction(ColumnFamilyData* cfd);
void SchedulePendingPurge(std::string fname, FileType type, uint64_t number, void SchedulePendingPurge(std::string fname, std::string dir_to_sync,
uint32_t path_id, int job_id); FileType type, uint64_t number, int job_id);
static void BGWorkCompaction(void* arg); static void BGWorkCompaction(void* arg);
// Runs a pre-chosen universal compaction involving bottom level in a // Runs a pre-chosen universal compaction involving bottom level in a
// separate, bottom-pri thread pool. // separate, bottom-pri thread pool.
@ -952,10 +962,12 @@ class DBImpl : public DB {
// helper function to call after some of the logs_ were synced // helper function to call after some of the logs_ were synced
void MarkLogsSynced(uint64_t up_to, bool synced_dir, const Status& status); void MarkLogsSynced(uint64_t up_to, bool synced_dir, const Status& status);
const Snapshot* GetSnapshotImpl(bool is_write_conflict_boundary); SnapshotImpl* GetSnapshotImpl(bool is_write_conflict_boundary);
uint64_t GetMaxTotalWalSize() const; uint64_t GetMaxTotalWalSize() const;
Directory* GetDataDir(ColumnFamilyData* cfd, size_t path_id) const;
Status CloseHelper(); Status CloseHelper();
// table_cache_ provides its own synchronization // table_cache_ provides its own synchronization
@ -1093,7 +1105,7 @@ class DBImpl : public DB {
const std::string& wal_dir, const std::string& wal_dir,
const std::vector<DbPath>& data_paths); const std::vector<DbPath>& data_paths);
Directory* GetDataDir(size_t path_id); Directory* GetDataDir(size_t path_id) const;
Directory* GetWalDir() { Directory* GetWalDir() {
if (wal_dir_) { if (wal_dir_) {
@ -1108,9 +1120,6 @@ class DBImpl : public DB {
std::unique_ptr<Directory> db_dir_; std::unique_ptr<Directory> db_dir_;
std::vector<std::unique_ptr<Directory>> data_dirs_; std::vector<std::unique_ptr<Directory>> data_dirs_;
std::unique_ptr<Directory> wal_dir_; std::unique_ptr<Directory> wal_dir_;
Status CreateAndNewDirectory(Env* env, const std::string& dirname,
std::unique_ptr<Directory>* directory) const;
}; };
Directories directories_; Directories directories_;
@ -1152,13 +1161,13 @@ class DBImpl : public DB {
// purge_queue_ // purge_queue_
struct PurgeFileInfo { struct PurgeFileInfo {
std::string fname; std::string fname;
std::string dir_to_sync;
FileType type; FileType type;
uint64_t number; uint64_t number;
uint32_t path_id;
int job_id; int job_id;
PurgeFileInfo(std::string fn, FileType t, uint64_t num, uint32_t pid, PurgeFileInfo(std::string fn, std::string d, FileType t, uint64_t num,
int jid) int jid)
: fname(fn), type(t), number(num), path_id(pid), job_id(jid) {} : fname(fn), dir_to_sync(d), type(t), number(num), job_id(jid) {}
}; };
// flush_queue_ and compaction_queue_ hold column families that we need to // flush_queue_ and compaction_queue_ hold column families that we need to
@ -1288,7 +1297,7 @@ class DBImpl : public DB {
// We must attempt to free the dependent memtables again // We must attempt to free the dependent memtables again
// at a later time after the transaction in the oldest // at a later time after the transaction in the oldest
// log is fully commited. // log is fully commited.
bool unable_to_flush_oldest_log_; bool unable_to_release_oldest_log_;
static const int KEEP_LOG_FILE_NUM = 1000; static const int KEEP_LOG_FILE_NUM = 1000;
// MSVC version 1800 still does not have constexpr for ::max() // MSVC version 1800 still does not have constexpr for ::max()
@ -1325,33 +1334,7 @@ class DBImpl : public DB {
// Indicate DB was opened successfully // Indicate DB was opened successfully
bool opened_successfully_; bool opened_successfully_;
// REQUIRES: logs_with_prep_mutex_ held LogsWithPrepTracker logs_with_prep_tracker_;
//
// sorted list of log numbers still containing prepared data.
// this is used by FindObsoleteFiles to determine which
// flushed logs we must keep around because they still
// contain prepared data which has not been committed or rolled back
struct LogCnt {
uint64_t log; // the log number
uint64_t cnt; // number of prepared sections in the log
};
std::vector<LogCnt> logs_with_prep_;
std::mutex logs_with_prep_mutex_;
// REQUIRES: prepared_section_completed_mutex_ held
//
// to be used in conjunction with logs_with_prep_.
// once a transaction with data in log L is committed or rolled back
// rather than updating logs_with_prep_ directly we keep track of that
// in prepared_section_completed_ which maps LOG -> instance_count. This helps
// avoiding contention between a commit thread and the prepare threads.
//
// when trying to determine the minimum log still active we first
// consult logs_with_prep_. while that root value maps to
// an equal value in prepared_section_completed_ we erase the log from
// both logs_with_prep_ and prepared_section_completed_.
std::unordered_map<uint64_t, uint64_t> prepared_section_completed_;
std::mutex prepared_section_completed_mutex_;
// Callback for compaction to check if a key is visible to a snapshot. // Callback for compaction to check if a key is visible to a snapshot.
// REQUIRES: mutex held // REQUIRES: mutex held
@ -1447,6 +1430,25 @@ extern CompressionType GetCompressionFlush(
const ImmutableCFOptions& ioptions, const ImmutableCFOptions& ioptions,
const MutableCFOptions& mutable_cf_options); const MutableCFOptions& mutable_cf_options);
// Return the earliest log file to keep after the memtable flush is
// finalized.
// `cfd_to_flush` is the column family whose memtable (specified in
// `memtables_to_flush`) will be flushed and thus will not depend on any WAL
// file.
// The function is only applicable to 2pc mode.
extern uint64_t PrecomputeMinLogNumberToKeep(
VersionSet* vset, const ColumnFamilyData& cfd_to_flush,
autovector<VersionEdit*> edit_list,
const autovector<MemTable*>& memtables_to_flush,
LogsWithPrepTracker* prep_tracker);
// `cfd_to_flush` is the column family whose memtable will be flushed and thus
// will not depend on any WAL file. nullptr means no memtable is being flushed.
// The function is only applicable to 2pc mode.
extern uint64_t FindMinPrepLogReferencedByMemTable(
VersionSet* vset, const ColumnFamilyData* cfd_to_flush,
const autovector<MemTable*>& memtables_to_flush);
// Fix user-supplied options to be reasonable // Fix user-supplied options to be reasonable
template <class T, class V> template <class T, class V>
static void ClipToRange(T* ptr, V minvalue, V maxvalue) { static void ClipToRange(T* ptr, V minvalue, V maxvalue) {

View File

@ -96,7 +96,7 @@ Status DBImpl::FlushMemTableToOutputFile(
env_options_for_compaction_, versions_.get(), &mutex_, &shutting_down_, env_options_for_compaction_, versions_.get(), &mutex_, &shutting_down_,
snapshot_seqs, earliest_write_conflict_snapshot, snapshot_checker, snapshot_seqs, earliest_write_conflict_snapshot, snapshot_checker,
job_context, log_buffer, directories_.GetDbDir(), job_context, log_buffer, directories_.GetDbDir(),
directories_.GetDataDir(0U), GetDataDir(cfd, 0U),
GetCompressionFlush(*cfd->ioptions(), mutable_cf_options), stats_, GetCompressionFlush(*cfd->ioptions(), mutable_cf_options), stats_,
&event_logger_, mutable_cf_options.report_bg_io_stats); &event_logger_, mutable_cf_options.report_bg_io_stats);
@ -129,7 +129,7 @@ Status DBImpl::FlushMemTableToOutputFile(
// and EventListener callback will be called when the db_mutex // and EventListener callback will be called when the db_mutex
// is unlocked by the current thread. // is unlocked by the current thread.
if (s.ok()) { if (s.ok()) {
s = flush_job.Run(&file_meta); s = flush_job.Run(&logs_with_prep_tracker_, &file_meta);
} else { } else {
flush_job.Cancel(); flush_job.Cancel();
} }
@ -169,7 +169,7 @@ Status DBImpl::FlushMemTableToOutputFile(
if (sfm) { if (sfm) {
// Notify sst_file_manager that a new file was added // Notify sst_file_manager that a new file was added
std::string file_path = MakeTableFileName( std::string file_path = MakeTableFileName(
immutable_db_options_.db_paths[0].path, file_meta.fd.GetNumber()); cfd->ioptions()->cf_paths[0].path, file_meta.fd.GetNumber());
sfm->OnAddFile(file_path); sfm->OnAddFile(file_path);
if (sfm->IsMaxAllowedSpaceReached() && bg_error_.ok()) { if (sfm->IsMaxAllowedSpaceReached() && bg_error_.ok()) {
Status new_bg_error = Status::IOError("Max allowed space was reached"); Status new_bg_error = Status::IOError("Max allowed space was reached");
@ -214,7 +214,7 @@ void DBImpl::NotifyOnFlushBegin(ColumnFamilyData* cfd, FileMetaData* file_meta,
info.cf_name = cfd->GetName(); info.cf_name = cfd->GetName();
// TODO(yhchiang): make db_paths dynamic in case flush does not // TODO(yhchiang): make db_paths dynamic in case flush does not
// go to L0 in the future. // go to L0 in the future.
info.file_path = MakeTableFileName(immutable_db_options_.db_paths[0].path, info.file_path = MakeTableFileName(cfd->ioptions()->cf_paths[0].path,
file_meta->fd.GetNumber()); file_meta->fd.GetNumber());
info.thread_id = env_->GetThreadID(); info.thread_id = env_->GetThreadID();
info.job_id = job_id; info.job_id = job_id;
@ -259,7 +259,7 @@ void DBImpl::NotifyOnFlushCompleted(ColumnFamilyData* cfd,
info.cf_name = cfd->GetName(); info.cf_name = cfd->GetName();
// TODO(yhchiang): make db_paths dynamic in case flush does not // TODO(yhchiang): make db_paths dynamic in case flush does not
// go to L0 in the future. // go to L0 in the future.
info.file_path = MakeTableFileName(immutable_db_options_.db_paths[0].path, info.file_path = MakeTableFileName(cfd->ioptions()->cf_paths[0].path,
file_meta->fd.GetNumber()); file_meta->fd.GetNumber());
info.thread_id = env_->GetThreadID(); info.thread_id = env_->GetThreadID();
info.job_id = job_id; info.job_id = job_id;
@ -282,12 +282,13 @@ void DBImpl::NotifyOnFlushCompleted(ColumnFamilyData* cfd,
Status DBImpl::CompactRange(const CompactRangeOptions& options, Status DBImpl::CompactRange(const CompactRangeOptions& options,
ColumnFamilyHandle* column_family, ColumnFamilyHandle* column_family,
const Slice* begin, const Slice* end) { const Slice* begin, const Slice* end) {
if (options.target_path_id >= immutable_db_options_.db_paths.size()) { auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
auto cfd = cfh->cfd();
if (options.target_path_id >= cfd->ioptions()->cf_paths.size()) {
return Status::InvalidArgument("Invalid target path ID"); return Status::InvalidArgument("Invalid target path ID");
} }
auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
auto cfd = cfh->cfd();
bool exclusive = options.exclusive_manual_compaction; bool exclusive = options.exclusive_manual_compaction;
bool flush_needed = true; bool flush_needed = true;
@ -553,7 +554,7 @@ Status DBImpl::CompactFilesImpl(
version->GetColumnFamilyMetaData(&cf_meta); version->GetColumnFamilyMetaData(&cf_meta);
if (output_path_id < 0) { if (output_path_id < 0) {
if (immutable_db_options_.db_paths.size() == 1U) { if (cfd->ioptions()->cf_paths.size() == 1U) {
output_path_id = 0; output_path_id = 0;
} else { } else {
return Status::NotSupported( return Status::NotSupported(
@ -615,8 +616,9 @@ Status DBImpl::CompactFilesImpl(
job_context->job_id, c.get(), immutable_db_options_, job_context->job_id, c.get(), immutable_db_options_,
env_options_for_compaction_, versions_.get(), &shutting_down_, env_options_for_compaction_, versions_.get(), &shutting_down_,
preserve_deletes_seqnum_.load(), log_buffer, directories_.GetDbDir(), preserve_deletes_seqnum_.load(), log_buffer, directories_.GetDbDir(),
directories_.GetDataDir(c->output_path_id()), stats_, &mutex_, &bg_error_, GetDataDir(c->column_family_data(), c->output_path_id()),
snapshot_seqs, earliest_write_conflict_snapshot, snapshot_checker, stats_, &mutex_, &bg_error_, snapshot_seqs,
earliest_write_conflict_snapshot, snapshot_checker,
table_cache_, &event_logger_, table_cache_, &event_logger_,
c->mutable_cf_options()->paranoid_file_checks, c->mutable_cf_options()->paranoid_file_checks,
c->mutable_cf_options()->report_bg_io_stats, dbname_, c->mutable_cf_options()->report_bg_io_stats, dbname_,
@ -685,7 +687,7 @@ Status DBImpl::CompactFilesImpl(
if (output_file_names != nullptr) { if (output_file_names != nullptr) {
for (const auto newf : c->edit()->GetNewFiles()) { for (const auto newf : c->edit()->GetNewFiles()) {
(*output_file_names).push_back(TableFileName( (*output_file_names).push_back(TableFileName(
immutable_db_options_.db_paths, newf.second.fd.GetNumber(), c->immutable_cf_options()->cf_paths, newf.second.fd.GetNumber(),
newf.second.fd.GetPathId()) ); newf.second.fd.GetPathId()) );
} }
} }
@ -760,7 +762,7 @@ void DBImpl::NotifyOnCompactionCompleted(
info.compression = c->output_compression(); info.compression = c->output_compression();
for (size_t i = 0; i < c->num_input_levels(); ++i) { for (size_t i = 0; i < c->num_input_levels(); ++i) {
for (const auto fmd : *c->inputs(i)) { for (const auto fmd : *c->inputs(i)) {
auto fn = TableFileName(immutable_db_options_.db_paths, auto fn = TableFileName(c->immutable_cf_options()->cf_paths,
fmd->fd.GetNumber(), fmd->fd.GetPathId()); fmd->fd.GetNumber(), fmd->fd.GetPathId());
info.input_files.push_back(fn); info.input_files.push_back(fn);
if (info.table_properties.count(fn) == 0) { if (info.table_properties.count(fn) == 0) {
@ -773,7 +775,8 @@ void DBImpl::NotifyOnCompactionCompleted(
} }
} }
for (const auto newf : c->edit()->GetNewFiles()) { for (const auto newf : c->edit()->GetNewFiles()) {
info.output_files.push_back(TableFileName(immutable_db_options_.db_paths, info.output_files.push_back(TableFileName(
c->immutable_cf_options()->cf_paths,
newf.second.fd.GetNumber(), newf.second.fd.GetNumber(),
newf.second.fd.GetPathId())); newf.second.fd.GetPathId()));
} }
@ -1246,11 +1249,10 @@ void DBImpl::SchedulePendingCompaction(ColumnFamilyData* cfd) {
} }
} }
void DBImpl::SchedulePendingPurge(std::string fname, FileType type, void DBImpl::SchedulePendingPurge(std::string fname, std::string dir_to_sync,
uint64_t number, uint32_t path_id, FileType type, uint64_t number, int job_id) {
int job_id) {
mutex_.AssertHeld(); mutex_.AssertHeld();
PurgeFileInfo file_info(fname, type, number, path_id, job_id); PurgeFileInfo file_info(fname, dir_to_sync, type, number, job_id);
purge_queue_.push_back(std::move(file_info)); purge_queue_.push_back(std::move(file_info));
} }
@ -1838,8 +1840,9 @@ Status DBImpl::BackgroundCompaction(bool* made_progress,
job_context->job_id, c.get(), immutable_db_options_, job_context->job_id, c.get(), immutable_db_options_,
env_options_for_compaction_, versions_.get(), &shutting_down_, env_options_for_compaction_, versions_.get(), &shutting_down_,
preserve_deletes_seqnum_.load(), log_buffer, directories_.GetDbDir(), preserve_deletes_seqnum_.load(), log_buffer, directories_.GetDbDir(),
directories_.GetDataDir(c->output_path_id()), stats_, &mutex_, GetDataDir(c->column_family_data(), c->output_path_id()),
&bg_error_, snapshot_seqs, earliest_write_conflict_snapshot, stats_, &mutex_, &bg_error_,
snapshot_seqs, earliest_write_conflict_snapshot,
snapshot_checker, table_cache_, &event_logger_, snapshot_checker, table_cache_, &event_logger_,
c->mutable_cf_options()->paranoid_file_checks, c->mutable_cf_options()->paranoid_file_checks,
c->mutable_cf_options()->report_bg_io_stats, dbname_, c->mutable_cf_options()->report_bg_io_stats, dbname_,

View File

@ -184,17 +184,21 @@ Status DBImpl::TEST_GetAllImmutableCFOptions(
} }
uint64_t DBImpl::TEST_FindMinLogContainingOutstandingPrep() { uint64_t DBImpl::TEST_FindMinLogContainingOutstandingPrep() {
return FindMinLogContainingOutstandingPrep(); return logs_with_prep_tracker_.FindMinLogContainingOutstandingPrep();
} }
size_t DBImpl::TEST_PreparedSectionCompletedSize() { size_t DBImpl::TEST_PreparedSectionCompletedSize() {
return prepared_section_completed_.size(); return logs_with_prep_tracker_.TEST_PreparedSectionCompletedSize();
} }
size_t DBImpl::TEST_LogsWithPrepSize() { return logs_with_prep_.size(); } size_t DBImpl::TEST_LogsWithPrepSize() {
return logs_with_prep_tracker_.TEST_LogsWithPrepSize();
}
uint64_t DBImpl::TEST_FindMinPrepLogReferencedByMemTable() { uint64_t DBImpl::TEST_FindMinPrepLogReferencedByMemTable() {
return FindMinPrepLogReferencedByMemTable(); autovector<MemTable*> empty_list;
return FindMinPrepLogReferencedByMemTable(versions_.get(), nullptr,
empty_list);
} }
Status DBImpl::TEST_GetLatestMutableCFOptions( Status DBImpl::TEST_GetLatestMutableCFOptions(

View File

@ -14,124 +14,17 @@
#include <inttypes.h> #include <inttypes.h>
#include <unordered_set> #include <unordered_set>
#include "db/event_helpers.h" #include "db/event_helpers.h"
#include "db/memtable_list.h"
#include "util/file_util.h" #include "util/file_util.h"
#include "util/sst_file_manager_impl.h" #include "util/sst_file_manager_impl.h"
namespace rocksdb { namespace rocksdb {
uint64_t DBImpl::FindMinPrepLogReferencedByMemTable() {
if (!allow_2pc()) {
return 0;
}
uint64_t min_log = 0;
// we must look through the memtables for two phase transactions
// that have been committed but not yet flushed
for (auto loop_cfd : *versions_->GetColumnFamilySet()) {
if (loop_cfd->IsDropped()) {
continue;
}
auto log = loop_cfd->imm()->GetMinLogContainingPrepSection();
if (log > 0 && (min_log == 0 || log < min_log)) {
min_log = log;
}
log = loop_cfd->mem()->GetMinLogContainingPrepSection();
if (log > 0 && (min_log == 0 || log < min_log)) {
min_log = log;
}
}
return min_log;
}
void DBImpl::MarkLogAsHavingPrepSectionFlushed(uint64_t log) {
assert(log != 0);
std::lock_guard<std::mutex> lock(prepared_section_completed_mutex_);
auto it = prepared_section_completed_.find(log);
if (UNLIKELY(it == prepared_section_completed_.end())) {
prepared_section_completed_[log] = 1;
} else {
it->second += 1;
}
}
void DBImpl::MarkLogAsContainingPrepSection(uint64_t log) {
assert(log != 0);
std::lock_guard<std::mutex> lock(logs_with_prep_mutex_);
auto rit = logs_with_prep_.rbegin();
bool updated = false;
// Most probably the last log is the one that is being marked for
// having a prepare section; so search from the end.
for (; rit != logs_with_prep_.rend() && rit->log >= log; ++rit) {
if (rit->log == log) {
rit->cnt++;
updated = true;
break;
}
}
if (!updated) {
// We are either at the start, or at a position with rit->log < log
logs_with_prep_.insert(rit.base(), {log, 1});
}
}
uint64_t DBImpl::FindMinLogContainingOutstandingPrep() {
std::lock_guard<std::mutex> lock(logs_with_prep_mutex_);
auto it = logs_with_prep_.begin();
// start with the smallest log
for (; it != logs_with_prep_.end();) {
auto min_log = it->log;
{
std::lock_guard<std::mutex> lock2(prepared_section_completed_mutex_);
auto completed_it = prepared_section_completed_.find(min_log);
if (completed_it == prepared_section_completed_.end() ||
completed_it->second < it->cnt) {
return min_log;
}
assert(completed_it != prepared_section_completed_.end() &&
completed_it->second == it->cnt);
prepared_section_completed_.erase(completed_it);
}
// erase from beginning in vector is not efficient but this function is not
// on the fast path.
it = logs_with_prep_.erase(it);
}
// no such log found
return 0;
}
uint64_t DBImpl::MinLogNumberToKeep() { uint64_t DBImpl::MinLogNumberToKeep() {
uint64_t log_number = versions_->MinLogNumber();
if (allow_2pc()) { if (allow_2pc()) {
// if are 2pc we must consider logs containing prepared return versions_->min_log_number_to_keep_2pc();
// sections of outstanding transactions. } else {
// return versions_->MinLogNumberWithUnflushedData();
// We must check min logs with outstanding prep before we check
// logs references by memtables because a log referenced by the
// first data structure could transition to the second under us.
//
// TODO(horuff): iterating over all column families under db mutex.
// should find more optimal solution
auto min_log_in_prep_heap = FindMinLogContainingOutstandingPrep();
if (min_log_in_prep_heap != 0 && min_log_in_prep_heap < log_number) {
log_number = min_log_in_prep_heap;
} }
auto min_log_refed_by_mem = FindMinPrepLogReferencedByMemTable();
if (min_log_refed_by_mem != 0 && min_log_refed_by_mem < log_number) {
log_number = min_log_refed_by_mem;
}
}
return log_number;
} }
// * Returns the list of live files in 'sst_live' // * Returns the list of live files in 'sst_live'
@ -191,8 +84,8 @@ void DBImpl::FindObsoleteFiles(JobContext* job_context, bool force,
// Mark the elements in job_context->sst_delete_files as grabbedForPurge // Mark the elements in job_context->sst_delete_files as grabbedForPurge
// so that other threads calling FindObsoleteFiles with full_scan=true // so that other threads calling FindObsoleteFiles with full_scan=true
// will not add these files to candidate list for purge. // will not add these files to candidate list for purge.
for (const auto sst_to_del : job_context->sst_delete_files) { for (const auto& sst_to_del : job_context->sst_delete_files) {
MarkAsGrabbedForPurge(sst_to_del->fd.GetNumber()); MarkAsGrabbedForPurge(sst_to_del.metadata->fd.GetNumber());
} }
// store the current filenum, lognum, etc // store the current filenum, lognum, etc
@ -200,20 +93,35 @@ void DBImpl::FindObsoleteFiles(JobContext* job_context, bool force,
job_context->pending_manifest_file_number = job_context->pending_manifest_file_number =
versions_->pending_manifest_file_number(); versions_->pending_manifest_file_number();
job_context->log_number = MinLogNumberToKeep(); job_context->log_number = MinLogNumberToKeep();
job_context->prev_log_number = versions_->prev_log_number(); job_context->prev_log_number = versions_->prev_log_number();
versions_->AddLiveFiles(&job_context->sst_live); versions_->AddLiveFiles(&job_context->sst_live);
if (doing_the_full_scan) { if (doing_the_full_scan) {
InfoLogPrefix info_log_prefix(!immutable_db_options_.db_log_dir.empty(), InfoLogPrefix info_log_prefix(!immutable_db_options_.db_log_dir.empty(),
dbname_); dbname_);
std::vector<std::string> paths;
for (size_t path_id = 0; path_id < immutable_db_options_.db_paths.size(); for (size_t path_id = 0; path_id < immutable_db_options_.db_paths.size();
path_id++) { path_id++) {
paths.emplace_back(immutable_db_options_.db_paths[path_id].path);
}
// Note that if cf_paths is not specified in the ColumnFamilyOptions
// of a particular column family, we use db_paths as the cf_paths
// setting. Hence, there can be multiple duplicates of files from db_paths
// in the following code. The duplicate are removed while identifying
// unique files in PurgeObsoleteFiles.
for (auto cfd : *versions_->GetColumnFamilySet()) {
for (size_t path_id = 0; path_id < cfd->ioptions()->cf_paths.size();
path_id++) {
paths.emplace_back(cfd->ioptions()->cf_paths[path_id].path);
}
}
for (auto& path : paths) {
// set of all files in the directory. We'll exclude files that are still // set of all files in the directory. We'll exclude files that are still
// alive in the subsequent processings. // alive in the subsequent processings.
std::vector<std::string> files; std::vector<std::string> files;
env_->GetChildren(immutable_db_options_.db_paths[path_id].path, env_->GetChildren(path, &files); // Ignore errors
&files); // Ignore errors
for (const std::string& file : files) { for (const std::string& file : files) {
uint64_t number; uint64_t number;
FileType type; FileType type;
@ -231,7 +139,7 @@ void DBImpl::FindObsoleteFiles(JobContext* job_context, bool force,
// TODO(icanadi) clean up this mess to avoid having one-off "/" prefixes // TODO(icanadi) clean up this mess to avoid having one-off "/" prefixes
job_context->full_scan_candidate_files.emplace_back( job_context->full_scan_candidate_files.emplace_back(
"/" + file, static_cast<uint32_t>(path_id)); "/" + file, path);
} }
} }
@ -241,7 +149,8 @@ void DBImpl::FindObsoleteFiles(JobContext* job_context, bool force,
env_->GetChildren(immutable_db_options_.wal_dir, env_->GetChildren(immutable_db_options_.wal_dir,
&log_files); // Ignore errors &log_files); // Ignore errors
for (const std::string& log_file : log_files) { for (const std::string& log_file : log_files) {
job_context->full_scan_candidate_files.emplace_back(log_file, 0); job_context->full_scan_candidate_files.emplace_back(log_file,
immutable_db_options_.wal_dir);
} }
} }
// Add info log files in db_log_dir // Add info log files in db_log_dir
@ -250,8 +159,9 @@ void DBImpl::FindObsoleteFiles(JobContext* job_context, bool force,
std::vector<std::string> info_log_files; std::vector<std::string> info_log_files;
// Ignore errors // Ignore errors
env_->GetChildren(immutable_db_options_.db_log_dir, &info_log_files); env_->GetChildren(immutable_db_options_.db_log_dir, &info_log_files);
for (std::string log_file : info_log_files) { for (std::string& log_file : info_log_files) {
job_context->full_scan_candidate_files.emplace_back(log_file, 0); job_context->full_scan_candidate_files.emplace_back(log_file,
immutable_db_options_.db_log_dir);
} }
} }
} }
@ -326,19 +236,19 @@ bool CompareCandidateFile(const JobContext::CandidateFileInfo& first,
} else if (first.file_name < second.file_name) { } else if (first.file_name < second.file_name) {
return false; return false;
} else { } else {
return (first.path_id > second.path_id); return (first.file_path > second.file_path);
} }
} }
}; // namespace }; // namespace
// Delete obsolete files and log status and information of file deletion // Delete obsolete files and log status and information of file deletion
void DBImpl::DeleteObsoleteFileImpl(int job_id, const std::string& fname, void DBImpl::DeleteObsoleteFileImpl(int job_id, const std::string& fname,
FileType type, uint64_t number, const std::string& path_to_sync,
uint32_t path_id) { FileType type, uint64_t number) {
Status file_deletion_status; Status file_deletion_status;
if (type == kTableFile) { if (type == kTableFile) {
file_deletion_status = file_deletion_status =
DeleteSSTFile(&immutable_db_options_, fname, path_id); DeleteSSTFile(&immutable_db_options_, fname, path_to_sync);
} else { } else {
file_deletion_status = env_->DeleteFile(fname); file_deletion_status = env_->DeleteFile(fname);
} }
@ -373,7 +283,7 @@ void DBImpl::DeleteObsoleteFileImpl(int job_id, const std::string& fname,
// belong to live files are possibly removed. Also, removes all the // belong to live files are possibly removed. Also, removes all the
// files in sst_delete_files and log_delete_files. // files in sst_delete_files and log_delete_files.
// It is not necessary to hold the mutex when invoking this method. // It is not necessary to hold the mutex when invoking this method.
void DBImpl::PurgeObsoleteFiles(const JobContext& state, bool schedule_only) { void DBImpl::PurgeObsoleteFiles(JobContext& state, bool schedule_only) {
TEST_SYNC_POINT("DBImpl::PurgeObsoleteFiles:Begin"); TEST_SYNC_POINT("DBImpl::PurgeObsoleteFiles:Begin");
// we'd better have sth to delete // we'd better have sth to delete
assert(state.HaveSomethingToDelete()); assert(state.HaveSomethingToDelete());
@ -396,23 +306,23 @@ void DBImpl::PurgeObsoleteFiles(const JobContext& state, bool schedule_only) {
state.log_delete_files.size() + state.manifest_delete_files.size()); state.log_delete_files.size() + state.manifest_delete_files.size());
// We may ignore the dbname when generating the file names. // We may ignore the dbname when generating the file names.
const char* kDumbDbName = ""; const char* kDumbDbName = "";
for (auto file : state.sst_delete_files) { for (auto& file : state.sst_delete_files) {
candidate_files.emplace_back( candidate_files.emplace_back(
MakeTableFileName(kDumbDbName, file->fd.GetNumber()), MakeTableFileName(kDumbDbName, file.metadata->fd.GetNumber()), file.path);
file->fd.GetPathId()); if (file.metadata->table_reader_handle) {
if (file->table_reader_handle) { table_cache_->Release(file.metadata->table_reader_handle);
table_cache_->Release(file->table_reader_handle);
} }
delete file; file.DeleteMetadata();
} }
for (auto file_num : state.log_delete_files) { for (auto file_num : state.log_delete_files) {
if (file_num > 0) { if (file_num > 0) {
candidate_files.emplace_back(LogFileName(kDumbDbName, file_num), 0); candidate_files.emplace_back(LogFileName(kDumbDbName, file_num),
immutable_db_options_.wal_dir);
} }
} }
for (const auto& filename : state.manifest_delete_files) { for (const auto& filename : state.manifest_delete_files) {
candidate_files.emplace_back(filename, 0); candidate_files.emplace_back(filename, dbname_);
} }
// dedup state.candidate_files so we don't try to delete the same // dedup state.candidate_files so we don't try to delete the same
@ -438,7 +348,6 @@ void DBImpl::PurgeObsoleteFiles(const JobContext& state, bool schedule_only) {
std::unordered_set<uint64_t> files_to_del; std::unordered_set<uint64_t> files_to_del;
for (const auto& candidate_file : candidate_files) { for (const auto& candidate_file : candidate_files) {
std::string to_delete = candidate_file.file_name; std::string to_delete = candidate_file.file_name;
uint32_t path_id = candidate_file.path_id;
uint64_t number; uint64_t number;
FileType type; FileType type;
// Ignore file if we cannot recognize it. // Ignore file if we cannot recognize it.
@ -502,13 +411,16 @@ void DBImpl::PurgeObsoleteFiles(const JobContext& state, bool schedule_only) {
} }
std::string fname; std::string fname;
std::string dir_to_sync;
if (type == kTableFile) { if (type == kTableFile) {
// evict from cache // evict from cache
TableCache::Evict(table_cache_.get(), number); TableCache::Evict(table_cache_.get(), number);
fname = TableFileName(immutable_db_options_.db_paths, number, path_id); fname = MakeTableFileName(candidate_file.file_path, number);
dir_to_sync = candidate_file.file_path;
} else { } else {
fname = ((type == kLogFile) ? immutable_db_options_.wal_dir : dbname_) + dir_to_sync =
"/" + to_delete; (type == kLogFile) ? immutable_db_options_.wal_dir : dbname_;
fname = dir_to_sync + "/" + to_delete;
} }
#ifndef ROCKSDB_LITE #ifndef ROCKSDB_LITE
@ -522,9 +434,9 @@ void DBImpl::PurgeObsoleteFiles(const JobContext& state, bool schedule_only) {
Status file_deletion_status; Status file_deletion_status;
if (schedule_only) { if (schedule_only) {
InstrumentedMutexLock guard_lock(&mutex_); InstrumentedMutexLock guard_lock(&mutex_);
SchedulePendingPurge(fname, type, number, path_id, state.job_id); SchedulePendingPurge(fname, dir_to_sync, type, number, state.job_id);
} else { } else {
DeleteObsoleteFileImpl(state.job_id, fname, type, number, path_id); DeleteObsoleteFileImpl(state.job_id, fname, dir_to_sync, type, number);
} }
} }
@ -601,4 +513,94 @@ void DBImpl::DeleteObsoleteFiles() {
mutex_.Lock(); mutex_.Lock();
} }
uint64_t FindMinPrepLogReferencedByMemTable(
VersionSet* vset, const ColumnFamilyData* cfd_to_flush,
const autovector<MemTable*>& memtables_to_flush) {
uint64_t min_log = 0;
// we must look through the memtables for two phase transactions
// that have been committed but not yet flushed
for (auto loop_cfd : *vset->GetColumnFamilySet()) {
if (loop_cfd->IsDropped() || loop_cfd == cfd_to_flush) {
continue;
}
auto log = loop_cfd->imm()->PrecomputeMinLogContainingPrepSection(
memtables_to_flush);
if (log > 0 && (min_log == 0 || log < min_log)) {
min_log = log;
}
log = loop_cfd->mem()->GetMinLogContainingPrepSection();
if (log > 0 && (min_log == 0 || log < min_log)) {
min_log = log;
}
}
return min_log;
}
uint64_t PrecomputeMinLogNumberToKeep(
VersionSet* vset, const ColumnFamilyData& cfd_to_flush,
autovector<VersionEdit*> edit_list,
const autovector<MemTable*>& memtables_to_flush,
LogsWithPrepTracker* prep_tracker) {
assert(vset != nullptr);
assert(prep_tracker != nullptr);
// Calculate updated min_log_number_to_keep
// Since the function should only be called in 2pc mode, log number in
// the version edit should be sufficient.
// Precompute the min log number containing unflushed data for the column
// family being flushed (`cfd_to_flush`).
uint64_t cf_min_log_number_to_keep = 0;
for (auto& e : edit_list) {
if (e->has_log_number()) {
cf_min_log_number_to_keep =
std::max(cf_min_log_number_to_keep, e->log_number());
}
}
if (cf_min_log_number_to_keep == 0) {
// No version edit contains information on log number. The log number
// for this column family should stay the same as it is.
cf_min_log_number_to_keep = cfd_to_flush.GetLogNumber();
}
// Get min log number containing unflushed data for other column families.
uint64_t min_log_number_to_keep =
vset->PreComputeMinLogNumberWithUnflushedData(&cfd_to_flush);
if (cf_min_log_number_to_keep != 0) {
min_log_number_to_keep =
std::min(cf_min_log_number_to_keep, min_log_number_to_keep);
}
// if are 2pc we must consider logs containing prepared
// sections of outstanding transactions.
//
// We must check min logs with outstanding prep before we check
// logs references by memtables because a log referenced by the
// first data structure could transition to the second under us.
//
// TODO: iterating over all column families under db mutex.
// should find more optimal solution
auto min_log_in_prep_heap =
prep_tracker->FindMinLogContainingOutstandingPrep();
if (min_log_in_prep_heap != 0 &&
min_log_in_prep_heap < min_log_number_to_keep) {
min_log_number_to_keep = min_log_in_prep_heap;
}
uint64_t min_log_refed_by_mem = FindMinPrepLogReferencedByMemTable(
vset, &cfd_to_flush, memtables_to_flush);
if (min_log_refed_by_mem != 0 &&
min_log_refed_by_mem < min_log_number_to_keep) {
min_log_number_to_keep = min_log_refed_by_mem;
}
return min_log_number_to_keep;
}
} // namespace rocksdb } // namespace rocksdb

View File

@ -163,28 +163,23 @@ static Status ValidateOptions(
if (s.ok() && db_options.allow_concurrent_memtable_write) { if (s.ok() && db_options.allow_concurrent_memtable_write) {
s = CheckConcurrentWritesSupported(cfd.options); s = CheckConcurrentWritesSupported(cfd.options);
} }
if (s.ok()) {
s = CheckCFPathsSupported(db_options, cfd.options);
}
if (!s.ok()) { if (!s.ok()) {
return s; return s;
} }
if (db_options.db_paths.size() > 1) {
if ((cfd.options.compaction_style != kCompactionStyleUniversal) && if (cfd.options.ttl > 0 || cfd.options.compaction_options_fifo.ttl > 0) {
(cfd.options.compaction_style != kCompactionStyleLevel)) {
return Status::NotSupported(
"More than one DB paths are only supported in "
"universal and level compaction styles. ");
}
}
if (cfd.options.compaction_options_fifo.ttl > 0) {
if (db_options.max_open_files != -1) { if (db_options.max_open_files != -1) {
return Status::NotSupported( return Status::NotSupported(
"FIFO Compaction with TTL is only supported when files are always " "TTL is only supported when files are always "
"kept open (set max_open_files = -1). "); "kept open (set max_open_files = -1). ");
} }
if (cfd.options.table_factory->Name() != if (cfd.options.table_factory->Name() !=
BlockBasedTableFactory().Name()) { BlockBasedTableFactory().Name()) {
return Status::NotSupported( return Status::NotSupported(
"FIFO Compaction with TTL is only supported in " "TTL is only supported in Block-Based Table format. ");
"Block-Based Table format. ");
} }
} }
} }
@ -254,9 +249,9 @@ Status DBImpl::NewDB() {
return s; return s;
} }
Status DBImpl::Directories::CreateAndNewDirectory( Status DBImpl::CreateAndNewDirectory(
Env* env, const std::string& dirname, Env* env, const std::string& dirname,
std::unique_ptr<Directory>* directory) const { std::unique_ptr<Directory>* directory) {
// We call CreateDirIfMissing() as the directory may already exist (if we // We call CreateDirIfMissing() as the directory may already exist (if we
// are reopening a DB), when this happens we don't want creating the // are reopening a DB), when this happens we don't want creating the
// directory to cause an error. However, we need to check if creating the // directory to cause an error. However, we need to check if creating the
@ -274,12 +269,12 @@ Status DBImpl::Directories::CreateAndNewDirectory(
Status DBImpl::Directories::SetDirectories( Status DBImpl::Directories::SetDirectories(
Env* env, const std::string& dbname, const std::string& wal_dir, Env* env, const std::string& dbname, const std::string& wal_dir,
const std::vector<DbPath>& data_paths) { const std::vector<DbPath>& data_paths) {
Status s = CreateAndNewDirectory(env, dbname, &db_dir_); Status s = DBImpl::CreateAndNewDirectory(env, dbname, &db_dir_);
if (!s.ok()) { if (!s.ok()) {
return s; return s;
} }
if (!wal_dir.empty() && dbname != wal_dir) { if (!wal_dir.empty() && dbname != wal_dir) {
s = CreateAndNewDirectory(env, wal_dir, &wal_dir_); s = DBImpl::CreateAndNewDirectory(env, wal_dir, &wal_dir_);
if (!s.ok()) { if (!s.ok()) {
return s; return s;
} }
@ -292,7 +287,7 @@ Status DBImpl::Directories::SetDirectories(
data_dirs_.emplace_back(nullptr); data_dirs_.emplace_back(nullptr);
} else { } else {
std::unique_ptr<Directory> path_directory; std::unique_ptr<Directory> path_directory;
s = CreateAndNewDirectory(env, db_path, &path_directory); s = DBImpl::CreateAndNewDirectory(env, db_path, &path_directory);
if (!s.ok()) { if (!s.ok()) {
return s; return s;
} }
@ -385,6 +380,14 @@ Status DBImpl::Recover(
if (immutable_db_options_.paranoid_checks && s.ok()) { if (immutable_db_options_.paranoid_checks && s.ok()) {
s = CheckConsistency(); s = CheckConsistency();
} }
if (s.ok() && !read_only) {
for (auto cfd : *versions_->GetColumnFamilySet()) {
s = cfd->AddDirectories();
if (!s.ok()) {
return s;
}
}
}
if (s.ok()) { if (s.ok()) {
SequenceNumber next_sequence(kMaxSequenceNumber); SequenceNumber next_sequence(kMaxSequenceNumber);
default_cf_handle_ = new ColumnFamilyHandleImpl( default_cf_handle_ = new ColumnFamilyHandleImpl(
@ -529,6 +532,13 @@ Status DBImpl::RecoverLogFiles(const std::vector<uint64_t>& log_numbers,
bool flushed = false; bool flushed = false;
uint64_t corrupted_log_number = kMaxSequenceNumber; uint64_t corrupted_log_number = kMaxSequenceNumber;
for (auto log_number : log_numbers) { for (auto log_number : log_numbers) {
if (log_number < versions_->min_log_number_to_keep_2pc()) {
ROCKS_LOG_INFO(immutable_db_options_.info_log,
"Skipping log #%" PRIu64
" since it is older than min log to keep #%" PRIu64,
log_number, versions_->min_log_number_to_keep_2pc());
continue;
}
// The previous incarnation may not have written any MANIFEST // The previous incarnation may not have written any MANIFEST
// records after allocating this log number. So we manually // records after allocating this log number. So we manually
// update the file number allocation counter in VersionSet. // update the file number allocation counter in VersionSet.
@ -1024,8 +1034,17 @@ Status DBImpl::Open(const DBOptions& db_options, const std::string& dbname,
DBImpl* impl = new DBImpl(db_options, dbname, seq_per_batch); DBImpl* impl = new DBImpl(db_options, dbname, seq_per_batch);
s = impl->env_->CreateDirIfMissing(impl->immutable_db_options_.wal_dir); s = impl->env_->CreateDirIfMissing(impl->immutable_db_options_.wal_dir);
if (s.ok()) { if (s.ok()) {
for (auto db_path : impl->immutable_db_options_.db_paths) { std::vector<std::string> paths;
s = impl->env_->CreateDirIfMissing(db_path.path); for (auto& db_path : impl->immutable_db_options_.db_paths) {
paths.emplace_back(db_path.path);
}
for (auto& cf : column_families) {
for (auto& cf_path : cf.options.cf_paths) {
paths.emplace_back(cf_path.path);
}
}
for (auto& path : paths) {
s = impl->env_->CreateDirIfMissing(path);
if (!s.ok()) { if (!s.ok()) {
break; break;
} }
@ -1168,20 +1187,31 @@ Status DBImpl::Open(const DBOptions& db_options, const std::string& dbname,
impl->immutable_db_options_.sst_file_manager.get()); impl->immutable_db_options_.sst_file_manager.get());
if (s.ok() && sfm) { if (s.ok() && sfm) {
// Notify SstFileManager about all sst files that already exist in // Notify SstFileManager about all sst files that already exist in
// db_paths[0] when the DB is opened. // db_paths[0] and cf_paths[0] when the DB is opened.
auto& db_path = impl->immutable_db_options_.db_paths[0]; std::vector<std::string> paths;
paths.emplace_back(impl->immutable_db_options_.db_paths[0].path);
for (auto& cf : column_families) {
if (!cf.options.cf_paths.empty()) {
paths.emplace_back(cf.options.cf_paths[0].path);
}
}
// Remove duplicate paths.
std::sort(paths.begin(), paths.end());
paths.erase(std::unique(paths.begin(), paths.end()), paths.end());
for (auto& path : paths) {
std::vector<std::string> existing_files; std::vector<std::string> existing_files;
impl->immutable_db_options_.env->GetChildren(db_path.path, &existing_files); impl->immutable_db_options_.env->GetChildren(path, &existing_files);
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;
std::string file_path = db_path.path + "/" + file_name; std::string file_path = path + "/" + file_name;
if (ParseFileName(file_name, &file_number, &file_type) && if (ParseFileName(file_name, &file_number, &file_type) &&
file_type == kTableFile) { file_type == kTableFile) {
sfm->OnAddFile(file_path); sfm->OnAddFile(file_path);
} }
} }
} }
}
#endif // !ROCKSDB_LITE #endif // !ROCKSDB_LITE
if (s.ok()) { if (s.ok()) {

View File

@ -1013,28 +1013,34 @@ Status DBImpl::SwitchWAL(WriteContext* write_context) {
} }
auto oldest_alive_log = alive_log_files_.begin()->number; auto oldest_alive_log = alive_log_files_.begin()->number;
auto oldest_log_with_uncommited_prep = FindMinLogContainingOutstandingPrep(); bool flush_wont_release_oldest_log = false;
if (allow_2pc()) {
auto oldest_log_with_uncommited_prep =
logs_with_prep_tracker_.FindMinLogContainingOutstandingPrep();
if (allow_2pc() && assert(oldest_log_with_uncommited_prep == 0 ||
oldest_log_with_uncommited_prep > 0 && oldest_log_with_uncommited_prep >= oldest_alive_log);
oldest_log_with_uncommited_prep <= oldest_alive_log) { if (oldest_log_with_uncommited_prep > 0 &&
if (unable_to_flush_oldest_log_) { oldest_log_with_uncommited_prep == oldest_alive_log) {
if (unable_to_release_oldest_log_) {
// we already attempted to flush all column families dependent on // we already attempted to flush all column families dependent on
// the oldest alive log but the log still contained uncommited transactions. // the oldest alive log but the log still contained uncommited
// the oldest alive log STILL contains uncommited transaction so there // transactions so there is still nothing that we can do.
// is still nothing that we can do.
return status; return status;
} else { } else {
ROCKS_LOG_WARN( ROCKS_LOG_WARN(
immutable_db_options_.info_log, immutable_db_options_.info_log,
"Unable to release oldest log due to uncommited transaction"); "Unable to release oldest log due to uncommited transaction");
unable_to_flush_oldest_log_ = true; unable_to_release_oldest_log_ = true;
flush_wont_release_oldest_log = true;
} }
} else { }
}
if (!flush_wont_release_oldest_log) {
// we only mark this log as getting flushed if we have successfully // we only mark this log as getting flushed if we have successfully
// flushed all data in this log. If this log contains outstanding prepared // flushed all data in this log. If this log contains outstanding prepared
// transactions then we cannot flush this log until those transactions are commited. // transactions then we cannot flush this log until those transactions are commited.
unable_to_flush_oldest_log_ = false; unable_to_release_oldest_log_ = false;
alive_log_files_.begin()->getting_flushed = true; alive_log_files_.begin()->getting_flushed = true;
} }

View File

@ -36,7 +36,9 @@ class TestIterator : public InternalIterator {
valid_(false), valid_(false),
sequence_number_(0), sequence_number_(0),
iter_(0), iter_(0),
cmp(comparator) {} cmp(comparator) {
data_.reserve(16);
}
void AddPut(std::string argkey, std::string argvalue) { void AddPut(std::string argkey, std::string argvalue) {
Add(argkey, kTypeValue, argvalue); Add(argkey, kTypeValue, argvalue);
@ -2549,7 +2551,7 @@ TEST_F(DBIterWithMergeIterTest, InnerMergeIteratorDataRace1) {
// MergeIterator::Prev() realized the mem table iterator is at its end // MergeIterator::Prev() realized the mem table iterator is at its end
// and before an SeekToLast() is called. // and before an SeekToLast() is called.
rocksdb::SyncPoint::GetInstance()->SetCallBack( rocksdb::SyncPoint::GetInstance()->SetCallBack(
"MergeIterator::Prev:BeforeSeekToLast", "MergeIterator::Prev:BeforePrev",
[&](void* arg) { internal_iter2_->Add("z", kTypeValue, "7", 12u); }); [&](void* arg) { internal_iter2_->Add("z", kTypeValue, "7", 12u); });
rocksdb::SyncPoint::GetInstance()->EnableProcessing(); rocksdb::SyncPoint::GetInstance()->EnableProcessing();
@ -2585,7 +2587,7 @@ TEST_F(DBIterWithMergeIterTest, InnerMergeIteratorDataRace2) {
// mem table after MergeIterator::Prev() realized the mem tableiterator is at // mem table after MergeIterator::Prev() realized the mem tableiterator is at
// its end and before an SeekToLast() is called. // its end and before an SeekToLast() is called.
rocksdb::SyncPoint::GetInstance()->SetCallBack( rocksdb::SyncPoint::GetInstance()->SetCallBack(
"MergeIterator::Prev:BeforeSeekToLast", [&](void* arg) { "MergeIterator::Prev:BeforePrev", [&](void* arg) {
internal_iter2_->Add("z", kTypeValue, "7", 12u); internal_iter2_->Add("z", kTypeValue, "7", 12u);
internal_iter2_->Add("z", kTypeValue, "7", 11u); internal_iter2_->Add("z", kTypeValue, "7", 11u);
}); });
@ -2623,7 +2625,7 @@ TEST_F(DBIterWithMergeIterTest, InnerMergeIteratorDataRace3) {
// mem table after MergeIterator::Prev() realized the mem table iterator is at // mem table after MergeIterator::Prev() realized the mem table iterator is at
// its end and before an SeekToLast() is called. // its end and before an SeekToLast() is called.
rocksdb::SyncPoint::GetInstance()->SetCallBack( rocksdb::SyncPoint::GetInstance()->SetCallBack(
"MergeIterator::Prev:BeforeSeekToLast", [&](void* arg) { "MergeIterator::Prev:BeforePrev", [&](void* arg) {
internal_iter2_->Add("z", kTypeValue, "7", 16u, true); internal_iter2_->Add("z", kTypeValue, "7", 16u, true);
internal_iter2_->Add("z", kTypeValue, "7", 15u, true); internal_iter2_->Add("z", kTypeValue, "7", 15u, true);
internal_iter2_->Add("z", kTypeValue, "7", 14u, true); internal_iter2_->Add("z", kTypeValue, "7", 14u, true);

View File

@ -2043,6 +2043,43 @@ TEST_P(DBIteratorTest, CreationFailure) {
delete iter; delete iter;
} }
TEST_P(DBIteratorTest, UpperBoundWithChangeDirection) {
Options options = CurrentOptions();
options.max_sequential_skip_in_iterations = 3;
DestroyAndReopen(options);
// write a bunch of kvs to the database.
ASSERT_OK(Put("a", "1"));
ASSERT_OK(Put("y", "1"));
ASSERT_OK(Put("y1", "1"));
ASSERT_OK(Put("y2", "1"));
ASSERT_OK(Put("y3", "1"));
ASSERT_OK(Put("z", "1"));
ASSERT_OK(Flush());
ASSERT_OK(Put("a", "1"));
ASSERT_OK(Put("z", "1"));
ASSERT_OK(Put("bar", "1"));
ASSERT_OK(Put("foo", "1"));
std::string upper_bound = "x";
Slice ub_slice(upper_bound);
ReadOptions ro;
ro.iterate_upper_bound = &ub_slice;
ro.max_skippable_internal_keys = 1000;
Iterator* iter = NewIterator(ro);
iter->Seek("foo");
ASSERT_TRUE(iter->Valid());
ASSERT_EQ("foo", iter->key().ToString());
iter->Prev();
ASSERT_TRUE(iter->Valid());
ASSERT_OK(iter->status());
ASSERT_EQ("bar", iter->key().ToString());
delete iter;
}
TEST_P(DBIteratorTest, TableFilter) { TEST_P(DBIteratorTest, TableFilter) {
ASSERT_OK(Put("a", "1")); ASSERT_OK(Put("a", "1"));
dbfull()->Flush(FlushOptions()); dbfull()->Flush(FlushOptions());
@ -2109,6 +2146,47 @@ TEST_P(DBIteratorTest, TableFilter) {
} }
} }
TEST_P(DBIteratorTest, UpperBoundWithPrevReseek) {
Options options = CurrentOptions();
options.max_sequential_skip_in_iterations = 3;
DestroyAndReopen(options);
// write a bunch of kvs to the database.
ASSERT_OK(Put("a", "1"));
ASSERT_OK(Put("y", "1"));
ASSERT_OK(Put("z", "1"));
ASSERT_OK(Flush());
ASSERT_OK(Put("a", "1"));
ASSERT_OK(Put("z", "1"));
ASSERT_OK(Put("bar", "1"));
ASSERT_OK(Put("foo", "1"));
ASSERT_OK(Put("foo", "2"));
ASSERT_OK(Put("foo", "3"));
ASSERT_OK(Put("foo", "4"));
ASSERT_OK(Put("foo", "5"));
const Snapshot* snapshot = db_->GetSnapshot();
ASSERT_OK(Put("foo", "6"));
std::string upper_bound = "x";
Slice ub_slice(upper_bound);
ReadOptions ro;
ro.snapshot = snapshot;
ro.iterate_upper_bound = &ub_slice;
Iterator* iter = NewIterator(ro);
iter->SeekForPrev("goo");
ASSERT_TRUE(iter->Valid());
ASSERT_EQ("foo", iter->key().ToString());
iter->Prev();
ASSERT_TRUE(iter->Valid());
ASSERT_EQ("bar", iter->key().ToString());
delete iter;
db_->ReleaseSnapshot(snapshot);
}
TEST_P(DBIteratorTest, SkipStatistics) { TEST_P(DBIteratorTest, SkipStatistics) {
Options options = CurrentOptions(); Options options = CurrentOptions();
options.statistics = rocksdb::CreateDBStatistics(); options.statistics = rocksdb::CreateDBStatistics();

View File

@ -289,6 +289,8 @@ int main(int argc, char** argv) {
::testing::InitGoogleTest(&argc, argv); ::testing::InitGoogleTest(&argc, argv);
return RUN_ALL_TESTS(); return RUN_ALL_TESTS();
#else #else
(void) argc;
(void) argv;
return 0; return 0;
#endif #endif
} }

View File

@ -72,7 +72,7 @@ TEST_F(DBSSTTest, SSTsWithLdbSuffixHandling) {
ASSERT_GT(num_files, 0); ASSERT_GT(num_files, 0);
std::vector<std::string> filenames; std::vector<std::string> filenames;
GetSstFiles(dbname_, &filenames); GetSstFiles(env_, dbname_, &filenames);
int num_ldb_files = 0; int num_ldb_files = 0;
for (size_t i = 0; i < filenames.size(); ++i) { for (size_t i = 0; i < filenames.size(); ++i) {
if (i & 1) { if (i & 1) {

View File

@ -809,6 +809,8 @@ int main(int argc, char** argv) {
::testing::InitGoogleTest(&argc, argv); ::testing::InitGoogleTest(&argc, argv);
return RUN_ALL_TESTS(); return RUN_ALL_TESTS();
#else #else
(void) argc;
(void) argv;
return 0; return 0;
#endif #endif
} }

View File

@ -1112,7 +1112,7 @@ TEST_F(DBTest2, PresetCompressionDict) {
size_t out_bytes = 0; size_t out_bytes = 0;
std::vector<std::string> files; std::vector<std::string> files;
GetSstFiles(dbname_, &files); GetSstFiles(env_, dbname_, &files);
for (const auto& file : files) { for (const auto& file : files) {
uint64_t curr_bytes; uint64_t curr_bytes;
env_->GetFileSize(dbname_ + "/" + file, &curr_bytes); env_->GetFileSize(dbname_ + "/" + file, &curr_bytes);

View File

@ -575,9 +575,17 @@ void DBTestBase::DestroyAndReopen(const Options& options) {
ASSERT_OK(TryReopen(options)); ASSERT_OK(TryReopen(options));
} }
void DBTestBase::Destroy(const Options& options) { void DBTestBase::Destroy(const Options& options, bool delete_cf_paths) {
std::vector<ColumnFamilyDescriptor> column_families;
if (delete_cf_paths) {
for (size_t i = 0; i < handles_.size(); ++i) {
ColumnFamilyDescriptor cfdescriptor;
handles_[i]->GetDescriptor(&cfdescriptor);
column_families.push_back(cfdescriptor);
}
}
Close(); Close();
ASSERT_OK(DestroyDB(dbname_, options)); ASSERT_OK(DestroyDB(dbname_, options, column_families));
} }
Status DBTestBase::ReadOnlyReopen(const Options& options) { Status DBTestBase::ReadOnlyReopen(const Options& options) {
@ -1017,9 +1025,9 @@ std::string DBTestBase::DumpSSTableList() {
return property; return property;
} }
void DBTestBase::GetSstFiles(std::string path, void DBTestBase::GetSstFiles(Env* env, std::string path,
std::vector<std::string>* files) { std::vector<std::string>* files) {
env_->GetChildren(path, files); env->GetChildren(path, files);
files->erase( files->erase(
std::remove_if(files->begin(), files->end(), [](std::string name) { std::remove_if(files->begin(), files->end(), [](std::string name) {
@ -1031,7 +1039,7 @@ void DBTestBase::GetSstFiles(std::string path,
int DBTestBase::GetSstFileCount(std::string path) { int DBTestBase::GetSstFileCount(std::string path) {
std::vector<std::string> files; std::vector<std::string> files;
GetSstFiles(path, &files); DBTestBase::GetSstFiles(env_, path, &files);
return static_cast<int>(files.size()); return static_cast<int>(files.size());
} }

View File

@ -451,7 +451,8 @@ class SpecialEnv : public EnvWrapper {
return s; return s;
} }
Status NewSequentialFile(const std::string& f, unique_ptr<SequentialFile>* r, virtual Status NewSequentialFile(const std::string& f,
unique_ptr<SequentialFile>* r,
const EnvOptions& soptions) override { const EnvOptions& soptions) override {
class CountingFile : public SequentialFile { class CountingFile : public SequentialFile {
public: public:
@ -803,7 +804,7 @@ class DBTestBase : public testing::Test {
void DestroyAndReopen(const Options& options); void DestroyAndReopen(const Options& options);
void Destroy(const Options& options); void Destroy(const Options& options, bool delete_cf_paths = false);
Status ReadOnlyReopen(const Options& options); Status ReadOnlyReopen(const Options& options);
@ -903,7 +904,8 @@ class DBTestBase : public testing::Test {
std::string DumpSSTableList(); std::string DumpSSTableList();
void GetSstFiles(std::string path, std::vector<std::string>* files); static void GetSstFiles(Env* env, std::string path,
std::vector<std::string>* files);
int GetSstFileCount(std::string path); int GetSstFileCount(std::string path);

View File

@ -1343,6 +1343,146 @@ TEST_P(DBTestUniversalCompaction, UniversalCompactionFourPaths) {
Destroy(options); Destroy(options);
} }
TEST_P(DBTestUniversalCompaction, UniversalCompactionCFPathUse) {
Options options = CurrentOptions();
options.db_paths.emplace_back(dbname_, 300 * 1024);
options.db_paths.emplace_back(dbname_ + "_2", 300 * 1024);
options.db_paths.emplace_back(dbname_ + "_3", 500 * 1024);
options.db_paths.emplace_back(dbname_ + "_4", 1024 * 1024 * 1024);
options.memtable_factory.reset(
new SpecialSkipListFactory(KNumKeysByGenerateNewFile - 1));
options.compaction_style = kCompactionStyleUniversal;
options.compaction_options_universal.size_ratio = 5;
options.write_buffer_size = 111 << 10; // 114KB
options.arena_block_size = 4 << 10;
options.level0_file_num_compaction_trigger = 2;
options.num_levels = 1;
std::vector<Options> option_vector;
option_vector.emplace_back(options);
ColumnFamilyOptions cf_opt1(options), cf_opt2(options);
// Configure CF1 specific paths.
cf_opt1.cf_paths.emplace_back(dbname_ + "cf1", 300 * 1024);
cf_opt1.cf_paths.emplace_back(dbname_ + "cf1_2", 300 * 1024);
cf_opt1.cf_paths.emplace_back(dbname_ + "cf1_3", 500 * 1024);
cf_opt1.cf_paths.emplace_back(dbname_ + "cf1_4", 1024 * 1024 * 1024);
option_vector.emplace_back(DBOptions(options), cf_opt1);
CreateColumnFamilies({"one"},option_vector[1]);
// Configura CF2 specific paths.
cf_opt2.cf_paths.emplace_back(dbname_ + "cf2", 300 * 1024);
cf_opt2.cf_paths.emplace_back(dbname_ + "cf2_2", 300 * 1024);
cf_opt2.cf_paths.emplace_back(dbname_ + "cf2_3", 500 * 1024);
cf_opt2.cf_paths.emplace_back(dbname_ + "cf2_4", 1024 * 1024 * 1024);
option_vector.emplace_back(DBOptions(options), cf_opt2);
CreateColumnFamilies({"two"},option_vector[2]);
ReopenWithColumnFamilies({"default", "one", "two"}, option_vector);
Random rnd(301);
int key_idx = 0;
int key_idx1 = 0;
int key_idx2 = 0;
auto generate_file = [&]() {
GenerateNewFile(0, &rnd, &key_idx);
GenerateNewFile(1, &rnd, &key_idx1);
GenerateNewFile(2, &rnd, &key_idx2);
};
auto check_sstfilecount = [&](int path_id, int expected) {
ASSERT_EQ(expected, GetSstFileCount(options.db_paths[path_id].path));
ASSERT_EQ(expected, GetSstFileCount(cf_opt1.cf_paths[path_id].path));
ASSERT_EQ(expected, GetSstFileCount(cf_opt2.cf_paths[path_id].path));
};
auto check_getvalues = [&]() {
for (int i = 0; i < key_idx; i++) {
auto v = Get(0, Key(i));
ASSERT_NE(v, "NOT_FOUND");
ASSERT_TRUE(v.size() == 1 || v.size() == 990);
}
for (int i = 0; i < key_idx1; i++) {
auto v = Get(1, Key(i));
ASSERT_NE(v, "NOT_FOUND");
ASSERT_TRUE(v.size() == 1 || v.size() == 990);
}
for (int i = 0; i < key_idx2; i++) {
auto v = Get(2, Key(i));
ASSERT_NE(v, "NOT_FOUND");
ASSERT_TRUE(v.size() == 1 || v.size() == 990);
}
};
// First three 110KB files are not going to second path.
// After that, (100K, 200K)
for (int num = 0; num < 3; num++) {
generate_file();
}
// Another 110KB triggers a compaction to 400K file to second path
generate_file();
check_sstfilecount(2, 1);
// (1, 4)
generate_file();
check_sstfilecount(2, 1);
check_sstfilecount(0, 1);
// (1,1,4) -> (2, 4)
generate_file();
check_sstfilecount(2, 1);
check_sstfilecount(1, 1);
check_sstfilecount(0, 0);
// (1, 2, 4) -> (3, 4)
generate_file();
check_sstfilecount(2, 1);
check_sstfilecount(1, 1);
check_sstfilecount(0, 0);
// (1, 3, 4) -> (8)
generate_file();
check_sstfilecount(3, 1);
// (1, 8)
generate_file();
check_sstfilecount(3, 1);
check_sstfilecount(0, 1);
// (1, 1, 8) -> (2, 8)
generate_file();
check_sstfilecount(3, 1);
check_sstfilecount(1, 1);
// (1, 2, 8) -> (3, 8)
generate_file();
check_sstfilecount(3, 1);
check_sstfilecount(1, 1);
check_sstfilecount(0, 0);
// (1, 3, 8) -> (4, 8)
generate_file();
check_sstfilecount(2, 1);
check_sstfilecount(3, 1);
// (1, 4, 8) -> (5, 8)
generate_file();
check_sstfilecount(3, 1);
check_sstfilecount(2, 1);
check_sstfilecount(0, 0);
check_getvalues();
ReopenWithColumnFamilies({"default", "one", "two"}, option_vector);
check_getvalues();
Destroy(options, true);
}
TEST_P(DBTestUniversalCompaction, IncreaseUniversalCompactionNumLevels) { TEST_P(DBTestUniversalCompaction, IncreaseUniversalCompactionNumLevels) {
std::function<void(int)> verify_func = [&](int num_keys_in_db) { std::function<void(int)> verify_func = [&](int num_keys_in_db) {
std::string keys_in_db; std::string keys_in_db;
@ -1759,6 +1899,8 @@ int main(int argc, char** argv) {
::testing::InitGoogleTest(&argc, argv); ::testing::InitGoogleTest(&argc, argv);
return RUN_ALL_TESTS(); return RUN_ALL_TESTS();
#else #else
(void) argc;
(void) argv;
return 0; return 0;
#endif #endif
} }

View File

@ -20,6 +20,106 @@ class DBWALTest : public DBTestBase {
DBWALTest() : DBTestBase("/db_wal_test") {} DBWALTest() : DBTestBase("/db_wal_test") {}
}; };
// A SpecialEnv enriched to give more insight about deleted files
class EnrichedSpecialEnv : public SpecialEnv {
public:
explicit EnrichedSpecialEnv(Env* base) : SpecialEnv(base) {}
Status NewSequentialFile(const std::string& f, unique_ptr<SequentialFile>* r,
const EnvOptions& soptions) override {
InstrumentedMutexLock l(&env_mutex_);
if (f == skipped_wal) {
deleted_wal_reopened = true;
if (IsWAL(f) && largetest_deleted_wal.size() != 0 &&
f.compare(largetest_deleted_wal) <= 0) {
gap_in_wals = true;
}
}
return SpecialEnv::NewSequentialFile(f, r, soptions);
}
Status DeleteFile(const std::string& fname) override {
if (IsWAL(fname)) {
deleted_wal_cnt++;
InstrumentedMutexLock l(&env_mutex_);
// If this is the first WAL, remember its name and skip deleting it. We
// remember its name partly because the application might attempt to
// delete the file again.
if (skipped_wal.size() != 0 && skipped_wal != fname) {
if (largetest_deleted_wal.size() == 0 ||
largetest_deleted_wal.compare(fname) < 0) {
largetest_deleted_wal = fname;
}
} else {
skipped_wal = fname;
return Status::OK();
}
}
return SpecialEnv::DeleteFile(fname);
}
bool IsWAL(const std::string& fname) {
// printf("iswal %s\n", fname.c_str());
return fname.compare(fname.size() - 3, 3, "log") == 0;
}
InstrumentedMutex env_mutex_;
// the wal whose actual delete was skipped by the env
std::string skipped_wal = "";
// the largest WAL that was requested to be deleted
std::string largetest_deleted_wal = "";
// number of WALs that were successfully deleted
std::atomic<size_t> deleted_wal_cnt = {0};
// the WAL whose delete from fs was skipped is reopened during recovery
std::atomic<bool> deleted_wal_reopened = {false};
// whether a gap in the WALs was detected during recovery
std::atomic<bool> gap_in_wals = {false};
};
class DBWALTestWithEnrichedEnv : public DBTestBase {
public:
DBWALTestWithEnrichedEnv() : DBTestBase("/db_wal_test") {
enriched_env_ = new EnrichedSpecialEnv(env_->target());
auto options = CurrentOptions();
options.env = enriched_env_;
Reopen(options);
delete env_;
// to be deleted by the parent class
env_ = enriched_env_;
}
protected:
EnrichedSpecialEnv* enriched_env_;
};
// Test that the recovery would successfully avoid the gaps between the logs.
// One known scenario that could cause this is that the application issue the
// WAL deletion out of order. For the sake of simplicity in the test, here we
// create the gap by manipulating the env to skip deletion of the first WAL but
// not the ones after it.
TEST_F(DBWALTestWithEnrichedEnv, SkipDeletedWALs) {
auto options = last_options_;
// To cause frequent WAL deletion
options.write_buffer_size = 128;
Reopen(options);
WriteOptions writeOpt = WriteOptions();
for (int i = 0; i < 128 * 5; i++) {
ASSERT_OK(dbfull()->Put(writeOpt, "foo", "v1"));
}
FlushOptions fo;
fo.wait = true;
ASSERT_OK(db_->Flush(fo));
// some wals are deleted
ASSERT_NE(0, enriched_env_->deleted_wal_cnt);
// but not the first one
ASSERT_NE(0, enriched_env_->skipped_wal.size());
// Test that the WAL that was not deleted will be skipped during recovery
options = last_options_;
Reopen(options);
ASSERT_FALSE(enriched_env_->deleted_wal_reopened);
ASSERT_FALSE(enriched_env_->gap_in_wals);
}
TEST_F(DBWALTest, WAL) { TEST_F(DBWALTest, WAL) {
do { do {
CreateAndReopenWithCF({"pikachu"}, CurrentOptions()); CreateAndReopenWithCF({"pikachu"}, CurrentOptions());
@ -891,7 +991,7 @@ TEST_F(DBWALTest, kPointInTimeRecoveryCFConsistency) {
// Record the offset at this point // Record the offset at this point
Env* env = options.env; Env* env = options.env;
int wal_file_id = RecoveryTestHelper::kWALFileOffset + 1; uint64_t wal_file_id = dbfull()->TEST_LogfileNumber();
std::string fname = LogFileName(dbname_, wal_file_id); std::string fname = LogFileName(dbname_, wal_file_id);
uint64_t offset_to_corrupt; uint64_t offset_to_corrupt;
ASSERT_OK(env->GetFileSize(fname, &offset_to_corrupt)); ASSERT_OK(env->GetFileSize(fname, &offset_to_corrupt));

View File

@ -500,7 +500,7 @@ int main(int argc, char** argv) {
#else #else
#include <stdio.h> #include <stdio.h>
int main(int argc, char** argv) { int main(int /*argc*/, char** /*argv*/) {
fprintf(stderr, fprintf(stderr,
"SKIPPED as DBImpl::DeleteFile is not supported in ROCKSDB_LITE\n"); "SKIPPED as DBImpl::DeleteFile is not supported in ROCKSDB_LITE\n");
return 0; return 0;

View File

@ -94,7 +94,8 @@ Status ExternalSstFileIngestionJob::Prepare(
const std::string path_outside_db = f.external_file_path; const std::string path_outside_db = f.external_file_path;
const std::string path_inside_db = const std::string path_inside_db =
TableFileName(db_options_.db_paths, f.fd.GetNumber(), f.fd.GetPathId()); TableFileName(cfd_->ioptions()->cf_paths, f.fd.GetNumber(),
f.fd.GetPathId());
if (ingestion_options_.move_files) { if (ingestion_options_.move_files) {
status = env_->LinkFile(path_outside_db, path_inside_db); status = env_->LinkFile(path_outside_db, path_inside_db);

View File

@ -1413,8 +1413,12 @@ TEST_F(ExternalSSTFileTest, AddFileTrivialMoveBug) {
// fit in L3 but will overlap with compaction so will be added // fit in L3 but will overlap with compaction so will be added
// to L2 but a compaction will trivially move it to L3 // to L2 but a compaction will trivially move it to L3
// and break LSM consistency // and break LSM consistency
static std::atomic<bool> called = {false};
if (!called) {
called = true;
ASSERT_OK(dbfull()->SetOptions({{"max_bytes_for_level_base", "1"}})); ASSERT_OK(dbfull()->SetOptions({{"max_bytes_for_level_base", "1"}}));
ASSERT_OK(GenerateAndAddExternalFile(options, {15, 16}, 7)); ASSERT_OK(GenerateAndAddExternalFile(options, {15, 16}, 7));
}
}); });
rocksdb::SyncPoint::GetInstance()->EnableProcessing(); rocksdb::SyncPoint::GetInstance()->EnableProcessing();
@ -2002,7 +2006,7 @@ int main(int argc, char** argv) {
#else #else
#include <stdio.h> #include <stdio.h>
int main(int argc, char** argv) { int main(int /*argc*/, char** /*argv*/) {
fprintf(stderr, fprintf(stderr,
"SKIPPED as External SST File Writer and Ingestion are not supported " "SKIPPED as External SST File Writer and Ingestion are not supported "
"in ROCKSDB_LITE\n"); "in ROCKSDB_LITE\n");

View File

@ -185,7 +185,8 @@ void FlushJob::PickMemTable() {
base_->Ref(); // it is likely that we do not need this reference base_->Ref(); // it is likely that we do not need this reference
} }
Status FlushJob::Run(FileMetaData* file_meta) { Status FlushJob::Run(LogsWithPrepTracker* prep_tracker,
FileMetaData* file_meta) {
db_mutex_->AssertHeld(); db_mutex_->AssertHeld();
assert(pick_memtable_called); assert(pick_memtable_called);
AutoThreadOperationStageUpdater stage_run( AutoThreadOperationStageUpdater stage_run(
@ -226,7 +227,7 @@ Status FlushJob::Run(FileMetaData* file_meta) {
TEST_SYNC_POINT("FlushJob::InstallResults"); TEST_SYNC_POINT("FlushJob::InstallResults");
// Replace immutable memtable with the generated Table // Replace immutable memtable with the generated Table
s = cfd_->imm()->InstallMemtableFlushResults( s = cfd_->imm()->InstallMemtableFlushResults(
cfd_, mutable_cf_options_, mems_, versions_, db_mutex_, cfd_, mutable_cf_options_, mems_, prep_tracker, versions_, db_mutex_,
meta_.fd.GetNumber(), &job_context_->memtables_to_free, db_directory_, meta_.fd.GetNumber(), &job_context_->memtables_to_free, db_directory_,
log_buffer_); log_buffer_);
} }

View File

@ -22,6 +22,7 @@
#include "db/internal_stats.h" #include "db/internal_stats.h"
#include "db/job_context.h" #include "db/job_context.h"
#include "db/log_writer.h" #include "db/log_writer.h"
#include "db/logs_with_prep_tracker.h"
#include "db/memtable_list.h" #include "db/memtable_list.h"
#include "db/snapshot_impl.h" #include "db/snapshot_impl.h"
#include "db/version_edit.h" #include "db/version_edit.h"
@ -42,6 +43,7 @@
namespace rocksdb { namespace rocksdb {
class DBImpl;
class MemTable; class MemTable;
class SnapshotChecker; class SnapshotChecker;
class TableCache; class TableCache;
@ -71,7 +73,8 @@ class FlushJob {
// Require db_mutex held. // Require db_mutex held.
// Once PickMemTable() is called, either Run() or Cancel() has to be called. // Once PickMemTable() is called, either Run() or Cancel() has to be called.
void PickMemTable(); void PickMemTable();
Status Run(FileMetaData* file_meta = nullptr); Status Run(LogsWithPrepTracker* prep_tracker = nullptr,
FileMetaData* file_meta = nullptr);
void Cancel(); void Cancel();
TableProperties GetTableProperties() const { return table_properties_; } TableProperties GetTableProperties() const { return table_properties_; }

View File

@ -150,7 +150,7 @@ TEST_F(FlushJobTest, NonEmpty) {
FileMetaData fd; FileMetaData fd;
mutex_.Lock(); mutex_.Lock();
flush_job.PickMemTable(); flush_job.PickMemTable();
ASSERT_OK(flush_job.Run(&fd)); ASSERT_OK(flush_job.Run(nullptr, &fd));
mutex_.Unlock(); mutex_.Unlock();
db_options_.statistics->histogramData(FLUSH_TIME, &hist); db_options_.statistics->histogramData(FLUSH_TIME, &hist);
ASSERT_GT(hist.average, 0.0); ASSERT_GT(hist.average, 0.0);

View File

@ -19,8 +19,8 @@ class ColumnFamilyData;
namespace rocksdb { namespace rocksdb {
class MemTableList;
class DBImpl; class DBImpl;
class MemTableList;
// Config for retrieving a property's value. // Config for retrieving a property's value.
struct DBPropertyInfo { struct DBPropertyInfo {

View File

@ -92,11 +92,12 @@ struct JobContext {
// Structure to store information for candidate files to delete. // Structure to store information for candidate files to delete.
struct CandidateFileInfo { struct CandidateFileInfo {
std::string file_name; std::string file_name;
uint32_t path_id; std::string file_path;
CandidateFileInfo(std::string name, uint32_t path) CandidateFileInfo(std::string name, std::string path)
: file_name(std::move(name)), path_id(path) {} : file_name(std::move(name)), file_path(path) {}
bool operator==(const CandidateFileInfo& other) const { bool operator==(const CandidateFileInfo& other) const {
return file_name == other.file_name && path_id == other.path_id; return file_name == other.file_name &&
file_path == other.file_path;
} }
}; };
@ -113,7 +114,7 @@ struct JobContext {
std::vector<FileDescriptor> sst_live; std::vector<FileDescriptor> sst_live;
// a list of sst files that we need to delete // a list of sst files that we need to delete
std::vector<FileMetaData*> sst_delete_files; std::vector<ObsoleteFileInfo> sst_delete_files;
// a list of log files that we need to delete // a list of log files that we need to delete
std::vector<uint64_t> log_delete_files; std::vector<uint64_t> log_delete_files;

View File

@ -0,0 +1,67 @@
// 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).
//
#include "db/logs_with_prep_tracker.h"
#include "port/likely.h"
namespace rocksdb {
void LogsWithPrepTracker::MarkLogAsHavingPrepSectionFlushed(uint64_t log) {
assert(log != 0);
std::lock_guard<std::mutex> lock(prepared_section_completed_mutex_);
auto it = prepared_section_completed_.find(log);
if (UNLIKELY(it == prepared_section_completed_.end())) {
prepared_section_completed_[log] = 1;
} else {
it->second += 1;
}
}
void LogsWithPrepTracker::MarkLogAsContainingPrepSection(uint64_t log) {
assert(log != 0);
std::lock_guard<std::mutex> lock(logs_with_prep_mutex_);
auto rit = logs_with_prep_.rbegin();
bool updated = false;
// Most probably the last log is the one that is being marked for
// having a prepare section; so search from the end.
for (; rit != logs_with_prep_.rend() && rit->log >= log; ++rit) {
if (rit->log == log) {
rit->cnt++;
updated = true;
break;
}
}
if (!updated) {
// We are either at the start, or at a position with rit->log < log
logs_with_prep_.insert(rit.base(), {log, 1});
}
}
uint64_t LogsWithPrepTracker::FindMinLogContainingOutstandingPrep() {
std::lock_guard<std::mutex> lock(logs_with_prep_mutex_);
auto it = logs_with_prep_.begin();
// start with the smallest log
for (; it != logs_with_prep_.end();) {
auto min_log = it->log;
{
std::lock_guard<std::mutex> lock2(prepared_section_completed_mutex_);
auto completed_it = prepared_section_completed_.find(min_log);
if (completed_it == prepared_section_completed_.end() ||
completed_it->second < it->cnt) {
return min_log;
}
assert(completed_it != prepared_section_completed_.end() &&
completed_it->second == it->cnt);
prepared_section_completed_.erase(completed_it);
}
// erase from beginning in vector is not efficient but this function is not
// on the fast path.
it = logs_with_prep_.erase(it);
}
// no such log found
return 0;
}
} // namespace rocksdb

View File

@ -0,0 +1,61 @@
// 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).
//
#pragma once
#include <stdint.h>
#include <cassert>
#include <cstdlib>
#include <mutex>
#include <unordered_map>
#include <vector>
namespace rocksdb {
// This class is used to track the log files with outstanding prepare entries.
class LogsWithPrepTracker {
public:
// Called when a transaction prepared in `log` has been committed or aborted.
void MarkLogAsHavingPrepSectionFlushed(uint64_t log);
// Called when a transaction is prepared in `log`.
void MarkLogAsContainingPrepSection(uint64_t log);
// Return the earliest log file with outstanding prepare entries.
uint64_t FindMinLogContainingOutstandingPrep();
size_t TEST_PreparedSectionCompletedSize() {
return prepared_section_completed_.size();
}
size_t TEST_LogsWithPrepSize() { return logs_with_prep_.size(); }
private:
// REQUIRES: logs_with_prep_mutex_ held
//
// sorted list of log numbers still containing prepared data.
// this is used by FindObsoleteFiles to determine which
// flushed logs we must keep around because they still
// contain prepared data which has not been committed or rolled back
struct LogCnt {
uint64_t log; // the log number
uint64_t cnt; // number of prepared sections in the log
};
std::vector<LogCnt> logs_with_prep_;
std::mutex logs_with_prep_mutex_;
// REQUIRES: prepared_section_completed_mutex_ held
//
// to be used in conjunction with logs_with_prep_.
// once a transaction with data in log L is committed or rolled back
// rather than updating logs_with_prep_ directly we keep track of that
// in prepared_section_completed_ which maps LOG -> instance_count. This helps
// avoiding contention between a commit thread and the prepare threads.
//
// when trying to determine the minimum log still active we first
// consult logs_with_prep_. while that root value maps to
// an equal value in prepared_section_completed_ we erase the log from
// both logs_with_prep_ and prepared_section_completed_.
std::unordered_map<uint64_t, uint64_t> prepared_section_completed_;
std::mutex prepared_section_completed_mutex_;
};
} // namespace rocksdb

View File

@ -479,12 +479,12 @@ bool MemTable::Add(SequenceNumber s, ValueType type,
insert_with_hint_prefix_extractor_->InDomain(key_slice)) { insert_with_hint_prefix_extractor_->InDomain(key_slice)) {
Slice prefix = insert_with_hint_prefix_extractor_->Transform(key_slice); Slice prefix = insert_with_hint_prefix_extractor_->Transform(key_slice);
bool res = table->InsertKeyWithHint(handle, &insert_hints_[prefix]); bool res = table->InsertKeyWithHint(handle, &insert_hints_[prefix]);
if (!res) { if (UNLIKELY(!res)) {
return res; return res;
} }
} else { } else {
bool res = table->InsertKey(handle); bool res = table->InsertKey(handle);
if (!res) { if (UNLIKELY(!res)) {
return res; return res;
} }
} }
@ -520,7 +520,7 @@ bool MemTable::Add(SequenceNumber s, ValueType type,
UpdateFlushState(); UpdateFlushState();
} else { } else {
bool res = table->InsertKeyConcurrently(handle); bool res = table->InsertKeyConcurrently(handle);
if (!res) { if (UNLIKELY(!res)) {
return res; return res;
} }

View File

@ -12,6 +12,7 @@
#include <inttypes.h> #include <inttypes.h>
#include <limits> #include <limits>
#include <string> #include <string>
#include "db/db_impl.h"
#include "db/memtable.h" #include "db/memtable.h"
#include "db/version_set.h" #include "db/version_set.h"
#include "monitoring/thread_status_util.h" #include "monitoring/thread_status_util.h"
@ -322,9 +323,10 @@ void MemTableList::RollbackMemtableFlush(const autovector<MemTable*>& mems,
// Record a successful flush in the manifest file // Record a successful flush in the manifest file
Status MemTableList::InstallMemtableFlushResults( Status MemTableList::InstallMemtableFlushResults(
ColumnFamilyData* cfd, const MutableCFOptions& mutable_cf_options, ColumnFamilyData* cfd, const MutableCFOptions& mutable_cf_options,
const autovector<MemTable*>& mems, VersionSet* vset, InstrumentedMutex* mu, const autovector<MemTable*>& mems, LogsWithPrepTracker* prep_tracker,
uint64_t file_number, autovector<MemTable*>* to_delete, VersionSet* vset, InstrumentedMutex* mu, uint64_t file_number,
Directory* db_directory, LogBuffer* log_buffer) { autovector<MemTable*>* to_delete, Directory* db_directory,
LogBuffer* log_buffer) {
AutoThreadOperationStageUpdater stage_updater( AutoThreadOperationStageUpdater stage_updater(
ThreadStatus::STAGE_MEMTABLE_INSTALL_FLUSH_RESULTS); ThreadStatus::STAGE_MEMTABLE_INSTALL_FLUSH_RESULTS);
mu->AssertHeld(); mu->AssertHeld();
@ -361,6 +363,7 @@ Status MemTableList::InstallMemtableFlushResults(
uint64_t batch_file_number = 0; uint64_t batch_file_number = 0;
size_t batch_count = 0; size_t batch_count = 0;
autovector<VersionEdit*> edit_list; autovector<VersionEdit*> edit_list;
autovector<MemTable*> memtables_to_flush;
// enumerate from the last (earliest) element to see how many batch finished // enumerate from the last (earliest) element to see how many batch finished
for (auto it = memlist.rbegin(); it != memlist.rend(); ++it) { for (auto it = memlist.rbegin(); it != memlist.rend(); ++it) {
MemTable* m = *it; MemTable* m = *it;
@ -373,11 +376,20 @@ Status MemTableList::InstallMemtableFlushResults(
"[%s] Level-0 commit table #%" PRIu64 " started", "[%s] Level-0 commit table #%" PRIu64 " started",
cfd->GetName().c_str(), m->file_number_); cfd->GetName().c_str(), m->file_number_);
edit_list.push_back(&m->edit_); edit_list.push_back(&m->edit_);
memtables_to_flush.push_back(m);
} }
batch_count++; batch_count++;
} }
if (batch_count > 0) { if (batch_count > 0) {
if (vset->db_options()->allow_2pc) {
assert(edit_list.size() > 0);
// We piggyback the information of earliest log file to keep in the
// manifest entry for the last file flushed.
edit_list.back()->SetMinLogNumberToKeep(PrecomputeMinLogNumberToKeep(
vset, *cfd, edit_list, memtables_to_flush, prep_tracker));
}
// this can release and reacquire the mutex. // this can release and reacquire the mutex.
s = vset->LogAndApply(cfd, mutable_cf_options, edit_list, mu, s = vset->LogAndApply(cfd, mutable_cf_options, edit_list, mu,
db_directory); db_directory);
@ -468,13 +480,21 @@ void MemTableList::InstallNewVersion() {
} }
} }
uint64_t MemTableList::GetMinLogContainingPrepSection() { uint64_t MemTableList::PrecomputeMinLogContainingPrepSection(
const autovector<MemTable*>& memtables_to_flush) {
uint64_t min_log = 0; uint64_t min_log = 0;
for (auto& m : current_->memlist_) { for (auto& m : current_->memlist_) {
// this mem has been flushed it no longer // Assume the list is very short, we can live with O(m*n). We can optimize
// needs to hold on the its prep section // if the performance has some problem.
if (m->flush_completed_) { bool should_skip = false;
for (MemTable* m_to_flush : memtables_to_flush) {
if (m == m_to_flush) {
should_skip = true;
break;
}
}
if (should_skip) {
continue; continue;
} }

View File

@ -13,6 +13,7 @@
#include <vector> #include <vector>
#include "db/dbformat.h" #include "db/dbformat.h"
#include "db/logs_with_prep_tracker.h"
#include "db/memtable.h" #include "db/memtable.h"
#include "db/range_del_aggregator.h" #include "db/range_del_aggregator.h"
#include "monitoring/instrumented_mutex.h" #include "monitoring/instrumented_mutex.h"
@ -210,9 +211,10 @@ class MemTableList {
// Commit a successful flush in the manifest file // Commit a successful flush in the manifest file
Status InstallMemtableFlushResults( Status InstallMemtableFlushResults(
ColumnFamilyData* cfd, const MutableCFOptions& mutable_cf_options, ColumnFamilyData* cfd, const MutableCFOptions& mutable_cf_options,
const autovector<MemTable*>& m, VersionSet* vset, InstrumentedMutex* mu, const autovector<MemTable*>& m, LogsWithPrepTracker* prep_tracker,
uint64_t file_number, autovector<MemTable*>* to_delete, VersionSet* vset, InstrumentedMutex* mu, uint64_t file_number,
Directory* db_directory, LogBuffer* log_buffer); autovector<MemTable*>* to_delete, Directory* db_directory,
LogBuffer* log_buffer);
// New memtables are inserted at the front of the list. // New memtables are inserted at the front of the list.
// Takes ownership of the referenced held on *m by the caller of Add(). // Takes ownership of the referenced held on *m by the caller of Add().
@ -243,7 +245,10 @@ class MemTableList {
size_t* current_memory_usage() { return &current_memory_usage_; } size_t* current_memory_usage() { return &current_memory_usage_; }
uint64_t GetMinLogContainingPrepSection(); // Returns the min log containing the prep section after memtables listsed in
// `memtables_to_flush` are flushed and their status is persisted in manifest.
uint64_t PrecomputeMinLogContainingPrepSection(
const autovector<MemTable*>& memtables_to_flush);
uint64_t GetEarliestMemTableID() const { uint64_t GetEarliestMemTableID() const {
auto& memlist = current_->memlist_; auto& memlist = current_->memlist_;

View File

@ -82,10 +82,10 @@ class MemTableListTest : public testing::Test {
// Create dummy mutex. // Create dummy mutex.
InstrumentedMutex mutex; InstrumentedMutex mutex;
InstrumentedMutexLock l(&mutex); InstrumentedMutexLock l(&mutex);
LogsWithPrepTracker dummy_prep_tracker;
return list->InstallMemtableFlushResults(cfd, mutable_cf_options, m, return list->InstallMemtableFlushResults(
&versions, &mutex, 1, to_delete, cfd, mutable_cf_options, m, &dummy_prep_tracker, &versions, &mutex, 1,
nullptr, &log_buffer); to_delete, nullptr, &log_buffer);
} }
}; };

View File

@ -208,7 +208,7 @@ int main(int argc, char** argv) {
#else #else
#include <stdio.h> #include <stdio.h>
int main(int argc, char** argv) { int main(int /*argc*/, char** /*argv*/) {
fprintf(stderr, fprintf(stderr,
"SKIPPED as DBImpl::DeleteFile is not supported in ROCKSDB_LITE\n"); "SKIPPED as DBImpl::DeleteFile is not supported in ROCKSDB_LITE\n");
return 0; return 0;

View File

@ -112,7 +112,7 @@ int main(int argc, char** argv) {
#include <cstdio> #include <cstdio>
int main(int argc, char** argv) { int main(int /*argc*/, char** /*argv*/) {
printf("Skipped as Options file is not supported in RocksDBLite.\n"); printf("Skipped as Options file is not supported in RocksDBLite.\n");
return 0; return 0;
} }

View File

@ -1170,7 +1170,7 @@ int main(int argc, char** argv) {
#else #else
#include <stdio.h> #include <stdio.h>
int main(int argc, char** argv) { int main(int /*argc*/, char** /*argv*/) {
fprintf(stderr, "SKIPPED as plain table is not supported in ROCKSDB_LITE\n"); fprintf(stderr, "SKIPPED as plain table is not supported in ROCKSDB_LITE\n");
return 0; return 0;
} }

View File

@ -887,7 +887,7 @@ int main(int argc, char** argv) {
#else #else
#include <stdio.h> #include <stdio.h>
int main(int argc, char** argv) { int main(int /*argc*/, char** /*argv*/) {
fprintf(stderr, fprintf(stderr,
"SKIPPED as HashSkipList and HashLinkList are not supported in " "SKIPPED as HashSkipList and HashLinkList are not supported in "
"ROCKSDB_LITE\n"); "ROCKSDB_LITE\n");

View File

@ -101,10 +101,12 @@ class Repairer {
db_options_(SanitizeOptions(dbname_, db_options)), db_options_(SanitizeOptions(dbname_, db_options)),
immutable_db_options_(ImmutableDBOptions(db_options_)), immutable_db_options_(ImmutableDBOptions(db_options_)),
icmp_(default_cf_opts.comparator), icmp_(default_cf_opts.comparator),
default_cf_opts_(default_cf_opts), default_cf_opts_(
SanitizeOptions(immutable_db_options_, default_cf_opts)),
default_cf_iopts_( default_cf_iopts_(
ImmutableCFOptions(immutable_db_options_, default_cf_opts)), ImmutableCFOptions(immutable_db_options_, default_cf_opts_)),
unknown_cf_opts_(unknown_cf_opts), unknown_cf_opts_(
SanitizeOptions(immutable_db_options_, unknown_cf_opts)),
create_unknown_cfs_(create_unknown_cfs), create_unknown_cfs_(create_unknown_cfs),
raw_table_cache_( raw_table_cache_(
// TableCache can be small since we expect each table to be opened // TableCache can be small since we expect each table to be opened

View File

@ -356,7 +356,7 @@ int main(int argc, char** argv) {
#else #else
#include <stdio.h> #include <stdio.h>
int main(int argc, char** argv) { int main(int /*argc*/, char** /*argv*/) {
fprintf(stderr, "SKIPPED as RepairDB is not supported in ROCKSDB_LITE\n"); fprintf(stderr, "SKIPPED as RepairDB is not supported in ROCKSDB_LITE\n");
return 0; return 0;
} }

View File

@ -21,6 +21,10 @@ class SnapshotList;
class SnapshotImpl : public Snapshot { class SnapshotImpl : public Snapshot {
public: public:
SequenceNumber number_; // const after creation SequenceNumber number_; // const after creation
// It indicates the smallest uncommitted data at the time the snapshot was
// taken. This is currently used by WritePrepared transactions to limit the
// scope of queries to IsInSnpashot.
SequenceNumber min_uncommitted_ = 0;
virtual SequenceNumber GetSequenceNumber() const override { return number_; } virtual SequenceNumber GetSequenceNumber() const override { return number_; }
@ -56,8 +60,8 @@ class SnapshotList {
SnapshotImpl* oldest() const { assert(!empty()); return list_.next_; } SnapshotImpl* oldest() const { assert(!empty()); return list_.next_; }
SnapshotImpl* newest() const { assert(!empty()); return list_.prev_; } SnapshotImpl* newest() const { assert(!empty()); return list_.prev_; }
const SnapshotImpl* New(SnapshotImpl* s, SequenceNumber seq, SnapshotImpl* New(SnapshotImpl* s, SequenceNumber seq, uint64_t unix_time,
uint64_t unix_time, bool is_write_conflict_boundary) { bool is_write_conflict_boundary) {
s->number_ = seq; s->number_ = seq;
s->unix_time_ = unix_time; s->unix_time_ = unix_time;
s->is_write_conflict_boundary_ = is_write_conflict_boundary; s->is_write_conflict_boundary_ = is_write_conflict_boundary;

View File

@ -92,7 +92,7 @@ Status TableCache::GetTableReader(
bool skip_filters, int level, bool prefetch_index_and_filter_in_cache, bool skip_filters, int level, bool prefetch_index_and_filter_in_cache,
bool for_compaction) { bool for_compaction) {
std::string fname = std::string fname =
TableFileName(ioptions_.db_paths, fd.GetNumber(), fd.GetPathId()); TableFileName(ioptions_.cf_paths, fd.GetNumber(), fd.GetPathId());
unique_ptr<RandomAccessFile> file; unique_ptr<RandomAccessFile> file;
Status s = ioptions_.env->NewRandomAccessFile(fname, &file, env_options); Status s = ioptions_.env->NewRandomAccessFile(fname, &file, env_options);

View File

@ -30,6 +30,7 @@ enum Tag {
kNewFile = 7, kNewFile = 7,
// 8 was used for large value refs // 8 was used for large value refs
kPrevLogNumber = 9, kPrevLogNumber = 9,
kMinLogNumberToKeep = 10,
// these are new formats divergent from open source leveldb // these are new formats divergent from open source leveldb
kNewFile2 = 100, kNewFile2 = 100,
@ -44,6 +45,11 @@ enum Tag {
enum CustomTag { enum CustomTag {
kTerminate = 1, // The end of customized fields kTerminate = 1, // The end of customized fields
kNeedCompaction = 2, kNeedCompaction = 2,
// Since Manifest is not entirely currently forward-compatible, and the only
// forward-compatbile part is the CutsomtTag of kNewFile, we currently encode
// kMinLogNumberToKeep as part of a CustomTag as a hack. This should be
// removed when manifest becomes forward-comptabile.
kMinLogNumberToKeepHack = 3,
kPathId = 65, kPathId = 65,
}; };
// If this bit for the custom tag is set, opening DB should fail if // If this bit for the custom tag is set, opening DB should fail if
@ -63,12 +69,14 @@ void VersionEdit::Clear() {
last_sequence_ = 0; last_sequence_ = 0;
next_file_number_ = 0; next_file_number_ = 0;
max_column_family_ = 0; max_column_family_ = 0;
min_log_number_to_keep_ = 0;
has_comparator_ = false; has_comparator_ = false;
has_log_number_ = false; has_log_number_ = false;
has_prev_log_number_ = false; has_prev_log_number_ = false;
has_next_file_number_ = false; has_next_file_number_ = false;
has_last_sequence_ = false; has_last_sequence_ = false;
has_max_column_family_ = false; has_max_column_family_ = false;
has_min_log_number_to_keep_ = false;
deleted_files_.clear(); deleted_files_.clear();
new_files_.clear(); new_files_.clear();
column_family_ = 0; column_family_ = 0;
@ -97,19 +105,19 @@ bool VersionEdit::EncodeTo(std::string* dst) const {
if (has_max_column_family_) { if (has_max_column_family_) {
PutVarint32Varint32(dst, kMaxColumnFamily, max_column_family_); PutVarint32Varint32(dst, kMaxColumnFamily, max_column_family_);
} }
for (const auto& deleted : deleted_files_) { for (const auto& deleted : deleted_files_) {
PutVarint32Varint32Varint64(dst, kDeletedFile, deleted.first /* level */, PutVarint32Varint32Varint64(dst, kDeletedFile, deleted.first /* level */,
deleted.second /* file number */); deleted.second /* file number */);
} }
bool min_log_num_written = false;
for (size_t i = 0; i < new_files_.size(); i++) { for (size_t i = 0; i < new_files_.size(); i++) {
const FileMetaData& f = new_files_[i].second; const FileMetaData& f = new_files_[i].second;
if (!f.smallest.Valid() || !f.largest.Valid()) { if (!f.smallest.Valid() || !f.largest.Valid()) {
return false; return false;
} }
bool has_customized_fields = false; bool has_customized_fields = false;
if (f.marked_for_compaction) { if (f.marked_for_compaction || has_min_log_number_to_keep_) {
PutVarint32(dst, kNewFile4); PutVarint32(dst, kNewFile4);
has_customized_fields = true; has_customized_fields = true;
} else if (f.fd.GetPathId() == 0) { } else if (f.fd.GetPathId() == 0) {
@ -165,6 +173,13 @@ bool VersionEdit::EncodeTo(std::string* dst) const {
char p = static_cast<char>(1); char p = static_cast<char>(1);
PutLengthPrefixedSlice(dst, Slice(&p, 1)); PutLengthPrefixedSlice(dst, Slice(&p, 1));
} }
if (has_min_log_number_to_keep_ && !min_log_num_written) {
PutVarint32(dst, CustomTag::kMinLogNumberToKeepHack);
std::string varint_log_number;
PutFixed64(&varint_log_number, min_log_number_to_keep_);
PutLengthPrefixedSlice(dst, Slice(varint_log_number));
min_log_num_written = true;
}
TEST_SYNC_POINT_CALLBACK("VersionEdit::EncodeTo:NewFile4:CustomizeFields", TEST_SYNC_POINT_CALLBACK("VersionEdit::EncodeTo:NewFile4:CustomizeFields",
dst); dst);
@ -218,6 +233,9 @@ const char* VersionEdit::DecodeNewFile4From(Slice* input) {
uint64_t number; uint64_t number;
uint32_t path_id = 0; uint32_t path_id = 0;
uint64_t file_size; uint64_t file_size;
// Since this is the only forward-compatible part of the code, we hack new
// extension into this record. When we do, we set this boolean to distinguish
// the record from the normal NewFile records.
if (GetLevel(input, &level, &msg) && GetVarint64(input, &number) && if (GetLevel(input, &level, &msg) && GetVarint64(input, &number) &&
GetVarint64(input, &file_size) && GetInternalKey(input, &f.smallest) && GetVarint64(input, &file_size) && GetInternalKey(input, &f.smallest) &&
GetInternalKey(input, &f.largest) && GetInternalKey(input, &f.largest) &&
@ -252,6 +270,14 @@ const char* VersionEdit::DecodeNewFile4From(Slice* input) {
} }
f.marked_for_compaction = (field[0] == 1); f.marked_for_compaction = (field[0] == 1);
break; break;
case kMinLogNumberToKeepHack:
// This is a hack to encode kMinLogNumberToKeep in a
// forward-compatbile fashion.
if (!GetFixed64(&field, &min_log_number_to_keep_)) {
return "deleted log number malformatted";
}
has_min_log_number_to_keep_ = true;
break;
default: default:
if ((custom_tag & kCustomTagNonSafeIgnoreMask) != 0) { if ((custom_tag & kCustomTagNonSafeIgnoreMask) != 0) {
// Should not proceed if cannot understand it // Should not proceed if cannot understand it
@ -331,6 +357,14 @@ Status VersionEdit::DecodeFrom(const Slice& src) {
} }
break; break;
case kMinLogNumberToKeep:
if (GetVarint64(&input, &min_log_number_to_keep_)) {
has_min_log_number_to_keep_ = true;
} else {
msg = "min log number to kee";
}
break;
case kCompactPointer: case kCompactPointer:
if (GetLevel(&input, &level, &msg) && if (GetLevel(&input, &level, &msg) &&
GetInternalKey(&input, &key)) { GetInternalKey(&input, &key)) {
@ -475,6 +509,10 @@ std::string VersionEdit::DebugString(bool hex_key) const {
r.append("\n NextFileNumber: "); r.append("\n NextFileNumber: ");
AppendNumberTo(&r, next_file_number_); AppendNumberTo(&r, next_file_number_);
} }
if (has_min_log_number_to_keep_) {
r.append("\n MinLogNumberToKeep: ");
AppendNumberTo(&r, min_log_number_to_keep_);
}
if (has_last_sequence_) { if (has_last_sequence_) {
r.append("\n LastSeq: "); r.append("\n LastSeq: ");
AppendNumberTo(&r, last_sequence_); AppendNumberTo(&r, last_sequence_);
@ -582,6 +620,9 @@ std::string VersionEdit::DebugJSON(int edit_num, bool hex_key) const {
if (has_max_column_family_) { if (has_max_column_family_) {
jw << "MaxColumnFamily" << max_column_family_; jw << "MaxColumnFamily" << max_column_family_;
} }
if (has_min_log_number_to_keep_) {
jw << "MinLogNumberToKeep" << min_log_number_to_keep_;
}
jw.EndObject(); jw.EndObject();

View File

@ -199,6 +199,14 @@ class VersionEdit {
has_max_column_family_ = true; has_max_column_family_ = true;
max_column_family_ = max_column_family; max_column_family_ = max_column_family;
} }
void SetMinLogNumberToKeep(uint64_t num) {
has_min_log_number_to_keep_ = true;
min_log_number_to_keep_ = num;
}
bool has_log_number() { return has_log_number_; }
uint64_t log_number() { return log_number_; }
// Add the specified file at the specified number. // Add the specified file at the specified number.
// REQUIRES: This version has not been saved (see VersionSet::SaveTo) // REQUIRES: This version has not been saved (see VersionSet::SaveTo)
@ -285,6 +293,8 @@ class VersionEdit {
uint64_t prev_log_number_; uint64_t prev_log_number_;
uint64_t next_file_number_; uint64_t next_file_number_;
uint32_t max_column_family_; uint32_t max_column_family_;
// The most recent WAL log number that is deleted
uint64_t min_log_number_to_keep_;
SequenceNumber last_sequence_; SequenceNumber last_sequence_;
bool has_comparator_; bool has_comparator_;
bool has_log_number_; bool has_log_number_;
@ -292,6 +302,7 @@ class VersionEdit {
bool has_next_file_number_; bool has_next_file_number_;
bool has_last_sequence_; bool has_last_sequence_;
bool has_max_column_family_; bool has_max_column_family_;
bool has_min_log_number_to_keep_;
DeletedFileSet deleted_files_; DeletedFileSet deleted_files_;
std::vector<std::pair<int, FileMetaData>> new_files_; std::vector<std::pair<int, FileMetaData>> new_files_;

View File

@ -181,6 +181,16 @@ TEST_F(VersionEditTest, ColumnFamilyTest) {
TestEncodeDecode(edit); TestEncodeDecode(edit);
} }
TEST_F(VersionEditTest, MinLogNumberToKeep) {
VersionEdit edit;
edit.SetMinLogNumberToKeep(13);
TestEncodeDecode(edit);
edit.Clear();
edit.SetMinLogNumberToKeep(23);
TestEncodeDecode(edit);
}
} // namespace rocksdb } // namespace rocksdb
int main(int argc, char** argv) { int main(int argc, char** argv) {

View File

@ -355,7 +355,11 @@ Version::~Version() {
assert(f->refs > 0); assert(f->refs > 0);
f->refs--; f->refs--;
if (f->refs <= 0) { if (f->refs <= 0) {
vset_->obsolete_files_.push_back(f); assert(cfd_ != nullptr);
uint32_t path_id = f->fd.GetPathId();
assert(path_id < cfd_->ioptions()->cf_paths.size());
vset_->obsolete_files_.push_back(
ObsoleteFileInfo(f, cfd_->ioptions()->cf_paths[path_id].path));
} }
} }
} }
@ -756,7 +760,7 @@ Status Version::GetTableProperties(std::shared_ptr<const TableProperties>* tp,
file_name = *fname; file_name = *fname;
} else { } else {
file_name = file_name =
TableFileName(vset_->db_options_->db_paths, file_meta->fd.GetNumber(), TableFileName(ioptions->cf_paths, file_meta->fd.GetNumber(),
file_meta->fd.GetPathId()); file_meta->fd.GetPathId());
} }
s = ioptions->env->NewRandomAccessFile(file_name, &file, env_options_); s = ioptions->env->NewRandomAccessFile(file_name, &file, env_options_);
@ -797,7 +801,7 @@ Status Version::GetPropertiesOfAllTables(TablePropertiesCollection* props,
int level) { int level) {
for (const auto& file_meta : storage_info_.files_[level]) { for (const auto& file_meta : storage_info_.files_[level]) {
auto fname = auto fname =
TableFileName(vset_->db_options_->db_paths, file_meta->fd.GetNumber(), TableFileName(cfd_->ioptions()->cf_paths, file_meta->fd.GetNumber(),
file_meta->fd.GetPathId()); file_meta->fd.GetPathId());
// 1. If the table is already present in table cache, load table // 1. If the table is already present in table cache, load table
// properties from there. // properties from there.
@ -825,7 +829,7 @@ Status Version::GetPropertiesOfTablesInRange(
false); false);
for (const auto& file_meta : files) { for (const auto& file_meta : files) {
auto fname = auto fname =
TableFileName(vset_->db_options_->db_paths, TableFileName(cfd_->ioptions()->cf_paths,
file_meta->fd.GetNumber(), file_meta->fd.GetPathId()); file_meta->fd.GetNumber(), file_meta->fd.GetPathId());
if (props->count(fname) == 0) { if (props->count(fname) == 0) {
// 1. If the table is already present in table cache, load table // 1. If the table is already present in table cache, load table
@ -897,11 +901,11 @@ void Version::GetColumnFamilyMetaData(ColumnFamilyMetaData* cf_meta) {
for (const auto& file : vstorage->LevelFiles(level)) { for (const auto& file : vstorage->LevelFiles(level)) {
uint32_t path_id = file->fd.GetPathId(); uint32_t path_id = file->fd.GetPathId();
std::string file_path; std::string file_path;
if (path_id < ioptions->db_paths.size()) { if (path_id < ioptions->cf_paths.size()) {
file_path = ioptions->db_paths[path_id].path; file_path = ioptions->cf_paths[path_id].path;
} else { } else {
assert(!ioptions->db_paths.empty()); assert(!ioptions->cf_paths.empty());
file_path = ioptions->db_paths.back().path; file_path = ioptions->cf_paths.back().path;
} }
files.emplace_back( files.emplace_back(
MakeTableFileName("", file->fd.GetNumber()), file_path, MakeTableFileName("", file->fd.GetNumber()), file_path,
@ -1667,6 +1671,9 @@ void VersionStorageInfo::ComputeCompactionScore(
} }
ComputeFilesMarkedForCompaction(); ComputeFilesMarkedForCompaction();
ComputeBottommostFilesMarkedForCompaction(); ComputeBottommostFilesMarkedForCompaction();
if (immutable_cf_options.ttl > 0) {
ComputeExpiredTtlFiles(immutable_cf_options);
}
EstimateCompactionBytesNeeded(mutable_cf_options); EstimateCompactionBytesNeeded(mutable_cf_options);
} }
@ -1693,6 +1700,34 @@ void VersionStorageInfo::ComputeFilesMarkedForCompaction() {
} }
} }
void VersionStorageInfo::ComputeExpiredTtlFiles(
const ImmutableCFOptions& ioptions) {
assert(ioptions.ttl > 0);
expired_ttl_files_.clear();
int64_t _current_time;
auto status = ioptions.env->GetCurrentTime(&_current_time);
if (!status.ok()) {
return;
}
const uint64_t current_time = static_cast<uint64_t>(_current_time);
for (int level = 0; level < num_levels() - 1; level++) {
for (auto f : files_[level]) {
if (!f->being_compacted && f->fd.table_reader != nullptr &&
f->fd.table_reader->GetTableProperties() != nullptr) {
auto creation_time =
f->fd.table_reader->GetTableProperties()->creation_time;
if (creation_time > 0 &&
creation_time < (current_time - ioptions.ttl)) {
expired_ttl_files_.emplace_back(level, f);
}
}
}
}
}
namespace { namespace {
// used to sort files by size // used to sort files by size
@ -2623,16 +2658,16 @@ struct VersionSet::ManifestWriter {
}; };
VersionSet::VersionSet(const std::string& dbname, VersionSet::VersionSet(const std::string& dbname,
const ImmutableDBOptions* db_options, const ImmutableDBOptions* _db_options,
const EnvOptions& storage_options, Cache* table_cache, const EnvOptions& storage_options, Cache* table_cache,
WriteBufferManager* write_buffer_manager, WriteBufferManager* write_buffer_manager,
WriteController* write_controller) WriteController* write_controller)
: column_family_set_( : column_family_set_(
new ColumnFamilySet(dbname, db_options, storage_options, table_cache, new ColumnFamilySet(dbname, _db_options, storage_options, table_cache,
write_buffer_manager, write_controller)), write_buffer_manager, write_controller)),
env_(db_options->env), env_(_db_options->env),
dbname_(dbname), dbname_(dbname),
db_options_(db_options), db_options_(_db_options),
next_file_number_(2), next_file_number_(2),
manifest_file_number_(0), // Filled by Recover() manifest_file_number_(0), // Filled by Recover()
options_file_number_(0), options_file_number_(0),
@ -2656,12 +2691,12 @@ VersionSet::~VersionSet() {
Cache* table_cache = column_family_set_->get_table_cache(); Cache* table_cache = column_family_set_->get_table_cache();
table_cache->ApplyToAllCacheEntries(&CloseTables, false /* thread_safe */); table_cache->ApplyToAllCacheEntries(&CloseTables, false /* thread_safe */);
column_family_set_.reset(); column_family_set_.reset();
for (auto file : obsolete_files_) { for (auto& file : obsolete_files_) {
if (file->table_reader_handle) { if (file.metadata->table_reader_handle) {
table_cache->Release(file->table_reader_handle); table_cache->Release(file.metadata->table_reader_handle);
TableCache::Evict(table_cache, file->fd.GetNumber()); TableCache::Evict(table_cache, file.metadata->fd.GetNumber());
} }
delete file; file.DeleteMetadata();
} }
obsolete_files_.clear(); obsolete_files_.clear();
} }
@ -2914,16 +2949,26 @@ Status VersionSet::LogAndApply(ColumnFamilyData* column_family_data,
} }
} else { } else {
uint64_t max_log_number_in_batch = 0; uint64_t max_log_number_in_batch = 0;
uint64_t min_log_number_to_keep = 0;
for (auto& e : batch_edits) { for (auto& e : batch_edits) {
if (e->has_log_number_) { if (e->has_log_number_) {
max_log_number_in_batch = max_log_number_in_batch =
std::max(max_log_number_in_batch, e->log_number_); std::max(max_log_number_in_batch, e->log_number_);
} }
if (e->has_min_log_number_to_keep_) {
min_log_number_to_keep =
std::max(min_log_number_to_keep, e->min_log_number_to_keep_);
}
} }
if (max_log_number_in_batch != 0) { if (max_log_number_in_batch != 0) {
assert(column_family_data->GetLogNumber() <= max_log_number_in_batch); assert(column_family_data->GetLogNumber() <= max_log_number_in_batch);
column_family_data->SetLogNumber(max_log_number_in_batch); column_family_data->SetLogNumber(max_log_number_in_batch);
} }
if (min_log_number_to_keep != 0) {
// Should only be set in 2PC mode.
MarkMinLogNumberToKeep2PC(min_log_number_to_keep);
}
AppendVersion(column_family_data, v); AppendVersion(column_family_data, v);
} }
@ -3076,6 +3121,7 @@ Status VersionSet::Recover(
uint64_t log_number = 0; uint64_t log_number = 0;
uint64_t previous_log_number = 0; uint64_t previous_log_number = 0;
uint32_t max_column_family = 0; uint32_t max_column_family = 0;
uint64_t min_log_number_to_keep = 0;
std::unordered_map<uint32_t, BaseReferencedVersionBuilder*> builders; std::unordered_map<uint32_t, BaseReferencedVersionBuilder*> builders;
// add default column family // add default column family
@ -3216,6 +3262,11 @@ Status VersionSet::Recover(
max_column_family = edit.max_column_family_; max_column_family = edit.max_column_family_;
} }
if (edit.has_min_log_number_to_keep_) {
min_log_number_to_keep =
std::max(min_log_number_to_keep, edit.min_log_number_to_keep_);
}
if (edit.has_last_sequence_) { if (edit.has_last_sequence_) {
last_sequence = edit.last_sequence_; last_sequence = edit.last_sequence_;
have_last_sequence = true; have_last_sequence = true;
@ -3238,6 +3289,9 @@ Status VersionSet::Recover(
column_family_set_->UpdateMaxColumnFamily(max_column_family); column_family_set_->UpdateMaxColumnFamily(max_column_family);
// When reading DB generated using old release, min_log_number_to_keep=0.
// All log files will be scanned for potential prepare entries.
MarkMinLogNumberToKeep2PC(min_log_number_to_keep);
MarkFileNumberUsed(previous_log_number); MarkFileNumberUsed(previous_log_number);
MarkFileNumberUsed(log_number); MarkFileNumberUsed(log_number);
} }
@ -3309,11 +3363,12 @@ Status VersionSet::Recover(
"manifest_file_number is %lu, next_file_number is %lu, " "manifest_file_number is %lu, next_file_number is %lu, "
"last_sequence is %lu, log_number is %lu," "last_sequence is %lu, log_number is %lu,"
"prev_log_number is %lu," "prev_log_number is %lu,"
"max_column_family is %u\n", "max_column_family is %u,"
"min_log_number_to_keep is %lu\n",
manifest_filename.c_str(), (unsigned long)manifest_file_number_, manifest_filename.c_str(), (unsigned long)manifest_file_number_,
(unsigned long)next_file_number_.load(), (unsigned long)last_sequence_, (unsigned long)next_file_number_.load(), (unsigned long)last_sequence_,
(unsigned long)log_number, (unsigned long)prev_log_number_, (unsigned long)log_number, (unsigned long)prev_log_number_,
column_family_set_->GetMaxColumnFamily()); column_family_set_->GetMaxColumnFamily(), min_log_number_to_keep_2pc());
for (auto cfd : *column_family_set_) { for (auto cfd : *column_family_set_) {
if (cfd->IsDropped()) { if (cfd->IsDropped()) {
@ -3601,6 +3656,7 @@ Status VersionSet::DumpManifest(Options& options, std::string& dscname,
cfd->SetLogNumber(edit.log_number_); cfd->SetLogNumber(edit.log_number_);
} }
if (edit.has_prev_log_number_) { if (edit.has_prev_log_number_) {
previous_log_number = edit.prev_log_number_; previous_log_number = edit.prev_log_number_;
have_prev_log_number = true; have_prev_log_number = true;
@ -3619,6 +3675,10 @@ Status VersionSet::DumpManifest(Options& options, std::string& dscname,
if (edit.has_max_column_family_) { if (edit.has_max_column_family_) {
column_family_set_->UpdateMaxColumnFamily(edit.max_column_family_); column_family_set_->UpdateMaxColumnFamily(edit.max_column_family_);
} }
if (edit.has_min_log_number_to_keep_) {
MarkMinLogNumberToKeep2PC(edit.min_log_number_to_keep_);
}
} }
} }
file_reader.reset(); file_reader.reset();
@ -3677,10 +3737,11 @@ Status VersionSet::DumpManifest(Options& options, std::string& dscname,
printf( printf(
"next_file_number %lu last_sequence " "next_file_number %lu last_sequence "
"%lu prev_log_number %lu max_column_family %u\n", "%lu prev_log_number %lu max_column_family %u min_log_number_to_keep "
"%" PRIu64 "\n",
(unsigned long)next_file_number_.load(), (unsigned long)last_sequence, (unsigned long)next_file_number_.load(), (unsigned long)last_sequence,
(unsigned long)previous_log_number, (unsigned long)previous_log_number,
column_family_set_->GetMaxColumnFamily()); column_family_set_->GetMaxColumnFamily(), min_log_number_to_keep_2pc());
} }
return s; return s;
@ -3695,6 +3756,14 @@ void VersionSet::MarkFileNumberUsed(uint64_t number) {
} }
} }
// Called only either from ::LogAndApply which is protected by mutex or during
// recovery which is single-threaded.
void VersionSet::MarkMinLogNumberToKeep2PC(uint64_t number) {
if (min_log_number_to_keep_2pc_.load(std::memory_order_relaxed) < number) {
min_log_number_to_keep_2pc_.store(number, std::memory_order_relaxed);
}
}
Status VersionSet::WriteSnapshot(log::Writer* log) { Status VersionSet::WriteSnapshot(log::Writer* log) {
// TODO: Break up into multiple records to reduce memory usage on recovery? // TODO: Break up into multiple records to reduce memory usage on recovery?
@ -4049,11 +4118,11 @@ void VersionSet::GetLiveFilesMetaData(std::vector<LiveFileMetaData>* metadata) {
LiveFileMetaData filemetadata; LiveFileMetaData filemetadata;
filemetadata.column_family_name = cfd->GetName(); filemetadata.column_family_name = cfd->GetName();
uint32_t path_id = file->fd.GetPathId(); uint32_t path_id = file->fd.GetPathId();
if (path_id < db_options_->db_paths.size()) { if (path_id < cfd->ioptions()->cf_paths.size()) {
filemetadata.db_path = db_options_->db_paths[path_id].path; filemetadata.db_path = cfd->ioptions()->cf_paths[path_id].path;
} else { } else {
assert(!db_options_->db_paths.empty()); assert(!cfd->ioptions()->cf_paths.empty());
filemetadata.db_path = db_options_->db_paths.back().path; filemetadata.db_path = cfd->ioptions()->cf_paths.back().path;
} }
filemetadata.name = MakeTableFileName("", file->fd.GetNumber()); filemetadata.name = MakeTableFileName("", file->fd.GetNumber());
filemetadata.level = level; filemetadata.level = level;
@ -4068,17 +4137,17 @@ void VersionSet::GetLiveFilesMetaData(std::vector<LiveFileMetaData>* metadata) {
} }
} }
void VersionSet::GetObsoleteFiles(std::vector<FileMetaData*>* files, void VersionSet::GetObsoleteFiles(std::vector<ObsoleteFileInfo>* files,
std::vector<std::string>* manifest_filenames, std::vector<std::string>* manifest_filenames,
uint64_t min_pending_output) { uint64_t min_pending_output) {
assert(manifest_filenames->empty()); assert(manifest_filenames->empty());
obsolete_manifests_.swap(*manifest_filenames); obsolete_manifests_.swap(*manifest_filenames);
std::vector<FileMetaData*> pending_files; std::vector<ObsoleteFileInfo> pending_files;
for (auto f : obsolete_files_) { for (auto& f : obsolete_files_) {
if (f->fd.GetNumber() < min_pending_output) { if (f.metadata->fd.GetNumber() < min_pending_output) {
files->push_back(f); files->push_back(std::move(f));
} else { } else {
pending_files.push_back(f); pending_files.push_back(std::move(f));
} }
} }
obsolete_files_.swap(pending_files); obsolete_files_.swap(pending_files);

View File

@ -135,6 +135,10 @@ class VersionStorageInfo {
// ComputeCompactionScore() // ComputeCompactionScore()
void ComputeFilesMarkedForCompaction(); void ComputeFilesMarkedForCompaction();
// This computes ttl_expired_files_ and is called by
// ComputeCompactionScore()
void ComputeExpiredTtlFiles(const ImmutableCFOptions& ioptions);
// This computes bottommost_files_marked_for_compaction_ and is called by // This computes bottommost_files_marked_for_compaction_ and is called by
// ComputeCompactionScore() or UpdateOldestSnapshot(). // ComputeCompactionScore() or UpdateOldestSnapshot().
// //
@ -286,6 +290,13 @@ class VersionStorageInfo {
return files_marked_for_compaction_; return files_marked_for_compaction_;
} }
// REQUIRES: This version has been saved (see VersionSet::SaveTo)
// REQUIRES: DB mutex held during access
const autovector<std::pair<int, FileMetaData*>>& ExpiredTtlFiles() const {
assert(finalized_);
return expired_ttl_files_;
}
// REQUIRES: This version has been saved (see VersionSet::SaveTo) // REQUIRES: This version has been saved (see VersionSet::SaveTo)
// REQUIRES: DB mutex held during access // REQUIRES: DB mutex held during access
const autovector<std::pair<int, FileMetaData*>>& const autovector<std::pair<int, FileMetaData*>>&
@ -446,6 +457,8 @@ class VersionStorageInfo {
// ComputeCompactionScore() // ComputeCompactionScore()
autovector<std::pair<int, FileMetaData*>> files_marked_for_compaction_; autovector<std::pair<int, FileMetaData*>> files_marked_for_compaction_;
autovector<std::pair<int, FileMetaData*>> expired_ttl_files_;
// These files are considered bottommost because none of their keys can exist // These files are considered bottommost because none of their keys can exist
// at lower levels. They are not necessarily all in the same level. The marked // at lower levels. They are not necessarily all in the same level. The marked
// ones are eligible for compaction because they contain duplicate key // ones are eligible for compaction because they contain duplicate key
@ -682,6 +695,36 @@ class Version {
void operator=(const Version&); void operator=(const Version&);
}; };
struct ObsoleteFileInfo {
FileMetaData* metadata;
std::string path;
ObsoleteFileInfo() noexcept : metadata(nullptr) {}
ObsoleteFileInfo(FileMetaData* f, const std::string& file_path)
: metadata(f), path(file_path) {}
ObsoleteFileInfo(const ObsoleteFileInfo&) = delete;
ObsoleteFileInfo& operator=(const ObsoleteFileInfo&) = delete;
ObsoleteFileInfo(ObsoleteFileInfo&& rhs) noexcept :
ObsoleteFileInfo() {
*this = std::move(rhs);
}
ObsoleteFileInfo& operator=(ObsoleteFileInfo&& rhs) noexcept {
path = std::move(rhs.path);
metadata = rhs.metadata;
rhs.metadata = nullptr;
return *this;
}
void DeleteMetadata() {
delete metadata;
metadata = nullptr;
}
};
class VersionSet { class VersionSet {
public: public:
VersionSet(const std::string& dbname, const ImmutableDBOptions* db_options, VersionSet(const std::string& dbname, const ImmutableDBOptions* db_options,
@ -759,6 +802,10 @@ class VersionSet {
uint64_t current_next_file_number() const { return next_file_number_.load(); } uint64_t current_next_file_number() const { return next_file_number_.load(); }
uint64_t min_log_number_to_keep_2pc() const {
return min_log_number_to_keep_2pc_.load();
}
// Allocate and return a new file number // Allocate and return a new file number
uint64_t NewFileNumber() { return next_file_number_.fetch_add(1); } uint64_t NewFileNumber() { return next_file_number_.fetch_add(1); }
@ -806,15 +853,31 @@ class VersionSet {
// REQUIRED: this is only called during single-threaded recovery or repair. // REQUIRED: this is only called during single-threaded recovery or repair.
void MarkFileNumberUsed(uint64_t number); void MarkFileNumberUsed(uint64_t number);
// Mark the specified log number as deleted
// REQUIRED: this is only called during single-threaded recovery or repair, or
// from ::LogAndApply where the global mutex is held.
void MarkMinLogNumberToKeep2PC(uint64_t number);
// Return the log file number for the log file that is currently // Return the log file number for the log file that is currently
// being compacted, or zero if there is no such log file. // being compacted, or zero if there is no such log file.
uint64_t prev_log_number() const { return prev_log_number_; } uint64_t prev_log_number() const { return prev_log_number_; }
// Returns the minimum log number such that all // Returns the minimum log number which still has data not flushed to any SST
// log numbers less than or equal to it can be deleted // file.
uint64_t MinLogNumber() const { // In non-2PC mode, all the log numbers smaller than this number can be safely
// deleted.
uint64_t MinLogNumberWithUnflushedData() const {
return PreComputeMinLogNumberWithUnflushedData(nullptr);
}
// Returns the minimum log number which still has data not flushed to any SST
// file, except data from `cfd_to_skip`.
uint64_t PreComputeMinLogNumberWithUnflushedData(
const ColumnFamilyData* cfd_to_skip) const {
uint64_t min_log_num = std::numeric_limits<uint64_t>::max(); uint64_t min_log_num = std::numeric_limits<uint64_t>::max();
for (auto cfd : *column_family_set_) { for (auto cfd : *column_family_set_) {
if (cfd == cfd_to_skip) {
continue;
}
// It's safe to ignore dropped column families here: // It's safe to ignore dropped column families here:
// cfd->IsDropped() becomes true after the drop is persisted in MANIFEST. // cfd->IsDropped() becomes true after the drop is persisted in MANIFEST.
if (min_log_num > cfd->GetLogNumber() && !cfd->IsDropped()) { if (min_log_num > cfd->GetLogNumber() && !cfd->IsDropped()) {
@ -854,7 +917,7 @@ class VersionSet {
// This function doesn't support leveldb SST filenames // This function doesn't support leveldb SST filenames
void GetLiveFilesMetaData(std::vector<LiveFileMetaData> *metadata); void GetLiveFilesMetaData(std::vector<LiveFileMetaData> *metadata);
void GetObsoleteFiles(std::vector<FileMetaData*>* files, void GetObsoleteFiles(std::vector<ObsoleteFileInfo>* files,
std::vector<std::string>* manifest_filenames, std::vector<std::string>* manifest_filenames,
uint64_t min_pending_output); uint64_t min_pending_output);
@ -865,6 +928,8 @@ class VersionSet {
new_options.writable_file_max_buffer_size; new_options.writable_file_max_buffer_size;
} }
const ImmutableDBOptions* db_options() const { return db_options_; }
static uint64_t GetNumLiveVersions(Version* dummy_versions); static uint64_t GetNumLiveVersions(Version* dummy_versions);
static uint64_t GetTotalSstFilesSize(Version* dummy_versions); static uint64_t GetTotalSstFilesSize(Version* dummy_versions);
@ -903,6 +968,10 @@ class VersionSet {
const std::string dbname_; const std::string dbname_;
const ImmutableDBOptions* const db_options_; const ImmutableDBOptions* const db_options_;
std::atomic<uint64_t> next_file_number_; std::atomic<uint64_t> next_file_number_;
// Any log number equal or lower than this should be ignored during recovery,
// and is qualified for being deleted in 2PC mode. In non-2PC mode, this
// number is ignored.
std::atomic<uint64_t> min_log_number_to_keep_2pc_ = {0};
uint64_t manifest_file_number_; uint64_t manifest_file_number_;
uint64_t options_file_number_; uint64_t options_file_number_;
uint64_t pending_manifest_file_number_; uint64_t pending_manifest_file_number_;
@ -935,7 +1004,7 @@ class VersionSet {
// Current size of manifest file // Current size of manifest file
uint64_t manifest_file_size_; uint64_t manifest_file_size_;
std::vector<FileMetaData*> obsolete_files_; std::vector<ObsoleteFileInfo> obsolete_files_;
std::vector<std::string> obsolete_manifests_; std::vector<std::string> obsolete_manifests_;
// env options for all reads and writes except compactions // env options for all reads and writes except compactions

View File

@ -303,7 +303,7 @@ int main(int argc, char** argv) {
#else #else
#include <stdio.h> #include <stdio.h>
int main(int argc, char** argv) { int main(int /*argc*/, char** /*argv*/) {
fprintf(stderr, "SKIPPED as WalManager is not supported in ROCKSDB_LITE\n"); fprintf(stderr, "SKIPPED as WalManager is not supported in ROCKSDB_LITE\n");
return 0; return 0;
} }

View File

@ -405,9 +405,11 @@ Status WriteBatch::Iterate(Handler* handler) const {
Status s; Status s;
char tag = 0; char tag = 0;
uint32_t column_family = 0; // default uint32_t column_family = 0; // default
while ((s.ok() || UNLIKELY(s.IsTryAgain())) && !input.empty() && bool last_was_try_again = false;
while (((s.ok() && !input.empty()) || UNLIKELY(s.IsTryAgain())) &&
handler->Continue()) { handler->Continue()) {
if (LIKELY(!s.IsTryAgain())) { if (LIKELY(!s.IsTryAgain())) {
last_was_try_again = false;
tag = 0; tag = 0;
column_family = 0; // default column_family = 0; // default
@ -418,6 +420,13 @@ Status WriteBatch::Iterate(Handler* handler) const {
} }
} else { } else {
assert(s.IsTryAgain()); assert(s.IsTryAgain());
assert(!last_was_try_again); // to detect infinite loop bugs
if (UNLIKELY(last_was_try_again)) {
return Status::Corruption(
"two consecutive TryAgain in WriteBatch handler; this is either a "
"software bug or data corruption.");
}
last_was_try_again = true;
s = Status::OK(); s = Status::OK();
} }

View File

@ -432,7 +432,7 @@ int main(int argc, char** argv) {
#else #else
#include <stdio.h> #include <stdio.h>
int main(int argc, char** argv) { int main(int /*argc*/, char** /*argv*/) {
fprintf(stderr, fprintf(stderr,
"SKIPPED as WriteWithCallback is not supported in ROCKSDB_LITE\n"); "SKIPPED as WriteWithCallback is not supported in ROCKSDB_LITE\n");
return 0; return 0;

8
env/io_posix.cc vendored
View File

@ -873,8 +873,12 @@ void PosixWritableFile::SetWriteLifeTimeHint(Env::WriteLifeTimeHint hint) {
if (fcntl(fd_, F_SET_RW_HINT, &hint) == 0) { if (fcntl(fd_, F_SET_RW_HINT, &hint) == 0) {
write_hint_ = hint; write_hint_ = hint;
} }
#endif #else
#endif (void)hint;
#endif // ROCKSDB_VALGRIND_RUN
#else
(void)hint;
#endif // OS_LINUX
} }
Status PosixWritableFile::InvalidateCache(size_t offset, size_t length) { Status PosixWritableFile::InvalidateCache(size_t offset, size_t length) {

View File

@ -570,6 +570,13 @@ struct AdvancedColumnFamilyOptions {
// Default: false // Default: false
bool report_bg_io_stats = false; bool report_bg_io_stats = false;
// Non-bottom-level files older than TTL will go through the compaction
// process. This needs max_open_files to be set to -1.
// Enabled only for level compaction for now.
//
// Default: 0 (disabled)
uint64_t ttl = 0;
// Create ColumnFamilyOptions with default values for all fields // Create ColumnFamilyOptions with default values for all fields
AdvancedColumnFamilyOptions(); AdvancedColumnFamilyOptions();
// Create ColumnFamilyOptions from Options // Create ColumnFamilyOptions from Options

View File

@ -1167,7 +1167,9 @@ class DB {
// Destroy the contents of the specified database. // Destroy the contents of the specified database.
// Be very careful using this method. // Be very careful using this method.
Status DestroyDB(const std::string& name, const Options& options); Status DestroyDB(const std::string& name, const Options& options,
const std::vector<ColumnFamilyDescriptor>& column_families =
std::vector<ColumnFamilyDescriptor>());
#ifndef ROCKSDB_LITE #ifndef ROCKSDB_LITE
// If a DB cannot be opened, you may attempt to call this method to // If a DB cannot be opened, you may attempt to call this method to

View File

@ -80,6 +80,7 @@ enum class CompactionReason {
// [Level] Automatic compaction within bottommost level to cleanup duplicate // [Level] Automatic compaction within bottommost level to cleanup duplicate
// versions of same user key, usually due to a released snapshot. // versions of same user key, usually due to a released snapshot.
kBottommostFiles, kBottommostFiles,
kTtl,
}; };
enum class FlushReason : int { enum class FlushReason : int {

View File

@ -77,6 +77,7 @@ enum CompressionType : unsigned char {
}; };
struct Options; struct Options;
struct DbPath;
struct ColumnFamilyOptions : public AdvancedColumnFamilyOptions { struct ColumnFamilyOptions : public AdvancedColumnFamilyOptions {
// The function recovers options to a previous version. Only 4.6 or later // The function recovers options to a previous version. Only 4.6 or later
@ -263,6 +264,20 @@ struct ColumnFamilyOptions : public AdvancedColumnFamilyOptions {
// BlockBasedTableOptions. // BlockBasedTableOptions.
std::shared_ptr<TableFactory> table_factory; std::shared_ptr<TableFactory> table_factory;
// A list of paths where SST files for this column family
// can be put into, with its target size. Similar to db_paths,
// newer data is placed into paths specified earlier in the
// vector while older data gradually moves to paths specified
// later in the vector.
// Note that, if a path is supplied to multiple column
// families, it would have files and total size from all
// the column families combined. User should privision for the
// total size(from all the column families) in such cases.
//
// If left empty, db_paths will be used.
// Default: empty
std::vector<DbPath> cf_paths;
// Create ColumnFamilyOptions with default values for all fields // Create ColumnFamilyOptions with default values for all fields
ColumnFamilyOptions(); ColumnFamilyOptions();
// Create ColumnFamilyOptions from Options // Create ColumnFamilyOptions from Options

View File

@ -98,12 +98,13 @@ class SstFileManager {
// DeleteScheduler immediately // DeleteScheduler immediately
// @param bytes_max_delete_chunk: if a single file is larger than delete chunk, // @param bytes_max_delete_chunk: if a single file is larger than delete chunk,
// ftruncate the file by this size each time, rather than dropping the whole // ftruncate the file by this size each time, rather than dropping the whole
// file. 0 means to always delete the whole file. // file. 0 means to always delete the whole file. NOTE this options may not
// work well with checkpoints, which relies on file system hard links.
extern SstFileManager* NewSstFileManager( extern SstFileManager* NewSstFileManager(
Env* env, std::shared_ptr<Logger> info_log = nullptr, Env* env, std::shared_ptr<Logger> info_log = nullptr,
std::string trash_dir = "", int64_t rate_bytes_per_sec = 0, std::string trash_dir = "", int64_t rate_bytes_per_sec = 0,
bool delete_existing_trash = true, Status* status = nullptr, bool delete_existing_trash = true, Status* status = nullptr,
double max_trash_db_ratio = 0.25, double max_trash_db_ratio = 0.25,
uint64_t bytes_max_delete_chunk = 64 * 1024 * 1024); uint64_t bytes_max_delete_chunk = 0);
} // namespace rocksdb } // namespace rocksdb

View File

@ -71,8 +71,13 @@ enum Tickers : uint32_t {
// # of bytes written into cache. // # of bytes written into cache.
BLOCK_CACHE_BYTES_WRITE, BLOCK_CACHE_BYTES_WRITE,
// # of times bloom filter has avoided file reads. // # of times bloom filter has avoided file reads, i.e., negatives.
BLOOM_FILTER_USEFUL, BLOOM_FILTER_USEFUL,
// # of times bloom FullFilter has not avoided the reads.
BLOOM_FILTER_FULL_POSITIVE,
// # of times bloom FullFilter has not avoided the reads and data actually
// exist.
BLOOM_FILTER_FULL_TRUE_POSITIVE,
// # persistent cache hit // # persistent cache hit
PERSISTENT_CACHE_HIT, PERSISTENT_CACHE_HIT,
@ -304,6 +309,17 @@ enum Tickers : uint32_t {
// # of bytes in the blob files evicted because of BlobDB is full. // # of bytes in the blob files evicted because of BlobDB is full.
BLOB_DB_FIFO_BYTES_EVICTED, BLOB_DB_FIFO_BYTES_EVICTED,
// These coutners indicate a performance issue in WritePrepared transactions.
// We should not seem them ticking them much.
// # of times prepare_mutex_ is acquired in the fast path.
TXN_PREPARE_MUTEX_OVERHEAD,
// # of times old_commit_map_mutex_ is acquired in the fast path.
TXN_OLD_COMMIT_MAP_MUTEX_OVERHEAD,
// # of times we checked a batch for duplicate keys.
TXN_DUPLICATE_KEY_OVERHEAD,
// # of times snapshot_mutex_ is acquired in the fast path.
TXN_SNAPSHOT_MUTEX_OVERHEAD,
TICKER_ENUM_MAX TICKER_ENUM_MAX
}; };
@ -332,6 +348,9 @@ const std::vector<std::pair<Tickers, std::string>> TickersNameMap = {
{BLOCK_CACHE_BYTES_READ, "rocksdb.block.cache.bytes.read"}, {BLOCK_CACHE_BYTES_READ, "rocksdb.block.cache.bytes.read"},
{BLOCK_CACHE_BYTES_WRITE, "rocksdb.block.cache.bytes.write"}, {BLOCK_CACHE_BYTES_WRITE, "rocksdb.block.cache.bytes.write"},
{BLOOM_FILTER_USEFUL, "rocksdb.bloom.filter.useful"}, {BLOOM_FILTER_USEFUL, "rocksdb.bloom.filter.useful"},
{BLOOM_FILTER_FULL_POSITIVE, "rocksdb.bloom.filter.full.positive"},
{BLOOM_FILTER_FULL_TRUE_POSITIVE,
"rocksdb.bloom.filter.full.true.positive"},
{PERSISTENT_CACHE_HIT, "rocksdb.persistent.cache.hit"}, {PERSISTENT_CACHE_HIT, "rocksdb.persistent.cache.hit"},
{PERSISTENT_CACHE_MISS, "rocksdb.persistent.cache.miss"}, {PERSISTENT_CACHE_MISS, "rocksdb.persistent.cache.miss"},
{SIM_BLOCK_CACHE_HIT, "rocksdb.sim.block.cache.hit"}, {SIM_BLOCK_CACHE_HIT, "rocksdb.sim.block.cache.hit"},
@ -349,8 +368,7 @@ const std::vector<std::pair<Tickers, std::string>> TickersNameMap = {
"rocksdb.compaction.range_del.drop.obsolete"}, "rocksdb.compaction.range_del.drop.obsolete"},
{COMPACTION_OPTIMIZED_DEL_DROP_OBSOLETE, {COMPACTION_OPTIMIZED_DEL_DROP_OBSOLETE,
"rocksdb.compaction.optimized.del.drop.obsolete"}, "rocksdb.compaction.optimized.del.drop.obsolete"},
{COMPACTION_CANCELLED, {COMPACTION_CANCELLED, "rocksdb.compaction.cancelled"},
"rocksdb.compaction.cancelled"},
{NUMBER_KEYS_WRITTEN, "rocksdb.number.keys.written"}, {NUMBER_KEYS_WRITTEN, "rocksdb.number.keys.written"},
{NUMBER_KEYS_READ, "rocksdb.number.keys.read"}, {NUMBER_KEYS_READ, "rocksdb.number.keys.read"},
{NUMBER_KEYS_UPDATED, "rocksdb.number.keys.updated"}, {NUMBER_KEYS_UPDATED, "rocksdb.number.keys.updated"},
@ -448,6 +466,11 @@ const std::vector<std::pair<Tickers, std::string>> TickersNameMap = {
{BLOB_DB_FIFO_NUM_FILES_EVICTED, "rocksdb.blobdb.fifo.num.files.evicted"}, {BLOB_DB_FIFO_NUM_FILES_EVICTED, "rocksdb.blobdb.fifo.num.files.evicted"},
{BLOB_DB_FIFO_NUM_KEYS_EVICTED, "rocksdb.blobdb.fifo.num.keys.evicted"}, {BLOB_DB_FIFO_NUM_KEYS_EVICTED, "rocksdb.blobdb.fifo.num.keys.evicted"},
{BLOB_DB_FIFO_BYTES_EVICTED, "rocksdb.blobdb.fifo.bytes.evicted"}, {BLOB_DB_FIFO_BYTES_EVICTED, "rocksdb.blobdb.fifo.bytes.evicted"},
{TXN_PREPARE_MUTEX_OVERHEAD, "rocksdb.txn.overhead.mutex.prepare"},
{TXN_OLD_COMMIT_MAP_MUTEX_OVERHEAD,
"rocksdb.txn.overhead.mutex.old.commit.map"},
{TXN_DUPLICATE_KEY_OVERHEAD, "rocksdb.txn.overhead.duplicate.key"},
{TXN_SNAPSHOT_MUTEX_OVERHEAD, "rocksdb.txn.overhead.mutex.snapshot"},
}; };
/** /**

View File

@ -215,6 +215,7 @@ class LDBCommand {
LDBOptions ldb_options_; LDBOptions ldb_options_;
private: private:
friend class WALDumperCommand;
/** /**
* Interpret command line options and flags to determine if the key * Interpret command line options and flags to determine if the key
* should be input/output in hex. * should be input/output in hex.

View File

@ -85,6 +85,14 @@ struct TransactionDBOptions {
// before the commit phase. The DB then needs to provide the mechanisms to // before the commit phase. The DB then needs to provide the mechanisms to
// tell apart committed from uncommitted data. // tell apart committed from uncommitted data.
TxnDBWritePolicy write_policy = TxnDBWritePolicy::WRITE_COMMITTED; TxnDBWritePolicy write_policy = TxnDBWritePolicy::WRITE_COMMITTED;
// TODO(myabandeh): remove this option
// Note: this is a temporary option as a hot fix in rollback of writeprepared
// txns in myrocks. MyRocks uses merge operands for autoinc column id without
// however obtaining locks. This breaks the assumption behind the rollback
// logic in myrocks. This hack of simply not rolling back merge operands works
// for the special way that myrocks uses this operands.
bool rollback_merge_operands = false;
}; };
struct TransactionOptions { struct TransactionOptions {
@ -98,9 +106,10 @@ struct TransactionOptions {
bool deadlock_detect = false; bool deadlock_detect = false;
// If set, it states that the CommitTimeWriteBatch represents the latest state // If set, it states that the CommitTimeWriteBatch represents the latest state
// of the application and meant to be used later during recovery. It enables // of the application, has only one sub-batch, i.e., no duplicate keys, and
// an optimization to postpone updating the memtable with CommitTimeWriteBatch // meant to be used later during recovery. It enables an optimization to
// to only SwitchMemtable or recovery. // postpone updating the memtable with CommitTimeWriteBatch to only
// SwitchMemtable or recovery.
bool use_only_the_last_commit_time_batch_for_recovery = false; bool use_only_the_last_commit_time_batch_for_recovery = false;
// TODO(agiardullo): TransactionDB does not yet support comparators that allow // TODO(agiardullo): TransactionDB does not yet support comparators that allow

View File

@ -74,7 +74,9 @@ ImmutableCFOptions::ImmutableCFOptions(const ImmutableDBOptions& db_options,
row_cache(db_options.row_cache), row_cache(db_options.row_cache),
max_subcompactions(db_options.max_subcompactions), max_subcompactions(db_options.max_subcompactions),
memtable_insert_with_hint_prefix_extractor( memtable_insert_with_hint_prefix_extractor(
cf_options.memtable_insert_with_hint_prefix_extractor.get()) {} cf_options.memtable_insert_with_hint_prefix_extractor.get()),
ttl(cf_options.ttl),
cf_paths(cf_options.cf_paths) {}
// Multiple two operands. If they overflow, return op1. // Multiple two operands. If they overflow, return op1.
uint64_t MultiplyCheckOverflow(uint64_t op1, double op2) { uint64_t MultiplyCheckOverflow(uint64_t op1, double op2) {

View File

@ -118,6 +118,10 @@ struct ImmutableCFOptions {
uint32_t max_subcompactions; uint32_t max_subcompactions;
const SliceTransform* memtable_insert_with_hint_prefix_extractor; const SliceTransform* memtable_insert_with_hint_prefix_extractor;
uint64_t ttl;
std::vector<DbPath> cf_paths;
}; };
struct MutableCFOptions { struct MutableCFOptions {

View File

@ -85,7 +85,8 @@ AdvancedColumnFamilyOptions::AdvancedColumnFamilyOptions(const Options& options)
optimize_filters_for_hits(options.optimize_filters_for_hits), optimize_filters_for_hits(options.optimize_filters_for_hits),
paranoid_file_checks(options.paranoid_file_checks), paranoid_file_checks(options.paranoid_file_checks),
force_consistency_checks(options.force_consistency_checks), force_consistency_checks(options.force_consistency_checks),
report_bg_io_stats(options.report_bg_io_stats) { report_bg_io_stats(options.report_bg_io_stats),
ttl(options.ttl) {
assert(memtable_factory.get() != nullptr); assert(memtable_factory.get() != nullptr);
if (max_bytes_for_level_multiplier_additional.size() < if (max_bytes_for_level_multiplier_additional.size() <
static_cast<unsigned int>(num_levels)) { static_cast<unsigned int>(num_levels)) {
@ -321,6 +322,7 @@ void ColumnFamilyOptions::Dump(Logger* log) const {
force_consistency_checks); force_consistency_checks);
ROCKS_LOG_HEADER(log, " Options.report_bg_io_stats: %d", ROCKS_LOG_HEADER(log, " Options.report_bg_io_stats: %d",
report_bg_io_stats); report_bg_io_stats);
ROCKS_LOG_HEADER(log, " Options.ttl: %d", ttl);
} // ColumnFamilyOptions::Dump } // ColumnFamilyOptions::Dump
void Options::Dump(Logger* log) const { void Options::Dump(Logger* log) const {

View File

@ -1601,6 +1601,7 @@ std::unordered_map<std::string, OptionTypeInfo>
uint34_t* existing_value_size, uint34_t* existing_value_size,
Slice delta_value, Slice delta_value,
std::string* merged_value); std::string* merged_value);
std::vector<DbPath> cf_paths;
*/ */
{"report_bg_io_stats", {"report_bg_io_stats",
{offset_of(&ColumnFamilyOptions::report_bg_io_stats), {offset_of(&ColumnFamilyOptions::report_bg_io_stats),
@ -1819,7 +1820,10 @@ std::unordered_map<std::string, OptionTypeInfo>
{offset_of(&ColumnFamilyOptions::compaction_options_universal), {offset_of(&ColumnFamilyOptions::compaction_options_universal),
OptionType::kCompactionOptionsUniversal, OptionType::kCompactionOptionsUniversal,
OptionVerificationType::kNormal, true, OptionVerificationType::kNormal, true,
offsetof(struct MutableCFOptions, compaction_options_universal)}}}; offsetof(struct MutableCFOptions, compaction_options_universal)}},
{"ttl",
{offset_of(&ColumnFamilyOptions::ttl), OptionType::kUInt64T,
OptionVerificationType::kNormal, false, 0}}};
std::unordered_map<std::string, OptionTypeInfo> std::unordered_map<std::string, OptionTypeInfo>
OptionsHelper::fifo_compaction_options_type_info = { OptionsHelper::fifo_compaction_options_type_info = {

View File

@ -345,6 +345,8 @@ TEST_F(OptionsSettableTest, ColumnFamilyOptionsAllFieldsSettable) {
sizeof(std::shared_ptr<const SliceTransform>)}, sizeof(std::shared_ptr<const SliceTransform>)},
{offset_of(&ColumnFamilyOptions::table_factory), {offset_of(&ColumnFamilyOptions::table_factory),
sizeof(std::shared_ptr<TableFactory>)}, sizeof(std::shared_ptr<TableFactory>)},
{offset_of(&ColumnFamilyOptions::cf_paths),
sizeof(std::vector<DbPath>)},
}; };
char* options_ptr = new char[sizeof(ColumnFamilyOptions)]; char* options_ptr = new char[sizeof(ColumnFamilyOptions)];
@ -437,6 +439,7 @@ TEST_F(OptionsSettableTest, ColumnFamilyOptionsAllFieldsSettable) {
"hard_pending_compaction_bytes_limit=0;" "hard_pending_compaction_bytes_limit=0;"
"disable_auto_compactions=false;" "disable_auto_compactions=false;"
"report_bg_io_stats=true;" "report_bg_io_stats=true;"
"ttl=60;"
"compaction_options_fifo={max_table_files_size=3;ttl=100;allow_" "compaction_options_fifo={max_table_files_size=3;ttl=100;allow_"
"compaction=false;};", "compaction=false;};",
new_options)); new_options));

1
src.mk
View File

@ -33,6 +33,7 @@ LIB_SOURCES = \
db/flush_scheduler.cc \ db/flush_scheduler.cc \
db/forward_iterator.cc \ db/forward_iterator.cc \
db/internal_stats.cc \ db/internal_stats.cc \
db/logs_with_prep_tracker.cc \
db/log_reader.cc \ db/log_reader.cc \
db/log_writer.cc \ db/log_writer.cc \
db/malloc_stats.cc \ db/malloc_stats.cc \

View File

@ -120,7 +120,7 @@ void ReleaseCachedEntry(void* arg, void* h) {
void ForceReleaseCachedEntry(void* arg, void* h) { void ForceReleaseCachedEntry(void* arg, void* h) {
Cache* cache = reinterpret_cast<Cache*>(arg); Cache* cache = reinterpret_cast<Cache*>(arg);
Cache::Handle* handle = reinterpret_cast<Cache::Handle*>(h); Cache::Handle* handle = reinterpret_cast<Cache::Handle*>(h);
cache->Release(handle, true); cache->Release(handle, true /* force_erase */);
} }
Slice GetCacheKeyFromOffset(const char* cache_key_prefix, Slice GetCacheKeyFromOffset(const char* cache_key_prefix,
@ -650,7 +650,7 @@ Status BlockBasedTable::Open(const ImmutableCFOptions& ioptions,
std::unique_ptr<FilePrefetchBuffer> prefetch_buffer; std::unique_ptr<FilePrefetchBuffer> prefetch_buffer;
// Before read footer, readahead backwards to prefetch data // Before read footer, readahead backwards to prefetch data
const size_t kTailPrefetchSize = 512 * 1024; const size_t kTailPrefetchSize = 4 * 1024;
size_t prefetch_off; size_t prefetch_off;
size_t prefetch_len; size_t prefetch_len;
if (file_size < kTailPrefetchSize) { if (file_size < kTailPrefetchSize) {
@ -1556,9 +1556,6 @@ BlockIter* BlockBasedTable::NewDataBlockIterator(
iter->RegisterCleanup(&ForceReleaseCachedEntry, block_cache, iter->RegisterCleanup(&ForceReleaseCachedEntry, block_cache,
cache_handle); cache_handle);
} }
} else {
delete block.value;
block.value = nullptr;
} }
} }
iter->RegisterCleanup(&DeleteHeldResource<Block>, block.value, nullptr); iter->RegisterCleanup(&DeleteHeldResource<Block>, block.value, nullptr);
@ -2030,19 +2027,23 @@ bool BlockBasedTable::FullFilterKeyMayMatch(const ReadOptions& read_options,
} }
Slice user_key = ExtractUserKey(internal_key); Slice user_key = ExtractUserKey(internal_key);
const Slice* const const_ikey_ptr = &internal_key; const Slice* const const_ikey_ptr = &internal_key;
bool may_match = true;
if (filter->whole_key_filtering()) { if (filter->whole_key_filtering()) {
return filter->KeyMayMatch(user_key, kNotValid, no_io, const_ikey_ptr); may_match = filter->KeyMayMatch(user_key, kNotValid, no_io, const_ikey_ptr);
} } else if (!read_options.total_order_seek &&
if (!read_options.total_order_seek && rep_->ioptions.prefix_extractor && rep_->ioptions.prefix_extractor &&
rep_->table_properties->prefix_extractor_name.compare( rep_->table_properties->prefix_extractor_name.compare(
rep_->ioptions.prefix_extractor->Name()) == 0 && rep_->ioptions.prefix_extractor->Name()) == 0 &&
rep_->ioptions.prefix_extractor->InDomain(user_key) && rep_->ioptions.prefix_extractor->InDomain(user_key) &&
!filter->PrefixMayMatch( !filter->PrefixMayMatch(
rep_->ioptions.prefix_extractor->Transform(user_key), kNotValid, rep_->ioptions.prefix_extractor->Transform(user_key),
false, const_ikey_ptr)) { kNotValid, false, const_ikey_ptr)) {
return false; may_match = false;
} }
return true; if (may_match) {
RecordTick(rep_->ioptions.statistics, BLOOM_FILTER_FULL_POSITIVE);
}
return may_match;
} }
Status BlockBasedTable::Get(const ReadOptions& read_options, const Slice& key, Status BlockBasedTable::Get(const ReadOptions& read_options, const Slice& key,
@ -2070,6 +2071,7 @@ Status BlockBasedTable::Get(const ReadOptions& read_options, const Slice& key,
iiter_unique_ptr.reset(iiter); iiter_unique_ptr.reset(iiter);
} }
bool matched = false; // if such user key mathced a key in SST
bool done = false; bool done = false;
for (iiter->Seek(key); iiter->Valid() && !done; iiter->Next()) { for (iiter->Seek(key); iiter->Valid() && !done; iiter->Next()) {
Slice handle_value = iiter->value(); Slice handle_value = iiter->value();
@ -2111,7 +2113,8 @@ Status BlockBasedTable::Get(const ReadOptions& read_options, const Slice& key,
s = Status::Corruption(Slice()); s = Status::Corruption(Slice());
} }
if (!get_context->SaveValue(parsed_key, biter.value(), &biter)) { if (!get_context->SaveValue(parsed_key, biter.value(), &matched,
&biter)) {
done = true; done = true;
break; break;
} }
@ -2123,6 +2126,9 @@ Status BlockBasedTable::Get(const ReadOptions& read_options, const Slice& key,
break; break;
} }
} }
if (matched && filter != nullptr && !filter->IsBlockBased()) {
RecordTick(rep_->ioptions.statistics, BLOOM_FILTER_FULL_TRUE_POSITIVE);
}
if (s.ok()) { if (s.ok()) {
s = iiter->status(); s = iiter->status();
} }

View File

@ -625,7 +625,7 @@ int main(int argc, char** argv) {
#else #else
#include <stdio.h> #include <stdio.h>
int main(int argc, char** argv) { int main(int /*argc*/, char** /*argv*/) {
fprintf(stderr, "SKIPPED as Cuckoo table is not supported in ROCKSDB_LITE\n"); fprintf(stderr, "SKIPPED as Cuckoo table is not supported in ROCKSDB_LITE\n");
return 0; return 0;
} }

View File

@ -170,7 +170,8 @@ Status CuckooTableReader::Get(const ReadOptions& /*readOptions*/,
Slice full_key(bucket, key_length_); Slice full_key(bucket, key_length_);
ParsedInternalKey found_ikey; ParsedInternalKey found_ikey;
ParseInternalKey(full_key, &found_ikey); ParseInternalKey(full_key, &found_ikey);
get_context->SaveValue(found_ikey, value); bool dont_care __attribute__((__unused__));
get_context->SaveValue(found_ikey, value, &dont_care);
} }
// We don't support merge operations. So, we return here. // We don't support merge operations. So, we return here.
return Status::OK(); return Status::OK();

View File

@ -560,7 +560,7 @@ int main(int argc, char** argv) {
#else #else
#include <stdio.h> #include <stdio.h>
int main(int argc, char** argv) { int main(int /*argc*/, char** /*argv*/) {
fprintf(stderr, "SKIPPED as Cuckoo table is not supported in ROCKSDB_LITE\n"); fprintf(stderr, "SKIPPED as Cuckoo table is not supported in ROCKSDB_LITE\n");
return 0; return 0;
} }

View File

@ -51,6 +51,7 @@ class FullFilterBitsBuilder : public FilterBitsBuilder {
uint32_t* num_lines); uint32_t* num_lines);
private: private:
friend class FullFilterBlockTest_DuplicateEntries_Test;
size_t bits_per_key_; size_t bits_per_key_;
size_t num_probes_; size_t num_probes_;
std::vector<uint32_t> hash_entries_; std::vector<uint32_t> hash_entries_;

View File

@ -17,16 +17,32 @@ FullFilterBlockBuilder::FullFilterBlockBuilder(
FilterBitsBuilder* filter_bits_builder) FilterBitsBuilder* filter_bits_builder)
: prefix_extractor_(prefix_extractor), : prefix_extractor_(prefix_extractor),
whole_key_filtering_(whole_key_filtering), whole_key_filtering_(whole_key_filtering),
last_whole_key_recorded_(false),
last_prefix_recorded_(false),
num_added_(0) { num_added_(0) {
assert(filter_bits_builder != nullptr); assert(filter_bits_builder != nullptr);
filter_bits_builder_.reset(filter_bits_builder); filter_bits_builder_.reset(filter_bits_builder);
} }
void FullFilterBlockBuilder::Add(const Slice& key) { void FullFilterBlockBuilder::Add(const Slice& key) {
const bool add_prefix = prefix_extractor_ && prefix_extractor_->InDomain(key);
if (whole_key_filtering_) { if (whole_key_filtering_) {
if (!add_prefix) {
AddKey(key); AddKey(key);
} else {
// if both whole_key and prefix are added to bloom then we will have whole
// key and prefix addition being interleaved and thus cannot rely on the
// bits builder to properly detect the duplicates by comparing with the
// last item.
Slice last_whole_key = Slice(last_whole_key_str_);
if (!last_whole_key_recorded_ || last_whole_key.compare(key) != 0) {
AddKey(key);
last_whole_key_recorded_ = true;
last_whole_key_str_.assign(key.data(), key.size());
} }
if (prefix_extractor_ && prefix_extractor_->InDomain(key)) { }
}
if (add_prefix) {
AddPrefix(key); AddPrefix(key);
} }
} }
@ -40,7 +56,20 @@ inline void FullFilterBlockBuilder::AddKey(const Slice& key) {
// Add prefix to filter if needed // Add prefix to filter if needed
inline void FullFilterBlockBuilder::AddPrefix(const Slice& key) { inline void FullFilterBlockBuilder::AddPrefix(const Slice& key) {
Slice prefix = prefix_extractor_->Transform(key); Slice prefix = prefix_extractor_->Transform(key);
if (whole_key_filtering_) {
// if both whole_key and prefix are added to bloom then we will have whole
// key and prefix addition being interleaved and thus cannot rely on the
// bits builder to properly detect the duplicates by comparing with the last
// item.
Slice last_prefix = Slice(last_prefix_str_);
if (!last_prefix_recorded_ || last_prefix.compare(prefix) != 0) {
AddKey(prefix); AddKey(prefix);
last_prefix_recorded_ = true;
last_prefix_str_.assign(prefix.data(), prefix.size());
}
} else {
AddKey(prefix);
}
} }
Slice FullFilterBlockBuilder::Finish(const BlockHandle& /*tmp*/, Slice FullFilterBlockBuilder::Finish(const BlockHandle& /*tmp*/,

View File

@ -59,6 +59,10 @@ class FullFilterBlockBuilder : public FilterBlockBuilder {
// should NOT dereference them. // should NOT dereference them.
const SliceTransform* prefix_extractor_; const SliceTransform* prefix_extractor_;
bool whole_key_filtering_; bool whole_key_filtering_;
bool last_whole_key_recorded_;
std::string last_whole_key_str_;
bool last_prefix_recorded_;
std::string last_prefix_str_;
uint32_t num_added_; uint32_t num_added_;
std::unique_ptr<const char[]> filter_data_; std::unique_ptr<const char[]> filter_data_;

View File

@ -6,6 +6,7 @@
#include "table/full_filter_block.h" #include "table/full_filter_block.h"
#include "rocksdb/filter_policy.h" #include "rocksdb/filter_policy.h"
#include "table/full_filter_bits_builder.h"
#include "util/coding.h" #include "util/coding.h"
#include "util/hash.h" #include "util/hash.h"
#include "util/string_util.h" #include "util/string_util.h"
@ -160,6 +161,39 @@ TEST_F(FullFilterBlockTest, EmptyBuilder) {
ASSERT_TRUE(reader.KeyMayMatch("foo")); ASSERT_TRUE(reader.KeyMayMatch("foo"));
} }
TEST_F(FullFilterBlockTest, DuplicateEntries) {
{ // empty prefixes
std::unique_ptr<const SliceTransform> prefix_extractor(
NewFixedPrefixTransform(0));
auto bits_builder = dynamic_cast<FullFilterBitsBuilder*>(
table_options_.filter_policy->GetFilterBitsBuilder());
const bool WHOLE_KEY = true;
FullFilterBlockBuilder builder(prefix_extractor.get(), WHOLE_KEY,
bits_builder);
ASSERT_EQ(0, builder.NumAdded());
builder.Add("key"); // test with empty prefix
ASSERT_EQ(2, bits_builder->hash_entries_.size());
}
// mix of empty and non-empty
std::unique_ptr<const SliceTransform> prefix_extractor(
NewFixedPrefixTransform(7));
auto bits_builder = dynamic_cast<FullFilterBitsBuilder*>(
table_options_.filter_policy->GetFilterBitsBuilder());
const bool WHOLE_KEY = true;
FullFilterBlockBuilder builder(prefix_extractor.get(), WHOLE_KEY,
bits_builder);
ASSERT_EQ(0, builder.NumAdded());
builder.Add(""); // test with empty key too
builder.Add("prefix1key1");
builder.Add("prefix1key1");
builder.Add("prefix1key2");
builder.Add("prefix1key3");
builder.Add("prefix2key4");
// two prefix adn 4 keys
ASSERT_EQ(1 + 2 + 4, bits_builder->hash_entries_.size());
}
TEST_F(FullFilterBlockTest, SingleChunk) { TEST_F(FullFilterBlockTest, SingleChunk) {
FullFilterBlockBuilder builder( FullFilterBlockBuilder builder(
nullptr, true, table_options_.filter_policy->GetFilterBitsBuilder()); nullptr, true, table_options_.filter_policy->GetFilterBitsBuilder());

View File

@ -95,10 +95,13 @@ void GetContext::RecordCounters(Tickers ticker, size_t val) {
} }
bool GetContext::SaveValue(const ParsedInternalKey& parsed_key, bool GetContext::SaveValue(const ParsedInternalKey& parsed_key,
const Slice& value, Cleanable* value_pinner) { const Slice& value, bool* matched,
Cleanable* value_pinner) {
assert(matched);
assert((state_ != kMerge && parsed_key.type != kTypeMerge) || assert((state_ != kMerge && parsed_key.type != kTypeMerge) ||
merge_context_ != nullptr); merge_context_ != nullptr);
if (ucmp_->Equal(parsed_key.user_key, user_key_)) { if (ucmp_->Equal(parsed_key.user_key, user_key_)) {
*matched = true;
// If the value is not in the snapshot, skip it // If the value is not in the snapshot, skip it
if (!CheckCallback(parsed_key.sequence)) { if (!CheckCallback(parsed_key.sequence)) {
return true; // to continue to the next seq return true; // to continue to the next seq
@ -231,11 +234,12 @@ void replayGetContextLog(const Slice& replay_log, const Slice& user_key,
assert(ret); assert(ret);
(void)ret; (void)ret;
bool dont_care __attribute__((__unused__));
// Since SequenceNumber is not stored and unknown, we will use // Since SequenceNumber is not stored and unknown, we will use
// kMaxSequenceNumber. // kMaxSequenceNumber.
get_context->SaveValue( get_context->SaveValue(
ParsedInternalKey(user_key, kMaxSequenceNumber, type), value, ParsedInternalKey(user_key, kMaxSequenceNumber, type), value,
value_pinner); &dont_care, value_pinner);
} }
#else // ROCKSDB_LITE #else // ROCKSDB_LITE
assert(false); assert(false);

View File

@ -42,10 +42,13 @@ class GetContext {
// Records this key, value, and any meta-data (such as sequence number and // Records this key, value, and any meta-data (such as sequence number and
// state) into this GetContext. // state) into this GetContext.
// //
// If the parsed_key matches the user key that we are looking for, sets
// mathced to true.
//
// Returns True if more keys need to be read (due to merges) or // Returns True if more keys need to be read (due to merges) or
// False if the complete value has been found. // False if the complete value has been found.
bool SaveValue(const ParsedInternalKey& parsed_key, const Slice& value, bool SaveValue(const ParsedInternalKey& parsed_key, const Slice& value,
Cleanable* value_pinner = nullptr); bool* matched, Cleanable* value_pinner = nullptr);
// Simplified version of the previous function. Should only be used when we // Simplified version of the previous function. Should only be used when we
// know that the operation is a Put. // know that the operation is a Put.

View File

@ -193,25 +193,13 @@ class MergingIterator : public InternalIterator {
InitMaxHeap(); InitMaxHeap();
for (auto& child : children_) { for (auto& child : children_) {
if (&child != current_) { if (&child != current_) {
if (!prefix_seek_mode_) {
child.Seek(key());
if (child.Valid()) {
// Child is at first entry >= key(). Step back one to be < key()
TEST_SYNC_POINT_CALLBACK("MergeIterator::Prev:BeforePrev",
&child);
child.Prev();
} else {
// Child has no entries >= key(). Position at last entry.
TEST_SYNC_POINT("MergeIterator::Prev:BeforeSeekToLast");
child.SeekToLast();
}
} else {
child.SeekForPrev(key()); child.SeekForPrev(key());
// Child is at first entry >= key(). Step back one to be < key()
TEST_SYNC_POINT_CALLBACK("MergeIterator::Prev:BeforePrev", &child);
if (child.Valid() && comparator_->Equal(key(), child.key())) { if (child.Valid() && comparator_->Equal(key(), child.key())) {
child.Prev(); child.Prev();
} }
} }
}
if (child.Valid()) { if (child.Valid()) {
maxHeap_->push(&child); maxHeap_->push(&child);
} }

View File

@ -41,7 +41,8 @@ Status MockTableReader::Get(const ReadOptions&, const Slice& key,
return Status::Corruption(Slice()); return Status::Corruption(Slice());
} }
if (!get_context->SaveValue(parsed_key, iter->value())) { bool dont_care __attribute__((__unused__));
if (!get_context->SaveValue(parsed_key, iter->value(), &dont_care)) {
break; break;
} }
} }

View File

@ -246,6 +246,13 @@ size_t PartitionedFilterBlockReader::ApproximateMemoryUsage() const {
return idx_on_fltr_blk_->size(); return idx_on_fltr_blk_->size();
} }
// Release the cached entry and decrement its ref count.
void ReleaseFilterCachedEntry(void* arg, void* h) {
Cache* cache = reinterpret_cast<Cache*>(arg);
Cache::Handle* handle = reinterpret_cast<Cache::Handle*>(h);
cache->Release(handle);
}
// TODO(myabandeh): merge this with the same function in IndexReader // TODO(myabandeh): merge this with the same function in IndexReader
void PartitionedFilterBlockReader::CacheDependencies(bool pin) { void PartitionedFilterBlockReader::CacheDependencies(bool pin) {
// Before read partitions, prefetch them to avoid lots of IOs // Before read partitions, prefetch them to avoid lots of IOs
@ -304,6 +311,8 @@ void PartitionedFilterBlockReader::CacheDependencies(bool pin) {
if (LIKELY(filter.IsSet())) { if (LIKELY(filter.IsSet())) {
if (pin) { if (pin) {
filter_map_[handle.offset()] = std::move(filter); filter_map_[handle.offset()] = std::move(filter);
RegisterCleanup(&ReleaseFilterCachedEntry, block_cache,
filter.cache_handle);
} else { } else {
block_cache->Release(filter.cache_handle); block_cache->Release(filter.cache_handle);
} }

View File

@ -65,7 +65,8 @@ class PartitionedFilterBlockBuilder : public FullFilterBlockBuilder {
size_t num_added_; size_t num_added_;
}; };
class PartitionedFilterBlockReader : public FilterBlockReader { class PartitionedFilterBlockReader : public FilterBlockReader,
public Cleanable {
public: public:
explicit PartitionedFilterBlockReader(const SliceTransform* prefix_extractor, explicit PartitionedFilterBlockReader(const SliceTransform* prefix_extractor,
bool whole_key_filtering, bool whole_key_filtering,

View File

@ -594,7 +594,8 @@ Status PlainTableReader::Get(const ReadOptions& /*ro*/, const Slice& target,
// TODO(ljin): since we know the key comparison result here, // TODO(ljin): since we know the key comparison result here,
// can we enable the fast path? // can we enable the fast path?
if (internal_comparator_.Compare(found_key, parsed_target) >= 0) { if (internal_comparator_.Compare(found_key, parsed_target) >= 0) {
if (!get_context->SaveValue(found_key, found_value)) { bool dont_care __attribute__((__unused__));
if (!get_context->SaveValue(found_key, found_value, &dont_care)) {
break; break;
} }
} }

View File

@ -302,9 +302,11 @@ class KeyConvertingIterator : public InternalIterator {
class TableConstructor: public Constructor { class TableConstructor: public Constructor {
public: public:
explicit TableConstructor(const Comparator* cmp, explicit TableConstructor(const Comparator* cmp,
bool convert_to_internal_key = false) bool convert_to_internal_key = false,
int level = -1)
: Constructor(cmp), : Constructor(cmp),
convert_to_internal_key_(convert_to_internal_key) {} convert_to_internal_key_(convert_to_internal_key),
level_(level) {}
~TableConstructor() { Reset(); } ~TableConstructor() { Reset(); }
virtual Status FinishImpl(const Options& options, virtual Status FinishImpl(const Options& options,
@ -319,14 +321,12 @@ class TableConstructor: public Constructor {
std::vector<std::unique_ptr<IntTblPropCollectorFactory>> std::vector<std::unique_ptr<IntTblPropCollectorFactory>>
int_tbl_prop_collector_factories; int_tbl_prop_collector_factories;
std::string column_family_name; std::string column_family_name;
int unknown_level = -1;
builder.reset(ioptions.table_factory->NewTableBuilder( builder.reset(ioptions.table_factory->NewTableBuilder(
TableBuilderOptions(ioptions, internal_comparator, TableBuilderOptions(
&int_tbl_prop_collector_factories, ioptions, internal_comparator, &int_tbl_prop_collector_factories,
options.compression, CompressionOptions(), options.compression, CompressionOptions(),
nullptr /* compression_dict */, nullptr /* compression_dict */, false /* skip_filters */,
false /* skip_filters */, column_family_name, column_family_name, level_),
unknown_level),
TablePropertiesCollectorFactory::Context::kUnknownColumnFamily, TablePropertiesCollectorFactory::Context::kUnknownColumnFamily,
file_writer_.get())); file_writer_.get()));
@ -351,8 +351,10 @@ class TableConstructor: public Constructor {
uniq_id_ = cur_uniq_id_++; uniq_id_ = cur_uniq_id_++;
file_reader_.reset(test::GetRandomAccessFileReader(new test::StringSource( file_reader_.reset(test::GetRandomAccessFileReader(new test::StringSource(
GetSink()->contents(), uniq_id_, ioptions.allow_mmap_reads))); GetSink()->contents(), uniq_id_, ioptions.allow_mmap_reads)));
const bool skip_filters = false;
return ioptions.table_factory->NewTableReader( return ioptions.table_factory->NewTableReader(
TableReaderOptions(ioptions, soptions, internal_comparator), TableReaderOptions(ioptions, soptions, internal_comparator,
skip_filters, level_),
std::move(file_reader_), GetSink()->contents().size(), &table_reader_); std::move(file_reader_), GetSink()->contents().size(), &table_reader_);
} }
@ -412,6 +414,7 @@ class TableConstructor: public Constructor {
unique_ptr<RandomAccessFileReader> file_reader_; unique_ptr<RandomAccessFileReader> file_reader_;
unique_ptr<TableReader> table_reader_; unique_ptr<TableReader> table_reader_;
bool convert_to_internal_key_; bool convert_to_internal_key_;
int level_;
TableConstructor(); TableConstructor();
@ -2249,6 +2252,7 @@ std::map<std::string, size_t> MockCache::marked_data_in_cache_;
// table is closed. This test makes sure that the only items remains in the // table is closed. This test makes sure that the only items remains in the
// cache after the table is closed are raw data blocks. // cache after the table is closed are raw data blocks.
TEST_F(BlockBasedTableTest, NoObjectInCacheAfterTableClose) { TEST_F(BlockBasedTableTest, NoObjectInCacheAfterTableClose) {
for (int level: {-1, 0, 1, 10}) {
for (auto index_type : for (auto index_type :
{BlockBasedTableOptions::IndexType::kBinarySearch, {BlockBasedTableOptions::IndexType::kBinarySearch,
BlockBasedTableOptions::IndexType::kTwoLevelIndexSearch}) { BlockBasedTableOptions::IndexType::kTwoLevelIndexSearch}) {
@ -2285,7 +2289,9 @@ TEST_F(BlockBasedTableTest, NoObjectInCacheAfterTableClose) {
rocksdb::NewBloomFilterPolicy(10, block_based_filter)); rocksdb::NewBloomFilterPolicy(10, block_based_filter));
opt.table_factory.reset(NewBlockBasedTableFactory(table_options)); opt.table_factory.reset(NewBlockBasedTableFactory(table_options));
TableConstructor c(BytewiseComparator()); bool convert_to_internal_key = false;
TableConstructor c(BytewiseComparator(), convert_to_internal_key,
level);
std::string user_key = "k01"; std::string user_key = "k01";
std::string key = std::string key =
InternalKey(user_key, 0, kTypeValue).Encode().ToString(); InternalKey(user_key, 0, kTypeValue).Encode().ToString();
@ -2326,6 +2332,7 @@ TEST_F(BlockBasedTableTest, NoObjectInCacheAfterTableClose) {
} }
} }
} }
} // level
} }
TEST_F(BlockBasedTableTest, BlockCacheLeak) { TEST_F(BlockBasedTableTest, BlockCacheLeak) {

View File

@ -24,7 +24,11 @@ class TwoLevelIterator : public InternalIterator {
explicit TwoLevelIterator(TwoLevelIteratorState* state, explicit TwoLevelIterator(TwoLevelIteratorState* state,
InternalIterator* first_level_iter); InternalIterator* first_level_iter);
virtual ~TwoLevelIterator() { delete state_; } virtual ~TwoLevelIterator() {
first_level_iter_.DeleteIter(false /* is_arena_mode */);
second_level_iter_.DeleteIter(false /* is_arena_mode */);
delete state_;
}
virtual void Seek(const Slice& target) override; virtual void Seek(const Slice& target) override;
virtual void SeekForPrev(const Slice& target) override; virtual void SeekForPrev(const Slice& target) override;

View File

@ -16,8 +16,8 @@ namespace rocksdb {
struct ReadOptions; struct ReadOptions;
class InternalKeyComparator; class InternalKeyComparator;
class Arena;
// TwoLevelIteratorState expects iterators are not created using the arena
struct TwoLevelIteratorState { struct TwoLevelIteratorState {
TwoLevelIteratorState() {} TwoLevelIteratorState() {}
@ -35,11 +35,7 @@ struct TwoLevelIteratorState {
// //
// Uses a supplied function to convert an index_iter value into // Uses a supplied function to convert an index_iter value into
// an iterator over the contents of the corresponding block. // an iterator over the contents of the corresponding block.
// arena: If not null, the arena is used to allocate the Iterator. // Note: this function expects first_level_iter was not created using the arena
// When destroying the iterator, the destructor will destroy
// all the states but those allocated in arena.
// need_free_iter_and_state: free `state` and `first_level_iter` if
// true. Otherwise, just call destructor.
extern InternalIterator* NewTwoLevelIterator( extern InternalIterator* NewTwoLevelIterator(
TwoLevelIteratorState* state, InternalIterator* first_level_iter); TwoLevelIteratorState* state, InternalIterator* first_level_iter);

Some files were not shown because too many files have changed in this diff Show More