diff --git a/CMakeLists.txt b/CMakeLists.txt index 51943b1a125..2092437f09e 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -705,6 +705,7 @@ set(SOURCES db/db_impl/db_impl_write.cc db/db_impl/db_impl_compaction_flush.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_debug.cc db/db_impl/db_impl_experimental.cc @@ -762,6 +763,7 @@ set(SOURCES env/env_encryption.cc env/file_system.cc env/file_system_tracer.cc + env/fs_on_demand.cc env/fs_remap.cc env/mock_env.cc env/unique_id_gen.cc @@ -1051,10 +1053,8 @@ 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() diff --git a/HISTORY.md b/HISTORY.md index feffb34b4cb..a1700f993a4 100644 --- a/HISTORY.md +++ b/HISTORY.md @@ -1,6 +1,31 @@ # Rocksdb Change Log > NOTE: Entries for next release do not go here. Follow instructions in `unreleased_history/README.txt` +## 9.3.0 (05/17/2024) +### New Features +* Optimistic transactions and pessimistic transactions with the WriteCommitted policy now support the `GetEntity` API. +* Added new `Iterator` property, "rocksdb.iterator.is-value-pinned", for checking whether the `Slice` returned by `Iterator::value()` can be used until the `Iterator` is destroyed. +* Optimistic transactions and WriteCommitted pessimistic transactions now support the `MultiGetEntity` API. +* Optimistic transactions and pessimistic transactions with the WriteCommitted policy now support the `PutEntity` API. Support for read APIs and other write policies (WritePrepared, WriteUnprepared) will be added later. + +### Public API Changes +* Exposed block based metadata cache options via C API +* Exposed compaction pri via c api. +* Add a kAdmPolicyAllowAll option to TieredAdmissionPolicy that admits all blocks evicted from the primary block cache into the compressed secondary cache. + +### Behavior Changes +* CompactRange() with change_level=true on a CF with FIFO compaction will return Status::NotSupported(). +* External file ingestion with FIFO compaction will always ingest to L0. + +### Bug Fixes +* Fixed a bug for databases using `DBOptions::allow_2pc == true` (all `TransactionDB`s except `OptimisticTransactionDB`) that have exactly one column family. Due to a missing WAL sync, attempting to open the DB could have returned a `Status::Corruption` with a message like "SST file is ahead of WALs". +* Fix a bug in CreateColumnFamilyWithImport() where if multiple CFs are imported, we were not resetting files' epoch number and L0 files can have overlapping key range but the same epoch number. +* Fixed race conditions when `ColumnFamilyOptions::inplace_update_support == true` between user overwrites and reads on the same key. +* Fix a bug where `CompactFiles()` can compact files of range conflict with other ongoing compactions' when `preclude_last_level_data_seconds > 0` is used +* Fixed a false positive `Status::Corruption` reported when reopening a DB that used `DBOptions::recycle_log_file_num > 0` and `DBOptions::wal_compression != kNoCompression`. +* While WAL is locked with LockWAL(), some operations like Flush() and IngestExternalFile() are now blocked as they should have been. +* Fixed a bug causing stale memory access when using the TieredSecondaryCache with an NVM secondary cache, and a file system that supports return an FS allocated buffer for MultiRead (FSSupportedOps::kFSBuffer is set). + ## 9.2.0 (05/01/2024) ### New Features * Added two options `deadline` and `max_size_bytes` for CacheDumper to exit early diff --git a/TARGETS b/TARGETS index cdfcdc701cd..a274da1518b 100644 --- a/TARGETS +++ b/TARGETS @@ -4,6 +4,7 @@ # This file is a Facebook-specific integration for buck builds, so can # only be validated by Facebook employees. load("//rocks/buckifier:defs.bzl", "cpp_library_wrapper","rocks_cpp_library_wrapper","cpp_binary_wrapper","cpp_unittest_wrapper","fancy_bench_wrapper","add_c_test_wrapper") +load("@fbcode_macros//build_defs:export_files.bzl", "export_file") cpp_library_wrapper(name="rocksdb_lib", srcs=[ @@ -410,6 +411,8 @@ rocks_cpp_library_wrapper(name="rocksdb_stress_lib", srcs=[ ], headers=None) +cpp_binary_wrapper(name="ldb", srcs=["tools/ldb.cc"], deps=[":rocksdb_tools_lib"], extra_preprocessor_flags=[], extra_bench_libs=False) + cpp_binary_wrapper(name="db_stress", srcs=["db_stress_tool/db_stress.cc"], deps=[":rocksdb_stress_lib"], extra_preprocessor_flags=[], extra_bench_libs=False) cpp_binary_wrapper(name="cache_bench", srcs=["cache/cache_bench.cc"], deps=[":rocksdb_cache_bench_tools_lib"], extra_preprocessor_flags=[], extra_bench_libs=False) diff --git a/buckifier/buckify_rocksdb.py b/buckifier/buckify_rocksdb.py index b56e1a82de4..f7831c6907d 100755 --- a/buckifier/buckify_rocksdb.py +++ b/buckifier/buckify_rocksdb.py @@ -193,6 +193,10 @@ def generate_targets(repo_path, deps_map): + src_mk.get("STRESS_LIB_SOURCES", []) + ["test_util/testutil.cc"], ) + # ldb binary + TARGETS.add_binary( + "ldb", ["tools/ldb.cc"], [":rocksdb_tools_lib"] + ) # db_stress binary TARGETS.add_binary( "db_stress", ["db_stress_tool/db_stress.cc"], [":rocksdb_stress_lib"] diff --git a/buckifier/targets_cfg.py b/buckifier/targets_cfg.py index ead6ac51a50..08f58628a7b 100644 --- a/buckifier/targets_cfg.py +++ b/buckifier/targets_cfg.py @@ -7,6 +7,7 @@ # This file is a Facebook-specific integration for buck builds, so can # only be validated by Facebook employees. load("//rocks/buckifier:defs.bzl", "cpp_library_wrapper","rocks_cpp_library_wrapper","cpp_binary_wrapper","cpp_unittest_wrapper","fancy_bench_wrapper","add_c_test_wrapper") +load("@fbcode_macros//build_defs:export_files.bzl", "export_file") """ diff --git a/cache/secondary_cache_adapter.cc b/cache/secondary_cache_adapter.cc index 6261b8ce6e7..9c1b9601022 100644 --- a/cache/secondary_cache_adapter.cc +++ b/cache/secondary_cache_adapter.cc @@ -134,12 +134,14 @@ bool CacheWithSecondaryAdapter::EvictionHandler(const Slice& key, auto obj = target_->Value(handle); // Ignore dummy entry if (obj != kDummyObj) { - bool hit = false; + bool force = false; if (adm_policy_ == TieredAdmissionPolicy::kAdmPolicyAllowCacheHits) { - hit = was_hit; + force = was_hit; + } else if (adm_policy_ == TieredAdmissionPolicy::kAdmPolicyAllowAll) { + force = true; } // Spill into secondary cache. - secondary_cache_->Insert(key, obj, helper, hit).PermitUncheckedError(); + secondary_cache_->Insert(key, obj, helper, force).PermitUncheckedError(); } } // Never takes ownership of obj @@ -661,6 +663,7 @@ std::shared_ptr NewTieredCache(const TieredCacheOptions& _opts) { break; case TieredAdmissionPolicy::kAdmPolicyPlaceholder: case TieredAdmissionPolicy::kAdmPolicyAllowCacheHits: + case TieredAdmissionPolicy::kAdmPolicyAllowAll: if (opts.nvm_sec_cache) { valid_adm_policy = false; } diff --git a/cache/tiered_secondary_cache_test.cc b/cache/tiered_secondary_cache_test.cc index 6a43b6dd526..b32033133e3 100644 --- a/cache/tiered_secondary_cache_test.cc +++ b/cache/tiered_secondary_cache_test.cc @@ -816,11 +816,174 @@ TEST_P(DBTieredAdmPolicyTest, CompressedOnlyTest) { Destroy(options); } +TEST_P(DBTieredAdmPolicyTest, CompressedCacheAdmission) { + if (!LZ4_Supported()) { + ROCKSDB_GTEST_SKIP("This test requires LZ4 support."); + return; + } + + BlockBasedTableOptions table_options; + // We want a block cache of size 5KB, and a compressed secondary cache of + // size 5KB. However, we specify a block cache size of 256KB here in order + // to take into account the cache reservation in the block cache on + // behalf of the compressed cache. The unit of cache reservation is 256KB. + // The effective block cache capacity will be calculated as 256 + 5 = 261KB, + // and 256KB will be reserved for the compressed cache, leaving 10KB for + // the primary block cache. We only have to worry about this here because + // the cache size is so small. + table_options.block_cache = NewCache(256 * 1024, 5 * 1024, 0, GetParam()); + table_options.block_size = 4 * 1024; + table_options.cache_index_and_filter_blocks = false; + Options options = GetDefaultOptions(); + options.create_if_missing = true; + options.table_factory.reset(NewBlockBasedTableFactory(table_options)); + + size_t comp_cache_usage = compressed_secondary_cache()->TEST_GetUsage(); + // Disable paranoid_file_checks so that flush will not read back the newly + // written file + options.paranoid_file_checks = false; + DestroyAndReopen(options); + Random rnd(301); + const int N = 256; + for (int i = 0; i < N; i++) { + std::string p_v; + test::CompressibleString(&rnd, 0.5, 1007, &p_v); + ASSERT_OK(Put(Key(i), p_v)); + } + + ASSERT_OK(Flush()); + + // The second Get (for 5) will evict the data block loaded by the first + // Get, which will be admitted into the compressed secondary cache only + // for the kAdmPolicyAllowAll policy + std::string v = Get(Key(0)); + ASSERT_EQ(1007, v.size()); + + v = Get(Key(5)); + ASSERT_EQ(1007, v.size()); + + if (GetParam() == TieredAdmissionPolicy::kAdmPolicyAllowAll) { + ASSERT_GT(compressed_secondary_cache()->TEST_GetUsage(), + comp_cache_usage + 128); + } else { + ASSERT_LT(compressed_secondary_cache()->TEST_GetUsage(), + comp_cache_usage + 128); + } + + Destroy(options); +} + +TEST_F(DBTieredSecondaryCacheTest, FSBufferTest) { + class WrapFS : public FileSystemWrapper { + public: + explicit WrapFS(const std::shared_ptr& _target) + : FileSystemWrapper(_target) {} + ~WrapFS() override {} + const char* Name() const override { return "WrapFS"; } + + IOStatus NewRandomAccessFile(const std::string& fname, + const FileOptions& opts, + std::unique_ptr* result, + IODebugContext* dbg) override { + class WrappedRandomAccessFile : public FSRandomAccessFileOwnerWrapper { + public: + explicit WrappedRandomAccessFile( + std::unique_ptr& file) + : FSRandomAccessFileOwnerWrapper(std::move(file)) {} + + IOStatus MultiRead(FSReadRequest* reqs, size_t num_reqs, + const IOOptions& options, + IODebugContext* dbg) override { + for (size_t i = 0; i < num_reqs; ++i) { + FSReadRequest& req = reqs[i]; + FSAllocationPtr buffer(new char[req.len], [](void* ptr) { + delete[] static_cast(ptr); + }); + req.fs_scratch = std::move(buffer); + req.status = Read(req.offset, req.len, options, &req.result, + static_cast(req.fs_scratch.get()), dbg); + } + return IOStatus::OK(); + } + }; + + std::unique_ptr file; + IOStatus s = target()->NewRandomAccessFile(fname, opts, &file, dbg); + EXPECT_OK(s); + result->reset(new WrappedRandomAccessFile(file)); + + return s; + } + + void SupportedOps(int64_t& supported_ops) override { + supported_ops = 1 << FSSupportedOps::kAsyncIO; + supported_ops |= 1 << FSSupportedOps::kFSBuffer; + } + }; + + if (!LZ4_Supported()) { + ROCKSDB_GTEST_SKIP("This test requires LZ4 support."); + return; + } + + std::shared_ptr wrap_fs = + std::make_shared(env_->GetFileSystem()); + std::unique_ptr wrap_env(new CompositeEnvWrapper(env_, wrap_fs)); + BlockBasedTableOptions table_options; + table_options.block_cache = NewCache(250 * 1024, 20 * 1024, 256 * 1024, + TieredAdmissionPolicy::kAdmPolicyAuto, + /*ready_before_wait=*/true); + table_options.block_size = 4 * 1024; + table_options.cache_index_and_filter_blocks = false; + Options options = GetDefaultOptions(); + options.create_if_missing = true; + options.table_factory.reset(NewBlockBasedTableFactory(table_options)); + options.statistics = CreateDBStatistics(); + options.env = wrap_env.get(); + + options.paranoid_file_checks = false; + DestroyAndReopen(options); + Random rnd(301); + const int N = 256; + for (int i = 0; i < N; i++) { + std::string p_v; + test::CompressibleString(&rnd, 0.5, 1007, &p_v); + ASSERT_OK(Put(Key(i), p_v)); + } + + ASSERT_OK(Flush()); + + std::vector keys; + std::vector values; + + keys.push_back(Key(0)); + keys.push_back(Key(4)); + keys.push_back(Key(8)); + values = MultiGet(keys, /*snapshot=*/nullptr, /*async=*/true); + ASSERT_EQ(values.size(), keys.size()); + for (const auto& value : values) { + ASSERT_EQ(1007, value.size()); + } + ASSERT_EQ(nvm_sec_cache()->num_insert_saved(), 3u); + ASSERT_EQ(nvm_sec_cache()->num_misses(), 3u); + ASSERT_EQ(nvm_sec_cache()->num_hits(), 0u); + + std::string v = Get(Key(12)); + ASSERT_EQ(1007, v.size()); + ASSERT_EQ(nvm_sec_cache()->num_insert_saved(), 4u); + ASSERT_EQ(nvm_sec_cache()->num_misses(), 4u); + ASSERT_EQ(options.statistics->getTickerCount(BLOCK_CACHE_MISS), 4u); + + Close(); + Destroy(options); +} + INSTANTIATE_TEST_CASE_P( DBTieredAdmPolicyTest, DBTieredAdmPolicyTest, ::testing::Values(TieredAdmissionPolicy::kAdmPolicyAuto, TieredAdmissionPolicy::kAdmPolicyPlaceholder, - TieredAdmissionPolicy::kAdmPolicyAllowCacheHits)); + TieredAdmissionPolicy::kAdmPolicyAllowCacheHits, + TieredAdmissionPolicy::kAdmPolicyAllowAll)); } // namespace ROCKSDB_NAMESPACE diff --git a/db/c.cc b/db/c.cc index 69cd665c0b6..970903826e1 100644 --- a/db/c.cc +++ b/db/c.cc @@ -1947,6 +1947,10 @@ void rocksdb_iter_get_error(const rocksdb_iterator_t* iter, char** errptr) { SaveError(errptr, iter->rep->status()); } +void rocksdb_iter_refresh(const rocksdb_iterator_t* iter, char** errptr) { + SaveError(errptr, iter->rep->Refresh()); +} + rocksdb_writebatch_t* rocksdb_writebatch_create() { return new rocksdb_writebatch_t; } @@ -1958,6 +1962,15 @@ rocksdb_writebatch_t* rocksdb_writebatch_create_from(const char* rep, return b; } +rocksdb_writebatch_t* rocksdb_writebatch_create_with_params( + size_t reserved_bytes, size_t max_bytes, size_t protection_bytes_per_key, + size_t default_cf_ts_sz) { + rocksdb_writebatch_t* b = new rocksdb_writebatch_t; + b->rep = WriteBatch(reserved_bytes, max_bytes, protection_bytes_per_key, + default_cf_ts_sz); + return b; +} + void rocksdb_writebatch_destroy(rocksdb_writebatch_t* b) { delete b; } void rocksdb_writebatch_clear(rocksdb_writebatch_t* b) { b->rep.Clear(); } @@ -2223,6 +2236,35 @@ rocksdb_writebatch_wi_t* rocksdb_writebatch_wi_create( return b; } +rocksdb_writebatch_wi_t* rocksdb_writebatch_wi_create_with_params( + rocksdb_comparator_t* backup_index_comparator, size_t reserved_bytes, + unsigned char overwrite_key, size_t max_bytes, + size_t protection_bytes_per_key) { + rocksdb_writebatch_wi_t* b = new rocksdb_writebatch_wi_t; + b->rep = new WriteBatchWithIndex(backup_index_comparator, reserved_bytes, + overwrite_key, max_bytes, + protection_bytes_per_key); + return b; +} + +void rocksdb_writebatch_update_timestamps( + rocksdb_writebatch_t* wb, const char* ts, size_t tslen, void* state, + size_t (*get_ts_size)(void*, uint32_t), char** errptr) { + SaveError(errptr, wb->rep.UpdateTimestamps( + Slice(ts, tslen), [&get_ts_size, &state](uint32_t cf) { + return (*get_ts_size)(state, cf); + })); +} + +void rocksdb_writebatch_wi_update_timestamps( + rocksdb_writebatch_wi_t* wb, const char* ts, size_t tslen, void* state, + size_t (*get_ts_size)(void*, uint32_t), char** errptr) { + SaveError(errptr, wb->rep->GetWriteBatch()->UpdateTimestamps( + Slice(ts, tslen), [&get_ts_size, &state](uint32_t cf) { + return (*get_ts_size)(state, cf); + })); +} + void rocksdb_writebatch_wi_destroy(rocksdb_writebatch_wi_t* b) { if (b->rep) { delete b->rep; @@ -2745,6 +2787,24 @@ void rocksdb_options_set_block_based_table_factory( } } +void rocksdb_block_based_options_set_top_level_index_pinning_tier( + rocksdb_block_based_table_options_t* options, int v) { + options->rep.metadata_cache_options.top_level_index_pinning = + static_cast(v); +} + +void rocksdb_block_based_options_set_partition_pinning_tier( + rocksdb_block_based_table_options_t* options, int v) { + options->rep.metadata_cache_options.partition_pinning = + static_cast(v); +} + +void rocksdb_block_based_options_set_unpartitioned_pinning_tier( + rocksdb_block_based_table_options_t* options, int v) { + options->rep.metadata_cache_options.unpartitioned_pinning = + static_cast(v); +} + rocksdb_cuckoo_table_options_t* rocksdb_cuckoo_options_create() { return new rocksdb_cuckoo_table_options_t; } @@ -3930,6 +3990,14 @@ void rocksdb_options_set_fifo_compaction_options( opt->rep.compaction_options_fifo = fifo->rep; } +void rocksdb_options_set_compaction_pri(rocksdb_options_t* opt, int pri) { + opt->rep.compaction_pri = static_cast(pri); +} + +int rocksdb_options_get_compaction_pri(rocksdb_options_t* opt) { + return opt->rep.compaction_pri; +} + char* rocksdb_options_statistics_get_string(rocksdb_options_t* opt) { ROCKSDB_NAMESPACE::Statistics* statistics = opt->rep.statistics.get(); if (statistics) { diff --git a/db/c_test.c b/db/c_test.c index b55b2cc96fa..81291dce5b1 100644 --- a/db/c_test.c +++ b/db/c_test.c @@ -704,6 +704,10 @@ int main(int argc, char** argv) { rocksdb_block_based_options_set_block_cache(table_options, cache); rocksdb_block_based_options_set_data_block_index_type(table_options, 1); rocksdb_block_based_options_set_data_block_hash_ratio(table_options, 0.75); + rocksdb_block_based_options_set_top_level_index_pinning_tier(table_options, + 1); + rocksdb_block_based_options_set_partition_pinning_tier(table_options, 2); + rocksdb_block_based_options_set_unpartitioned_pinning_tier(table_options, 3); rocksdb_options_set_block_based_table_factory(options, table_options); rocksdb_options_set_compression(options, rocksdb_no_compression); @@ -2041,7 +2045,6 @@ int main(int argc, char** argv) { rocksdb_options_set_advise_random_on_open(o, 1); CheckCondition(1 == rocksdb_options_get_advise_random_on_open(o)); - rocksdb_options_set_use_adaptive_mutex(o, 1); CheckCondition(1 == rocksdb_options_get_use_adaptive_mutex(o)); @@ -2112,6 +2115,9 @@ int main(int argc, char** argv) { rocksdb_options_set_compaction_style(o, 2); CheckCondition(2 == rocksdb_options_get_compaction_style(o)); + rocksdb_options_set_compaction_pri(o, 4); + CheckCondition(4 == rocksdb_options_get_compaction_pri(o)); + rocksdb_options_set_atomic_flush(o, 1); CheckCondition(1 == rocksdb_options_get_atomic_flush(o)); @@ -2616,6 +2622,10 @@ int main(int argc, char** argv) { CheckCondition(1 == rocksdb_options_get_compaction_style(copy)); CheckCondition(2 == rocksdb_options_get_compaction_style(o)); + rocksdb_options_set_compaction_pri(copy, 1); + CheckCondition(1 == rocksdb_options_get_compaction_pri(copy)); + CheckCondition(4 == rocksdb_options_get_compaction_pri(o)); + rocksdb_options_set_atomic_flush(copy, 0); CheckCondition(0 == rocksdb_options_get_atomic_flush(copy)); CheckCondition(1 == rocksdb_options_get_atomic_flush(o)); diff --git a/db/column_family.cc b/db/column_family.cc index 2b606ec4095..bcc0008cedd 100644 --- a/db/column_family.cc +++ b/db/column_family.cc @@ -1528,6 +1528,20 @@ Status ColumnFamilyData::ValidateOptions( } } } + + if (cf_options.compaction_style == kCompactionStyleUniversal) { + int max_read_amp = cf_options.compaction_options_universal.max_read_amp; + if (max_read_amp < -1) { + return Status::NotSupported( + "CompactionOptionsUniversal::max_read_amp should be at least -1."); + } else if (0 < max_read_amp && + max_read_amp < cf_options.level0_file_num_compaction_trigger) { + return Status::NotSupported( + "CompactionOptionsUniversal::max_read_amp limits the number of sorted" + " runs but is smaller than the compaction trigger " + "level0_file_num_compaction_trigger."); + } + } return s; } diff --git a/db/compaction/compaction_iterator.cc b/db/compaction/compaction_iterator.cc index bc19de8ec28..60592489bcc 100644 --- a/db/compaction/compaction_iterator.cc +++ b/db/compaction/compaction_iterator.cc @@ -1280,6 +1280,21 @@ void CompactionIterator::DecideOutputLevel() { } #endif // NDEBUG + // saved_seq_for_penul_check_ is populated in `NextFromInput` when the + // entry's sequence number is non zero and validity context for output this + // entry is kSwapPreferredSeqno for use in `DecideOutputLevel`. It should be + // cleared out here unconditionally. Otherwise, it may end up getting consumed + // incorrectly by a different entry. + 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; + saved_seq_for_penul_check_ = std::nullopt; + ParsedInternalKey ikey_for_range_check = ikey_; + if (seq_for_range_check != ikey_.sequence) { + ikey_for_range_check.sequence = seq_for_range_check; + } if (output_to_penultimate_level_) { // If it's decided to output to the penultimate level, but unsafe to do so, // still output to the last level. For example, moving the data from a lower @@ -1287,16 +1302,6 @@ 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_for_range_check); if (!safe_to_penultimate_level) { @@ -1310,7 +1315,7 @@ void CompactionIterator::DecideOutputLevel() { // snapshot is released before enabling `last_level_temperature` feature // We will migrate the feature to `last_level_temperature` and maybe make // it not dynamically changeable. - if (ikey_.sequence > earliest_snapshot_) { + if (seq_for_range_check > earliest_snapshot_) { status_ = Status::Corruption( "Unsafe to store Seq later than snapshot in the last level if " "per_key_placement is enabled"); diff --git a/db/compaction/compaction_picker.cc b/db/compaction/compaction_picker.cc index 57e4457db12..e049d95b24e 100644 --- a/db/compaction/compaction_picker.cc +++ b/db/compaction/compaction_picker.cc @@ -340,8 +340,8 @@ Compaction* CompactionPicker::CompactFiles( #ifndef NDEBUG assert(input_files.size()); // This compaction output should not overlap with a running compaction as - // `SanitizeCompactionInputFiles` should've checked earlier and db mutex - // shouldn't have been released since. + // `SanitizeAndConvertCompactionInputFiles` should've checked earlier and db + // mutex shouldn't have been released since. int start_level = Compaction::kInvalidLevel; for (const auto& in : input_files) { // input_files should already be sorted by level @@ -1040,19 +1040,18 @@ Status CompactionPicker::SanitizeCompactionInputFilesForAllLevels( } } } - if (RangeOverlapWithCompaction(smallestkey, largestkey, output_level)) { - return Status::Aborted( - "A running compaction is writing to the same output level in an " - "overlapping key range"); - } return Status::OK(); } -Status CompactionPicker::SanitizeCompactionInputFiles( +Status CompactionPicker::SanitizeAndConvertCompactionInputFiles( std::unordered_set* input_files, - const ColumnFamilyMetaData& cf_meta, const int output_level) const { + const ColumnFamilyMetaData& cf_meta, const int output_level, + const VersionStorageInfo* vstorage, + std::vector* converted_input_files) const { assert(static_cast(cf_meta.levels.size()) - 1 == cf_meta.levels[cf_meta.levels.size() - 1].level); + assert(converted_input_files); + if (output_level >= static_cast(cf_meta.levels.size())) { return Status::InvalidArgument( "Output level for column family " + cf_meta.name + @@ -1078,7 +1077,6 @@ Status CompactionPicker::SanitizeCompactionInputFiles( Status s = SanitizeCompactionInputFilesForAllLevels(input_files, cf_meta, output_level); - if (!s.ok()) { return s; } @@ -1119,6 +1117,22 @@ Status CompactionPicker::SanitizeCompactionInputFiles( } } + s = GetCompactionInputsFromFileNumbers(converted_input_files, input_files, + vstorage, CompactionOptions()); + if (!s.ok()) { + return s; + } + assert(converted_input_files->size() > 0); + if (output_level != 0 && + FilesRangeOverlapWithCompaction( + *converted_input_files, output_level, + Compaction::EvaluatePenultimateLevel( + vstorage, ioptions_, (*converted_input_files)[0].level, + output_level))) { + return Status::Aborted( + "A running compaction is writing to the same output level(s) in an " + "overlapping key range"); + } return Status::OK(); } diff --git a/db/compaction/compaction_picker.h b/db/compaction/compaction_picker.h index 0cec71b475f..88915d45946 100644 --- a/db/compaction/compaction_picker.h +++ b/db/compaction/compaction_picker.h @@ -86,16 +86,20 @@ class CompactionPicker { virtual bool NeedsCompaction(const VersionStorageInfo* vstorage) const = 0; -// Sanitize the input set of compaction input files. -// When the input parameters do not describe a valid compaction, the -// function will try to fix the input_files by adding necessary -// files. If it's not possible to conver an invalid input_files -// into a valid one by adding more files, the function will return a -// non-ok status with specific reason. -// - Status SanitizeCompactionInputFiles(std::unordered_set* input_files, - const ColumnFamilyMetaData& cf_meta, - const int output_level) const; + // Sanitize the input set of compaction input files and convert it to + // `std::vector` in the output parameter + // `converted_input_files`. + // When the input parameters do not describe a valid + // compaction, the function will try to fix the input_files by adding + // necessary files. If it's not possible to convert an invalid input_files + // into a valid one by adding more files, the function will return a + // non-ok status with specific reason. + // + Status SanitizeAndConvertCompactionInputFiles( + std::unordered_set* input_files, + const ColumnFamilyMetaData& cf_meta, const int output_level, + const VersionStorageInfo* vstorage, + std::vector* converted_input_files) const; // Free up the files that participated in a compaction // @@ -109,8 +113,8 @@ class CompactionPicker { // object. // // Caller must provide a set of input files that has been passed through - // `SanitizeCompactionInputFiles` earlier. The lock should not be released - // between that call and this one. + // `SanitizeAndConvertCompactionInputFiles` earlier. The lock should not be + // released between that call and this one. Compaction* CompactFiles(const CompactionOptions& compact_options, const std::vector& input_files, int output_level, VersionStorageInfo* vstorage, @@ -120,6 +124,7 @@ class CompactionPicker { // Converts a set of compaction input file numbers into // a list of CompactionInputFiles. + // TODO(hx235): remove the unused paramter `compact_options` Status GetCompactionInputsFromFileNumbers( std::vector* input_files, std::unordered_set* input_set, @@ -225,8 +230,8 @@ class CompactionPicker { protected: const ImmutableOptions& ioptions_; -// A helper function to SanitizeCompactionInputFiles() that -// sanitizes "input_files" by adding necessary files. + // A helper function to SanitizeAndConvertCompactionInputFiles() that + // sanitizes "input_files" by adding necessary files. virtual Status SanitizeCompactionInputFilesForAllLevels( std::unordered_set* input_files, const ColumnFamilyMetaData& cf_meta, const int output_level) const; diff --git a/db/compaction/compaction_picker_test.cc b/db/compaction/compaction_picker_test.cc index 93b2c7bae6f..70f59a87656 100644 --- a/db/compaction/compaction_picker_test.cc +++ b/db/compaction/compaction_picker_test.cc @@ -4330,6 +4330,118 @@ TEST_F(CompactionPickerTest, IntraL0WhenL0IsSmall) { } } +TEST_F(CompactionPickerTest, UniversalMaxReadAmpLargeDB) { + ioptions_.compaction_style = kCompactionStyleUniversal; + ioptions_.num_levels = 50; + mutable_cf_options_.RefreshDerivedOptions(ioptions_); + mutable_cf_options_.compaction_options_universal.size_ratio = 10; + mutable_cf_options_.write_buffer_size = 256 << 20; + // Avoid space amp compaction + mutable_cf_options_.compaction_options_universal + .max_size_amplification_percent = 200; + const int kMaxRuns = 8; + for (int max_read_amp : {kMaxRuns, 0, -1}) { + SCOPED_TRACE("max_read_amp = " + std::to_string(max_read_amp)); + if (max_read_amp == -1) { + mutable_cf_options_.level0_file_num_compaction_trigger = kMaxRuns; + } else { + mutable_cf_options_.level0_file_num_compaction_trigger = 4; + } + mutable_cf_options_.compaction_options_universal.max_read_amp = + max_read_amp; + UniversalCompactionPicker universal_compaction_picker(ioptions_, &icmp_); + uint64_t max_run_size = 20ull << 30; + // When max_read_amp = 0, we estimate the number of levels needed based on + // size_ratio and write_buffer_size. See more in + // UniversalCompactionBuilder::PickCompaction(). + // With a 20GB last level, we estimate that 8 levels are needed: + // L0 256MB + // L1 256MB * 1.1 (size_ratio) = 282MB + // L2 (256MB + 282MB) * 1.1 = 592MB + // L3 1243MB + // L4 2610MB + // L5 5481MB + // L6 11510MB + // L7 24171MB > 20GB + for (int i = 0; i <= kMaxRuns; ++i) { + SCOPED_TRACE("i = " + std::to_string(i)); + NewVersionStorage(/*num_levels=*/50, kCompactionStyleUniversal); + Add(/*level=*/49, /*file_number=*/10, /*smallest=*/"100", + /*largest=*/"200", /*file_size=*/max_run_size, /*path_id=*/0, + /*smallest_seq=*/0, /*largest_seq=*/0, + /*compensated_file_size=*/max_run_size); + // Besides the last sorted run, we add additional `i` sorted runs + // without triggering space-amp or size-amp compactions. + uint64_t file_size = 1 << 20; + for (int j = 0; j < i; ++j) { + Add(/*level=*/j, /*file_number=*/100 - j, /*smallest=*/"100", + /*largest=*/"200", /*file_size=*/file_size, /*path_id=*/0, + /*smallest_seq=*/100 - j, /*largest_seq=*/100 - j, + /*compensated_file_size=*/file_size); + // to avoid space-amp and size-amp compaction + file_size *= 2; + } + UpdateVersionStorageInfo(); + // level0_file_num_compaction_trigger is still used as trigger to + // check potential compactions + ASSERT_EQ( + universal_compaction_picker.NeedsCompaction(vstorage_.get()), + i + 1 >= mutable_cf_options_.level0_file_num_compaction_trigger); + std::unique_ptr compaction( + universal_compaction_picker.PickCompaction( + cf_name_, mutable_cf_options_, mutable_db_options_, + vstorage_.get(), &log_buffer_)); + if (i == kMaxRuns) { + // There are in total i + 1 > kMaxRuns sorted runs. + // This triggers compaction ignoring size_ratio. + ASSERT_NE(nullptr, compaction); + ASSERT_EQ(CompactionReason::kUniversalSortedRunNum, + compaction->compaction_reason()); + // First two runs are compacted + ASSERT_EQ(0, compaction->start_level()); + ASSERT_EQ(1, compaction->output_level()); + ASSERT_EQ(1U, compaction->num_input_files(0)); + ASSERT_EQ(1U, compaction->num_input_files(1)); + } else { + ASSERT_EQ(nullptr, compaction); + } + } + } +} + +TEST_F(CompactionPickerTest, UniversalMaxReadAmpSmallDB) { + ioptions_.compaction_style = kCompactionStyleUniversal; + ioptions_.num_levels = 50; + mutable_cf_options_.RefreshDerivedOptions(ioptions_); + mutable_cf_options_.level0_file_num_compaction_trigger = 1; + mutable_cf_options_.compaction_options_universal.size_ratio = 10; + mutable_cf_options_.write_buffer_size = 256 << 20; + mutable_cf_options_.compaction_options_universal + .max_size_amplification_percent = 200; + const int kMaxRuns = 1; + for (int max_read_amp : {-1, kMaxRuns, 0}) { + SCOPED_TRACE("max_read_amp = " + std::to_string(max_read_amp)); + mutable_cf_options_.compaction_options_universal.max_read_amp = + max_read_amp; + UniversalCompactionPicker universal_compaction_picker(ioptions_, &icmp_); + NewVersionStorage(/*num_levels=*/50, kCompactionStyleUniversal); + // max_run_size is much smaller than write_buffer_size, + // only 1 level is needed. + uint64_t max_run_size = 8 << 10; + Add(/*level=*/49, /*file_number=*/10, /*smallest=*/"100", + /*largest=*/"200", /*file_size=*/max_run_size, /*path_id=*/0, + /*smallest_seq=*/0, /*largest_seq=*/0, + /*compensated_file_size=*/max_run_size); + UpdateVersionStorageInfo(); + ASSERT_TRUE(universal_compaction_picker.NeedsCompaction(vstorage_.get())); + std::unique_ptr compaction( + universal_compaction_picker.PickCompaction( + cf_name_, mutable_cf_options_, mutable_db_options_, vstorage_.get(), + &log_buffer_)); + ASSERT_EQ(nullptr, compaction); + } +} + } // namespace ROCKSDB_NAMESPACE int main(int argc, char** argv) { diff --git a/db/compaction/compaction_picker_universal.cc b/db/compaction/compaction_picker_universal.cc index da71e398779..0b5426149ea 100644 --- a/db/compaction/compaction_picker_universal.cc +++ b/db/compaction/compaction_picker_universal.cc @@ -227,6 +227,7 @@ class UniversalCompactionBuilder { const InternalKeyComparator* icmp_; double score_; std::vector sorted_runs_; + uint64_t max_run_size_; const std::string& cf_name_; const MutableCFOptions& mutable_cf_options_; const MutableDBOptions& mutable_db_options_; @@ -235,7 +236,8 @@ class UniversalCompactionBuilder { LogBuffer* log_buffer_; static std::vector CalculateSortedRuns( - const VersionStorageInfo& vstorage, int last_level); + const VersionStorageInfo& vstorage, int last_level, + uint64_t* max_run_size); // Pick a path ID to place a newly generated file, with its estimated file // size. @@ -440,11 +442,15 @@ void UniversalCompactionBuilder::SortedRun::DumpSizeInfo( std::vector UniversalCompactionBuilder::CalculateSortedRuns( - const VersionStorageInfo& vstorage, int last_level) { + const VersionStorageInfo& vstorage, int last_level, + uint64_t* max_run_size) { + assert(max_run_size); + *max_run_size = 0; std::vector ret; for (FileMetaData* f : vstorage.LevelFiles(0)) { ret.emplace_back(0, f, f->fd.GetFileSize(), f->compensated_file_size, f->being_compacted); + *max_run_size = std::max(*max_run_size, f->fd.GetFileSize()); } for (int level = 1; level <= last_level; level++) { uint64_t total_compensated_size = 0U; @@ -466,6 +472,7 @@ UniversalCompactionBuilder::CalculateSortedRuns( ret.emplace_back(level, nullptr, total_size, total_compensated_size, being_compacted); } + *max_run_size = std::max(*max_run_size, total_size); } return ret; } @@ -477,13 +484,16 @@ Compaction* UniversalCompactionBuilder::PickCompaction() { score_ = vstorage_->CompactionScore(kLevel0); int max_output_level = vstorage_->MaxOutputLevel(ioptions_.allow_ingest_behind); - sorted_runs_ = CalculateSortedRuns(*vstorage_, max_output_level); + max_run_size_ = 0; + sorted_runs_ = + CalculateSortedRuns(*vstorage_, max_output_level, &max_run_size_); + int file_num_compaction_trigger = + mutable_cf_options_.level0_file_num_compaction_trigger; if (sorted_runs_.size() == 0 || (vstorage_->FilesMarkedForPeriodicCompaction().empty() && vstorage_->FilesMarkedForCompaction().empty() && - sorted_runs_.size() < (unsigned int)mutable_cf_options_ - .level0_file_num_compaction_trigger)) { + sorted_runs_.size() < (unsigned int)file_num_compaction_trigger)) { ROCKS_LOG_BUFFER(log_buffer_, "[%s] Universal: nothing to do\n", cf_name_.c_str()); TEST_SYNC_POINT_CALLBACK( @@ -505,11 +515,9 @@ Compaction* UniversalCompactionBuilder::PickCompaction() { TEST_SYNC_POINT_CALLBACK("PostPickPeriodicCompaction", c); } - // Check for size amplification. if (c == nullptr && - sorted_runs_.size() >= - static_cast( - mutable_cf_options_.level0_file_num_compaction_trigger)) { + sorted_runs_.size() >= static_cast(file_num_compaction_trigger)) { + // Check for size amplification. if ((c = PickCompactionToReduceSizeAmp()) != nullptr) { TEST_SYNC_POINT("PickCompactionToReduceSizeAmpReturnNonnullptr"); ROCKS_LOG_BUFFER(log_buffer_, "[%s] Universal: compacting for size amp\n", @@ -527,13 +535,48 @@ Compaction* UniversalCompactionBuilder::PickCompaction() { cf_name_.c_str()); } else { // Size amplification and file size ratios are within configured limits. - // If max read amplification is exceeding configured limits, then force - // compaction without looking at filesize ratios and try to reduce - // the number of files to fewer than level0_file_num_compaction_trigger. + // If max read amplification exceeds configured limits, then force + // compaction to reduce the number sorted runs without looking at file + // size ratios. + // This is guaranteed by NeedsCompaction() assert(sorted_runs_.size() >= - static_cast( - mutable_cf_options_.level0_file_num_compaction_trigger)); + static_cast(file_num_compaction_trigger)); + int max_num_runs = + mutable_cf_options_.compaction_options_universal.max_read_amp; + if (max_num_runs < 0) { + // any value < -1 is not valid + assert(max_num_runs == -1); + // By default, fall back to `level0_file_num_compaction_trigger` + max_num_runs = file_num_compaction_trigger; + } else if (max_num_runs == 0) { + if (mutable_cf_options_.compaction_options_universal.stop_style == + kCompactionStopStyleTotalSize) { + // 0 means auto-tuning by RocksDB. We estimate max num run based on + // max_run_size, size_ratio and write buffer size: + // Assume the size of the lowest level size is equal to + // write_buffer_size. Each subsequent level is the max size without + // triggering size_ratio compaction. `max_num_runs` is the minimum + // number of levels required such that the target size of the + // largest level is at least `max_run_size_`. + max_num_runs = 1; + double cur_level_max_size = + static_cast(mutable_cf_options_.write_buffer_size); + double total_run_size = 0; + while (cur_level_max_size < static_cast(max_run_size_)) { + // This loop should not take too many iterations since + // cur_level_max_size at least doubles each iteration. + total_run_size += cur_level_max_size; + cur_level_max_size = (100.0 + ratio) / 100.0 * total_run_size; + ++max_num_runs; + } + } else { + // TODO: implement the auto-tune logic for this stop style + max_num_runs = file_num_compaction_trigger; + } + } else { + // max_num_runs > 0, it's the limit on the number of sorted run + } // Get the total number of sorted runs that are not being compacted int num_sr_not_compacted = 0; for (size_t i = 0; i < sorted_runs_.size(); i++) { @@ -544,17 +587,25 @@ Compaction* UniversalCompactionBuilder::PickCompaction() { // The number of sorted runs that are not being compacted is greater // than the maximum allowed number of sorted runs - if (num_sr_not_compacted > - mutable_cf_options_.level0_file_num_compaction_trigger) { - unsigned int num_files = - num_sr_not_compacted - - mutable_cf_options_.level0_file_num_compaction_trigger + 1; + if (num_sr_not_compacted > max_num_runs) { + unsigned int num_files = num_sr_not_compacted - max_num_runs + 1; if ((c = PickCompactionToReduceSortedRuns(UINT_MAX, num_files)) != nullptr) { ROCKS_LOG_BUFFER(log_buffer_, - "[%s] Universal: compacting for file num -- %u\n", - cf_name_.c_str(), num_files); + "[%s] Universal: compacting for file num, to " + "compact file num -- %u, max num runs allowed" + "-- %d, max_run_size -- %" PRIu64 "\n", + cf_name_.c_str(), num_files, max_num_runs, + max_run_size_); } + } else { + ROCKS_LOG_BUFFER( + log_buffer_, + "[%s] Universal: skipping compaction for file num, num runs not " + "being compacted -- %u, max num runs allowed -- %d, max_run_size " + "-- %" PRIu64 "\n", + cf_name_.c_str(), num_sr_not_compacted, max_num_runs, + max_run_size_); } } } diff --git a/db/compaction/tiered_compaction_test.cc b/db/compaction/tiered_compaction_test.cc index d8b48b991ef..c99db985098 100644 --- a/db/compaction/tiered_compaction_test.cc +++ b/db/compaction/tiered_compaction_test.cc @@ -1651,6 +1651,31 @@ TEST_P(TimedPutPrecludeLastLevelTest, FastTrackTimedPutToLastLevel) { Close(); } +TEST_P(TimedPutPrecludeLastLevelTest, InterleavedTimedPutAndPut) { + Options options = CurrentOptions(); + options.compaction_style = kCompactionStyleUniversal; + options.disable_auto_compactions = true; + options.preclude_last_level_data_seconds = 1 * 24 * 60 * 60; + 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(); + + // Start time: kMockStartTime = 10000000; + ASSERT_OK(TimedPut(0, Key(0), "v0", kMockStartTime - 1 * 24 * 60 * 60, wo)); + ASSERT_OK(Put(Key(1), "v1", wo)); + ASSERT_OK(Flush()); + + ASSERT_OK(db_->CompactRange(CompactRangeOptions(), nullptr, nullptr)); + ASSERT_EQ("0,0,0,0,0,1,1", FilesPerLevel()); + ASSERT_GT(GetSstSizeHelper(Temperature::kHot), 0); + ASSERT_GT(GetSstSizeHelper(Temperature::kCold), 0); + Close(); +} + TEST_P(TimedPutPrecludeLastLevelTest, PreserveTimedPutOnPenultimateLevel) { Options options = CurrentOptions(); options.compaction_style = kCompactionStyleUniversal; diff --git a/db/db_compaction_test.cc b/db/db_compaction_test.cc index e03093077f4..83b39a5218f 100644 --- a/db/db_compaction_test.cc +++ b/db/db_compaction_test.cc @@ -16,6 +16,7 @@ #include "env/mock_env.h" #include "port/port.h" #include "port/stack_trace.h" +#include "rocksdb/advanced_options.h" #include "rocksdb/concurrent_task_limiter.h" #include "rocksdb/experimental.h" #include "rocksdb/sst_file_writer.h" @@ -6204,29 +6205,14 @@ TEST_F(DBCompactionTest, CompactionLimiter) { NumTableFilesAtLevel(0, 0)); } - // All CFs are pending compaction + // Wait until all CFs are pending compaction. WaitForFlushMemtable() can + // return before the next compaction is scheduled, so we need to do some + // waiting here. 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()); - } + for (int i = 0; i < 10000 && tp_len < cf_count; i++) { + env_->SleepForMicroseconds(1000); + tp_len = env_->GetThreadPoolQueueLen(Env::LOW); } - ASSERT_EQ(cf_count, tp_len); // Unblock all compaction threads @@ -7135,7 +7121,8 @@ class DBCompactionTestWithOngoingFileIngestionParam SyncPoint::GetInstance()->SetCallBack( "ExternalSstFileIngestionJob::Run", [&](void*) { SyncPoint::GetInstance()->LoadDependency( - {{"DBImpl::CompactFilesImpl::PostSanitizeCompactionInputFiles", + {{"DBImpl::CompactFilesImpl::" + "PostSanitizeAndConvertCompactionInputFiles", "VersionSet::LogAndApply:WriteManifest"}}); }); } else { @@ -7617,6 +7604,72 @@ class DBCompactionTestL0FilesMisorderCorruption : public DBCompactionTest { std::shared_ptr sleeping_task_; }; +TEST_F(DBCompactionTest, CompactFilesSupportKeyPlacementRangeConflict) { + Options options; + options.create_if_missing = true; + options.compaction_style = kCompactionStyleUniversal; + options.num_levels = 3; + DestroyAndReopen(options); + + // To create LSM of below shape: + // L0: [k2] + // L1: [k3],[k4] + // L2: [k1, k5] + ASSERT_OK(Put("k1", "v")); + ASSERT_OK(Put("k5", "v")); + ASSERT_OK(Flush()); + CompactRangeOptions cro; + cro.bottommost_level_compaction = BottommostLevelCompaction::kForceOptimized; + ASSERT_OK(db_->CompactRange(cro, nullptr, nullptr)); + ASSERT_EQ("0,0,1", FilesPerLevel()); + + ASSERT_OK(Put("k3", "v")); + ASSERT_OK(Flush()); + ASSERT_OK(Put("k4", "v")); + ASSERT_OK(Flush()); + ASSERT_OK(experimental::PromoteL0(db_, db_->DefaultColumnFamily(), 1)); + ASSERT_EQ("0,2,1", FilesPerLevel()); + + ASSERT_OK(Put("k2", "v")); + ASSERT_OK(Flush()); + ASSERT_EQ("1,2,1", FilesPerLevel()); + + Close(); + + // To force below two CompactFiles() in order to coerce range conflict on L1 + // upon (2) + // (1): Compact [k2] at L0 and [k3] at L1 with output to L1 + // (2): Compact [k4] at L1 and [k1, k5] at L2 and output to L1 and L2 + options.preclude_last_level_data_seconds = 1; + Reopen(options); + + ColumnFamilyMetaData cf_meta_data; + db_->GetColumnFamilyMetaData(&cf_meta_data); + + ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->SetCallBack( + "CompactFilesImpl:0", [&](void* /*arg*/) { + std::vector c2_input_files; + c2_input_files.push_back(cf_meta_data.levels[1].files[1].name); + c2_input_files.push_back(cf_meta_data.levels[2].files[0].name); + // To verify CompactFiles() is aborted upon range conflict instead + // of crashing upon internal assertion + Status s = db_->CompactFiles(CompactionOptions(), c2_input_files, + 2 /* output_level */); + ASSERT_TRUE(s.IsAborted()); + }); + ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing(); + + std::vector c1_input_files; + c1_input_files.push_back(cf_meta_data.levels[0].files[0].name); + c1_input_files.push_back(cf_meta_data.levels[1].files[0].name); + Status s = db_->CompactFiles(CompactionOptions(), c1_input_files, + 1 /* output_level */); + ASSERT_OK(s); + + ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing(); + ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->ClearAllCallBacks(); +} + TEST_F(DBCompactionTestL0FilesMisorderCorruption, FlushAfterIntraL0LevelCompactionWithIngestedFile) { SetupOptions(CompactionStyle::kCompactionStyleLevel, ""); diff --git a/db/db_filesnapshot.cc b/db/db_filesnapshot.cc index 610ec67cfbe..747aa0c4976 100644 --- a/db/db_filesnapshot.cc +++ b/db/db_filesnapshot.cc @@ -91,7 +91,7 @@ Status DBImpl::GetLiveFiles(std::vector& ret, return Status::OK(); } -Status DBImpl::GetSortedWalFiles(VectorLogPtr& files) { +Status DBImpl::GetSortedWalFiles(VectorWalPtr& files) { // Record tracked WALs as a (minimum) cross-check for directory scan std::vector required_by_manifest; @@ -152,11 +152,11 @@ Status DBImpl::GetSortedWalFiles(VectorLogPtr& files) { } if (s.ok()) { - size_t wal_size = files.size(); + size_t wal_count = files.size(); ROCKS_LOG_INFO(immutable_db_options_.info_log, - "Number of log files %" ROCKSDB_PRIszt " (%" ROCKSDB_PRIszt + "Number of WAL files %" ROCKSDB_PRIszt " (%" ROCKSDB_PRIszt " required by manifest)", - wal_size, required_by_manifest.size()); + wal_count, required_by_manifest.size()); #ifndef NDEBUG std::ostringstream wal_names; for (const auto& wal : files) { @@ -177,7 +177,7 @@ Status DBImpl::GetSortedWalFiles(VectorLogPtr& files) { return s; } -Status DBImpl::GetCurrentWalFile(std::unique_ptr* current_log_file) { +Status DBImpl::GetCurrentWalFile(std::unique_ptr* current_log_file) { uint64_t current_logfile_number; { InstrumentedMutexLock l(&mutex_); @@ -198,13 +198,13 @@ Status DBImpl::GetLiveFilesStorageInfo( // NOTE: This implementation was largely migrated from Checkpoint. Status s; - VectorLogPtr live_wal_files; + VectorWalPtr live_wal_files; bool flush_memtable = true; if (!immutable_db_options_.allow_2pc) { if (opts.wal_size_for_flush == std::numeric_limits::max()) { flush_memtable = false; } else if (opts.wal_size_for_flush > 0) { - // If the outstanding log files are small, we skip the flush. + // If the outstanding WAL files are small, we skip the flush. s = GetSortedWalFiles(live_wal_files); if (!s.ok()) { @@ -229,12 +229,18 @@ Status DBImpl::GetLiveFilesStorageInfo( // metadata. mutex_.Lock(); if (flush_memtable) { - Status status = FlushForGetLiveFiles(); - if (!status.ok()) { - mutex_.Unlock(); - ROCKS_LOG_ERROR(immutable_db_options_.info_log, "Cannot Flush data %s\n", - status.ToString().c_str()); - return status; + bool wal_locked = lock_wal_count_ > 0; + if (wal_locked) { + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "Can't FlushForGetLiveFiles while WAL is locked"); + } else { + Status status = FlushForGetLiveFiles(); + if (!status.ok()) { + mutex_.Unlock(); + ROCKS_LOG_ERROR(immutable_db_options_.info_log, + "Cannot Flush data %s\n", status.ToString().c_str()); + return status; + } } } @@ -393,11 +399,11 @@ Status DBImpl::GetLiveFilesStorageInfo( return s; } - size_t wal_size = live_wal_files.size(); + size_t wal_count = 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(); - for (size_t i = 0; s.ok() && i < wal_size; ++i) { + for (size_t i = 0; s.ok() && i < wal_count; ++i) { auto const include_file = [&]{ if (opts.include_all_wal_files) { return true; @@ -418,7 +424,7 @@ Status DBImpl::GetLiveFilesStorageInfo( // Trim the log either if its the last one, or log file recycling is // enabled. In the latter case, a hard link doesn't prevent the file // from being renamed and recycled. So we need to copy it instead. - info.trim_to_size = (i + 1 == wal_size) || + info.trim_to_size = (i + 1 == wal_count) || (immutable_db_options_.recycle_log_file_num > 0); if (opts.include_checksum_info) { info.file_checksum_func_name = kUnknownFileChecksumFuncName; diff --git a/db/db_flush_test.cc b/db/db_flush_test.cc index 6846d96d0ef..b72de9a6886 100644 --- a/db/db_flush_test.cc +++ b/db/db_flush_test.cc @@ -86,8 +86,8 @@ TEST_F(DBFlushTest, SyncFail) { options.env = fault_injection_env.get(); SyncPoint::GetInstance()->LoadDependency( - {{"DBFlushTest::SyncFail:1", "DBImpl::SyncClosedLogs:Start"}, - {"DBImpl::SyncClosedLogs:Failed", "DBFlushTest::SyncFail:2"}}); + {{"DBFlushTest::SyncFail:1", "DBImpl::SyncClosedWals:Start"}, + {"DBImpl::SyncClosedWals:Failed", "DBFlushTest::SyncFail:2"}}); SyncPoint::GetInstance()->EnableProcessing(); CreateAndReopenWithCF({"pikachu"}, options); @@ -111,8 +111,8 @@ TEST_F(DBFlushTest, SyncSkip) { Options options = CurrentOptions(); SyncPoint::GetInstance()->LoadDependency( - {{"DBFlushTest::SyncSkip:1", "DBImpl::SyncClosedLogs:Skip"}, - {"DBImpl::SyncClosedLogs:Skip", "DBFlushTest::SyncSkip:2"}}); + {{"DBFlushTest::SyncSkip:1", "DBImpl::SyncClosedWals:Skip"}, + {"DBImpl::SyncClosedWals:Skip", "DBFlushTest::SyncSkip:2"}}); SyncPoint::GetInstance()->EnableProcessing(); Reopen(options); @@ -2381,7 +2381,7 @@ TEST_F(DBFlushTest, PickRightMemtables) { SyncPoint::GetInstance()->DisableProcessing(); SyncPoint::GetInstance()->ClearAllCallBacks(); SyncPoint::GetInstance()->SetCallBack( - "DBImpl::SyncClosedLogs:BeforeReLock", [&](void* /*arg*/) { + "DBImpl::SyncClosedWals:BeforeReLock", [&](void* /*arg*/) { ASSERT_OK(db_->Put(WriteOptions(), handles_[1], "what", "v")); auto* cfhi = static_cast_with_check(handles_[1]); diff --git a/db/db_follower_test.cc b/db/db_follower_test.cc index 86bf8cc7c53..febc5ae4a46 100644 --- a/db/db_follower_test.cc +++ b/db/db_follower_test.cc @@ -17,6 +17,7 @@ class DBFollowerTest : public DBTestBase { // Create the leader DB object DBFollowerTest() : DBTestBase("/db_follower_test", /*env_do_fsync*/ false) { follower_name_ = dbname_ + "/follower"; + db_parent_ = dbname_; Close(); Destroy(CurrentOptions()); EXPECT_EQ(env_->CreateDirIfMissing(dbname_), Status::OK()); @@ -27,17 +28,215 @@ class DBFollowerTest : public DBTestBase { ~DBFollowerTest() { follower_.reset(); EXPECT_EQ(DestroyDB(follower_name_, CurrentOptions()), Status::OK()); + Destroy(CurrentOptions()); + dbname_ = db_parent_; } protected: + class DBFollowerTestFS : public FileSystemWrapper { + public: + explicit DBFollowerTestFS(const std::shared_ptr& target) + : FileSystemWrapper(target), + cv_(&mutex_), + barrier_(false), + count_(0), + reinit_count_(0) {} + + const char* Name() const override { return "DBFollowerTestFS"; } + + IOStatus NewSequentialFile(const std::string& fname, + const FileOptions& file_opts, + std::unique_ptr* result, + IODebugContext* dbg = nullptr) override { + class DBFollowerTestSeqFile : public FSSequentialFileWrapper { + public: + DBFollowerTestSeqFile(DBFollowerTestFS* fs, + std::unique_ptr&& file, + uint64_t /*size*/) + : FSSequentialFileWrapper(file.get()), + fs_(fs), + file_(std::move(file)) {} + + IOStatus Read(size_t n, const IOOptions& options, Slice* result, + char* scratch, IODebugContext* dbg) override { + fs_->BarrierWait(); + return target()->Read(n, options, result, scratch, dbg); + } + + private: + DBFollowerTestFS* fs_; + std::unique_ptr file_; + }; + + std::unique_ptr file; + IOStatus s = target()->NewSequentialFile(fname, file_opts, &file, dbg); + + if (s.ok() && test::GetFileType(fname) == kDescriptorFile) { + uint64_t size = 0; + EXPECT_EQ(target()->GetFileSize(fname, IOOptions(), &size, nullptr), + IOStatus::OK()); + result->reset(new DBFollowerTestSeqFile(this, std::move(file), size)); + } else { + *result = std::move(file); + } + return s; + } + + void BarrierInit(int count) { + MutexLock l(&mutex_); + barrier_ = true; + count_ = count; + } + + void BarrierWait() { + MutexLock l(&mutex_); + if (!barrier_) { + return; + } + if (--count_ == 0) { + if (reinit_count_ > 0) { + count_ = reinit_count_; + reinit_count_ = 0; + } else { + barrier_ = false; + } + cv_.SignalAll(); + } else { + cv_.Wait(); + } + } + + void BarrierWaitAndReinit(int count) { + MutexLock l(&mutex_); + if (!barrier_) { + return; + } + reinit_count_ = count; + if (--count_ == 0) { + if (reinit_count_ > 0) { + count_ = reinit_count_; + reinit_count_ = 0; + } else { + barrier_ = false; + } + cv_.SignalAll(); + } else { + cv_.Wait(); + } + } + + private: + port::Mutex mutex_; + port::CondVar cv_; + bool barrier_; + int count_; + int reinit_count_; + }; + + class DBFollowerTestSstPartitioner : public SstPartitioner { + public: + explicit DBFollowerTestSstPartitioner(uint64_t max_keys) + : max_keys_(max_keys), num_keys_(0) {} + + const char* Name() const override { return "DBFollowerTestSstPartitioner"; } + + PartitionerResult ShouldPartition( + const PartitionerRequest& /*request*/) override { + if (++num_keys_ > max_keys_) { + num_keys_ = 0; + return PartitionerResult::kRequired; + } else { + return PartitionerResult::kNotRequired; + } + } + + bool CanDoTrivialMove(const Slice& /*smallest_user_key*/, + const Slice& /*largest_user_key*/) override { + return true; + } + + private: + uint64_t max_keys_; + uint64_t num_keys_; + }; + + class DBFollowerTestSstPartitionerFactory : public SstPartitionerFactory { + public: + explicit DBFollowerTestSstPartitionerFactory(uint64_t max_keys) + : max_keys_(max_keys) {} + + std::unique_ptr CreatePartitioner( + const SstPartitioner::Context& /*context*/) const override { + std::unique_ptr partitioner; + partitioner.reset(new DBFollowerTestSstPartitioner(max_keys_)); + return partitioner; + } + + const char* Name() const override { + return "DBFollowerTestSstPartitionerFactory"; + } + + private: + uint64_t max_keys_; + }; + Status OpenAsFollower() { - return DB::OpenAsFollower(CurrentOptions(), follower_name_, dbname_, - &follower_); + Options opts = CurrentOptions(); + if (!follower_env_) { + follower_env_ = NewCompositeEnv( + std::make_shared(env_->GetFileSystem())); + } + opts.env = follower_env_.get(); + opts.follower_refresh_catchup_period_ms = 100; + return DB::OpenAsFollower(opts, follower_name_, dbname_, &follower_); + } + + std::string FollowerGet(const std::string& k) { + ReadOptions options; + options.verify_checksums = true; + std::string result; + Status s = follower()->Get(options, k, &result); + if (s.IsNotFound()) { + result = "NOT_FOUND"; + } else if (!s.ok()) { + result = s.ToString(); + } + return result; } + DB* follower() { return follower_.get(); } + DBFollowerTestFS* follower_fs() { + return static_cast(follower_env_->GetFileSystem().get()); + } + + void CheckDirs() { + std::vector db_children; + std::vector follower_children; + EXPECT_OK(env_->GetChildren(dbname_, &db_children)); + EXPECT_OK(env_->GetChildren(follower_name_, &follower_children)); + + std::set db_filenums; + std::set follower_filenums; + for (auto& name : db_children) { + if (test::GetFileType(name) != kTableFile) { + continue; + } + db_filenums.insert(test::GetFileNumber(name)); + } + for (auto& name : follower_children) { + if (test::GetFileType(name) != kTableFile) { + continue; + } + follower_filenums.insert(test::GetFileNumber(name)); + } + db_filenums.merge(follower_filenums); + EXPECT_EQ(follower_filenums.size(), db_filenums.size()); + } private: std::string follower_name_; + std::string db_parent_; + std::unique_ptr follower_env_; std::unique_ptr follower_; }; @@ -51,8 +250,273 @@ TEST_F(DBFollowerTest, Basic) { std::string val; ASSERT_OK(follower()->Get(ReadOptions(), "k1", &val)); ASSERT_EQ(val, "v1"); + CheckDirs(); +} + +TEST_F(DBFollowerTest, Flush) { + SyncPoint::GetInstance()->LoadDependency({ + {"DBImplFollower::TryCatchupWithLeader:Begin1", "Leader::Start"}, + {"Leader::Done", "DBImplFollower::TryCatchupWithLeader:Begin2"}, + {"DBImplFollower::TryCatchupWithLeader:End", "Follower::WaitForCatchup"}, + }); + SyncPoint::GetInstance()->EnableProcessing(); + + ASSERT_OK(OpenAsFollower()); + TEST_SYNC_POINT("Leader::Start"); + ASSERT_OK(Put("k1", "v1")); + ASSERT_OK(Flush()); + TEST_SYNC_POINT("Leader::Done"); + + TEST_SYNC_POINT("Follower::WaitForCatchup"); + std::string val; + ASSERT_OK(follower()->Get(ReadOptions(), "k1", &val)); + ASSERT_EQ(val, "v1"); + CheckDirs(); + + SyncPoint::GetInstance()->DisableProcessing(); +} + +// This test creates 4 L0 files, immediately followed by a compaction to L1. +// The follower replays the 4 flush records from the MANIFEST unsuccessfully, +// and then successfully recovers a Version from the compaction record +TEST_F(DBFollowerTest, RetryCatchup) { + Options opts = CurrentOptions(); + opts.disable_auto_compactions = true; + Reopen(opts); + + ASSERT_OK(OpenAsFollower()); + SyncPoint::GetInstance()->LoadDependency({ + {"DBImplFollower::TryCatchupWithLeader:Begin1", "Leader::Start"}, + {"DBImpl::BackgroundCompaction:Start", + "DBImplFollower::TryCatchupWithLeader:Begin2"}, + {"VersionEditHandlerPointInTime::MaybeCreateVersion:Begin1", + "DBImpl::BackgroundCompaction:BeforeCompaction"}, + {"DBImpl::BackgroundCallCompaction:PurgedObsoleteFiles", + "VersionEditHandlerPointInTime::MaybeCreateVersion:Begin2"}, + {"DBImplFollower::TryCatchupWithLeader:End", "Follower::WaitForCatchup"}, + }); + SyncPoint::GetInstance()->EnableProcessing(); + TEST_SYNC_POINT("Leader::Start"); + ASSERT_OK(Put("k1", "v1")); + ASSERT_OK(Flush()); + ASSERT_OK(Put("k1", "v2")); + ASSERT_OK(Flush()); + ASSERT_OK(Put("k1", "v3")); + ASSERT_OK(Flush()); + ASSERT_OK(Put("k1", "v4")); + ASSERT_OK(Flush()); + + ASSERT_OK(dbfull()->TEST_CompactRange(0, nullptr, nullptr, nullptr, true)); + + TEST_SYNC_POINT("Follower::WaitForCatchup"); + ASSERT_EQ(FollowerGet("k1"), "v4"); + CheckDirs(); + + SyncPoint::GetInstance()->ClearAllCallBacks(); + SyncPoint::GetInstance()->DisableProcessing(); } +// This test validates the same as the previous test, except there is a +// MANIFEST rollover between the flushes and compaction. The follower +// does not switch to a new MANIFEST in ReadAndApply. So it would require +// another round of refresh before catching up. +TEST_F(DBFollowerTest, RetryCatchupManifestRollover) { + Options opts = CurrentOptions(); + opts.disable_auto_compactions = true; + Reopen(opts); + + ASSERT_OK(Put("k1", "v1")); + ASSERT_OK(Flush()); + ASSERT_OK(Put("k1", "v2")); + ASSERT_OK(Flush()); + ASSERT_OK(Put("k1", "v3")); + ASSERT_OK(Flush()); + ASSERT_OK(OpenAsFollower()); + SyncPoint::GetInstance()->LoadDependency({ + {"DBImplFollower::TryCatchupWithLeader:Begin1", "Leader::Start"}, + {"Leader::Flushed", "DBImplFollower::TryCatchupWithLeader:Begin2"}, + {"VersionEditHandlerPointInTime::MaybeCreateVersion:Begin1", + "Leader::Done"}, + {"DBImpl::BackgroundCallCompaction:PurgedObsoleteFiles", + "VersionEditHandlerPointInTime::MaybeCreateVersion:Begin2"}, + {"DBImplFollower::TryCatchupWithLeader:End", + "Follower::WaitForCatchup:1"}, + }); + SyncPoint::GetInstance()->EnableProcessing(); + TEST_SYNC_POINT("Leader::Start"); + ASSERT_OK(Put("k1", "v4")); + ASSERT_OK(Flush()); + + TEST_SYNC_POINT("Leader::Flushed"); + TEST_SYNC_POINT("Leader::Done"); + Reopen(opts); + ASSERT_OK(dbfull()->TEST_CompactRange(0, nullptr, nullptr, nullptr, true)); + + TEST_SYNC_POINT("Follower::WaitForCatchup:1"); + SyncPoint::GetInstance()->LoadDependency({ + {"DBImplFollower::TryCatchupWithLeader:End", + "Follower::WaitForCatchup:2"}, + }); + TEST_SYNC_POINT("Follower::WaitForCatchup:2"); + ASSERT_EQ(FollowerGet("k1"), "v4"); + + SyncPoint::GetInstance()->ClearAllCallBacks(); + SyncPoint::GetInstance()->DisableProcessing(); +} + +// This test creates 4 L0 files and compacts them. The follower, during catchup, +// successfully instantiates 4 Versions corresponding to the 4 files (but +// donesn't install them yet), followed by deleting those 4 and adding a new +// file from compaction. The test verifies that the 4 L0 files are deleted +// correctly by the follower. +// We use teh Barrier* functions to ensure that the follower first sees the 4 +// L0 files and is able to link them, and then sees the compaction that +// obsoletes those L0 files (so those L0 files are intermediates that it has +// to explicitly delete). Suppose we don't have any barriers, its possible +// the follower reads the L0 records and compaction records from the MANIFEST +// in one read, which means those L0 files would have already been deleted +// by the leader and the follower cannot link to them. +TEST_F(DBFollowerTest, IntermediateObsoleteFiles) { + Options opts = CurrentOptions(); + opts.disable_auto_compactions = true; + Reopen(opts); + ASSERT_OK(OpenAsFollower()); + + follower_fs()->BarrierInit(2); + ASSERT_OK(Put("k1", "v1")); + ASSERT_OK(Flush()); + ASSERT_OK(Put("k1", "v2")); + ASSERT_OK(Flush()); + ASSERT_OK(Put("k1", "v3")); + ASSERT_OK(Flush()); + ASSERT_OK(Put("k1", "v4")); + ASSERT_OK(Flush()); + follower_fs()->BarrierWaitAndReinit(2); + + ASSERT_OK(dbfull()->TEST_CompactRange(0, nullptr, nullptr, nullptr, true)); + follower_fs()->BarrierWait(); + + SyncPoint::GetInstance()->LoadDependency({ + {"DBImplFollower::TryCatchupWithLeader:End", + "Follower::WaitForCatchup:1"}, + }); + SyncPoint::GetInstance()->EnableProcessing(); + TEST_SYNC_POINT("Follower::WaitForCatchup:1"); + CheckDirs(); + ASSERT_EQ(FollowerGet("k1"), "v4"); +} + +// This test verifies a scenario where the follower can recover a Version +// partially (i.e some of the additions cannot be found), and the files +// that are found are obsoleted by a subsequent VersionEdit. +TEST_F(DBFollowerTest, PartialVersionRecovery) { + Options opts = CurrentOptions(); + opts.disable_auto_compactions = true; + opts.sst_partitioner_factory = + std::make_shared(1); + Reopen(opts); + + ASSERT_OK(Put("k1", "v1")); + ASSERT_OK(Put("k2", "v1")); + ASSERT_OK(Put("k3", "v1")); + ASSERT_OK(Flush()); + MoveFilesToLevel(2); + + ASSERT_OK(Put("k1", "v2")); + ASSERT_OK(Flush()); + ASSERT_OK(Put("k3", "v2")); + ASSERT_OK(Flush()); + MoveFilesToLevel(1); + + ASSERT_OK(OpenAsFollower()); + ASSERT_OK(dbfull()->SetOptions(dbfull()->DefaultColumnFamily(), + {{"max_compaction_bytes", "1"}})); + + follower_fs()->BarrierInit(2); + Slice key("k1"); + ASSERT_OK(dbfull()->TEST_CompactRange(1, &key, &key, nullptr, true)); + + follower_fs()->BarrierWaitAndReinit(2); + + // The second compaction input overlaps the previous compaction outputs + // by one file. This file is never added to VersionStorageInfo since it + // was added and deleted before the catch up completes. We later verify that + // the follower correctly deleted this file. + key = Slice("k3"); + ASSERT_OK(dbfull()->TEST_CompactRange(1, &key, &key, nullptr, true)); + follower_fs()->BarrierWait(); + + SyncPoint::GetInstance()->LoadDependency({ + {"DBImplFollower::TryCatchupWithLeader:End", + "Follower::WaitForCatchup:1"}, + }); + SyncPoint::GetInstance()->EnableProcessing(); + TEST_SYNC_POINT("Follower::WaitForCatchup:1"); + CheckDirs(); + ASSERT_EQ(FollowerGet("k1"), "v2"); + ASSERT_EQ(FollowerGet("k2"), "v1"); + ASSERT_EQ(FollowerGet("k3"), "v2"); + SyncPoint::GetInstance()->DisableProcessing(); +} + +// This test verifies a scenario similar to the PartialVersionRecovery, except +// with a MANIFEST rollover in between. When there is a rollover, the +// follower's attempt ends without installing a new Version. The next catch up +// attempt will recover a full Version. +TEST_F(DBFollowerTest, PartialVersionRecoveryWithRollover) { + Options opts = CurrentOptions(); + opts.disable_auto_compactions = true; + opts.sst_partitioner_factory = + std::make_shared(1); + Reopen(opts); + + ASSERT_OK(Put("k1", "v1")); + ASSERT_OK(Put("k2", "v1")); + ASSERT_OK(Put("k3", "v1")); + ASSERT_OK(Flush()); + MoveFilesToLevel(2); + + ASSERT_OK(Put("k1", "v2")); + ASSERT_OK(Flush()); + ASSERT_OK(Put("k3", "v2")); + ASSERT_OK(Flush()); + MoveFilesToLevel(1); + + opts.max_compaction_bytes = 1; + Reopen(opts); + + ASSERT_OK(OpenAsFollower()); + + follower_fs()->BarrierInit(2); + Slice key("k1"); + ASSERT_OK(dbfull()->TEST_CompactRange(1, &key, &key, nullptr, true)); + + follower_fs()->BarrierWaitAndReinit(2); + Reopen(opts); + key = Slice("k3"); + ASSERT_OK(dbfull()->TEST_CompactRange(1, &key, &key, nullptr, true)); + follower_fs()->BarrierWait(); + + SyncPoint::GetInstance()->LoadDependency({ + {"DBImplFollower::TryCatchupWithLeader:Begin1", + "Follower::WaitForCatchup:1"}, + {"Follower::WaitForCatchup:2", + "DBImplFollower::TryCatchupWithLeader:Begin2"}, + }); + SyncPoint::GetInstance()->EnableProcessing(); + TEST_SYNC_POINT("Follower::WaitForCatchup:1"); + TEST_SYNC_POINT("Follower::WaitForCatchup:2"); + SyncPoint::GetInstance()->LoadDependency({ + {"DBImplFollower::TryCatchupWithLeader:End", + "Follower::WaitForCatchup:3"}, + }); + TEST_SYNC_POINT("Follower::WaitForCatchup:3"); + CheckDirs(); + ASSERT_EQ(FollowerGet("k1"), "v2"); + ASSERT_EQ(FollowerGet("k2"), "v1"); + ASSERT_EQ(FollowerGet("k3"), "v2"); + SyncPoint::GetInstance()->DisableProcessing(); +} #endif } // namespace ROCKSDB_NAMESPACE diff --git a/db/db_impl/db_impl.cc b/db/db_impl/db_impl.cc index 12b0afc2bb7..d338a584220 100644 --- a/db/db_impl/db_impl.cc +++ b/db/db_impl/db_impl.cc @@ -626,6 +626,19 @@ Status DBImpl::CloseHelper() { job_context.Clean(); mutex_.Lock(); } + if (!mutable_db_options_.avoid_sync_during_shutdown && !logs_.empty()) { + mutex_.Unlock(); + Status s = FlushWAL(true /* sync */); + mutex_.Lock(); + if (!s.ok()) { + ROCKS_LOG_WARN(immutable_db_options_.info_log, + "Unable to flush and sync WALs with error -- %s", + s.ToString().c_str()); + if (ret.ok()) { + ret = s; + } + } + } { InstrumentedMutexLock lock(&log_write_mutex_); for (auto l : logs_to_free_) { @@ -637,7 +650,7 @@ Status DBImpl::CloseHelper() { if (!s.ok()) { ROCKS_LOG_WARN( immutable_db_options_.info_log, - "Unable to Sync WAL file %s with error -- %s", + "Unable to clear writer for WAL %s with error -- %s", LogFileName(immutable_db_options_.GetWalDir(), log_number).c_str(), s.ToString().c_str()); // Retain the first error @@ -1551,62 +1564,104 @@ bool DBImpl::WALBufferIsEmpty() { Status DBImpl::SyncWAL() { TEST_SYNC_POINT("DBImpl::SyncWAL:Begin"); - autovector logs_to_sync; - bool need_log_dir_sync; - uint64_t current_log_number; + WriteOptions write_options; + VersionEdit synced_wals; + Status s = SyncWalImpl(/*include_current_wal=*/true, write_options, + /*job_context=*/nullptr, &synced_wals, + /*error_recovery_in_prog=*/false); + + if (s.ok() && synced_wals.IsWalAddition()) { + InstrumentedMutexLock l(&mutex_); + // TODO: plumb Env::IOActivity, Env::IOPriority + const ReadOptions read_options; + s = ApplyWALToManifest(read_options, write_options, &synced_wals); + } + + TEST_SYNC_POINT("DBImpl::SyncWAL:BeforeMarkLogsSynced:2"); + return s; +} + +IOStatus DBImpl::SyncWalImpl(bool include_current_wal, + const WriteOptions& write_options, + JobContext* job_context, VersionEdit* synced_wals, + bool error_recovery_in_prog) { + autovector wals_to_sync; + bool need_wal_dir_sync; + // Number of a WAL that was active at the start of call and maybe is by + // the end of the call. + uint64_t maybe_active_number; + // Sync WALs up to this number + uint64_t up_to_number; { InstrumentedMutexLock l(&log_write_mutex_); assert(!logs_.empty()); - // This SyncWAL() call only cares about logs up to this number. - current_log_number = logfile_number_; + maybe_active_number = logfile_number_; + up_to_number = + include_current_wal ? maybe_active_number : maybe_active_number - 1; - while (logs_.front().number <= current_log_number && - logs_.front().IsSyncing()) { + while (logs_.front().number <= up_to_number && logs_.front().IsSyncing()) { log_sync_cv_.Wait(); } // First check that logs are safe to sync in background. - for (auto it = logs_.begin(); - it != logs_.end() && it->number <= current_log_number; ++it) { - if (!it->writer->file()->writable_file()->IsSyncThreadSafe()) { - return Status::NotSupported( - "SyncWAL() is not supported for this implementation of WAL file", - immutable_db_options_.allow_mmap_writes - ? "try setting Options::allow_mmap_writes to false" - : Slice()); - } + if (include_current_wal && + !logs_.back().writer->file()->writable_file()->IsSyncThreadSafe()) { + return IOStatus::NotSupported( + "SyncWAL() is not supported for this implementation of WAL file", + immutable_db_options_.allow_mmap_writes + ? "try setting Options::allow_mmap_writes to false" + : Slice()); } for (auto it = logs_.begin(); - it != logs_.end() && it->number <= current_log_number; ++it) { + it != logs_.end() && it->number <= up_to_number; ++it) { auto& log = *it; log.PrepareForSync(); - logs_to_sync.push_back(log.writer); + wals_to_sync.push_back(log.writer); } - need_log_dir_sync = !log_dir_synced_; + need_wal_dir_sync = !log_dir_synced_; } - TEST_SYNC_POINT("DBWALTest::SyncWALNotWaitWrite:1"); + if (include_current_wal) { + TEST_SYNC_POINT("DBWALTest::SyncWALNotWaitWrite:1"); + } RecordTick(stats_, WAL_FILE_SYNCED); - Status status; - IOStatus io_s; - // TODO: plumb Env::IOActivity, Env::IOPriority - const ReadOptions read_options; - const WriteOptions write_options; IOOptions opts; - io_s = WritableFileWriter::PrepareIOOptions(write_options, opts); - if (!io_s.ok()) { - status = io_s; - } + IOStatus io_s = WritableFileWriter::PrepareIOOptions(write_options, opts); if (io_s.ok()) { - for (log::Writer* log : logs_to_sync) { - io_s = - log->file()->SyncWithoutFlush(opts, immutable_db_options_.use_fsync); + for (log::Writer* log : wals_to_sync) { + if (job_context) { + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "[JOB %d] Syncing log #%" PRIu64, job_context->job_id, + log->get_log_number()); + } + if (error_recovery_in_prog) { + log->file()->reset_seen_error(); + } + if (log->get_log_number() >= maybe_active_number) { + assert(log->get_log_number() == maybe_active_number); + io_s = log->file()->SyncWithoutFlush(opts, + immutable_db_options_.use_fsync); + } else { + io_s = log->file()->Sync(opts, immutable_db_options_.use_fsync); + } if (!io_s.ok()) { - status = io_s; break; } + // Normally the log file is closed when purging obsolete file, but if + // log recycling is enabled, the log file is closed here so that it + // can be reused. + if (log->get_log_number() < maybe_active_number && + immutable_db_options_.recycle_log_file_num > 0) { + if (error_recovery_in_prog) { + log->file()->reset_seen_error(); + } + io_s = log->Close(write_options); + if (!io_s.ok()) { + break; + } + } } } if (!io_s.ok()) { @@ -1616,31 +1671,28 @@ Status DBImpl::SyncWAL() { // future writes IOStatusCheck(io_s); } - if (status.ok() && need_log_dir_sync) { - status = directories_.GetWalDir()->FsyncWithDirOptions( + if (io_s.ok() && need_wal_dir_sync) { + io_s = directories_.GetWalDir()->FsyncWithDirOptions( IOOptions(), nullptr, DirFsyncOptions(DirFsyncOptions::FsyncReason::kNewFileSynced)); } - TEST_SYNC_POINT("DBWALTest::SyncWALNotWaitWrite:2"); + if (include_current_wal) { + TEST_SYNC_POINT("DBWALTest::SyncWALNotWaitWrite:2"); - TEST_SYNC_POINT("DBImpl::SyncWAL:BeforeMarkLogsSynced:1"); - VersionEdit synced_wals; + TEST_SYNC_POINT("DBImpl::SyncWAL:BeforeMarkLogsSynced:1"); + } else { + TEST_SYNC_POINT_CALLBACK("DBImpl::SyncClosedWals:BeforeReLock", + /*arg=*/nullptr); + } { InstrumentedMutexLock l(&log_write_mutex_); - if (status.ok()) { - MarkLogsSynced(current_log_number, need_log_dir_sync, &synced_wals); + if (io_s.ok()) { + MarkLogsSynced(up_to_number, need_wal_dir_sync, synced_wals); } else { - MarkLogsNotSynced(current_log_number); + MarkLogsNotSynced(up_to_number); } } - if (status.ok() && synced_wals.IsWalAddition()) { - InstrumentedMutexLock l(&mutex_); - status = ApplyWALToManifest(read_options, write_options, &synced_wals); - } - - TEST_SYNC_POINT("DBImpl::SyncWAL:BeforeMarkLogsSynced:2"); - - return status; + return io_s; } Status DBImpl::ApplyWALToManifest(const ReadOptions& read_options, @@ -2099,7 +2151,7 @@ Status DBImpl::GetEntity(const ReadOptions& _read_options, if (_read_options.io_activity != Env::IOActivity::kUnknown && _read_options.io_activity != Env::IOActivity::kGetEntity) { return Status::InvalidArgument( - "Cannot call GetEntity with `ReadOptions::io_activity` != " + "Can only call GetEntity with `ReadOptions::io_activity` set to " "`Env::IOActivity::kUnknown` or `Env::IOActivity::kGetEntity`"); } ReadOptions read_options(_read_options); @@ -2126,7 +2178,7 @@ Status DBImpl::GetEntity(const ReadOptions& _read_options, const Slice& key, if (_read_options.io_activity != Env::IOActivity::kUnknown && _read_options.io_activity != Env::IOActivity::kGetEntity) { s = Status::InvalidArgument( - "Cannot call GetEntity with `ReadOptions::io_activity` != " + "Can only call GetEntity with `ReadOptions::io_activity` set to " "`Env::IOActivity::kUnknown` or `Env::IOActivity::kGetEntity`"); for (size_t i = 0; i < num_column_families; ++i) { (*result)[i].SetStatus(s); @@ -3185,22 +3237,55 @@ void DBImpl::MultiGetEntity(const ReadOptions& _read_options, size_t num_keys, ColumnFamilyHandle** column_families, const Slice* keys, PinnableWideColumns* results, Status* statuses, bool sorted_input) { + assert(statuses); + + if (!column_families) { + const Status s = Status::InvalidArgument( + "Cannot call MultiGetEntity without column families"); + for (size_t i = 0; i < num_keys; ++i) { + statuses[i] = s; + } + + return; + } + + if (!keys) { + const Status s = + Status::InvalidArgument("Cannot call MultiGetEntity without keys"); + for (size_t i = 0; i < num_keys; ++i) { + statuses[i] = s; + } + + return; + } + + if (!results) { + const Status s = Status::InvalidArgument( + "Cannot call MultiGetEntity without PinnableWideColumns objects"); + for (size_t i = 0; i < num_keys; ++i) { + statuses[i] = s; + } + + return; + } + if (_read_options.io_activity != Env::IOActivity::kUnknown && _read_options.io_activity != Env::IOActivity::kMultiGetEntity) { - Status s = Status::InvalidArgument( - "Can only call MultiGetEntity with `ReadOptions::io_activity` is " + const Status s = Status::InvalidArgument( + "Can only call MultiGetEntity with `ReadOptions::io_activity` set to " "`Env::IOActivity::kUnknown` or `Env::IOActivity::kMultiGetEntity`"); for (size_t i = 0; i < num_keys; ++i) { - if (statuses[i].ok()) { - statuses[i] = s; - } + statuses[i] = s; } + return; } + ReadOptions read_options(_read_options); if (read_options.io_activity == Env::IOActivity::kUnknown) { read_options.io_activity = Env::IOActivity::kMultiGetEntity; } + MultiGetCommon(read_options, num_keys, column_families, keys, /* values */ nullptr, results, /* timestamps */ nullptr, statuses, sorted_input); @@ -3210,22 +3295,54 @@ void DBImpl::MultiGetEntity(const ReadOptions& _read_options, ColumnFamilyHandle* column_family, size_t num_keys, const Slice* keys, PinnableWideColumns* results, Status* statuses, bool sorted_input) { + assert(statuses); + + if (!column_family) { + const Status s = Status::InvalidArgument( + "Cannot call MultiGetEntity without a column family handle"); + for (size_t i = 0; i < num_keys; ++i) { + statuses[i] = s; + } + + return; + } + + if (!keys) { + const Status s = + Status::InvalidArgument("Cannot call MultiGetEntity without keys"); + for (size_t i = 0; i < num_keys; ++i) { + statuses[i] = s; + } + + return; + } + + if (!results) { + const Status s = Status::InvalidArgument( + "Cannot call MultiGetEntity without PinnableWideColumns objects"); + for (size_t i = 0; i < num_keys; ++i) { + statuses[i] = s; + } + + return; + } + if (_read_options.io_activity != Env::IOActivity::kUnknown && _read_options.io_activity != Env::IOActivity::kMultiGetEntity) { - Status s = Status::InvalidArgument( - "Can only call MultiGetEntity with `ReadOptions::io_activity` is " + const Status s = Status::InvalidArgument( + "Can only call MultiGetEntity with `ReadOptions::io_activity` set to " "`Env::IOActivity::kUnknown` or `Env::IOActivity::kMultiGetEntity`"); for (size_t i = 0; i < num_keys; ++i) { - if (statuses[i].ok()) { - statuses[i] = s; - } + statuses[i] = s; } return; } + ReadOptions read_options(_read_options); if (read_options.io_activity == Env::IOActivity::kUnknown) { read_options.io_activity = Env::IOActivity::kMultiGetEntity; } + MultiGetCommon(read_options, column_family, num_keys, keys, /* values */ nullptr, results, /* timestamps */ nullptr, statuses, sorted_input); @@ -3234,18 +3351,34 @@ void DBImpl::MultiGetEntity(const ReadOptions& _read_options, void DBImpl::MultiGetEntity(const ReadOptions& _read_options, size_t num_keys, const Slice* keys, PinnableAttributeGroups* results) { + assert(results); + + if (!keys) { + const Status s = + Status::InvalidArgument("Cannot call MultiGetEntity without keys"); + for (size_t i = 0; i < num_keys; ++i) { + for (size_t j = 0; j < results[i].size(); ++j) { + results[i][j].SetStatus(s); + } + } + + return; + } + if (_read_options.io_activity != Env::IOActivity::kUnknown && _read_options.io_activity != Env::IOActivity::kMultiGetEntity) { - Status s = Status::InvalidArgument( - "Can only call MultiGetEntity with ReadOptions::io_activity` is " + const Status s = Status::InvalidArgument( + "Can only call MultiGetEntity with `ReadOptions::io_activity` set to " "`Env::IOActivity::kUnknown` or `Env::IOActivity::kMultiGetEntity`"); for (size_t i = 0; i < num_keys; ++i) { for (size_t j = 0; j < results[i].size(); ++j) { results[i][j].SetStatus(s); } } + return; } + ReadOptions read_options(_read_options); if (read_options.io_activity == Env::IOActivity::kUnknown) { read_options.io_activity = Env::IOActivity::kMultiGetEntity; @@ -3263,6 +3396,7 @@ void DBImpl::MultiGetEntity(const ReadOptions& _read_options, size_t num_keys, ++total_count; } } + std::vector statuses(total_count); std::vector columns(total_count); MultiGetCommon(read_options, total_count, column_families.data(), @@ -3283,6 +3417,15 @@ void DBImpl::MultiGetEntity(const ReadOptions& _read_options, size_t num_keys, } } +void DBImpl::MultiGetEntityWithCallback( + const ReadOptions& read_options, ColumnFamilyHandle* column_family, + ReadCallback* callback, + autovector* sorted_keys) { + assert(read_options.io_activity == Env::IOActivity::kMultiGetEntity); + + MultiGetWithCallbackImpl(read_options, column_family, callback, sorted_keys); +} + Status DBImpl::WrapUpCreateColumnFamilies( const ReadOptions& read_options, const WriteOptions& write_options, const std::vector& cf_options) { diff --git a/db/db_impl/db_impl.h b/db/db_impl/db_impl.h index f4a95b52b85..58751a339ac 100644 --- a/db/db_impl/db_impl.h +++ b/db/db_impl/db_impl.h @@ -57,6 +57,7 @@ #include "rocksdb/status.h" #include "rocksdb/trace_reader_writer.h" #include "rocksdb/transaction_log.h" +#include "rocksdb/user_write_callback.h" #include "rocksdb/utilities/replayer.h" #include "rocksdb/write_buffer_manager.h" #include "table/merging_iterator.h" @@ -231,6 +232,10 @@ class DBImpl : public DB { using DB::Write; Status Write(const WriteOptions& options, WriteBatch* updates) override; + using DB::WriteWithCallback; + Status WriteWithCallback(const WriteOptions& options, WriteBatch* updates, + UserWriteCallback* user_write_cb) override; + using DB::Get; Status Get(const ReadOptions& _read_options, ColumnFamilyHandle* column_family, const Slice& key, @@ -291,6 +296,11 @@ class DBImpl : public DB { const Slice* keys, PinnableAttributeGroups* results) override; + void MultiGetEntityWithCallback( + const ReadOptions& read_options, ColumnFamilyHandle* column_family, + ReadCallback* callback, + autovector* sorted_keys); + Status CreateColumnFamily(const ColumnFamilyOptions& cf_options, const std::string& column_family, ColumnFamilyHandle** handle) override { @@ -496,8 +506,8 @@ class DBImpl : public DB { // All the returned filenames start with "/" Status GetLiveFiles(std::vector&, uint64_t* manifest_file_size, bool flush_memtable = true) override; - Status GetSortedWalFiles(VectorLogPtr& files) override; - Status GetCurrentWalFile(std::unique_ptr* current_log_file) override; + Status GetSortedWalFiles(VectorWalPtr& files) override; + Status GetCurrentWalFile(std::unique_ptr* current_log_file) override; Status GetCreationTimeOfOldestFile(uint64_t* creation_time) override; Status GetUpdatesSince( @@ -683,7 +693,8 @@ class DBImpl : public DB { // thread to determine whether it is safe to perform the write. virtual Status WriteWithCallback(const WriteOptions& write_options, WriteBatch* my_batch, - WriteCallback* callback); + WriteCallback* callback, + UserWriteCallback* user_write_cb = nullptr); // Returns the sequence number that is guaranteed to be smaller than or equal // to the sequence number of any key that could be inserted into the current @@ -1492,6 +1503,7 @@ class DBImpl : public DB { // batch that does not have duplicate keys. Status WriteImpl(const WriteOptions& options, WriteBatch* updates, WriteCallback* callback = nullptr, + UserWriteCallback* user_write_cb = nullptr, uint64_t* log_used = nullptr, uint64_t log_ref = 0, bool disable_memtable = false, uint64_t* seq_used = nullptr, size_t batch_cnt = 0, @@ -1500,6 +1512,7 @@ class DBImpl : public DB { Status PipelinedWriteImpl(const WriteOptions& options, WriteBatch* updates, WriteCallback* callback = nullptr, + UserWriteCallback* user_write_cb = nullptr, uint64_t* log_used = nullptr, uint64_t log_ref = 0, bool disable_memtable = false, uint64_t* seq_used = nullptr); @@ -1526,7 +1539,8 @@ class DBImpl : public DB { // marks start of a new sub-batch. Status WriteImplWALOnly( WriteThread* write_thread, const WriteOptions& options, - WriteBatch* updates, WriteCallback* callback, uint64_t* log_used, + WriteBatch* updates, WriteCallback* callback, + UserWriteCallback* user_write_cb, uint64_t* log_used, const uint64_t log_ref, uint64_t* seq_used, const size_t sub_batch_cnt, PreReleaseCallback* pre_release_callback, const AssignOrder assign_order, const PublishLastSeq publish_last_seq, const bool disable_memtable); @@ -1643,6 +1657,7 @@ class DBImpl : public DB { friend class ForwardIterator; friend struct SuperVersion; friend class CompactedDBImpl; + friend class DBImplFollower; #ifndef NDEBUG friend class DBTest_ConcurrentFlushWAL_Test; friend class DBTest_MixedSlowdownOptionsStop_Test; @@ -1914,7 +1929,7 @@ class DBImpl : public DB { void ReleaseFileNumberFromPendingOutputs( std::unique_ptr::iterator>& v); - IOStatus SyncClosedLogs(const WriteOptions& write_options, + IOStatus SyncClosedWals(const WriteOptions& write_options, JobContext* job_context, VersionEdit* synced_wals, bool error_recovery_in_prog); @@ -2048,17 +2063,22 @@ class DBImpl : public DB { mutex_.Lock(); } - if (!immutable_db_options_.unordered_write) { - // Then the writes are finished before the next write group starts - return; + if (immutable_db_options_.unordered_write) { + // Wait for the ones who already wrote to the WAL to finish their + // memtable write. + if (pending_memtable_writes_.load() != 0) { + // XXX: suspicious wait while holding DB mutex? + std::unique_lock guard(switch_mutex_); + switch_cv_.wait(guard, + [&] { return pending_memtable_writes_.load() == 0; }); + } + } else { + // (Writes are finished before the next write group starts.) } - // Wait for the ones who already wrote to the WAL to finish their - // memtable write. - if (pending_memtable_writes_.load() != 0) { - std::unique_lock guard(switch_mutex_); - switch_cv_.wait(guard, - [&] { return pending_memtable_writes_.load() == 0; }); + // Wait for any LockWAL to clear + while (lock_wal_count_ > 0) { + bg_cv_.Wait(); } } @@ -2244,6 +2264,11 @@ class DBImpl : public DB { ColumnFamilyData* PickCompactionFromQueue( std::unique_ptr* token, LogBuffer* log_buffer); + IOStatus SyncWalImpl(bool include_current_wal, + const WriteOptions& write_options, + JobContext* job_context, VersionEdit* synced_wals, + bool error_recovery_in_prog); + // helper function to call after some of the logs_ were synced void MarkLogsSynced(uint64_t up_to, bool synced_dir, VersionEdit* edit); Status ApplyWALToManifest(const ReadOptions& read_options, @@ -2568,7 +2593,7 @@ class DBImpl : public DB { // 8. read by MarkLogsNotSynced() and MarkLogsSynced() are protected by // log_write_mutex_. // 9. erase() by MarkLogsSynced() protected by log_write_mutex_. - // 10. read by SyncClosedLogs() protected by only log_write_mutex_. This can + // 10. read by SyncClosedWals() protected by only log_write_mutex_. This can // happen in bg flush threads after DB::Open() returns success to // applications. // 11. reads, e.g. front(), iteration, and back() called by PreprocessWrite() @@ -2581,7 +2606,7 @@ class DBImpl : public DB { // 13. emplace_back() by SwitchMemtable() hold both mutex_ and // log_write_mutex_. This happens in the write group leader. Can conflict // with bg threads calling FindObsoleteFiles(), MarkLogsSynced(), - // SyncClosedLogs(), etc. as well as application threads calling + // SyncClosedWals(), etc. as well as application threads calling // FlushWAL(), SyncWAL(), LockWAL(). This is fine because all parties // require at least log_write_mutex_. // 14. iteration called in WriteToWAL(write_group) protected by @@ -2987,9 +3012,9 @@ inline Status DBImpl::FailIfReadCollapsedHistory(const ColumnFamilyData* cfd, if (!full_history_ts_low.empty() && ucmp->CompareTimestamp(ts, full_history_ts_low) < 0) { std::stringstream oss; - oss << "Read timestamp: " << ts.ToString(true) + oss << "Read timestamp: " << ucmp->TimestampToString(ts) << " is smaller than full_history_ts_low: " - << Slice(full_history_ts_low).ToString(true) << std::endl; + << ucmp->TimestampToString(full_history_ts_low) << std::endl; return Status::InvalidArgument(oss.str()); } return Status::OK(); diff --git a/db/db_impl/db_impl_compaction_flush.cc b/db/db_impl/db_impl_compaction_flush.cc index c3e29c200b0..3eef4c5fdfb 100644 --- a/db/db_impl/db_impl_compaction_flush.cc +++ b/db/db_impl/db_impl_compaction_flush.cc @@ -116,89 +116,19 @@ bool DBImpl::ShouldRescheduleFlushRequestToRetainUDT( return true; } -IOStatus DBImpl::SyncClosedLogs(const WriteOptions& write_options, +IOStatus DBImpl::SyncClosedWals(const WriteOptions& write_options, JobContext* job_context, VersionEdit* synced_wals, bool error_recovery_in_prog) { - TEST_SYNC_POINT("DBImpl::SyncClosedLogs:Start"); - InstrumentedMutexLock l(&log_write_mutex_); - autovector logs_to_sync; - uint64_t current_log_number = logfile_number_; - while (logs_.front().number < current_log_number && - logs_.front().IsSyncing()) { - log_sync_cv_.Wait(); - } - for (auto it = logs_.begin(); - it != logs_.end() && it->number < current_log_number; ++it) { - auto& log = *it; - log.PrepareForSync(); - logs_to_sync.push_back(log.writer); - } - - IOStatus io_s; - if (!logs_to_sync.empty()) { - log_write_mutex_.Unlock(); - - assert(job_context); - - for (log::Writer* log : logs_to_sync) { - ROCKS_LOG_INFO(immutable_db_options_.info_log, - "[JOB %d] Syncing log #%" PRIu64, job_context->job_id, - log->get_log_number()); - if (error_recovery_in_prog) { - log->file()->reset_seen_error(); - } - - IOOptions io_options; - io_s = WritableFileWriter::PrepareIOOptions(write_options, io_options); - if (!io_s.ok()) { - break; - } - io_s = log->file()->Sync(io_options, immutable_db_options_.use_fsync); - if (!io_s.ok()) { - break; - } - - if (immutable_db_options_.recycle_log_file_num > 0) { - if (error_recovery_in_prog) { - log->file()->reset_seen_error(); - } - // Normally the log file is closed when purging obsolete file, but if - // log recycling is enabled, the log file is closed here so that it - // can be reused. - io_s = log->Close(write_options); - if (!io_s.ok()) { - break; - } - } - } - if (io_s.ok()) { - IOOptions io_options; - io_s = WritableFileWriter::PrepareIOOptions(write_options, io_options); - if (io_s.ok()) { - io_s = directories_.GetWalDir()->FsyncWithDirOptions( - io_options, nullptr, - DirFsyncOptions(DirFsyncOptions::FsyncReason::kNewFileSynced)); - } - } - - TEST_SYNC_POINT_CALLBACK("DBImpl::SyncClosedLogs:BeforeReLock", - /*arg=*/nullptr); - log_write_mutex_.Lock(); + TEST_SYNC_POINT("DBImpl::SyncClosedWals:Start"); - // "number <= current_log_number - 1" is equivalent to - // "number < current_log_number". - if (io_s.ok()) { - MarkLogsSynced(current_log_number - 1, true, synced_wals); - } else { - MarkLogsNotSynced(current_log_number - 1); - } - if (!io_s.ok()) { - TEST_SYNC_POINT("DBImpl::SyncClosedLogs:Failed"); - return io_s; - } + IOStatus io_s = SyncWalImpl(/*include_current_wal*/ false, write_options, + job_context, synced_wals, error_recovery_in_prog); + if (!io_s.ok()) { + TEST_SYNC_POINT("DBImpl::SyncClosedWals:Failed"); + } else { + TEST_SYNC_POINT("DBImpl::SyncClosedWals:end"); } - TEST_SYNC_POINT("DBImpl::SyncClosedLogs:end"); return io_s; } @@ -224,19 +154,25 @@ Status DBImpl::FlushMemTableToOutputFile( // the host crashes after flushing and before WAL is persistent, the // flushed SST may contain data from write batches whose updates to // other (unflushed) column families are missing. + // + // When 2PC is enabled, non-recent WAL(s) may be needed for crash-recovery, + // even when there is only one CF in the DB, for prepared transactions that + // had not been committed yet. Make sure we sync them to keep the persisted + // WAL state at least as new as the persisted SST state. const bool needs_to_sync_closed_wals = logfile_number_ > 0 && - versions_->GetColumnFamilySet()->NumberOfColumnFamilies() > 1; + (versions_->GetColumnFamilySet()->NumberOfColumnFamilies() > 1 || + allow_2pc()); // If needs_to_sync_closed_wals is true, we need to record the current // maximum memtable ID of this column family so that a later PickMemtables() // call will not pick memtables whose IDs are higher. This is due to the fact - // that SyncClosedLogs() may release the db mutex, and memtable switch can + // that SyncClosedWals() may release the db mutex, and memtable switch can // happen for this column family in the meantime. The newly created memtables // have their data backed by unsynced WALs, thus they cannot be included in // this flush job. // Another reason why we must record the current maximum memtable ID of this - // column family: SyncClosedLogs() may release db mutex, thus it's possible + // column family: SyncClosedWals() may release db mutex, thus it's possible // for application to continue to insert into memtables increasing db's // sequence number. The application may take a snapshot, but this snapshot is // not included in `snapshot_seqs` which will be passed to flush job because @@ -250,7 +186,7 @@ Status DBImpl::FlushMemTableToOutputFile( // If needs_to_sync_closed_wals is false, then the flush job will pick ALL // existing memtables of the column family when PickMemTable() is called - // later. Although we won't call SyncClosedLogs() in this case, we may still + // later. Although we won't call SyncClosedWals() in this case, we may still // call the callbacks of the listeners, i.e. NotifyOnFlushBegin() which also // releases and re-acquires the db mutex. In the meantime, the application // can still insert into the memtables and increase the db's sequence number. @@ -280,12 +216,12 @@ Status DBImpl::FlushMemTableToOutputFile( bool need_cancel = false; IOStatus log_io_s = IOStatus::OK(); if (needs_to_sync_closed_wals) { - // SyncClosedLogs() may unlock and re-lock the log_write_mutex multiple + // SyncClosedWals() may unlock and re-lock the log_write_mutex multiple // times. VersionEdit synced_wals; bool error_recovery_in_prog = error_handler_.IsRecoveryInProgress(); mutex_.Unlock(); - log_io_s = SyncClosedLogs(write_options, job_context, &synced_wals, + log_io_s = SyncClosedWals(write_options, job_context, &synced_wals, error_recovery_in_prog); mutex_.Lock(); if (log_io_s.ok() && synced_wals.IsWalAddition()) { @@ -300,7 +236,7 @@ Status DBImpl::FlushMemTableToOutputFile( error_handler_.SetBGError(log_io_s, BackgroundErrorReason::kFlush); } } else { - TEST_SYNC_POINT("DBImpl::SyncClosedLogs:Skip"); + TEST_SYNC_POINT("DBImpl::SyncClosedWals:Skip"); } s = log_io_s; @@ -574,7 +510,7 @@ Status DBImpl::AtomicFlushMemTablesToOutputFiles( VersionEdit synced_wals; bool error_recovery_in_prog = error_handler_.IsRecoveryInProgress(); mutex_.Unlock(); - log_io_s = SyncClosedLogs(write_options, job_context, &synced_wals, + log_io_s = SyncClosedWals(write_options, job_context, &synced_wals, error_recovery_in_prog); mutex_.Lock(); if (log_io_s.ok() && synced_wals.IsWalAddition()) { @@ -1088,7 +1024,12 @@ Status DBImpl::IncreaseFullHistoryTsLowImpl(ColumnFamilyData* cfd, assert(ucmp->timestamp_size() == ts_low.size() && !ts_low.empty()); if (!current_ts_low.empty() && ucmp->CompareTimestamp(ts_low, current_ts_low) < 0) { - return Status::InvalidArgument("Cannot decrease full_history_ts_low"); + std::stringstream oss; + oss << "Current full_history_ts_low: " + << ucmp->TimestampToString(current_ts_low) + << " is higher than provided ts: " << ucmp->TimestampToString(ts_low) + << std::endl; + return Status::InvalidArgument(oss.str()); } Status s = versions_->LogAndApply(cfd, *cfd->GetLatestMutableCFOptions(), @@ -1120,6 +1061,11 @@ Status DBImpl::CompactRangeInternal(const CompactRangeOptions& options, if (options.target_path_id >= cfd->ioptions()->cf_paths.size()) { return Status::InvalidArgument("Invalid target path ID"); } + if (options.change_level && + cfd->ioptions()->compaction_style == kCompactionStyleFIFO) { + return Status::NotSupported( + "FIFO compaction does not support change_level."); + } bool flush_needed = true; @@ -1180,6 +1126,16 @@ Status DBImpl::CompactRangeInternal(const CompactRangeOptions& options, final_output_level, options, begin, end, exclusive, false /* disable_trivial_move */, std::numeric_limits::max(), trim_ts); + } else if (cfd->ioptions()->compaction_style == kCompactionStyleFIFO) { + // FIFOCompactionPicker::CompactRange() will ignore the input key range + // [begin, end] and just try to pick compaction based on the configured + // option `compaction_options_fifo`. So we skip checking if [begin, end] + // overlaps with the DB here. + final_output_level = 0; + s = RunManualCompaction(cfd, /*input_level=*/0, final_output_level, options, + begin, end, exclusive, + false /* disable_trivial_move */, + std::numeric_limits::max(), trim_ts); } else { int first_overlapped_level = kInvalidLevel; { @@ -1264,8 +1220,7 @@ Status DBImpl::CompactRangeInternal(const CompactRangeOptions& options, CleanupSuperVersion(super_version); } if (s.ok() && first_overlapped_level != kInvalidLevel) { - if (cfd->ioptions()->compaction_style == kCompactionStyleUniversal || - cfd->ioptions()->compaction_style == kCompactionStyleFIFO) { + if (cfd->ioptions()->compaction_style == kCompactionStyleUniversal) { assert(first_overlapped_level == 0); s = RunManualCompaction( cfd, first_overlapped_level, first_overlapped_level, options, begin, @@ -1517,16 +1472,11 @@ Status DBImpl::CompactFilesImpl( std::to_string(cfd->ioptions()->num_levels - 1)); } - Status s = cfd->compaction_picker()->SanitizeCompactionInputFiles( - &input_set, cf_meta, output_level); - TEST_SYNC_POINT("DBImpl::CompactFilesImpl::PostSanitizeCompactionInputFiles"); - if (!s.ok()) { - return s; - } - std::vector input_files; - s = cfd->compaction_picker()->GetCompactionInputsFromFileNumbers( - &input_files, &input_set, version->storage_info(), compact_options); + Status s = cfd->compaction_picker()->SanitizeAndConvertCompactionInputFiles( + &input_set, cf_meta, output_level, version->storage_info(), &input_files); + TEST_SYNC_POINT( + "DBImpl::CompactFilesImpl::PostSanitizeAndConvertCompactionInputFiles"); if (!s.ok()) { return s; } @@ -3340,7 +3290,7 @@ void DBImpl::BackgroundCallFlush(Env::Priority thread_pri) { bg_cv_.SignalAll(); // In case a waiter can proceed despite the error mutex_.Unlock(); ROCKS_LOG_ERROR(immutable_db_options_.info_log, - "[JOB %d] Waiting after background flush error: %s" + "[JOB %d] Waiting after background flush error: %s, " "Accumulated background error counts: %" PRIu64, job_context.job_id, s.ToString().c_str(), error_cnt); log_buffer.FlushBufferToLog(); diff --git a/db/db_impl/db_impl_follower.cc b/db/db_impl/db_impl_follower.cc index 8d21f530cd0..a104a83d6bf 100644 --- a/db/db_impl/db_impl_follower.cc +++ b/db/db_impl/db_impl_follower.cc @@ -5,6 +5,7 @@ #include "db/db_impl/db_impl_follower.h" +#include #include #include "db/arena_wrapped_db_iter.h" @@ -95,17 +96,28 @@ Status DBImplFollower::TryCatchUpWithLeader() { assert(versions_.get() != nullptr); assert(manifest_reader_.get() != nullptr); Status s; + + TEST_SYNC_POINT("DBImplFollower::TryCatchupWithLeader:Begin1"); + TEST_SYNC_POINT("DBImplFollower::TryCatchupWithLeader:Begin2"); // 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_); + std::vector files_to_delete; s = static_cast_with_check(versions_.get()) ->ReadAndApply(&mutex_, &manifest_reader_, - manifest_reader_status_.get(), &cfds_changed); + manifest_reader_status_.get(), &cfds_changed, + &files_to_delete); + ReleaseFileNumberFromPendingOutputs(pending_outputs_inserted_elem_); + pending_outputs_inserted_elem_.reset(new std::list::iterator( + CaptureCurrentFileNumberInPendingOutputs())); ROCKS_LOG_INFO(immutable_db_options_.info_log, "Last sequence is %" PRIu64, static_cast(versions_->LastSequence())); + ROCKS_LOG_INFO( + immutable_db_options_.info_log, "Next file number is %" PRIu64, + static_cast(versions_->current_next_file_number())); for (ColumnFamilyData* cfd : cfds_changed) { if (cfd->IsDropped()) { ROCKS_LOG_DEBUG(immutable_db_options_.info_log, "[%s] is dropped\n", @@ -147,9 +159,33 @@ Status DBImplFollower::TryCatchUpWithLeader() { sv_context.NewSuperVersion(); } } + + for (auto& file : files_to_delete) { + IOStatus io_s = fs_->DeleteFile(file, IOOptions(), nullptr); + if (!io_s.ok()) { + ROCKS_LOG_INFO(immutable_db_options_.info_log, + "Cannot delete file %s: %s", file.c_str(), + io_s.ToString().c_str()); + } + } } job_context.Clean(); + // Cleanup unused, obsolete files. + JobContext purge_files_job_context(0); + { + InstrumentedMutexLock lock_guard(&mutex_); + // Currently, follower instance does not create any database files, thus + // is unnecessary for the follower to force full scan. + FindObsoleteFiles(&purge_files_job_context, /*force=*/false); + } + if (purge_files_job_context.HaveSomethingToDelete()) { + PurgeObsoleteFiles(purge_files_job_context); + } + purge_files_job_context.Clean(); + + TEST_SYNC_POINT("DBImplFollower::TryCatchupWithLeader:End"); + return s; } @@ -199,6 +235,8 @@ Status DBImplFollower::Close() { catch_up_thread_.reset(); } + ReleaseFileNumberFromPendingOutputs(pending_outputs_inserted_elem_); + return DBImpl::Close(); } diff --git a/db/db_impl/db_impl_follower.h b/db/db_impl/db_impl_follower.h index 60992c111e5..374c60d5c64 100644 --- a/db/db_impl/db_impl_follower.h +++ b/db/db_impl/db_impl_follower.h @@ -27,7 +27,7 @@ class DBImplFollower : public DBImplSecondary { bool OwnTablesAndLogs() const override { // TODO: Change this to true once we've properly implemented file // deletion for the read scaling case - return false; + return true; } Status Recover(const std::vector& column_families, @@ -49,5 +49,6 @@ class DBImplFollower : public DBImplSecondary { std::string src_path_; port::Mutex mu_; port::CondVar cv_; + std::unique_ptr::iterator> pending_outputs_inserted_elem_; }; } // namespace ROCKSDB_NAMESPACE diff --git a/db/db_impl/db_impl_secondary.cc b/db/db_impl/db_impl_secondary.cc index a5509c4f121..92944d11818 100644 --- a/db/db_impl/db_impl_secondary.cc +++ b/db/db_impl/db_impl_secondary.cc @@ -680,7 +680,8 @@ Status DBImplSecondary::TryCatchUpWithPrimary() { InstrumentedMutexLock lock_guard(&mutex_); s = static_cast_with_check(versions_.get()) ->ReadAndApply(&mutex_, &manifest_reader_, - manifest_reader_status_.get(), &cfds_changed); + manifest_reader_status_.get(), &cfds_changed, + /*files_to_delete=*/nullptr); ROCKS_LOG_INFO(immutable_db_options_.info_log, "Last sequence is %" PRIu64, static_cast(versions_->LastSequence())); diff --git a/db/db_impl/db_impl_write.cc b/db/db_impl/db_impl_write.cc index 5f18e01d24e..ed95549b0ac 100644 --- a/db/db_impl/db_impl_write.cc +++ b/db/db_impl/db_impl_write.cc @@ -155,21 +155,36 @@ Status DBImpl::Write(const WriteOptions& write_options, WriteBatch* my_batch) { } if (s.ok()) { s = WriteImpl(write_options, my_batch, /*callback=*/nullptr, + /*user_write_cb=*/nullptr, /*log_used=*/nullptr); } return s; } +Status DBImpl::WriteWithCallback(const WriteOptions& write_options, + WriteBatch* my_batch, WriteCallback* callback, + UserWriteCallback* user_write_cb) { + Status s; + if (write_options.protection_bytes_per_key > 0) { + s = WriteBatchInternal::UpdateProtectionInfo( + my_batch, write_options.protection_bytes_per_key); + } + if (s.ok()) { + s = WriteImpl(write_options, my_batch, callback, user_write_cb); + } + return s; +} + Status DBImpl::WriteWithCallback(const WriteOptions& write_options, WriteBatch* my_batch, - WriteCallback* callback) { + UserWriteCallback* user_write_cb) { Status s; if (write_options.protection_bytes_per_key > 0) { s = WriteBatchInternal::UpdateProtectionInfo( my_batch, write_options.protection_bytes_per_key); } if (s.ok()) { - s = WriteImpl(write_options, my_batch, callback, nullptr); + s = WriteImpl(write_options, my_batch, /*callback=*/nullptr, user_write_cb); } return s; } @@ -179,9 +194,9 @@ Status DBImpl::WriteWithCallback(const WriteOptions& write_options, // published sequence. Status DBImpl::WriteImpl(const WriteOptions& write_options, WriteBatch* my_batch, WriteCallback* callback, - uint64_t* log_used, uint64_t log_ref, - bool disable_memtable, uint64_t* seq_used, - size_t batch_cnt, + UserWriteCallback* user_write_cb, uint64_t* log_used, + uint64_t log_ref, bool disable_memtable, + uint64_t* seq_used, size_t batch_cnt, PreReleaseCallback* pre_release_callback, PostMemTableCallback* post_memtable_callback) { assert(!seq_per_batch_ || batch_cnt != 0); @@ -221,7 +236,11 @@ Status DBImpl::WriteImpl(const WriteOptions& write_options, return Status::InvalidArgument( "`WriteOptions::protection_bytes_per_key` must be zero or eight"); } else if (write_options.disableWAL && - immutable_db_options_.recycle_log_file_num > 0) { + immutable_db_options_.recycle_log_file_num > 0 && + !(two_write_queues_ && disable_memtable)) { + // Corruption detection in recycled WALs relies on sequential sequence + // numbers, but WritePreparedTxnDB uses disableWAL internally for split + // writes return Status::InvalidArgument( "WriteOptions::disableWAL option is not supported if " "DBOptions::recycle_log_file_num > 0"); @@ -264,6 +283,10 @@ Status DBImpl::WriteImpl(const WriteOptions& write_options, return Status::NotSupported( "seq_per_batch currently does not honor post_memtable_callback"); } + if (my_batch->HasDeleteRange() && immutable_db_options_.row_cache) { + return Status::NotSupported( + "DeleteRange is not compatible with row cache."); + } // Otherwise IsLatestPersistentState optimization does not make sense assert(!WriteBatchInternal::IsLatestPersistentState(my_batch) || disable_memtable); @@ -280,10 +303,10 @@ Status DBImpl::WriteImpl(const WriteOptions& write_options, seq_per_batch_ ? kDoAssignOrder : kDontAssignOrder; // Otherwise it is WAL-only Prepare batches in WriteCommitted policy and // they don't consume sequence. - return WriteImplWALOnly(&nonmem_write_thread_, write_options, my_batch, - callback, log_used, log_ref, seq_used, batch_cnt, - pre_release_callback, assign_order, - kDontPublishLastSeq, disable_memtable); + return WriteImplWALOnly( + &nonmem_write_thread_, write_options, my_batch, callback, user_write_cb, + log_used, log_ref, seq_used, batch_cnt, pre_release_callback, + assign_order, kDontPublishLastSeq, disable_memtable); } if (immutable_db_options_.unordered_write) { @@ -295,9 +318,9 @@ Status DBImpl::WriteImpl(const WriteOptions& write_options, // Use a write thread to i) optimize for WAL write, ii) publish last // sequence in in increasing order, iii) call pre_release_callback serially Status status = WriteImplWALOnly( - &write_thread_, write_options, my_batch, callback, log_used, log_ref, - &seq, sub_batch_cnt, pre_release_callback, kDoAssignOrder, - kDoPublishLastSeq, disable_memtable); + &write_thread_, write_options, my_batch, callback, user_write_cb, + log_used, log_ref, &seq, sub_batch_cnt, pre_release_callback, + kDoAssignOrder, kDoPublishLastSeq, disable_memtable); TEST_SYNC_POINT("DBImpl::WriteImpl:UnorderedWriteAfterWriteWAL"); if (!status.ok()) { return status; @@ -314,17 +337,20 @@ Status DBImpl::WriteImpl(const WriteOptions& write_options, } if (immutable_db_options_.enable_pipelined_write) { - return PipelinedWriteImpl(write_options, my_batch, callback, log_used, - log_ref, disable_memtable, seq_used); + return PipelinedWriteImpl(write_options, my_batch, callback, user_write_cb, + log_used, log_ref, disable_memtable, seq_used); } PERF_TIMER_GUARD(write_pre_and_post_process_time); - WriteThread::Writer w(write_options, my_batch, callback, log_ref, - disable_memtable, batch_cnt, pre_release_callback, - post_memtable_callback); + WriteThread::Writer w(write_options, my_batch, callback, user_write_cb, + log_ref, disable_memtable, batch_cnt, + pre_release_callback, post_memtable_callback); StopWatch write_sw(immutable_db_options_.clock, stats_, DB_WRITE); write_thread_.JoinBatchGroup(&w); + if (w.state == WriteThread::STATE_PARALLEL_MEMTABLE_CALLER) { + write_thread_.SetMemWritersEachStride(&w); + } if (w.state == WriteThread::STATE_PARALLEL_MEMTABLE_WRITER) { // we are a non-leader in a parallel group @@ -675,6 +701,7 @@ Status DBImpl::WriteImpl(const WriteOptions& write_options, Status DBImpl::PipelinedWriteImpl(const WriteOptions& write_options, WriteBatch* my_batch, WriteCallback* callback, + UserWriteCallback* user_write_cb, uint64_t* log_used, uint64_t log_ref, bool disable_memtable, uint64_t* seq_used) { PERF_TIMER_GUARD(write_pre_and_post_process_time); @@ -682,8 +709,8 @@ Status DBImpl::PipelinedWriteImpl(const WriteOptions& write_options, WriteContext write_context; - WriteThread::Writer w(write_options, my_batch, callback, log_ref, - disable_memtable, /*_batch_cnt=*/0, + WriteThread::Writer w(write_options, my_batch, callback, user_write_cb, + log_ref, disable_memtable, /*_batch_cnt=*/0, /*_pre_release_callback=*/nullptr); write_thread_.JoinBatchGroup(&w); TEST_SYNC_POINT("DBImplWrite::PipelinedWriteImpl:AfterJoinBatchGroup"); @@ -822,7 +849,9 @@ Status DBImpl::PipelinedWriteImpl(const WriteOptions& write_options, // so we need to set its status to pass ASSERT_STATUS_CHECKED memtable_write_group.status.PermitUncheckedError(); } - + if (w.state == WriteThread::STATE_PARALLEL_MEMTABLE_CALLER) { + write_thread_.SetMemWritersEachStride(&w); + } if (w.state == WriteThread::STATE_PARALLEL_MEMTABLE_WRITER) { PERF_TIMER_STOP(write_pre_and_post_process_time); PERF_TIMER_FOR_WAIT_GUARD(write_memtable_time); @@ -862,7 +891,8 @@ Status DBImpl::UnorderedWriteMemtable(const WriteOptions& write_options, PERF_TIMER_GUARD(write_pre_and_post_process_time); StopWatch write_sw(immutable_db_options_.clock, stats_, DB_WRITE); - WriteThread::Writer w(write_options, my_batch, callback, log_ref, + WriteThread::Writer w(write_options, my_batch, callback, + /*user_write_cb=*/nullptr, log_ref, false /*disable_memtable*/); if (w.CheckCallback(this) && w.ShouldWriteToMemtable()) { @@ -912,13 +942,15 @@ Status DBImpl::UnorderedWriteMemtable(const WriteOptions& write_options, // applicable in a two-queue setting. Status DBImpl::WriteImplWALOnly( WriteThread* write_thread, const WriteOptions& write_options, - WriteBatch* my_batch, WriteCallback* callback, uint64_t* log_used, + WriteBatch* my_batch, WriteCallback* callback, + UserWriteCallback* user_write_cb, uint64_t* log_used, const uint64_t log_ref, uint64_t* seq_used, const size_t sub_batch_cnt, PreReleaseCallback* pre_release_callback, const AssignOrder assign_order, const PublishLastSeq publish_last_seq, const bool disable_memtable) { PERF_TIMER_GUARD(write_pre_and_post_process_time); - WriteThread::Writer w(write_options, my_batch, callback, log_ref, - disable_memtable, sub_batch_cnt, pre_release_callback); + WriteThread::Writer w(write_options, my_batch, callback, user_write_cb, + log_ref, disable_memtable, sub_batch_cnt, + pre_release_callback); StopWatch write_sw(immutable_db_options_.clock, stats_, DB_WRITE); write_thread->JoinBatchGroup(&w); @@ -1485,6 +1517,11 @@ IOStatus DBImpl::WriteToWAL(const WriteThread::WriteGroup& write_group, RecordTick(stats_, WAL_FILE_BYTES, log_size); stats->AddDBStats(InternalStats::kIntStatsWriteWithWal, write_with_wal); RecordTick(stats_, WRITE_WITH_WAL, write_with_wal); + for (auto* writer : write_group) { + if (!writer->CallbackFailed()) { + writer->CheckPostWalWriteCallback(); + } + } } return io_s; } @@ -1549,6 +1586,11 @@ IOStatus DBImpl::ConcurrentWriteToWAL( stats->AddDBStats(InternalStats::kIntStatsWriteWithWal, write_with_wal, concurrent); RecordTick(stats_, WRITE_WITH_WAL, write_with_wal); + for (auto* writer : write_group) { + if (!writer->CallbackFailed()) { + writer->CheckPostWalWriteCallback(); + } + } } return io_s; } @@ -2152,6 +2194,8 @@ void DBImpl::NotifyOnMemTableSealed(ColumnFamilyData* /*cfd*/, // two_write_queues_ is true (This is to simplify the reasoning.) Status DBImpl::SwitchMemtable(ColumnFamilyData* cfd, WriteContext* context) { mutex_.AssertHeld(); + assert(lock_wal_count_ == 0); + // TODO: plumb Env::IOActivity, Env::IOPriority const ReadOptions read_options; const WriteOptions write_options; diff --git a/db/db_iter.cc b/db/db_iter.cc index 63dabb9ca98..b42acc4bc6a 100644 --- a/db/db_iter.cc +++ b/db/db_iter.cc @@ -109,6 +109,16 @@ Status DBIter::GetProperty(std::string prop_name, std::string* prop) { *prop = "Iterator is not valid."; } return Status::OK(); + } else if (prop_name == "rocksdb.iterator.is-value-pinned") { + if (valid_) { + *prop = (pin_thru_lifetime_ && iter_.Valid() && + iter_.value().data() == value_.data()) + ? "1" + : "0"; + } else { + *prop = "Iterator is not valid."; + } + return Status::OK(); } else if (prop_name == "rocksdb.iterator.internal-key") { *prop = saved_key_.GetUserKey().ToString(); return Status::OK(); diff --git a/db/db_iterator_test.cc b/db/db_iterator_test.cc index 8247333b0ba..ef6a3efc273 100644 --- a/db/db_iterator_test.cc +++ b/db/db_iterator_test.cc @@ -133,11 +133,17 @@ TEST_P(DBIteratorTest, IteratorProperty) { ASSERT_NOK(iter->GetProperty("non_existing.value", &prop_value)); ASSERT_OK(iter->GetProperty("rocksdb.iterator.is-key-pinned", &prop_value)); ASSERT_EQ("0", prop_value); + ASSERT_OK( + iter->GetProperty("rocksdb.iterator.is-value-pinned", &prop_value)); + ASSERT_EQ("0", prop_value); ASSERT_OK(iter->GetProperty("rocksdb.iterator.internal-key", &prop_value)); ASSERT_EQ("1", prop_value); iter->Next(); ASSERT_OK(iter->GetProperty("rocksdb.iterator.is-key-pinned", &prop_value)); ASSERT_EQ("Iterator is not valid.", prop_value); + ASSERT_OK( + iter->GetProperty("rocksdb.iterator.is-value-pinned", &prop_value)); + ASSERT_EQ("Iterator is not valid.", prop_value); // Get internal key at which the iteration stopped (tombstone in this case). ASSERT_OK(iter->GetProperty("rocksdb.iterator.internal-key", &prop_value)); @@ -1680,12 +1686,15 @@ TEST_P(DBIteratorTest, PinnedDataIteratorMultipleFiles) { ro.pin_data = true; auto iter = NewIterator(ro); - std::vector> results; + std::vector> results; for (iter->SeekToFirst(); iter->Valid(); iter->Next()) { std::string prop_value; ASSERT_OK(iter->GetProperty("rocksdb.iterator.is-key-pinned", &prop_value)); ASSERT_EQ("1", prop_value); - results.emplace_back(iter->key(), iter->value().ToString()); + ASSERT_OK( + iter->GetProperty("rocksdb.iterator.is-value-pinned", &prop_value)); + ASSERT_EQ("1", prop_value); + results.emplace_back(iter->key(), iter->value()); } ASSERT_EQ(results.size(), true_data.size()); @@ -1739,6 +1748,9 @@ TEST_P(DBIteratorTest, PinnedDataIteratorMergeOperator) { std::string prop_value; ASSERT_OK(iter->GetProperty("rocksdb.iterator.is-key-pinned", &prop_value)); ASSERT_EQ("1", prop_value); + ASSERT_OK( + iter->GetProperty("rocksdb.iterator.is-value-pinned", &prop_value)); + ASSERT_EQ("0", prop_value); results.emplace_back(iter->key(), iter->value().ToString()); } ASSERT_OK(iter->status()); @@ -1792,12 +1804,15 @@ TEST_P(DBIteratorTest, PinnedDataIteratorReadAfterUpdate) { } } - std::vector> results; + std::vector> results; for (iter->SeekToFirst(); iter->Valid(); iter->Next()) { std::string prop_value; ASSERT_OK(iter->GetProperty("rocksdb.iterator.is-key-pinned", &prop_value)); ASSERT_EQ("1", prop_value); - results.emplace_back(iter->key(), iter->value().ToString()); + ASSERT_OK( + iter->GetProperty("rocksdb.iterator.is-value-pinned", &prop_value)); + ASSERT_EQ("1", prop_value); + results.emplace_back(iter->key(), iter->value()); } ASSERT_OK(iter->status()); diff --git a/db/db_kv_checksum_test.cc b/db/db_kv_checksum_test.cc index d3108e73529..3b85ef50ebc 100644 --- a/db/db_kv_checksum_test.cc +++ b/db/db_kv_checksum_test.cc @@ -732,7 +732,7 @@ TEST_P(DbMemtableKVChecksumTest, GetWithCorruptAfterMemtableInsert) { }); SyncPoint::GetInstance()->SetCallBack( - "Memtable::SaveValue:Begin:entry", [&](void* entry) { + "Memtable::SaveValue:Found:entry", [&](void* entry) { char* buf = *static_cast(entry); buf[corrupt_byte_offset_] += corrupt_byte_addend_; ++corrupt_byte_offset_; @@ -769,7 +769,7 @@ TEST_P(DbMemtableKVChecksumTest, }); SyncPoint::GetInstance()->SetCallBack( - "Memtable::SaveValue:Begin:entry", [&](void* entry) { + "Memtable::SaveValue:Found:entry", [&](void* entry) { char* buf = *static_cast(entry); buf[corrupt_byte_offset_] += corrupt_byte_addend_; ++corrupt_byte_offset_; diff --git a/db/db_range_del_test.cc b/db/db_range_del_test.cc index c7c473e73cc..5122aedc97a 100644 --- a/db/db_range_del_test.cc +++ b/db/db_range_del_test.cc @@ -3820,6 +3820,10 @@ TEST_F(DBRangeDelTest, RowCache) { ASSERT_OK(wb.DeleteRange(Key(1), Key(5))); ASSERT_TRUE(db_->Write(WriteOptions(), &wb).IsNotSupported()); ASSERT_EQ(Get(Key(3)), "val"); + // By default, memtable insertion failure will turn the DB to read-only mode. + // The check for delete range should happen before that to fail early + // and should not turn db into read-only mdoe. + ASSERT_OK(Put(Key(5), "foo")); } } // namespace ROCKSDB_NAMESPACE diff --git a/db/db_test.cc b/db/db_test.cc index c3ec250940e..879636176ce 100644 --- a/db/db_test.cc +++ b/db/db_test.cc @@ -3844,6 +3844,9 @@ TEST_P(DBTestWithParam, FIFOCompactionTest) { } else { CompactRangeOptions cro; cro.exclusive_manual_compaction = exclusive_manual_compaction_; + cro.change_level = true; + ASSERT_TRUE(db_->CompactRange(cro, nullptr, nullptr).IsNotSupported()); + cro.change_level = false; ASSERT_OK(db_->CompactRange(cro, nullptr, nullptr)); } // only 5 files should survive @@ -4674,24 +4677,27 @@ TEST_F(DBTest, DynamicMemtableOptions) { #ifdef ROCKSDB_USING_THREAD_STATUS namespace { -void VerifyOperationCount(Env* env, ThreadStatus::OperationType op_type, +bool VerifyOperationCount(Env* env, ThreadStatus::OperationType op_type, int expected_count) { int op_count = 0; std::vector thread_list; - ASSERT_OK(env->GetThreadList(&thread_list)); + EXPECT_OK(env->GetThreadList(&thread_list)); for (const auto& thread : thread_list) { if (thread.operation_type == op_type) { op_count++; } } if (op_count != expected_count) { + fprintf(stderr, "op_count: %d, expected_count %d\n", op_count, + expected_count); for (const auto& thread : thread_list) { - fprintf(stderr, "thread id: %" PRIu64 ", thread status: %s\n", + fprintf(stderr, "thread id: %" PRIu64 ", thread status: %s, cf_name %s\n", thread.thread_id, - thread.GetOperationName(thread.operation_type).c_str()); + thread.GetOperationName(thread.operation_type).c_str(), + thread.cf_name.c_str()); } } - ASSERT_EQ(op_count, expected_count); + return op_count == expected_count; } } // anonymous namespace @@ -4791,11 +4797,11 @@ TEST_F(DBTest, ThreadStatusFlush) { ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing(); CreateAndReopenWithCF({"pikachu"}, options); - VerifyOperationCount(env_, ThreadStatus::OP_FLUSH, 0); + ASSERT_TRUE(VerifyOperationCount(env_, ThreadStatus::OP_FLUSH, 0)); ASSERT_OK(Put(1, "foo", "v1")); ASSERT_EQ("v1", Get(1, "foo")); - VerifyOperationCount(env_, ThreadStatus::OP_FLUSH, 0); + ASSERT_TRUE(VerifyOperationCount(env_, ThreadStatus::OP_FLUSH, 0)); uint64_t num_running_flushes = 0; ASSERT_TRUE(db_->GetIntProperty(DB::Properties::kNumRunningFlushes, @@ -4808,7 +4814,7 @@ TEST_F(DBTest, ThreadStatusFlush) { // The first sync point is to make sure there's one flush job // running when we perform VerifyOperationCount(). TEST_SYNC_POINT("DBTest::ThreadStatusFlush:1"); - VerifyOperationCount(env_, ThreadStatus::OP_FLUSH, 1); + ASSERT_TRUE(VerifyOperationCount(env_, ThreadStatus::OP_FLUSH, 1)); ASSERT_TRUE(db_->GetIntProperty(DB::Properties::kNumRunningFlushes, &num_running_flushes)); ASSERT_EQ(num_running_flushes, 1); @@ -4819,17 +4825,11 @@ TEST_F(DBTest, ThreadStatusFlush) { } TEST_P(DBTestWithParam, ThreadStatusSingleCompaction) { - const int kTestKeySize = 16; const int kTestValueSize = 984; - const int kEntrySize = kTestKeySize + kTestValueSize; const int kEntriesPerBuffer = 100; Options options; options.create_if_missing = true; - options.write_buffer_size = kEntrySize * kEntriesPerBuffer; options.compaction_style = kCompactionStyleLevel; - options.target_file_size_base = options.write_buffer_size; - options.max_bytes_for_level_base = options.target_file_size_base * 2; - options.max_bytes_for_level_multiplier = 2; options.compression = kNoCompression; options = CurrentOptions(options); options.env = env_; @@ -4864,7 +4864,7 @@ TEST_P(DBTestWithParam, ThreadStatusSingleCompaction) { &num_running_compactions)); ASSERT_EQ(num_running_compactions, 0); TEST_SYNC_POINT("DBTest::ThreadStatusSingleCompaction:0"); - ASSERT_GE(NumTableFilesAtLevel(0), + ASSERT_EQ(NumTableFilesAtLevel(0), options.level0_file_num_compaction_trigger); // This makes sure at least one compaction is running. @@ -4872,10 +4872,18 @@ TEST_P(DBTestWithParam, ThreadStatusSingleCompaction) { if (options.enable_thread_tracking) { // expecting one single L0 to L1 compaction - VerifyOperationCount(env_, ThreadStatus::OP_COMPACTION, 1); + // This test is flaky and fails here. + bool match = VerifyOperationCount(env_, ThreadStatus::OP_COMPACTION, 1); + if (!match) { + ASSERT_TRUE(db_->GetIntProperty(DB::Properties::kNumRunningCompactions, + &num_running_compactions)); + fprintf(stderr, "running compaction: %" PRIu64 " lsm state: %s\n", + num_running_compactions, FilesPerLevel().c_str()); + } + ASSERT_TRUE(match); } else { // If thread tracking is not enabled, compaction count should be 0. - VerifyOperationCount(env_, ThreadStatus::OP_COMPACTION, 0); + ASSERT_TRUE(VerifyOperationCount(env_, ThreadStatus::OP_COMPACTION, 0)); } ASSERT_TRUE(db_->GetIntProperty(DB::Properties::kNumRunningCompactions, &num_running_compactions)); @@ -5637,6 +5645,8 @@ TEST_F(DBTest, DynamicUniversalCompactionOptions) { ASSERT_EQ( dbfull()->GetOptions().compaction_options_universal.allow_trivial_move, false); + ASSERT_EQ(dbfull()->GetOptions().compaction_options_universal.max_read_amp, + -1); ASSERT_OK(dbfull()->SetOptions( {{"compaction_options_universal", "{size_ratio=7;}"}})); @@ -5658,9 +5668,11 @@ TEST_F(DBTest, DynamicUniversalCompactionOptions) { ASSERT_EQ( dbfull()->GetOptions().compaction_options_universal.allow_trivial_move, false); + ASSERT_EQ(dbfull()->GetOptions().compaction_options_universal.max_read_amp, + -1); - ASSERT_OK(dbfull()->SetOptions( - {{"compaction_options_universal", "{min_merge_width=11;}"}})); + ASSERT_OK(dbfull()->SetOptions({{"compaction_options_universal", + "{min_merge_width=11;max_read_amp=0;}"}})); ASSERT_EQ(dbfull()->GetOptions().compaction_options_universal.size_ratio, 7u); ASSERT_EQ(dbfull()->GetOptions().compaction_options_universal.min_merge_width, 11u); @@ -5679,6 +5691,8 @@ TEST_F(DBTest, DynamicUniversalCompactionOptions) { ASSERT_EQ( dbfull()->GetOptions().compaction_options_universal.allow_trivial_move, false); + ASSERT_EQ(dbfull()->GetOptions().compaction_options_universal.max_read_amp, + 0); } TEST_F(DBTest, FileCreationRandomFailure) { diff --git a/db/db_test2.cc b/db/db_test2.cc index 2da4b563d04..e6a3adf9b8b 100644 --- a/db/db_test2.cc +++ b/db/db_test2.cc @@ -4154,7 +4154,7 @@ TEST_F(DBTest2, LiveFilesOmitObsoleteFiles) { TEST_SYNC_POINT("DBTest2::LiveFilesOmitObsoleteFiles:FlushTriggered"); ASSERT_OK(db_->DisableFileDeletions()); - VectorLogPtr log_files; + VectorWalPtr log_files; ASSERT_OK(db_->GetSortedWalFiles(log_files)); TEST_SYNC_POINT("DBTest2::LiveFilesOmitObsoleteFiles:LiveFilesCaptured"); for (const auto& log_file : log_files) { diff --git a/db/db_test_util.h b/db/db_test_util.h index dce96467bff..a31b95a015c 100644 --- a/db/db_test_util.h +++ b/db/db_test_util.h @@ -276,16 +276,16 @@ class SpecialEnv : public EnvWrapper { SpecialEnv* env_; std::unique_ptr base_; }; - class WalFile : public WritableFile { + class SpecialWalFile : public WritableFile { public: - WalFile(SpecialEnv* env, std::unique_ptr&& b) + SpecialWalFile(SpecialEnv* env, std::unique_ptr&& b) : env_(env), base_(std::move(b)) { env_->num_open_wal_file_.fetch_add(1); } - virtual ~WalFile() { env_->num_open_wal_file_.fetch_add(-1); } + virtual ~SpecialWalFile() { env_->num_open_wal_file_.fetch_add(-1); } Status Append(const Slice& data) override { #if !(defined NDEBUG) || !defined(OS_WIN) - TEST_SYNC_POINT("SpecialEnv::WalFile::Append:1"); + TEST_SYNC_POINT("SpecialEnv::SpecialWalFile::Append:1"); #endif Status s; if (env_->log_write_error_.load(std::memory_order_acquire)) { @@ -299,7 +299,7 @@ class SpecialEnv : public EnvWrapper { s = base_->Append(data); } #if !(defined NDEBUG) || !defined(OS_WIN) - TEST_SYNC_POINT("SpecialEnv::WalFile::Append:2"); + TEST_SYNC_POINT("SpecialEnv::SpecialWalFile::Append:2"); #endif return s; } @@ -419,7 +419,7 @@ class SpecialEnv : public EnvWrapper { } else if (strstr(f.c_str(), "MANIFEST") != nullptr) { r->reset(new ManifestFile(this, std::move(*r))); } else if (strstr(f.c_str(), "log") != nullptr) { - r->reset(new WalFile(this, std::move(*r))); + r->reset(new SpecialWalFile(this, std::move(*r))); } else { r->reset(new OtherFile(this, std::move(*r))); } diff --git a/db/db_wal_test.cc b/db/db_wal_test.cc index 1ff16ee9a4f..a4976c3ed35 100644 --- a/db/db_wal_test.cc +++ b/db/db_wal_test.cc @@ -270,8 +270,10 @@ TEST_F(DBWALTest, SyncWALNotWaitWrite) { ASSERT_OK(Put("foo3", "bar3")); ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->LoadDependency({ - {"SpecialEnv::WalFile::Append:1", "DBWALTest::SyncWALNotWaitWrite:1"}, - {"DBWALTest::SyncWALNotWaitWrite:2", "SpecialEnv::WalFile::Append:2"}, + {"SpecialEnv::SpecialWalFile::Append:1", + "DBWALTest::SyncWALNotWaitWrite:1"}, + {"DBWALTest::SyncWALNotWaitWrite:2", + "SpecialEnv::SpecialWalFile::Append:2"}, }); ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing(); diff --git a/db/db_write_test.cc b/db/db_write_test.cc index e074dd6b42f..2dfcd864f5a 100644 --- a/db/db_write_test.cc +++ b/db/db_write_test.cc @@ -666,11 +666,25 @@ TEST_P(DBWriteTest, LockWALInEffect) { // try the 1st WAL created during open ASSERT_OK(Put("key0", "value")); ASSERT_NE(options.manual_wal_flush, dbfull()->WALBufferIsEmpty()); + ASSERT_OK(db_->LockWAL()); + ASSERT_TRUE(dbfull()->WALBufferIsEmpty()); + uint64_t wal_num = dbfull()->TEST_GetCurrentLogNumber(); + // Manual flush with wait=false should abruptly fail with TryAgain + FlushOptions flush_opts; + flush_opts.wait = false; + for (bool allow_write_stall : {true, false}) { + flush_opts.allow_write_stall = allow_write_stall; + ASSERT_TRUE(db_->Flush(flush_opts).IsTryAgain()); + } + ASSERT_EQ(wal_num, dbfull()->TEST_GetCurrentLogNumber()); + ASSERT_OK(db_->UnlockWAL()); - // try the 2nd wal created during SwitchWAL + + // try the 2nd wal created during SwitchWAL (not locked this time) ASSERT_OK(dbfull()->TEST_SwitchWAL()); + ASSERT_NE(wal_num, dbfull()->TEST_GetCurrentLogNumber()); ASSERT_OK(Put("key1", "value")); ASSERT_NE(options.manual_wal_flush, dbfull()->WALBufferIsEmpty()); ASSERT_OK(db_->LockWAL()); @@ -709,21 +723,57 @@ TEST_P(DBWriteTest, LockWALInEffect) { } TEST_P(DBWriteTest, LockWALConcurrentRecursive) { + // This is a micro-stress test of LockWAL and concurrency handling. + // It is considered the most convenient way to balance functional + // coverage and reproducibility (vs. the two extremes of (a) unit tests + // tailored to specific interleavings and (b) db_stress) Options options = GetOptions(); Reopen(options); - ASSERT_OK(Put("k1", "val")); + ASSERT_OK(Put("k1", "k1_orig")); ASSERT_OK(db_->LockWAL()); // 0 -> 1 auto frozen_seqno = db_->GetLatestSequenceNumber(); - std::atomic t1_completed{false}; - port::Thread t1{[&]() { - // Won't finish until WAL unlocked - ASSERT_OK(Put("k1", "val2")); - t1_completed = true; + + std::string ingest_file = dbname_ + "/external.sst"; + { + SstFileWriter sst_file_writer(EnvOptions(), options); + ASSERT_OK(sst_file_writer.Open(ingest_file)); + ASSERT_OK(sst_file_writer.Put("k2", "k2_val")); + ExternalSstFileInfo external_info; + ASSERT_OK(sst_file_writer.Finish(&external_info)); + } + AcqRelAtomic parallel_ingest_completed{false}; + port::Thread parallel_ingest{[&]() { + IngestExternalFileOptions ingest_opts; + ingest_opts.move_files = true; // faster than copy + // Shouldn't finish until WAL unlocked + ASSERT_OK(db_->IngestExternalFile({ingest_file}, ingest_opts)); + parallel_ingest_completed.Store(true); + }}; + + AcqRelAtomic flush_completed{false}; + port::Thread parallel_flush{[&]() { + FlushOptions flush_opts; + // NB: Flush with wait=false case is tested above in LockWALInEffect + flush_opts.wait = true; + // allow_write_stall = true blocks in fewer cases + flush_opts.allow_write_stall = true; + // Shouldn't finish until WAL unlocked + ASSERT_OK(db_->Flush(flush_opts)); + flush_completed.Store(true); + }}; + + AcqRelAtomic parallel_put_completed{false}; + port::Thread parallel_put{[&]() { + // This can make certain failure scenarios more likely: + // sleep(1); + // Shouldn't finish until WAL unlocked + ASSERT_OK(Put("k1", "k1_mod")); + parallel_put_completed.Store(true); }}; ASSERT_OK(db_->LockWAL()); // 1 -> 2 // Read-only ops are OK - ASSERT_EQ(Get("k1"), "val"); + ASSERT_EQ(Get("k1"), "k1_orig"); { std::vector files; LiveFilesStorageInfoOptions lf_opts; @@ -732,29 +782,35 @@ TEST_P(DBWriteTest, LockWALConcurrentRecursive) { ASSERT_OK(db_->GetLiveFilesStorageInfo({lf_opts}, &files)); } - port::Thread t2{[&]() { + port::Thread parallel_lock_wal{[&]() { ASSERT_OK(db_->LockWAL()); // 2 -> 3 or 1 -> 2 }}; ASSERT_OK(db_->UnlockWAL()); // 2 -> 1 or 3 -> 2 - // Give t1 an extra chance to jump in case of bug + // Give parallel_put an extra chance to jump in case of bug std::this_thread::yield(); - t2.join(); - ASSERT_FALSE(t1_completed.load()); + parallel_lock_wal.join(); + ASSERT_FALSE(parallel_put_completed.Load()); + ASSERT_FALSE(parallel_ingest_completed.Load()); + ASSERT_FALSE(flush_completed.Load()); // Should now have 2 outstanding LockWAL - ASSERT_EQ(Get("k1"), "val"); + ASSERT_EQ(Get("k1"), "k1_orig"); ASSERT_OK(db_->UnlockWAL()); // 2 -> 1 - ASSERT_FALSE(t1_completed.load()); - ASSERT_EQ(Get("k1"), "val"); + ASSERT_FALSE(parallel_put_completed.Load()); + ASSERT_FALSE(parallel_ingest_completed.Load()); + ASSERT_FALSE(flush_completed.Load()); + + ASSERT_EQ(Get("k1"), "k1_orig"); + ASSERT_EQ(Get("k2"), "NOT_FOUND"); ASSERT_EQ(frozen_seqno, db_->GetLatestSequenceNumber()); // Ensure final Unlock is concurrency safe and extra Unlock is safe but // non-OK std::atomic unlock_ok{0}; - port::Thread t3{[&]() { + port::Thread parallel_stuff{[&]() { if (db_->UnlockWAL().ok()) { unlock_ok++; } @@ -767,18 +823,23 @@ TEST_P(DBWriteTest, LockWALConcurrentRecursive) { if (db_->UnlockWAL().ok()) { unlock_ok++; } - t3.join(); + parallel_stuff.join(); // There was one extra unlock, so just one non-ok ASSERT_EQ(unlock_ok.load(), 2); // Write can proceed - t1.join(); - ASSERT_TRUE(t1_completed.load()); - ASSERT_EQ(Get("k1"), "val2"); + parallel_put.join(); + ASSERT_TRUE(parallel_put_completed.Load()); + ASSERT_EQ(Get("k1"), "k1_mod"); + parallel_ingest.join(); + ASSERT_TRUE(parallel_ingest_completed.Load()); + ASSERT_EQ(Get("k2"), "k2_val"); + parallel_flush.join(); + ASSERT_TRUE(flush_completed.Load()); // And new writes - ASSERT_OK(Put("k2", "val")); - ASSERT_EQ(Get("k2"), "val"); + ASSERT_OK(Put("k3", "val")); + ASSERT_EQ(Get("k3"), "val"); } TEST_P(DBWriteTest, ConcurrentlyDisabledWAL) { diff --git a/db/dbformat.cc b/db/dbformat.cc index 2378ba488b9..4a613c7d4f2 100644 --- a/db/dbformat.cc +++ b/db/dbformat.cc @@ -47,6 +47,8 @@ EntryType GetEntryType(ValueType value_type) { return kEntryBlobIndex; case kTypeWideColumnEntity: return kEntryWideColumnEntity; + case kTypeValuePreferredSeqno: + return kEntryTimedPut; default: return kEntryOther; } @@ -155,10 +157,23 @@ void ReplaceInternalKeyWithMinTimestamp(std::string* result, const Slice& key, result->append(key.data() + key_sz - kNumInternalBytes, kNumInternalBytes); } -std::string ParsedInternalKey::DebugString(bool log_err_key, bool hex) const { +std::string ParsedInternalKey::DebugString(bool log_err_key, bool hex, + const Comparator* ucmp) const { std::string result = "'"; + size_t ts_sz_for_debug = ucmp == nullptr ? 0 : ucmp->timestamp_size(); if (log_err_key) { - result += user_key.ToString(hex); + if (ts_sz_for_debug == 0) { + result += user_key.ToString(hex); + } else { + assert(user_key.size() >= ts_sz_for_debug); + Slice user_key_without_ts = user_key; + user_key_without_ts.remove_suffix(ts_sz_for_debug); + result += user_key_without_ts.ToString(hex); + Slice ts = Slice(user_key.data() + user_key.size() - ts_sz_for_debug, + ts_sz_for_debug); + result += "|timestamp:"; + result += ucmp->TimestampToString(ts); + } } else { result += ""; } @@ -171,11 +186,11 @@ std::string ParsedInternalKey::DebugString(bool log_err_key, bool hex) const { return result; } -std::string InternalKey::DebugString(bool hex) const { +std::string InternalKey::DebugString(bool hex, const Comparator* ucmp) const { std::string result; ParsedInternalKey parsed; if (ParseInternalKey(rep_, &parsed, false /* log_err_key */).ok()) { - result = parsed.DebugString(true /* log_err_key */, hex); // TODO + result = parsed.DebugString(true /* log_err_key */, hex, ucmp); // TODO } else { result = "(bad)"; result.append(EscapeString(rep_)); diff --git a/db/dbformat.h b/db/dbformat.h index 5b16726693e..bd0c8f03de5 100644 --- a/db/dbformat.h +++ b/db/dbformat.h @@ -148,7 +148,8 @@ struct ParsedInternalKey { // u contains timestamp if user timestamp feature is enabled. ParsedInternalKey(const Slice& u, const SequenceNumber& seq, ValueType t) : user_key(u), sequence(seq), type(t) {} - std::string DebugString(bool log_err_key, bool hex) const; + std::string DebugString(bool log_err_key, bool hex, + const Comparator* ucmp = nullptr) const; void clear() { user_key.clear(); @@ -503,7 +504,7 @@ class InternalKey { AppendInternalKeyFooter(&rep_, s, t); } - std::string DebugString(bool hex) const; + std::string DebugString(bool hex, const Comparator* ucmp = nullptr) const; }; inline int InternalKeyComparator::Compare(const InternalKey& a, diff --git a/db/error_handler_fs_test.cc b/db/error_handler_fs_test.cc index ce351e3695d..a4eaad15317 100644 --- a/db/error_handler_fs_test.cc +++ b/db/error_handler_fs_test.cc @@ -30,6 +30,14 @@ class DBErrorHandlingFSTest : public DBTestBase { fault_env_.reset(new CompositeEnvWrapper(env_, fault_fs_)); } + ~DBErrorHandlingFSTest() { + // Before destroying fault_env_ + SyncPoint::GetInstance()->DisableProcessing(); + SyncPoint::GetInstance()->LoadDependency({}); + SyncPoint::GetInstance()->ClearAllCallBacks(); + Close(); + } + std::string GetManifestNameFromLiveFiles() { std::vector live_files; uint64_t manifest_size; @@ -417,7 +425,7 @@ TEST_F(DBErrorHandlingFSTest, FlushWALWriteRetryableError) { listener->EnableAutoRecovery(false); SyncPoint::GetInstance()->SetCallBack( - "DBImpl::SyncClosedLogs:Start", + "DBImpl::SyncClosedWals:Start", [&](void*) { fault_fs_->SetFilesystemActive(false, error_msg); }); SyncPoint::GetInstance()->EnableProcessing(); @@ -462,7 +470,7 @@ TEST_F(DBErrorHandlingFSTest, FlushWALAtomicWriteRetryableError) { listener->EnableAutoRecovery(false); SyncPoint::GetInstance()->SetCallBack( - "DBImpl::SyncClosedLogs:Start", + "DBImpl::SyncClosedWals:Start", [&](void*) { fault_fs_->SetFilesystemActive(false, error_msg); }); SyncPoint::GetInstance()->EnableProcessing(); diff --git a/db/external_sst_file_ingestion_job.cc b/db/external_sst_file_ingestion_job.cc index 684952c98ec..3b77025d660 100644 --- a/db/external_sst_file_ingestion_job.cc +++ b/db/external_sst_file_ingestion_job.cc @@ -953,13 +953,15 @@ Status ExternalSstFileIngestionJob::AssignLevelAndSeqnoForIngestedFile( *assigned_seqno = 0; auto ucmp = cfd_->user_comparator(); const size_t ts_sz = ucmp->timestamp_size(); - if (force_global_seqno || files_overlap_) { + if (force_global_seqno || files_overlap_ || + compaction_style == kCompactionStyleFIFO) { *assigned_seqno = last_seqno + 1; // If files overlap, we have to ingest them at level 0. - if (files_overlap_) { + if (files_overlap_ || compaction_style == kCompactionStyleFIFO) { assert(ts_sz == 0); file_to_ingest->picked_level = 0; - if (ingestion_options_.fail_if_not_bottommost_level) { + if (ingestion_options_.fail_if_not_bottommost_level && + cfd_->NumberLevels() > 1) { status = Status::TryAgain( "Files cannot be ingested to Lmax. Please make sure key range of " "Lmax does not overlap with files to ingest."); diff --git a/db/external_sst_file_test.cc b/db/external_sst_file_test.cc index 64f8a03d46d..26bdaa2fddb 100644 --- a/db/external_sst_file_test.cc +++ b/db/external_sst_file_test.cc @@ -3082,6 +3082,68 @@ TEST_P(ExternalSSTFileTest, delete iter; } +TEST_F(ExternalSSTFileTest, FIFOCompaction) { + // FIFO always ingests SST files to L0 and assign latest sequence number. + Options options = CurrentOptions(); + options.num_levels = 1; + options.compaction_style = kCompactionStyleFIFO; + options.max_open_files = -1; + DestroyAndReopen(options); + std::map true_data; + + for (int i = 0; i < 100; ++i) { + ASSERT_OK(Put(Key(i), Key(i) + "_val")); + true_data[Key(i)] = Key(i) + "_val"; + } + ASSERT_OK(Flush()); + ASSERT_EQ("1", FilesPerLevel()); + std::vector> file_data; + for (int i = 0; i <= 20; i++) { + file_data.emplace_back(Key(i), Key(i) + "_ingest"); + } + // Overlaps with memtable, will trigger flush + ASSERT_OK(GenerateAndAddExternalFile(options, file_data, -1, + /*allow_global_seqno=*/true, true, false, + false, false, &true_data)); + ASSERT_EQ("2", FilesPerLevel()); + + file_data.clear(); + for (int i = 100; i <= 120; i++) { + file_data.emplace_back(Key(i), Key(i) + "_ingest"); + } + // global sequence number is always assigned, so this will fail + ASSERT_NOK(GenerateAndAddExternalFile(options, file_data, -1, + /*allow_global_seqno=*/false, true, + false, false, false, &true_data)); + ASSERT_OK(GenerateAndAddExternalFile(options, file_data, -1, + /*allow_global_seqno=*/true, true, false, + false, false, &true_data)); + + // Compact to data to lower level to test multi-level FIFO later + options.num_levels = 7; + options.compaction_style = kCompactionStyleUniversal; + ASSERT_OK(TryReopen(options)); + CompactRangeOptions cro; + cro.bottommost_level_compaction = BottommostLevelCompaction::kForceOptimized; + ASSERT_OK(db_->CompactRange(cro, nullptr, nullptr)); + ASSERT_EQ("0,0,0,0,0,0,1", FilesPerLevel()); + + options.num_levels = 7; + options.compaction_style = kCompactionStyleFIFO; + ASSERT_OK(TryReopen(options)); + file_data.clear(); + for (int i = 200; i <= 220; i++) { + file_data.emplace_back(Key(i), Key(i) + "_ingest"); + } + // Files are ingested into L0 for multi-level FIFO + ASSERT_OK(GenerateAndAddExternalFile(options, file_data, -1, + /*allow_global_seqno=*/true, true, false, + false, false, &true_data)); + + ASSERT_EQ("1,0,0,0,0,0,1", FilesPerLevel()); + VerifyDBFromMap(true_data); +} + class ExternalSSTFileWithTimestampTest : public ExternalSSTFileTest { public: ExternalSSTFileWithTimestampTest() = default; diff --git a/db/fault_injection_test.cc b/db/fault_injection_test.cc index 7b93326ca8b..ea90f1e46b5 100644 --- a/db/fault_injection_test.cc +++ b/db/fault_injection_test.cc @@ -12,6 +12,7 @@ // file data (or entire files) not protected by a "sync". #include "db/db_impl/db_impl.h" +#include "db/db_test_util.h" #include "db/log_format.h" #include "db/version_set.h" #include "env/mock_env.h" @@ -629,6 +630,35 @@ INSTANTIATE_TEST_CASE_P( std::make_tuple(false, kSyncWal, kEnd), std::make_tuple(true, kSyncWal, kEnd))); +class FaultInjectionFSTest : public DBTestBase { + public: + FaultInjectionFSTest() + : DBTestBase("fault_injection_fs_test", /*env_do_fsync=*/false) {} +}; + +TEST_F(FaultInjectionFSTest, SyncWALDuringDBClose) { + std::shared_ptr fault_fs = + std::make_shared(env_->GetFileSystem()); + std::unique_ptr env(new CompositeEnvWrapper(env_, fault_fs)); + Options options = CurrentOptions(); + options.manual_wal_flush = true; + options.avoid_sync_during_shutdown = true; + options.env = env.get(); + Reopen(options); + ASSERT_OK(Put("k1", "v1")); + Close(); + Reopen(options); + ASSERT_EQ("NOT_FOUND", Get("k1")); + Destroy(options); + + options.avoid_sync_during_shutdown = false; + Reopen(options); + ASSERT_OK(Put("k1", "v1")); + Close(); + Reopen(options); + ASSERT_EQ("v1", Get("k1")); + Destroy(options); +} } // namespace ROCKSDB_NAMESPACE int main(int argc, char** argv) { diff --git a/db/import_column_family_job.cc b/db/import_column_family_job.cc index 43c32e39b4f..270f889a9ab 100644 --- a/db/import_column_family_job.cc +++ b/db/import_column_family_job.cc @@ -175,22 +175,29 @@ Status ImportColumnFamilyJob::Run() { static_cast(temp_current_time); } - // Recover files' epoch number using dummy VersionStorageInfo - VersionBuilder dummy_version_builder( - cfd_->current()->version_set()->file_options(), cfd_->ioptions(), - cfd_->table_cache(), cfd_->current()->storage_info(), - cfd_->current()->version_set(), - cfd_->GetFileMetadataCacheReservationManager()); - VersionStorageInfo dummy_vstorage( - &cfd_->internal_comparator(), cfd_->user_comparator(), - cfd_->NumberLevels(), cfd_->ioptions()->compaction_style, - nullptr /* src_vstorage */, cfd_->ioptions()->force_consistency_checks, - EpochNumberRequirement::kMightMissing, cfd_->ioptions()->clock, - cfd_->GetLatestMutableCFOptions()->bottommost_file_compaction_delay, - cfd_->current()->version_set()->offpeak_time_option()); Status s; - + // When importing multiple CFs, we should not reuse epoch number from ingested + // files. Since these epoch numbers were assigned by different CFs, there may + // be different files from different CFs with the same epoch number. With a + // subsequent intra-L0 compaction we may end up with files with overlapping + // key range but the same epoch number. Here we will create a dummy + // VersionStorageInfo per CF being imported. Each CF's files will be assigned + // increasing epoch numbers to avoid duplicated epoch number. This is done by + // only resetting epoch number of the new CF in the first call to + // RecoverEpochNumbers() below. for (size_t i = 0; s.ok() && i < files_to_import_.size(); ++i) { + VersionBuilder dummy_version_builder( + cfd_->current()->version_set()->file_options(), cfd_->ioptions(), + cfd_->table_cache(), cfd_->current()->storage_info(), + cfd_->current()->version_set(), + cfd_->GetFileMetadataCacheReservationManager()); + VersionStorageInfo dummy_vstorage( + &cfd_->internal_comparator(), cfd_->user_comparator(), + cfd_->NumberLevels(), cfd_->ioptions()->compaction_style, + nullptr /* src_vstorage */, cfd_->ioptions()->force_consistency_checks, + EpochNumberRequirement::kMightMissing, cfd_->ioptions()->clock, + cfd_->GetLatestMutableCFOptions()->bottommost_file_compaction_delay, + cfd_->current()->version_set()->offpeak_time_option()); for (size_t j = 0; s.ok() && j < files_to_import_[i].size(); ++j) { const auto& f = files_to_import_[i][j]; const auto& file_metadata = *metadatas_[i][j]; @@ -218,42 +225,39 @@ Status ImportColumnFamilyJob::Run() { f.table_properties.user_defined_timestamps_persisted)); s = dummy_version_builder.Apply(&dummy_version_edit); } - } - - if (s.ok()) { - s = dummy_version_builder.SaveTo(&dummy_vstorage); - } - if (s.ok()) { - dummy_vstorage.RecoverEpochNumbers(cfd_); - } - - // Record changes from this CF import in VersionEdit, including files with - // recovered epoch numbers - if (s.ok()) { - edit_.SetColumnFamily(cfd_->GetID()); - + if (s.ok()) { + s = dummy_version_builder.SaveTo(&dummy_vstorage); + } + if (s.ok()) { + // force resetting epoch number for each file + dummy_vstorage.RecoverEpochNumbers(cfd_, /*restart_epoch=*/i == 0, + /*force=*/true); + edit_.SetColumnFamily(cfd_->GetID()); + + for (int level = 0; level < dummy_vstorage.num_levels(); level++) { + for (FileMetaData* file_meta : dummy_vstorage.LevelFiles(level)) { + edit_.AddFile(level, *file_meta); + // If incoming sequence number is higher, update local sequence + // number. + if (file_meta->fd.largest_seqno > versions_->LastSequence()) { + versions_->SetLastAllocatedSequence(file_meta->fd.largest_seqno); + versions_->SetLastPublishedSequence(file_meta->fd.largest_seqno); + versions_->SetLastSequence(file_meta->fd.largest_seqno); + } + } + } + } + // Release resources occupied by the dummy VersionStorageInfo for (int level = 0; level < dummy_vstorage.num_levels(); level++) { for (FileMetaData* file_meta : dummy_vstorage.LevelFiles(level)) { - edit_.AddFile(level, *file_meta); - // If incoming sequence number is higher, update local sequence number. - if (file_meta->fd.largest_seqno > versions_->LastSequence()) { - versions_->SetLastAllocatedSequence(file_meta->fd.largest_seqno); - versions_->SetLastPublishedSequence(file_meta->fd.largest_seqno); - versions_->SetLastSequence(file_meta->fd.largest_seqno); + file_meta->refs--; + if (file_meta->refs <= 0) { + delete file_meta; } } } } - // Release resources occupied by the dummy VersionStorageInfo - for (int level = 0; level < dummy_vstorage.num_levels(); level++) { - for (FileMetaData* file_meta : dummy_vstorage.LevelFiles(level)) { - file_meta->refs--; - if (file_meta->refs <= 0) { - delete file_meta; - } - } - } return s; } diff --git a/db/import_column_family_test.cc b/db/import_column_family_test.cc index e3a36073f42..89fdbb7e300 100644 --- a/db/import_column_family_test.cc +++ b/db/import_column_family_test.cc @@ -946,6 +946,108 @@ TEST_F(ImportColumnFamilyTest, ImportMultiColumnFamilySeveralFilesWithOverlap) { ASSERT_OK(db_->DestroyColumnFamilyHandle(second_cfh)); } +TEST_F(ImportColumnFamilyTest, AssignEpochNumberToMultipleCF) { + // Test ingesting CFs where L0 files could have the same epoch number. + Options options = CurrentOptions(); + options.level_compaction_dynamic_level_bytes = true; + options.max_background_jobs = 8; + env_->SetBackgroundThreads(2, Env::LOW); + env_->SetBackgroundThreads(0, Env::BOTTOM); + CreateAndReopenWithCF({"CF1", "CF2"}, options); + + // CF1: + // L6: [0, 99], [100, 199] + // CF2: + // L6: [1000, 1099], [1100, 1199] + for (int i = 100; i < 200; ++i) { + ASSERT_OK(Put(1, Key(i), Key(i) + "_val")); + ASSERT_OK(Put(2, Key(1000 + i), Key(1000 + i) + "_val")); + } + ASSERT_OK(Flush(1)); + ASSERT_OK(Flush(2)); + for (int i = 0; i < 100; ++i) { + ASSERT_OK(Put(1, Key(i), Key(i) + "_val")); + ASSERT_OK(Put(2, Key(1000 + i), Key(1000 + i) + "_val")); + } + ASSERT_OK(Flush(1)); + ASSERT_OK(Flush(2)); + MoveFilesToLevel(6, 1); + MoveFilesToLevel(6, 2); + + // CF1: + // level 0 epoch: 5 file num 30 smallest key000010 - key000019 + // level 0 epoch: 4 file num 27 smallest key000000 - key000009 + // level 0 epoch: 3 file num 23 smallest key000100 - key000199 + // level 6 epoch: 2 file num 20 smallest key000000 - key000099 + // level 6 epoch: 1 file num 17 smallest key000100 - key000199 + // CF2: + // level 0 epoch: 5 file num 31 smallest key001010 - key001019 + // level 0 epoch: 4 file num 28 smallest key001000 - key001009 + // level 0 epoch: 3 file num 25 smallest key001020 - key001029 + // level 6 epoch: 2 file num 21 smallest key001000 - key001099 + // level 6 epoch: 1 file num 18 smallest key001100 - key001199 + for (int i = 100; i < 200; ++i) { + ASSERT_OK(Put(1, Key(i), Key(i) + "_val")); + } + ASSERT_OK(Flush(1)); + for (int i = 20; i < 30; ++i) { + ASSERT_OK(Put(2, Key(i + 1000), Key(i + 1000) + "_val")); + } + ASSERT_OK(Flush(2)); + + for (int i = 0; i < 20; ++i) { + ASSERT_OK(Put(1, Key(i), Key(i) + "_val")); + ASSERT_OK(Put(2, Key(i + 1000), Key(i + 1000) + "_val")); + if (i % 10 == 9) { + ASSERT_OK(Flush(1)); + ASSERT_OK(Flush(2)); + } + } + ASSERT_OK(Flush(1)); + ASSERT_OK(Flush(2)); + + // Create a CF by importing these two CF1 and CF2. + // Then two compactions will be triggerred, one to compact from L0 + // to L6 (files #23 and #17), and another to do intra-L0 compaction + // for the rest of the L0 files. Before a bug fix, we used to + // directly use the epoch numbers from the ingested files in the new CF. + // This means different files from different CFs can have the same epoch + // number. If the intra-L0 compaction finishes first, it can cause a + // corruption where two L0 files can have the same epoch number but + // with overlapping key range. + Checkpoint* checkpoint1; + ASSERT_OK(Checkpoint::Create(db_, &checkpoint1)); + ASSERT_OK(checkpoint1->ExportColumnFamily(handles_[1], export_files_dir_, + &metadata_ptr_)); + ASSERT_OK(checkpoint1->ExportColumnFamily(handles_[2], export_files_dir2_, + &metadata_ptr2_)); + ASSERT_NE(metadata_ptr_, nullptr); + ASSERT_NE(metadata_ptr2_, nullptr); + + std::atomic_int compaction_counter = 0; + SyncPoint::GetInstance()->SetCallBack( + "DBImpl::BackgroundCompaction:NonTrivial:BeforeRun", + [&compaction_counter](void*) { + compaction_counter++; + if (compaction_counter == 1) { + // Wait for the next compaction to finish + TEST_SYNC_POINT("WaitForSecondCompaction"); + } + }); + SyncPoint::GetInstance()->LoadDependency( + {{"DBImpl::BackgroundCompaction:AfterCompaction", + "WaitForSecondCompaction"}}); + SyncPoint::GetInstance()->EnableProcessing(); + ImportColumnFamilyOptions import_options; + import_options.move_files = false; + std::vector metadatas = {metadata_ptr_, + metadata_ptr2_}; + ASSERT_OK(db_->CreateColumnFamilyWithImport(options, "CF3", import_options, + metadatas, &import_cfh_)); + WaitForCompactOptions o; + ASSERT_OK(db_->WaitForCompact(o)); + delete checkpoint1; +} } // namespace ROCKSDB_NAMESPACE int main(int argc, char** argv) { diff --git a/db/log_reader.cc b/db/log_reader.cc index 110eb2c27c8..d6eefec05c5 100644 --- a/db/log_reader.cc +++ b/db/log_reader.cc @@ -487,9 +487,11 @@ unsigned int Reader::ReadPhysicalRecord(Slice* result, size_t* drop_size, type == kRecyclableUserDefinedTimestampSizeType); if (is_recyclable_type) { header_size = kRecyclableHeaderSize; - if (end_of_buffer_offset_ - buffer_.size() == 0) { - recycled_ = true; + if (first_record_read_ && !recycled_) { + // A recycled log should have started with a recycled record + return kBadRecord; } + recycled_ = true; // We need enough for the larger header if (buffer_.size() < static_cast(kRecyclableHeaderSize)) { int r = kEof; @@ -867,9 +869,12 @@ bool FragmentBufferedReader::TryReadFragment( int header_size = kHeaderSize; if ((type >= kRecyclableFullType && type <= kRecyclableLastType) || type == kRecyclableUserDefinedTimestampSizeType) { - if (end_of_buffer_offset_ - buffer_.size() == 0) { - recycled_ = true; + if (first_record_read_ && !recycled_) { + // A recycled log should have started with a recycled record + *fragment_type_or_err = kBadRecord; + return true; } + recycled_ = true; header_size = kRecyclableHeaderSize; while (buffer_.size() < static_cast(kRecyclableHeaderSize)) { size_t old_size = buffer_.size(); diff --git a/db/log_test.cc b/db/log_test.cc index 79ff02a04b7..51f88ac5b7d 100644 --- a/db/log_test.cc +++ b/db/log_test.cc @@ -774,6 +774,32 @@ TEST_P(LogTest, RecycleWithTimestampSize) { ASSERT_EQ("EOF", Read()); } +// Validates that `MaybeAddUserDefinedTimestampSizeRecord`` adds padding to the +// tail of a block and switches to a new block, if there's not enough space for +// the record. +TEST_P(LogTest, TimestampSizeRecordPadding) { + bool recyclable_log = (std::get<0>(GetParam()) != 0); + const size_t header_size = + recyclable_log ? kRecyclableHeaderSize : kHeaderSize; + const size_t data_len = kBlockSize - 2 * header_size; + + const auto first_str = BigString("foo", data_len); + Write(first_str); + + UnorderedMap ts_sz = { + {2, sizeof(uint64_t)}, + }; + ASSERT_OK( + writer_->MaybeAddUserDefinedTimestampSizeRecord(WriteOptions(), ts_sz)); + ASSERT_LT(writer_->TEST_block_offset(), kBlockSize); + + const auto second_str = BigString("bar", 1000); + Write(second_str); + + ASSERT_EQ(first_str, Read()); + CheckRecordAndTimestampSize(second_str, ts_sz); +} + // Do NOT enable compression for this instantiation. INSTANTIATE_TEST_CASE_P( Log, LogTest, @@ -1131,6 +1157,42 @@ TEST_P(CompressionLogTest, AlignedFragmentation) { ASSERT_EQ("EOF", Read()); } +TEST_P(CompressionLogTest, ChecksumMismatch) { + const CompressionType kCompressionType = std::get<2>(GetParam()); + const bool kCompressionEnabled = kCompressionType != kNoCompression; + const bool kRecyclableLog = (std::get<0>(GetParam()) != 0); + if (!StreamingCompressionTypeSupported(kCompressionType)) { + ROCKSDB_GTEST_SKIP("Test requires support for compression type"); + return; + } + ASSERT_OK(SetupTestEnv()); + + Write("foooooo"); + int header_len; + if (kRecyclableLog) { + header_len = kRecyclableHeaderSize; + } else { + header_len = kHeaderSize; + } + int compression_record_len; + if (kCompressionEnabled) { + compression_record_len = header_len + 4; + } else { + compression_record_len = 0; + } + IncrementByte(compression_record_len + header_len /* offset */, + 14 /* delta */); + + ASSERT_EQ("EOF", Read()); + if (!kRecyclableLog) { + ASSERT_GT(DroppedBytes(), 0U); + ASSERT_EQ("OK", MatchError("checksum mismatch")); + } else { + ASSERT_EQ(0U, DroppedBytes()); + ASSERT_EQ("", ReportMessage()); + } +} + INSTANTIATE_TEST_CASE_P( Compression, CompressionLogTest, ::testing::Combine(::testing::Values(0, 1), ::testing::Bool(), diff --git a/db/log_writer.cc b/db/log_writer.cc index e61efc9eefc..2cd6bbd788c 100644 --- a/db/log_writer.cc +++ b/db/log_writer.cc @@ -27,6 +27,8 @@ Writer::Writer(std::unique_ptr&& dest, uint64_t log_number, block_offset_(0), log_number_(log_number), recycle_log_files_(recycle_log_files), + // Header size varies depending on whether we are recycling or not. + header_size_(recycle_log_files ? kRecyclableHeaderSize : kHeaderSize), manual_flush_(manual_flush), compression_type_(compression_type), compress_(nullptr) { @@ -80,10 +82,6 @@ IOStatus Writer::AddRecord(const WriteOptions& write_options, const char* ptr = slice.data(); size_t left = slice.size(); - // Header size varies depending on whether we are recycling or not. - const int header_size = - recycle_log_files_ ? kRecyclableHeaderSize : kHeaderSize; - // Fragment the record if necessary and emit it. Note that if slice // is empty, we still want to iterate once to emit a single // zero-length record @@ -102,12 +100,12 @@ IOStatus Writer::AddRecord(const WriteOptions& write_options, do { const int64_t leftover = kBlockSize - block_offset_; assert(leftover >= 0); - if (leftover < header_size) { + if (leftover < header_size_) { // Switch to a new block if (leftover > 0) { // Fill the trailer (literal below relies on kHeaderSize and // kRecyclableHeaderSize being <= 11) - assert(header_size <= 11); + assert(header_size_ <= 11); s = dest_->Append(opts, Slice("\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00", static_cast(leftover)), @@ -120,9 +118,9 @@ IOStatus Writer::AddRecord(const WriteOptions& write_options, } // Invariant: we never leave < header_size bytes in a block. - assert(static_cast(kBlockSize - block_offset_) >= header_size); + assert(static_cast(kBlockSize - block_offset_) >= header_size_); - const size_t avail = kBlockSize - block_offset_ - header_size; + const size_t avail = kBlockSize - block_offset_ - header_size_; // Compress the record if compression is enabled. // Compress() is called at least once (compress_start=true) and after the @@ -203,8 +201,7 @@ IOStatus Writer::AddCompressionTypeRecord(const WriteOptions& write_options) { } } // Initialize fields required for compression - const size_t max_output_buffer_len = - kBlockSize - (recycle_log_files_ ? kRecyclableHeaderSize : kHeaderSize); + const size_t max_output_buffer_len = kBlockSize - header_size_; CompressionOptions opts; constexpr uint32_t compression_format_version = 2; compress_ = StreamingCompress::Create(compression_type_, opts, @@ -244,6 +241,25 @@ IOStatus Writer::MaybeAddUserDefinedTimestampSizeRecord( record.EncodeTo(&encoded); RecordType type = recycle_log_files_ ? kRecyclableUserDefinedTimestampSizeType : kUserDefinedTimestampSizeType; + + // If there's not enough space for this record, switch to a new block. + const int64_t leftover = kBlockSize - block_offset_; + if (leftover < header_size_ + (int)encoded.size()) { + IOOptions opts; + IOStatus s = WritableFileWriter::PrepareIOOptions(write_options, opts); + if (!s.ok()) { + return s; + } + + std::vector trailer(leftover, '\x00'); + s = dest_->Append(opts, Slice(trailer.data(), trailer.size())); + if (!s.ok()) { + return s; + } + + block_offset_ = 0; + } + return EmitPhysicalRecord(write_options, type, encoded.data(), encoded.size()); } diff --git a/db/log_writer.h b/db/log_writer.h index 1bbf72569ec..48fd3db7c28 100644 --- a/db/log_writer.h +++ b/db/log_writer.h @@ -109,11 +109,14 @@ class Writer { bool BufferIsEmpty(); + size_t TEST_block_offset() const { return block_offset_; } + private: std::unique_ptr dest_; size_t block_offset_; // Current offset in block uint64_t log_number_; bool recycle_log_files_; + int header_size_; // crc32c values for all supported record types. These are // pre-computed to reduce the overhead of computing the crc of the diff --git a/db/memtable.cc b/db/memtable.cc index c3a6433dc14..91080673ffa 100644 --- a/db/memtable.cc +++ b/db/memtable.cc @@ -13,6 +13,7 @@ #include #include #include +#include #include "db/dbformat.h" #include "db/kv_checksum.h" @@ -928,21 +929,10 @@ struct Saver { } // anonymous namespace static bool SaveValue(void* arg, const char* entry) { - TEST_SYNC_POINT_CALLBACK("Memtable::SaveValue:Begin:entry", &entry); Saver* s = static_cast(arg); assert(s != nullptr); assert(!s->value || !s->columns); - if (s->protection_bytes_per_key > 0) { - *(s->status) = MemTable::VerifyEntryChecksum( - entry, s->protection_bytes_per_key, s->allow_data_in_errors); - if (!s->status->ok()) { - ROCKS_LOG_ERROR(s->logger, "In SaveValue: %s", s->status->getState()); - // Memtable entry corrupted - return false; - } - } - MergeContext* merge_context = s->merge_context; SequenceNumber max_covering_tombstone_seq = s->max_covering_tombstone_seq; const MergeOperator* merge_operator = s->merge_operator; @@ -965,6 +955,22 @@ static bool SaveValue(void* arg, const char* entry) { if (user_comparator->EqualWithoutTimestamp(user_key_slice, s->key->user_key())) { // Correct user key + TEST_SYNC_POINT_CALLBACK("Memtable::SaveValue:Found:entry", &entry); + std::optional read_lock; + if (s->inplace_update_support) { + read_lock.emplace(s->mem->GetLock(s->key->user_key())); + } + + if (s->protection_bytes_per_key > 0) { + *(s->status) = MemTable::VerifyEntryChecksum( + entry, s->protection_bytes_per_key, s->allow_data_in_errors); + if (!s->status->ok()) { + ROCKS_LOG_ERROR(s->logger, "In SaveValue: %s", s->status->getState()); + // Memtable entry corrupted + return false; + } + } + const uint64_t tag = DecodeFixed64(key_ptr + key_length - 8); ValueType type; SequenceNumber seq; @@ -1035,10 +1041,6 @@ static bool SaveValue(void* arg, const char* entry) { return false; } - if (s->inplace_update_support) { - s->mem->GetLock(s->key->user_key())->ReadLock(); - } - Slice v = GetLengthPrefixedSlice(key_ptr + key_length); *(s->status) = Status::OK(); @@ -1049,10 +1051,6 @@ static bool SaveValue(void* arg, const char* entry) { s->columns->SetPlainValue(v); } - if (s->inplace_update_support) { - s->mem->GetLock(s->key->user_key())->ReadUnlock(); - } - *(s->found_final_value) = true; *(s->is_blob_index) = true; @@ -1060,10 +1058,6 @@ static bool SaveValue(void* arg, const char* entry) { } case kTypeValue: case kTypeValuePreferredSeqno: { - if (s->inplace_update_support) { - s->mem->GetLock(s->key->user_key())->ReadLock(); - } - Slice v = GetLengthPrefixedSlice(key_ptr + key_length); if (type == kTypeValuePreferredSeqno) { @@ -1100,10 +1094,6 @@ static bool SaveValue(void* arg, const char* entry) { s->columns->SetPlainValue(v); } - if (s->inplace_update_support) { - s->mem->GetLock(s->key->user_key())->ReadUnlock(); - } - *(s->found_final_value) = true; if (s->is_blob_index != nullptr) { @@ -1113,10 +1103,6 @@ static bool SaveValue(void* arg, const char* entry) { return false; } case kTypeWideColumnEntity: { - if (s->inplace_update_support) { - s->mem->GetLock(s->key->user_key())->ReadLock(); - } - Slice v = GetLengthPrefixedSlice(key_ptr + key_length); *(s->status) = Status::OK(); @@ -1158,10 +1144,6 @@ static bool SaveValue(void* arg, const char* entry) { *(s->status) = s->columns->SetWideColumnValue(v); } - if (s->inplace_update_support) { - s->mem->GetLock(s->key->user_key())->ReadUnlock(); - } - *(s->found_final_value) = true; if (s->is_blob_index != nullptr) { @@ -1499,9 +1481,9 @@ Status MemTable::Update(SequenceNumber seq, ValueType value_type, // Update value, if new value size <= previous value size if (new_size <= prev_size) { + WriteLock wl(GetLock(lkey.user_key())); char* p = EncodeVarint32(const_cast(key_ptr) + key_length, new_size); - WriteLock wl(GetLock(lkey.user_key())); memcpy(p, value.data(), value.size()); assert((unsigned)((p + value.size()) - entry) == (unsigned)(VarintLength(key_length) + key_length + diff --git a/db/range_tombstone_fragmenter.h b/db/range_tombstone_fragmenter.h index 8bdbf03be97..9c644cb6872 100644 --- a/db/range_tombstone_fragmenter.h +++ b/db/range_tombstone_fragmenter.h @@ -17,16 +17,6 @@ #include "table/internal_iterator.h" namespace ROCKSDB_NAMESPACE { -struct FragmentedRangeTombstoneList; - -struct FragmentedRangeTombstoneListCache { - // ensure only the first reader needs to initialize l - std::mutex reader_mutex; - std::unique_ptr tombstones = nullptr; - // readers will first check this bool to avoid - std::atomic initialized = false; -}; - struct FragmentedRangeTombstoneList { public: // A compact representation of a "stack" of range tombstone fragments, which @@ -124,6 +114,14 @@ struct FragmentedRangeTombstoneList { uint64_t total_tombstone_payload_bytes_; }; +struct FragmentedRangeTombstoneListCache { + // ensure only the first reader needs to initialize l + std::mutex reader_mutex; + std::unique_ptr tombstones = nullptr; + // readers will first check this bool to avoid + std::atomic initialized = false; +}; + // FragmentedRangeTombstoneIterator converts an InternalIterator of a range-del // meta block into an iterator over non-overlapping tombstone fragments. The // tombstone fragmentation process should be more efficient than the range diff --git a/db/transaction_log_impl.cc b/db/transaction_log_impl.cc index 8841b8cf3b2..bb8d6b0ffd1 100644 --- a/db/transaction_log_impl.cc +++ b/db/transaction_log_impl.cc @@ -18,7 +18,7 @@ TransactionLogIteratorImpl::TransactionLogIteratorImpl( const std::string& dir, const ImmutableDBOptions* options, const TransactionLogIterator::ReadOptions& read_options, const EnvOptions& soptions, const SequenceNumber seq, - std::unique_ptr files, VersionSet const* const versions, + std::unique_ptr files, VersionSet const* const versions, const bool seq_per_batch, const std::shared_ptr& io_tracer) : dir_(dir), options_(options), @@ -44,7 +44,7 @@ TransactionLogIteratorImpl::TransactionLogIteratorImpl( } Status TransactionLogIteratorImpl::OpenLogFile( - const LogFile* log_file, + const WalFile* log_file, std::unique_ptr* file_reader) { FileSystemPtr fs(options_->fs, io_tracer_); std::unique_ptr file; @@ -281,7 +281,7 @@ void TransactionLogIteratorImpl::UpdateCurrentWriteBatch(const Slice& record) { current_status_ = Status::OK(); } -Status TransactionLogIteratorImpl::OpenLogReader(const LogFile* log_file) { +Status TransactionLogIteratorImpl::OpenLogReader(const WalFile* log_file) { std::unique_ptr file; Status s = OpenLogFile(log_file, &file); if (!s.ok()) { diff --git a/db/transaction_log_impl.h b/db/transaction_log_impl.h index eb700036110..e8552a47d59 100644 --- a/db/transaction_log_impl.h +++ b/db/transaction_log_impl.h @@ -19,9 +19,9 @@ namespace ROCKSDB_NAMESPACE { -class LogFileImpl : public LogFile { +class WalFileImpl : public WalFile { public: - LogFileImpl(uint64_t logNum, WalFileType logType, SequenceNumber startSeq, + WalFileImpl(uint64_t logNum, WalFileType logType, SequenceNumber startSeq, uint64_t sizeBytes) : logNumber_(logNum), type_(logType), @@ -43,7 +43,7 @@ class LogFileImpl : public LogFile { uint64_t SizeFileBytes() const override { return sizeFileBytes_; } - bool operator<(const LogFile& that) const { + bool operator<(const WalFile& that) const { return LogNumber() < that.LogNumber(); } @@ -60,7 +60,7 @@ class TransactionLogIteratorImpl : public TransactionLogIterator { const std::string& dir, const ImmutableDBOptions* options, const TransactionLogIterator::ReadOptions& read_options, const EnvOptions& soptions, const SequenceNumber seqNum, - std::unique_ptr files, VersionSet const* const versions, + std::unique_ptr files, VersionSet const* const versions, const bool seq_per_batch, const std::shared_ptr& io_tracer); bool Valid() override; @@ -77,7 +77,7 @@ class TransactionLogIteratorImpl : public TransactionLogIterator { const TransactionLogIterator::ReadOptions read_options_; const EnvOptions& soptions_; SequenceNumber starting_sequence_number_; - std::unique_ptr files_; + std::unique_ptr files_; // Used only to get latest seq. num // TODO(icanadi) can this be just a callback? VersionSet const* const versions_; @@ -92,7 +92,7 @@ class TransactionLogIteratorImpl : public TransactionLogIterator { std::unique_ptr current_batch_; std::unique_ptr current_log_reader_; std::string scratch_; - Status OpenLogFile(const LogFile* log_file, + Status OpenLogFile(const WalFile* log_file, std::unique_ptr* file); struct LogReporter : public log::Reader::Reporter { @@ -123,6 +123,6 @@ class TransactionLogIteratorImpl : public TransactionLogIterator { bool IsBatchExpected(const WriteBatch* batch, SequenceNumber expected_seq); // Update current batch if a continuous batch is found. void UpdateCurrentWriteBatch(const Slice& record); - Status OpenLogReader(const LogFile* file); + Status OpenLogReader(const WalFile* file); }; } // namespace ROCKSDB_NAMESPACE diff --git a/db/version_edit_handler.cc b/db/version_edit_handler.cc index 42bdcd3c33a..3284768046a 100644 --- a/db/version_edit_handler.cc +++ b/db/version_edit_handler.cc @@ -152,7 +152,7 @@ Status FileChecksumRetriever::ApplyVersionEdit(VersionEdit& edit, VersionEditHandler::VersionEditHandler( bool read_only, std::vector column_families, - VersionSet* version_set, bool track_missing_files, + VersionSet* version_set, bool track_found_and_missing_files, bool no_error_if_files_missing, const std::shared_ptr& io_tracer, const ReadOptions& read_options, bool skip_load_table_files, EpochNumberRequirement epoch_number_requirement) @@ -160,7 +160,7 @@ VersionEditHandler::VersionEditHandler( read_only_(read_only), column_families_(std::move(column_families)), version_set_(version_set), - track_missing_files_(track_missing_files), + track_found_and_missing_files_(track_found_and_missing_files), no_error_if_files_missing_(no_error_if_files_missing), io_tracer_(io_tracer), skip_load_table_files_(skip_load_table_files), @@ -500,7 +500,8 @@ ColumnFamilyData* VersionEditHandler::CreateCfAndInit( assert(builders_.find(cf_id) == builders_.end()); builders_.emplace(cf_id, VersionBuilderUPtr(new BaseReferencedVersionBuilder(cfd))); - if (track_missing_files_) { + if (track_found_and_missing_files_) { + cf_to_found_files_.emplace(cf_id, std::unordered_set()); cf_to_missing_files_.emplace(cf_id, std::unordered_set()); cf_to_missing_blob_files_high_.emplace(cf_id, kInvalidBlobFileNumber); } @@ -513,7 +514,11 @@ ColumnFamilyData* VersionEditHandler::DestroyCfAndCleanup( auto builder_iter = builders_.find(cf_id); assert(builder_iter != builders_.end()); builders_.erase(builder_iter); - if (track_missing_files_) { + if (track_found_and_missing_files_) { + auto found_files_iter = cf_to_found_files_.find(cf_id); + assert(found_files_iter != cf_to_found_files_.end()); + cf_to_found_files_.erase(found_files_iter); + auto missing_files_iter = cf_to_missing_files_.find(cf_id); assert(missing_files_iter != cf_to_missing_files_.end()); cf_to_missing_files_.erase(missing_files_iter); @@ -729,7 +734,7 @@ VersionEditHandlerPointInTime::VersionEditHandlerPointInTime( const ReadOptions& read_options, EpochNumberRequirement epoch_number_requirement) : VersionEditHandler(read_only, column_families, version_set, - /*track_missing_files=*/true, + /*track_found_and_missing_files=*/true, /*no_error_if_files_missing=*/true, io_tracer, read_options, epoch_number_requirement) {} @@ -824,6 +829,12 @@ void VersionEditHandlerPointInTime::CheckIterationResult( version_set_->AppendVersion(cfd, v_iter->second); versions_.erase(v_iter); + // Let's clear found_files, since any files in that are part of the + // installed Version. Any files that got obsoleted would have already + // been moved to intermediate_files_ + auto found_files_iter = cf_to_found_files_.find(cfd->GetID()); + assert(found_files_iter != cf_to_found_files_.end()); + found_files_iter->second.clear(); } } } else { @@ -854,10 +865,16 @@ ColumnFamilyData* VersionEditHandlerPointInTime::DestroyCfAndCleanup( Status VersionEditHandlerPointInTime::MaybeCreateVersion( const VersionEdit& edit, ColumnFamilyData* cfd, bool force_create_version) { + TEST_SYNC_POINT("VersionEditHandlerPointInTime::MaybeCreateVersion:Begin1"); + TEST_SYNC_POINT("VersionEditHandlerPointInTime::MaybeCreateVersion:Begin2"); assert(cfd != nullptr); if (!force_create_version) { assert(edit.GetColumnFamily() == cfd->GetID()); } + auto found_files_iter = cf_to_found_files_.find(cfd->GetID()); + assert(found_files_iter != cf_to_found_files_.end()); + std::unordered_set& found_files = found_files_iter->second; + auto missing_files_iter = cf_to_missing_files_.find(cfd->GetID()); assert(missing_files_iter != cf_to_missing_files_.end()); std::unordered_set& missing_files = missing_files_iter->second; @@ -889,6 +906,18 @@ Status VersionEditHandlerPointInTime::MaybeCreateVersion( auto fiter = missing_files.find(file_num); if (fiter != missing_files.end()) { missing_files.erase(fiter); + } else { + fiter = found_files.find(file_num); + // Only mark new files added during this catchup attempt for deletion. + // These files were never installed in VersionStorageInfo. + // Already referenced files that are deleted by a VersionEdit will + // be added to the VersionStorageInfo's obsolete files when the old + // version is dereferenced. + if (fiter != found_files.end()) { + intermediate_files_.emplace_back( + MakeTableFileName(cfd->ioptions()->cf_paths[0].path, file_num)); + found_files.erase(fiter); + } } } @@ -904,9 +933,14 @@ Status VersionEditHandlerPointInTime::MaybeCreateVersion( s = VerifyFile(cfd, fpath, level, meta); if (s.IsPathNotFound() || s.IsNotFound() || s.IsCorruption()) { missing_files.insert(file_num); + if (s.IsCorruption()) { + found_files.insert(file_num); + } s = Status::OK(); } else if (!s.ok()) { break; + } else { + found_files.insert(file_num); } } diff --git a/db/version_edit_handler.h b/db/version_edit_handler.h index 4caa9c08988..e19ead31df3 100644 --- a/db/version_edit_handler.h +++ b/db/version_edit_handler.h @@ -104,7 +104,7 @@ using VersionBuilderUPtr = std::unique_ptr; // To use this class and its subclasses, // 1. Create an object of VersionEditHandler or its subclasses. // VersionEditHandler handler(read_only, column_families, version_set, -// track_missing_files, +// track_found_and_missing_files, // no_error_if_files_missing); // 2. Status s = handler.Iterate(reader, &db_id); // 3. Check s and handle possible errors. @@ -116,16 +116,17 @@ class VersionEditHandler : public VersionEditHandlerBase { explicit VersionEditHandler( bool read_only, const std::vector& column_families, - VersionSet* version_set, bool track_missing_files, + VersionSet* version_set, bool track_found_and_missing_files, bool no_error_if_files_missing, const std::shared_ptr& io_tracer, const ReadOptions& read_options, EpochNumberRequirement epoch_number_requirement = EpochNumberRequirement::kMustPresent) - : VersionEditHandler( - read_only, column_families, version_set, track_missing_files, - no_error_if_files_missing, io_tracer, read_options, - /*skip_load_table_files=*/false, epoch_number_requirement) {} + : VersionEditHandler(read_only, column_families, version_set, + track_found_and_missing_files, + no_error_if_files_missing, io_tracer, read_options, + /*skip_load_table_files=*/false, + epoch_number_requirement) {} ~VersionEditHandler() override {} @@ -144,7 +145,7 @@ class VersionEditHandler : public VersionEditHandlerBase { protected: explicit VersionEditHandler( bool read_only, std::vector column_families, - VersionSet* version_set, bool track_missing_files, + VersionSet* version_set, bool track_found_and_missing_files, bool no_error_if_files_missing, const std::shared_ptr& io_tracer, const ReadOptions& read_options, bool skip_load_table_files, @@ -195,7 +196,8 @@ class VersionEditHandler : public VersionEditHandlerBase { // by subsequent manifest records, Recover() will return failure status. std::unordered_map column_families_not_found_; VersionEditParams version_edit_params_; - const bool track_missing_files_; + const bool track_found_and_missing_files_; + std::unordered_map> cf_to_found_files_; std::unordered_map> cf_to_missing_files_; std::unordered_map cf_to_missing_blob_files_high_; @@ -273,6 +275,8 @@ class VersionEditHandlerPointInTime : public VersionEditHandler { bool in_atomic_group_ = false; + std::vector intermediate_files_; + private: bool AtomicUpdateVersionsCompleted(); bool AtomicUpdateVersionsContains(uint32_t cfid); @@ -310,6 +314,10 @@ class ManifestTailer : public VersionEditHandlerPointInTime { return cfds_changed_; } + std::vector& GetIntermediateFiles() { + return intermediate_files_; + } + protected: Status Initialize() override; @@ -342,7 +350,7 @@ class DumpManifestHandler : public VersionEditHandler { bool json) : VersionEditHandler( /*read_only=*/true, column_families, version_set, - /*track_missing_files=*/false, + /*track_found_and_missing_files=*/false, /*no_error_if_files_missing=*/false, io_tracer, read_options, /*skip_load_table_files=*/true), verbose_(verbose), @@ -356,14 +364,15 @@ class DumpManifestHandler : public VersionEditHandler { Status ApplyVersionEdit(VersionEdit& edit, ColumnFamilyData** cfd) override { // Write out each individual edit - if (verbose_ && !json_) { + if (json_) { // Print out DebugStrings. Can include non-terminating null characters. - fwrite(edit.DebugString(hex_).data(), sizeof(char), - edit.DebugString(hex_).size(), stdout); - } else if (json_) { + std::string edit_dump_str = edit.DebugJSON(count_, hex_); + fwrite(edit_dump_str.data(), sizeof(char), edit_dump_str.size(), stdout); + fwrite("\n", sizeof(char), 1, stdout); + } else if (verbose_) { // Print out DebugStrings. Can include non-terminating null characters. - fwrite(edit.DebugString(hex_).data(), sizeof(char), - edit.DebugString(hex_).size(), stdout); + std::string edit_dump_str = edit.DebugString(hex_); + fwrite(edit_dump_str.data(), sizeof(char), edit_dump_str.size(), stdout); } ++count_; return VersionEditHandler::ApplyVersionEdit(edit, cfd); diff --git a/db/version_set.cc b/db/version_set.cc index 031a8de17c4..a8c575f5a23 100644 --- a/db/version_set.cc +++ b/db/version_set.cc @@ -3130,6 +3130,10 @@ bool Version::MaybeInitializeFileMetaData(const ReadOptions& read_options, file_meta->raw_value_size = tp->raw_value_size; file_meta->raw_key_size = tp->raw_key_size; file_meta->num_range_deletions = tp->num_range_deletions; + // Ensure new invariants on old files + file_meta->num_deletions = + std::max(tp->num_deletions, tp->num_range_deletions); + file_meta->num_entries = std::max(tp->num_entries, tp->num_deletions); return true; } @@ -3141,6 +3145,7 @@ void VersionStorageInfo::UpdateAccumulatedStats(FileMetaData* file_meta) { accumulated_file_size_ += file_meta->fd.GetFileSize(); accumulated_raw_key_size_ += file_meta->raw_key_size; accumulated_raw_value_size_ += file_meta->raw_value_size; + assert(file_meta->num_entries >= file_meta->num_deletions); accumulated_num_non_deletions_ += file_meta->num_entries - file_meta->num_deletions; accumulated_num_deletions_ += file_meta->num_deletions; @@ -3496,6 +3501,10 @@ void VersionStorageInfo::ComputeCompactionScore( score = kScoreForNeedCompaction; } } else { + // For universal compaction, if a user configures `max_read_amp`, then + // the score may be a false positive signal. + // `level0_file_num_compaction_trigger` is used as a trigger to check + // if there is any compaction work to do. score = static_cast(num_sorted_runs) / mutable_cf_options.level0_file_num_compaction_trigger; if (compaction_style_ == kCompactionStyleLevel && num_levels() > 1) { @@ -4612,25 +4621,27 @@ uint64_t VersionStorageInfo::GetMaxEpochNumberOfFiles() const { return max_epoch_number; } -void VersionStorageInfo::RecoverEpochNumbers(ColumnFamilyData* cfd) { - cfd->ResetNextEpochNumber(); +void VersionStorageInfo::RecoverEpochNumbers(ColumnFamilyData* cfd, + bool restart_epoch, bool force) { + if (restart_epoch) { + cfd->ResetNextEpochNumber(); - bool reserve_epoch_num_for_file_ingested_behind = - cfd->ioptions()->allow_ingest_behind; - if (reserve_epoch_num_for_file_ingested_behind) { - uint64_t reserved_epoch_number = cfd->NewEpochNumber(); - assert(reserved_epoch_number == kReservedEpochNumberForFileIngestedBehind); - ROCKS_LOG_INFO(cfd->ioptions()->info_log.get(), - "[%s]CF has reserved epoch number %" PRIu64 - " for files ingested " - "behind since `Options::allow_ingest_behind` is true", - cfd->GetName().c_str(), reserved_epoch_number); + bool reserve_epoch_num_for_file_ingested_behind = + cfd->ioptions()->allow_ingest_behind; + if (reserve_epoch_num_for_file_ingested_behind) { + uint64_t reserved_epoch_number = cfd->NewEpochNumber(); + assert(reserved_epoch_number == + kReservedEpochNumberForFileIngestedBehind); + ROCKS_LOG_INFO(cfd->ioptions()->info_log.get(), + "[%s]CF has reserved epoch number %" PRIu64 + " for files ingested " + "behind since `Options::allow_ingest_behind` is true", + cfd->GetName().c_str(), reserved_epoch_number); + } } - if (HasMissingEpochNumber()) { - assert(epoch_number_requirement_ == EpochNumberRequirement::kMightMissing); - assert(num_levels_ >= 1); - + bool missing_epoch_number = HasMissingEpochNumber(); + if (missing_epoch_number || force) { for (int level = num_levels_ - 1; level >= 1; --level) { auto& files_at_level = files_[level]; if (files_at_level.empty()) { @@ -4641,17 +4652,19 @@ void VersionStorageInfo::RecoverEpochNumbers(ColumnFamilyData* cfd) { f->epoch_number = next_epoch_number; } } - for (auto file_meta_iter = files_[0].rbegin(); file_meta_iter != files_[0].rend(); file_meta_iter++) { FileMetaData* f = *file_meta_iter; f->epoch_number = cfd->NewEpochNumber(); } - - ROCKS_LOG_WARN(cfd->ioptions()->info_log.get(), - "[%s]CF's epoch numbers are inferred based on seqno", - cfd->GetName().c_str()); - epoch_number_requirement_ = EpochNumberRequirement::kMustPresent; + if (missing_epoch_number) { + assert(epoch_number_requirement_ == + EpochNumberRequirement::kMightMissing); + ROCKS_LOG_WARN(cfd->ioptions()->info_log.get(), + "[%s]CF's epoch numbers are inferred based on seqno", + cfd->GetName().c_str()); + epoch_number_requirement_ = EpochNumberRequirement::kMustPresent; + } } else { assert(epoch_number_requirement_ == EpochNumberRequirement::kMustPresent); cfd->SetNextEpochNumber( @@ -6054,8 +6067,8 @@ Status VersionSet::Recover( true /* checksum */, 0 /* log_number */); VersionEditHandler handler( read_only, column_families, const_cast(this), - /*track_missing_files=*/false, no_error_if_files_missing, io_tracer_, - read_options, EpochNumberRequirement::kMightMissing); + /*track_found_and_missing_files=*/false, no_error_if_files_missing, + io_tracer_, read_options, EpochNumberRequirement::kMightMissing); handler.Iterate(reader, &log_read_status); s = handler.status(); if (s.ok()) { @@ -7430,7 +7443,8 @@ Status ReactiveVersionSet::ReadAndApply( InstrumentedMutex* mu, std::unique_ptr* manifest_reader, Status* manifest_read_status, - std::unordered_set* cfds_changed) { + std::unordered_set* cfds_changed, + std::vector* files_to_delete) { assert(manifest_reader != nullptr); assert(cfds_changed != nullptr); mu->AssertHeld(); @@ -7447,6 +7461,9 @@ Status ReactiveVersionSet::ReadAndApply( if (s.ok()) { *cfds_changed = std::move(manifest_tailer_->GetUpdatedColumnFamilies()); } + if (files_to_delete) { + *files_to_delete = std::move(manifest_tailer_->GetIntermediateFiles()); + } return s; } diff --git a/db/version_set.h b/db/version_set.h index 82e6211f450..ba86878d406 100644 --- a/db/version_set.h +++ b/db/version_set.h @@ -341,7 +341,15 @@ class VersionStorageInfo { EpochNumberRequirement epoch_number_requirement) { epoch_number_requirement_ = epoch_number_requirement; } - void RecoverEpochNumbers(ColumnFamilyData* cfd); + // Ensure all files have epoch number set. + // If there is a file missing epoch number, all files' epoch number will be + // reset according to CF's epoch number. Otherwise, the CF will be updated + // with the max epoch number of the files. + // + // @param restart_epoch This CF's epoch number will be reset to start from 0. + // @param force Force resetting all files' epoch number. + void RecoverEpochNumbers(ColumnFamilyData* cfd, bool restart_epoch = true, + bool force = false); class FileLocation { public: @@ -1733,7 +1741,8 @@ class ReactiveVersionSet : public VersionSet { InstrumentedMutex* mu, std::unique_ptr* manifest_reader, Status* manifest_read_status, - std::unordered_set* cfds_changed); + std::unordered_set* cfds_changed, + std::vector* files_to_delete); Status Recover(const std::vector& column_families, std::unique_ptr* manifest_reader, diff --git a/db/version_set_test.cc b/db/version_set_test.cc index f6a983d6b21..d4b748db7b9 100644 --- a/db/version_set_test.cc +++ b/db/version_set_test.cc @@ -2742,7 +2742,8 @@ TEST_F(VersionSetAtomicGroupTest, std::unordered_set cfds_changed; mu.Lock(); EXPECT_OK(reactive_versions_->ReadAndApply( - &mu, &manifest_reader, manifest_reader_status.get(), &cfds_changed)); + &mu, &manifest_reader, manifest_reader_status.get(), &cfds_changed, + /*files_to_delete=*/nullptr)); mu.Unlock(); EXPECT_TRUE(first_in_atomic_group_); EXPECT_TRUE(last_in_atomic_group_); @@ -2797,7 +2798,8 @@ TEST_F(VersionSetAtomicGroupTest, std::unordered_set cfds_changed; mu.Lock(); EXPECT_OK(reactive_versions_->ReadAndApply( - &mu, &manifest_reader, manifest_reader_status.get(), &cfds_changed)); + &mu, &manifest_reader, manifest_reader_status.get(), &cfds_changed, + /*files_to_delete=*/nullptr)); mu.Unlock(); // Reactive version set should be empty now. EXPECT_TRUE(reactive_versions_->TEST_read_edits_in_atomic_group() == 0); @@ -2826,7 +2828,8 @@ TEST_F(VersionSetAtomicGroupTest, std::unordered_set cfds_changed; mu.Lock(); EXPECT_OK(reactive_versions_->ReadAndApply( - &mu, &manifest_reader, manifest_reader_status.get(), &cfds_changed)); + &mu, &manifest_reader, manifest_reader_status.get(), &cfds_changed, + /*files_to_delete=*/nullptr)); mu.Unlock(); EXPECT_TRUE(first_in_atomic_group_); EXPECT_FALSE(last_in_atomic_group_); @@ -2882,7 +2885,8 @@ TEST_F(VersionSetAtomicGroupTest, AddNewEditsToLog(kAtomicGroupSize); mu.Lock(); EXPECT_NOK(reactive_versions_->ReadAndApply( - &mu, &manifest_reader, manifest_reader_status.get(), &cfds_changed)); + &mu, &manifest_reader, manifest_reader_status.get(), &cfds_changed, + /*files_to_delete=*/nullptr)); mu.Unlock(); EXPECT_EQ(edits_[kAtomicGroupSize / 2].DebugString(), corrupted_edit_.DebugString()); @@ -2932,7 +2936,8 @@ TEST_F(VersionSetAtomicGroupTest, AddNewEditsToLog(kAtomicGroupSize); mu.Lock(); EXPECT_NOK(reactive_versions_->ReadAndApply( - &mu, &manifest_reader, manifest_reader_status.get(), &cfds_changed)); + &mu, &manifest_reader, manifest_reader_status.get(), &cfds_changed, + /*files_to_delete=*/nullptr)); mu.Unlock(); EXPECT_EQ(edits_[1].DebugString(), edit_with_incorrect_group_size_.DebugString()); diff --git a/db/wal_manager.cc b/db/wal_manager.cc index 5f33cc8a6d2..15f72d00e4c 100644 --- a/db/wal_manager.cc +++ b/db/wal_manager.cc @@ -44,13 +44,13 @@ Status WalManager::DeleteFile(const std::string& fname, uint64_t number) { return s; } -Status WalManager::GetSortedWalFiles(VectorLogPtr& files) { +Status WalManager::GetSortedWalFiles(VectorWalPtr& files) { // First get sorted files in db dir, then get sorted files from archived // dir, to avoid a race condition where a log file is moved to archived // dir in between. Status s; // list wal files in main db dir. - VectorLogPtr logs; + VectorWalPtr logs; s = GetSortedWalsOfType(wal_dir_, logs, kAliveLogFile); if (!s.ok()) { return s; @@ -113,7 +113,7 @@ Status WalManager::GetUpdatesSince( // Get all sorted Wal Files. // Do binary search and open files and find the seq number. - std::unique_ptr wal_files(new VectorLogPtr); + std::unique_ptr wal_files(new VectorWalPtr); Status s = GetSortedWalFiles(*wal_files); if (!s.ok()) { return s; @@ -249,7 +249,7 @@ void WalManager::PurgeObsoleteWALFiles() { } size_t files_del_num = log_files_num - files_keep_num; - VectorLogPtr archived_logs; + VectorWalPtr archived_logs; s = GetSortedWalsOfType(archival_dir, archived_logs, kArchivedLogFile); if (!s.ok()) { ROCKS_LOG_WARN(db_options_.info_log, @@ -291,7 +291,7 @@ void WalManager::ArchiveWALFile(const std::string& fname, uint64_t number) { } Status WalManager::GetSortedWalsOfType(const std::string& path, - VectorLogPtr& log_files, + VectorWalPtr& log_files, WalFileType log_type) { std::vector all_files; const Status status = env_->GetChildren(path, &all_files); @@ -358,20 +358,20 @@ Status WalManager::GetSortedWalsOfType(const std::string& path, } log_files.emplace_back( - new LogFileImpl(number, log_type, sequence, size_bytes)); + new WalFileImpl(number, log_type, sequence, size_bytes)); } } std::sort( log_files.begin(), log_files.end(), - [](const std::unique_ptr& a, const std::unique_ptr& b) { - LogFileImpl* a_impl = static_cast_with_check(a.get()); - LogFileImpl* b_impl = static_cast_with_check(b.get()); + [](const std::unique_ptr& a, const std::unique_ptr& b) { + WalFileImpl* a_impl = static_cast_with_check(a.get()); + WalFileImpl* b_impl = static_cast_with_check(b.get()); return *a_impl < *b_impl; }); return status; } -Status WalManager::RetainProbableWalFiles(VectorLogPtr& all_logs, +Status WalManager::RetainProbableWalFiles(VectorWalPtr& all_logs, const SequenceNumber target) { int64_t start = 0; // signed to avoid overflow when target is < first file. int64_t end = static_cast(all_logs.size()) - 1; @@ -444,7 +444,7 @@ Status WalManager::ReadFirstRecord(const WalFileType type, } Status WalManager::GetLiveWalFile(uint64_t number, - std::unique_ptr* log_file) { + std::unique_ptr* log_file) { if (!log_file) { return Status::InvalidArgument("log_file not preallocated."); } @@ -462,7 +462,7 @@ Status WalManager::GetLiveWalFile(uint64_t number, return s; } - log_file->reset(new LogFileImpl(number, kAliveLogFile, + log_file->reset(new WalFileImpl(number, kAliveLogFile, 0, // SequenceNumber size_bytes)); diff --git a/db/wal_manager.h b/db/wal_manager.h index d8acba8afa3..64af8e6a042 100644 --- a/db/wal_manager.h +++ b/db/wal_manager.h @@ -49,7 +49,7 @@ class WalManager { wal_in_db_path_(db_options_.IsWalDirSameAsDBPath()), io_tracer_(io_tracer) {} - Status GetSortedWalFiles(VectorLogPtr& files); + Status GetSortedWalFiles(VectorWalPtr& files); // Allow user to tail transaction log to find all recent changes to the // database that are newer than `seq_number`. @@ -64,7 +64,7 @@ class WalManager { Status DeleteFile(const std::string& fname, uint64_t number); - Status GetLiveWalFile(uint64_t number, std::unique_ptr* log_file); + Status GetLiveWalFile(uint64_t number, std::unique_ptr* log_file); Status TEST_ReadFirstRecord(const WalFileType type, const uint64_t number, SequenceNumber* sequence) { @@ -77,12 +77,12 @@ class WalManager { } private: - Status GetSortedWalsOfType(const std::string& path, VectorLogPtr& log_files, + Status GetSortedWalsOfType(const std::string& path, VectorWalPtr& log_files, WalFileType type); // Requires: all_logs should be sorted with earliest log file first // Retains all log files in all_logs which contain updates with seq no. // Greater Than or Equal to the requested SequenceNumber. - Status RetainProbableWalFiles(VectorLogPtr& all_logs, + Status RetainProbableWalFiles(VectorWalPtr& all_logs, const SequenceNumber target); // ReadFirstRecord checks the read_first_record_cache_ to see if the entry diff --git a/db/wide/db_wide_basic_test.cc b/db/wide/db_wide_basic_test.cc index 6f3bc9be7ee..886f71d7452 100644 --- a/db/wide/db_wide_basic_test.cc +++ b/db/wide/db_wide_basic_test.cc @@ -1794,6 +1794,144 @@ TEST_F(DBWideBasicTest, PinnableWideColumnsMove) { test_move(/* fill_cache*/ true); } +TEST_F(DBWideBasicTest, SanityChecks) { + constexpr char foo[] = "foo"; + constexpr char bar[] = "bar"; + constexpr size_t num_keys = 2; + + { + constexpr ColumnFamilyHandle* column_family = nullptr; + PinnableWideColumns columns; + ASSERT_TRUE(db_->GetEntity(ReadOptions(), column_family, foo, &columns) + .IsInvalidArgument()); + } + + { + constexpr PinnableWideColumns* columns = nullptr; + ASSERT_TRUE( + db_->GetEntity(ReadOptions(), db_->DefaultColumnFamily(), foo, columns) + .IsInvalidArgument()); + } + + { + ReadOptions read_options; + read_options.io_activity = Env::IOActivity::kGet; + + PinnableWideColumns columns; + ASSERT_TRUE( + db_->GetEntity(read_options, db_->DefaultColumnFamily(), foo, &columns) + .IsInvalidArgument()); + } + + { + constexpr ColumnFamilyHandle* column_family = nullptr; + std::array keys{{foo, bar}}; + std::array results; + std::array statuses; + + db_->MultiGetEntity(ReadOptions(), column_family, num_keys, keys.data(), + results.data(), statuses.data()); + + ASSERT_TRUE(statuses[0].IsInvalidArgument()); + ASSERT_TRUE(statuses[1].IsInvalidArgument()); + } + + { + constexpr Slice* keys = nullptr; + std::array results; + std::array statuses; + + db_->MultiGetEntity(ReadOptions(), db_->DefaultColumnFamily(), num_keys, + keys, results.data(), statuses.data()); + + ASSERT_TRUE(statuses[0].IsInvalidArgument()); + ASSERT_TRUE(statuses[1].IsInvalidArgument()); + } + + { + std::array keys{{foo, bar}}; + constexpr PinnableWideColumns* results = nullptr; + std::array statuses; + + db_->MultiGetEntity(ReadOptions(), db_->DefaultColumnFamily(), num_keys, + keys.data(), results, statuses.data()); + + ASSERT_TRUE(statuses[0].IsInvalidArgument()); + ASSERT_TRUE(statuses[1].IsInvalidArgument()); + } + + { + ReadOptions read_options; + read_options.io_activity = Env::IOActivity::kMultiGet; + + std::array keys{{foo, bar}}; + std::array results; + std::array statuses; + + db_->MultiGetEntity(read_options, db_->DefaultColumnFamily(), num_keys, + keys.data(), results.data(), statuses.data()); + ASSERT_TRUE(statuses[0].IsInvalidArgument()); + ASSERT_TRUE(statuses[1].IsInvalidArgument()); + } + + { + constexpr ColumnFamilyHandle** column_families = nullptr; + std::array keys{{foo, bar}}; + std::array results; + std::array statuses; + + db_->MultiGetEntity(ReadOptions(), num_keys, column_families, keys.data(), + results.data(), statuses.data()); + + ASSERT_TRUE(statuses[0].IsInvalidArgument()); + ASSERT_TRUE(statuses[1].IsInvalidArgument()); + } + + { + std::array column_families{ + {db_->DefaultColumnFamily(), db_->DefaultColumnFamily()}}; + constexpr Slice* keys = nullptr; + std::array results; + std::array statuses; + + db_->MultiGetEntity(ReadOptions(), num_keys, column_families.data(), keys, + results.data(), statuses.data()); + + ASSERT_TRUE(statuses[0].IsInvalidArgument()); + ASSERT_TRUE(statuses[1].IsInvalidArgument()); + } + + { + std::array column_families{ + {db_->DefaultColumnFamily(), db_->DefaultColumnFamily()}}; + std::array keys{{foo, bar}}; + constexpr PinnableWideColumns* results = nullptr; + std::array statuses; + + db_->MultiGetEntity(ReadOptions(), num_keys, column_families.data(), + keys.data(), results, statuses.data()); + + ASSERT_TRUE(statuses[0].IsInvalidArgument()); + ASSERT_TRUE(statuses[1].IsInvalidArgument()); + } + + { + ReadOptions read_options; + read_options.io_activity = Env::IOActivity::kMultiGet; + + std::array column_families{ + {db_->DefaultColumnFamily(), db_->DefaultColumnFamily()}}; + std::array keys{{foo, bar}}; + std::array results; + std::array statuses; + + db_->MultiGetEntity(read_options, num_keys, column_families.data(), + keys.data(), results.data(), statuses.data()); + ASSERT_TRUE(statuses[0].IsInvalidArgument()); + ASSERT_TRUE(statuses[1].IsInvalidArgument()); + } +} + } // namespace ROCKSDB_NAMESPACE int main(int argc, char** argv) { diff --git a/db/wide/wide_columns_helper.cc b/db/wide/wide_columns_helper.cc index cf829ce7969..9f90d6b3181 100644 --- a/db/wide/wide_columns_helper.cc +++ b/db/wide/wide_columns_helper.cc @@ -6,6 +6,7 @@ #include "db/wide/wide_columns_helper.h" #include +#include #include "db/wide/wide_column_serialization.h" @@ -15,6 +16,9 @@ void WideColumnsHelper::DumpWideColumns(const WideColumns& columns, if (columns.empty()) { return; } + + const std::ios_base::fmtflags orig_flags = os.flags(); + if (hex) { os << std::hex; } @@ -23,6 +27,8 @@ void WideColumnsHelper::DumpWideColumns(const WideColumns& columns, for (++it; it != columns.end(); ++it) { os << ' ' << *it; } + + os.flags(orig_flags); } Status WideColumnsHelper::DumpSliceAsWideColumns(const Slice& value, diff --git a/db/write_batch.cc b/db/write_batch.cc index d58e1a46683..7294d9845f1 100644 --- a/db/write_batch.cc +++ b/db/write_batch.cc @@ -65,6 +65,7 @@ #include "port/lang.h" #include "rocksdb/merge_operator.h" #include "rocksdb/system_clock.h" +#include "util/aligned_storage.h" #include "util/autovector.h" #include "util/cast_util.h" #include "util/coding.h" @@ -1900,7 +1901,7 @@ class MemTableInserter : public WriteBatch::Handler { // Make creation optional but do not incur // std::unique_ptr additional allocation using MemPostInfoMap = std::map; - using PostMapType = std::aligned_storage::type; + using PostMapType = aligned_storage::type; PostMapType mem_post_info_map_; // current recovered transaction we are rebuilding (recovery) WriteBatch* rebuilding_trx_; @@ -1914,7 +1915,7 @@ class MemTableInserter : public WriteBatch::Handler { bool write_before_prepare_; // Whether this batch was unprepared or not bool unprepared_batch_; - using DupDetector = std::aligned_storage::type; + using DupDetector = aligned_storage::type; DupDetector duplicate_detector_; bool dup_dectector_on_; @@ -1922,7 +1923,7 @@ class MemTableInserter : public WriteBatch::Handler { bool hint_created_; // Hints for this batch using HintMap = std::unordered_map; - using HintMapType = std::aligned_storage::type; + using HintMapType = aligned_storage::type; HintMapType hint_; HintMap& GetHintMap() { @@ -2121,14 +2122,15 @@ class MemTableInserter : public WriteBatch::Handler { return true; } + template Status PutCFImpl(uint32_t column_family_id, const Slice& key, const Slice& value, ValueType value_type, + RebuildTxnOp rebuild_txn_op, const ProtectionInfoKVOS64* kv_prot_info) { // optimize for non-recovery mode if (UNLIKELY(write_after_commit_ && rebuilding_trx_ != nullptr)) { // TODO(ajkr): propagate `ProtectionInfoKVOS64`. - return WriteBatchInternal::Put(rebuilding_trx_, column_family_id, key, - value); + return rebuild_txn_op(rebuilding_trx_, column_family_id, key, value); // else insert the values to the memtable right away } @@ -2139,8 +2141,8 @@ class MemTableInserter : public WriteBatch::Handler { // The CF is probably flushed and hence no need for insert but we still // need to keep track of the keys for upcoming rollback/commit. // TODO(ajkr): propagate `ProtectionInfoKVOS64`. - ret_status = WriteBatchInternal::Put(rebuilding_trx_, column_family_id, - key, value); + ret_status = + rebuild_txn_op(rebuilding_trx_, column_family_id, key, value); if (ret_status.ok()) { MaybeAdvanceSeq(IsDuplicateKeySeq(column_family_id, key)); } @@ -2264,8 +2266,8 @@ class MemTableInserter : public WriteBatch::Handler { if (UNLIKELY(ret_status.ok() && rebuilding_trx_ != nullptr)) { assert(!write_after_commit_); // TODO(ajkr): propagate `ProtectionInfoKVOS64`. - ret_status = WriteBatchInternal::Put(rebuilding_trx_, column_family_id, - key, value); + ret_status = + rebuild_txn_op(rebuilding_trx_, column_family_id, key, value); } return ret_status; } @@ -2274,15 +2276,21 @@ class MemTableInserter : public WriteBatch::Handler { const Slice& value) override { const auto* kv_prot_info = NextProtectionInfo(); Status ret_status; + + auto rebuild_txn_op = [](WriteBatch* rebuilding_trx, uint32_t cf_id, + const Slice& k, const Slice& v) -> Status { + return WriteBatchInternal::Put(rebuilding_trx, cf_id, k, v); + }; + if (kv_prot_info != nullptr) { // Memtable needs seqno, doesn't need CF ID auto mem_kv_prot_info = kv_prot_info->StripC(column_family_id).ProtectS(sequence_); ret_status = PutCFImpl(column_family_id, key, value, kTypeValue, - &mem_kv_prot_info); + rebuild_txn_op, &mem_kv_prot_info); } else { ret_status = PutCFImpl(column_family_id, key, value, kTypeValue, - nullptr /* kv_prot_info */); + rebuild_txn_op, nullptr /* kv_prot_info */); } // TODO: this assumes that if TryAgain status is returned to the caller, // the operation is actually tried again. The proper way to do this is to @@ -2301,15 +2309,23 @@ class MemTableInserter : public WriteBatch::Handler { std::string value_buf; Slice packed_value = PackValueAndWriteTime(value, unix_write_time, &value_buf); + + auto rebuild_txn_op = [](WriteBatch* /* rebuilding_trx */, + uint32_t /* cf_id */, const Slice& /* k */, + const Slice& /* v */) -> Status { + return Status::NotSupported(); + }; + if (kv_prot_info != nullptr) { auto mem_kv_prot_info = kv_prot_info->StripC(column_family_id).ProtectS(sequence_); ret_status = PutCFImpl(column_family_id, key, packed_value, - kTypeValuePreferredSeqno, &mem_kv_prot_info); + kTypeValuePreferredSeqno, rebuild_txn_op, + &mem_kv_prot_info); } else { - ret_status = - PutCFImpl(column_family_id, key, packed_value, - kTypeValuePreferredSeqno, nullptr /* kv_prot_info */); + ret_status = PutCFImpl(column_family_id, key, packed_value, + kTypeValuePreferredSeqno, rebuild_txn_op, + nullptr /* kv_prot_info */); } // TODO: this assumes that if TryAgain status is returned to the caller, @@ -2327,14 +2343,27 @@ class MemTableInserter : public WriteBatch::Handler { const auto* kv_prot_info = NextProtectionInfo(); Status s; + + auto rebuild_txn_op = [](WriteBatch* rebuilding_trx, uint32_t cf_id, + const Slice& k, Slice entity) -> Status { + WideColumns columns; + const Status st = WideColumnSerialization::Deserialize(entity, columns); + if (!st.ok()) { + return st; + } + + return WriteBatchInternal::PutEntity(rebuilding_trx, cf_id, k, columns); + }; + if (kv_prot_info) { // Memtable needs seqno, doesn't need CF ID auto mem_kv_prot_info = kv_prot_info->StripC(column_family_id).ProtectS(sequence_); s = PutCFImpl(column_family_id, key, value, kTypeWideColumnEntity, - &mem_kv_prot_info); + rebuild_txn_op, &mem_kv_prot_info); } else { s = PutCFImpl(column_family_id, key, value, kTypeWideColumnEntity, + rebuild_txn_op, /* kv_prot_info */ nullptr); } @@ -2521,11 +2550,6 @@ 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(); @@ -2778,16 +2802,23 @@ class MemTableInserter : public WriteBatch::Handler { const Slice& value) override { const auto* kv_prot_info = NextProtectionInfo(); Status ret_status; + + auto rebuild_txn_op = [](WriteBatch* /* rebuilding_trx */, + uint32_t /* cf_id */, const Slice& /* k */, + const Slice& /* v */) -> Status { + return Status::NotSupported(); + }; + if (kv_prot_info != nullptr) { // Memtable needs seqno, doesn't need CF ID auto mem_kv_prot_info = kv_prot_info->StripC(column_family_id).ProtectS(sequence_); // Same as PutCF except for value type. ret_status = PutCFImpl(column_family_id, key, value, kTypeBlobIndex, - &mem_kv_prot_info); + rebuild_txn_op, &mem_kv_prot_info); } else { ret_status = PutCFImpl(column_family_id, key, value, kTypeBlobIndex, - nullptr /* kv_prot_info */); + rebuild_txn_op, nullptr /* kv_prot_info */); } if (UNLIKELY(ret_status.IsTryAgain())) { DecrementProtectionInfoIdxForTryAgain(); diff --git a/db/write_callback_test.cc b/db/write_callback_test.cc index c2f19c85a5e..c76720b2348 100644 --- a/db/write_callback_test.cc +++ b/db/write_callback_test.cc @@ -2,8 +2,6 @@ // 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/write_callback.h" #include @@ -15,6 +13,7 @@ #include "db/db_impl/db_impl.h" #include "port/port.h" #include "rocksdb/db.h" +#include "rocksdb/user_write_callback.h" #include "rocksdb/write_batch.h" #include "test_util/sync_point.h" #include "test_util/testharness.h" @@ -84,6 +83,28 @@ class MockWriteCallback : public WriteCallback { bool AllowWriteBatching() override { return allow_batching_; } }; +class MockUserWriteCallback : public UserWriteCallback { + public: + std::atomic write_enqueued_{false}; + std::atomic wal_write_done_{false}; + + MockUserWriteCallback() = default; + + MockUserWriteCallback(const MockUserWriteCallback& other) { + write_enqueued_.store(other.write_enqueued_.load()); + wal_write_done_.store(other.wal_write_done_.load()); + } + + void OnWriteEnqueued() override { write_enqueued_.store(true); } + + void OnWalWriteFinish() override { wal_write_done_.store(true); } + + void Reset() { + write_enqueued_.store(false); + wal_write_done_.store(false); + } +}; + #if !defined(ROCKSDB_VALGRIND_RUN) || defined(ROCKSDB_FULL_VALGRIND_RUN) class WriteCallbackPTest : public WriteCallbackTest, @@ -119,9 +140,11 @@ TEST_P(WriteCallbackPTest, WriteWithCallbackTest) { kvs_.clear(); write_batch_.Clear(); callback_.was_called_.store(false); + user_write_cb_.Reset(); } MockWriteCallback callback_; + MockUserWriteCallback user_write_cb_; WriteBatch write_batch_; std::vector> kvs_; }; @@ -327,18 +350,26 @@ TEST_P(WriteCallbackPTest, WriteWithCallbackTest) { ASSERT_OK(WriteBatchInternal::InsertNoop(&write_op.write_batch_)); const size_t ONE_BATCH = 1; s = db_impl->WriteImpl(woptions, &write_op.write_batch_, - &write_op.callback_, nullptr, 0, false, nullptr, - ONE_BATCH, + &write_op.callback_, &write_op.user_write_cb_, + nullptr, 0, false, nullptr, ONE_BATCH, two_queues_ ? &publish_seq_callback : nullptr); } else { s = db_impl->WriteWithCallback(woptions, &write_op.write_batch_, - &write_op.callback_); + &write_op.callback_, + &write_op.user_write_cb_); } + ASSERT_TRUE(write_op.user_write_cb_.write_enqueued_.load()); if (write_op.callback_.should_fail_) { ASSERT_TRUE(s.IsBusy()); + ASSERT_FALSE(write_op.user_write_cb_.wal_write_done_.load()); } else { ASSERT_OK(s); + if (enable_WAL_) { + ASSERT_TRUE(write_op.user_write_cb_.wal_write_done_.load()); + } else { + ASSERT_FALSE(write_op.user_write_cb_.wal_write_done_.load()); + } } }; @@ -440,6 +471,16 @@ TEST_F(WriteCallbackTest, WriteCallBackTest) { ASSERT_OK(s); ASSERT_EQ("value.a2", value); + MockUserWriteCallback user_write_cb; + WriteBatch wb4; + ASSERT_OK(wb4.Put("a", "value.a4")); + + ASSERT_OK(db->WriteWithCallback(write_options, &wb4, &user_write_cb)); + ASSERT_OK(db->Get(read_options, "a", &value)); + ASSERT_EQ(value, "value.a4"); + ASSERT_TRUE(user_write_cb.write_enqueued_.load()); + ASSERT_TRUE(user_write_cb.wal_write_done_.load()); + delete db; ASSERT_OK(DestroyDB(dbname, options)); } diff --git a/db/write_thread.cc b/db/write_thread.cc index 39f13c31875..87c82627249 100644 --- a/db/write_thread.cc +++ b/db/write_thread.cc @@ -404,6 +404,8 @@ void WriteThread::JoinBatchGroup(Writer* w) { bool linked_as_leader = LinkOne(w, &newest_writer_); + w->CheckWriteEnqueuedCallback(); + if (linked_as_leader) { SetState(w, STATE_GROUP_LEADER); } @@ -428,6 +430,7 @@ void WriteThread::JoinBatchGroup(Writer* w) { TEST_SYNC_POINT_CALLBACK("WriteThread::JoinBatchGroup:BeganWaiting", w); AwaitState(w, STATE_GROUP_LEADER | STATE_MEMTABLE_WRITER_LEADER | + STATE_PARALLEL_MEMTABLE_CALLER | STATE_PARALLEL_MEMTABLE_WRITER | STATE_COMPLETED, &jbg_ctx); TEST_SYNC_POINT_CALLBACK("WriteThread::JoinBatchGroup:DoneWaiting", w); @@ -656,12 +659,57 @@ void WriteThread::ExitAsMemTableWriter(Writer* /*self*/, SetState(leader, STATE_COMPLETED); } +void WriteThread::SetMemWritersEachStride(Writer* w) { + WriteGroup* write_group = w->write_group; + Writer* last_writer = write_group->last_writer; + + // The stride is the same for each writer in write_group, so w will + // call the writers with the same number in write_group mod total size + size_t stride = static_cast(std::sqrt(write_group->size)); + size_t count = 0; + while (w) { + if (count++ % stride == 0) { + SetState(w, STATE_PARALLEL_MEMTABLE_WRITER); + } + w = (w == last_writer) ? nullptr : w->link_newer; + } +} + void WriteThread::LaunchParallelMemTableWriters(WriteGroup* write_group) { assert(write_group != nullptr); - write_group->running.store(write_group->size); - for (auto w : *write_group) { - SetState(w, STATE_PARALLEL_MEMTABLE_WRITER); + size_t group_size = write_group->size; + write_group->running.store(group_size); + + // The minimum number to allow the group use parallel caller mode. + // The number must no lower than 3; + const size_t MinParallelSize = 20; + + // The group_size is too small, and there is no need to have + // the parallel partial callers. + if (group_size < MinParallelSize) { + for (auto w : *write_group) { + SetState(w, STATE_PARALLEL_MEMTABLE_WRITER); + } + return; } + + // The stride is equal to std::sqrt(group_size) which can minimize + // the total number of leader SetSate. + // Set the leader itself STATE_PARALLEL_MEMTABLE_WRITER, and set + // (stride-1) writers to be STATE_PARALLEL_MEMTABLE_CALLER. + size_t stride = static_cast(std::sqrt(group_size)); + auto w = write_group->leader; + SetState(w, STATE_PARALLEL_MEMTABLE_WRITER); + + for (size_t i = 1; i < stride; i++) { + w = w->link_newer; + SetState(w, STATE_PARALLEL_MEMTABLE_CALLER); + } + + // After setting all STATE_PARALLEL_MEMTABLE_CALLER, the leader also + // does the job as STATE_PARALLEL_MEMTABLE_CALLER. + w = w->link_newer; + SetMemWritersEachStride(w); } static WriteThread::AdaptationContext cpmtw_ctx( @@ -788,8 +836,8 @@ void WriteThread::ExitAsBatchGroupLeader(WriteGroup& write_group, } AwaitState(leader, - STATE_MEMTABLE_WRITER_LEADER | STATE_PARALLEL_MEMTABLE_WRITER | - STATE_COMPLETED, + STATE_MEMTABLE_WRITER_LEADER | STATE_PARALLEL_MEMTABLE_CALLER | + STATE_PARALLEL_MEMTABLE_WRITER | STATE_COMPLETED, &eabgl_ctx); } else { Writer* head = newest_writer_.load(std::memory_order_acquire); diff --git a/db/write_thread.h b/db/write_thread.h index dc64601f9f4..16af946b124 100644 --- a/db/write_thread.h +++ b/db/write_thread.h @@ -22,7 +22,9 @@ #include "rocksdb/options.h" #include "rocksdb/status.h" #include "rocksdb/types.h" +#include "rocksdb/user_write_callback.h" #include "rocksdb/write_batch.h" +#include "util/aligned_storage.h" #include "util/autovector.h" namespace ROCKSDB_NAMESPACE { @@ -71,6 +73,12 @@ class WriteThread { // A state indicating that the thread may be waiting using StateMutex() // and StateCondVar() STATE_LOCKED_WAITING = 32, + + // The state used to inform a waiting writer that it has become a + // caller to call some other waiting writers to write to memtable + // by calling SetMemWritersEachStride. After doing + // this, it will also write to memtable. + STATE_PARALLEL_MEMTABLE_CALLER = 64, }; struct Writer; @@ -127,6 +135,7 @@ class WriteThread { uint64_t log_used; // log number that this batch was inserted into uint64_t log_ref; // log number that memtable insert should reference WriteCallback* callback; + UserWriteCallback* user_write_cb; bool made_waitable; // records lazy construction of mutex and cv std::atomic state; // write under StateMutex() or pre-link WriteGroup* write_group; @@ -134,8 +143,8 @@ class WriteThread { Status status; Status callback_status; // status returned by callback->Callback() - std::aligned_storage::type state_mutex_bytes; - std::aligned_storage::type state_cv_bytes; + aligned_storage::type state_mutex_bytes; + aligned_storage::type state_cv_bytes; Writer* link_older; // read/write only before linking, or as leader Writer* link_newer; // lazy, read/write only before linking, or as leader @@ -153,6 +162,7 @@ class WriteThread { log_used(0), log_ref(0), callback(nullptr), + user_write_cb(nullptr), made_waitable(false), state(STATE_INIT), write_group(nullptr), @@ -161,8 +171,8 @@ class WriteThread { link_newer(nullptr) {} Writer(const WriteOptions& write_options, WriteBatch* _batch, - WriteCallback* _callback, uint64_t _log_ref, bool _disable_memtable, - size_t _batch_cnt = 0, + WriteCallback* _callback, UserWriteCallback* _user_write_cb, + uint64_t _log_ref, bool _disable_memtable, size_t _batch_cnt = 0, PreReleaseCallback* _pre_release_callback = nullptr, PostMemTableCallback* _post_memtable_callback = nullptr) : batch(_batch), @@ -180,6 +190,7 @@ class WriteThread { log_used(0), log_ref(_log_ref), callback(_callback), + user_write_cb(_user_write_cb), made_waitable(false), state(STATE_INIT), write_group(nullptr), @@ -203,6 +214,18 @@ class WriteThread { return callback_status.ok(); } + void CheckWriteEnqueuedCallback() { + if (user_write_cb != nullptr) { + user_write_cb->OnWriteEnqueued(); + } + } + + void CheckPostWalWriteCallback() { + if (user_write_cb != nullptr) { + user_write_cb->OnWalWriteFinish(); + } + } + void CreateMutex() { if (!made_waitable) { // Note that made_waitable is tracked separately from state @@ -323,10 +346,19 @@ class WriteThread { // Causes JoinBatchGroup to return STATE_PARALLEL_MEMTABLE_WRITER for all of // the non-leader members of this write batch group. Sets Writer::sequence // before waking them up. + // If the size of write_group n is not small, the leader will call n^0.5 + // members to be PARALLEL_MEMTABLE_CALLER in the write_group to help to set + // other's status parallel. This ensures that the cost to call SetState + // sequentially does not exceed 2(n^0.5). // // WriteGroup* write_group: Extra state used to coordinate the parallel add void LaunchParallelMemTableWriters(WriteGroup* write_group); + // One of the every stride=N number writer in the WriteGroup are set to the + // MemTableWriters, where N is equal to square of the total number of this + // write_group, and all of these MemTableWriters will write to memtable. + void SetMemWritersEachStride(Writer* w); + // Reports the completion of w's batch to the parallel group leader, and // waits for the rest of the parallel batch to complete. Returns true // if this thread is the last to complete, and hence should advance diff --git a/db_stress_tool/batched_ops_stress.cc b/db_stress_tool/batched_ops_stress.cc index 4830c57e92a..a5760207e95 100644 --- a/db_stress_tool/batched_ops_stress.cc +++ b/db_stress_tool/batched_ops_stress.cc @@ -54,7 +54,12 @@ class BatchedOpsStressTest : public StressTest { const std::string v = value_body + num; if (FLAGS_use_put_entity_one_in > 0 && (value_base % FLAGS_use_put_entity_one_in) == 0) { - status = batch.PutEntity(cfh, k, GenerateWideColumns(value_base, v)); + if (FLAGS_use_attribute_group) { + status = + batch.PutEntity(k, GenerateAttributeGroups({cfh}, value_base, v)); + } else { + 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) { @@ -300,15 +305,30 @@ class BatchedOpsStressTest : public StressTest { constexpr size_t num_keys = 10; - std::array results; + std::array column_results; + std::array attribute_group_results; + + std::string error_msg_header = FLAGS_use_attribute_group + ? "GetEntity (AttributeGroup) error" + : "GetEntity error"; for (size_t i = 0; i < num_keys; ++i) { const std::string key = std::to_string(i) + key_suffix; - const Status s = db_->GetEntity(read_opts_copy, cfh, key, &results[i]); + Status s; + if (FLAGS_use_attribute_group) { + attribute_group_results[i].emplace_back(cfh); + s = db_->GetEntity(read_opts_copy, key, &attribute_group_results[i]); + if (s.ok()) { + s = attribute_group_results[i].back().status(); + } + } else { + s = db_->GetEntity(read_opts_copy, cfh, key, &column_results[i]); + } if (!s.ok() && !s.IsNotFound()) { - fprintf(stderr, "GetEntity error: %s\n", s.ToString().c_str()); + fprintf(stderr, "%s: %s\n", error_msg_header.c_str(), + s.ToString().c_str()); thread->stats.AddErrors(1); } else if (s.IsNotFound()) { thread->stats.AddGets(1, 0); @@ -317,14 +337,20 @@ class BatchedOpsStressTest : public StressTest { } } - for (size_t i = 0; i < num_keys; ++i) { - const WideColumns& columns = results[i].columns(); + const WideColumns& columns_to_compare = + FLAGS_use_attribute_group ? attribute_group_results[0].front().columns() + : column_results[0].columns(); - if (!CompareColumns(results[0].columns(), columns)) { - fprintf(stderr, - "GetEntity error: inconsistent entities for key %s: %s, %s\n", - StringToHex(key_suffix).c_str(), - WideColumnsToHex(results[0].columns()).c_str(), + for (size_t i = 1; i < num_keys; ++i) { + const WideColumns& columns = + FLAGS_use_attribute_group + ? attribute_group_results[i].front().columns() + : column_results[i].columns(); + + if (!CompareColumns(columns_to_compare, columns)) { + fprintf(stderr, "%s: inconsistent entities for key %s: %s, %s\n", + error_msg_header.c_str(), StringToHex(key_suffix).c_str(), + WideColumnsToHex(columns_to_compare).c_str(), WideColumnsToHex(columns).c_str()); } @@ -338,20 +364,18 @@ class BatchedOpsStressTest : public StressTest { if (value.empty() || value[value.size() - 1] != expected) { fprintf(stderr, - "GetEntity error: incorrect column value for key " + "%s: incorrect column value for key " "%s, entity %s, column value %s, expected %c\n", - StringToHex(key_suffix).c_str(), + error_msg_header.c_str(), StringToHex(key_suffix).c_str(), WideColumnsToHex(columns).c_str(), value.ToString(/* hex */ true).c_str(), expected); } } if (!VerifyWideColumns(columns)) { - fprintf( - stderr, - "GetEntity error: inconsistent columns for key %s, entity %s\n", - StringToHex(key_suffix).c_str(), - WideColumnsToHex(columns).c_str()); + fprintf(stderr, "%s: inconsistent columns for key %s, entity %s\n", + error_msg_header.c_str(), StringToHex(key_suffix).c_str(), + WideColumnsToHex(columns).c_str()); } } } @@ -385,66 +409,140 @@ class BatchedOpsStressTest : public StressTest { std::array keys; std::array key_slices; - std::array results; - std::array statuses; for (size_t j = 0; j < num_prefixes; ++j) { keys[j] = std::to_string(j) + key_suffix; key_slices[j] = keys[j]; } - db_->MultiGetEntity(read_opts_copy, cfh, num_prefixes, key_slices.data(), - results.data(), statuses.data()); + if (FLAGS_use_attribute_group) { + // AttributeGroup MultiGetEntity verification - for (size_t j = 0; j < num_prefixes; ++j) { - const Status& s = statuses[j]; + std::vector results; + results.reserve(num_prefixes); + for (size_t j = 0; j < num_prefixes; ++j) { + PinnableAttributeGroups attribute_groups; + attribute_groups.emplace_back(cfh); + results.emplace_back(std::move(attribute_groups)); + } + db_->MultiGetEntity(read_opts_copy, num_prefixes, key_slices.data(), + results.data()); + + const WideColumns& cmp_columns = results[0][0].columns(); + + for (size_t j = 0; j < num_prefixes; ++j) { + const auto& attribute_groups = results[j]; + assert(attribute_groups.size() == 1); + const Status& s = attribute_groups[0].status(); + if (!s.ok() && !s.IsNotFound()) { + fprintf(stderr, "MultiGetEntity (AttributeGroup) error: %s\n", + s.ToString().c_str()); + thread->stats.AddErrors(1); + } else if (s.IsNotFound()) { + thread->stats.AddGets(1, 0); + } else { + thread->stats.AddGets(1, 1); + } - if (!s.ok() && !s.IsNotFound()) { - fprintf(stderr, "MultiGetEntity error: %s\n", s.ToString().c_str()); - thread->stats.AddErrors(1); - } else if (s.IsNotFound()) { - thread->stats.AddGets(1, 0); - } else { - thread->stats.AddGets(1, 1); + const WideColumns& columns = results[j][0].columns(); + if (!CompareColumns(cmp_columns, columns)) { + fprintf(stderr, + "MultiGetEntity (AttributeGroup) error: inconsistent " + "entities for key %s: %s, " + "%s\n", + StringToHex(key_suffix).c_str(), + WideColumnsToHex(cmp_columns).c_str(), + WideColumnsToHex(columns).c_str()); + } + if (!columns.empty()) { + // The last character of each column value should be 'j' as a + // decimal digit + const char expected = static_cast('0' + j); + + for (const auto& column : columns) { + const Slice& value = column.value(); + + if (value.empty() || value[value.size() - 1] != expected) { + fprintf(stderr, + "MultiGetEntity (AttributeGroup) error: incorrect " + "column value for key " + "%s, entity %s, column value %s, expected %c\n", + StringToHex(key_suffix).c_str(), + WideColumnsToHex(columns).c_str(), + value.ToString(/* hex */ true).c_str(), expected); + } + } + + if (!VerifyWideColumns(columns)) { + fprintf(stderr, + "MultiGetEntity (AttributeGroup) error: inconsistent " + "columns for key %s, " + "entity %s\n", + StringToHex(key_suffix).c_str(), + WideColumnsToHex(columns).c_str()); + } + } } + } else { + // Non-AttributeGroup MultiGetEntity verification + + std::array results; + std::array statuses; + + db_->MultiGetEntity(read_opts_copy, cfh, num_prefixes, + key_slices.data(), results.data(), statuses.data()); const WideColumns& cmp_columns = results[0].columns(); - const WideColumns& columns = results[j].columns(); - if (!CompareColumns(cmp_columns, columns)) { - fprintf(stderr, - "MultiGetEntity error: inconsistent entities for key %s: %s, " - "%s\n", - StringToHex(key_suffix).c_str(), - WideColumnsToHex(cmp_columns).c_str(), - WideColumnsToHex(columns).c_str()); - } + for (size_t j = 0; j < num_prefixes; ++j) { + const Status& s = statuses[j]; - if (!columns.empty()) { - // The last character of each column value should be 'j' as a decimal - // digit - const char expected = static_cast('0' + j); + if (!s.ok() && !s.IsNotFound()) { + fprintf(stderr, "MultiGetEntity error: %s\n", s.ToString().c_str()); + thread->stats.AddErrors(1); + } else if (s.IsNotFound()) { + thread->stats.AddGets(1, 0); + } else { + thread->stats.AddGets(1, 1); + } + const WideColumns& columns = results[j].columns(); + + if (!CompareColumns(cmp_columns, columns)) { + fprintf( + stderr, + "MultiGetEntity error: inconsistent entities for key %s: %s, " + "%s\n", + StringToHex(key_suffix).c_str(), + WideColumnsToHex(cmp_columns).c_str(), + WideColumnsToHex(columns).c_str()); + } - for (const auto& column : columns) { - const Slice& value = column.value(); + if (!columns.empty()) { + // The last character of each column value should be 'j' as a + // decimal digit + const char expected = static_cast('0' + j); + + for (const auto& column : columns) { + const Slice& value = column.value(); + + if (value.empty() || value[value.size() - 1] != expected) { + fprintf(stderr, + "MultiGetEntity error: incorrect column value for key " + "%s, entity %s, column value %s, expected %c\n", + StringToHex(key_suffix).c_str(), + WideColumnsToHex(columns).c_str(), + value.ToString(/* hex */ true).c_str(), expected); + } + } - if (value.empty() || value[value.size() - 1] != expected) { + if (!VerifyWideColumns(columns)) { fprintf(stderr, - "MultiGetEntity error: incorrect column value for key " - "%s, entity %s, column value %s, expected %c\n", + "MultiGetEntity error: inconsistent columns for key %s, " + "entity %s\n", StringToHex(key_suffix).c_str(), - WideColumnsToHex(columns).c_str(), - value.ToString(/* hex */ true).c_str(), expected); + WideColumnsToHex(columns).c_str()); } } - - if (!VerifyWideColumns(columns)) { - fprintf(stderr, - "MultiGetEntity error: inconsistent columns for key %s, " - "entity %s\n", - StringToHex(key_suffix).c_str(), - WideColumnsToHex(columns).c_str()); - } } } } diff --git a/db_stress_tool/cf_consistency_stress.cc b/db_stress_tool/cf_consistency_stress.cc index 1a098e98235..cd0beabf359 100644 --- a/db_stress_tool/cf_consistency_stress.cc +++ b/db_stress_tool/cf_consistency_stress.cc @@ -37,25 +37,35 @@ 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) { - 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) { - status = batch.Merge(cfh, k, v); - } else { - status = batch.Put(cfh, k, v); + if (FLAGS_use_attribute_group && FLAGS_use_put_entity_one_in > 0 && + (value_base % FLAGS_use_put_entity_one_in) == 0) { + std::vector cfhs; + cfhs.reserve(rand_column_families.size()); + for (auto cf : rand_column_families) { + cfhs.push_back(column_families_[cf]); } - if (!status.ok()) { - break; + status = batch.PutEntity(k, GenerateAttributeGroups(cfhs, value_base, v)); + } else { + 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) { + 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) { + status = batch.Merge(cfh, k, v); + } else { + status = batch.Put(cfh, k, v); + } + if (!status.ok()) { + break; + } } } @@ -328,57 +338,122 @@ class CfConsistencyStressTest : public StressTest { } if (is_consistent) { - for (size_t i = 1; i < rand_column_families.size(); ++i) { - assert(rand_column_families[i] >= 0); - assert(rand_column_families[i] < - static_cast(column_families_.size())); - - PinnableWideColumns result; - s = db_->GetEntity(read_opts_copy, - column_families_[rand_column_families[i]], key, - &result); - - if (!s.ok() && !s.IsNotFound()) { - break; + if (FLAGS_use_attribute_group) { + PinnableAttributeGroups result; + result.reserve(rand_column_families.size()); + for (size_t i = 1; i < rand_column_families.size(); ++i) { + assert(rand_column_families[i] >= 0); + assert(rand_column_families[i] < + static_cast(column_families_.size())); + + result.emplace_back(column_families_[rand_column_families[i]]); } - - const bool found = s.ok(); - - assert(!column_family_names_.empty()); - assert(i < column_family_names_.size()); - - if (!cmp_found && found) { - fprintf(stderr, - "GetEntity returns different results for key %s: CF %s " - "returns not found, CF %s returns entity %s\n", + s = db_->GetEntity(read_opts_copy, key, &result); + if (s.ok()) { + for (auto& attribute_group : result) { + s = attribute_group.status(); + if (!s.ok() && !s.IsNotFound()) { + break; + } + + const bool found = s.ok(); + + if (!cmp_found && found) { + fprintf( + stderr, + "Non-AttributeGroup GetEntity returns different results " + "than AttributeGroup GetEntity for key %s: CF %s " + "returns not found, CF %s returns entity %s \n", StringToHex(key).c_str(), column_family_names_[0].c_str(), - column_family_names_[i].c_str(), - WideColumnsToHex(result.columns()).c_str()); - is_consistent = false; - break; - } - - if (cmp_found && !found) { - fprintf(stderr, - "GetEntity returns different results for key %s: CF %s " - "returns entity %s, CF %s returns not found\n", + attribute_group.column_family()->GetName().c_str(), + WideColumnsToHex(attribute_group.columns()).c_str()); + is_consistent = false; + break; + } + if (cmp_found && !found) { + fprintf( + stderr, + "Non-AttributeGroup GetEntity returns different results " + "than AttributeGroup GetEntity for key %s: CF %s " + "returns entity %s, CF %s returns not found \n", StringToHex(key).c_str(), column_family_names_[0].c_str(), WideColumnsToHex(cmp_result.columns()).c_str(), - column_family_names_[i].c_str()); - is_consistent = false; - break; - } - - if (found && result != cmp_result) { - fprintf(stderr, - "GetEntity returns different results for key %s: CF %s " + attribute_group.column_family()->GetName().c_str()); + is_consistent = false; + break; + } + if (found && + attribute_group.columns() != cmp_result.columns()) { + fprintf( + stderr, + "Non-AttributeGroup GetEntity returns different results " + "than AttributeGroup GetEntity for key %s: CF %s " "returns entity %s, CF %s returns entity %s\n", StringToHex(key).c_str(), column_family_names_[0].c_str(), WideColumnsToHex(cmp_result.columns()).c_str(), - column_family_names_[i].c_str(), - WideColumnsToHex(result.columns()).c_str()); - is_consistent = false; - break; + attribute_group.column_family()->GetName().c_str(), + WideColumnsToHex(attribute_group.columns()).c_str()); + is_consistent = false; + break; + } + } + } + } else { + for (size_t i = 1; i < rand_column_families.size(); ++i) { + assert(rand_column_families[i] >= 0); + assert(rand_column_families[i] < + static_cast(column_families_.size())); + + PinnableWideColumns result; + s = db_->GetEntity(read_opts_copy, + column_families_[rand_column_families[i]], key, + &result); + + if (!s.ok() && !s.IsNotFound()) { + break; + } + + const bool found = s.ok(); + + assert(!column_family_names_.empty()); + assert(i < column_family_names_.size()); + + if (!cmp_found && found) { + fprintf(stderr, + "GetEntity returns different results for key %s: CF %s " + "returns not found, CF %s returns entity %s\n", + StringToHex(key).c_str(), + column_family_names_[0].c_str(), + column_family_names_[i].c_str(), + WideColumnsToHex(result.columns()).c_str()); + is_consistent = false; + break; + } + + if (cmp_found && !found) { + fprintf(stderr, + "GetEntity returns different results for key %s: CF %s " + "returns entity %s, CF %s returns not found\n", + StringToHex(key).c_str(), + column_family_names_[0].c_str(), + WideColumnsToHex(cmp_result.columns()).c_str(), + column_family_names_[i].c_str()); + is_consistent = false; + break; + } + + if (found && result != cmp_result) { + fprintf(stderr, + "GetEntity returns different results for key %s: CF %s " + "returns entity %s, CF %s returns entity %s\n", + StringToHex(key).c_str(), + column_family_names_[0].c_str(), + WideColumnsToHex(cmp_result.columns()).c_str(), + column_family_names_[i].c_str(), + WideColumnsToHex(result.columns()).c_str()); + is_consistent = false; + break; + } } } } @@ -431,95 +506,220 @@ class CfConsistencyStressTest : public StressTest { const size_t num_keys = rand_keys.size(); - for (size_t i = 0; i < num_keys; ++i) { - const std::string key = Key(rand_keys[i]); + if (FLAGS_use_attribute_group) { + // AttributeGroup MultiGetEntity verification + + std::vector results; + std::vector key_slices; + std::vector key_strs; + results.reserve(num_keys); + key_slices.reserve(num_keys); + key_strs.reserve(num_keys); + + for (size_t i = 0; i < num_keys; ++i) { + key_strs.emplace_back(Key(rand_keys[i])); + key_slices.emplace_back(key_strs.back()); + PinnableAttributeGroups attribute_groups; + for (auto* cfh : cfhs) { + attribute_groups.emplace_back(cfh); + } + results.emplace_back(std::move(attribute_groups)); + } + db_->MultiGetEntity(read_opts_copy, num_keys, key_slices.data(), + results.data()); - std::vector key_slices(num_cfs, key); - std::vector results(num_cfs); - std::vector statuses(num_cfs); + bool is_consistent = true; - db_->MultiGetEntity(read_opts_copy, num_cfs, cfhs.data(), - key_slices.data(), results.data(), statuses.data()); + for (size_t i = 0; i < num_keys; ++i) { + const auto& result = results[i]; + const Status& cmp_s = result[0].status(); + const WideColumns& cmp_columns = result[0].columns(); - bool is_consistent = true; + bool has_error = false; - for (size_t j = 0; j < num_cfs; ++j) { - const Status& s = statuses[j]; - const Status& cmp_s = statuses[0]; - const WideColumns& columns = results[j].columns(); - const WideColumns& cmp_columns = results[0].columns(); + for (size_t j = 0; j < num_cfs; ++j) { + const Status& s = result[j].status(); + const WideColumns& columns = result[j].columns(); + if (!s.ok() && !s.IsNotFound()) { + fprintf(stderr, "TestMultiGetEntity (AttributeGroup) error: %s\n", + s.ToString().c_str()); + thread->stats.AddErrors(1); + has_error = true; + break; + } - if (!s.ok() && !s.IsNotFound()) { - fprintf(stderr, "TestMultiGetEntity error: %s\n", - s.ToString().c_str()); - thread->stats.AddErrors(1); - break; - } + assert(cmp_s.ok() || cmp_s.IsNotFound()); + + if (s.IsNotFound()) { + if (cmp_s.ok()) { + fprintf(stderr, + "MultiGetEntity (AttributeGroup) returns different " + "results for key %s: CF %s " + "returns entity %s, CF %s returns not found\n", + key_slices[i].ToString(true).c_str(), + column_family_names_[0].c_str(), + WideColumnsToHex(cmp_columns).c_str(), + column_family_names_[j].c_str()); + is_consistent = false; + break; + } - assert(cmp_s.ok() || cmp_s.IsNotFound()); + continue; + } - if (s.IsNotFound()) { - if (cmp_s.ok()) { - fprintf( - stderr, - "MultiGetEntity returns different results for key %s: CF %s " - "returns entity %s, CF %s returns not found\n", - StringToHex(key).c_str(), column_family_names_[0].c_str(), - WideColumnsToHex(cmp_columns).c_str(), - column_family_names_[j].c_str()); + assert(s.ok()); + if (cmp_s.IsNotFound()) { + fprintf(stderr, + "MultiGetEntity (AttributeGroup) returns different results " + "for key %s: CF %s " + "returns not found, CF %s returns entity %s\n", + key_slices[i].ToString(true).c_str(), + column_family_names_[0].c_str(), + column_family_names_[j].c_str(), + WideColumnsToHex(columns).c_str()); is_consistent = false; break; } - continue; - } + if (columns != cmp_columns) { + fprintf(stderr, + "MultiGetEntity (AttributeGroup) returns different results " + "for key %s: CF %s " + "returns entity %s, CF %s returns entity %s\n", + key_slices[i].ToString(true).c_str(), + column_family_names_[0].c_str(), + WideColumnsToHex(cmp_columns).c_str(), + column_family_names_[j].c_str(), + WideColumnsToHex(columns).c_str()); + is_consistent = false; + break; + } - assert(s.ok()); - if (cmp_s.IsNotFound()) { + if (!VerifyWideColumns(columns)) { + fprintf(stderr, + "MultiGetEntity (AttributeGroup) error: inconsistent " + "columns for key %s, " + "entity %s\n", + key_slices[i].ToString(true).c_str(), + WideColumnsToHex(columns).c_str()); + is_consistent = false; + break; + } + } + if (has_error) { + break; + } else if (!is_consistent) { fprintf(stderr, - "MultiGetEntity returns different results for key %s: CF %s " - "returns not found, CF %s returns entity %s\n", - StringToHex(key).c_str(), column_family_names_[0].c_str(), - column_family_names_[j].c_str(), - WideColumnsToHex(columns).c_str()); - is_consistent = false; + "TestMultiGetEntity (AttributeGroup) error: results are not " + "consistent\n"); + thread->stats.AddErrors(1); + // Fail fast to preserve the DB state. + thread->shared->SetVerificationFailure(); break; + } else if (cmp_s.ok()) { + thread->stats.AddGets(1, 1); + } else if (cmp_s.IsNotFound()) { + thread->stats.AddGets(1, 0); } + } - if (columns != cmp_columns) { - fprintf(stderr, + } else { + // Non-AttributeGroup MultiGetEntity verification + + for (size_t i = 0; i < num_keys; ++i) { + const std::string key = Key(rand_keys[i]); + + std::vector key_slices(num_cfs, key); + std::vector results(num_cfs); + std::vector statuses(num_cfs); + + db_->MultiGetEntity(read_opts_copy, num_cfs, cfhs.data(), + key_slices.data(), results.data(), statuses.data()); + + bool is_consistent = true; + + const Status& cmp_s = statuses[0]; + const WideColumns& cmp_columns = results[0].columns(); + + for (size_t j = 0; j < num_cfs; ++j) { + const Status& s = statuses[j]; + const WideColumns& columns = results[j].columns(); + + if (!s.ok() && !s.IsNotFound()) { + fprintf(stderr, "TestMultiGetEntity error: %s\n", + s.ToString().c_str()); + thread->stats.AddErrors(1); + break; + } + + assert(cmp_s.ok() || cmp_s.IsNotFound()); + + if (s.IsNotFound()) { + if (cmp_s.ok()) { + fprintf( + stderr, "MultiGetEntity returns different results for key %s: CF %s " - "returns entity %s, CF %s returns entity %s\n", + "returns entity %s, CF %s returns not found\n", StringToHex(key).c_str(), column_family_names_[0].c_str(), WideColumnsToHex(cmp_columns).c_str(), - column_family_names_[j].c_str(), - WideColumnsToHex(columns).c_str()); - is_consistent = false; - break; + column_family_names_[j].c_str()); + is_consistent = false; + break; + } + + continue; + } + + assert(s.ok()); + if (cmp_s.IsNotFound()) { + fprintf( + stderr, + "MultiGetEntity returns different results for key %s: CF %s " + "returns not found, CF %s returns entity %s\n", + StringToHex(key).c_str(), column_family_names_[0].c_str(), + column_family_names_[j].c_str(), + WideColumnsToHex(columns).c_str()); + is_consistent = false; + break; + } + + if (columns != cmp_columns) { + fprintf( + stderr, + "MultiGetEntity returns different results for key %s: CF %s " + "returns entity %s, CF %s returns entity %s\n", + StringToHex(key).c_str(), column_family_names_[0].c_str(), + WideColumnsToHex(cmp_columns).c_str(), + column_family_names_[j].c_str(), + WideColumnsToHex(columns).c_str()); + is_consistent = false; + break; + } + + if (!VerifyWideColumns(columns)) { + fprintf(stderr, + "MultiGetEntity error: inconsistent columns for key %s, " + "entity %s\n", + StringToHex(key).c_str(), + WideColumnsToHex(columns).c_str()); + is_consistent = false; + break; + } } - if (!VerifyWideColumns(columns)) { + if (!is_consistent) { fprintf(stderr, - "MultiGetEntity error: inconsistent columns for key %s, " - "entity %s\n", - StringToHex(key).c_str(), WideColumnsToHex(columns).c_str()); - is_consistent = false; + "TestMultiGetEntity error: results are not consistent\n"); + thread->stats.AddErrors(1); + // Fail fast to preserve the DB state. + thread->shared->SetVerificationFailure(); break; + } else if (statuses[0].ok()) { + thread->stats.AddGets(1, 1); + } else if (statuses[0].IsNotFound()) { + thread->stats.AddGets(1, 0); } } - - if (!is_consistent) { - fprintf(stderr, - "TestMultiGetEntity error: results are not consistent\n"); - thread->stats.AddErrors(1); - // Fail fast to preserve the DB state. - thread->shared->SetVerificationFailure(); - break; - } else if (statuses[0].ok()) { - thread->stats.AddGets(1, 1); - } else if (statuses[0].IsNotFound()) { - thread->stats.AddGets(1, 0); - } } } diff --git a/db_stress_tool/db_stress_common.cc b/db_stress_tool/db_stress_common.cc index 3683b246abb..1ba3ca7a623 100644 --- a/db_stress_tool/db_stress_common.cc +++ b/db_stress_tool/db_stress_common.cc @@ -321,6 +321,17 @@ uint32_t GetValueBase(Slice s) { return res; } +AttributeGroups GenerateAttributeGroups( + const std::vector& cfhs, uint32_t value_base, + const Slice& slice) { + WideColumns columns = GenerateWideColumns(value_base, slice); + AttributeGroups attribute_groups; + for (auto* cfh : cfhs) { + attribute_groups.emplace_back(cfh, columns); + } + return attribute_groups; +} + WideColumns GenerateWideColumns(uint32_t value_base, const Slice& slice) { WideColumns columns; diff --git a/db_stress_tool/db_stress_common.h b/db_stress_tool/db_stress_common.h index 12be9fbc5fd..3ff8fdbe233 100644 --- a/db_stress_tool/db_stress_common.h +++ b/db_stress_tool/db_stress_common.h @@ -136,6 +136,7 @@ DECLARE_int32(universal_size_ratio); DECLARE_int32(universal_min_merge_width); DECLARE_int32(universal_max_merge_width); DECLARE_int32(universal_max_size_amplification_percent); +DECLARE_int32(universal_max_read_amp); DECLARE_int32(clear_column_family_one_in); DECLARE_int32(get_live_files_apis_one_in); DECLARE_int32(get_all_column_family_metadata_one_in); @@ -209,8 +210,11 @@ DECLARE_int32(ingest_external_file_one_in); DECLARE_int32(ingest_external_file_width); DECLARE_int32(compact_files_one_in); DECLARE_int32(compact_range_one_in); +DECLARE_int32(promote_l0_one_in); DECLARE_int32(mark_for_compaction_one_file_in); DECLARE_int32(flush_one_in); +DECLARE_int32(key_may_exist_one_in); +DECLARE_int32(reset_stats_one_in); DECLARE_int32(pause_background_one_in); DECLARE_int32(disable_file_deletions_one_in); DECLARE_int32(disable_manual_compaction_one_in); @@ -250,6 +254,8 @@ DECLARE_string(memtablerep); DECLARE_int32(prefix_size); DECLARE_bool(use_merge); DECLARE_uint32(use_put_entity_one_in); +DECLARE_bool(use_attribute_group); +DECLARE_bool(use_multi_cf_iterator); DECLARE_bool(use_full_merge_v1); DECLARE_int32(sync_wal_one_in); DECLARE_bool(avoid_unnecessary_blocking_io); @@ -375,6 +381,7 @@ DECLARE_uint64(WAL_ttl_seconds); DECLARE_uint64(WAL_size_limit_MB); DECLARE_bool(strict_bytes_per_sync); DECLARE_bool(avoid_flush_during_shutdown); +DECLARE_bool(avoid_sync_during_shutdown); DECLARE_bool(fill_cache); DECLARE_bool(optimize_multiget_for_io); DECLARE_bool(memtable_insert_hint_per_batch); @@ -758,6 +765,10 @@ WideColumns GenerateExpectedWideColumns(uint32_t value_base, bool VerifyWideColumns(const Slice& value, const WideColumns& columns); bool VerifyWideColumns(const WideColumns& columns); +AttributeGroups GenerateAttributeGroups( + const std::vector& cfhs, uint32_t value_base, + const Slice& slice); + StressTest* CreateCfConsistencyStressTest(); StressTest* CreateBatchedOpsStressTest(); StressTest* CreateNonBatchedOpsStressTest(); diff --git a/db_stress_tool/db_stress_gflags.cc b/db_stress_tool/db_stress_gflags.cc index b037b8d97ae..6b690af29f5 100644 --- a/db_stress_tool/db_stress_gflags.cc +++ b/db_stress_tool/db_stress_gflags.cc @@ -306,6 +306,9 @@ DEFINE_int32(universal_max_merge_width, 0, DEFINE_int32(universal_max_size_amplification_percent, 0, "The max size amplification for universal style compaction"); +DEFINE_int32(universal_max_read_amp, -1, + "The limit on the number of sorted runs"); + DEFINE_int32(clear_column_family_one_in, 1000000, "With a chance of 1/N, delete a column family and then recreate " "it again. If N == 0, never drop/create column families. " @@ -756,6 +759,10 @@ DEFINE_int32(compact_range_one_in, 0, "If non-zero, then CompactRange() will be called once for every N " "operations on average. 0 indicates CompactRange() is disabled."); +DEFINE_int32(promote_l0_one_in, 0, + "If non-zero, then PromoteL0() will be called once for every N " + "operations on average. 0 indicates PromoteL0() is disabled."); + DEFINE_int32(mark_for_compaction_one_file_in, 0, "A `TablePropertiesCollectorFactory` will be registered, which " "creates a `TablePropertiesCollector` with `NeedCompact()` " @@ -766,6 +773,14 @@ DEFINE_int32(flush_one_in, 0, "If non-zero, then Flush() will be called once for every N ops " "on average. 0 indicates calls to Flush() are disabled."); +DEFINE_int32(key_may_exist_one_in, 0, + "If non-zero, then KeyMayExist() will be called " + "once for every N ops on average. 0 disables."); + +DEFINE_int32(reset_stats_one_in, 0, + "If non-zero, then ResetStats() will be called " + "once for every N ops on average. 0 disables."); + 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."); @@ -940,6 +955,12 @@ DEFINE_uint32(use_put_entity_one_in, 0, "If greater than zero, PutEntity will be used once per every N " "write ops on average."); +DEFINE_bool(use_attribute_group, false, + "If set, use the attribute_group API to put/get entities"); + +DEFINE_bool(use_multi_cf_iterator, false, + "If set, use the multi_cf_iterator for TestIterate"); + DEFINE_bool(use_full_merge_v1, false, "On true, use a merge operator that implement the deprecated " "version of FullMerge"); @@ -961,6 +982,10 @@ DEFINE_bool(avoid_flush_during_recovery, ROCKSDB_NAMESPACE::Options().avoid_flush_during_recovery, "Avoid flush during recovery"); +DEFINE_bool(avoid_sync_during_shutdown, + ROCKSDB_NAMESPACE::Options().avoid_sync_during_shutdown, + "Options.avoid_sync_during_shutdown"); + DEFINE_uint64(max_write_batch_group_size_bytes, ROCKSDB_NAMESPACE::Options().max_write_batch_group_size_bytes, "Max write batch group size"); diff --git a/db_stress_tool/db_stress_test_base.cc b/db_stress_tool/db_stress_test_base.cc index 9de43d6691e..deb88455db0 100644 --- a/db_stress_tool/db_stress_test_base.cc +++ b/db_stress_tool/db_stress_test_base.cc @@ -505,11 +505,13 @@ Status StressTest::AssertSame(DB* db, ColumnFamilyHandle* cf, } void StressTest::ProcessStatus(SharedState* shared, std::string opname, - Status s) const { + const Status& s, + bool ignore_injected_error) const { if (s.ok()) { return; } - if (!s.IsIOError() || !std::strstr(s.getState(), "injected")) { + if (!s.IsIOError() || !std::strstr(s.getState(), "injected") || + !ignore_injected_error) { std::ostringstream oss; oss << opname << " failed: " << s.ToString(); VerificationAbort(shared, oss.str()); @@ -615,8 +617,21 @@ void StressTest::PreloadDbAndReopenAsReadOnly(int64_t number_of_keys, if (FLAGS_use_put_entity_one_in > 0 && (value_base % FLAGS_use_put_entity_one_in) == 0) { - s = db_->PutEntity(write_opts, cfh, key, - GenerateWideColumns(value_base, v)); + if (!FLAGS_use_txn) { + if (FLAGS_use_attribute_group) { + s = db_->PutEntity(write_opts, key, + GenerateAttributeGroups({cfh}, value_base, v)); + } else { + s = db_->PutEntity(write_opts, cfh, key, + GenerateWideColumns(value_base, v)); + } + } else { + s = ExecuteTransaction( + write_opts, /*thread=*/nullptr, [&](Transaction& txn) { + return txn.PutEntity(cfh, key, + GenerateWideColumns(value_base, v)); + }); + } } else if (FLAGS_use_merge) { if (!FLAGS_use_txn) { if (FLAGS_user_timestamp_size > 0) { @@ -956,12 +971,48 @@ void StressTest::OperateDb(ThreadState* thread) { if (!s.ok()) { fprintf(stderr, "LockWAL() failed: %s\n", s.ToString().c_str()); } else { + // Verify no writes during LockWAL auto old_seqno = db_->GetLatestSequenceNumber(); - // Yield for a while - do { - std::this_thread::yield(); - } while (thread->rand.OneIn(2)); - // Latest seqno should not have changed + // And also that WAL is not changed during LockWAL() + std::unique_ptr old_wal; + s = db_->GetCurrentWalFile(&old_wal); + if (!s.ok()) { + fprintf(stderr, "GetCurrentWalFile() failed: %s\n", + s.ToString().c_str()); + } else { + // Yield for a while + do { + std::this_thread::yield(); + } while (thread->rand.OneIn(2)); + // Current WAL and size should not have changed + std::unique_ptr new_wal; + s = db_->GetCurrentWalFile(&new_wal); + if (!s.ok()) { + fprintf(stderr, "GetCurrentWalFile() failed: %s\n", + s.ToString().c_str()); + } else { + if (old_wal->LogNumber() != new_wal->LogNumber()) { + fprintf(stderr, + "Failed: WAL number changed during LockWAL(): %" PRIu64 + " to %" PRIu64 "\n", + old_wal->LogNumber(), new_wal->LogNumber()); + } + // FIXME: FaultInjectionTestFS does not report file sizes that + // reflect what has been flushed. Either that needs to be fixed + // or GetSortedWals/GetLiveWalFile need to stop relying on + // asking the FS for sizes. + if (!fault_fs_guard && + old_wal->SizeFileBytes() != new_wal->SizeFileBytes()) { + fprintf(stderr, + "Failed: WAL %" PRIu64 + " size changed during LockWAL(): %" PRIu64 + " to %" PRIu64 "\n", + old_wal->LogNumber(), old_wal->SizeFileBytes(), + new_wal->SizeFileBytes()); + } + } + } + // Verify no writes during LockWAL auto new_seqno = db_->GetLatestSequenceNumber(); if (old_seqno != new_seqno) { fprintf( @@ -969,6 +1020,7 @@ void StressTest::OperateDb(ThreadState* thread) { "Failure: latest seqno changed from %u to %u with WAL locked\n", (unsigned)old_seqno, (unsigned)new_seqno); } + // Verification done. Now unlock WAL s = db_->UnlockWAL(); if (!s.ok()) { fprintf(stderr, "UnlockWAL() failed: %s\n", s.ToString().c_str()); @@ -1001,6 +1053,10 @@ void StressTest::OperateDb(ThreadState* thread) { } } + if (thread->rand.OneInOpt(FLAGS_promote_l0_one_in)) { + TestPromoteL0(thread, column_family); + } + std::vector rand_column_families = GenerateColumnFamilies(FLAGS_column_families, rand_column_family); @@ -1041,6 +1097,11 @@ void StressTest::OperateDb(ThreadState* thread) { ProcessStatus(shared, "VerifyGetCurrentWalFile", status); } + if (thread->rand.OneInOpt(FLAGS_reset_stats_one_in)) { + Status status = TestResetStats(); + ProcessStatus(shared, "ResetStats", status); + } + if (thread->rand.OneInOpt(FLAGS_pause_background_one_in)) { Status status = TestPauseBackground(thread); ProcessStatus(shared, "Pause/ContinueBackgroundWork", status); @@ -1135,6 +1196,10 @@ void StressTest::OperateDb(ThreadState* thread) { read_opts.timestamp = &read_ts; } + if (thread->rand.OneInOpt(FLAGS_key_may_exist_one_in)) { + TestKeyMayExist(thread, read_opts, rand_column_families, rand_keys); + } + int prob_op = thread->rand.Uniform(100); // Reset this in case we pick something other than a read op. We don't // want to use a stale value when deciding at the beginning of the loop @@ -1360,15 +1425,28 @@ Status StressTest::TestIterate(ThreadState* thread, ro.iterate_lower_bound = &lower_bound; } - ColumnFamilyHandle* const cfh = column_families_[rand_column_families[0]]; - assert(cfh); + std::unique_ptr iter; - std::unique_ptr iter(db_->NewIterator(ro, cfh)); + if (FLAGS_use_multi_cf_iterator) { + std::vector cfhs; + cfhs.reserve(rand_column_families.size()); + for (auto cf_index : rand_column_families) { + cfhs.emplace_back(column_families_[cf_index]); + } + assert(!cfhs.empty()); + iter = db_->NewCoalescingIterator(ro, cfhs); + } else { + ColumnFamilyHandle* const cfh = column_families_[rand_column_families[0]]; + assert(cfh); + iter = std::unique_ptr(db_->NewIterator(ro, cfh)); + } std::vector key_strs; if (thread->rand.OneIn(16)) { // Generate keys close to lower or upper bound of SST files. - key_strs = GetWhiteBoxKeys(thread, db_, cfh, rand_keys.size()); + key_strs = + GetWhiteBoxKeys(thread, db_, column_families_[rand_column_families[0]], + rand_keys.size()); } if (key_strs.empty()) { // Use the random keys passed in. @@ -1430,9 +1508,10 @@ Status StressTest::TestIterate(ThreadState* thread, const bool support_seek_first_or_last = expect_total_order; - // Write-prepared and Write-unprepared do not support Refresh() yet. + // Write-prepared and Write-unprepared and multi-cf-iterator do not support + // Refresh() yet. if (!(FLAGS_use_txn && FLAGS_txn_write_policy != 0 /* write committed */) && - thread->rand.OneIn(4)) { + !FLAGS_use_multi_cf_iterator && thread->rand.OneIn(4)) { Status s = iter->Refresh(snapshot_guard.snapshot()); assert(s.ok()); op_logs += "Refresh "; @@ -1527,13 +1606,13 @@ Status StressTest::VerifyGetAllColumnFamilyMetaData() const { // Test the return status of GetSortedWalFiles. Status StressTest::VerifyGetSortedWalFiles() const { - VectorLogPtr log_ptr; + VectorWalPtr log_ptr; return db_->GetSortedWalFiles(log_ptr); } // Test the return status of GetCurrentWalFile. Status StressTest::VerifyGetCurrentWalFile() const { - std::unique_ptr cur_wal_file; + std::unique_ptr cur_wal_file; return db_->GetCurrentWalFile(&cur_wal_file); } @@ -2428,8 +2507,32 @@ void StressTest::TestCompactFiles(ThreadState* thread, } } +void StressTest::TestPromoteL0(ThreadState* thread, + ColumnFamilyHandle* column_family) { + int target_level = thread->rand.Next() % options_.num_levels; + Status s = db_->PromoteL0(column_family, target_level); + if (!s.ok()) { + // The second error occurs when another concurrent PromoteL0() moving the + // same files finishes first which is an allowed behavior + bool non_ok_status_allowed = + s.IsInvalidArgument() || + (s.IsCorruption() && + s.ToString().find("VersionBuilder: Cannot delete table file") != + std::string::npos && + s.ToString().find("since it is on level") != std::string::npos); + fprintf(non_ok_status_allowed ? stdout : stderr, + "Unable to perform PromoteL0(): %s under specified " + "target_level: %d.\n", + s.ToString().c_str(), target_level); + if (!non_ok_status_allowed) { + thread->shared->SafeTerminate(); + } + } +} + Status StressTest::TestFlush(const std::vector& rand_column_families) { FlushOptions flush_opts; + assert(flush_opts.wait); if (FLAGS_atomic_flush) { return db_->Flush(flush_opts, column_families_); } @@ -2439,6 +2542,8 @@ Status StressTest::TestFlush(const std::vector& rand_column_families) { return db_->Flush(flush_opts, cfhs); } +Status StressTest::TestResetStats() { return db_->ResetStats(); } + Status StressTest::TestPauseBackground(ThreadState* thread) { Status status = db_->PauseBackgroundWork(); if (!status.ok()) { @@ -2583,7 +2688,10 @@ void StressTest::TestCompactRange(ThreadState* thread, int64_t rand_key, CompactRangeOptions cro; cro.exclusive_manual_compaction = static_cast(thread->rand.Next() % 2); - cro.change_level = static_cast(thread->rand.Next() % 2); + if (static_cast(FLAGS_compaction_style) != + ROCKSDB_NAMESPACE::CompactionStyle::kCompactionStyleFIFO) { + cro.change_level = static_cast(thread->rand.Next() % 2); + } if (thread->rand.OneIn(2)) { cro.target_level = thread->rand.Next() % options_.num_levels; } @@ -2801,7 +2909,8 @@ void StressTest::PrintEnv() const { fprintf(stdout, "Num times DB reopens : %d\n", FLAGS_reopen); fprintf(stdout, "Batches/snapshots : %d\n", FLAGS_test_batches_snapshots); - fprintf(stdout, "Do update in place : %d\n", FLAGS_in_place_update); + fprintf(stdout, "Do update in place : %d\n", + FLAGS_inplace_update_support); fprintf(stdout, "Num keys per lock : %d\n", 1 << FLAGS_log2_keys_per_lock); std::string compression = CompressionTypeToString(compression_type_e); @@ -3597,7 +3706,6 @@ void InitializeOptionsFromFlags( options.compression_opts.checksum = true; } options.max_manifest_file_size = FLAGS_max_manifest_file_size; - options.inplace_update_support = FLAGS_in_place_update; options.max_subcompactions = static_cast(FLAGS_subcompactions); options.allow_concurrent_memtable_write = FLAGS_allow_concurrent_memtable_write; @@ -3618,6 +3726,8 @@ void InitializeOptionsFromFlags( FLAGS_universal_max_merge_width; options.compaction_options_universal.max_size_amplification_percent = FLAGS_universal_max_size_amplification_percent; + options.compaction_options_universal.max_read_amp = + FLAGS_universal_max_read_amp; options.atomic_flush = FLAGS_atomic_flush; options.manual_wal_flush = FLAGS_manual_wal_flush_one_in > 0 ? true : false; options.avoid_unnecessary_blocking_io = FLAGS_avoid_unnecessary_blocking_io; @@ -3744,6 +3854,7 @@ void InitializeOptionsFromFlags( options.wal_bytes_per_sync = FLAGS_wal_bytes_per_sync; options.strict_bytes_per_sync = FLAGS_strict_bytes_per_sync; options.avoid_flush_during_shutdown = FLAGS_avoid_flush_during_shutdown; + options.avoid_sync_during_shutdown = FLAGS_avoid_sync_during_shutdown; options.dump_malloc_stats = FLAGS_dump_malloc_stats; options.stats_history_buffer_size = FLAGS_stats_history_buffer_size; options.skip_stats_update_on_db_open = FLAGS_skip_stats_update_on_db_open; diff --git a/db_stress_tool/db_stress_test_base.h b/db_stress_tool/db_stress_test_base.h index 3532f40e571..5e4593823be 100644 --- a/db_stress_tool/db_stress_test_base.h +++ b/db_stress_tool/db_stress_test_base.h @@ -89,6 +89,10 @@ class StressTest { return {rand_key}; } + virtual void TestKeyMayExist(ThreadState*, const ReadOptions&, + const std::vector&, + const std::vector&) {} + virtual Status TestGet(ThreadState* thread, const ReadOptions& read_opts, const std::vector& rand_column_families, const std::vector& rand_keys) = 0; @@ -136,6 +140,9 @@ class StressTest { const Slice& start_key, ColumnFamilyHandle* column_family); + virtual void TestPromoteL0(ThreadState* thread, + ColumnFamilyHandle* column_family); + // Calculate a hash value for all keys in range [start_key, end_key] // at a certain snapshot. uint32_t GetRangeHash(ThreadState* thread, const Snapshot* snapshot, @@ -204,6 +211,8 @@ class StressTest { Status TestFlush(const std::vector& rand_column_families); + Status TestResetStats(); + Status TestPauseBackground(ThreadState* thread); Status TestDisableFileDeletions(ThreadState* thread); @@ -236,7 +245,8 @@ class StressTest { return Status::NotSupported("TestCustomOperations() must be overridden"); } - void ProcessStatus(SharedState* shared, std::string msg, Status s) const; + void ProcessStatus(SharedState* shared, std::string msg, const Status& s, + bool ignore_injected_error = true) const; void VerificationAbort(SharedState* shared, std::string msg) const; diff --git a/db_stress_tool/db_stress_tool.cc b/db_stress_tool/db_stress_tool.cc index 813467efc4c..34a45a603c1 100644 --- a/db_stress_tool/db_stress_tool.cc +++ b/db_stress_tool/db_stress_tool.cc @@ -251,13 +251,14 @@ int db_stress_tool(int argc, char** argv) { } 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_check_multiget_consistency || + FLAGS_iterpercent > 0 || FLAGS_prefixpercent > 0 || + FLAGS_test_batches_snapshots || 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, test_cf_consistency, " + "prefixpercent, 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); @@ -300,11 +301,11 @@ int db_stress_tool(int argc, char** argv) { } if (FLAGS_use_put_entity_one_in > 0 && - (FLAGS_use_full_merge_v1 || FLAGS_use_txn || FLAGS_test_multi_ops_txns || + (FLAGS_use_full_merge_v1 || FLAGS_test_multi_ops_txns || FLAGS_user_timestamp_size > 0)) { fprintf(stderr, - "Wide columns are incompatible with V1 Merge, transactions, and " - "user-defined timestamps\n"); + "Wide columns are incompatible with V1 Merge, the multi-op " + "transaction test, and user-defined timestamps\n"); exit(1); } diff --git a/db_stress_tool/multi_ops_txns_stress.cc b/db_stress_tool/multi_ops_txns_stress.cc index ee90711b1f7..eb31eded28a 100644 --- a/db_stress_tool/multi_ops_txns_stress.cc +++ b/db_stress_tool/multi_ops_txns_stress.cc @@ -1331,14 +1331,16 @@ uint32_t MultiOpsTxnsStressTest::GenerateNextC(ThreadState* thread) { } void MultiOpsTxnsStressTest::ProcessRecoveredPreparedTxnsHelper( - Transaction* txn, SharedState*) { + Transaction* txn, SharedState* shared) { thread_local Random rand(static_cast(FLAGS_seed)); if (rand.OneIn(2)) { Status s = txn->Commit(); - assert(s.ok()); + ProcessStatus(shared, "ProcessRecoveredPreparedTxnsHelper", s, + /*ignore_injected_error=*/false); } else { Status s = txn->Rollback(); - assert(s.ok()); + ProcessStatus(shared, "ProcessRecoveredPreparedTxnsHelper", s, + /*ignore_injected_error=*/false); } } @@ -1517,14 +1519,15 @@ void MultiOpsTxnsStressTest::PreloadDb(SharedState* shared, int threads, WriteBatch wb; const auto primary_index_entry = record.EncodePrimaryIndexEntry(); Status s = wb.Put(primary_index_entry.first, primary_index_entry.second); - assert(s.ok()); + ProcessStatus(shared, "PreloadDB", s, /*ignore_injected_error=*/false); const auto secondary_index_entry = record.EncodeSecondaryIndexEntry(); s = wb.Put(secondary_index_entry.first, secondary_index_entry.second); - assert(s.ok()); + ProcessStatus(shared, "PreloadDB", s, /*ignore_injected_error=*/false); s = txn_db_->Write(wopts, &wb); assert(s.ok()); + ProcessStatus(shared, "PreloadDB", s, /*ignore_injected_error=*/false); // TODO (yanqin): make the following check optional, especially when data // size is large. @@ -1532,7 +1535,7 @@ void MultiOpsTxnsStressTest::PreloadDb(SharedState* shared, int threads, tmp_rec.SetB(record.b_value()); s = tmp_rec.DecodeSecondaryIndexEntry(secondary_index_entry.first, secondary_index_entry.second); - assert(s.ok()); + ProcessStatus(shared, "PreloadDB", s, /*ignore_injected_error=*/false); assert(tmp_rec == record); existing_a_uniqs[tid].insert(a); diff --git a/db_stress_tool/no_batched_ops_stress.cc b/db_stress_tool/no_batched_ops_stress.cc index 4099d312391..1a081d25fbc 100644 --- a/db_stress_tool/no_batched_ops_stress.cc +++ b/db_stress_tool/no_batched_ops_stress.cc @@ -462,6 +462,44 @@ class NonBatchedOpsStressTest : public StressTest { bool IsStateTracked() const override { return true; } + void TestKeyMayExist(ThreadState* thread, const ReadOptions& read_opts, + const std::vector& rand_column_families, + const std::vector& rand_keys) override { + auto cfh = column_families_[rand_column_families[0]]; + std::string key_str = Key(rand_keys[0]); + Slice key = key_str; + std::string ignore; + ReadOptions read_opts_copy = read_opts; + + std::string read_ts_str; + Slice read_ts_slice; + if (FLAGS_user_timestamp_size > 0) { + read_ts_str = GetNowNanos(); + read_ts_slice = read_ts_str; + read_opts_copy.timestamp = &read_ts_slice; + } + bool read_older_ts = MaybeUseOlderTimestampForPointLookup( + thread, read_ts_str, read_ts_slice, read_opts_copy); + + const ExpectedValue pre_read_expected_value = + thread->shared->Get(rand_column_families[0], rand_keys[0]); + bool key_may_exist = db_->KeyMayExist(read_opts_copy, cfh, key, &ignore); + const ExpectedValue post_read_expected_value = + thread->shared->Get(rand_column_families[0], rand_keys[0]); + + if (!key_may_exist && !FLAGS_skip_verifydb && !read_older_ts) { + if (ExpectedValueHelper::MustHaveExisted(pre_read_expected_value, + post_read_expected_value)) { + thread->shared->SetVerificationFailure(); + fprintf(stderr, + "error : inconsistent values for key %s: expected state has " + "the key, TestKeyMayExist() returns false indicating the key " + "must not exist.\n", + key.ToString(true).c_str()); + } + } + } + Status TestGet(ThreadState* thread, const ReadOptions& read_opts, const std::vector& rand_column_families, const std::vector& rand_keys) override { @@ -572,10 +610,10 @@ class NonBatchedOpsStressTest : public StressTest { std::vector values(num_keys); std::vector statuses(num_keys); // When Flags_use_txn is enabled, we also do a read your write check. - std::vector> ryw_expected_values; - ryw_expected_values.reserve(num_keys); + std::unordered_map ryw_expected_values; SharedState* shared = thread->shared; + assert(shared); int column_family = rand_column_families[0]; ColumnFamilyHandle* cfh = column_families_[column_family]; @@ -612,7 +650,7 @@ class NonBatchedOpsStressTest : public StressTest { Status s = NewTxn(wo, &txn); if (!s.ok()) { fprintf(stderr, "NewTxn error: %s\n", s.ToString().c_str()); - thread->shared->SafeTerminate(); + shared->SafeTerminate(); } } for (size_t i = 0; i < num_keys; ++i) { @@ -620,54 +658,8 @@ class NonBatchedOpsStressTest : public StressTest { key_str.emplace_back(Key(rand_key)); keys.emplace_back(key_str.back()); if (use_txn) { - if (!shared->AllowsOverwrite(rand_key) && - shared->Exists(column_family, rand_key)) { - // Just do read your write checks for keys that allow overwrites. - ryw_expected_values.emplace_back(std::nullopt); - continue; - } - // With a 1 in 10 probability, insert the just added key in the batch - // into the transaction. This will create an overlap with the MultiGet - // keys and exercise some corner cases in the code - if (thread->rand.OneIn(10)) { - int op = thread->rand.Uniform(2); - Status s; - assert(txn); - switch (op) { - case 0: - case 1: { - ExpectedValue put_value; - put_value.Put(false /* pending */); - ryw_expected_values.emplace_back(put_value); - char value[100]; - size_t sz = - GenerateValue(put_value.GetValueBase(), value, sizeof(value)); - Slice v(value, sz); - if (op == 0) { - s = txn->Put(cfh, keys.back(), v); - } else { - s = txn->Merge(cfh, keys.back(), v); - } - break; - } - case 2: { - ExpectedValue delete_value; - delete_value.Delete(false /* pending */); - ryw_expected_values.emplace_back(delete_value); - s = txn->Delete(cfh, keys.back()); - break; - } - default: - assert(false); - } - if (!s.ok()) { - fprintf(stderr, "Transaction put error: %s\n", - s.ToString().c_str()); - thread->shared->SafeTerminate(); - } - } else { - ryw_expected_values.emplace_back(std::nullopt); - } + MaybeAddKeyToTxnForRYW(thread, column_family, rand_key, txn.get(), + ryw_expected_values); } } @@ -698,7 +690,7 @@ class NonBatchedOpsStressTest : public StressTest { if (stat_nok < error_count) { // Grab mutex so multiple thread don't try to print the // stack trace at the same time - MutexLock l(thread->shared->GetMutex()); + MutexLock l(shared->GetMutex()); fprintf(stderr, "Didn't get expected error from MultiGet. \n"); fprintf(stderr, "num_keys %zu Expected %d errors, seen %d\n", num_keys, error_count, stat_nok); @@ -833,13 +825,13 @@ class NonBatchedOpsStressTest : public StressTest { fprintf(stderr, "TestMultiGet error: is_consistent is false\n"); thread->stats.AddErrors(1); // Fail fast to preserve the DB state - thread->shared->SetVerificationFailure(); + shared->SetVerificationFailure(); return false; } else if (!is_ryw_correct) { fprintf(stderr, "TestMultiGet error: is_ryw_correct is false\n"); thread->stats.AddErrors(1); // Fail fast to preserve the DB state - thread->shared->SetVerificationFailure(); + shared->SetVerificationFailure(); return false; } else if (s.ok()) { // found case @@ -868,9 +860,15 @@ class NonBatchedOpsStressTest : public StressTest { for (size_t i = 0; i < num_of_keys; ++i) { bool check_result = true; if (use_txn) { - assert(ryw_expected_values.size() == num_of_keys); - check_result = check_multiget(keys[i], values[i], statuses[i], - ryw_expected_values[i]); + std::optional ryw_expected_value; + + const auto it = ryw_expected_values.find(key_str[i]); + if (it != ryw_expected_values.end()) { + ryw_expected_value = it->second; + } + + check_result = + check_multiget(keys[i], values[i], statuses[i], ryw_expected_value); } else { check_result = check_multiget(keys[i], values[i], statuses[i], std::nullopt /* ryw_expected_value */); @@ -903,20 +901,22 @@ class NonBatchedOpsStressTest : public StressTest { assert(shared); assert(!rand_column_families.empty()); - assert(!rand_keys.empty()); - std::unique_ptr lock(new MutexLock( - shared->GetMutexForKey(rand_column_families[0], rand_keys[0]))); + const int column_family = rand_column_families[0]; - assert(rand_column_families[0] >= 0); - assert(rand_column_families[0] < static_cast(column_families_.size())); + assert(column_family >= 0); + assert(column_family < static_cast(column_families_.size())); - ColumnFamilyHandle* const cfh = column_families_[rand_column_families[0]]; + ColumnFamilyHandle* const cfh = column_families_[column_family]; assert(cfh); - const std::string key = Key(rand_keys[0]); + assert(!rand_keys.empty()); - PinnableWideColumns from_db; + const int64_t key = rand_keys[0]; + const std::string key_str = Key(key); + + PinnableWideColumns columns_from_db; + PinnableAttributeGroups attribute_groups_from_db; ReadOptions read_opts_copy = read_opts; std::string read_ts_str; @@ -926,10 +926,25 @@ class NonBatchedOpsStressTest : public StressTest { read_ts_slice = read_ts_str; read_opts_copy.timestamp = &read_ts_slice; } - bool read_older_ts = MaybeUseOlderTimestampForPointLookup( + const bool read_older_ts = MaybeUseOlderTimestampForPointLookup( thread, read_ts_str, read_ts_slice, read_opts_copy); - const Status s = db_->GetEntity(read_opts_copy, cfh, key, &from_db); + const ExpectedValue pre_read_expected_value = + thread->shared->Get(column_family, key); + + Status s; + if (FLAGS_use_attribute_group) { + attribute_groups_from_db.emplace_back(cfh); + s = db_->GetEntity(read_opts_copy, key_str, &attribute_groups_from_db); + if (s.ok()) { + s = attribute_groups_from_db.back().status(); + } + } else { + s = db_->GetEntity(read_opts_copy, cfh, key_str, &columns_from_db); + } + + const ExpectedValue post_read_expected_value = + thread->shared->Get(column_family, key); int error_count = 0; @@ -953,37 +968,56 @@ class NonBatchedOpsStressTest : public StressTest { thread->stats.AddGets(1, 1); if (!FLAGS_skip_verifydb && !read_older_ts) { - const WideColumns& columns = from_db.columns(); - ExpectedValue expected = - shared->Get(rand_column_families[0], rand_keys[0]); + if (FLAGS_use_attribute_group) { + assert(!attribute_groups_from_db.empty()); + } + const WideColumns& columns = + FLAGS_use_attribute_group + ? attribute_groups_from_db.back().columns() + : columns_from_db.columns(); if (!VerifyWideColumns(columns)) { shared->SetVerificationFailure(); fprintf(stderr, "error : inconsistent columns returned by GetEntity for key " "%s: %s\n", - StringToHex(key).c_str(), WideColumnsToHex(columns).c_str()); - } else if (ExpectedValueHelper::MustHaveNotExisted(expected, - expected)) { + StringToHex(key_str).c_str(), + WideColumnsToHex(columns).c_str()); + } else if (ExpectedValueHelper::MustHaveNotExisted( + pre_read_expected_value, post_read_expected_value)) { shared->SetVerificationFailure(); fprintf( stderr, "error : inconsistent values for key %s: GetEntity returns %s, " "expected state does not have the key.\n", - StringToHex(key).c_str(), WideColumnsToHex(columns).c_str()); + StringToHex(key_str).c_str(), WideColumnsToHex(columns).c_str()); + } else { + const uint32_t value_base_from_db = + GetValueBase(WideColumnsHelper::GetDefaultColumn(columns)); + if (!ExpectedValueHelper::InExpectedValueBaseRange( + value_base_from_db, pre_read_expected_value, + post_read_expected_value)) { + shared->SetVerificationFailure(); + fprintf( + stderr, + "error : inconsistent values for key %s: GetEntity returns %s " + "with value base %d that falls out of expected state's value " + "base range.\n", + StringToHex(key_str).c_str(), WideColumnsToHex(columns).c_str(), + value_base_from_db); + } } } } else if (s.IsNotFound()) { thread->stats.AddGets(1, 0); if (!FLAGS_skip_verifydb && !read_older_ts) { - ExpectedValue expected = - shared->Get(rand_column_families[0], rand_keys[0]); - if (ExpectedValueHelper::MustHaveExisted(expected, expected)) { + if (ExpectedValueHelper::MustHaveExisted(pre_read_expected_value, + post_read_expected_value)) { shared->SetVerificationFailure(); fprintf(stderr, "error : inconsistent values for key %s: expected state has " "the key, GetEntity returns NotFound.\n", - StringToHex(key).c_str()); + StringToHex(key_str).c_str()); } } } else { @@ -1010,45 +1044,63 @@ class NonBatchedOpsStressTest : public StressTest { read_opts_copy.snapshot = snapshot_guard.snapshot(); assert(!rand_column_families.empty()); - assert(rand_column_families[0] >= 0); - assert(rand_column_families[0] < static_cast(column_families_.size())); - ColumnFamilyHandle* const cfh = column_families_[rand_column_families[0]]; + const int column_family = rand_column_families[0]; + + assert(column_family >= 0); + assert(column_family < static_cast(column_families_.size())); + + ColumnFamilyHandle* const cfh = column_families_[column_family]; assert(cfh); assert(!rand_keys.empty()); const size_t num_keys = rand_keys.size(); + std::unique_ptr txn; + + if (FLAGS_use_txn) { + WriteOptions write_options; + if (FLAGS_rate_limit_auto_wal_flush) { + write_options.rate_limiter_priority = Env::IO_USER; + } + + const Status s = NewTxn(write_options, &txn); + if (!s.ok()) { + fprintf(stderr, "NewTxn error: %s\n", s.ToString().c_str()); + thread->shared->SafeTerminate(); + } + } + std::vector keys(num_keys); std::vector key_slices(num_keys); + std::unordered_map ryw_expected_values; for (size_t i = 0; i < num_keys; ++i) { - keys[i] = Key(rand_keys[i]); - key_slices[i] = keys[i]; - } + const int64_t key = rand_keys[i]; - std::vector results(num_keys); - std::vector statuses(num_keys); + keys[i] = Key(key); + key_slices[i] = keys[i]; - if (fault_fs_guard) { - fault_fs_guard->EnableErrorInjection(); - SharedState::ignore_read_error = false; + if (FLAGS_use_txn) { + MaybeAddKeyToTxnForRYW(thread, column_family, key, txn.get(), + ryw_expected_values); + } } - db_->MultiGetEntity(read_opts_copy, cfh, num_keys, key_slices.data(), - results.data(), statuses.data()); - int error_count = 0; - if (fault_fs_guard) { + auto verify_expected_errors = [&](auto get_status) { + assert(fault_fs_guard); + error_count = fault_fs_guard->GetAndResetErrorCount(); if (error_count && !SharedState::ignore_read_error) { int stat_nok = 0; - for (const auto& s : statuses) { + for (size_t i = 0; i < num_keys; ++i) { + const Status& s = get_status(i); if (!s.ok() && !s.IsNotFound()) { - stat_nok++; + ++stat_nok; } } @@ -1066,89 +1118,238 @@ class NonBatchedOpsStressTest : public StressTest { std::terminate(); } } + }; - fault_fs_guard->DisableErrorInjection(); - } - - const bool check_get_entity = - !error_count && FLAGS_check_multiget_entity_consistency; + auto check_results = [&](auto get_columns, auto get_status, + auto do_extra_check, auto call_get_entity) { + 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]; + for (size_t i = 0; i < num_keys; ++i) { + const WideColumns& columns = get_columns(i); + const Status& s = get_status(i); - bool is_consistent = true; + bool is_consistent = true; - if (s.ok() && !VerifyWideColumns(results[i].columns())) { - fprintf( - stderr, - "error : inconsistent columns returned by MultiGetEntity for key " - "%s: %s\n", - StringToHex(keys[i]).c_str(), - WideColumnsToHex(results[i].columns()).c_str()); - is_consistent = false; - } else if (check_get_entity && (s.ok() || s.IsNotFound())) { - PinnableWideColumns cmp_result; - ThreadStatusUtil::SetThreadOperation( - ThreadStatus::OperationType::OP_GETENTITY); - const Status cmp_s = - db_->GetEntity(read_opts_copy, cfh, key_slices[i], &cmp_result); - - if (!cmp_s.ok() && !cmp_s.IsNotFound()) { - fprintf(stderr, "GetEntity error: %s\n", cmp_s.ToString().c_str()); + if (s.ok() && !VerifyWideColumns(columns)) { + fprintf( + stderr, + "error : inconsistent columns returned by MultiGetEntity for key " + "%s: %s\n", + StringToHex(keys[i]).c_str(), WideColumnsToHex(columns).c_str()); is_consistent = false; - } else if (cmp_s.IsNotFound()) { - if (s.ok()) { - fprintf(stderr, + } else if (s.ok() || s.IsNotFound()) { + if (!do_extra_check(keys[i], columns, s)) { + is_consistent = false; + } else if (check_get_entity) { + PinnableWideColumns cmp_result; + ThreadStatusUtil::SetThreadOperation( + ThreadStatus::OperationType::OP_GETENTITY); + const Status cmp_s = call_get_entity(key_slices[i], &cmp_result); + + if (!cmp_s.ok() && !cmp_s.IsNotFound()) { + fprintf(stderr, "GetEntity error: %s\n", + cmp_s.ToString().c_str()); + is_consistent = false; + } else if (cmp_s.IsNotFound()) { + if (s.ok()) { + fprintf( + stderr, "Inconsistent results for key %s: MultiGetEntity returned " "ok, GetEntity returned not found\n", StringToHex(keys[i]).c_str()); - is_consistent = false; - } - } else { - assert(cmp_s.ok()); + is_consistent = false; + } + } else { + assert(cmp_s.ok()); - if (s.IsNotFound()) { - fprintf(stderr, + if (s.IsNotFound()) { + fprintf( + stderr, "Inconsistent results for key %s: MultiGetEntity returned " "not found, GetEntity returned ok\n", StringToHex(keys[i]).c_str()); - is_consistent = false; + is_consistent = false; + } else { + assert(s.ok()); + + const WideColumns& cmp_columns = cmp_result.columns(); + + if (columns != cmp_columns) { + fprintf(stderr, + "Inconsistent results for key %s: MultiGetEntity " + "returned " + "%s, GetEntity returned %s\n", + StringToHex(keys[i]).c_str(), + WideColumnsToHex(columns).c_str(), + WideColumnsToHex(cmp_columns).c_str()); + is_consistent = false; + } + } + } + } + } + + if (!is_consistent) { + fprintf(stderr, + "TestMultiGetEntity error: results are not consistent\n"); + thread->stats.AddErrors(1); + // Fail fast to preserve the DB state + thread->shared->SetVerificationFailure(); + break; + } else if (s.ok()) { + thread->stats.AddGets(1, 1); + } else if (s.IsNotFound()) { + thread->stats.AddGets(1, 0); + } else { + if (error_count == 0) { + fprintf(stderr, "MultiGetEntity error: %s\n", s.ToString().c_str()); + thread->stats.AddErrors(1); } else { - assert(s.ok()); + thread->stats.AddVerifiedErrors(1); + } + } + } + }; + + if (FLAGS_use_txn) { + // Transactional/read-your-own-writes MultiGetEntity verification + std::vector results(num_keys); + std::vector statuses(num_keys); + + assert(txn); + txn->MultiGetEntity(read_opts_copy, cfh, num_keys, key_slices.data(), + results.data(), statuses.data()); + + auto ryw_check = [&](const std::string& key, const WideColumns& columns, + const Status& s) -> bool { + const auto it = ryw_expected_values.find(key); + if (it == ryw_expected_values.end()) { + return true; + } - if (results[i] != cmp_result) { + const auto& ryw_expected_value = it->second; + + if (s.ok()) { + if (ryw_expected_value.IsDeleted()) { + fprintf( + stderr, + "MultiGetEntity failed the read-your-own-write check for key " + "%s\n", + Slice(key).ToString(true).c_str()); + fprintf(stderr, + "MultiGetEntity returned ok, transaction has non-committed " + "delete\n"); + return false; + } else { + const uint32_t value_base = ryw_expected_value.GetValueBase(); + char expected_value[100]; + const size_t sz = GenerateValue(value_base, expected_value, + sizeof(expected_value)); + const Slice expected_slice(expected_value, sz); + const WideColumns expected_columns = + GenerateExpectedWideColumns(value_base, expected_slice); + + if (columns != expected_columns) { fprintf( stderr, - "Inconsistent results for key %s: MultiGetEntity returned " - "%s, GetEntity returned %s\n", - StringToHex(keys[i]).c_str(), - WideColumnsToHex(results[i].columns()).c_str(), - WideColumnsToHex(cmp_result.columns()).c_str()); - is_consistent = false; + "MultiGetEntity failed the read-your-own-write check for key " + "%s\n", + Slice(key).ToString(true).c_str()); + fprintf(stderr, "MultiGetEntity returned %s\n", + WideColumnsToHex(columns).c_str()); + fprintf(stderr, "Transaction has non-committed write %s\n", + WideColumnsToHex(expected_columns).c_str()); + return false; } + + return true; } } - } - if (!is_consistent) { - fprintf(stderr, - "TestMultiGetEntity error: results are not consistent\n"); - thread->stats.AddErrors(1); - // Fail fast to preserve the DB state - thread->shared->SetVerificationFailure(); - break; - } else if (s.ok()) { - thread->stats.AddGets(1, 1); - } else if (s.IsNotFound()) { - thread->stats.AddGets(1, 0); - } else { - if (error_count == 0) { - fprintf(stderr, "MultiGetEntity error: %s\n", s.ToString().c_str()); - thread->stats.AddErrors(1); - } else { - thread->stats.AddVerifiedErrors(1); + assert(s.IsNotFound()); + if (!ryw_expected_value.IsDeleted()) { + fprintf(stderr, + "MultiGetEntity failed the read-your-own-write check for key " + "%s\n", + Slice(key).ToString(true).c_str()); + fprintf(stderr, + "MultiGetEntity returned not found, transaction has " + "non-committed write\n"); + return false; } + + return true; + }; + + check_results([&](size_t i) { return results[i].columns(); }, + [&](size_t i) { return statuses[i]; }, ryw_check, + [&](const Slice& key, PinnableWideColumns* result) { + return txn->GetEntity(read_opts_copy, cfh, key, result); + }); + + txn->Rollback().PermitUncheckedError(); + } else if (FLAGS_use_attribute_group) { + // AttributeGroup MultiGetEntity verification + + if (fault_fs_guard) { + fault_fs_guard->EnableErrorInjection(); + SharedState::ignore_read_error = false; + } + + std::vector results; + results.reserve(num_keys); + for (size_t i = 0; i < num_keys; ++i) { + PinnableAttributeGroups attribute_groups; + attribute_groups.emplace_back(cfh); + results.emplace_back(std::move(attribute_groups)); + } + + db_->MultiGetEntity(read_opts_copy, num_keys, key_slices.data(), + results.data()); + + if (fault_fs_guard) { + verify_expected_errors( + [&](size_t i) { return results[i][0].status(); }); + + fault_fs_guard->DisableErrorInjection(); + } + + // Compare against non-attribute-group GetEntity result + check_results([&](size_t i) { return results[i][0].columns(); }, + [&](size_t i) { return results[i][0].status(); }, + [](const Slice& /* key */, const WideColumns& /* columns */, + const Status& /* s */) { return true; }, + [&](const Slice& key, PinnableWideColumns* result) { + return db_->GetEntity(read_opts_copy, cfh, key, result); + }); + } else { + // Non-AttributeGroup MultiGetEntity verification + + if (fault_fs_guard) { + fault_fs_guard->EnableErrorInjection(); + SharedState::ignore_read_error = false; + } + + std::vector results(num_keys); + std::vector statuses(num_keys); + + db_->MultiGetEntity(read_opts_copy, cfh, num_keys, key_slices.data(), + results.data(), statuses.data()); + + if (fault_fs_guard) { + verify_expected_errors([&](size_t i) { return statuses[i]; }); + + fault_fs_guard->DisableErrorInjection(); } + + check_results([&](size_t i) { return results[i].columns(); }, + [&](size_t i) { return statuses[i]; }, + [](const Slice& /* key */, const WideColumns& /* columns */, + const Status& /* s */) { return true; }, + [&](const Slice& key, PinnableWideColumns* result) { + return db_->GetEntity(read_opts_copy, cfh, key, result); + }); } } @@ -1302,8 +1503,19 @@ class NonBatchedOpsStressTest : public StressTest { if (FLAGS_use_put_entity_one_in > 0 && (value_base % FLAGS_use_put_entity_one_in) == 0) { - s = db_->PutEntity(write_opts, cfh, k, - GenerateWideColumns(value_base, v)); + if (!FLAGS_use_txn) { + if (FLAGS_use_attribute_group) { + s = db_->PutEntity(write_opts, k, + GenerateAttributeGroups({cfh}, value_base, v)); + } else { + s = db_->PutEntity(write_opts, cfh, k, + GenerateWideColumns(value_base, v)); + } + } else { + s = ExecuteTransaction(write_opts, thread, [&](Transaction& txn) { + return txn.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) { @@ -1702,7 +1914,19 @@ class NonBatchedOpsStressTest : public StressTest { pre_read_expected_values.push_back( shared->Get(rand_column_family, i + lb)); } - std::unique_ptr iter(db_->NewIterator(ro, cfh)); + std::unique_ptr iter; + if (FLAGS_use_multi_cf_iterator) { + std::vector cfhs; + cfhs.reserve(rand_column_families.size()); + for (auto cf_index : rand_column_families) { + cfhs.emplace_back(column_families_[cf_index]); + } + assert(!cfhs.empty()); + iter = db_->NewCoalescingIterator(ro, cfhs); + } else { + iter = std::unique_ptr(db_->NewIterator(ro, cfh)); + } + for (int64_t i = 0; i < static_cast(expected_values_size); ++i) { post_read_expected_values.push_back( shared->Get(rand_column_family, i + lb)); @@ -1892,9 +2116,10 @@ class NonBatchedOpsStressTest : public StressTest { op_logs += "P"; } - // Write-prepared and Write-unprepared do not support Refresh() yet. + // Write-prepared/write-unprepared transactions and multi-CF iterator do not + // support Refresh() yet. if (!(FLAGS_use_txn && FLAGS_txn_write_policy != 0) && - thread->rand.OneIn(2)) { + !FLAGS_use_multi_cf_iterator && thread->rand.OneIn(2)) { pre_read_expected_values.clear(); post_read_expected_values.clear(); // Refresh after forward/backward scan to allow higher chance of SV @@ -2079,7 +2304,7 @@ class NonBatchedOpsStressTest : public StressTest { return Status::OK(); } - bool VerifyOrSyncValue(int cf, int64_t key, const ReadOptions& /*opts*/, + bool VerifyOrSyncValue(int cf, int64_t key, const ReadOptions& opts, SharedState* shared, const std::string& value_from_db, std::string msg_prefix, const Status& s) const { if (shared->HasVerificationFailedYet()) { @@ -2105,27 +2330,43 @@ class NonBatchedOpsStressTest : public StressTest { GenerateValue(expected_value.GetValueBase(), expected_value_data, sizeof(expected_value_data)); + std::ostringstream read_u64ts; + if (opts.timestamp) { + read_u64ts << " while read with timestamp: "; + uint64_t read_ts; + if (DecodeU64Ts(*opts.timestamp, &read_ts).ok()) { + read_u64ts << std::to_string(read_ts) << ", "; + } else { + read_u64ts << s.ToString() + << " Encoded read timestamp: " << opts.timestamp->ToString() + << ", "; + } + } + // compare value_from_db with the value in the shared state if (s.ok()) { const Slice slice(value_from_db); const uint32_t value_base_from_db = GetValueBase(slice); if (ExpectedValueHelper::MustHaveNotExisted(expected_value, expected_value)) { - VerificationAbort(shared, msg_prefix + ": Unexpected value found", cf, - key, value_from_db, ""); + VerificationAbort( + shared, msg_prefix + ": Unexpected value found" + read_u64ts.str(), + cf, key, value_from_db, ""); return false; } if (!ExpectedValueHelper::InExpectedValueBaseRange( value_base_from_db, expected_value, expected_value)) { - VerificationAbort(shared, msg_prefix + ": Unexpected value found", cf, - key, value_from_db, - Slice(expected_value_data, expected_value_data_size)); + VerificationAbort( + shared, msg_prefix + ": Unexpected value found" + read_u64ts.str(), + cf, key, value_from_db, + Slice(expected_value_data, expected_value_data_size)); return false; } // TODO: are the length/memcmp() checks repetitive? if (value_from_db.length() != expected_value_data_size) { VerificationAbort(shared, - msg_prefix + ": Length of value read is not equal", + msg_prefix + ": Length of value read is not equal" + + read_u64ts.str(), cf, key, value_from_db, Slice(expected_value_data, expected_value_data_size)); return false; @@ -2133,7 +2374,8 @@ class NonBatchedOpsStressTest : public StressTest { if (memcmp(value_from_db.data(), expected_value_data, expected_value_data_size) != 0) { VerificationAbort(shared, - msg_prefix + ": Contents of value read don't match", + msg_prefix + ": Contents of value read don't match" + + read_u64ts.str(), cf, key, value_from_db, Slice(expected_value_data, expected_value_data_size)); return false; @@ -2142,14 +2384,16 @@ class NonBatchedOpsStressTest : public StressTest { if (ExpectedValueHelper::MustHaveExisted(expected_value, expected_value)) { VerificationAbort( - shared, msg_prefix + ": Value not found: " + s.ToString(), cf, key, - "", Slice(expected_value_data, expected_value_data_size)); + shared, + msg_prefix + ": Value not found " + read_u64ts.str() + s.ToString(), + cf, key, "", Slice(expected_value_data, expected_value_data_size)); return false; } } else { - VerificationAbort(shared, msg_prefix + "Non-OK status: " + s.ToString(), - cf, key, "", - Slice(expected_value_data, expected_value_data_size)); + VerificationAbort( + shared, + msg_prefix + "Non-OK status " + read_u64ts.str() + s.ToString(), cf, + key, "", Slice(expected_value_data, expected_value_data_size)); return false; } return true; @@ -2167,6 +2411,93 @@ class NonBatchedOpsStressTest : public StressTest { return !shared->AllowsOverwrite(key_num); }; } + + void MaybeAddKeyToTxnForRYW( + ThreadState* thread, int column_family, int64_t key, Transaction* txn, + std::unordered_map& ryw_expected_values) { + assert(thread); + assert(txn); + + SharedState* const shared = thread->shared; + assert(shared); + + if (!shared->AllowsOverwrite(key) && shared->Exists(column_family, key)) { + // Just do read your write checks for keys that allow overwrites. + return; + } + + // With a 1 in 10 probability, insert the just added key in the batch + // into the transaction. This will create an overlap with the MultiGet + // keys and exercise some corner cases in the code + if (thread->rand.OneIn(10)) { + assert(column_family >= 0); + assert(column_family < static_cast(column_families_.size())); + + ColumnFamilyHandle* const cfh = column_families_[column_family]; + assert(cfh); + + const std::string k = Key(key); + + enum class Op { + PutOrPutEntity, + Merge, + Delete, + // add new operations above this line + NumberOfOps + }; + + const Op op = static_cast( + thread->rand.Uniform(static_cast(Op::NumberOfOps))); + + Status s; + + switch (op) { + case Op::PutOrPutEntity: + case Op::Merge: { + ExpectedValue put_value; + put_value.SyncPut(static_cast(thread->rand.Uniform( + static_cast(ExpectedValue::GetValueBaseMask())))); + ryw_expected_values[k] = put_value; + + const uint32_t value_base = put_value.GetValueBase(); + + char value[100]; + const size_t sz = GenerateValue(value_base, value, sizeof(value)); + const Slice v(value, sz); + + if (op == Op::PutOrPutEntity) { + if (FLAGS_use_put_entity_one_in > 0 && + (value_base % FLAGS_use_put_entity_one_in) == 0) { + s = txn->PutEntity(cfh, k, GenerateWideColumns(value_base, v)); + } else { + s = txn->Put(cfh, k, v); + } + } else { + s = txn->Merge(cfh, k, v); + } + + break; + } + case Op::Delete: { + ExpectedValue delete_value; + delete_value.SyncDelete(); + ryw_expected_values[k] = delete_value; + + s = txn->Delete(cfh, k); + break; + } + default: + assert(false); + } + + if (!s.ok()) { + fprintf(stderr, + "Transaction write error in read-your-own-write test: %s\n", + s.ToString().c_str()); + shared->SafeTerminate(); + } + } + } }; StressTest* CreateNonBatchedOpsStressTest() { diff --git a/env/fs_on_demand.cc b/env/fs_on_demand.cc index bac424264a3..a2f23f5c41a 100644 --- a/env/fs_on_demand.cc +++ b/env/fs_on_demand.cc @@ -5,6 +5,7 @@ #include "env/fs_on_demand.h" +#include #include #include "file/filename.h" diff --git a/env/io_posix.cc b/env/io_posix.cc index 29efb055bae..f31ee7d167a 100644 --- a/env/io_posix.cc +++ b/env/io_posix.cc @@ -1441,10 +1441,12 @@ void PosixWritableFile::SetWriteLifeTimeHint(Env::WriteLifeTimeHint hint) { #ifdef OS_LINUX // Suppress Valgrind "Unimplemented functionality" error. #ifndef ROCKSDB_VALGRIND_RUN + uint64_t fcntl_hint = hint; + if (hint == write_hint_) { return; } - if (fcntl(fd_, F_SET_RW_HINT, &hint) == 0) { + if (fcntl(fd_, F_SET_RW_HINT, &fcntl_hint) == 0) { write_hint_ = hint; } #else diff --git a/file/writable_file_writer.cc b/file/writable_file_writer.cc index d36023c0e00..2859f66299a 100644 --- a/file/writable_file_writer.cc +++ b/file/writable_file_writer.cc @@ -64,7 +64,7 @@ IOStatus WritableFileWriter::Create(const std::shared_ptr& fs, IOStatus WritableFileWriter::Append(const IOOptions& opts, const Slice& data, uint32_t crc32c_checksum) { if (seen_error()) { - return AssertFalseAndGetStatusForPrevError(); + return GetWriterHasPreviousErrorStatus(); } StopWatch sw(clock_, stats_, hist_type_, @@ -199,7 +199,7 @@ IOStatus WritableFileWriter::Append(const IOOptions& opts, const Slice& data, IOStatus WritableFileWriter::Pad(const IOOptions& opts, const size_t pad_bytes) { if (seen_error()) { - return AssertFalseAndGetStatusForPrevError(); + return GetWriterHasPreviousErrorStatus(); } const IOOptions io_options = FinalizeIOOptions(opts); assert(pad_bytes < kDefaultPageSize); @@ -348,7 +348,7 @@ IOStatus WritableFileWriter::Close(const IOOptions& opts) { // enabled IOStatus WritableFileWriter::Flush(const IOOptions& opts) { if (seen_error()) { - return AssertFalseAndGetStatusForPrevError(); + return GetWriterHasPreviousErrorStatus(); } const IOOptions io_options = FinalizeIOOptions(opts); @@ -458,7 +458,7 @@ IOStatus WritableFileWriter::PrepareIOOptions(const WriteOptions& wo, IOStatus WritableFileWriter::Sync(const IOOptions& opts, bool use_fsync) { if (seen_error()) { - return AssertFalseAndGetStatusForPrevError(); + return GetWriterHasPreviousErrorStatus(); } IOOptions io_options = FinalizeIOOptions(opts); @@ -483,7 +483,7 @@ IOStatus WritableFileWriter::Sync(const IOOptions& opts, bool use_fsync) { IOStatus WritableFileWriter::SyncWithoutFlush(const IOOptions& opts, bool use_fsync) { if (seen_error()) { - return AssertFalseAndGetStatusForPrevError(); + return GetWriterHasPreviousErrorStatus(); } IOOptions io_options = FinalizeIOOptions(opts); if (!writable_file_->IsSyncThreadSafe()) { @@ -495,9 +495,6 @@ IOStatus WritableFileWriter::SyncWithoutFlush(const IOOptions& opts, IOStatus s = SyncInternal(io_options, use_fsync); TEST_SYNC_POINT("WritableFileWriter::SyncWithoutFlush:2"); if (!s.ok()) { -#ifndef NDEBUG - sync_without_flush_called_ = true; -#endif // NDEBUG set_seen_error(); } return s; @@ -543,7 +540,7 @@ IOStatus WritableFileWriter::SyncInternal(const IOOptions& opts, IOStatus WritableFileWriter::RangeSync(const IOOptions& opts, uint64_t offset, uint64_t nbytes) { if (seen_error()) { - return AssertFalseAndGetStatusForPrevError(); + return GetWriterHasPreviousErrorStatus(); } IOSTATS_TIMER_GUARD(range_sync_nanos); @@ -572,7 +569,7 @@ IOStatus WritableFileWriter::RangeSync(const IOOptions& opts, uint64_t offset, IOStatus WritableFileWriter::WriteBuffered(const IOOptions& opts, const char* data, size_t size) { if (seen_error()) { - return AssertFalseAndGetStatusForPrevError(); + return GetWriterHasPreviousErrorStatus(); } IOStatus s; @@ -663,7 +660,7 @@ IOStatus WritableFileWriter::WriteBufferedWithChecksum(const IOOptions& opts, const char* data, size_t size) { if (seen_error()) { - return AssertFalseAndGetStatusForPrevError(); + return GetWriterHasPreviousErrorStatus(); } IOStatus s; @@ -877,7 +874,7 @@ IOStatus WritableFileWriter::WriteDirect(const IOOptions& opts) { IOStatus WritableFileWriter::WriteDirectWithChecksum(const IOOptions& opts) { if (seen_error()) { - return AssertFalseAndGetStatusForPrevError(); + return GetWriterHasPreviousErrorStatus(); } assert(use_direct_io()); diff --git a/file/writable_file_writer.h b/file/writable_file_writer.h index 6b71cfa64c6..40ae239cddd 100644 --- a/file/writable_file_writer.h +++ b/file/writable_file_writer.h @@ -149,13 +149,6 @@ class WritableFileWriter { uint64_t next_write_offset_; bool pending_sync_; std::atomic seen_error_; -#ifndef NDEBUG - // SyncWithoutFlush() is the function that is allowed to be called - // concurrently with other function. One of the concurrent call - // could set seen_error_, and the other one would hit assertion - // in debug mode. - std::atomic sync_without_flush_called_ = false; -#endif // NDEBUG uint64_t last_sync_size_; uint64_t bytes_per_sync_; RateLimiter* rate_limiter_; @@ -304,9 +297,7 @@ class WritableFileWriter { } void set_seen_error() { seen_error_.store(true, std::memory_order_relaxed); } - IOStatus AssertFalseAndGetStatusForPrevError() { - // This should only happen if SyncWithoutFlush() was called. - assert(sync_without_flush_called_); + IOStatus GetWriterHasPreviousErrorStatus() { return IOStatus::IOError("Writer has previous error."); } diff --git a/include/rocksdb/c.h b/include/rocksdb/c.h index 9de8770965f..090b156f052 100644 --- a/include/rocksdb/c.h +++ b/include/rocksdb/c.h @@ -727,6 +727,8 @@ extern ROCKSDB_LIBRARY_API const char* rocksdb_iter_timestamp( const rocksdb_iterator_t*, size_t* tslen); extern ROCKSDB_LIBRARY_API void rocksdb_iter_get_error( const rocksdb_iterator_t*, char** errptr); +extern ROCKSDB_LIBRARY_API void rocksdb_iter_refresh( + const rocksdb_iterator_t* iter, char** errptr); extern ROCKSDB_LIBRARY_API void rocksdb_wal_iter_next( rocksdb_wal_iterator_t* iter); @@ -747,6 +749,10 @@ extern ROCKSDB_LIBRARY_API rocksdb_writebatch_t* rocksdb_writebatch_create( void); extern ROCKSDB_LIBRARY_API rocksdb_writebatch_t* rocksdb_writebatch_create_from( const char* rep, size_t size); +extern ROCKSDB_LIBRARY_API rocksdb_writebatch_t* +rocksdb_writebatch_create_with_params(size_t reserved_bytes, size_t max_bytes, + size_t protection_bytes_per_key, + size_t default_cf_ts_sz); extern ROCKSDB_LIBRARY_API void rocksdb_writebatch_destroy( rocksdb_writebatch_t*); extern ROCKSDB_LIBRARY_API void rocksdb_writebatch_clear(rocksdb_writebatch_t*); @@ -842,6 +848,9 @@ extern ROCKSDB_LIBRARY_API void rocksdb_writebatch_rollback_to_save_point( rocksdb_writebatch_t*, char** errptr); extern ROCKSDB_LIBRARY_API void rocksdb_writebatch_pop_save_point( rocksdb_writebatch_t*, char** errptr); +extern ROCKSDB_LIBRARY_API void rocksdb_writebatch_update_timestamps( + rocksdb_writebatch_t* wb, const char* ts, size_t tslen, void* state, + size_t (*get_ts_size)(void*, uint32_t), char** errptr); /* Write batch with index */ @@ -850,6 +859,11 @@ rocksdb_writebatch_wi_create(size_t reserved_bytes, unsigned char overwrite_keys); extern ROCKSDB_LIBRARY_API rocksdb_writebatch_wi_t* rocksdb_writebatch_wi_create_from(const char* rep, size_t size); +extern ROCKSDB_LIBRARY_API rocksdb_writebatch_wi_t* +rocksdb_writebatch_wi_create_with_params( + rocksdb_comparator_t* backup_index_comparator, size_t reserved_bytes, + unsigned char overwrite_key, size_t max_bytes, + size_t protection_bytes_per_key); extern ROCKSDB_LIBRARY_API void rocksdb_writebatch_wi_destroy( rocksdb_writebatch_wi_t*); extern ROCKSDB_LIBRARY_API void rocksdb_writebatch_wi_clear( @@ -960,6 +974,9 @@ extern ROCKSDB_LIBRARY_API rocksdb_iterator_t* rocksdb_writebatch_wi_create_iterator_with_base_cf( rocksdb_writebatch_wi_t* wbwi, rocksdb_iterator_t* base_iterator, rocksdb_column_family_handle_t* cf); +extern ROCKSDB_LIBRARY_API void rocksdb_writebatch_wi_update_timestamps( + rocksdb_writebatch_wi_t* wbwi, const char* ts, size_t tslen, void* state, + size_t (*get_ts_size)(void*, uint32_t), char** errptr); /* Options utils */ @@ -1064,6 +1081,21 @@ rocksdb_block_based_options_set_pin_top_level_index_and_filter( rocksdb_block_based_table_options_t*, unsigned char); extern ROCKSDB_LIBRARY_API void rocksdb_options_set_block_based_table_factory( rocksdb_options_t* opt, rocksdb_block_based_table_options_t* table_options); +enum { + rocksdb_block_based_k_fallback_pinning_tier = 0, + rocksdb_block_based_k_none_pinning_tier = 1, + rocksdb_block_based_k_flush_and_similar_pinning_tier = 2, + rocksdb_block_based_k_all_pinning_tier = 3, +}; +extern ROCKSDB_LIBRARY_API void +rocksdb_block_based_options_set_top_level_index_pinning_tier( + rocksdb_block_based_table_options_t*, int); +extern ROCKSDB_LIBRARY_API void +rocksdb_block_based_options_set_partition_pinning_tier( + rocksdb_block_based_table_options_t*, int); +extern ROCKSDB_LIBRARY_API void +rocksdb_block_based_options_set_unpartitioned_pinning_tier( + rocksdb_block_based_table_options_t*, int); extern ROCKSDB_LIBRARY_API void rocksdb_options_set_write_buffer_manager( rocksdb_options_t* opt, rocksdb_write_buffer_manager_t* wbm); @@ -1682,6 +1714,18 @@ extern ROCKSDB_LIBRARY_API void rocksdb_options_set_wal_compression( extern ROCKSDB_LIBRARY_API int rocksdb_options_get_wal_compression( rocksdb_options_t* opt); +enum { + rocksdb_k_by_compensated_size_compaction_pri = 0, + rocksdb_k_oldest_largest_seq_first_compaction_pri = 1, + rocksdb_k_oldest_smallest_seq_first_compaction_pri = 2, + rocksdb_k_min_overlapping_ratio_compaction_pri = 3, + rocksdb_k_round_robin_compaction_pri = 4 +}; +extern ROCKSDB_LIBRARY_API void rocksdb_options_set_compaction_pri( + rocksdb_options_t*, int); +extern ROCKSDB_LIBRARY_API int rocksdb_options_get_compaction_pri( + rocksdb_options_t*); + /* RateLimiter */ extern ROCKSDB_LIBRARY_API rocksdb_ratelimiter_t* rocksdb_ratelimiter_create( int64_t rate_bytes_per_sec, int64_t refill_period_us, int32_t fairness); diff --git a/include/rocksdb/cache.h b/include/rocksdb/cache.h index 59805881f5e..54e9e88aacb 100644 --- a/include/rocksdb/cache.h +++ b/include/rocksdb/cache.h @@ -523,6 +523,11 @@ enum TieredAdmissionPolicy { // compressed secondary, and a compressed local flash (non-volatile) cache. // Each tier is managed as an independent queue. kAdmPolicyThreeQueue, + // Allow all blocks evicted from the primary block cache into the secondary + // cache. This may increase CPU overhead due to more blocks being admitted + // and compressed, but may increase the compressed secondary cache hit rate + // for some workloads + kAdmPolicyAllowAll, kAdmPolicyMax, }; diff --git a/include/rocksdb/comparator.h b/include/rocksdb/comparator.h index 087610efd5e..c1ea96b5924 100644 --- a/include/rocksdb/comparator.h +++ b/include/rocksdb/comparator.h @@ -142,6 +142,11 @@ class Comparator : public Customizable, public CompareInterface { return Slice(); } + // Return a human readable user-defined timestamp for debugging. + virtual std::string TimestampToString(const Slice& /*timestamp*/) const { + return ""; + } + 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 ecfae5b6939..b2648ecc7cd 100644 --- a/include/rocksdb/db.h +++ b/include/rocksdb/db.h @@ -28,6 +28,7 @@ #include "rocksdb/thread_status.h" #include "rocksdb/transaction_log.h" #include "rocksdb/types.h" +#include "rocksdb/user_write_callback.h" #include "rocksdb/version.h" #include "rocksdb/wide_columns.h" @@ -583,6 +584,15 @@ class DB { // Note: consider setting options.sync = true. virtual Status Write(const WriteOptions& options, WriteBatch* updates) = 0; + // Same as DB::Write, and takes a `UserWriteCallback` argument to allow + // users to plug in custom logic in callback functions during the write. + virtual Status WriteWithCallback(const WriteOptions& /*options*/, + WriteBatch* /*updates*/, + UserWriteCallback* /*user_write_cb*/) { + return Status::NotSupported( + "WriteWithCallback not implemented for this interface."); + } + // If the column family specified by "column_family" contains an entry for // "key", return the corresponding value in "*value". If the entry is a plain // key-value, return the value as-is; if it is a wide-column entity, return @@ -1475,6 +1485,9 @@ class DB { // move the files back to the minimum level capable of holding the data set // or a given level (specified by non-negative options.target_level). // + // For FIFO compaction, this will trigger a compaction (if available) + // based on CompactionOptionsFIFO. + // // In case of user-defined timestamp, if enabled, `begin` and `end` should // not contain timestamp. virtual Status CompactRange(const CompactRangeOptions& options, @@ -1676,8 +1689,8 @@ class DB { // Freezes the logical state of the DB (by stopping writes), and if WAL is // enabled, ensures that state has been flushed to DB files (as in // FlushWAL()). This can be used for taking a Checkpoint at a known DB - // state, though the user must use options to insure no DB flush is invoked - // in this frozen state. Other operations allowed on a "read only" DB should + // state, though while the WAL is locked, flushes as part of CreateCheckpoint + // and simiar are skipped. Other operations allowed on a "read only" DB should // work while frozen. Each LockWAL() call that returns OK must eventually be // followed by a corresponding call to UnlockWAL(). Where supported, non-OK // status is generally only possible with some kind of corruption or I/O @@ -1828,7 +1841,7 @@ class DB { bool flush_memtable = true) = 0; // Retrieve the sorted list of all wal files with earliest file first - virtual Status GetSortedWalFiles(VectorLogPtr& files) = 0; + virtual Status GetSortedWalFiles(VectorWalPtr& files) = 0; // Retrieve information about the current wal file // @@ -1838,7 +1851,7 @@ class DB { // Additionally, for the sake of optimization current_log_file->StartSequence // would always be set to 0 virtual Status GetCurrentWalFile( - std::unique_ptr* current_log_file) = 0; + std::unique_ptr* current_log_file) = 0; // IngestExternalFile() will load a list of external SST files (1) into the DB // Two primary modes are supported: @@ -1858,6 +1871,7 @@ class DB { // supported. 4) When an ingested file contains point data and range deletion // for the same key, the point data currently overrides the range deletion // regardless which one has the higher user-defined timestamps. + // For FIFO compaction, SST files will always be ingested into L0. // // (1) External SST files can be created using SstFileWriter // (2) We will try to ingest the files to the lowest possible level @@ -2005,6 +2019,8 @@ class DB { return Status::NotSupported("SuggestCompactRange() is not implemented."); } + // Trivially move L0 files to target level. Should not be called with another + // PromoteL0() concurrently virtual Status PromoteL0(ColumnFamilyHandle* /*column_family*/, int /*target_level*/) { return Status::NotSupported("PromoteL0() is not implemented."); diff --git a/include/rocksdb/iterator.h b/include/rocksdb/iterator.h index 0334aa7511e..b6e1d866728 100644 --- a/include/rocksdb/iterator.h +++ b/include/rocksdb/iterator.h @@ -32,7 +32,7 @@ class Iterator : public IteratorBase { Iterator(const Iterator&) = delete; void operator=(const Iterator&) = delete; - virtual ~Iterator() {} + virtual ~Iterator() override {} // 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 @@ -78,6 +78,12 @@ class Iterator : public IteratorBase { // - Iterator created with ReadOptions::pin_data = true // - DB tables were created with // BlockBasedTableOptions::use_delta_encoding = false. + // Property "rocksdb.iterator.is-value-pinned": + // If returning "1", this means that the Slice returned by value() is valid + // as long as the iterator is not deleted. + // It is guaranteed to always return "1" if + // - Iterator created with ReadOptions::pin_data = true + // - The value is found in a `kTypeValue` record // Property "rocksdb.iterator.super-version-number": // LSM version used by the iterator. The same format as DB Property // kCurrentSuperVersionNumber. See its comment for more information. diff --git a/include/rocksdb/options.h b/include/rocksdb/options.h index 459ba1d5172..91c18464647 100644 --- a/include/rocksdb/options.h +++ b/include/rocksdb/options.h @@ -234,6 +234,12 @@ struct ColumnFamilyOptions : public AdvancedColumnFamilyOptions { // Number of files to trigger level-0 compaction. A value <0 means that // level-0 compaction will not be triggered by number of files at all. // + // Universal compaction: RocksDB will try to keep the number of sorted runs + // no more than this number. If CompactionOptionsUniversal::max_read_amp is + // set, then this option will be used only as a trigger to look for + // compaction. CompactionOptionsUniversal::max_read_amp will be the limit + // on the number of sorted runs. + // // Default: 4 // // Dynamically changeable through SetOptions() API @@ -1265,6 +1271,16 @@ struct DBOptions { // Dynamically changeable through SetDBOptions() API. bool avoid_flush_during_shutdown = false; + // By default RocksDB will not flush (if `manual_wal_flush` = true) and sync + // WAL on DB close even if there are unpersisted data (i.e. unflushed or + // unsynced WAL data). This can speed up DB close. Unpersisted data WILL BE + // LOST. + // + // DEFAULT: true + // + // Dynamically changeable through SetDBOptions() API. + bool avoid_sync_during_shutdown = true; + // Set this option to true during creation of database if you want // to be able to ingest behind (call IngestExternalFile() skipping keys // that already exist, rather than overwriting matching keys). diff --git a/include/rocksdb/transaction_log.h b/include/rocksdb/transaction_log.h index e13ad8f80a5..d3e33574863 100644 --- a/include/rocksdb/transaction_log.h +++ b/include/rocksdb/transaction_log.h @@ -14,8 +14,10 @@ namespace ROCKSDB_NAMESPACE { -class LogFile; -using VectorLogPtr = std::vector>; +class WalFile; +using VectorWalPtr = std::vector>; +// DEPRECATED old name +using VectorLogPtr = VectorWalPtr; enum WalFileType { /* Indicates that WAL file is in archive directory. WAL files are moved from @@ -30,10 +32,10 @@ enum WalFileType { kAliveLogFile = 1 }; -class LogFile { +class WalFile { public: - LogFile() {} - virtual ~LogFile() {} + WalFile() {} + virtual ~WalFile() {} // Returns log file's pathname relative to the main db dir // Eg. For a live-log-file = /000003.log @@ -50,10 +52,14 @@ class LogFile { // Starting sequence number of writebatch written in this log file virtual SequenceNumber StartSequence() const = 0; - // Size of log file on disk in Bytes + // The position of the last flushed write to the file (which for + // recycled WAL files is typically less than the full file size). virtual uint64_t SizeFileBytes() const = 0; }; +// DEPRECATED old name for WalFile. (Confusing with "Logger" etc.) +using LogFile = WalFile; + struct BatchResult { SequenceNumber sequence = 0; std::unique_ptr writeBatchPtr; diff --git a/include/rocksdb/types.h b/include/rocksdb/types.h index 3b9791c4824..dceacbbefc6 100644 --- a/include/rocksdb/types.h +++ b/include/rocksdb/types.h @@ -67,6 +67,7 @@ enum EntryType { kEntryBlobIndex, kEntryDeleteWithTimestamp, kEntryWideColumnEntity, + kEntryTimedPut, // That hasn't yet converted to a standard Put entry kEntryOther, }; diff --git a/include/rocksdb/universal_compaction.h b/include/rocksdb/universal_compaction.h index 0b0a85e1c86..c52d8363319 100644 --- a/include/rocksdb/universal_compaction.h +++ b/include/rocksdb/universal_compaction.h @@ -65,6 +65,36 @@ class CompactionOptionsUniversal { // Default: -1 int compression_size_percent; + // The limit on the number of sorted runs. RocksDB will try to keep + // the number of sorted runs at most this number. While compactions are + // running, the number of sorted runs may be temporarily higher than + // this number. + // + // Since universal compaction checks if there is compaction to do when + // the number of sorted runs is at least level0_file_num_compaction_trigger, + // it is suggested to set level0_file_num_compaction_trigger to be no larger + // than max_read_amp. + // + // Values: + // -1: special flag to let RocksDB pick default. Currently, + // RocksDB will fall back to the behavior before this option is introduced, + // which is to use level0_file_num_compaction_trigger as the limit. + // This may change in the future to behave as 0 below. + // 0: Let RocksDB auto-tune. Currently, we determine the max number of + // sorted runs based on the current DB size, size_ratio and + // write_buffer_size. Note that this is only supported for the default + // stop_style kCompactionStopStyleTotalSize. For + // kCompactionStopStyleSimilarSize, this behaves as if -1 is configured. + // N > 0: limit the number of sorted runs to be at most N. + // N should be at least the compaction trigger specified by + // level0_file_num_compaction_trigger. If 0 < max_read_amp < + // level0_file_num_compaction_trigger, Status::NotSupported() will be + // returned during DB open. + // N < -1: Status::NotSupported() will be returned during DB open. + // + // Default: -1 + int max_read_amp; + // The algorithm used to stop picking files into a single compaction run // Default: kCompactionStopStyleTotalSize CompactionStopStyle stop_style; @@ -88,6 +118,7 @@ class CompactionOptionsUniversal { max_merge_width(UINT_MAX), max_size_amplification_percent(200), compression_size_percent(-1), + max_read_amp(-1), stop_style(kCompactionStopStyleTotalSize), allow_trivial_move(false), incremental(false) {} diff --git a/include/rocksdb/user_write_callback.h b/include/rocksdb/user_write_callback.h new file mode 100644 index 00000000000..d002b9caaeb --- /dev/null +++ b/include/rocksdb/user_write_callback.h @@ -0,0 +1,29 @@ +// 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/status.h" + +namespace ROCKSDB_NAMESPACE { + +// Custom callback functions to support users to plug in logic while data is +// being written to the DB. It's intended for better synchronization between +// concurrent writes. Note that these callbacks are in the write's critical path +// It's desirable to keep them fast and minimum to not affect the write's +// latency. These callbacks may be called in the context of a different thread. +class UserWriteCallback { + public: + virtual ~UserWriteCallback() {} + + // This function will be called after the write is enqueued. + virtual void OnWriteEnqueued() = 0; + + // This function will be called after wal write finishes if it applies. + virtual void OnWalWriteFinish() = 0; +}; + +} // namespace ROCKSDB_NAMESPACE \ No newline at end of file diff --git a/include/rocksdb/utilities/ldb_cmd.h b/include/rocksdb/utilities/ldb_cmd.h index ed4f5de7e86..c3a12b6943d 100644 --- a/include/rocksdb/utilities/ldb_cmd.h +++ b/include/rocksdb/utilities/ldb_cmd.h @@ -35,6 +35,7 @@ class LDBCommand { static const std::string ARG_DB; static const std::string ARG_PATH; static const std::string ARG_SECONDARY_PATH; + static const std::string ARG_LEADER_PATH; static const std::string ARG_HEX; static const std::string ARG_KEY_HEX; static const std::string ARG_VALUE_HEX; @@ -72,6 +73,7 @@ class LDBCommand { static const std::string ARG_PREPOPULATE_BLOB_CACHE; static const std::string ARG_DECODE_BLOB_INDEX; static const std::string ARG_DUMP_UNCOMPRESSED_BLOBS; + static const std::string ARG_READ_TIMESTAMP; struct ParsedParams { std::string cmd; @@ -82,6 +84,10 @@ class LDBCommand { static LDBCommand* SelectCommand(const ParsedParams& parsed_parms); + static void ParseSingleParam(const std::string& param, + ParsedParams& parsed_params, + std::vector& cmd_tokens); + static LDBCommand* InitFromCmdLineArgs( const std::vector& args, const Options& options, const LDBOptions& ldb_options, @@ -155,10 +161,12 @@ class LDBCommand { // with this secondary path. When running against a database opened by // another process, ldb wll leave the source directory completely intact. std::string secondary_path_; + std::string leader_path_; std::string column_family_name_; DB* db_; DBWithTTL* db_ttl_; std::map cf_handles_; + std::map ucmps_; /** * true implies that this command can work if the db is opened in read-only @@ -190,6 +198,9 @@ class LDBCommand { bool create_if_missing_; + /** Encoded user provided uint64_t read timestamp. */ + std::string read_timestamp_; + /** * Map of options passed on the command-line. */ @@ -220,17 +231,19 @@ class LDBCommand { ColumnFamilyHandle* GetCfHandle(); static std::string PrintKeyValue(const std::string& key, + const std::string& timestamp, const std::string& value, bool is_key_hex, - bool is_value_hex); + bool is_value_hex, const Comparator* ucmp); static std::string PrintKeyValue(const std::string& key, - const std::string& value, bool is_hex); + const std::string& timestamp, + const std::string& value, bool is_hex, + const Comparator* ucmp); - static std::string PrintKeyValueOrWideColumns(const Slice& key, - const Slice& value, - const WideColumns& wide_columns, - bool is_key_hex, - bool is_value_hex); + static std::string PrintKeyValueOrWideColumns( + const Slice& key, const Slice& timestamp, const Slice& value, + const WideColumns& wide_columns, bool is_key_hex, bool is_value_hex, + const Comparator* ucmp); /** * Return true if the specified flag is present in the specified flags vector @@ -275,6 +288,10 @@ class LDBCommand { bool ParseBooleanOption(const std::map& options, const std::string& option, bool default_val); + /* Populate `ropts.timestamp` from command line flag --read_timestamp */ + Status MaybePopulateReadTimestamp(ColumnFamilyHandle* cfh, ReadOptions& ropts, + Slice* read_timestamp); + Options options_; std::vector column_families_; ConfigOptions config_options_; diff --git a/include/rocksdb/utilities/stackable_db.h b/include/rocksdb/utilities/stackable_db.h index 6ce05a4ad24..414a28dfbfe 100644 --- a/include/rocksdb/utilities/stackable_db.h +++ b/include/rocksdb/utilities/stackable_db.h @@ -501,12 +501,12 @@ class StackableDB : public DB { return db_->GetFullHistoryTsLow(column_family, ts_low); } - Status GetSortedWalFiles(VectorLogPtr& files) override { + Status GetSortedWalFiles(VectorWalPtr& files) override { return db_->GetSortedWalFiles(files); } Status GetCurrentWalFile( - std::unique_ptr* current_log_file) override { + std::unique_ptr* current_log_file) override { return db_->GetCurrentWalFile(current_log_file); } diff --git a/include/rocksdb/utilities/transaction.h b/include/rocksdb/utilities/transaction.h index 7625a3e38a4..4a52d1d6c92 100644 --- a/include/rocksdb/utilities/transaction.h +++ b/include/rocksdb/utilities/transaction.h @@ -5,7 +5,6 @@ #pragma once - #include #include #include @@ -165,7 +164,7 @@ class Transaction { virtual void SetSnapshot() = 0; // Similar to SetSnapshot(), but will not change the current snapshot - // until Put/Merge/Delete/GetForUpdate/MultigetForUpdate is called. + // until Put/PutEntity/Merge/Delete/GetForUpdate/MultigetForUpdate is called. // By calling this function, the transaction will essentially call // SetSnapshot() for you right before performing the next write/GetForUpdate. // @@ -268,10 +267,10 @@ class Transaction { // points. virtual void SetSavePoint() = 0; - // Undo all operations in this transaction (Put, Merge, Delete, PutLogData) - // since the most recent call to SetSavePoint() and removes the most recent - // SetSavePoint(). - // If there is no previous call to SetSavePoint(), returns Status::NotFound() + // Undo all operations in this transaction (Put, PutEntity, Merge, Delete, + // PutLogData) since the most recent call to SetSavePoint() and removes the + // most recent SetSavePoint(). If there is no previous call to SetSavePoint(), + // returns Status::NotFound() virtual Status RollbackToSavePoint() = 0; // Pop the most recent save point. @@ -318,6 +317,10 @@ class Transaction { return s; } + virtual Status GetEntity(const ReadOptions& options, + ColumnFamilyHandle* column_family, const Slice& key, + PinnableWideColumns* columns) = 0; + virtual std::vector MultiGet( const ReadOptions& options, const std::vector& column_family, @@ -354,6 +357,12 @@ class Transaction { } } + virtual void MultiGetEntity(const ReadOptions& options, + ColumnFamilyHandle* column_family, + size_t num_keys, const Slice* keys, + PinnableWideColumns* results, Status* statuses, + bool sorted_input = false) = 0; + // Read this key and ensure that this transaction will only // be able to be committed if this key is not written outside this // transaction after it has first been read (or after the snapshot if a @@ -434,6 +443,13 @@ class Transaction { } } + virtual Status GetEntityForUpdate(const ReadOptions& read_options, + ColumnFamilyHandle* column_family, + const Slice& key, + PinnableWideColumns* columns, + bool exclusive = true, + bool do_validate = true) = 0; + virtual std::vector MultiGetForUpdate( const ReadOptions& options, const std::vector& column_family, @@ -461,9 +477,9 @@ class Transaction { virtual Iterator* GetIterator(const ReadOptions& read_options, ColumnFamilyHandle* column_family) = 0; - // Put, Merge, Delete, and SingleDelete behave similarly to the corresponding - // functions in WriteBatch, but will also do conflict checking on the - // keys being written. + // Put, PutEntity, Merge, Delete, and SingleDelete behave similarly to the + // corresponding functions in WriteBatch, but will also do conflict checking + // on the keys being written. // // assume_tracked=true expects the key be already tracked. More // specifically, it means the the key was previous tracked in the same @@ -489,6 +505,10 @@ class Transaction { const bool assume_tracked = false) = 0; virtual Status Put(const SliceParts& key, const SliceParts& value) = 0; + virtual Status PutEntity(ColumnFamilyHandle* column_family, const Slice& key, + const WideColumns& columns, + bool assume_tracked = false) = 0; + virtual Status Merge(ColumnFamilyHandle* column_family, const Slice& key, const Slice& value, const bool assume_tracked = false) = 0; @@ -528,6 +548,10 @@ class Transaction { virtual Status PutUntracked(const SliceParts& key, const SliceParts& value) = 0; + virtual Status PutEntityUntracked(ColumnFamilyHandle* column_family, + const Slice& key, + const WideColumns& columns) = 0; + virtual Status MergeUntracked(ColumnFamilyHandle* column_family, const Slice& key, const Slice& value) = 0; virtual Status MergeUntracked(const Slice& key, const Slice& value) = 0; @@ -556,18 +580,18 @@ class Transaction { // Similar to WriteBatch::PutLogData virtual void PutLogData(const Slice& blob) = 0; - // By default, all Put/Merge/Delete operations will be indexed in the - // transaction so that Get/GetForUpdate/GetIterator can search for these + // By default, all Put/PutEntity/Merge/Delete operations will be indexed in + // the transaction so that Get/GetForUpdate/GetIterator can search for these // keys. // // If the caller does not want to fetch the keys about to be written, // they may want to avoid indexing as a performance optimization. // Calling DisableIndexing() will turn off indexing for all future - // Put/Merge/Delete operations until EnableIndexing() is called. + // Put/PutEntity/Merge/Delete operations until EnableIndexing() is called. // - // If a key is Put/Merge/Deleted after DisableIndexing is called and then - // is fetched via Get/GetForUpdate/GetIterator, the result of the fetch is - // undefined. + // If a key is written (using Put/PutEntity/Merge/Delete) after + // DisableIndexing is called and then is fetched via + // Get/GetForUpdate/GetIterator, the result of the fetch is undefined. virtual void DisableIndexing() = 0; virtual void EnableIndexing() = 0; @@ -578,9 +602,10 @@ class Transaction { // number of keys that need to be checked for conflicts at commit time. virtual uint64_t GetNumKeys() const = 0; - // Returns the number of Puts/Deletes/Merges that have been applied to this - // transaction so far. + // Returns the number of Put/PutEntity/Delete/Merge operations that have been + // applied to this transaction so far. virtual uint64_t GetNumPuts() const = 0; + virtual uint64_t GetNumPutEntities() const = 0; virtual uint64_t GetNumDeletes() const = 0; virtual uint64_t GetNumMerges() const = 0; diff --git a/include/rocksdb/utilities/write_batch_with_index.h b/include/rocksdb/utilities/write_batch_with_index.h index de24ae889b2..ad662364783 100644 --- a/include/rocksdb/utilities/write_batch_with_index.h +++ b/include/rocksdb/utilities/write_batch_with_index.h @@ -130,7 +130,7 @@ class WriteBatchWithIndex : public WriteBatchBase { "Cannot call this method without attribute groups"); } return Status::NotSupported( - "PutEntity not supported by WriteBatchWithIndex"); + "PutEntity with AttributeGroups not supported by WriteBatchWithIndex"); } using WriteBatchBase::Merge; @@ -284,7 +284,12 @@ class WriteBatchWithIndex : public WriteBatchBase { Status GetEntityFromBatchAndDB(DB* db, const ReadOptions& read_options, ColumnFamilyHandle* column_family, const Slice& key, - PinnableWideColumns* columns); + PinnableWideColumns* columns) { + constexpr ReadCallback* callback = nullptr; + + return GetEntityFromBatchAndDB(db, read_options, column_family, key, + columns, callback); + } void MultiGetFromBatchAndDB(DB* db, const ReadOptions& read_options, ColumnFamilyHandle* column_family, @@ -310,7 +315,13 @@ class WriteBatchWithIndex : public WriteBatchBase { ColumnFamilyHandle* column_family, size_t num_keys, const Slice* keys, PinnableWideColumns* results, - Status* statuses, bool sorted_input); + Status* statuses, bool sorted_input) { + constexpr ReadCallback* callback = nullptr; + + MultiGetEntityFromBatchAndDB(db, read_options, column_family, num_keys, + keys, results, statuses, sorted_input, + callback); + } // Records the state of the batch for future calls to RollbackToSavePoint(). // May be called multiple times to set multiple save points. diff --git a/include/rocksdb/version.h b/include/rocksdb/version.h index 24506703fbc..a7fd1780404 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 3 +#define ROCKSDB_MINOR 4 #define ROCKSDB_PATCH 0 // Do not use these. We made the mistake of declaring macros starting with diff --git a/java/rocksjni/options.cc b/java/rocksjni/options.cc index d5dc2f0676f..a6120f67dca 100644 --- a/java/rocksjni/options.cc +++ b/java/rocksjni/options.cc @@ -3917,7 +3917,7 @@ jlongArray Java_org_rocksdb_Options_tablePropertiesCollectorFactory( // exception thrown: OutOfMemoryError return nullptr; } - jlong* buf = env->GetLongArrayElements(retVal, NULL); + jlong* buf = env->GetLongArrayElements(retVal, nullptr); if (buf == nullptr) { // exception thrown: OutOfMemoryError return nullptr; @@ -3943,7 +3943,7 @@ void Java_org_rocksdb_Options_setTablePropertiesCollectorFactory( auto* opt = reinterpret_cast(jhandle); const jsize size = env->GetArrayLength(j_factory_handles); - jlong* buf = env->GetLongArrayElements(j_factory_handles, NULL); + jlong* buf = env->GetLongArrayElements(j_factory_handles, nullptr); if (buf == nullptr) { // exception thrown: OutOfMemoryError return; diff --git a/java/rocksjni/portal.h b/java/rocksjni/portal.h index 8a95b995e56..29b74733137 100644 --- a/java/rocksjni/portal.h +++ b/java/rocksjni/portal.h @@ -1689,7 +1689,7 @@ class JniUtil { } else if (env_rs == JNI_EDETACHED) { // current thread is not attached, attempt to attach const jint rs_attach = - jvm->AttachCurrentThread(reinterpret_cast(&env), NULL); + jvm->AttachCurrentThread(reinterpret_cast(&env), nullptr); if (rs_attach == JNI_OK) { *attached = JNI_TRUE; return env; diff --git a/java/rocksjni/ttl.cc b/java/rocksjni/ttl.cc index 8e23c10279b..4ab0854f502 100644 --- a/java/rocksjni/ttl.cc +++ b/java/rocksjni/ttl.cc @@ -145,7 +145,7 @@ jlongArray Java_org_rocksdb_TtlDB_openCF(JNIEnv* env, jclass, jlong jopt_handle, return jresults; } else { ROCKSDB_NAMESPACE::RocksDBExceptionJni::ThrowNew(env, s); - return NULL; + return nullptr; } } diff --git a/options/cf_options.cc b/options/cf_options.cc index 3f3af21a074..d6eacf3118b 100644 --- a/options/cf_options.cc +++ b/options/cf_options.cc @@ -239,6 +239,10 @@ static std::unordered_map {offsetof(class CompactionOptionsUniversal, compression_size_percent), OptionType::kInt, OptionVerificationType::kNormal, OptionTypeFlags::kMutable}}, + {"max_read_amp", + {offsetof(class CompactionOptionsUniversal, max_read_amp), + OptionType::kInt, OptionVerificationType::kNormal, + OptionTypeFlags::kMutable}}, {"stop_style", {offsetof(class CompactionOptionsUniversal, stop_style), OptionType::kCompactionStopStyle, OptionVerificationType::kNormal, @@ -1137,6 +1141,8 @@ void MutableCFOptions::Dump(Logger* log) const { ROCKS_LOG_INFO(log, "compaction_options_universal.compression_size_percent : %d", compaction_options_universal.compression_size_percent); + ROCKS_LOG_INFO(log, "compaction_options_universal.max_read_amp: %d", + compaction_options_universal.max_read_amp); ROCKS_LOG_INFO(log, "compaction_options_universal.stop_style : %d", compaction_options_universal.stop_style); ROCKS_LOG_INFO( diff --git a/options/db_options.cc b/options/db_options.cc index 6da12a12156..012c53292e0 100644 --- a/options/db_options.cc +++ b/options/db_options.cc @@ -72,6 +72,10 @@ static std::unordered_map {offsetof(struct MutableDBOptions, avoid_flush_during_shutdown), OptionType::kBoolean, OptionVerificationType::kNormal, OptionTypeFlags::kMutable}}, + {"avoid_sync_during_shutdown", + {offsetof(struct MutableDBOptions, avoid_sync_during_shutdown), + OptionType::kBoolean, OptionVerificationType::kNormal, + OptionTypeFlags::kMutable}}, {"writable_file_max_buffer_size", {offsetof(struct MutableDBOptions, writable_file_max_buffer_size), OptionType::kSizeT, OptionVerificationType::kNormal, @@ -990,6 +994,7 @@ MutableDBOptions::MutableDBOptions() max_background_compactions(-1), max_subcompactions(0), avoid_flush_during_shutdown(false), + avoid_sync_during_shutdown(true), writable_file_max_buffer_size(1024 * 1024), delayed_write_rate(2 * 1024U * 1024U), max_total_wal_size(0), @@ -1009,6 +1014,7 @@ MutableDBOptions::MutableDBOptions(const DBOptions& options) max_background_compactions(options.max_background_compactions), max_subcompactions(options.max_subcompactions), avoid_flush_during_shutdown(options.avoid_flush_during_shutdown), + avoid_sync_during_shutdown(options.avoid_sync_during_shutdown), writable_file_max_buffer_size(options.writable_file_max_buffer_size), delayed_write_rate(options.delayed_write_rate), max_total_wal_size(options.max_total_wal_size), @@ -1034,6 +1040,8 @@ void MutableDBOptions::Dump(Logger* log) const { max_subcompactions); ROCKS_LOG_HEADER(log, " Options.avoid_flush_during_shutdown: %d", avoid_flush_during_shutdown); + ROCKS_LOG_HEADER(log, " Options.avoid_sync_during_shutdown: %d", + avoid_sync_during_shutdown); ROCKS_LOG_HEADER( log, " Options.writable_file_max_buffer_size: %" ROCKSDB_PRIszt, writable_file_max_buffer_size); diff --git a/options/db_options.h b/options/db_options.h index ff7ddc880ff..b95f75ba4ab 100644 --- a/options/db_options.h +++ b/options/db_options.h @@ -125,6 +125,7 @@ struct MutableDBOptions { int max_background_compactions; uint32_t max_subcompactions; bool avoid_flush_during_shutdown; + bool avoid_sync_during_shutdown; size_t writable_file_max_buffer_size; uint64_t delayed_write_rate; uint64_t max_total_wal_size; diff --git a/options/options.cc b/options/options.cc index 514fdb6d012..8336aaa131d 100644 --- a/options/options.cc +++ b/options/options.cc @@ -360,6 +360,9 @@ void ColumnFamilyOptions::Dump(Logger* log) const { ROCKS_LOG_HEADER(log, "Options.compaction_options_universal.stop_style: %s", str_compaction_stop_style.c_str()); + ROCKS_LOG_HEADER(log, + "Options.compaction_options_universal.max_read_amp: %d", + compaction_options_universal.max_read_amp); ROCKS_LOG_HEADER( log, "Options.compaction_options_fifo.max_table_files_size: %" PRIu64, compaction_options_fifo.max_table_files_size); diff --git a/options/options_settable_test.cc b/options/options_settable_test.cc index 747b888af5e..9c6b7657f62 100644 --- a/options/options_settable_test.cc +++ b/options/options_settable_test.cc @@ -348,6 +348,7 @@ TEST_F(OptionsSettableTest, DBOptionsAllFieldsSettable) { "allow_2pc=false;" "avoid_flush_during_recovery=false;" "avoid_flush_during_shutdown=false;" + "avoid_sync_during_shutdown=false;" "allow_ingest_behind=false;" "concurrent_prepare=false;" "two_write_queues=false;" diff --git a/port/stack_trace.cc b/port/stack_trace.cc index f4909f91d5d..ebbd4c1700d 100644 --- a/port/stack_trace.cc +++ b/port/stack_trace.cc @@ -39,6 +39,7 @@ void* SaveStack(int* /*num_frames*/, int /*first_frames_to_skip*/) { #endif // OS_OPENBSD #ifdef OS_FREEBSD #include +#include #endif // OS_FREEBSD #ifdef OS_LINUX #include diff --git a/table/block_based/block_based_table_iterator.cc b/table/block_based/block_based_table_iterator.cc index 3e2f4cc16fb..55eaedff31e 100644 --- a/table/block_based/block_based_table_iterator.cc +++ b/table/block_based/block_based_table_iterator.cc @@ -338,8 +338,8 @@ void BlockBasedTableIterator::InitDataBlock() { bool use_block_cache_for_lookup = true; if (DoesContainBlockHandles()) { - data_block_handle = block_handles_.front().handle_; - is_in_cache = block_handles_.front().is_cache_hit_; + data_block_handle = block_handles_->front().handle_; + is_in_cache = block_handles_->front().is_cache_hit_; use_block_cache_for_lookup = false; } else { data_block_handle = index_iter_->value().handle; @@ -361,7 +361,7 @@ void BlockBasedTableIterator::InitDataBlock() { Status s; block_iter_.Invalidate(Status::OK()); table_->NewDataBlockIterator( - read_options_, (block_handles_.front().cachable_entry_).As(), + read_options_, (block_handles_->front().cachable_entry_).As(), &block_iter_, s); } else { auto* rep = table_->get_rep(); @@ -466,8 +466,8 @@ void BlockBasedTableIterator::AsyncInitDataBlock(bool is_first_pass) { bool is_in_cache = false; if (DoesContainBlockHandles()) { - data_block_handle = block_handles_.front().handle_; - is_in_cache = block_handles_.front().is_cache_hit_; + data_block_handle = block_handles_->front().handle_; + is_in_cache = block_handles_->front().is_cache_hit_; } else { data_block_handle = index_iter_->value().handle; } @@ -477,7 +477,7 @@ void BlockBasedTableIterator::AsyncInitDataBlock(bool is_first_pass) { if (is_in_cache) { block_iter_.Invalidate(Status::OK()); table_->NewDataBlockIterator( - read_options_, (block_handles_.front().cachable_entry_).As(), + read_options_, (block_handles_->front().cachable_entry_).As(), &block_iter_, s); } else { table_->NewDataBlockIterator( @@ -524,7 +524,7 @@ bool BlockBasedTableIterator::MaterializeCurrentBlock() { // BlockCacheLookupForReadAheadSize is called. Slice first_internal_key; if (DoesContainBlockHandles()) { - first_internal_key = block_handles_.front().first_internal_key_; + first_internal_key = block_handles_->front().first_internal_key_; } else { first_internal_key = index_iter_->value().first_internal_key; } @@ -580,7 +580,7 @@ void BlockBasedTableIterator::FindBlockForward() { if (DoesContainBlockHandles()) { // Advance and point to that next Block handle to make that block handle // current. - block_handles_.pop_front(); + block_handles_->pop_front(); } if (!DoesContainBlockHandles()) { @@ -681,7 +681,8 @@ void BlockBasedTableIterator::InitializeStartAndEndOffsets( bool read_curr_block, bool& found_first_miss_block, uint64_t& start_updated_offset, uint64_t& end_updated_offset, size_t& prev_handles_size) { - prev_handles_size = block_handles_.size(); + assert(block_handles_ != nullptr); + prev_handles_size = block_handles_->size(); size_t footer = table_->get_rep()->footer.GetBlockTrailerSize(); // It initialize start and end offset to begin which is covered by following @@ -701,7 +702,7 @@ void BlockBasedTableIterator::InitializeStartAndEndOffsets( end_updated_offset = block_handle_info.handle_.offset() + footer + block_handle_info.handle_.size(); - block_handles_.emplace_back(std::move(block_handle_info)); + block_handles_->emplace_back(std::move(block_handle_info)); index_iter_->Next(); is_index_at_curr_block_ = false; @@ -717,17 +718,17 @@ void BlockBasedTableIterator::InitializeStartAndEndOffsets( // Initialize prev_handles_size to 0 as all those handles need to be read // again. prev_handles_size = 0; - start_updated_offset = block_handles_.front().handle_.offset(); - end_updated_offset = block_handles_.back().handle_.offset() + footer + - block_handles_.back().handle_.size(); + start_updated_offset = block_handles_->front().handle_.offset(); + end_updated_offset = block_handles_->back().handle_.offset() + footer + + block_handles_->back().handle_.size(); } } else { // Scenario 3 : read_curr_block is false (callback made to do additional // prefetching in buffers) and the queue already has some // handles from first buffer. if (DoesContainBlockHandles()) { - start_updated_offset = block_handles_.back().handle_.offset() + footer + - block_handles_.back().handle_.size(); + start_updated_offset = block_handles_->back().handle_.offset() + footer + + block_handles_->back().handle_.size(); end_updated_offset = start_updated_offset; } else { // Scenario 4 : read_curr_block is false (callback made to do additional @@ -789,6 +790,9 @@ void BlockBasedTableIterator::BlockCacheLookupForReadAheadSize( // Initialize start and end offsets based on exisiting handles in the queue // and read_curr_block argument passed. + if (block_handles_ == nullptr) { + block_handles_.reset(new std::deque()); + } InitializeStartAndEndOffsets(read_curr_block, found_first_miss_block, start_updated_offset, end_updated_offset, prev_handles_size); @@ -833,7 +837,7 @@ void BlockBasedTableIterator::BlockCacheLookupForReadAheadSize( } // Add the handle to the queue. - block_handles_.emplace_back(std::move(block_handle_info)); + block_handles_->emplace_back(std::move(block_handle_info)); // Can't figure out for current block if current block // is out of bound. But for next block we can find that. @@ -851,9 +855,9 @@ void BlockBasedTableIterator::BlockCacheLookupForReadAheadSize( if (found_first_miss_block) { // Iterate cache hit block handles from the end till a Miss is there, to // truncate and update the end offset till that Miss. - auto it = block_handles_.rbegin(); + auto it = block_handles_->rbegin(); auto it_end = - block_handles_.rbegin() + (block_handles_.size() - prev_handles_size); + block_handles_->rbegin() + (block_handles_->size() - prev_handles_size); while (it != it_end && (*it).is_cache_hit_ && start_updated_offset != (*it).handle_.offset()) { diff --git a/table/block_based/block_based_table_iterator.h b/table/block_based/block_based_table_iterator.h index d9e29a75f3b..2b562ef0679 100644 --- a/table/block_based/block_based_table_iterator.h +++ b/table/block_based/block_based_table_iterator.h @@ -338,7 +338,8 @@ class BlockBasedTableIterator : public InternalIteratorBase { // different blocks when readahead_size is calculated in // BlockCacheLookupForReadAheadSize, to avoid index_iter_ reseek, // block_handles_ is used. - std::deque block_handles_; + // `block_handles_` is lazily constructed to save CPU when it is unused + std::unique_ptr> block_handles_; // During cache lookup to find readahead size, index_iter_ is iterated and it // can point to a different block. is_index_at_curr_block_ keeps track of @@ -418,7 +419,11 @@ class BlockBasedTableIterator : public InternalIteratorBase { : false); } - void ClearBlockHandles() { block_handles_.clear(); } + void ClearBlockHandles() { + if (block_handles_ != nullptr) { + block_handles_->clear(); + } + } // Reset prev_block_offset_. If index_iter_ has moved ahead, it won't get // accurate prev_block_offset_. @@ -426,7 +431,9 @@ class BlockBasedTableIterator : public InternalIteratorBase { prev_block_offset_ = std::numeric_limits::max(); } - bool DoesContainBlockHandles() { return !block_handles_.empty(); } + bool DoesContainBlockHandles() { + return block_handles_ != nullptr && !block_handles_->empty(); + } void InitializeStartAndEndOffsets(bool read_curr_block, bool& found_first_miss_block, diff --git a/table/block_fetcher.cc b/table/block_fetcher.cc index 31b6d93888f..1316d7302dd 100644 --- a/table/block_fetcher.cc +++ b/table/block_fetcher.cc @@ -241,7 +241,7 @@ inline void BlockFetcher::GetBlockContents() { // Read a block from the file and verify its checksum. Upon return, io_status_ // will be updated with the status of the read, and slice_ will be updated // with a pointer to the data. -void BlockFetcher::ReadBlock(bool retry, FSAllocationPtr& fs_buf) { +void BlockFetcher::ReadBlock(bool retry) { FSReadRequest read_req; IOOptions opts; io_status_ = file_->PrepareIOOptions(read_options_, opts); @@ -336,7 +336,7 @@ void BlockFetcher::ReadBlock(bool retry, FSAllocationPtr& fs_buf) { if (io_status_.ok()) { InsertCompressedBlockToPersistentCacheIfNeeded(); - fs_buf = std::move(read_req.fs_scratch); + fs_buf_ = std::move(read_req.fs_scratch); } else { ReleaseFileSystemProvidedBuffer(&read_req); direct_io_buf_.reset(); @@ -347,7 +347,6 @@ void BlockFetcher::ReadBlock(bool retry, FSAllocationPtr& fs_buf) { } IOStatus BlockFetcher::ReadBlockContents() { - FSAllocationPtr fs_buf; if (TryGetUncompressBlockFromPersistentCache()) { compression_type_ = kNoCompression; #ifndef NDEBUG @@ -360,15 +359,15 @@ IOStatus BlockFetcher::ReadBlockContents() { return io_status_; } } else if (!TryGetSerializedBlockFromPersistentCache()) { - ReadBlock(/*retry =*/false, fs_buf); + ReadBlock(/*retry =*/false); // If the file system supports retry after corruption, then try to // re-read the block and see if it succeeds. if (io_status_.IsCorruption() && retry_corrupt_read_) { - assert(!fs_buf); - ReadBlock(/*retry=*/true, fs_buf); + assert(!fs_buf_); + ReadBlock(/*retry=*/true); } if (!io_status_.ok()) { - assert(!fs_buf); + assert(!fs_buf_); return io_status_; } } @@ -417,16 +416,15 @@ IOStatus BlockFetcher::ReadAsyncBlockContents() { return io_s; } if (io_s.ok()) { - FSAllocationPtr fs_buf; // Data Block is already in prefetch. got_from_prefetch_buffer_ = true; ProcessTrailerIfPresent(); if (io_status_.IsCorruption() && retry_corrupt_read_) { got_from_prefetch_buffer_ = false; - ReadBlock(/*retry = */ true, fs_buf); + ReadBlock(/*retry = */ true); } if (!io_status_.ok()) { - assert(!fs_buf); + assert(!fs_buf_); return io_status_; } used_buf_ = const_cast(slice_.data()); diff --git a/table/block_fetcher.h b/table/block_fetcher.h index 46e643f91b2..9441e0a73ca 100644 --- a/table/block_fetcher.h +++ b/table/block_fetcher.h @@ -137,6 +137,7 @@ class BlockFetcher { bool for_compaction_ = false; bool use_fs_scratch_ = false; bool retry_corrupt_read_ = false; + FSAllocationPtr fs_buf_; // return true if found bool TryGetUncompressBlockFromPersistentCache(); @@ -152,7 +153,7 @@ class BlockFetcher { void InsertCompressedBlockToPersistentCacheIfNeeded(); void InsertUncompressedBlockToPersistentCacheIfNeeded(); void ProcessTrailerIfPresent(); - void ReadBlock(bool retry, FSAllocationPtr& fs_buf); + void ReadBlock(bool retry); void ReleaseFileSystemProvidedBuffer(FSReadRequest* read_req) { if (use_fs_scratch_) { diff --git a/table/iterator.cc b/table/iterator.cc index 14e280a07b6..8306f5a0464 100644 --- a/table/iterator.cc +++ b/table/iterator.cc @@ -23,6 +23,10 @@ Status Iterator::GetProperty(std::string prop_name, std::string* prop) { *prop = "0"; return Status::OK(); } + if (prop_name == "rocksdb.iterator.is-value-pinned") { + *prop = "0"; + return Status::OK(); + } return Status::InvalidArgument("Unidentified property."); } diff --git a/table/sst_file_dumper.cc b/table/sst_file_dumper.cc index d201163808d..72729ffc3c1 100644 --- a/table/sst_file_dumper.cc +++ b/table/sst_file_dumper.cc @@ -521,22 +521,22 @@ Status SstFileDumper::ReadSequential(bool print_kv, uint64_t read_num_limit, iter->value(), oss, output_hex_); if (!s.ok()) { fprintf(stderr, "%s => error deserializing wide columns\n", - ikey.DebugString(true, output_hex_).c_str()); + ikey.DebugString(true, output_hex_, ucmp).c_str()); continue; } fprintf(stdout, "%s => %s\n", - ikey.DebugString(true, output_hex_).c_str(), + ikey.DebugString(true, output_hex_, ucmp).c_str(), oss.str().c_str()); } else if (ikey.type == kTypeValuePreferredSeqno) { auto [unpacked_value, preferred_seqno] = ParsePackedValueWithSeqno(value); fprintf(stdout, "%s => %s, %llu\n", - ikey.DebugString(true, output_hex_).c_str(), + ikey.DebugString(true, output_hex_, ucmp).c_str(), unpacked_value.ToString(output_hex_).c_str(), static_cast(preferred_seqno)); } else { fprintf(stdout, "%s => %s\n", - ikey.DebugString(true, output_hex_).c_str(), + ikey.DebugString(true, output_hex_, ucmp).c_str(), value.ToString(output_hex_).c_str()); } } else { @@ -545,12 +545,12 @@ Status SstFileDumper::ReadSequential(bool print_kv, uint64_t read_num_limit, const Status s = blob_index.DecodeFrom(value); if (!s.ok()) { fprintf(stderr, "%s => error decoding blob index\n", - ikey.DebugString(true, output_hex_).c_str()); + ikey.DebugString(true, output_hex_, ucmp).c_str()); continue; } fprintf(stdout, "%s => %s\n", - ikey.DebugString(true, output_hex_).c_str(), + ikey.DebugString(true, output_hex_, ucmp).c_str(), blob_index.DebugString(output_hex_).c_str()); } } diff --git a/table/sst_file_reader.cc b/table/sst_file_reader.cc index 9b940df11a3..8fb3d7267f4 100644 --- a/table/sst_file_reader.cc +++ b/table/sst_file_reader.cc @@ -140,7 +140,7 @@ Status SstFileReader::VerifyNumEntries(const ReadOptions& read_options) { uint64_t num_read = 0; for (; internal_iter->Valid(); internal_iter->Next()) { ++num_read; - }; + } s = internal_iter->status(); if (!s.ok()) { return s; diff --git a/test_util/testutil.cc b/test_util/testutil.cc index 5372126ef5f..ab926cef219 100644 --- a/test_util/testutil.cc +++ b/test_util/testutil.cc @@ -565,6 +565,30 @@ void DeleteDir(Env* env, const std::string& dirname) { TryDeleteDir(env, dirname).PermitUncheckedError(); } +FileType GetFileType(const std::string& path) { + FileType type = kTempFile; + std::size_t found = path.find_last_of('/'); + if (found == std::string::npos) { + found = 0; + } + std::string file_name = path.substr(found); + uint64_t number = 0; + ParseFileName(file_name, &number, &type); + return type; +} + +uint64_t GetFileNumber(const std::string& path) { + FileType type = kTempFile; + std::size_t found = path.find_last_of('/'); + if (found == std::string::npos) { + found = 0; + } + std::string file_name = path.substr(found); + uint64_t number = 0; + ParseFileName(file_name, &number, &type); + return number; +} + Status CreateEnvFromSystem(const ConfigOptions& config_options, Env** result, std::shared_ptr* guard) { const char* env_uri = getenv("TEST_ENV_URI"); diff --git a/test_util/testutil.h b/test_util/testutil.h index b3fa0954cbf..02accdc5219 100644 --- a/test_util/testutil.h +++ b/test_util/testutil.h @@ -882,6 +882,12 @@ Status TryDeleteDir(Env* env, const std::string& dirname); // Delete a directory if it exists void DeleteDir(Env* env, const std::string& dirname); +// Find the FileType from the file path +FileType GetFileType(const std::string& path); + +// Get the file number given the file path +uint64_t GetFileNumber(const std::string& path); + // Creates an Env from the system environment by looking at the system // environment variables. Status CreateEnvFromSystem(const ConfigOptions& options, Env** result, diff --git a/tools/check_format_compatible.sh b/tools/check_format_compatible.sh index 6da1000f4ab..07b7cf346bd 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" "9.1.fb" "9.2.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" "9.3.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 ce14bce5da0..39f3d804b18 100644 --- a/tools/db_bench_tool.cc +++ b/tools/db_bench_tool.cc @@ -544,12 +544,20 @@ DEFINE_int32(universal_compression_size_percent, -1, "The percentage of the database to compress for universal " "compaction. -1 means compress everything."); +DEFINE_int32(universal_max_read_amp, -1, + "The limit on the number of sorted runs"); + DEFINE_bool(universal_allow_trivial_move, false, "Allow trivial move in universal compaction."); DEFINE_bool(universal_incremental, false, "Enable incremental compactions in universal compaction."); +DEFINE_int32( + universal_stop_style, + (int32_t)ROCKSDB_NAMESPACE::CompactionOptionsUniversal().stop_style, + "Universal compaction stop style."); + DEFINE_int64(cache_size, 32 << 20, // 32MB "Number of bytes to use as a cache of uncompressed data"); @@ -1299,6 +1307,8 @@ static enum ROCKSDB_NAMESPACE::TieredAdmissionPolicy StringToAdmissionPolicy( return ROCKSDB_NAMESPACE::kAdmPolicyAllowCacheHits; } else if (!strcasecmp(policy, "three_queue")) { return ROCKSDB_NAMESPACE::kAdmPolicyThreeQueue; + } else if (!strcasecmp(policy, "allow_all")) { + return ROCKSDB_NAMESPACE::kAdmPolicyAllowAll; } else { fprintf(stderr, "Cannot parse admission policy %s\n", policy); exit(1); @@ -4662,10 +4672,14 @@ class Benchmark { options.compaction_options_universal.compression_size_percent = FLAGS_universal_compression_size_percent; } + options.compaction_options_universal.max_read_amp = + FLAGS_universal_max_read_amp; options.compaction_options_universal.allow_trivial_move = FLAGS_universal_allow_trivial_move; options.compaction_options_universal.incremental = FLAGS_universal_incremental; + options.compaction_options_universal.stop_style = + static_cast(FLAGS_universal_stop_style); if (FLAGS_thread_status_per_interval > 0) { options.enable_thread_tracking = true; } diff --git a/tools/db_crashtest.py b/tools/db_crashtest.py index 2045c4d6998..3bea01c5b14 100644 --- a/tools/db_crashtest.py +++ b/tools/db_crashtest.py @@ -67,7 +67,11 @@ "clear_column_family_one_in": 0, "compact_files_one_in": lambda: random.choice([1000, 1000000]), "compact_range_one_in": lambda: random.choice([1000, 1000000]), + # Disabled because of various likely related failures with + # "Cannot delete table file #N from level 0 since it is on level X" + "promote_l0_one_in": 0, "compaction_pri": random.randint(0, 4), + "key_may_exist_one_in": lambda: random.choice([100, 100000]), "data_block_index_type": lambda: random.choice([0, 1]), "delpercent": 4, "delrangepercent": 1, @@ -116,6 +120,7 @@ "optimize_filters_for_memory": lambda: random.randint(0, 1), "partition_filters": lambda: random.randint(0, 1), "partition_pinning": lambda: random.randint(0, 3), + "reset_stats_one_in": lambda: random.choice([10000, 1000000]), "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]), @@ -147,6 +152,8 @@ "use_merge": lambda: random.randint(0, 1), # use_put_entity_one_in has to be the same across invocations for verification to work, hence no lambda "use_put_entity_one_in": random.choice([0] * 7 + [1, 5, 10]), + "use_attribute_group": lambda: random.randint(0, 1), + "use_multi_cf_iterator": 0, # TODO(jaykorean) - re-enable this after fixing the test # 999 -> use Bloom API "bloom_before_level": lambda: random.choice([random.randint(-1, 2), random.randint(-1, 10), 0x7fffffff - 1, 0x7fffffff]), "value_size_mult": 32, @@ -265,6 +272,7 @@ "WAL_size_limit_MB": lambda: random.choice([0, 1]), "strict_bytes_per_sync": lambda: random.choice([0, 1]), "avoid_flush_during_shutdown": lambda: random.choice([0, 1]), + "avoid_sync_during_shutdown": lambda: random.choice([0, 1]), "fill_cache": lambda: random.choice([0, 1]), "optimize_multiget_for_io": lambda: random.choice([0, 1]), "memtable_insert_hint_per_batch": lambda: random.choice([0, 1]), @@ -305,6 +313,7 @@ "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]), + "universal_max_read_amp": lambda : random.choice([-1] * 3 + [0, 4, 10]), } _TEST_DIR_ENV_VAR = "TEST_TMPDIR" # If TEST_TMPDIR_EXPECTED is not specified, default value will be TEST_TMPDIR @@ -486,8 +495,6 @@ def is_direct_io_supported(dbname): # pipeline write is not currnetly compatible with WritePrepared txns "enable_pipelined_write": 0, "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 @@ -501,8 +508,6 @@ def is_direct_io_supported(dbname): "occ_validation_policy": random.randint(0, 1), "share_occ_lock_buckets": random.randint(0, 1), "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 @@ -681,6 +686,11 @@ def finalize_and_sanitize(src_params): if dest_params["prefix_size"] < 0: dest_params["prefix_size"] = 1 + # BER disables WAL and tests unsynced data loss which + # does not work with inplace_update_support. + if dest_params.get("best_efforts_recovery") == 1: + dest_params["inplace_update_support"] = 0 + # Multi-key operations are not currently compatible with transactions or # timestamp. if ( @@ -695,11 +705,22 @@ def finalize_and_sanitize(src_params): ): dest_params["delpercent"] += dest_params["delrangepercent"] dest_params["delrangepercent"] = 0 + # Since the value of inplace_update_support needs to be fixed across runs, + # we disable other incompatible options here instead of disabling + # inplace_update_support based on other option values, which may change + # across runs. 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 + dest_params["allow_concurrent_memtable_write"] = 0 + # inplace_update_support does not update sequence number. Our stress test recovery + # logic for unsynced data loss relies on max sequence number stored + # in MANIFEST, so they don't work together. + dest_params["disable_wal"] = 0 + dest_params["sync_fault_injection"] = 0 + dest_params["manual_wal_flush_one_in"] = 0 if ( dest_params.get("disable_wal") == 1 or dest_params.get("sync_fault_injection") == 1 @@ -717,12 +738,15 @@ 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 + # 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 + if (dest_params.get("avoid_sync_during_shutdown") == 1): + dest_params["reopen"] = 0 # Only under WritePrepared txns, unordered_write would provide the same guarnatees as vanilla rocksdb + # unordered_write is only enabled with --txn, and txn_params disables inplace_update_support, so + # setting allow_concurrent_memtable_write=1 won't conflcit with inplace_update_support. if dest_params.get("unordered_write", 0) == 1: dest_params["txn_write_policy"] = 1 dest_params["allow_concurrent_memtable_write"] = 1 @@ -772,9 +796,12 @@ def finalize_and_sanitize(src_params): dest_params["unordered_write"] = 0 # For TransactionDB, correctness testing with unsync data loss is currently # compatible with only write committed policy - if dest_params.get("use_txn") == 1 and dest_params.get("txn_write_policy") != 0: + if dest_params.get("use_txn") == 1 and dest_params.get("txn_write_policy", 0) != 0: dest_params["sync_fault_injection"] = 0 dest_params["manual_wal_flush_one_in"] = 0 + # Wide-column pessimistic transaction APIs are initially supported for + # WriteCommitted only + dest_params["use_put_entity_one_in"] = 0 # Wide column stress tests require FullMergeV3 if dest_params["use_put_entity_one_in"] != 0: dest_params["use_full_merge_v1"] = 0 @@ -826,7 +853,6 @@ def finalize_and_sanitize(src_params): 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 @@ -834,9 +860,10 @@ def finalize_and_sanitize(src_params): # 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) + # Redistribute to maintain 100% total + dest_params["readpercent"] += dest_params.get("iterpercent", 10) + dest_params.get("prefixpercent", 20) dest_params["iterpercent"] = 0 + dest_params["prefixpercent"] = 0 dest_params["check_multiget_consistency"] = 0 dest_params["check_multiget_entity_consistency"] = 0 if dest_params.get("disable_wal") == 1: @@ -859,7 +886,6 @@ def finalize_and_sanitize(src_params): dest_params["use_timed_put_one_in"] = 3 return dest_params - def gen_cmd_params(args): params = {} @@ -1109,13 +1135,13 @@ def whitebox_crash_main(args, unknown_args): } # Single level universal has a lot of special logic. Ensure we cover # it sometimes. - if random.randint(0, 1) == 1: + if not args.test_tiered_storage and random.randint(0, 1) == 1: additional_opts.update( { "num_levels": 1, } ) - elif check_mode == 2: + elif check_mode == 2 and not args.test_tiered_storage: # normal run with FIFO compaction mode # ops_per_thread is divided by 5 because FIFO compaction # style is quite a bit slower on reads with lot of files diff --git a/tools/ldb_cmd.cc b/tools/ldb_cmd.cc index 1f0e8b4ea51..e0b1b547122 100644 --- a/tools/ldb_cmd.cc +++ b/tools/ldb_cmd.cc @@ -45,6 +45,7 @@ #include "util/file_checksum_helper.h" #include "util/stderr_logger.h" #include "util/string_util.h" +#include "util/write_batch_util.h" #include "utilities/blob_db/blob_dump_tool.h" #include "utilities/merge_operators.h" #include "utilities/ttl/db_ttl_impl.h" @@ -59,6 +60,7 @@ const std::string LDBCommand::ARG_FS_URI = "fs_uri"; const std::string LDBCommand::ARG_DB = "db"; const std::string LDBCommand::ARG_PATH = "path"; const std::string LDBCommand::ARG_SECONDARY_PATH = "secondary_path"; +const std::string LDBCommand::ARG_LEADER_PATH = "leader_path"; const std::string LDBCommand::ARG_HEX = "hex"; const std::string LDBCommand::ARG_KEY_HEX = "key_hex"; const std::string LDBCommand::ARG_VALUE_HEX = "value_hex"; @@ -107,6 +109,7 @@ const std::string LDBCommand::ARG_PREPOPULATE_BLOB_CACHE = const std::string LDBCommand::ARG_DECODE_BLOB_INDEX = "decode_blob_index"; const std::string LDBCommand::ARG_DUMP_UNCOMPRESSED_BLOBS = "dump_uncompressed_blobs"; +const std::string LDBCommand::ARG_READ_TIMESTAMP = "read_timestamp"; const char* LDBCommand::DELIM = " ==> "; @@ -114,6 +117,7 @@ namespace { void DumpWalFile(Options options, std::string wal_file, bool print_header, bool print_values, bool is_write_committed, + const std::map& ucmps, LDBCommandExecuteResult* exec_state); void DumpSstFile(Options options, std::string filename, bool output_hex, @@ -122,6 +126,9 @@ void DumpSstFile(Options options, std::string filename, bool output_hex, void DumpBlobFile(const std::string& filename, bool is_key_hex, bool is_value_hex, bool dump_uncompressed_blobs); + +Status EncodeUserProvidedTimestamp(const std::string& user_timestamp, + std::string* ts_buf); } // namespace LDBCommand* LDBCommand::InitFromCmdLineArgs( @@ -136,6 +143,32 @@ LDBCommand* LDBCommand::InitFromCmdLineArgs( SelectCommand); } +void LDBCommand::ParseSingleParam(const std::string& param, + ParsedParams& parsed_params, + std::vector& cmd_tokens) { + const std::string OPTION_PREFIX = "--"; + + if (param[0] == '-' && param[1] == '-') { + std::vector splits = StringSplit(param, '='); + // --option_name=option_value + if (splits.size() == 2) { + std::string optionKey = splits[0].substr(OPTION_PREFIX.size()); + parsed_params.option_map[optionKey] = splits[1]; + } else if (splits.size() == 1) { + // --flag_name + std::string optionKey = splits[0].substr(OPTION_PREFIX.size()); + parsed_params.flags.push_back(optionKey); + } else { + // --option_name=option_value, option_value contains '=' + std::string optionKey = splits[0].substr(OPTION_PREFIX.size()); + parsed_params.option_map[optionKey] = + param.substr(splits[0].length() + 1); + } + } else { + cmd_tokens.push_back(param); + } +} + /** * Parse the command-line arguments and create the appropriate LDBCommand2 * instance. @@ -162,28 +195,8 @@ LDBCommand* LDBCommand::InitFromCmdLineArgs( // and their parameters. For eg: put key1 value1 go into this vector. std::vector cmdTokens; - const std::string OPTION_PREFIX = "--"; - for (const auto& arg : args) { - if (arg[0] == '-' && arg[1] == '-') { - std::vector splits = StringSplit(arg, '='); - // --option_name=option_value - if (splits.size() == 2) { - std::string optionKey = splits[0].substr(OPTION_PREFIX.size()); - parsed_params.option_map[optionKey] = splits[1]; - } else if (splits.size() == 1) { - // --flag_name - std::string optionKey = splits[0].substr(OPTION_PREFIX.size()); - parsed_params.flags.push_back(optionKey); - } else { - // --option_name=option_value, option_value contains '=' - std::string optionKey = splits[0].substr(OPTION_PREFIX.size()); - parsed_params.option_map[optionKey] = - arg.substr(splits[0].length() + 1); - } - } else { - cmdTokens.push_back(arg); - } + ParseSingleParam(arg, parsed_params, cmdTokens); } if (cmdTokens.size() < 1) { @@ -426,6 +439,12 @@ LDBCommand::LDBCommand(const std::map& options, secondary_path_ = itr->second; } + itr = options.find(ARG_LEADER_PATH); + leader_path_ = ""; + if (itr != options.end()) { + leader_path_ = itr->second; + } + is_key_hex_ = IsKeyHex(options, flags); is_value_hex_ = IsValueHex(options, flags); is_db_ttl_ = IsFlagPresent(flags, ARG_TTL); @@ -458,9 +477,9 @@ void LDBCommand::OpenDB() { exec_state_ = LDBCommandExecuteResult::Failed( "ldb doesn't support TTL DB with multiple column families"); } - if (!secondary_path_.empty()) { + if (!secondary_path_.empty() || !leader_path_.empty()) { exec_state_ = LDBCommandExecuteResult::Failed( - "Open as secondary is not supported for TTL DB yet."); + "Open as secondary or follower is not supported for TTL DB yet."); } if (is_read_only_) { st = DBWithTTL::Open(options_, db_path_, &db_ttl_, 0, true); @@ -469,7 +488,11 @@ void LDBCommand::OpenDB() { } db_ = db_ttl_; } else { - if (is_read_only_ && secondary_path_.empty()) { + if (!secondary_path_.empty() && !leader_path_.empty()) { + exec_state_ = LDBCommandExecuteResult::Failed( + "Cannot provide both secondary and leader paths"); + } + if (is_read_only_ && secondary_path_.empty() && leader_path_.empty()) { if (column_families_.empty()) { st = DB::OpenForReadOnly(options_, db_path_, &db_); } else { @@ -478,18 +501,27 @@ void LDBCommand::OpenDB() { } } else { if (column_families_.empty()) { - if (secondary_path_.empty()) { + if (secondary_path_.empty() && leader_path_.empty()) { st = DB::Open(options_, db_path_, &db_); - } else { + } else if (!secondary_path_.empty()) { st = DB::OpenAsSecondary(options_, db_path_, secondary_path_, &db_); + } else { + std::unique_ptr dbptr; + st = DB::OpenAsFollower(options_, db_path_, leader_path_, &dbptr); + db_ = dbptr.release(); } } else { - if (secondary_path_.empty()) { + if (secondary_path_.empty() && leader_path_.empty()) { st = DB::Open(options_, db_path_, column_families_, &handles_opened, &db_); - } else { + } else if (!secondary_path_.empty()) { st = DB::OpenAsSecondary(options_, db_path_, secondary_path_, column_families_, &handles_opened, &db_); + } else { + std::unique_ptr dbptr; + st = DB::OpenAsFollower(options_, db_path_, leader_path_, + column_families_, &handles_opened, &dbptr); + db_ = dbptr.release(); } } } @@ -502,6 +534,7 @@ void LDBCommand::OpenDB() { bool found_cf_name = false; for (size_t i = 0; i < handles_opened.size(); i++) { cf_handles_[column_families_[i].name] = handles_opened[i]; + ucmps_[handles_opened[i]->GetID()] = handles_opened[i]->GetComparator(); if (column_family_name_ == column_families_[i].name) { found_cf_name = true; } @@ -511,6 +544,8 @@ void LDBCommand::OpenDB() { "Non-existing column family " + column_family_name_); CloseDB(); } + ColumnFamilyHandle* default_cf = db_->DefaultColumnFamily(); + ucmps_[default_cf->GetID()] = default_cf->GetComparator(); } else { // We successfully opened DB in single column family mode. assert(column_families_.empty()); @@ -519,6 +554,8 @@ void LDBCommand::OpenDB() { "Non-existing column family " + column_family_name_); CloseDB(); } + ColumnFamilyHandle* default_cf = db_->DefaultColumnFamily(); + ucmps_[default_cf->GetID()] = default_cf->GetComparator(); } } @@ -553,6 +590,7 @@ std::vector LDBCommand::BuildCmdLineOptions( ARG_FS_URI, ARG_DB, ARG_SECONDARY_PATH, + ARG_LEADER_PATH, ARG_BLOOM_BITS, ARG_BLOCK_SIZE, ARG_AUTO_COMPACTION, @@ -722,6 +760,42 @@ bool LDBCommand::ParseCompressionTypeOption( return false; } +Status LDBCommand::MaybePopulateReadTimestamp(ColumnFamilyHandle* cfh, + ReadOptions& ropts, + Slice* read_timestamp) { + const size_t ts_sz = cfh->GetComparator()->timestamp_size(); + + auto iter = option_map_.find(ARG_READ_TIMESTAMP); + if (iter == option_map_.end()) { + if (ts_sz == 0) { + return Status::OK(); + } + return Status::InvalidArgument( + "column family enables user-defined timestamp while --read_timestamp " + "is not provided."); + } + if (iter->second.empty()) { + if (ts_sz == 0) { + return Status::OK(); + } + return Status::InvalidArgument( + "column family enables user-defined timestamp while --read_timestamp " + "is empty."); + } + if (ts_sz == 0) { + return Status::InvalidArgument( + "column family does not enable user-defined timestamps while " + "--read_timestamp is provided."); + } + Status s = EncodeUserProvidedTimestamp(iter->second, &read_timestamp_); + if (!s.ok()) { + return s; + } + *read_timestamp = read_timestamp_; + ropts.timestamp = read_timestamp; + return Status::OK(); +} + void LDBCommand::OverrideBaseOptions() { options_.create_if_missing = false; @@ -769,7 +843,10 @@ void LDBCommand::OverrideBaseCFOptions(ColumnFamilyOptions* cf_opts) { } } - if (options_.comparator != nullptr) { + // Default comparator is BytewiseComparator, so only when it's not, it + // means user has a command line override. + if (options_.comparator != nullptr && + options_.comparator != BytewiseComparator()) { cf_opts->comparator = options_.comparator; } @@ -1102,27 +1179,36 @@ std::string LDBCommand::StringToHex(const std::string& str) { } std::string LDBCommand::PrintKeyValue(const std::string& key, + const std::string& timestamp, const std::string& value, bool is_key_hex, - bool is_value_hex) { + bool is_value_hex, + const Comparator* ucmp) { std::string result; result.append(is_key_hex ? StringToHex(key) : key); + if (!timestamp.empty()) { + result.append("|timestamp:"); + result.append(ucmp->TimestampToString(timestamp)); + } result.append(DELIM); result.append(is_value_hex ? StringToHex(value) : value); return result; } std::string LDBCommand::PrintKeyValue(const std::string& key, - const std::string& value, bool is_hex) { - return PrintKeyValue(key, value, is_hex, is_hex); + const std::string& timestamp, + const std::string& value, bool is_hex, + const Comparator* ucmp) { + return PrintKeyValue(key, timestamp, value, is_hex, is_hex, ucmp); } std::string LDBCommand::PrintKeyValueOrWideColumns( - const Slice& key, const Slice& value, const WideColumns& wide_columns, - bool is_key_hex, bool is_value_hex) { + const Slice& key, const Slice& timestamp, const Slice& value, + const WideColumns& wide_columns, bool is_key_hex, bool is_value_hex, + const Comparator* ucmp) { if (wide_columns.empty() || WideColumnsHelper::HasDefaultColumnOnly(wide_columns)) { - return PrintKeyValue(key.ToString(), value.ToString(), is_key_hex, - is_value_hex); + return PrintKeyValue(key.ToString(), timestamp.ToString(), value.ToString(), + is_key_hex, is_value_hex, ucmp); } /* // Sample plaintext output (first column is kDefaultWideColumnName) @@ -1133,9 +1219,10 @@ std::string LDBCommand::PrintKeyValueOrWideColumns( */ std::ostringstream oss; WideColumnsHelper::DumpWideColumns(wide_columns, oss, is_value_hex); - return PrintKeyValue(key.ToString(), oss.str().c_str(), is_key_hex, - false); // is_value_hex_ is already honored in oss. - // avoid double-hexing it. + return PrintKeyValue(key.ToString(), timestamp.ToString(), oss.str().c_str(), + is_key_hex, false, + ucmp); // is_value_hex_ is already honored in oss. + // avoid double-hexing it. } std::string LDBCommand::HelpRangeCmdArgs() { @@ -1885,10 +1972,12 @@ void InternalDumpCommand::DoCommand() { assert(GetExecuteState().IsFailed()); return; } - + ColumnFamilyHandle* cfh = GetCfHandle(); + const Comparator* ucmp = cfh->GetComparator(); + size_t ts_sz = ucmp->timestamp_size(); if (print_stats_) { std::string stats; - if (db_->GetProperty(GetCfHandle(), "rocksdb.stats", &stats)) { + if (db_->GetProperty(cfh, "rocksdb.stats", &stats)) { fprintf(stdout, "%s\n", stats.c_str()); } } @@ -1910,7 +1999,11 @@ void InternalDumpCommand::DoCommand() { for (auto& key_version : key_versions) { ValueType value_type = static_cast(key_version.type); InternalKey ikey(key_version.user_key, key_version.sequence, value_type); - if (has_to_ && ikey.user_key() == to_) { + Slice user_key_without_ts = ikey.user_key(); + if (ts_sz > 0) { + user_key_without_ts.remove_suffix(ts_sz); + } + if (has_to_ && ucmp->Compare(user_key_without_ts, to_) == 0) { // GetAllKeyVersions() includes keys with user key `to_`, but idump has // traditionally excluded such keys. break; @@ -1946,7 +2039,7 @@ void InternalDumpCommand::DoCommand() { } if (!count_only_ && !count_delim_) { - std::string key = ikey.DebugString(is_key_hex_); + std::string key = ikey.DebugString(is_key_hex_, ucmp); Slice value(key_version.value); if (!decode_blob_index_ || value_type != kTypeBlobIndex) { if (value_type == kTypeWideColumnEntity) { @@ -2122,7 +2215,7 @@ void DBDumperCommand::DoCommand() { // TODO(myabandeh): allow configuring is_write_commited DumpWalFile(options_, path_, /* print_header_ */ true, /* print_values_ */ true, true /* is_write_commited */, - &exec_state_); + ucmps_, &exec_state_); break; case kTableFile: DumpSstFile(options_, path_, is_key_hex_, /* show_properties */ true, @@ -2162,8 +2255,16 @@ void DBDumperCommand::DoDumpCommand() { // Setup key iterator ReadOptions scan_read_opts; + Slice read_timestamp; + ColumnFamilyHandle* cfh = GetCfHandle(); + const Comparator* ucmp = cfh->GetComparator(); + size_t ts_sz = ucmp->timestamp_size(); + if (ucmp->timestamp_size() > 0) { + read_timestamp = ucmp->GetMaxTimestamp(); + scan_read_opts.timestamp = &read_timestamp; + } scan_read_opts.total_order_seek = true; - Iterator* iter = db_->NewIterator(scan_read_opts, GetCfHandle()); + Iterator* iter = db_->NewIterator(scan_read_opts, cfh); Status st = iter->status(); if (!st.ok()) { exec_state_ = @@ -2218,7 +2319,7 @@ void DBDumperCommand::DoDumpCommand() { for (; iter->Valid(); iter->Next()) { int rawtime = 0; // If end marker was specified, we stop before it - if (!null_to_ && (iter->key().ToString() >= to_)) { + if (!null_to_ && ucmp->Compare(iter->key(), to_) >= 0) { break; } // Terminate if maximum number of keys have been dumped @@ -2272,11 +2373,14 @@ void DBDumperCommand::DoDumpCommand() { // (TODO) TTL Iterator does not support wide columns yet. std::string str = is_db_ttl_ - ? PrintKeyValue(iter->key().ToString(), iter->value().ToString(), - is_key_hex_, is_value_hex_) - : PrintKeyValueOrWideColumns(iter->key(), iter->value(), - iter->columns(), is_key_hex_, - is_value_hex_); + ? PrintKeyValue(iter->key().ToString(), + ts_sz == 0 ? "" : iter->timestamp().ToString(), + iter->value().ToString(), is_key_hex_, + is_value_hex_, ucmp) + : PrintKeyValueOrWideColumns( + iter->key(), ts_sz == 0 ? "" : iter->timestamp().ToString(), + iter->value(), iter->columns(), is_key_hex_, is_value_hex_, + ucmp); fprintf(stdout, "%s\n", str.c_str()); } } @@ -2597,14 +2701,16 @@ struct StdErrReporter : public log::Reader::Reporter { class InMemoryHandler : public WriteBatch::Handler { public: InMemoryHandler(std::stringstream& row, bool print_values, - bool write_after_commit = false) + bool write_after_commit, + const std::map& ucmps) : Handler(), row_(row), print_values_(print_values), - write_after_commit_(write_after_commit) {} + write_after_commit_(write_after_commit), + ucmps_(ucmps) {} - void commonPutMerge(const Slice& key, const Slice& value) { - std::string k = LDBCommand::StringToHex(key.ToString()); + void commonPutMerge(uint32_t cf, const Slice& key, const Slice& value) { + std::string k = PrintKey(cf, key); if (print_values_) { std::string v = LDBCommand::StringToHex(value.ToString()); row_ << k << " : "; @@ -2616,23 +2722,29 @@ class InMemoryHandler : public WriteBatch::Handler { Status PutCF(uint32_t cf, const Slice& key, const Slice& value) override { row_ << "PUT(" << cf << ") : "; - commonPutMerge(key, value); + commonPutMerge(cf, key, value); return Status::OK(); } Status PutEntityCF(uint32_t cf, const Slice& key, const Slice& value) override { - row_ << "PUT_ENTITY(" << cf << ") : "; - std::string k = LDBCommand::StringToHex(key.ToString()); + row_ << "PUT_ENTITY(" << cf << ") : " << PrintKey(cf, key); if (print_values_) { - return WideColumnsHelper::DumpSliceAsWideColumns(value, row_, true); + row_ << " : "; + const Status s = + WideColumnsHelper::DumpSliceAsWideColumns(value, row_, true); + if (!s.ok()) { + return s; + } } + + row_ << ' '; return Status::OK(); } Status MergeCF(uint32_t cf, const Slice& key, const Slice& value) override { row_ << "MERGE(" << cf << ") : "; - commonPutMerge(key, value); + commonPutMerge(cf, key, value); return Status::OK(); } @@ -2643,21 +2755,21 @@ class InMemoryHandler : public WriteBatch::Handler { Status DeleteCF(uint32_t cf, const Slice& key) override { row_ << "DELETE(" << cf << ") : "; - row_ << LDBCommand::StringToHex(key.ToString()) << " "; + row_ << PrintKey(cf, key) << " "; return Status::OK(); } Status SingleDeleteCF(uint32_t cf, const Slice& key) override { row_ << "SINGLE_DELETE(" << cf << ") : "; - row_ << LDBCommand::StringToHex(key.ToString()) << " "; + row_ << PrintKey(cf, key) << " "; return Status::OK(); } Status DeleteRangeCF(uint32_t cf, const Slice& begin_key, const Slice& end_key) override { row_ << "DELETE_RANGE(" << cf << ") : "; - row_ << LDBCommand::StringToHex(begin_key.ToString()) << " "; - row_ << LDBCommand::StringToHex(end_key.ToString()) << " "; + row_ << PrintKey(cf, begin_key) << " "; + row_ << PrintKey(cf, end_key) << " "; return Status::OK(); } @@ -2702,13 +2814,37 @@ class InMemoryHandler : public WriteBatch::Handler { } private: + std::string PrintKey(uint32_t cf, const Slice& key) { + auto ucmp_iter = ucmps_.find(cf); + if (ucmp_iter == ucmps_.end()) { + // Fallback to default print slice as hex + return LDBCommand::StringToHex(key.ToString()); + } + size_t ts_sz = ucmp_iter->second->timestamp_size(); + if (ts_sz == 0) { + return LDBCommand::StringToHex(key.ToString()); + } else { + // This could happen if there is corruption or undetected comparator + // change. + if (key.size() < ts_sz) { + return "CORRUPT KEY"; + } + Slice user_key_without_ts = key; + user_key_without_ts.remove_suffix(ts_sz); + Slice ts = Slice(key.data() + key.size() - ts_sz, ts_sz); + return LDBCommand::StringToHex(user_key_without_ts.ToString()) + + "|timestamp:" + ucmp_iter->second->TimestampToString(ts); + } + } std::stringstream& row_; bool print_values_; bool write_after_commit_; + const std::map ucmps_; }; void DumpWalFile(Options options, std::string wal_file, bool print_header, bool print_values, bool is_write_committed, + const std::map& ucmps, LDBCommandExecuteResult* exec_state) { const auto& fs = options.env->GetFileSystem(); FileOptions soptions(options); @@ -2729,6 +2865,12 @@ void DumpWalFile(Options options, std::string wal_file, bool print_header, uint64_t log_number; FileType type; + // Comparators are available and will be used for formatting user key if DB + // is opened for this dump wal operation. + UnorderedMap running_ts_sz; + for (const auto& [cf_id, ucmp] : ucmps) { + running_ts_sz.emplace(cf_id, ucmp->timestamp_size()); + } // we need the log number, but ParseFilename expects dbname/NNN.log. std::string sanitized = wal_file; size_t lastslash = sanitized.rfind('/'); @@ -2741,6 +2883,7 @@ void DumpWalFile(Options options, std::string wal_file, bool print_header, } log::Reader reader(options.info_log, std::move(wal_file_reader), &reporter, true /* checksum */, log_number); + std::unordered_set encountered_cf_ids; std::string scratch; WriteBatch batch; Slice record; @@ -2769,11 +2912,51 @@ void DumpWalFile(Options options, std::string wal_file, bool print_header, } break; } + const UnorderedMap recorded_ts_sz = + reader.GetRecordedTimestampSize(); + if (!running_ts_sz.empty()) { + status = HandleWriteBatchTimestampSizeDifference( + &batch, running_ts_sz, recorded_ts_sz, + TimestampSizeConsistencyMode::kVerifyConsistency, + /*new_batch=*/nullptr); + if (!status.ok()) { + std::stringstream oss; + oss << "Format for user keys in WAL file is inconsistent with the " + "comparator used to open the DB. Timestamp size recorded in " + "WAL vs specified by " + "comparator: {"; + bool first_cf = true; + for (const auto& [cf_id, ts_sz] : running_ts_sz) { + if (first_cf) { + first_cf = false; + } else { + oss << ", "; + } + auto record_ts_iter = recorded_ts_sz.find(cf_id); + size_t ts_sz_in_wal = (record_ts_iter == recorded_ts_sz.end()) + ? 0 + : record_ts_iter->second; + oss << "(cf_id: " << cf_id << ", [recorded: " << ts_sz_in_wal + << ", comparator: " << ts_sz << "])"; + } + oss << "}"; + if (exec_state) { + *exec_state = LDBCommandExecuteResult::Failed(oss.str()); + } else { + std::cerr << oss.str() << std::endl; + } + break; + } + } row << WriteBatchInternal::Sequence(&batch) << ","; row << WriteBatchInternal::Count(&batch) << ","; row << WriteBatchInternal::ByteSize(&batch) << ","; row << reader.LastRecordOffset() << ","; - InMemoryHandler handler(row, print_values, is_write_committed); + ColumnFamilyCollector cf_collector; + status = batch.Iterate(&cf_collector); + auto cf_ids = cf_collector.column_families(); + encountered_cf_ids.insert(cf_ids.begin(), cf_ids.end()); + InMemoryHandler handler(row, print_values, is_write_committed, ucmps); status = batch.Iterate(&handler); if (!status.ok()) { if (exec_state) { @@ -2788,6 +2971,29 @@ void DumpWalFile(Options options, std::string wal_file, bool print_header, } std::cout << row.str(); } + + std::stringstream cf_ids_oss; + bool empty_cfs = true; + for (uint32_t cf_id : encountered_cf_ids) { + if (ucmps.find(cf_id) == ucmps.end()) { + if (empty_cfs) { + cf_ids_oss << "["; + empty_cfs = false; + } else { + cf_ids_oss << ","; + } + cf_ids_oss << cf_id; + } + } + if (!empty_cfs) { + cf_ids_oss << "]"; + std::cout + << "(Column family id: " << cf_ids_oss.str() + << " contained in WAL are not opened in DB. Applied default " + "hex formatting for user key. Specify --db= to " + "open DB for better user key formatting if it contains timestamp.)" + << std::endl; + } } } @@ -2803,7 +3009,7 @@ WALDumperCommand::WALDumperCommand( const std::map& options, const std::vector& flags) : LDBCommand(options, flags, true, - BuildCmdLineOptions({ARG_WAL_FILE, ARG_WRITE_COMMITTED, + BuildCmdLineOptions({ARG_WAL_FILE, ARG_DB, ARG_WRITE_COMMITTED, ARG_PRINT_HEADER, ARG_PRINT_VALUE})), print_header_(false), print_values_(false), @@ -2823,12 +3029,17 @@ WALDumperCommand::WALDumperCommand( exec_state_ = LDBCommandExecuteResult::Failed("Argument " + ARG_WAL_FILE + " must be specified."); } + + if (!db_path_.empty()) { + no_db_open_ = false; + } } void WALDumperCommand::Help(std::string& ret) { ret.append(" "); ret.append(WALDumperCommand::Name()); ret.append(" --" + ARG_WAL_FILE + "="); + ret.append(" [--" + ARG_DB + "=]"); ret.append(" [--" + ARG_PRINT_HEADER + "] "); ret.append(" [--" + ARG_PRINT_VALUE + "] "); ret.append(" [--" + ARG_WRITE_COMMITTED + "=true|false] "); @@ -2838,7 +3049,7 @@ void WALDumperCommand::Help(std::string& ret) { void WALDumperCommand::DoCommand() { PrepareOptions(); DumpWalFile(options_, wal_file_, print_header_, print_values_, - is_write_committed_, &exec_state_); + is_write_committed_, ucmps_, &exec_state_); } // ---------------------------------------------------------------------------- @@ -2846,9 +3057,9 @@ void WALDumperCommand::DoCommand() { GetCommand::GetCommand(const std::vector& params, const std::map& options, const std::vector& flags) - : LDBCommand( - options, flags, true, - BuildCmdLineOptions({ARG_TTL, ARG_HEX, ARG_KEY_HEX, ARG_VALUE_HEX})) { + : LDBCommand(options, flags, true, + BuildCmdLineOptions({ARG_TTL, ARG_HEX, ARG_KEY_HEX, + ARG_VALUE_HEX, ARG_READ_TIMESTAMP})) { if (params.size() != 1) { exec_state_ = LDBCommandExecuteResult::Failed( " must be specified for the get command"); @@ -2865,6 +3076,7 @@ void GetCommand::Help(std::string& ret) { ret.append(" "); ret.append(GetCommand::Name()); ret.append(" "); + ret.append(" [--" + ARG_READ_TIMESTAMP + "=] "); ret.append(" [--" + ARG_TTL + "]"); ret.append("\n"); } @@ -2874,8 +3086,18 @@ void GetCommand::DoCommand() { assert(GetExecuteState().IsFailed()); return; } + ReadOptions ropts; + Slice read_timestamp; + ColumnFamilyHandle* cfh = GetCfHandle(); + Status st = MaybePopulateReadTimestamp(cfh, ropts, &read_timestamp); + if (!st.ok()) { + std::stringstream oss; + oss << "Get failed: " << st.ToString(); + exec_state_ = LDBCommandExecuteResult::Failed(oss.str()); + return; + } std::string value; - Status st = db_->Get(ReadOptions(), GetCfHandle(), key_, &value); + st = db_->Get(ropts, cfh, key_, &value); if (st.ok()) { fprintf(stdout, "%s\n", (is_value_hex_ ? StringToHex(value) : value).c_str()); @@ -2895,7 +3117,8 @@ MultiGetCommand::MultiGetCommand( const std::map& options, const std::vector& flags) : LDBCommand(options, flags, true, - BuildCmdLineOptions({ARG_HEX, ARG_KEY_HEX, ARG_VALUE_HEX})) { + BuildCmdLineOptions({ARG_HEX, ARG_KEY_HEX, ARG_VALUE_HEX, + ARG_READ_TIMESTAMP})) { if (params.size() < 1) { exec_state_ = LDBCommandExecuteResult::Failed( "At least one must be specified for multi_get."); @@ -2911,6 +3134,7 @@ void MultiGetCommand::Help(std::string& ret) { ret.append(" "); ret.append(MultiGetCommand::Name()); ret.append(" ..."); + ret.append(" [--" + ARG_READ_TIMESTAMP + "=] "); ret.append("\n"); } @@ -2919,6 +3143,16 @@ void MultiGetCommand::DoCommand() { assert(GetExecuteState().IsFailed()); return; } + ReadOptions ropts; + Slice read_timestamp; + ColumnFamilyHandle* cfh = GetCfHandle(); + Status st = MaybePopulateReadTimestamp(cfh, ropts, &read_timestamp); + if (!st.ok()) { + std::stringstream oss; + oss << "MultiGet failed: " << st.ToString(); + exec_state_ = LDBCommandExecuteResult::Failed(oss.str()); + return; + } size_t num_keys = keys_.size(); std::vector key_slices; std::vector values(num_keys); @@ -2926,8 +3160,8 @@ void MultiGetCommand::DoCommand() { for (const std::string& key : keys_) { key_slices.emplace_back(key); } - db_->MultiGet(ReadOptions(), GetCfHandle(), num_keys, key_slices.data(), - values.data(), statuses.data()); + db_->MultiGet(ropts, cfh, num_keys, key_slices.data(), values.data(), + statuses.data()); bool failed = false; for (size_t i = 0; i < num_keys; ++i) { @@ -2957,9 +3191,9 @@ GetEntityCommand::GetEntityCommand( const std::vector& params, const std::map& options, const std::vector& flags) - : LDBCommand( - options, flags, true, - BuildCmdLineOptions({ARG_TTL, ARG_HEX, ARG_KEY_HEX, ARG_VALUE_HEX})) { + : LDBCommand(options, flags, true, + BuildCmdLineOptions({ARG_TTL, ARG_HEX, ARG_KEY_HEX, + ARG_VALUE_HEX, ARG_READ_TIMESTAMP})) { if (params.size() != 1) { exec_state_ = LDBCommandExecuteResult::Failed( " must be specified for the get_entity command"); @@ -2976,6 +3210,7 @@ void GetEntityCommand::Help(std::string& ret) { ret.append(" "); ret.append(GetEntityCommand::Name()); ret.append(" "); + ret.append(" [--" + ARG_READ_TIMESTAMP + "=] "); ret.append(" [--" + ARG_TTL + "]"); ret.append("\n"); } @@ -2985,9 +3220,18 @@ void GetEntityCommand::DoCommand() { assert(GetExecuteState().IsFailed()); return; } + ReadOptions ropt; + Slice read_timestamp; + ColumnFamilyHandle* cfh = GetCfHandle(); + Status st = MaybePopulateReadTimestamp(cfh, ropt, &read_timestamp); + if (!st.ok()) { + std::stringstream oss; + oss << "GetEntity failed: " << st.ToString(); + exec_state_ = LDBCommandExecuteResult::Failed(oss.str()); + return; + } PinnableWideColumns pinnable_wide_columns; - Status st = db_->GetEntity(ReadOptions(), GetCfHandle(), key_, - &pinnable_wide_columns); + st = db_->GetEntity(ropt, cfh, key_, &pinnable_wide_columns); if (st.ok()) { std::ostringstream oss; WideColumnsHelper::DumpWideColumns(pinnable_wide_columns.columns(), oss, @@ -3007,7 +3251,8 @@ MultiGetEntityCommand::MultiGetEntityCommand( const std::map& options, const std::vector& flags) : LDBCommand(options, flags, true /* is_read_only */, - BuildCmdLineOptions({ARG_HEX, ARG_KEY_HEX, ARG_VALUE_HEX})) { + BuildCmdLineOptions({ARG_HEX, ARG_KEY_HEX, ARG_VALUE_HEX, + ARG_READ_TIMESTAMP})) { if (params.size() < 1) { exec_state_ = LDBCommandExecuteResult::Failed( "At least one must be specified for the multi_get_entity " @@ -3024,6 +3269,7 @@ void MultiGetEntityCommand::Help(std::string& ret) { ret.append(" "); ret.append(MultiGetEntityCommand::Name()); ret.append(" ..."); + ret.append(" [--" + ARG_READ_TIMESTAMP + "=] "); ret.append("\n"); } @@ -3033,6 +3279,16 @@ void MultiGetEntityCommand::DoCommand() { return; } + ReadOptions ropt; + Slice read_timestamp; + ColumnFamilyHandle* cfh = GetCfHandle(); + Status st = MaybePopulateReadTimestamp(cfh, ropt, &read_timestamp); + if (!st.ok()) { + std::stringstream oss; + oss << "MultiGetEntity failed: " << st.ToString(); + exec_state_ = LDBCommandExecuteResult::Failed(oss.str()); + return; + } size_t num_keys = keys_.size(); std::vector key_slices; std::vector results(num_keys); @@ -3041,8 +3297,8 @@ void MultiGetEntityCommand::DoCommand() { key_slices.emplace_back(key); } - db_->MultiGetEntity(ReadOptions(), GetCfHandle(), num_keys, key_slices.data(), - results.data(), statuses.data()); + db_->MultiGetEntity(ropt, cfh, num_keys, key_slices.data(), results.data(), + statuses.data()); bool failed = false; for (size_t i = 0; i < num_keys; ++i) { @@ -3200,11 +3456,11 @@ void BatchPutCommand::OverrideBaseOptions() { ScanCommand::ScanCommand(const std::vector& /*params*/, const std::map& options, const std::vector& flags) - : LDBCommand( - options, flags, true, - BuildCmdLineOptions({ARG_TTL, ARG_NO_VALUE, ARG_HEX, ARG_KEY_HEX, - ARG_TO, ARG_VALUE_HEX, ARG_FROM, ARG_TIMESTAMP, - ARG_MAX_KEYS, ARG_TTL_START, ARG_TTL_END})), + : LDBCommand(options, flags, true, + BuildCmdLineOptions( + {ARG_TTL, ARG_NO_VALUE, ARG_HEX, ARG_KEY_HEX, ARG_TO, + ARG_VALUE_HEX, ARG_FROM, ARG_TIMESTAMP, ARG_MAX_KEYS, + ARG_TTL_START, ARG_TTL_END, ARG_READ_TIMESTAMP})), start_key_specified_(false), end_key_specified_(false), max_keys_scanned_(-1), @@ -3260,6 +3516,7 @@ void ScanCommand::Help(std::string& ret) { ret.append(" [--" + ARG_TTL_START + "=:- is inclusive]"); ret.append(" [--" + ARG_TTL_END + "=:- is exclusive]"); ret.append(" [--" + ARG_NO_VALUE + "]"); + ret.append(" [--" + ARG_READ_TIMESTAMP + "=] "); ret.append("\n"); } @@ -3271,8 +3528,19 @@ void ScanCommand::DoCommand() { int num_keys_scanned = 0; ReadOptions scan_read_opts; + ColumnFamilyHandle* cfh = GetCfHandle(); + const Comparator* ucmp = cfh->GetComparator(); + size_t ts_sz = ucmp->timestamp_size(); + Slice read_timestamp; + Status st = MaybePopulateReadTimestamp(cfh, scan_read_opts, &read_timestamp); + if (!st.ok()) { + std::stringstream oss; + oss << "Scan failed: " << st.ToString(); + exec_state_ = LDBCommandExecuteResult::Failed(oss.str()); + return; + } scan_read_opts.total_order_seek = true; - Iterator* it = db_->NewIterator(scan_read_opts, GetCfHandle()); + Iterator* it = db_->NewIterator(scan_read_opts, cfh); if (start_key_specified_) { it->Seek(start_key_); } else { @@ -3319,12 +3587,15 @@ void ScanCommand::DoCommand() { } fprintf(stdout, "%s\n", key_str.c_str()); } else { - std::string str = is_db_ttl_ ? PrintKeyValue(it->key().ToString(), - it->value().ToString(), - is_key_hex_, is_value_hex_) - : PrintKeyValueOrWideColumns( - it->key(), it->value(), it->columns(), - is_key_hex_, is_value_hex_); + std::string str = + is_db_ttl_ + ? PrintKeyValue(it->key().ToString(), + ts_sz == 0 ? "" : it->timestamp().ToString(), + it->value().ToString(), is_key_hex_, + is_value_hex_, ucmp) + : PrintKeyValueOrWideColumns( + it->key(), ts_sz == 0 ? "" : it->timestamp(), it->value(), + it->columns(), is_key_hex_, is_value_hex_, ucmp); fprintf(stdout, "%s\n", str.c_str()); } @@ -3583,6 +3854,7 @@ const char* DBQuerierCommand::HELP_CMD = "help"; const char* DBQuerierCommand::GET_CMD = "get"; const char* DBQuerierCommand::PUT_CMD = "put"; const char* DBQuerierCommand::DELETE_CMD = "delete"; +const char* DBQuerierCommand::COUNT_CMD = "count"; DBQuerierCommand::DBQuerierCommand( const std::vector& /*params*/, @@ -3611,72 +3883,211 @@ void DBQuerierCommand::DoCommand() { return; } - ReadOptions read_options; - WriteOptions write_options; - std::string line; - std::string key; - std::string value; Status s; - std::stringstream oss; - while (s.ok() && getline(std::cin, line, '\n')) { + ColumnFamilyHandle* cfh = GetCfHandle(); + const Comparator* ucmp = cfh->GetComparator(); + while ((s.ok() || s.IsNotFound() || s.IsInvalidArgument()) && + getline(std::cin, line, '\n')) { + std::string key; + std::string timestamp; + std::string value; + // Reset to OK status before parsing and executing next user command. + s = Status::OK(); + std::stringstream oss; // Parse line into std::vector std::vector tokens; + ParsedParams parsed_params; size_t pos = 0; while (true) { size_t pos2 = line.find(' ', pos); + std::string token = + line.substr(pos, (pos2 == std::string::npos) ? pos2 : (pos2 - pos)); + ParseSingleParam(token, parsed_params, tokens); if (pos2 == std::string::npos) { break; } - tokens.push_back(line.substr(pos, pos2 - pos)); pos = pos2 + 1; } - tokens.push_back(line.substr(pos)); + + if (tokens.empty() || !parsed_params.flags.empty()) { + fprintf(stdout, "Bad command\n"); + continue; + } const std::string& cmd = tokens[0]; + ReadOptions read_options; + WriteOptions write_options; + Slice read_timestamp; if (cmd == HELP_CMD) { fprintf(stdout, - "get \n" - "put \n" - "delete \n"); - } else if (cmd == DELETE_CMD && tokens.size() == 2) { + "get [--read_timestamp=]\n" + "put [] \n" + "delete []\n" + "count [--from=] [--to=] " + "[--read_timestamp=]\n"); + } else if (cmd == DELETE_CMD && parsed_params.option_map.empty()) { key = (is_key_hex_ ? HexToString(tokens[1]) : tokens[1]); - s = db_->Delete(write_options, GetCfHandle(), Slice(key)); + if (tokens.size() == 2) { + s = db_->Delete(write_options, cfh, Slice(key)); + } else if (tokens.size() == 3) { + Status encode_s = EncodeUserProvidedTimestamp(tokens[2], ×tamp); + if (encode_s.ok()) { + s = db_->Delete(write_options, cfh, Slice(key), Slice(timestamp)); + } else { + fprintf(stdout, "delete gets invalid argument: %s\n", + encode_s.ToString().c_str()); + continue; + } + } else { + fprintf(stdout, "delete gets invalid arguments\n"); + continue; + } + oss << "delete " << (is_key_hex_ ? StringToHex(key) : key); + if (!timestamp.empty()) { + oss << " write_ts: " << ucmp->TimestampToString(timestamp); + } if (s.ok()) { - fprintf(stdout, "Successfully deleted %s\n", tokens[1].c_str()); + oss << " succeeded"; } else { - oss << "delete " << key << " failed: " << s.ToString(); + oss << " failed: " << s.ToString(); } - } else if (cmd == PUT_CMD && tokens.size() == 3) { + fprintf(stdout, "%s\n", oss.str().c_str()); + } else if (cmd == PUT_CMD && parsed_params.option_map.empty()) { key = (is_key_hex_ ? HexToString(tokens[1]) : tokens[1]); - value = (is_value_hex_ ? HexToString(tokens[2]) : tokens[2]); - s = db_->Put(write_options, GetCfHandle(), Slice(key), Slice(value)); + if (tokens.size() == 3) { + value = (is_value_hex_ ? HexToString(tokens[2]) : tokens[2]); + s = db_->Put(write_options, cfh, Slice(key), Slice(value)); + } else if (tokens.size() == 4) { + value = (is_value_hex_ ? HexToString(tokens[3]) : tokens[3]); + Status encode_s = EncodeUserProvidedTimestamp(tokens[2], ×tamp); + if (encode_s.ok()) { + s = db_->Put(write_options, cfh, Slice(key), Slice(timestamp), + Slice(value)); + } else { + fprintf(stdout, "put gets invalid argument: %s\n", + encode_s.ToString().c_str()); + continue; + } + } else { + fprintf(stdout, "put gets invalid arguments\n"); + continue; + } + + oss << "put " << (is_key_hex_ ? StringToHex(key) : key); + if (!timestamp.empty()) { + oss << " write_ts: " << ucmp->TimestampToString(timestamp); + } + oss << " => " << (is_value_hex_ ? StringToHex(value) : value); if (s.ok()) { - fprintf(stdout, "Successfully put %s %s\n", tokens[1].c_str(), - tokens[2].c_str()); + oss << " succeeded"; } else { - oss << "put " << key << "=>" << value << " failed: " << s.ToString(); + oss << " failed: " << s.ToString(); } + fprintf(stdout, "%s\n", oss.str().c_str()); } else if (cmd == GET_CMD && tokens.size() == 2) { key = (is_key_hex_ ? HexToString(tokens[1]) : tokens[1]); - s = db_->Get(read_options, GetCfHandle(), Slice(key), &value); + bool bad_option = false; + for (auto& option : parsed_params.option_map) { + if (option.first == "read_timestamp") { + Status encode_s = + EncodeUserProvidedTimestamp(option.second, ×tamp); + if (!encode_s.ok()) { + fprintf(stdout, "get gets invalid argument: %s\n", + encode_s.ToString().c_str()); + bad_option = true; + break; + } + read_timestamp = timestamp; + read_options.timestamp = &read_timestamp; + } else { + fprintf(stdout, "get gets invalid arguments\n"); + bad_option = true; + break; + } + } + if (bad_option) { + continue; + } + s = db_->Get(read_options, cfh, Slice(key), &value); if (s.ok()) { fprintf(stdout, "%s\n", - PrintKeyValue(key, value, is_key_hex_, is_value_hex_).c_str()); + PrintKeyValue(key, timestamp, value, is_key_hex_, is_value_hex_, + ucmp) + .c_str()); } else { - if (s.IsNotFound()) { - fprintf(stdout, "Not found %s\n", tokens[1].c_str()); + oss << "get " << (is_key_hex_ ? StringToHex(key) : key); + if (!timestamp.empty()) { + oss << " read_timestamp: " << ucmp->TimestampToString(timestamp); + } + oss << " status: " << s.ToString(); + fprintf(stdout, "%s\n", oss.str().c_str()); + } + } else if (cmd == COUNT_CMD) { + std::string start_key; + std::string end_key; + bool bad_option = false; + for (auto& option : parsed_params.option_map) { + if (option.first == "from") { + start_key = + (is_key_hex_ ? HexToString(option.second) : option.second); + } else if (option.first == "to") { + end_key = (is_key_hex_ ? HexToString(option.second) : option.second); + } else if (option.first == "read_timestamp") { + Status encode_s = + EncodeUserProvidedTimestamp(option.second, ×tamp); + if (!encode_s.ok()) { + bad_option = true; + fprintf(stdout, "count gets invalid argument: %s\n", + encode_s.ToString().c_str()); + break; + } + read_timestamp = timestamp; + read_options.timestamp = &read_timestamp; } else { - oss << "get " << key << " error: " << s.ToString(); + fprintf(stdout, "count gets invalid arguments\n"); + bad_option = true; + break; + } + } + if (bad_option) { + continue; + } + + Slice end_key_slice(end_key); + uint64_t count = 0; + if (!end_key.empty()) { + read_options.iterate_upper_bound = &end_key_slice; + } + std::unique_ptr iter(db_->NewIterator(read_options, cfh)); + if (start_key.empty()) { + iter->SeekToFirst(); + } else { + iter->Seek(start_key); + } + while (iter->status().ok() && iter->Valid()) { + count++; + iter->Next(); + } + if (iter->status().ok()) { + fprintf(stdout, "%" PRIu64 "\n", count); + } else { + oss << "scan from " + << (is_key_hex_ ? StringToHex(start_key) : start_key); + if (!timestamp.empty()) { + oss << " read_timestamp: " << ucmp->TimestampToString(timestamp); } + oss << " to " << (is_key_hex_ ? StringToHex(end_key) : end_key) + << " failed: " << iter->status().ToString(); + fprintf(stdout, "%s\n", oss.str().c_str()); } } else { fprintf(stdout, "Unknown command %s\n", line.c_str()); } } - if (!s.ok()) { - exec_state_ = LDBCommandExecuteResult::Failed(oss.str()); + if (!(s.ok() || s.IsNotFound() || s.IsInvalidArgument())) { + exec_state_ = LDBCommandExecuteResult::Failed(s.ToString()); } } @@ -4012,6 +4423,18 @@ void DumpBlobFile(const std::string& filename, bool is_key_hex, fprintf(stderr, "Failed: %s\n", s.ToString().c_str()); } } + +Status EncodeUserProvidedTimestamp(const std::string& user_timestamp, + std::string* ts_buf) { + uint64_t int_timestamp; + std::istringstream iss(user_timestamp); + if (!(iss >> int_timestamp)) { + return Status::InvalidArgument( + "user provided timestamp is not a valid uint64 value."); + } + EncodeU64Ts(int_timestamp, ts_buf); + return Status::OK(); +} } // namespace DBFileDumperCommand::DBFileDumperCommand( @@ -4106,7 +4529,7 @@ void DBFileDumperCommand::DoCommand() { std::cout << "Write Ahead Log Files" << std::endl; std::cout << "==============================" << std::endl; - ROCKSDB_NAMESPACE::VectorLogPtr wal_files; + ROCKSDB_NAMESPACE::VectorWalPtr wal_files; s = db_->GetSortedWalFiles(wal_files); if (!s.ok()) { std::cerr << "Error when getting WAL files" << std::endl; @@ -4123,7 +4546,7 @@ void DBFileDumperCommand::DoCommand() { std::cout << filename << std::endl; // TODO(myabandeh): allow configuring is_write_commited DumpWalFile(options_, filename, true, true, true /* is_write_commited */, - &exec_state_); + ucmps_, &exec_state_); } } } diff --git a/tools/ldb_cmd_impl.h b/tools/ldb_cmd_impl.h index 15ff571b219..32bf4da5988 100644 --- a/tools/ldb_cmd_impl.h +++ b/tools/ldb_cmd_impl.h @@ -369,7 +369,7 @@ class WALDumperCommand : public LDBCommand { const std::map& options, const std::vector& flags); - bool NoDBOpen() override { return true; } + bool NoDBOpen() override { return no_db_open_; } static void Help(std::string& ret); @@ -380,6 +380,7 @@ class WALDumperCommand : public LDBCommand { std::string wal_file_; bool print_values_; bool is_write_committed_; // default will be set to true + bool no_db_open_ = true; static const std::string ARG_WAL_FILE; static const std::string ARG_WRITE_COMMITTED; @@ -619,6 +620,7 @@ class DBQuerierCommand : public LDBCommand { static const char* GET_CMD; static const char* PUT_CMD; static const char* DELETE_CMD; + static const char* COUNT_CMD; }; class CheckConsistencyCommand : public LDBCommand { diff --git a/tools/ldb_tool.cc b/tools/ldb_tool.cc index 86fd37ef4ff..a3dd22dfcfc 100644 --- a/tools/ldb_tool.cc +++ b/tools/ldb_tool.cc @@ -28,6 +28,9 @@ void LDBCommandRunner::PrintHelp(const LDBOptions& ldb_options, ret.append(" --" + LDBCommand::ARG_SECONDARY_PATH + "= to open DB as secondary instance. Operations " "not supported in secondary instance will fail.\n\n"); + ret.append(" --" + LDBCommand::ARG_LEADER_PATH + + "= to open DB as a follower instance. Operations " + "not supported in follower instance will fail.\n\n"); ret.append( "The following optional parameters control if keys/values are " "input/output as hex or as plain strings:\n"); @@ -85,6 +88,9 @@ void LDBCommandRunner::PrintHelp(const LDBOptions& ldb_options, "=\n"); ret.append(" --" + LDBCommand::ARG_BLOB_COMPACTION_READAHEAD_SIZE + "=\n"); + ret.append(" --" + LDBCommand::ARG_READ_TIMESTAMP + + "= : read timestamp, required if column " + "family enables timestamp, otherwise invalid if provided."); ret.append("\n\n"); ret.append("Data Access Commands:\n"); diff --git a/unreleased_history/behavior_changes/flush_wal_close.md b/unreleased_history/behavior_changes/flush_wal_close.md new file mode 100644 index 00000000000..27fd0594eb1 --- /dev/null +++ b/unreleased_history/behavior_changes/flush_wal_close.md @@ -0,0 +1 @@ +* When `manual_wal_flush=true`, `avoid_sync_during_shutdown=false` will flush WALs before syncing them diff --git a/unreleased_history/behavior_changes/wal_sync_on_close.md b/unreleased_history/behavior_changes/wal_sync_on_close.md new file mode 100644 index 00000000000..1a34b95b3f6 --- /dev/null +++ b/unreleased_history/behavior_changes/wal_sync_on_close.md @@ -0,0 +1 @@ +Introduce a new DB option `avoid_sync_during_shutdown`. If set true, it will sync WALs during DB close diff --git a/unreleased_history/new_features/get_entity_for_update.md b/unreleased_history/new_features/get_entity_for_update.md new file mode 100644 index 00000000000..2a9c2afa04a --- /dev/null +++ b/unreleased_history/new_features/get_entity_for_update.md @@ -0,0 +1 @@ +Optimistic transactions and pessimistic transactions with the WriteCommitted policy now support the `GetEntityForUpdate` API. diff --git a/unreleased_history/new_features/ldb_count_command.md b/unreleased_history/new_features/ldb_count_command.md new file mode 100644 index 00000000000..d6312221478 --- /dev/null +++ b/unreleased_history/new_features/ldb_count_command.md @@ -0,0 +1 @@ +Added a new "count" command to the ldb repl shell. By default, it prints a count of keys in the database from start to end. The options --from= and/or --to= can be specified to limit the range. diff --git a/unreleased_history/new_features/write_batch_c_api b/unreleased_history/new_features/write_batch_c_api new file mode 100644 index 00000000000..150af09df86 --- /dev/null +++ b/unreleased_history/new_features/write_batch_c_api @@ -0,0 +1,3 @@ +* Add `rocksdb_writebatch_update_timestamps`, `rocksdb_writebatch_wi_update_timestamps` in C API. +* Add `rocksdb_iter_refresh` in C API. +* Add `rocksdb_writebatch_create_with_params`, `rocksdb_writebatch_wi_create_with_params` to create WB and WBWI with all options in C API diff --git a/unreleased_history/performance_improvements/parallel_notify-one_i_LaunchParallelMemTableWriters.md b/unreleased_history/performance_improvements/parallel_notify-one_i_LaunchParallelMemTableWriters.md new file mode 100644 index 00000000000..8fbfca124bc --- /dev/null +++ b/unreleased_history/performance_improvements/parallel_notify-one_i_LaunchParallelMemTableWriters.md @@ -0,0 +1 @@ +Improved write throughput to memtable when there's a large number of concurrent writers and allow_concurrent_memtable_write=true(#12545) \ No newline at end of file diff --git a/unreleased_history/public_api_changes/rename_to_wal_file.md b/unreleased_history/public_api_changes/rename_to_wal_file.md new file mode 100644 index 00000000000..58df40c473f --- /dev/null +++ b/unreleased_history/public_api_changes/rename_to_wal_file.md @@ -0,0 +1 @@ +* Deprecated names `LogFile` and `VectorLogPtr` in favor of new names `WalFile` and `VectorWalPtr`. diff --git a/unreleased_history/public_api_changes/universal_max_read_amp.md b/unreleased_history/public_api_changes/universal_max_read_amp.md new file mode 100644 index 00000000000..0c10a37b974 --- /dev/null +++ b/unreleased_history/public_api_changes/universal_max_read_amp.md @@ -0,0 +1 @@ +* Introduce a new universal compaction option CompactionOptionsUniversal::max_read_amp which allows user to define the limit on the number of sorted runs separately from the trigger for compaction (`level0_file_num_compaction_trigger`) #12477. \ No newline at end of file diff --git a/util/aligned_storage.h b/util/aligned_storage.h new file mode 100644 index 00000000000..70cf1e346d1 --- /dev/null +++ b/util/aligned_storage.h @@ -0,0 +1,24 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). +// +// Copyright (c) 2011 The LevelDB Authors. All rights reserved. +// Use of this source code is governed by a BSD-style license that can be +// found in the LICENSE file. See the AUTHORS file for names of contributors. +#pragma once + +#include + +#include "rocksdb/rocksdb_namespace.h" + +namespace ROCKSDB_NAMESPACE { + +template +struct aligned_storage { + struct type { + alignas(Align) unsigned char data[sizeof(T)]; + }; +}; + +} // namespace ROCKSDB_NAMESPACE diff --git a/util/comparator.cc b/util/comparator.cc index 78e54aaf40e..b31f5f3209d 100644 --- a/util/comparator.cc +++ b/util/comparator.cc @@ -276,6 +276,13 @@ class ComparatorWithU64TsImpl : public Comparator { Slice GetMinTimestamp() const override { return MinU64Ts(); } + std::string TimestampToString(const Slice& timestamp) const override { + assert(timestamp.size() == sizeof(uint64_t)); + uint64_t ts = 0; + DecodeU64Ts(timestamp, &ts).PermitUncheckedError(); + return std::to_string(ts); + } + 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/random.cc b/util/random.cc index 17396d32ff3..e43eb3198ef 100644 --- a/util/random.cc +++ b/util/random.cc @@ -13,6 +13,7 @@ #include #include "port/likely.h" +#include "util/aligned_storage.h" #include "util/thread_local.h" #define STORAGE_DECL static thread_local @@ -21,7 +22,7 @@ namespace ROCKSDB_NAMESPACE { Random* Random::GetTLSInstance() { STORAGE_DECL Random* tls_instance; - STORAGE_DECL std::aligned_storage::type tls_instance_bytes; + STORAGE_DECL aligned_storage::type tls_instance_bytes; auto rv = tls_instance; if (UNLIKELY(rv == nullptr)) { diff --git a/util/stderr_logger.cc b/util/stderr_logger.cc index 69e9989f04e..1005b724b5d 100644 --- a/util/stderr_logger.cc +++ b/util/stderr_logger.cc @@ -38,12 +38,12 @@ void StderrLogger::Logv(const char* format, va_list ap) { va_list ap_copy; va_copy(ap_copy, ap); - const size_t log_suffix_len = vsnprintf(nullptr, 0, format, ap_copy); + const size_t log_suffix_len = vsnprintf(nullptr, 0, format, ap_copy) + 1; va_end(ap_copy); // Allocate space for the context, log_prefix, and log itself // Extra byte for null termination - size_t buf_len = ctx_len + log_prefix_len + log_suffix_len + 1; + size_t buf_len = ctx_len + log_prefix_len + log_suffix_len; std::unique_ptr buf(new char[buf_len]); // If the logger was created without a prefix, the prefix is a nullptr @@ -55,8 +55,7 @@ void StderrLogger::Logv(const char* format, va_list ap) { t.tm_year + 1900, t.tm_mon + 1, t.tm_mday, t.tm_hour, t.tm_min, t.tm_sec, static_cast(now_tv.tv_usec), static_cast(thread_id), prefix); - written += vsnprintf(buf.get() + written, log_suffix_len, format, ap); - buf[written] = '\0'; + vsnprintf(buf.get() + written, log_suffix_len, format, ap); fprintf(stderr, "%s%c", buf.get(), '\n'); } diff --git a/util/udt_util.cc b/util/udt_util.cc index 40cf1e4964b..c349ddf586d 100644 --- a/util/udt_util.cc +++ b/util/udt_util.cc @@ -158,6 +158,39 @@ Status TimestampRecoveryHandler::PutCF(uint32_t cf, const Slice& key, return WriteBatchInternal::Put(new_batch_.get(), cf, new_key, value); } +Status TimestampRecoveryHandler::PutEntityCF(uint32_t cf, const Slice& key, + const Slice& entity) { + std::string new_key_buf; + Slice new_key; + Status status = TimestampRecoveryHandler::ReconcileTimestampDiscrepancy( + cf, key, &new_key_buf, &new_key); + if (!status.ok()) { + return status; + } + Slice entity_copy = entity; + WideColumns columns; + if (!WideColumnSerialization::Deserialize(entity_copy, columns).ok()) { + return Status::Corruption("Unable to deserialize entity", + entity.ToString(/* hex */ true)); + } + + return WriteBatchInternal::PutEntity(new_batch_.get(), cf, new_key, columns); +} + +Status TimestampRecoveryHandler::TimedPutCF(uint32_t cf, const Slice& key, + const Slice& value, + uint64_t write_time) { + std::string new_key_buf; + Slice new_key; + Status status = + ReconcileTimestampDiscrepancy(cf, key, &new_key_buf, &new_key); + if (!status.ok()) { + return status; + } + return WriteBatchInternal::TimedPut(new_batch_.get(), cf, new_key, value, + write_time); +} + Status TimestampRecoveryHandler::DeleteCF(uint32_t cf, const Slice& key) { std::string new_key_buf; Slice new_key; diff --git a/util/udt_util.h b/util/udt_util.h index b524fceab0d..a926e8e8126 100644 --- a/util/udt_util.h +++ b/util/udt_util.h @@ -11,6 +11,7 @@ #include #include +#include "db/wide/wide_column_serialization.h" #include "db/write_batch_internal.h" #include "rocksdb/slice.h" #include "rocksdb/status.h" @@ -116,6 +117,12 @@ class TimestampRecoveryHandler : public WriteBatch::Handler { Status PutCF(uint32_t cf, const Slice& key, const Slice& value) override; + Status PutEntityCF(uint32_t cf, const Slice& key, + const Slice& entity) override; + + Status TimedPutCF(uint32_t cf, const Slice& key, const Slice& value, + uint64_t write_time) override; + Status DeleteCF(uint32_t cf, const Slice& key) override; Status SingleDeleteCF(uint32_t cf, const Slice& key) override; diff --git a/util/udt_util_test.cc b/util/udt_util_test.cc index 8f45d564a5b..ecd5e1773f2 100644 --- a/util/udt_util_test.cc +++ b/util/udt_util_test.cc @@ -16,6 +16,7 @@ namespace ROCKSDB_NAMESPACE { namespace { static const std::string kTestKeyWithoutTs = "key"; static const std::string kValuePlaceHolder = "value"; +static const uint64_t kWriteUnixTime = 100; } // namespace class HandleTimestampSizeDifferenceTest : public testing::Test { @@ -38,6 +39,34 @@ class HandleTimestampSizeDifferenceTest : public testing::Test { return AddKey(cf, key); } + Status TimedPutCF(uint32_t cf, const Slice& key, const Slice& value, + uint64_t write_unix_time) override { + if (value.compare(kValuePlaceHolder) != 0) { + return Status::InvalidArgument(); + } + if (write_unix_time != kWriteUnixTime) { + return Status::InvalidArgument(); + } + return AddKey(cf, key); + } + + Status PutEntityCF(uint32_t cf, const Slice& key, + const Slice& entity) override { + Slice entity_copy = entity; + WideColumns columns; + Status s = WideColumnSerialization::Deserialize(entity_copy, columns); + if (!s.ok()) { + return s; + } + if (columns.size() != 1) { + return Status::InvalidArgument(); + } + if (columns[0].value().compare(kValuePlaceHolder) != 0) { + return Status::InvalidArgument(); + } + return AddKey(cf, key); + } + Status DeleteCF(uint32_t cf, const Slice& key) override { return AddKey(cf, key); } @@ -117,6 +146,10 @@ class HandleTimestampSizeDifferenceTest : public testing::Test { WriteBatchInternal::Merge(batch, cf_id, key, kValuePlaceHolder)); ASSERT_OK(WriteBatchInternal::PutBlobIndex(batch, cf_id, key, kValuePlaceHolder)); + ASSERT_OK(WriteBatchInternal::TimedPut( + batch, cf_id, key, kValuePlaceHolder, kWriteUnixTime)); + WideColumns columns{{kDefaultWideColumnName, kValuePlaceHolder}}; + ASSERT_OK(WriteBatchInternal::PutEntity(batch, cf_id, key, columns)); } } diff --git a/util/write_batch_util.h b/util/write_batch_util.h index 6986d25d0d0..294b302831e 100644 --- a/util/write_batch_util.h +++ b/util/write_batch_util.h @@ -32,6 +32,11 @@ class ColumnFamilyCollector : public WriteBatch::Handler { return AddColumnFamilyId(column_family_id); } + Status PutEntityCF(uint32_t column_family_id, const Slice&, + const Slice&) override { + return AddColumnFamilyId(column_family_id); + } + Status TimedPutCF(uint32_t column_family_id, const Slice&, const Slice&, uint64_t) override { return AddColumnFamilyId(column_family_id); diff --git a/util/xxhash.h b/util/xxhash.h index 2bf111b9565..8f2303b7e77 100644 --- a/util/xxhash.h +++ b/util/xxhash.h @@ -11,6 +11,15 @@ #ifndef XXH_NAMESPACE #define XXH_NAMESPACE ROCKSDB_ #endif // !defined(XXH_NAMESPACE) + +#if (defined(XXH_INLINE_ALL) || defined(XXH_PRIVATE_API) || \ + defined(XXH_IMPLEMENTATION)) && \ + !defined(XXH_IMPLEM_13a8737387) +#if defined(__cplusplus) && (__cplusplus > 202002L) +/* C++23 and future versions have std::unreachable() */ +#include /* std::unreachable() */ +#endif +#endif /* END RocksDB customizations */ // clang-format off @@ -2064,8 +2073,6 @@ static int XXH_isLittleEndian(void) # define XXH_UNREACHABLE() unreachable() #elif defined(__cplusplus) && (__cplusplus > 202002L) -/* C++23 and future versions have std::unreachable() */ -# include /* std::unreachable() */ # define XXH_UNREACHABLE() std::unreachable() #elif XXH_HAS_BUILTIN(__builtin_unreachable) diff --git a/utilities/backup/backup_engine_test.cc b/utilities/backup/backup_engine_test.cc index 917effceab8..9882ea7daaf 100644 --- a/utilities/backup/backup_engine_test.cc +++ b/utilities/backup/backup_engine_test.cc @@ -4406,9 +4406,9 @@ TEST_F(BackupEngineTest, ExcludeFiles) { delete db; db = nullptr; - for (auto be_pair : - {std::make_pair(backup_engine_.get(), alt_backup_engine), - std::make_pair(alt_backup_engine, backup_engine_.get())}) { + auto backup_engine = backup_engine_.get(); + for (auto be_pair : {std::make_pair(backup_engine, alt_backup_engine), + std::make_pair(alt_backup_engine, backup_engine)}) { ASSERT_OK(DestroyDB(dbname_, options_)); RestoreOptions ro; // Fails without alternate dir @@ -4430,9 +4430,9 @@ TEST_F(BackupEngineTest, ExcludeFiles) { CloseBackupEngine(); OpenBackupEngine(); - for (auto be_pair : - {std::make_pair(backup_engine_.get(), alt_backup_engine), - std::make_pair(alt_backup_engine, backup_engine_.get())}) { + backup_engine = backup_engine_.get(); + for (auto be_pair : {std::make_pair(backup_engine, alt_backup_engine), + std::make_pair(alt_backup_engine, backup_engine)}) { ASSERT_OK(DestroyDB(dbname_, options_)); RestoreOptions ro; ro.alternate_dirs.push_front(be_pair.second); @@ -4459,9 +4459,9 @@ TEST_F(BackupEngineTest, ExcludeFiles) { AssertBackupInfoConsistency(/*allow excluded*/ true); // Excluded file(s) deleted, unable to restore - for (auto be_pair : - {std::make_pair(backup_engine_.get(), alt_backup_engine), - std::make_pair(alt_backup_engine, backup_engine_.get())}) { + backup_engine = backup_engine_.get(); + for (auto be_pair : {std::make_pair(backup_engine, alt_backup_engine), + std::make_pair(alt_backup_engine, backup_engine)}) { RestoreOptions ro; ro.alternate_dirs.push_front(be_pair.second); ASSERT_TRUE(be_pair.first->RestoreDBFromLatestBackup(dbname_, dbname_, ro) @@ -4475,9 +4475,9 @@ TEST_F(BackupEngineTest, ExcludeFiles) { AssertBackupInfoConsistency(/*allow excluded*/ true); // Excluded file(s) deleted, unable to restore - for (auto be_pair : - {std::make_pair(backup_engine_.get(), alt_backup_engine), - std::make_pair(alt_backup_engine, backup_engine_.get())}) { + backup_engine = backup_engine_.get(); + for (auto be_pair : {std::make_pair(backup_engine, alt_backup_engine), + std::make_pair(alt_backup_engine, backup_engine)}) { RestoreOptions ro; ro.alternate_dirs.push_front(be_pair.second); ASSERT_TRUE(be_pair.first->RestoreDBFromLatestBackup(dbname_, dbname_, ro) diff --git a/utilities/debug.cc b/utilities/debug.cc index 6274cd6c2d0..f5d4d55de3a 100644 --- a/utilities/debug.cc +++ b/utilities/debug.cc @@ -85,9 +85,21 @@ Status GetAllKeyVersions(DB* db, ColumnFamilyHandle* cfh, Slice begin_key, ScopedArenaPtr iter( idb->NewInternalIterator(read_options, &arena, kMaxSequenceNumber, cfh)); - if (!begin_key.empty()) { + const Comparator* ucmp = icmp.user_comparator(); + size_t ts_sz = ucmp->timestamp_size(); + + Slice from_slice = begin_key; + bool has_begin = !begin_key.empty(); + Slice end_slice = end_key; + bool has_end = !end_key.empty(); + std::string begin_key_buf, end_key_buf; + auto [from, end] = MaybeAddTimestampsToRange( + has_begin ? &from_slice : nullptr, has_end ? &end_slice : nullptr, ts_sz, + &begin_key_buf, &end_key_buf); + if (has_begin) { + assert(from.has_value()); InternalKey ikey; - ikey.SetMinPossibleForUserKey(begin_key); + ikey.SetMinPossibleForUserKey(from.value()); iter->Seek(ikey.Encode()); } else { iter->SeekToFirst(); @@ -102,8 +114,8 @@ Status GetAllKeyVersions(DB* db, ColumnFamilyHandle* cfh, Slice begin_key, return pik_status; } - if (!end_key.empty() && - icmp.user_comparator()->Compare(ikey.user_key, end_key) > 0) { + if (has_end && end.has_value() && + icmp.user_comparator()->Compare(ikey.user_key, end.value()) > 0) { break; } diff --git a/utilities/transactions/optimistic_transaction_test.cc b/utilities/transactions/optimistic_transaction_test.cc index eb3511be592..833f03ce917 100644 --- a/utilities/transactions/optimistic_transaction_test.cc +++ b/utilities/transactions/optimistic_transaction_test.cc @@ -1142,8 +1142,12 @@ TEST_P(OptimisticTransactionTest, UntrackedWrites) { delete txn; txn = txn_db->BeginTransaction(write_options); + const WideColumns untracked_columns{{"hello", "world"}}; + ASSERT_OK(txn->Put("tracked", "1")); ASSERT_OK(txn->PutUntracked("untracked", "1")); + ASSERT_OK(txn->PutEntityUntracked(txn_db->DefaultColumnFamily(), "untracked", + untracked_columns)); ASSERT_OK(txn->MergeUntracked("untracked", "2")); ASSERT_OK(txn->DeleteUntracked("untracked")); @@ -1159,8 +1163,12 @@ TEST_P(OptimisticTransactionTest, UntrackedWrites) { delete txn; txn = txn_db->BeginTransaction(write_options); + const WideColumns untracked_new_columns{{"foo", "bar"}}; + ASSERT_OK(txn->Put("tracked", "10")); ASSERT_OK(txn->PutUntracked("untracked", "A")); + ASSERT_OK(txn->PutEntityUntracked(txn_db->DefaultColumnFamily(), "untracked", + untracked_new_columns)); // Write to tracked key outside of the transaction and verify that the // untracked keys are not written when the commit fails. @@ -1687,6 +1695,533 @@ TEST_P(OptimisticTransactionTest, TimestampedSnapshotSetCommitTs) { ASSERT_TRUE(s.IsNotSupported()); } +TEST_P(OptimisticTransactionTest, PutEntitySuccess) { + constexpr char foo[] = "foo"; + const WideColumns foo_columns{ + {kDefaultWideColumnName, "bar"}, {"col1", "val1"}, {"col2", "val2"}}; + const WideColumns foo_new_columns{ + {kDefaultWideColumnName, "baz"}, {"colA", "valA"}, {"colB", "valB"}}; + + ASSERT_OK(txn_db->PutEntity(WriteOptions(), txn_db->DefaultColumnFamily(), + foo, foo_columns)); + + { + std::unique_ptr txn(txn_db->BeginTransaction(WriteOptions())); + + ASSERT_NE(txn, nullptr); + ASSERT_EQ(txn->GetNumPutEntities(), 0); + + { + PinnableWideColumns columns; + ASSERT_OK(txn->GetEntity(ReadOptions(), txn_db->DefaultColumnFamily(), + foo, &columns)); + ASSERT_EQ(columns.columns(), foo_columns); + } + + { + PinnableWideColumns columns; + ASSERT_OK(txn->GetEntityForUpdate( + ReadOptions(), txn_db->DefaultColumnFamily(), foo, &columns)); + ASSERT_EQ(columns.columns(), foo_columns); + } + + ASSERT_OK( + txn->PutEntity(txn_db->DefaultColumnFamily(), foo, foo_new_columns)); + + ASSERT_EQ(txn->GetNumPutEntities(), 1); + + { + PinnableWideColumns columns; + ASSERT_OK(txn->GetEntity(ReadOptions(), txn_db->DefaultColumnFamily(), + foo, &columns)); + ASSERT_EQ(columns.columns(), foo_new_columns); + } + + { + PinnableWideColumns columns; + ASSERT_OK(txn->GetEntityForUpdate( + ReadOptions(), txn_db->DefaultColumnFamily(), foo, &columns)); + ASSERT_EQ(columns.columns(), foo_new_columns); + } + + ASSERT_OK(txn->Commit()); + } + + { + PinnableWideColumns columns; + ASSERT_OK(txn_db->GetEntity(ReadOptions(), txn_db->DefaultColumnFamily(), + foo, &columns)); + ASSERT_EQ(columns.columns(), foo_new_columns); + } +} + +TEST_P(OptimisticTransactionTest, PutEntityWriteConflict) { + constexpr char foo[] = "foo"; + const WideColumns foo_columns{ + {kDefaultWideColumnName, "bar"}, {"col1", "val1"}, {"col2", "val2"}}; + constexpr char baz[] = "baz"; + const WideColumns baz_columns{ + {kDefaultWideColumnName, "quux"}, {"colA", "valA"}, {"colB", "valB"}}; + + ASSERT_OK(txn_db->PutEntity(WriteOptions(), txn_db->DefaultColumnFamily(), + foo, foo_columns)); + ASSERT_OK(txn_db->PutEntity(WriteOptions(), txn_db->DefaultColumnFamily(), + baz, baz_columns)); + + std::unique_ptr txn(txn_db->BeginTransaction(WriteOptions())); + ASSERT_NE(txn, nullptr); + + { + PinnableWideColumns columns; + ASSERT_OK(txn->GetEntity(ReadOptions(), txn_db->DefaultColumnFamily(), foo, + &columns)); + ASSERT_EQ(columns.columns(), foo_columns); + } + + { + PinnableWideColumns columns; + ASSERT_OK(txn->GetEntity(ReadOptions(), txn_db->DefaultColumnFamily(), baz, + &columns)); + ASSERT_EQ(columns.columns(), baz_columns); + } + + { + constexpr size_t num_keys = 2; + + std::array keys{{foo, baz}}; + std::array results; + std::array statuses; + + txn->MultiGetEntity(ReadOptions(), txn_db->DefaultColumnFamily(), num_keys, + keys.data(), results.data(), statuses.data()); + + ASSERT_OK(statuses[0]); + ASSERT_OK(statuses[1]); + + ASSERT_EQ(results[0].columns(), foo_columns); + ASSERT_EQ(results[1].columns(), baz_columns); + } + + const WideColumns foo_new_columns{{kDefaultWideColumnName, "FOO"}, + {"hello", "world"}}; + const WideColumns baz_new_columns{{kDefaultWideColumnName, "BAZ"}, + {"ping", "pong"}}; + + ASSERT_OK( + txn->PutEntity(txn_db->DefaultColumnFamily(), foo, foo_new_columns)); + ASSERT_OK( + txn->PutEntity(txn_db->DefaultColumnFamily(), baz, baz_new_columns)); + + { + PinnableWideColumns columns; + ASSERT_OK(txn->GetEntity(ReadOptions(), txn_db->DefaultColumnFamily(), foo, + &columns)); + ASSERT_EQ(columns.columns(), foo_new_columns); + } + + { + PinnableWideColumns columns; + ASSERT_OK(txn->GetEntity(ReadOptions(), txn_db->DefaultColumnFamily(), baz, + &columns)); + ASSERT_EQ(columns.columns(), baz_new_columns); + } + + { + constexpr size_t num_keys = 2; + + std::array keys{{foo, baz}}; + std::array results; + std::array statuses; + + txn->MultiGetEntity(ReadOptions(), txn_db->DefaultColumnFamily(), num_keys, + keys.data(), results.data(), statuses.data()); + + ASSERT_OK(statuses[0]); + ASSERT_OK(statuses[1]); + + ASSERT_EQ(results[0].columns(), foo_new_columns); + ASSERT_EQ(results[1].columns(), baz_new_columns); + } + + // This PutEntity outside of a transaction will conflict with the previous + // write + const WideColumns foo_conflict_columns{{kDefaultWideColumnName, "X"}, + {"conflicting", "write"}}; + ASSERT_OK(txn_db->PutEntity(WriteOptions(), txn_db->DefaultColumnFamily(), + foo, foo_conflict_columns)); + + { + PinnableWideColumns columns; + ASSERT_OK(txn_db->GetEntity(ReadOptions(), txn_db->DefaultColumnFamily(), + foo, &columns)); + ASSERT_EQ(columns.columns(), foo_conflict_columns); + } + + ASSERT_TRUE(txn->Commit().IsBusy()); // Txn should not commit + + // Verify that transaction did not write anything + { + PinnableWideColumns columns; + ASSERT_OK(txn_db->GetEntity(ReadOptions(), txn_db->DefaultColumnFamily(), + foo, &columns)); + ASSERT_EQ(columns.columns(), foo_conflict_columns); + } + + { + PinnableWideColumns columns; + ASSERT_OK(txn_db->GetEntity(ReadOptions(), txn_db->DefaultColumnFamily(), + baz, &columns)); + ASSERT_EQ(columns.columns(), baz_columns); + } + + { + constexpr size_t num_keys = 2; + + std::array keys{{foo, baz}}; + std::array results; + std::array statuses; + constexpr bool sorted_input = false; + + txn_db->MultiGetEntity(ReadOptions(), txn_db->DefaultColumnFamily(), + num_keys, keys.data(), results.data(), + statuses.data(), sorted_input); + + ASSERT_OK(statuses[0]); + ASSERT_OK(statuses[1]); + + ASSERT_EQ(results[0].columns(), foo_conflict_columns); + ASSERT_EQ(results[1].columns(), baz_columns); + } +} + +TEST_P(OptimisticTransactionTest, PutEntityWriteConflictTxnTxn) { + constexpr char foo[] = "foo"; + const WideColumns foo_columns{ + {kDefaultWideColumnName, "bar"}, {"col1", "val1"}, {"col2", "val2"}}; + constexpr char baz[] = "baz"; + const WideColumns baz_columns{ + {kDefaultWideColumnName, "quux"}, {"colA", "valA"}, {"colB", "valB"}}; + + ASSERT_OK(txn_db->PutEntity(WriteOptions(), txn_db->DefaultColumnFamily(), + foo, foo_columns)); + ASSERT_OK(txn_db->PutEntity(WriteOptions(), txn_db->DefaultColumnFamily(), + baz, baz_columns)); + + std::unique_ptr txn(txn_db->BeginTransaction(WriteOptions())); + ASSERT_NE(txn, nullptr); + + { + PinnableWideColumns columns; + ASSERT_OK(txn->GetEntity(ReadOptions(), txn_db->DefaultColumnFamily(), foo, + &columns)); + ASSERT_EQ(columns.columns(), foo_columns); + } + + { + PinnableWideColumns columns; + ASSERT_OK(txn->GetEntity(ReadOptions(), txn_db->DefaultColumnFamily(), baz, + &columns)); + ASSERT_EQ(columns.columns(), baz_columns); + } + + { + constexpr size_t num_keys = 2; + + std::array keys{{foo, baz}}; + std::array results; + std::array statuses; + + txn->MultiGetEntity(ReadOptions(), txn_db->DefaultColumnFamily(), num_keys, + keys.data(), results.data(), statuses.data()); + + ASSERT_OK(statuses[0]); + ASSERT_OK(statuses[1]); + + ASSERT_EQ(results[0].columns(), foo_columns); + ASSERT_EQ(results[1].columns(), baz_columns); + } + + const WideColumns foo_new_columns{{kDefaultWideColumnName, "FOO"}, + {"hello", "world"}}; + const WideColumns baz_new_columns{{kDefaultWideColumnName, "BAZ"}, + {"ping", "pong"}}; + + ASSERT_OK( + txn->PutEntity(txn_db->DefaultColumnFamily(), foo, foo_new_columns)); + ASSERT_OK( + txn->PutEntity(txn_db->DefaultColumnFamily(), baz, baz_new_columns)); + + { + PinnableWideColumns columns; + ASSERT_OK(txn->GetEntity(ReadOptions(), txn_db->DefaultColumnFamily(), foo, + &columns)); + ASSERT_EQ(columns.columns(), foo_new_columns); + } + + { + PinnableWideColumns columns; + ASSERT_OK(txn->GetEntity(ReadOptions(), txn_db->DefaultColumnFamily(), baz, + &columns)); + ASSERT_EQ(columns.columns(), baz_new_columns); + } + + { + constexpr size_t num_keys = 2; + + std::array keys{{foo, baz}}; + std::array results; + std::array statuses; + + txn->MultiGetEntity(ReadOptions(), txn_db->DefaultColumnFamily(), num_keys, + keys.data(), results.data(), statuses.data()); + + ASSERT_OK(statuses[0]); + ASSERT_OK(statuses[1]); + + ASSERT_EQ(results[0].columns(), foo_new_columns); + ASSERT_EQ(results[1].columns(), baz_new_columns); + } + + std::unique_ptr conflicting_txn( + txn_db->BeginTransaction(WriteOptions())); + ASSERT_NE(conflicting_txn, nullptr); + + const WideColumns foo_conflict_columns{{kDefaultWideColumnName, "X"}, + {"conflicting", "write"}}; + ASSERT_OK(conflicting_txn->PutEntity(txn_db->DefaultColumnFamily(), foo, + foo_conflict_columns)); + ASSERT_OK(conflicting_txn->Commit()); + + { + PinnableWideColumns columns; + ASSERT_OK(txn_db->GetEntity(ReadOptions(), txn_db->DefaultColumnFamily(), + foo, &columns)); + ASSERT_EQ(columns.columns(), foo_conflict_columns); + } + + ASSERT_TRUE(txn->Commit().IsBusy()); // Txn should not commit + + // Verify that transaction did not write anything + { + PinnableWideColumns columns; + ASSERT_OK(txn_db->GetEntity(ReadOptions(), txn_db->DefaultColumnFamily(), + foo, &columns)); + ASSERT_EQ(columns.columns(), foo_conflict_columns); + } + + { + PinnableWideColumns columns; + ASSERT_OK(txn_db->GetEntity(ReadOptions(), txn_db->DefaultColumnFamily(), + baz, &columns)); + ASSERT_EQ(columns.columns(), baz_columns); + } + + { + constexpr size_t num_keys = 2; + + std::array keys{{foo, baz}}; + std::array results; + std::array statuses; + constexpr bool sorted_input = false; + + txn_db->MultiGetEntity(ReadOptions(), txn_db->DefaultColumnFamily(), + num_keys, keys.data(), results.data(), + statuses.data(), sorted_input); + + ASSERT_OK(statuses[0]); + ASSERT_OK(statuses[1]); + + ASSERT_EQ(results[0].columns(), foo_conflict_columns); + ASSERT_EQ(results[1].columns(), baz_columns); + } +} + +TEST_P(OptimisticTransactionTest, PutEntityReadConflict) { + constexpr char foo[] = "foo"; + const WideColumns foo_columns{ + {kDefaultWideColumnName, "bar"}, {"col1", "val1"}, {"col2", "val2"}}; + + ASSERT_OK(txn_db->PutEntity(WriteOptions(), txn_db->DefaultColumnFamily(), + foo, foo_columns)); + + std::unique_ptr txn(txn_db->BeginTransaction(WriteOptions())); + ASSERT_NE(txn, nullptr); + + txn->SetSnapshot(); + + ReadOptions snapshot_read_options; + snapshot_read_options.snapshot = txn->GetSnapshot(); + + { + PinnableWideColumns columns; + ASSERT_OK(txn->GetEntityForUpdate( + snapshot_read_options, txn_db->DefaultColumnFamily(), foo, &columns)); + ASSERT_EQ(columns.columns(), foo_columns); + } + + // This PutEntity outside of a transaction will conflict with the previous + // write + const WideColumns foo_conflict_columns{{kDefaultWideColumnName, "X"}, + {"conflicting", "write"}}; + ASSERT_OK(txn_db->PutEntity(WriteOptions(), txn_db->DefaultColumnFamily(), + foo, foo_conflict_columns)); + + { + PinnableWideColumns columns; + ASSERT_OK(txn_db->GetEntity(ReadOptions(), txn_db->DefaultColumnFamily(), + foo, &columns)); + ASSERT_EQ(columns.columns(), foo_conflict_columns); + } + + { + PinnableWideColumns columns; + ASSERT_OK(txn->GetEntity(ReadOptions(), txn_db->DefaultColumnFamily(), foo, + &columns)); + ASSERT_EQ(columns.columns(), foo_conflict_columns); + } + + ASSERT_TRUE(txn->Commit().IsBusy()); // Txn should not commit + + { + PinnableWideColumns columns; + ASSERT_OK(txn_db->GetEntity(ReadOptions(), txn_db->DefaultColumnFamily(), + foo, &columns)); + ASSERT_EQ(columns.columns(), foo_conflict_columns); + } +} + +TEST_P(OptimisticTransactionTest, EntityReadSanityChecks) { + constexpr char foo[] = "foo"; + constexpr char bar[] = "bar"; + constexpr size_t num_keys = 2; + + std::unique_ptr txn(txn_db->BeginTransaction(WriteOptions())); + ASSERT_NE(txn, nullptr); + + { + constexpr ColumnFamilyHandle* column_family = nullptr; + PinnableWideColumns columns; + ASSERT_TRUE(txn->GetEntity(ReadOptions(), column_family, foo, &columns) + .IsInvalidArgument()); + } + + { + constexpr PinnableWideColumns* columns = nullptr; + ASSERT_TRUE(txn->GetEntity(ReadOptions(), txn_db->DefaultColumnFamily(), + foo, columns) + .IsInvalidArgument()); + } + + { + ReadOptions read_options; + read_options.io_activity = Env::IOActivity::kGet; + + PinnableWideColumns columns; + ASSERT_TRUE(txn->GetEntity(read_options, txn_db->DefaultColumnFamily(), foo, + &columns) + .IsInvalidArgument()); + } + + { + constexpr ColumnFamilyHandle* column_family = nullptr; + std::array keys{{foo, bar}}; + std::array results; + std::array statuses; + constexpr bool sorted_input = false; + + txn->MultiGetEntity(ReadOptions(), column_family, num_keys, keys.data(), + results.data(), statuses.data(), sorted_input); + + ASSERT_TRUE(statuses[0].IsInvalidArgument()); + ASSERT_TRUE(statuses[1].IsInvalidArgument()); + } + + { + constexpr Slice* keys = nullptr; + std::array results; + std::array statuses; + constexpr bool sorted_input = false; + + txn->MultiGetEntity(ReadOptions(), txn_db->DefaultColumnFamily(), num_keys, + keys, results.data(), statuses.data(), sorted_input); + + ASSERT_TRUE(statuses[0].IsInvalidArgument()); + ASSERT_TRUE(statuses[1].IsInvalidArgument()); + } + + { + std::array keys{{foo, bar}}; + constexpr PinnableWideColumns* results = nullptr; + std::array statuses; + constexpr bool sorted_input = false; + + txn->MultiGetEntity(ReadOptions(), txn_db->DefaultColumnFamily(), num_keys, + keys.data(), results, statuses.data(), sorted_input); + + ASSERT_TRUE(statuses[0].IsInvalidArgument()); + ASSERT_TRUE(statuses[1].IsInvalidArgument()); + } + + { + ReadOptions read_options; + read_options.io_activity = Env::IOActivity::kMultiGet; + + std::array keys{{foo, bar}}; + std::array results; + std::array statuses; + constexpr bool sorted_input = false; + + txn->MultiGetEntity(read_options, txn_db->DefaultColumnFamily(), num_keys, + keys.data(), results.data(), statuses.data(), + sorted_input); + ASSERT_TRUE(statuses[0].IsInvalidArgument()); + ASSERT_TRUE(statuses[1].IsInvalidArgument()); + } + + { + constexpr ColumnFamilyHandle* column_family = nullptr; + PinnableWideColumns columns; + ASSERT_TRUE( + txn->GetEntityForUpdate(ReadOptions(), column_family, foo, &columns) + .IsInvalidArgument()); + } + + { + constexpr PinnableWideColumns* columns = nullptr; + ASSERT_TRUE(txn->GetEntityForUpdate(ReadOptions(), + txn_db->DefaultColumnFamily(), foo, + columns) + .IsInvalidArgument()); + } + + { + ReadOptions read_options; + read_options.io_activity = Env::IOActivity::kGet; + + PinnableWideColumns columns; + ASSERT_TRUE(txn->GetEntityForUpdate(read_options, + txn_db->DefaultColumnFamily(), foo, + &columns) + .IsInvalidArgument()); + } + + { + txn->SetSnapshot(); + + ReadOptions read_options; + read_options.snapshot = txn->GetSnapshot(); + + PinnableWideColumns columns; + constexpr bool exclusive = true; + constexpr bool do_validate = false; + + ASSERT_TRUE(txn->GetEntityForUpdate(read_options, + txn_db->DefaultColumnFamily(), foo, + &columns, exclusive, do_validate) + .IsInvalidArgument()); + } +} + INSTANTIATE_TEST_CASE_P( InstanceOccGroup, OptimisticTransactionTest, testing::Values(OccValidationPolicy::kValidateSerial, diff --git a/utilities/transactions/pessimistic_transaction.cc b/utilities/transactions/pessimistic_transaction.cc index 4e798de2a45..bc988a603b5 100644 --- a/utilities/transactions/pessimistic_transaction.cc +++ b/utilities/transactions/pessimistic_transaction.cc @@ -3,7 +3,6 @@ // COPYING file in the root directory) and Apache 2.0 License // (found in the LICENSE.Apache file in the root directory). - #include "utilities/transactions/pessimistic_transaction.h" #include @@ -218,6 +217,75 @@ inline Status WriteCommittedTxn::GetForUpdateImpl( value, exclusive, do_validate); } +Status WriteCommittedTxn::GetEntityForUpdate(const ReadOptions& read_options, + ColumnFamilyHandle* column_family, + const Slice& key, + PinnableWideColumns* columns, + bool exclusive, bool do_validate) { + if (!column_family) { + return Status::InvalidArgument( + "Cannot call GetEntityForUpdate without a column family handle"); + } + + const Comparator* const ucmp = column_family->GetComparator(); + assert(ucmp); + const size_t ts_sz = ucmp->timestamp_size(); + + if (ts_sz == 0) { + return TransactionBaseImpl::GetEntityForUpdate( + read_options, column_family, key, columns, exclusive, do_validate); + } + + assert(ts_sz > 0); + + if (!do_validate) { + if (read_timestamp_ != kMaxTxnTimestamp) { + return Status::InvalidArgument( + "Read timestamp must not be set if validation is disabled"); + } + } else { + if (read_timestamp_ == kMaxTxnTimestamp) { + return Status::InvalidArgument( + "Read timestamp must be set for validation"); + } + } + + std::string ts_buf; + PutFixed64(&ts_buf, read_timestamp_); + Slice ts(ts_buf); + + if (!read_options.timestamp) { + ReadOptions read_options_copy = read_options; + read_options_copy.timestamp = &ts; + + return TransactionBaseImpl::GetEntityForUpdate( + read_options_copy, column_family, key, columns, exclusive, do_validate); + } + + assert(read_options.timestamp); + if (*read_options.timestamp != ts) { + return Status::InvalidArgument("Must read from the same read timestamp"); + } + + return TransactionBaseImpl::GetEntityForUpdate( + read_options, column_family, key, columns, exclusive, do_validate); +} + +Status WriteCommittedTxn::PutEntityImpl(ColumnFamilyHandle* column_family, + const Slice& key, + const WideColumns& columns, + bool do_validate, bool assume_tracked) { + return Operate(column_family, key, do_validate, assume_tracked, + [column_family, &key, &columns, this]() { + Status s = GetBatchForWrite()->PutEntity(column_family, key, + columns); + if (s.ok()) { + ++num_put_entities_; + } + return s; + }); +} + Status WriteCommittedTxn::Put(ColumnFamilyHandle* column_family, const Slice& key, const Slice& value, const bool assume_tracked) { @@ -570,9 +638,9 @@ Status WriteCommittedTxn::PrepareInternal() { SequenceNumber* const KIgnoreSeqUsed = nullptr; const size_t kNoBatchCount = 0; s = db_impl_->WriteImpl(write_options, GetWriteBatch()->GetWriteBatch(), - kNoWriteCallback, &log_number_, kRefNoLog, - kDisableMemtable, KIgnoreSeqUsed, kNoBatchCount, - &mark_log_callback); + kNoWriteCallback, /*user_write_cb=*/nullptr, + &log_number_, kRefNoLog, kDisableMemtable, + KIgnoreSeqUsed, kNoBatchCount, &mark_log_callback); return s; } @@ -705,11 +773,11 @@ Status WriteCommittedTxn::CommitWithoutPrepareInternal() { post_mem_cb = &snapshot_creation_cb; } } - auto s = db_impl_->WriteImpl(write_options_, wb, - /*callback*/ nullptr, /*log_used*/ nullptr, - /*log_ref*/ 0, /*disable_memtable*/ false, - &seq_used, /*batch_cnt=*/0, - /*pre_release_callback=*/nullptr, post_mem_cb); + auto s = db_impl_->WriteImpl( + write_options_, wb, + /*callback*/ nullptr, /*user_write_cb=*/nullptr, /*log_used*/ nullptr, + /*log_ref*/ 0, /*disable_memtable*/ false, &seq_used, /*batch_cnt=*/0, + /*pre_release_callback=*/nullptr, post_mem_cb); assert(!s.ok() || seq_used != kMaxSequenceNumber); if (s.ok()) { SetId(seq_used); @@ -720,6 +788,7 @@ Status WriteCommittedTxn::CommitWithoutPrepareInternal() { Status WriteCommittedTxn::CommitBatchInternal(WriteBatch* batch, size_t) { uint64_t seq_used = kMaxSequenceNumber; auto s = db_impl_->WriteImpl(write_options_, batch, /*callback*/ nullptr, + /*user_write_cb=*/nullptr, /*log_used*/ nullptr, /*log_ref*/ 0, /*disable_memtable*/ false, &seq_used); assert(!s.ok() || seq_used != kMaxSequenceNumber); @@ -793,6 +862,7 @@ Status WriteCommittedTxn::CommitInternal() { } } s = db_impl_->WriteImpl(write_options_, working_batch, /*callback*/ nullptr, + /*user_write_cb=*/nullptr, /*log_used*/ nullptr, /*log_ref*/ log_number_, /*disable_memtable*/ false, &seq_used, /*batch_cnt=*/0, /*pre_release_callback=*/nullptr, @@ -896,6 +966,11 @@ Status PessimisticTransaction::LockBatch(WriteBatch* batch, RecordKey(column_family_id, key); return Status::OK(); } + Status PutEntityCF(uint32_t column_family_id, const Slice& key, + const Slice& /* unused */) override { + RecordKey(column_family_id, key); + return Status::OK(); + } Status MergeCF(uint32_t column_family_id, const Slice& key, const Slice& /* unused */) override { RecordKey(column_family_id, key); diff --git a/utilities/transactions/pessimistic_transaction.h b/utilities/transactions/pessimistic_transaction.h index 5f8942f4a42..9cdea759866 100644 --- a/utilities/transactions/pessimistic_transaction.h +++ b/utilities/transactions/pessimistic_transaction.h @@ -5,7 +5,6 @@ #pragma once - #include #include #include @@ -235,6 +234,11 @@ class WriteCommittedTxn : public PessimisticTransaction { PinnableSlice* pinnable_val, bool exclusive, const bool do_validate) override; + Status GetEntityForUpdate(const ReadOptions& read_options, + ColumnFamilyHandle* column_family, const Slice& key, + PinnableWideColumns* columns, bool exclusive, + bool do_validate) override; + using TransactionBaseImpl::Put; // `key` does NOT include timestamp even when it's enabled. Status Put(ColumnFamilyHandle* column_family, const Slice& key, @@ -249,6 +253,25 @@ class WriteCommittedTxn : public PessimisticTransaction { Status PutUntracked(ColumnFamilyHandle* column_family, const SliceParts& key, const SliceParts& value) override; + // `key` does NOT include timestamp even when it's enabled. + Status PutEntity(ColumnFamilyHandle* column_family, const Slice& key, + const WideColumns& columns, + bool assume_tracked = false) override { + const bool do_validate = !assume_tracked; + + return PutEntityImpl(column_family, key, columns, do_validate, + assume_tracked); + } + + Status PutEntityUntracked(ColumnFamilyHandle* column_family, const Slice& key, + const WideColumns& columns) override { + constexpr bool do_validate = false; + constexpr bool assume_tracked = false; + + return PutEntityImpl(column_family, key, columns, do_validate, + assume_tracked); + } + using TransactionBaseImpl::Delete; // `key` does NOT include timestamp even when it's enabled. Status Delete(ColumnFamilyHandle* column_family, const Slice& key, @@ -288,6 +311,10 @@ class WriteCommittedTxn : public PessimisticTransaction { TValue* value, bool exclusive, const bool do_validate); + Status PutEntityImpl(ColumnFamilyHandle* column_family, const Slice& key, + const WideColumns& columns, bool do_validate, + bool assume_tracked); + template Status Operate(ColumnFamilyHandle* column_family, const TKey& key, const bool do_validate, const bool assume_tracked, diff --git a/utilities/transactions/pessimistic_transaction_db.cc b/utilities/transactions/pessimistic_transaction_db.cc index 75e69867b82..7aca2bb523c 100644 --- a/utilities/transactions/pessimistic_transaction_db.cc +++ b/utilities/transactions/pessimistic_transaction_db.cc @@ -3,10 +3,10 @@ // COPYING file in the root directory) and Apache 2.0 License // (found in the LICENSE.Apache file in the root directory). - #include "utilities/transactions/pessimistic_transaction_db.h" #include +#include #include #include #include @@ -512,15 +512,15 @@ Transaction* PessimisticTransactionDB::BeginInternalTransaction( return txn; } -// All user Put, Merge, Delete, and Write requests must be intercepted to make -// sure that they lock all keys that they are writing to avoid causing conflicts -// with any concurrent transactions. The easiest way to do this is to wrap all -// write operations in a transaction. +// All user Put, PutEntity, Merge, Delete, and Write requests must be +// intercepted to make sure that they lock all keys that they are writing to +// avoid causing conflicts with any concurrent transactions. The easiest way to +// do this is to wrap all write operations in a transaction. // -// Put(), Merge(), and Delete() only lock a single key per call. Write() will -// sort its keys before locking them. This guarantees that TransactionDB write -// methods cannot deadlock with each other (but still could deadlock with a -// Transaction). +// Put(), PutEntity(), Merge(), and Delete() only lock a single key per call. +// Write() will sort its keys before locking them. This guarantees that +// TransactionDB write methods cannot deadlock with each other (but still could +// deadlock with a Transaction). Status PessimisticTransactionDB::Put(const WriteOptions& options, ColumnFamilyHandle* column_family, const Slice& key, const Slice& val) { @@ -545,6 +545,42 @@ Status PessimisticTransactionDB::Put(const WriteOptions& options, return s; } +Status PessimisticTransactionDB::PutEntity(const WriteOptions& options, + ColumnFamilyHandle* column_family, + const Slice& key, + const WideColumns& columns) { + { + const Status s = FailIfCfEnablesTs(this, column_family); + if (!s.ok()) { + return s; + } + } + + { + std::unique_ptr txn(BeginInternalTransaction(options)); + txn->DisableIndexing(); + + // Since the client didn't create a transaction, they don't care about + // conflict checking for this write. So we just need to do + // PutEntityUntracked(). + { + const Status s = txn->PutEntityUntracked(column_family, key, columns); + if (!s.ok()) { + return s; + } + } + + { + const Status s = txn->Commit(); + if (!s.ok()) { + return s; + } + } + } + + return Status::OK(); +} + Status PessimisticTransactionDB::Delete(const WriteOptions& wopts, ColumnFamilyHandle* column_family, const Slice& key) { diff --git a/utilities/transactions/pessimistic_transaction_db.h b/utilities/transactions/pessimistic_transaction_db.h index 6654aa80963..77670e7d4aa 100644 --- a/utilities/transactions/pessimistic_transaction_db.h +++ b/utilities/transactions/pessimistic_transaction_db.h @@ -50,6 +50,20 @@ class PessimisticTransactionDB : public TransactionDB { Status Put(const WriteOptions& options, ColumnFamilyHandle* column_family, const Slice& key, const Slice& val) override; + Status PutEntity(const WriteOptions& options, + ColumnFamilyHandle* column_family, const Slice& key, + const WideColumns& columns) override; + Status PutEntity(const WriteOptions& /* options */, const Slice& /* key */, + const AttributeGroups& attribute_groups) override { + if (attribute_groups.empty()) { + return Status::InvalidArgument( + "Cannot call this method without attribute groups"); + } + return Status::NotSupported( + "PutEntity with AttributeGroups not supported by " + "PessimisticTransactionDB"); + } + using StackableDB::Delete; Status Delete(const WriteOptions& wopts, ColumnFamilyHandle* column_family, const Slice& key) override; diff --git a/utilities/transactions/transaction_base.cc b/utilities/transactions/transaction_base.cc index b232736cfb7..9d129c23e9b 100644 --- a/utilities/transactions/transaction_base.cc +++ b/utilities/transactions/transaction_base.cc @@ -3,7 +3,6 @@ // COPYING file in the root directory) and Apache 2.0 License // (found in the LICENSE.Apache file in the root directory). - #include "utilities/transactions/transaction_base.h" #include @@ -90,6 +89,7 @@ void TransactionBaseImpl::Clear() { commit_time_batch_.Clear(); tracked_locks_->Clear(); num_puts_ = 0; + num_put_entities_ = 0; num_deletes_ = 0; num_merges_ = 0; @@ -177,7 +177,7 @@ void TransactionBaseImpl::SetSavePoint() { autovector>()); } save_points_->emplace(snapshot_, snapshot_needed_, snapshot_notifier_, - num_puts_, num_deletes_, num_merges_, + num_puts_, num_put_entities_, num_deletes_, num_merges_, lock_tracker_factory_); write_batch_.SetSavePoint(); } @@ -190,6 +190,7 @@ Status TransactionBaseImpl::RollbackToSavePoint() { snapshot_needed_ = save_point.snapshot_needed_; snapshot_notifier_ = save_point.snapshot_notifier_; num_puts_ = save_point.num_puts_; + num_put_entities_ = save_point.num_put_entities_; num_deletes_ = save_point.num_deletes_; num_merges_ = save_point.num_merges_; @@ -288,6 +289,13 @@ Status TransactionBaseImpl::GetImpl(const ReadOptions& read_options, pinnable_val); } +Status TransactionBaseImpl::GetEntity(const ReadOptions& read_options, + ColumnFamilyHandle* column_family, + const Slice& key, + PinnableWideColumns* columns) { + return GetEntityImpl(read_options, column_family, key, columns); +} + Status TransactionBaseImpl::GetForUpdate(const ReadOptions& read_options, ColumnFamilyHandle* column_family, const Slice& key, std::string* value, @@ -343,6 +351,24 @@ Status TransactionBaseImpl::GetForUpdate(const ReadOptions& read_options, return s; } +Status TransactionBaseImpl::GetEntityForUpdate( + const ReadOptions& read_options, ColumnFamilyHandle* column_family, + const Slice& key, PinnableWideColumns* columns, bool exclusive, + bool do_validate) { + if (!do_validate && read_options.snapshot != nullptr) { + return Status::InvalidArgument( + "Snapshot must not be set if validation is disabled"); + } + + const Status s = + TryLock(column_family, key, true /* read_only */, exclusive, do_validate); + if (!s.ok()) { + return s; + } + + return GetEntityImpl(read_options, column_family, key, columns); +} + std::vector TransactionBaseImpl::MultiGet( const ReadOptions& _read_options, const std::vector& column_family, @@ -400,6 +426,15 @@ void TransactionBaseImpl::MultiGet(const ReadOptions& _read_options, sorted_input); } +void TransactionBaseImpl::MultiGetEntity(const ReadOptions& read_options, + ColumnFamilyHandle* column_family, + size_t num_keys, const Slice* keys, + PinnableWideColumns* results, + Status* statuses, bool sorted_input) { + MultiGetEntityImpl(read_options, column_family, num_keys, keys, results, + statuses, sorted_input); +} + std::vector TransactionBaseImpl::MultiGetForUpdate( const ReadOptions& read_options, const std::vector& column_family, @@ -451,6 +486,32 @@ Iterator* TransactionBaseImpl::GetIterator(const ReadOptions& read_options, &read_options); } +Status TransactionBaseImpl::PutEntityImpl(ColumnFamilyHandle* column_family, + const Slice& key, + const WideColumns& columns, + bool do_validate, + bool assume_tracked) { + { + constexpr bool read_only = false; + constexpr bool exclusive = true; + const Status s = TryLock(column_family, key, read_only, exclusive, + do_validate, assume_tracked); + if (!s.ok()) { + return s; + } + } + + { + const Status s = GetBatchForWrite()->PutEntity(column_family, key, columns); + if (!s.ok()) { + return s; + } + } + + ++num_put_entities_; + return Status::OK(); +} + Status TransactionBaseImpl::Put(ColumnFamilyHandle* column_family, const Slice& key, const Slice& value, const bool assume_tracked) { @@ -678,6 +739,10 @@ uint64_t TransactionBaseImpl::GetElapsedTime() const { uint64_t TransactionBaseImpl::GetNumPuts() const { return num_puts_; } +uint64_t TransactionBaseImpl::GetNumPutEntities() const { + return num_put_entities_; +} + uint64_t TransactionBaseImpl::GetNumDeletes() const { return num_deletes_; } uint64_t TransactionBaseImpl::GetNumMerges() const { return num_merges_; } @@ -705,7 +770,8 @@ void TransactionBaseImpl::TrackKey(uint32_t cfh_id, const std::string& key, } } -// Gets the write batch that should be used for Put/Merge/Deletes. +// Gets the write batch that should be used for Put/PutEntity/Merge/Delete +// operations. // // Returns either a WriteBatch or WriteBatchWithIndex depending on whether // DisableIndexing() has been called. @@ -769,6 +835,19 @@ Status TransactionBaseImpl::RebuildFromWriteBatch(WriteBatch* src_batch) { return txn_->Put(db_->GetColumnFamilyHandle(cf), key, val); } + Status PutEntityCF(uint32_t cf, const Slice& key, + const Slice& entity) override { + Slice entity_copy = entity; + WideColumns columns; + const Status s = + WideColumnSerialization::Deserialize(entity_copy, columns); + if (!s.ok()) { + return s; + } + + return txn_->PutEntity(db_->GetColumnFamilyHandle(cf), key, columns); + } + Status DeleteCF(uint32_t cf, const Slice& key) override { return txn_->Delete(db_->GetColumnFamilyHandle(cf), key); } diff --git a/utilities/transactions/transaction_base.h b/utilities/transactions/transaction_base.h index 7dcf412cb88..631562eb353 100644 --- a/utilities/transactions/transaction_base.h +++ b/utilities/transactions/transaction_base.h @@ -5,7 +5,6 @@ #pragma once - #include #include #include @@ -37,10 +36,11 @@ class TransactionBaseImpl : public Transaction { void Reinitialize(DB* db, const WriteOptions& write_options); - // Called before executing Put, Merge, Delete, and GetForUpdate. If TryLock - // returns non-OK, the Put/Merge/Delete/GetForUpdate will be failed. - // do_validate will be false if called from PutUntracked, DeleteUntracked, - // MergeUntracked, or GetForUpdate(do_validate=false) + // Called before executing Put, PutEntity, Merge, Delete, and GetForUpdate. If + // TryLock returns non-OK, the Put/PutEntity/Merge/Delete/GetForUpdate will be + // failed. do_validate will be false if called from PutUntracked, + // PutEntityUntracked, DeleteUntracked, MergeUntracked, or + // GetForUpdate(do_validate=false) virtual Status TryLock(ColumnFamilyHandle* column_family, const Slice& key, bool read_only, bool exclusive, const bool do_validate = true, @@ -66,6 +66,10 @@ class TransactionBaseImpl : public Transaction { return Get(options, db_->DefaultColumnFamily(), key, value); } + Status GetEntity(const ReadOptions& options, + ColumnFamilyHandle* column_family, const Slice& key, + PinnableWideColumns* columns) override; + using Transaction::GetForUpdate; Status GetForUpdate(const ReadOptions& options, ColumnFamilyHandle* column_family, const Slice& key, @@ -91,6 +95,11 @@ class TransactionBaseImpl : public Transaction { exclusive, do_validate); } + Status GetEntityForUpdate(const ReadOptions& read_options, + ColumnFamilyHandle* column_family, const Slice& key, + PinnableWideColumns* columns, bool exclusive = true, + bool do_validate = true) override; + using Transaction::MultiGet; std::vector MultiGet( const ReadOptions& _read_options, @@ -112,6 +121,11 @@ class TransactionBaseImpl : public Transaction { const Slice* keys, PinnableSlice* values, Status* statuses, const bool sorted_input = false) override; + void MultiGetEntity(const ReadOptions& options, + ColumnFamilyHandle* column_family, size_t num_keys, + const Slice* keys, PinnableWideColumns* results, + Status* statuses, bool sorted_input = false) override; + using Transaction::MultiGetForUpdate; std::vector MultiGetForUpdate( const ReadOptions& options, @@ -145,6 +159,15 @@ class TransactionBaseImpl : public Transaction { return Put(nullptr, key, value); } + Status PutEntity(ColumnFamilyHandle* column_family, const Slice& key, + const WideColumns& columns, + bool assume_tracked = false) override { + const bool do_validate = !assume_tracked; + + return PutEntityImpl(column_family, key, columns, do_validate, + assume_tracked); + } + Status Merge(ColumnFamilyHandle* column_family, const Slice& key, const Slice& value, const bool assume_tracked = false) override; Status Merge(const Slice& key, const Slice& value) override { @@ -181,6 +204,15 @@ class TransactionBaseImpl : public Transaction { return PutUntracked(nullptr, key, value); } + Status PutEntityUntracked(ColumnFamilyHandle* column_family, const Slice& key, + const WideColumns& columns) override { + constexpr bool do_validate = false; + constexpr bool assume_tracked = false; + + return PutEntityImpl(column_family, key, columns, do_validate, + assume_tracked); + } + Status MergeUntracked(ColumnFamilyHandle* column_family, const Slice& key, const Slice& value) override; Status MergeUntracked(const Slice& key, const Slice& value) override { @@ -240,6 +272,8 @@ class TransactionBaseImpl : public Transaction { uint64_t GetNumPuts() const override; + uint64_t GetNumPutEntities() const override; + uint64_t GetNumDeletes() const override; uint64_t GetNumMerges() const override; @@ -276,6 +310,26 @@ class TransactionBaseImpl : public Transaction { Status GetImpl(const ReadOptions& options, ColumnFamilyHandle* column_family, const Slice& key, PinnableSlice* value) override; + Status GetEntityImpl(const ReadOptions& options, + ColumnFamilyHandle* column_family, const Slice& key, + PinnableWideColumns* columns) { + return write_batch_.GetEntityFromBatchAndDB(db_, options, column_family, + key, columns); + } + + void MultiGetEntityImpl(const ReadOptions& options, + ColumnFamilyHandle* column_family, size_t num_keys, + const Slice* keys, PinnableWideColumns* results, + Status* statuses, bool sorted_input) { + write_batch_.MultiGetEntityFromBatchAndDB(db_, options, column_family, + num_keys, keys, results, statuses, + sorted_input); + } + + Status PutEntityImpl(ColumnFamilyHandle* column_family, const Slice& key, + const WideColumns& columns, bool do_validate, + bool assume_tracked); + // Add a key to the list of tracked keys. // // seqno is the earliest seqno this key was involved with this transaction. @@ -320,6 +374,7 @@ class TransactionBaseImpl : public Transaction { // Count of various operations pending in this transaction uint64_t num_puts_ = 0; + uint64_t num_put_entities_ = 0; uint64_t num_deletes_ = 0; uint64_t num_merges_ = 0; @@ -328,6 +383,7 @@ class TransactionBaseImpl : public Transaction { bool snapshot_needed_ = false; std::shared_ptr snapshot_notifier_; uint64_t num_puts_ = 0; + uint64_t num_put_entities_ = 0; uint64_t num_deletes_ = 0; uint64_t num_merges_ = 0; @@ -336,12 +392,14 @@ class TransactionBaseImpl : public Transaction { SavePoint(std::shared_ptr snapshot, bool snapshot_needed, std::shared_ptr snapshot_notifier, - uint64_t num_puts, uint64_t num_deletes, uint64_t num_merges, + uint64_t num_puts, uint64_t num_put_entities, + uint64_t num_deletes, uint64_t num_merges, const LockTrackerFactory& lock_tracker_factory) : snapshot_(snapshot), snapshot_needed_(snapshot_needed), snapshot_notifier_(snapshot_notifier), num_puts_(num_puts), + num_put_entities_(num_put_entities), num_deletes_(num_deletes), num_merges_(num_merges), new_locks_(lock_tracker_factory.Create()) {} @@ -373,10 +431,10 @@ class TransactionBaseImpl : public Transaction { // prepare phase is not skipped. WriteBatch commit_time_batch_; - // If true, future Put/Merge/Deletes will be indexed in the - // WriteBatchWithIndex. - // If false, future Put/Merge/Deletes will be inserted directly into the - // underlying WriteBatch and not indexed in the WriteBatchWithIndex. + // If true, future Put/PutEntity/Merge/Delete operations will be indexed in + // the WriteBatchWithIndex. If false, future Put/PutEntity/Merge/Delete + // operations will be inserted directly into the underlying WriteBatch and not + // indexed in the WriteBatchWithIndex. bool indexing_enabled_; // SetSnapshotOnNextOperation() has been called and the snapshot has not yet diff --git a/utilities/transactions/transaction_test.cc b/utilities/transactions/transaction_test.cc index a18af717f10..94629602240 100644 --- a/utilities/transactions/transaction_test.cc +++ b/utilities/transactions/transaction_test.cc @@ -3,10 +3,10 @@ // COPYING file in the root directory) and Apache 2.0 License // (found in the LICENSE.Apache file in the root directory). - #include "utilities/transactions/transaction_test.h" #include +#include #include #include #include @@ -2210,6 +2210,10 @@ TEST_P(TransactionTest, TwoPhaseLogRollingTest2) { * hidden behind improperly summed sequence ids */ TEST_P(TransactionTest, TwoPhaseOutOfOrderDelete) { + // WAL recycling incompatible with disableWAL (below) + options.recycle_log_file_num = 0; + ASSERT_OK(ReOpenNoDelete()); + DBImpl* db_impl = static_cast_with_check(db->GetRootDB()); WriteOptions wal_on, wal_off; wal_on.sync = true; @@ -3621,6 +3625,9 @@ TEST_P(TransactionTest, UntrackedWrites) { // Untracked writes should succeed even though key was written after snapshot s = txn->PutUntracked("untracked", "1"); ASSERT_OK(s); + s = txn->PutEntityUntracked(db->DefaultColumnFamily(), "untracked", + {{"hello", "world"}}); + ASSERT_OK(s); s = txn->MergeUntracked("untracked", "2"); ASSERT_OK(s); s = txn->DeleteUntracked("untracked"); @@ -3629,6 +3636,8 @@ TEST_P(TransactionTest, UntrackedWrites) { // Conflict s = txn->Put("untracked", "3"); ASSERT_TRUE(s.IsBusy()); + s = txn->PutEntity(db->DefaultColumnFamily(), "untracked", {{"foo", "bar"}}); + ASSERT_TRUE(s.IsBusy()); s = txn->Commit(); ASSERT_OK(s); @@ -6984,6 +6993,477 @@ TEST_P(TransactionTest, UnlockWALStallCleared) { } } +TEST_P(TransactionTest, PutEntitySuccess) { + const TxnDBWritePolicy write_policy = std::get<2>(GetParam()); + if (write_policy != TxnDBWritePolicy::WRITE_COMMITTED) { + ROCKSDB_GTEST_BYPASS("Test only WriteCommitted for now"); + return; + } + + constexpr char foo[] = "foo"; + const WideColumns foo_columns{ + {kDefaultWideColumnName, "bar"}, {"col1", "val1"}, {"col2", "val2"}}; + const WideColumns foo_new_columns{ + {kDefaultWideColumnName, "baz"}, {"colA", "valA"}, {"colB", "valB"}}; + + ASSERT_OK(db->PutEntity(WriteOptions(), db->DefaultColumnFamily(), foo, + foo_columns)); + + { + std::unique_ptr txn( + db->BeginTransaction(WriteOptions(), TransactionOptions())); + + ASSERT_NE(txn, nullptr); + ASSERT_NE(txn->GetID(), 0); + ASSERT_EQ(txn->GetNumPutEntities(), 0); + + { + PinnableWideColumns columns; + ASSERT_OK(txn->GetEntity(ReadOptions(), db->DefaultColumnFamily(), foo, + &columns)); + ASSERT_EQ(columns.columns(), foo_columns); + } + + { + PinnableWideColumns columns; + ASSERT_OK(txn->GetEntityForUpdate( + ReadOptions(), db->DefaultColumnFamily(), foo, &columns)); + ASSERT_EQ(columns.columns(), foo_columns); + } + + ASSERT_OK(txn->PutEntity(db->DefaultColumnFamily(), foo, foo_new_columns)); + + ASSERT_EQ(txn->GetNumPutEntities(), 1); + + { + PinnableWideColumns columns; + ASSERT_OK(txn->GetEntity(ReadOptions(), db->DefaultColumnFamily(), foo, + &columns)); + ASSERT_EQ(columns.columns(), foo_new_columns); + } + + { + PinnableWideColumns columns; + ASSERT_OK(txn->GetEntityForUpdate( + ReadOptions(), db->DefaultColumnFamily(), foo, &columns)); + ASSERT_EQ(columns.columns(), foo_new_columns); + } + + ASSERT_OK(txn->Commit()); + } + + { + PinnableWideColumns columns; + ASSERT_OK( + db->GetEntity(ReadOptions(), db->DefaultColumnFamily(), foo, &columns)); + ASSERT_EQ(columns.columns(), foo_new_columns); + } +} + +TEST_P(TransactionTest, PutEntityWriteConflict) { + const TxnDBWritePolicy write_policy = std::get<2>(GetParam()); + if (write_policy != TxnDBWritePolicy::WRITE_COMMITTED) { + ROCKSDB_GTEST_BYPASS("Test only WriteCommitted for now"); + return; + } + + constexpr char foo[] = "foo"; + const WideColumns foo_columns{ + {kDefaultWideColumnName, "bar"}, {"col1", "val1"}, {"col2", "val2"}}; + constexpr char baz[] = "baz"; + const WideColumns baz_columns{ + {kDefaultWideColumnName, "quux"}, {"colA", "valA"}, {"colB", "valB"}}; + + ASSERT_OK(db->PutEntity(WriteOptions(), db->DefaultColumnFamily(), foo, + foo_columns)); + ASSERT_OK(db->PutEntity(WriteOptions(), db->DefaultColumnFamily(), baz, + baz_columns)); + + std::unique_ptr txn(db->BeginTransaction(WriteOptions())); + ASSERT_NE(txn, nullptr); + + { + PinnableWideColumns columns; + ASSERT_OK(txn->GetEntity(ReadOptions(), db->DefaultColumnFamily(), foo, + &columns)); + ASSERT_EQ(columns.columns(), foo_columns); + } + + { + PinnableWideColumns columns; + ASSERT_OK(txn->GetEntity(ReadOptions(), db->DefaultColumnFamily(), baz, + &columns)); + ASSERT_EQ(columns.columns(), baz_columns); + } + + { + constexpr size_t num_keys = 2; + + std::array keys{{foo, baz}}; + std::array results; + std::array statuses; + + txn->MultiGetEntity(ReadOptions(), db->DefaultColumnFamily(), num_keys, + keys.data(), results.data(), statuses.data()); + + ASSERT_OK(statuses[0]); + ASSERT_OK(statuses[1]); + + ASSERT_EQ(results[0].columns(), foo_columns); + ASSERT_EQ(results[1].columns(), baz_columns); + } + + const WideColumns foo_new_columns{{kDefaultWideColumnName, "FOO"}, + {"hello", "world"}}; + const WideColumns baz_new_columns{{kDefaultWideColumnName, "BAZ"}, + {"ping", "pong"}}; + + ASSERT_OK(txn->PutEntity(db->DefaultColumnFamily(), foo, foo_new_columns)); + ASSERT_OK(txn->PutEntity(db->DefaultColumnFamily(), baz, baz_new_columns)); + + { + PinnableWideColumns columns; + ASSERT_OK(txn->GetEntity(ReadOptions(), db->DefaultColumnFamily(), foo, + &columns)); + ASSERT_EQ(columns.columns(), foo_new_columns); + } + + { + PinnableWideColumns columns; + ASSERT_OK(txn->GetEntity(ReadOptions(), db->DefaultColumnFamily(), baz, + &columns)); + ASSERT_EQ(columns.columns(), baz_new_columns); + } + + { + constexpr size_t num_keys = 2; + + std::array keys{{foo, baz}}; + std::array results; + std::array statuses; + + txn->MultiGetEntity(ReadOptions(), db->DefaultColumnFamily(), num_keys, + keys.data(), results.data(), statuses.data()); + + ASSERT_OK(statuses[0]); + ASSERT_OK(statuses[1]); + + ASSERT_EQ(results[0].columns(), foo_new_columns); + ASSERT_EQ(results[1].columns(), baz_new_columns); + } + + // This PutEntity outside of a transaction will conflict with the previous + // write + const WideColumns foo_conflict_columns{{kDefaultWideColumnName, "X"}, + {"conflicting", "write"}}; + ASSERT_TRUE(db->PutEntity(WriteOptions(), db->DefaultColumnFamily(), foo, + foo_conflict_columns) + .IsTimedOut()); + + { + PinnableWideColumns columns; + ASSERT_OK( + db->GetEntity(ReadOptions(), db->DefaultColumnFamily(), foo, &columns)); + ASSERT_EQ(columns.columns(), foo_columns); + } + + { + PinnableWideColumns columns; + ASSERT_OK( + db->GetEntity(ReadOptions(), db->DefaultColumnFamily(), baz, &columns)); + ASSERT_EQ(columns.columns(), baz_columns); + } + + { + constexpr size_t num_keys = 2; + + std::array keys{{foo, baz}}; + std::array results; + std::array statuses; + constexpr bool sorted_input = false; + + db->MultiGetEntity(ReadOptions(), db->DefaultColumnFamily(), num_keys, + keys.data(), results.data(), statuses.data(), + sorted_input); + + ASSERT_OK(statuses[0]); + ASSERT_OK(statuses[1]); + + ASSERT_EQ(results[0].columns(), foo_columns); + ASSERT_EQ(results[1].columns(), baz_columns); + } + + ASSERT_OK(txn->Commit()); + + { + PinnableWideColumns columns; + ASSERT_OK( + db->GetEntity(ReadOptions(), db->DefaultColumnFamily(), foo, &columns)); + ASSERT_EQ(columns.columns(), foo_new_columns); + } + + { + PinnableWideColumns columns; + ASSERT_OK( + db->GetEntity(ReadOptions(), db->DefaultColumnFamily(), baz, &columns)); + ASSERT_EQ(columns.columns(), baz_new_columns); + } + + { + constexpr size_t num_keys = 2; + + std::array keys{{foo, baz}}; + std::array results; + std::array statuses; + constexpr bool sorted_input = false; + + db->MultiGetEntity(ReadOptions(), db->DefaultColumnFamily(), num_keys, + keys.data(), results.data(), statuses.data(), + sorted_input); + + ASSERT_OK(statuses[0]); + ASSERT_OK(statuses[1]); + + ASSERT_EQ(results[0].columns(), foo_new_columns); + ASSERT_EQ(results[1].columns(), baz_new_columns); + } +} + +TEST_P(TransactionTest, PutEntityReadConflict) { + const TxnDBWritePolicy write_policy = std::get<2>(GetParam()); + if (write_policy != TxnDBWritePolicy::WRITE_COMMITTED) { + ROCKSDB_GTEST_BYPASS("Test only WriteCommitted for now"); + return; + } + + constexpr char foo[] = "foo"; + const WideColumns foo_columns{ + {kDefaultWideColumnName, "bar"}, {"col1", "val1"}, {"col2", "val2"}}; + + ASSERT_OK(db->PutEntity(WriteOptions(), db->DefaultColumnFamily(), foo, + foo_columns)); + + std::unique_ptr txn(db->BeginTransaction(WriteOptions())); + ASSERT_NE(txn, nullptr); + + txn->SetSnapshot(); + + ReadOptions snapshot_read_options; + snapshot_read_options.snapshot = txn->GetSnapshot(); + + { + PinnableWideColumns columns; + ASSERT_OK(txn->GetEntityForUpdate( + snapshot_read_options, db->DefaultColumnFamily(), foo, &columns)); + ASSERT_EQ(columns.columns(), foo_columns); + } + + // This PutEntity outside of a transaction will conflict with the previous + // write + const WideColumns foo_conflict_columns{{kDefaultWideColumnName, "X"}, + {"conflicting", "write"}}; + ASSERT_TRUE(db->PutEntity(WriteOptions(), db->DefaultColumnFamily(), foo, + foo_conflict_columns) + .IsTimedOut()); + + { + PinnableWideColumns columns; + ASSERT_OK( + db->GetEntity(ReadOptions(), db->DefaultColumnFamily(), foo, &columns)); + ASSERT_EQ(columns.columns(), foo_columns); + } + + { + PinnableWideColumns columns; + ASSERT_OK(txn->GetEntity(ReadOptions(), db->DefaultColumnFamily(), foo, + &columns)); + ASSERT_EQ(columns.columns(), foo_columns); + } + + ASSERT_OK(txn->Commit()); + + { + PinnableWideColumns columns; + ASSERT_OK( + db->GetEntity(ReadOptions(), db->DefaultColumnFamily(), foo, &columns)); + ASSERT_EQ(columns.columns(), foo_columns); + } +} + +TEST_P(TransactionTest, EntityReadSanityChecks) { + constexpr char foo[] = "foo"; + constexpr char bar[] = "bar"; + constexpr size_t num_keys = 2; + + std::unique_ptr txn(db->BeginTransaction(WriteOptions())); + ASSERT_NE(txn, nullptr); + + { + constexpr ColumnFamilyHandle* column_family = nullptr; + PinnableWideColumns columns; + ASSERT_TRUE(txn->GetEntity(ReadOptions(), column_family, foo, &columns) + .IsInvalidArgument()); + } + + { + constexpr PinnableWideColumns* columns = nullptr; + ASSERT_TRUE( + txn->GetEntity(ReadOptions(), db->DefaultColumnFamily(), foo, columns) + .IsInvalidArgument()); + } + + { + ReadOptions read_options; + read_options.io_activity = Env::IOActivity::kGet; + + PinnableWideColumns columns; + ASSERT_TRUE( + txn->GetEntity(read_options, db->DefaultColumnFamily(), foo, &columns) + .IsInvalidArgument()); + } + + { + constexpr ColumnFamilyHandle* column_family = nullptr; + std::array keys{{foo, bar}}; + std::array results; + std::array statuses; + constexpr bool sorted_input = false; + + txn->MultiGetEntity(ReadOptions(), column_family, num_keys, keys.data(), + results.data(), statuses.data(), sorted_input); + + ASSERT_TRUE(statuses[0].IsInvalidArgument()); + ASSERT_TRUE(statuses[1].IsInvalidArgument()); + } + + { + constexpr Slice* keys = nullptr; + std::array results; + std::array statuses; + constexpr bool sorted_input = false; + + txn->MultiGetEntity(ReadOptions(), db->DefaultColumnFamily(), num_keys, + keys, results.data(), statuses.data(), sorted_input); + + ASSERT_TRUE(statuses[0].IsInvalidArgument()); + ASSERT_TRUE(statuses[1].IsInvalidArgument()); + } + + { + std::array keys{{foo, bar}}; + constexpr PinnableWideColumns* results = nullptr; + std::array statuses; + constexpr bool sorted_input = false; + + txn->MultiGetEntity(ReadOptions(), db->DefaultColumnFamily(), num_keys, + keys.data(), results, statuses.data(), sorted_input); + + ASSERT_TRUE(statuses[0].IsInvalidArgument()); + ASSERT_TRUE(statuses[1].IsInvalidArgument()); + } + + { + ReadOptions read_options; + read_options.io_activity = Env::IOActivity::kMultiGet; + + std::array keys{{foo, bar}}; + std::array results; + std::array statuses; + constexpr bool sorted_input = false; + + txn->MultiGetEntity(read_options, db->DefaultColumnFamily(), num_keys, + keys.data(), results.data(), statuses.data(), + sorted_input); + ASSERT_TRUE(statuses[0].IsInvalidArgument()); + ASSERT_TRUE(statuses[1].IsInvalidArgument()); + } + + { + constexpr ColumnFamilyHandle* column_family = nullptr; + PinnableWideColumns columns; + ASSERT_TRUE( + txn->GetEntityForUpdate(ReadOptions(), column_family, foo, &columns) + .IsInvalidArgument()); + } + + { + constexpr PinnableWideColumns* columns = nullptr; + ASSERT_TRUE(txn->GetEntityForUpdate(ReadOptions(), + db->DefaultColumnFamily(), foo, columns) + .IsInvalidArgument()); + } + + { + ReadOptions read_options; + read_options.io_activity = Env::IOActivity::kGet; + + PinnableWideColumns columns; + ASSERT_TRUE(txn->GetEntityForUpdate(read_options, db->DefaultColumnFamily(), + foo, &columns) + .IsInvalidArgument()); + } + + { + txn->SetSnapshot(); + + ReadOptions read_options; + read_options.snapshot = txn->GetSnapshot(); + + PinnableWideColumns columns; + constexpr bool exclusive = true; + constexpr bool do_validate = false; + + ASSERT_TRUE(txn->GetEntityForUpdate(read_options, db->DefaultColumnFamily(), + foo, &columns, exclusive, do_validate) + .IsInvalidArgument()); + } +} + +TEST_P(TransactionTest, PutEntityRecovery) { + const TxnDBWritePolicy write_policy = std::get<2>(GetParam()); + if (write_policy != TxnDBWritePolicy::WRITE_COMMITTED) { + ROCKSDB_GTEST_BYPASS("Test only WriteCommitted for now"); + return; + } + + constexpr char foo[] = "foo"; + const WideColumns foo_columns{ + {kDefaultWideColumnName, "bar"}, {"col1", "val1"}, {"col2", "val2"}}; + + constexpr char xid[] = "xid"; + + { + WriteOptions write_options; + write_options.sync = true; + write_options.disableWAL = false; + + std::unique_ptr txn(db->BeginTransaction(write_options)); + ASSERT_NE(txn, nullptr); + + ASSERT_OK(txn->SetName(xid)); + + ASSERT_OK(txn->PutEntity(db->DefaultColumnFamily(), foo, foo_columns)); + + ASSERT_OK(txn->Prepare()); + } + + ASSERT_OK(ReOpenNoDelete()); + + { + std::unique_ptr txn(db->GetTransactionByName(xid)); + ASSERT_NE(txn, nullptr); + + ASSERT_OK(txn->Commit()); + } + + { + PinnableWideColumns columns; + ASSERT_OK( + db->GetEntity(ReadOptions(), db->DefaultColumnFamily(), foo, &columns)); + ASSERT_EQ(columns.columns(), foo_columns); + } +} + TEST_F(TransactionDBTest, CollapseKey) { ASSERT_OK(ReOpen()); ASSERT_OK(db->Put({}, "hello", "world")); @@ -7033,6 +7513,51 @@ TEST_F(TransactionDBTest, CollapseKey) { } } +TEST_F(TransactionDBTest, FlushedLogWithPendingPrepareIsSynced) { + // Repro for a bug where we missed a necessary sync of the old WAL during + // memtable flush. It happened due to applying an optimization to skip syncing + // the old WAL in too many scenarios (all memtable flushes on single CF + // databases). That optimization is only valid when memtable flush can + // guarantee the old WAL will not be read by crash-recovery. When the old WAL + // contains a prepare record without its corresponding commit, that WAL will + // be read by crash-recovery and therefore it must be synced. + const int kStartIndex = 1000; + int next_index = kStartIndex; + ASSERT_OK(ReOpen()); + + ASSERT_OK( + db->Put(WriteOptions(), "key" + std::to_string(next_index), "value")); + next_index++; + + Transaction* txn = db->BeginTransaction(WriteOptions(), TransactionOptions()); + ASSERT_OK(txn->SetName("xid" + std::to_string(next_index))); + ASSERT_OK( + txn->Put(Slice("key" + std::to_string(next_index)), Slice("value"))); + next_index++; + ASSERT_OK(txn->Prepare()); + + // Set it directly writable so new WAL containing the commit record will be + // recovered despite not being explicitly synced. + fault_fs->SetFilesystemDirectWritable(true); + ASSERT_OK(db->Flush(FlushOptions())); + + ASSERT_OK(txn->Commit()); + delete txn; + + ASSERT_OK( + db->Put(WriteOptions(), "key" + std::to_string(next_index), "value")); + next_index++; + + ASSERT_OK(ReOpenNoDelete()); + + for (int i = kStartIndex; i < next_index; i++) { + PinnableSlice value; + ASSERT_OK(db->Get(ReadOptions(), db->DefaultColumnFamily(), + "key" + std::to_string(i), &value)); + ASSERT_EQ("value", value); + } +} + } // namespace ROCKSDB_NAMESPACE int main(int argc, char** argv) { diff --git a/utilities/transactions/transaction_test.h b/utilities/transactions/transaction_test.h index a8fa874a782..a779bf8767c 100644 --- a/utilities/transactions/transaction_test.h +++ b/utilities/transactions/transaction_test.h @@ -63,6 +63,8 @@ class TransactionTestBase : public ::testing::Test { options.unordered_write = write_ordering == kUnorderedWrite; options.level0_file_num_compaction_trigger = 2; options.merge_operator = MergeOperators::CreateFromStringId("stringappend"); + // Recycling log file is generally more challenging for correctness + options.recycle_log_file_num = 2; special_env.skip_fsync_ = true; fault_fs.reset(new FaultInjectionTestFS(FileSystem::Default())); env.reset(new CompositeEnvWrapper(&special_env, fault_fs)); @@ -461,7 +463,7 @@ class TransactionTestBase : public ::testing::Test { } db_impl = static_cast_with_check(db->GetRootDB()); // Check that WAL is empty - VectorLogPtr log_files; + VectorWalPtr log_files; ASSERT_OK(db_impl->GetSortedWalFiles(log_files)); ASSERT_EQ(0, log_files.size()); diff --git a/utilities/transactions/write_committed_transaction_ts_test.cc b/utilities/transactions/write_committed_transaction_ts_test.cc index 0afd57bcc77..86c09ac9089 100644 --- a/utilities/transactions/write_committed_transaction_ts_test.cc +++ b/utilities/transactions/write_committed_transaction_ts_test.cc @@ -717,6 +717,147 @@ TEST_P(WriteCommittedTxnWithTsTest, CheckKeysForConflicts) { SyncPoint::GetInstance()->ClearAllCallBacks(); } +TEST_P(WriteCommittedTxnWithTsTest, GetEntityForUpdate) { + ASSERT_OK(ReOpenNoDelete()); + + ColumnFamilyOptions cf_options; + cf_options.comparator = test::BytewiseComparatorWithU64TsWrapper(); + + const std::string test_cf_name = "test_cf"; + + ColumnFamilyHandle* cfh = nullptr; + ASSERT_OK(db->CreateColumnFamily(cf_options, test_cf_name, &cfh)); + std::unique_ptr cfh_guard(cfh); + + constexpr char foo[] = "foo"; + constexpr char bar[] = "bar"; + constexpr char baz[] = "baz"; + constexpr char quux[] = "quux"; + + { + std::unique_ptr txn0( + NewTxn(WriteOptions(), TransactionOptions())); + + { + std::unique_ptr txn1( + NewTxn(WriteOptions(), TransactionOptions())); + ASSERT_OK(txn1->Put(cfh, foo, bar)); + ASSERT_OK(txn1->Put(cfh, baz, quux)); + ASSERT_OK(txn1->SetCommitTimestamp(24)); + ASSERT_OK(txn1->Commit()); + } + + ASSERT_OK(txn0->SetReadTimestampForValidation(23)); + + // Validation fails: timestamp from db(24) > validation timestamp(23) + PinnableWideColumns columns; + ASSERT_TRUE( + txn0->GetEntityForUpdate(ReadOptions(), cfh, foo, &columns).IsBusy()); + + ASSERT_OK(txn0->Rollback()); + } + + { + std::unique_ptr txn2( + NewTxn(WriteOptions(), TransactionOptions())); + + ASSERT_OK(txn2->SetReadTimestampForValidation(25)); + + // Validation successful: timestamp from db(24) < validation timestamp (25) + { + PinnableWideColumns columns; + ASSERT_OK(txn2->GetEntityForUpdate(ReadOptions(), cfh, foo, &columns)); + } + + // Using a different read timestamp in ReadOptions while doing validation is + // not allowed + { + ReadOptions read_options; + std::string read_timestamp; + Slice diff_read_ts = EncodeU64Ts(24, &read_timestamp); + read_options.timestamp = &diff_read_ts; + + PinnableWideColumns columns; + ASSERT_TRUE(txn2->GetEntityForUpdate(read_options, cfh, foo, &columns) + .IsInvalidArgument()); + + ASSERT_OK(txn2->SetCommitTimestamp(26)); + ASSERT_OK(txn2->Commit()); + } + } + + // GetEntityForUpdate with validation timestamp set but no validation is not + // allowed + { + std::unique_ptr txn3( + NewTxn(WriteOptions(), TransactionOptions())); + + ASSERT_OK(txn3->SetReadTimestampForValidation(27)); + + PinnableWideColumns columns; + ASSERT_TRUE(txn3->GetEntityForUpdate(ReadOptions(), cfh, foo, &columns, + /*exclusive=*/true, + /*do_validate=*/false) + .IsInvalidArgument()); + + ASSERT_OK(txn3->Rollback()); + } + + // GetEntityForUpdate with validation but no validation timestamp is not + // allowed + { + std::unique_ptr txn4( + NewTxn(WriteOptions(), TransactionOptions())); + + // ReadOptions.timestamp is not set + { + PinnableWideColumns columns; + ASSERT_TRUE(txn4->GetEntityForUpdate(ReadOptions(), cfh, foo, &columns) + .IsInvalidArgument()); + } + + // ReadOptions.timestamp is set + { + ReadOptions read_options; + std::string read_timestamp; + Slice read_ts = EncodeU64Ts(27, &read_timestamp); + read_options.timestamp = &read_ts; + + PinnableWideColumns columns; + ASSERT_TRUE(txn4->GetEntityForUpdate(read_options, cfh, foo, &columns) + .IsInvalidArgument()); + } + + ASSERT_OK(txn4->Rollback()); + } + + // Validation disabled + { + std::unique_ptr txn5( + NewTxn(WriteOptions(), TransactionOptions())); + + // ReadOptions.timestamp is not set => success + { + PinnableWideColumns columns; + ASSERT_OK(txn5->GetEntityForUpdate(ReadOptions(), cfh, foo, &columns, + /*exclusive=*/true, + /*do_validate=*/false)); + } + + // ReadOptions.timestamp explicitly set to max timestamp => success + { + ReadOptions read_options; + Slice max_ts = MaxU64Ts(); + read_options.timestamp = &max_ts; + + PinnableWideColumns columns; + ASSERT_OK(txn5->GetEntityForUpdate(read_options, cfh, baz, &columns, + /*exclusive=*/true, + /*do_validate=*/false)); + } + } +} + } // namespace ROCKSDB_NAMESPACE int main(int argc, char** argv) { @@ -724,4 +865,3 @@ int main(int argc, char** argv) { ::testing::InitGoogleTest(&argc, argv); return RUN_ALL_TESTS(); } - diff --git a/utilities/transactions/write_prepared_txn.cc b/utilities/transactions/write_prepared_txn.cc index ddaf077ac3f..9ef2d3f2b17 100644 --- a/utilities/transactions/write_prepared_txn.cc +++ b/utilities/transactions/write_prepared_txn.cc @@ -154,8 +154,9 @@ Status WritePreparedTxn::PrepareInternal() { const bool DISABLE_MEMTABLE = true; uint64_t seq_used = kMaxSequenceNumber; s = db_impl_->WriteImpl(write_options, GetWriteBatch()->GetWriteBatch(), - /*callback*/ nullptr, &log_number_, /*log ref*/ 0, - !DISABLE_MEMTABLE, &seq_used, prepare_batch_cnt_, + /*callback*/ nullptr, /*user_write_cb=*/nullptr, + &log_number_, /*log ref*/ 0, !DISABLE_MEMTABLE, + &seq_used, prepare_batch_cnt_, &add_prepared_callback); assert(!s.ok() || seq_used != kMaxSequenceNumber); auto prepare_seq = seq_used; @@ -247,9 +248,10 @@ Status WritePreparedTxn::CommitInternal() { // TransactionOptions::use_only_the_last_commit_time_batch_for_recovery to // true. See the comments about GetCommitTimeWriteBatch() in // include/rocksdb/utilities/transaction.h. - s = db_impl_->WriteImpl(write_options_, working_batch, nullptr, nullptr, - zero_log_number, disable_memtable, &seq_used, - batch_cnt, pre_release_callback); + s = db_impl_->WriteImpl(write_options_, working_batch, nullptr, + /*user_write_cb=*/nullptr, nullptr, zero_log_number, + disable_memtable, &seq_used, batch_cnt, + pre_release_callback); assert(!s.ok() || seq_used != kMaxSequenceNumber); const SequenceNumber commit_batch_seq = seq_used; if (LIKELY(do_one_write || !s.ok())) { @@ -284,8 +286,9 @@ Status WritePreparedTxn::CommitInternal() { const bool DISABLE_MEMTABLE = true; const size_t ONE_BATCH = 1; const uint64_t NO_REF_LOG = 0; - s = db_impl_->WriteImpl(write_options_, &empty_batch, nullptr, nullptr, - NO_REF_LOG, DISABLE_MEMTABLE, &seq_used, ONE_BATCH, + s = db_impl_->WriteImpl(write_options_, &empty_batch, nullptr, + /*user_write_cb=*/nullptr, nullptr, NO_REF_LOG, + DISABLE_MEMTABLE, &seq_used, ONE_BATCH, &update_commit_map_with_aux_batch); assert(!s.ok() || seq_used != kMaxSequenceNumber); return s; @@ -450,8 +453,9 @@ Status WritePreparedTxn::RollbackInternal() { // DB in one shot. min_uncommitted still works since it requires capturing // data that is written to DB but not yet committed, while // the rollback batch commits with PreReleaseCallback. - s = db_impl_->WriteImpl(write_options_, &rollback_batch, nullptr, nullptr, - NO_REF_LOG, !DISABLE_MEMTABLE, &seq_used, ONE_BATCH, + s = db_impl_->WriteImpl(write_options_, &rollback_batch, nullptr, + /*user_write_cb=*/nullptr, nullptr, NO_REF_LOG, + !DISABLE_MEMTABLE, &seq_used, ONE_BATCH, pre_release_callback); assert(!s.ok() || seq_used != kMaxSequenceNumber); if (!s.ok()) { @@ -476,8 +480,9 @@ Status WritePreparedTxn::RollbackInternal() { // In the absence of Prepare markers, use Noop as a batch separator s = WriteBatchInternal::InsertNoop(&empty_batch); assert(s.ok()); - s = db_impl_->WriteImpl(write_options_, &empty_batch, nullptr, nullptr, - NO_REF_LOG, DISABLE_MEMTABLE, &seq_used, ONE_BATCH, + s = db_impl_->WriteImpl(write_options_, &empty_batch, nullptr, + /*user_write_cb=*/nullptr, nullptr, NO_REF_LOG, + DISABLE_MEMTABLE, &seq_used, ONE_BATCH, &update_commit_map_with_prepare); assert(!s.ok() || seq_used != kMaxSequenceNumber); ROCKS_LOG_DETAILS(db_impl_->immutable_db_options().info_log, diff --git a/utilities/transactions/write_prepared_txn_db.cc b/utilities/transactions/write_prepared_txn_db.cc index a68e635f634..3c42b7fe0ea 100644 --- a/utilities/transactions/write_prepared_txn_db.cc +++ b/utilities/transactions/write_prepared_txn_db.cc @@ -213,8 +213,8 @@ Status WritePreparedTxnDB::WriteInternal(const WriteOptions& write_options_orig, } else { pre_release_callback = &add_prepared_callback; } - s = db_impl_->WriteImpl(write_options, batch, nullptr, nullptr, no_log_ref, - !DISABLE_MEMTABLE, &seq_used, batch_cnt, + s = db_impl_->WriteImpl(write_options, batch, nullptr, nullptr, nullptr, + no_log_ref, !DISABLE_MEMTABLE, &seq_used, batch_cnt, pre_release_callback); assert(!s.ok() || seq_used != kMaxSequenceNumber); uint64_t prepare_seq = seq_used; @@ -240,8 +240,8 @@ Status WritePreparedTxnDB::WriteInternal(const WriteOptions& write_options_orig, write_options.sync = false; const size_t ONE_BATCH = 1; // Just to inc the seq s = db_impl_->WriteImpl(write_options, &empty_batch, nullptr, nullptr, - no_log_ref, DISABLE_MEMTABLE, &seq_used, ONE_BATCH, - &update_commit_map_with_prepare); + nullptr, no_log_ref, DISABLE_MEMTABLE, &seq_used, + ONE_BATCH, &update_commit_map_with_prepare); assert(!s.ok() || seq_used != kMaxSequenceNumber); // Note: RemovePrepared is called from within PreReleaseCallback return s; diff --git a/utilities/transactions/write_unprepared_txn.cc b/utilities/transactions/write_unprepared_txn.cc index cdc888b5d79..1400aef38c1 100644 --- a/utilities/transactions/write_unprepared_txn.cc +++ b/utilities/transactions/write_unprepared_txn.cc @@ -378,7 +378,8 @@ Status WriteUnpreparedTxn::FlushWriteBatchToDBInternal(bool prepared) { // WriteImpl should not overwrite that value, so set log_used to nullptr if // log_number_ is already set. s = db_impl_->WriteImpl(write_options, GetWriteBatch()->GetWriteBatch(), - /*callback*/ nullptr, &last_log_number_, + /*callback*/ nullptr, /*user_write_cb=*/nullptr, + &last_log_number_, /*log ref*/ 0, !DISABLE_MEMTABLE, &seq_used, prepare_batch_cnt_, &add_prepared_callback); if (log_number_ == 0) { @@ -595,7 +596,7 @@ Status WriteUnpreparedTxn::CommitInternal() { const uint64_t zero_log_number = 0ull; size_t batch_cnt = UNLIKELY(commit_batch_cnt) ? commit_batch_cnt : 1; s = db_impl_->WriteImpl(write_options_, working_batch, nullptr, nullptr, - zero_log_number, disable_memtable, &seq_used, + nullptr, zero_log_number, disable_memtable, &seq_used, batch_cnt, pre_release_callback); assert(!s.ok() || seq_used != kMaxSequenceNumber); const SequenceNumber commit_batch_seq = seq_used; @@ -639,8 +640,8 @@ Status WriteUnpreparedTxn::CommitInternal() { const size_t ONE_BATCH = 1; const uint64_t NO_REF_LOG = 0; s = db_impl_->WriteImpl(write_options_, &empty_batch, nullptr, nullptr, - NO_REF_LOG, DISABLE_MEMTABLE, &seq_used, ONE_BATCH, - &update_commit_map_with_commit_batch); + nullptr, NO_REF_LOG, DISABLE_MEMTABLE, &seq_used, + ONE_BATCH, &update_commit_map_with_commit_batch); assert(!s.ok() || seq_used != kMaxSequenceNumber); // Note RemovePrepared should be called after WriteImpl that publishsed the // seq. Otherwise SmallestUnCommittedSeq optimization breaks. @@ -771,8 +772,8 @@ Status WriteUnpreparedTxn::RollbackInternal() { // data that is written to DB but not yet committed, while the rollback // batch commits with PreReleaseCallback. s = db_impl_->WriteImpl(write_options_, rollback_batch.GetWriteBatch(), - nullptr, nullptr, NO_REF_LOG, !DISABLE_MEMTABLE, - &seq_used, rollback_batch_cnt, + nullptr, nullptr, nullptr, NO_REF_LOG, + !DISABLE_MEMTABLE, &seq_used, rollback_batch_cnt, do_one_write ? &update_commit_map : nullptr); assert(!s.ok() || seq_used != kMaxSequenceNumber); if (!s.ok()) { @@ -807,8 +808,8 @@ Status WriteUnpreparedTxn::RollbackInternal() { s = WriteBatchInternal::InsertNoop(&empty_batch); assert(s.ok()); s = db_impl_->WriteImpl(write_options_, &empty_batch, nullptr, nullptr, - NO_REF_LOG, DISABLE_MEMTABLE, &seq_used, ONE_BATCH, - &update_commit_map_with_rollback_batch); + nullptr, NO_REF_LOG, DISABLE_MEMTABLE, &seq_used, + ONE_BATCH, &update_commit_map_with_rollback_batch); assert(!s.ok() || seq_used != kMaxSequenceNumber); // Mark the txn as rolled back if (s.ok()) { diff --git a/utilities/transactions/write_unprepared_txn_db.cc b/utilities/transactions/write_unprepared_txn_db.cc index 0f52cd2861c..90e827d2d9b 100644 --- a/utilities/transactions/write_unprepared_txn_db.cc +++ b/utilities/transactions/write_unprepared_txn_db.cc @@ -180,7 +180,8 @@ Status WriteUnpreparedTxnDB::RollbackRecoveredTransaction( const size_t kOneBatch = 1; uint64_t seq_used = kMaxSequenceNumber; s = db_impl_->WriteImpl(w_options, &rollback_batch, nullptr, nullptr, - kNoLogRef, !kDisableMemtable, &seq_used, kOneBatch); + nullptr, kNoLogRef, !kDisableMemtable, &seq_used, + kOneBatch); if (!s.ok()) { return s; } 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 d2e1816d4e6..13e3cb9cd19 100644 --- a/utilities/write_batch_with_index/write_batch_with_index.cc +++ b/utilities/write_batch_with_index/write_batch_with_index.cc @@ -823,11 +823,55 @@ void WriteBatchWithIndex::MultiGetFromBatchAndDB( } Status WriteBatchWithIndex::GetEntityFromBatchAndDB( - DB* db, const ReadOptions& read_options, ColumnFamilyHandle* column_family, + DB* db, const ReadOptions& _read_options, ColumnFamilyHandle* column_family, const Slice& key, PinnableWideColumns* columns, ReadCallback* callback) { - assert(db); - assert(column_family); - assert(columns); + if (!db) { + return Status::InvalidArgument( + "Cannot call GetEntityFromBatchAndDB without a DB object"); + } + + if (_read_options.io_activity != Env::IOActivity::kUnknown && + _read_options.io_activity != Env::IOActivity::kGetEntity) { + return Status::InvalidArgument( + "Can only call GetEntityFromBatchAndDB with `ReadOptions::io_activity` " + "set to `Env::IOActivity::kUnknown` or `Env::IOActivity::kGetEntity`"); + } + + ReadOptions read_options(_read_options); + if (read_options.io_activity == Env::IOActivity::kUnknown) { + read_options.io_activity = Env::IOActivity::kGetEntity; + } + + 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) { + if (!read_options.timestamp) { + return Status::InvalidArgument("Must specify timestamp"); + } + + if (read_options.timestamp->size() != ts_sz) { + return Status::InvalidArgument( + "Timestamp size does not match the timestamp size of the " + "column family"); + } + } else { + if (read_options.timestamp) { + return Status::InvalidArgument( + "Cannot specify timestamp since the column family does not have " + "timestamps enabled"); + } + } + + if (!columns) { + return Status::InvalidArgument( + "Cannot call GetEntityFromBatchAndDB without a PinnableWideColumns " + "object"); + } columns->Reset(); @@ -872,46 +916,100 @@ Status WriteBatchWithIndex::GetEntityFromBatchAndDB( return s; } -Status WriteBatchWithIndex::GetEntityFromBatchAndDB( - DB* db, const ReadOptions& read_options, ColumnFamilyHandle* column_family, - const Slice& key, PinnableWideColumns* columns) { +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(statuses); + if (!db) { - return Status::InvalidArgument( - "Cannot call GetEntityFromBatchAndDB without a DB object"); + const Status s = Status::InvalidArgument( + "Cannot call MultiGetEntityFromBatchAndDB without a DB object"); + for (size_t i = 0; i < num_keys; ++i) { + statuses[i] = s; + } + return; } - if (!column_family) { - return Status::InvalidArgument( - "Cannot call GetEntityFromBatchAndDB without a column family handle"); + if (_read_options.io_activity != Env::IOActivity::kUnknown && + _read_options.io_activity != Env::IOActivity::kMultiGetEntity) { + const Status s = Status::InvalidArgument( + "Can only call MultiGetEntityFromBatchAndDB with " + "`ReadOptions::io_activity` set to `Env::IOActivity::kUnknown` or " + "`Env::IOActivity::kMultiGetEntity`"); + for (size_t i = 0; i < num_keys; ++i) { + if (statuses[i].ok()) { + statuses[i] = s; + } + } + return; } - 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"); + ReadOptions read_options(_read_options); + if (read_options.io_activity == Env::IOActivity::kUnknown) { + read_options.io_activity = Env::IOActivity::kMultiGetEntity; } - if (!columns) { - return Status::InvalidArgument( - "Cannot call GetEntityFromBatchAndDB without a PinnableWideColumns " - "object"); + if (!column_family) { + const Status s = Status::InvalidArgument( + "Cannot call MultiGetEntityFromBatchAndDB without a column family " + "handle"); + for (size_t i = 0; i < num_keys; ++i) { + statuses[i] = s; + } + return; } - constexpr ReadCallback* callback = nullptr; + const Comparator* const ucmp = rep->comparator.GetComparator(column_family); + const size_t ts_sz = ucmp ? ucmp->timestamp_size() : 0; + if (ts_sz > 0) { + if (!read_options.timestamp) { + const Status s = Status::InvalidArgument("Must specify timestamp"); + for (size_t i = 0; i < num_keys; ++i) { + statuses[i] = s; + } + return; + } + + if (read_options.timestamp->size() != ts_sz) { + const Status s = Status::InvalidArgument( + "Timestamp size does not match the timestamp size of the " + "column family"); + for (size_t i = 0; i < num_keys; ++i) { + statuses[i] = s; + } + return; + } + } else { + if (read_options.timestamp) { + const Status s = Status::InvalidArgument( + "Cannot specify timestamp since the column family does not have " + "timestamps enabled"); + for (size_t i = 0; i < num_keys; ++i) { + statuses[i] = s; + } + return; + } + } - return GetEntityFromBatchAndDB(db, read_options, column_family, key, columns, - callback); -} + if (!keys) { + const Status s = Status::InvalidArgument( + "Cannot call MultiGetEntityFromBatchAndDB without keys"); + for (size_t i = 0; i < num_keys; ++i) { + statuses[i] = s; + } + return; + } -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); + if (!results) { + const Status s = Status::InvalidArgument( + "Cannot call MultiGetEntityFromBatchAndDB without " + "PinnableWideColumns objects"); + for (size_t i = 0; i < num_keys; ++i) { + statuses[i] = s; + } + return; + } struct MergeTuple { MergeTuple(const Slice& _key, Status* _s, MergeContext&& _merge_context, @@ -990,8 +1088,8 @@ void WriteBatchWithIndex::MultiGetEntityFromBatchAndDB( 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); + ->MultiGetEntityWithCallback(read_options, column_family, callback, + &sorted_keys); for (const auto& merge : merges) { if (merge.s->ok() || merge.s->IsNotFound()) { // DB lookup succeeded @@ -1001,57 +1099,6 @@ void WriteBatchWithIndex::MultiGetEntityFromBatchAndDB( } } -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 5a8031423e1..e2e60ab6df4 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 @@ -36,6 +36,8 @@ BaseDeltaIterator::BaseDeltaIterator(ColumnFamilyHandle* column_family, assert(comparator_); } +BaseDeltaIterator::~BaseDeltaIterator() = default; + bool BaseDeltaIterator::Valid() const { return status_.ok() ? (current_at_base_ ? BaseValid() : DeltaValid()) : false; } diff --git a/utilities/write_batch_with_index/write_batch_with_index_internal.h b/utilities/write_batch_with_index/write_batch_with_index_internal.h index 163de2014d4..45c3716b326 100644 --- a/utilities/write_batch_with_index/write_batch_with_index_internal.h +++ b/utilities/write_batch_with_index/write_batch_with_index_internal.h @@ -38,7 +38,7 @@ class BaseDeltaIterator : public Iterator { WBWIIteratorImpl* delta_iterator, const Comparator* comparator); - ~BaseDeltaIterator() override {} + ~BaseDeltaIterator() override; bool Valid() const override; void SeekToFirst() override; 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 bf723033133..d706682a5fb 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 @@ -2973,6 +2973,128 @@ TEST_P(WriteBatchWithIndexTest, GetEntityFromBatch) { } } +TEST_P(WriteBatchWithIndexTest, EntityReadSanityChecks) { + ASSERT_OK(OpenDB()); + + constexpr char foo[] = "foo"; + constexpr char bar[] = "bar"; + constexpr size_t num_keys = 2; + + { + constexpr DB* db = nullptr; + PinnableWideColumns columns; + ASSERT_TRUE(batch_ + ->GetEntityFromBatchAndDB(db, ReadOptions(), + db_->DefaultColumnFamily(), foo, + &columns) + .IsInvalidArgument()); + } + + { + constexpr ColumnFamilyHandle* column_family = nullptr; + PinnableWideColumns columns; + ASSERT_TRUE(batch_ + ->GetEntityFromBatchAndDB(db_, ReadOptions(), column_family, + foo, &columns) + .IsInvalidArgument()); + } + + { + constexpr PinnableWideColumns* columns = nullptr; + ASSERT_TRUE(batch_ + ->GetEntityFromBatchAndDB(db_, ReadOptions(), + db_->DefaultColumnFamily(), foo, + columns) + .IsInvalidArgument()); + } + + { + ReadOptions read_options; + read_options.io_activity = Env::IOActivity::kGet; + + PinnableWideColumns columns; + ASSERT_TRUE(batch_ + ->GetEntityFromBatchAndDB(db_, read_options, + db_->DefaultColumnFamily(), foo, + &columns) + .IsInvalidArgument()); + } + + { + constexpr DB* db = nullptr; + std::array keys{{foo, bar}}; + std::array results; + std::array statuses; + constexpr bool sorted_input = false; + + batch_->MultiGetEntityFromBatchAndDB( + db, ReadOptions(), db_->DefaultColumnFamily(), num_keys, keys.data(), + results.data(), statuses.data(), sorted_input); + + ASSERT_TRUE(statuses[0].IsInvalidArgument()); + ASSERT_TRUE(statuses[1].IsInvalidArgument()); + } + + { + constexpr ColumnFamilyHandle* column_family = nullptr; + std::array keys{{foo, bar}}; + std::array results; + std::array statuses; + constexpr bool sorted_input = false; + + batch_->MultiGetEntityFromBatchAndDB(db_, ReadOptions(), column_family, + num_keys, keys.data(), results.data(), + statuses.data(), sorted_input); + + ASSERT_TRUE(statuses[0].IsInvalidArgument()); + ASSERT_TRUE(statuses[1].IsInvalidArgument()); + } + + { + constexpr Slice* keys = nullptr; + std::array results; + std::array statuses; + constexpr bool sorted_input = false; + + batch_->MultiGetEntityFromBatchAndDB( + db_, ReadOptions(), db_->DefaultColumnFamily(), num_keys, keys, + results.data(), statuses.data(), sorted_input); + + ASSERT_TRUE(statuses[0].IsInvalidArgument()); + ASSERT_TRUE(statuses[1].IsInvalidArgument()); + } + + { + std::array keys{{foo, bar}}; + constexpr PinnableWideColumns* results = nullptr; + std::array statuses; + constexpr bool sorted_input = false; + + batch_->MultiGetEntityFromBatchAndDB( + db_, ReadOptions(), db_->DefaultColumnFamily(), num_keys, keys.data(), + results, statuses.data(), sorted_input); + + ASSERT_TRUE(statuses[0].IsInvalidArgument()); + ASSERT_TRUE(statuses[1].IsInvalidArgument()); + } + + { + ReadOptions read_options; + read_options.io_activity = Env::IOActivity::kMultiGet; + + std::array keys{{foo, bar}}; + std::array results; + std::array statuses; + constexpr bool sorted_input = false; + + batch_->MultiGetEntityFromBatchAndDB( + db_, read_options, db_->DefaultColumnFamily(), num_keys, keys.data(), + results.data(), statuses.data(), sorted_input); + ASSERT_TRUE(statuses[0].IsInvalidArgument()); + ASSERT_TRUE(statuses[1].IsInvalidArgument()); + } +} + INSTANTIATE_TEST_CASE_P(WBWI, WriteBatchWithIndexTest, testing::Bool()); } // namespace ROCKSDB_NAMESPACE