Remove iter_start_seqnum and preserve_deletes (#9430)

Summary:
According to https://github.com/facebook/rocksdb/blob/6.27.fb/db/db_impl/db_impl.cc#L2896:L2911 and https://github.com/facebook/rocksdb/blob/6.27.fb/db/db_impl/db_impl_open.cc#L203:L208,
we are going to remove `iter_start_seqnum` and `preserve_deletes` starting from RocksDB 7.0

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

Test Plan: make check and CI

Reviewed By: ajkr

Differential Revision: D33753639

Pulled By: riversand963

fbshipit-source-id: c80aab8e8d8fc33e52472fed524ed703d0ffc8b6
main
Yanqin Jin 2 years ago committed by Facebook GitHub Bot
parent e58cc56fb5
commit d10c5c08d3
  1. 2
      HISTORY.md
  2. 4
      db/compaction/compaction_iterator.h
  3. 83
      db/db_compaction_test.cc
  4. 32
      db/db_impl/db_impl.cc
  5. 16
      db/db_impl/db_impl.h
  6. 7
      db/db_impl/db_impl_open.cc
  7. 2
      db/db_iter.cc
  8. 78
      db/db_iter_test.cc
  9. 8
      db/db_properties_test.cc
  10. 4
      db/db_test.cc
  11. 4
      db/db_test_util.cc
  12. 2
      db/db_test_util.h
  13. 105
      db/db_with_timestamp_basic_test.cc
  14. 8
      include/rocksdb/db.h
  15. 10
      include/rocksdb/options.h
  16. 5
      include/rocksdb/utilities/stackable_db.h
  17. 66
      java/rocksjni/options.cc
  18. 25
      java/rocksjni/rocksjni.cc
  19. 16
      java/src/main/java/org/rocksdb/DBOptions.java
  20. 26
      java/src/main/java/org/rocksdb/DBOptionsInterface.java
  21. 16
      java/src/main/java/org/rocksdb/Options.java
  22. 34
      java/src/main/java/org/rocksdb/ReadOptions.java
  23. 21
      java/src/main/java/org/rocksdb/RocksDB.java
  24. 9
      java/src/test/java/org/rocksdb/DBOptionsTest.java
  25. 9
      java/src/test/java/org/rocksdb/OptionsTest.java
  26. 10
      java/src/test/java/org/rocksdb/ReadOptionsTest.java
  27. 11
      java/src/test/java/org/rocksdb/RocksDBTest.java
  28. 6
      options/db_options.cc
  29. 1
      options/db_options.h
  30. 2
      options/options.cc
  31. 5
      options/options_helper.cc
  32. 1
      options/options_settable_test.cc
  33. 2
      options/options_test.cc

@ -17,6 +17,8 @@
* Remove deprecated API DBOptions::purge_redundant_kvs_while_flush.
* Remove deprecated overloads of API DB::CompactRange.
* Remove deprecated option DBOptions::skip_log_error_on_recovery.
* Remove ReadOptions::iter_start_seqnum which has been deprecated.
* Remove DBOptions::preserved_deletes and DB::SetPreserveDeletesSequenceNumber().
### Behavior Changes
* Disallow the combination of DBOptions.use_direct_io_for_flush_and_compaction == true and DBOptions.writable_file_max_buffer_size == 0. This combination can cause WritableFileWriter::Append() to loop forever, and it does not make much sense in direct IO.

@ -139,9 +139,7 @@ class CompactionIterator {
return compaction_->immutable_options()->allow_ingest_behind;
}
bool preserve_deletes() const override {
return compaction_->immutable_options()->preserve_deletes;
}
bool preserve_deletes() const override { return false; }
bool allow_mmap_reads() const override {
return compaction_->immutable_options()->allow_mmap_reads;

@ -369,89 +369,6 @@ TEST_P(DBCompactionTestWithParam, CompactionDeletionTrigger) {
}
#endif // !defined(ROCKSDB_VALGRIND_RUN) || defined(ROCKSDB_FULL_VALGRIND_RUN)
TEST_P(DBCompactionTestWithParam, CompactionsPreserveDeletes) {
// For each options type we test following
// - Enable preserve_deletes
// - write bunch of keys and deletes
// - Set start_seqnum to the beginning; compact; check that keys are present
// - rewind start_seqnum way forward; compact; check that keys are gone
for (int tid = 0; tid < 3; ++tid) {
Options options = DeletionTriggerOptions(CurrentOptions());
options.max_subcompactions = max_subcompactions_;
options.preserve_deletes=true;
options.num_levels = 2;
if (tid == 1) {
options.skip_stats_update_on_db_open = true;
} else if (tid == 2) {
// third pass with universal compaction
options.compaction_style = kCompactionStyleUniversal;
}
DestroyAndReopen(options);
Random rnd(301);
// highlight the default; all deletes should be preserved
SetPreserveDeletesSequenceNumber(0);
const int kTestSize = kCDTKeysPerBuffer;
std::vector<std::string> values;
for (int k = 0; k < kTestSize; ++k) {
values.push_back(rnd.RandomString(kCDTValueSize));
ASSERT_OK(Put(Key(k), values[k]));
}
for (int k = 0; k < kTestSize; ++k) {
ASSERT_OK(Delete(Key(k)));
}
// to ensure we tackle all tombstones
CompactRangeOptions cro;
cro.change_level = true;
cro.target_level = 2;
cro.bottommost_level_compaction =
BottommostLevelCompaction::kForceOptimized;
ASSERT_OK(dbfull()->TEST_WaitForFlushMemTable());
ASSERT_TRUE(
dbfull()->CompactRange(cro, nullptr, nullptr).IsInvalidArgument());
// check that normal user iterator doesn't see anything
Iterator* db_iter = dbfull()->NewIterator(ReadOptions());
int i = 0;
for (db_iter->SeekToFirst(); db_iter->Valid(); db_iter->Next()) {
i++;
}
ASSERT_OK(db_iter->status());
ASSERT_EQ(i, 0);
delete db_iter;
// check that iterator that sees internal keys sees tombstones
ReadOptions ro;
ro.iter_start_seqnum=1;
db_iter = dbfull()->NewIterator(ro);
ASSERT_OK(db_iter->status());
i = 0;
for (db_iter->SeekToFirst(); db_iter->Valid(); db_iter->Next()) {
i++;
}
ASSERT_EQ(i, 4);
delete db_iter;
// now all deletes should be gone
SetPreserveDeletesSequenceNumber(100000000);
ASSERT_NOK(dbfull()->CompactRange(cro, nullptr, nullptr));
db_iter = dbfull()->NewIterator(ro);
ASSERT_TRUE(db_iter->status().IsInvalidArgument());
i = 0;
for (db_iter->SeekToFirst(); db_iter->Valid(); db_iter->Next()) {
i++;
}
ASSERT_EQ(i, 0);
delete db_iter;
}
}
TEST_F(DBCompactionTest, SkipStatsUpdateTest) {
// This test verify UpdateAccumulatedStats is not on
// if options.skip_stats_update_on_db_open = true

@ -234,7 +234,6 @@ DBImpl::DBImpl(const DBOptions& options, const std::string& dbname,
use_custom_gc_(seq_per_batch),
shutdown_initiated_(false),
own_sfm_(options.sst_file_manager == nullptr),
preserve_deletes_(options.preserve_deletes),
closed_(false),
atomic_flush_install_cv_(&mutex_),
blob_callback_(immutable_db_options_.sst_file_manager.get(), &mutex_,
@ -272,11 +271,6 @@ DBImpl::DBImpl(const DBOptions& options, const std::string& dbname,
mutable_db_options_.Dump(immutable_db_options_.info_log.get());
DumpSupportInfo(immutable_db_options_.info_log.get());
// always open the DB with 0 here, which means if preserve_deletes_==true
// we won't drop any deletion markers until SetPreserveDeletesSequenceNumber()
// is called by client and this seqnum is advanced.
preserve_deletes_seqnum_.store(0);
if (write_buffer_manager_) {
wbm_stall_.reset(new WBMStallInterface());
}
@ -1487,15 +1481,6 @@ void DBImpl::SetLastPublishedSequence(SequenceNumber seq) {
versions_->SetLastPublishedSequence(seq);
}
bool DBImpl::SetPreserveDeletesSequenceNumber(SequenceNumber seqnum) {
if (seqnum > preserve_deletes_seqnum_.load()) {
preserve_deletes_seqnum_.store(seqnum);
return true;
} else {
return false;
}
}
Status DBImpl::GetFullHistoryTsLow(ColumnFamilyHandle* column_family,
std::string* ts_low) {
if (ts_low == nullptr) {
@ -2917,22 +2902,7 @@ Iterator* DBImpl::NewIterator(const ReadOptions& read_options,
return NewErrorIterator(Status::NotSupported(
"ReadTier::kPersistedData is not yet supported in iterators."));
}
// if iterator wants internal keys, we can only proceed if
// we can guarantee the deletes haven't been processed yet
if (read_options.iter_start_seqnum > 0 &&
!iter_start_seqnum_deprecation_warned_.exchange(true)) {
ROCKS_LOG_WARN(
immutable_db_options_.info_log,
"iter_start_seqnum is deprecated, will be removed in a future release. "
"Please try using user-defined timestamp instead.");
}
if (immutable_db_options_.preserve_deletes &&
read_options.iter_start_seqnum > 0 &&
read_options.iter_start_seqnum < preserve_deletes_seqnum_.load()) {
return NewErrorIterator(Status::InvalidArgument(
"Iterator requested internal keys which are too old and are not"
" guaranteed to be preserved, try larger iter_start_seqnum opt."));
}
auto cfh = static_cast_with_check<ColumnFamilyHandleImpl>(column_family);
ColumnFamilyData* cfd = cfh->cfd();
assert(cfd != nullptr);

@ -353,8 +353,6 @@ class DBImpl : public DB {
virtual SequenceNumber GetLatestSequenceNumber() const override;
virtual bool SetPreserveDeletesSequenceNumber(SequenceNumber seqnum) override;
// IncreaseFullHistoryTsLow(ColumnFamilyHandle*, std::string) will acquire
// and release db_mutex
Status IncreaseFullHistoryTsLow(ColumnFamilyHandle* column_family,
@ -2306,12 +2304,10 @@ class DBImpl : public DB {
// DB::Open() or passed to us
bool own_sfm_;
// Clients must periodically call SetPreserveDeletesSequenceNumber()
// to advance this seqnum. Default value is 0 which means ALL deletes are
// preserved. Note that this has no effect if DBOptions.preserve_deletes
// is set to false.
std::atomic<SequenceNumber> preserve_deletes_seqnum_;
const bool preserve_deletes_;
// Default value is 0 which means ALL deletes are
// preserved. Note that this has no effect if preserve_deletes is false.
const std::atomic<SequenceNumber> preserve_deletes_seqnum_{0};
const bool preserve_deletes_ = false;
// Flag to check whether Close() has been called on this DB
bool closed_;
@ -2337,10 +2333,6 @@ class DBImpl : public DB {
// Pointer to WriteBufferManager stalling interface.
std::unique_ptr<StallInterface> wbm_stall_;
// Indicate if deprecation warning message is logged before. Will be removed
// soon with the deprecated feature.
std::atomic_bool iter_start_seqnum_deprecation_warned_{false};
};
extern Options SanitizeOptions(const std::string& db, const Options& src,

@ -208,13 +208,6 @@ DBOptions SanitizeOptions(const std::string& dbname, const DBOptions& src,
"file size check will be skipped during open.");
}
if (result.preserve_deletes) {
ROCKS_LOG_WARN(
result.info_log,
"preserve_deletes is deprecated, will be removed in a future release. "
"Please try using user-defined timestamp instead.");
}
return result;
}

@ -78,7 +78,7 @@ DBIter::DBIter(Env* _env, const ReadOptions& read_options,
range_del_agg_(&ioptions.internal_comparator, s),
db_impl_(db_impl),
cfd_(cfd),
start_seqnum_(read_options.iter_start_seqnum),
start_seqnum_(0ULL),
timestamp_ub_(read_options.timestamp),
timestamp_lb_(read_options.iter_start_ts),
timestamp_size_(timestamp_ub_ ? timestamp_ub_->size() : 0) {

@ -2562,84 +2562,6 @@ TEST_F(DBIteratorTest, DBIterator14) {
ASSERT_EQ(db_iter->value().ToString(), "4");
}
TEST_F(DBIteratorTest, DBIteratorTestDifferentialSnapshots) {
{ // test that KVs earlier that iter_start_seqnum are filtered out
ReadOptions ro;
ro.iter_start_seqnum=5;
Options options;
options.statistics = ROCKSDB_NAMESPACE::CreateDBStatistics();
TestIterator* internal_iter = new TestIterator(BytewiseComparator());
for (size_t i = 0; i < 10; ++i) {
internal_iter->AddPut(std::to_string(i), std::to_string(i) + "a");
internal_iter->AddPut(std::to_string(i), std::to_string(i) + "b");
internal_iter->AddPut(std::to_string(i), std::to_string(i) + "c");
}
internal_iter->Finish();
std::unique_ptr<Iterator> db_iter(NewDBIterator(
env_, ro, ImmutableOptions(options), MutableCFOptions(options),
BytewiseComparator(), internal_iter, nullptr /* version */,
13 /* sequence */, options.max_sequential_skip_in_iterations,
nullptr /* read_callback */));
// Expecting InternalKeys in [5,8] range with correct type
int seqnums[4] = {5,8,11,13};
std::string user_keys[4] = {"1","2","3","4"};
std::string values[4] = {"1c", "2c", "3c", "4b"};
int i = 0;
for (db_iter->SeekToFirst(); db_iter->Valid(); db_iter->Next()) {
ParsedInternalKey fkey;
ASSERT_OK(
ParseInternalKey(db_iter->key(), &fkey, true /* log_err_key */));
ASSERT_EQ(user_keys[i], fkey.user_key.ToString());
ASSERT_EQ(kTypeValue, fkey.type);
ASSERT_EQ(seqnums[i], fkey.sequence);
ASSERT_EQ(values[i], db_iter->value().ToString());
i++;
}
ASSERT_EQ(i, 4);
}
{ // Test that deletes are returned correctly as internal KVs
ReadOptions ro;
ro.iter_start_seqnum=5;
Options options;
options.statistics = ROCKSDB_NAMESPACE::CreateDBStatistics();
TestIterator* internal_iter = new TestIterator(BytewiseComparator());
for (size_t i = 0; i < 10; ++i) {
internal_iter->AddPut(std::to_string(i), std::to_string(i) + "a");
internal_iter->AddPut(std::to_string(i), std::to_string(i) + "b");
internal_iter->AddDeletion(std::to_string(i));
}
internal_iter->Finish();
std::unique_ptr<Iterator> db_iter(NewDBIterator(
env_, ro, ImmutableOptions(options), MutableCFOptions(options),
BytewiseComparator(), internal_iter, nullptr /* version */,
13 /* sequence */, options.max_sequential_skip_in_iterations,
nullptr /* read_callback */));
// Expecting InternalKeys in [5,8] range with correct type
int seqnums[4] = {5,8,11,13};
ValueType key_types[4] = {kTypeDeletion, kTypeDeletion, kTypeDeletion,
kTypeValue};
std::string user_keys[4] = {"1","2","3","4"};
std::string values[4] = {"", "", "", "4b"};
int i = 0;
for (db_iter->SeekToFirst(); db_iter->Valid(); db_iter->Next()) {
ParsedInternalKey fkey;
ASSERT_OK(
ParseInternalKey(db_iter->key(), &fkey, true /* log_err_key */));
ASSERT_EQ(user_keys[i], fkey.user_key.ToString());
ASSERT_EQ(key_types[i], fkey.type);
ASSERT_EQ(seqnums[i], fkey.sequence);
ASSERT_EQ(values[i], db_iter->value().ToString());
i++;
}
ASSERT_EQ(i, 4);
}
}
class DBIterWithMergeIterTest : public testing::Test {
public:
DBIterWithMergeIterTest()

@ -337,7 +337,7 @@ TEST_F(DBPropertiesTest, ValidateSampleNumber) {
TEST_F(DBPropertiesTest, AggregatedTableProperties) {
for (int kTableCount = 40; kTableCount <= 100; kTableCount += 30) {
const int kDeletionsPerTable = 5;
const int kDeletionsPerTable = 0;
const int kMergeOperandsPerTable = 15;
const int kRangeDeletionsPerTable = 5;
const int kPutsPerTable = 100;
@ -349,7 +349,6 @@ TEST_F(DBPropertiesTest, AggregatedTableProperties) {
options.level0_file_num_compaction_trigger = 8;
options.compression = kNoCompression;
options.create_if_missing = true;
options.preserve_deletes = true;
options.merge_operator.reset(new TestPutOperator());
BlockBasedTableOptions table_options;
@ -530,7 +529,7 @@ TEST_F(DBPropertiesTest, ReadLatencyHistogramByLevel) {
TEST_F(DBPropertiesTest, AggregatedTablePropertiesAtLevel) {
const int kTableCount = 100;
const int kDeletionsPerTable = 2;
const int kDeletionsPerTable = 0;
const int kMergeOperandsPerTable = 2;
const int kRangeDeletionsPerTable = 2;
const int kPutsPerTable = 10;
@ -549,7 +548,6 @@ TEST_F(DBPropertiesTest, AggregatedTablePropertiesAtLevel) {
options.max_bytes_for_level_multiplier = 2;
// This ensures there no compaction happening when we call GetProperty().
options.disable_auto_compactions = true;
options.preserve_deletes = true;
options.merge_operator.reset(new TestPutOperator());
BlockBasedTableOptions table_options;
@ -628,7 +626,7 @@ TEST_F(DBPropertiesTest, AggregatedTablePropertiesAtLevel) {
// Gives larger bias here as index block size, filter block size,
// and data block size become much harder to estimate in this test.
VerifyTableProperties(expected_tp, tp, CACHE_LINE_SIZE >= 256 ? 0.6 : 0.5,
0.4, 0.4, 0.25);
0.5, 0.5, 0.25);
}
}
}

@ -3199,10 +3199,6 @@ class ModelDB : public DB {
SequenceNumber GetLatestSequenceNumber() const override { return 0; }
bool SetPreserveDeletesSequenceNumber(SequenceNumber /*seqnum*/) override {
return true;
}
Status IncreaseFullHistoryTsLow(ColumnFamilyHandle* /*cf*/,
std::string /*ts_low*/) override {
return Status::OK();

@ -786,10 +786,6 @@ Status DBTestBase::SingleDelete(int cf, const std::string& k) {
return db_->SingleDelete(WriteOptions(), handles_[cf], k);
}
bool DBTestBase::SetPreserveDeletesSequenceNumber(SequenceNumber sn) {
return db_->SetPreserveDeletesSequenceNumber(sn);
}
std::string DBTestBase::Get(const std::string& k, const Snapshot* snapshot) {
ReadOptions options;
options.verify_checksums = true;

@ -1022,8 +1022,6 @@ class DBTestBase : public testing::Test {
Status SingleDelete(int cf, const std::string& k);
bool SetPreserveDeletesSequenceNumber(SequenceNumber sn);
std::string Get(const std::string& k, const Snapshot* snapshot = nullptr);
std::string Get(int cf, const std::string& k,

@ -1108,111 +1108,6 @@ TEST_F(DBBasicTestWithTimestamp, SimpleForwardIterateLowerTsBound) {
Close();
}
class DBBasicDeletionTestWithTimestamp
: public DBBasicTestWithTimestampBase,
public testing::WithParamInterface<enum ValueType> {
public:
DBBasicDeletionTestWithTimestamp()
: DBBasicTestWithTimestampBase("db_basic_deletion_test_with_timestamp") {}
};
INSTANTIATE_TEST_CASE_P(
Timestamp, DBBasicDeletionTestWithTimestamp,
::testing::Values(ValueType::kTypeSingleDeletion,
ValueType::kTypeDeletionWithTimestamp));
TEST_P(DBBasicDeletionTestWithTimestamp, ForwardIterateStartSeqnum) {
const int kNumKeysPerFile = 128;
const uint64_t kMaxKey = 0xffffffffffffffff;
const uint64_t kMinKey = kMaxKey - 1023;
Options options = CurrentOptions();
options.env = env_;
options.create_if_missing = true;
ValueType op_type = GetParam();
// Need to disable compaction to bottommost level when sequence number will be
// zeroed out, causing the verification of sequence number to fail in this
// test.
options.disable_auto_compactions = true;
const size_t kTimestampSize = Timestamp(0, 0).size();
TestComparator test_cmp(kTimestampSize);
options.comparator = &test_cmp;
options.memtable_factory.reset(
test::NewSpecialSkipListFactory(kNumKeysPerFile));
DestroyAndReopen(options);
std::vector<SequenceNumber> start_seqs;
const int kNumTimestamps = 4;
std::vector<std::string> write_ts_list;
for (int t = 0; t != kNumTimestamps; ++t) {
write_ts_list.push_back(Timestamp(2 * t, /*do not care*/ 17));
}
WriteOptions write_opts;
for (size_t i = 0; i != write_ts_list.size(); ++i) {
Slice write_ts = write_ts_list[i];
write_opts.timestamp = &write_ts;
for (uint64_t k = kMaxKey; k >= kMinKey; --k) {
Status s;
if (k % 2) {
s = db_->Put(write_opts, Key1(k), "value" + std::to_string(i));
} else {
if (op_type == ValueType::kTypeDeletionWithTimestamp) {
s = db_->Delete(write_opts, Key1(k));
} else if (op_type == ValueType::kTypeSingleDeletion) {
s = db_->SingleDelete(write_opts, Key1(k));
}
}
ASSERT_OK(s);
}
start_seqs.push_back(db_->GetLatestSequenceNumber());
}
std::vector<std::string> read_ts_list;
for (int t = 0; t != kNumTimestamps - 1; ++t) {
read_ts_list.push_back(Timestamp(2 * t + 3, /*do not care*/ 17));
}
ReadOptions read_opts;
// Scan with only read_opts.iter_start_seqnum set.
for (size_t i = 0; i != read_ts_list.size(); ++i) {
Slice read_ts = read_ts_list[i];
read_opts.timestamp = &read_ts;
read_opts.iter_start_seqnum = start_seqs[i] + 1;
std::unique_ptr<Iterator> iter(db_->NewIterator(read_opts));
SequenceNumber expected_seq = start_seqs[i] + (kMaxKey - kMinKey) + 1;
uint64_t key = kMinKey;
for (iter->Seek(Key1(kMinKey)); iter->Valid(); iter->Next()) {
CheckIterEntry(
iter.get(), Key1(key), expected_seq, (key % 2) ? kTypeValue : op_type,
(key % 2) ? "value" + std::to_string(i + 1) : std::string(),
write_ts_list[i + 1]);
++key;
--expected_seq;
}
}
// Scan with both read_opts.iter_start_seqnum and read_opts.iter_start_ts set.
std::vector<std::string> read_ts_lb_list;
for (int t = 0; t < kNumTimestamps - 1; ++t) {
read_ts_lb_list.push_back(Timestamp(2 * t, /*do not care*/ 17));
}
for (size_t i = 0; i < read_ts_list.size(); ++i) {
Slice read_ts = read_ts_list[i];
Slice read_ts_lb = read_ts_lb_list[i];
read_opts.timestamp = &read_ts;
read_opts.iter_start_ts = &read_ts_lb;
read_opts.iter_start_seqnum = start_seqs[i] + 1;
std::unique_ptr<Iterator> it(db_->NewIterator(read_opts));
uint64_t key = kMinKey;
SequenceNumber expected_seq = start_seqs[i] + (kMaxKey - kMinKey) + 1;
for (it->Seek(Key1(kMinKey)); it->Valid(); it->Next()) {
CheckIterEntry(it.get(), Key1(key), expected_seq,
(key % 2) ? kTypeValue : op_type,
"value" + std::to_string(i + 1), write_ts_list[i + 1]);
++key;
--expected_seq;
}
}
Close();
}
TEST_F(DBBasicTestWithTimestamp, ReseekToTargetTimestamp) {
Options options = CurrentOptions();
options.env = env_;

@ -1326,14 +1326,6 @@ class DB {
// The sequence number of the most recent transaction.
virtual SequenceNumber GetLatestSequenceNumber() const = 0;
// Instructs DB to preserve deletes with sequence numbers >= passed seqnum.
// Has no effect if DBOptions.preserve_deletes is set to false.
// This function assumes that user calls this function with monotonically
// increasing seqnums (otherwise we can't guarantee that a particular delete
// hasn't been already processed); returns true if the value was successfully
// updated, false if user attempted to call if with seqnum <= current value.
virtual bool SetPreserveDeletesSequenceNumber(SequenceNumber seqnum) = 0;
// Prevent file deletions. Compactions will continue to occur,
// but no obsolete files will be deleted. Calling this multiple
// times have the same effect as calling it once.

@ -1199,11 +1199,6 @@ struct DBOptions {
// Immutable.
bool allow_ingest_behind = false;
// Deprecated, will be removed in a future release.
// Please try using user-defined timestamp instead.
// DEFAULT: false
bool preserve_deletes = false;
// If enabled it uses two queues for writes, one for the ones with
// disable_memtable and one for the ones that also write to memtable. This
// allows the memtable writes not to lag behind other writes. It can be used
@ -1555,11 +1550,6 @@ struct ReadOptions {
// Default: empty (every table will be scanned)
std::function<bool(const TableProperties&)> table_filter;
// Deprecated, will be removed in a future release.
// Please try using user-defined timestamp instead.
// Default: 0 (don't filter by seqnum, return user keys)
SequenceNumber iter_start_seqnum;
// Timestamp of operation. Read should return the latest data visible to the
// specified timestamp. All timestamps of the same database must be of the
// same length and format. The user is responsible for providing a customized

@ -414,11 +414,6 @@ class StackableDB : public DB {
return db_->GetLatestSequenceNumber();
}
virtual bool SetPreserveDeletesSequenceNumber(
SequenceNumber seqnum) override {
return db_->SetPreserveDeletesSequenceNumber(seqnum);
}
Status IncreaseFullHistoryTsLow(ColumnFamilyHandle* column_family,
std::string ts_low) override {
return db_->IncreaseFullHistoryTsLow(column_family, ts_low);

@ -2387,28 +2387,6 @@ jboolean Java_org_rocksdb_Options_allowIngestBehind(
return static_cast<jboolean>(opt->allow_ingest_behind);
}
/*
* Class: org_rocksdb_Options
* Method: setPreserveDeletes
* Signature: (JZ)V
*/
void Java_org_rocksdb_Options_setPreserveDeletes(
JNIEnv*, jobject, jlong jhandle, jboolean jpreserve_deletes) {
auto* opt = reinterpret_cast<ROCKSDB_NAMESPACE::Options*>(jhandle);
opt->preserve_deletes = jpreserve_deletes == JNI_TRUE;
}
/*
* Class: org_rocksdb_Options
* Method: preserveDeletes
* Signature: (J)Z
*/
jboolean Java_org_rocksdb_Options_preserveDeletes(
JNIEnv*, jobject, jlong jhandle) {
auto* opt = reinterpret_cast<ROCKSDB_NAMESPACE::Options*>(jhandle);
return static_cast<jboolean>(opt->preserve_deletes);
}
/*
* Class: org_rocksdb_Options
* Method: setTwoWriteQueues
@ -7384,28 +7362,6 @@ jboolean Java_org_rocksdb_DBOptions_allowIngestBehind(
return static_cast<jboolean>(opt->allow_ingest_behind);
}
/*
* Class: org_rocksdb_DBOptions
* Method: setPreserveDeletes
* Signature: (JZ)V
*/
void Java_org_rocksdb_DBOptions_setPreserveDeletes(
JNIEnv*, jobject, jlong jhandle, jboolean jpreserve_deletes) {
auto* opt = reinterpret_cast<ROCKSDB_NAMESPACE::DBOptions*>(jhandle);
opt->preserve_deletes = jpreserve_deletes == JNI_TRUE;
}
/*
* Class: org_rocksdb_DBOptions
* Method: preserveDeletes
* Signature: (J)Z
*/
jboolean Java_org_rocksdb_DBOptions_preserveDeletes(
JNIEnv*, jobject, jlong jhandle) {
auto* opt = reinterpret_cast<ROCKSDB_NAMESPACE::DBOptions*>(jhandle);
return static_cast<jboolean>(opt->preserve_deletes);
}
/*
* Class: org_rocksdb_DBOptions
* Method: setTwoWriteQueues
@ -8200,28 +8156,6 @@ void Java_org_rocksdb_ReadOptions_setTableFilter(
opt->table_filter = jni_table_filter->GetTableFilterFunction();
}
/*
* Class: org_rocksdb_ReadOptions
* Method: setIterStartSeqnum
* Signature: (JJ)V
*/
void Java_org_rocksdb_ReadOptions_setIterStartSeqnum(
JNIEnv*, jobject, jlong jhandle, jlong jiter_start_seqnum) {
auto* opt = reinterpret_cast<ROCKSDB_NAMESPACE::ReadOptions*>(jhandle);
opt->iter_start_seqnum = static_cast<uint64_t>(jiter_start_seqnum);
}
/*
* Class: org_rocksdb_ReadOptions
* Method: iterStartSeqnum
* Signature: (J)J
*/
jlong Java_org_rocksdb_ReadOptions_iterStartSeqnum(
JNIEnv*, jobject, jlong jhandle) {
auto* opt = reinterpret_cast<ROCKSDB_NAMESPACE::ReadOptions*>(jhandle);
return static_cast<jlong>(opt->iter_start_seqnum);
}
/*
* Class: org_rocksdb_ReadOptions
* Method: autoPrefixMode

@ -2380,7 +2380,8 @@ jobjectArray Java_org_rocksdb_RocksDB_keyMayExistFoundValue(
// exception thrown: OutOfMemoryError
return nullptr;
}
env->SetByteArrayRegion(jresult_value, 0, jvalue_len,
env->SetByteArrayRegion(
jresult_value, 0, jvalue_len,
const_cast<jbyte*>(reinterpret_cast<const jbyte*>(value.data())));
if (env->ExceptionCheck()) {
// exception thrown: ArrayIndexOutOfBoundsException
@ -3297,22 +3298,6 @@ jlong Java_org_rocksdb_RocksDB_getLatestSequenceNumber(
return db->GetLatestSequenceNumber();
}
/*
* Class: org_rocksdb_RocksDB
* Method: setPreserveDeletesSequenceNumber
* Signature: (JJ)Z
*/
jboolean Java_org_rocksdb_RocksDB_setPreserveDeletesSequenceNumber(
JNIEnv*, jobject, jlong jdb_handle, jlong jseq_number) {
auto* db = reinterpret_cast<ROCKSDB_NAMESPACE::DB*>(jdb_handle);
if (db->SetPreserveDeletesSequenceNumber(
static_cast<uint64_t>(jseq_number))) {
return JNI_TRUE;
} else {
return JNI_FALSE;
}
}
/*
* Class: org_rocksdb_RocksDB
* Method: disableFileDeletions
@ -3461,7 +3446,7 @@ jobjectArray Java_org_rocksdb_RocksDB_getLiveFilesMetaData(
auto* db = reinterpret_cast<ROCKSDB_NAMESPACE::DB*>(jdb_handle);
std::vector<ROCKSDB_NAMESPACE::LiveFileMetaData> live_files_meta_data;
db->GetLiveFilesMetaData(&live_files_meta_data);
// convert to Java type
const jsize jlen = static_cast<jsize>(live_files_meta_data.size());
jobjectArray jlive_files_meta_data = env->NewObjectArray(
@ -3594,7 +3579,7 @@ jobject Java_org_rocksdb_RocksDB_getPropertiesOfAllTables(
if (!s.ok()) {
ROCKSDB_NAMESPACE::RocksDBExceptionJni::ThrowNew(env, s);
}
// convert to Java type
jobject jhash_map = ROCKSDB_NAMESPACE::HashMapJni::construct(
env, static_cast<uint32_t>(table_properties_collection.size()));
@ -3772,7 +3757,7 @@ void Java_org_rocksdb_RocksDB_startTrace(
jlong jtrace_writer_jnicallback_handle) {
auto* db = reinterpret_cast<ROCKSDB_NAMESPACE::DB*>(jdb_handle);
ROCKSDB_NAMESPACE::TraceOptions trace_options;
trace_options.max_trace_file_size =
trace_options.max_trace_file_size =
static_cast<uint64_t>(jmax_trace_file_size);
// transfer ownership of trace writer from Java to C++
auto trace_writer =

@ -1126,19 +1126,6 @@ public class DBOptions extends RocksObject
return allowIngestBehind(nativeHandle_);
}
@Override
public DBOptions setPreserveDeletes(final boolean preserveDeletes) {
assert(isOwningHandle());
setPreserveDeletes(nativeHandle_, preserveDeletes);
return this;
}
@Override
public boolean preserveDeletes() {
assert(isOwningHandle());
return preserveDeletes(nativeHandle_);
}
@Override
public DBOptions setTwoWriteQueues(final boolean twoWriteQueues) {
assert(isOwningHandle());
@ -1485,9 +1472,6 @@ public class DBOptions extends RocksObject
private native void setAllowIngestBehind(final long handle,
final boolean allowIngestBehind);
private native boolean allowIngestBehind(final long handle);
private native void setPreserveDeletes(final long handle,
final boolean preserveDeletes);
private native boolean preserveDeletes(final long handle);
private native void setTwoWriteQueues(final long handle,
final boolean twoWriteQueues);
private native boolean twoWriteQueues(final long handle);

@ -1523,32 +1523,6 @@ public interface DBOptionsInterface<T extends DBOptionsInterface<T>> {
*/
boolean allowIngestBehind();
/**
* Needed to support differential snapshots.
* If set to true then DB will only process deletes with sequence number
* less than what was set by SetPreserveDeletesSequenceNumber(uint64_t ts).
* Clients are responsible to periodically call this method to advance
* the cutoff time. If this method is never called and preserve_deletes
* is set to true NO deletes will ever be processed.
* At the moment this only keeps normal deletes, SingleDeletes will
* not be preserved.
*
* DEFAULT: false
*
* @param preserveDeletes true to preserve deletes.
*
* @return the reference to the current options.
*/
T setPreserveDeletes(final boolean preserveDeletes);
/**
* Returns true if deletes are preserved.
* See {@link #setPreserveDeletes(boolean)}.
*
* @return true if deletes are preserved, false otherwise.
*/
boolean preserveDeletes();
/**
* If enabled it uses two queues for writes, one for the ones with
* disable_memtable and one for the ones that also write to memtable. This

@ -1211,19 +1211,6 @@ public class Options extends RocksObject
return allowIngestBehind(nativeHandle_);
}
@Override
public Options setPreserveDeletes(final boolean preserveDeletes) {
assert(isOwningHandle());
setPreserveDeletes(nativeHandle_, preserveDeletes);
return this;
}
@Override
public boolean preserveDeletes() {
assert(isOwningHandle());
return preserveDeletes(nativeHandle_);
}
@Override
public Options setTwoWriteQueues(final boolean twoWriteQueues) {
assert(isOwningHandle());
@ -2304,9 +2291,6 @@ public class Options extends RocksObject
private native void setAllowIngestBehind(final long handle,
final boolean allowIngestBehind);
private native boolean allowIngestBehind(final long handle);
private native void setPreserveDeletes(final long handle,
final boolean preserveDeletes);
private native boolean preserveDeletes(final long handle);
private native void setTwoWriteQueues(final long handle,
final boolean twoWriteQueues);
private native boolean twoWriteQueues(final long handle);

@ -533,35 +533,6 @@ public class ReadOptions extends RocksObject {
return this;
}
/**
* Needed to support differential snapshots. Has 2 effects:
* 1) Iterator will skip all internal keys with seqnum &lt; iter_start_seqnum
* 2) if this param &gt; 0 iterator will return INTERNAL keys instead of user
* keys; e.g. return tombstones as well.
*
* Default: 0 (don't filter by seqnum, return user keys)
*
* @param startSeqnum the starting sequence number.
*
* @return the reference to the current ReadOptions.
*/
public ReadOptions setIterStartSeqnum(final long startSeqnum) {
assert(isOwningHandle());
setIterStartSeqnum(nativeHandle_, startSeqnum);
return this;
}
/**
* Returns the starting Sequence Number of any iterator.
* See {@link #setIterStartSeqnum(long)}.
*
* @return the starting sequence number of any iterator.
*/
public long iterStartSeqnum() {
assert(isOwningHandle());
return iterStartSeqnum(nativeHandle_);
}
/**
* When true, by default use total_order_seek = true, and RocksDB can
* selectively enable prefix seek mode if won't generate a different result
@ -844,10 +815,7 @@ public class ReadOptions extends RocksObject {
private native void setIterateLowerBound(final long handle,
final long lowerBoundSliceHandle);
private native long iterateLowerBound(final long handle);
private native void setTableFilter(final long handle,
final long tableFilterHandle);
private native void setIterStartSeqnum(final long handle, final long seqNum);
private native long iterStartSeqnum(final long handle);
private native void setTableFilter(final long handle, final long tableFilterHandle);
private native boolean autoPrefixMode(final long handle);
private native void setAutoPrefixMode(final long handle, final boolean autoPrefixMode);
private native long timestamp(final long handle);

@ -4283,25 +4283,6 @@ public class RocksDB extends RocksObject {
return getLatestSequenceNumber(nativeHandle_);
}
/**
* Instructs DB to preserve deletes with sequence numbers &gt;= sequenceNumber.
*
* Has no effect if DBOptions#preserveDeletes() is set to false.
*
* This function assumes that user calls this function with monotonically
* increasing seqnums (otherwise we can't guarantee that a particular delete
* hasn't been already processed).
*
* @param sequenceNumber the minimum sequence number to preserve
*
* @return true if the value was successfully updated,
* false if user attempted to call if with
* sequenceNumber &lt;= current value.
*/
public boolean setPreserveDeletesSequenceNumber(final long sequenceNumber) {
return setPreserveDeletesSequenceNumber(nativeHandle_, sequenceNumber);
}
/**
* <p>Prevent file deletions. Compactions will continue to occur,
* but no obsolete files will be deleted. Calling this multiple
@ -5085,8 +5066,6 @@ public class RocksDB extends RocksObject {
throws RocksDBException;
private native void syncWal(final long handle) throws RocksDBException;
private native long getLatestSequenceNumber(final long handle);
private native boolean setPreserveDeletesSequenceNumber(final long handle,
final long sequenceNumber);
private native void disableFileDeletions(long handle) throws RocksDBException;
private native void enableFileDeletions(long handle, boolean force)
throws RocksDBException;

@ -730,15 +730,6 @@ public class DBOptionsTest {
}
}
@Test
public void preserveDeletes() {
try (final DBOptions opt = new DBOptions()) {
assertThat(opt.preserveDeletes()).isFalse();
opt.setPreserveDeletes(true);
assertThat(opt.preserveDeletes()).isTrue();
}
}
@Test
public void twoWriteQueues() {
try (final DBOptions opt = new DBOptions()) {

@ -961,15 +961,6 @@ public class OptionsTest {
}
}
@Test
public void preserveDeletes() {
try (final Options opt = new Options()) {
assertThat(opt.preserveDeletes()).isFalse();
opt.setPreserveDeletes(true);
assertThat(opt.preserveDeletes()).isTrue();
}
}
@Test
public void twoWriteQueues() {
try (final Options opt = new Options()) {

@ -201,16 +201,6 @@ public class ReadOptionsTest {
}
}
@Test
public void iterStartSeqnum() {
try (final ReadOptions opt = new ReadOptions()) {
assertThat(opt.iterStartSeqnum()).isEqualTo(0);
opt.setIterStartSeqnum(10);
assertThat(opt.iterStartSeqnum()).isEqualTo(10);
}
}
@Test
public void autoPrefixMode() {
try (final ReadOptions opt = new ReadOptions()) {

@ -1458,17 +1458,6 @@ public class RocksDBTest {
}
}
@Test
public void setPreserveDeletesSequenceNumber() throws RocksDBException {
try (final Options options = new Options().setCreateIfMissing(true)) {
final String dbPath = dbFolder.getRoot().getAbsolutePath();
try (final RocksDB db = RocksDB.open(options, dbPath)) {
assertThat(db.setPreserveDeletesSequenceNumber(db.getLatestSequenceNumber()))
.isFalse();
}
}
}
@Test
public void getLiveFiles() throws RocksDBException {
try (final Options options = new Options().setCreateIfMissing(true)) {

@ -369,8 +369,7 @@ static std::unordered_map<std::string, OptionTypeInfo>
OptionType::kBoolean, OptionVerificationType::kNormal,
OptionTypeFlags::kNone}},
{"preserve_deletes",
{offsetof(struct ImmutableDBOptions, preserve_deletes),
OptionType::kBoolean, OptionVerificationType::kNormal,
{0, OptionType::kBoolean, OptionVerificationType::kDeprecated,
OptionTypeFlags::kNone}},
{"concurrent_prepare", // Deprecated by two_write_queues
{0, OptionType::kBoolean, OptionVerificationType::kDeprecated,
@ -725,7 +724,6 @@ ImmutableDBOptions::ImmutableDBOptions(const DBOptions& options)
dump_malloc_stats(options.dump_malloc_stats),
avoid_flush_during_recovery(options.avoid_flush_during_recovery),
allow_ingest_behind(options.allow_ingest_behind),
preserve_deletes(options.preserve_deletes),
two_write_queues(options.two_write_queues),
manual_wal_flush(options.manual_wal_flush),
wal_compression(options.wal_compression),
@ -889,8 +887,6 @@ void ImmutableDBOptions::Dump(Logger* log) const {
avoid_flush_during_recovery);
ROCKS_LOG_HEADER(log, " Options.allow_ingest_behind: %d",
allow_ingest_behind);
ROCKS_LOG_HEADER(log, " Options.preserve_deletes: %d",
preserve_deletes);
ROCKS_LOG_HEADER(log, " Options.two_write_queues: %d",
two_write_queues);
ROCKS_LOG_HEADER(log, " Options.manual_wal_flush: %d",

@ -84,7 +84,6 @@ struct ImmutableDBOptions {
bool dump_malloc_stats;
bool avoid_flush_during_recovery;
bool allow_ingest_behind;
bool preserve_deletes;
bool two_write_queues;
bool manual_wal_flush;
CompressionType wal_compression;

@ -662,7 +662,6 @@ ReadOptions::ReadOptions()
pin_data(false),
background_purge_on_iterator_cleanup(false),
ignore_range_deletions(false),
iter_start_seqnum(0),
timestamp(nullptr),
iter_start_ts(nullptr),
deadline(std::chrono::microseconds::zero()),
@ -687,7 +686,6 @@ ReadOptions::ReadOptions(bool cksum, bool cache)
pin_data(false),
background_purge_on_iterator_cleanup(false),
ignore_range_deletions(false),
iter_start_seqnum(0),
timestamp(nullptr),
iter_start_ts(nullptr),
deadline(std::chrono::microseconds::zero()),

@ -164,10 +164,7 @@ DBOptions BuildDBOptions(const ImmutableDBOptions& immutable_db_options,
immutable_db_options.avoid_flush_during_recovery;
options.avoid_flush_during_shutdown =
mutable_db_options.avoid_flush_during_shutdown;
options.allow_ingest_behind =
immutable_db_options.allow_ingest_behind;
options.preserve_deletes =
immutable_db_options.preserve_deletes;
options.allow_ingest_behind = immutable_db_options.allow_ingest_behind;
options.two_write_queues = immutable_db_options.two_write_queues;
options.manual_wal_flush = immutable_db_options.manual_wal_flush;
options.wal_compression = immutable_db_options.wal_compression;

@ -328,7 +328,6 @@ TEST_F(OptionsSettableTest, DBOptionsAllFieldsSettable) {
"avoid_flush_during_recovery=false;"
"avoid_flush_during_shutdown=false;"
"allow_ingest_behind=false;"
"preserve_deletes=false;"
"concurrent_prepare=false;"
"two_write_queues=false;"
"manual_wal_flush=false;"

@ -155,6 +155,7 @@ TEST_F(OptionsTest, GetOptionsFromMapTest) {
{"bytes_per_sync", "47"},
{"wal_bytes_per_sync", "48"},
{"strict_bytes_per_sync", "true"},
{"preserve_deletes", "false"},
};
ColumnFamilyOptions base_cf_opt;
@ -2317,6 +2318,7 @@ TEST_F(OptionsOldApiTest, GetOptionsFromMapTest) {
{"bytes_per_sync", "47"},
{"wal_bytes_per_sync", "48"},
{"strict_bytes_per_sync", "true"},
{"preserve_deletes", "false"},
};
ColumnFamilyOptions base_cf_opt;

Loading…
Cancel
Save