Get `CompactionJobInfo` from CompactFiles

Summary: Pull Request resolved: https://github.com/facebook/rocksdb/pull/4716

Differential Revision: D13207677

Pulled By: ajkr

fbshipit-source-id: d0ccf5a66df6cbb07288b0c5ebad81fd9df3926b
main
DorianZheng 6 years ago committed by Facebook Github Bot
parent a8b9891f95
commit 2670fe8c73
  1. 6
      db/c.cc
  2. 5
      db/c_test.c
  3. 45
      db/compact_files_test.cc
  4. 24
      db/db_impl.h
  5. 100
      db/db_impl_compaction_flush.cc
  6. 4
      db/db_impl_readonly.h
  7. 4
      db/db_test.cc
  8. 1
      include/rocksdb/c.h
  9. 19
      include/rocksdb/db.h
  10. 4
      include/rocksdb/sst_file_reader.h
  11. 4
      include/rocksdb/table.h
  12. 13
      include/rocksdb/utilities/stackable_db.h
  13. 30
      java/rocksjni/statisticsjni.cc
  14. 17
      java/src/main/java/org/rocksdb/WriteBatchWithIndex.java
  15. 5
      java/src/test/java/org/rocksdb/WriteBatchWithIndexTest.java
  16. 11
      monitoring/statistics.cc
  17. 18
      table/sst_file_reader.cc
  18. 11
      table/sst_file_reader_test.cc
  19. 4
      tools/db_bench_tool.cc
  20. 3
      tools/ldb_cmd.cc
  21. 3
      tools/sst_dump_tool.cc
  22. 4
      utilities/transactions/transaction_base.cc

@ -1784,11 +1784,11 @@ rocksdb_iterator_t* rocksdb_writebatch_wi_create_iterator_with_base(
} }
rocksdb_iterator_t* rocksdb_writebatch_wi_create_iterator_with_base_cf( rocksdb_iterator_t* rocksdb_writebatch_wi_create_iterator_with_base_cf(
rocksdb_writebatch_wi_t* wbwi, rocksdb_writebatch_wi_t* wbwi, rocksdb_iterator_t* base_iterator,
rocksdb_iterator_t* base_iterator,
rocksdb_column_family_handle_t* column_family) { rocksdb_column_family_handle_t* column_family) {
rocksdb_iterator_t* result = new rocksdb_iterator_t; rocksdb_iterator_t* result = new rocksdb_iterator_t;
result->rep = wbwi->rep->NewIteratorWithBase(column_family->rep, base_iterator->rep); result->rep =
wbwi->rep->NewIteratorWithBase(column_family->rep, base_iterator->rep);
delete base_iterator; delete base_iterator;
return result; return result;
} }

@ -917,7 +917,8 @@ int main(int argc, char** argv) {
rocksdb_writebatch_wi_t* wbi = rocksdb_writebatch_wi_create(0, 1); rocksdb_writebatch_wi_t* wbi = rocksdb_writebatch_wi_create(0, 1);
rocksdb_writebatch_wi_put(wbi, "bar", 3, "b", 1); rocksdb_writebatch_wi_put(wbi, "bar", 3, "b", 1);
rocksdb_writebatch_wi_delete(wbi, "foo", 3); rocksdb_writebatch_wi_delete(wbi, "foo", 3);
rocksdb_iterator_t* iter = rocksdb_writebatch_wi_create_iterator_with_base(wbi, base_iter); rocksdb_iterator_t* iter =
rocksdb_writebatch_wi_create_iterator_with_base(wbi, base_iter);
CheckCondition(!rocksdb_iter_valid(iter)); CheckCondition(!rocksdb_iter_valid(iter));
rocksdb_iter_seek_to_first(iter); rocksdb_iter_seek_to_first(iter);
CheckCondition(rocksdb_iter_valid(iter)); CheckCondition(rocksdb_iter_valid(iter));
@ -1527,7 +1528,7 @@ int main(int argc, char** argv) {
const rocksdb_snapshot_t* snapshot; const rocksdb_snapshot_t* snapshot;
snapshot = rocksdb_transactiondb_create_snapshot(txn_db); snapshot = rocksdb_transactiondb_create_snapshot(txn_db);
rocksdb_readoptions_set_snapshot(roptions, snapshot); rocksdb_readoptions_set_snapshot(roptions, snapshot);
rocksdb_transactiondb_put(txn_db, woptions, "foo", 3, "hey", 3, &err); rocksdb_transactiondb_put(txn_db, woptions, "foo", 3, "hey", 3, &err);
CheckNoError(err); CheckNoError(err);

@ -357,6 +357,51 @@ TEST_F(CompactFilesTest, SentinelCompressionType) {
} }
} }
TEST_F(CompactFilesTest, GetCompactionJobInfo) {
Options options;
options.create_if_missing = true;
// Disable RocksDB background compaction.
options.compaction_style = kCompactionStyleNone;
options.level0_slowdown_writes_trigger = 1000;
options.level0_stop_writes_trigger = 1000;
options.write_buffer_size = 65536;
options.max_write_buffer_number = 2;
options.compression = kNoCompression;
options.max_compaction_bytes = 5000;
// Add listener
FlushedFileCollector* collector = new FlushedFileCollector();
options.listeners.emplace_back(collector);
DB* db = nullptr;
DestroyDB(db_name_, options);
Status s = DB::Open(options, db_name_, &db);
assert(s.ok());
assert(db);
// create couple files
for (int i = 0; i < 500; ++i) {
db->Put(WriteOptions(), ToString(i), std::string(1000, 'a' + (i % 26)));
}
reinterpret_cast<DBImpl*>(db)->TEST_WaitForFlushMemTable();
auto l0_files_1 = collector->GetFlushedFiles();
CompactionOptions co;
co.compression = CompressionType::kLZ4Compression;
CompactionJobInfo compaction_job_info;
ASSERT_OK(
db->CompactFiles(co, l0_files_1, 0, -1, nullptr, &compaction_job_info));
ASSERT_EQ(compaction_job_info.base_input_level, 0);
ASSERT_EQ(compaction_job_info.cf_id, db->DefaultColumnFamily()->GetID());
ASSERT_EQ(compaction_job_info.cf_name, db->DefaultColumnFamily()->GetName());
ASSERT_EQ(compaction_job_info.compaction_reason,
CompactionReason::kManualCompaction);
ASSERT_EQ(compaction_job_info.compression, CompressionType::kLZ4Compression);
ASSERT_EQ(compaction_job_info.output_level, 0);
ASSERT_OK(compaction_job_info.status);
// no assertion failure
delete db;
}
} // namespace rocksdb } // namespace rocksdb
int main(int argc, char** argv) { int main(int argc, char** argv) {

@ -192,13 +192,13 @@ class DBImpl : public DB {
const Slice* begin, const Slice* end) override; const Slice* begin, const Slice* end) override;
using DB::CompactFiles; using DB::CompactFiles;
virtual Status CompactFiles(const CompactionOptions& compact_options, virtual Status CompactFiles(
ColumnFamilyHandle* column_family, const CompactionOptions& compact_options,
const std::vector<std::string>& input_file_names, ColumnFamilyHandle* column_family,
const int output_level, const std::vector<std::string>& input_file_names, const int output_level,
const int output_path_id = -1, const int output_path_id = -1,
std::vector<std::string>* const output_file_names std::vector<std::string>* const output_file_names = nullptr,
= nullptr) override; CompactionJobInfo* compaction_job_info = nullptr) override;
virtual Status PauseBackgroundWork() override; virtual Status PauseBackgroundWork() override;
virtual Status ContinueBackgroundWork() override; virtual Status ContinueBackgroundWork() override;
@ -1054,7 +1054,8 @@ class DBImpl : public DB {
const std::vector<std::string>& input_file_names, const std::vector<std::string>& input_file_names,
std::vector<std::string>* const output_file_names, std::vector<std::string>* const output_file_names,
const int output_level, int output_path_id, const int output_level, int output_path_id,
JobContext* job_context, LogBuffer* log_buffer); JobContext* job_context, LogBuffer* log_buffer,
CompactionJobInfo* compaction_job_info);
// Wait for current IngestExternalFile() calls to finish. // Wait for current IngestExternalFile() calls to finish.
// REQUIRES: mutex_ held // REQUIRES: mutex_ held
@ -1572,6 +1573,13 @@ class DBImpl : public DB {
bool ShouldntRunManualCompaction(ManualCompactionState* m); bool ShouldntRunManualCompaction(ManualCompactionState* m);
bool HaveManualCompaction(ColumnFamilyData* cfd); bool HaveManualCompaction(ColumnFamilyData* cfd);
bool MCOverlap(ManualCompactionState* m, ManualCompactionState* m1); bool MCOverlap(ManualCompactionState* m, ManualCompactionState* m1);
#ifndef ROCKSDB_LITE
void BuildCompactionJobInfo(const ColumnFamilyData* cfd, Compaction* c,
const Status& st,
const CompactionJobStats& compaction_job_stats,
const int job_id, const Version* current,
CompactionJobInfo* compaction_job_info) const;
#endif
bool ShouldPurge(uint64_t file_number) const; bool ShouldPurge(uint64_t file_number) const;
void MarkAsGrabbedForPurge(uint64_t file_number); void MarkAsGrabbedForPurge(uint64_t file_number);

@ -727,7 +727,8 @@ Status DBImpl::CompactFiles(const CompactionOptions& compact_options,
ColumnFamilyHandle* column_family, ColumnFamilyHandle* column_family,
const std::vector<std::string>& input_file_names, const std::vector<std::string>& input_file_names,
const int output_level, const int output_path_id, const int output_level, const int output_path_id,
std::vector<std::string>* const output_file_names) { std::vector<std::string>* const output_file_names,
CompactionJobInfo* compaction_job_info) {
#ifdef ROCKSDB_LITE #ifdef ROCKSDB_LITE
(void)compact_options; (void)compact_options;
(void)column_family; (void)column_family;
@ -735,6 +736,7 @@ Status DBImpl::CompactFiles(const CompactionOptions& compact_options,
(void)output_level; (void)output_level;
(void)output_path_id; (void)output_path_id;
(void)output_file_names; (void)output_file_names;
(void)compaction_job_info;
// not supported in lite version // not supported in lite version
return Status::NotSupported("Not supported in ROCKSDB LITE"); return Status::NotSupported("Not supported in ROCKSDB LITE");
#else #else
@ -766,7 +768,7 @@ Status DBImpl::CompactFiles(const CompactionOptions& compact_options,
s = CompactFilesImpl(compact_options, cfd, current, input_file_names, s = CompactFilesImpl(compact_options, cfd, current, input_file_names,
output_file_names, output_level, output_path_id, output_file_names, output_level, output_path_id,
&job_context, &log_buffer); &job_context, &log_buffer, compaction_job_info);
current->Unref(); current->Unref();
} }
@ -806,7 +808,8 @@ Status DBImpl::CompactFilesImpl(
const CompactionOptions& compact_options, ColumnFamilyData* cfd, const CompactionOptions& compact_options, ColumnFamilyData* cfd,
Version* version, const std::vector<std::string>& input_file_names, Version* version, const std::vector<std::string>& input_file_names,
std::vector<std::string>* const output_file_names, const int output_level, std::vector<std::string>* const output_file_names, const int output_level,
int output_path_id, JobContext* job_context, LogBuffer* log_buffer) { int output_path_id, JobContext* job_context, LogBuffer* log_buffer,
CompactionJobInfo* compaction_job_info) {
mutex_.AssertHeld(); mutex_.AssertHeld();
if (shutting_down_.load(std::memory_order_acquire)) { if (shutting_down_.load(std::memory_order_acquire)) {
@ -892,6 +895,7 @@ Status DBImpl::CompactFilesImpl(
snapshot_checker = DisableGCSnapshotChecker::Instance(); snapshot_checker = DisableGCSnapshotChecker::Instance();
} }
assert(is_snapshot_supported_ || snapshots_.empty()); assert(is_snapshot_supported_ || snapshots_.empty());
CompactionJobStats compaction_job_stats;
CompactionJob compaction_job( CompactionJob compaction_job(
job_context->job_id, c.get(), immutable_db_options_, job_context->job_id, c.get(), immutable_db_options_,
env_options_for_compaction_, versions_.get(), &shutting_down_, env_options_for_compaction_, versions_.get(), &shutting_down_,
@ -901,19 +905,7 @@ Status DBImpl::CompactFilesImpl(
snapshot_checker, table_cache_, &event_logger_, snapshot_checker, table_cache_, &event_logger_,
c->mutable_cf_options()->paranoid_file_checks, c->mutable_cf_options()->paranoid_file_checks,
c->mutable_cf_options()->report_bg_io_stats, dbname_, c->mutable_cf_options()->report_bg_io_stats, dbname_,
nullptr); // Here we pass a nullptr for CompactionJobStats because &compaction_job_stats);
// CompactFiles does not trigger OnCompactionCompleted(),
// which is the only place where CompactionJobStats is
// returned. The idea of not triggering OnCompationCompleted()
// is that CompactFiles runs in the caller thread, so the user
// should always know when it completes. As a result, it makes
// less sense to notify the users something they should already
// know.
//
// In the future, if we would like to add CompactionJobStats
// support for CompactFiles, we should have CompactFiles API
// pass a pointer of CompactionJobStats as the out-value
// instead of using EventListener.
// Creating a compaction influences the compaction score because the score // Creating a compaction influences the compaction score because the score
// takes running compactions into account (by skipping files that are already // takes running compactions into account (by skipping files that are already
@ -950,6 +942,11 @@ Status DBImpl::CompactFilesImpl(
ReleaseFileNumberFromPendingOutputs(pending_outputs_inserted_elem); ReleaseFileNumberFromPendingOutputs(pending_outputs_inserted_elem);
if (compaction_job_info != nullptr) {
BuildCompactionJobInfo(cfd, c.get(), s, compaction_job_stats,
job_context->job_id, version, compaction_job_info);
}
if (status.ok()) { if (status.ok()) {
// Done // Done
} else if (status.IsShutdownInProgress()) { } else if (status.IsShutdownInProgress()) {
@ -1092,36 +1089,8 @@ void DBImpl::NotifyOnCompactionCompleted(
TEST_SYNC_POINT("DBImpl::NotifyOnCompactionCompleted::UnlockMutex"); TEST_SYNC_POINT("DBImpl::NotifyOnCompactionCompleted::UnlockMutex");
{ {
CompactionJobInfo info; CompactionJobInfo info;
info.cf_id = cfd->GetID(); BuildCompactionJobInfo(cfd, c, st, compaction_job_stats, job_id, current,
info.cf_name = cfd->GetName(); &info);
info.status = st;
info.thread_id = env_->GetThreadID();
info.job_id = job_id;
info.base_input_level = c->start_level();
info.output_level = c->output_level();
info.stats = compaction_job_stats;
info.table_properties = c->GetOutputTableProperties();
info.compaction_reason = c->compaction_reason();
info.compression = c->output_compression();
for (size_t i = 0; i < c->num_input_levels(); ++i) {
for (const auto fmd : *c->inputs(i)) {
auto fn = TableFileName(c->immutable_cf_options()->cf_paths,
fmd->fd.GetNumber(), fmd->fd.GetPathId());
info.input_files.push_back(fn);
if (info.table_properties.count(fn) == 0) {
std::shared_ptr<const TableProperties> tp;
auto s = current->GetTableProperties(&tp, fmd, &fn);
if (s.ok()) {
info.table_properties[fn] = tp;
}
}
}
}
for (const auto newf : c->edit()->GetNewFiles()) {
info.output_files.push_back(TableFileName(
c->immutable_cf_options()->cf_paths, newf.second.fd.GetNumber(),
newf.second.fd.GetPathId()));
}
for (auto listener : immutable_db_options_.listeners) { for (auto listener : immutable_db_options_.listeners) {
listener->OnCompactionCompleted(this, info); listener->OnCompactionCompleted(this, info);
} }
@ -2762,6 +2731,45 @@ bool DBImpl::MCOverlap(ManualCompactionState* m, ManualCompactionState* m1) {
return true; return true;
} }
#ifndef ROCKSDB_LITE
void DBImpl::BuildCompactionJobInfo(
const ColumnFamilyData* cfd, Compaction* c, const Status& st,
const CompactionJobStats& compaction_job_stats, const int job_id,
const Version* current, CompactionJobInfo* compaction_job_info) const {
assert(compaction_job_info != nullptr);
compaction_job_info->cf_id = cfd->GetID();
compaction_job_info->cf_name = cfd->GetName();
compaction_job_info->status = st;
compaction_job_info->thread_id = env_->GetThreadID();
compaction_job_info->job_id = job_id;
compaction_job_info->base_input_level = c->start_level();
compaction_job_info->output_level = c->output_level();
compaction_job_info->stats = compaction_job_stats;
compaction_job_info->table_properties = c->GetOutputTableProperties();
compaction_job_info->compaction_reason = c->compaction_reason();
compaction_job_info->compression = c->output_compression();
for (size_t i = 0; i < c->num_input_levels(); ++i) {
for (const auto fmd : *c->inputs(i)) {
auto fn = TableFileName(c->immutable_cf_options()->cf_paths,
fmd->fd.GetNumber(), fmd->fd.GetPathId());
compaction_job_info->input_files.push_back(fn);
if (compaction_job_info->table_properties.count(fn) == 0) {
shared_ptr<const TableProperties> tp;
auto s = current->GetTableProperties(&tp, fmd, &fn);
if (s.ok()) {
compaction_job_info->table_properties[fn] = tp;
}
}
}
}
for (const auto& newf : c->edit()->GetNewFiles()) {
compaction_job_info->output_files.push_back(
TableFileName(c->immutable_cf_options()->cf_paths,
newf.second.fd.GetNumber(), newf.second.fd.GetPathId()));
}
}
#endif
// SuperVersionContext gets created and destructed outside of the lock -- // SuperVersionContext gets created and destructed outside of the lock --
// we use this conveniently to: // we use this conveniently to:
// * malloc one SuperVersion() outside of the lock -- new_superversion // * malloc one SuperVersion() outside of the lock -- new_superversion

@ -77,8 +77,8 @@ class DBImplReadOnly : public DBImpl {
ColumnFamilyHandle* /*column_family*/, ColumnFamilyHandle* /*column_family*/,
const std::vector<std::string>& /*input_file_names*/, const std::vector<std::string>& /*input_file_names*/,
const int /*output_level*/, const int /*output_path_id*/ = -1, const int /*output_level*/, const int /*output_path_id*/ = -1,
std::vector<std::string>* const /*output_file_names*/ = nullptr std::vector<std::string>* const /*output_file_names*/ = nullptr,
) override { CompactionJobInfo* /*compaction_job_info*/ = nullptr) override {
return Status::NotSupported("Not supported operation in read only mode."); return Status::NotSupported("Not supported operation in read only mode.");
} }

@ -2606,8 +2606,8 @@ class ModelDB : public DB {
ColumnFamilyHandle* /*column_family*/, ColumnFamilyHandle* /*column_family*/,
const std::vector<std::string>& /*input_file_names*/, const std::vector<std::string>& /*input_file_names*/,
const int /*output_level*/, const int /*output_path_id*/ = -1, const int /*output_level*/, const int /*output_path_id*/ = -1,
std::vector<std::string>* const /*output_file_names*/ = nullptr std::vector<std::string>* const /*output_file_names*/ = nullptr,
) override { CompactionJobInfo* /*compaction_job_info*/ = nullptr) override {
return Status::NotSupported("Not supported operation."); return Status::NotSupported("Not supported operation.");
} }

@ -637,7 +637,6 @@ extern ROCKSDB_LIBRARY_API rocksdb_iterator_t* rocksdb_writebatch_wi_create_iter
rocksdb_iterator_t* base_iterator, rocksdb_iterator_t* base_iterator,
rocksdb_column_family_handle_t* cf); rocksdb_column_family_handle_t* cf);
/* Block based table options */ /* Block based table options */
extern ROCKSDB_LIBRARY_API rocksdb_block_based_table_options_t* extern ROCKSDB_LIBRARY_API rocksdb_block_based_table_options_t*

@ -53,6 +53,9 @@ class WriteBatch;
class Env; class Env;
class EventListener; class EventListener;
class TraceWriter; class TraceWriter;
#ifdef ROCKSDB_LITE
class CompactionJobInfo;
#endif
using std::unique_ptr; using std::unique_ptr;
@ -834,18 +837,20 @@ class DB {
virtual Status CompactFiles( virtual Status CompactFiles(
const CompactionOptions& compact_options, const CompactionOptions& compact_options,
ColumnFamilyHandle* column_family, ColumnFamilyHandle* column_family,
const std::vector<std::string>& input_file_names, const std::vector<std::string>& input_file_names, const int output_level,
const int output_level, const int output_path_id = -1, const int output_path_id = -1,
std::vector<std::string>* const output_file_names = nullptr) = 0; std::vector<std::string>* const output_file_names = nullptr,
CompactionJobInfo* compaction_job_info = nullptr) = 0;
virtual Status CompactFiles( virtual Status CompactFiles(
const CompactionOptions& compact_options, const CompactionOptions& compact_options,
const std::vector<std::string>& input_file_names, const std::vector<std::string>& input_file_names, const int output_level,
const int output_level, const int output_path_id = -1, const int output_path_id = -1,
std::vector<std::string>* const output_file_names = nullptr) { std::vector<std::string>* const output_file_names = nullptr,
CompactionJobInfo* compaction_job_info = nullptr) {
return CompactFiles(compact_options, DefaultColumnFamily(), return CompactFiles(compact_options, DefaultColumnFamily(),
input_file_names, output_level, output_path_id, input_file_names, output_level, output_path_id,
output_file_names); output_file_names, compaction_job_info);
} }
// This function will wait until all currently running background processes // This function will wait until all currently running background processes

@ -7,9 +7,9 @@
#ifndef ROCKSDB_LITE #ifndef ROCKSDB_LITE
#include "rocksdb/slice.h"
#include "rocksdb/options.h"
#include "rocksdb/iterator.h" #include "rocksdb/iterator.h"
#include "rocksdb/options.h"
#include "rocksdb/slice.h"
#include "rocksdb/table_properties.h" #include "rocksdb/table_properties.h"
namespace rocksdb { namespace rocksdb {

@ -454,8 +454,8 @@ class TableFactory {
// and cache the table object returned. // and cache the table object returned.
// (2) SstFileDumper (for SST Dump) opens the table and dump the table // (2) SstFileDumper (for SST Dump) opens the table and dump the table
// contents using the iterator of the table. // contents using the iterator of the table.
// (3) DBImpl::IngestExternalFile() calls this function to read the contents of // (3) DBImpl::IngestExternalFile() calls this function to read the contents
// the sst file it's attempting to add // of the sst file it's attempting to add
// //
// table_reader_options is a TableReaderOptions which contain all the // table_reader_options is a TableReaderOptions which contain all the
// needed parameters and configuration to open the table. // needed parameters and configuration to open the table.

@ -218,12 +218,13 @@ class StackableDB : public DB {
virtual Status CompactFiles( virtual Status CompactFiles(
const CompactionOptions& compact_options, const CompactionOptions& compact_options,
ColumnFamilyHandle* column_family, ColumnFamilyHandle* column_family,
const std::vector<std::string>& input_file_names, const std::vector<std::string>& input_file_names, const int output_level,
const int output_level, const int output_path_id = -1, const int output_path_id = -1,
std::vector<std::string>* const output_file_names = nullptr) override { std::vector<std::string>* const output_file_names = nullptr,
return db_->CompactFiles( CompactionJobInfo* compaction_job_info = nullptr) override {
compact_options, column_family, input_file_names, return db_->CompactFiles(compact_options, column_family, input_file_names,
output_level, output_path_id, output_file_names); output_level, output_path_id, output_file_names,
compaction_job_info);
} }
virtual Status PauseBackgroundWork() override { virtual Status PauseBackgroundWork() override {

@ -10,25 +10,23 @@
namespace rocksdb { namespace rocksdb {
StatisticsJni::StatisticsJni(std::shared_ptr<Statistics> stats) StatisticsJni::StatisticsJni(std::shared_ptr<Statistics> stats)
: StatisticsImpl(stats), m_ignore_histograms() { : StatisticsImpl(stats), m_ignore_histograms() {}
}
StatisticsJni::StatisticsJni(std::shared_ptr<Statistics> stats, StatisticsJni::StatisticsJni(std::shared_ptr<Statistics> stats,
const std::set<uint32_t> ignore_histograms) : StatisticsImpl(stats), const std::set<uint32_t> ignore_histograms)
m_ignore_histograms(ignore_histograms) { : StatisticsImpl(stats), m_ignore_histograms(ignore_histograms) {}
}
bool StatisticsJni::HistEnabledForType(uint32_t type) const { bool StatisticsJni::HistEnabledForType(uint32_t type) const {
if (type >= HISTOGRAM_ENUM_MAX) { if (type >= HISTOGRAM_ENUM_MAX) {
return false; return false;
} }
if (m_ignore_histograms.count(type) > 0) {
return false;
}
return true; if (m_ignore_histograms.count(type) > 0) {
return false;
} }
return true;
}
// @lint-ignore TXT4 T25377293 Grandfathered in // @lint-ignore TXT4 T25377293 Grandfathered in
}; };

@ -129,12 +129,10 @@ public class WriteBatchWithIndex extends AbstractWriteBatch {
public RocksIterator newIteratorWithBase( public RocksIterator newIteratorWithBase(
final ColumnFamilyHandle columnFamilyHandle, final ColumnFamilyHandle columnFamilyHandle,
final RocksIterator baseIterator) { final RocksIterator baseIterator) {
RocksIterator iterator = new RocksIterator( RocksIterator iterator = new RocksIterator(baseIterator.parent_,
baseIterator.parent_, iteratorWithBase(
iteratorWithBase(nativeHandle_, nativeHandle_, columnFamilyHandle.nativeHandle_, baseIterator.nativeHandle_));
columnFamilyHandle.nativeHandle_, // when the iterator is deleted it will also delete the baseIterator
baseIterator.nativeHandle_));
//when the iterator is deleted it will also delete the baseIterator
baseIterator.disOwnNativeHandle(); baseIterator.disOwnNativeHandle();
return iterator; return iterator;
} }
@ -151,8 +149,7 @@ public class WriteBatchWithIndex extends AbstractWriteBatch {
* point-in-timefrom baseIterator and modifications made in this write batch. * point-in-timefrom baseIterator and modifications made in this write batch.
*/ */
public RocksIterator newIteratorWithBase(final RocksIterator baseIterator) { public RocksIterator newIteratorWithBase(final RocksIterator baseIterator) {
return newIteratorWithBase(baseIterator.parent_.getDefaultColumnFamily(), return newIteratorWithBase(baseIterator.parent_.getDefaultColumnFamily(), baseIterator);
baseIterator);
} }
/** /**
@ -295,8 +292,8 @@ public class WriteBatchWithIndex extends AbstractWriteBatch {
final boolean overwriteKey); final boolean overwriteKey);
private native long iterator0(final long handle); private native long iterator0(final long handle);
private native long iterator1(final long handle, final long cfHandle); private native long iterator1(final long handle, final long cfHandle);
private native long iteratorWithBase(final long handle, private native long iteratorWithBase(
final long baseIteratorHandle, final long cfHandle); final long handle, final long baseIteratorHandle, final long cfHandle);
private native byte[] getFromBatch(final long handle, final long optHandle, private native byte[] getFromBatch(final long handle, final long optHandle,
final byte[] key, final int keyLen); final byte[] key, final int keyLen);
private native byte[] getFromBatch(final long handle, final long optHandle, private native byte[] getFromBatch(final long handle, final long optHandle,

@ -47,7 +47,6 @@ public class WriteBatchWithIndexTest {
try (final WriteBatchWithIndex wbwi = new WriteBatchWithIndex(true); try (final WriteBatchWithIndex wbwi = new WriteBatchWithIndex(true);
final RocksIterator base = db.newIterator(); final RocksIterator base = db.newIterator();
final RocksIterator it = wbwi.newIteratorWithBase(base)) { final RocksIterator it = wbwi.newIteratorWithBase(base)) {
it.seek(k1); it.seek(k1);
assertThat(it.isValid()).isTrue(); assertThat(it.isValid()).isTrue();
assertThat(it.key()).isEqualTo(k1); assertThat(it.key()).isEqualTo(k1);
@ -421,8 +420,8 @@ public class WriteBatchWithIndexTest {
final ReadOptions readOptions, final WriteBatchWithIndex wbwi, final ReadOptions readOptions, final WriteBatchWithIndex wbwi,
final String skey) { final String skey) {
final byte[] key = skey.getBytes(); final byte[] key = skey.getBytes();
try(final RocksIterator baseIterator = db.newIterator(readOptions); try (final RocksIterator baseIterator = db.newIterator(readOptions);
final RocksIterator iterator = wbwi.newIteratorWithBase(baseIterator)) { final RocksIterator iterator = wbwi.newIteratorWithBase(baseIterator)) {
iterator.seek(key); iterator.seek(key);
// Arrays.equals(key, iterator.key()) ensures an exact match in Rocks, // Arrays.equals(key, iterator.key()) ensures an exact match in Rocks,

@ -357,11 +357,12 @@ std::string StatisticsImpl::ToString() const {
getHistogramImplLocked(h.first)->Data(&hData); getHistogramImplLocked(h.first)->Data(&hData);
// don't handle failures - buffer should always be big enough and arguments // don't handle failures - buffer should always be big enough and arguments
// should be provided correctly // should be provided correctly
int ret = snprintf( int ret =
buffer, kTmpStrBufferSize, snprintf(buffer, kTmpStrBufferSize,
"%s P50 : %f P95 : %f P99 : %f P100 : %f COUNT : %" PRIu64 " SUM : %" "%s P50 : %f P95 : %f P99 : %f P100 : %f COUNT : %" PRIu64
PRIu64 "\n", h.second.c_str(), hData.median, hData.percentile95, " SUM : %" PRIu64 "\n",
hData.percentile99, hData.max, hData.count, hData.sum); h.second.c_str(), hData.median, hData.percentile95,
hData.percentile99, hData.max, hData.count, hData.sum);
if (ret < 0 || ret >= kTmpStrBufferSize) { if (ret < 0 || ret >= kTmpStrBufferSize) {
assert(false); assert(false);
continue; continue;

@ -10,8 +10,8 @@
#include "db/db_iter.h" #include "db/db_iter.h"
#include "options/cf_options.h" #include "options/cf_options.h"
#include "table/get_context.h" #include "table/get_context.h"
#include "table/table_reader.h"
#include "table/table_builder.h" #include "table/table_builder.h"
#include "table/table_reader.h"
#include "util/file_reader_writer.h" #include "util/file_reader_writer.h"
namespace rocksdb { namespace rocksdb {
@ -31,8 +31,7 @@ struct SstFileReader::Rep {
moptions(ColumnFamilyOptions(options)) {} moptions(ColumnFamilyOptions(options)) {}
}; };
SstFileReader::SstFileReader(const Options& options) SstFileReader::SstFileReader(const Options& options) : rep_(new Rep(options)) {}
: rep_(new Rep(options)) {}
SstFileReader::~SstFileReader() {} SstFileReader::~SstFileReader() {}
@ -60,18 +59,19 @@ Status SstFileReader::Open(const std::string& file_path) {
Iterator* SstFileReader::NewIterator(const ReadOptions& options) { Iterator* SstFileReader::NewIterator(const ReadOptions& options) {
auto r = rep_.get(); auto r = rep_.get();
auto sequence = options.snapshot != nullptr ? auto sequence = options.snapshot != nullptr
options.snapshot->GetSequenceNumber() : ? options.snapshot->GetSequenceNumber()
kMaxSequenceNumber; : kMaxSequenceNumber;
auto internal_iter = r->table_reader->NewIterator( auto internal_iter =
options, r->moptions.prefix_extractor.get()); r->table_reader->NewIterator(options, r->moptions.prefix_extractor.get());
return NewDBIterator(r->options.env, options, r->ioptions, r->moptions, return NewDBIterator(r->options.env, options, r->ioptions, r->moptions,
r->ioptions.user_comparator, internal_iter, sequence, r->ioptions.user_comparator, internal_iter, sequence,
r->moptions.max_sequential_skip_in_iterations, r->moptions.max_sequential_skip_in_iterations,
nullptr /* read_callback */); nullptr /* read_callback */);
} }
std::shared_ptr<const TableProperties> SstFileReader::GetTableProperties() const { std::shared_ptr<const TableProperties> SstFileReader::GetTableProperties()
const {
return rep_->table_reader->GetTableProperties(); return rep_->table_reader->GetTableProperties();
} }

@ -39,8 +39,8 @@ class SstFileReaderTest : public testing::Test {
ASSERT_OK(writer.Open(sst_name_)); ASSERT_OK(writer.Open(sst_name_));
for (size_t i = 0; i + 2 < keys.size(); i += 3) { for (size_t i = 0; i + 2 < keys.size(); i += 3) {
ASSERT_OK(writer.Put(keys[i], keys[i])); ASSERT_OK(writer.Put(keys[i], keys[i]));
ASSERT_OK(writer.Merge(keys[i+1], EncodeAsUint64(i+1))); ASSERT_OK(writer.Merge(keys[i + 1], EncodeAsUint64(i + 1)));
ASSERT_OK(writer.Delete(keys[i+2])); ASSERT_OK(writer.Delete(keys[i + 2]));
} }
ASSERT_OK(writer.Finish()); ASSERT_OK(writer.Finish());
@ -56,8 +56,8 @@ class SstFileReaderTest : public testing::Test {
ASSERT_EQ(iter->value().compare(keys[i]), 0); ASSERT_EQ(iter->value().compare(keys[i]), 0);
iter->Next(); iter->Next();
ASSERT_TRUE(iter->Valid()); ASSERT_TRUE(iter->Valid());
ASSERT_EQ(iter->key().compare(keys[i+1]), 0); ASSERT_EQ(iter->key().compare(keys[i + 1]), 0);
ASSERT_EQ(iter->value().compare(EncodeAsUint64(i+1)), 0); ASSERT_EQ(iter->value().compare(EncodeAsUint64(i + 1)), 0);
iter->Next(); iter->Next();
} }
ASSERT_FALSE(iter->Valid()); ASSERT_FALSE(iter->Valid());
@ -99,7 +99,8 @@ int main(int argc, char** argv) {
#include <stdio.h> #include <stdio.h>
int main(int /*argc*/, char** /*argv*/) { int main(int /*argc*/, char** /*argv*/) {
fprintf(stderr, "SKIPPED as SstFileReader is not supported in ROCKSDB_LITE\n"); fprintf(stderr,
"SKIPPED as SstFileReader is not supported in ROCKSDB_LITE\n");
return 0; return 0;
} }

@ -4580,8 +4580,8 @@ void VerifyDBFromDB(std::string& truth_db_name) {
if (FLAGS_max_scan_distance != 0) { if (FLAGS_max_scan_distance != 0) {
if (FLAGS_reverse_iterator) { if (FLAGS_reverse_iterator) {
GenerateKeyFromInt( GenerateKeyFromInt(
(uint64_t)std::max((int64_t)0, static_cast<uint64_t>(std::max(
seek_pos - FLAGS_max_scan_distance), static_cast<int64_t>(0), seek_pos - FLAGS_max_scan_distance)),
FLAGS_num, &lower_bound); FLAGS_num, &lower_bound);
options.iterate_lower_bound = &lower_bound; options.iterate_lower_bound = &lower_bound;
} else { } else {

@ -2846,7 +2846,8 @@ void DumpSstFile(std::string filename, bool output_hex, bool show_properties) {
} }
// no verification // no verification
rocksdb::SstFileDumper dumper(filename, false, output_hex); rocksdb::SstFileDumper dumper(filename, false, output_hex);
Status st = dumper.ReadSequential(true, std::numeric_limits<uint64_t>::max(), false, // has_from Status st = dumper.ReadSequential(true, std::numeric_limits<uint64_t>::max(),
false, // has_from
from_key, false, // has_to from_key, false, // has_to
to_key); to_key);
if (!st.ok()) { if (!st.ok()) {

@ -570,8 +570,7 @@ int SSTDumpTool::Run(int argc, char** argv) {
filename = std::string(dir_or_file) + "/" + filename; filename = std::string(dir_or_file) + "/" + filename;
} }
rocksdb::SstFileDumper dumper(filename, verify_checksum, rocksdb::SstFileDumper dumper(filename, verify_checksum, output_hex);
output_hex);
if (!dumper.getStatus().ok()) { if (!dumper.getStatus().ok()) {
fprintf(stderr, "%s: %s\n", filename.c_str(), fprintf(stderr, "%s: %s\n", filename.c_str(),
dumper.getStatus().ToString().c_str()); dumper.getStatus().ToString().c_str());

@ -180,7 +180,7 @@ Status TransactionBaseImpl::RollbackToSavePoint() {
return Status::NotFound(); return Status::NotFound();
} }
} }
Status TransactionBaseImpl::PopSavePoint() { Status TransactionBaseImpl::PopSavePoint() {
if (save_points_ == nullptr || if (save_points_ == nullptr ||
save_points_->empty()) { save_points_->empty()) {
@ -189,7 +189,7 @@ Status TransactionBaseImpl::PopSavePoint() {
return Status::NotFound(); return Status::NotFound();
} }
assert(!save_points_->empty()); assert(!save_points_->empty());
save_points_->pop(); save_points_->pop();
return write_batch_.PopSavePoint(); return write_batch_.PopSavePoint();
} }

Loading…
Cancel
Save