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
This commit is contained in:
parent
e58cc56fb5
commit
d10c5c08d3
@ -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 < iter_start_seqnum
|
||||
* 2) if this param > 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 >= 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 <= 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…
Reference in New Issue
Block a user