diff --git a/.github/workflows/nightly.yml b/.github/workflows/nightly.yml index 1749b52ba7e..3c0ad9f2559 100644 --- a/.github/workflows/nightly.yml +++ b/.github/workflows/nightly.yml @@ -96,3 +96,13 @@ jobs: steps: - uses: actions/checkout@v4.1.0 - uses: "./.github/actions/windows-build-steps" + build-linux-arm-test-full: + if: ${{ github.repository_owner == 'facebook' }} + runs-on: + labels: 4-core-ubuntu-arm + steps: + - uses: actions/checkout@v4.1.0 + - uses: "./.github/actions/pre-steps" + - run: sudo apt-get update && sudo apt-get install -y build-essential libgflags-dev + - run: make V=1 J=4 -j4 check + - uses: "./.github/actions/post-steps" diff --git a/.github/workflows/pr-jobs.yml b/.github/workflows/pr-jobs.yml index 00f1305a2e4..eaab8e6d732 100644 --- a/.github/workflows/pr-jobs.yml +++ b/.github/workflows/pr-jobs.yml @@ -607,3 +607,13 @@ jobs: with: name: maven-site path: "${{ github.workspace }}/java/target/site" + build-linux-arm: + if: ${{ github.repository_owner == 'facebook' }} + runs-on: + labels: 4-core-ubuntu-arm + steps: + - uses: actions/checkout@v4.1.0 + - uses: "./.github/actions/pre-steps" + - run: sudo apt-get update && sudo apt-get install -y build-essential + - run: ROCKSDBTESTS_PLATFORM_DEPENDENT=only make V=1 J=4 -j4 all_but_some_tests check_some + - uses: "./.github/actions/post-steps" diff --git a/CMakeLists.txt b/CMakeLists.txt index d936ea5426e..51943b1a125 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -667,6 +667,7 @@ set(SOURCES cache/sharded_cache.cc cache/tiered_secondary_cache.cc db/arena_wrapped_db_iter.cc + db/attribute_group_iterator_impl.cc db/blob/blob_contents.cc db/blob/blob_fetcher.cc db/blob/blob_file_addition.cc @@ -683,6 +684,7 @@ set(SOURCES db/blob/prefetch_buffer_collection.cc db/builder.cc db/c.cc + db/coalescing_iterator.cc db/column_family.cc db/compaction/compaction.cc db/compaction/compaction_iterator.cc @@ -729,7 +731,6 @@ set(SOURCES db/memtable_list.cc db/merge_helper.cc db/merge_operator.cc - db/multi_cf_iterator.cc db/output_validator.cc db/periodic_task_scheduler.cc db/range_del_aggregator.cc @@ -968,6 +969,7 @@ set(SOURCES utilities/transactions/write_prepared_txn_db.cc utilities/transactions/write_unprepared_txn.cc utilities/transactions/write_unprepared_txn_db.cc + utilities/types_util.cc utilities/ttl/db_ttl_impl.cc utilities/wal_filter.cc utilities/write_batch_with_index/write_batch_with_index.cc @@ -1049,8 +1051,10 @@ endif() else() list(APPEND SOURCES + db/db_impl/db_impl_follower.cc port/port_posix.cc env/env_posix.cc + env/fs_on_demand.cc env/fs_posix.cc env/io_posix.cc) endif() @@ -1378,6 +1382,7 @@ if(WITH_TESTS) db/file_indexer_test.cc db/filename_test.cc db/flush_job_test.cc + db/db_follower_test.cc db/import_column_family_test.cc db/listener_test.cc db/log_test.cc @@ -1501,6 +1506,7 @@ if(WITH_TESTS) utilities/transactions/lock/range/range_locking_test.cc utilities/transactions/timestamped_snapshot_test.cc utilities/ttl/ttl_test.cc + utilities/types_util_test.cc utilities/util_merge_operators_test.cc utilities/write_batch_with_index/write_batch_with_index_test.cc ${PLUGIN_TESTS} diff --git a/HISTORY.md b/HISTORY.md index 058f326c618..feffb34b4cb 100644 --- a/HISTORY.md +++ b/HISTORY.md @@ -1,6 +1,43 @@ # Rocksdb Change Log > NOTE: Entries for next release do not go here. Follow instructions in `unreleased_history/README.txt` +## 9.2.0 (05/01/2024) +### New Features +* Added two options `deadline` and `max_size_bytes` for CacheDumper to exit early +* Added a new API `GetEntityFromBatchAndDB` to `WriteBatchWithIndex` that can be used for wide-column point lookups with read-your-own-writes consistency. Similarly to `GetFromBatchAndDB`, the API can combine data from the write batch with data from the underlying database if needed. See the API comments for more details. +* [Experimental] Introduce two new cross-column-family iterators - CoalescingIterator and AttributeGroupIterator. The CoalescingIterator enables users to iterate over multiple column families and access their values and columns. During this iteration, if the same key exists in more than one column family, the keys in the later column family will overshadow the previous ones. The AttributeGroupIterator allows users to gather wide columns per Column Family and create attribute groups while iterating over keys across all CFs. +* Added a new API `MultiGetEntityFromBatchAndDB` to `WriteBatchWithIndex` that can be used for batched wide-column point lookups with read-your-own-writes consistency. Similarly to `MultiGetFromBatchAndDB`, the API can combine data from the write batch with data from the underlying database if needed. See the API comments for more details. +* *Adds a `SstFileReader::NewTableIterator` API to support programmatically read a SST file as a raw table file. +* Add an option to `WaitForCompactOptions` - `wait_for_purge` to make `WaitForCompact()` API wait for background purge to complete + +### Public API Changes +* DeleteRange() will return NotSupported() if row_cache is configured since they don't work together in some cases. +* Deprecated `CompactionOptions::compression` since `CompactionOptions`'s API for configuring compression was incomplete, unsafe, and likely unnecessary +* Using `OptionChangeMigration()` to migrate from non-FIFO to FIFO compaction +with `Options::compaction_options_fifo.max_table_files_size` > 0 can cause +the whole DB to be dropped right after migration if the migrated data is larger than +`max_table_files_size` + +### Behavior Changes +* Enabling `BlockBasedTableOptions::block_align` is now incompatible (i.e., APIs will return `Status::InvalidArgument`) with more ways of enabling compression: `CompactionOptions::compression`, `ColumnFamilyOptions::compression_per_level`, and `ColumnFamilyOptions::bottommost_compression`. +* Changed the default value of `CompactionOptions::compression` to `kDisableCompressionOption`, which means the compression type is determined by the `ColumnFamilyOptions`. +* `BlockBasedTableOptions::optimize_filters_for_memory` is now set to true by default. When `partition_filters=false`, this could lead to somewhat increased average RSS memory usage by the block cache, but this "extra" usage is within the allowed memory budget and should make memory usage more consistent (by minimizing internal fragmentation for more kinds of blocks). +* Dump all keys for cache dumper impl if `SetDumpFilter()` is not called +* `CompactRange()` with `CompactRangeOptions::change_level = true` and `CompactRangeOptions::target_level = 0` that ends up moving more than 1 file from non-L0 to L0 will return `Status::Aborted()`. +* On distributed file systems that support file system level checksum verification and reconstruction reads, RocksDB will now retry a file read if the initial read fails RocksDB block level or record level checksum verification. This applies to MANIFEST file reads when the DB is opened, and to SST file reads at all times. + +### Bug Fixes +* Fix a bug causing `VerifyFileChecksums()` to return false-positive corruption under `BlockBasedTableOptions::block_align=true` +* Provide consistent view of the database across the column families for `NewIterators()` API. +* Fixed feature interaction bug for `DeleteRange()` together with `ColumnFamilyOptions::memtable_insert_with_hint_prefix_extractor`. The impact of this bug would likely be corruption or crashing. +* Fixed hang in `DisableManualCompactions()` where compactions waiting to be scheduled due to conflicts would not be canceled promptly +* Fixed a regression when `ColumnFamilyOptions::max_successive_merges > 0` where the CPU overhead for deciding whether to merge could have increased unless the user had set the option `ColumnFamilyOptions::strict_max_successive_merges` +* Fixed a bug in `MultiGet()` and `MultiGetEntity()` together with blob files (`ColumnFamilyOptions::enable_blob_files == true`). An error looking up one of the keys could cause the results to be wrong for other keys for which the statuses were `Status::OK`. +* Fixed a bug where wrong padded bytes are used to generate file checksum and `DataVerificationInfo::checksum` upon file creation +* Correctly implemented the move semantics of `PinnableWideColumns`. +* Fixed a bug when the recycle_log_file_num in DBOptions is changed from 0 to non-zero when a DB is reopened. On a subsequent reopen, if a log file created when recycle_log_file_num==0 was reused previously, is alive and is empty, we could end up inserting stale WAL records into the memtable. +* *Fix a bug where obsolete files' deletion during DB::Open are not rate limited with `SstFilemManager`'s slow deletion feature even if it's configured. + ## 9.1.0 (03/22/2024) ### New Features * Added an option, `GetMergeOperandsOptions::continue_cb`, to give users the ability to end `GetMergeOperands()`'s lookup process before all merge operands were found. diff --git a/Makefile b/Makefile index cafefb49546..f22727f9254 100644 --- a/Makefile +++ b/Makefile @@ -1610,6 +1610,9 @@ object_registry_test: $(OBJ_DIR)/utilities/object_registry_test.o $(TEST_LIBRARY ttl_test: $(OBJ_DIR)/utilities/ttl/ttl_test.o $(TEST_LIBRARY) $(LIBRARY) $(AM_LINK) +types_util_test: $(OBJ_DIR)/utilities/types_util_test.o $(TEST_LIBRARY) $(LIBRARY) + $(AM_LINK) + write_batch_with_index_test: $(OBJ_DIR)/utilities/write_batch_with_index/write_batch_with_index_test.o $(TEST_LIBRARY) $(LIBRARY) $(AM_LINK) @@ -1919,6 +1922,9 @@ sst_file_reader_test: $(OBJ_DIR)/table/sst_file_reader_test.o $(TEST_LIBRARY) $( db_secondary_test: $(OBJ_DIR)/db/db_secondary_test.o $(TEST_LIBRARY) $(LIBRARY) $(AM_LINK) +db_follower_test: $(OBJ_DIR)/db/db_follower_test.o $(TEST_LIBRARY) $(LIBRARY) + $(AM_LINK) + block_cache_tracer_test: $(OBJ_DIR)/trace_replay/block_cache_tracer_test.o $(TEST_LIBRARY) $(LIBRARY) $(AM_LINK) diff --git a/TARGETS b/TARGETS index 73ddeabc310..cdfcdc701cd 100644 --- a/TARGETS +++ b/TARGETS @@ -21,6 +21,7 @@ cpp_library_wrapper(name="rocksdb_lib", srcs=[ "cache/sharded_cache.cc", "cache/tiered_secondary_cache.cc", "db/arena_wrapped_db_iter.cc", + "db/attribute_group_iterator_impl.cc", "db/blob/blob_contents.cc", "db/blob/blob_fetcher.cc", "db/blob/blob_file_addition.cc", @@ -37,6 +38,7 @@ cpp_library_wrapper(name="rocksdb_lib", srcs=[ "db/blob/prefetch_buffer_collection.cc", "db/builder.cc", "db/c.cc", + "db/coalescing_iterator.cc", "db/column_family.cc", "db/compaction/compaction.cc", "db/compaction/compaction_iterator.cc", @@ -58,6 +60,7 @@ cpp_library_wrapper(name="rocksdb_lib", srcs=[ "db/db_impl/db_impl_debug.cc", "db/db_impl/db_impl_experimental.cc", "db/db_impl/db_impl_files.cc", + "db/db_impl/db_impl_follower.cc", "db/db_impl/db_impl_open.cc", "db/db_impl/db_impl_readonly.cc", "db/db_impl/db_impl_secondary.cc", @@ -83,7 +86,6 @@ cpp_library_wrapper(name="rocksdb_lib", srcs=[ "db/memtable_list.cc", "db/merge_helper.cc", "db/merge_operator.cc", - "db/multi_cf_iterator.cc", "db/output_validator.cc", "db/periodic_task_scheduler.cc", "db/range_del_aggregator.cc", @@ -116,6 +118,7 @@ cpp_library_wrapper(name="rocksdb_lib", srcs=[ "env/env_posix.cc", "env/file_system.cc", "env/file_system_tracer.cc", + "env/fs_on_demand.cc", "env/fs_posix.cc", "env/fs_remap.cc", "env/io_posix.cc", @@ -346,6 +349,7 @@ cpp_library_wrapper(name="rocksdb_lib", srcs=[ "utilities/transactions/write_unprepared_txn.cc", "utilities/transactions/write_unprepared_txn_db.cc", "utilities/ttl/db_ttl_impl.cc", + "utilities/types_util.cc", "utilities/wal_filter.cc", "utilities/write_batch_with_index/write_batch_with_index.cc", "utilities/write_batch_with_index/write_batch_with_index_internal.cc", @@ -4793,6 +4797,12 @@ cpp_unittest_wrapper(name="db_flush_test", extra_compiler_flags=[]) +cpp_unittest_wrapper(name="db_follower_test", + srcs=["db/db_follower_test.cc"], + deps=[":rocksdb_test_lib"], + extra_compiler_flags=[]) + + cpp_unittest_wrapper(name="db_inplace_update_test", srcs=["db/db_inplace_update_test.cc"], deps=[":rocksdb_test_lib"], @@ -5527,6 +5537,12 @@ cpp_unittest_wrapper(name="ttl_test", extra_compiler_flags=[]) +cpp_unittest_wrapper(name="types_util_test", + srcs=["utilities/types_util_test.cc"], + deps=[":rocksdb_test_lib"], + extra_compiler_flags=[]) + + cpp_unittest_wrapper(name="udt_util_test", srcs=["util/udt_util_test.cc"], deps=[":rocksdb_test_lib"], diff --git a/build_tools/build_detect_platform b/build_tools/build_detect_platform index 72983c188f2..2e6d7bc20ac 100755 --- a/build_tools/build_detect_platform +++ b/build_tools/build_detect_platform @@ -163,24 +163,6 @@ case "$TARGET_OS" in PLATFORM_LDFLAGS="$PLATFORM_LDFLAGS -latomic" fi PLATFORM_LDFLAGS="$PLATFORM_LDFLAGS -lpthread -lrt -ldl" - if test -z "$ROCKSDB_USE_IO_URING"; then - ROCKSDB_USE_IO_URING=1 - fi - if test "$ROCKSDB_USE_IO_URING" -ne 0; then - # check for liburing - $CXX $PLATFORM_CXXFLAGS -x c++ - -luring -o test.o 2>/dev/null < - int main() { - struct io_uring ring; - io_uring_queue_init(1, &ring, 0); - return 0; - } -EOF - if [ "$?" = 0 ]; then - PLATFORM_LDFLAGS="$PLATFORM_LDFLAGS -luring" - COMMON_FLAGS="$COMMON_FLAGS -DROCKSDB_IOURING_PRESENT" - fi - fi # PORT_FILES=port/linux/linux_specific.cc ;; SunOS) @@ -614,6 +596,24 @@ EOF fi fi + if test -z "$ROCKSDB_USE_IO_URING"; then + ROCKSDB_USE_IO_URING=1 + fi + if [ "$ROCKSDB_USE_IO_URING" -ne 0 -a "$PLATFORM" = OS_LINUX ]; then + # check for liburing + $CXX $PLATFORM_CXXFLAGS -x c++ - -luring -o test.o 2>/dev/null < + int main() { + struct io_uring ring; + io_uring_queue_init(1, &ring, 0); + return 0; + } +EOF + if [ "$?" = 0 ]; then + PLATFORM_LDFLAGS="$PLATFORM_LDFLAGS -luring" + COMMON_FLAGS="$COMMON_FLAGS -DROCKSDB_IOURING_PRESENT" + fi + fi fi # TODO(tec): Fix -Wshorten-64-to-32 errors on FreeBSD and enable the warning. diff --git a/build_tools/fbcode_config.sh b/build_tools/fbcode_config.sh index fa629af9780..02732bde3d1 100644 --- a/build_tools/fbcode_config.sh +++ b/build_tools/fbcode_config.sh @@ -113,7 +113,7 @@ CLANG_LIB="$CLANG_BASE/lib" CLANG_SRC="$CLANG_BASE/../../src" CLANG_ANALYZER="$CLANG_BIN/clang++" -CLANG_SCAN_BUILD="$CLANG_SRC/llvm/tools/clang/tools/scan-build/bin/scan-build" +CLANG_SCAN_BUILD="$CLANG_BIN/scan-build if [ -z "$USE_CLANG" ]; then # gcc diff --git a/build_tools/fbcode_config_platform010.sh b/build_tools/fbcode_config_platform010.sh index 25835d09108..143954d210b 100644 --- a/build_tools/fbcode_config_platform010.sh +++ b/build_tools/fbcode_config_platform010.sh @@ -110,7 +110,7 @@ CLANG_LIB="$CLANG_BASE/lib" CLANG_SRC="$CLANG_BASE/../../src" CLANG_ANALYZER="$CLANG_BIN/clang++" -CLANG_SCAN_BUILD="$CLANG_SRC/llvm/clang/tools/scan-build/bin/scan-build" +CLANG_SCAN_BUILD="$CLANG_BIN/scan-build" if [ -z "$USE_CLANG" ]; then # gcc diff --git a/cache/cache_reservation_manager_test.cc b/cache/cache_reservation_manager_test.cc index 2a0c318e090..132975ce285 100644 --- a/cache/cache_reservation_manager_test.cc +++ b/cache/cache_reservation_manager_test.cc @@ -129,7 +129,6 @@ TEST_F(CacheReservationManagerTest, TEST(CacheReservationManagerIncreaseReservcationOnFullCacheTest, IncreaseCacheReservationOnFullCache) { - ; constexpr std::size_t kSizeDummyEntry = CacheReservationManagerImpl::GetDummyEntrySize(); constexpr std::size_t kSmallCacheCapacity = 4 * kSizeDummyEntry; diff --git a/db/attribute_group_iterator_impl.cc b/db/attribute_group_iterator_impl.cc new file mode 100644 index 00000000000..747005ec454 --- /dev/null +++ b/db/attribute_group_iterator_impl.cc @@ -0,0 +1,20 @@ +// Copyright (c) Meta Platforms, Inc. and affiliates. +// 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/attribute_group_iterator_impl.h" + +namespace ROCKSDB_NAMESPACE { + +const AttributeGroups kNoAttributeGroups; +const IteratorAttributeGroups kNoIteratorAttributeGroups; + +void AttributeGroupIteratorImpl::AddToAttributeGroups( + const autovector& items) { + for (const auto& item : items) { + attribute_groups_.emplace_back(item.cfh, &item.iterator->columns()); + } +} + +} // namespace ROCKSDB_NAMESPACE diff --git a/db/attribute_group_iterator_impl.h b/db/attribute_group_iterator_impl.h new file mode 100644 index 00000000000..3977fe42827 --- /dev/null +++ b/db/attribute_group_iterator_impl.h @@ -0,0 +1,83 @@ +// Copyright (c) Meta Platforms, Inc. and affiliates. +// 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 "db/multi_cf_iterator_impl.h" +#include "rocksdb/attribute_groups.h" + +namespace ROCKSDB_NAMESPACE { + +class AttributeGroupIteratorImpl : public AttributeGroupIterator { + public: + AttributeGroupIteratorImpl( + const Comparator* comparator, + const std::vector& column_families, + const std::vector& child_iterators) + : impl_( + comparator, column_families, child_iterators, [this]() { Reset(); }, + [this](const autovector& items) { + AddToAttributeGroups(items); + }) {} + ~AttributeGroupIteratorImpl() override {} + + // No copy allowed + AttributeGroupIteratorImpl(const AttributeGroupIteratorImpl&) = delete; + AttributeGroupIteratorImpl& operator=(const AttributeGroupIteratorImpl&) = + delete; + + bool Valid() const override { return impl_.Valid(); } + void SeekToFirst() override { impl_.SeekToFirst(); } + void SeekToLast() override { impl_.SeekToLast(); } + void Seek(const Slice& target) override { impl_.Seek(target); } + void SeekForPrev(const Slice& target) override { impl_.SeekForPrev(target); } + void Next() override { impl_.Next(); } + void Prev() override { impl_.Prev(); } + Slice key() const override { return impl_.key(); } + Status status() const override { return impl_.status(); } + + const IteratorAttributeGroups& attribute_groups() const override { + assert(Valid()); + return attribute_groups_; + } + + void Reset() { attribute_groups_.clear(); } + + private: + MultiCfIteratorImpl impl_; + IteratorAttributeGroups attribute_groups_; + void AddToAttributeGroups(const autovector& items); +}; + +class EmptyAttributeGroupIterator : public AttributeGroupIterator { + public: + explicit EmptyAttributeGroupIterator(const Status& s) : status_(s) {} + bool Valid() const override { return false; } + void Seek(const Slice& /*target*/) override {} + void SeekForPrev(const Slice& /*target*/) override {} + void SeekToFirst() override {} + void SeekToLast() override {} + void Next() override { assert(false); } + void Prev() override { assert(false); } + Slice key() const override { + assert(false); + return Slice(); + } + Status status() const override { return status_; } + + const IteratorAttributeGroups& attribute_groups() const override { + return kNoIteratorAttributeGroups; + } + + private: + Status status_; +}; + +inline std::unique_ptr NewAttributeGroupErrorIterator( + const Status& status) { + return std::make_unique(status); +} + +} // namespace ROCKSDB_NAMESPACE diff --git a/db/blob/db_blob_basic_test.cc b/db/blob/db_blob_basic_test.cc index e41933d3549..0e17df7aa02 100644 --- a/db/blob/db_blob_basic_test.cc +++ b/db/blob/db_blob_basic_test.cc @@ -1551,6 +1551,57 @@ TEST_P(DBBlobBasicIOErrorMultiGetTest, MultipleBlobFiles) { ASSERT_TRUE(statuses[1].IsIOError()); } +TEST_F(DBBlobBasicTest, MultiGetFindTable_IOError) { + // Repro test for a specific bug where `MultiGet()` would fail to open a table + // in `FindTable()` and then proceed to return raw blob handles for the other + // keys. + Options options = GetDefaultOptions(); + options.enable_blob_files = true; + options.min_blob_size = 0; + + Reopen(options); + + // Force no table cache so every read will preload the SST file. + dbfull()->TEST_table_cache()->SetCapacity(0); + + constexpr size_t num_keys = 2; + + constexpr char key1[] = "key1"; + constexpr char value1[] = "blob1"; + + ASSERT_OK(Put(key1, value1)); + ASSERT_OK(Flush()); + + constexpr char key2[] = "key2"; + constexpr char value2[] = "blob2"; + + ASSERT_OK(Put(key2, value2)); + ASSERT_OK(Flush()); + + std::atomic num_files_opened = 0; + // This test would be more realistic if we injected an `IOError` from the + // `FileSystem` + SyncPoint::GetInstance()->SetCallBack( + "TableCache::MultiGet:FindTable", [&](void* status) { + num_files_opened++; + if (num_files_opened == 2) { + Status* s = static_cast(status); + *s = Status::IOError(); + } + }); + SyncPoint::GetInstance()->EnableProcessing(); + + std::array keys{{key1, key2}}; + std::array values; + std::array statuses; + db_->MultiGet(ReadOptions(), db_->DefaultColumnFamily(), num_keys, + keys.data(), values.data(), statuses.data()); + + ASSERT_TRUE(statuses[0].IsIOError()); + ASSERT_OK(statuses[1]); + ASSERT_EQ(value2, values[1]); +} + namespace { class ReadBlobCompactionFilter : public CompactionFilter { diff --git a/db/builder.cc b/db/builder.cc index a3c15ad11e5..d2557a0b0cc 100644 --- a/db/builder.cc +++ b/db/builder.cc @@ -210,6 +210,7 @@ Status BuildTable( const bool logical_strip_timestamp = ts_sz > 0 && !ioptions.persist_user_defined_timestamps; + SequenceNumber smallest_preferred_seqno = kMaxSequenceNumber; std::string key_after_flush_buf; std::string value_buf; c_iter.SeekToFirst(); @@ -242,6 +243,8 @@ Status BuildTable( if (preferred_seqno < ikey.sequence) { value_after_flush = PackValueAndSeqno(unpacked_value, preferred_seqno, &value_buf); + smallest_preferred_seqno = + std::min(smallest_preferred_seqno, preferred_seqno); } else { // Cannot get a useful preferred seqno, convert it to a kTypeValue. UpdateInternalKey(&key_after_flush_buf, ikey.sequence, kTypeValue); @@ -326,9 +329,10 @@ Status BuildTable( } else { SeqnoToTimeMapping relevant_mapping; if (seqno_to_time_mapping) { - relevant_mapping.CopyFromSeqnoRange(*seqno_to_time_mapping, - meta->fd.smallest_seqno, - meta->fd.largest_seqno); + relevant_mapping.CopyFromSeqnoRange( + *seqno_to_time_mapping, + std::min(meta->fd.smallest_seqno, smallest_preferred_seqno), + meta->fd.largest_seqno); relevant_mapping.SetCapacity(kMaxSeqnoTimePairsPerSST); relevant_mapping.Enforce(tboptions.file_creation_time); } diff --git a/db/c_test.c b/db/c_test.c index 656e2fa5f71..b55b2cc96fa 100644 --- a/db/c_test.c +++ b/db/c_test.c @@ -1318,6 +1318,8 @@ int main(int argc, char** argv) { policy = rocksdb_filterpolicy_create_ribbon_hybrid(8.0, 1); } rocksdb_block_based_options_set_filter_policy(table_options, policy); + rocksdb_block_based_options_set_optimize_filters_for_memory(table_options, + 0); // Create new database rocksdb_close(db); diff --git a/db/coalescing_iterator.cc b/db/coalescing_iterator.cc new file mode 100644 index 00000000000..200cece55ad --- /dev/null +++ b/db/coalescing_iterator.cc @@ -0,0 +1,47 @@ +// Copyright (c) Meta Platforms, Inc. and affiliates. +// 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/coalescing_iterator.h" + +#include "db/wide/wide_columns_helper.h" + +namespace ROCKSDB_NAMESPACE { + +void CoalescingIterator::Coalesce( + const autovector& items) { + assert(wide_columns_.empty()); + MinHeap heap; + for (const auto& item : items) { + assert(item.iterator); + for (auto& column : item.iterator->columns()) { + heap.push(WideColumnWithOrder{&column, item.order}); + } + } + if (heap.empty()) { + return; + } + wide_columns_.reserve(heap.size()); + auto current = heap.top(); + heap.pop(); + while (!heap.empty()) { + int comparison = current.column->name().compare(heap.top().column->name()); + if (comparison < 0) { + wide_columns_.push_back(*current.column); + } else if (comparison > 0) { + // Shouldn't reach here. + // Current item in the heap is greater than the top item in the min heap + assert(false); + } + current = heap.top(); + heap.pop(); + } + wide_columns_.push_back(*current.column); + + if (WideColumnsHelper::HasDefaultColumn(wide_columns_)) { + value_ = WideColumnsHelper::GetDefaultColumn(wide_columns_); + } +} + +} // namespace ROCKSDB_NAMESPACE diff --git a/db/coalescing_iterator.h b/db/coalescing_iterator.h new file mode 100644 index 00000000000..a4d156a6dac --- /dev/null +++ b/db/coalescing_iterator.h @@ -0,0 +1,79 @@ +// Copyright (c) Meta Platforms, Inc. and affiliates. +// 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 "db/multi_cf_iterator_impl.h" + +namespace ROCKSDB_NAMESPACE { + +// EXPERIMENTAL +class CoalescingIterator : public Iterator { + public: + CoalescingIterator(const Comparator* comparator, + const std::vector& column_families, + const std::vector& child_iterators) + : impl_( + comparator, column_families, child_iterators, [this]() { Reset(); }, + [this](const autovector& items) { + Coalesce(items); + }) {} + ~CoalescingIterator() override {} + + // No copy allowed + CoalescingIterator(const CoalescingIterator&) = delete; + CoalescingIterator& operator=(const CoalescingIterator&) = delete; + + bool Valid() const override { return impl_.Valid(); } + void SeekToFirst() override { impl_.SeekToFirst(); } + void SeekToLast() override { impl_.SeekToLast(); } + void Seek(const Slice& target) override { impl_.Seek(target); } + void SeekForPrev(const Slice& target) override { impl_.SeekForPrev(target); } + void Next() override { impl_.Next(); } + void Prev() override { impl_.Prev(); } + Slice key() const override { return impl_.key(); } + Status status() const override { return impl_.status(); } + + Slice value() const override { + assert(Valid()); + return value_; + } + const WideColumns& columns() const override { + assert(Valid()); + return wide_columns_; + } + + void Reset() { + value_.clear(); + wide_columns_.clear(); + } + + private: + MultiCfIteratorImpl impl_; + Slice value_; + WideColumns wide_columns_; + + struct WideColumnWithOrder { + const WideColumn* column; + int order; + }; + + class WideColumnWithOrderComparator { + public: + explicit WideColumnWithOrderComparator() {} + bool operator()(const WideColumnWithOrder& a, + const WideColumnWithOrder& b) const { + int c = a.column->name().compare(b.column->name()); + return c == 0 ? a.order - b.order > 0 : c > 0; + } + }; + + using MinHeap = + BinaryHeap; + + void Coalesce(const autovector& items); +}; + +} // namespace ROCKSDB_NAMESPACE diff --git a/db/compact_files_test.cc b/db/compact_files_test.cc index 129b29c99ff..5a8100041fd 100644 --- a/db/compact_files_test.cc +++ b/db/compact_files_test.cc @@ -441,6 +441,50 @@ TEST_F(CompactFilesTest, SentinelCompressionType) { } } +TEST_F(CompactFilesTest, CompressionWithBlockAlign) { + Options options; + options.compression = CompressionType::kNoCompression; + options.create_if_missing = true; + options.disable_auto_compactions = true; + + std::shared_ptr collector = + std::make_shared(); + options.listeners.push_back(collector); + + { + BlockBasedTableOptions bbto; + bbto.block_align = true; + options.table_factory.reset(NewBlockBasedTableFactory(bbto)); + } + + std::unique_ptr db; + { + DB* _db = nullptr; + ASSERT_OK(DB::Open(options, db_name_, &_db)); + db.reset(_db); + } + + ASSERT_OK(db->Put(WriteOptions(), "key", "val")); + ASSERT_OK(db->Flush(FlushOptions())); + + // Ensure background work is fully finished including listener callbacks + // before accessing listener state. + ASSERT_OK( + static_cast_with_check(db.get())->TEST_WaitForBackgroundWork()); + auto l0_files = collector->GetFlushedFiles(); + ASSERT_EQ(1, l0_files.size()); + + // We can run this test even without Snappy support because we expect the + // `CompactFiles()` to fail before actually invoking Snappy compression. + CompactionOptions compaction_opts; + compaction_opts.compression = CompressionType::kSnappyCompression; + ASSERT_TRUE(db->CompactFiles(compaction_opts, l0_files, 1 /* output_level */) + .IsInvalidArgument()); + + compaction_opts.compression = CompressionType::kDisableCompressionOption; + ASSERT_OK(db->CompactFiles(compaction_opts, l0_files, 1 /* output_level */)); +} + TEST_F(CompactFilesTest, GetCompactionJobInfo) { Options options; options.create_if_missing = true; diff --git a/db/compaction/compaction_iterator.cc b/db/compaction/compaction_iterator.cc index ebda5a6ff77..bc19de8ec28 100644 --- a/db/compaction/compaction_iterator.cc +++ b/db/compaction/compaction_iterator.cc @@ -997,25 +997,37 @@ void CompactionIterator::NextFromInput() { // A special case involving range deletion is handled separately below. auto [unpacked_value, preferred_seqno] = ParsePackedValueWithSeqno(value_); - assert(preferred_seqno < ikey_.sequence); - InternalKey ikey_after_swap(ikey_.user_key, preferred_seqno, kTypeValue); - Slice ikey_after_swap_slice(*ikey_after_swap.rep()); + assert(preferred_seqno < ikey_.sequence || ikey_.sequence == 0); if (range_del_agg_->ShouldDelete( - ikey_after_swap_slice, - RangeDelPositioningMode::kForwardTraversal)) { - // A range tombstone that doesn't cover this kTypeValuePreferredSeqno - // entry may end up covering the entry, so it's not safe to swap - // preferred sequence number. In this case, we output the entry as is. - validity_info_.SetValid(ValidContext::kNewUserKey); + key_, RangeDelPositioningMode::kForwardTraversal)) { + ++iter_stats_.num_record_drop_hidden; + ++iter_stats_.num_record_drop_range_del; + AdvanceInputIter(); } else { - iter_stats_.num_timed_put_swap_preferred_seqno++; - ikey_.sequence = preferred_seqno; - ikey_.type = kTypeValue; - current_key_.UpdateInternalKey(ikey_.sequence, ikey_.type); - key_ = current_key_.GetInternalKey(); - ikey_.user_key = current_key_.GetUserKey(); - value_ = unpacked_value; - validity_info_.SetValid(ValidContext::kSwapPreferredSeqno); + InternalKey ikey_after_swap(ikey_.user_key, + std::min(preferred_seqno, ikey_.sequence), + kTypeValue); + Slice ikey_after_swap_slice(*ikey_after_swap.rep()); + if (range_del_agg_->ShouldDelete( + ikey_after_swap_slice, + RangeDelPositioningMode::kForwardTraversal)) { + // A range tombstone that doesn't cover this kTypeValuePreferredSeqno + // entry will end up covering the entry, so it's not safe to swap + // preferred sequence number. In this case, we output the entry as is. + validity_info_.SetValid(ValidContext::kNewUserKey); + } else { + if (ikey_.sequence != 0) { + iter_stats_.num_timed_put_swap_preferred_seqno++; + saved_seq_for_penul_check_ = ikey_.sequence; + ikey_.sequence = preferred_seqno; + } + ikey_.type = kTypeValue; + current_key_.UpdateInternalKey(ikey_.sequence, ikey_.type); + key_ = current_key_.GetInternalKey(); + ikey_.user_key = current_key_.GetUserKey(); + value_ = unpacked_value; + validity_info_.SetValid(ValidContext::kSwapPreferredSeqno); + } } } else if (ikey_.type == kTypeMerge) { if (!merge_helper_->HasOperator()) { @@ -1275,8 +1287,18 @@ void CompactionIterator::DecideOutputLevel() { // considered unsafe, because the key may conflict with higher-level SSTs // not from this compaction. // TODO: add statistic for declined output_to_penultimate_level + SequenceNumber seq_for_range_check = + (saved_seq_for_penul_check_.has_value() && + saved_seq_for_penul_check_.value() != kMaxSequenceNumber) + ? saved_seq_for_penul_check_.value() + : ikey_.sequence; + ParsedInternalKey ikey_for_range_check = ikey_; + if (seq_for_range_check != ikey_.sequence) { + ikey_for_range_check.sequence = seq_for_range_check; + saved_seq_for_penul_check_ = std::nullopt; + } bool safe_to_penultimate_level = - compaction_->WithinPenultimateLevelOutputRange(ikey_); + compaction_->WithinPenultimateLevelOutputRange(ikey_for_range_check); if (!safe_to_penultimate_level) { output_to_penultimate_level_ = false; // It could happen when disable/enable `last_level_temperature` while diff --git a/db/compaction/compaction_iterator.h b/db/compaction/compaction_iterator.h index eeb75efac4e..5c546feca9c 100644 --- a/db/compaction/compaction_iterator.h +++ b/db/compaction/compaction_iterator.h @@ -437,6 +437,17 @@ class CompactionIterator { // iterator output (or current key in the underlying iterator during // NextFromInput()). ParsedInternalKey ikey_; + + // When a kTypeValuePreferredSeqno entry's preferred seqno is safely swapped + // in in this compaction, this field saves its original sequence number for + // range checking whether it's safe to be placed on the penultimate level. + // This is to ensure when such an entry happens to be the right boundary of + // penultimate safe range, it won't get excluded because with the preferred + // seqno swapped in, it's now larger than the right boundary (itself before + // the swap). This is safe to do, because preferred seqno is swapped in only + // when no entries with the same user key exist on lower levels and this entry + // is already visible in the earliest snapshot. + std::optional saved_seq_for_penul_check_ = kMaxSequenceNumber; // Stores whether ikey_.user_key is valid. If set to false, the user key is // not compared against the current key in the underlying iterator. bool has_current_user_key_ = false; diff --git a/db/compaction/compaction_iterator_test.cc b/db/compaction/compaction_iterator_test.cc index 7558a3e5c75..f2fb7e4bb03 100644 --- a/db/compaction/compaction_iterator_test.cc +++ b/db/compaction/compaction_iterator_test.cc @@ -1260,6 +1260,28 @@ TEST_F(CompactionIteratorWithSnapshotCheckerTest, false /*key_not_exists_beyond_output_level*/); } +TEST_F(CompactionIteratorWithSnapshotCheckerTest, + TimedPut_ShouldBeCoverredByRangeDeletionBeforeSwap_NoOutput) { + InitIterators({test::KeyStr("morning", 5, kTypeValuePreferredSeqno), + test::KeyStr("morning", 2, kTypeValuePreferredSeqno), + test::KeyStr("night", 6, kTypeValue)}, + {ValueWithPreferredSeqno("zao", 3), + ValueWithPreferredSeqno("zao", 1), "wan"}, + {test::KeyStr("ma", 6, kTypeRangeDeletion)}, {"mz"}, 6, + kMaxSequenceNumber /*last_committed_sequence*/, + nullptr /*merge_op*/, nullptr /*filter*/, + false /*bottommost_level*/, + kMaxSequenceNumber /*earliest_write_conflict_snapshot*/, + true /*key_not_exists_beyond_output_level*/); + c_iter_->SeekToFirst(); + ASSERT_TRUE(c_iter_->Valid()); + ASSERT_EQ(test::KeyStr("night", 6, kTypeValue), c_iter_->key().ToString()); + ASSERT_EQ("wan", c_iter_->value().ToString()); + c_iter_->Next(); + ASSERT_FALSE(c_iter_->Valid()); + ASSERT_OK(c_iter_->status()); +} + TEST_F(CompactionIteratorWithSnapshotCheckerTest, TimedPut_WillBeHiddenByRangeDeletionAfterSwap_NoSwap) { InitIterators({test::KeyStr("morning", 5, kTypeValuePreferredSeqno), @@ -1316,6 +1338,19 @@ TEST_F( true /*key_not_exists_beyond_output_level*/); } +TEST_F(CompactionIteratorWithSnapshotCheckerTest, + TimedPut_SequenceNumberAlreadyZeroedOut_ChangeType) { + RunTest( + {test::KeyStr("bar", 0, kTypeValuePreferredSeqno), + test::KeyStr("bar", 0, kTypeValuePreferredSeqno), + test::KeyStr("foo", 0, kTypeValue)}, + {ValueWithPreferredSeqno("bv2", 2), ValueWithPreferredSeqno("bv1", 1), + "fv1"}, + {test::KeyStr("bar", 0, kTypeValue), test::KeyStr("foo", 0, kTypeValue)}, + {"bv2", "fv1"}, 6 /*last_committed_seq*/, nullptr /*merge_operator*/, + nullptr /*compaction_filter*/, true /*bottommost_level*/); +} + // Compaction filter should keep uncommitted key as-is, and // * Convert the latest value to deletion, and/or // * if latest value is a merge, apply filter to all subsequent merges. diff --git a/db/compaction/compaction_outputs.cc b/db/compaction/compaction_outputs.cc index f201da56cd0..e596b13cb09 100644 --- a/db/compaction/compaction_outputs.cc +++ b/db/compaction/compaction_outputs.cc @@ -27,7 +27,9 @@ Status CompactionOutputs::Finish( if (s.ok()) { SeqnoToTimeMapping relevant_mapping; relevant_mapping.CopyFromSeqnoRange( - seqno_to_time_mapping, meta->fd.smallest_seqno, meta->fd.largest_seqno); + seqno_to_time_mapping, + std::min(smallest_preferred_seqno_, meta->fd.smallest_seqno), + meta->fd.largest_seqno); relevant_mapping.SetCapacity(kMaxSeqnoTimePairsPerSST); builder_->SetSeqnoTimeTableProperties(relevant_mapping, meta->oldest_ancester_time); @@ -422,6 +424,11 @@ Status CompactionOutputs::AddToOutput( } const ParsedInternalKey& ikey = c_iter.ikey(); + if (ikey.type == kTypeValuePreferredSeqno) { + SequenceNumber preferred_seqno = ParsePackedValueForSeqno(value); + smallest_preferred_seqno_ = + std::min(smallest_preferred_seqno_, preferred_seqno); + } s = current_output().meta.UpdateBoundaries(key, value, ikey.sequence, ikey.type); diff --git a/db/compaction/compaction_outputs.h b/db/compaction/compaction_outputs.h index f232214e3b7..1b02fb0e9ea 100644 --- a/db/compaction/compaction_outputs.h +++ b/db/compaction/compaction_outputs.h @@ -297,6 +297,7 @@ class CompactionOutputs { std::unique_ptr builder_; std::unique_ptr file_writer_; uint64_t current_output_file_size_ = 0; + SequenceNumber smallest_preferred_seqno_ = kMaxSequenceNumber; // all the compaction outputs so far std::vector outputs_; diff --git a/db/compaction/compaction_picker.cc b/db/compaction/compaction_picker.cc index 53ef7bc6df1..57e4457db12 100644 --- a/db/compaction/compaction_picker.cc +++ b/db/compaction/compaction_picker.cc @@ -457,9 +457,10 @@ bool CompactionPicker::IsRangeInCompaction(VersionStorageInfo* vstorage, // Returns false if files on parent level are currently in compaction, which // means that we can't compact them bool CompactionPicker::SetupOtherInputs( - const std::string& cf_name, VersionStorageInfo* vstorage, - CompactionInputFiles* inputs, CompactionInputFiles* output_level_inputs, - int* parent_index, int base_index, bool only_expand_towards_right) { + const std::string& cf_name, const MutableCFOptions& mutable_cf_options, + VersionStorageInfo* vstorage, CompactionInputFiles* inputs, + CompactionInputFiles* output_level_inputs, int* parent_index, + int base_index, bool only_expand_towards_right) { assert(!inputs->empty()); assert(output_level_inputs->empty()); const int input_level = inputs->level; @@ -525,8 +526,15 @@ bool CompactionPicker::SetupOtherInputs( if (!ExpandInputsToCleanCut(cf_name, vstorage, &expanded_inputs)) { try_overlapping_inputs = false; } + // It helps to reduce write amp and avoid a further separate compaction + // to include more input level files without expanding output level files. + // So we apply a softer limit. We still need a limit to avoid overly large + // compactions and potential high space amp spikes. + const uint64_t limit = + MultiplyCheckOverflow(mutable_cf_options.max_compaction_bytes, 2.0); if (try_overlapping_inputs && expanded_inputs.size() > inputs->size() && - !AreFilesInCompaction(expanded_inputs.files)) { + !AreFilesInCompaction(expanded_inputs.files) && + output_level_inputs_size + expanded_inputs_size < limit) { InternalKey new_start, new_limit; GetRange(expanded_inputs, &new_start, &new_limit); CompactionInputFiles expanded_output_level_inputs; @@ -548,7 +556,8 @@ bool CompactionPicker::SetupOtherInputs( base_index, nullptr); expanded_inputs_size = TotalFileSize(expanded_inputs.files); if (expanded_inputs.size() > inputs->size() && - !AreFilesInCompaction(expanded_inputs.files)) { + !AreFilesInCompaction(expanded_inputs.files) && + (output_level_inputs_size + expanded_inputs_size) < limit) { expand_inputs = true; } } @@ -808,8 +817,8 @@ Compaction* CompactionPicker::CompactRange( output_level_inputs.level = output_level; if (input_level != output_level) { int parent_index = -1; - if (!SetupOtherInputs(cf_name, vstorage, &inputs, &output_level_inputs, - &parent_index, -1)) { + if (!SetupOtherInputs(cf_name, mutable_cf_options, vstorage, &inputs, + &output_level_inputs, &parent_index, -1)) { // manual compaction is now multi-threaded, so it can // happen that SetupOtherInputs fails // we handle it higher in RunManualCompaction diff --git a/db/compaction/compaction_picker.h b/db/compaction/compaction_picker.h index 63542a387a7..0cec71b475f 100644 --- a/db/compaction/compaction_picker.h +++ b/db/compaction/compaction_picker.h @@ -186,6 +186,7 @@ class CompactionPicker { int penultimate_level) const; bool SetupOtherInputs(const std::string& cf_name, + const MutableCFOptions& mutable_cf_options, VersionStorageInfo* vstorage, CompactionInputFiles* inputs, CompactionInputFiles* output_level_inputs, diff --git a/db/compaction/compaction_picker_level.cc b/db/compaction/compaction_picker_level.cc index 3cb45211298..92cf8650161 100644 --- a/db/compaction/compaction_picker_level.cc +++ b/db/compaction/compaction_picker_level.cc @@ -467,8 +467,9 @@ bool LevelCompactionBuilder::SetupOtherInputsIfNeeded() { } if (!is_l0_trivial_move_ && !compaction_picker_->SetupOtherInputs( - cf_name_, vstorage_, &start_level_inputs_, &output_level_inputs_, - &parent_index_, base_index_, round_robin_expanding)) { + cf_name_, mutable_cf_options_, vstorage_, &start_level_inputs_, + &output_level_inputs_, &parent_index_, base_index_, + round_robin_expanding)) { return false; } diff --git a/db/compaction/compaction_picker_test.cc b/db/compaction/compaction_picker_test.cc index beac419d98e..93b2c7bae6f 100644 --- a/db/compaction/compaction_picker_test.cc +++ b/db/compaction/compaction_picker_test.cc @@ -2465,7 +2465,7 @@ TEST_F(CompactionPickerTest, IsBottommostLevelTest) { DeleteVersionStorage(); } -TEST_F(CompactionPickerTest, IgnoreCompactionLimitWhenAddFileFromInputLevel) { +TEST_F(CompactionPickerTest, CompactionLimitWhenAddFileFromInputLevel) { mutable_cf_options_.max_bytes_for_level_base = 1000000u; mutable_cf_options_.max_compaction_bytes = 800000u; ioptions_.level_compaction_dynamic_level_bytes = false; @@ -2473,8 +2473,10 @@ TEST_F(CompactionPickerTest, IgnoreCompactionLimitWhenAddFileFromInputLevel) { // A compaction should be triggered and pick file 2 and 5. // It pulls in other compaction input file from the input level L1 // without pulling in more output level files. - // Files 1, 3, 4 will be included in the compaction. + // Files 1, 3, 4 are eligible. // File 6 is excluded since it overlaps with file 7. + // It can expand input level since in this case, the limit on compaction size + // is 2 * max_compaction_bytes. Add(1, 1U, "100", "150", 300000U); Add(1, 2U, "151", "200", 300001U, 0, 0); Add(1, 3U, "201", "250", 300000U, 0, 0); @@ -2498,6 +2500,38 @@ TEST_F(CompactionPickerTest, IgnoreCompactionLimitWhenAddFileFromInputLevel) { ASSERT_EQ(5U, compaction->input(1, 0)->fd.GetNumber()); } +TEST_F(CompactionPickerTest, HitCompactionLimitWhenAddFileFromInputLevel) { + mutable_cf_options_.max_bytes_for_level_base = 1000000u; + mutable_cf_options_.max_compaction_bytes = 800000u; + ioptions_.level_compaction_dynamic_level_bytes = false; + NewVersionStorage(6, kCompactionStyleLevel); + // A compaction should be triggered and pick file 2 and 5. + // It pulls in other compaction input file from the input level L1 + // without pulling in more output level files. + // Files 1, 3, 4 are eligible. + // File 6 is excluded since it overlaps with file 7. + // It can not expand input level since total compaction size hit the limit + // 2 * max_compaction_bytes. + Add(1, 1U, "100", "150", 400000U); + Add(1, 2U, "151", "200", 400001U, 0, 0); + Add(1, 3U, "201", "250", 400000U, 0, 0); + Add(1, 4U, "251", "300", 400000U, 0, 0); + Add(1, 6U, "325", "400", 400000U, 0, 0); + Add(2, 5U, "100", "350", 1U); + Add(2, 7U, "375", "425", 1U); + UpdateVersionStorageInfo(); + + std::unique_ptr compaction(level_compaction_picker.PickCompaction( + cf_name_, mutable_cf_options_, mutable_db_options_, vstorage_.get(), + &log_buffer_)); + ASSERT_TRUE(compaction.get() != nullptr); + ASSERT_EQ(2U, compaction->num_input_levels()); + ASSERT_EQ(1U, compaction->num_input_files(0)); + ASSERT_EQ(1U, compaction->num_input_files(1)); + ASSERT_EQ(2U, compaction->input(0, 0)->fd.GetNumber()); + ASSERT_EQ(5U, compaction->input(1, 0)->fd.GetNumber()); +} + TEST_F(CompactionPickerTest, IsTrivialMoveOn) { mutable_cf_options_.max_bytes_for_level_base = 10000u; mutable_cf_options_.max_compaction_bytes = 10001u; diff --git a/db/compaction/compaction_picker_universal.cc b/db/compaction/compaction_picker_universal.cc index 597edb7919a..da71e398779 100644 --- a/db/compaction/compaction_picker_universal.cc +++ b/db/compaction/compaction_picker_universal.cc @@ -1139,7 +1139,8 @@ Compaction* UniversalCompactionBuilder::PickIncrementalForReduceSizeAmp( // from bottom_start_idx and bottom_end_idx, but for now, we use // SetupOtherInputs() for simplicity. int parent_index = -1; // Create and use bottom_start_idx? - if (!picker_->SetupOtherInputs(cf_name_, vstorage_, &second_last_level_inputs, + if (!picker_->SetupOtherInputs(cf_name_, mutable_cf_options_, vstorage_, + &second_last_level_inputs, &bottom_level_inputs, &parent_index, /*base_index=*/-1)) { return nullptr; @@ -1310,8 +1311,9 @@ Compaction* UniversalCompactionBuilder::PickDeleteTriggeredCompaction() { int parent_index = -1; output_level_inputs.level = output_level; - if (!picker_->SetupOtherInputs(cf_name_, vstorage_, &start_level_inputs, - &output_level_inputs, &parent_index, -1)) { + if (!picker_->SetupOtherInputs(cf_name_, mutable_cf_options_, vstorage_, + &start_level_inputs, &output_level_inputs, + &parent_index, -1)) { return nullptr; } inputs.push_back(start_level_inputs); diff --git a/db/compaction/tiered_compaction_test.cc b/db/compaction/tiered_compaction_test.cc index 8e72dce9d03..d8b48b991ef 100644 --- a/db/compaction/tiered_compaction_test.cc +++ b/db/compaction/tiered_compaction_test.cc @@ -1584,7 +1584,16 @@ TEST_F(PrecludeLastLevelTest, SmallPrecludeTime) { Close(); } -TEST_F(PrecludeLastLevelTest, FastTrackTimedPutToLastLevel) { +// Test Param: protection_bytes_per_key for WriteBatch +class TimedPutPrecludeLastLevelTest + : public PrecludeLastLevelTest, + public testing::WithParamInterface { + public: + TimedPutPrecludeLastLevelTest() + : PrecludeLastLevelTest("timed_put_preclude_last_level_test") {} +}; + +TEST_P(TimedPutPrecludeLastLevelTest, FastTrackTimedPutToLastLevel) { const int kNumTrigger = 4; const int kNumLevels = 7; const int kNumKeys = 100; @@ -1598,6 +1607,8 @@ TEST_F(PrecludeLastLevelTest, FastTrackTimedPutToLastLevel) { options.num_levels = kNumLevels; options.last_level_temperature = Temperature::kCold; DestroyAndReopen(options); + WriteOptions wo; + wo.protection_bytes_per_key = GetParam(); Random rnd(301); @@ -1606,7 +1617,7 @@ TEST_F(PrecludeLastLevelTest, FastTrackTimedPutToLastLevel) { }); for (int i = 0; i < kNumKeys / 2; i++) { - ASSERT_OK(Put(Key(i), rnd.RandomString(100))); + ASSERT_OK(Put(Key(i), rnd.RandomString(100), wo)); dbfull()->TEST_WaitForPeriodicTaskRun([&] { mock_clock_->MockSleepForSeconds(static_cast(rnd.Uniform(2))); }); @@ -1620,7 +1631,7 @@ TEST_F(PrecludeLastLevelTest, FastTrackTimedPutToLastLevel) { // These data are eligible to be put on the last level once written to db // and compaction will fast track them to the last level. for (int i = kNumKeys / 2; i < kNumKeys; i++) { - ASSERT_OK(TimedPut(0, Key(i), rnd.RandomString(100), 50)); + ASSERT_OK(TimedPut(0, Key(i), rnd.RandomString(100), 50, wo)); } ASSERT_OK(Flush()); @@ -1640,6 +1651,67 @@ TEST_F(PrecludeLastLevelTest, FastTrackTimedPutToLastLevel) { Close(); } +TEST_P(TimedPutPrecludeLastLevelTest, PreserveTimedPutOnPenultimateLevel) { + Options options = CurrentOptions(); + options.compaction_style = kCompactionStyleUniversal; + options.disable_auto_compactions = true; + options.preclude_last_level_data_seconds = 3 * 24 * 60 * 60; + int seconds_between_recording = (3 * 24 * 60 * 60) / kMaxSeqnoTimePairsPerCF; + options.env = mock_env_.get(); + options.num_levels = 7; + options.last_level_temperature = Temperature::kCold; + options.default_write_temperature = Temperature::kHot; + DestroyAndReopen(options); + WriteOptions wo; + wo.protection_bytes_per_key = GetParam(); + + // Creating a snapshot to manually control when preferred sequence number is + // swapped in. An entry's preferred seqno won't get swapped in until it's + // visible to the earliest snapshot. With this, we can test relevant seqno to + // time mapping recorded in SST file also covers preferred seqno, not just + // the seqno in the internal keys. + auto* snap1 = db_->GetSnapshot(); + // Start time: kMockStartTime = 10000000; + ASSERT_OK(TimedPut(0, Key(0), "v0", kMockStartTime - 1 * 24 * 60 * 60, wo)); + ASSERT_OK(TimedPut(0, Key(1), "v1", kMockStartTime - 1 * 24 * 60 * 60, wo)); + ASSERT_OK(TimedPut(0, Key(2), "v2", kMockStartTime - 1 * 24 * 60 * 60, wo)); + ASSERT_OK(Flush()); + + // Should still be in penultimate level. + ASSERT_OK(db_->CompactRange(CompactRangeOptions(), nullptr, nullptr)); + ASSERT_EQ("0,0,0,0,0,1", FilesPerLevel()); + ASSERT_GT(GetSstSizeHelper(Temperature::kHot), 0); + ASSERT_EQ(GetSstSizeHelper(Temperature::kCold), 0); + + // Wait one more day and release snapshot. Data's preferred seqno should be + // swapped in, but data should still stay in penultimate level. SST file's + // seqno to time mapping should continue to cover preferred seqno after + // compaction. + db_->ReleaseSnapshot(snap1); + mock_clock_->MockSleepForSeconds(1 * 24 * 60 * 60); + ASSERT_OK(db_->CompactRange(CompactRangeOptions(), nullptr, nullptr)); + ASSERT_EQ("0,0,0,0,0,1", FilesPerLevel()); + ASSERT_GT(GetSstSizeHelper(Temperature::kHot), 0); + ASSERT_EQ(GetSstSizeHelper(Temperature::kCold), 0); + + // Wait one more day and data are eligible to be placed on last level. + // Instead of waiting exactly one more day, here we waited + // `seconds_between_recording` less seconds to show that it's not precise. + // Data could start to be placed on cold tier one recording interval before + // they exactly become cold based on the setting. For this one column family + // setting preserving 3 days of recording, it's about 43 minutes. + mock_clock_->MockSleepForSeconds(1 * 24 * 60 * 60 - + seconds_between_recording); + ASSERT_OK(db_->CompactRange(CompactRangeOptions(), nullptr, nullptr)); + ASSERT_EQ("0,0,0,0,0,0,1", FilesPerLevel()); + ASSERT_EQ(GetSstSizeHelper(Temperature::kHot), 0); + ASSERT_GT(GetSstSizeHelper(Temperature::kCold), 0); + Close(); +} + +INSTANTIATE_TEST_CASE_P(TimedPutPrecludeLastLevelTest, + TimedPutPrecludeLastLevelTest, ::testing::Values(0, 8)); + TEST_F(PrecludeLastLevelTest, LastLevelOnlyCompactionPartial) { const int kNumTrigger = 4; const int kNumLevels = 7; diff --git a/db/db_basic_test.cc b/db/db_basic_test.cc index dec2cfb9ca1..1e6c392bf00 100644 --- a/db/db_basic_test.cc +++ b/db/db_basic_test.cc @@ -464,12 +464,14 @@ TEST_F(DBBasicTest, TimedPutBasic) { ASSERT_EQ("bv1", Get(1, "bar")); ASSERT_OK(TimedPut(1, "baz", "bzv1", /*write_unix_time=*/0)); ASSERT_EQ("bzv1", Get(1, "baz")); - std::string range_del_begin = "b"; - std::string range_del_end = "baz"; - Slice begin_rdel = range_del_begin, end_rdel = range_del_end; - ASSERT_OK( - db_->DeleteRange(WriteOptions(), handles_[1], begin_rdel, end_rdel)); - ASSERT_EQ("NOT_FOUND", Get(1, "bar")); + if (option_config_ != kRowCache) { + std::string range_del_begin = "b"; + std::string range_del_end = "baz"; + Slice begin_rdel = range_del_begin, end_rdel = range_del_end; + ASSERT_OK( + db_->DeleteRange(WriteOptions(), handles_[1], begin_rdel, end_rdel)); + ASSERT_EQ("NOT_FOUND", Get(1, "bar")); + } ASSERT_EQ("bzv1", Get(1, "baz")); ASSERT_OK(SingleDelete(1, "baz")); @@ -1413,7 +1415,7 @@ TEST_P(DBMultiGetTestWithParam, MultiGetMultiCF) { int get_sv_count = 0; ROCKSDB_NAMESPACE::DBImpl* db = static_cast_with_check(db_); ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack( - "DBImpl::MultiGet::AfterRefSV", [&](void* /*arg*/) { + "DBImpl::MultiCFSnapshot::AfterRefSV", [&](void* /*arg*/) { if (++get_sv_count == 2) { // After MultiGet refs a couple of CFs, flush all CFs so MultiGet // is forced to repeat the process @@ -1513,9 +1515,10 @@ TEST_P(DBMultiGetTestWithParam, MultiGetMultiCFMutex) { int retries = 0; bool last_try = false; ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack( - "DBImpl::MultiGet::LastTry", [&](void* /*arg*/) { last_try = true; }); + "DBImpl::MultiCFSnapshot::LastTry", + [&](void* /*arg*/) { last_try = true; }); ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack( - "DBImpl::MultiGet::AfterRefSV", [&](void* /*arg*/) { + "DBImpl::MultiCFSnapshot::AfterRefSV", [&](void* /*arg*/) { if (last_try) { return; } @@ -1531,10 +1534,10 @@ TEST_P(DBMultiGetTestWithParam, MultiGetMultiCFMutex) { } }); ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->LoadDependency({ - {"DBImpl::MultiGet::AfterLastTryRefSV", + {"DBImpl::MultiCFSnapshot::AfterLastTryRefSV", "DBMultiGetTestWithParam::MultiGetMultiCFMutex:BeforeCreateSV"}, {"DBMultiGetTestWithParam::MultiGetMultiCFMutex:AfterCreateSV", - "DBImpl::MultiGet::BeforeLastTryUnRefSV"}, + "DBImpl::MultiCFSnapshot::BeforeLastTryUnRefSV"}, }); ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing(); @@ -1600,7 +1603,7 @@ TEST_P(DBMultiGetTestWithParam, MultiGetMultiCFSnapshot) { int get_sv_count = 0; ROCKSDB_NAMESPACE::DBImpl* db = static_cast_with_check(db_); ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack( - "DBImpl::MultiGet::AfterRefSV", [&](void* /*arg*/) { + "DBImpl::MultiCFSnapshot::AfterRefSV", [&](void* /*arg*/) { if (++get_sv_count == 2) { for (int i = 0; i < 8; ++i) { ASSERT_OK(Flush(i)); diff --git a/db/db_bloom_filter_test.cc b/db/db_bloom_filter_test.cc index fa2d45d2505..a13820535d6 100644 --- a/db/db_bloom_filter_test.cc +++ b/db/db_bloom_filter_test.cc @@ -529,6 +529,7 @@ TEST_P(DBBloomFilterTestWithParam, BloomFilter) { BlockBasedTableOptions table_options; table_options.no_block_cache = true; table_options.filter_policy = Create(10, bfp_impl_); + table_options.optimize_filters_for_memory = false; table_options.partition_filters = partition_filters_; if (partition_filters_) { table_options.index_type = @@ -1695,6 +1696,7 @@ TEST_F(DBBloomFilterTest, ContextCustomFilterPolicy) { BlockBasedTableOptions table_options; table_options.filter_policy = policy; table_options.format_version = 5; + table_options.optimize_filters_for_memory = false; options.table_factory.reset(NewBlockBasedTableFactory(table_options)); ASSERT_OK(TryReopen(options)); diff --git a/db/db_compaction_test.cc b/db/db_compaction_test.cc index 612a1f21d60..e03093077f4 100644 --- a/db/db_compaction_test.cc +++ b/db/db_compaction_test.cc @@ -3997,7 +3997,7 @@ TEST_P(DBCompactionTestWithParam, FullCompactionInBottomPriThreadPool) { Env::Default()->SetBackgroundThreads(0, Env::Priority::BOTTOM); } -TEST_F(DBCompactionTest, CancelCompactionWaitingOnConflict) { +TEST_F(DBCompactionTest, CancelCompactionWaitingOnRunningConflict) { // This test verifies cancellation of a compaction waiting to be scheduled due // to conflict with a running compaction. // @@ -4036,7 +4036,7 @@ TEST_F(DBCompactionTest, CancelCompactionWaitingOnConflict) { // Make sure the manual compaction has seen the conflict before being canceled ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->LoadDependency( {{"ColumnFamilyData::CompactRange:Return", - "DBCompactionTest::CancelCompactionWaitingOnConflict:" + "DBCompactionTest::CancelCompactionWaitingOnRunningConflict:" "PreDisableManualCompaction"}}); auto manual_compaction_thread = port::Thread([this]() { ASSERT_TRUE(db_->CompactRange(CompactRangeOptions(), nullptr, nullptr) @@ -4047,12 +4047,73 @@ TEST_F(DBCompactionTest, CancelCompactionWaitingOnConflict) { // despite finding a conflict with an automatic compaction that is still // running TEST_SYNC_POINT( - "DBCompactionTest::CancelCompactionWaitingOnConflict:" + "DBCompactionTest::CancelCompactionWaitingOnRunningConflict:" "PreDisableManualCompaction"); db_->DisableManualCompaction(); manual_compaction_thread.join(); } +TEST_F(DBCompactionTest, CancelCompactionWaitingOnScheduledConflict) { + // This test verifies cancellation of a compaction waiting to be scheduled due + // to conflict with a scheduled (but not running) compaction. + // + // A `CompactRange()` in universal compacts all files, waiting for files to + // become available if they are locked for another compaction. This test + // blocks the compaction thread pool and then calls `CompactRange()` twice. + // The first call to `CompactRange()` schedules a compaction that is queued + // in the thread pool. The second call to `CompactRange()` blocks on the first + // call due to the conflict in file picking. The test verifies that + // `DisableManualCompaction()` can cancel both while the thread pool remains + // blocked. + const int kNumSortedRuns = 4; + + Options options = CurrentOptions(); + options.compaction_style = kCompactionStyleUniversal; + options.disable_auto_compactions = true; + options.memtable_factory.reset( + test::NewSpecialSkipListFactory(KNumKeysByGenerateNewFile - 1)); + Reopen(options); + + test::SleepingBackgroundTask sleeping_task_low; + env_->Schedule(&test::SleepingBackgroundTask::DoSleepTask, &sleeping_task_low, + Env::Priority::LOW); + + // Fill overlapping files in L0 + Random rnd(301); + for (int i = 0; i < kNumSortedRuns; ++i) { + int key_idx = 0; + GenerateNewFile(&rnd, &key_idx, false /* nowait */); + } + + std::atomic num_compact_range_calls{0}; + ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack( + "ColumnFamilyData::CompactRange:Return", + [&](void* /* arg */) { num_compact_range_calls++; }); + + ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing(); + + const int kNumManualCompactions = 2; + port::Thread manual_compaction_threads[kNumManualCompactions]; + for (int i = 0; i < kNumManualCompactions; i++) { + manual_compaction_threads[i] = port::Thread([this]() { + ASSERT_TRUE(db_->CompactRange(CompactRangeOptions(), nullptr, nullptr) + .IsIncomplete()); + }); + } + while (num_compact_range_calls < kNumManualCompactions) { + } + + // Cancel it. Threads should be joinable, i.e., both the scheduled and blocked + // manual compactions were canceled despite no compaction could have ever run. + db_->DisableManualCompaction(); + for (int i = 0; i < kNumManualCompactions; i++) { + manual_compaction_threads[i].join(); + } + + sleeping_task_low.WakeUp(); + sleeping_task_low.WaitUntilDone(); +} + TEST_F(DBCompactionTest, OptimizedDeletionObsoleting) { // Deletions can be dropped when compacted to non-last level if they fall // outside the lower-level files' key-ranges. @@ -6082,6 +6143,14 @@ TEST_F(DBCompactionTest, CompactionLimiter) { } }); + std::vector pending_compaction_cfs; + ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack( + "SchedulePendingCompaction::cfd", [&](void* arg) { + const std::string& cf_name = + static_cast(arg)->GetName(); + pending_compaction_cfs.emplace_back(cf_name); + }); + ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing(); // Block all compact threads in thread pool. @@ -6136,7 +6205,29 @@ TEST_F(DBCompactionTest, CompactionLimiter) { } // All CFs are pending compaction - ASSERT_EQ(cf_count, env_->GetThreadPoolQueueLen(Env::LOW)); + unsigned int tp_len = env_->GetThreadPoolQueueLen(Env::LOW); + if (cf_count != tp_len) { + // The test is flaky and fails the assertion below. + // Print some debug information. + uint64_t num_running_flushes = 0; + if (db_->GetIntProperty(DB::Properties::kNumRunningFlushes, + &num_running_flushes)) { + fprintf(stdout, "Running flushes: %" PRIu64 "\n", num_running_flushes); + } + fprintf(stdout, + "%zu CF in compaction queue: ", pending_compaction_cfs.size()); + for (const auto& cf_name : pending_compaction_cfs) { + fprintf(stdout, "%s, ", cf_name.c_str()); + } + fprintf(stdout, "\n"); + + // print lsm + for (unsigned int cf = 0; cf < cf_count; cf++) { + fprintf(stdout, "%s: %s\n", cf_names[cf], FilesPerLevel(cf).c_str()); + } + } + + ASSERT_EQ(cf_count, tp_len); // Unblock all compaction threads for (size_t i = 0; i < kTotalCompactTasks; i++) { @@ -9890,6 +9981,62 @@ TEST_F(DBCompactionTest, TurnOnLevelCompactionDynamicLevelBytesUCToLC) { ASSERT_EQ(expected_lsm, FilesPerLevel(1)); } +TEST_F(DBCompactionTest, DisallowRefitFilesFromNonL0ToL02) { + Options options = CurrentOptions(); + options.compaction_style = CompactionStyle::kCompactionStyleLevel; + options.num_levels = 3; + DestroyAndReopen(options); + + // To set up LSM shape: + // L0 + // L1 + // L2:[a@1, k@3], [k@2, z@4] (sorted by ascending smallest key) + // Both of these 2 files have epoch number = 1 + const Snapshot* s1 = db_->GetSnapshot(); + ASSERT_OK(Put("a", "@1")); + ASSERT_OK(Put("k", "@2")); + const Snapshot* s2 = db_->GetSnapshot(); + ASSERT_OK(Put("k", "@3")); + ASSERT_OK(Put("z", "v3")); + ASSERT_OK(Flush()); + // Cut file between k@3 and k@2 + SyncPoint::GetInstance()->SetCallBack( + "CompactionOutputs::ShouldStopBefore::manual_decision", + [options](void* p) { + auto* pair = (std::pair*)p; + if ((options.comparator->Compare(ExtractUserKey(pair->second), "k") == + 0) && + (GetInternalKeySeqno(pair->second) == 2)) { + *(pair->first) = true; + } + }); + SyncPoint::GetInstance()->EnableProcessing(); + CompactRangeOptions cro; + cro.bottommost_level_compaction = BottommostLevelCompaction::kForceOptimized; + cro.change_level = true; + cro.target_level = 2; + Status s = dbfull()->CompactRange(cro, nullptr, nullptr); + ASSERT_OK(s); + ASSERT_EQ("0,0,2", FilesPerLevel()); + std::vector files; + dbfull()->GetLiveFilesMetaData(&files); + ASSERT_EQ(files.size(), 2); + ASSERT_EQ(files[0].smallestkey, "a"); + ASSERT_EQ(files[0].largestkey, "k"); + ASSERT_EQ(files[1].smallestkey, "k"); + ASSERT_EQ(files[1].largestkey, "z"); + + // Disallow moving 2 non-L0 files to L0 + CompactRangeOptions cro2; + cro2.change_level = true; + cro2.target_level = 0; + s = dbfull()->CompactRange(cro2, nullptr, nullptr); + ASSERT_TRUE(s.IsAborted()); + + db_->ReleaseSnapshot(s1); + db_->ReleaseSnapshot(s2); +} + TEST_F(DBCompactionTest, DrainUnnecessaryLevelsAfterMultiplierChanged) { // When the level size multiplier increases such that fewer levels become // necessary, unnecessary levels should to be drained. diff --git a/db/db_filesnapshot.cc b/db/db_filesnapshot.cc index edd1b4ab993..610ec67cfbe 100644 --- a/db/db_filesnapshot.cc +++ b/db/db_filesnapshot.cc @@ -151,6 +151,29 @@ Status DBImpl::GetSortedWalFiles(VectorLogPtr& files) { } } + if (s.ok()) { + size_t wal_size = files.size(); + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "Number of log files %" ROCKSDB_PRIszt " (%" ROCKSDB_PRIszt + " required by manifest)", + wal_size, required_by_manifest.size()); +#ifndef NDEBUG + std::ostringstream wal_names; + for (const auto& wal : files) { + wal_names << wal->PathName() << " "; + } + + std::ostringstream wal_required_by_manifest_names; + for (const auto& wal : required_by_manifest) { + wal_required_by_manifest_names << wal << ".log "; + } + + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "Log files : %s .Log files required by manifest: %s.", + wal_names.str().c_str(), + wal_required_by_manifest_names.str().c_str()); +#endif // NDEBUG + } return s; } @@ -371,10 +394,6 @@ Status DBImpl::GetLiveFilesStorageInfo( } size_t wal_size = live_wal_files.size(); - - ROCKS_LOG_INFO(immutable_db_options_.info_log, - "Number of log files %" ROCKSDB_PRIszt, live_wal_files.size()); - // Link WAL files. Copy exact size of last one because it is the only one // that has changes after the last flush. auto wal_dir = immutable_db_options_.GetWalDir(); diff --git a/db/db_follower_test.cc b/db/db_follower_test.cc new file mode 100644 index 00000000000..86bf8cc7c53 --- /dev/null +++ b/db/db_follower_test.cc @@ -0,0 +1,63 @@ +// Copyright (c) 2024-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/db_test_util.h" +#include "port/stack_trace.h" +#include "test_util/sync_point.h" + +namespace ROCKSDB_NAMESPACE { + +#ifdef OS_LINUX + +class DBFollowerTest : public DBTestBase { + public: + // Create directories for leader and follower + // Create the leader DB object + DBFollowerTest() : DBTestBase("/db_follower_test", /*env_do_fsync*/ false) { + follower_name_ = dbname_ + "/follower"; + Close(); + Destroy(CurrentOptions()); + EXPECT_EQ(env_->CreateDirIfMissing(dbname_), Status::OK()); + dbname_ = dbname_ + "/leader"; + Reopen(CurrentOptions()); + } + + ~DBFollowerTest() { + follower_.reset(); + EXPECT_EQ(DestroyDB(follower_name_, CurrentOptions()), Status::OK()); + } + + protected: + Status OpenAsFollower() { + return DB::OpenAsFollower(CurrentOptions(), follower_name_, dbname_, + &follower_); + } + DB* follower() { return follower_.get(); } + + private: + std::string follower_name_; + std::unique_ptr follower_; +}; + +TEST_F(DBFollowerTest, Basic) { + ASSERT_OK(Put("k1", "v1")); + ASSERT_OK(Flush()); + ASSERT_OK(Put("k2", "v2")); + ASSERT_OK(Flush()); + + ASSERT_OK(OpenAsFollower()); + std::string val; + ASSERT_OK(follower()->Get(ReadOptions(), "k1", &val)); + ASSERT_EQ(val, "v1"); +} + +#endif +} // namespace ROCKSDB_NAMESPACE + +int main(int argc, char** argv) { + ROCKSDB_NAMESPACE::port::InstallStackTraceHandler(); + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} diff --git a/db/db_impl/db_impl.cc b/db/db_impl/db_impl.cc index f7e6f9692b2..12b0afc2bb7 100644 --- a/db/db_impl/db_impl.cc +++ b/db/db_impl/db_impl.cc @@ -26,7 +26,9 @@ #include #include "db/arena_wrapped_db_iter.h" +#include "db/attribute_group_iterator_impl.h" #include "db/builder.h" +#include "db/coalescing_iterator.h" #include "db/compaction/compaction_job.h" #include "db/convenience_impl.h" #include "db/db_info_dumper.h" @@ -45,7 +47,6 @@ #include "db/memtable.h" #include "db/memtable_list.h" #include "db/merge_context.h" -#include "db/multi_cf_iterator.h" #include "db/periodic_task_scheduler.h" #include "db/range_tombstone_fragmenter.h" #include "db/table_cache.h" @@ -1853,6 +1854,7 @@ void DBImpl::SchedulePurge() { } void DBImpl::BackgroundCallPurge() { + TEST_SYNC_POINT("DBImpl::BackgroundCallPurge:beforeMutexLock"); mutex_.Lock(); while (!logs_to_free_queue_.empty()) { @@ -2515,12 +2517,12 @@ Status DBImpl::GetImpl(const ReadOptions& read_options, const Slice& key, return s; } -template -Status DBImpl::MultiCFSnapshot( - const ReadOptions& read_options, ReadCallback* callback, - std::function& - iter_deref_func, - T* cf_list, SequenceNumber* snapshot, bool* sv_from_thread_local) { +template +Status DBImpl::MultiCFSnapshot(const ReadOptions& read_options, + ReadCallback* callback, + IterDerefFuncType iter_deref_func, T* cf_list, + bool extra_sv_ref, SequenceNumber* snapshot, + bool* sv_from_thread_local) { PERF_TIMER_GUARD(get_snapshot_time); assert(sv_from_thread_local); @@ -2537,7 +2539,7 @@ Status DBImpl::MultiCFSnapshot( SuperVersion* super_version = node->super_version; ColumnFamilyData* cfd = node->cfd; if (super_version != nullptr) { - if (*sv_from_thread_local) { + if (*sv_from_thread_local && !extra_sv_ref) { ReturnAndCleanupSuperVersion(cfd, super_version); } else { CleanupSuperVersion(super_version); @@ -2553,7 +2555,11 @@ Status DBImpl::MultiCFSnapshot( // super version auto cf_iter = cf_list->begin(); auto node = iter_deref_func(cf_iter); - node->super_version = GetAndRefSuperVersion(node->cfd); + if (extra_sv_ref) { + node->super_version = node->cfd->GetReferencedSuperVersion(this); + } else { + node->super_version = GetAndRefSuperVersion(node->cfd); + } if (check_read_ts) { s = FailIfReadCollapsedHistory(node->cfd, node->super_version, *(read_options.timestamp)); @@ -2600,7 +2606,7 @@ Status DBImpl::MultiCFSnapshot( } if (read_options.snapshot == nullptr) { if (last_try) { - TEST_SYNC_POINT("DBImpl::MultiGet::LastTry"); + TEST_SYNC_POINT("DBImpl::MultiCFSnapshot::LastTry"); // We're close to max number of retries. For the last retry, // acquire the lock so we're sure to succeed mutex_.Lock(); @@ -2615,11 +2621,15 @@ Status DBImpl::MultiCFSnapshot( ++cf_iter) { auto node = iter_deref_func(cf_iter); if (!last_try) { - node->super_version = GetAndRefSuperVersion(node->cfd); + if (extra_sv_ref) { + node->super_version = node->cfd->GetReferencedSuperVersion(this); + } else { + node->super_version = GetAndRefSuperVersion(node->cfd); + } } else { node->super_version = node->cfd->GetSuperVersion()->Ref(); } - TEST_SYNC_POINT("DBImpl::MultiGet::AfterRefSV"); + TEST_SYNC_POINT("DBImpl::MultiCFSnapshot::AfterRefSV"); if (check_read_ts) { s = FailIfReadCollapsedHistory(node->cfd, node->super_version, *(read_options.timestamp)); @@ -2633,6 +2643,7 @@ Status DBImpl::MultiCFSnapshot( break; } } + TEST_SYNC_POINT("DBImpl::MultiCFSnapshot::BeforeCheckingSnapshot"); if (read_options.snapshot != nullptr || last_try) { // If user passed a snapshot, then we don't care if a memtable is // sealed or compaction happens because the snapshot would ensure @@ -2656,7 +2667,7 @@ Status DBImpl::MultiCFSnapshot( if (!retry) { if (last_try) { mutex_.Unlock(); - TEST_SYNC_POINT("DBImpl::MultiGet::AfterLastTryRefSV"); + TEST_SYNC_POINT("DBImpl::MultiCFSnapshot::AfterLastTryRefSV"); } break; } @@ -2768,37 +2779,37 @@ void DBImpl::MultiGetCommon(const ReadOptions& read_options, } PrepareMultiGetKeys(num_keys, sorted_input, &sorted_keys); - autovector - multiget_cf_data; + autovector + key_range_per_cf; + autovector + cf_sv_pairs; size_t cf_start = 0; ColumnFamilyHandle* cf = sorted_keys[0]->column_family; for (size_t i = 0; i < num_keys; ++i) { KeyContext* key_ctx = sorted_keys[i]; if (key_ctx->column_family != cf) { - multiget_cf_data.emplace_back(cf, cf_start, i - cf_start, nullptr); + key_range_per_cf.emplace_back(cf_start, i - cf_start); + cf_sv_pairs.emplace_back(cf, nullptr); cf_start = i; cf = key_ctx->column_family; } } - multiget_cf_data.emplace_back(cf, cf_start, num_keys - cf_start, nullptr); - - std::function::iterator&)> - iter_deref_lambda = - [](autovector::iterator& cf_iter) { - return &(*cf_iter); - }; + key_range_per_cf.emplace_back(cf_start, num_keys - cf_start); + cf_sv_pairs.emplace_back(cf, nullptr); - SequenceNumber consistent_seqnum; - bool sv_from_thread_local; - Status s = MultiCFSnapshot< - autovector>( - read_options, nullptr, iter_deref_lambda, &multiget_cf_data, - &consistent_seqnum, &sv_from_thread_local); + SequenceNumber consistent_seqnum = kMaxSequenceNumber; + bool sv_from_thread_local = false; + Status s = MultiCFSnapshot>( + read_options, nullptr, + [](autovector::iterator& cf_iter) { + return &(*cf_iter); + }, + &cf_sv_pairs, + /* extra_sv_ref */ false, &consistent_seqnum, &sv_from_thread_local); if (!s.ok()) { for (size_t i = 0; i < num_keys; ++i) { @@ -2816,31 +2827,40 @@ void DBImpl::MultiGetCommon(const ReadOptions& read_options, read_callback = ×tamp_read_callback; } - auto cf_iter = multiget_cf_data.begin(); - for (; cf_iter != multiget_cf_data.end(); ++cf_iter) { - s = MultiGetImpl(read_options, cf_iter->start, cf_iter->num_keys, - &sorted_keys, cf_iter->super_version, consistent_seqnum, + assert(key_range_per_cf.size() == cf_sv_pairs.size()); + auto key_range_per_cf_iter = key_range_per_cf.begin(); + auto cf_sv_pair_iter = cf_sv_pairs.begin(); + while (key_range_per_cf_iter != key_range_per_cf.end() && + cf_sv_pair_iter != cf_sv_pairs.end()) { + s = MultiGetImpl(read_options, key_range_per_cf_iter->start, + key_range_per_cf_iter->num_keys, &sorted_keys, + cf_sv_pair_iter->super_version, consistent_seqnum, read_callback); if (!s.ok()) { break; } + ++key_range_per_cf_iter; + ++cf_sv_pair_iter; } if (!s.ok()) { assert(s.IsTimedOut() || s.IsAborted()); - for (++cf_iter; cf_iter != multiget_cf_data.end(); ++cf_iter) { - for (size_t i = cf_iter->start; i < cf_iter->start + cf_iter->num_keys; + for (++key_range_per_cf_iter; + key_range_per_cf_iter != key_range_per_cf.end(); + ++key_range_per_cf_iter) { + for (size_t i = key_range_per_cf_iter->start; + i < key_range_per_cf_iter->start + key_range_per_cf_iter->num_keys; ++i) { *sorted_keys[i]->s = s; } } } - for (const auto& iter : multiget_cf_data) { + for (const auto& cf_sv_pair : cf_sv_pairs) { if (sv_from_thread_local) { - ReturnAndCleanupSuperVersion(iter.cfd, iter.super_version); + ReturnAndCleanupSuperVersion(cf_sv_pair.cfd, cf_sv_pair.super_version); } else { - TEST_SYNC_POINT("DBImpl::MultiGet::BeforeLastTryUnRefSV"); - CleanupSuperVersion(iter.super_version); + TEST_SYNC_POINT("DBImpl::MultiCFSnapshot::BeforeLastTryUnRefSV"); + CleanupSuperVersion(cf_sv_pair.super_version); } } } @@ -2972,21 +2992,18 @@ void DBImpl::MultiGetWithCallbackImpl( const ReadOptions& read_options, ColumnFamilyHandle* column_family, ReadCallback* callback, autovector* sorted_keys) { - std::array multiget_cf_data; - multiget_cf_data[0] = MultiGetColumnFamilyData(column_family, nullptr); - std::function::iterator&)> - iter_deref_lambda = - [](std::array::iterator& cf_iter) { - return &(*cf_iter); - }; - + std::array cf_sv_pairs; + cf_sv_pairs[0] = ColumnFamilySuperVersionPair(column_family, nullptr); size_t num_keys = sorted_keys->size(); - SequenceNumber consistent_seqnum; - bool sv_from_thread_local; - Status s = MultiCFSnapshot>( - read_options, callback, iter_deref_lambda, &multiget_cf_data, - &consistent_seqnum, &sv_from_thread_local); + SequenceNumber consistent_seqnum = kMaxSequenceNumber; + bool sv_from_thread_local = false; + Status s = MultiCFSnapshot>( + read_options, callback, + [](std::array::iterator& cf_iter) { + return &(*cf_iter); + }, + &cf_sv_pairs, + /* extra_sv_ref */ false, &consistent_seqnum, &sv_from_thread_local); if (!s.ok()) { return; } @@ -3025,11 +3042,11 @@ void DBImpl::MultiGetWithCallbackImpl( } s = MultiGetImpl(read_options, 0, num_keys, sorted_keys, - multiget_cf_data[0].super_version, consistent_seqnum, + cf_sv_pairs[0].super_version, consistent_seqnum, read_callback); assert(s.ok() || s.IsTimedOut() || s.IsAborted()); - ReturnAndCleanupSuperVersion(multiget_cf_data[0].cfd, - multiget_cf_data[0].super_version); + ReturnAndCleanupSuperVersion(cf_sv_pairs[0].cfd, + cf_sv_pairs[0].super_version); } // The actual implementation of batched MultiGet. Parameters - @@ -3744,29 +3761,49 @@ ArenaWrappedDBIter* DBImpl::NewIteratorImpl( return db_iter; } -std::unique_ptr DBImpl::NewMultiCfIterator( +std::unique_ptr DBImpl::NewCoalescingIterator( const ReadOptions& _read_options, const std::vector& column_families) { + return NewMultiCfIterator( + _read_options, column_families, [](const Status& s) { + return std::unique_ptr(NewErrorIterator(s)); + }); +} + +std::unique_ptr DBImpl::NewAttributeGroupIterator( + const ReadOptions& _read_options, + const std::vector& column_families) { + return NewMultiCfIterator( + _read_options, column_families, + [](const Status& s) { return NewAttributeGroupErrorIterator(s); }); +} + +template +std::unique_ptr DBImpl::NewMultiCfIterator( + const ReadOptions& _read_options, + const std::vector& column_families, + ErrorIteratorFuncType error_iterator_func) { if (column_families.size() == 0) { - return std::unique_ptr(NewErrorIterator( - Status::InvalidArgument("No Column Family was provided"))); + return error_iterator_func( + Status::InvalidArgument("No Column Family was provided")); } const Comparator* first_comparator = column_families[0]->GetComparator(); for (size_t i = 1; i < column_families.size(); ++i) { const Comparator* cf_comparator = column_families[i]->GetComparator(); if (first_comparator != cf_comparator && first_comparator->GetId().compare(cf_comparator->GetId()) != 0) { - return std::unique_ptr(NewErrorIterator(Status::InvalidArgument( - "Different comparators are being used across CFs"))); + return error_iterator_func(Status::InvalidArgument( + "Different comparators are being used across CFs")); } } std::vector child_iterators; Status s = NewIterators(_read_options, column_families, &child_iterators); - if (s.ok()) { - return std::make_unique(first_comparator, column_families, - std::move(child_iterators)); + if (!s.ok()) { + return error_iterator_func(s); } - return std::unique_ptr(NewErrorIterator(s)); + return std::make_unique(column_families[0]->GetComparator(), + column_families, + std::move(child_iterators)); } Status DBImpl::NewIterators( @@ -3791,69 +3828,62 @@ Status DBImpl::NewIterators( "ReadTier::kPersistedData is not yet supported in iterators."); } - if (read_options.timestamp) { - for (auto* cf : column_families) { - assert(cf); - const Status s = FailIfTsMismatchCf(cf, *(read_options.timestamp)); - if (!s.ok()) { - return s; - } + autovector + cf_sv_pairs; + + Status s; + for (auto* cf : column_families) { + assert(cf); + if (read_options.timestamp) { + s = FailIfTsMismatchCf(cf, *(read_options.timestamp)); + } else { + s = FailIfCfHasTs(cf); } - } else { - for (auto* cf : column_families) { - assert(cf); - const Status s = FailIfCfHasTs(cf); - if (!s.ok()) { - return s; - } + if (!s.ok()) { + return s; } + cf_sv_pairs.emplace_back(cf, nullptr); } - iterators->clear(); iterators->reserve(column_families.size()); - autovector> cfh_to_sv; - const bool check_read_ts = - read_options.timestamp && read_options.timestamp->size() > 0; - for (auto cf : column_families) { - auto cfh = static_cast_with_check(cf); - auto cfd = cfh->cfd(); - SuperVersion* sv = cfd->GetReferencedSuperVersion(this); - cfh_to_sv.emplace_back(cfh, sv); - if (check_read_ts) { - const Status s = - FailIfReadCollapsedHistory(cfd, sv, *(read_options.timestamp)); - if (!s.ok()) { - for (auto prev_entry : cfh_to_sv) { - CleanupSuperVersion(std::get<1>(prev_entry)); - } - return s; - } - } + + SequenceNumber consistent_seqnum = kMaxSequenceNumber; + bool sv_from_thread_local = false; + s = MultiCFSnapshot>( + read_options, nullptr /* read_callback*/, + [](autovector::iterator& cf_iter) { + return &(*cf_iter); + }, + &cf_sv_pairs, + /* extra_sv_ref */ true, &consistent_seqnum, &sv_from_thread_local); + if (!s.ok()) { + return s; } - assert(cfh_to_sv.size() == column_families.size()); + + assert(cf_sv_pairs.size() == column_families.size()); if (read_options.tailing) { - for (auto [cfh, sv] : cfh_to_sv) { - auto iter = new ForwardIterator(this, read_options, cfh->cfd(), sv, + for (const auto& cf_sv_pair : cf_sv_pairs) { + auto iter = new ForwardIterator(this, read_options, cf_sv_pair.cfd, + cf_sv_pair.super_version, /* allow_unprepared_value */ true); - iterators->push_back(NewDBIterator( - env_, read_options, *cfh->cfd()->ioptions(), sv->mutable_cf_options, - cfh->cfd()->user_comparator(), iter, sv->current, kMaxSequenceNumber, - sv->mutable_cf_options.max_sequential_skip_in_iterations, - nullptr /*read_callback*/, cfh)); + iterators->push_back( + NewDBIterator(env_, read_options, *cf_sv_pair.cfd->ioptions(), + cf_sv_pair.super_version->mutable_cf_options, + cf_sv_pair.cfd->user_comparator(), iter, + cf_sv_pair.super_version->current, kMaxSequenceNumber, + cf_sv_pair.super_version->mutable_cf_options + .max_sequential_skip_in_iterations, + nullptr /*read_callback*/, cf_sv_pair.cfh)); } } else { - // Note: no need to consider the special case of - // last_seq_same_as_publish_seq_==false since NewIterators is overridden in - // WritePreparedTxnDB - auto snapshot = read_options.snapshot != nullptr - ? read_options.snapshot->GetSequenceNumber() - : versions_->LastSequence(); - for (auto [cfh, sv] : cfh_to_sv) { - iterators->push_back(NewIteratorImpl(read_options, cfh, sv, snapshot, - nullptr /*read_callback*/)); + for (const auto& cf_sv_pair : cf_sv_pairs) { + iterators->push_back(NewIteratorImpl( + read_options, cf_sv_pair.cfh, cf_sv_pair.super_version, + consistent_seqnum, nullptr /*read_callback*/)); } } - return Status::OK(); } @@ -3968,8 +3998,8 @@ DBImpl::CreateTimestampedSnapshotImpl(SequenceNumber snapshot_seq, uint64_t ts, std::shared_ptr latest = timestamped_snapshots_.GetSnapshot(std::numeric_limits::max()); - // If there is already a latest timestamped snapshot, then we need to do some - // checks. + // If there is already a latest timestamped snapshot, then we need to do + // some checks. if (latest) { uint64_t latest_snap_ts = latest->GetTimestamp(); SequenceNumber latest_snap_seq = latest->GetSequenceNumber(); @@ -3978,8 +4008,8 @@ DBImpl::CreateTimestampedSnapshotImpl(SequenceNumber snapshot_seq, uint64_t ts, Status status; std::shared_ptr ret; if (latest_snap_ts > ts) { - // A snapshot created later cannot have smaller timestamp than a previous - // timestamped snapshot. + // A snapshot created later cannot have smaller timestamp than a + // previous timestamped snapshot. needs_create_snap = false; std::ostringstream oss; oss << "snapshot exists with larger timestamp " << latest_snap_ts << " > " @@ -4093,7 +4123,8 @@ void DBImpl::ReleaseSnapshot(const Snapshot* s) { // Calculate a new threshold, skipping those CFs where compactions are // scheduled. We do not do the same pass as the previous loop because - // mutex might be unlocked during the loop, making the result inaccurate. + // mutex might be unlocked during the loop, making the result + // inaccurate. SequenceNumber new_bottommost_files_mark_threshold = kMaxSequenceNumber; for (auto* cfd : *versions_->GetColumnFamilySet()) { if (CfdListContains(cf_scheduled, cfd) || @@ -4521,7 +4552,8 @@ Status DBImpl::GetApproximateSizes(const SizeApproximationOptions& options, sizes[i] = 0; if (options.include_files) { sizes[i] += versions_->ApproximateSize( - options, read_options, v, k1.Encode(), k2.Encode(), /*start_level=*/0, + options, read_options, v, k1.Encode(), k2.Encode(), + /*start_level=*/0, /*end_level=*/-1, TableReaderCaller::kUserApproximateSize); } if (options.include_memtables) { @@ -4806,9 +4838,9 @@ void DBImpl::GetColumnFamilyMetaData(ColumnFamilyHandle* column_family, static_cast_with_check(column_family)->cfd(); auto* sv = GetAndRefSuperVersion(cfd); { - // Without mutex, Version::GetColumnFamilyMetaData will have data race with - // Compaction::MarkFilesBeingCompacted. One solution is to use mutex, but - // this may cause regression. An alternative is to make + // Without mutex, Version::GetColumnFamilyMetaData will have data race + // with Compaction::MarkFilesBeingCompacted. One solution is to use mutex, + // but this may cause regression. An alternative is to make // FileMetaData::being_compacted atomic, but it will make FileMetaData // non-copy-able. Another option is to separate these variables from // original FileMetaData struct, and this requires re-organization of data diff --git a/db/db_impl/db_impl.h b/db/db_impl/db_impl.h index 244d979c1a5..f4a95b52b85 100644 --- a/db/db_impl/db_impl.h +++ b/db/db_impl/db_impl.h @@ -50,6 +50,7 @@ #include "monitoring/instrumented_mutex.h" #include "options/db_options.h" #include "port/port.h" +#include "rocksdb/attribute_groups.h" #include "rocksdb/db.h" #include "rocksdb/env.h" #include "rocksdb/memtablerep.h" @@ -351,9 +352,13 @@ class DBImpl : public DB { const Snapshot* GetSnapshot() override; void ReleaseSnapshot(const Snapshot* snapshot) override; - // UNDER CONSTRUCTION - DO NOT USE - // Return a cross-column-family iterator from a consistent database state. - std::unique_ptr NewMultiCfIterator( + // EXPERIMENTAL + std::unique_ptr NewCoalescingIterator( + const ReadOptions& options, + const std::vector& column_families) override; + + // EXPERIMENTAL + std::unique_ptr NewAttributeGroupIterator( const ReadOptions& options, const std::vector& column_families) override; @@ -816,6 +821,8 @@ class DBImpl : public DB { uint64_t MinLogNumberToKeep(); + uint64_t MinLogNumberToRecycle(); + // Returns the lower bound file number for SSTs that won't be deleted, even if // they're obsolete. This lower bound is used internally to prevent newly // created flush/compaction output files from being deleted before they're @@ -1052,7 +1059,8 @@ class DBImpl : public DB { static Status Open(const DBOptions& db_options, const std::string& name, const std::vector& column_families, std::vector* handles, DB** dbptr, - const bool seq_per_batch, const bool batch_per_txn); + const bool seq_per_batch, const bool batch_per_txn, + const bool is_retry, bool* can_retry); static IOStatus CreateAndNewDirectory( FileSystem* fs, const std::string& dirname, @@ -1413,9 +1421,8 @@ class DBImpl : public DB { autovector cfds_; autovector mutable_cf_opts_; autovector> edit_lists_; - // Stale SST files to delete found upon recovery. This stores a mapping from - // such a file's absolute path to its parent directory. - std::unordered_map files_to_delete_; + // All existing data files (SST files and Blob files) found during DB::Open. + std::vector existing_data_files_; bool is_new_db_ = false; }; @@ -1529,7 +1536,7 @@ class DBImpl : public DB { Status WriteRecoverableState(); // Actual implementation of Close() - Status CloseImpl(); + virtual Status CloseImpl(); // Recover the descriptor from persistent storage. May do a significant // amount of work to recover recently logged updates. Any changes to @@ -1541,9 +1548,9 @@ class DBImpl : public DB { virtual Status Recover( const std::vector& column_families, bool read_only = false, bool error_if_wal_file_exists = false, - bool error_if_data_exists_in_wals = false, + bool error_if_data_exists_in_wals = false, bool is_retry = false, uint64_t* recovered_seq = nullptr, - RecoveryContext* recovery_ctx = nullptr); + RecoveryContext* recovery_ctx = nullptr, bool* can_retry = nullptr); virtual bool OwnTablesAndLogs() const { return true; } @@ -1553,22 +1560,36 @@ class DBImpl : public DB { // Assign db_id_ and write DB ID to manifest if necessary. void SetDBId(std::string&& id, bool read_only, RecoveryContext* recovery_ctx); + // Collect a deduplicated collection of paths used by this DB, including + // dbname_, DBOptions.db_paths, ColumnFamilyOptions.cf_paths. + std::set CollectAllDBPaths(); + // REQUIRES: db mutex held when calling this function, but the db mutex can // be released and re-acquired. Db mutex will be held when the function // returns. - // After recovery, there may be SST files in db/cf paths that are - // not referenced in the MANIFEST (e.g. + // It stores all existing data files (SST and Blob) in RecoveryContext. In + // the meantime, we find out the largest file number present in the paths, and + // bump up the version set's next_file_number_ to be 1 + largest_file_number. + // recovery_ctx stores the context about version edits. All those edits are + // persisted to new Manifest after successfully syncing the new WAL. + Status MaybeUpdateNextFileNumber(RecoveryContext* recovery_ctx); + + // Track existing data files, including both referenced and unreferenced SST + // and Blob files in SstFileManager. This is only called during DB::Open and + // it's called before any file deletion start so that their deletion can be + // properly rate limited. + // Files may not be referenced in the MANIFEST because (e.g. // 1. It's best effort recovery; // 2. The VersionEdits referencing the SST files are appended to // RecoveryContext, DB crashes when syncing the MANIFEST, the VersionEdits are // still not synced to MANIFEST during recovery.) - // It stores the SST files to be deleted in RecoveryContext. In the - // meantime, we find out the largest file number present in the paths, and - // bump up the version set's next_file_number_ to be 1 + largest_file_number. - // recovery_ctx stores the context about version edits and files to be - // deleted. All those edits are persisted to new Manifest after successfully - // syncing the new WAL. - Status DeleteUnreferencedSstFiles(RecoveryContext* recovery_ctx); + // + // If the file is referenced in Manifest (typically that's the + // vast majority of all files), since it already has the file size + // on record, we don't need to query the file system. Otherwise, we query the + // file system for the size of an unreferenced file. + void TrackExistingDataFiles( + const std::vector& existing_data_files); // SetDbSessionId() should be called in the constuctor DBImpl() // to ensure that db_session_id_ gets updated every time the DB is opened @@ -1923,7 +1944,7 @@ class DBImpl : public DB { // corrupted_log_found is set to true if we recover from a corrupted log file. Status RecoverLogFiles(const std::vector& log_numbers, SequenceNumber* next_sequence, bool read_only, - bool* corrupted_log_found, + bool is_retry, bool* corrupted_log_found, RecoveryContext* recovery_ctx); // The following two methods are used to flush a memtable to @@ -2331,10 +2352,7 @@ class DBImpl : public DB { // A structure to hold the information required to process MultiGet of keys // belonging to one column family. For a multi column family MultiGet, there // will be a container of these objects. - struct MultiGetColumnFamilyData { - ColumnFamilyHandle* cf; - ColumnFamilyData* cfd; - + struct MultiGetKeyRangePerCf { // For the batched MultiGet which relies on sorted keys, start specifies // the index of first key belonging to this column family in the sorted // list. @@ -2344,31 +2362,33 @@ class DBImpl : public DB { // belonging to this column family in the sorted list size_t num_keys; + MultiGetKeyRangePerCf() : start(0), num_keys(0) {} + + MultiGetKeyRangePerCf(size_t first, size_t count) + : start(first), num_keys(count) {} + }; + + // A structure to contain ColumnFamilyData and the SuperVersion obtained for + // the consistent view of DB + struct ColumnFamilySuperVersionPair { + ColumnFamilyHandleImpl* cfh; + ColumnFamilyData* cfd; + // SuperVersion for the column family obtained in a manner that ensures a // consistent view across all column families in the DB SuperVersion* super_version; - MultiGetColumnFamilyData(ColumnFamilyHandle* column_family, - SuperVersion* sv) - : cf(column_family), - cfd(static_cast(cf)->cfd()), - start(0), - num_keys(0), + ColumnFamilySuperVersionPair(ColumnFamilyHandle* column_family, + SuperVersion* sv) + : cfh(static_cast(column_family)), + cfd(cfh->cfd()), super_version(sv) {} - MultiGetColumnFamilyData(ColumnFamilyHandle* column_family, size_t first, - size_t count, SuperVersion* sv) - : cf(column_family), - cfd(static_cast(cf)->cfd()), - start(first), - num_keys(count), - super_version(sv) {} - - MultiGetColumnFamilyData() = default; + ColumnFamilySuperVersionPair() = default; }; // A common function to obtain a consistent snapshot, which can be implicit // if the user doesn't specify a snapshot in read_options, across - // multiple column families for MultiGet. It will attempt to get an implicit + // multiple column families. It will attempt to get an implicit // snapshot without acquiring the db_mutes, but will give up after a few // tries and acquire the mutex if a memtable flush happens. The template // allows both the batched and non-batched MultiGet to call this with @@ -2377,18 +2397,26 @@ class DBImpl : public DB { // If callback is non-null, the callback is refreshed with the snapshot // sequence number // + // `extra_sv_ref` is used to indicate whether thread-local SuperVersion + // should be obtained with an extra ref (by GetReferencedSuperVersion()) or + // not (by GetAndRefSuperVersion()). For instance, point lookup like MultiGet + // does not require SuperVersion to be re-acquired throughout the entire + // invocation (no need extra ref), while MultiCfIterators may need the + // SuperVersion to be updated during Refresh() (requires extra ref). + // // `sv_from_thread_local` being set to false indicates that the SuperVersion // obtained from the ColumnFamilyData, whereas true indicates they are thread // local. + // // A non-OK status will be returned if for a column family that enables // user-defined timestamp feature, the specified `ReadOptions.timestamp` // attemps to read collapsed history. - template - Status MultiCFSnapshot( - const ReadOptions& read_options, ReadCallback* callback, - std::function& - iter_deref_func, - T* cf_list, SequenceNumber* snapshot, bool* sv_from_thread_local); + template + Status MultiCFSnapshot(const ReadOptions& read_options, + ReadCallback* callback, + IterDerefFuncType iter_deref_func, T* cf_list, + bool extra_sv_ref, SequenceNumber* snapshot, + bool* sv_from_thread_local); // The actual implementation of the batching MultiGet. The caller is expected // to have acquired the SuperVersion and pass in a snapshot sequence number @@ -2411,6 +2439,13 @@ class DBImpl : public DB { bool ShouldReferenceSuperVersion(const MergeContext& merge_context); + template + std::unique_ptr NewMultiCfIterator( + const ReadOptions& _read_options, + const std::vector& column_families, + ErrorIteratorFuncType error_iterator_func); + // Lock over the persistent DB state. Non-nullptr iff successfully acquired. FileLock* db_lock_; @@ -2462,6 +2497,11 @@ class DBImpl : public DB { uint64_t logfile_number_; // Log files that we can recycle. Must be protected by db mutex_. std::deque log_recycle_files_; + // The minimum log file number taht can be recycled, if log recycling is + // enabled. This is used to ensure that log files created by previous + // instances of the database are not recycled, as we cannot be sure they + // were created in the recyclable format. + uint64_t min_log_number_to_recycle_; // Protected by log_write_mutex_. bool log_dir_synced_; // Without two_write_queues, read and writes to log_empty_ are protected by diff --git a/db/db_impl/db_impl_compaction_flush.cc b/db/db_impl/db_impl_compaction_flush.cc index 15b0377d427..c3e29c200b0 100644 --- a/db/db_impl/db_impl_compaction_flush.cc +++ b/db/db_impl/db_impl_compaction_flush.cc @@ -1420,6 +1420,14 @@ Status DBImpl::CompactFiles(const CompactionOptions& compact_options, LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL, immutable_db_options_.info_log.get()); + if (compact_options.compression != + CompressionType::kDisableCompressionOption) { + ROCKS_LOG_WARN(immutable_db_options_.info_log, + "[%s] [JOB %d] Found use of deprecated option " + "`CompactionOptions::compression`", + cfd->GetName().c_str(), job_context.job_id); + } + // Perform CompactFiles TEST_SYNC_POINT("TestCompactFiles::IngestExternalFile2"); TEST_SYNC_POINT_CALLBACK("TestCompactFiles:PausingManualCompaction:3", @@ -1774,6 +1782,8 @@ void DBImpl::NotifyOnCompactionCompleted( // REQUIREMENT: block all background work by calling PauseBackgroundWork() // before calling this function +// TODO (hx235): Replace Status::NotSupported() with Status::Aborted() for +// better semantics like CompactFiles() Status DBImpl::ReFitLevel(ColumnFamilyData* cfd, int level, int target_level) { assert(level < cfd->NumberLevels()); if (target_level >= cfd->NumberLevels()) { @@ -1809,6 +1819,8 @@ Status DBImpl::ReFitLevel(ColumnFamilyData* cfd, int level, int target_level) { if (to_level != level) { std::vector input(1); input[0].level = level; + // TODO (hx235): Only refit the output files in the current manual + // compaction instead of all the files in the output level for (auto& f : vstorage->LevelFiles(level)) { input[0].files.push_back(f); } @@ -1840,6 +1852,12 @@ Status DBImpl::ReFitLevel(ColumnFamilyData* cfd, int level, int target_level) { } } else { // to_level < level + if (to_level == 0 && input[0].files.size() > 1) { + refitting_level_ = false; + return Status::Aborted( + "Moving more than 1 file from non-L0 to L0 is not allowed as it " + "does not bring any benefit to read nor write throughput."); + } // Check levels are empty for a trivial move for (int l = to_level; l < level; l++) { if (vstorage->NumLevelFiles(l) > 0) { @@ -2154,16 +2172,6 @@ Status DBImpl::RunManualCompaction( manual.begin, manual.end, &manual.manual_end, &manual_conflict, max_file_num_to_ignore, trim_ts)) == nullptr && manual_conflict))) { - if (!scheduled) { - // There is a conflicting compaction - if (manual_compaction_paused_ > 0 || manual.canceled == true) { - // Stop waiting since it was canceled. Pretend the error came from - // compaction so the below cleanup/error handling code can process it. - manual.done = true; - manual.status = - Status::Incomplete(Status::SubCode::kManualCompactionPaused); - } - } if (!manual.done) { bg_cv_.Wait(); } @@ -2238,6 +2246,17 @@ Status DBImpl::RunManualCompaction( *final_output_level = compaction->output_level(); } } + if (!scheduled) { + // There is nothing scheduled to wait on, so any cancellation can end the + // manual now. + if (manual_compaction_paused_ > 0 || manual.canceled == true) { + // Stop waiting since it was canceled. Pretend the error came from + // compaction so the below cleanup/error handling code can process it. + manual.done = true; + manual.status = + Status::Incomplete(Status::SubCode::kManualCompactionPaused); + } + } } log_buffer.FlushBufferToLog(); @@ -3032,6 +3051,8 @@ void DBImpl::SchedulePendingCompaction(ColumnFamilyData* cfd) { return; } if (!cfd->queued_for_compaction() && cfd->NeedsCompaction()) { + TEST_SYNC_POINT_CALLBACK("SchedulePendingCompaction::cfd", + static_cast(cfd)); AddToCompactionQueue(cfd); ++unscheduled_compactions_; } @@ -4344,6 +4365,7 @@ Status DBImpl::WaitForCompact( } if ((bg_bottom_compaction_scheduled_ || bg_compaction_scheduled_ || bg_flush_scheduled_ || unscheduled_compactions_ || + (wait_for_compact_options.wait_for_purge && bg_purge_scheduled_) || unscheduled_flushes_ || error_handler_.IsRecoveryInProgress()) && (error_handler_.GetBGError().ok())) { if (wait_for_compact_options.timeout.count()) { @@ -4351,6 +4373,7 @@ Status DBImpl::WaitForCompact( return Status::TimedOut(); } } else { + TEST_SYNC_POINT("DBImpl::WaitForCompact:InsideLoop"); bg_cv_.Wait(); } } else if (wait_for_compact_options.close_db) { diff --git a/db/db_impl/db_impl_files.cc b/db/db_impl/db_impl_files.cc index e1ba41e4ac7..4813978583c 100644 --- a/db/db_impl/db_impl_files.cc +++ b/db/db_impl/db_impl_files.cc @@ -28,6 +28,8 @@ uint64_t DBImpl::MinLogNumberToKeep() { return versions_->min_log_number_to_keep(); } +uint64_t DBImpl::MinLogNumberToRecycle() { return min_log_number_to_recycle_; } + uint64_t DBImpl::MinObsoleteSstNumberToKeep() { mutex_.AssertHeld(); if (!pending_outputs_.empty()) { @@ -175,31 +177,10 @@ void DBImpl::FindObsoleteFiles(JobContext* job_context, bool force, versions_->AddLiveFiles(&job_context->sst_live, &job_context->blob_live); InfoLogPrefix info_log_prefix(!immutable_db_options_.db_log_dir.empty(), dbname_); - std::set paths; - for (size_t path_id = 0; path_id < immutable_db_options_.db_paths.size(); - path_id++) { - paths.insert(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++) { - auto& path = cfd->ioptions()->cf_paths[path_id].path; - - if (paths.find(path) == paths.end()) { - paths.insert(path); - } - } - } - + // PurgeObsoleteFiles will dedupe duplicate files. IOOptions io_opts; io_opts.do_not_recurse = true; - for (auto& path : paths) { + for (auto& path : CollectAllDBPaths()) { // set of all files in the directory. We'll exclude files that are still // alive in the subsequent processings. std::vector files; @@ -298,7 +279,8 @@ void DBImpl::FindObsoleteFiles(JobContext* job_context, bool force, while (alive_log_files_.begin()->number < min_log_number) { auto& earliest = *alive_log_files_.begin(); if (immutable_db_options_.recycle_log_file_num > - log_recycle_files_.size()) { + log_recycle_files_.size() && + earliest.number >= MinLogNumberToRecycle()) { ROCKS_LOG_INFO(immutable_db_options_.info_log, "adding log %" PRIu64 " to recycle list\n", earliest.number); @@ -963,28 +945,26 @@ Status DBImpl::SetupDBId(const WriteOptions& write_options, bool read_only, return s; } -Status DBImpl::DeleteUnreferencedSstFiles(RecoveryContext* recovery_ctx) { - mutex_.AssertHeld(); - std::vector paths; - paths.push_back(NormalizePath(dbname_ + std::string(1, kFilePathSeparator))); +std::set DBImpl::CollectAllDBPaths() { + std::set all_db_paths; + all_db_paths.insert(NormalizePath(dbname_)); for (const auto& db_path : immutable_db_options_.db_paths) { - paths.push_back( - NormalizePath(db_path.path + std::string(1, kFilePathSeparator))); + all_db_paths.insert(NormalizePath(db_path.path)); } for (const auto* cfd : *versions_->GetColumnFamilySet()) { for (const auto& cf_path : cfd->ioptions()->cf_paths) { - paths.push_back( - NormalizePath(cf_path.path + std::string(1, kFilePathSeparator))); + all_db_paths.insert(NormalizePath(cf_path.path)); } } - // Dedup paths - std::sort(paths.begin(), paths.end()); - paths.erase(std::unique(paths.begin(), paths.end()), paths.end()); + return all_db_paths; +} +Status DBImpl::MaybeUpdateNextFileNumber(RecoveryContext* recovery_ctx) { + mutex_.AssertHeld(); uint64_t next_file_number = versions_->current_next_file_number(); uint64_t largest_file_number = next_file_number; Status s; - for (const auto& path : paths) { + for (const auto& path : CollectAllDBPaths()) { std::vector files; s = env_->GetChildren(path, &files); if (!s.ok()) { @@ -996,13 +976,10 @@ Status DBImpl::DeleteUnreferencedSstFiles(RecoveryContext* recovery_ctx) { if (!ParseFileName(fname, &number, &type)) { continue; } - // path ends with '/' or '\\' - const std::string normalized_fpath = path + fname; + const std::string normalized_fpath = path + kFilePathSeparator + fname; largest_file_number = std::max(largest_file_number, number); - if (type == kTableFile && number >= next_file_number && - recovery_ctx->files_to_delete_.find(normalized_fpath) == - recovery_ctx->files_to_delete_.end()) { - recovery_ctx->files_to_delete_.emplace(normalized_fpath, path); + if ((type == kTableFile || type == kBlobFile)) { + recovery_ctx->existing_data_files_.push_back(normalized_fpath); } } } @@ -1022,5 +999,4 @@ Status DBImpl::DeleteUnreferencedSstFiles(RecoveryContext* recovery_ctx) { recovery_ctx->UpdateVersionEdits(default_cfd, edit); return s; } - } // namespace ROCKSDB_NAMESPACE diff --git a/db/db_impl/db_impl_follower.cc b/db/db_impl/db_impl_follower.cc new file mode 100644 index 00000000000..8d21f530cd0 --- /dev/null +++ b/db/db_impl/db_impl_follower.cc @@ -0,0 +1,310 @@ +// Copyright (c) 2024-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/db_impl/db_impl_follower.h" + +#include + +#include "db/arena_wrapped_db_iter.h" +#include "db/merge_context.h" +#include "env/composite_env_wrapper.h" +#include "env/fs_on_demand.h" +#include "logging/auto_roll_logger.h" +#include "logging/logging.h" +#include "monitoring/perf_context_imp.h" +#include "rocksdb/configurable.h" +#include "rocksdb/db.h" +#include "util/cast_util.h" +#include "util/write_batch_util.h" + +namespace ROCKSDB_NAMESPACE { + +DBImplFollower::DBImplFollower(const DBOptions& db_options, + std::unique_ptr&& env, + const std::string& dbname, std::string src_path) + : DBImplSecondary(db_options, dbname, ""), + env_guard_(std::move(env)), + stop_requested_(false), + src_path_(std::move(src_path)), + cv_(&mu_) { + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "Opening the db in follower mode"); + LogFlush(immutable_db_options_.info_log); +} + +DBImplFollower::~DBImplFollower() { + Status s = Close(); + if (!s.ok()) { + ROCKS_LOG_INFO(immutable_db_options_.info_log, "Error closing DB : %s", + s.ToString().c_str()); + } +} + +// Recover a follower DB instance by reading the MANIFEST. The verification +// as part of the MANIFEST replay will ensure that local links to the +// leader's files are created, thus ensuring we can continue reading them +// even if the leader deletes those files due to compaction. +// TODO: +// 1. Devise a mechanism to prevent misconfiguration by, for example, +// keeping a local copy of the IDENTITY file and cross checking +// 2. Make the recovery more robust by retrying if the first attempt +// fails. +Status DBImplFollower::Recover( + const std::vector& column_families, + bool /*readonly*/, bool /*error_if_wal_file_exists*/, + bool /*error_if_data_exists_in_wals*/, bool /*is_retry*/, uint64_t*, + RecoveryContext* /*recovery_ctx*/, bool* /*can_retry*/) { + mutex_.AssertHeld(); + + JobContext job_context(0); + Status s; + s = static_cast(versions_.get()) + ->Recover(column_families, &manifest_reader_, &manifest_reporter_, + &manifest_reader_status_); + if (!s.ok()) { + if (manifest_reader_status_) { + manifest_reader_status_->PermitUncheckedError(); + } + return s; + } + if (immutable_db_options_.paranoid_checks && s.ok()) { + s = CheckConsistency(); + } + if (s.ok()) { + default_cf_handle_ = new ColumnFamilyHandleImpl( + versions_->GetColumnFamilySet()->GetDefault(), this, &mutex_); + default_cf_internal_stats_ = default_cf_handle_->cfd()->internal_stats(); + + // Start the periodic catch-up thread + // TODO: See if it makes sense to have a threadpool, rather than a thread + // per follower DB instance + catch_up_thread_.reset( + new port::Thread(&DBImplFollower::PeriodicRefresh, this)); + } + + return s; +} + +// Try to catch up by tailing the MANIFEST. +// TODO: +// 1. Cleanup obsolete files afterward +// 2. Add some error notifications and statistics +Status DBImplFollower::TryCatchUpWithLeader() { + assert(versions_.get() != nullptr); + assert(manifest_reader_.get() != nullptr); + Status s; + // read the manifest and apply new changes to the follower instance + std::unordered_set cfds_changed; + JobContext job_context(0, true /*create_superversion*/); + { + InstrumentedMutexLock lock_guard(&mutex_); + s = static_cast_with_check(versions_.get()) + ->ReadAndApply(&mutex_, &manifest_reader_, + manifest_reader_status_.get(), &cfds_changed); + + ROCKS_LOG_INFO(immutable_db_options_.info_log, "Last sequence is %" PRIu64, + static_cast(versions_->LastSequence())); + for (ColumnFamilyData* cfd : cfds_changed) { + if (cfd->IsDropped()) { + ROCKS_LOG_DEBUG(immutable_db_options_.info_log, "[%s] is dropped\n", + cfd->GetName().c_str()); + continue; + } + VersionStorageInfo::LevelSummaryStorage tmp; + ROCKS_LOG_DEBUG(immutable_db_options_.info_log, + "[%s] Level summary: %s\n", cfd->GetName().c_str(), + cfd->current()->storage_info()->LevelSummary(&tmp)); + } + + if (s.ok()) { + for (auto cfd : cfds_changed) { + if (cfd->mem()->GetEarliestSequenceNumber() < + versions_->LastSequence()) { + // Construct a new memtable with earliest sequence number set to the + // last sequence number in the VersionSet. This matters when + // DBImpl::MultiCFSnapshot tries to get consistent references + // to super versions in a lock free manner, it checks the earliest + // sequence number to detect if there was a change in version in + // the meantime. + const MutableCFOptions mutable_cf_options = + *cfd->GetLatestMutableCFOptions(); + MemTable* new_mem = cfd->ConstructNewMemtable( + mutable_cf_options, versions_->LastSequence()); + cfd->mem()->SetNextLogNumber(cfd->GetLogNumber()); + cfd->mem()->ConstructFragmentedRangeTombstones(); + cfd->imm()->Add(cfd->mem(), &job_context.memtables_to_free); + new_mem->Ref(); + cfd->SetMemtable(new_mem); + } + + // This will check if the old memtable is still referenced + cfd->imm()->RemoveOldMemTables(cfd->GetLogNumber(), + &job_context.memtables_to_free); + auto& sv_context = job_context.superversion_contexts.back(); + cfd->InstallSuperVersion(&sv_context, &mutex_); + sv_context.NewSuperVersion(); + } + } + } + job_context.Clean(); + + return s; +} + +void DBImplFollower::PeriodicRefresh() { + while (!stop_requested_.load()) { + MutexLock l(&mu_); + int64_t wait_until = + immutable_db_options_.clock->NowMicros() + + immutable_db_options_.follower_refresh_catchup_period_ms * 1000; + immutable_db_options_.clock->TimedWait( + &cv_, std::chrono::microseconds(wait_until)); + if (stop_requested_.load()) { + break; + } + Status s; + for (uint64_t i = 0; + i < immutable_db_options_.follower_catchup_retry_count && + !stop_requested_.load(); + ++i) { + s = TryCatchUpWithLeader(); + + if (s.ok()) { + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "Successful catch up on attempt %llu", + static_cast(i)); + break; + } + wait_until = immutable_db_options_.clock->NowMicros() + + immutable_db_options_.follower_catchup_retry_wait_ms * 1000; + immutable_db_options_.clock->TimedWait( + &cv_, std::chrono::microseconds(wait_until)); + } + if (!s.ok()) { + ROCKS_LOG_INFO(immutable_db_options_.info_log, "Catch up unsuccessful"); + } + } +} + +Status DBImplFollower::Close() { + if (catch_up_thread_) { + stop_requested_.store(true); + { + MutexLock l(&mu_); + cv_.SignalAll(); + } + catch_up_thread_->join(); + catch_up_thread_.reset(); + } + + return DBImpl::Close(); +} + +Status DB::OpenAsFollower(const Options& options, const std::string& dbname, + const std::string& leader_path, + std::unique_ptr* dbptr) { + dbptr->reset(); + + DBOptions db_options(options); + ColumnFamilyOptions cf_options(options); + std::vector column_families; + column_families.emplace_back(kDefaultColumnFamilyName, cf_options); + std::vector handles; + + Status s = DB::OpenAsFollower(db_options, dbname, leader_path, + column_families, &handles, dbptr); + if (s.ok()) { + assert(handles.size() == 1); + delete handles[0]; + } + return s; +} + +Status DB::OpenAsFollower( + const DBOptions& db_options, const std::string& dbname, + const std::string& src_path, + const std::vector& column_families, + std::vector* handles, std::unique_ptr* dbptr) { + dbptr->reset(); + + FileSystem* fs = db_options.env->GetFileSystem().get(); + { + IOStatus io_s; + if (db_options.create_if_missing) { + io_s = fs->CreateDirIfMissing(dbname, IOOptions(), nullptr); + } else { + io_s = fs->FileExists(dbname, IOOptions(), nullptr); + } + if (!io_s.ok()) { + return static_cast(io_s); + } + } + std::unique_ptr new_env(new CompositeEnvWrapper( + db_options.env, NewOnDemandFileSystem(db_options.env->GetFileSystem(), + src_path, dbname))); + + DBOptions tmp_opts(db_options); + Status s; + tmp_opts.env = new_env.get(); + if (nullptr == tmp_opts.info_log) { + s = CreateLoggerFromOptions(dbname, tmp_opts, &tmp_opts.info_log); + if (!s.ok()) { + tmp_opts.info_log = nullptr; + return s; + } + } + + handles->clear(); + DBImplFollower* impl = + new DBImplFollower(tmp_opts, std::move(new_env), dbname, src_path); + impl->versions_.reset(new ReactiveVersionSet( + dbname, &impl->immutable_db_options_, impl->file_options_, + impl->table_cache_.get(), impl->write_buffer_manager_, + &impl->write_controller_, impl->io_tracer_)); + impl->column_family_memtables_.reset( + new ColumnFamilyMemTablesImpl(impl->versions_->GetColumnFamilySet())); + impl->wal_in_db_path_ = impl->immutable_db_options_.IsWalDirSameAsDBPath(); + + impl->mutex_.Lock(); + s = impl->Recover(column_families, /*read_only=*/true, + /*error_if_wal_file_exists=*/false, + /*error_if_data_exists_in_wals=*/false); + if (s.ok()) { + for (const auto& cf : column_families) { + auto cfd = + impl->versions_->GetColumnFamilySet()->GetColumnFamily(cf.name); + if (nullptr == cfd) { + s = Status::InvalidArgument("Column family not found", cf.name); + break; + } + handles->push_back(new ColumnFamilyHandleImpl(cfd, impl, &impl->mutex_)); + } + } + SuperVersionContext sv_context(false /* create_superversion */); + if (s.ok()) { + for (auto cfd : *impl->versions_->GetColumnFamilySet()) { + sv_context.NewSuperVersion(); + cfd->InstallSuperVersion(&sv_context, &impl->mutex_); + } + } + impl->mutex_.Unlock(); + sv_context.Clean(); + if (s.ok()) { + dbptr->reset(impl); + for (auto h : *handles) { + impl->NewThreadStatusCfInfo( + static_cast_with_check(h)->cfd()); + } + } else { + for (auto h : *handles) { + delete h; + } + handles->clear(); + delete impl; + } + return s; +} + +} // namespace ROCKSDB_NAMESPACE diff --git a/db/db_impl/db_impl_follower.h b/db/db_impl/db_impl_follower.h new file mode 100644 index 00000000000..60992c111e5 --- /dev/null +++ b/db/db_impl/db_impl_follower.h @@ -0,0 +1,53 @@ +// Copyright (c) 2024-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 +#include + +#include "db/db_impl/db_impl.h" +#include "db/db_impl/db_impl_secondary.h" +#include "logging/logging.h" +#include "port/port.h" + +namespace ROCKSDB_NAMESPACE { + +class DBImplFollower : public DBImplSecondary { + public: + DBImplFollower(const DBOptions& db_options, std::unique_ptr&& env, + const std::string& dbname, std::string src_path); + ~DBImplFollower(); + + Status Close() override; + + protected: + bool OwnTablesAndLogs() const override { + // TODO: Change this to true once we've properly implemented file + // deletion for the read scaling case + return false; + } + + Status Recover(const std::vector& column_families, + bool /*readonly*/, bool /*error_if_wal_file_exists*/, + bool /*error_if_data_exists_in_wals*/, + bool /*is_retry*/ = false, uint64_t* = nullptr, + RecoveryContext* /*recovery_ctx*/ = nullptr, + bool* /*can_retry*/ = nullptr) override; + + private: + friend class DB; + + Status TryCatchUpWithLeader(); + void PeriodicRefresh(); + + std::unique_ptr env_guard_; + std::unique_ptr catch_up_thread_; + std::atomic stop_requested_; + std::string src_path_; + port::Mutex mu_; + port::CondVar cv_; +}; +} // namespace ROCKSDB_NAMESPACE diff --git a/db/db_impl/db_impl_open.cc b/db/db_impl/db_impl_open.cc index a0e3f7cf152..61aa289b713 100644 --- a/db/db_impl/db_impl_open.cc +++ b/db/db_impl/db_impl_open.cc @@ -175,15 +175,6 @@ DBOptions SanitizeOptions(const std::string& dbname, const DBOptions& src, } } } - // 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(result.sst_file_manager.get()); - for (size_t i = 0; i < result.db_paths.size(); i++) { - DeleteScheduler::CleanupDirectory(result.env, sfm, result.db_paths[i].path) - .PermitUncheckedError(); - } // Create a default SstFileManager for purposes of tracking compaction size // and facilitating recovery from out of space errors. @@ -413,7 +404,8 @@ IOStatus Directories::SetDirectories(FileSystem* fs, const std::string& dbname, Status DBImpl::Recover( const std::vector& column_families, bool read_only, bool error_if_wal_file_exists, bool error_if_data_exists_in_wals, - uint64_t* recovered_seq, RecoveryContext* recovery_ctx) { + bool is_retry, uint64_t* recovered_seq, RecoveryContext* recovery_ctx, + bool* can_retry) { mutex_.AssertHeld(); const WriteOptions write_options(Env::IOActivity::kDBOpen); @@ -529,7 +521,31 @@ Status DBImpl::Recover( Status s; bool missing_table_file = false; if (!immutable_db_options_.best_efforts_recovery) { - s = versions_->Recover(column_families, read_only, &db_id_); + // Status of reading the descriptor file + Status desc_status; + s = versions_->Recover(column_families, read_only, &db_id_, + /*no_error_if_files_missing=*/false, is_retry, + &desc_status); + desc_status.PermitUncheckedError(); + if (can_retry) { + // If we're opening for the first time and the failure is likely due to + // a corrupt MANIFEST file (could result in either the log::Reader + // detecting a corrupt record, or SST files not found error due to + // discarding badly formed tail records) + if (!is_retry && + (desc_status.IsCorruption() || s.IsNotFound() || s.IsCorruption()) && + CheckFSFeatureSupport(fs_.get(), + FSSupportedOps::kVerifyAndReconstructRead)) { + *can_retry = true; + ROCKS_LOG_ERROR( + immutable_db_options_.info_log, + "Possible corruption detected while replaying MANIFEST %s, %s. " + "Will be retried.", + desc_status.ToString().c_str(), s.ToString().c_str()); + } else { + *can_retry = false; + } + } } else { assert(!files_in_dbname.empty()); s = versions_->TryRecover(column_families, read_only, files_in_dbname, @@ -644,7 +660,7 @@ Status DBImpl::Recover( s = SetupDBId(write_options, read_only, recovery_ctx); ROCKS_LOG_INFO(immutable_db_options_.info_log, "DB ID: %s\n", db_id_.c_str()); if (s.ok() && !read_only) { - s = DeleteUnreferencedSstFiles(recovery_ctx); + s = MaybeUpdateNextFileNumber(recovery_ctx); } if (immutable_db_options_.paranoid_checks && s.ok()) { @@ -767,8 +783,8 @@ Status DBImpl::Recover( std::sort(wals.begin(), wals.end()); bool corrupted_wal_found = false; - s = RecoverLogFiles(wals, &next_sequence, read_only, &corrupted_wal_found, - recovery_ctx); + s = RecoverLogFiles(wals, &next_sequence, read_only, is_retry, + &corrupted_wal_found, recovery_ctx); if (corrupted_wal_found && recovered_seq != nullptr) { *recovered_seq = next_sequence; } @@ -946,19 +962,6 @@ Status DBImpl::LogAndApplyForRecovery(const RecoveryContext& recovery_ctx) { recovery_ctx.mutable_cf_opts_, read_options, write_options, recovery_ctx.edit_lists_, &mutex_, directories_.GetDbDir()); - if (s.ok() && !(recovery_ctx.files_to_delete_.empty())) { - mutex_.Unlock(); - for (const auto& stale_sst_file : recovery_ctx.files_to_delete_) { - s = DeleteDBFile(&immutable_db_options_, stale_sst_file.first, - stale_sst_file.second, - /*force_bg=*/false, - /*force_fg=*/false); - if (!s.ok()) { - break; - } - } - mutex_.Lock(); - } return s; } @@ -1078,7 +1081,7 @@ bool DBImpl::InvokeWalFilterIfNeededOnWalRecord(uint64_t wal_number, // REQUIRES: wal_numbers are sorted in ascending order Status DBImpl::RecoverLogFiles(const std::vector& wal_numbers, SequenceNumber* next_sequence, bool read_only, - bool* corrupted_wal_found, + bool is_retry, bool* corrupted_wal_found, RecoveryContext* recovery_ctx) { struct LogReporter : public log::Reader::Reporter { Env* env; @@ -1189,7 +1192,7 @@ Status DBImpl::RecoverLogFiles(const std::vector& wal_numbers, } file_reader.reset(new SequentialFileReader( std::move(file), fname, immutable_db_options_.log_readahead_size, - io_tracer_)); + io_tracer_, /*listeners=*/{}, /*rate_limiter=*/nullptr, is_retry)); } // Create the log reader. @@ -1833,8 +1836,12 @@ Status DB::Open(const DBOptions& db_options, const std::string& dbname, const bool kBatchPerTxn = true; ThreadStatusUtil::SetEnableTracking(db_options.enable_thread_tracking); ThreadStatusUtil::SetThreadOperation(ThreadStatus::OperationType::OP_DBOPEN); - Status s = DBImpl::Open(db_options, dbname, column_families, handles, dbptr, - !kSeqPerBatch, kBatchPerTxn); + bool can_retry = false; + Status s; + do { + s = DBImpl::Open(db_options, dbname, column_families, handles, dbptr, + !kSeqPerBatch, kBatchPerTxn, can_retry, &can_retry); + } while (!s.ok() && can_retry); ThreadStatusUtil::ResetThreadStatus(); return s; } @@ -1953,10 +1960,55 @@ IOStatus DBImpl::CreateWAL(const WriteOptions& write_options, return io_s; } +void DBImpl::TrackExistingDataFiles( + const std::vector& existing_data_files) { + auto sfm = static_cast( + immutable_db_options_.sst_file_manager.get()); + assert(sfm); + std::vector metadata; + GetAllColumnFamilyMetaData(&metadata); + + std::unordered_set referenced_files; + for (const auto& md : metadata) { + for (const auto& lmd : md.levels) { + for (const auto& fmd : lmd.files) { + // We're assuming that each sst file name exists in at most one of + // the paths. + std::string file_path = + fmd.directory + kFilePathSeparator + fmd.relative_filename; + sfm->OnAddFile(file_path, fmd.size).PermitUncheckedError(); + referenced_files.insert(file_path); + } + } + for (const auto& bmd : md.blob_files) { + std::string name = bmd.blob_file_name; + // The BlobMetaData.blob_file_name may start with "/". + if (!name.empty() && name[0] == kFilePathSeparator) { + name = name.substr(1); + } + // We're assuming that each blob file name exists in at most one of + // the paths. + std::string file_path = bmd.blob_file_path + kFilePathSeparator + name; + sfm->OnAddFile(file_path, bmd.blob_file_size).PermitUncheckedError(); + referenced_files.insert(file_path); + } + } + + for (const auto& file_path : existing_data_files) { + if (referenced_files.find(file_path) != referenced_files.end()) { + continue; + } + // There shouldn't be any duplicated files. In case there is, SstFileManager + // will take care of deduping it. + sfm->OnAddFile(file_path).PermitUncheckedError(); + } +} + Status DBImpl::Open(const DBOptions& db_options, const std::string& dbname, const std::vector& column_families, std::vector* handles, DB** dbptr, - const bool seq_per_batch, const bool batch_per_txn) { + const bool seq_per_batch, const bool batch_per_txn, + const bool is_retry, bool* can_retry) { const WriteOptions write_options(Env::IOActivity::kDBOpen); const ReadOptions read_options(Env::IOActivity::kDBOpen); @@ -1999,7 +2051,7 @@ Status DBImpl::Open(const DBOptions& db_options, const std::string& dbname, paths.emplace_back(cf_path.path); } } - for (auto& path : paths) { + for (const auto& path : paths) { s = impl->env_->CreateDirIfMissing(path); if (!s.ok()) { break; @@ -2029,8 +2081,8 @@ Status DBImpl::Open(const DBOptions& db_options, const std::string& dbname, uint64_t recovered_seq(kMaxSequenceNumber); s = impl->Recover(column_families, false /* read_only */, false /* error_if_wal_file_exists */, - false /* error_if_data_exists_in_wals */, &recovered_seq, - &recovery_ctx); + false /* error_if_data_exists_in_wals */, is_retry, + &recovered_seq, &recovery_ctx, can_retry); if (s.ok()) { uint64_t new_log_number = impl->versions_->NewFileNumber(); log::Writer* new_log = nullptr; @@ -2038,6 +2090,11 @@ Status DBImpl::Open(const DBOptions& db_options, const std::string& dbname, impl->GetWalPreallocateBlockSize(max_write_buffer_size); s = impl->CreateWAL(write_options, new_log_number, 0 /*recycle_log_number*/, preallocate_block_size, &new_log); + if (s.ok()) { + // Prevent log files created by previous instance from being recycled. + // They might be in alive_log_file_, and might get recycled otherwise. + impl->min_log_number_to_recycle_ = new_log_number; + } if (s.ok()) { InstrumentedMutexLock wl(&impl->log_write_mutex_); impl->logfile_number_ = new_log_number; @@ -2165,9 +2222,6 @@ Status DBImpl::Open(const DBOptions& db_options, const std::string& dbname, impl->WriteOptionsFile(write_options, true /*db_mutex_already_held*/); *dbptr = impl; impl->opened_successfully_ = true; - impl->DeleteObsoleteFiles(); - TEST_SYNC_POINT("DBImpl::Open:AfterDeleteFiles"); - impl->MaybeScheduleFlushOrCompaction(); } else { persist_options_status.PermitUncheckedError(); } @@ -2182,73 +2236,10 @@ Status DBImpl::Open(const DBOptions& db_options, const std::string& dbname, ROCKS_LOG_INFO(impl->immutable_db_options_.info_log, "SstFileManager instance %p", sfm); - // Notify SstFileManager about all sst files that already exist in - // db_paths[0] and cf_paths[0] when the DB is opened. - - // SstFileManagerImpl needs to know sizes of the files. For files whose size - // we already know (sst files that appear in manifest - typically that's the - // vast majority of all files), we'll pass the size to SstFileManager. - // For all other files SstFileManager will query the size from filesystem. - - std::vector metadata; - impl->GetAllColumnFamilyMetaData(&metadata); - - std::unordered_map known_file_sizes; - for (const auto& md : metadata) { - for (const auto& lmd : md.levels) { - for (const auto& fmd : lmd.files) { - known_file_sizes[fmd.relative_filename] = fmd.size; - } - } - for (const auto& bmd : md.blob_files) { - std::string name = bmd.blob_file_name; - // The BlobMetaData.blob_file_name may start with "/". - if (!name.empty() && name[0] == '/') { - name = name.substr(1); - } - known_file_sizes[name] = bmd.blob_file_size; - } - } - - std::vector 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()); - IOOptions io_opts; - io_opts.do_not_recurse = true; - for (auto& path : paths) { - std::vector existing_files; - impl->immutable_db_options_.fs - ->GetChildren(path, io_opts, &existing_files, - /*IODebugContext*=*/nullptr) - .PermitUncheckedError(); //**TODO: What do to on error? - for (auto& file_name : existing_files) { - uint64_t file_number; - FileType file_type; - std::string file_path = path + "/" + file_name; - if (ParseFileName(file_name, &file_number, &file_type) && - (file_type == kTableFile || file_type == kBlobFile)) { - // TODO: Check for errors from OnAddFile? - if (known_file_sizes.count(file_name)) { - // We're assuming that each sst file name exists in at most one of - // the paths. - sfm->OnAddFile(file_path, known_file_sizes.at(file_name)) - .PermitUncheckedError(); - } else { - sfm->OnAddFile(file_path).PermitUncheckedError(); - } - } - } - } + impl->TrackExistingDataFiles(recovery_ctx.existing_data_files_); // Reserve some disk buffer space. This is a heuristic - when we run out - // of disk space, this ensures that there is atleast write_buffer_size + // of disk space, this ensures that there is at least write_buffer_size // amount of free space before we resume DB writes. In low disk space // conditions, we want to avoid a lot of small L0 files due to frequent // WAL write failures and resultant forced flushes @@ -2256,6 +2247,27 @@ Status DBImpl::Open(const DBOptions& db_options, const std::string& dbname, impl->immutable_db_options_.db_paths[0].path); } + if (s.ok()) { + // 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). + // Note that we only start doing this and below delete obsolete file after + // `TrackExistingDataFiles` are called, the `max_trash_db_ratio` is + // ineffective otherwise and these files' deletion won't be rate limited + // which can cause discard stall. + for (const auto& path : impl->CollectAllDBPaths()) { + DeleteScheduler::CleanupDirectory(impl->immutable_db_options_.env, sfm, + path) + .PermitUncheckedError(); + } + impl->mutex_.Lock(); + // This will do a full scan. + impl->DeleteObsoleteFiles(); + TEST_SYNC_POINT("DBImpl::Open:AfterDeleteFiles"); + impl->MaybeScheduleFlushOrCompaction(); + impl->mutex_.Unlock(); + } if (s.ok()) { ROCKS_LOG_HEADER(impl->immutable_db_options_.info_log, "DB pointer %p", diff --git a/db/db_impl/db_impl_secondary.cc b/db/db_impl/db_impl_secondary.cc index f41884626e6..a5509c4f121 100644 --- a/db/db_impl/db_impl_secondary.cc +++ b/db/db_impl/db_impl_secondary.cc @@ -33,8 +33,8 @@ DBImplSecondary::~DBImplSecondary() = default; Status DBImplSecondary::Recover( const std::vector& column_families, bool /*readonly*/, bool /*error_if_wal_file_exists*/, - bool /*error_if_data_exists_in_wals*/, uint64_t*, - RecoveryContext* /*recovery_ctx*/) { + bool /*error_if_data_exists_in_wals*/, bool /*is_retry*/, uint64_t*, + RecoveryContext* /*recovery_ctx*/, bool* /*can_retry*/) { mutex_.AssertHeld(); JobContext job_context(0); diff --git a/db/db_impl/db_impl_secondary.h b/db/db_impl/db_impl_secondary.h index f1a40af3792..124cee3f3bb 100644 --- a/db/db_impl/db_impl_secondary.h +++ b/db/db_impl/db_impl_secondary.h @@ -82,8 +82,9 @@ class DBImplSecondary : public DBImpl { // and log_readers_ to facilitate future operations. Status Recover(const std::vector& column_families, bool read_only, bool error_if_wal_file_exists, - bool error_if_data_exists_in_wals, uint64_t* = nullptr, - RecoveryContext* recovery_ctx = nullptr) override; + bool error_if_data_exists_in_wals, bool is_retry = false, + uint64_t* = nullptr, RecoveryContext* recovery_ctx = nullptr, + bool* can_retry = nullptr) override; // Can return IOError due to files being deleted by the primary. To avoid // IOError in this case, application can coordinate between primary and @@ -276,6 +277,10 @@ class DBImplSecondary : public DBImpl { return false; } + std::unique_ptr manifest_reader_; + std::unique_ptr manifest_reporter_; + std::unique_ptr manifest_reader_status_; + private: friend class DB; @@ -304,10 +309,6 @@ class DBImplSecondary : public DBImpl { const CompactionServiceInput& input, CompactionServiceResult* result); - std::unique_ptr manifest_reader_; - std::unique_ptr manifest_reporter_; - std::unique_ptr manifest_reader_status_; - // Cache log readers for each log number, used for continue WAL replay // after recovery std::map> log_readers_; diff --git a/db/db_io_failure_test.cc b/db/db_io_failure_test.cc index 32e41e83ffe..c570c6c2c78 100644 --- a/db/db_io_failure_test.cc +++ b/db/db_io_failure_test.cc @@ -21,14 +21,15 @@ class CorruptionFS : public FileSystemWrapper { int num_writable_file_errors_; explicit CorruptionFS(const std::shared_ptr& _target, - bool fs_buffer) + bool fs_buffer, bool verify_read) : FileSystemWrapper(_target), writable_file_error_(false), num_writable_file_errors_(0), corruption_trigger_(INT_MAX), read_count_(0), rnd_(300), - fs_buffer_(fs_buffer) {} + fs_buffer_(fs_buffer), + verify_read_(verify_read) {} ~CorruptionFS() override { // Assert that the corruption was reset, which means it got triggered assert(corruption_trigger_ == INT_MAX); @@ -113,11 +114,13 @@ class CorruptionFS : public FileSystemWrapper { } void SupportedOps(int64_t& supported_ops) override { - supported_ops = 1 << FSSupportedOps::kVerifyAndReconstructRead | - 1 << FSSupportedOps::kAsyncIO; + supported_ops = 1 << FSSupportedOps::kAsyncIO; if (fs_buffer_) { supported_ops |= 1 << FSSupportedOps::kFSBuffer; } + if (verify_read_) { + supported_ops |= 1 << FSSupportedOps::kVerifyAndReconstructRead; + } } private: @@ -125,6 +128,7 @@ class CorruptionFS : public FileSystemWrapper { int read_count_; Random rnd_; bool fs_buffer_; + bool verify_read_; }; } // anonymous namespace @@ -696,23 +700,24 @@ TEST_F(DBIOFailureTest, CompactionSstSyncError) { class DBIOCorruptionTest : public DBIOFailureTest, - public testing::WithParamInterface> { + public testing::WithParamInterface> { public: DBIOCorruptionTest() : DBIOFailureTest() { BlockBasedTableOptions bbto; - Options options = CurrentOptions(); + options_ = CurrentOptions(); base_env_ = env_; EXPECT_NE(base_env_, nullptr); - fs_.reset( - new CorruptionFS(base_env_->GetFileSystem(), std::get<0>(GetParam()))); + fs_.reset(new CorruptionFS(base_env_->GetFileSystem(), + std::get<0>(GetParam()), + std::get<2>(GetParam()))); env_guard_ = NewCompositeEnv(fs_); - options.env = env_guard_.get(); + options_.env = env_guard_.get(); bbto.num_file_reads_for_auto_readahead = 0; - options.table_factory.reset(NewBlockBasedTableFactory(bbto)); - options.disable_auto_compactions = true; + options_.table_factory.reset(NewBlockBasedTableFactory(bbto)); + options_.disable_auto_compactions = true; - Reopen(options); + Reopen(options_); } ~DBIOCorruptionTest() { @@ -720,10 +725,13 @@ class DBIOCorruptionTest db_ = nullptr; } + Status ReopenDB() { return TryReopen(options_); } + protected: std::unique_ptr env_guard_; std::shared_ptr fs_; Env* base_env_; + Options options_; }; TEST_P(DBIOCorruptionTest, GetReadCorruptionRetry) { @@ -737,8 +745,13 @@ TEST_P(DBIOCorruptionTest, GetReadCorruptionRetry) { std::string val; ReadOptions ro; ro.async_io = std::get<1>(GetParam()); - ASSERT_OK(dbfull()->Get(ReadOptions(), "key1", &val)); - ASSERT_EQ(val, "val1"); + Status s = dbfull()->Get(ReadOptions(), "key1", &val); + if (std::get<2>(GetParam())) { + ASSERT_OK(s); + ASSERT_EQ(val, "val1"); + } else { + ASSERT_TRUE(s.IsCorruption()); + } } TEST_P(DBIOCorruptionTest, IterReadCorruptionRetry) { @@ -758,7 +771,11 @@ TEST_P(DBIOCorruptionTest, IterReadCorruptionRetry) { while (iter->status().ok() && iter->Valid()) { iter->Next(); } - ASSERT_OK(iter->status()); + if (std::get<2>(GetParam())) { + ASSERT_OK(iter->status()); + } else { + ASSERT_TRUE(iter->status().IsCorruption()); + } delete iter; } @@ -779,8 +796,13 @@ TEST_P(DBIOCorruptionTest, MultiGetReadCorruptionRetry) { ro.async_io = std::get<1>(GetParam()); dbfull()->MultiGet(ro, dbfull()->DefaultColumnFamily(), keys.size(), keys.data(), values.data(), statuses.data()); - ASSERT_EQ(values[0].ToString(), "val1"); - ASSERT_EQ(values[1].ToString(), "val2"); + if (std::get<2>(GetParam())) { + ASSERT_EQ(values[0].ToString(), "val1"); + ASSERT_EQ(values[1].ToString(), "val2"); + } else { + ASSERT_TRUE(statuses[0].IsCorruption()); + ASSERT_TRUE(statuses[1].IsCorruption()); + } } TEST_P(DBIOCorruptionTest, CompactionReadCorruptionRetry) { @@ -793,13 +815,18 @@ TEST_P(DBIOCorruptionTest, CompactionReadCorruptionRetry) { ASSERT_OK(Put("key2", "val2")); ASSERT_OK(Flush()); fs->SetCorruptionTrigger(1); - ASSERT_OK(dbfull()->CompactRange(CompactRangeOptions(), nullptr, nullptr)); - - std::string val; - ReadOptions ro; - ro.async_io = std::get<1>(GetParam()); - ASSERT_OK(dbfull()->Get(ro, "key1", &val)); - ASSERT_EQ(val, "val1"); + Status s = dbfull()->CompactRange(CompactRangeOptions(), nullptr, nullptr); + if (std::get<2>(GetParam())) { + ASSERT_OK(s); + + std::string val; + ReadOptions ro; + ro.async_io = std::get<1>(GetParam()); + ASSERT_OK(dbfull()->Get(ro, "key1", &val)); + ASSERT_EQ(val, "val1"); + } else { + ASSERT_TRUE(s.IsCorruption()); + } } TEST_P(DBIOCorruptionTest, FlushReadCorruptionRetry) { @@ -808,17 +835,44 @@ TEST_P(DBIOCorruptionTest, FlushReadCorruptionRetry) { ASSERT_OK(Put("key1", "val1")); fs->SetCorruptionTrigger(1); - ASSERT_OK(Flush()); + Status s = Flush(); + if (std::get<2>(GetParam())) { + ASSERT_OK(s); + + std::string val; + ReadOptions ro; + ro.async_io = std::get<1>(GetParam()); + ASSERT_OK(dbfull()->Get(ro, "key1", &val)); + ASSERT_EQ(val, "val1"); + } else { + ASSERT_NOK(s); + } +} - std::string val; - ReadOptions ro; - ro.async_io = std::get<1>(GetParam()); - ASSERT_OK(dbfull()->Get(ro, "key1", &val)); - ASSERT_EQ(val, "val1"); +TEST_P(DBIOCorruptionTest, ManifestCorruptionRetry) { + CorruptionFS* fs = + static_cast(env_guard_->GetFileSystem().get()); + + ASSERT_OK(Put("key1", "val1")); + ASSERT_OK(Flush()); + SyncPoint::GetInstance()->SetCallBack( + "VersionSet::Recover:StartManifestRead", + [&](void* /*arg*/) { fs->SetCorruptionTrigger(0); }); + SyncPoint::GetInstance()->EnableProcessing(); + + if (std::get<2>(GetParam())) { + ASSERT_OK(ReopenDB()); + } else { + ASSERT_EQ(ReopenDB(), Status::Corruption()); + } + SyncPoint::GetInstance()->DisableProcessing(); } +// The parameters are - 1. Use FS provided buffer, 2. Use async IO ReadOption, +// 3. Retry with verify_and_reconstruct_read IOOption INSTANTIATE_TEST_CASE_P(DBIOCorruptionTest, DBIOCorruptionTest, - testing::Combine(testing::Bool(), testing::Bool())); + testing::Combine(testing::Bool(), testing::Bool(), + testing::Bool())); } // namespace ROCKSDB_NAMESPACE int main(int argc, char** argv) { diff --git a/db/db_iter.cc b/db/db_iter.cc index 65e2fc1654b..63dabb9ca98 100644 --- a/db/db_iter.cc +++ b/db/db_iter.cc @@ -231,6 +231,7 @@ bool DBIter::SetValueAndColumnsFromEntity(Slice slice) { if (!s.ok()) { status_ = s; valid_ = false; + wide_columns_.clear(); return false; } diff --git a/db/db_iterator_test.cc b/db/db_iterator_test.cc index 7c3bdd850ff..8247333b0ba 100644 --- a/db/db_iterator_test.cc +++ b/db/db_iterator_test.cc @@ -3555,6 +3555,121 @@ TEST_F(DBIteratorTest, ErrorWhenReadFile) { iter->Reset(); } +TEST_F(DBIteratorTest, IteratorsConsistentViewImplicitSnapshot) { + Options options = GetDefaultOptions(); + CreateAndReopenWithCF({"cf_1", "cf_2"}, options); + + for (int i = 0; i < 3; ++i) { + ASSERT_OK(Put(i, "cf" + std::to_string(i) + "_key", + "cf" + std::to_string(i) + "_val")); + } + + ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->LoadDependency( + {{"DBImpl::BGWorkFlush:done", + "DBImpl::MultiCFSnapshot::BeforeCheckingSnapshot"}}); + + bool flushed = false; + ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack( + "DBImpl::MultiCFSnapshot::AfterRefSV", [&](void* /*arg*/) { + if (!flushed) { + for (int i = 0; i < 3; ++i) { + ASSERT_OK(Put(i, "cf" + std::to_string(i) + "_key", + "cf" + std::to_string(i) + "_val_new")); + } + // After SV is obtained for the first CF, flush for the second CF + ASSERT_OK(Flush(1)); + flushed = true; + } + }); + ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing(); + ReadOptions read_options; + std::vector iters; + ASSERT_OK(db_->NewIterators(read_options, handles_, &iters)); + + for (int i = 0; i < 3; ++i) { + auto iter = iters[i]; + ASSERT_OK(iter->status()); + iter->SeekToFirst(); + ASSERT_EQ(IterStatus(iter), "cf" + std::to_string(i) + "_key->cf" + + std::to_string(i) + "_val_new"); + } + for (auto* iter : iters) { + delete iter; + } + + // Thread-local SVs are no longer obsolete nor in use + for (int i = 0; i < 3; ++i) { + auto* cfd = + static_cast_with_check(handles_[i])->cfd(); + ASSERT_NE(cfd->TEST_GetLocalSV()->Get(), SuperVersion::kSVObsolete); + ASSERT_NE(cfd->TEST_GetLocalSV()->Get(), SuperVersion::kSVInUse); + } +} + +TEST_F(DBIteratorTest, IteratorsConsistentViewExplicitSnapshot) { + Options options = GetDefaultOptions(); + options.atomic_flush = true; + CreateAndReopenWithCF({"cf_1", "cf_2"}, options); + + for (int i = 0; i < 3; ++i) { + ASSERT_OK(Put(i, "cf" + std::to_string(i) + "_key", + "cf" + std::to_string(i) + "_val")); + } + + ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->LoadDependency( + {{"DBImpl::BGWorkFlush:done", + "DBImpl::MultiCFSnapshot::BeforeCheckingSnapshot"}}); + + bool flushed = false; + ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack( + "DBImpl::MultiCFSnapshot::AfterRefSV", [&](void* /*arg*/) { + if (!flushed) { + for (int i = 0; i < 3; ++i) { + ASSERT_OK(Put(i, "cf" + std::to_string(i) + "_key", + "cf" + std::to_string(i) + "_val_new")); + } + // After SV is obtained for the first CF, do the atomic flush() + ASSERT_OK(Flush()); + flushed = true; + } + }); + ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing(); + // Explicit snapshot wouldn't force reloading all svs. We should expect old + // values + const Snapshot* snapshot = db_->GetSnapshot(); + ReadOptions read_options; + read_options.snapshot = snapshot; + std::vector iters; + ASSERT_OK(db_->NewIterators(read_options, handles_, &iters)); + + for (int i = 0; i < 3; ++i) { + auto iter = iters[i]; + ASSERT_OK(iter->status()); + iter->SeekToFirst(); + ASSERT_EQ(IterStatus(iter), "cf" + std::to_string(i) + "_key->cf" + + std::to_string(i) + "_val"); + } + db_->ReleaseSnapshot(snapshot); + for (auto* iter : iters) { + delete iter; + } + + // Thread-local SV for cf_0 is obsolete (atomic flush happened after the first + // SV Ref) + auto* cfd0 = + static_cast_with_check(handles_[0])->cfd(); + ASSERT_EQ(cfd0->TEST_GetLocalSV()->Get(), SuperVersion::kSVObsolete); + ASSERT_NE(cfd0->TEST_GetLocalSV()->Get(), SuperVersion::kSVInUse); + + // Rest are not InUse nor Obsolete + for (int i = 1; i < 3; ++i) { + auto* cfd = + static_cast_with_check(handles_[i])->cfd(); + ASSERT_NE(cfd->TEST_GetLocalSV()->Get(), SuperVersion::kSVObsolete); + ASSERT_NE(cfd->TEST_GetLocalSV()->Get(), SuperVersion::kSVInUse); + } +} + } // namespace ROCKSDB_NAMESPACE int main(int argc, char** argv) { diff --git a/db/db_memtable_test.cc b/db/db_memtable_test.cc index 385ccb43c75..5c8b6db2ba3 100644 --- a/db/db_memtable_test.cc +++ b/db/db_memtable_test.cc @@ -313,6 +313,10 @@ TEST_F(DBMemTableTest, InsertWithHint) { ASSERT_EQ("foo_v3", Get("foo_k3")); ASSERT_EQ("bar_v1", Get("bar_k1")); ASSERT_EQ("bar_v2", Get("bar_k2")); + ASSERT_OK(db_->DeleteRange(WriteOptions(), "foo_k1", "foo_k4")); + ASSERT_EQ(hint_bar, rep->last_hint_in()); + ASSERT_EQ(hint_bar, rep->last_hint_out()); + ASSERT_EQ(5, rep->num_insert_with_hint()); ASSERT_EQ("vvv", Get("NotInPrefixDomain")); } diff --git a/db/db_range_del_test.cc b/db/db_range_del_test.cc index f92fa27aed5..c7c473e73cc 100644 --- a/db/db_range_del_test.cc +++ b/db/db_range_del_test.cc @@ -3806,6 +3806,21 @@ TEST_F(DBRangeDelTest, RefreshWithSnapshot) { iter.reset(); db_->ReleaseSnapshot(snapshot); } + +TEST_F(DBRangeDelTest, RowCache) { + Options options = CurrentOptions(); + options.row_cache = NewLRUCache(8 << 10); + DestroyAndReopen(options); + ASSERT_OK(Put(Key(3), "val")); + ASSERT_TRUE(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), + Key(3), Key(5)) + .IsNotSupported()); + WriteBatch wb; + ASSERT_OK(wb.Put(Key(6), "abc")); + ASSERT_OK(wb.DeleteRange(Key(1), Key(5))); + ASSERT_TRUE(db_->Write(WriteOptions(), &wb).IsNotSupported()); + ASSERT_EQ(Get(Key(3)), "val"); +} } // namespace ROCKSDB_NAMESPACE int main(int argc, char** argv) { diff --git a/db/db_sst_test.cc b/db/db_sst_test.cc index d7eee829bee..57cde407342 100644 --- a/db/db_sst_test.cc +++ b/db/db_sst_test.cc @@ -21,7 +21,8 @@ namespace ROCKSDB_NAMESPACE { class DBSSTTest : public DBTestBase { public: - DBSSTTest() : DBTestBase("db_sst_test", /*env_do_fsync=*/true) {} + DBSSTTest(const std::string& test_name = "db_sst_test") + : DBTestBase(test_name, /*env_do_fsync=*/true) {} }; // A class which remembers the name of each flushed file. @@ -937,15 +938,23 @@ INSTANTIATE_TEST_CASE_P(DBWALTestWithParam, DBWALTestWithParam, ::testing::Values(std::make_tuple("", true), std::make_tuple("_wal_dir", false))); -TEST_F(DBSSTTest, OpenDBWithExistingTrashAndObsoleteSstFile) { +// Test param: max_trash_db_ratio for DeleteScheduler +class DBObsoleteFileDeletionOnOpenTest + : public DBSSTTest, + public ::testing::WithParamInterface { + public: + explicit DBObsoleteFileDeletionOnOpenTest() + : DBSSTTest("db_sst_deletion_on_open_test") {} +}; + +TEST_P(DBObsoleteFileDeletionOnOpenTest, Basic) { Options options = CurrentOptions(); options.sst_file_manager.reset( NewSstFileManager(env_, nullptr, "", 1024 * 1024 /* 1 MB/sec */)); auto sfm = static_cast(options.sst_file_manager.get()); - // Set an extra high trash ratio to prevent immediate/non-rate limited - // deletions sfm->SetDeleteRateBytesPerSecond(1024 * 1024); - sfm->delete_scheduler()->SetMaxTrashDBRatio(1000.0); + double max_trash_db_ratio = GetParam(); + sfm->delete_scheduler()->SetMaxTrashDBRatio(max_trash_db_ratio); int bg_delete_file = 0; ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack( @@ -965,10 +974,19 @@ TEST_F(DBSSTTest, OpenDBWithExistingTrashAndObsoleteSstFile) { WriteStringToFile(env_, "abc", dbname_ + "/" + "003.sst.trash", false)); // Manually add an obsolete sst file. Obsolete SST files are discovered and // deleted upon recovery. - constexpr uint64_t kSstFileNumber = 100; - const std::string kObsoleteSstFile = - MakeTableFileName(dbname_, kSstFileNumber); - ASSERT_OK(WriteStringToFile(env_, "abc", kObsoleteSstFile, false)); + uint64_t sst_file_number = 100; + const std::string kObsoleteSstFileOne = + MakeTableFileName(dbname_, sst_file_number); + ASSERT_OK(WriteStringToFile(env_, "abc", kObsoleteSstFileOne, false)); + // The slow deletion on recovery had a bug before where a file's size is not + // first tracked in `total_size_` in SstFileManager before passed to + // DeleteScheduler. The first obsolete file is still slow deleted because + // 0 (total_trash_size_) > 0 (total_size_) * 1000 (max_trash_db_ratio) + // is always false. + // Here we explicitly create a second obsolete file to verify this bug's fix + const std::string kObsoleteSstFileTwo = + MakeTableFileName(dbname_, sst_file_number - 1); + ASSERT_OK(WriteStringToFile(env_, "abc", kObsoleteSstFileTwo, false)); // Reopen the DB and verify that it deletes existing trash files and obsolete // SST files with rate limiting. @@ -977,10 +995,26 @@ TEST_F(DBSSTTest, OpenDBWithExistingTrashAndObsoleteSstFile) { ASSERT_NOK(env_->FileExists(dbname_ + "/" + "001.sst.trash")); ASSERT_NOK(env_->FileExists(dbname_ + "/" + "002.sst.trash")); ASSERT_NOK(env_->FileExists(dbname_ + "/" + "003.sst.trash")); - ASSERT_NOK(env_->FileExists(kObsoleteSstFile)); - ASSERT_EQ(bg_delete_file, 4); + ASSERT_NOK(env_->FileExists(kObsoleteSstFileOne)); + ASSERT_NOK(env_->FileExists(kObsoleteSstFileTwo)); + // The files in the DB's directory are all either trash or obsolete sst files. + // So the trash/db ratio is 1. A ratio equal to or higher than 1 should + // schedule all files' deletion in background. A ratio lower than 1 may + // send some files to be deleted immediately. + if (max_trash_db_ratio < 1) { + ASSERT_LE(bg_delete_file, 5); + } else { + ASSERT_EQ(bg_delete_file, 5); + } + + ASSERT_EQ(sfm->GetTotalSize(), 0); + ASSERT_EQ(sfm->delete_scheduler()->GetTotalTrashSize(), 0); } +INSTANTIATE_TEST_CASE_P(DBObsoleteFileDeletionOnOpenTest, + DBObsoleteFileDeletionOnOpenTest, + ::testing::Values(0, 0.5, 1, 1.2)); + // Create a DB with 2 db_paths, and generate multiple files in the 2 // db_paths using CompactRangeOptions, make sure that files that were // deleted from first db_path were deleted using DeleteScheduler and diff --git a/db/db_test.cc b/db/db_test.cc index 3d514e39ad2..c3ec250940e 100644 --- a/db/db_test.cc +++ b/db/db_test.cc @@ -26,6 +26,7 @@ #endif #include "cache/lru_cache.h" +#include "db/attribute_group_iterator_impl.h" #include "db/blob/blob_index.h" #include "db/blob/blob_log_format.h" #include "db/db_impl/db_impl.h" @@ -3199,11 +3200,18 @@ class ModelDB : public DB { return Status::NotSupported("Not supported yet"); } - // UNDER CONSTRUCTION - DO NOT USE - std::unique_ptr NewMultiCfIterator( + std::unique_ptr NewCoalescingIterator( const ReadOptions& /*options*/, const std::vector& /*column_families*/) override { - return nullptr; + return std::unique_ptr( + NewErrorIterator(Status::NotSupported("Not supported yet"))); + } + + std::unique_ptr NewAttributeGroupIterator( + const ReadOptions& /*options*/, + const std::vector& /*column_families*/) override { + return NewAttributeGroupErrorIterator( + Status::NotSupported("Not supported yet")); } const Snapshot* GetSnapshot() override { diff --git a/db/db_test_util.cc b/db/db_test_util.cc index cbc394b0426..88df4257634 100644 --- a/db/db_test_util.cc +++ b/db/db_test_util.cc @@ -154,6 +154,9 @@ bool DBTestBase::ShouldSkipOptions(int option_config, int skip_mask) { if ((skip_mask & kSkipMmapReads) && option_config == kWalDirAndMmapReads) { return true; } + if ((skip_mask & kSkipRowCache) && option_config == kRowCache) { + return true; + } return false; } @@ -766,7 +769,9 @@ Status DBTestBase::TimedPut(const Slice& k, const Slice& v, Status DBTestBase::TimedPut(int cf, const Slice& k, const Slice& v, uint64_t write_unix_time, WriteOptions wo) { - WriteBatch wb; + WriteBatch wb(/*reserved_bytes=*/0, /*max_bytes=*/0, + wo.protection_bytes_per_key, + /*default_cf_ts_sz=*/0); ColumnFamilyHandle* cfh; if (cf != 0) { cfh = handles_[cf]; diff --git a/db/db_test_util.h b/db/db_test_util.h index 775c161d36b..dce96467bff 100644 --- a/db/db_test_util.h +++ b/db/db_test_util.h @@ -1071,6 +1071,7 @@ class DBTestBase : public testing::Test { kSkipNoSeekToLast = 32, kSkipFIFOCompaction = 128, kSkipMmapReads = 256, + kSkipRowCache = 512, }; const int kRangeDelSkipConfigs = @@ -1078,7 +1079,9 @@ class DBTestBase : public testing::Test { kSkipPlainTable | // MmapReads disables the iterator pinning that RangeDelAggregator // requires. - kSkipMmapReads; + kSkipMmapReads | + // Not compatible yet. + kSkipRowCache; // `env_do_fsync` decides whether the special Env would do real // fsync for files and directories. Skipping fsync can speed up diff --git a/db/db_wal_test.cc b/db/db_wal_test.cc index 91070e298b6..1ff16ee9a4f 100644 --- a/db/db_wal_test.cc +++ b/db/db_wal_test.cc @@ -814,6 +814,7 @@ TEST_F(DBWALTest, WALWithChecksumHandoff) { writeOpt.disableWAL = false; // Data is persisted in the WAL ASSERT_OK(dbfull()->Put(writeOpt, handles_[1], "zoo", "v3")); + ASSERT_OK(dbfull()->SyncWAL()); // The hash does not match, write fails fault_fs->SetChecksumHandoffFuncType(ChecksumType::kxxHash); writeOpt.disableWAL = false; diff --git a/db/db_write_test.cc b/db/db_write_test.cc index f464a3036b8..e074dd6b42f 100644 --- a/db/db_write_test.cc +++ b/db/db_write_test.cc @@ -795,7 +795,7 @@ TEST_P(DBWriteTest, ConcurrentlyDisabledWAL) { std::thread threads[10]; for (int t = 0; t < 10; t++) { threads[t] = std::thread([t, wal_key_prefix, wal_value, no_wal_key_prefix, - no_wal_value, this] { + no_wal_value, &options, this] { for (int i = 0; i < 10; i++) { ROCKSDB_NAMESPACE::WriteOptions write_option_disable; write_option_disable.disableWAL = true; @@ -806,7 +806,10 @@ TEST_P(DBWriteTest, ConcurrentlyDisabledWAL) { std::string wal_key = wal_key_prefix + std::to_string(i) + "_" + std::to_string(i); ASSERT_OK(this->Put(wal_key, wal_value, write_option_default)); - ASSERT_OK(dbfull()->SyncWAL()); + ASSERT_OK(dbfull()->SyncWAL()) + << "options.env: " << options.env << ", env_: " << env_ + << ", env_->is_wal_sync_thread_safe_: " + << env_->is_wal_sync_thread_safe_.load(); } return; }); @@ -910,6 +913,32 @@ TEST_P(DBWriteTest, RecycleLogTestCFAheadOfWAL) { Status::Corruption()); } +TEST_P(DBWriteTest, RecycleLogToggleTest) { + Options options = GetOptions(); + options.recycle_log_file_num = 0; + options.avoid_flush_during_recovery = true; + options.wal_recovery_mode = WALRecoveryMode::kPointInTimeRecovery; + + Destroy(options); + Reopen(options); + // After opening, a new log gets created, say 1.log + ASSERT_OK(Put(Key(1), "val1")); + + options.recycle_log_file_num = 1; + Reopen(options); + // 1.log is added to alive_log_files_ + ASSERT_OK(Put(Key(2), "val1")); + ASSERT_OK(Flush()); + // 1.log should be deleted and not recycled, since it + // was created by the previous Reopen + ASSERT_OK(Put(Key(1), "val2")); + ASSERT_OK(Flush()); + + options.recycle_log_file_num = 1; + Reopen(options); + ASSERT_EQ(Get(Key(1)), "val2"); +} + INSTANTIATE_TEST_CASE_P(DBWriteTestInstance, DBWriteTest, testing::Values(DBTestBase::kDefault, DBTestBase::kConcurrentWALWrites, diff --git a/db/deletefile_test.cc b/db/deletefile_test.cc index 7a86e913936..2550ef5ec60 100644 --- a/db/deletefile_test.cc +++ b/db/deletefile_test.cc @@ -223,6 +223,49 @@ TEST_F(DeleteFileTest, PurgeObsoleteFilesTest) { CheckFileTypeCounts(dbname_, 0, 1, 1); } +TEST_F(DeleteFileTest, WaitForCompactWithWaitForPurgeOptionTest) { + Options options = CurrentOptions(); + SetOptions(&options); + Destroy(options); + options.create_if_missing = true; + Reopen(options); + + std::string first("0"), last("999999"); + CompactRangeOptions compact_options; + compact_options.change_level = true; + compact_options.target_level = 2; + Slice first_slice(first), last_slice(last); + + CreateTwoLevels(); + Iterator* itr = nullptr; + ReadOptions read_options; + read_options.background_purge_on_iterator_cleanup = true; + itr = db_->NewIterator(read_options); + ASSERT_OK(itr->status()); + ASSERT_OK(db_->CompactRange(compact_options, &first_slice, &last_slice)); + SyncPoint::GetInstance()->LoadDependency( + {{"DBImpl::BGWorkPurge:start", "DeleteFileTest::WaitForPurgeTest"}, + {"DBImpl::WaitForCompact:InsideLoop", + "DBImpl::BackgroundCallPurge:beforeMutexLock"}}); + SyncPoint::GetInstance()->EnableProcessing(); + + delete itr; + + TEST_SYNC_POINT("DeleteFileTest::WaitForPurgeTest"); + // At this point, purge got started, but can't finish due to sync points + // not purged yet + CheckFileTypeCounts(dbname_, 0, 3, 1); + + // The sync point in WaitForCompact should unblock the purge + WaitForCompactOptions wait_for_compact_options; + wait_for_compact_options.wait_for_purge = true; + Status s = dbfull()->WaitForCompact(wait_for_compact_options); + ASSERT_OK(s); + + // Now files should be purged + CheckFileTypeCounts(dbname_, 0, 1, 1); +} + TEST_F(DeleteFileTest, BackgroundPurgeIteratorTest) { Options options = CurrentOptions(); SetOptions(&options); @@ -600,4 +643,3 @@ int main(int argc, char** argv) { RegisterCustomObjects(argc, argv); return RUN_ALL_TESTS(); } - diff --git a/db/error_handler_fs_test.cc b/db/error_handler_fs_test.cc index 2d33a7a694f..ce351e3695d 100644 --- a/db/error_handler_fs_test.cc +++ b/db/error_handler_fs_test.cc @@ -1238,6 +1238,7 @@ TEST_F(DBErrorHandlingFSTest, AutoRecoverFlushError) { ERROR_HANDLER_AUTORESUME_RETRY_TOTAL_COUNT)); ASSERT_EQ(0, options.statistics->getAndResetTickerCount( ERROR_HANDLER_AUTORESUME_SUCCESS_COUNT)); + ASSERT_OK(dbfull()->SyncWAL()); Reopen(options); ASSERT_EQ("val", Get(Key(0))); diff --git a/db/fault_injection_test.cc b/db/fault_injection_test.cc index 17b4c034283..7b93326ca8b 100644 --- a/db/fault_injection_test.cc +++ b/db/fault_injection_test.cc @@ -578,8 +578,10 @@ TEST_P(FaultInjectionTest, NoDuplicateTrailingEntries) { fault_fs->DisableWriteErrorInjection(); - // Closing the log writer will cause WritableFileWriter::Close() and flush - // remaining data from its buffer to underlying file. + // Flush remaining data from its buffer to underlying file. + ASSERT_OK(log_writer->file()->writable_file()->Sync(IOOptions(), + nullptr /* dbg */)); + // Closing the log writer will cause WritableFileWriter::Close() log_writer.reset(); { diff --git a/db/import_column_family_job.cc b/db/import_column_family_job.cc index 4d5c65616e1..43c32e39b4f 100644 --- a/db/import_column_family_job.cc +++ b/db/import_column_family_job.cc @@ -65,11 +65,11 @@ Status ImportColumnFamilyJob::Prepare(uint64_t next_file_number, largest = file_to_import.largest_internal_key; } else { if (cfd_->internal_comparator().Compare( - smallest, file_to_import.smallest_internal_key) < 0) { + smallest, file_to_import.smallest_internal_key) > 0) { smallest = file_to_import.smallest_internal_key; } if (cfd_->internal_comparator().Compare( - largest, file_to_import.largest_internal_key) > 0) { + largest, file_to_import.largest_internal_key) < 0) { largest = file_to_import.largest_internal_key; } } diff --git a/db/import_column_family_test.cc b/db/import_column_family_test.cc index 89586bcd18e..e3a36073f42 100644 --- a/db/import_column_family_test.cc +++ b/db/import_column_family_test.cc @@ -881,6 +881,71 @@ TEST_F(ImportColumnFamilyTest, ImportMultiColumnFamilyWithOverlap) { delete db_copy; ASSERT_OK(DestroyDir(env_, dbname_ + "/db_copy")); } + +TEST_F(ImportColumnFamilyTest, ImportMultiColumnFamilySeveralFilesWithOverlap) { + Options options = CurrentOptions(); + CreateAndReopenWithCF({"koko"}, options); + + SstFileWriter sfw_cf1(EnvOptions(), options, handles_[1]); + const std::string file1_sst_name = "file1.sst"; + const std::string file1_sst = sst_files_dir_ + file1_sst_name; + ASSERT_OK(sfw_cf1.Open(file1_sst)); + ASSERT_OK(sfw_cf1.Put("K1", "V1")); + ASSERT_OK(sfw_cf1.Put("K2", "V2")); + ASSERT_OK(sfw_cf1.Finish()); + + SstFileWriter sfw_cf2(EnvOptions(), options, handles_[1]); + const std::string file2_sst_name = "file2.sst"; + const std::string file2_sst = sst_files_dir_ + file2_sst_name; + ASSERT_OK(sfw_cf2.Open(file2_sst)); + ASSERT_OK(sfw_cf2.Put("K2", "V2")); + ASSERT_OK(sfw_cf2.Put("K3", "V3")); + ASSERT_OK(sfw_cf2.Finish()); + + ColumnFamilyHandle* second_cfh = nullptr; + ASSERT_OK(db_->CreateColumnFamily(options, "toto", &second_cfh)); + + SstFileWriter sfw_cf3(EnvOptions(), options, second_cfh); + const std::string file3_sst_name = "file3.sst"; + const std::string file3_sst = sst_files_dir_ + file3_sst_name; + ASSERT_OK(sfw_cf3.Open(file3_sst)); + ASSERT_OK(sfw_cf3.Put("K3", "V3")); + ASSERT_OK(sfw_cf3.Put("K4", "V4")); + ASSERT_OK(sfw_cf3.Finish()); + + SstFileWriter sfw_cf4(EnvOptions(), options, second_cfh); + const std::string file4_sst_name = "file4.sst"; + const std::string file4_sst = sst_files_dir_ + file4_sst_name; + ASSERT_OK(sfw_cf4.Open(file4_sst)); + ASSERT_OK(sfw_cf4.Put("K4", "V4")); + ASSERT_OK(sfw_cf4.Put("K5", "V5")); + ASSERT_OK(sfw_cf4.Finish()); + + ExportImportFilesMetaData metadata1, metadata2; + metadata1.files.push_back( + LiveFileMetaDataInit(file1_sst_name, sst_files_dir_, 1, 1, 2)); + metadata1.files.push_back( + LiveFileMetaDataInit(file2_sst_name, sst_files_dir_, 1, 3, 4)); + metadata1.db_comparator_name = options.comparator->Name(); + metadata2.files.push_back( + LiveFileMetaDataInit(file3_sst_name, sst_files_dir_, 1, 1, 2)); + metadata2.files.push_back( + LiveFileMetaDataInit(file4_sst_name, sst_files_dir_, 1, 3, 4)); + metadata2.db_comparator_name = options.comparator->Name(); + + std::vector metadatas{&metadata1, + &metadata2}; + + ASSERT_EQ(db_->CreateColumnFamilyWithImport(ColumnFamilyOptions(), "yoyo", + ImportColumnFamilyOptions(), + metadatas, &import_cfh_), + Status::InvalidArgument("CFs have overlapping ranges")); + ASSERT_EQ(import_cfh_, nullptr); + + ASSERT_OK(db_->DropColumnFamily(second_cfh)); + ASSERT_OK(db_->DestroyColumnFamilyHandle(second_cfh)); +} + } // namespace ROCKSDB_NAMESPACE int main(int argc, char** argv) { diff --git a/db/log_reader.cc b/db/log_reader.cc index da979a1ee1e..110eb2c27c8 100644 --- a/db/log_reader.cc +++ b/db/log_reader.cc @@ -232,7 +232,9 @@ bool Reader::ReadRecord(Slice* record, std::string* scratch, // produce a hole in the recovered data. Report an error here, which // higher layers can choose to ignore when it's provable there is no // hole. - ReportCorruption(scratch->size(), "error reading trailing data"); + ReportCorruption( + scratch->size(), + "error reading trailing data due to encountering EOF"); } // This can be caused by the writer dying immediately after // writing a physical record but before completing the next; don't @@ -252,7 +254,9 @@ bool Reader::ReadRecord(Slice* record, std::string* scratch, // produce a hole in the recovered data. Report an error here, // which higher layers can choose to ignore when it's provable // there is no hole. - ReportCorruption(scratch->size(), "error reading trailing data"); + ReportCorruption( + scratch->size(), + "error reading trailing data due to encountering old record"); } // This can be caused by the writer dying immediately after // writing a physical record but before completing the next; don't diff --git a/db/memtable.cc b/db/memtable.cc index ba4f0da824a..c3a6433dc14 100644 --- a/db/memtable.cc +++ b/db/memtable.cc @@ -765,8 +765,9 @@ Status MemTable::Add(SequenceNumber s, ValueType type, Slice key_without_ts = StripTimestampFromUserKey(key, ts_sz_); if (!allow_concurrent) { - // Extract prefix for insert with hint. - if (insert_with_hint_prefix_extractor_ != nullptr && + // Extract prefix for insert with hint. Hints are for point key table + // (`table_`) only, not `range_del_table_`. + if (table == table_ && insert_with_hint_prefix_extractor_ != nullptr && insert_with_hint_prefix_extractor_->InDomain(key_slice)) { Slice prefix = insert_with_hint_prefix_extractor_->Transform(key_slice); bool res = table->InsertKeyWithHint(handle, &insert_hints_[prefix]); @@ -1621,7 +1622,8 @@ Status MemTable::UpdateCallback(SequenceNumber seq, const Slice& key, return Status::NotFound(); } -size_t MemTable::CountSuccessiveMergeEntries(const LookupKey& key) { +size_t MemTable::CountSuccessiveMergeEntries(const LookupKey& key, + size_t limit) { Slice memkey = key.memtable_key(); // A total ordered iterator is costly for some memtablerep (prefix aware @@ -1633,7 +1635,7 @@ size_t MemTable::CountSuccessiveMergeEntries(const LookupKey& key) { size_t num_successive_merges = 0; - for (; iter->Valid(); iter->Next()) { + for (; iter->Valid() && num_successive_merges < limit; iter->Next()) { const char* entry = iter->key(); uint32_t key_length = 0; const char* iter_key_ptr = GetVarint32Ptr(entry, entry + 5, &key_length); diff --git a/db/memtable.h b/db/memtable.h index 730258f05c8..11ab6ea41b8 100644 --- a/db/memtable.h +++ b/db/memtable.h @@ -326,9 +326,10 @@ class MemTable { const ProtectionInfoKVOS64* kv_prot_info); // Returns the number of successive merge entries starting from the newest - // entry for the key up to the last non-merge entry or last entry for the - // key in the memtable. - size_t CountSuccessiveMergeEntries(const LookupKey& key); + // entry for the key. The count ends when the oldest entry in the memtable + // with which the newest entry would be merged is reached, or the count + // reaches `limit`. + size_t CountSuccessiveMergeEntries(const LookupKey& key, size_t limit); // Update counters and flush status after inserting a whole write batch // Used in concurrent memtable inserts. diff --git a/db/multi_cf_iterator.cc b/db/multi_cf_iterator.cc deleted file mode 100644 index 80e4171d54d..00000000000 --- a/db/multi_cf_iterator.cc +++ /dev/null @@ -1,102 +0,0 @@ -// Copyright (c) Meta Platforms, Inc. and affiliates. -// 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/multi_cf_iterator.h" - -#include - -namespace ROCKSDB_NAMESPACE { - -template -void MultiCfIterator::SeekCommon(BinaryHeap& heap, - ChildSeekFuncType child_seek_func) { - heap.clear(); - int i = 0; - for (auto& cfh_iter_pair : cfh_iter_pairs_) { - auto& cfh = cfh_iter_pair.first; - auto& iter = cfh_iter_pair.second; - child_seek_func(iter.get()); - if (iter->Valid()) { - assert(iter->status().ok()); - heap.push(MultiCfIteratorInfo{iter.get(), cfh, i}); - } else { - considerStatus(iter->status()); - } - ++i; - } -} - -template -void MultiCfIterator::AdvanceIterator(BinaryHeap& heap, - AdvanceFuncType advance_func) { - // 1. Keep the top iterator (by popping it from the heap) - // 2. Make sure all others have iterated past the top iterator key slice - // 3. Advance the top iterator, and add it back to the heap if valid - auto top = heap.top(); - heap.pop(); - if (!heap.empty()) { - auto* current = heap.top().iterator; - while (current->Valid() && - comparator_->Compare(top.iterator->key(), current->key()) == 0) { - assert(current->status().ok()); - advance_func(current); - if (current->Valid()) { - heap.replace_top(heap.top()); - } else { - considerStatus(current->status()); - heap.pop(); - } - if (!heap.empty()) { - current = heap.top().iterator; - } - } - } - advance_func(top.iterator); - if (top.iterator->Valid()) { - assert(top.iterator->status().ok()); - heap.push(top); - } else { - considerStatus(top.iterator->status()); - } -} - -void MultiCfIterator::SeekToFirst() { - auto& min_heap = GetHeap([this]() { InitMinHeap(); }); - SeekCommon(min_heap, [](Iterator* iter) { iter->SeekToFirst(); }); -} -void MultiCfIterator::Seek(const Slice& target) { - auto& min_heap = GetHeap([this]() { InitMinHeap(); }); - SeekCommon(min_heap, [&target](Iterator* iter) { iter->Seek(target); }); -} -void MultiCfIterator::SeekToLast() { - auto& max_heap = GetHeap([this]() { InitMaxHeap(); }); - SeekCommon(max_heap, [](Iterator* iter) { iter->SeekToLast(); }); -} -void MultiCfIterator::SeekForPrev(const Slice& target) { - auto& max_heap = GetHeap([this]() { InitMaxHeap(); }); - SeekCommon(max_heap, - [&target](Iterator* iter) { iter->SeekForPrev(target); }); -} - -void MultiCfIterator::Next() { - assert(Valid()); - auto& min_heap = GetHeap([this]() { - Slice target = key(); - InitMinHeap(); - Seek(target); - }); - AdvanceIterator(min_heap, [](Iterator* iter) { iter->Next(); }); -} -void MultiCfIterator::Prev() { - assert(Valid()); - auto& max_heap = GetHeap([this]() { - Slice target = key(); - InitMaxHeap(); - SeekForPrev(target); - }); - AdvanceIterator(max_heap, [](Iterator* iter) { iter->Prev(); }); -} - -} // namespace ROCKSDB_NAMESPACE diff --git a/db/multi_cf_iterator.h b/db/multi_cf_iterator.h deleted file mode 100644 index cdd09c16df0..00000000000 --- a/db/multi_cf_iterator.h +++ /dev/null @@ -1,159 +0,0 @@ -// Copyright (c) Meta Platforms, Inc. and affiliates. -// 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 - -#include "rocksdb/comparator.h" -#include "rocksdb/iterator.h" -#include "rocksdb/options.h" -#include "util/heap.h" -#include "util/overload.h" - -namespace ROCKSDB_NAMESPACE { - -// UNDER CONSTRUCTION - DO NOT USE -// A cross-column-family iterator from a consistent database state. -// When the same key exists in more than one column families, the iterator -// selects the value from the first column family containing the key, in the -// order provided in the `column_families` parameter. -class MultiCfIterator : public Iterator { - public: - MultiCfIterator(const Comparator* comparator, - const std::vector& column_families, - const std::vector& child_iterators) - : comparator_(comparator), - heap_(MultiCfMinHeap( - MultiCfHeapItemComparator>(comparator_))) { - assert(column_families.size() > 0 && - column_families.size() == child_iterators.size()); - cfh_iter_pairs_.reserve(column_families.size()); - for (size_t i = 0; i < column_families.size(); ++i) { - cfh_iter_pairs_.emplace_back( - column_families[i], std::unique_ptr(child_iterators[i])); - } - } - ~MultiCfIterator() override { status_.PermitUncheckedError(); } - - // No copy allowed - MultiCfIterator(const MultiCfIterator&) = delete; - MultiCfIterator& operator=(const MultiCfIterator&) = delete; - - private: - std::vector>> - cfh_iter_pairs_; - ReadOptions read_options_; - Status status_; - - AttributeGroups attribute_groups_; - - struct MultiCfIteratorInfo { - Iterator* iterator; - ColumnFamilyHandle* cfh; - int order; - }; - - template - class MultiCfHeapItemComparator { - public: - explicit MultiCfHeapItemComparator(const Comparator* comparator) - : comparator_(comparator) {} - bool operator()(const MultiCfIteratorInfo& a, - const MultiCfIteratorInfo& b) const { - assert(a.iterator); - assert(b.iterator); - assert(a.iterator->Valid()); - assert(b.iterator->Valid()); - int c = comparator_->Compare(a.iterator->key(), b.iterator->key()); - assert(c != 0 || a.order != b.order); - return c == 0 ? a.order - b.order > 0 : CompareOp()(c, 0); - } - - private: - const Comparator* comparator_; - }; - const Comparator* comparator_; - using MultiCfMinHeap = - BinaryHeap>>; - using MultiCfMaxHeap = BinaryHeap>>; - - using MultiCfIterHeap = std::variant; - - MultiCfIterHeap heap_; - - // TODO: Lower and Upper bounds - - Iterator* current() const { - if (std::holds_alternative(heap_)) { - auto& max_heap = std::get(heap_); - return max_heap.top().iterator; - } - auto& min_heap = std::get(heap_); - return min_heap.top().iterator; - } - - Slice key() const override { - assert(Valid()); - return current()->key(); - } - Slice value() const override { - assert(Valid()); - return current()->value(); - } - const WideColumns& columns() const override { - assert(Valid()); - return current()->columns(); - } - - bool Valid() const override { - if (std::holds_alternative(heap_)) { - auto& max_heap = std::get(heap_); - return !max_heap.empty() && status_.ok(); - } - auto& min_heap = std::get(heap_); - return !min_heap.empty() && status_.ok(); - } - - Status status() const override { return status_; } - void considerStatus(Status s) { - if (!s.ok() && status_.ok()) { - status_ = std::move(s); - } - } - - template - HeapType& GetHeap(InitFunc initFunc) { - if (!std::holds_alternative(heap_)) { - initFunc(); - } - return std::get(heap_); - } - - void InitMinHeap() { - heap_.emplace( - MultiCfHeapItemComparator>(comparator_)); - } - void InitMaxHeap() { - heap_.emplace( - MultiCfHeapItemComparator>(comparator_)); - } - - template - void SeekCommon(BinaryHeap& heap, ChildSeekFuncType child_seek_func); - template - void AdvanceIterator(BinaryHeap& heap, AdvanceFuncType advance_func); - - void SeekToFirst() override; - void SeekToLast() override; - void Seek(const Slice& /*target*/) override; - void SeekForPrev(const Slice& /*target*/) override; - void Next() override; - void Prev() override; -}; - -} // namespace ROCKSDB_NAMESPACE diff --git a/db/multi_cf_iterator_impl.h b/db/multi_cf_iterator_impl.h new file mode 100644 index 00000000000..af9084c9812 --- /dev/null +++ b/db/multi_cf_iterator_impl.h @@ -0,0 +1,289 @@ +// Copyright (c) Meta Platforms, Inc. and affiliates. +// 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 +#include + +#include "rocksdb/comparator.h" +#include "rocksdb/iterator.h" +#include "rocksdb/options.h" +#include "util/heap.h" + +namespace ROCKSDB_NAMESPACE { + +struct MultiCfIteratorInfo { + ColumnFamilyHandle* cfh; + Iterator* iterator; + int order; +}; + +class MultiCfIteratorImpl { + public: + MultiCfIteratorImpl( + const Comparator* comparator, + const std::vector& column_families, + const std::vector& child_iterators, + std::function reset_func, + std::function&)> populate_func) + : comparator_(comparator), + heap_(MultiCfMinHeap( + MultiCfHeapItemComparator>(comparator_))), + reset_func_(std::move(reset_func)), + populate_func_(std::move(populate_func)) { + assert(column_families.size() > 0 && + column_families.size() == child_iterators.size()); + cfh_iter_pairs_.reserve(column_families.size()); + for (size_t i = 0; i < column_families.size(); ++i) { + cfh_iter_pairs_.emplace_back( + column_families[i], std::unique_ptr(child_iterators[i])); + } + } + ~MultiCfIteratorImpl() { status_.PermitUncheckedError(); } + + // No copy allowed + MultiCfIteratorImpl(const MultiCfIteratorImpl&) = delete; + MultiCfIteratorImpl& operator=(const MultiCfIteratorImpl&) = delete; + + Slice key() const { + assert(Valid()); + return current()->key(); + } + + bool Valid() const { + if (std::holds_alternative(heap_)) { + auto& max_heap = std::get(heap_); + return !max_heap.empty() && status_.ok(); + } + auto& min_heap = std::get(heap_); + return !min_heap.empty() && status_.ok(); + } + + Status status() const { return status_; } + + void SeekToFirst() { + auto& min_heap = GetHeap([this]() { InitMinHeap(); }); + SeekCommon(min_heap, [](Iterator* iter) { iter->SeekToFirst(); }); + } + void Seek(const Slice& target) { + auto& min_heap = GetHeap([this]() { InitMinHeap(); }); + SeekCommon(min_heap, [&target](Iterator* iter) { iter->Seek(target); }); + } + void SeekToLast() { + auto& max_heap = GetHeap([this]() { InitMaxHeap(); }); + SeekCommon(max_heap, [](Iterator* iter) { iter->SeekToLast(); }); + } + void SeekForPrev(const Slice& target) { + auto& max_heap = GetHeap([this]() { InitMaxHeap(); }); + SeekCommon(max_heap, + [&target](Iterator* iter) { iter->SeekForPrev(target); }); + } + + void Next() { + assert(Valid()); + auto& min_heap = GetHeap([this]() { + Slice target = key(); + InitMinHeap(); + Seek(target); + }); + AdvanceIterator(min_heap, [](Iterator* iter) { iter->Next(); }); + } + void Prev() { + assert(Valid()); + auto& max_heap = GetHeap([this]() { + Slice target = key(); + InitMaxHeap(); + SeekForPrev(target); + }); + AdvanceIterator(max_heap, [](Iterator* iter) { iter->Prev(); }); + } + + private: + std::vector>> + cfh_iter_pairs_; + Status status_; + + template + class MultiCfHeapItemComparator { + public: + explicit MultiCfHeapItemComparator(const Comparator* comparator) + : comparator_(comparator) {} + bool operator()(const MultiCfIteratorInfo& a, + const MultiCfIteratorInfo& b) const { + assert(a.iterator); + assert(b.iterator); + assert(a.iterator->Valid()); + assert(b.iterator->Valid()); + int c = comparator_->Compare(a.iterator->key(), b.iterator->key()); + assert(c != 0 || a.order != b.order); + return c == 0 ? a.order - b.order > 0 : CompareOp()(c, 0); + } + + private: + const Comparator* comparator_; + }; + const Comparator* comparator_; + using MultiCfMinHeap = + BinaryHeap>>; + using MultiCfMaxHeap = BinaryHeap>>; + + using MultiCfIterHeap = std::variant; + + MultiCfIterHeap heap_; + + std::function reset_func_; + std::function)> populate_func_; + + Iterator* current() const { + if (std::holds_alternative(heap_)) { + auto& max_heap = std::get(heap_); + return max_heap.top().iterator; + } + auto& min_heap = std::get(heap_); + return min_heap.top().iterator; + } + + void considerStatus(Status s) { + if (!s.ok() && status_.ok()) { + status_ = std::move(s); + } + } + + template + HeapType& GetHeap(InitFunc initFunc) { + if (!std::holds_alternative(heap_)) { + initFunc(); + } + return std::get(heap_); + } + + void InitMinHeap() { + heap_.emplace( + MultiCfHeapItemComparator>(comparator_)); + } + void InitMaxHeap() { + heap_.emplace( + MultiCfHeapItemComparator>(comparator_)); + } + + template + void SeekCommon(BinaryHeap& heap, ChildSeekFuncType child_seek_func) { + reset_func_(); + heap.clear(); + int i = 0; + for (auto& [cfh, iter] : cfh_iter_pairs_) { + child_seek_func(iter.get()); + if (iter->Valid()) { + assert(iter->status().ok()); + heap.push(MultiCfIteratorInfo{cfh, iter.get(), i}); + } else { + considerStatus(iter->status()); + if (!status_.ok()) { + // Non-OK status from the iterator. Bail out early + heap.clear(); + break; + } + } + ++i; + } + if (!heap.empty()) { + PopulateIterator(heap); + } + } + + template + void AdvanceIterator(BinaryHeap& heap, AdvanceFuncType advance_func) { + assert(!heap.empty()); + reset_func_(); + + // 1. Keep the top iterator (by popping it from the heap) + // 2. Make sure all others have iterated past the top iterator key slice + // 3. Advance the top iterator, and add it back to the heap if valid + auto top = heap.top(); + heap.pop(); + if (!heap.empty()) { + auto current = heap.top(); + assert(current.iterator); + while (current.iterator->Valid() && + comparator_->Compare(top.iterator->key(), + current.iterator->key()) == 0) { + assert(current.iterator->status().ok()); + advance_func(current.iterator); + if (current.iterator->Valid()) { + heap.replace_top(heap.top()); + } else { + considerStatus(current.iterator->status()); + if (!status_.ok()) { + heap.clear(); + return; + } else { + heap.pop(); + } + } + if (!heap.empty()) { + current = heap.top(); + } + } + } + advance_func(top.iterator); + if (top.iterator->Valid()) { + assert(top.iterator->status().ok()); + heap.push(top); + } else { + considerStatus(top.iterator->status()); + if (!status_.ok()) { + heap.clear(); + return; + } + } + + if (!heap.empty()) { + PopulateIterator(heap); + } + } + + template + void PopulateIterator(BinaryHeap& heap) { + // 1. Keep the top iterator (by popping it from the heap) and add it to list + // to populate + // 2. For all non-top iterators having the same key as top iter popped + // from the previous step, add them to the same list and pop it + // temporarily from the heap + // 3. Once no other iters have the same key as the top iter from step 1, + // populate the value/columns and attribute_groups from the list + // collected in step 1 and 2 and add all the iters back to the heap + assert(!heap.empty()); + auto top = heap.top(); + heap.pop(); + autovector to_populate; + to_populate.push_back(top); + if (!heap.empty()) { + auto current = heap.top(); + assert(current.iterator); + while (current.iterator->Valid() && + comparator_->Compare(top.iterator->key(), + current.iterator->key()) == 0) { + assert(current.iterator->status().ok()); + to_populate.push_back(current); + heap.pop(); + if (!heap.empty()) { + current = heap.top(); + } else { + break; + } + } + } + // Add the items back to the heap + for (auto& item : to_populate) { + heap.push(item); + } + populate_func_(to_populate); + } +}; + +} // namespace ROCKSDB_NAMESPACE diff --git a/db/multi_cf_iterator_test.cc b/db/multi_cf_iterator_test.cc index f4d146ca14a..f3094f358e6 100644 --- a/db/multi_cf_iterator_test.cc +++ b/db/multi_cf_iterator_test.cc @@ -3,43 +3,37 @@ // COPYING file in the root directory) and Apache 2.0 License // (found in the LICENSE.Apache file in the root directory). -#include - #include "db/db_test_util.h" +#include "rocksdb/attribute_groups.h" namespace ROCKSDB_NAMESPACE { -class MultiCfIteratorTest : public DBTestBase { +class CoalescingIteratorTest : public DBTestBase { public: - MultiCfIteratorTest() - : DBTestBase("multi_cf_iterator_test", /*env_do_fsync=*/true) {} + CoalescingIteratorTest() + : DBTestBase("coalescing_iterator_test", /*env_do_fsync=*/true) {} - // Verify Iteration of MultiCfIterator + // Verify Iteration of CoalescingIterator // by SeekToFirst() + Next() and SeekToLast() + Prev() - void verifyMultiCfIterator( - const std::vector& cfhs, - const std::vector& expected_keys, - const std::optional>& expected_values = std::nullopt, - const std::optional>& expected_wide_columns = - std::nullopt, - const std::optional>& - expected_attribute_groups = std::nullopt) { + void verifyCoalescingIterator(const std::vector& cfhs, + const std::vector& expected_keys, + const std::vector& expected_values, + const std::optional>& + expected_wide_columns = std::nullopt, + const Slice* lower_bound = nullptr, + const Slice* upper_bound = nullptr) { int i = 0; + ReadOptions read_options; + read_options.iterate_lower_bound = lower_bound; + read_options.iterate_upper_bound = upper_bound; std::unique_ptr iter = - db_->NewMultiCfIterator(ReadOptions(), cfhs); + db_->NewCoalescingIterator(read_options, cfhs); for (iter->SeekToFirst(); iter->Valid(); iter->Next()) { ASSERT_EQ(expected_keys[i], iter->key()); - if (expected_values.has_value()) { - ASSERT_EQ(expected_values.value()[i], iter->value()); - } + ASSERT_EQ(expected_values[i], iter->value()); if (expected_wide_columns.has_value()) { ASSERT_EQ(expected_wide_columns.value()[i], iter->columns()); } - if (expected_attribute_groups.has_value()) { - // TODO - Add this back when attribute_groups() API is added - // ASSERT_EQ(expected_attribute_groups.value()[i], - // iter->attribute_groups()); - } ++i; } ASSERT_EQ(expected_keys.size(), i); @@ -48,17 +42,10 @@ class MultiCfIteratorTest : public DBTestBase { int rev_i = i - 1; for (iter->SeekToLast(); iter->Valid(); iter->Prev()) { ASSERT_EQ(expected_keys[rev_i], iter->key()); - if (expected_values.has_value()) { - ASSERT_EQ(expected_values.value()[rev_i], iter->value()); - } + ASSERT_EQ(expected_values[rev_i], iter->value()); if (expected_wide_columns.has_value()) { ASSERT_EQ(expected_wide_columns.value()[rev_i], iter->columns()); } - if (expected_attribute_groups.has_value()) { - // TODO - Add this back when attribute_groups() API is added - // ASSERT_EQ(expected_attribute_groups.value()[rev_i], - // iter->attribute_groups()); - } rev_i--; } ASSERT_OK(iter->status()); @@ -78,20 +65,20 @@ class MultiCfIteratorTest : public DBTestBase { } }; -TEST_F(MultiCfIteratorTest, InvalidArguments) { +TEST_F(CoalescingIteratorTest, InvalidArguments) { Options options = GetDefaultOptions(); { CreateAndReopenWithCF({"cf_1", "cf_2", "cf_3"}, options); // Invalid - No CF is provided std::unique_ptr iter_with_no_cf = - db_->NewMultiCfIterator(ReadOptions(), {}); + db_->NewCoalescingIterator(ReadOptions(), {}); ASSERT_NOK(iter_with_no_cf->status()); ASSERT_TRUE(iter_with_no_cf->status().IsInvalidArgument()); } } -TEST_F(MultiCfIteratorTest, SimpleValues) { +TEST_F(CoalescingIteratorTest, SimpleValues) { Options options = GetDefaultOptions(); { // Case 1: Unique key per CF @@ -109,19 +96,21 @@ TEST_F(MultiCfIteratorTest, SimpleValues) { // Test for iteration over CF default->1->2->3 std::vector cfhs_order_0_1_2_3 = { handles_[0], handles_[1], handles_[2], handles_[3]}; - verifyMultiCfIterator(cfhs_order_0_1_2_3, expected_keys, expected_values); + verifyCoalescingIterator(cfhs_order_0_1_2_3, expected_keys, + expected_values); // Test for iteration over CF 3->1->default_cf->2 std::vector cfhs_order_3_1_0_2 = { handles_[3], handles_[1], handles_[0], handles_[2]}; // Iteration order and the return values should be the same since keys are // unique per CF - verifyMultiCfIterator(cfhs_order_3_1_0_2, expected_keys, expected_values); + verifyCoalescingIterator(cfhs_order_3_1_0_2, expected_keys, + expected_values); // Verify Seek() { std::unique_ptr iter = - db_->NewMultiCfIterator(ReadOptions(), cfhs_order_0_1_2_3); + db_->NewCoalescingIterator(ReadOptions(), cfhs_order_0_1_2_3); iter->Seek(""); ASSERT_EQ(IterStatus(iter.get()), "key_1->key_1_cf_0_val"); iter->Seek("key_1"); @@ -136,7 +125,7 @@ TEST_F(MultiCfIteratorTest, SimpleValues) { // Verify SeekForPrev() { std::unique_ptr iter = - db_->NewMultiCfIterator(ReadOptions(), cfhs_order_0_1_2_3); + db_->NewCoalescingIterator(ReadOptions(), cfhs_order_0_1_2_3); iter->SeekForPrev(""); ASSERT_EQ(IterStatus(iter.get()), "(invalid)"); iter->SeekForPrev("key_1"); @@ -172,56 +161,358 @@ TEST_F(MultiCfIteratorTest, SimpleValues) { // Test for iteration over CFs default->1->2->3 std::vector cfhs_order_0_1_2_3 = { handles_[0], handles_[1], handles_[2], handles_[3]}; - std::vector expected_values = {"key_1_cf_0_val", "key_2_cf_1_val", - "key_3_cf_0_val"}; - verifyMultiCfIterator(cfhs_order_0_1_2_3, expected_keys, expected_values); + std::vector expected_values = {"key_1_cf_3_val", "key_2_cf_2_val", + "key_3_cf_3_val"}; + verifyCoalescingIterator(cfhs_order_0_1_2_3, expected_keys, + expected_values); // Test for iteration over CFs 3->2->default_cf->1 std::vector cfhs_order_3_2_0_1 = { handles_[3], handles_[2], handles_[0], handles_[1]}; - expected_values = {"key_1_cf_3_val", "key_2_cf_2_val", "key_3_cf_3_val"}; - verifyMultiCfIterator(cfhs_order_3_2_0_1, expected_keys, expected_values); + expected_values = {"key_1_cf_0_val", "key_2_cf_1_val", "key_3_cf_1_val"}; + verifyCoalescingIterator(cfhs_order_3_2_0_1, expected_keys, + expected_values); // Verify Seek() { std::unique_ptr iter = - db_->NewMultiCfIterator(ReadOptions(), cfhs_order_3_2_0_1); + db_->NewCoalescingIterator(ReadOptions(), cfhs_order_3_2_0_1); iter->Seek(""); - ASSERT_EQ(IterStatus(iter.get()), "key_1->key_1_cf_3_val"); + ASSERT_EQ(IterStatus(iter.get()), "key_1->key_1_cf_0_val"); iter->Seek("key_1"); - ASSERT_EQ(IterStatus(iter.get()), "key_1->key_1_cf_3_val"); + ASSERT_EQ(IterStatus(iter.get()), "key_1->key_1_cf_0_val"); iter->Seek("key_2"); - ASSERT_EQ(IterStatus(iter.get()), "key_2->key_2_cf_2_val"); + ASSERT_EQ(IterStatus(iter.get()), "key_2->key_2_cf_1_val"); iter->Next(); - ASSERT_EQ(IterStatus(iter.get()), "key_3->key_3_cf_3_val"); + ASSERT_EQ(IterStatus(iter.get()), "key_3->key_3_cf_1_val"); iter->Seek("key_x"); ASSERT_EQ(IterStatus(iter.get()), "(invalid)"); } // Verify SeekForPrev() { std::unique_ptr iter = - db_->NewMultiCfIterator(ReadOptions(), cfhs_order_3_2_0_1); + db_->NewCoalescingIterator(ReadOptions(), cfhs_order_3_2_0_1); iter->SeekForPrev(""); ASSERT_EQ(IterStatus(iter.get()), "(invalid)"); iter->SeekForPrev("key_1"); - ASSERT_EQ(IterStatus(iter.get()), "key_1->key_1_cf_3_val"); + ASSERT_EQ(IterStatus(iter.get()), "key_1->key_1_cf_0_val"); iter->Next(); - ASSERT_EQ(IterStatus(iter.get()), "key_2->key_2_cf_2_val"); + ASSERT_EQ(IterStatus(iter.get()), "key_2->key_2_cf_1_val"); iter->SeekForPrev("key_x"); - ASSERT_EQ(IterStatus(iter.get()), "key_3->key_3_cf_3_val"); + ASSERT_EQ(IterStatus(iter.get()), "key_3->key_3_cf_1_val"); iter->Next(); ASSERT_EQ(IterStatus(iter.get()), "(invalid)"); } } } -TEST_F(MultiCfIteratorTest, EmptyCfs) { +TEST_F(CoalescingIteratorTest, LowerAndUpperBounds) { + Options options = GetDefaultOptions(); + { + // Case 1: Unique key per CF + CreateAndReopenWithCF({"cf_1", "cf_2", "cf_3"}, options); + + ASSERT_OK(Put(0, "key_1", "key_1_cf_0_val")); + ASSERT_OK(Put(1, "key_2", "key_2_cf_1_val")); + ASSERT_OK(Put(2, "key_3", "key_3_cf_2_val")); + ASSERT_OK(Put(3, "key_4", "key_4_cf_3_val")); + + std::vector cfhs_order_0_1_2_3 = { + handles_[0], handles_[1], handles_[2], handles_[3]}; + + // with lower_bound + { + // lower_bound is inclusive + Slice lb = Slice("key_2"); + std::vector expected_keys = {"key_2", "key_3", "key_4"}; + std::vector expected_values = {"key_2_cf_1_val", "key_3_cf_2_val", + "key_4_cf_3_val"}; + verifyCoalescingIterator(cfhs_order_0_1_2_3, expected_keys, + expected_values, std::nullopt, &lb); + } + // with upper_bound + { + // upper_bound is exclusive + Slice ub = Slice("key_3"); + std::vector expected_keys = {"key_1", "key_2"}; + std::vector expected_values = {"key_1_cf_0_val", "key_2_cf_1_val"}; + verifyCoalescingIterator(cfhs_order_0_1_2_3, expected_keys, + expected_values, std::nullopt, nullptr, &ub); + } + // with lower and upper bound + { + Slice lb = Slice("key_2"); + Slice ub = Slice("key_4"); + std::vector expected_keys = {"key_2", "key_3"}; + std::vector expected_values = {"key_2_cf_1_val", "key_3_cf_2_val"}; + verifyCoalescingIterator(cfhs_order_0_1_2_3, expected_keys, + expected_values, std::nullopt, &lb, &ub); + } + + { + Slice lb = Slice("key_2"); + Slice ub = Slice("key_4"); + ReadOptions read_options; + read_options.iterate_lower_bound = &lb; + read_options.iterate_upper_bound = &ub; + // Verify Seek() with bounds + { + std::unique_ptr iter = + db_->NewCoalescingIterator(read_options, cfhs_order_0_1_2_3); + iter->Seek(""); + ASSERT_EQ(IterStatus(iter.get()), "key_2->key_2_cf_1_val"); + iter->Next(); + ASSERT_EQ(IterStatus(iter.get()), "key_3->key_3_cf_2_val"); + iter->Seek("key_x"); + ASSERT_EQ(IterStatus(iter.get()), "(invalid)"); + } + // Verify SeekForPrev() with bounds + { + std::unique_ptr iter = + db_->NewCoalescingIterator(read_options, cfhs_order_0_1_2_3); + iter->SeekForPrev(""); + ASSERT_EQ(IterStatus(iter.get()), "(invalid)"); + iter->SeekForPrev("key_1"); + ASSERT_EQ(IterStatus(iter.get()), "(invalid)"); + iter->SeekForPrev("key_2"); + ASSERT_EQ(IterStatus(iter.get()), "key_2->key_2_cf_1_val"); + iter->Next(); + ASSERT_EQ(IterStatus(iter.get()), "key_3->key_3_cf_2_val"); + iter->SeekForPrev("key_x"); + ASSERT_EQ(IterStatus(iter.get()), "key_3->key_3_cf_2_val"); + iter->Prev(); + ASSERT_EQ(IterStatus(iter.get()), "key_2->key_2_cf_1_val"); + iter->Next(); + ASSERT_EQ(IterStatus(iter.get()), "key_3->key_3_cf_2_val"); + iter->Next(); + ASSERT_EQ(IterStatus(iter.get()), "(invalid)"); + } + } + } + { + // Case 2: Same key in multiple CFs + options = CurrentOptions(options); + DestroyAndReopen(options); + CreateAndReopenWithCF({"cf_1", "cf_2", "cf_3"}, options); + + ASSERT_OK(Put(0, "key_1", "key_1_cf_0_val")); + ASSERT_OK(Put(3, "key_1", "key_1_cf_3_val")); + ASSERT_OK(Put(1, "key_2", "key_2_cf_1_val")); + ASSERT_OK(Put(2, "key_2", "key_2_cf_2_val")); + ASSERT_OK(Put(0, "key_3", "key_3_cf_0_val")); + ASSERT_OK(Put(1, "key_3", "key_3_cf_1_val")); + ASSERT_OK(Put(3, "key_3", "key_3_cf_3_val")); + + // Test for iteration over CFs default->1->2->3 + std::vector cfhs_order_0_1_2_3 = { + handles_[0], handles_[1], handles_[2], handles_[3]}; + // with lower_bound + { + // lower_bound is inclusive + Slice lb = Slice("key_2"); + std::vector expected_keys = {"key_2", "key_3"}; + std::vector expected_values = {"key_2_cf_2_val", "key_3_cf_3_val"}; + verifyCoalescingIterator(cfhs_order_0_1_2_3, expected_keys, + expected_values, std::nullopt, &lb); + } + // with upper_bound + { + // upper_bound is exclusive + Slice ub = Slice("key_3"); + std::vector expected_keys = {"key_1", "key_2"}; + std::vector expected_values = {"key_1_cf_3_val", "key_2_cf_2_val"}; + verifyCoalescingIterator(cfhs_order_0_1_2_3, expected_keys, + expected_values, std::nullopt, nullptr, &ub); + } + // with lower and upper bound + { + Slice lb = Slice("key_2"); + Slice ub = Slice("key_3"); + std::vector expected_keys = {"key_2"}; + std::vector expected_values = {"key_2_cf_2_val"}; + verifyCoalescingIterator(cfhs_order_0_1_2_3, expected_keys, + expected_values, std::nullopt, &lb, &ub); + } + + // Test for iteration over CFs 3->2->default_cf->1 + std::vector cfhs_order_3_2_0_1 = { + handles_[3], handles_[2], handles_[0], handles_[1]}; + { + // lower_bound is inclusive + Slice lb = Slice("key_2"); + std::vector expected_keys = {"key_2", "key_3"}; + std::vector expected_values = {"key_2_cf_1_val", "key_3_cf_1_val"}; + verifyCoalescingIterator(cfhs_order_3_2_0_1, expected_keys, + expected_values, std::nullopt, &lb); + } + // with upper_bound + { + // upper_bound is exclusive + Slice ub = Slice("key_3"); + std::vector expected_keys = {"key_1", "key_2"}; + std::vector expected_values = {"key_1_cf_0_val", "key_2_cf_1_val"}; + verifyCoalescingIterator(cfhs_order_3_2_0_1, expected_keys, + expected_values, std::nullopt, nullptr, &ub); + } + // with lower and upper bound + { + Slice lb = Slice("key_2"); + Slice ub = Slice("key_3"); + std::vector expected_keys = {"key_2"}; + std::vector expected_values = {"key_2_cf_1_val"}; + verifyCoalescingIterator(cfhs_order_3_2_0_1, expected_keys, + expected_values, std::nullopt, &lb, &ub); + } + { + Slice lb = Slice("key_2"); + Slice ub = Slice("key_3"); + ReadOptions read_options; + read_options.iterate_lower_bound = &lb; + read_options.iterate_upper_bound = &ub; + // Verify Seek() with bounds + { + std::unique_ptr iter = + db_->NewCoalescingIterator(read_options, cfhs_order_3_2_0_1); + iter->Seek(""); + ASSERT_EQ(IterStatus(iter.get()), "key_2->key_2_cf_1_val"); + iter->Next(); + ASSERT_EQ(IterStatus(iter.get()), "(invalid)"); + iter->Seek("key_x"); + ASSERT_EQ(IterStatus(iter.get()), "(invalid)"); + } + // Verify SeekForPrev() with bounds + { + std::unique_ptr iter = + db_->NewCoalescingIterator(read_options, cfhs_order_3_2_0_1); + iter->SeekForPrev(""); + ASSERT_EQ(IterStatus(iter.get()), "(invalid)"); + iter->SeekForPrev("key_1"); + ASSERT_EQ(IterStatus(iter.get()), "(invalid)"); + iter->SeekForPrev("key_2"); + ASSERT_EQ(IterStatus(iter.get()), "key_2->key_2_cf_1_val"); + iter->SeekForPrev("key_x"); + ASSERT_EQ(IterStatus(iter.get()), "key_2->key_2_cf_1_val"); + iter->Next(); + ASSERT_EQ(IterStatus(iter.get()), "(invalid)"); + } + } + } +} + +TEST_F(CoalescingIteratorTest, ConsistentViewExplicitSnapshot) { + Options options = GetDefaultOptions(); + options.atomic_flush = true; + CreateAndReopenWithCF({"cf_1", "cf_2", "cf_3"}, options); + + for (int i = 0; i < 4; ++i) { + ASSERT_OK(Put(i, "cf" + std::to_string(i) + "_key", + "cf" + std::to_string(i) + "_val")); + } + + ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->LoadDependency( + {{"DBImpl::BGWorkFlush:done", + "DBImpl::MultiCFSnapshot::BeforeCheckingSnapshot"}}); + + bool flushed = false; + ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack( + "DBImpl::MultiCFSnapshot::AfterRefSV", [&](void* /*arg*/) { + if (!flushed) { + for (int i = 0; i < 4; ++i) { + ASSERT_OK(Put(i, "cf" + std::to_string(i) + "_key", + "cf" + std::to_string(i) + "_val_new")); + } + ASSERT_OK(Flush()); + flushed = true; + } + }); + ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing(); + + std::vector cfhs_order_0_1_2_3 = { + handles_[0], handles_[1], handles_[2], handles_[3]}; + ReadOptions read_options; + const Snapshot* snapshot = db_->GetSnapshot(); + read_options.snapshot = snapshot; + // Verify Seek() + { + std::unique_ptr iter = + db_->NewCoalescingIterator(read_options, cfhs_order_0_1_2_3); + iter->Seek(""); + ASSERT_EQ(IterStatus(iter.get()), "cf0_key->cf0_val"); + iter->Next(); + ASSERT_EQ(IterStatus(iter.get()), "cf1_key->cf1_val"); + } + // Verify SeekForPrev() + { + std::unique_ptr iter = + db_->NewCoalescingIterator(read_options, cfhs_order_0_1_2_3); + iter->SeekForPrev(""); + ASSERT_EQ(IterStatus(iter.get()), "(invalid)"); + iter->SeekForPrev("cf2_key"); + ASSERT_EQ(IterStatus(iter.get()), "cf2_key->cf2_val"); + iter->Prev(); + ASSERT_EQ(IterStatus(iter.get()), "cf1_key->cf1_val"); + } + db_->ReleaseSnapshot(snapshot); +} + +TEST_F(CoalescingIteratorTest, ConsistentViewImplicitSnapshot) { + Options options = GetDefaultOptions(); + CreateAndReopenWithCF({"cf_1", "cf_2", "cf_3"}, options); + + for (int i = 0; i < 4; ++i) { + ASSERT_OK(Put(i, "cf" + std::to_string(i) + "_key", + "cf" + std::to_string(i) + "_val")); + } + + ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->LoadDependency( + {{"DBImpl::BGWorkFlush:done", + "DBImpl::MultiCFSnapshot::BeforeCheckingSnapshot"}}); + + bool flushed = false; + ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack( + "DBImpl::MultiCFSnapshot::AfterRefSV", [&](void* /*arg*/) { + if (!flushed) { + for (int i = 0; i < 4; ++i) { + ASSERT_OK(Put(i, "cf" + std::to_string(i) + "_key", + "cf" + std::to_string(i) + "_val_new")); + } + ASSERT_OK(Flush(1)); + flushed = true; + } + }); + ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing(); + + std::vector cfhs_order_0_1_2_3 = { + handles_[0], handles_[1], handles_[2], handles_[3]}; + // Verify Seek() + { + std::unique_ptr iter = + db_->NewCoalescingIterator(ReadOptions(), cfhs_order_0_1_2_3); + iter->Seek("cf2_key"); + ASSERT_EQ(IterStatus(iter.get()), "cf2_key->cf2_val_new"); + iter->Next(); + ASSERT_EQ(IterStatus(iter.get()), "cf3_key->cf3_val_new"); + } + // Verify SeekForPrev() + { + std::unique_ptr iter = + db_->NewCoalescingIterator(ReadOptions(), cfhs_order_0_1_2_3); + iter->SeekForPrev(""); + ASSERT_EQ(IterStatus(iter.get()), "(invalid)"); + iter->SeekForPrev("cf1_key"); + ASSERT_EQ(IterStatus(iter.get()), "cf1_key->cf1_val_new"); + iter->Prev(); + ASSERT_EQ(IterStatus(iter.get()), "cf0_key->cf0_val_new"); + } +} + +TEST_F(CoalescingIteratorTest, EmptyCfs) { Options options = GetDefaultOptions(); { // Case 1: No keys in any of the CFs CreateAndReopenWithCF({"cf_1", "cf_2", "cf_3"}, options); std::unique_ptr iter = - db_->NewMultiCfIterator(ReadOptions(), handles_); + db_->NewCoalescingIterator(ReadOptions(), handles_); iter->SeekToFirst(); ASSERT_EQ(IterStatus(iter.get()), "(invalid)"); iter->SeekToLast(); @@ -237,7 +528,7 @@ TEST_F(MultiCfIteratorTest, EmptyCfs) { // Case 2: A single key exists in only one of the CF. Rest CFs are empty. ASSERT_OK(Put(1, "key_1", "key_1_cf_1_val")); std::unique_ptr iter = - db_->NewMultiCfIterator(ReadOptions(), handles_); + db_->NewCoalescingIterator(ReadOptions(), handles_); iter->SeekToFirst(); ASSERT_EQ(IterStatus(iter.get()), "key_1->key_1_cf_1_val"); iter->Next(); @@ -251,21 +542,21 @@ TEST_F(MultiCfIteratorTest, EmptyCfs) { // Case 3: same key exists in all of the CFs except one (cf_2) ASSERT_OK(Put(0, "key_1", "key_1_cf_0_val")); ASSERT_OK(Put(3, "key_1", "key_1_cf_3_val")); - // handles_ are in the order of 0->1->2->3. We should expect value from cf_0 + // handles_ are in the order of 0->1->2->3 std::unique_ptr iter = - db_->NewMultiCfIterator(ReadOptions(), handles_); + db_->NewCoalescingIterator(ReadOptions(), handles_); iter->SeekToFirst(); - ASSERT_EQ(IterStatus(iter.get()), "key_1->key_1_cf_0_val"); + ASSERT_EQ(IterStatus(iter.get()), "key_1->key_1_cf_3_val"); iter->Next(); ASSERT_EQ(IterStatus(iter.get()), "(invalid)"); iter->SeekToLast(); - ASSERT_EQ(IterStatus(iter.get()), "key_1->key_1_cf_0_val"); + ASSERT_EQ(IterStatus(iter.get()), "key_1->key_1_cf_3_val"); iter->Prev(); ASSERT_EQ(IterStatus(iter.get()), "(invalid)"); } } -TEST_F(MultiCfIteratorTest, WideColumns) { +TEST_F(CoalescingIteratorTest, WideColumns) { // Set up the DB and Column Families Options options = GetDefaultOptions(); CreateAndReopenWithCF({"cf_1", "cf_2", "cf_3"}, options); @@ -274,30 +565,65 @@ TEST_F(MultiCfIteratorTest, WideColumns) { WideColumns key_1_columns_in_cf_2{ {kDefaultWideColumnName, "cf_2_col_val_0_key_1"}, {"cf_2_col_name_1", "cf_2_col_val_1_key_1"}, - {"cf_2_col_name_2", "cf_2_col_val_2_key_1"}}; + {"cf_2_col_name_2", "cf_2_col_val_2_key_1"}, + {"cf_overlap_col_name", "cf_2_overlap_value_key_1"}}; WideColumns key_1_columns_in_cf_3{ {"cf_3_col_name_1", "cf_3_col_val_1_key_1"}, {"cf_3_col_name_2", "cf_3_col_val_2_key_1"}, - {"cf_3_col_name_3", "cf_3_col_val_3_key_1"}}; + {"cf_3_col_name_3", "cf_3_col_val_3_key_1"}, + {"cf_overlap_col_name", "cf_3_overlap_value_key_1"}}; + WideColumns key_1_expected_columns_cfh_order_2_3{ + {kDefaultWideColumnName, "cf_2_col_val_0_key_1"}, + {"cf_2_col_name_1", "cf_2_col_val_1_key_1"}, + {"cf_2_col_name_2", "cf_2_col_val_2_key_1"}, + {"cf_3_col_name_1", "cf_3_col_val_1_key_1"}, + {"cf_3_col_name_2", "cf_3_col_val_2_key_1"}, + {"cf_3_col_name_3", "cf_3_col_val_3_key_1"}, + {"cf_overlap_col_name", "cf_3_overlap_value_key_1"}}; + WideColumns key_1_expected_columns_cfh_order_3_2{ + {kDefaultWideColumnName, "cf_2_col_val_0_key_1"}, + {"cf_2_col_name_1", "cf_2_col_val_1_key_1"}, + {"cf_2_col_name_2", "cf_2_col_val_2_key_1"}, + {"cf_3_col_name_1", "cf_3_col_val_1_key_1"}, + {"cf_3_col_name_2", "cf_3_col_val_2_key_1"}, + {"cf_3_col_name_3", "cf_3_col_val_3_key_1"}, + {"cf_overlap_col_name", "cf_2_overlap_value_key_1"}}; constexpr char key_2[] = "key_2"; WideColumns key_2_columns_in_cf_1{ - {"cf_1_col_name_1", "cf_1_col_val_1_key_2"}}; + {"cf_overlap_col_name", "cf_1_overlap_value_key_2"}}; WideColumns key_2_columns_in_cf_2{ {"cf_2_col_name_1", "cf_2_col_val_1_key_2"}, - {"cf_2_col_name_2", "cf_2_col_val_2_key_2"}}; + {"cf_2_col_name_2", "cf_2_col_val_2_key_2"}, + {"cf_overlap_col_name", "cf_2_overlap_value_key_2"}}; + WideColumns key_2_expected_columns_cfh_order_1_2{ + {"cf_2_col_name_1", "cf_2_col_val_1_key_2"}, + {"cf_2_col_name_2", "cf_2_col_val_2_key_2"}, + {"cf_overlap_col_name", "cf_2_overlap_value_key_2"}}; + WideColumns key_2_expected_columns_cfh_order_2_1{ + {"cf_2_col_name_1", "cf_2_col_val_1_key_2"}, + {"cf_2_col_name_2", "cf_2_col_val_2_key_2"}, + {"cf_overlap_col_name", "cf_1_overlap_value_key_2"}}; constexpr char key_3[] = "key_3"; WideColumns key_3_columns_in_cf_1{ {"cf_1_col_name_1", "cf_1_col_val_1_key_3"}}; WideColumns key_3_columns_in_cf_3{ {"cf_3_col_name_1", "cf_3_col_val_1_key_3"}}; + WideColumns key_3_expected_columns{ + {"cf_1_col_name_1", "cf_1_col_val_1_key_3"}, + {"cf_3_col_name_1", "cf_3_col_val_1_key_3"}, + }; constexpr char key_4[] = "key_4"; WideColumns key_4_columns_in_cf_0{ {"cf_0_col_name_1", "cf_0_col_val_1_key_4"}}; WideColumns key_4_columns_in_cf_2{ {"cf_2_col_name_1", "cf_2_col_val_1_key_4"}}; + WideColumns key_4_expected_columns{ + {"cf_0_col_name_1", "cf_0_col_val_1_key_4"}, + {"cf_2_col_name_1", "cf_2_col_val_1_key_4"}, + }; // Use AttributeGroup PutEntity API to insert them together AttributeGroups key_1_attribute_groups{ @@ -318,31 +644,47 @@ TEST_F(MultiCfIteratorTest, WideColumns) { ASSERT_OK(db_->PutEntity(WriteOptions(), key_3, key_3_attribute_groups)); ASSERT_OK(db_->PutEntity(WriteOptions(), key_4, key_4_attribute_groups)); - // Test for iteration over CF default->1->2->3 - std::vector cfhs_order_0_1_2_3 = { - handles_[0], handles_[1], handles_[2], handles_[3]}; + // Keys should be returned in order regardless of cfh order std::vector expected_keys = {key_1, key_2, key_3, key_4}; - // Pick what DBIter would return for value() in the first CF that key exists + // Since value for kDefaultWideColumnName only exists for key_1, rest will - // return empty value + // return empty value after coalesced std::vector expected_values = {"cf_2_col_val_0_key_1", "", "", ""}; - // Pick columns from the first CF that the key exists and value is stored as - // wide column - std::vector expected_wide_columns = { - {{kDefaultWideColumnName, "cf_2_col_val_0_key_1"}, - {"cf_2_col_name_1", "cf_2_col_val_1_key_1"}, - {"cf_2_col_name_2", "cf_2_col_val_2_key_1"}}, - {{"cf_1_col_name_1", "cf_1_col_val_1_key_2"}}, - {{"cf_1_col_name_1", "cf_1_col_val_1_key_3"}}, - {{"cf_0_col_name_1", "cf_0_col_val_1_key_4"}}}; - verifyMultiCfIterator(cfhs_order_0_1_2_3, expected_keys, expected_values, - expected_wide_columns); + // Test for iteration over CF default->1->2->3 + { + std::vector cfhs_order_0_1_2_3 = { + handles_[0], handles_[1], handles_[2], handles_[3]}; + + // Coalesced columns + std::vector expected_wide_columns_0_1_2_3 = { + key_1_expected_columns_cfh_order_2_3, + key_2_expected_columns_cfh_order_1_2, key_3_expected_columns, + key_4_expected_columns}; + + verifyCoalescingIterator(cfhs_order_0_1_2_3, expected_keys, expected_values, + expected_wide_columns_0_1_2_3); + } + + // Test for iteration over CF 3->2->default->1 + { + std::vector cfhs_order_3_2_0_1 = { + handles_[3], handles_[2], handles_[0], handles_[1]}; + + // Coalesced columns + std::vector expected_wide_columns_3_2_0_1 = { + key_1_expected_columns_cfh_order_3_2, + key_2_expected_columns_cfh_order_2_1, key_3_expected_columns, + key_4_expected_columns}; + + verifyCoalescingIterator(cfhs_order_3_2_0_1, expected_keys, expected_values, + expected_wide_columns_3_2_0_1); + } } -TEST_F(MultiCfIteratorTest, DifferentComparatorsInMultiCFs) { +TEST_F(CoalescingIteratorTest, DifferentComparatorsInMultiCFs) { // This test creates two column families with two different comparators. - // Attempting to create the MultiCFIterator should fail. + // Attempting to create the CoalescingIterator should fail. Options options = GetDefaultOptions(); options.create_if_missing = true; DestroyAndReopen(options); @@ -362,15 +704,15 @@ TEST_F(MultiCfIteratorTest, DifferentComparatorsInMultiCFs) { verifyExpectedKeys(handles_[1], {"key_3", "key_2", "key_1"}); std::unique_ptr iter = - db_->NewMultiCfIterator(ReadOptions(), handles_); + db_->NewCoalescingIterator(ReadOptions(), handles_); ASSERT_NOK(iter->status()); ASSERT_TRUE(iter->status().IsInvalidArgument()); } -TEST_F(MultiCfIteratorTest, CustomComparatorsInMultiCFs) { +TEST_F(CoalescingIteratorTest, CustomComparatorsInMultiCFs) { // This test creates two column families with the same custom test // comparators (but instantiated independently). Attempting to create the - // MultiCFIterator should not fail. + // CoalescingIterator should not fail. Options options = GetDefaultOptions(); options.create_if_missing = true; DestroyAndReopen(options); @@ -410,12 +752,12 @@ TEST_F(MultiCfIteratorTest, CustomComparatorsInMultiCFs) { std::vector expected_keys = { "key_001_003", "key_001_002", "key_001_001", "key_002_003", "key_002_002", "key_002_001", "key_003_006", "key_003_005", "key_003_004"}; - std::vector expected_values = {"value_0_1", "value_0_2", "value_0_3", + std::vector expected_values = {"value_1_1", "value_1_2", "value_1_3", "value_0_4", "value_0_5", "value_0_6", "value_1_4", "value_1_5", "value_1_6"}; int i = 0; std::unique_ptr iter = - db_->NewMultiCfIterator(ReadOptions(), handles_); + db_->NewCoalescingIterator(ReadOptions(), handles_); for (iter->SeekToFirst(); iter->Valid(); iter->Next()) { ASSERT_EQ(expected_keys[i], iter->key()); ASSERT_EQ(expected_values[i], iter->value()); @@ -424,7 +766,59 @@ TEST_F(MultiCfIteratorTest, CustomComparatorsInMultiCFs) { ASSERT_OK(iter->status()); } -TEST_F(MultiCfIteratorTest, DISABLED_IterateAttributeGroups) { +class AttributeGroupIteratorTest : public DBTestBase { + public: + AttributeGroupIteratorTest() + : DBTestBase("attribute_group_iterator_test", /*env_do_fsync=*/true) {} + + void verifyAttributeGroupIterator( + const std::vector& cfhs, + const std::vector& expected_keys, + const std::vector& expected_attribute_groups, + const Slice* lower_bound = nullptr, const Slice* upper_bound = nullptr) { + int i = 0; + ReadOptions read_options; + read_options.iterate_lower_bound = lower_bound; + read_options.iterate_upper_bound = upper_bound; + std::unique_ptr iter = + db_->NewAttributeGroupIterator(read_options, cfhs); + for (iter->SeekToFirst(); iter->Valid(); iter->Next()) { + ASSERT_EQ(expected_keys[i], iter->key()); + auto iterator_attribute_groups = iter->attribute_groups(); + ASSERT_EQ(expected_attribute_groups[i].size(), + iterator_attribute_groups.size()); + for (size_t cfh_i = 0; cfh_i < iterator_attribute_groups.size(); + cfh_i++) { + ASSERT_EQ(expected_attribute_groups[i][cfh_i].column_family(), + iterator_attribute_groups[cfh_i].column_family()); + ASSERT_EQ(expected_attribute_groups[i][cfh_i].columns(), + iterator_attribute_groups[cfh_i].columns()); + } + ++i; + } + ASSERT_EQ(expected_keys.size(), i); + ASSERT_OK(iter->status()); + + int rev_i = i - 1; + for (iter->SeekToLast(); iter->Valid(); iter->Prev()) { + ASSERT_EQ(expected_keys[rev_i], iter->key()); + auto iterator_attribute_groups = iter->attribute_groups(); + ASSERT_EQ(expected_attribute_groups[rev_i].size(), + iterator_attribute_groups.size()); + for (size_t cfh_i = 0; cfh_i < iterator_attribute_groups.size(); + cfh_i++) { + ASSERT_EQ(expected_attribute_groups[rev_i][cfh_i].column_family(), + iterator_attribute_groups[cfh_i].column_family()); + ASSERT_EQ(expected_attribute_groups[rev_i][cfh_i].columns(), + iterator_attribute_groups[cfh_i].columns()); + } + rev_i--; + } + ASSERT_OK(iter->status()); + } +}; + +TEST_F(AttributeGroupIteratorTest, IterateAttributeGroups) { // Set up the DB and Column Families Options options = GetDefaultOptions(); CreateAndReopenWithCF({"cf_1", "cf_2", "cf_3"}, options); @@ -479,13 +873,40 @@ TEST_F(MultiCfIteratorTest, DISABLED_IterateAttributeGroups) { // Test for iteration over CF default->1->2->3 std::vector cfhs_order_0_1_2_3 = { handles_[0], handles_[1], handles_[2], handles_[3]}; - std::vector expected_keys = {key_1, key_2, key_3, key_4}; - std::vector expected_attribute_groups = { - key_1_attribute_groups, key_2_attribute_groups, key_3_attribute_groups, - key_4_attribute_groups}; - verifyMultiCfIterator( - cfhs_order_0_1_2_3, expected_keys, std::nullopt /* expected_values */, - std::nullopt /* expected_wide_columns */, expected_attribute_groups); + { + std::vector expected_keys = {key_1, key_2, key_3, key_4}; + std::vector expected_attribute_groups = { + key_1_attribute_groups, key_2_attribute_groups, key_3_attribute_groups, + key_4_attribute_groups}; + verifyAttributeGroupIterator(cfhs_order_0_1_2_3, expected_keys, + expected_attribute_groups); + } + Slice lb = Slice("key_2"); + Slice ub = Slice("key_4"); + // Test for lower bound only + { + std::vector expected_keys = {key_2, key_3, key_4}; + std::vector expected_attribute_groups = { + key_2_attribute_groups, key_3_attribute_groups, key_4_attribute_groups}; + verifyAttributeGroupIterator(cfhs_order_0_1_2_3, expected_keys, + expected_attribute_groups, &lb); + } + // Test for upper bound only + { + std::vector expected_keys = {key_1, key_2, key_3}; + std::vector expected_attribute_groups = { + key_1_attribute_groups, key_2_attribute_groups, key_3_attribute_groups}; + verifyAttributeGroupIterator(cfhs_order_0_1_2_3, expected_keys, + expected_attribute_groups, nullptr, &ub); + } + // Test for lower and upper bound + { + std::vector expected_keys = {key_2, key_3}; + std::vector expected_attribute_groups = { + key_2_attribute_groups, key_3_attribute_groups}; + verifyAttributeGroupIterator(cfhs_order_0_1_2_3, expected_keys, + expected_attribute_groups, &lb, &ub); + } } } // namespace ROCKSDB_NAMESPACE diff --git a/db/version_set.cc b/db/version_set.cc index 23f6d770f6c..031a8de17c4 100644 --- a/db/version_set.cc +++ b/db/version_set.cc @@ -2777,7 +2777,7 @@ void Version::MultiGet(const ReadOptions& read_options, MultiGetRange* range, } } - if (s.ok() && !blob_ctxs.empty()) { + if (!blob_ctxs.empty()) { MultiGetBlob(read_options, keys_with_blobs_range, blob_ctxs); } @@ -6013,7 +6013,8 @@ Status VersionSet::GetCurrentManifestPath(const std::string& dbname, Status VersionSet::Recover( const std::vector& column_families, bool read_only, - std::string* db_id, bool no_error_if_files_missing) { + std::string* db_id, bool no_error_if_files_missing, bool is_retry, + Status* log_status) { const ReadOptions read_options(Env::IOActivity::kDBOpen); // Read "CURRENT" file, which contains a pointer to the current manifest // file @@ -6038,8 +6039,11 @@ Status VersionSet::Recover( } manifest_file_reader.reset(new SequentialFileReader( std::move(manifest_file), manifest_path, - db_options_->log_readahead_size, io_tracer_, db_options_->listeners)); + db_options_->log_readahead_size, io_tracer_, db_options_->listeners, + /*rate_limiter=*/nullptr, is_retry)); } + TEST_SYNC_POINT("VersionSet::Recover:StartManifestRead"); + uint64_t current_manifest_file_size = 0; uint64_t log_number = 0; { @@ -6063,6 +6067,9 @@ Status VersionSet::Recover( if (s.ok()) { RecoverEpochNumbers(); } + if (log_status) { + *log_status = log_read_status; + } } if (s.ok()) { @@ -7137,6 +7144,20 @@ Status VersionSet::GetMetadataForFile(uint64_t number, int* filelevel, } void VersionSet::GetLiveFilesMetaData(std::vector* metadata) { + if (!metadata) { + return; + } + assert(metadata); + size_t count = 0; + for (auto cfd : *column_family_set_) { + if (cfd->IsDropped() || !cfd->initialized()) { + continue; + } + for (int level = 0; level < cfd->NumberLevels(); level++) { + count += cfd->current()->storage_info()->LevelFiles(level).size(); + } + } + metadata->reserve(count); for (auto cfd : *column_family_set_) { if (cfd->IsDropped() || !cfd->initialized()) { continue; diff --git a/db/version_set.h b/db/version_set.h index 12304348fea..82e6211f450 100644 --- a/db/version_set.h +++ b/db/version_set.h @@ -1258,7 +1258,8 @@ class VersionSet { // are not opened Status Recover(const std::vector& column_families, bool read_only = false, std::string* db_id = nullptr, - bool no_error_if_files_missing = false); + bool no_error_if_files_missing = false, bool is_retry = false, + Status* log_status = nullptr); Status TryRecover(const std::vector& column_families, bool read_only, diff --git a/db/wide/db_wide_basic_test.cc b/db/wide/db_wide_basic_test.cc index 15d2fdff7f9..6f3bc9be7ee 100644 --- a/db/wide/db_wide_basic_test.cc +++ b/db/wide/db_wide_basic_test.cc @@ -1723,6 +1723,77 @@ TEST_F(DBWideBasicTest, PutEntitySerializationError) { ASSERT_OK(db_->Write(WriteOptions(), &batch)); } +TEST_F(DBWideBasicTest, PinnableWideColumnsMove) { + Options options = GetDefaultOptions(); + + constexpr char key1[] = "foo"; + constexpr char value[] = "bar"; + ASSERT_OK(db_->Put(WriteOptions(), db_->DefaultColumnFamily(), key1, value)); + + constexpr char key2[] = "baz"; + const WideColumns columns{{"quux", "corge"}}; + ASSERT_OK(db_->PutEntity(WriteOptions(), db_->DefaultColumnFamily(), key2, + columns)); + + ASSERT_OK(db_->Flush(FlushOptions())); + + const auto test_move = [&](bool fill_cache) { + ReadOptions read_options; + read_options.fill_cache = fill_cache; + + { + const WideColumns expected_columns{{kDefaultWideColumnName, value}}; + + { + PinnableWideColumns result; + ASSERT_OK(db_->GetEntity(read_options, db_->DefaultColumnFamily(), key1, + &result)); + ASSERT_EQ(result.columns(), expected_columns); + + PinnableWideColumns move_target(std::move(result)); + ASSERT_EQ(move_target.columns(), expected_columns); + } + + { + PinnableWideColumns result; + ASSERT_OK(db_->GetEntity(read_options, db_->DefaultColumnFamily(), key1, + &result)); + ASSERT_EQ(result.columns(), expected_columns); + + PinnableWideColumns move_target; + move_target = std::move(result); + ASSERT_EQ(move_target.columns(), expected_columns); + } + } + + { + PinnableWideColumns result; + ASSERT_OK(db_->GetEntity(read_options, db_->DefaultColumnFamily(), key2, + &result)); + ASSERT_EQ(result.columns(), columns); + + PinnableWideColumns move_target(std::move(result)); + ASSERT_EQ(move_target.columns(), columns); + } + + { + PinnableWideColumns result; + ASSERT_OK(db_->GetEntity(read_options, db_->DefaultColumnFamily(), key2, + &result)); + ASSERT_EQ(result.columns(), columns); + + PinnableWideColumns move_target; + move_target = std::move(result); + ASSERT_EQ(move_target.columns(), columns); + } + }; + + // Test with and without fill_cache to cover both the case when pointers are + // invalidated during PinnableSlice's move and when they are not. + test_move(/* fill_cache*/ false); + test_move(/* fill_cache*/ true); +} + } // namespace ROCKSDB_NAMESPACE int main(int argc, char** argv) { diff --git a/db/wide/wide_columns.cc b/db/wide/wide_columns.cc index 61f56d715f6..67b734781d2 100644 --- a/db/wide/wide_columns.cc +++ b/db/wide/wide_columns.cc @@ -4,7 +4,6 @@ // (found in the LICENSE.Apache file in the root directory). #include "rocksdb/wide_columns.h" - #include "db/wide/wide_column_serialization.h" namespace ROCKSDB_NAMESPACE { @@ -12,11 +11,11 @@ namespace ROCKSDB_NAMESPACE { const Slice kDefaultWideColumnName; const WideColumns kNoWideColumns; -const AttributeGroups kNoAttributeGroups; Status PinnableWideColumns::CreateIndexForWideColumns() { - Slice value_copy = value_; + columns_.clear(); + Slice value_copy = value_; return WideColumnSerialization::Deserialize(value_copy, columns_); } diff --git a/db/write_batch.cc b/db/write_batch.cc index 4adba1de84b..d58e1a46683 100644 --- a/db/write_batch.cc +++ b/db/write_batch.cc @@ -484,13 +484,22 @@ Status ReadRecordFromWriteBatch(Slice* input, char* tag, return Status::Corruption("bad WriteBatch TimedPut"); } FALLTHROUGH_INTENDED; - case kTypeValuePreferredSeqno: + case kTypeValuePreferredSeqno: { + Slice packed_value; if (!GetLengthPrefixedSlice(input, key) || - !GetLengthPrefixedSlice(input, value) || - !GetFixed64(input, write_unix_time)) { + !GetLengthPrefixedSlice(input, &packed_value)) { return Status::Corruption("bad WriteBatch TimedPut"); } + if (write_unix_time) { + std::tie(*value, *write_unix_time) = + ParsePackedValueWithWriteTime(packed_value); + } else { + // Caller doesn't want to unpack write_unix_time, so keep it packed in + // the value. + *value = packed_value; + } break; + } default: return Status::Corruption("unknown WriteBatch tag"); } @@ -883,12 +892,11 @@ Status WriteBatchInternal::TimedPut(WriteBatch* b, uint32_t column_family_id, b->rep_.push_back(static_cast(kTypeColumnFamilyValuePreferredSeqno)); PutVarint32(&b->rep_, column_family_id); } + std::string value_buf; + Slice packed_value = + PackValueAndWriteTime(value, write_unix_time, &value_buf); PutLengthPrefixedSlice(&b->rep_, key); - PutLengthPrefixedSlice(&b->rep_, value); - // For a kTypeValuePreferredSeqno entry, its write time is encoded separately - // from value in an encoded WriteBatch. They are packed into one value Slice - // once it's written to the database. - PutFixed64(&b->rep_, write_unix_time); + PutLengthPrefixedSlice(&b->rep_, packed_value); b->content_flags_.store(b->content_flags_.load(std::memory_order_relaxed) | ContentFlags::HAS_TIMED_PUT, @@ -899,7 +907,7 @@ Status WriteBatchInternal::TimedPut(WriteBatch* b, uint32_t column_family_id, // `kTypeColumnFamilyValuePreferredSeqno` here. b->prot_info_->entries_.emplace_back( ProtectionInfo64() - .ProtectKVO(key, value, kTypeValuePreferredSeqno) + .ProtectKVO(key, packed_value, kTypeValuePreferredSeqno) .ProtectC(column_family_id)); } return save.commit(); @@ -1779,7 +1787,6 @@ Status WriteBatch::VerifyChecksum() const { Slice input(rep_.data() + WriteBatchInternal::kHeader, rep_.size() - WriteBatchInternal::kHeader); Slice key, value, blob, xid; - uint64_t unix_write_time = 0; char tag = 0; uint32_t column_family = 0; // default Status s; @@ -1792,7 +1799,7 @@ Status WriteBatch::VerifyChecksum() const { value.clear(); column_family = 0; s = ReadRecordFromWriteBatch(&input, &tag, &column_family, &key, &value, - &blob, &xid, &unix_write_time); + &blob, &xid, /*write_unix_time=*/nullptr); if (!s.ok()) { return s; } @@ -2514,6 +2521,11 @@ class MemTableInserter : public WriteBatch::Handler { assert(ret_status.ok()); if (db_ != nullptr) { + if (db_->immutable_db_options().row_cache) { + ret_status.PermitUncheckedError(); + return Status::NotSupported( + "DeleteRange is not compatible with row cache."); + } auto cf_handle = cf_mems_->GetColumnFamilyHandle(); if (cf_handle == nullptr) { cf_handle = db_->DefaultColumnFamily(); @@ -2623,8 +2635,10 @@ class MemTableInserter : public WriteBatch::Handler { LookupKey lkey(key, sequence_); // Count the number of successive merges at the head - // of the key in the memtable - size_t num_merges = mem->CountSuccessiveMergeEntries(lkey); + // of the key in the memtable. Limit the count to the threshold for + // triggering merge to prevent unnecessary counting overhead. + size_t num_merges = mem->CountSuccessiveMergeEntries( + lkey, moptions->max_successive_merges /* limit */); if (num_merges >= moptions->max_successive_merges) { perform_merge = true; @@ -3160,8 +3174,12 @@ class ProtectionInfoUpdater : public WriteBatch::Handler { } Status TimedPutCF(uint32_t cf, const Slice& key, const Slice& val, - uint64_t /*unix_write_time*/) override { - return UpdateProtInfo(cf, key, val, kTypeValuePreferredSeqno); + uint64_t unix_write_time) override { + std::string encoded_write_time; + PutFixed64(&encoded_write_time, unix_write_time); + std::array value_with_time{{val, encoded_write_time}}; + SliceParts packed_value(value_with_time.data(), 2); + return UpdateProtInfo(cf, key, packed_value, kTypeValuePreferredSeqno); } Status PutEntityCF(uint32_t cf, const Slice& key, @@ -3220,6 +3238,17 @@ class ProtectionInfoUpdater : public WriteBatch::Handler { return Status::OK(); } + Status UpdateProtInfo(uint32_t cf, const Slice& key, const SliceParts& val, + const ValueType op_type) { + if (prot_info_) { + prot_info_->entries_.emplace_back( + ProtectionInfo64() + .ProtectKVO(SliceParts(&key, 1), val, op_type) + .ProtectC(cf)); + } + return Status::OK(); + } + // No copy or move. ProtectionInfoUpdater(const ProtectionInfoUpdater&) = delete; ProtectionInfoUpdater(ProtectionInfoUpdater&&) = delete; diff --git a/db/write_callback_test.cc b/db/write_callback_test.cc index b0d0367b33e..c2f19c85a5e 100644 --- a/db/write_callback_test.cc +++ b/db/write_callback_test.cc @@ -179,7 +179,8 @@ TEST_P(WriteCallbackPTest, WriteWithCallbackTest) { column_families.emplace_back(kDefaultColumnFamilyName, cf_options); std::vector handles; auto open_s = DBImpl::Open(db_options, dbname, column_families, &handles, - &db, seq_per_batch_, true /* batch_per_txn */); + &db, seq_per_batch_, true /* batch_per_txn */, + false /* is_retry */, nullptr /* can_retry */); ASSERT_OK(open_s); assert(handles.size() == 1); delete handles[0]; diff --git a/db_stress_tool/batched_ops_stress.cc b/db_stress_tool/batched_ops_stress.cc index 25e4d23722e..4830c57e92a 100644 --- a/db_stress_tool/batched_ops_stress.cc +++ b/db_stress_tool/batched_ops_stress.cc @@ -42,6 +42,7 @@ class BatchedOpsStressTest : public StressTest { ColumnFamilyHandle* const cfh = column_families_[rand_column_families[0]]; assert(cfh); + Status status; for (int i = 9; i >= 0; --i) { const std::string num = std::to_string(i); @@ -51,28 +52,37 @@ class BatchedOpsStressTest : public StressTest { // batched, non-batched, CF consistency). const std::string k = num + key_body; const std::string v = value_body + num; - if (FLAGS_use_put_entity_one_in > 0 && (value_base % FLAGS_use_put_entity_one_in) == 0) { - batch.PutEntity(cfh, k, GenerateWideColumns(value_base, v)); + status = batch.PutEntity(cfh, k, GenerateWideColumns(value_base, v)); + } else if (FLAGS_use_timed_put_one_in > 0 && + ((value_base + kLargePrimeForCommonFactorSkew) % + FLAGS_use_timed_put_one_in) == 0) { + uint64_t write_unix_time = GetWriteUnixTime(thread); + status = batch.TimedPut(cfh, k, v, write_unix_time); } else if (FLAGS_use_merge) { - batch.Merge(cfh, k, v); + status = batch.Merge(cfh, k, v); } else { - batch.Put(cfh, k, v); + status = batch.Put(cfh, k, v); + } + if (!status.ok()) { + break; } } - const Status s = db_->Write(write_opts, &batch); + if (status.ok()) { + status = db_->Write(write_opts, &batch); + } - if (!s.ok()) { - fprintf(stderr, "multiput error: %s\n", s.ToString().c_str()); + if (!status.ok()) { + fprintf(stderr, "multiput error: %s\n", status.ToString().c_str()); thread->stats.AddErrors(1); } else { // we did 10 writes each of size sz + 1 thread->stats.AddBytesForWrites(10, (sz + 1) * 10); } - return s; + return status; } // Given a key K, this deletes ("0"+K), ("1"+K), ..., ("9"+K) diff --git a/db_stress_tool/cf_consistency_stress.cc b/db_stress_tool/cf_consistency_stress.cc index da382ae3b8d..1a098e98235 100644 --- a/db_stress_tool/cf_consistency_stress.cc +++ b/db_stress_tool/cf_consistency_stress.cc @@ -36,31 +36,43 @@ class CfConsistencyStressTest : public StressTest { WriteBatch batch; + Status status; for (auto cf : rand_column_families) { ColumnFamilyHandle* const cfh = column_families_[cf]; assert(cfh); if (FLAGS_use_put_entity_one_in > 0 && (value_base % FLAGS_use_put_entity_one_in) == 0) { - batch.PutEntity(cfh, k, GenerateWideColumns(value_base, v)); + status = batch.PutEntity(cfh, k, GenerateWideColumns(value_base, v)); + } else if (FLAGS_use_timed_put_one_in > 0 && + ((value_base + kLargePrimeForCommonFactorSkew) % + FLAGS_use_timed_put_one_in) == 0) { + uint64_t write_unix_time = GetWriteUnixTime(thread); + status = batch.TimedPut(cfh, k, v, write_unix_time); } else if (FLAGS_use_merge) { - batch.Merge(cfh, k, v); + status = batch.Merge(cfh, k, v); } else { - batch.Put(cfh, k, v); + status = batch.Put(cfh, k, v); + } + if (!status.ok()) { + break; } } - Status s = db_->Write(write_opts, &batch); + if (status.ok()) { + status = db_->Write(write_opts, &batch); + } - if (!s.ok()) { - fprintf(stderr, "multi put or merge error: %s\n", s.ToString().c_str()); + if (!status.ok()) { + fprintf(stderr, "multi put or merge error: %s\n", + status.ToString().c_str()); thread->stats.AddErrors(1); } else { auto num = static_cast(rand_column_families.size()); thread->stats.AddBytesForWrites(num, (sz + 1) * num); } - return s; + return status; } Status TestDelete(ThreadState* thread, WriteOptions& write_opts, diff --git a/db_stress_tool/db_stress_common.cc b/db_stress_tool/db_stress_common.cc index 27a4299da48..3683b246abb 100644 --- a/db_stress_tool/db_stress_common.cc +++ b/db_stress_tool/db_stress_common.cc @@ -391,6 +391,27 @@ std::string GetNowNanos() { return ret; } +uint64_t GetWriteUnixTime(ThreadState* thread) { + static uint64_t kPreserveSeconds = + std::max(FLAGS_preserve_internal_time_seconds, + FLAGS_preclude_last_level_data_seconds); + static uint64_t kFallbackTime = std::numeric_limits::max(); + int64_t write_time = 0; + Status s = db_stress_env->GetCurrentTime(&write_time); + uint32_t write_time_mode = thread->rand.Uniform(3); + if (write_time_mode == 0 || !s.ok()) { + return kFallbackTime; + } else if (write_time_mode == 1) { + uint64_t delta = kPreserveSeconds > 0 + ? static_cast(thread->rand.Uniform( + static_cast(kPreserveSeconds))) + : 0; + return static_cast(write_time) - delta; + } else { + return static_cast(write_time) - kPreserveSeconds; + } +} + namespace { class MyXXH64Checksum : public FileChecksumGenerator { diff --git a/db_stress_tool/db_stress_common.h b/db_stress_tool/db_stress_common.h index 4a426f44ca8..12be9fbc5fd 100644 --- a/db_stress_tool/db_stress_common.h +++ b/db_stress_tool/db_stress_common.h @@ -46,6 +46,7 @@ #include "monitoring/histogram.h" #include "options/options_helper.h" #include "port/port.h" +#include "rocksdb/advanced_options.h" #include "rocksdb/cache.h" #include "rocksdb/env.h" #include "rocksdb/slice.h" @@ -136,7 +137,8 @@ DECLARE_int32(universal_min_merge_width); DECLARE_int32(universal_max_merge_width); DECLARE_int32(universal_max_size_amplification_percent); DECLARE_int32(clear_column_family_one_in); -DECLARE_int32(get_live_files_one_in); +DECLARE_int32(get_live_files_apis_one_in); +DECLARE_int32(get_all_column_family_metadata_one_in); DECLARE_int32(get_sorted_wal_files_one_in); DECLARE_int32(get_current_wal_file_one_in); DECLARE_int32(set_options_one_in); @@ -155,6 +157,7 @@ DECLARE_int32(unpartitioned_pinning); DECLARE_string(cache_type); DECLARE_uint64(subcompactions); DECLARE_uint64(periodic_compaction_seconds); +DECLARE_string(daily_offpeak_time_utc); DECLARE_uint64(compaction_ttl); DECLARE_bool(fifo_allow_compaction); DECLARE_bool(allow_concurrent_memtable_write); @@ -209,6 +212,8 @@ DECLARE_int32(compact_range_one_in); DECLARE_int32(mark_for_compaction_one_file_in); DECLARE_int32(flush_one_in); DECLARE_int32(pause_background_one_in); +DECLARE_int32(disable_file_deletions_one_in); +DECLARE_int32(disable_manual_compaction_one_in); DECLARE_int32(compact_range_width); DECLARE_int32(acquire_snapshot_one_in); DECLARE_bool(compare_full_db_state_snapshot); @@ -257,8 +262,12 @@ DECLARE_int32(verify_file_checksums_one_in); DECLARE_int32(verify_db_one_in); DECLARE_int32(continuous_verification_interval); DECLARE_int32(get_property_one_in); +DECLARE_int32(get_properties_of_all_tables_one_in); DECLARE_string(file_checksum_impl); DECLARE_bool(verification_only); +DECLARE_string(last_level_temperature); +DECLARE_string(default_write_temperature); +DECLARE_string(default_temperature); // Options for transaction dbs. // Use TransactionDB (a.k.a. Pessimistic Transaction DB) @@ -340,9 +349,9 @@ DECLARE_uint32(memtable_max_range_deletions); DECLARE_uint32(bottommost_file_compaction_delay); // Tiered storage -DECLARE_bool(enable_tiered_storage); // set last_level_temperature DECLARE_int64(preclude_last_level_data_seconds); DECLARE_int64(preserve_internal_time_seconds); +DECLARE_uint32(use_timed_put_one_in); DECLARE_int32(verify_iterator_with_expected_state_one_in); DECLARE_bool(preserve_unverified_changes); @@ -392,10 +401,21 @@ DECLARE_double(low_pri_pool_ratio); DECLARE_uint64(soft_pending_compaction_bytes_limit); DECLARE_uint64(hard_pending_compaction_bytes_limit); DECLARE_uint64(max_sequential_skip_in_iterations); +DECLARE_bool(enable_sst_partitioner_factory); +DECLARE_bool(enable_do_not_compress_roles); +DECLARE_bool(block_align); +DECLARE_uint32(lowest_used_cache_tier); +DECLARE_bool(enable_custom_split_merge); +DECLARE_uint32(adm_policy); +DECLARE_bool(enable_memtable_insert_with_hint_prefix_extractor); +DECLARE_bool(check_multiget_consistency); +DECLARE_bool(check_multiget_entity_consistency); +DECLARE_bool(inplace_update_support); constexpr long KB = 1024; constexpr int kRandomValueMaxFactor = 3; constexpr int kValueMaxLen = 100; +constexpr uint32_t kLargePrimeForCommonFactorSkew = 1872439133; // wrapped posix environment extern ROCKSDB_NAMESPACE::Env* db_stress_env; @@ -487,6 +507,28 @@ inline std::string ChecksumTypeToString(ROCKSDB_NAMESPACE::ChecksumType ctype) { return iter->first; } +inline enum ROCKSDB_NAMESPACE::Temperature StringToTemperature( + const char* ctype) { + assert(ctype); + auto iter = std::find_if( + ROCKSDB_NAMESPACE::temperature_to_string.begin(), + ROCKSDB_NAMESPACE::temperature_to_string.end(), + [&](const std::pair& + temp_and_string_val) { + return ctype == temp_and_string_val.second; + }); + assert(iter != ROCKSDB_NAMESPACE::temperature_to_string.end()); + return iter->first; +} + +inline std::string TemperatureToString( + ROCKSDB_NAMESPACE::Temperature temperature) { + auto iter = + ROCKSDB_NAMESPACE::OptionsHelper::temperature_to_string.find(temperature); + assert(iter != ROCKSDB_NAMESPACE::OptionsHelper::temperature_to_string.end()); + return iter->second; +} + inline std::vector SplitString(std::string src) { std::vector ret; if (src.empty()) { @@ -726,6 +768,8 @@ int64_t GetOneHotKeyID(double rand_seed, int64_t max_key); std::string GetNowNanos(); +uint64_t GetWriteUnixTime(ThreadState* thread); + std::shared_ptr GetFileChecksumImpl( const std::string& name); diff --git a/db_stress_tool/db_stress_gflags.cc b/db_stress_tool/db_stress_gflags.cc index bada4396150..b037b8d97ae 100644 --- a/db_stress_tool/db_stress_gflags.cc +++ b/db_stress_tool/db_stress_gflags.cc @@ -311,9 +311,17 @@ DEFINE_int32(clear_column_family_one_in, 1000000, "it again. If N == 0, never drop/create column families. " "When test_batches_snapshots is true, this flag has no effect"); -DEFINE_int32(get_live_files_one_in, 1000000, - "With a chance of 1/N, call GetLiveFiles to verify if it returns " - "correctly. If N == 0, do not call the interface."); +DEFINE_int32( + get_live_files_apis_one_in, 1000000, + "With a chance of 1/N, call GetLiveFiles(), GetLiveFilesMetaData() and " + "GetLiveFilesChecksumInfo() to verify if it returns " + "OK or violate any internal assertion. If N == 0, do not call the " + "interface."); + +DEFINE_int32( + get_all_column_family_metadata_one_in, 1000000, + "With a chance of 1/N, call GetAllColumnFamilyMetaData to verify if it " + "violates any internal assertion. If N == 0, do not call the interface."); DEFINE_int32( get_sorted_wal_files_one_in, 1000000, @@ -395,6 +403,8 @@ DEFINE_uint64(subcompactions, 1, DEFINE_uint64(periodic_compaction_seconds, 1000, "Files older than this value will be picked up for compaction."); +DEFINE_string(daily_offpeak_time_utc, "", + "If set, process periodic compactions during this period only"); DEFINE_uint64(compaction_ttl, 1000, "Files older than TTL will be compacted to the next level."); @@ -516,8 +526,6 @@ DEFINE_int32(prepopulate_blob_cache, 0, "[Integrated BlobDB] Pre-populate hot/warm blobs in blob cache. 0 " "to disable and 1 to insert during flush."); -DEFINE_bool(enable_tiered_storage, false, "Set last_level_temperature"); - DEFINE_int64(preclude_last_level_data_seconds, 0, "Preclude data from the last level. Used with tiered storage " "feature to preclude new data from comacting to the last level."); @@ -526,6 +534,10 @@ DEFINE_int64( preserve_internal_time_seconds, 0, "Preserve internal time information which is attached to each SST."); +DEFINE_uint32(use_timed_put_one_in, 0, + "If greater than zero, TimedPut is used per every N write ops on " + "on average."); + static const bool FLAGS_subcompactions_dummy __attribute__((__unused__)) = RegisterFlagValidator(&FLAGS_subcompactions, &ValidateUint32Range); @@ -758,6 +770,15 @@ DEFINE_int32(pause_background_one_in, 0, "If non-zero, then PauseBackgroundWork()+Continue will be called " "once for every N ops on average. 0 disables."); +DEFINE_int32(disable_file_deletions_one_in, 0, + "If non-zero, then DisableFileDeletions()+Enable will be called " + "once for every N ops on average. 0 disables."); + +DEFINE_int32( + disable_manual_compaction_one_in, 0, + "If non-zero, then DisableManualCompaction()+Enable will be called " + "once for every N ops on average. 0 disables."); + DEFINE_int32(compact_range_width, 10000, "The width of the ranges passed to CompactRange()."); @@ -981,6 +1002,11 @@ DEFINE_int32(get_property_one_in, 1000, " properties for every N ops on average. 0 indicates that" " GetProperty() will be not be called."); +DEFINE_int32(get_properties_of_all_tables_one_in, 1000, + "If non-zero, then DB::GetPropertiesOfAllTables() will be called " + "for every N ops on average. 0 indicates that" + " it will be not be called."); + DEFINE_bool(sync_fault_injection, false, "If true, FaultInjectionTestFS will be used for write operations, " "and unsynced data in DB will lost after crash. In such a case we " @@ -1304,4 +1330,59 @@ DEFINE_uint64(soft_pending_compaction_bytes_limit, DEFINE_uint64(hard_pending_compaction_bytes_limit, ROCKSDB_NAMESPACE::Options().hard_pending_compaction_bytes_limit, "Options.hard_pending_compaction_bytes_limit"); + +DEFINE_bool(enable_sst_partitioner_factory, false, + "If true, set Options.sst_partitioner_factory to " + "SstPartitionerFixedPrefixFactory with prefix length equal to 1"); + +DEFINE_bool( + enable_do_not_compress_roles, false, + "If true, set CompressedSecondaryCacheOptions.do_not_compress_roles to " + "include all cache roles"); + +DEFINE_bool(block_align, + ROCKSDB_NAMESPACE::BlockBasedTableOptions().block_align, + "BlockBasedTableOptions.block_align"); + +DEFINE_uint32( + lowest_used_cache_tier, + static_cast(ROCKSDB_NAMESPACE::Options().lowest_used_cache_tier), + "Options.lowest_used_cache_tier"); + +DEFINE_bool(enable_custom_split_merge, + ROCKSDB_NAMESPACE::CompressedSecondaryCacheOptions() + .enable_custom_split_merge, + "CompressedSecondaryCacheOptions.enable_custom_split_merge"); + +DEFINE_uint32( + adm_policy, + static_cast(ROCKSDB_NAMESPACE::TieredCacheOptions().adm_policy), + "TieredCacheOptions.adm_policy"); + +DEFINE_string(last_level_temperature, "kUnknown", + "Options.last_level_temperature"); + +DEFINE_string(default_write_temperature, "kUnknown", + "Options.default_write_temperature"); + +DEFINE_string(default_temperature, "kUnknown", "Options.default_temperature"); + +DEFINE_bool(enable_memtable_insert_with_hint_prefix_extractor, + ROCKSDB_NAMESPACE::Options() + .memtable_insert_with_hint_prefix_extractor != nullptr, + "If true and FLAGS_prefix_size > 0, set " + "Options.memtable_insert_with_hint_prefix_extractor to " + "be Options.prefix_extractor"); + +DEFINE_bool(check_multiget_consistency, true, + "If true, check consistency of MultiGet result by comparing it " + "with Get's under a snapshot"); + +DEFINE_bool(check_multiget_entity_consistency, true, + "If true, check consistency of MultiGetEntity result by comparing " + "it GetEntity's under a snapshot"); + +DEFINE_bool(inplace_update_support, + ROCKSDB_NAMESPACE::Options().inplace_update_support, + "Options.inplace_update_support"); #endif // GFLAGS diff --git a/db_stress_tool/db_stress_test_base.cc b/db_stress_tool/db_stress_test_base.cc index 4510634fffd..9de43d6691e 100644 --- a/db_stress_tool/db_stress_test_base.cc +++ b/db_stress_tool/db_stress_test_base.cc @@ -12,6 +12,7 @@ #include #include "rocksdb/options.h" +#include "rocksdb/slice_transform.h" #include "util/compression.h" #ifdef GFLAGS #include "db_stress_tool/db_stress_common.h" @@ -142,6 +143,10 @@ std::shared_ptr StressTest::NewCache(size_t capacity, CompressedSecondaryCacheOptions opts; opts.capacity = FLAGS_compressed_secondary_cache_size; opts.compress_format_version = FLAGS_compress_format_version; + if (FLAGS_enable_do_not_compress_roles) { + opts.do_not_compress_roles = {CacheEntryRoleSet::All()}; + } + opts.enable_custom_split_merge = FLAGS_enable_custom_split_merge; secondary_cache = NewCompressedSecondaryCache(opts); if (secondary_cache == nullptr) { fprintf(stderr, "Failed to allocate compressed secondary cache\n"); @@ -183,6 +188,15 @@ std::shared_ptr StressTest::NewCache(size_t capacity, tiered_opts.cache_type = PrimaryCacheType::kCacheTypeHCC; tiered_opts.total_capacity = cache_size; tiered_opts.compressed_secondary_ratio = 0.5; + tiered_opts.adm_policy = + static_cast(FLAGS_adm_policy); + if (tiered_opts.adm_policy == + TieredAdmissionPolicy::kAdmPolicyThreeQueue) { + CompressedSecondaryCacheOptions nvm_sec_cache_opts; + nvm_sec_cache_opts.capacity = cache_size; + tiered_opts.nvm_sec_cache = + NewCompressedSecondaryCache(nvm_sec_cache_opts); + } block_cache = NewTieredCache(tiered_opts); } else { opts.secondary_cache = std::move(secondary_cache); @@ -203,6 +217,15 @@ std::shared_ptr StressTest::NewCache(size_t capacity, tiered_opts.cache_type = PrimaryCacheType::kCacheTypeLRU; tiered_opts.total_capacity = cache_size; tiered_opts.compressed_secondary_ratio = 0.5; + tiered_opts.adm_policy = + static_cast(FLAGS_adm_policy); + if (tiered_opts.adm_policy == + TieredAdmissionPolicy::kAdmPolicyThreeQueue) { + CompressedSecondaryCacheOptions nvm_sec_cache_opts; + nvm_sec_cache_opts.capacity = cache_size; + tiered_opts.nvm_sec_cache = + NewCompressedSecondaryCache(nvm_sec_cache_opts); + } block_cache = NewTieredCache(tiered_opts); } else { opts.secondary_cache = std::move(secondary_cache); @@ -990,10 +1013,20 @@ void StressTest::OperateDb(ThreadState* thread) { } // Verify GetLiveFiles with a 1 in N chance. - if (thread->rand.OneInOpt(FLAGS_get_live_files_one_in) && + if (thread->rand.OneInOpt(FLAGS_get_live_files_apis_one_in) && !FLAGS_write_fault_one_in) { - Status status = VerifyGetLiveFiles(); - ProcessStatus(shared, "VerifyGetLiveFiles", status); + Status s_1 = VerifyGetLiveFiles(); + ProcessStatus(shared, "VerifyGetLiveFiles", s_1); + Status s_2 = VerifyGetLiveFilesMetaData(); + ProcessStatus(shared, "VerifyGetLiveFilesMetaData", s_2); + Status s_3 = VerifyGetLiveFilesStorageInfo(); + ProcessStatus(shared, "VerifyGetLiveFilesStorageInfo", s_3); + } + + // Verify GetAllColumnFamilyMetaData with a 1 in N chance. + if (thread->rand.OneInOpt(FLAGS_get_all_column_family_metadata_one_in)) { + Status status = VerifyGetAllColumnFamilyMetaData(); + ProcessStatus(shared, "VerifyGetAllColumnFamilyMetaData", status); } // Verify GetSortedWalFiles with a 1 in N chance. @@ -1013,6 +1046,16 @@ void StressTest::OperateDb(ThreadState* thread) { ProcessStatus(shared, "Pause/ContinueBackgroundWork", status); } + if (thread->rand.OneInOpt(FLAGS_disable_file_deletions_one_in)) { + Status status = TestDisableFileDeletions(thread); + ProcessStatus(shared, "TestDisableFileDeletions", status); + } + + if (thread->rand.OneInOpt(FLAGS_disable_manual_compaction_one_in)) { + Status status = TestDisableManualCompaction(thread); + ProcessStatus(shared, "TestDisableManualCompaction", status); + } + if (thread->rand.OneInOpt(FLAGS_verify_checksum_one_in)) { ThreadStatusUtil::SetEnableTracking(FLAGS_enable_thread_tracking); ThreadStatusUtil::SetThreadOperation( @@ -1035,6 +1078,11 @@ void StressTest::OperateDb(ThreadState* thread) { TestGetProperty(thread); } + if (thread->rand.OneInOpt(FLAGS_get_properties_of_all_tables_one_in)) { + Status status = TestGetPropertiesOfAllTables(); + ProcessStatus(shared, "TestGetPropertiesOfAllTables", status); + } + std::vector rand_keys = GenerateKeys(rand_key); if (thread->rand.OneInOpt(FLAGS_ingest_external_file_one_in)) { @@ -1449,13 +1497,34 @@ Status StressTest::TestIterate(ThreadState* thread, return Status::OK(); } -// Test the return status of GetLiveFiles. +// Test the return status of GetLiveFiles() Status StressTest::VerifyGetLiveFiles() const { std::vector live_file; uint64_t manifest_size = 0; return db_->GetLiveFiles(live_file, &manifest_size); } +// Test VerifyGetLiveFilesMetaData() +Status StressTest::VerifyGetLiveFilesMetaData() const { + std::vector live_file_metadata; + db_->GetLiveFilesMetaData(&live_file_metadata); + return Status::OK(); +} + +// Test the return status of GetLiveFilesStorageInfo() +Status StressTest::VerifyGetLiveFilesStorageInfo() const { + std::vector live_file_storage_info; + return db_->GetLiveFilesStorageInfo(LiveFilesStorageInfoOptions(), + &live_file_storage_info); +} + +// Test GetAllColumnFamilyMetaData() +Status StressTest::VerifyGetAllColumnFamilyMetaData() const { + std::vector all_cf_metadata; + db_->GetAllColumnFamilyMetaData(&all_cf_metadata); + return Status::OK(); +} + // Test the return status of GetSortedWalFiles. Status StressTest::VerifyGetSortedWalFiles() const { VectorLogPtr log_ptr; @@ -1718,6 +1787,7 @@ Status StressTest::TestBackupRestore( FLAGS_backup_max_size * 1000000 /* rate_bytes_per_sec */, 1 /* refill_period_us */)); } + backup_opts.current_temperatures_override_manifest = thread->rand.OneIn(2); std::ostringstream backup_opt_oss; backup_opt_oss << "share_table_files: " << backup_opts.share_table_files << ", share_files_with_checksum: " @@ -1730,7 +1800,9 @@ Status StressTest::TestBackupRestore( << ", backup_rate_limiter: " << backup_opts.backup_rate_limiter.get() << ", restore_rate_limiter: " - << backup_opts.restore_rate_limiter.get(); + << backup_opts.restore_rate_limiter.get() + << ", current_temperatures_override_manifest: " + << backup_opts.current_temperatures_override_manifest; std::ostringstream create_backup_opt_oss; std::ostringstream restore_opts_oss; @@ -2281,6 +2353,11 @@ void StressTest::TestGetProperty(ThreadState* thread) const { } } +Status StressTest::TestGetPropertiesOfAllTables() const { + TablePropertiesCollection props; + return db_->GetPropertiesOfAllTables(&props); +} + void StressTest::TestCompactFiles(ThreadState* thread, ColumnFamilyHandle* column_family) { ROCKSDB_NAMESPACE::ColumnFamilyMetaData cf_meta_data; @@ -2378,6 +2455,28 @@ Status StressTest::TestPauseBackground(ThreadState* thread) { return db_->ContinueBackgroundWork(); } +Status StressTest::TestDisableFileDeletions(ThreadState* thread) { + Status status = db_->DisableFileDeletions(); + if (!status.ok()) { + return status; + } + // Similar to TestPauseBackground() + int pwr2_micros = + std::min(thread->rand.Uniform(25), thread->rand.Uniform(25)); + clock_->SleepForMicroseconds(1 << pwr2_micros); + return db_->EnableFileDeletions(); +} + +Status StressTest::TestDisableManualCompaction(ThreadState* thread) { + db_->DisableManualCompaction(); + // Similar to TestPauseBackground() + int pwr2_micros = + std::min(thread->rand.Uniform(25), thread->rand.Uniform(25)); + clock_->SleepForMicroseconds(1 << pwr2_micros); + db_->EnableManualCompaction(); + return Status::OK(); +} + void StressTest::TestAcquireSnapshot(ThreadState* thread, int rand_column_family, const std::string& keystr, uint64_t i) { @@ -2537,7 +2636,8 @@ void StressTest::TestCompactRange(ThreadState* thread, int64_t rand_key, bool non_ok_status_allowed = status.IsManualCompactionPaused() || (status.getState() && std::strstr(status.getState(), "injected")) || - status.IsInvalidArgument() || status.IsNotSupported(); + status.IsAborted() || status.IsInvalidArgument() || + status.IsNotSupported(); fprintf(non_ok_status_allowed ? stdout : stderr, "Unable to perform CompactRange(): %s under specified " "CompactRangeOptions: %s (Empty string or " @@ -2754,6 +2854,8 @@ void StressTest::PrintEnv() const { #endif fprintf(stdout, "Periodic Compaction Secs : %" PRIu64 "\n", FLAGS_periodic_compaction_seconds); + fprintf(stdout, "Daily Offpeak UTC : %s\n", + FLAGS_daily_offpeak_time_utc.c_str()); fprintf(stdout, "Compaction TTL : %" PRIu64 "\n", FLAGS_compaction_ttl); const char* compaction_pri = ""; @@ -3414,6 +3516,7 @@ void InitializeOptionsFromFlags( block_based_options.index_shortening = static_cast( FLAGS_index_shortening); + block_based_options.block_align = FLAGS_block_align; options.table_factory.reset(NewBlockBasedTableFactory(block_based_options)); options.db_write_buffer_size = FLAGS_db_write_buffer_size; options.write_buffer_size = FLAGS_write_buffer_size; @@ -3446,6 +3549,10 @@ void InitializeOptionsFromFlags( options.num_levels = FLAGS_num_levels; if (FLAGS_prefix_size >= 0) { options.prefix_extractor.reset(NewFixedPrefixTransform(FLAGS_prefix_size)); + if (FLAGS_enable_memtable_insert_with_hint_prefix_extractor) { + options.memtable_insert_with_hint_prefix_extractor = + options.prefix_extractor; + } } options.max_open_files = FLAGS_open_files; options.statistics = dbstats; @@ -3497,6 +3604,7 @@ void InitializeOptionsFromFlags( options.experimental_mempurge_threshold = FLAGS_experimental_mempurge_threshold; options.periodic_compaction_seconds = FLAGS_periodic_compaction_seconds; + options.daily_offpeak_time_utc = FLAGS_daily_offpeak_time_utc; options.stats_dump_period_sec = static_cast(FLAGS_stats_dump_period_sec); options.ttl = FLAGS_compaction_ttl; @@ -3572,9 +3680,13 @@ void InitializeOptionsFromFlags( options.wal_compression = StringToCompressionType(FLAGS_wal_compression.c_str()); - if (FLAGS_enable_tiered_storage) { - options.last_level_temperature = Temperature::kCold; - } + options.last_level_temperature = + StringToTemperature(FLAGS_last_level_temperature.c_str()); + options.default_write_temperature = + StringToTemperature(FLAGS_default_write_temperature.c_str()); + options.default_temperature = + StringToTemperature(FLAGS_default_temperature.c_str()); + options.preclude_last_level_data_seconds = FLAGS_preclude_last_level_data_seconds; options.preserve_internal_time_seconds = FLAGS_preserve_internal_time_seconds; @@ -3651,6 +3763,13 @@ void InitializeOptionsFromFlags( FLAGS_hard_pending_compaction_bytes_limit; options.max_sequential_skip_in_iterations = FLAGS_max_sequential_skip_in_iterations; + if (FLAGS_enable_sst_partitioner_factory) { + options.sst_partitioner_factory = std::shared_ptr( + NewSstPartitionerFixedPrefixFactory(1)); + } + options.lowest_used_cache_tier = + static_cast(FLAGS_lowest_used_cache_tier); + options.inplace_update_support = FLAGS_inplace_update_support; } void InitializeOptionsGeneral( diff --git a/db_stress_tool/db_stress_test_base.h b/db_stress_tool/db_stress_test_base.h index f578ae2a0d9..3532f40e571 100644 --- a/db_stress_tool/db_stress_test_base.h +++ b/db_stress_tool/db_stress_test_base.h @@ -206,14 +206,24 @@ class StressTest { Status TestPauseBackground(ThreadState* thread); + Status TestDisableFileDeletions(ThreadState* thread); + + Status TestDisableManualCompaction(ThreadState* thread); + void TestAcquireSnapshot(ThreadState* thread, int rand_column_family, const std::string& keystr, uint64_t i); Status MaybeReleaseSnapshots(ThreadState* thread, uint64_t i); + Status VerifyGetLiveFiles() const; + Status VerifyGetLiveFilesMetaData() const; + Status VerifyGetLiveFilesStorageInfo() const; + Status VerifyGetAllColumnFamilyMetaData() const; + Status VerifyGetSortedWalFiles() const; Status VerifyGetCurrentWalFile() const; void TestGetProperty(ThreadState* thread) const; + Status TestGetPropertiesOfAllTables() const; virtual Status TestApproximateSize( ThreadState* thread, uint64_t iteration, diff --git a/db_stress_tool/db_stress_tool.cc b/db_stress_tool/db_stress_tool.cc index 54147ea9caa..813467efc4c 100644 --- a/db_stress_tool/db_stress_tool.cc +++ b/db_stress_tool/db_stress_tool.cc @@ -249,14 +249,17 @@ int db_stress_tool(int argc, char** argv) { exit(1); } } - if (FLAGS_enable_compaction_filter && + if ((FLAGS_enable_compaction_filter || FLAGS_inplace_update_support) && (FLAGS_acquire_snapshot_one_in > 0 || FLAGS_compact_range_one_in > 0 || FLAGS_iterpercent > 0 || FLAGS_test_batches_snapshots || - FLAGS_test_cf_consistency)) { + FLAGS_test_cf_consistency || FLAGS_check_multiget_consistency || + FLAGS_check_multiget_entity_consistency)) { fprintf( stderr, "Error: acquire_snapshot_one_in, compact_range_one_in, iterpercent, " - "test_batches_snapshots must all be 0 when using compaction filter\n"); + "test_batches_snapshots, test_cf_consistency, " + "check_multiget_consistency, check_multiget_entity_consistency must " + "all be 0 when using compaction filter or inplace update support\n"); exit(1); } if (FLAGS_test_multi_ops_txns) { diff --git a/db_stress_tool/expected_state.cc b/db_stress_tool/expected_state.cc index 91be3bea630..63bb5c1715f 100644 --- a/db_stress_tool/expected_state.cc +++ b/db_stress_tool/expected_state.cc @@ -437,6 +437,28 @@ class ExpectedStateTraceRecordHandler : public TraceRecord::Handler, return Status::OK(); } + Status TimedPutCF(uint32_t column_family_id, const Slice& key_with_ts, + const Slice& value, uint64_t write_unix_time) override { + Slice key = + StripTimestampFromUserKey(key_with_ts, FLAGS_user_timestamp_size); + uint64_t key_id; + if (!GetIntVal(key.ToString(), &key_id)) { + return Status::Corruption("unable to parse key", key.ToString()); + } + uint32_t value_base = GetValueBase(value); + + bool should_buffer_write = !(buffered_writes_ == nullptr); + if (should_buffer_write) { + return WriteBatchInternal::TimedPut(buffered_writes_.get(), + column_family_id, key, value, + write_unix_time); + } + + state_->SyncPut(column_family_id, static_cast(key_id), value_base); + ++num_write_ops_; + return Status::OK(); + } + Status PutEntityCF(uint32_t column_family_id, const Slice& key_with_ts, const Slice& entity) override { Slice key = diff --git a/db_stress_tool/no_batched_ops_stress.cc b/db_stress_tool/no_batched_ops_stress.cc index a94de38971d..4099d312391 100644 --- a/db_stress_tool/no_batched_ops_stress.cc +++ b/db_stress_tool/no_batched_ops_stress.cc @@ -580,13 +580,8 @@ class NonBatchedOpsStressTest : public StressTest { int column_family = rand_column_families[0]; ColumnFamilyHandle* cfh = column_families_[column_family]; int error_count = 0; - // Do a consistency check between Get and MultiGet. Don't do it too - // often as it will slow db_stress down - // - // CompactionFilter can make snapshot non-repeatable by removing keys - // protected by snapshot - bool do_consistency_check = - !FLAGS_enable_compaction_filter && thread->rand.OneIn(4); + + bool do_consistency_check = FLAGS_check_multiget_consistency; ReadOptions readoptionscopy = read_opts; @@ -1075,10 +1070,8 @@ class NonBatchedOpsStressTest : public StressTest { fault_fs_guard->DisableErrorInjection(); } - // CompactionFilter can make snapshot non-repeatable by removing keys - // protected by snapshot - const bool check_get_entity = !FLAGS_enable_compaction_filter && - !error_count && thread->rand.OneIn(4); + const bool check_get_entity = + !error_count && FLAGS_check_multiget_entity_consistency; for (size_t i = 0; i < num_keys; ++i) { const Status& s = statuses[i]; @@ -1311,6 +1304,15 @@ class NonBatchedOpsStressTest : public StressTest { (value_base % FLAGS_use_put_entity_one_in) == 0) { s = db_->PutEntity(write_opts, cfh, k, GenerateWideColumns(value_base, v)); + } else if (FLAGS_use_timed_put_one_in > 0 && + ((value_base + kLargePrimeForCommonFactorSkew) % + FLAGS_use_timed_put_one_in) == 0) { + WriteBatch wb; + uint64_t write_unix_time = GetWriteUnixTime(thread); + s = wb.TimedPut(cfh, k, v, write_unix_time); + if (s.ok()) { + s = db_->Write(write_opts, &wb); + } } else if (FLAGS_use_merge) { if (!FLAGS_use_txn) { if (FLAGS_user_timestamp_size == 0) { diff --git a/env/env.cc b/env/env.cc index 7405f66e08e..77e242a42e3 100644 --- a/env/env.cc +++ b/env/env.cc @@ -825,6 +825,15 @@ WritableFile::~WritableFile() = default; MemoryMappedFileBuffer::~MemoryMappedFileBuffer() = default; +// This const variable can be used in public headers without introducing the +// possibility of ODR violations due to varying macro definitions. +const InfoLogLevel Logger::kDefaultLogLevel = +#ifdef NDEBUG + INFO_LEVEL; +#else + DEBUG_LEVEL; +#endif // NDEBUG + Logger::~Logger() = default; Status Logger::Close() { diff --git a/env/fs_on_demand.cc b/env/fs_on_demand.cc new file mode 100644 index 00000000000..bac424264a3 --- /dev/null +++ b/env/fs_on_demand.cc @@ -0,0 +1,330 @@ +// Copyright (c) 2024-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 "env/fs_on_demand.h" + +#include + +#include "file/filename.h" +#include "port/port.h" +#include "rocksdb/types.h" + +namespace ROCKSDB_NAMESPACE { +// Check if the input path is under orig (typically the local directory), and if +// so, change it to the equivalent path under replace (typically the remote +// directory). For example, if orig is "/data/follower", replace is +// "/data/leader", and the given path is "/data/follower/000010.sst", on return +// the path would be changed to +// "/data/leader/000010.sst". +// Return value is true if the path was modified, false otherwise +bool OnDemandFileSystem::CheckPathAndAdjust(const std::string& orig, + const std::string& replace, + std::string& path) { + size_t pos = path.find(orig); + if (pos > 0) { + return false; + } + path.replace(pos, orig.length(), replace); + return true; +} + +bool OnDemandFileSystem::LookupFileType(const std::string& name, + FileType* type) { + std::size_t found = name.find_last_of('/'); + std::string file_name = name.substr(found); + uint64_t number = 0; + return ParseFileName(file_name, &number, type); +} + +// RocksDB opens non-SST files for reading in sequential file mode. This +// includes CURRENT, OPTIONS, MANIFEST etc. For these files, we open them +// in place in the source directory. For files that are appendable or +// can be renamed, which is MANIFEST and CURRENT files, we wrap the +// underlying FSSequentialFile with another class that checks when EOF +// has been reached and re-opens the file to see the latest data. On some +// distributed file systems, this is necessary. +IOStatus OnDemandFileSystem::NewSequentialFile( + const std::string& fname, const FileOptions& file_opts, + std::unique_ptr* result, IODebugContext* dbg) { + FileType type; + static std::unordered_set valid_types( + {kWalFile, kDescriptorFile, kCurrentFile, kIdentityFile, kOptionsFile}); + if (!LookupFileType(fname, &type) || + (valid_types.find(type) == valid_types.end())) { + return IOStatus::NotSupported(); + } + + IOStatus s; + std::string rname = fname; + if (CheckPathAndAdjust(local_path_, remote_path_, rname)) { + // First clear any local directory cache as it may be out of date + target()->DiscardCacheForDirectory(rname); + + std::unique_ptr inner_file; + s = target()->NewSequentialFile(rname, file_opts, &inner_file, dbg); + if (s.ok() && type == kDescriptorFile) { + result->reset(new OnDemandSequentialFile(std::move(inner_file), this, + file_opts, rname)); + } else { + *result = std::move(inner_file); + } + } else { + s = target()->NewSequentialFile(fname, file_opts, result, dbg); + } + return s; +} + +// This is only supported for SST files. If the file is present locally, +// i.e in the destination dir, we just open it and return. If its in the +// remote, i.e source dir, we link it locally and open the link. +// TODO: Add support for blob files belonging to the new BlobDB +IOStatus OnDemandFileSystem::NewRandomAccessFile( + const std::string& fname, const FileOptions& file_opts, + std::unique_ptr* result, IODebugContext* dbg) { + FileType type; + if (!LookupFileType(fname, &type) || type != kTableFile) { + return IOStatus::NotSupported(); + } + + IOStatus s = target()->FileExists(fname, file_opts.io_options, nullptr); + if (s.IsNotFound() || s.IsPathNotFound()) { + std::string rname = fname; + if (CheckPathAndAdjust(local_path_, remote_path_, rname)) { + // First clear any local directory cache as it may be out of date + target()->DiscardCacheForDirectory(rname); + + s = target()->LinkFile(rname, fname, IOOptions(), nullptr); + if (!s.ok()) { + return s; + } + } + } + + return s.ok() ? target()->NewRandomAccessFile(fname, file_opts, result, dbg) + : s; +} + +// We don't expect to create any writable file other than info LOG files. +IOStatus OnDemandFileSystem::NewWritableFile( + const std::string& fname, const FileOptions& file_opts, + std::unique_ptr* result, IODebugContext* dbg) { + FileType type; + if (!LookupFileType(fname, &type) || type != kInfoLogFile) { + return IOStatus::NotSupported(); + } + + std::string rname = fname; + if (CheckPathAndAdjust(local_path_, remote_path_, rname)) { + // First clear any local directory cache as it may be out of date + target()->DiscardCacheForDirectory(rname); + + IOStatus s = target()->FileExists(rname, file_opts.io_options, dbg); + if (s.ok()) { + return IOStatus::InvalidArgument( + "Writing to a file present in the remote directory not supoprted"); + } + } + + return target()->NewWritableFile(fname, file_opts, result, dbg); +} + +// Currently not supported, as there's no need for RocksDB to create a +// directory object for a DB in read-only mode. +IOStatus OnDemandFileSystem::NewDirectory( + const std::string& /*name*/, const IOOptions& /*io_opts*/, + std::unique_ptr* /*result*/, IODebugContext* /*dbg*/) { + return IOStatus::NotSupported(); +} + +// Check if the given file exists, either locally or remote. If the file is an +// SST file, then link it locally. We assume if the file existence is being +// checked, its for verification purposes, for example while replaying the +// MANIFEST. The file will be opened for reading some time in the future. +IOStatus OnDemandFileSystem::FileExists(const std::string& fname, + const IOOptions& options, + IODebugContext* dbg) { + IOStatus s = target()->FileExists(fname, options, dbg); + if (!s.IsNotFound() && !s.IsPathNotFound()) { + return s; + } + + std::string rname = fname; + if (CheckPathAndAdjust(local_path_, remote_path_, rname)) { + // First clear any local directory cache as it may be out of date + target()->DiscardCacheForDirectory(rname); + + FileType type; + if (LookupFileType(fname, &type) && type == kTableFile) { + s = target()->LinkFile(rname, fname, options, dbg); + } else { + s = target()->FileExists(rname, options, dbg); + } + } + return s; +} + +// Doa listing of both the local and remote directories and merge the two. +IOStatus OnDemandFileSystem::GetChildren(const std::string& dir, + const IOOptions& options, + std::vector* result, + IODebugContext* dbg) { + std::string rdir = dir; + IOStatus s = target()->GetChildren(dir, options, result, dbg); + if (!s.ok() || !CheckPathAndAdjust(local_path_, remote_path_, rdir)) { + return s; + } + + std::vector rchildren; + // First clear any local directory cache as it may be out of date + target()->DiscardCacheForDirectory(rdir); + s = target()->GetChildren(rdir, options, &rchildren, dbg); + if (s.ok()) { + std::for_each(rchildren.begin(), rchildren.end(), [&](std::string& name) { + // Adjust name + (void)CheckPathAndAdjust(remote_path_, local_path_, name); + }); + std::sort(result->begin(), result->end()); + std::sort(rchildren.begin(), rchildren.end()); + + std::vector output; + output.reserve(result->size() + rchildren.size()); + std::set_union(result->begin(), result->end(), rchildren.begin(), + rchildren.end(), std::back_inserter(output)); + *result = std::move(output); + } + return s; +} + +// Doa listing of both the local and remote directories and merge the two. +IOStatus OnDemandFileSystem::GetChildrenFileAttributes( + const std::string& dir, const IOOptions& options, + std::vector* result, IODebugContext* dbg) { + std::string rdir = dir; + IOStatus s = target()->GetChildrenFileAttributes(dir, options, result, dbg); + if (!s.ok() || !CheckPathAndAdjust(local_path_, remote_path_, rdir)) { + return s; + } + + std::vector rchildren; + // First clear any local directory cache as it may be out of date + target()->DiscardCacheForDirectory(rdir); + s = target()->GetChildrenFileAttributes(rdir, options, &rchildren, dbg); + if (s.ok()) { + struct FileAttributeSorter { + bool operator()(const FileAttributes& lhs, const FileAttributes& rhs) { + return lhs.name < rhs.name; + } + } file_attr_sorter; + + std::for_each( + rchildren.begin(), rchildren.end(), [&](FileAttributes& file) { + // Adjust name + (void)CheckPathAndAdjust(remote_path_, local_path_, file.name); + }); + std::sort(result->begin(), result->end(), file_attr_sorter); + std::sort(rchildren.begin(), rchildren.end(), file_attr_sorter); + + std::vector output; + output.reserve(result->size() + rchildren.size()); + std::set_union(rchildren.begin(), rchildren.end(), result->begin(), + result->end(), std::back_inserter(output), file_attr_sorter); + *result = std::move(output); + } + return s; +} + +IOStatus OnDemandFileSystem::GetFileSize(const std::string& fname, + const IOOptions& options, + uint64_t* file_size, + IODebugContext* dbg) { + uint64_t local_size = 0; + IOStatus s = target()->GetFileSize(fname, options, &local_size, dbg); + if (!s.ok() && !s.IsNotFound() && !s.IsPathNotFound()) { + return s; + } + + if (s.IsNotFound() || s.IsPathNotFound()) { + std::string rname = fname; + if (CheckPathAndAdjust(local_path_, remote_path_, rname)) { + // First clear any local directory cache as it may be out of date + target()->DiscardCacheForDirectory(rname); + + FileType type; + if (LookupFileType(fname, &type) && type == kTableFile) { + s = target()->LinkFile(rname, fname, options, dbg); + if (s.ok()) { + s = target()->GetFileSize(fname, options, &local_size, dbg); + } + } else { + s = target()->GetFileSize(rname, options, &local_size, dbg); + } + } + } + *file_size = local_size; + return s; +} + +// An implementation of Read that tracks whether we've reached EOF. If so, +// re-open the file to try to read past the previous EOF offset. After +// re-opening, positing it back to the last read offset. +IOStatus OnDemandSequentialFile::Read(size_t n, const IOOptions& options, + Slice* result, char* scratch, + IODebugContext* dbg) { + IOStatus s; + if (eof_) { + // Reopen the file. With some distributed file systems, this is required + // in order to get the new size + file_.reset(); + s = fs_->NewSequentialFile(path_, file_opts_, &file_, dbg); + if (!s.ok()) { + return IOStatus::IOError("While opening file after relinking, got error ", + s.ToString()); + } + s = file_->Skip(offset_); + if (!s.ok()) { + return IOStatus::IOError( + "While seeking to offset" + std::to_string(offset_) + "got error", + s.ToString()); + } + eof_ = false; + } + + s = file_->Read(n, options, result, scratch, dbg); + if (s.ok()) { + offset_ += result->size(); + if (result->size() < n) { + // We reached EOF. Mark it so we know to relink next time + eof_ = true; + } + } + return s; +} + +IOStatus OnDemandSequentialFile::Skip(uint64_t n) { + IOStatus s = file_->Skip(n); + if (s.ok()) { + offset_ += n; + } + return s; +} + +bool OnDemandSequentialFile::use_direct_io() const { + return file_->use_direct_io(); +} + +size_t OnDemandSequentialFile::GetRequiredBufferAlignment() const { + return file_->GetRequiredBufferAlignment(); +} + +Temperature OnDemandSequentialFile::GetTemperature() const { + return file_->GetTemperature(); +} + +std::shared_ptr NewOnDemandFileSystem( + const std::shared_ptr& fs, std::string src_path, + std::string dest_path) { + return std::make_shared(fs, src_path, dest_path); +} +} // namespace ROCKSDB_NAMESPACE diff --git a/env/fs_on_demand.h b/env/fs_on_demand.h new file mode 100644 index 00000000000..e313103d6c5 --- /dev/null +++ b/env/fs_on_demand.h @@ -0,0 +1,139 @@ +// Copyright (c) 2024-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 + +#include "rocksdb/file_system.h" + +namespace ROCKSDB_NAMESPACE { +// A FileSystem that links files to a local (destination) directory from a +// corresponding remote (source) directory on demand. The decision to link +// depends on the file type, with appendable or rename-able files, such as, +// descriptors, logs, CURRENT, being read in place in the remote directory, +// and SST files being linked. In the future, files read in place may be +// mirrored to the local directory, so the local dir has a complete database +// for troubleshooting purposes. + +class OnDemandFileSystem : public FileSystemWrapper { + public: + OnDemandFileSystem(const std::shared_ptr& target, + const std::string& remote_path, + const std::string& local_path) + : FileSystemWrapper(target), + remote_path_(remote_path), + local_path_(local_path) {} + + const char* Name() const override { return "OnDemandFileSystem"; } + + IOStatus NewSequentialFile(const std::string& fname, + const FileOptions& file_opts, + std::unique_ptr* result, + IODebugContext* dbg) override; + + IOStatus NewRandomAccessFile(const std::string& fname, + const FileOptions& file_opts, + std::unique_ptr* result, + IODebugContext* dbg) override; + + IOStatus NewWritableFile(const std::string& fname, + const FileOptions& file_opts, + std::unique_ptr* result, + IODebugContext* dbg) override; + + IOStatus ReuseWritableFile(const std::string& /*fname*/, + const std::string& /*old_fname*/, + const FileOptions& /*fopts*/, + std::unique_ptr* /*result*/, + IODebugContext* /*dbg*/) override { + return IOStatus::NotSupported("ReuseWritableFile"); + } + + IOStatus NewDirectory(const std::string& name, const IOOptions& io_opts, + std::unique_ptr* result, + IODebugContext* dbg) override; + + IOStatus FileExists(const std::string& fname, const IOOptions& options, + IODebugContext* dbg) override; + + IOStatus GetChildren(const std::string& dir, const IOOptions& options, + std::vector* result, + IODebugContext* dbg) override; + + IOStatus GetChildrenFileAttributes(const std::string& dir, + const IOOptions& options, + std::vector* result, + IODebugContext* dbg) override; + + IOStatus GetFileSize(const std::string& fname, const IOOptions& options, + uint64_t* file_size, IODebugContext* dbg) override; + + private: + bool CheckPathAndAdjust(const std::string& orig, const std::string& replace, + std::string& path); + bool LookupFileType(const std::string& name, FileType* type); + + const std::string remote_path_; + const std::string local_path_; +}; + +// A wrapper class around an FSSequentialFile object. Its mainly +// intended to be used for appendable files like MANIFEST and logs. +// Beneath the covers, it tracks when EOF is reached, and reopens +// the file in order to read the latest appended data. This is +// necessary on some distributed file systems as they may have +// stale metadata about the file. +// TODO: Mirror the data read to a local file for troubleshooting +// purposes, as well as recovery in case the source dir is +// deleted. +class OnDemandSequentialFile : public FSSequentialFile { + public: + OnDemandSequentialFile(std::unique_ptr&& file, + OnDemandFileSystem* fs, const FileOptions& file_opts, + const std::string& path) + : file_(std::move(file)), + fs_(fs), + file_opts_(file_opts), + path_(path), + eof_(false), + offset_(0) {} + + virtual ~OnDemandSequentialFile() {} + + IOStatus Read(size_t n, const IOOptions& options, Slice* result, + char* scratch, IODebugContext* dbg) override; + + IOStatus Skip(uint64_t n) override; + + bool use_direct_io() const override; + + size_t GetRequiredBufferAlignment() const override; + + IOStatus InvalidateCache(size_t /*offset*/, size_t /*length*/) override { + return IOStatus::NotSupported("InvalidateCache not supported."); + } + + IOStatus PositionedRead(uint64_t /*offset*/, size_t /*n*/, + const IOOptions& /*options*/, Slice* /*result*/, + char* /*scratch*/, IODebugContext* /*dbg*/) override { + return IOStatus::NotSupported("PositionedRead"); + } + + Temperature GetTemperature() const override; + + private: + std::unique_ptr file_; + OnDemandFileSystem* fs_; + const FileOptions file_opts_; + const std::string path_; + bool eof_; + uint64_t offset_; +}; + +std::shared_ptr NewOnDemandFileSystem( + const std::shared_ptr& fs, std::string remote_path, + std::string local_path); + +} // namespace ROCKSDB_NAMESPACE diff --git a/file/delete_scheduler.cc b/file/delete_scheduler.cc index 1adbf3846ea..203a3265307 100644 --- a/file/delete_scheduler.cc +++ b/file/delete_scheduler.cc @@ -60,10 +60,10 @@ DeleteScheduler::~DeleteScheduler() { Status DeleteScheduler::DeleteFile(const std::string& file_path, const std::string& dir_to_sync, const bool force_bg) { + uint64_t total_size = sst_file_manager_->GetTotalSize(); if (rate_bytes_per_sec_.load() <= 0 || (!force_bg && - total_trash_size_.load() > - sst_file_manager_->GetTotalSize() * max_trash_db_ratio_.load())) { + total_trash_size_.load() > total_size * max_trash_db_ratio_.load())) { // Rate limiting is disabled or trash size makes up more than // max_trash_db_ratio_ (default 25%) of the total DB size TEST_SYNC_POINT("DeleteScheduler::DeleteFile"); @@ -72,9 +72,11 @@ Status DeleteScheduler::DeleteFile(const std::string& file_path, s = sst_file_manager_->OnDeleteFile(file_path); ROCKS_LOG_INFO(info_log_, "Deleted file %s immediately, rate_bytes_per_sec %" PRIi64 - ", total_trash_size %" PRIu64 " max_trash_db_ratio %lf", + ", total_trash_size %" PRIu64 ", total_size %" PRIi64 + ", max_trash_db_ratio %lf", file_path.c_str(), rate_bytes_per_sec_.load(), - total_trash_size_.load(), max_trash_db_ratio_.load()); + total_trash_size_.load(), total_size, + max_trash_db_ratio_.load()); InstrumentedMutexLock l(&mu_); RecordTick(stats_.get(), FILES_DELETED_IMMEDIATELY); } diff --git a/file/file_util.h b/file/file_util.h index 7bd536756d3..af6106cf121 100644 --- a/file/file_util.h +++ b/file/file_util.h @@ -46,6 +46,11 @@ inline IOStatus CreateFile(const std::shared_ptr& fs, return CreateFile(fs.get(), destination, contents, use_fsync); } +// Delete a DB file, if this file is a SST file or Blob file and SstFileManager +// is used, it should have already been tracked by SstFileManager via its +// `OnFileAdd` API before passing to this API to be deleted, to ensure +// SstFileManager and its DeleteScheduler are tracking DB size and trash size +// properly. Status DeleteDBFile(const ImmutableDBOptions* db_options, const std::string& fname, const std::string& path_to_sync, const bool force_bg, const bool force_fg); diff --git a/file/sequence_file_reader.cc b/file/sequence_file_reader.cc index ac2f37b0d0d..82432fa0f1c 100644 --- a/file/sequence_file_reader.cc +++ b/file/sequence_file_reader.cc @@ -41,6 +41,7 @@ IOStatus SequentialFileReader::Read(size_t n, Slice* result, char* scratch, IOStatus io_s; IOOptions io_opts; io_opts.rate_limiter_priority = rate_limiter_priority; + io_opts.verify_and_reconstruct_read = verify_and_reconstruct_read_; if (use_direct_io()) { // // |-offset_advance-|---bytes returned--| diff --git a/file/sequence_file_reader.h b/file/sequence_file_reader.h index dc0e61bd2a1..187adefb610 100644 --- a/file/sequence_file_reader.h +++ b/file/sequence_file_reader.h @@ -56,6 +56,7 @@ class SequentialFileReader { std::atomic offset_{0}; // read offset std::vector> listeners_{}; RateLimiter* rate_limiter_; + bool verify_and_reconstruct_read_; public: explicit SequentialFileReader( @@ -63,11 +64,13 @@ class SequentialFileReader { const std::shared_ptr& io_tracer = nullptr, const std::vector>& listeners = {}, RateLimiter* rate_limiter = - nullptr) // TODO: migrate call sites to provide rate limiter + nullptr, // TODO: migrate call sites to provide rate limiter + bool verify_and_reconstruct_read = false) : file_name_(_file_name), file_(std::move(_file), io_tracer, _file_name), listeners_(), - rate_limiter_(rate_limiter) { + rate_limiter_(rate_limiter), + verify_and_reconstruct_read_(verify_and_reconstruct_read) { AddFileIOListeners(listeners); } @@ -77,12 +80,14 @@ class SequentialFileReader { const std::shared_ptr& io_tracer = nullptr, const std::vector>& listeners = {}, RateLimiter* rate_limiter = - nullptr) // TODO: migrate call sites to provide rate limiter + nullptr, // TODO: migrate call sites to provide rate limiter + bool verify_and_reconstruct_read = false) : file_name_(_file_name), file_(NewReadaheadSequentialFile(std::move(_file), _readahead_size), io_tracer, _file_name), listeners_(), - rate_limiter_(rate_limiter) { + rate_limiter_(rate_limiter), + verify_and_reconstruct_read_(verify_and_reconstruct_read) { AddFileIOListeners(listeners); } static IOStatus Create(const std::shared_ptr& fs, diff --git a/file/sst_file_manager_impl.h b/file/sst_file_manager_impl.h index 75aa4fe55a3..a45663f8f16 100644 --- a/file/sst_file_manager_impl.h +++ b/file/sst_file_manager_impl.h @@ -30,6 +30,10 @@ class SstFileManagerImpl : public SstFileManager { double max_trash_db_ratio, uint64_t bytes_max_delete_chunk); + // No copy + SstFileManagerImpl(const SstFileManagerImpl& sfm) = delete; + SstFileManagerImpl& operator=(const SstFileManagerImpl& sfm) = delete; + ~SstFileManagerImpl(); // DB will call OnAddFile whenever a new sst/blob file is added. diff --git a/file/writable_file_writer.cc b/file/writable_file_writer.cc index 4fadf1d71a3..d36023c0e00 100644 --- a/file/writable_file_writer.cc +++ b/file/writable_file_writer.cc @@ -205,7 +205,6 @@ IOStatus WritableFileWriter::Pad(const IOOptions& opts, assert(pad_bytes < kDefaultPageSize); size_t left = pad_bytes; size_t cap = buf_.Capacity() - buf_.CurrentSize(); - size_t pad_start = buf_.CurrentSize(); // Assume pad_bytes is small compared to buf_ capacity. So we always // use buf_ rather than write directly to file in certain cases like @@ -214,6 +213,16 @@ IOStatus WritableFileWriter::Pad(const IOOptions& opts, size_t append_bytes = std::min(cap, left); buf_.PadWith(append_bytes, 0); left -= append_bytes; + + Slice data(buf_.BufferStart() + buf_.CurrentSize() - append_bytes, + append_bytes); + UpdateFileChecksum(data); + if (perform_data_verification_) { + buffered_data_crc32c_checksum_ = crc32c::Extend( + buffered_data_crc32c_checksum_, + buf_.BufferStart() + buf_.CurrentSize() - append_bytes, append_bytes); + } + if (left > 0) { IOStatus s = Flush(io_options); if (!s.ok()) { @@ -226,11 +235,7 @@ IOStatus WritableFileWriter::Pad(const IOOptions& opts, pending_sync_ = true; uint64_t cur_size = filesize_.load(std::memory_order_acquire); filesize_.store(cur_size + pad_bytes, std::memory_order_release); - if (perform_data_verification_) { - buffered_data_crc32c_checksum_ = - crc32c::Extend(buffered_data_crc32c_checksum_, - buf_.BufferStart() + pad_start, pad_bytes); - } + return IOStatus::OK(); } diff --git a/include/rocksdb/advanced_options.h b/include/rocksdb/advanced_options.h index fdf2af058de..80933b14977 100644 --- a/include/rocksdb/advanced_options.h +++ b/include/rocksdb/advanced_options.h @@ -229,11 +229,18 @@ struct AdvancedColumnFamilyOptions { // if it is not explicitly set by the user. Otherwise, the default is 0. int64_t max_write_buffer_size_to_maintain = 0; - // Allows thread-safe inplace updates. If this is true, there is no way to + // Allows thread-safe inplace updates. + // + // If this is true, there is no way to // achieve point-in-time consistency using snapshot or iterator (assuming // concurrent updates). Hence iterator and multi-get will return results // which are not consistent as of any point-in-time. + // // Backward iteration on memtables will not work either. + // + // It is intended to work or be compatible with a limited set of features: + // (1) Non-snapshot Get() + // // If inplace_callback function is not set, // Put(key, new_value) will update inplace the existing_value iff // * key exists in current memtable diff --git a/include/rocksdb/attribute_groups.h b/include/rocksdb/attribute_groups.h new file mode 100644 index 00000000000..bbc62170356 --- /dev/null +++ b/include/rocksdb/attribute_groups.h @@ -0,0 +1,114 @@ +// Copyright (c) Meta Platforms, Inc. and affiliates. +// 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 "rocksdb/iterator_base.h" +#include "rocksdb/wide_columns.h" + +namespace ROCKSDB_NAMESPACE { + +class ColumnFamilyHandle; + +// Class representing attribute group. Attribute group is a logical grouping of +// wide-column entities by leveraging Column Families. +// Used in Write Path +class AttributeGroup { + public: + explicit AttributeGroup(ColumnFamilyHandle* column_family, + const WideColumns& columns) + : column_family_(column_family), columns_(columns) {} + + ColumnFamilyHandle* column_family() const { return column_family_; } + const WideColumns& columns() const { return columns_; } + WideColumns& columns() { return columns_; } + + private: + ColumnFamilyHandle* column_family_; + WideColumns columns_; +}; + +inline bool operator==(const AttributeGroup& lhs, const AttributeGroup& rhs) { + return lhs.column_family() == rhs.column_family() && + lhs.columns() == rhs.columns(); +} + +// A collection of Attribute Groups. +using AttributeGroups = std::vector; + +// An empty set of Attribute Groups. +extern const AttributeGroups kNoAttributeGroups; + +// Used in Read Path. Wide-columns returned from the query are pinnable. +class PinnableAttributeGroup { + public: + explicit PinnableAttributeGroup(ColumnFamilyHandle* column_family) + : column_family_(column_family), status_(Status::OK()) {} + + ColumnFamilyHandle* column_family() const { return column_family_; } + const Status& status() const { return status_; } + const WideColumns& columns() const { return columns_.columns(); } + + void SetStatus(const Status& status); + void SetColumns(PinnableWideColumns&& columns); + + void Reset(); + + private: + ColumnFamilyHandle* column_family_; + Status status_; + PinnableWideColumns columns_; +}; + +inline void PinnableAttributeGroup::SetStatus(const Status& status) { + status_ = status; +} +inline void PinnableAttributeGroup::SetColumns(PinnableWideColumns&& columns) { + columns_ = std::move(columns); +} + +inline void PinnableAttributeGroup::Reset() { + SetStatus(Status::OK()); + columns_.Reset(); +} + +// A collection of Pinnable Attribute Groups. +using PinnableAttributeGroups = std::vector; + +// Used in Iterator Path. Uses pointers to the columns to avoid having to copy +// all WideColumns objs during iteration. +class IteratorAttributeGroup { + public: + explicit IteratorAttributeGroup(ColumnFamilyHandle* column_family, + const WideColumns* columns) + : column_family_(column_family), columns_(columns) {} + ColumnFamilyHandle* column_family() const { return column_family_; } + const WideColumns& columns() const { return *columns_; } + + private: + ColumnFamilyHandle* column_family_; + const WideColumns* columns_; +}; + +using IteratorAttributeGroups = std::vector; + +extern const IteratorAttributeGroups kNoIteratorAttributeGroups; + +// EXPERIMENTAL +// A cross-column-family iterator that collects and returns attribute groups for +// each key in order provided by comparator +class AttributeGroupIterator : public IteratorBase { + public: + AttributeGroupIterator() {} + ~AttributeGroupIterator() override {} + + // No copy allowed + AttributeGroupIterator(const AttributeGroupIterator&) = delete; + AttributeGroupIterator& operator=(const AttributeGroupIterator&) = delete; + + virtual const IteratorAttributeGroups& attribute_groups() const = 0; +}; + +} // namespace ROCKSDB_NAMESPACE diff --git a/include/rocksdb/comparator.h b/include/rocksdb/comparator.h index 4b39a25851a..087610efd5e 100644 --- a/include/rocksdb/comparator.h +++ b/include/rocksdb/comparator.h @@ -120,6 +120,28 @@ class Comparator : public Customizable, public CompareInterface { inline size_t timestamp_size() const { return timestamp_size_; } + // Return what this Comparator considers as the maximum timestamp. + // The default implementation only works for when `timestamp_size_` is 0, + // subclasses for which this is not the case needs to override this function. + virtual Slice GetMaxTimestamp() const { + if (timestamp_size_ == 0) { + return Slice(); + } + assert(false); + return Slice(); + } + + // Return what this Comparator considers as the min timestamp. + // The default implementation only works for when `timestamp_size_` is 0, + // subclasses for which this is not the case needs to override this function. + virtual Slice GetMinTimestamp() const { + if (timestamp_size_ == 0) { + return Slice(); + } + assert(false); + return Slice(); + } + int CompareWithoutTimestamp(const Slice& a, const Slice& b) const { return CompareWithoutTimestamp(a, /*a_has_ts=*/true, b, /*b_has_ts=*/true); } diff --git a/include/rocksdb/db.h b/include/rocksdb/db.h index d31408e681b..ecfae5b6939 100644 --- a/include/rocksdb/db.h +++ b/include/rocksdb/db.h @@ -17,6 +17,7 @@ #include #include +#include "rocksdb/attribute_groups.h" #include "rocksdb/block_cache_trace_writer.h" #include "rocksdb/iterator.h" #include "rocksdb/listener.h" @@ -297,6 +298,29 @@ class DB { const std::vector& column_families, std::vector* handles, DB** dbptr); + // EXPERIMENTAL + + // Open a database as a follower. The difference between this and opening + // as secondary is that the follower database has its own directory with + // links to the actual files, and can tolarate obsolete file deletions by + // the leader to its own database. Another difference is the follower + // tries to keep up with the leader by periodically tailing the leader's + // MANIFEST, and (in the future) memtable updates, rather than relying on + // the user to manually call TryCatchupWithPrimary(). + + // Open as a follower with the default column family + static Status OpenAsFollower(const Options& options, const std::string& name, + const std::string& leader_path, + std::unique_ptr* dbptr); + + // Open as a follower with multiple column families + static Status OpenAsFollower( + const DBOptions& db_options, const std::string& name, + const std::string& leader_path, + const std::vector& column_families, + std::vector* handles, std::unique_ptr* dbptr); + // End EXPERIMENTAL + // Open DB and run the compaction. // It's a read-only operation, the result won't be installed to the DB, it // will be output to the `output_directory`. The API should only be used with @@ -517,6 +541,8 @@ class DB { // 2) Limiting the maximum number of open files in the presence of range // tombstones can degrade read performance. To avoid this problem, set // max_open_files to -1 whenever possible. + // 3) Incompatible with row_cache, will return Status::NotSupported() if + // row_cache is configured. virtual Status DeleteRange(const WriteOptions& options, ColumnFamilyHandle* column_family, const Slice& begin_key, const Slice& end_key); @@ -971,12 +997,28 @@ class DB { const std::vector& column_families, std::vector* iterators) = 0; - // UNDER CONSTRUCTION - DO NOT USE + // EXPERIMENTAL // Return a cross-column-family iterator from a consistent database state. - // When the same key is present in multiple column families, the iterator - // selects the value or columns from the first column family containing the - // key, in the order specified by the `column_families` parameter. - virtual std::unique_ptr NewMultiCfIterator( + // + // If a key exists in more than one column family, value() will be determined + // by the wide column value of kDefaultColumnName after coalesced as described + // below. + // + // Each wide column will be independently shadowed by the CFs. + // For example, if CF1 has "key_1" ==> {"col_1": "foo", + // "col_2", "baz"} and CF2 has "key_1" ==> {"col_2": "quux", "col_3", "bla"}, + // and when the iterator is at key_1, columns() will return + // {"col_1": "foo", "col_2", "quux", "col_3", "bla"} + // In this example, value() will be empty, because none of them have values + // for kDefaultColumnName + virtual std::unique_ptr NewCoalescingIterator( + const ReadOptions& options, + const std::vector& column_families) = 0; + + // EXPERIMENTAL + // A cross-column-family iterator that collects and returns attribute groups + // for each key in order provided by comparator + virtual std::unique_ptr NewAttributeGroupIterator( const ReadOptions& options, const std::vector& column_families) = 0; diff --git a/include/rocksdb/env.h b/include/rocksdb/env.h index d81960c437f..0be90ca2a3e 100644 --- a/include/rocksdb/env.h +++ b/include/rocksdb/env.h @@ -1209,6 +1209,10 @@ class Logger { public: static constexpr size_t kDoNotSupportGetLogFileSize = SIZE_MAX; + // Set to INFO_LEVEL when RocksDB is compiled in release mode, and + // DEBUG_LEVEL when compiled in debug mode. See DBOptions::info_log_level. + static const InfoLogLevel kDefaultLogLevel; + explicit Logger(const InfoLogLevel log_level = InfoLogLevel::INFO_LEVEL) : closed_(false), log_level_(log_level) {} // No copying allowed diff --git a/include/rocksdb/file_system.h b/include/rocksdb/file_system.h index d56eed1adce..8d21c919466 100644 --- a/include/rocksdb/file_system.h +++ b/include/rocksdb/file_system.h @@ -702,6 +702,16 @@ class FileSystem : public Customizable { return IOStatus::OK(); } + // EXPERIMENTAL + // Discard any directory metadata cached in memory for the specified + // directory and its descendants. Useful for distributed file systems + // where the local cache may be out of sync with the actual directory state. + // + // The implementation is not required to be thread safe. Its the caller's + // responsibility to ensure that no directory operations happen + // concurrently. + virtual void DiscardCacheForDirectory(const std::string& /*path*/) {} + // Indicates to upper layers which FileSystem operations mentioned in // FSSupportedOps are supported by underlying FileSystem. Each bit in // supported_ops argument represent corresponding FSSupportedOps operation. @@ -1624,6 +1634,10 @@ class FileSystemWrapper : public FileSystem { return target_->AbortIO(io_handles); } + void DiscardCacheForDirectory(const std::string& path) override { + target_->DiscardCacheForDirectory(path); + } + void SupportedOps(int64_t& supported_ops) override { return target_->SupportedOps(supported_ops); } diff --git a/include/rocksdb/iterator.h b/include/rocksdb/iterator.h index 0cddf4a3346..0334aa7511e 100644 --- a/include/rocksdb/iterator.h +++ b/include/rocksdb/iterator.h @@ -20,14 +20,12 @@ #include -#include "rocksdb/cleanable.h" -#include "rocksdb/slice.h" -#include "rocksdb/status.h" +#include "rocksdb/iterator_base.h" #include "rocksdb/wide_columns.h" namespace ROCKSDB_NAMESPACE { -class Iterator : public Cleanable { +class Iterator : public IteratorBase { public: Iterator() {} // No copying allowed @@ -36,51 +34,6 @@ class Iterator : public Cleanable { virtual ~Iterator() {} - // An iterator is either positioned at a key/value pair, or - // not valid. This method returns true iff the iterator is valid. - // Always returns false if !status().ok(). - virtual bool Valid() const = 0; - - // Position at the first key in the source. The iterator is Valid() - // after this call iff the source is not empty. - virtual void SeekToFirst() = 0; - - // Position at the last key in the source. The iterator is - // Valid() after this call iff the source is not empty. - virtual void SeekToLast() = 0; - - // Position at the first key in the source that at or past target. - // The iterator is Valid() after this call iff the source contains - // an entry that comes at or past target. - // All Seek*() methods clear any error status() that the iterator had prior to - // the call; after the seek, status() indicates only the error (if any) that - // happened during the seek, not any past errors. - // Target does not contain timestamp. - virtual void Seek(const Slice& target) = 0; - - // Position at the last key in the source that at or before target. - // The iterator is Valid() after this call iff the source contains - // an entry that comes at or before target. - // Target does not contain timestamp. - virtual void SeekForPrev(const Slice& target) = 0; - - // Moves to the next entry in the source. After this call, Valid() is - // true iff the iterator was not positioned at the last entry in the source. - // REQUIRES: Valid() - virtual void Next() = 0; - - // Moves to the previous entry in the source. After this call, Valid() is - // true iff the iterator was not positioned at the first entry in source. - // REQUIRES: Valid() - virtual void Prev() = 0; - - // Return the key for the current entry. The underlying storage for - // the returned slice is valid only until the next modification of the - // iterator (i.e. the next SeekToFirst/SeekToLast/Seek/SeekForPrev/Next/Prev - // operation). - // REQUIRES: Valid() - virtual Slice key() const = 0; - // Return the value for the current entry. If the entry is a plain key-value, // return the value as-is; if it is a wide-column entity, return the value of // the default anonymous column (see kDefaultWideColumnName) if any, or an @@ -102,11 +55,6 @@ class Iterator : public Cleanable { return kNoWideColumns; } - // If an error has occurred, return it. Else return an ok status. - // If non-blocking IO is requested and this operation cannot be - // satisfied without doing some IO, then this returns Status::Incomplete(). - virtual Status status() const = 0; - // If supported, the DB state that the iterator reads from is updated to // the latest state. The iterator will be invalidated after the call. // Regardless of whether the iterator was created/refreshed previously diff --git a/include/rocksdb/iterator_base.h b/include/rocksdb/iterator_base.h new file mode 100644 index 00000000000..036936e2b67 --- /dev/null +++ b/include/rocksdb/iterator_base.h @@ -0,0 +1,74 @@ +// Copyright (c) Meta Platforms, Inc. and affiliates. +// 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 "rocksdb/cleanable.h" +#include "rocksdb/slice.h" +#include "rocksdb/status.h" + +namespace ROCKSDB_NAMESPACE { + +class IteratorBase : public Cleanable { + public: + IteratorBase() {} + // No copying allowed + IteratorBase(const IteratorBase&) = delete; + void operator=(const IteratorBase&) = delete; + + virtual ~IteratorBase() {} + + // An iterator is either positioned at a key/value pair, or + // not valid. This method returns true iff the iterator is valid. + // Always returns false if !status().ok(). + virtual bool Valid() const = 0; + + // Position at the first key in the source. The iterator is Valid() + // after this call iff the source is not empty. + virtual void SeekToFirst() = 0; + + // Position at the last key in the source. The iterator is + // Valid() after this call iff the source is not empty. + virtual void SeekToLast() = 0; + + // Position at the first key in the source that at or past target. + // The iterator is Valid() after this call iff the source contains + // an entry that comes at or past target. + // All Seek*() methods clear any error status() that the iterator had prior to + // the call; after the seek, status() indicates only the error (if any) that + // happened during the seek, not any past errors. + // Target does not contain timestamp. + virtual void Seek(const Slice& target) = 0; + + // Position at the last key in the source that at or before target. + // The iterator is Valid() after this call iff the source contains + // an entry that comes at or before target. + // Target does not contain timestamp. + virtual void SeekForPrev(const Slice& target) = 0; + + // Moves to the next entry in the source. After this call, Valid() is + // true iff the iterator was not positioned at the last entry in the source. + // REQUIRES: Valid() + virtual void Next() = 0; + + // Moves to the previous entry in the source. After this call, Valid() is + // true iff the iterator was not positioned at the first entry in source. + // REQUIRES: Valid() + virtual void Prev() = 0; + + // Return the key for the current entry. The underlying storage for + // the returned slice is valid only until the next modification of the + // iterator (i.e. the next SeekToFirst/SeekToLast/Seek/SeekForPrev/Next/Prev + // operation). + // REQUIRES: Valid() + virtual Slice key() const = 0; + + // If an error has occurred, return it. Else return an ok status. + // If non-blocking IO is requested and this operation cannot be + // satisfied without doing some IO, then this returns Status::Incomplete(). + virtual Status status() const = 0; +}; + +} // namespace ROCKSDB_NAMESPACE diff --git a/include/rocksdb/options.h b/include/rocksdb/options.h index 93531be0706..459ba1d5172 100644 --- a/include/rocksdb/options.h +++ b/include/rocksdb/options.h @@ -643,11 +643,10 @@ struct DBOptions { // Default: nullptr std::shared_ptr info_log = nullptr; -#ifdef NDEBUG - InfoLogLevel info_log_level = INFO_LEVEL; -#else - InfoLogLevel info_log_level = DEBUG_LEVEL; -#endif // NDEBUG + // Minimum level for sending log messages to info_log. The default is + // INFO_LEVEL when RocksDB is compiled in release mode, and DEBUG_LEVEL + // when it is compiled in debug mode. + InfoLogLevel info_log_level = Logger::kDefaultLogLevel; // Number of open files that can be used by the DB. You may need to // increase this if your database has a large working set. Value -1 means @@ -1053,6 +1052,9 @@ struct DBOptions { uint64_t bytes_per_sync = 0; // Same as bytes_per_sync, but applies to WAL files + // This does not gaurantee the WALs are synced in the order of creation. New + // WAL can be synced while an older WAL doesn't. Therefore upon system crash, + // this hole in the WAL data can create partial data loss. // // Default: 0, turned off // @@ -1220,6 +1222,8 @@ struct DBOptions { bool allow_2pc = false; // A global cache for table-level rows. + // Used to speed up Get() queries. + // NOTE: does not work with DeleteRange() yet. // Default: nullptr (disabled) std::shared_ptr row_cache = nullptr; @@ -1484,6 +1488,30 @@ struct DBOptions { // use "0:00-23:59". To make an entire day have no offpeak period, leave // this field blank. Default: Empty string (no offpeak). std::string daily_offpeak_time_utc = ""; + + // EXPERIMENTAL + + // When a RocksDB database is opened in follower mode, this option + // is set by the user to request the frequency of the follower + // attempting to refresh its view of the leader. RocksDB may choose to + // trigger catch ups more frequently if it detects any changes in the + // database state. + // Default every 10s. + uint64_t follower_refresh_catchup_period_ms = 10000; + + // For a given catch up attempt, this option specifies the number of times + // to tail the MANIFEST and try to install a new, consistent version before + // giving up. Though it should be extremely rare, the catch up may fail if + // the leader is mutating the LSM at a very high rate and the follower is + // unable to get a consistent view. + // Default to 10 attempts + uint64_t follower_catchup_retry_count = 10; + + // Time to wait between consecutive catch up attempts + // Default 100ms + uint64_t follower_catchup_retry_wait_ms = 100; + + // End EXPERIMENTAL }; // Options to control the behavior of a database (passed to DB::Open) @@ -1908,20 +1936,29 @@ Status CreateLoggerFromOptions(const std::string& dbname, // CompactionOptions are used in CompactFiles() call. struct CompactionOptions { + // DEPRECATED: this option is unsafe because it allows the user to set any + // `CompressionType` while always using `CompressionOptions` from the + // `ColumnFamilyOptions`. As a result the `CompressionType` and + // `CompressionOptions` can easily be inconsistent. + // // Compaction output compression type - // Default: snappy + // + // Default: `kDisableCompressionOption` + // // If set to `kDisableCompressionOption`, RocksDB will choose compression type - // according to the `ColumnFamilyOptions`, taking into account the output - // level if `compression_per_level` is specified. + // according to the `ColumnFamilyOptions`. RocksDB takes into account the + // output level in case the `ColumnFamilyOptions` has level-specific settings. CompressionType compression; + // Compaction will create files of size `output_file_size_limit`. // Default: MAX, which means that compaction will create a single file uint64_t output_file_size_limit; + // If > 0, it will replace the option in the DBOptions for this compaction. uint32_t max_subcompactions; CompactionOptions() - : compression(kSnappyCompression), + : compression(kDisableCompressionOption), output_file_size_limit(std::numeric_limits::max()), max_subcompactions(0) {} }; @@ -2213,6 +2250,9 @@ struct WaitForCompactOptions { // A boolean to flush all column families before starting to wait. bool flush = false; + // A boolean to wait for purge to complete + bool wait_for_purge = false; + // A boolean to call Close() after waiting is done. By the time Close() is // called here, there should be no background jobs in progress and no new // background jobs should be added. DB may not have been closed if Close() diff --git a/include/rocksdb/perf_context.h b/include/rocksdb/perf_context.h index 80792131cf7..42452be1517 100644 --- a/include/rocksdb/perf_context.h +++ b/include/rocksdb/perf_context.h @@ -280,7 +280,7 @@ struct PerfContextBase { struct PerfContext : public PerfContextBase { ~PerfContext(); - PerfContext() {} + PerfContext() { Reset(); } PerfContext(const PerfContext&); PerfContext& operator=(const PerfContext&); diff --git a/include/rocksdb/sst_file_reader.h b/include/rocksdb/sst_file_reader.h index 4e5cda130a2..b588d9f42a4 100644 --- a/include/rocksdb/sst_file_reader.h +++ b/include/rocksdb/sst_file_reader.h @@ -24,11 +24,21 @@ class SstFileReader { // Prepares to read from the file located at "file_path". Status Open(const std::string& file_path); - // Returns a new iterator over the table contents. + // Returns a new iterator over the table contents as a DB iterator, a.k.a + // a `DBIter` that iterates logically visible entries, for example, a delete + // entry is not logically visible. // Most read options provide the same control as we read from DB. // If "snapshot" is nullptr, the iterator returns only the latest keys. Iterator* NewIterator(const ReadOptions& options); + // Returns a new iterator over the table contents as a raw table iterator, + // a.k.a a `TableIterator`that iterates all point data entries in the table + // including logically invisible entries like delete entries. + // This API is intended to provide a programmatic way to observe SST files + // created by a DB, to be used by third party tools. DB optimization + // capabilities like filling cache, read ahead are disabled. + std::unique_ptr NewTableIterator(); + std::shared_ptr GetTableProperties() const; // Verifies whether there is corruption in this table. diff --git a/include/rocksdb/table.h b/include/rocksdb/table.h index b0b276a6396..a801a3349a0 100644 --- a/include/rocksdb/table.h +++ b/include/rocksdb/table.h @@ -426,12 +426,12 @@ struct BlockBasedTableOptions { // the block cache better at using space it is allowed. (These issues // should not arise with partitioned filters.) // - // NOTE: Do not set to true if you do not trust malloc_usable_size. With - // this option, RocksDB might access an allocated memory object beyond its - // original size if malloc_usable_size says it is safe to do so. While this - // can be considered bad practice, it should not produce undefined behavior - // unless malloc_usable_size is buggy or broken. - bool optimize_filters_for_memory = false; + // NOTE: Set to false if you do not trust malloc_usable_size. When set to + // true, RocksDB might access an allocated memory object beyond its original + // size if malloc_usable_size says it is safe to do so. While this can be + // considered bad practice, it should not produce undefined behavior unless + // malloc_usable_size is buggy or broken. + bool optimize_filters_for_memory = true; // Use delta encoding to compress keys in blocks. // ReadOptions::pin_data requires this option to be disabled. diff --git a/include/rocksdb/types.h b/include/rocksdb/types.h index c9c2146865a..3b9791c4824 100644 --- a/include/rocksdb/types.h +++ b/include/rocksdb/types.h @@ -70,6 +70,17 @@ enum EntryType { kEntryOther, }; +// Structured user-oriented representation of an internal key. It includes user +// key, sequence number, and type. +// If user-defined timestamp is enabled, `timestamp` contains the user-defined +// timestamp, it's otherwise an empty Slice. +struct ParsedEntryInfo { + Slice user_key; + Slice timestamp; + SequenceNumber sequence; + EntryType type; +}; + enum class WriteStallCause { // Beginning of CF-scope write stall causes // diff --git a/include/rocksdb/utilities/cache_dump_load.h b/include/rocksdb/utilities/cache_dump_load.h index 8b91bb7e15b..8f41839cd9d 100644 --- a/include/rocksdb/utilities/cache_dump_load.h +++ b/include/rocksdb/utilities/cache_dump_load.h @@ -67,6 +67,10 @@ class CacheDumpReader { // dump or load process related control variables can be added here. struct CacheDumpOptions { SystemClock* clock; + // Deadline for dumper or loader in microseconds + std::chrono::microseconds deadline = std::chrono::microseconds::zero(); + // Max size bytes for dumper or loader + uint64_t max_size_bytes = 0; }; // NOTE that: this class is EXPERIMENTAL! May be changed in the future! diff --git a/include/rocksdb/utilities/env_mirror.h b/include/rocksdb/utilities/env_mirror.h index 01293f0d0cc..0172b20a7e2 100644 --- a/include/rocksdb/utilities/env_mirror.h +++ b/include/rocksdb/utilities/env_mirror.h @@ -83,7 +83,7 @@ class EnvMirror : public EnvWrapper { std::sort(ar.begin(), ar.end()); std::sort(br.begin(), br.end()); if (!as.ok() || ar != br) { - assert(0 == "getchildren results don't match"); + assert(nullptr == "getchildren results don't match"); } *r = ar; return as; diff --git a/include/rocksdb/utilities/option_change_migration.h b/include/rocksdb/utilities/option_change_migration.h index a73324a9e34..0ad00cc860e 100644 --- a/include/rocksdb/utilities/option_change_migration.h +++ b/include/rocksdb/utilities/option_change_migration.h @@ -15,10 +15,10 @@ namespace ROCKSDB_NAMESPACE { // Multiple column families is not supported. // It is best-effort. No guarantee to succeed. // A full compaction may be executed. -// If the target options use FIFO compaction, the FIFO condition might be -// sacrificed: for data migrated, data inserted later might be dropped -// earlier. This is to gurantee FIFO compaction won't drop all the -// migrated data to fit max_table_files_size. +// WARNING: using this to migrate from non-FIFO to FIFO compaction +// with `Options::compaction_options_fifo.max_table_files_size` > 0 can cause +// the whole DB to be dropped right after migration if the migrated data is +// larger than `max_table_files_size` Status OptionChangeMigration(std::string dbname, const Options& old_opts, const Options& new_opts); } // namespace ROCKSDB_NAMESPACE diff --git a/include/rocksdb/utilities/stackable_db.h b/include/rocksdb/utilities/stackable_db.h index 0aa6a65ebc5..6ce05a4ad24 100644 --- a/include/rocksdb/utilities/stackable_db.h +++ b/include/rocksdb/utilities/stackable_db.h @@ -103,12 +103,18 @@ class StackableDB : public DB { } using DB::GetEntity; + Status GetEntity(const ReadOptions& options, ColumnFamilyHandle* column_family, const Slice& key, PinnableWideColumns* columns) override { return db_->GetEntity(options, column_family, key, columns); } + Status GetEntity(const ReadOptions& options, const Slice& key, + PinnableAttributeGroups* result) override { + return db_->GetEntity(options, key, result); + } + using DB::GetMergeOperands; Status GetMergeOperands(const ReadOptions& options, ColumnFamilyHandle* column_family, const Slice& key, @@ -147,6 +153,12 @@ class StackableDB : public DB { statuses, sorted_input); } + void MultiGetEntity(const ReadOptions& options, size_t num_keys, + const Slice* keys, + PinnableAttributeGroups* results) override { + db_->MultiGetEntity(options, num_keys, keys, results); + } + using DB::IngestExternalFile; Status IngestExternalFile(ColumnFamilyHandle* column_family, const std::vector& external_files, @@ -259,11 +271,18 @@ class StackableDB : public DB { return db_->NewIterators(options, column_families, iterators); } - using DB::NewMultiCfIterator; - std::unique_ptr NewMultiCfIterator( + using DB::NewCoalescingIterator; + std::unique_ptr NewCoalescingIterator( + const ReadOptions& options, + const std::vector& column_families) override { + return db_->NewCoalescingIterator(options, column_families); + } + + using DB::NewAttributeGroupIterator; + std::unique_ptr NewAttributeGroupIterator( const ReadOptions& options, const std::vector& column_families) override { - return db_->NewMultiCfIterator(options, column_families); + return db_->NewAttributeGroupIterator(options, column_families); } const Snapshot* GetSnapshot() override { return db_->GetSnapshot(); } diff --git a/include/rocksdb/utilities/types_util.h b/include/rocksdb/utilities/types_util.h new file mode 100644 index 00000000000..d1531cf12fd --- /dev/null +++ b/include/rocksdb/utilities/types_util.h @@ -0,0 +1,36 @@ +// Copyright (c) Meta Platforms, Inc. and affiliates. +// +// 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 "rocksdb/comparator.h" +#include "rocksdb/slice.h" +#include "rocksdb/status.h" +#include "rocksdb/types.h" + +namespace ROCKSDB_NAMESPACE { + +// Given a user key, creates the internal key used for `Seek` operation for a +// raw table iterator. The internal key is stored in `buf`. +// `comparator` should be the same as the `Options.comparator` used to create +// the column family or the `SstFileWriter`. +Status GetInternalKeyForSeek(const Slice& user_key, + const Comparator* comparator, std::string* buf); + +// Given a user key, creates the internal key used for `SeekForPrev` operation +// for a raw table iterator. The internal key is stored in `buf`. +// `comparator`: see doc for `GetInternalKeyForSeek`. +Status GetInternalKeyForSeekForPrev(const Slice& user_key, + const Comparator* comparator, + std::string* buf); + +// Util method that takes an internal key and parse it to get `ParsedEntryInfo`. +// Such an internal key usually comes from a table iterator. +// `comparator`: see doc for `GetInternalKeyForSeek`. +Status ParseEntry(const Slice& internal_key, const Comparator* comparator, + ParsedEntryInfo* parsed_entry); + +} // namespace ROCKSDB_NAMESPACE diff --git a/include/rocksdb/utilities/write_batch_with_index.h b/include/rocksdb/utilities/write_batch_with_index.h index 3fa168d0f9f..de24ae889b2 100644 --- a/include/rocksdb/utilities/write_batch_with_index.h +++ b/include/rocksdb/utilities/write_batch_with_index.h @@ -268,7 +268,23 @@ class WriteBatchWithIndex : public WriteBatchBase { ColumnFamilyHandle* column_family, const Slice& key, PinnableSlice* value); - // TODO: implement GetEntityFromBatchAndDB + // Similar to DB::GetEntity() but also reads writes from this batch. + // + // This method queries the batch for the key and if the result can be + // determined based on the batch alone, it is returned (assuming the key is + // found, in the form of a wide-column entity). If the batch does not contain + // enough information to determine the result (the key is not present in the + // batch at all or a merge is in progress), the DB is queried and the result + // is merged with the entries from the batch if necessary. + // + // Setting read_options.snapshot will affect what is read from the DB + // but will NOT change which keys are read from the batch (the keys in + // this batch do not yet belong to any snapshot and will be fetched + // regardless). + Status GetEntityFromBatchAndDB(DB* db, const ReadOptions& read_options, + ColumnFamilyHandle* column_family, + const Slice& key, + PinnableWideColumns* columns); void MultiGetFromBatchAndDB(DB* db, const ReadOptions& read_options, ColumnFamilyHandle* column_family, @@ -276,7 +292,25 @@ class WriteBatchWithIndex : public WriteBatchBase { PinnableSlice* values, Status* statuses, bool sorted_input); - // TODO: implement MultiGetEntityFromBatchAndDB + // Similar to DB::MultiGetEntity() but also reads writes from this batch. + // + // For each key, this method queries the batch and if the result can be + // determined based on the batch alone, it is returned in the appropriate + // PinnableWideColumns object (assuming the key is found). For all keys for + // which the batch does not contain enough information to determine the result + // (the key is not present in the batch at all or a merge is in progress), the + // DB is queried and the result is merged with the entries from the batch if + // necessary. + // + // Setting read_options.snapshot will affect what is read from the DB + // but will NOT change which keys are read from the batch (the keys in + // this batch do not yet belong to any snapshot and will be fetched + // regardless). + void MultiGetEntityFromBatchAndDB(DB* db, const ReadOptions& read_options, + ColumnFamilyHandle* column_family, + size_t num_keys, const Slice* keys, + PinnableWideColumns* results, + Status* statuses, bool sorted_input); // Records the state of the batch for future calls to RollbackToSavePoint(). // May be called multiple times to set multiple save points. @@ -314,11 +348,23 @@ class WriteBatchWithIndex : public WriteBatchBase { // last sub-batch. size_t SubBatchCnt(); + void MergeAcrossBatchAndDBImpl(ColumnFamilyHandle* column_family, + const Slice& key, + const PinnableWideColumns& existing, + const MergeContext& merge_context, + std::string* value, + PinnableWideColumns* columns, Status* status); void MergeAcrossBatchAndDB(ColumnFamilyHandle* column_family, const Slice& key, const PinnableWideColumns& existing, const MergeContext& merge_context, PinnableSlice* value, Status* status); + void MergeAcrossBatchAndDB(ColumnFamilyHandle* column_family, + const Slice& key, + const PinnableWideColumns& existing, + const MergeContext& merge_context, + PinnableWideColumns* columns, Status* status); + Status GetFromBatchAndDB(DB* db, const ReadOptions& read_options, ColumnFamilyHandle* column_family, const Slice& key, PinnableSlice* value, ReadCallback* callback); @@ -327,6 +373,17 @@ class WriteBatchWithIndex : public WriteBatchBase { const size_t num_keys, const Slice* keys, PinnableSlice* values, Status* statuses, bool sorted_input, ReadCallback* callback); + Status GetEntityFromBatchAndDB(DB* db, const ReadOptions& read_options, + ColumnFamilyHandle* column_family, + const Slice& key, PinnableWideColumns* columns, + ReadCallback* callback); + void MultiGetEntityFromBatchAndDB(DB* db, const ReadOptions& read_options, + ColumnFamilyHandle* column_family, + size_t num_keys, const Slice* keys, + PinnableWideColumns* results, + Status* statuses, bool sorted_input, + ReadCallback* callback); + struct Rep; std::unique_ptr rep; }; diff --git a/include/rocksdb/version.h b/include/rocksdb/version.h index 10603a77394..24506703fbc 100644 --- a/include/rocksdb/version.h +++ b/include/rocksdb/version.h @@ -12,7 +12,7 @@ // NOTE: in 'main' development branch, this should be the *next* // minor or major version number planned for release. #define ROCKSDB_MAJOR 9 -#define ROCKSDB_MINOR 1 +#define ROCKSDB_MINOR 3 #define ROCKSDB_PATCH 0 // Do not use these. We made the mistake of declaring macros starting with diff --git a/include/rocksdb/wide_columns.h b/include/rocksdb/wide_columns.h index 79763755102..93616cba81e 100644 --- a/include/rocksdb/wide_columns.h +++ b/include/rocksdb/wide_columns.h @@ -105,6 +105,16 @@ extern const WideColumns kNoWideColumns; // wide-column queries. class PinnableWideColumns { public: + PinnableWideColumns() = default; + + PinnableWideColumns(const PinnableWideColumns&) = delete; + PinnableWideColumns& operator=(const PinnableWideColumns&) = delete; + + PinnableWideColumns(PinnableWideColumns&&); + PinnableWideColumns& operator=(PinnableWideColumns&&); + + ~PinnableWideColumns() = default; + const WideColumns& columns() const { return columns_; } size_t serialized_size() const { return value_.size(); } @@ -121,6 +131,7 @@ class PinnableWideColumns { void Reset(); private: + void Move(PinnableWideColumns&& other); void CopyValue(const Slice& value); void PinOrCopyValue(const Slice& value, Cleanable* cleanable); void MoveValue(PinnableSlice&& value); @@ -133,6 +144,42 @@ class PinnableWideColumns { WideColumns columns_; }; +inline void PinnableWideColumns::Reset() { + value_.Reset(); + columns_.clear(); +} + +inline void PinnableWideColumns::Move(PinnableWideColumns&& other) { + assert(columns_.empty()); + + if (other.columns_.empty()) { + return; + } + + const char* const data = other.value_.data(); + const bool is_plain_value = + other.columns_.size() == 1 && + other.columns_.front().name() == kDefaultWideColumnName && + other.columns_.front().value() == other.value_; + + MoveValue(std::move(other.value_)); + + if (value_.data() == data) { + columns_ = std::move(other.columns_); + } else { + if (is_plain_value) { + CreateIndexForPlainValue(); + } else { + const Status s = CreateIndexForWideColumns(); + assert(s.ok()); + + s.PermitUncheckedError(); + } + } + + other.Reset(); +} + inline void PinnableWideColumns::CopyValue(const Slice& value) { value_.PinSelf(value); } @@ -186,28 +233,61 @@ inline void PinnableWideColumns::SetPlainValue(std::string&& value) { inline Status PinnableWideColumns::SetWideColumnValue(const Slice& value) { CopyValue(value); - return CreateIndexForWideColumns(); + + const Status s = CreateIndexForWideColumns(); + if (!s.ok()) { + Reset(); + } + + return s; } inline Status PinnableWideColumns::SetWideColumnValue(const Slice& value, Cleanable* cleanable) { PinOrCopyValue(value, cleanable); - return CreateIndexForWideColumns(); + + const Status s = CreateIndexForWideColumns(); + if (!s.ok()) { + Reset(); + } + + return s; } inline Status PinnableWideColumns::SetWideColumnValue(PinnableSlice&& value) { MoveValue(std::move(value)); - return CreateIndexForWideColumns(); + + const Status s = CreateIndexForWideColumns(); + if (!s.ok()) { + Reset(); + } + + return s; } inline Status PinnableWideColumns::SetWideColumnValue(std::string&& value) { MoveValue(std::move(value)); - return CreateIndexForWideColumns(); + + const Status s = CreateIndexForWideColumns(); + if (!s.ok()) { + Reset(); + } + + return s; } -inline void PinnableWideColumns::Reset() { - value_.Reset(); - columns_.clear(); +inline PinnableWideColumns::PinnableWideColumns(PinnableWideColumns&& other) { + Move(std::move(other)); +} + +inline PinnableWideColumns& PinnableWideColumns::operator=( + PinnableWideColumns&& other) { + if (this != &other) { + Reset(); + Move(std::move(other)); + } + + return *this; } inline bool operator==(const PinnableWideColumns& lhs, @@ -220,69 +300,4 @@ inline bool operator!=(const PinnableWideColumns& lhs, return !(lhs == rhs); } -// Class representing attribute group. Attribute group is a logical grouping of -// wide-column entities by leveraging Column Families. -// Used in Write Path -class AttributeGroup { - public: - ColumnFamilyHandle* column_family() const { return column_family_; } - const WideColumns& columns() const { return columns_; } - WideColumns& columns() { return columns_; } - - explicit AttributeGroup(ColumnFamilyHandle* column_family, - const WideColumns& columns) - : column_family_(column_family), columns_(columns) {} - - private: - ColumnFamilyHandle* column_family_; - WideColumns columns_; -}; - -inline bool operator==(const AttributeGroup& lhs, const AttributeGroup& rhs) { - return lhs.column_family() == rhs.column_family() && - lhs.columns() == rhs.columns(); -} - -// A collection of Attribute Groups. -using AttributeGroups = std::vector; - -// An empty set of Attribute Groups. -extern const AttributeGroups kNoAttributeGroups; - -// Used in Read Path. Wide-columns returned from the query are pinnable. -class PinnableAttributeGroup { - public: - ColumnFamilyHandle* column_family() const { return column_family_; } - const Status& status() const { return status_; } - const WideColumns& columns() const { return columns_.columns(); } - - explicit PinnableAttributeGroup(ColumnFamilyHandle* column_family) - : column_family_(column_family), status_(Status::OK()) {} - - void SetStatus(const Status& status); - void SetColumns(PinnableWideColumns&& columns); - - void Reset(); - - private: - ColumnFamilyHandle* column_family_; - Status status_; - PinnableWideColumns columns_; -}; - -inline void PinnableAttributeGroup::SetStatus(const Status& status) { - status_ = status; -} -inline void PinnableAttributeGroup::SetColumns(PinnableWideColumns&& columns) { - columns_ = std::move(columns); -} - -inline void PinnableAttributeGroup::Reset() { - SetStatus(Status::OK()); - columns_.Reset(); -} - -// A collection of Pinnable Attribute Groups. -using PinnableAttributeGroups = std::vector; - } // namespace ROCKSDB_NAMESPACE diff --git a/include/rocksdb/write_batch_base.h b/include/rocksdb/write_batch_base.h index 2bf6a3c424a..3334a129256 100644 --- a/include/rocksdb/write_batch_base.h +++ b/include/rocksdb/write_batch_base.h @@ -10,8 +10,8 @@ #include +#include "rocksdb/attribute_groups.h" #include "rocksdb/rocksdb_namespace.h" -#include "rocksdb/wide_columns.h" namespace ROCKSDB_NAMESPACE { diff --git a/java/CMakeLists.txt b/java/CMakeLists.txt index 053e4a33335..a60847ead37 100644 --- a/java/CMakeLists.txt +++ b/java/CMakeLists.txt @@ -16,12 +16,13 @@ set(CMAKE_JAVA_COMPILE_FLAGS -source 8) set(JNI_NATIVE_SOURCES rocksjni/backup_engine_options.cc rocksjni/backupenginejni.cc + rocksjni/cache.cc rocksjni/cassandra_compactionfilterjni.cc rocksjni/cassandra_value_operator.cc rocksjni/checkpoint.cc rocksjni/clock_cache.cc - rocksjni/cache.cc rocksjni/columnfamilyhandle.cc + rocksjni/compact_range_options.cc rocksjni/compaction_filter.cc rocksjni/compaction_filter_factory.cc rocksjni/compaction_filter_factory_jnicallback.cc @@ -30,7 +31,6 @@ set(JNI_NATIVE_SOURCES rocksjni/compaction_options.cc rocksjni/compaction_options_fifo.cc rocksjni/compaction_options_universal.cc - rocksjni/compact_range_options.cc rocksjni/comparator.cc rocksjni/comparatorjnicallback.cc rocksjni/compression_options.cc @@ -42,11 +42,12 @@ set(JNI_NATIVE_SOURCES rocksjni/event_listener_jnicallback.cc rocksjni/export_import_files_metadatajni.cc rocksjni/filter.cc - rocksjni/import_column_family_options.cc rocksjni/hyper_clock_cache.cc + rocksjni/import_column_family_options.cc rocksjni/ingest_external_file_options.cc rocksjni/iterator.cc rocksjni/jni_multiget_helpers.cc + rocksjni/jni_perf_context.cc rocksjni/jnicallback.cc rocksjni/loggerjnicallback.cc rocksjni/lru_cache.cc @@ -59,7 +60,6 @@ set(JNI_NATIVE_SOURCES rocksjni/options.cc rocksjni/options_util.cc rocksjni/persistent_cache.cc - rocksjni/jni_perf_context.cc rocksjni/ratelimiterjni.cc rocksjni/remove_emptyvalue_compactionfilterjni.cc rocksjni/restorejni.cc @@ -69,9 +69,9 @@ set(JNI_NATIVE_SOURCES rocksjni/slice.cc rocksjni/snapshot.cc rocksjni/sst_file_manager.cc - rocksjni/sst_file_writerjni.cc - rocksjni/sst_file_readerjni.cc rocksjni/sst_file_reader_iterator.cc + rocksjni/sst_file_readerjni.cc + rocksjni/sst_file_writerjni.cc rocksjni/sst_partitioner.cc rocksjni/statistics.cc rocksjni/statisticsjni.cc @@ -95,16 +95,17 @@ set(JNI_NATIVE_SOURCES rocksjni/wal_filter.cc rocksjni/wal_filter_jnicallback.cc rocksjni/write_batch.cc - rocksjni/writebatchhandlerjnicallback.cc rocksjni/write_batch_test.cc rocksjni/write_batch_with_index.cc rocksjni/write_buffer_manager.cc + rocksjni/writebatchhandlerjnicallback.cc ) set(JAVA_MAIN_CLASSES src/main/java/org/rocksdb/AbstractCompactionFilter.java src/main/java/org/rocksdb/AbstractCompactionFilterFactory.java src/main/java/org/rocksdb/AbstractComparator.java + src/main/java/org/rocksdb/AbstractComparatorJniBridge.java src/main/java/org/rocksdb/AbstractEventListener.java src/main/java/org/rocksdb/AbstractImmutableNativeReference.java src/main/java/org/rocksdb/AbstractMutableOptions.java @@ -119,8 +120,8 @@ set(JAVA_MAIN_CLASSES src/main/java/org/rocksdb/AdvancedColumnFamilyOptionsInterface.java src/main/java/org/rocksdb/AdvancedMutableColumnFamilyOptionsInterface.java src/main/java/org/rocksdb/BackgroundErrorReason.java - src/main/java/org/rocksdb/BackupEngineOptions.java src/main/java/org/rocksdb/BackupEngine.java + src/main/java/org/rocksdb/BackupEngineOptions.java src/main/java/org/rocksdb/BackupInfo.java src/main/java/org/rocksdb/BlockBasedTableConfig.java src/main/java/org/rocksdb/BloomFilter.java @@ -135,8 +136,9 @@ set(JAVA_MAIN_CLASSES src/main/java/org/rocksdb/ColumnFamilyDescriptor.java src/main/java/org/rocksdb/ColumnFamilyHandle.java src/main/java/org/rocksdb/ColumnFamilyMetaData.java - src/main/java/org/rocksdb/ColumnFamilyOptionsInterface.java src/main/java/org/rocksdb/ColumnFamilyOptions.java + src/main/java/org/rocksdb/ColumnFamilyOptionsInterface.java + src/main/java/org/rocksdb/CompactRangeOptions.java src/main/java/org/rocksdb/CompactionJobInfo.java src/main/java/org/rocksdb/CompactionJobStats.java src/main/java/org/rocksdb/CompactionOptions.java @@ -144,17 +146,18 @@ set(JAVA_MAIN_CLASSES src/main/java/org/rocksdb/CompactionOptionsUniversal.java src/main/java/org/rocksdb/CompactionPriority.java src/main/java/org/rocksdb/CompactionReason.java - src/main/java/org/rocksdb/CompactRangeOptions.java src/main/java/org/rocksdb/CompactionStopStyle.java src/main/java/org/rocksdb/CompactionStyle.java src/main/java/org/rocksdb/ComparatorOptions.java src/main/java/org/rocksdb/ComparatorType.java src/main/java/org/rocksdb/CompressionOptions.java src/main/java/org/rocksdb/CompressionType.java + src/main/java/org/rocksdb/ConcurrentTaskLimiter.java + src/main/java/org/rocksdb/ConcurrentTaskLimiterImpl.java src/main/java/org/rocksdb/ConfigOptions.java - src/main/java/org/rocksdb/DataBlockIndexType.java - src/main/java/org/rocksdb/DBOptionsInterface.java src/main/java/org/rocksdb/DBOptions.java + src/main/java/org/rocksdb/DBOptionsInterface.java + src/main/java/org/rocksdb/DataBlockIndexType.java src/main/java/org/rocksdb/DbPath.java src/main/java/org/rocksdb/DirectSlice.java src/main/java/org/rocksdb/EncodingType.java @@ -164,38 +167,36 @@ set(JAVA_MAIN_CLASSES src/main/java/org/rocksdb/Experimental.java src/main/java/org/rocksdb/ExportImportFilesMetaData.java src/main/java/org/rocksdb/ExternalFileIngestionInfo.java + src/main/java/org/rocksdb/FileOperationInfo.java src/main/java/org/rocksdb/Filter.java src/main/java/org/rocksdb/FilterPolicyType.java - src/main/java/org/rocksdb/FileOperationInfo.java src/main/java/org/rocksdb/FlushJobInfo.java - src/main/java/org/rocksdb/FlushReason.java src/main/java/org/rocksdb/FlushOptions.java + src/main/java/org/rocksdb/FlushReason.java src/main/java/org/rocksdb/GetStatus.java src/main/java/org/rocksdb/HashLinkedListMemTableConfig.java src/main/java/org/rocksdb/HashSkipListMemTableConfig.java src/main/java/org/rocksdb/HistogramData.java src/main/java/org/rocksdb/HistogramType.java src/main/java/org/rocksdb/Holder.java - src/main/java/org/rocksdb/ImportColumnFamilyOptions.java src/main/java/org/rocksdb/HyperClockCache.java + src/main/java/org/rocksdb/ImportColumnFamilyOptions.java src/main/java/org/rocksdb/IndexShorteningMode.java src/main/java/org/rocksdb/IndexType.java src/main/java/org/rocksdb/InfoLogLevel.java src/main/java/org/rocksdb/IngestExternalFileOptions.java - src/main/java/org/rocksdb/LevelMetaData.java - src/main/java/org/rocksdb/ConcurrentTaskLimiter.java - src/main/java/org/rocksdb/ConcurrentTaskLimiterImpl.java src/main/java/org/rocksdb/KeyMayExist.java + src/main/java/org/rocksdb/LRUCache.java + src/main/java/org/rocksdb/LevelMetaData.java src/main/java/org/rocksdb/LiveFileMetaData.java src/main/java/org/rocksdb/LogFile.java src/main/java/org/rocksdb/Logger.java src/main/java/org/rocksdb/LoggerInterface.java src/main/java/org/rocksdb/LoggerType.java - src/main/java/org/rocksdb/LRUCache.java - src/main/java/org/rocksdb/MemoryUsageType.java - src/main/java/org/rocksdb/MemoryUtil.java src/main/java/org/rocksdb/MemTableConfig.java src/main/java/org/rocksdb/MemTableInfo.java + src/main/java/org/rocksdb/MemoryUsageType.java + src/main/java/org/rocksdb/MemoryUtil.java src/main/java/org/rocksdb/MergeOperator.java src/main/java/org/rocksdb/MutableColumnFamilyOptions.java src/main/java/org/rocksdb/MutableColumnFamilyOptionsInterface.java @@ -209,12 +210,12 @@ set(JAVA_MAIN_CLASSES src/main/java/org/rocksdb/OperationType.java src/main/java/org/rocksdb/OptimisticTransactionDB.java src/main/java/org/rocksdb/OptimisticTransactionOptions.java - src/main/java/org/rocksdb/Options.java src/main/java/org/rocksdb/OptionString.java + src/main/java/org/rocksdb/Options.java src/main/java/org/rocksdb/OptionsUtil.java - src/main/java/org/rocksdb/PersistentCache.java src/main/java/org/rocksdb/PerfContext.java src/main/java/org/rocksdb/PerfLevel.java + src/main/java/org/rocksdb/PersistentCache.java src/main/java/org/rocksdb/PlainTableConfig.java src/main/java/org/rocksdb/PrepopulateBlobCache.java src/main/java/org/rocksdb/Priority.java @@ -227,11 +228,11 @@ set(JAVA_MAIN_CLASSES src/main/java/org/rocksdb/RestoreOptions.java src/main/java/org/rocksdb/ReusedSynchronisationType.java src/main/java/org/rocksdb/RocksCallbackObject.java - src/main/java/org/rocksdb/RocksDBException.java src/main/java/org/rocksdb/RocksDB.java + src/main/java/org/rocksdb/RocksDBException.java src/main/java/org/rocksdb/RocksEnv.java - src/main/java/org/rocksdb/RocksIteratorInterface.java src/main/java/org/rocksdb/RocksIterator.java + src/main/java/org/rocksdb/RocksIteratorInterface.java src/main/java/org/rocksdb/RocksMemEnv.java src/main/java/org/rocksdb/RocksMutableObject.java src/main/java/org/rocksdb/RocksObject.java @@ -248,9 +249,9 @@ set(JAVA_MAIN_CLASSES src/main/java/org/rocksdb/SstPartitionerFactory.java src/main/java/org/rocksdb/SstPartitionerFixedPrefixFactory.java src/main/java/org/rocksdb/StateType.java - src/main/java/org/rocksdb/StatisticsCollectorCallback.java - src/main/java/org/rocksdb/StatisticsCollector.java src/main/java/org/rocksdb/Statistics.java + src/main/java/org/rocksdb/StatisticsCollector.java + src/main/java/org/rocksdb/StatisticsCollectorCallback.java src/main/java/org/rocksdb/StatsCollectorInput.java src/main/java/org/rocksdb/StatsLevel.java src/main/java/org/rocksdb/Status.java @@ -260,35 +261,36 @@ set(JAVA_MAIN_CLASSES src/main/java/org/rocksdb/TableFileCreationReason.java src/main/java/org/rocksdb/TableFileDeletionInfo.java src/main/java/org/rocksdb/TableFilter.java + src/main/java/org/rocksdb/TableFormatConfig.java src/main/java/org/rocksdb/TableProperties.java src/main/java/org/rocksdb/TablePropertiesCollectorFactory.java - src/main/java/org/rocksdb/TableFormatConfig.java - src/main/java/org/rocksdb/ThreadType.java src/main/java/org/rocksdb/ThreadStatus.java + src/main/java/org/rocksdb/ThreadType.java src/main/java/org/rocksdb/TickerType.java src/main/java/org/rocksdb/TimedEnv.java src/main/java/org/rocksdb/TraceOptions.java src/main/java/org/rocksdb/TraceWriter.java - src/main/java/org/rocksdb/TransactionalDB.java - src/main/java/org/rocksdb/TransactionalOptions.java + src/main/java/org/rocksdb/Transaction.java src/main/java/org/rocksdb/TransactionDB.java src/main/java/org/rocksdb/TransactionDBOptions.java - src/main/java/org/rocksdb/Transaction.java src/main/java/org/rocksdb/TransactionLogIterator.java src/main/java/org/rocksdb/TransactionOptions.java + src/main/java/org/rocksdb/TransactionalDB.java + src/main/java/org/rocksdb/TransactionalOptions.java src/main/java/org/rocksdb/TtlDB.java src/main/java/org/rocksdb/TxnDBWritePolicy.java + src/main/java/org/rocksdb/UInt64AddOperator.java src/main/java/org/rocksdb/VectorMemTableConfig.java + src/main/java/org/rocksdb/WALRecoveryMode.java + src/main/java/org/rocksdb/WBWIRocksIterator.java src/main/java/org/rocksdb/WalFileType.java src/main/java/org/rocksdb/WalFilter.java src/main/java/org/rocksdb/WalProcessingOption.java - src/main/java/org/rocksdb/WALRecoveryMode.java - src/main/java/org/rocksdb/WBWIRocksIterator.java src/main/java/org/rocksdb/WriteBatch.java src/main/java/org/rocksdb/WriteBatchInterface.java src/main/java/org/rocksdb/WriteBatchWithIndex.java - src/main/java/org/rocksdb/WriteOptions.java src/main/java/org/rocksdb/WriteBufferManager.java + src/main/java/org/rocksdb/WriteOptions.java src/main/java/org/rocksdb/WriteStallCondition.java src/main/java/org/rocksdb/WriteStallInfo.java src/main/java/org/rocksdb/util/BufferUtil.java @@ -298,224 +300,240 @@ set(JAVA_MAIN_CLASSES src/main/java/org/rocksdb/util/IntComparator.java src/main/java/org/rocksdb/util/ReverseBytewiseComparator.java src/main/java/org/rocksdb/util/SizeUnit.java - src/main/java/org/rocksdb/UInt64AddOperator.java + src/main/java/org/rocksdb/util/StdErrLogger.java src/test/java/org/rocksdb/NativeComparatorWrapperTest.java src/test/java/org/rocksdb/RocksDBExceptionTest.java - src/test/java/org/rocksdb/test/TestableEventListener.java - src/test/java/org/rocksdb/WriteBatchTest.java src/test/java/org/rocksdb/RocksNativeLibraryResource.java + src/test/java/org/rocksdb/WriteBatchTest.java + src/test/java/org/rocksdb/test/TestableEventListener.java src/test/java/org/rocksdb/util/CapturingWriteBatchHandler.java - src/main/java/org/rocksdb/util/StdErrLogger.java src/test/java/org/rocksdb/util/WriteBatchGetter.java ) set(JAVA_TEST_CLASSES + src/test/java/org/rocksdb/AbstractTransactionTest.java + src/test/java/org/rocksdb/BackupEngineOptionsTest.java + src/test/java/org/rocksdb/BackupEngineTest.java + src/test/java/org/rocksdb/BlobOptionsTest.java + src/test/java/org/rocksdb/BlockBasedTableConfigTest.java + src/test/java/org/rocksdb/BuiltinComparatorTest.java + src/test/java/org/rocksdb/ByteBufferUnsupportedOperationTest.java + src/test/java/org/rocksdb/BytewiseComparatorRegressionTest.java + src/test/java/org/rocksdb/CheckPointTest.java + src/test/java/org/rocksdb/ClockCacheTest.java + src/test/java/org/rocksdb/ColumnFamilyOptionsTest.java + src/test/java/org/rocksdb/ColumnFamilyTest.java + src/test/java/org/rocksdb/CompactRangeOptionsTest.java + src/test/java/org/rocksdb/CompactionFilterFactoryTest.java + src/test/java/org/rocksdb/CompactionJobInfoTest.java + src/test/java/org/rocksdb/CompactionJobStatsTest.java + src/test/java/org/rocksdb/CompactionOptionsFIFOTest.java + src/test/java/org/rocksdb/CompactionOptionsTest.java + src/test/java/org/rocksdb/CompactionOptionsUniversalTest.java + src/test/java/org/rocksdb/CompactionPriorityTest.java + src/test/java/org/rocksdb/CompactionStopStyleTest.java + src/test/java/org/rocksdb/ComparatorOptionsTest.java + src/test/java/org/rocksdb/CompressionOptionsTest.java + src/test/java/org/rocksdb/CompressionTypesTest.java src/test/java/org/rocksdb/ConcurrentTaskLimiterTest.java + src/test/java/org/rocksdb/DBOptionsTest.java + src/test/java/org/rocksdb/DefaultEnvTest.java + src/test/java/org/rocksdb/DirectSliceTest.java + src/test/java/org/rocksdb/EnvOptionsTest.java src/test/java/org/rocksdb/EventListenerTest.java - src/test/java/org/rocksdb/CompactionOptionsTest.java - src/test/java/org/rocksdb/PlatformRandomHelper.java + src/test/java/org/rocksdb/FilterTest.java + src/test/java/org/rocksdb/FlushOptionsTest.java + src/test/java/org/rocksdb/FlushTest.java + src/test/java/org/rocksdb/HyperClockCacheTest.java + src/test/java/org/rocksdb/ImportColumnFamilyTest.java + src/test/java/org/rocksdb/InfoLogLevelTest.java src/test/java/org/rocksdb/IngestExternalFileOptionsTest.java - src/test/java/org/rocksdb/MutableDBOptionsTest.java - src/test/java/org/rocksdb/WriteOptionsTest.java - src/test/java/org/rocksdb/SstPartitionerTest.java - src/test/java/org/rocksdb/RocksMemEnvTest.java - src/test/java/org/rocksdb/CompactionOptionsUniversalTest.java - src/test/java/org/rocksdb/ClockCacheTest.java - src/test/java/org/rocksdb/BytewiseComparatorRegressionTest.java - src/test/java/org/rocksdb/SnapshotTest.java - src/test/java/org/rocksdb/CompactionJobStatsTest.java + src/test/java/org/rocksdb/KeyExistsTest.java + src/test/java/org/rocksdb/KeyMayExistTest.java + src/test/java/org/rocksdb/LRUCacheTest.java + src/test/java/org/rocksdb/LoggerTest.java src/test/java/org/rocksdb/MemTableTest.java - src/test/java/org/rocksdb/CompactionFilterFactoryTest.java - src/test/java/org/rocksdb/DefaultEnvTest.java - src/test/java/org/rocksdb/DBOptionsTest.java - src/test/java/org/rocksdb/RocksIteratorTest.java - src/test/java/org/rocksdb/SliceTest.java + src/test/java/org/rocksdb/MemoryUtilTest.java + src/test/java/org/rocksdb/MergeCFVariantsTest.java + src/test/java/org/rocksdb/MergeTest.java + src/test/java/org/rocksdb/MergeVariantsTest.java + src/test/java/org/rocksdb/MixedOptionsTest.java + src/test/java/org/rocksdb/MultiColumnRegressionTest.java + src/test/java/org/rocksdb/MultiGetManyKeysTest.java src/test/java/org/rocksdb/MultiGetTest.java - src/test/java/org/rocksdb/ComparatorOptionsTest.java + src/test/java/org/rocksdb/MutableColumnFamilyOptionsTest.java + src/test/java/org/rocksdb/MutableDBOptionsTest.java + src/test/java/org/rocksdb/MutableOptionsGetSetTest.java src/test/java/org/rocksdb/NativeLibraryLoaderTest.java - src/test/java/org/rocksdb/StatisticsTest.java - src/test/java/org/rocksdb/WALRecoveryModeTest.java - src/test/java/org/rocksdb/TransactionLogIteratorTest.java + src/test/java/org/rocksdb/OptimisticTransactionDBTest.java + src/test/java/org/rocksdb/OptimisticTransactionOptionsTest.java + src/test/java/org/rocksdb/OptimisticTransactionTest.java + src/test/java/org/rocksdb/OptionsTest.java + src/test/java/org/rocksdb/OptionsUtilTest.java + src/test/java/org/rocksdb/PerfContextTest.java + src/test/java/org/rocksdb/PerfLevelTest.java + src/test/java/org/rocksdb/PlainTableConfigTest.java + src/test/java/org/rocksdb/PlatformRandomHelper.java + src/test/java/org/rocksdb/PutCFVariantsTest.java + src/test/java/org/rocksdb/PutMultiplePartsTest.java + src/test/java/org/rocksdb/PutVariantsTest.java + src/test/java/org/rocksdb/RateLimiterTest.java + src/test/java/org/rocksdb/ReadOnlyTest.java src/test/java/org/rocksdb/ReadOptionsTest.java + src/test/java/org/rocksdb/RocksDBTest.java + src/test/java/org/rocksdb/RocksIteratorTest.java + src/test/java/org/rocksdb/RocksMemEnvTest.java src/test/java/org/rocksdb/SecondaryDBTest.java - src/test/java/org/rocksdb/KeyMayExistTest.java - src/test/java/org/rocksdb/BlobOptionsTest.java - src/test/java/org/rocksdb/InfoLogLevelTest.java - src/test/java/org/rocksdb/CompactionPriorityTest.java - src/test/java/org/rocksdb/FlushOptionsTest.java - src/test/java/org/rocksdb/VerifyChecksumsTest.java - src/test/java/org/rocksdb/MultiColumnRegressionTest.java - src/test/java/org/rocksdb/FlushTest.java - src/test/java/org/rocksdb/HyperClockCacheTest.java - src/test/java/org/rocksdb/PutMultiplePartsTest.java + src/test/java/org/rocksdb/SliceTest.java + src/test/java/org/rocksdb/SnapshotTest.java + src/test/java/org/rocksdb/SstFileManagerTest.java + src/test/java/org/rocksdb/SstFileReaderTest.java + src/test/java/org/rocksdb/SstFileWriterTest.java + src/test/java/org/rocksdb/SstPartitionerTest.java src/test/java/org/rocksdb/StatisticsCollectorTest.java - src/test/java/org/rocksdb/LRUCacheTest.java - src/test/java/org/rocksdb/ColumnFamilyOptionsTest.java - src/test/java/org/rocksdb/TransactionTest.java - src/test/java/org/rocksdb/CompactionOptionsFIFOTest.java - src/test/java/org/rocksdb/BackupEngineOptionsTest.java - src/test/java/org/rocksdb/CheckPointTest.java - src/test/java/org/rocksdb/PlainTableConfigTest.java + src/test/java/org/rocksdb/StatisticsTest.java + src/test/java/org/rocksdb/StatsCallbackMock.java + src/test/java/org/rocksdb/TableFilterTest.java + src/test/java/org/rocksdb/TimedEnvTest.java src/test/java/org/rocksdb/TransactionDBOptionsTest.java - src/test/java/org/rocksdb/ReadOnlyTest.java - src/test/java/org/rocksdb/EnvOptionsTest.java + src/test/java/org/rocksdb/TransactionDBTest.java + src/test/java/org/rocksdb/TransactionLogIteratorTest.java + src/test/java/org/rocksdb/TransactionOptionsTest.java + src/test/java/org/rocksdb/TransactionTest.java + src/test/java/org/rocksdb/TtlDBTest.java + src/test/java/org/rocksdb/Types.java + src/test/java/org/rocksdb/VerifyChecksumsTest.java + src/test/java/org/rocksdb/WALRecoveryModeTest.java + src/test/java/org/rocksdb/WalFilterTest.java + src/test/java/org/rocksdb/WriteBatchHandlerTest.java + src/test/java/org/rocksdb/WriteBatchThreadedTest.java + src/test/java/org/rocksdb/WriteBatchWithIndexTest.java + src/test/java/org/rocksdb/WriteOptionsTest.java src/test/java/org/rocksdb/test/RemoveEmptyValueCompactionFilterFactory.java src/test/java/org/rocksdb/test/RemoveEmptyValueCompactionFilterFactory.java - src/test/java/org/rocksdb/test/TestableEventListener.java src/test/java/org/rocksdb/test/RemoveEmptyValueCompactionFilterFactory.java - src/test/java/org/rocksdb/test/TestableEventListener.java src/test/java/org/rocksdb/test/RocksJunitRunner.java - src/test/java/org/rocksdb/LoggerTest.java - src/test/java/org/rocksdb/FilterTest.java - src/test/java/org/rocksdb/ByteBufferUnsupportedOperationTest.java - src/test/java/org/rocksdb/util/IntComparatorTest.java - src/test/java/org/rocksdb/util/JNIComparatorTest.java + src/test/java/org/rocksdb/test/TestableEventListener.java + src/test/java/org/rocksdb/test/TestableEventListener.java src/test/java/org/rocksdb/util/ByteBufferAllocator.java - src/test/java/org/rocksdb/util/SizeUnitTest.java - src/test/java/org/rocksdb/util/BytewiseComparatorTest.java - src/test/java/org/rocksdb/util/EnvironmentTest.java src/test/java/org/rocksdb/util/BytewiseComparatorIntTest.java + src/test/java/org/rocksdb/util/BytewiseComparatorTest.java src/test/java/org/rocksdb/util/DirectByteBufferAllocator.java + src/test/java/org/rocksdb/util/EnvironmentTest.java src/test/java/org/rocksdb/util/HeapByteBufferAllocator.java - src/test/java/org/rocksdb/util/TestUtil.java + src/test/java/org/rocksdb/util/IntComparatorTest.java + src/test/java/org/rocksdb/util/JNIComparatorTest.java src/test/java/org/rocksdb/util/ReverseBytewiseComparatorIntTest.java - src/test/java/org/rocksdb/Types.java - src/test/java/org/rocksdb/MixedOptionsTest.java - src/test/java/org/rocksdb/CompactRangeOptionsTest.java - src/test/java/org/rocksdb/SstFileWriterTest.java - src/test/java/org/rocksdb/WalFilterTest.java - src/test/java/org/rocksdb/AbstractTransactionTest.java - src/test/java/org/rocksdb/MergeTest.java - src/test/java/org/rocksdb/OptionsTest.java - src/test/java/org/rocksdb/WriteBatchThreadedTest.java - src/test/java/org/rocksdb/MultiGetManyKeysTest.java - src/test/java/org/rocksdb/TimedEnvTest.java - src/test/java/org/rocksdb/CompactionStopStyleTest.java - src/test/java/org/rocksdb/CompactionJobInfoTest.java - src/test/java/org/rocksdb/BlockBasedTableConfigTest.java - src/test/java/org/rocksdb/BuiltinComparatorTest.java - src/test/java/org/rocksdb/RateLimiterTest.java - src/test/java/org/rocksdb/TransactionOptionsTest.java - src/test/java/org/rocksdb/WriteBatchWithIndexTest.java - src/test/java/org/rocksdb/WriteBatchHandlerTest.java - src/test/java/org/rocksdb/OptimisticTransactionDBTest.java - src/test/java/org/rocksdb/OptionsUtilTest.java - src/test/java/org/rocksdb/OptimisticTransactionTest.java - src/test/java/org/rocksdb/MutableColumnFamilyOptionsTest.java - src/test/java/org/rocksdb/CompressionOptionsTest.java - src/test/java/org/rocksdb/ColumnFamilyTest.java - src/test/java/org/rocksdb/SstFileReaderTest.java - src/test/java/org/rocksdb/TransactionDBTest.java - src/test/java/org/rocksdb/RocksDBTest.java - src/test/java/org/rocksdb/MutableOptionsGetSetTest.java - src/test/java/org/rocksdb/OptimisticTransactionOptionsTest.java - src/test/java/org/rocksdb/SstFileManagerTest.java - src/test/java/org/rocksdb/BackupEngineTest.java - src/test/java/org/rocksdb/DirectSliceTest.java - src/test/java/org/rocksdb/StatsCallbackMock.java - src/test/java/org/rocksdb/CompressionTypesTest.java - src/test/java/org/rocksdb/MemoryUtilTest.java - src/test/java/org/rocksdb/TableFilterTest.java - src/test/java/org/rocksdb/TtlDBTest.java + src/test/java/org/rocksdb/util/SizeUnitTest.java src/test/java/org/rocksdb/util/StdErrLoggerTest.java + src/test/java/org/rocksdb/util/TestUtil.java ) set(JAVA_TEST_RUNNING_CLASSES - org.rocksdb.ConcurrentTaskLimiterTest - org.rocksdb.EventListenerTest - org.rocksdb.CompactionOptionsTest - org.rocksdb.IngestExternalFileOptionsTest - org.rocksdb.MutableDBOptionsTest - org.rocksdb.WriteOptionsTest - org.rocksdb.SstPartitionerTest - org.rocksdb.RocksMemEnvTest - org.rocksdb.CompactionOptionsUniversalTest + org.rocksdb.BackupEngineOptionsTest + org.rocksdb.BackupEngineTest + org.rocksdb.BlobOptionsTest + org.rocksdb.BlockBasedTableConfigTest + org.rocksdb.BuiltinComparatorTest + org.rocksdb.ByteBufferUnsupportedOperationTest + org.rocksdb.BytewiseComparatorRegressionTest + org.rocksdb.CheckPointTest org.rocksdb.ClockCacheTest - # org.rocksdb.BytewiseComparatorRegressionTest - org.rocksdb.SnapshotTest - org.rocksdb.CompactionJobStatsTest - org.rocksdb.MemTableTest + org.rocksdb.ColumnFamilyOptionsTest + org.rocksdb.ColumnFamilyTest + org.rocksdb.CompactRangeOptionsTest org.rocksdb.CompactionFilterFactoryTest - # org.rocksdb.DefaultEnvTest - org.rocksdb.DBOptionsTest - org.rocksdb.WriteBatchTest - org.rocksdb.RocksIteratorTest - org.rocksdb.SliceTest - org.rocksdb.MultiGetTest - org.rocksdb.ComparatorOptionsTest - # org.rocksdb.NativeLibraryLoaderTest - org.rocksdb.StatisticsTest - org.rocksdb.WALRecoveryModeTest - org.rocksdb.TransactionLogIteratorTest - org.rocksdb.ReadOptionsTest - org.rocksdb.SecondaryDBTest - org.rocksdb.KeyMayExistTest - org.rocksdb.BlobOptionsTest - org.rocksdb.InfoLogLevelTest + org.rocksdb.CompactionJobInfoTest + org.rocksdb.CompactionJobStatsTest + org.rocksdb.CompactionOptionsFIFOTest + org.rocksdb.CompactionOptionsTest + org.rocksdb.CompactionOptionsUniversalTest org.rocksdb.CompactionPriorityTest + org.rocksdb.CompactionStopStyleTest + org.rocksdb.ComparatorOptionsTest + org.rocksdb.CompressionOptionsTest + org.rocksdb.CompressionTypesTest + org.rocksdb.ConcurrentTaskLimiterTest + org.rocksdb.DBOptionsTest + org.rocksdb.DefaultEnvTest + org.rocksdb.DirectSliceTest + org.rocksdb.EnvOptionsTest + org.rocksdb.EventListenerTest + org.rocksdb.FilterTest org.rocksdb.FlushOptionsTest - org.rocksdb.VerifyChecksumsTest - org.rocksdb.MultiColumnRegressionTest org.rocksdb.FlushTest org.rocksdb.HyperClockCacheTest - org.rocksdb.PutMultiplePartsTest - org.rocksdb.StatisticsCollectorTest + org.rocksdb.ImportColumnFamilyTest + org.rocksdb.InfoLogLevelTest + org.rocksdb.IngestExternalFileOptionsTest + org.rocksdb.KeyExistsTest + org.rocksdb.KeyMayExistTest org.rocksdb.LRUCacheTest - org.rocksdb.ColumnFamilyOptionsTest - org.rocksdb.TransactionTest - org.rocksdb.CompactionOptionsFIFOTest - org.rocksdb.BackupEngineOptionsTest - org.rocksdb.CheckPointTest - org.rocksdb.PlainTableConfigTest - # org.rocksdb.TransactionDBOptionsTest - org.rocksdb.ReadOnlyTest - org.rocksdb.EnvOptionsTest org.rocksdb.LoggerTest - org.rocksdb.FilterTest - # org.rocksdb.ByteBufferUnsupportedOperationTest - # org.rocksdb.util.IntComparatorTest - # org.rocksdb.util.JNIComparatorTest - org.rocksdb.util.SizeUnitTest - # org.rocksdb.util.BytewiseComparatorTest - org.rocksdb.util.EnvironmentTest - # org.rocksdb.util.BytewiseComparatorIntTest - # org.rocksdb.util.ReverseBytewiseComparatorIntTest - org.rocksdb.MixedOptionsTest - org.rocksdb.CompactRangeOptionsTest - # org.rocksdb.SstFileWriterTest - org.rocksdb.WalFilterTest - # org.rocksdb.AbstractTransactionTest + org.rocksdb.MemTableTest + org.rocksdb.MemoryUtilTest + org.rocksdb.MergeCFVariantsTest org.rocksdb.MergeTest - org.rocksdb.OptionsTest - org.rocksdb.WriteBatchThreadedTest + org.rocksdb.MergeVariantsTest + org.rocksdb.MixedOptionsTest + org.rocksdb.MultiColumnRegressionTest org.rocksdb.MultiGetManyKeysTest - org.rocksdb.TimedEnvTest - org.rocksdb.CompactionStopStyleTest - org.rocksdb.CompactionJobInfoTest - org.rocksdb.BlockBasedTableConfigTest - org.rocksdb.BuiltinComparatorTest - org.rocksdb.RateLimiterTest - # org.rocksdb.TransactionOptionsTest - org.rocksdb.WriteBatchWithIndexTest - org.rocksdb.WriteBatchHandlerTest + org.rocksdb.MultiGetTest + org.rocksdb.MutableColumnFamilyOptionsTest + org.rocksdb.MutableDBOptionsTest + org.rocksdb.MutableOptionsGetSetTest + org.rocksdb.NativeComparatorWrapperTest + org.rocksdb.NativeLibraryLoaderTest org.rocksdb.OptimisticTransactionDBTest - org.rocksdb.OptionsUtilTest + org.rocksdb.OptimisticTransactionOptionsTest org.rocksdb.OptimisticTransactionTest - org.rocksdb.MutableColumnFamilyOptionsTest - org.rocksdb.CompressionOptionsTest - org.rocksdb.ColumnFamilyTest - org.rocksdb.SstFileReaderTest - org.rocksdb.TransactionDBTest + org.rocksdb.OptionsTest + org.rocksdb.OptionsUtilTest + org.rocksdb.PerfContextTest + org.rocksdb.PerfLevelTest + org.rocksdb.PlainTableConfigTest + org.rocksdb.PutCFVariantsTest + org.rocksdb.PutMultiplePartsTest + org.rocksdb.PutVariantsTest + org.rocksdb.RateLimiterTest + org.rocksdb.ReadOnlyTest + org.rocksdb.ReadOptionsTest + org.rocksdb.RocksDBExceptionTest org.rocksdb.RocksDBTest - org.rocksdb.MutableOptionsGetSetTest - # org.rocksdb.OptimisticTransactionOptionsTest + org.rocksdb.RocksIteratorTest + org.rocksdb.RocksMemEnvTest + org.rocksdb.SecondaryDBTest + org.rocksdb.SliceTest + org.rocksdb.SnapshotTest org.rocksdb.SstFileManagerTest - org.rocksdb.BackupEngineTest - org.rocksdb.DirectSliceTest - org.rocksdb.CompressionTypesTest - org.rocksdb.MemoryUtilTest + org.rocksdb.SstFileReaderTest + org.rocksdb.SstFileWriterTest + org.rocksdb.SstPartitionerTest + org.rocksdb.StatisticsCollectorTest + org.rocksdb.StatisticsTest org.rocksdb.TableFilterTest + org.rocksdb.TimedEnvTest + org.rocksdb.TransactionDBOptionsTest + org.rocksdb.TransactionDBTest + org.rocksdb.TransactionLogIteratorTest + org.rocksdb.TransactionOptionsTest + org.rocksdb.TransactionTest org.rocksdb.TtlDBTest + org.rocksdb.VerifyChecksumsTest + org.rocksdb.WALRecoveryModeTest + org.rocksdb.WalFilterTest + org.rocksdb.WriteBatchHandlerTest + org.rocksdb.WriteBatchTest + org.rocksdb.WriteBatchThreadedTest + org.rocksdb.WriteBatchWithIndexTest + org.rocksdb.WriteOptionsTest + org.rocksdb.util.BytewiseComparatorIntTest + org.rocksdb.util.BytewiseComparatorTest + org.rocksdb.util.EnvironmentTest + org.rocksdb.util.IntComparatorTest + org.rocksdb.util.JNIComparatorTest + org.rocksdb.util.ReverseBytewiseComparatorIntTest + org.rocksdb.util.SizeUnitTest org.rocksdb.util.StdErrLoggerTest ) @@ -877,7 +895,7 @@ foreach (CLAZZ ${JAVA_TEST_RUNNING_CLASSES}) else() add_test( NAME jtest_${CLAZZ} - COMMAND ${Java_JAVA_EXECUTABLE} ${JVMARGS} -ea -Xcheck:jni -Djava.library.path=${PROJECT_BINARY_DIR}/java -classpath ${JAVA_RUN_TESTCLASSPATH}:${ROCKSDBJNI_CLASSES_TEST_JAR_FILE} org.rocksdb.test.RocksJunitRunner ${CLAZZ} + COMMAND ${Java_JAVA_EXECUTABLE} ${JVMARGS} -ea -Xcheck:jni -Djava.library.path=${PROJECT_BINARY_DIR}/java -classpath ${JAVA_RUN_TESTCLASSPATH}:${ROCKSDBJNI_CLASSES_TEST_JAR_FILE}:${CMAKE_CURRENT_BINARY_DIR}/${ROCKSDB_JAR} org.rocksdb.test.RocksJunitRunner ${CLAZZ} ) endif() endforeach(CLAZZ) diff --git a/java/rocksjni/jni_perf_context.cc b/java/rocksjni/jni_perf_context.cc index 8ef5c7d369f..c160c2db1b3 100644 --- a/java/rocksjni/jni_perf_context.cc +++ b/java/rocksjni/jni_perf_context.cc @@ -6,6 +6,7 @@ #include #include "include/org_rocksdb_PerfContext.h" +#include "portal.h" #include "rocksdb/db.h" #include "rocksdb/perf_context.h" @@ -1234,3 +1235,14 @@ jlong Java_org_rocksdb_PerfContext_getNumberAsyncSeek(JNIEnv*, jobject, reinterpret_cast(jpc_handle); return perf_context->number_async_seek; } + +jstring Java_org_rocksdb_PerfContext_toString(JNIEnv* env, jobject, + jlong jpc_handle, + jboolean exclude_zero_counters) { + ROCKSDB_NAMESPACE::PerfContext* perf_context = + reinterpret_cast(jpc_handle); + + auto result = perf_context->ToString(exclude_zero_counters); + + return ROCKSDB_NAMESPACE::JniUtil::toJavaString(env, &result, false); +} diff --git a/java/rocksjni/portal.h b/java/rocksjni/portal.h index 58fdfd9fa5c..8a95b995e56 100644 --- a/java/rocksjni/portal.h +++ b/java/rocksjni/portal.h @@ -7740,7 +7740,8 @@ class SstFileMetaDataJni : public JavaClass { } jmethodID mid = env->GetMethodID( - jclazz, "", "(Ljava/lang/String;Ljava/lang/String;JJJ[B[BJZJJ)V"); + jclazz, "", + "(Ljava/lang/String;Ljava/lang/String;JJJ[B[BJZJJ[B)V"); if (mid == nullptr) { // exception thrown: NoSuchMethodException or OutOfMemoryError return nullptr; @@ -7780,6 +7781,17 @@ class SstFileMetaDataJni : public JavaClass { return nullptr; } + jbyteArray jfile_checksum = ROCKSDB_NAMESPACE::JniUtil::copyBytes( + env, sst_file_meta_data->file_checksum); + if (env->ExceptionCheck()) { + // exception occurred creating java string + env->DeleteLocalRef(jfile_name); + env->DeleteLocalRef(jpath); + env->DeleteLocalRef(jsmallest_key); + env->DeleteLocalRef(jlargest_key); + return nullptr; + } + jobject jsst_file_meta_data = env->NewObject( jclazz, mid, jfile_name, jpath, static_cast(sst_file_meta_data->size), @@ -7788,13 +7800,14 @@ class SstFileMetaDataJni : public JavaClass { jlargest_key, static_cast(sst_file_meta_data->num_reads_sampled), static_cast(sst_file_meta_data->being_compacted), static_cast(sst_file_meta_data->num_entries), - static_cast(sst_file_meta_data->num_deletions)); + static_cast(sst_file_meta_data->num_deletions), jfile_checksum); if (env->ExceptionCheck()) { env->DeleteLocalRef(jfile_name); env->DeleteLocalRef(jpath); env->DeleteLocalRef(jsmallest_key); env->DeleteLocalRef(jlargest_key); + env->DeleteLocalRef(jfile_checksum); return nullptr; } @@ -7803,6 +7816,7 @@ class SstFileMetaDataJni : public JavaClass { env->DeleteLocalRef(jpath); env->DeleteLocalRef(jsmallest_key); env->DeleteLocalRef(jlargest_key); + env->DeleteLocalRef(jfile_checksum); return jsst_file_meta_data; } diff --git a/java/src/main/java/org/rocksdb/BlockBasedTableConfig.java b/java/src/main/java/org/rocksdb/BlockBasedTableConfig.java index d066131458c..c8159db2ddc 100644 --- a/java/src/main/java/org/rocksdb/BlockBasedTableConfig.java +++ b/java/src/main/java/org/rocksdb/BlockBasedTableConfig.java @@ -31,7 +31,7 @@ public BlockBasedTableConfig() { indexBlockRestartInterval = 1; metadataBlockSize = 4096; partitionFilters = false; - optimizeFiltersForMemory = false; + optimizeFiltersForMemory = true; useDeltaEncoding = true; filterPolicy = null; wholeKeyFiltering = true; diff --git a/java/src/main/java/org/rocksdb/PerfContext.java b/java/src/main/java/org/rocksdb/PerfContext.java index 3934e4115cf..2d9ac320309 100644 --- a/java/src/main/java/org/rocksdb/PerfContext.java +++ b/java/src/main/java/org/rocksdb/PerfContext.java @@ -651,6 +651,15 @@ public long getNumberAsyncSeek() { return getNumberAsyncSeek(nativeHandle_); } + @Override + public String toString() { + return toString(true); + } + + public String toString(final boolean excludeZeroCounters) { + return toString(nativeHandle_, excludeZeroCounters); + } + @Override protected void disposeInternal(long handle) { // Nothing to do. Perf context is valid for all the time of application is running. @@ -758,4 +767,6 @@ protected void disposeInternal(long handle) { private native long getEncryptDataNanos(long nativeHandle_); private native long getDecryptDataNanos(long nativeHandle_); private native long getNumberAsyncSeek(long nativeHandle_); + + private native String toString(final long nativeHandle, final boolean excludeZeroCounters); } diff --git a/java/src/test/java/org/rocksdb/CompactionOptionsTest.java b/java/src/test/java/org/rocksdb/CompactionOptionsTest.java index 9b7d7969458..b3820b8d172 100644 --- a/java/src/test/java/org/rocksdb/CompactionOptionsTest.java +++ b/java/src/test/java/org/rocksdb/CompactionOptionsTest.java @@ -20,10 +20,9 @@ public class CompactionOptionsTest { public void compression() { try (final CompactionOptions compactionOptions = new CompactionOptions()) { assertThat(compactionOptions.compression()) - .isEqualTo(CompressionType.SNAPPY_COMPRESSION); - compactionOptions.setCompression(CompressionType.NO_COMPRESSION); - assertThat(compactionOptions.compression()) - .isEqualTo(CompressionType.NO_COMPRESSION); + .isEqualTo(CompressionType.DISABLE_COMPRESSION_OPTION); + compactionOptions.setCompression(CompressionType.SNAPPY_COMPRESSION); + assertThat(compactionOptions.compression()).isEqualTo(CompressionType.SNAPPY_COMPRESSION); } } diff --git a/java/src/test/java/org/rocksdb/DefaultEnvTest.java b/java/src/test/java/org/rocksdb/DefaultEnvTest.java index 3fb563ecbbc..cf61f70f27b 100644 --- a/java/src/test/java/org/rocksdb/DefaultEnvTest.java +++ b/java/src/test/java/org/rocksdb/DefaultEnvTest.java @@ -5,16 +5,16 @@ package org.rocksdb; +import static org.assertj.core.api.Assertions.assertThat; + +import java.nio.charset.StandardCharsets; +import java.util.Collection; +import java.util.List; import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; -import java.util.Collection; -import java.util.List; - -import static org.assertj.core.api.Assertions.assertThat; - public class DefaultEnvTest { @ClassRule @@ -88,9 +88,14 @@ public void lowerThreadPoolCPUPriority() { @Test public void threadList() throws RocksDBException { - try (final Env defaultEnv = RocksEnv.getDefault()) { - final Collection threadList = defaultEnv.getThreadList(); - assertThat(threadList.size()).isGreaterThan(0); + // We need to open DB first to get at least one thread in thread list. + try (final RocksDB db = RocksDB.open(dbFolder.getRoot().getAbsolutePath())) { + db.put("test-key".getBytes(StandardCharsets.UTF_8), + "test-value".getBytes(StandardCharsets.UTF_8)); + try (final Env defaultEnv = RocksEnv.getDefault()) { + final Collection threadList = defaultEnv.getThreadList(); + assertThat(threadList.size()).isGreaterThan(0); + } } } diff --git a/java/src/test/java/org/rocksdb/OptimisticTransactionOptionsTest.java b/java/src/test/java/org/rocksdb/OptimisticTransactionOptionsTest.java index ef656b95893..ac291576490 100644 --- a/java/src/test/java/org/rocksdb/OptimisticTransactionOptionsTest.java +++ b/java/src/test/java/org/rocksdb/OptimisticTransactionOptionsTest.java @@ -5,18 +5,23 @@ package org.rocksdb; -import org.junit.Test; -import org.rocksdb.util.BytewiseComparator; +import static org.assertj.core.api.Assertions.assertThat; import java.util.Random; - -import static org.assertj.core.api.Assertions.assertThat; +import org.junit.BeforeClass; +import org.junit.Test; +import org.rocksdb.util.BytewiseComparator; public class OptimisticTransactionOptionsTest { private static final Random rand = PlatformRandomHelper. getPlatformSpecificRandomFactory(); + @BeforeClass + public static void beforeAll() { + RocksDB.loadLibrary(); + } + @Test public void setSnapshot() { try (final OptimisticTransactionOptions opt = new OptimisticTransactionOptions()) { diff --git a/java/src/test/java/org/rocksdb/OptionsUtilTest.java b/java/src/test/java/org/rocksdb/OptionsUtilTest.java index e14fb58fa69..0998ae83fa7 100644 --- a/java/src/test/java/org/rocksdb/OptionsUtilTest.java +++ b/java/src/test/java/org/rocksdb/OptionsUtilTest.java @@ -289,7 +289,7 @@ private void verifyTableFormatOptions(final LoaderUnderTest loaderUnderTest) altCFTableConfig.setIndexBlockRestartInterval(6); altCFTableConfig.setMetadataBlockSize(12 * 1024); altCFTableConfig.setPartitionFilters(true); - altCFTableConfig.setOptimizeFiltersForMemory(true); + altCFTableConfig.setOptimizeFiltersForMemory(false); altCFTableConfig.setUseDeltaEncoding(false); altCFTableConfig.setFilterPolicy(new BloomFilter(7.5)); altCFTableConfig.setWholeKeyFiltering(false); diff --git a/java/src/test/java/org/rocksdb/PerfContextTest.java b/java/src/test/java/org/rocksdb/PerfContextTest.java index 5be1c47e928..62d032721bb 100644 --- a/java/src/test/java/org/rocksdb/PerfContextTest.java +++ b/java/src/test/java/org/rocksdb/PerfContextTest.java @@ -102,4 +102,26 @@ public void testGetPostProcessTime() throws RocksDBException { assertThat(ctx).isNotNull(); assertThat(ctx.getPostProcessTime()).isGreaterThan(0); } + + @Test + public void testToString() throws RocksDBException { + db.setPerfLevel(PerfLevel.ENABLE_TIME_AND_CPU_TIME_EXCEPT_FOR_MUTEX); + db.put("key".getBytes(), "value".getBytes()); + db.compactRange(); + db.get("key".getBytes()); + PerfContext ctx = db.getPerfContext(); + assertThat(ctx).isNotNull(); + assertThat(ctx.toString(false)).isNotEmpty(); + } + + @Test + public void testDefaultToString() throws RocksDBException { + db.setPerfLevel(PerfLevel.ENABLE_TIME_AND_CPU_TIME_EXCEPT_FOR_MUTEX); + db.put("key".getBytes(), "value".getBytes()); + db.compactRange(); + db.get("key".getBytes()); + PerfContext ctx = db.getPerfContext(); + assertThat(ctx).isNotNull(); + assertThat(ctx.toString()).isNotEmpty(); + } } diff --git a/java/src/test/java/org/rocksdb/RocksDBExceptionTest.java b/java/src/test/java/org/rocksdb/RocksDBExceptionTest.java index d3bd4ece7f1..94a65693059 100644 --- a/java/src/test/java/org/rocksdb/RocksDBExceptionTest.java +++ b/java/src/test/java/org/rocksdb/RocksDBExceptionTest.java @@ -5,14 +5,19 @@ package org.rocksdb; -import org.junit.Test; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.Assert.fail; +import org.junit.BeforeClass; +import org.junit.Test; import org.rocksdb.Status.Code; import org.rocksdb.Status.SubCode; -import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.Assert.fail; public class RocksDBExceptionTest { + @BeforeClass + public static void beforeAll() { + RocksDB.loadLibrary(); + } @Test public void exception() { diff --git a/java/src/test/java/org/rocksdb/RocksDBTest.java b/java/src/test/java/org/rocksdb/RocksDBTest.java index 18b82929aa8..1459f03b05a 100644 --- a/java/src/test/java/org/rocksdb/RocksDBTest.java +++ b/java/src/test/java/org/rocksdb/RocksDBTest.java @@ -1423,6 +1423,33 @@ public void getLiveFilesMetadataWithChecksum() throws RocksDBException { } } + @Test + public void getColumnFamilyMetadataWithChecksum() throws RocksDBException { + final Properties props = new Properties(); + props.put("file_checksum_gen_factory", "FileChecksumGenCrc32cFactory"); + final String dbPath = dbFolder.getRoot().getAbsolutePath(); + + try (final DBOptions dbOptions = DBOptions.getDBOptionsFromProps(props); + final ColumnFamilyOptions cfOptions = new ColumnFamilyOptions(); + final Options options = new Options(dbOptions, cfOptions).setCreateIfMissing(true)) { + try (final RocksDB db = RocksDB.open(options, dbPath); + final WriteOptions writeOptions = new WriteOptions().setDisableWAL(true)) { + db.put("key".getBytes(UTF_8), "value".getBytes(UTF_8)); + } + + try (final RocksDB db = RocksDB.open(options, dbFolder.getRoot().getAbsolutePath())) { + ColumnFamilyMetaData metadata = db.getColumnFamilyMetaData(); // Exception here + List levels = metadata.levels(); + assertThat(levels).isNotEmpty(); + List filesMetadata = levels.get(0).files(); + assertThat(filesMetadata).isNotEmpty(); + assertThat(filesMetadata.get(0).fileChecksum()).isNotNull(); + assertThat(filesMetadata.get(0).fileChecksum()).hasSize(4); + assertThat(filesMetadata.get(0).fileChecksum()).isNotEqualTo(new byte[] {0, 0, 0, 0}); + } + } + } + @Ignore("TODO(AR) re-enable when ready!") @Test public void compactFiles() throws RocksDBException { diff --git a/java/src/test/java/org/rocksdb/TransactionDBOptionsTest.java b/java/src/test/java/org/rocksdb/TransactionDBOptionsTest.java index 7eaa6b16cdf..bd8357b95f5 100644 --- a/java/src/test/java/org/rocksdb/TransactionDBOptionsTest.java +++ b/java/src/test/java/org/rocksdb/TransactionDBOptionsTest.java @@ -5,13 +5,18 @@ package org.rocksdb; -import org.junit.Test; +import static org.assertj.core.api.Assertions.assertThat; import java.util.Random; - -import static org.assertj.core.api.Assertions.assertThat; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; public class TransactionDBOptionsTest { + @BeforeClass + public static void beforeAll() { + RocksDB.loadLibrary(); + } private static final Random rand = PlatformRandomHelper. getPlatformSpecificRandomFactory(); diff --git a/java/src/test/java/org/rocksdb/TransactionOptionsTest.java b/java/src/test/java/org/rocksdb/TransactionOptionsTest.java index add0439e03a..1dc397b4534 100644 --- a/java/src/test/java/org/rocksdb/TransactionOptionsTest.java +++ b/java/src/test/java/org/rocksdb/TransactionOptionsTest.java @@ -5,14 +5,17 @@ package org.rocksdb; -import org.junit.Test; +import static org.assertj.core.api.Assertions.assertThat; import java.util.Random; - -import static org.assertj.core.api.Assertions.assertThat; +import org.junit.BeforeClass; +import org.junit.Test; public class TransactionOptionsTest { - + @BeforeClass + public static void beforeAll() { + RocksDB.loadLibrary(); + } private static final Random rand = PlatformRandomHelper. getPlatformSpecificRandomFactory(); diff --git a/memory/memory_allocator.cc b/memory/memory_allocator.cc index d0de26b94d5..fe183e403f0 100644 --- a/memory/memory_allocator.cc +++ b/memory/memory_allocator.cc @@ -77,4 +77,5 @@ Status MemoryAllocator::CreateFromString( copy.invoke_prepare_options = true; return LoadManagedObject(copy, value, result); } + } // namespace ROCKSDB_NAMESPACE diff --git a/options/db_options.cc b/options/db_options.cc index c7d227254e0..6da12a12156 100644 --- a/options/db_options.cc +++ b/options/db_options.cc @@ -37,6 +37,7 @@ static std::unordered_map static std::unordered_map cache_tier_string_map = { {"kVolatileTier", CacheTier::kVolatileTier}, + {"kVolatileCompressedTier", CacheTier::kVolatileCompressedTier}, {"kNonVolatileBlockTier", CacheTier::kNonVolatileBlockTier}}; static std::unordered_map info_log_level_string_map = @@ -558,6 +559,19 @@ static std::unordered_map {offsetof(struct ImmutableDBOptions, enforce_single_del_contracts), OptionType::kBoolean, OptionVerificationType::kNormal, OptionTypeFlags::kNone}}, + {"follower_refresh_catchup_period_ms", + {offsetof(struct ImmutableDBOptions, + follower_refresh_catchup_period_ms), + OptionType::kUInt64T, OptionVerificationType::kNormal, + OptionTypeFlags::kNone}}, + {"follower_catchup_retry_count", + {offsetof(struct ImmutableDBOptions, follower_catchup_retry_count), + OptionType::kUInt64T, OptionVerificationType::kNormal, + OptionTypeFlags::kNone}}, + {"follower_catchup_retry_wait_ms", + {offsetof(struct ImmutableDBOptions, follower_catchup_retry_wait_ms), + OptionType::kUInt64T, OptionVerificationType::kNormal, + OptionTypeFlags::kNone}}, }; const std::string OptionsHelper::kDBOptionsName = "DBOptions"; @@ -755,7 +769,11 @@ ImmutableDBOptions::ImmutableDBOptions(const DBOptions& options) checksum_handoff_file_types(options.checksum_handoff_file_types), lowest_used_cache_tier(options.lowest_used_cache_tier), compaction_service(options.compaction_service), - enforce_single_del_contracts(options.enforce_single_del_contracts) { + enforce_single_del_contracts(options.enforce_single_del_contracts), + follower_refresh_catchup_period_ms( + options.follower_refresh_catchup_period_ms), + follower_catchup_retry_count(options.follower_catchup_retry_count), + follower_catchup_retry_wait_ms(options.follower_catchup_retry_wait_ms) { fs = env->GetFileSystem(); clock = env->GetSystemClock().get(); logger = info_log.get(); diff --git a/options/db_options.h b/options/db_options.h index b0432df81f5..ff7ddc880ff 100644 --- a/options/db_options.h +++ b/options/db_options.h @@ -104,6 +104,9 @@ struct ImmutableDBOptions { Logger* logger; std::shared_ptr compaction_service; bool enforce_single_del_contracts; + uint64_t follower_refresh_catchup_period_ms; + uint64_t follower_catchup_retry_count; + uint64_t follower_catchup_retry_wait_ms; bool IsWalDirSameAsDBPath() const; bool IsWalDirSameAsDBPath(const std::string& path) const; diff --git a/src.mk b/src.mk index b06ad05edb3..23cf348e1eb 100644 --- a/src.mk +++ b/src.mk @@ -12,8 +12,9 @@ LIB_SOURCES = \ cache/secondary_cache.cc \ cache/secondary_cache_adapter.cc \ cache/sharded_cache.cc \ - cache/tiered_secondary_cache.cc \ + cache/tiered_secondary_cache.cc \ db/arena_wrapped_db_iter.cc \ + db/attribute_group_iterator_impl.cc \ db/blob/blob_contents.cc \ db/blob/blob_fetcher.cc \ db/blob/blob_file_addition.cc \ @@ -30,6 +31,7 @@ LIB_SOURCES = \ db/blob/prefetch_buffer_collection.cc \ db/builder.cc \ db/c.cc \ + db/coalescing_iterator.cc \ db/column_family.cc \ db/compaction/compaction.cc \ db/compaction/compaction_iterator.cc \ @@ -51,6 +53,7 @@ LIB_SOURCES = \ db/db_impl/db_impl_debug.cc \ db/db_impl/db_impl_experimental.cc \ db/db_impl/db_impl_files.cc \ + db/db_impl/db_impl_follower.cc \ db/db_impl/db_impl_open.cc \ db/db_impl/db_impl_readonly.cc \ db/db_impl/db_impl_secondary.cc \ @@ -76,7 +79,6 @@ LIB_SOURCES = \ db/memtable_list.cc \ db/merge_helper.cc \ db/merge_operator.cc \ - db/multi_cf_iterator.cc \ db/output_validator.cc \ db/periodic_task_scheduler.cc \ db/range_del_aggregator.cc \ @@ -108,6 +110,7 @@ LIB_SOURCES = \ env/env_encryption.cc \ env/env_posix.cc \ env/file_system.cc \ + env/fs_on_demand.cc \ env/fs_posix.cc \ env/fs_remap.cc \ env/file_system_tracer.cc \ @@ -320,6 +323,7 @@ LIB_SOURCES = \ utilities/transactions/write_unprepared_txn.cc \ utilities/transactions/write_unprepared_txn_db.cc \ utilities/ttl/db_ttl_impl.cc \ + utilities/types_util.cc \ utilities/wal_filter.cc \ utilities/write_batch_with_index/write_batch_with_index.cc \ utilities/write_batch_with_index/write_batch_with_index_internal.cc \ @@ -472,6 +476,7 @@ TEST_MAIN_SOURCES = \ db/db_dynamic_level_test.cc \ db/db_encryption_test.cc \ db/db_flush_test.cc \ + db/db_follower_test.cc \ db/db_readonly_with_timestamp_test.cc \ db/db_with_timestamp_basic_test.cc \ db/import_column_family_test.cc \ @@ -633,6 +638,7 @@ TEST_MAIN_SOURCES = \ utilities/transactions/write_committed_transaction_ts_test.cc \ utilities/transactions/timestamped_snapshot_test.cc \ utilities/ttl/ttl_test.cc \ + utilities/types_util_test.cc \ utilities/util_merge_operators_test.cc \ utilities/write_batch_with_index/write_batch_with_index_test.cc \ diff --git a/table/block_based/block_based_table_builder.cc b/table/block_based/block_based_table_builder.cc index eb2ae4ddba9..060a541e396 100644 --- a/table/block_based/block_based_table_builder.cc +++ b/table/block_based/block_based_table_builder.cc @@ -626,6 +626,14 @@ struct BlockBasedTableBuilder::Rep { } else { base_context_checksum = 0; } + + if (alignment > 0 && compression_type != kNoCompression) { + // With better sanitization in `CompactionPicker::CompactFiles()`, we + // would not need to handle this case here and could change it to an + // assertion instead. + SetStatus(Status::InvalidArgument( + "Enable block_align, but compression enabled")); + } } Rep(const Rep&) = delete; diff --git a/table/block_based/block_based_table_factory.cc b/table/block_based/block_based_table_factory.cc index 6da594c10d6..34081621a4c 100644 --- a/table/block_based/block_based_table_factory.cc +++ b/table/block_based/block_based_table_factory.cc @@ -619,6 +619,21 @@ Status BlockBasedTableFactory::ValidateOptions( "Enable block_align, but compression " "enabled"); } + if (table_options_.block_align && + cf_opts.bottommost_compression != kDisableCompressionOption && + cf_opts.bottommost_compression != kNoCompression) { + return Status::InvalidArgument( + "Enable block_align, but bottommost_compression enabled"); + } + if (table_options_.block_align) { + for (auto level_compression : cf_opts.compression_per_level) { + if (level_compression != kDisableCompressionOption && + level_compression != kNoCompression) { + return Status::InvalidArgument( + "Enable block_align, but compression_per_level enabled"); + } + } + } if (table_options_.block_align && (table_options_.block_size & (table_options_.block_size - 1))) { return Status::InvalidArgument( diff --git a/table/block_based/filter_policy.cc b/table/block_based/filter_policy.cc index f3c3fb256b2..3cd63ffad80 100644 --- a/table/block_based/filter_policy.cc +++ b/table/block_based/filter_policy.cc @@ -425,6 +425,9 @@ class FastLocalBloomBitsBuilder : public XXPH3FilterBitsBuilder { } double EstimatedFpRate(size_t keys, size_t len_with_metadata) override { + if (len_with_metadata <= kMetadataLen) { + return keys > 0 ? 1.0 : 0.0; + } int num_probes = GetNumProbes(keys, len_with_metadata); return FastLocalBloomImpl::EstimatedFpRate( keys, len_with_metadata - kMetadataLen, num_probes, /*hash bits*/ 64); @@ -891,6 +894,9 @@ class Standard128RibbonBitsBuilder : public XXPH3FilterBitsBuilder { double EstimatedFpRate(size_t num_entries, size_t len_with_metadata) override { + if (len_with_metadata <= kMetadataLen) { + return num_entries > 0 ? 1.0 : 0.0; + } if (num_entries > kMaxRibbonEntries) { // More entries than supported by this Ribbon return bloom_fallback_.EstimatedFpRate(num_entries, len_with_metadata); @@ -1030,9 +1036,12 @@ class LegacyBloomBitsBuilder : public BuiltinFilterBitsBuilder { return CalculateSpace(num_entries, &dont_care1, &dont_care2); } - double EstimatedFpRate(size_t keys, size_t bytes) override { - return LegacyBloomImpl::EstimatedFpRate(keys, bytes - kMetadataLen, - num_probes_); + double EstimatedFpRate(size_t keys, size_t len_with_metadata) override { + if (len_with_metadata <= kMetadataLen) { + return keys > 0 ? 1.0 : 0.0; + } + return LegacyBloomImpl::EstimatedFpRate( + keys, len_with_metadata - kMetadataLen, num_probes_); } size_t ApproximateNumEntries(size_t bytes) override; diff --git a/table/sst_file_reader.cc b/table/sst_file_reader.cc index da50ff037c8..9b940df11a3 100644 --- a/table/sst_file_reader.cc +++ b/table/sst_file_reader.cc @@ -15,6 +15,7 @@ #include "rocksdb/file_system.h" #include "table/get_context.h" #include "table/table_builder.h" +#include "table/table_iterator.h" #include "table/table_reader.h" namespace ROCKSDB_NAMESPACE { @@ -24,6 +25,9 @@ struct SstFileReader::Rep { EnvOptions soptions; ImmutableOptions ioptions; MutableCFOptions moptions; + // Keep a member variable for this, since `NewIterator()` uses a const + // reference of `ReadOptions`. + ReadOptions roptions_for_table_iter; std::unique_ptr table_reader; @@ -31,7 +35,10 @@ struct SstFileReader::Rep { : options(opts), soptions(options), ioptions(options), - moptions(ColumnFamilyOptions(options)) {} + moptions(ColumnFamilyOptions(options)) { + roptions_for_table_iter = + ReadOptions(/*_verify_checksums=*/true, /*_fill_cache=*/false); + } }; SstFileReader::SstFileReader(const Options& options) : rep_(new Rep(options)) {} @@ -94,6 +101,21 @@ Iterator* SstFileReader::NewIterator(const ReadOptions& roptions) { return res; } +std::unique_ptr SstFileReader::NewTableIterator() { + auto r = rep_.get(); + InternalIterator* internal_iter = r->table_reader->NewIterator( + r->roptions_for_table_iter, r->moptions.prefix_extractor.get(), + /*arena*/ nullptr, false /* skip_filters */, + TableReaderCaller::kSSTFileReader); + assert(internal_iter); + if (internal_iter == nullptr) { + // Do not attempt to create a TableIterator if we cannot get a valid + // InternalIterator. + return nullptr; + } + return std::make_unique(internal_iter); +} + std::shared_ptr SstFileReader::GetTableProperties() const { return rep_->table_reader->GetTableProperties(); diff --git a/table/sst_file_reader_test.cc b/table/sst_file_reader_test.cc index 597909925a9..c5f8079e457 100644 --- a/table/sst_file_reader_test.cc +++ b/table/sst_file_reader_test.cc @@ -8,10 +8,12 @@ #include +#include "db/db_test_util.h" #include "port/stack_trace.h" #include "rocksdb/convenience.h" #include "rocksdb/db.h" #include "rocksdb/sst_file_writer.h" +#include "rocksdb/utilities/types_util.h" #include "table/sst_file_writer_collectors.h" #include "test_util/testharness.h" #include "test_util/testutil.h" @@ -578,6 +580,195 @@ TEST_F(SstFileReaderTest, VerifyNumEntriesCorruption) { ASSERT_TRUE(std::strstr(oss.str().c_str(), s.getState())); } +class SstFileReaderTableIteratorTest : public DBTestBase { + public: + SstFileReaderTableIteratorTest() + : DBTestBase("sst_file_reader_table_iterator_test", + /*env_do_fsync=*/false) {} + + void VerifyTableEntry(Iterator* iter, const std::string& user_key, + ValueType value_type, + std::optional expected_value, + bool backward_iteration = false) { + ASSERT_TRUE(iter->Valid()); + ASSERT_TRUE(iter->status().ok()); + ParsedInternalKey pikey; + ASSERT_OK(ParseInternalKey(iter->key(), &pikey, /*log_err_key=*/false)); + ASSERT_EQ(pikey.user_key, user_key); + ASSERT_EQ(pikey.type, value_type); + if (expected_value.has_value()) { + ASSERT_EQ(iter->value(), expected_value.value()); + } + if (!backward_iteration) { + iter->Next(); + } else { + iter->Prev(); + } + } +}; + +TEST_F(SstFileReaderTableIteratorTest, Basic) { + Options options = CurrentOptions(); + const Comparator* ucmp = BytewiseComparator(); + options.comparator = ucmp; + options.disable_auto_compactions = true; + + DestroyAndReopen(options); + + // Create a L0 sst file with 4 entries, two for each user key. + // The file should have these entries in ascending internal key order: + // 'bar, seq: 4, type: kTypeValue => val2' + // 'bar, seq: 3, type: kTypeDeletion' + // 'foo, seq: 2, type: kTypeDeletion' + // 'foo, seq: 1, type: kTypeValue => val1' + ASSERT_OK(Put("foo", "val1")); + const Snapshot* snapshot1 = dbfull()->GetSnapshot(); + ASSERT_OK(Delete("foo")); + ASSERT_OK(Delete("bar")); + const Snapshot* snapshot2 = dbfull()->GetSnapshot(); + ASSERT_OK(Put("bar", "val2")); + + ASSERT_OK(Flush()); + + std::vector files; + dbfull()->GetLiveFilesMetaData(&files); + ASSERT_TRUE(files.size() == 1); + ASSERT_TRUE(files[0].level == 0); + std::string file_name = files[0].directory + "/" + files[0].relative_filename; + + SstFileReader reader(options); + ASSERT_OK(reader.Open(file_name)); + ASSERT_OK(reader.VerifyChecksum()); + + // When iterating the file as a DB iterator, only one data entry for "bar" is + // visible. + std::unique_ptr db_iter(reader.NewIterator(ReadOptions())); + db_iter->SeekToFirst(); + ASSERT_TRUE(db_iter->Valid()); + ASSERT_EQ(db_iter->key(), "bar"); + ASSERT_EQ(db_iter->value(), "val2"); + db_iter->Next(); + ASSERT_FALSE(db_iter->Valid()); + db_iter.reset(); + + // When iterating the file with a raw table iterator, all the data entries are + // surfaced in ascending internal key order. + std::unique_ptr table_iter = reader.NewTableIterator(); + + table_iter->SeekToFirst(); + VerifyTableEntry(table_iter.get(), "bar", kTypeValue, "val2"); + VerifyTableEntry(table_iter.get(), "bar", kTypeDeletion, std::nullopt); + VerifyTableEntry(table_iter.get(), "foo", kTypeDeletion, std::nullopt); + VerifyTableEntry(table_iter.get(), "foo", kTypeValue, "val1"); + ASSERT_FALSE(table_iter->Valid()); + + std::string seek_key_buf; + ASSERT_OK(GetInternalKeyForSeek("foo", ucmp, &seek_key_buf)); + Slice seek_target = seek_key_buf; + table_iter->Seek(seek_target); + VerifyTableEntry(table_iter.get(), "foo", kTypeDeletion, std::nullopt); + VerifyTableEntry(table_iter.get(), "foo", kTypeValue, "val1"); + ASSERT_FALSE(table_iter->Valid()); + + ASSERT_OK(GetInternalKeyForSeekForPrev("bar", ucmp, &seek_key_buf)); + Slice seek_for_prev_target = seek_key_buf; + table_iter->SeekForPrev(seek_for_prev_target); + VerifyTableEntry(table_iter.get(), "bar", kTypeDeletion, std::nullopt, + /*backward_iteration=*/true); + VerifyTableEntry(table_iter.get(), "bar", kTypeValue, "val2", + /*backward_iteration=*/true); + ASSERT_FALSE(table_iter->Valid()); + + dbfull()->ReleaseSnapshot(snapshot1); + dbfull()->ReleaseSnapshot(snapshot2); + Close(); +} + +TEST_F(SstFileReaderTableIteratorTest, UserDefinedTimestampsEnabled) { + Options options = CurrentOptions(); + const Comparator* ucmp = test::BytewiseComparatorWithU64TsWrapper(); + options.comparator = ucmp; + options.disable_auto_compactions = true; + + DestroyAndReopen(options); + + // Create a L0 sst file with 4 entries, two for each user key. + // The file should have these entries in ascending internal key order: + // 'bar, ts=3, seq: 4, type: kTypeValue => val2' + // 'bar, ts=2, seq: 3, type: kTypeDeletionWithTimestamp' + // 'foo, ts=4, seq: 2, type: kTypeDeletionWithTimestamp' + // 'foo, ts=3, seq: 1, type: kTypeValue => val1' + WriteOptions wopt; + ColumnFamilyHandle* cfd = db_->DefaultColumnFamily(); + ASSERT_OK(db_->Put(wopt, cfd, "foo", EncodeAsUint64(3), "val1")); + ASSERT_OK(db_->Delete(wopt, cfd, "foo", EncodeAsUint64(4))); + ASSERT_OK(db_->Delete(wopt, cfd, "bar", EncodeAsUint64(2))); + ASSERT_OK(db_->Put(wopt, cfd, "bar", EncodeAsUint64(3), "val2")); + + ASSERT_OK(Flush()); + + std::vector files; + dbfull()->GetLiveFilesMetaData(&files); + ASSERT_TRUE(files.size() == 1); + ASSERT_TRUE(files[0].level == 0); + std::string file_name = files[0].directory + "/" + files[0].relative_filename; + + SstFileReader reader(options); + ASSERT_OK(reader.Open(file_name)); + ASSERT_OK(reader.VerifyChecksum()); + + // When iterating the file as a DB iterator, only one data entry for "bar" is + // visible. + ReadOptions ropts; + std::string read_ts = EncodeAsUint64(4); + Slice read_ts_slice = read_ts; + ropts.timestamp = &read_ts_slice; + std::unique_ptr db_iter(reader.NewIterator(ropts)); + db_iter->SeekToFirst(); + ASSERT_TRUE(db_iter->Valid()); + ASSERT_EQ(db_iter->key(), "bar"); + ASSERT_EQ(db_iter->value(), "val2"); + ASSERT_EQ(db_iter->timestamp(), EncodeAsUint64(3)); + db_iter->Next(); + ASSERT_FALSE(db_iter->Valid()); + db_iter.reset(); + + std::unique_ptr table_iter = reader.NewTableIterator(); + + table_iter->SeekToFirst(); + VerifyTableEntry(table_iter.get(), "bar" + EncodeAsUint64(3), kTypeValue, + "val2"); + VerifyTableEntry(table_iter.get(), "bar" + EncodeAsUint64(2), + kTypeDeletionWithTimestamp, std::nullopt); + VerifyTableEntry(table_iter.get(), "foo" + EncodeAsUint64(4), + kTypeDeletionWithTimestamp, std::nullopt); + VerifyTableEntry(table_iter.get(), "foo" + EncodeAsUint64(3), kTypeValue, + "val1"); + ASSERT_FALSE(table_iter->Valid()); + + std::string seek_key_buf; + ASSERT_OK(GetInternalKeyForSeek("foo", ucmp, &seek_key_buf)); + Slice seek_target = seek_key_buf; + table_iter->Seek(seek_target); + VerifyTableEntry(table_iter.get(), "foo" + EncodeAsUint64(4), + kTypeDeletionWithTimestamp, std::nullopt); + VerifyTableEntry(table_iter.get(), "foo" + EncodeAsUint64(3), kTypeValue, + "val1"); + ASSERT_FALSE(table_iter->Valid()); + + ASSERT_OK(GetInternalKeyForSeekForPrev("bar", ucmp, &seek_key_buf)); + Slice seek_for_prev_target = seek_key_buf; + table_iter->SeekForPrev(seek_for_prev_target); + VerifyTableEntry(table_iter.get(), "bar" + EncodeAsUint64(2), + kTypeDeletionWithTimestamp, std::nullopt, + /*backward_iteration=*/true); + VerifyTableEntry(table_iter.get(), "bar" + EncodeAsUint64(3), kTypeValue, + "val2", /*backward_iteration=*/true); + ASSERT_FALSE(table_iter->Valid()); + + Close(); +} + } // namespace ROCKSDB_NAMESPACE int main(int argc, char** argv) { diff --git a/table/table_iterator.h b/table/table_iterator.h new file mode 100644 index 00000000000..7d18924e2d8 --- /dev/null +++ b/table/table_iterator.h @@ -0,0 +1,69 @@ +// Copyright (c) Meta Platforms, Inc. and affiliates. +// +// 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 "rocksdb/iterator.h" +#include "table/internal_iterator.h" + +namespace ROCKSDB_NAMESPACE { +// An iterator wrapper class used to wrap an `InternalIterator` created by API +// `TableReader::NewIterator`. The `InternalIterator` should be allocated with +// the default allocator, not on an arena. +// NOTE: Callers should ensure the wrapped `InternalIterator*` is a valid +// pointer before constructing a `TableIterator` with it. +class TableIterator : public Iterator { + void reset(InternalIterator* iter) noexcept { + if (iter_ != nullptr) { + delete iter_; + } + iter_ = iter; + } + + public: + explicit TableIterator(InternalIterator* iter) : iter_(iter) {} + + TableIterator(const TableIterator&) = delete; + TableIterator& operator=(const TableIterator&) = delete; + + TableIterator(TableIterator&& o) noexcept { + iter_ = o.iter_; + o.iter_ = nullptr; + } + + TableIterator& operator=(TableIterator&& o) noexcept { + reset(o.iter_); + o.iter_ = nullptr; + return *this; + } + + InternalIterator* operator->() { return iter_; } + InternalIterator* get() { return iter_; } + + ~TableIterator() override { reset(nullptr); } + + bool Valid() const override { return iter_->Valid(); } + void SeekToFirst() override { return iter_->SeekToFirst(); } + void SeekToLast() override { return iter_->SeekToLast(); } + void Seek(const Slice& target) override { return iter_->Seek(target); } + void SeekForPrev(const Slice& target) override { + return iter_->SeekForPrev(target); + } + void Next() override { return iter_->Next(); } + void Prev() override { return iter_->Prev(); } + Slice key() const override { return iter_->key(); } + Slice value() const override { return iter_->value(); } + Status status() const override { return iter_->status(); } + Status GetProperty(std::string /*prop_name*/, + std::string* /*prop*/) override { + assert(false); + return Status::NotSupported("TableIterator does not support GetProperty."); + } + + private: + InternalIterator* iter_; +}; +} // namespace ROCKSDB_NAMESPACE diff --git a/table/table_test.cc b/table/table_test.cc index 02a8d899d77..1ffd53c8fe5 100644 --- a/table/table_test.cc +++ b/table/table_test.cc @@ -3188,6 +3188,7 @@ TEST_P(BlockBasedTableTest, BlockCacheLookupSeqScans) { Options options; BlockBasedTableOptions table_options = GetBlockBasedTableOptions(); options.create_if_missing = true; + options.compression = kNoCompression; options.statistics = CreateDBStatistics(); table_options.index_type = BlockBasedTableOptions::IndexType::kTwoLevelIndexSearch; @@ -3196,6 +3197,8 @@ TEST_P(BlockBasedTableTest, BlockCacheLookupSeqScans) { table_options.filter_policy.reset(NewBloomFilterPolicy(10, true)); table_options.block_align = true; options.table_factory.reset(new BlockBasedTableFactory(table_options)); + ASSERT_OK(options.table_factory->ValidateOptions( + DBOptions(options), ColumnFamilyOptions(options))); TableConstructor c(BytewiseComparator()); GenerateKVMap(&c); @@ -3326,6 +3329,7 @@ TEST_P(BlockBasedTableTest, BlockCacheLookupAsyncScansSeek) { std::unique_ptr env( new CompositeEnvWrapper(c.env_, FileSystem::Default())); options.env = env.get(); + options.compression = kNoCompression; options.statistics = CreateDBStatistics(); c.env_ = env.get(); @@ -3338,6 +3342,8 @@ TEST_P(BlockBasedTableTest, BlockCacheLookupAsyncScansSeek) { table_options.filter_policy.reset(NewBloomFilterPolicy(10, true)); table_options.block_align = true; options.table_factory.reset(new BlockBasedTableFactory(table_options)); + ASSERT_OK(options.table_factory->ValidateOptions( + DBOptions(options), ColumnFamilyOptions(options))); GenerateKVMap(&c); @@ -5425,6 +5431,8 @@ TEST_P(BlockBasedTableTest, BlockAlignTest) { Options options; options.compression = kNoCompression; options.table_factory.reset(NewBlockBasedTableFactory(bbto)); + ASSERT_OK(options.table_factory->ValidateOptions( + DBOptions(options), ColumnFamilyOptions(options))); const ImmutableOptions ioptions(options); const MutableCFOptions moptions(options); InternalKeyComparator ikc(options.comparator); @@ -5475,7 +5483,10 @@ TEST_P(BlockBasedTableTest, BlockAlignTest) { std::unique_ptr table_reader; bbto.block_align = false; Options options2; + options2.compression = kNoCompression; options2.table_factory.reset(NewBlockBasedTableFactory(bbto)); + ASSERT_OK(options2.table_factory->ValidateOptions( + DBOptions(options2), ColumnFamilyOptions(options2))); ImmutableOptions ioptions2(options2); const MutableCFOptions moptions2(options2); @@ -5505,6 +5516,95 @@ TEST_P(BlockBasedTableTest, BlockAlignTest) { table_reader.reset(); } +TEST_P(BlockBasedTableTest, FixBlockAlignMismatchedFileChecksums) { + Options options; + options.create_if_missing = true; + options.compression = kNoCompression; + options.file_checksum_gen_factory = GetFileChecksumGenCrc32cFactory(); + BlockBasedTableOptions bbto; + bbto.block_align = true; + bbto.block_size = 1024; + options.table_factory.reset(NewBlockBasedTableFactory(bbto)); + ASSERT_OK(options.table_factory->ValidateOptions( + DBOptions(options), ColumnFamilyOptions(options))); + const std::string kDBPath = + test::PerThreadDBPath("block_align_padded_bytes_verify_file_checksums"); + ASSERT_OK(DestroyDB(kDBPath, options)); + DB* db; + ASSERT_OK(DB::Open(options, kDBPath, &db)); + ASSERT_OK(db->Put(WriteOptions(), "k1", "v1")); + ASSERT_OK(db->Flush(FlushOptions())); + // Before the fix, VerifyFileChecksums() will fail as padded bytes from + // aligning blocks are used to generate the checksum to compare against the + // one not generated by padded bytes + ASSERT_OK(db->VerifyFileChecksums(ReadOptions())); + delete db; +} + +class NoBufferAlignmenttWritableFile : public FSWritableFileOwnerWrapper { + public: + explicit NoBufferAlignmenttWritableFile( + std::unique_ptr&& file) + : FSWritableFileOwnerWrapper(std::move(file)) {} + size_t GetRequiredBufferAlignment() const override { return 1; } +}; + +class NoBufferAlignmenttWritableFileFileSystem : public FileSystemWrapper { + public: + explicit NoBufferAlignmenttWritableFileFileSystem( + const std::shared_ptr& base) + : FileSystemWrapper(base) {} + + static const char* kClassName() { + return "NoBufferAlignmenttWritableFileFileSystem"; + } + const char* Name() const override { return kClassName(); } + + IOStatus NewWritableFile(const std::string& fname, + const FileOptions& file_opts, + std::unique_ptr* result, + IODebugContext* dbg) override { + IOStatus s = target()->NewWritableFile(fname, file_opts, result, dbg); + EXPECT_OK(s); + result->reset(new NoBufferAlignmenttWritableFile(std::move(*result))); + return s; + } +}; + +TEST_P(BlockBasedTableTest, + FixBlockAlignFlushDuringPadMismatchedFileChecksums) { + Options options; + options.create_if_missing = true; + options.compression = kNoCompression; + options.file_checksum_gen_factory = GetFileChecksumGenCrc32cFactory(); + // To force flush during pad by enforcing a small buffer size + options.writable_file_max_buffer_size = 1; + // To help enforce a small buffer size by removing buffer alignment + Env* raw_env = Env::Default(); + std::shared_ptr fs = + std::make_shared( + raw_env->GetFileSystem()); + std::unique_ptr env(new CompositeEnvWrapper(raw_env, fs)); + options.env = env.get(); + + BlockBasedTableOptions bbto; + bbto.block_align = true; + options.table_factory.reset(NewBlockBasedTableFactory(bbto)); + const std::string kDBPath = test::PerThreadDBPath( + "block_align_flush_during_flush_verify_file_checksums"); + ASSERT_OK(DestroyDB(kDBPath, options)); + DB* db; + ASSERT_OK(DB::Open(options, kDBPath, &db)); + + ASSERT_OK(db->Put(WriteOptions(), "k1", "k2")); + ASSERT_OK(db->Flush(FlushOptions())); + + // Before the fix, VerifyFileChecksums() will fail as incorrect padded bytes + // were used to generate checksum upon file creation + ASSERT_OK(db->VerifyFileChecksums(ReadOptions())); + delete db; +} + TEST_P(BlockBasedTableTest, PropertiesBlockRestartPointTest) { BlockBasedTableOptions bbto = GetBlockBasedTableOptions(); bbto.block_align = true; @@ -5516,6 +5616,8 @@ TEST_P(BlockBasedTableTest, PropertiesBlockRestartPointTest) { Options options; options.compression = kNoCompression; options.table_factory.reset(NewBlockBasedTableFactory(bbto)); + ASSERT_OK(options.table_factory->ValidateOptions( + DBOptions(options), ColumnFamilyOptions(options))); const ImmutableOptions ioptions(options); const MutableCFOptions moptions(options); @@ -5822,6 +5924,7 @@ TEST_P(BlockBasedTableTest, SeekMetaBlocks) { TEST_P(BlockBasedTableTest, BadOptions) { ROCKSDB_NAMESPACE::Options options; options.compression = kNoCompression; + options.create_if_missing = true; BlockBasedTableOptions bbto = GetBlockBasedTableOptions(); bbto.block_size = 4000; bbto.block_align = true; @@ -5830,13 +5933,29 @@ TEST_P(BlockBasedTableTest, BadOptions) { test::PerThreadDBPath("block_based_table_bad_options_test"); options.table_factory.reset(NewBlockBasedTableFactory(bbto)); ASSERT_OK(DestroyDB(kDBPath, options)); - ROCKSDB_NAMESPACE::DB* db; - ASSERT_NOK(ROCKSDB_NAMESPACE::DB::Open(options, kDBPath, &db)); - bbto.block_size = 4096; - options.compression = kSnappyCompression; - options.table_factory.reset(NewBlockBasedTableFactory(bbto)); - ASSERT_NOK(ROCKSDB_NAMESPACE::DB::Open(options, kDBPath, &db)); + std::unique_ptr db; + { + ROCKSDB_NAMESPACE::DB* _db; + ASSERT_NOK(ROCKSDB_NAMESPACE::DB::Open(options, kDBPath, &_db)); + + bbto.block_size = 4096; + options.compression = kSnappyCompression; + options.table_factory.reset(NewBlockBasedTableFactory(bbto)); + ASSERT_NOK(ROCKSDB_NAMESPACE::DB::Open(options, kDBPath, &_db)); + + options.compression = kNoCompression; + options.bottommost_compression = kSnappyCompression; + ASSERT_NOK(ROCKSDB_NAMESPACE::DB::Open(options, kDBPath, &_db)); + + options.bottommost_compression = kNoCompression; + options.compression_per_level.emplace_back(kSnappyCompression); + ASSERT_NOK(ROCKSDB_NAMESPACE::DB::Open(options, kDBPath, &_db)); + + options.compression_per_level.clear(); + ASSERT_OK(ROCKSDB_NAMESPACE::DB::Open(options, kDBPath, &_db)); + db.reset(_db); + } } TEST_F(BBTTailPrefetchTest, TestTailPrefetchStats) { diff --git a/tools/check_format_compatible.sh b/tools/check_format_compatible.sh index fc00556ecd0..6da1000f4ab 100755 --- a/tools/check_format_compatible.sh +++ b/tools/check_format_compatible.sh @@ -125,7 +125,7 @@ EOF # To check for DB forward compatibility with loading options (old version # reading data from new), as well as backward compatibility -declare -a db_forward_with_options_refs=("8.6.fb" "8.7.fb" "8.8.fb" "8.9.fb" "8.10.fb" "8.11.fb" "9.0.fb") +declare -a db_forward_with_options_refs=("8.6.fb" "8.7.fb" "8.8.fb" "8.9.fb" "8.10.fb" "8.11.fb" "9.0.fb" "9.1.fb" "9.2.fb") # To check for DB forward compatibility without loading options (in addition # to the "with loading options" set), as well as backward compatibility declare -a db_forward_no_options_refs=() # N/A at the moment diff --git a/tools/db_bench_tool.cc b/tools/db_bench_tool.cc index d47ffb5385f..ce14bce5da0 100644 --- a/tools/db_bench_tool.cc +++ b/tools/db_bench_tool.cc @@ -1135,6 +1135,11 @@ DEFINE_int32(secondary_update_interval, 5, "Secondary instance attempts to catch up with the primary every " "secondary_update_interval seconds."); +DEFINE_bool(open_as_follower, false, + "Open a RocksDB DB as a follower. The leader instance can be " + "running in another db_bench process."); + +DEFINE_string(leader_path, "", "Path to the directory of the leader DB"); DEFINE_bool(report_bg_io_stats, false, "Measure times spents on I/Os while in compactions. "); @@ -4979,6 +4984,12 @@ class Benchmark { }, FLAGS_secondary_update_interval, db)); } + } else if (FLAGS_open_as_follower) { + std::unique_ptr dbptr; + s = DB::OpenAsFollower(options, db_name, FLAGS_leader_path, &dbptr); + if (s.ok()) { + db->db = dbptr.release(); + } } else { s = DB::Open(options, db_name, &db->db); } diff --git a/tools/db_crashtest.py b/tools/db_crashtest.py index 3fbd5729c2c..2045c4d6998 100644 --- a/tools/db_crashtest.py +++ b/tools/db_crashtest.py @@ -60,9 +60,7 @@ ), "compression_max_dict_bytes": lambda: 16384 * random.randint(0, 1), "compression_zstd_max_train_bytes": lambda: 65536 * random.randint(0, 1), - # Disabled compression_parallel_threads as the feature is not stable - # lambda: random.choice([1] * 9 + [4]) - "compression_parallel_threads": 1, + "compression_parallel_threads": lambda: random.choice([1] * 3 + [4, 8, 16]), "compression_max_dict_buffer_bytes": lambda: (1 << random.randint(0, 40)) - 1, "compression_use_zstd_dict_trainer": lambda: random.randint(0, 1), "compression_checksum": lambda: random.randint(0, 1), @@ -76,12 +74,23 @@ "destroy_db_initially": 0, "enable_pipelined_write": lambda: random.randint(0, 1), "enable_compaction_filter": lambda: random.choice([0, 0, 0, 1]), + # `inplace_update_support` is incompatible with DB that has delete + # range data in memtables. + # Such data can result from any of the previous db stress runs + # using delete range. + # Since there is no easy way to keep track of whether delete range + # is used in any of the previous runs, + # to simpify our testing, we set `inplace_update_support` across + # runs and to disable delete range accordingly + # (see below `finalize_and_sanitize`). + "inplace_update_support": random.choice([0] * 9 + [1]), "expected_values_dir": lambda: setup_expected_values_dir(), "fail_if_options_file_error": lambda: random.randint(0, 1), "flush_one_in": lambda: random.choice([1000, 1000000]), "manual_wal_flush_one_in": lambda: random.choice([0, 1000]), "file_checksum_impl": lambda: random.choice(["none", "crc32c", "xxh64", "big"]), - "get_live_files_one_in": lambda: random.choice([10000, 1000000]), + "get_live_files_apis_one_in": lambda: random.choice([10000, 1000000]), + "get_all_column_family_metadata_one_in": lambda: random.choice([10000, 1000000]), # Note: the following two are intentionally disabled as the corresponding # APIs are not guaranteed to succeed. "get_sorted_wal_files_one_in": 0, @@ -108,6 +117,8 @@ "partition_filters": lambda: random.randint(0, 1), "partition_pinning": lambda: random.randint(0, 3), "pause_background_one_in": lambda: random.choice([10000, 1000000]), + "disable_file_deletions_one_in": lambda: random.choice([10000, 1000000]), + "disable_manual_compaction_one_in": lambda: random.choice([10000, 1000000]), "prefix_size": lambda: random.choice([-1, 1, 5, 7, 8]), "prefixpercent": 5, "progress_reports": 0, @@ -149,6 +160,15 @@ "use_get_entity": lambda: random.choice([0] * 7 + [1]), "use_multi_get_entity": lambda: random.choice([0] * 7 + [1]), "periodic_compaction_seconds": lambda: random.choice([0, 0, 1, 2, 10, 100, 1000]), + "daily_offpeak_time_utc": lambda: random.choice( + [ + "", + "", + "00:00-23:59", + "04:00-08:00", + "23:30-03:15" + ] + ), # 0 = never (used by some), 10 = often (for threading bugs), 600 = default "stats_dump_period_sec": lambda: random.choice([0, 10, 600]), "compaction_ttl": lambda: random.choice([0, 0, 1, 2, 10, 100, 1000]), @@ -161,7 +181,9 @@ # Sync mode might make test runs slower so running it in a smaller chance "sync": lambda: random.choice([1 if t == 0 else 0 for t in range(0, 20)]), "bytes_per_sync": lambda: random.choice([0, 262144]), - "wal_bytes_per_sync": lambda: random.choice([0, 524288]), + # TODO(hx235): Enable `wal_bytes_per_sync` after fixing the DB recovery such + # that it won't recover past the WAL data hole created by this option + "wal_bytes_per_sync": 0, "compaction_readahead_size": lambda: random.choice( [0, 0, 1024 * 1024]), "db_write_buffer_size": lambda: random.choice( @@ -177,8 +199,8 @@ [16, 64, 1024 * 1024, 16 * 1024 * 1024] ), "level_compaction_dynamic_level_bytes": lambda: random.randint(0, 1), - "verify_checksum_one_in": lambda: random.choice([100000, 1000000]), - "verify_file_checksums_one_in": lambda: random.choice([100000, 1000000]), + "verify_checksum_one_in": lambda: random.choice([1000, 1000000]), + "verify_file_checksums_one_in": lambda: random.choice([1000, 1000000]), "verify_db_one_in": lambda: random.choice([10000, 100000]), "continuous_verification_interval": 0, "max_key_len": 3, @@ -190,6 +212,7 @@ "open_read_fault_one_in": lambda: random.choice([0, 0, 32]), "sync_fault_injection": lambda: random.randint(0, 1), "get_property_one_in": lambda: random.choice([100000, 1000000]), + "get_properties_of_all_tables_one_in": lambda: random.choice([100000, 1000000]), "paranoid_file_checks": lambda: random.choice([0, 1, 1, 1]), "max_write_buffer_size_to_maintain": lambda: random.choice( [0, 1024 * 1024, 2 * 1024 * 1024, 4 * 1024 * 1024, 8 * 1024 * 1024] @@ -240,7 +263,6 @@ "advise_random_on_open": lambda: random.choice([0] + [1] * 3), "WAL_ttl_seconds": lambda: random.choice([0, 60]), "WAL_size_limit_MB": lambda: random.choice([0, 1]), - "wal_bytes_per_sync": lambda: random.choice([0, 1024 * 1024]), "strict_bytes_per_sync": lambda: random.choice([0, 1]), "avoid_flush_during_shutdown": lambda: random.choice([0, 1]), "fill_cache": lambda: random.choice([0, 1]), @@ -262,12 +284,27 @@ "use_adaptive_mutex_lru": lambda: random.choice([0, 1]), "compress_format_version": lambda: random.choice([1, 2]), "manifest_preallocation_size": lambda: random.choice([0, 5 * 1024]), - "enable_checksum_handoff": 0, + "enable_checksum_handoff": lambda: random.choice([0, 1]), "max_total_wal_size": lambda: random.choice([0] * 4 + [64 * 1024 * 1024]), "high_pri_pool_ratio": lambda: random.choice([0, 0.5]), "low_pri_pool_ratio": lambda: random.choice([0, 0.5]), "soft_pending_compaction_bytes_limit" : lambda: random.choice([1024 * 1024] + [64 * 1073741824] * 4), "hard_pending_compaction_bytes_limit" : lambda: random.choice([2 * 1024 * 1024] + [256 * 1073741824] * 4), + "enable_sst_partitioner_factory": lambda: random.choice([0, 1]), + "enable_do_not_compress_roles": lambda: random.choice([0, 1]), + "block_align": lambda: random.choice([0, 1]), + "lowest_used_cache_tier": lambda: random.choice([0, 1, 2]), + "enable_custom_split_merge": lambda: random.choice([0, 1]), + "adm_policy": lambda: random.choice([0, 1, 2, 3]), + "last_level_temperature": lambda: random.choice(["kUnknown", "kHot", "kWarm", "kCold"]), + "default_write_temperature": lambda: random.choice(["kUnknown", "kHot", "kWarm", "kCold"]), + "default_temperature": lambda: random.choice(["kUnknown", "kHot", "kWarm", "kCold"]), + # TODO(hx235): enable `enable_memtable_insert_with_hint_prefix_extractor` + # after fixing the surfaced issue with delete range + "enable_memtable_insert_with_hint_prefix_extractor": 0, + "check_multiget_consistency": lambda: random.choice([0, 0, 0, 1]), + "check_multiget_entity_consistency": lambda: random.choice([0, 0, 0, 1]), + "use_timed_put_one_in": lambda: random.choice([0] * 7 + [1, 5, 10]), } _TEST_DIR_ENV_VAR = "TEST_TMPDIR" # If TEST_TMPDIR_EXPECTED is not specified, default value will be TEST_TMPDIR @@ -425,8 +462,9 @@ def is_direct_io_supported(dbname): "write_buffer_size": 1024 * 1024, "enable_pipelined_write": lambda: random.randint(0, 1), # Snapshots are used heavily in this test mode, while they are incompatible - # with compaction filter. + # with compaction filter, inplace_update_support "enable_compaction_filter": 0, + "inplace_update_support": 0, # `CfConsistencyStressTest::TestIngestExternalFile()` is not implemented. "ingest_external_file_one_in": 0, # `CfConsistencyStressTest::TestIterateAgainstExpected()` is not implemented. @@ -450,6 +488,10 @@ def is_direct_io_supported(dbname): "create_timestamped_snapshot_one_in": random.choice([0, 20]), # PutEntity in transactions is not yet implemented "use_put_entity_one_in": 0, + # Should not be used with TransactionDB which uses snapshot. + "inplace_update_support": 0, + # TimedPut is not supported in transaction + "use_timed_put_one_in": 0, } # For optimistic transaction db @@ -461,6 +503,10 @@ def is_direct_io_supported(dbname): "occ_lock_bucket_count": lambda: random.choice([10, 100, 500]), # PutEntity in transactions is not yet implemented "use_put_entity_one_in": 0, + # Should not be used with OptimisticTransactionDB which uses snapshot. + "inplace_update_support": 0, + # TimedPut is not supported in transaction + "use_timed_put_one_in": 0, } best_efforts_recovery_params = { @@ -506,10 +552,11 @@ def is_direct_io_supported(dbname): "ingest_external_file_one_in": 0, # PutEntity with timestamps is not yet implemented "use_put_entity_one_in": 0, + # TimedPut is not compatible with user-defined timestamps yet. + "use_timed_put_one_in": 0, } tiered_params = { - "enable_tiered_storage": 1, # Set tiered compaction hot data time as: 1 minute, 1 hour, 10 hour "preclude_last_level_data_seconds": lambda: random.choice([60, 3600, 36000]), # only test universal compaction for now, level has known issue of @@ -518,6 +565,8 @@ def is_direct_io_supported(dbname): # tiered storage doesn't support blob db yet "enable_blob_files": 0, "use_blob_db": 0, + "default_write_temperature": lambda: random.choice(["kUnknown", "kHot", "kWarm"]), + "last_level_temperature": "kCold", } multiops_txn_default_params = { @@ -569,6 +618,10 @@ def is_direct_io_supported(dbname): "use_multi_get_entity": 0, # `MultiOpsTxnsStressTest::TestIterateAgainstExpected()` is not implemented. "verify_iterator_with_expected_state_one_in": 0, + # This test uses snapshot heavily which is incompatible with this option. + "inplace_update_support": 0, + # TimedPut not supported in transaction + "use_timed_put_one_in": 0, } multiops_wc_txn_params = { @@ -605,8 +658,6 @@ def finalize_and_sanitize(src_params): dest_params["compression_max_dict_buffer_bytes"] = 0 if dest_params.get("compression_type") != "zstd": dest_params["compression_zstd_max_train_bytes"] = 0 - if dest_params.get("allow_concurrent_memtable_write", 1) == 1: - dest_params["memtablerep"] = "skip_list" if dest_params["mmap_read"] == 1: dest_params["use_direct_io_for_flush_and_compaction"] = 0 dest_params["use_direct_reads"] = 0 @@ -626,6 +677,7 @@ def finalize_and_sanitize(src_params): if dest_params["test_batches_snapshots"] == 1: dest_params["enable_compaction_filter"] = 0 + dest_params["inplace_update_support"] = 0 if dest_params["prefix_size"] < 0: dest_params["prefix_size"] = 1 @@ -643,6 +695,11 @@ def finalize_and_sanitize(src_params): ): dest_params["delpercent"] += dest_params["delrangepercent"] dest_params["delrangepercent"] = 0 + if dest_params["inplace_update_support"] == 1: + dest_params["delpercent"] += dest_params["delrangepercent"] + dest_params["delrangepercent"] = 0 + dest_params["readpercent"] += dest_params["prefixpercent"] + dest_params["prefixpercent"] = 0 if ( dest_params.get("disable_wal") == 1 or dest_params.get("sync_fault_injection") == 1 @@ -660,6 +717,11 @@ def finalize_and_sanitize(src_params): # files, which would be problematic when unsynced data can be lost in # crash recoveries. dest_params["enable_compaction_filter"] = 0 + # TODO(hx235): re-enable "reopen" after supporting unsynced data loss + # verification upon reopen. Currently reopen does not restore expected state + # with potential data loss in mind like start of each `./db_stress` run. + # Therefore it always expects no data loss. + dest_params["reopen"] = 0 # Only under WritePrepared txns, unordered_write would provide the same guarnatees as vanilla rocksdb if dest_params.get("unordered_write", 0) == 1: dest_params["txn_write_policy"] = 1 @@ -686,15 +748,6 @@ def finalize_and_sanitize(src_params): dest_params["enable_pipelined_write"] = 0 if dest_params.get("sst_file_manager_bytes_per_sec", 0) == 0: dest_params["sst_file_manager_bytes_per_truncate"] = 0 - if dest_params.get("enable_compaction_filter", 0) == 1: - # Compaction filter is incompatible with snapshots. Need to avoid taking - # snapshots, as well as avoid operations that use snapshots for - # verification. - dest_params["acquire_snapshot_one_in"] = 0 - dest_params["compact_range_one_in"] = 0 - # Give the iterator ops away to reads. - dest_params["readpercent"] += dest_params.get("iterpercent", 10) - dest_params["iterpercent"] = 0 if dest_params.get("prefix_size") == -1: dest_params["readpercent"] += dest_params.get("prefixpercent", 20) dest_params["prefixpercent"] = 0 @@ -765,16 +818,45 @@ def finalize_and_sanitize(src_params): # with each other. There is no external APIs to ensure that. dest_params["use_multiget"] = 0 dest_params["use_multi_get_entity"] = 0 - dest_params["readpercent"] += dest_params.get("iterpercent", 10); + dest_params["readpercent"] += dest_params.get("iterpercent", 10) dest_params["iterpercent"] = 0 # Only best efforts recovery test support disabling wal and # disable atomic flush. if dest_params["test_best_efforts_recovery"] == 0: dest_params["disable_wal"] = 0 + if dest_params.get("allow_concurrent_memtable_write", 1) == 1: + dest_params["memtablerep"] = "skip_list" + dest_params["inplace_update_support"] = 0 + if (dest_params.get("enable_compaction_filter", 0) == 1 + or dest_params.get("inplace_update_support", 0) == 1): + # Compaction filter, inplace update support are incompatible with snapshots. Need to avoid taking + # snapshots, as well as avoid operations that use snapshots for + # verification. + dest_params["acquire_snapshot_one_in"] = 0 + dest_params["compact_range_one_in"] = 0 + # Give the iterator ops away to reads. + dest_params["readpercent"] += dest_params.get("iterpercent", 10) + dest_params["iterpercent"] = 0 + dest_params["check_multiget_consistency"] = 0 + dest_params["check_multiget_entity_consistency"] = 0 if dest_params.get("disable_wal") == 1: # disableWAL and recycle_log_file_num options are not mutually # compatible at the moment dest_params["recycle_log_file_num"] = 0 + # Enabling block_align with compression is not supported + if dest_params.get("block_align") == 1: + dest_params["compression_type"] = "none" + dest_params["bottommost_compression_type"] = "none" + # If periodic_compaction_seconds is not set, daily_offpeak_time_utc doesn't do anything + if dest_params.get("periodic_compaction_seconds") == 0: + dest_params["daily_offpeak_time_utc"] = "" + # `use_put_entity_one_in` cannot be enabled/disabled across runs, modify + # `use_timed_put_one_in` option so that they make sense together. + if dest_params.get("use_put_entity_one_in") == 1: + dest_params["use_timed_put_one_in"] = 0 + elif (dest_params.get("use_put_entity_one_in") > 1 and + dest_params.get("use_timed_put_one_in") == 1): + dest_params["use_timed_put_one_in"] = 3 return dest_params diff --git a/tools/ldb_cmd.cc b/tools/ldb_cmd.cc index 355e26fa7c0..1f0e8b4ea51 100644 --- a/tools/ldb_cmd.cc +++ b/tools/ldb_cmd.cc @@ -214,6 +214,10 @@ LDBCommand* LDBCommand::SelectCommand(const ParsedParams& parsed_params) { } else if (parsed_params.cmd == GetEntityCommand::Name()) { return new GetEntityCommand(parsed_params.cmd_params, parsed_params.option_map, parsed_params.flags); + } else if (parsed_params.cmd == MultiGetEntityCommand::Name()) { + return new MultiGetEntityCommand(parsed_params.cmd_params, + parsed_params.option_map, + parsed_params.flags); } else if (parsed_params.cmd == PutCommand::Name()) { return new PutCommand(parsed_params.cmd_params, parsed_params.option_map, parsed_params.flags); @@ -2938,7 +2942,7 @@ void MultiGetCommand::DoCommand() { fprintf(stderr, "Status for key %s: %s\n", (is_key_hex_ ? StringToHex(keys_[i]) : keys_[i]).c_str(), statuses[i].ToString().c_str()); - failed = false; + failed = true; } } if (failed) { @@ -2998,6 +3002,73 @@ void GetEntityCommand::DoCommand() { // ---------------------------------------------------------------------------- +MultiGetEntityCommand::MultiGetEntityCommand( + const std::vector& params, + const std::map& options, + const std::vector& flags) + : LDBCommand(options, flags, true /* is_read_only */, + BuildCmdLineOptions({ARG_HEX, ARG_KEY_HEX, ARG_VALUE_HEX})) { + if (params.size() < 1) { + exec_state_ = LDBCommandExecuteResult::Failed( + "At least one must be specified for the multi_get_entity " + "command"); + } else { + for (size_t i = 0; i < params.size(); i++) { + std::string key = params.at(i); + keys_.emplace_back(is_key_hex_ ? HexToString(key) : key); + } + } +} + +void MultiGetEntityCommand::Help(std::string& ret) { + ret.append(" "); + ret.append(MultiGetEntityCommand::Name()); + ret.append(" ..."); + ret.append("\n"); +} + +void MultiGetEntityCommand::DoCommand() { + if (!db_) { + assert(GetExecuteState().IsFailed()); + return; + } + + size_t num_keys = keys_.size(); + std::vector key_slices; + std::vector results(num_keys); + std::vector statuses(num_keys); + for (const std::string& key : keys_) { + key_slices.emplace_back(key); + } + + db_->MultiGetEntity(ReadOptions(), GetCfHandle(), num_keys, key_slices.data(), + results.data(), statuses.data()); + + bool failed = false; + for (size_t i = 0; i < num_keys; ++i) { + std::string key = is_key_hex_ ? StringToHex(keys_[i]) : keys_[i]; + if (statuses[i].ok()) { + std::ostringstream oss; + oss << key << DELIM; + WideColumnsHelper::DumpWideColumns(results[i].columns(), oss, + is_value_hex_); + fprintf(stdout, "%s\n", oss.str().c_str()); + } else if (statuses[i].IsNotFound()) { + fprintf(stdout, "Key not found: %s\n", key.c_str()); + } else { + fprintf(stderr, "Status for key %s: %s\n", key.c_str(), + statuses[i].ToString().c_str()); + failed = true; + } + } + if (failed) { + exec_state_ = + LDBCommandExecuteResult::Failed("one or more keys had non-okay status"); + } +} + +// ---------------------------------------------------------------------------- + ApproxSizeCommand::ApproxSizeCommand( const std::vector& /*params*/, const std::map& options, @@ -3473,7 +3544,7 @@ PutEntityCommand::PutEntityCommand( void PutEntityCommand::Help(std::string& ret) { ret.append(" "); - ret.append(PutCommand::Name()); + ret.append(PutEntityCommand::Name()); ret.append( " : : " "<...>"); diff --git a/tools/ldb_cmd_impl.h b/tools/ldb_cmd_impl.h index e5d0d1fe91c..15ff571b219 100644 --- a/tools/ldb_cmd_impl.h +++ b/tools/ldb_cmd_impl.h @@ -435,6 +435,22 @@ class GetEntityCommand : public LDBCommand { std::string key_; }; +class MultiGetEntityCommand : public LDBCommand { + public: + static std::string Name() { return "multi_get_entity"; } + + MultiGetEntityCommand(const std::vector& params, + const std::map& options, + const std::vector& flags); + + void DoCommand() override; + + static void Help(std::string& ret); + + private: + std::vector keys_; +}; + class ApproxSizeCommand : public LDBCommand { public: static std::string Name() { return "approxsize"; } diff --git a/tools/ldb_tool.cc b/tools/ldb_tool.cc index 381f70c4d16..86fd37ef4ff 100644 --- a/tools/ldb_tool.cc +++ b/tools/ldb_tool.cc @@ -89,8 +89,11 @@ void LDBCommandRunner::PrintHelp(const LDBOptions& ldb_options, ret.append("\n\n"); ret.append("Data Access Commands:\n"); PutCommand::Help(ret); + PutEntityCommand::Help(ret); GetCommand::Help(ret); + GetEntityCommand::Help(ret); MultiGetCommand::Help(ret); + MultiGetEntityCommand::Help(ret); BatchPutCommand::Help(ret); ScanCommand::Help(ret); DeleteCommand::Help(ret); diff --git a/util/autovector.h b/util/autovector.h index 39c7aabee57..56aa4a3314b 100644 --- a/util/autovector.h +++ b/util/autovector.h @@ -371,6 +371,9 @@ autovector& autovector::assign( // copy array num_stack_items_ = other.num_stack_items_; + for (size_t i = 0; i < num_stack_items_; ++i) { + new ((void*)(&values_[i])) value_type(); + } std::copy(other.values_, other.values_ + num_stack_items_, values_); return *this; @@ -385,6 +388,7 @@ autovector& autovector::operator=( num_stack_items_ = n; other.num_stack_items_ = 0; for (size_t i = 0; i < n; ++i) { + new ((void*)(&values_[i])) value_type(); values_[i] = std::move(other.values_[i]); } return *this; diff --git a/util/bloom_test.cc b/util/bloom_test.cc index b0a5cae5661..f3dbe637353 100644 --- a/util/bloom_test.cc +++ b/util/bloom_test.cc @@ -290,6 +290,9 @@ TEST_P(FullBloomTest, FullSmall) { } TEST_P(FullBloomTest, FullVaryingLengths) { + // Match how this test was originally built + table_options_.optimize_filters_for_memory = false; + char buffer[sizeof(int)]; // Count number of filters that significantly exceed the false positive rate @@ -335,6 +338,9 @@ TEST_P(FullBloomTest, FullVaryingLengths) { } TEST_P(FullBloomTest, OptimizeForMemory) { + // Verify default option + EXPECT_EQ(BlockBasedTableOptions().optimize_filters_for_memory, true); + char buffer[sizeof(int)]; for (bool offm : {true, false}) { table_options_.optimize_filters_for_memory = offm; @@ -354,8 +360,9 @@ TEST_P(FullBloomTest, OptimizeForMemory) { Build(); size_t size = FilterData().size(); total_size += size; - // optimize_filters_for_memory currently depends on malloc_usable_size - // but we run the rest of the test to ensure no bad behavior without it. + // optimize_filters_for_memory currently only has an effect with + // malloc_usable_size support, but we run the rest of the test to ensure + // no bad behavior without it. #ifdef ROCKSDB_MALLOC_USABLE_SIZE size = malloc_usable_size(const_cast(FilterData().data())); #endif // ROCKSDB_MALLOC_USABLE_SIZE @@ -508,6 +515,9 @@ inline uint32_t SelectByCacheLineSize(uint32_t for64, uint32_t for128, // ability to read filters generated using other cache line sizes. // See RawSchema. TEST_P(FullBloomTest, Schema) { + // Match how this test was originally built + table_options_.optimize_filters_for_memory = false; + #define EXPECT_EQ_Bloom(a, b) \ { \ if (GetParam() != kStandard128Ribbon) { \ diff --git a/util/coding_test.cc b/util/coding_test.cc index 79dd7b82e92..83d89506149 100644 --- a/util/coding_test.cc +++ b/util/coding_test.cc @@ -127,7 +127,7 @@ TEST(Coding, Varint64) { values.push_back(power); values.push_back(power - 1); values.push_back(power + 1); - }; + } std::string s; for (unsigned int i = 0; i < values.size(); i++) { diff --git a/util/comparator.cc b/util/comparator.cc index a5d7a7ca0bd..78e54aaf40e 100644 --- a/util/comparator.cc +++ b/util/comparator.cc @@ -271,6 +271,11 @@ class ComparatorWithU64TsImpl : public Comparator { return -CompareTimestamp(ExtractTimestampFromUserKey(a, ts_sz), ExtractTimestampFromUserKey(b, ts_sz)); } + + Slice GetMaxTimestamp() const override { return MaxU64Ts(); } + + Slice GetMinTimestamp() const override { return MinU64Ts(); } + using Comparator::CompareWithoutTimestamp; int CompareWithoutTimestamp(const Slice& a, bool a_has_ts, const Slice& b, bool b_has_ts) const override { diff --git a/util/xxhash.h b/util/xxhash.h index 3413ebf855e..2bf111b9565 100644 --- a/util/xxhash.h +++ b/util/xxhash.h @@ -2329,7 +2329,7 @@ XXH32_finalize(xxh_u32 hash, const xxh_u8* ptr, size_t len, XXH_alignment align) hash = XXH_rotl32(hash, 17) * XXH_PRIME32_4; \ } while (0) - if (ptr==NULL) XXH_ASSERT(len == 0) + if (ptr==nullptr) XXH_ASSERT(len == 0) /* Compact rerolled version; generally faster */ if (!XXH32_ENDJMP) { @@ -2409,7 +2409,7 @@ XXH32_endian_align(const xxh_u8* input, size_t len, xxh_u32 seed, XXH_alignment { xxh_u32 h32; - if (input==NULL) XXH_ASSERT(len == 0) + if (input==nullptr) XXH_ASSERT(len == 0) if (len>=16) { const xxh_u8* const bEnd = input + len; @@ -2495,7 +2495,7 @@ XXH_PUBLIC_API XXH_errorcode XXH32_reset(XXH32_state_t* statePtr, XXH32_hash_t s XXH_PUBLIC_API XXH_errorcode XXH32_update(XXH32_state_t* state, const void* input, size_t len) { - if (input==NULL) { + if (input==nullptr) { XXH_ASSERT(len == 0) return XXH_OK; } @@ -2802,7 +2802,7 @@ static xxh_u64 XXH64_avalanche(xxh_u64 hash) static XXH_PUREF xxh_u64 XXH64_finalize(xxh_u64 hash, const xxh_u8* ptr, size_t len, XXH_alignment align) { - if (ptr==NULL) XXH_ASSERT(len == 0) + if (ptr==nullptr) XXH_ASSERT(len == 0) len &= 31; while (len >= 8) { xxh_u64 const k1 = XXH64_round(0, XXH_get64bits(ptr)); @@ -2847,7 +2847,7 @@ XXH_FORCE_INLINE XXH_PUREF xxh_u64 XXH64_endian_align(const xxh_u8* input, size_t len, xxh_u64 seed, XXH_alignment align) { xxh_u64 h64; - if (input==NULL) XXH_ASSERT(len == 0) + if (input==nullptr) XXH_ASSERT(len == 0) if (len>=32) { const xxh_u8* const bEnd = input + len; @@ -2936,7 +2936,7 @@ XXH_PUBLIC_API XXH_errorcode XXH64_reset(XXH_NOESCAPE XXH64_state_t* statePtr, X XXH_PUBLIC_API XXH_errorcode XXH64_update (XXH_NOESCAPE XXH64_state_t* state, XXH_NOESCAPE const void* input, size_t len) { - if (input==NULL) { + if (input==nullptr) { XXH_ASSERT(len == 0) return XXH_OK; } @@ -5331,7 +5331,7 @@ XXH_PUBLIC_API XXH64_hash_t XXH3_64bits_withSecretandSeed(XXH_NOESCAPE const void* input, size_t length, XXH_NOESCAPE const void* secret, size_t secretSize, XXH64_hash_t seed) { if (length <= XXH3_MIDSIZE_MAX) - return XXH3_64bits_internal(input, length, seed, XXH3_kSecret, sizeof(XXH3_kSecret), NULL); + return XXH3_64bits_internal(input, length, seed, XXH3_kSecret, sizeof(XXH3_kSecret), nullptr); return XXH3_hashLong_64b_withSecret(input, length, seed, (const xxh_u8*)secret, secretSize); } @@ -5368,7 +5368,7 @@ static XXH_MALLOCF void* XXH_alignedMalloc(size_t s, size_t align) XXH_ASSERT(s != 0 && s < (s + align)) /* empty/overflow */ { /* Overallocate to make room for manual realignment and an offset byte */ xxh_u8* base = (xxh_u8*)XXH_malloc(s + align); - if (base != NULL) { + if (base != nullptr) { /* * Get the offset needed to align this pointer. * @@ -5385,7 +5385,7 @@ static XXH_MALLOCF void* XXH_alignedMalloc(size_t s, size_t align) ptr[-1] = (xxh_u8)offset; return ptr; } - return NULL; + return nullptr; } } /* @@ -5394,7 +5394,7 @@ static XXH_MALLOCF void* XXH_alignedMalloc(size_t s, size_t align) */ static void XXH_alignedFree(void* p) { - if (p != NULL) { + if (p != nullptr) { xxh_u8* ptr = (xxh_u8*)p; /* Get the offset byte we added in XXH_malloc. */ xxh_u8 offset = ptr[-1]; @@ -5407,7 +5407,7 @@ static void XXH_alignedFree(void* p) XXH_PUBLIC_API XXH3_state_t* XXH3_createState(void) { XXH3_state_t* const state = (XXH3_state_t*)XXH_alignedMalloc(sizeof(XXH3_state_t), 64); - if (state==NULL) return NULL; + if (state==nullptr) return nullptr; XXH3_INITSTATE(state); return state; } @@ -5457,7 +5457,7 @@ XXH3_reset_internal(XXH3_state_t* statePtr, XXH_PUBLIC_API XXH_errorcode XXH3_64bits_reset(XXH_NOESCAPE XXH3_state_t* statePtr) { - if (statePtr == NULL) return XXH_ERROR; + if (statePtr == nullptr) return XXH_ERROR; XXH3_reset_internal(statePtr, 0, XXH3_kSecret, XXH_SECRET_DEFAULT_SIZE); return XXH_OK; } @@ -5466,9 +5466,9 @@ XXH3_64bits_reset(XXH_NOESCAPE XXH3_state_t* statePtr) XXH_PUBLIC_API XXH_errorcode XXH3_64bits_reset_withSecret(XXH_NOESCAPE XXH3_state_t* statePtr, XXH_NOESCAPE const void* secret, size_t secretSize) { - if (statePtr == NULL) return XXH_ERROR; + if (statePtr == nullptr) return XXH_ERROR; XXH3_reset_internal(statePtr, 0, secret, secretSize); - if (secret == NULL) return XXH_ERROR; + if (secret == nullptr) return XXH_ERROR; if (secretSize < XXH3_SECRET_SIZE_MIN) return XXH_ERROR; return XXH_OK; } @@ -5477,11 +5477,11 @@ XXH3_64bits_reset_withSecret(XXH_NOESCAPE XXH3_state_t* statePtr, XXH_NOESCAPE c XXH_PUBLIC_API XXH_errorcode XXH3_64bits_reset_withSeed(XXH_NOESCAPE XXH3_state_t* statePtr, XXH64_hash_t seed) { - if (statePtr == NULL) return XXH_ERROR; + if (statePtr == nullptr) return XXH_ERROR; if (seed==0) return XXH3_64bits_reset(statePtr); - if ((seed != statePtr->seed) || (statePtr->extSecret != NULL)) + if ((seed != statePtr->seed) || (statePtr->extSecret != nullptr)) XXH3_initCustomSecret(statePtr->customSecret, seed); - XXH3_reset_internal(statePtr, seed, NULL, XXH_SECRET_DEFAULT_SIZE); + XXH3_reset_internal(statePtr, seed, nullptr, XXH_SECRET_DEFAULT_SIZE); return XXH_OK; } @@ -5489,8 +5489,8 @@ XXH3_64bits_reset_withSeed(XXH_NOESCAPE XXH3_state_t* statePtr, XXH64_hash_t see XXH_PUBLIC_API XXH_errorcode XXH3_64bits_reset_withSecretandSeed(XXH_NOESCAPE XXH3_state_t* statePtr, XXH_NOESCAPE const void* secret, size_t secretSize, XXH64_hash_t seed64) { - if (statePtr == NULL) return XXH_ERROR; - if (secret == NULL) return XXH_ERROR; + if (statePtr == nullptr) return XXH_ERROR; + if (secret == nullptr) return XXH_ERROR; if (secretSize < XXH3_SECRET_SIZE_MIN) return XXH_ERROR; XXH3_reset_internal(statePtr, seed64, secret, secretSize); statePtr->useSeed = 1; /* always, even if seed64==0 */ @@ -5538,14 +5538,14 @@ XXH3_update(XXH3_state_t* XXH_RESTRICT const state, XXH3_f_accumulate f_acc, XXH3_f_scrambleAcc f_scramble) { - if (input==NULL) { + if (input==nullptr) { XXH_ASSERT(len == 0) return XXH_OK; } XXH_ASSERT(state != NULL) { const xxh_u8* const bEnd = input + len; - const unsigned char* const secret = (state->extSecret == NULL) ? state->customSecret : state->extSecret; + const unsigned char* const secret = (state->extSecret == nullptr) ? state->customSecret : state->extSecret; #if defined(XXH3_STREAM_USE_STACK) && XXH3_STREAM_USE_STACK >= 1 /* For some reason, gcc and MSVC seem to suffer greatly * when operating accumulators directly into state. @@ -5693,7 +5693,7 @@ XXH3_digest_long (XXH64_hash_t* acc, /*! @ingroup XXH3_family */ XXH_PUBLIC_API XXH64_hash_t XXH3_64bits_digest (XXH_NOESCAPE const XXH3_state_t* state) { - const unsigned char* const secret = (state->extSecret == NULL) ? state->customSecret : state->extSecret; + const unsigned char* const secret = (state->extSecret == nullptr) ? state->customSecret : state->extSecret; if (state->totalLen > XXH3_MIDSIZE_MAX) { XXH_ALIGN(XXH_ACC_ALIGN) XXH64_hash_t acc[XXH_ACC_NB]; XXH3_digest_long(acc, state, secret); @@ -6131,7 +6131,7 @@ XXH_PUBLIC_API XXH128_hash_t XXH3_128bits_withSecretandSeed(XXH_NOESCAPE const void* input, size_t len, XXH_NOESCAPE const void* secret, size_t secretSize, XXH64_hash_t seed) { if (len <= XXH3_MIDSIZE_MAX) - return XXH3_128bits_internal(input, len, seed, XXH3_kSecret, sizeof(XXH3_kSecret), NULL); + return XXH3_128bits_internal(input, len, seed, XXH3_kSecret, sizeof(XXH3_kSecret), nullptr); return XXH3_hashLong_128b_withSecret(input, len, seed, secret, secretSize); } @@ -6189,7 +6189,7 @@ XXH3_128bits_update(XXH_NOESCAPE XXH3_state_t* state, XXH_NOESCAPE const void* i /*! @ingroup XXH3_family */ XXH_PUBLIC_API XXH128_hash_t XXH3_128bits_digest (XXH_NOESCAPE const XXH3_state_t* state) { - const unsigned char* const secret = (state->extSecret == NULL) ? state->customSecret : state->extSecret; + const unsigned char* const secret = (state->extSecret == nullptr) ? state->customSecret : state->extSecret; if (state->totalLen > XXH3_MIDSIZE_MAX) { XXH_ALIGN(XXH_ACC_ALIGN) XXH64_hash_t acc[XXH_ACC_NB]; XXH3_digest_long(acc, state, secret); @@ -6287,7 +6287,7 @@ XXH3_generateSecret(XXH_NOESCAPE void* secretBuffer, size_t secretSize, XXH_NOES XXH_ASSERT(secretSize >= XXH3_SECRET_SIZE_MIN) #else /* production mode, assert() are disabled */ - if (secretBuffer == NULL) return XXH_ERROR; + if (secretBuffer == nullptr) return XXH_ERROR; if (secretSize < XXH3_SECRET_SIZE_MIN) return XXH_ERROR; #endif @@ -6298,7 +6298,7 @@ XXH3_generateSecret(XXH_NOESCAPE void* secretBuffer, size_t secretSize, XXH_NOES #if (XXH_DEBUGLEVEL >= 1) XXH_ASSERT(customSeed != NULL) #else - if (customSeed == NULL) return XXH_ERROR; + if (customSeed == nullptr) return XXH_ERROR; #endif /* Fill secretBuffer with a copy of customSeed - repeat as needed */ diff --git a/utilities/blob_db/blob_db_impl.h b/utilities/blob_db/blob_db_impl.h index 6cbc0d594a5..9fc9deb5e03 100644 --- a/utilities/blob_db/blob_db_impl.h +++ b/utilities/blob_db/blob_db_impl.h @@ -162,6 +162,9 @@ class BlobDBImpl : public BlobDB { Status GetLiveFiles(std::vector&, uint64_t* manifest_file_size, bool flush_memtable = true) override; void GetLiveFilesMetaData(std::vector*) override; + Status GetLiveFilesStorageInfo( + const LiveFilesStorageInfoOptions& opts, + std::vector* files) override; ~BlobDBImpl(); diff --git a/utilities/blob_db/blob_db_impl_filesnapshot.cc b/utilities/blob_db/blob_db_impl_filesnapshot.cc index 7f9189b12eb..b669a42e8d7 100644 --- a/utilities/blob_db/blob_db_impl_filesnapshot.cc +++ b/utilities/blob_db/blob_db_impl_filesnapshot.cc @@ -104,4 +104,24 @@ void BlobDBImpl::GetLiveFilesMetaData(std::vector* metadata) { } } +Status BlobDBImpl::GetLiveFilesStorageInfo( + const LiveFilesStorageInfoOptions& opts, + std::vector* files) { + ReadLock rl(&mutex_); + Status s = db_->GetLiveFilesStorageInfo(opts, files); + if (s.ok()) { + files->reserve(files->size() + blob_files_.size()); + for (const auto& [blob_number, blob_file] : blob_files_) { + LiveFileStorageInfo file; + file.size = blob_file->GetFileSize(); + file.directory = blob_dir_; + file.relative_filename = BlobFileName(blob_number); + file.file_type = kBlobFile; + file.trim_to_size = true; + files->push_back(std::move(file)); + } + } + return s; +} + } // namespace ROCKSDB_NAMESPACE::blob_db diff --git a/utilities/blob_db/blob_db_test.cc b/utilities/blob_db/blob_db_test.cc index 3f57cbfaa2d..923c8b46a99 100644 --- a/utilities/blob_db/blob_db_test.cc +++ b/utilities/blob_db/blob_db_test.cc @@ -1013,6 +1013,27 @@ TEST_F(BlobDBTest, GetLiveFilesMetaData) { ASSERT_EQ(5U, livefile.size()); ASSERT_EQ(filename1, livefile[3]); ASSERT_EQ(filename2, livefile[4]); + + std::vector all_files, blob_files; + ASSERT_OK(blob_db_->GetLiveFilesStorageInfo(LiveFilesStorageInfoOptions(), + &all_files)); + for (size_t i = 0; i < all_files.size(); i++) { + if (all_files[i].file_type == kBlobFile) { + blob_files.push_back(all_files[i]); + } + } + + ASSERT_EQ(2U, blob_files.size()); + ASSERT_GT(all_files.size(), blob_files.size()); + + ASSERT_EQ("000001.blob", blob_files[0].relative_filename); + ASSERT_EQ(blob_db_impl()->TEST_blob_dir(), blob_files[0].directory); + ASSERT_GT(blob_files[0].size, 0); + + ASSERT_EQ("000002.blob", blob_files[1].relative_filename); + ASSERT_EQ(blob_db_impl()->TEST_blob_dir(), blob_files[1].directory); + ASSERT_GT(blob_files[1].size, 0); + VerifyDB(data); } diff --git a/utilities/cache_dump_load_impl.cc b/utilities/cache_dump_load_impl.cc index 52f2a4df7d9..f0b2a73448c 100644 --- a/utilities/cache_dump_load_impl.cc +++ b/utilities/cache_dump_load_impl.cc @@ -3,18 +3,20 @@ // COPYING file in the root directory) and Apache 2.0 License // (found in the LICENSE.Apache file in the root directory). -#include "cache/cache_key.h" -#include "table/block_based/block_based_table_reader.h" +#include "utilities/cache_dump_load_impl.h" + +#include #include "cache/cache_entry_roles.h" +#include "cache/cache_key.h" #include "file/writable_file_writer.h" #include "port/lang.h" #include "rocksdb/env.h" #include "rocksdb/file_system.h" #include "rocksdb/utilities/ldb_cmd.h" +#include "table/block_based/block_based_table_reader.h" #include "table/format.h" #include "util/crc32c.h" -#include "utilities/cache_dump_load_impl.h" namespace ROCKSDB_NAMESPACE { @@ -24,6 +26,7 @@ namespace ROCKSDB_NAMESPACE { // requirement. Status CacheDumperImpl::SetDumpFilter(std::vector db_list) { Status s = Status::OK(); + dump_all_keys_ = false; for (size_t i = 0; i < db_list.size(); i++) { assert(i < db_list.size()); TablePropertiesCollection ptc; @@ -67,6 +70,8 @@ IOStatus CacheDumperImpl::DumpCacheEntriesToWriter() { } clock_ = options_.clock; + deadline_ = options_.deadline; + // Set the sequence number sequence_num_ = 0; @@ -117,6 +122,19 @@ CacheDumperImpl::DumpOneBlockCallBack(std::string& buf) { return; } + if (options_.max_size_bytes > 0 && + dumped_size_bytes_ > options_.max_size_bytes) { + return; + } + + uint64_t timestamp = clock_->NowMicros(); + if (deadline_.count()) { + std::chrono::microseconds now = std::chrono::microseconds(timestamp); + if (now >= deadline_) { + return; + } + } + CacheEntryRole role = helper->role; CacheDumpUnitType type = CacheDumpUnitType::kBlockTypeMax; @@ -140,7 +158,7 @@ CacheDumperImpl::DumpOneBlockCallBack(std::string& buf) { } // based on the key prefix, check if the block should be filter out. - if (ShouldFilterOut(key)) { + if (!dump_all_keys_ && ShouldFilterOut(key)) { return; } @@ -154,7 +172,8 @@ CacheDumperImpl::DumpOneBlockCallBack(std::string& buf) { if (s.ok()) { // Write it out - WriteBlock(type, key, buf).PermitUncheckedError(); + WriteBlock(type, key, buf, timestamp).PermitUncheckedError(); + dumped_size_bytes_ += len; } }; } @@ -168,8 +187,7 @@ CacheDumperImpl::DumpOneBlockCallBack(std::string& buf) { // First, we write the metadata first, which is a fixed size string. Then, we // Append the dump unit string to the writer. IOStatus CacheDumperImpl::WriteBlock(CacheDumpUnitType type, const Slice& key, - const Slice& value) { - uint64_t timestamp = clock_->NowMicros(); + const Slice& value, uint64_t timestamp) { uint32_t value_checksum = crc32c::Value(value.data(), value.size()); // First, serialize the block information in a string @@ -219,7 +237,8 @@ IOStatus CacheDumperImpl::WriteHeader() { "block_size, block_data, block_checksum> cache_value\n"; std::string header_value(s.str()); CacheDumpUnitType type = CacheDumpUnitType::kHeader; - return WriteBlock(type, header_key, header_value); + uint64_t timestamp = clock_->NowMicros(); + return WriteBlock(type, header_key, header_value, timestamp); } // Write the footer after all the blocks are stored to indicate the ending. @@ -227,7 +246,8 @@ IOStatus CacheDumperImpl::WriteFooter() { std::string footer_key = "footer"; std::string footer_value("cache dump completed"); CacheDumpUnitType type = CacheDumpUnitType::kFooter; - return WriteBlock(type, footer_key, footer_value); + uint64_t timestamp = clock_->NowMicros(); + return WriteBlock(type, footer_key, footer_value, timestamp); } // This is the main function to restore the cache entries to secondary cache. diff --git a/utilities/cache_dump_load_impl.h b/utilities/cache_dump_load_impl.h index 4ec42ef633f..ee892f47488 100644 --- a/utilities/cache_dump_load_impl.h +++ b/utilities/cache_dump_load_impl.h @@ -96,14 +96,16 @@ class CacheDumperImpl : public CacheDumper { CacheDumperImpl(const CacheDumpOptions& dump_options, const std::shared_ptr& cache, std::unique_ptr&& writer) - : options_(dump_options), cache_(cache), writer_(std::move(writer)) {} + : options_(dump_options), cache_(cache), writer_(std::move(writer)) { + dumped_size_bytes_ = 0; + } ~CacheDumperImpl() { writer_.reset(); } Status SetDumpFilter(std::vector db_list) override; IOStatus DumpCacheEntriesToWriter() override; private: IOStatus WriteBlock(CacheDumpUnitType type, const Slice& key, - const Slice& value); + const Slice& value, uint64_t timestamp); IOStatus WriteHeader(); IOStatus WriteFooter(); bool ShouldFilterOut(const Slice& key); @@ -121,6 +123,11 @@ class CacheDumperImpl : public CacheDumper { // improvement can be applied like BloomFilter or others to speedup the // filtering. std::set prefix_filter_; + // Deadline for dumper in microseconds. + std::chrono::microseconds deadline_; + uint64_t dumped_size_bytes_; + // dump all keys of cache if user doesn't call SetDumpFilter + bool dump_all_keys_ = true; }; // The default implementation of CacheDumpedLoader @@ -187,8 +194,12 @@ class ToFileCacheDumpWriter : public CacheDumpWriter { // Reset the writer IOStatus Close() override { + IOStatus io_s; + if (file_writer_ != nullptr && !file_writer_->seen_error()) { + io_s = file_writer_->Sync(IOOptions(), false /* use_fsync */); + } file_writer_.reset(); - return IOStatus::OK(); + return io_s; } private: diff --git a/utilities/fault_injection_fs.cc b/utilities/fault_injection_fs.cc index 0ffb43ea60d..25ccbab173d 100644 --- a/utilities/fault_injection_fs.cc +++ b/utilities/fault_injection_fs.cc @@ -255,15 +255,10 @@ IOStatus TestFSWritableFile::Close(const IOOptions& options, } writable_file_opened_ = false; IOStatus io_s; - if (!target_->use_direct_io()) { - io_s = target_->Append(state_.buffer_, options, dbg); - } - if (io_s.ok()) { - state_.buffer_.resize(0); - // Ignore sync errors - target_->Sync(options, dbg).PermitUncheckedError(); - io_s = target_->Close(options, dbg); - } + // Drop buffered data that was never synced because close is not a syncing + // mechanism in POSIX file semantics. + state_.buffer_.resize(0); + io_s = target_->Close(options, dbg); if (io_s.ok()) { IOStatus in_s = fs_->InjectMetadataWriteError(); if (!in_s.ok()) { @@ -633,6 +628,17 @@ IOStatus FaultInjectionTestFS::ReopenWritableFile( return io_s; } +IOStatus FaultInjectionTestFS::ReuseWritableFile( + const std::string& fname, const std::string& old_fname, + const FileOptions& file_opts, std::unique_ptr* result, + IODebugContext* dbg) { + IOStatus s = RenameFile(old_fname, fname, file_opts.io_options, dbg); + if (!s.ok()) { + return s; + } + return NewWritableFile(fname, file_opts, result, dbg); +} + IOStatus FaultInjectionTestFS::NewRandomRWFile( const std::string& fname, const FileOptions& file_opts, std::unique_ptr* result, IODebugContext* dbg) { diff --git a/utilities/fault_injection_fs.h b/utilities/fault_injection_fs.h index 356d21f5701..01487b6848e 100644 --- a/utilities/fault_injection_fs.h +++ b/utilities/fault_injection_fs.h @@ -210,6 +210,7 @@ class FaultInjectionTestFS : public FileSystemWrapper { metadata_write_error_one_in_(0), read_error_one_in_(0), ingest_data_corruption_before_write_(false), + checksum_handoff_func_type_(kCRC32c), fail_get_file_unique_id_(false) {} virtual ~FaultInjectionTestFS() { error_.PermitUncheckedError(); } @@ -230,6 +231,12 @@ class FaultInjectionTestFS : public FileSystemWrapper { std::unique_ptr* result, IODebugContext* dbg) override; + IOStatus ReuseWritableFile(const std::string& fname, + const std::string& old_fname, + const FileOptions& file_opts, + std::unique_ptr* result, + IODebugContext* dbg) override; + IOStatus NewRandomRWFile(const std::string& fname, const FileOptions& file_opts, std::unique_ptr* result, @@ -368,12 +375,12 @@ class FaultInjectionTestFS : public FileSystemWrapper { void SetChecksumHandoffFuncType(const ChecksumType& func_type) { MutexLock l(&mutex_); - checksum_handoff_func_tpye_ = func_type; + checksum_handoff_func_type_ = func_type; } const ChecksumType& GetChecksumHandoffFuncType() { MutexLock l(&mutex_); - return checksum_handoff_func_tpye_; + return checksum_handoff_func_type_; } void SetFailGetUniqueId(bool flag) { @@ -581,7 +588,7 @@ class FaultInjectionTestFS : public FileSystemWrapper { // File types where direct writable is skipped. std::set direct_writable_types_; bool ingest_data_corruption_before_write_; - ChecksumType checksum_handoff_func_tpye_; + ChecksumType checksum_handoff_func_type_; bool fail_get_file_unique_id_; // Extract number of type from file name. Return false if failing to fine diff --git a/utilities/option_change_migration/option_change_migration.cc b/utilities/option_change_migration/option_change_migration.cc index ea3cf685712..a08c5b59292 100644 --- a/utilities/option_change_migration/option_change_migration.cc +++ b/utilities/option_change_migration/option_change_migration.cc @@ -160,14 +160,7 @@ Status OptionChangeMigration(std::string dbname, const Options& old_opts, return MigrateToLevelBase(dbname, old_opts, new_opts); } else if (new_opts.compaction_style == CompactionStyle::kCompactionStyleFIFO) { - uint64_t l0_file_size = 0; - if (new_opts.compaction_options_fifo.max_table_files_size > 0) { - // Create at least 8 files when max_table_files_size hits, so that the DB - // doesn't just disappear. This in fact violates the FIFO condition, but - // otherwise, the migrated DB is unlikley to be usable. - l0_file_size = new_opts.compaction_options_fifo.max_table_files_size / 8; - } - return CompactToLevel(old_opts, dbname, 0, l0_file_size, true); + return CompactToLevel(old_opts, dbname, 0, 0 /* l0_file_size */, true); } else { return Status::NotSupported( "Do not how to migrate to this compaction style"); diff --git a/utilities/option_change_migration/option_change_migration_test.cc b/utilities/option_change_migration/option_change_migration_test.cc index d6114f8331a..9984f0dd456 100644 --- a/utilities/option_change_migration/option_change_migration_test.cc +++ b/utilities/option_change_migration/option_change_migration_test.cc @@ -9,6 +9,8 @@ #include "rocksdb/utilities/option_change_migration.h" +#include +#include #include #include "db/db_test_util.h" @@ -31,6 +33,8 @@ class DBOptionChangeMigrationTests level2_ = std::get<3>(GetParam()); compaction_style2_ = std::get<4>(GetParam()); is_dynamic2_ = std::get<5>(GetParam()); + // This is set to be extremely large if not zero to avoid dropping any data + // right after migration, which makes test verification difficult fifo_max_table_files_size_ = std::get<6>(GetParam()); } @@ -457,26 +461,30 @@ INSTANTIATE_TEST_CASE_P( false /* is dynamic leveling in old option */, 4 /* old num_levels */, 2 /* new compaction style */, false /* is dynamic leveling in new option */, 0), - std::make_tuple(4 /* old num_levels */, 0 /* old compaction style */, - false /* is dynamic leveling in old option */, - 1 /* old num_levels */, 2 /* new compaction style */, - false /* is dynamic leveling in new option */, - 5 * 1024 * 1024 /*fifo max_table_files_size*/), - std::make_tuple(3 /* old num_levels */, 0 /* old compaction style */, - true /* is dynamic leveling in old option */, - 2 /* old num_levels */, 2 /* new compaction style */, - false /* is dynamic leveling in new option */, - 5 * 1024 * 1024 /*fifo max_table_files_size*/), - std::make_tuple(3 /* old num_levels */, 1 /* old compaction style */, - false /* is dynamic leveling in old option */, - 3 /* old num_levels */, 2 /* new compaction style */, - false /* is dynamic leveling in new option */, - 5 * 1024 * 1024 /*fifo max_table_files_size*/), + std::make_tuple( + 4 /* old num_levels */, 0 /* old compaction style */, + false /* is dynamic leveling in old option */, + 1 /* old num_levels */, 2 /* new compaction style */, + false /* is dynamic leveling in new option */, + std::numeric_limits::max() /*fifo max_table_files_size*/), + std::make_tuple( + 3 /* old num_levels */, 0 /* old compaction style */, + true /* is dynamic leveling in old option */, + 2 /* old num_levels */, 2 /* new compaction style */, + false /* is dynamic leveling in new option */, + std::numeric_limits::max() /*fifo max_table_files_size*/), + std::make_tuple( + 3 /* old num_levels */, 1 /* old compaction style */, + false /* is dynamic leveling in old option */, + 3 /* old num_levels */, 2 /* new compaction style */, + false /* is dynamic leveling in new option */, + std::numeric_limits::max() /*fifo max_table_files_size*/), std::make_tuple(1 /* old num_levels */, 1 /* old compaction style */, false /* is dynamic leveling in old option */, 4 /* old num_levels */, 2 /* new compaction style */, false /* is dynamic leveling in new option */, - 5 * 1024 * 1024 /*fifo max_table_files_size*/))); + std::numeric_limits< + uint64_t>::max() /*fifo max_table_files_size*/))); class DBOptionChangeMigrationTest : public DBTestBase { public: diff --git a/utilities/transactions/lock/range/range_tree/lib/portability/toku_time.h b/utilities/transactions/lock/range/range_tree/lib/portability/toku_time.h index 9b83c53511c..451959aed4d 100644 --- a/utilities/transactions/lock/range/range_tree/lib/portability/toku_time.h +++ b/utilities/transactions/lock/range/range_tree/lib/portability/toku_time.h @@ -165,7 +165,7 @@ static inline tokutime_t toku_time_now(void) { static inline uint64_t toku_current_time_microsec(void) { struct timeval t; - gettimeofday(&t, NULL); + gettimeofday(&t, nullptr); return t.tv_sec * (1UL * 1000 * 1000) + t.tv_usec; } diff --git a/utilities/transactions/lock/range/range_tree/lib/util/growable_array.h b/utilities/transactions/lock/range/range_tree/lib/util/growable_array.h index 158750fdbaf..39e8c091c32 100644 --- a/utilities/transactions/lock/range/range_tree/lib/util/growable_array.h +++ b/utilities/transactions/lock/range/range_tree/lib/util/growable_array.h @@ -78,7 +78,7 @@ class GrowableArray { void init(void) // Effect: Initialize the array to contain no elements. { - m_array = NULL; + m_array = nullptr; m_size = 0; m_size_limit = 0; } @@ -87,7 +87,7 @@ class GrowableArray { // Effect: Deinitialize the array (freeing any memory it uses, for example). { toku_free(m_array); - m_array = NULL; + m_array = nullptr; m_size = 0; m_size_limit = 0; } @@ -113,7 +113,7 @@ class GrowableArray { // constant. { if (m_size >= m_size_limit) { - if (m_array == NULL) { + if (m_array == nullptr) { m_size_limit = 1; } else { m_size_limit *= 2; diff --git a/utilities/transactions/pessimistic_transaction_db.cc b/utilities/transactions/pessimistic_transaction_db.cc index 57c14b5f7b7..75e69867b82 100644 --- a/utilities/transactions/pessimistic_transaction_db.cc +++ b/utilities/transactions/pessimistic_transaction_db.cc @@ -255,7 +255,8 @@ Status TransactionDB::Open( txn_db_options.write_policy == WRITE_COMMITTED || txn_db_options.write_policy == WRITE_PREPARED; s = DBImpl::Open(db_options_2pc, dbname, column_families_copy, handles, &db, - use_seq_per_batch, use_batch_per_txn); + use_seq_per_batch, use_batch_per_txn, + /*is_retry=*/false, /*can_retry=*/nullptr); if (s.ok()) { ROCKS_LOG_WARN(db->GetDBOptions().info_log, "Transaction write_policy is %" PRId32, @@ -426,6 +427,27 @@ Status PessimisticTransactionDB::CreateColumnFamilies( return s; } +Status PessimisticTransactionDB::CreateColumnFamilyWithImport( + const ColumnFamilyOptions& options, const std::string& column_family_name, + const ImportColumnFamilyOptions& import_options, + const std::vector& metadatas, + ColumnFamilyHandle** handle) { + InstrumentedMutexLock l(&column_family_mutex_); + Status s = VerifyCFOptions(options); + if (!s.ok()) { + return s; + } + + s = db_->CreateColumnFamilyWithImport(options, column_family_name, + import_options, metadatas, handle); + if (s.ok()) { + lock_manager_->AddColumnFamily(*handle); + UpdateCFComparatorMap(*handle); + } + + return s; +} + // Let LockManager know that it can deallocate the LockMap for this // column family. Status PessimisticTransactionDB::DropColumnFamily( diff --git a/utilities/transactions/pessimistic_transaction_db.h b/utilities/transactions/pessimistic_transaction_db.h index a125c6c3562..6654aa80963 100644 --- a/utilities/transactions/pessimistic_transaction_db.h +++ b/utilities/transactions/pessimistic_transaction_db.h @@ -106,6 +106,23 @@ class PessimisticTransactionDB : public TransactionDB { const std::vector& column_families, std::vector* handles) override; + using StackableDB::CreateColumnFamilyWithImport; + Status CreateColumnFamilyWithImport( + const ColumnFamilyOptions& options, const std::string& column_family_name, + const ImportColumnFamilyOptions& import_options, + const ExportImportFilesMetaData& metadata, + ColumnFamilyHandle** handle) override { + const std::vector& metadatas{&metadata}; + return CreateColumnFamilyWithImport(options, column_family_name, + import_options, metadatas, handle); + } + + Status CreateColumnFamilyWithImport( + const ColumnFamilyOptions& options, const std::string& column_family_name, + const ImportColumnFamilyOptions& import_options, + const std::vector& metadatas, + ColumnFamilyHandle** handle) override; + using StackableDB::DropColumnFamily; Status DropColumnFamily(ColumnFamilyHandle* column_family) override; diff --git a/utilities/transactions/transaction_test.cc b/utilities/transactions/transaction_test.cc index a2fa9ce051c..a18af717f10 100644 --- a/utilities/transactions/transaction_test.cc +++ b/utilities/transactions/transaction_test.cc @@ -334,6 +334,7 @@ TEST_P(TransactionTest, SwitchMemtableDuringPrepareAndCommit_WC) { ASSERT_EQ("value", value); } + ASSERT_OK(dbimpl->SyncWAL()); delete db; db = nullptr; Status s; @@ -2981,6 +2982,79 @@ TEST_P(TransactionTest, ColumnFamiliesTest) { } } +TEST_P(TransactionTest, WriteImportedColumnFamilyTest) { + WriteOptions write_options; + ReadOptions read_options; + ColumnFamilyOptions cf_options; + ImportColumnFamilyOptions import_options; + ExportImportFilesMetaData* metadata_ptr = nullptr; + ColumnFamilyHandle* import_cf = nullptr; + ColumnFamilyHandle* export_cf = nullptr; + std::string export_files_dir = test::PerThreadDBPath(env.get(), "cf_export"); + std::string value; + Status s; + + { + // Create a column family to export + s = db->CreateColumnFamily(cf_options, "CF_EXPORT", &export_cf); + ASSERT_OK(s); + + // Write some data to the db + WriteBatch batch; + ASSERT_OK(batch.Put(export_cf, "K1", "V1")); + ASSERT_OK(batch.Put(export_cf, "K2", "V2")); + s = db->Write(write_options, &batch); + ASSERT_OK(s); + + Checkpoint* checkpoint = nullptr; + s = Checkpoint::Create(db, &checkpoint); + ASSERT_OK(s); + s = checkpoint->ExportColumnFamily(export_cf, export_files_dir, + &metadata_ptr); + ASSERT_OK(s); + ASSERT_NE(metadata_ptr, nullptr); + delete checkpoint; + + s = db->DropColumnFamily(export_cf); + ASSERT_OK(s); + delete export_cf; + } + + { + // Create a new column family with import + s = db->CreateColumnFamilyWithImport( + cf_options, "CF_IMPORT", import_options, *metadata_ptr, &import_cf); + ASSERT_OK(s); + s = db->Get(read_options, import_cf, "K1", &value); + ASSERT_OK(s); + ASSERT_EQ(value, "V1"); + s = db->Get(read_options, import_cf, "K2", &value); + ASSERT_OK(s); + ASSERT_EQ(value, "V2"); + + // Wirte a new key-value pair + Transaction* txn = db->BeginTransaction(write_options); + ASSERT_TRUE(txn); + s = txn->Put(import_cf, "K3", "V3"); + ASSERT_OK(s); + s = txn->Commit(); + ASSERT_OK(s); + delete txn; + + s = db->Get(read_options, import_cf, "K3", &value); + ASSERT_OK(s); + ASSERT_EQ(value, "V3"); + + s = db->DropColumnFamily(import_cf); + ASSERT_OK(s); + delete import_cf; + } + + delete metadata_ptr; + + EXPECT_OK(DestroyDir(env.get(), export_files_dir)); +} + TEST_P(TransactionTest, MultiGetBatchedTest) { WriteOptions write_options; ReadOptions read_options, snapshot_read_options; diff --git a/utilities/transactions/transaction_test.h b/utilities/transactions/transaction_test.h index 60c5c8a4ba0..a8fa874a782 100644 --- a/utilities/transactions/transaction_test.h +++ b/utilities/transactions/transaction_test.h @@ -170,7 +170,8 @@ class TransactionTestBase : public ::testing::Test { txn_db_options.write_policy == WRITE_COMMITTED || txn_db_options.write_policy == WRITE_PREPARED; Status s = DBImpl::Open(options_copy, dbname, cfs, handles, &root_db, - use_seq_per_batch, use_batch_per_txn); + use_seq_per_batch, use_batch_per_txn, + /*is_retry=*/false, /*can_retry=*/nullptr); auto stackable_db = std::make_unique(root_db); if (s.ok()) { assert(root_db != nullptr); @@ -200,7 +201,8 @@ class TransactionTestBase : public ::testing::Test { txn_db_options.write_policy == WRITE_COMMITTED || txn_db_options.write_policy == WRITE_PREPARED; Status s = DBImpl::Open(options_copy, dbname, column_families, &handles, - &root_db, use_seq_per_batch, use_batch_per_txn); + &root_db, use_seq_per_batch, use_batch_per_txn, + /*is_retry=*/false, /*can_retry=*/nullptr); if (!s.ok()) { delete root_db; return s; diff --git a/utilities/types_util.cc b/utilities/types_util.cc new file mode 100644 index 00000000000..cbd427425ed --- /dev/null +++ b/utilities/types_util.cc @@ -0,0 +1,88 @@ +// Copyright (c) Meta Platforms, Inc. and affiliates. +// +// 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 "rocksdb/utilities/types_util.h" + +#include "db/dbformat.h" + +namespace ROCKSDB_NAMESPACE { + +Status GetInternalKeyForSeek(const Slice& user_key, + const Comparator* comparator, std::string* buf) { + if (!comparator) { + return Status::InvalidArgument( + "Constructing an internal key requires user key comparator."); + } + size_t ts_sz = comparator->timestamp_size(); + Slice max_ts = comparator->GetMaxTimestamp(); + if (ts_sz != max_ts.size()) { + return Status::InvalidArgument( + "The maximum timestamp returned by Comparator::GetMaxTimestamp is " + "invalid."); + } + buf->reserve(user_key.size() + ts_sz + kNumInternalBytes); + buf->assign(user_key.data(), user_key.size()); + if (ts_sz) { + buf->append(max_ts.data(), max_ts.size()); + } + PutFixed64(buf, PackSequenceAndType(kMaxSequenceNumber, kValueTypeForSeek)); + return Status::OK(); +} + +Status GetInternalKeyForSeekForPrev(const Slice& user_key, + const Comparator* comparator, + std::string* buf) { + if (!comparator) { + return Status::InvalidArgument( + "Constructing an internal key requires user key comparator."); + } + size_t ts_sz = comparator->timestamp_size(); + Slice min_ts = comparator->GetMinTimestamp(); + if (ts_sz != min_ts.size()) { + return Status::InvalidArgument( + "The minimum timestamp returned by Comparator::GetMinTimestamp is " + "invalid."); + } + buf->reserve(user_key.size() + ts_sz + kNumInternalBytes); + buf->assign(user_key.data(), user_key.size()); + if (ts_sz) { + buf->append(min_ts.data(), min_ts.size()); + } + PutFixed64(buf, PackSequenceAndType(0, kValueTypeForSeekForPrev)); + return Status::OK(); +} + +Status ParseEntry(const Slice& internal_key, const Comparator* comparator, + ParsedEntryInfo* parsed_entry) { + if (internal_key.size() < kNumInternalBytes) { + return Status::InvalidArgument("Internal key size invalid."); + } + if (!comparator) { + return Status::InvalidArgument( + "Parsing an internal key requires user key comparator."); + } + ParsedInternalKey pikey; + Status status = ParseInternalKey(internal_key, &pikey, /*log_err_key=*/false); + if (!status.ok()) { + return status; + } + + size_t ts_sz = comparator->timestamp_size(); + if (pikey.user_key.size() < ts_sz) { + return Status::InvalidArgument("User key(with timestamp) size invalid."); + } + if (ts_sz == 0) { + parsed_entry->user_key = pikey.user_key; + } else { + parsed_entry->user_key = StripTimestampFromUserKey(pikey.user_key, ts_sz); + parsed_entry->timestamp = + ExtractTimestampFromUserKey(pikey.user_key, ts_sz); + } + parsed_entry->sequence = pikey.sequence; + parsed_entry->type = ROCKSDB_NAMESPACE::GetEntryType(pikey.type); + return Status::OK(); +} +} // namespace ROCKSDB_NAMESPACE diff --git a/utilities/types_util_test.cc b/utilities/types_util_test.cc new file mode 100644 index 00000000000..8535aaec9eb --- /dev/null +++ b/utilities/types_util_test.cc @@ -0,0 +1,98 @@ +// Copyright (c) Meta Platforms, Inc. and affiliates. +// +// 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 "rocksdb/utilities/types_util.h" + +#include "db/dbformat.h" +#include "port/stack_trace.h" +#include "rocksdb/types.h" +#include "test_util/testharness.h" + +namespace ROCKSDB_NAMESPACE { +namespace { +std::string EncodeAsUint64(uint64_t v) { + std::string dst; + PutFixed64(&dst, v); + return dst; +} +std::string IKey(const std::string& user_key, uint64_t seq, ValueType vt, + std::optional timestamp) { + std::string encoded; + encoded.assign(user_key.data(), user_key.size()); + if (timestamp.has_value()) { + PutFixed64(&encoded, timestamp.value()); + } + PutFixed64(&encoded, PackSequenceAndType(seq, vt)); + return encoded; +} +} // namespace + +TEST(ParseEntryTest, InvalidInternalKey) { + const Comparator* ucmp = BytewiseComparator(); + std::string invalid_ikey = "foo"; + Slice ikey_slice = invalid_ikey; + ParsedEntryInfo parsed_entry; + ASSERT_TRUE(ParseEntry(ikey_slice, ucmp, &parsed_entry).IsInvalidArgument()); + + std::string ikey = + IKey("foo", 3, ValueType::kTypeValue, /*timestamp=*/std::nullopt); + ikey_slice = ikey; + ASSERT_TRUE( + ParseEntry(ikey_slice, nullptr, &parsed_entry).IsInvalidArgument()); +} + +TEST(ParseEntryTest, Basic) { + const Comparator* ucmp = BytewiseComparator(); + std::string ikey = + IKey("foo", 3, ValueType::kTypeValue, /*timestamp=*/std::nullopt); + Slice ikey_slice = ikey; + + ParsedEntryInfo parsed_entry; + ASSERT_OK(ParseEntry(ikey_slice, ucmp, &parsed_entry)); + ASSERT_EQ(parsed_entry.user_key, "foo"); + ASSERT_EQ(parsed_entry.timestamp, ""); + ASSERT_EQ(parsed_entry.sequence, 3); + ASSERT_EQ(parsed_entry.type, EntryType::kEntryPut); + + ikey = IKey("bar", 5, ValueType::kTypeDeletion, /*timestamp=*/std::nullopt); + ikey_slice = ikey; + + ASSERT_OK(ParseEntry(ikey_slice, ucmp, &parsed_entry)); + ASSERT_EQ(parsed_entry.user_key, "bar"); + ASSERT_EQ(parsed_entry.timestamp, ""); + ASSERT_EQ(parsed_entry.sequence, 5); + ASSERT_EQ(parsed_entry.type, EntryType::kEntryDelete); +} + +TEST(ParseEntryTest, UserKeyIncludesTimestamp) { + const Comparator* ucmp = BytewiseComparatorWithU64Ts(); + std::string ikey = IKey("foo", 3, ValueType::kTypeValue, 50); + Slice ikey_slice = ikey; + + ParsedEntryInfo parsed_entry; + ASSERT_OK(ParseEntry(ikey_slice, ucmp, &parsed_entry)); + ASSERT_EQ(parsed_entry.user_key, "foo"); + ASSERT_EQ(parsed_entry.timestamp, EncodeAsUint64(50)); + ASSERT_EQ(parsed_entry.sequence, 3); + ASSERT_EQ(parsed_entry.type, EntryType::kEntryPut); + + ikey = IKey("bar", 5, ValueType::kTypeDeletion, 30); + ikey_slice = ikey; + + ASSERT_OK(ParseEntry(ikey_slice, ucmp, &parsed_entry)); + ASSERT_EQ(parsed_entry.user_key, "bar"); + ASSERT_EQ(parsed_entry.timestamp, EncodeAsUint64(30)); + ASSERT_EQ(parsed_entry.sequence, 5); + ASSERT_EQ(parsed_entry.type, EntryType::kEntryDelete); +} + +} // namespace ROCKSDB_NAMESPACE + +int main(int argc, char** argv) { + ROCKSDB_NAMESPACE::port::InstallStackTraceHandler(); + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} diff --git a/utilities/write_batch_with_index/write_batch_with_index.cc b/utilities/write_batch_with_index/write_batch_with_index.cc index 0011401900e..d2e1816d4e6 100644 --- a/utilities/write_batch_with_index/write_batch_with_index.cc +++ b/utilities/write_batch_with_index/write_batch_with_index.cc @@ -567,43 +567,62 @@ Status WriteBatchWithIndex::GetFromBatchAndDB(DB* db, nullptr); } -void WriteBatchWithIndex::MergeAcrossBatchAndDB( +void WriteBatchWithIndex::MergeAcrossBatchAndDBImpl( ColumnFamilyHandle* column_family, const Slice& key, const PinnableWideColumns& existing, const MergeContext& merge_context, - PinnableSlice* value, Status* status) { - assert(value); + std::string* value, PinnableWideColumns* columns, Status* status) { + assert(value || columns); + assert(!value || !columns); assert(status); - assert(status->ok() || status->IsNotFound()); - - std::string result_value; if (status->ok()) { if (WideColumnsHelper::HasDefaultColumnOnly(existing.columns())) { *status = WriteBatchWithIndexInternal::MergeKeyWithBaseValue( column_family, key, MergeHelper::kPlainBaseValue, WideColumnsHelper::GetDefaultColumn(existing.columns()), - merge_context, &result_value, - static_cast(nullptr)); + merge_context, value, columns); } else { *status = WriteBatchWithIndexInternal::MergeKeyWithBaseValue( column_family, key, MergeHelper::kWideBaseValue, existing.columns(), - merge_context, &result_value, - static_cast(nullptr)); + merge_context, value, columns); } } else { assert(status->IsNotFound()); *status = WriteBatchWithIndexInternal::MergeKeyWithNoBaseValue( - column_family, key, merge_context, &result_value, - static_cast(nullptr)); + column_family, key, merge_context, value, columns); } +} + +void WriteBatchWithIndex::MergeAcrossBatchAndDB( + ColumnFamilyHandle* column_family, const Slice& key, + const PinnableWideColumns& existing, const MergeContext& merge_context, + PinnableSlice* value, Status* status) { + assert(value); + assert(status); + + std::string result_value; + constexpr PinnableWideColumns* result_entity = nullptr; + MergeAcrossBatchAndDBImpl(column_family, key, existing, merge_context, + &result_value, result_entity, status); if (status->ok()) { - value->Reset(); *value->GetSelf() = std::move(result_value); value->PinSelf(); } } +void WriteBatchWithIndex::MergeAcrossBatchAndDB( + ColumnFamilyHandle* column_family, const Slice& key, + const PinnableWideColumns& existing, const MergeContext& merge_context, + PinnableWideColumns* columns, Status* status) { + assert(columns); + assert(status); + + constexpr std::string* value = nullptr; + MergeAcrossBatchAndDBImpl(column_family, key, existing, merge_context, value, + columns, status); +} + Status WriteBatchWithIndex::GetFromBatchAndDB( DB* db, const ReadOptions& read_options, ColumnFamilyHandle* column_family, const Slice& key, PinnableSlice* pinnable_val, ReadCallback* callback) { @@ -620,6 +639,8 @@ Status WriteBatchWithIndex::GetFromBatchAndDB( return Status::InvalidArgument("Must specify timestamp"); } + pinnable_val->Reset(); + // Since the lifetime of the WriteBatch is the same as that of the transaction // we cannot pin it as otherwise the returned value will not be available // after the transaction finishes. @@ -634,7 +655,8 @@ Status WriteBatchWithIndex::GetFromBatchAndDB( return s; } - if (!s.ok() || result == WBWIIteratorImpl::kError) { + assert(!s.ok() == (result == WBWIIteratorImpl::kError)); + if (result == WBWIIteratorImpl::kError) { return s; } @@ -800,6 +822,236 @@ void WriteBatchWithIndex::MultiGetFromBatchAndDB( } } +Status WriteBatchWithIndex::GetEntityFromBatchAndDB( + DB* db, const ReadOptions& read_options, ColumnFamilyHandle* column_family, + const Slice& key, PinnableWideColumns* columns, ReadCallback* callback) { + assert(db); + assert(column_family); + assert(columns); + + columns->Reset(); + + MergeContext merge_context; + Status s; + + auto result = WriteBatchWithIndexInternal::GetEntityFromBatch( + this, column_family, key, &merge_context, columns, &s); + + assert(!s.ok() == (result == WBWIIteratorImpl::kError)); + + if (result == WBWIIteratorImpl::kFound || + result == WBWIIteratorImpl::kError) { + return s; + } + + if (result == WBWIIteratorImpl::kDeleted) { + return Status::NotFound(); + } + + assert(result == WBWIIteratorImpl::kMergeInProgress || + result == WBWIIteratorImpl::kNotFound); + + PinnableWideColumns existing; + + DBImpl::GetImplOptions get_impl_options; + get_impl_options.column_family = column_family; + get_impl_options.columns = + (result == WBWIIteratorImpl::kMergeInProgress) ? &existing : columns; + get_impl_options.callback = callback; + + s = static_cast_with_check(db->GetRootDB()) + ->GetImpl(read_options, key, get_impl_options); + + if (result == WBWIIteratorImpl::kMergeInProgress) { + if (s.ok() || s.IsNotFound()) { // DB lookup succeeded + MergeAcrossBatchAndDB(column_family, key, existing, merge_context, + columns, &s); + } + } + + return s; +} + +Status WriteBatchWithIndex::GetEntityFromBatchAndDB( + DB* db, const ReadOptions& read_options, ColumnFamilyHandle* column_family, + const Slice& key, PinnableWideColumns* columns) { + if (!db) { + return Status::InvalidArgument( + "Cannot call GetEntityFromBatchAndDB without a DB object"); + } + + if (!column_family) { + return Status::InvalidArgument( + "Cannot call GetEntityFromBatchAndDB without a column family handle"); + } + + const Comparator* const ucmp = rep->comparator.GetComparator(column_family); + size_t ts_sz = ucmp ? ucmp->timestamp_size() : 0; + if (ts_sz > 0 && !read_options.timestamp) { + return Status::InvalidArgument("Must specify timestamp"); + } + + if (!columns) { + return Status::InvalidArgument( + "Cannot call GetEntityFromBatchAndDB without a PinnableWideColumns " + "object"); + } + + constexpr ReadCallback* callback = nullptr; + + return GetEntityFromBatchAndDB(db, read_options, column_family, key, columns, + callback); +} + +void WriteBatchWithIndex::MultiGetEntityFromBatchAndDB( + DB* db, const ReadOptions& read_options, ColumnFamilyHandle* column_family, + size_t num_keys, const Slice* keys, PinnableWideColumns* results, + Status* statuses, bool sorted_input, ReadCallback* callback) { + assert(db); + assert(column_family); + assert(keys); + assert(results); + assert(statuses); + + struct MergeTuple { + MergeTuple(const Slice& _key, Status* _s, MergeContext&& _merge_context, + PinnableWideColumns* _columns) + : key(_key), + s(_s), + merge_context(std::move(_merge_context)), + columns(_columns) { + assert(s); + assert(columns); + } + + Slice key; + Status* s; + PinnableWideColumns existing; + MergeContext merge_context; + PinnableWideColumns* columns; + }; + + autovector merges; + + autovector key_contexts; + + for (size_t i = 0; i < num_keys; ++i) { + const Slice& key = keys[i]; + MergeContext merge_context; + PinnableWideColumns* const columns = &results[i]; + Status* const s = &statuses[i]; + + columns->Reset(); + + auto result = WriteBatchWithIndexInternal::GetEntityFromBatch( + this, column_family, key, &merge_context, columns, s); + + if (result == WBWIIteratorImpl::kFound || + result == WBWIIteratorImpl::kError) { + continue; + } + + if (result == WBWIIteratorImpl::kDeleted) { + *s = Status::NotFound(); + continue; + } + + if (result == WBWIIteratorImpl::kMergeInProgress) { + merges.emplace_back(key, s, std::move(merge_context), columns); + + // The columns field will be populated by the loop below to prevent issues + // with dangling pointers. + key_contexts.emplace_back(column_family, key, /* value */ nullptr, + /* columns */ nullptr, /* timestamp */ nullptr, + s); + continue; + } + + assert(result == WBWIIteratorImpl::kNotFound); + key_contexts.emplace_back(column_family, key, /* value */ nullptr, columns, + /* timestamp */ nullptr, s); + } + + autovector sorted_keys; + sorted_keys.reserve(key_contexts.size()); + + size_t merges_idx = 0; + for (KeyContext& key_context : key_contexts) { + if (!key_context.columns) { + assert(*key_context.key == merges[merges_idx].key); + + key_context.columns = &merges[merges_idx].existing; + ++merges_idx; + } + + sorted_keys.emplace_back(&key_context); + } + + static_cast_with_check(db->GetRootDB()) + ->PrepareMultiGetKeys(sorted_keys.size(), sorted_input, &sorted_keys); + static_cast_with_check(db->GetRootDB()) + ->MultiGetWithCallback(read_options, column_family, callback, + &sorted_keys); + + for (const auto& merge : merges) { + if (merge.s->ok() || merge.s->IsNotFound()) { // DB lookup succeeded + MergeAcrossBatchAndDB(column_family, merge.key, merge.existing, + merge.merge_context, merge.columns, merge.s); + } + } +} + +void WriteBatchWithIndex::MultiGetEntityFromBatchAndDB( + DB* db, const ReadOptions& read_options, ColumnFamilyHandle* column_family, + size_t num_keys, const Slice* keys, PinnableWideColumns* results, + Status* statuses, bool sorted_input) { + assert(statuses); + + if (!db) { + for (size_t i = 0; i < num_keys; ++i) { + statuses[i] = Status::InvalidArgument( + "Cannot call MultiGetEntityFromBatchAndDB without a DB object"); + } + } + + if (!column_family) { + for (size_t i = 0; i < num_keys; ++i) { + statuses[i] = Status::InvalidArgument( + "Cannot call MultiGetEntityFromBatchAndDB without a column family " + "handle"); + } + } + + const Comparator* const ucmp = rep->comparator.GetComparator(column_family); + const size_t ts_sz = ucmp ? ucmp->timestamp_size() : 0; + if (ts_sz > 0 && !read_options.timestamp) { + for (size_t i = 0; i < num_keys; ++i) { + statuses[i] = Status::InvalidArgument("Must specify timestamp"); + } + return; + } + + if (!keys) { + for (size_t i = 0; i < num_keys; ++i) { + statuses[i] = Status::InvalidArgument( + "Cannot call MultiGetEntityFromBatchAndDB without keys"); + } + } + + if (!results) { + for (size_t i = 0; i < num_keys; ++i) { + statuses[i] = Status::InvalidArgument( + "Cannot call MultiGetEntityFromBatchAndDB without " + "PinnableWideColumns objects"); + } + } + + constexpr ReadCallback* callback = nullptr; + + MultiGetEntityFromBatchAndDB(db, read_options, column_family, num_keys, keys, + results, statuses, sorted_input, callback); +} + void WriteBatchWithIndex::SetSavePoint() { rep->write_batch.SetSavePoint(); } Status WriteBatchWithIndex::RollbackToSavePoint() { diff --git a/utilities/write_batch_with_index/write_batch_with_index_internal.cc b/utilities/write_batch_with_index/write_batch_with_index_internal.cc index 84e30b7cc8c..5a8031423e1 100644 --- a/utilities/write_batch_with_index/write_batch_with_index_internal.cc +++ b/utilities/write_batch_with_index/write_batch_with_index_internal.cc @@ -281,6 +281,7 @@ void BaseDeltaIterator::SetValueAndColumnsFromDelta() { status_ = WideColumnSerialization::Deserialize(value_copy, columns_); if (!status_.ok()) { + columns_.clear(); return; } @@ -340,6 +341,7 @@ void BaseDeltaIterator::SetValueAndColumnsFromDelta() { status_ = WideColumnSerialization::Deserialize(entity, columns_); if (!status_.ok()) { + columns_.clear(); return; } @@ -838,6 +840,9 @@ WBWIIteratorImpl::Result WriteBatchWithIndexInternal::GetFromBatchImpl( Traits::ClearOutput(output); result = WBWIIteratorImpl::Result::kError; } + } else { + Traits::ClearOutput(output); + *s = Status::OK(); } return result; diff --git a/utilities/write_batch_with_index/write_batch_with_index_test.cc b/utilities/write_batch_with_index/write_batch_with_index_test.cc index cb8e3a343c6..bf723033133 100644 --- a/utilities/write_batch_with_index/write_batch_with_index_test.cc +++ b/utilities/write_batch_with_index/write_batch_with_index_test.cc @@ -2614,8 +2614,40 @@ TEST_P(WriteBatchWithIndexTest, WideColumnsBatchOnly) { } } - // TODO: add tests for GetEntityFromBatchAndDB and - // MultiGetEntityFromBatchAndDB once they are implemented + // GetEntityFromBatchAndDB + { + PinnableWideColumns columns; + ASSERT_TRUE(batch_ + ->GetEntityFromBatchAndDB(db_, read_opts_, + db_->DefaultColumnFamily(), + delete_key, &columns) + .IsNotFound()); + } + + for (size_t i = 1; i < num_keys; ++i) { + PinnableWideColumns columns; + ASSERT_OK(batch_->GetEntityFromBatchAndDB( + db_, read_opts_, db_->DefaultColumnFamily(), keys[i], &columns)); + ASSERT_EQ(columns.columns(), expected[i]); + } + + // MultiGetEntityFromBatchAndDB + { + std::array results; + std::array statuses; + constexpr bool sorted_input = false; + + batch_->MultiGetEntityFromBatchAndDB( + db_, read_opts_, db_->DefaultColumnFamily(), num_keys, keys.data(), + results.data(), statuses.data(), sorted_input); + + ASSERT_TRUE(statuses[0].IsNotFound()); + + for (size_t i = 1; i < num_keys; ++i) { + ASSERT_OK(statuses[i]); + ASSERT_EQ(results[i].columns(), expected[i]); + } + } // Iterator std::unique_ptr iter(batch_->NewIteratorWithBase( @@ -2733,8 +2765,40 @@ TEST_P(WriteBatchWithIndexTest, WideColumnsBatchAndDB) { ASSERT_TRUE(statuses[num_keys - 1].IsNotFound()); } - // TODO: add tests for GetEntityFromBatchAndDB and - // MultiGetEntityFromBatchAndDB once they are implemented + // GetEntityFromBatchAndDB + for (size_t i = 0; i < num_keys - 1; ++i) { + PinnableWideColumns columns; + ASSERT_OK(batch_->GetEntityFromBatchAndDB( + db_, read_opts_, db_->DefaultColumnFamily(), keys[i], &columns)); + ASSERT_EQ(columns.columns(), expected[i]); + } + + { + PinnableWideColumns columns; + ASSERT_TRUE(batch_ + ->GetEntityFromBatchAndDB(db_, read_opts_, + db_->DefaultColumnFamily(), + no_merge_c_key, &columns) + .IsNotFound()); + } + + // MultiGetEntityFromBatchAndDB + { + std::array results; + std::array statuses; + constexpr bool sorted_input = false; + + batch_->MultiGetEntityFromBatchAndDB( + db_, read_opts_, db_->DefaultColumnFamily(), num_keys, keys.data(), + results.data(), statuses.data(), sorted_input); + + for (size_t i = 0; i < num_keys - 1; ++i) { + ASSERT_OK(statuses[i]); + ASSERT_EQ(results[i].columns(), expected[i]); + } + + ASSERT_TRUE(statuses[num_keys - 1].IsNotFound()); + } // Iterator std::unique_ptr iter(batch_->NewIteratorWithBase(