diff --git a/HISTORY.md b/HISTORY.md index 59641ce89..535f37aa9 100644 --- a/HISTORY.md +++ b/HISTORY.md @@ -1,6 +1,7 @@ # Rocksdb Change Log ## Unreleased ### Behavior Changes +* Added two additional tickers, MEMTABLE_PAYLOAD_BYTES_AT_FLUSH and MEMTABLE_GARBAGE_BYTES_AT_FLUSH. These stats can be used to estimate the ratio of "garbage" (outdated) bytes in the memtable that are discarded at flush time. * Added API comments clarifying safe usage of Disable/EnableManualCompaction and EventListener callbacks for compaction. ### Bug Fixes diff --git a/db/builder.cc b/db/builder.cc index b160ccea7..88c99a084 100644 --- a/db/builder.cc +++ b/db/builder.cc @@ -69,7 +69,8 @@ Status BuildTable( int job_id, const Env::IOPriority io_priority, TableProperties* table_properties, Env::WriteLifeTimeHint write_hint, const std::string* full_history_ts_low, - BlobFileCompletionCallback* blob_callback, uint64_t* num_input_entries) { + BlobFileCompletionCallback* blob_callback, uint64_t* num_input_entries, + uint64_t* memtable_payload_bytes, uint64_t* memtable_garbage_bytes) { assert((tboptions.column_family_id == TablePropertiesCollectorFactory::Context::kUnknownColumnFamily) == tboptions.column_family_name.empty()); @@ -89,9 +90,12 @@ Status BuildTable( new CompactionRangeDelAggregator(&tboptions.internal_comparator, snapshots)); uint64_t num_unfragmented_tombstones = 0; + uint64_t total_tombstone_payload_bytes = 0; for (auto& range_del_iter : range_del_iters) { num_unfragmented_tombstones += range_del_iter->num_unfragmented_tombstones(); + total_tombstone_payload_bytes += + range_del_iter->total_tombstone_payload_bytes(); range_del_agg->AddTombstones(std::move(range_del_iter)); } @@ -254,6 +258,25 @@ Status BuildTable( meta->marked_for_compaction = builder->NeedCompact(); assert(meta->fd.GetFileSize() > 0); tp = builder->GetTableProperties(); // refresh now that builder is finished + if (memtable_payload_bytes != nullptr && + memtable_garbage_bytes != nullptr) { + const CompactionIterationStats& ci_stats = c_iter.iter_stats(); + uint64_t total_payload_bytes = ci_stats.total_input_raw_key_bytes + + ci_stats.total_input_raw_value_bytes + + total_tombstone_payload_bytes; + uint64_t total_payload_bytes_written = + (tp.raw_key_size + tp.raw_value_size); + // Prevent underflow, which may still happen at this point + // since we only support inserts, deletes, and deleteRanges. + if (total_payload_bytes_written <= total_payload_bytes) { + *memtable_payload_bytes = total_payload_bytes; + *memtable_garbage_bytes = + total_payload_bytes - total_payload_bytes_written; + } else { + *memtable_payload_bytes = 0; + *memtable_garbage_bytes = 0; + } + } if (table_properties) { *table_properties = tp; } diff --git a/db/builder.h b/db/builder.h index 879fb2bd8..f8828f5c4 100644 --- a/db/builder.h +++ b/db/builder.h @@ -66,6 +66,8 @@ extern Status BuildTable( Env::WriteLifeTimeHint write_hint = Env::WLTH_NOT_SET, const std::string* full_history_ts_low = nullptr, BlobFileCompletionCallback* blob_callback = nullptr, - uint64_t* num_input_entries = nullptr); + uint64_t* num_input_entries = nullptr, + uint64_t* memtable_payload_bytes = nullptr, + uint64_t* memtable_garbage_bytes = nullptr); } // namespace ROCKSDB_NAMESPACE diff --git a/db/db_flush_test.cc b/db/db_flush_test.cc index e94901cbc..b5d3026d8 100644 --- a/db/db_flush_test.cc +++ b/db/db_flush_test.cc @@ -285,6 +285,379 @@ TEST_F(DBFlushTest, ScheduleOnlyOneBgThread) { SyncPoint::GetInstance()->ClearAllCallBacks(); } +// The following 3 tests are designed for testing garbage statistics at flush +// time. +// +// ======= General Information ======= (from GitHub Wiki). +// There are three scenarios where memtable flush can be triggered: +// +// 1 - Memtable size exceeds ColumnFamilyOptions::write_buffer_size +// after a write. +// 2 - Total memtable size across all column families exceeds +// DBOptions::db_write_buffer_size, +// or DBOptions::write_buffer_manager signals a flush. In this scenario +// the largest memtable will be flushed. +// 3 - Total WAL file size exceeds DBOptions::max_total_wal_size. +// In this scenario the memtable with the oldest data will be flushed, +// in order to allow the WAL file with data from this memtable to be +// purged. +// +// As a result, a memtable can be flushed before it is full. This is one +// reason the generated SST file can be smaller than the corresponding +// memtable. Compression is another factor to make SST file smaller than +// corresponding memtable, since data in memtable is uncompressed. + +TEST_F(DBFlushTest, StatisticsGarbageBasic) { + Options options = CurrentOptions(); + + // The following options are used to enforce several values that + // may already exist as default values to make this test resilient + // to default value updates in the future. + options.statistics = CreateDBStatistics(); + + // Record all statistics. + options.statistics->set_stats_level(StatsLevel::kAll); + + // create the DB if it's not already present + options.create_if_missing = true; + + // Useful for now as we are trying to compare uncompressed data savings on + // flush(). + options.compression = kNoCompression; + + // Prevent memtable in place updates. Should already be disabled + // (from Wiki: + // In place updates can be enabled by toggling on the bool + // inplace_update_support flag. However, this flag is by default set to + // false + // because this thread-safe in-place update support is not compatible + // with concurrent memtable writes. Note that the bool + // allow_concurrent_memtable_write is set to true by default ) + options.inplace_update_support = false; + options.allow_concurrent_memtable_write = true; + + // Enforce size of a single MemTable to 64MB (64MB = 67108864 bytes). + options.write_buffer_size = 64 << 20; + + ASSERT_OK(TryReopen(options)); + + // Put multiple times the same key-values. + // The encoded length of a db entry in the memtable is + // defined in db/memtable.cc (MemTable::Add) as the variable: + // encoded_len= VarintLength(internal_key_size) --> = + // log_256(internal_key). + // Min # of bytes + // necessary to + // store + // internal_key_size. + // + internal_key_size --> = actual key string, + // (size key_size: w/o term null char) + // + 8 bytes for + // fixed uint64 "seq + // number + // + + // insertion type" + // + VarintLength(val_size) --> = min # of bytes to + // store val_size + // + val_size --> = actual value + // string + // For example, in our situation, "key1" : size 4, "value1" : size 6 + // (the terminating null characters are not copied over to the memtable). + // And therefore encoded_len = 1 + (4+8) + 1 + 6 = 20 bytes per entry. + // However in terms of raw data contained in the memtable, and written + // over to the SSTable, we only count internal_key_size and val_size, + // because this is the only raw chunk of bytes that contains everything + // necessary to reconstruct a user entry: sequence number, insertion type, + // key, and value. + + // To test the relevance of our Memtable garbage statistics, + // namely MEMTABLE_PAYLOAD_BYTES_AT_FLUSH and MEMTABLE_GARBAGE_BYTES_AT_FLUSH, + // we insert K-V pairs with 3 distinct keys (of length 4), + // and random values of arbitrary length RAND_VALUES_LENGTH, + // and we repeat this step NUM_REPEAT times total. + // At the end, we insert 3 final K-V pairs with the same 3 keys + // and known values (these will be the final values, of length 6). + // I chose NUM_REPEAT=2,000 such that no automatic flush is + // triggered (the number of bytes in the memtable is therefore + // well below any meaningful heuristic for a memtable of size 64MB). + // As a result, since each K-V pair is inserted as a payload + // of N meaningful bytes (sequence number, insertion type, + // key, and value = 8 + 4 + RAND_VALUE_LENGTH), + // MEMTABLE_GARBAGE_BYTES_AT_FLUSH should be equal to 2,000 * N bytes + // and MEMTABLE_PAYLAOD_BYTES_AT_FLUSH = MEMTABLE_GARBAGE_BYTES_AT_FLUSH + + // (3*(8 + 4 + 6)) bytes. For RAND_VALUE_LENGTH = 172 (arbitrary value), we + // expect: + // N = 8 + 4 + 172 = 184 bytes + // MEMTABLE_GARBAGE_BYTES_AT_FLUSH = 2,000 * 184 = 368,000 bytes. + // MEMTABLE_PAYLOAD_BYTES_AT_FLUSH = 368,000 + 3*18 = 368,054 bytes. + + const size_t NUM_REPEAT = 2000; + const size_t RAND_VALUES_LENGTH = 172; + const std::string KEY1 = "key1"; + const std::string KEY2 = "key2"; + const std::string KEY3 = "key3"; + const std::string VALUE1 = "value1"; + const std::string VALUE2 = "value2"; + const std::string VALUE3 = "value3"; + uint64_t EXPECTED_MEMTABLE_PAYLOAD_BYTES_AT_FLUSH = 0; + uint64_t EXPECTED_MEMTABLE_GARBAGE_BYTES_AT_FLUSH = 0; + + Random rnd(301); + // Insertion of of K-V pairs, multiple times. + for (size_t i = 0; i < NUM_REPEAT; i++) { + // Create value strings of arbitrary length RAND_VALUES_LENGTH bytes. + std::string p_v1 = rnd.RandomString(RAND_VALUES_LENGTH); + std::string p_v2 = rnd.RandomString(RAND_VALUES_LENGTH); + std::string p_v3 = rnd.RandomString(RAND_VALUES_LENGTH); + ASSERT_OK(Put(KEY1, p_v1)); + ASSERT_OK(Put(KEY2, p_v2)); + ASSERT_OK(Put(KEY3, p_v3)); + EXPECTED_MEMTABLE_GARBAGE_BYTES_AT_FLUSH += + KEY1.size() + p_v1.size() + sizeof(uint64_t); + EXPECTED_MEMTABLE_GARBAGE_BYTES_AT_FLUSH += + KEY2.size() + p_v2.size() + sizeof(uint64_t); + EXPECTED_MEMTABLE_GARBAGE_BYTES_AT_FLUSH += + KEY3.size() + p_v3.size() + sizeof(uint64_t); + } + + // The memtable data bytes includes the "garbage" + // bytes along with the useful payload. + EXPECTED_MEMTABLE_PAYLOAD_BYTES_AT_FLUSH = + EXPECTED_MEMTABLE_GARBAGE_BYTES_AT_FLUSH; + + ASSERT_OK(Put(KEY1, VALUE1)); + ASSERT_OK(Put(KEY2, VALUE2)); + ASSERT_OK(Put(KEY3, VALUE3)); + + // Add useful payload to the memtable data bytes: + EXPECTED_MEMTABLE_PAYLOAD_BYTES_AT_FLUSH += + KEY1.size() + VALUE1.size() + KEY2.size() + VALUE2.size() + KEY3.size() + + VALUE3.size() + 3 * sizeof(uint64_t); + + // We assert that the last K-V pairs have been successfully inserted, + // and that the valid values are VALUE1, VALUE2, VALUE3. + PinnableSlice value; + ASSERT_OK(Get(KEY1, &value)); + ASSERT_EQ(value.ToString(), VALUE1); + ASSERT_OK(Get(KEY2, &value)); + ASSERT_EQ(value.ToString(), VALUE2); + ASSERT_OK(Get(KEY3, &value)); + ASSERT_EQ(value.ToString(), VALUE3); + + // Force flush to SST. Increments the statistics counter. + ASSERT_OK(Flush()); + + // Collect statistics. + uint64_t mem_data_bytes = + TestGetTickerCount(options, MEMTABLE_PAYLOAD_BYTES_AT_FLUSH); + uint64_t mem_garbage_bytes = + TestGetTickerCount(options, MEMTABLE_GARBAGE_BYTES_AT_FLUSH); + + EXPECT_EQ(mem_data_bytes, EXPECTED_MEMTABLE_PAYLOAD_BYTES_AT_FLUSH); + EXPECT_EQ(mem_garbage_bytes, EXPECTED_MEMTABLE_GARBAGE_BYTES_AT_FLUSH); + + Close(); +} + +TEST_F(DBFlushTest, StatisticsGarbageInsertAndDeletes) { + Options options = CurrentOptions(); + options.statistics = CreateDBStatistics(); + options.statistics->set_stats_level(StatsLevel::kAll); + options.create_if_missing = true; + options.compression = kNoCompression; + options.inplace_update_support = false; + options.allow_concurrent_memtable_write = true; + options.write_buffer_size = 67108864; + + ASSERT_OK(TryReopen(options)); + + const size_t NUM_REPEAT = 2000; + const size_t RAND_VALUES_LENGTH = 37; + const std::string KEY1 = "key1"; + const std::string KEY2 = "key2"; + const std::string KEY3 = "key3"; + const std::string KEY4 = "key4"; + const std::string KEY5 = "key5"; + const std::string KEY6 = "key6"; + + uint64_t EXPECTED_MEMTABLE_PAYLOAD_BYTES_AT_FLUSH = 0; + uint64_t EXPECTED_MEMTABLE_GARBAGE_BYTES_AT_FLUSH = 0; + + WriteBatch batch; + + Random rnd(301); + // Insertion of of K-V pairs, multiple times. + for (size_t i = 0; i < NUM_REPEAT; i++) { + // Create value strings of arbitrary length RAND_VALUES_LENGTH bytes. + std::string p_v1 = rnd.RandomString(RAND_VALUES_LENGTH); + std::string p_v2 = rnd.RandomString(RAND_VALUES_LENGTH); + std::string p_v3 = rnd.RandomString(RAND_VALUES_LENGTH); + ASSERT_OK(Put(KEY1, p_v1)); + ASSERT_OK(Put(KEY2, p_v2)); + ASSERT_OK(Put(KEY3, p_v3)); + EXPECTED_MEMTABLE_GARBAGE_BYTES_AT_FLUSH += + KEY1.size() + p_v1.size() + sizeof(uint64_t); + EXPECTED_MEMTABLE_GARBAGE_BYTES_AT_FLUSH += + KEY2.size() + p_v2.size() + sizeof(uint64_t); + EXPECTED_MEMTABLE_GARBAGE_BYTES_AT_FLUSH += + KEY3.size() + p_v3.size() + sizeof(uint64_t); + ASSERT_OK(Delete(KEY1)); + ASSERT_OK(Delete(KEY2)); + ASSERT_OK(Delete(KEY3)); + EXPECTED_MEMTABLE_GARBAGE_BYTES_AT_FLUSH += + KEY1.size() + KEY2.size() + KEY3.size() + 3 * sizeof(uint64_t); + } + + // The memtable data bytes includes the "garbage" + // bytes along with the useful payload. + EXPECTED_MEMTABLE_PAYLOAD_BYTES_AT_FLUSH = + EXPECTED_MEMTABLE_GARBAGE_BYTES_AT_FLUSH; + + // Note : one set of delete for KEY1, KEY2, KEY3 is written to + // SSTable to propagate the delete operations to K-V pairs + // that could have been inserted into the database during past Flush + // opeartions. + EXPECTED_MEMTABLE_GARBAGE_BYTES_AT_FLUSH -= + KEY1.size() + KEY2.size() + KEY3.size() + 3 * sizeof(uint64_t); + + // Additional useful paylaod. + ASSERT_OK(Delete(KEY4)); + ASSERT_OK(Delete(KEY5)); + ASSERT_OK(Delete(KEY6)); + + // // Add useful payload to the memtable data bytes: + EXPECTED_MEMTABLE_PAYLOAD_BYTES_AT_FLUSH += + KEY4.size() + KEY5.size() + KEY6.size() + 3 * sizeof(uint64_t); + + // We assert that the K-V pairs have been successfully deleted. + PinnableSlice value; + ASSERT_NOK(Get(KEY1, &value)); + ASSERT_NOK(Get(KEY2, &value)); + ASSERT_NOK(Get(KEY3, &value)); + + // Force flush to SST. Increments the statistics counter. + ASSERT_OK(Flush()); + + // Collect statistics. + uint64_t mem_data_bytes = + TestGetTickerCount(options, MEMTABLE_PAYLOAD_BYTES_AT_FLUSH); + uint64_t mem_garbage_bytes = + TestGetTickerCount(options, MEMTABLE_GARBAGE_BYTES_AT_FLUSH); + + EXPECT_EQ(mem_data_bytes, EXPECTED_MEMTABLE_PAYLOAD_BYTES_AT_FLUSH); + EXPECT_EQ(mem_garbage_bytes, EXPECTED_MEMTABLE_GARBAGE_BYTES_AT_FLUSH); + + Close(); +} + +TEST_F(DBFlushTest, StatisticsGarbageRangeDeletes) { + Options options = CurrentOptions(); + options.statistics = CreateDBStatistics(); + options.statistics->set_stats_level(StatsLevel::kAll); + options.create_if_missing = true; + options.compression = kNoCompression; + options.inplace_update_support = false; + options.allow_concurrent_memtable_write = true; + options.write_buffer_size = 67108864; + + ASSERT_OK(TryReopen(options)); + + const size_t NUM_REPEAT = 1000; + const size_t RAND_VALUES_LENGTH = 42; + const std::string KEY1 = "key1"; + const std::string KEY2 = "key2"; + const std::string KEY3 = "key3"; + const std::string KEY4 = "key4"; + const std::string KEY5 = "key5"; + const std::string KEY6 = "key6"; + const std::string VALUE3 = "value3"; + + uint64_t EXPECTED_MEMTABLE_PAYLOAD_BYTES_AT_FLUSH = 0; + uint64_t EXPECTED_MEMTABLE_GARBAGE_BYTES_AT_FLUSH = 0; + + Random rnd(301); + // Insertion of of K-V pairs, multiple times. + // Also insert DeleteRange + for (size_t i = 0; i < NUM_REPEAT; i++) { + // Create value strings of arbitrary length RAND_VALUES_LENGTH bytes. + std::string p_v1 = rnd.RandomString(RAND_VALUES_LENGTH); + std::string p_v2 = rnd.RandomString(RAND_VALUES_LENGTH); + std::string p_v3 = rnd.RandomString(RAND_VALUES_LENGTH); + ASSERT_OK(Put(KEY1, p_v1)); + ASSERT_OK(Put(KEY2, p_v2)); + ASSERT_OK(Put(KEY3, p_v3)); + EXPECTED_MEMTABLE_GARBAGE_BYTES_AT_FLUSH += + KEY1.size() + p_v1.size() + sizeof(uint64_t); + EXPECTED_MEMTABLE_GARBAGE_BYTES_AT_FLUSH += + KEY2.size() + p_v2.size() + sizeof(uint64_t); + EXPECTED_MEMTABLE_GARBAGE_BYTES_AT_FLUSH += + KEY3.size() + p_v3.size() + sizeof(uint64_t); + ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), KEY1, + KEY2)); + // Note: DeleteRange have an exclusive upper bound, e.g. here: [KEY2,KEY3) + // is deleted. + ASSERT_OK(db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), KEY2, + KEY3)); + // Delete ranges are stored as a regular K-V pair, with key=STARTKEY, + // value=ENDKEY. + EXPECTED_MEMTABLE_GARBAGE_BYTES_AT_FLUSH += + (KEY1.size() + KEY2.size() + sizeof(uint64_t)) + + (KEY2.size() + KEY3.size() + sizeof(uint64_t)); + } + + // The memtable data bytes includes the "garbage" + // bytes along with the useful payload. + EXPECTED_MEMTABLE_PAYLOAD_BYTES_AT_FLUSH = + EXPECTED_MEMTABLE_GARBAGE_BYTES_AT_FLUSH; + + // Note : one set of deleteRange for (KEY1, KEY2) and (KEY2, KEY3) is written + // to SSTable to propagate the deleteRange operations to K-V pairs that could + // have been inserted into the database during past Flush opeartions. + EXPECTED_MEMTABLE_GARBAGE_BYTES_AT_FLUSH -= + (KEY1.size() + KEY2.size() + sizeof(uint64_t)) + + (KEY2.size() + KEY3.size() + sizeof(uint64_t)); + + // Overwrite KEY3 with known value (VALUE3) + // Note that during the whole time KEY3 has never been deleted + // by the RangeDeletes. + ASSERT_OK(Put(KEY3, VALUE3)); + EXPECTED_MEMTABLE_PAYLOAD_BYTES_AT_FLUSH += + KEY3.size() + VALUE3.size() + sizeof(uint64_t); + + // Additional useful paylaod. + ASSERT_OK( + db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), KEY4, KEY5)); + ASSERT_OK( + db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), KEY5, KEY6)); + + // Add useful payload to the memtable data bytes: + EXPECTED_MEMTABLE_PAYLOAD_BYTES_AT_FLUSH += + (KEY4.size() + KEY5.size() + sizeof(uint64_t)) + + (KEY5.size() + KEY6.size() + sizeof(uint64_t)); + + // We assert that the K-V pairs have been successfully deleted. + PinnableSlice value; + ASSERT_NOK(Get(KEY1, &value)); + ASSERT_NOK(Get(KEY2, &value)); + // And that KEY3's value is correct. + ASSERT_OK(Get(KEY3, &value)); + ASSERT_EQ(value, VALUE3); + + // Force flush to SST. Increments the statistics counter. + ASSERT_OK(Flush()); + + // Collect statistics. + uint64_t mem_data_bytes = + TestGetTickerCount(options, MEMTABLE_PAYLOAD_BYTES_AT_FLUSH); + uint64_t mem_garbage_bytes = + TestGetTickerCount(options, MEMTABLE_GARBAGE_BYTES_AT_FLUSH); + + EXPECT_EQ(mem_data_bytes, EXPECTED_MEMTABLE_PAYLOAD_BYTES_AT_FLUSH); + EXPECT_EQ(mem_garbage_bytes, EXPECTED_MEMTABLE_GARBAGE_BYTES_AT_FLUSH); + + Close(); +} + TEST_P(DBFlushDirectIOTest, DirectIO) { Options options; options.create_if_missing = true; diff --git a/db/flush_job.cc b/db/flush_job.cc index cd5b4fbe6..10d6ed108 100644 --- a/db/flush_job.cc +++ b/db/flush_job.cc @@ -403,6 +403,8 @@ Status FlushJob::WriteLevel0Table() { : meta_.oldest_ancester_time; uint64_t num_input_entries = 0; + uint64_t memtable_payload_bytes = 0; + uint64_t memtable_garbage_bytes = 0; IOStatus io_s; const std::string* const full_history_ts_low = (full_history_ts_low_.empty()) ? nullptr : &full_history_ts_low_; @@ -422,7 +424,7 @@ Status FlushJob::WriteLevel0Table() { mutable_cf_options_.paranoid_file_checks, cfd_->internal_stats(), &io_s, io_tracer_, event_logger_, job_context_->job_id, Env::IO_HIGH, &table_properties_, write_hint, full_history_ts_low, blob_callback_, - &num_input_entries); + &num_input_entries, &memtable_payload_bytes, &memtable_garbage_bytes); if (!io_s.ok()) { io_status_ = io_s; } @@ -437,6 +439,12 @@ Status FlushJob::WriteLevel0Table() { s = Status::Corruption(msg); } } + if (tboptions.reason == TableFileCreationReason::kFlush) { + RecordTick(stats_, MEMTABLE_PAYLOAD_BYTES_AT_FLUSH, + memtable_payload_bytes); + RecordTick(stats_, MEMTABLE_GARBAGE_BYTES_AT_FLUSH, + memtable_garbage_bytes); + } LogFlush(db_options_.info_log); } ROCKS_LOG_INFO(db_options_.info_log, diff --git a/db/range_tombstone_fragmenter.cc b/db/range_tombstone_fragmenter.cc index e9f353d58..0985fee0b 100644 --- a/db/range_tombstone_fragmenter.cc +++ b/db/range_tombstone_fragmenter.cc @@ -28,8 +28,11 @@ FragmentedRangeTombstoneList::FragmentedRangeTombstoneList( InternalKey pinned_last_start_key; Slice last_start_key; num_unfragmented_tombstones_ = 0; + total_tombstone_payload_bytes_ = 0; for (unfragmented_tombstones->SeekToFirst(); unfragmented_tombstones->Valid(); unfragmented_tombstones->Next(), num_unfragmented_tombstones_++) { + total_tombstone_payload_bytes_ += unfragmented_tombstones->key().size() + + unfragmented_tombstones->value().size(); if (num_unfragmented_tombstones_ > 0 && icmp.Compare(last_start_key, unfragmented_tombstones->key()) > 0) { is_sorted = false; @@ -52,8 +55,12 @@ FragmentedRangeTombstoneList::FragmentedRangeTombstoneList( std::vector keys, values; keys.reserve(num_unfragmented_tombstones_); values.reserve(num_unfragmented_tombstones_); + // Reset the counter to zero for the next iteration over keys. + total_tombstone_payload_bytes_ = 0; for (unfragmented_tombstones->SeekToFirst(); unfragmented_tombstones->Valid(); unfragmented_tombstones->Next()) { + total_tombstone_payload_bytes_ += unfragmented_tombstones->key().size() + + unfragmented_tombstones->value().size(); keys.emplace_back(unfragmented_tombstones->key().data(), unfragmented_tombstones->key().size()); values.emplace_back(unfragmented_tombstones->value().data(), diff --git a/db/range_tombstone_fragmenter.h b/db/range_tombstone_fragmenter.h index d0d07e4e2..f323db5d7 100644 --- a/db/range_tombstone_fragmenter.h +++ b/db/range_tombstone_fragmenter.h @@ -72,6 +72,10 @@ struct FragmentedRangeTombstoneList { return num_unfragmented_tombstones_; } + uint64_t total_tombstone_payload_bytes() const { + return total_tombstone_payload_bytes_; + } + private: // Given an ordered range tombstone iterator unfragmented_tombstones, // "fragment" the tombstones into non-overlapping pieces, and store them in @@ -87,6 +91,7 @@ struct FragmentedRangeTombstoneList { std::list pinned_slices_; PinnedIteratorsManager pinned_iters_mgr_; uint64_t num_unfragmented_tombstones_; + uint64_t total_tombstone_payload_bytes_; }; // FragmentedRangeTombstoneIterator converts an InternalIterator of a range-del @@ -188,6 +193,9 @@ class FragmentedRangeTombstoneIterator : public InternalIterator { uint64_t num_unfragmented_tombstones() const { return tombstones_->num_unfragmented_tombstones(); } + uint64_t total_tombstone_payload_bytes() const { + return tombstones_->total_tombstone_payload_bytes(); + } private: using RangeTombstoneStack = FragmentedRangeTombstoneList::RangeTombstoneStack; diff --git a/include/rocksdb/statistics.h b/include/rocksdb/statistics.h index 70c0c8489..8fc5a2eed 100644 --- a/include/rocksdb/statistics.h +++ b/include/rocksdb/statistics.h @@ -383,6 +383,12 @@ enum Tickers : uint32_t { ERROR_HANDLER_AUTORESUME_RETRY_TOTAL_COUNT, ERROR_HANDLER_AUTORESUME_SUCCESS_COUNT, + // Statistics for memtable garbage collection: + // Raw bytes of data (payload) present on memtable at flush time. + MEMTABLE_PAYLOAD_BYTES_AT_FLUSH, + // Outdated bytes of data present on memtable at flush time. + MEMTABLE_GARBAGE_BYTES_AT_FLUSH, + TICKER_ENUM_MAX }; diff --git a/java/rocksjni/portal.h b/java/rocksjni/portal.h index 330b9e9bd..3800c8324 100644 --- a/java/rocksjni/portal.h +++ b/java/rocksjni/portal.h @@ -4996,6 +4996,10 @@ class TickerTypeJni { return -0x1A; case ROCKSDB_NAMESPACE::Tickers::ERROR_HANDLER_AUTORESUME_SUCCESS_COUNT: return -0x1B; + case ROCKSDB_NAMESPACE::Tickers::MEMTABLE_PAYLOAD_BYTES_AT_FLUSH: + return -0x1C; + case ROCKSDB_NAMESPACE::Tickers::MEMTABLE_GARBAGE_BYTES_AT_FLUSH: + return -0x1D; case ROCKSDB_NAMESPACE::Tickers::TICKER_ENUM_MAX: // 0x5F for backwards compatibility on current minor version. return 0x5F; @@ -5322,6 +5326,10 @@ class TickerTypeJni { case -0x1B: return ROCKSDB_NAMESPACE::Tickers:: ERROR_HANDLER_AUTORESUME_SUCCESS_COUNT; + case -0x1C: + return ROCKSDB_NAMESPACE::Tickers::MEMTABLE_PAYLOAD_BYTES_AT_FLUSH; + case -0x1D: + return ROCKSDB_NAMESPACE::Tickers::MEMTABLE_GARBAGE_BYTES_AT_FLUSH; case 0x5F: // 0x5F for backwards compatibility on current minor version. return ROCKSDB_NAMESPACE::Tickers::TICKER_ENUM_MAX; diff --git a/java/src/main/java/org/rocksdb/TickerType.java b/java/src/main/java/org/rocksdb/TickerType.java index a2c708588..0d6cc5a92 100644 --- a/java/src/main/java/org/rocksdb/TickerType.java +++ b/java/src/main/java/org/rocksdb/TickerType.java @@ -752,6 +752,18 @@ public enum TickerType { ERROR_HANDLER_AUTORESUME_RETRY_TOTAL_COUNT((byte) -0x1A), ERROR_HANDLER_AUTORESUME_SUCCESS_COUNT((byte) -0x1B), + /** + * Bytes of raw data (payload) found on memtable at flush time. + * Contains the sum of garbage payload (bytes that are discarded + * at flush time) and useful payload (bytes of data that will + * eventually be written to SSTable). + */ + MEMTABLE_PAYLOAD_BYTES_AT_FLUSH((byte) -0x1C), + /** + * Outdated bytes of data present on memtable at flush time. + */ + MEMTABLE_GARBAGE_BYTES_AT_FLUSH((byte) -0x1D), + TICKER_ENUM_MAX((byte) 0x5F); private final byte value; diff --git a/monitoring/statistics.cc b/monitoring/statistics.cc index 2b690dd50..18d8eb160 100644 --- a/monitoring/statistics.cc +++ b/monitoring/statistics.cc @@ -201,6 +201,10 @@ const std::vector> TickersNameMap = { "rocksdb.error.handler.autoresume.retry.total.count"}, {ERROR_HANDLER_AUTORESUME_SUCCESS_COUNT, "rocksdb.error.handler.autoresume.success.count"}, + {MEMTABLE_PAYLOAD_BYTES_AT_FLUSH, + "rocksdb.memtable.payload.bytes.at.flush"}, + {MEMTABLE_GARBAGE_BYTES_AT_FLUSH, + "rocksdb.memtable.garbage.bytes.at.flush"}, }; const std::vector> HistogramsNameMap = {