Fix db_test under ROCKSDB_LITE

Summary:
This diff exclude alot of tests in db_test that are not compiling / failing under ROCKSD_LITE

Test Plan:
OPT=-DROCKSDB_LITE make check -j64
make check -j64

Reviewers: yhchiang, igor, sdong

Reviewed By: sdong

Subscribers: dhruba

Differential Revision: https://reviews.facebook.net/D48771
main
Islam AbdelRahman 9 years ago
parent 6d730b4ae7
commit 952ad994a9
  1. 2
      Makefile
  2. 183
      db/db_test.cc
  3. 6
      db/db_test_util.cc

@ -550,8 +550,10 @@ check: all
echo "===== Running $$t"; ./$$t || exit 1; done; \
fi
rm -rf $(TMPD)
ifeq ($(filter -DROCKSDB_LITE,$(OPT)),)
python tools/ldb_test.py
sh tools/rocksdb_dump_test.sh
endif
check_some: $(SUBSET) ldb_tests
for t in $(SUBSET); do echo "===== Running $$t"; ./$$t || exit 1; done

@ -78,6 +78,7 @@ static long TestGetTickerCount(const Options& options, Tickers ticker_type) {
return options.statistics->getTickerCount(ticker_type);
}
#ifndef ROCKSDB_LITE
// A helper function that ensures the table properties returned in
// `GetPropertiesOfAllTablesTest` is correct.
// This test assumes entries size is different for each of the tables.
@ -113,6 +114,7 @@ uint64_t GetNumberOfSstFilesForColumnFamily(DB* db,
}
} // namespace
#endif // ROCKSDB_LITE
class DBTest : public DBTestBase {
public:
@ -131,46 +133,7 @@ class DBTestWithParam : public DBTest,
uint32_t max_subcompactions_;
};
class BloomStatsTestWithParam
: public DBTest,
public testing::WithParamInterface<std::tuple<bool, bool>> {
public:
BloomStatsTestWithParam() {
use_block_table_ = std::get<0>(GetParam());
use_block_based_builder_ = std::get<1>(GetParam());
options_.create_if_missing = true;
options_.prefix_extractor.reset(rocksdb::NewFixedPrefixTransform(4));
options_.memtable_prefix_bloom_bits = 8 * 1024;
if (use_block_table_) {
BlockBasedTableOptions table_options;
table_options.hash_index_allow_collision = false;
table_options.filter_policy.reset(
NewBloomFilterPolicy(10, use_block_based_builder_));
options_.table_factory.reset(NewBlockBasedTableFactory(table_options));
} else {
PlainTableOptions table_options;
options_.table_factory.reset(NewPlainTableFactory(table_options));
}
perf_context.Reset();
DestroyAndReopen(options_);
}
~BloomStatsTestWithParam() {
perf_context.Reset();
Destroy(options_);
}
// Required if inheriting from testing::WithParamInterface<>
static void SetUpTestCase() {}
static void TearDownTestCase() {}
bool use_block_table_;
bool use_block_based_builder_;
Options options_;
};
#ifndef ROCKSDB_LITE
TEST_F(DBTest, Empty) {
do {
Options options;
@ -232,6 +195,7 @@ TEST_F(DBTest, Empty) {
ASSERT_EQ("0", num);
} while (ChangeOptions());
}
#endif // ROCKSDB_LITE
TEST_F(DBTest, WriteEmptyBatch) {
Options options;
@ -254,6 +218,7 @@ TEST_F(DBTest, WriteEmptyBatch) {
ASSERT_EQ("bar", Get(1, "foo"));
}
#ifndef ROCKSDB_LITE
TEST_F(DBTest, ReadOnlyDB) {
ASSERT_OK(Put("foo", "v1"));
ASSERT_OK(Put("bar", "v2"));
@ -790,6 +755,7 @@ TEST_F(DBTest, AggregatedTablePropertiesAtLevel) {
}
}
}
#endif // ROCKSDB_LITE
class CoutingUserTblPropCollector : public TablePropertiesCollector {
public:
@ -842,6 +808,7 @@ class CoutingUserTblPropCollectorFactory
uint32_t num_created_;
};
#ifndef ROCKSDB_LITE
TEST_F(DBTest, GetUserDefinedTableProperties) {
Options options = CurrentOptions();
options.level0_file_num_compaction_trigger = (1<<30);
@ -881,6 +848,7 @@ TEST_F(DBTest, GetUserDefinedTableProperties) {
dbfull()->TEST_CompactRange(0, nullptr, nullptr);
ASSERT_GT(collector_factory->num_created_, 0);
}
#endif // ROCKSDB_LITE
TEST_F(DBTest, UserDefinedTablePropertiesContext) {
Options options = CurrentOptions();
@ -943,6 +911,7 @@ TEST_F(DBTest, UserDefinedTablePropertiesContext) {
ASSERT_GT(collector_factory->num_created_, 0);
}
#ifndef ROCKSDB_LITE
TEST_F(DBTest, LevelLimitReopen) {
Options options = CurrentOptions();
CreateAndReopenWithCF({"pikachu"}, options);
@ -964,6 +933,7 @@ TEST_F(DBTest, LevelLimitReopen) {
options.max_bytes_for_level_multiplier_additional.resize(10, 1);
ASSERT_OK(TryReopenWithColumnFamilies({"default", "pikachu"}, options));
}
#endif // ROCKSDB_LITE
TEST_F(DBTest, PutDeleteGet) {
do {
@ -1110,6 +1080,7 @@ TEST_F(DBTest, GetFromVersions) {
} while (ChangeOptions());
}
#ifndef ROCKSDB_LITE
TEST_F(DBTest, GetSnapshot) {
anon::OptionsOverride options_override;
options_override.skip_policy = kSkipNoSnapshot;
@ -1135,6 +1106,7 @@ TEST_F(DBTest, GetSnapshot) {
}
} while (ChangeOptions());
}
#endif // ROCKSDB_LITE
TEST_F(DBTest, GetLevel0Ordering) {
do {
@ -1162,6 +1134,7 @@ TEST_F(DBTest, WrongLevel0Config) {
ASSERT_OK(DB::Open(options, dbname_, &db_));
}
#ifndef ROCKSDB_LITE
TEST_F(DBTest, GetOrderedByLevels) {
do {
CreateAndReopenWithCF({"pikachu"}, CurrentOptions());
@ -1233,6 +1206,7 @@ TEST_F(DBTest, GetEncountersEmptyLevel) {
ASSERT_EQ(NumTableFilesAtLevel(0, 1), 1); // XXX
} while (ChangeOptions(kSkipUniversalCompaction | kSkipFIFOCompaction));
}
#endif // ROCKSDB_LITE
// KeyMayExist can lead to a few false positives, but not false negatives.
// To make test deterministic, use a much larger number of bits per key-20 than
@ -1365,6 +1339,7 @@ TEST_F(DBTest, NonBlockingIteration) {
kSkipMmapReads));
}
#ifndef ROCKSDB_LITE
TEST_F(DBTest, ManagedNonBlockingIteration) {
do {
ReadOptions non_blocking_opts, regular_opts;
@ -1429,6 +1404,7 @@ TEST_F(DBTest, ManagedNonBlockingIteration) {
} while (ChangeOptions(kSkipPlainTable | kSkipNoSeekToLast | kSkipHashCuckoo |
kSkipMmapReads));
}
#endif // ROCKSDB_LITE
// A delete is skipped for key if KeyMayExist(key) returns False
// Tests Writebatch consistency and proper delete behaviour
@ -2333,6 +2309,7 @@ TEST_F(DBTest, FlushMultipleMemtable) {
} while (ChangeCompactOptions());
}
#ifndef ROCKSDB_LITE
TEST_F(DBTest, NumImmutableMemTable) {
do {
Options options = CurrentOptions();
@ -2442,6 +2419,7 @@ TEST_F(DBTest, NumImmutableMemTable) {
ASSERT_TRUE(GetPerfLevel() == kDisable);
} while (ChangeCompactOptions());
}
#endif // ROCKSDB_LITE
TEST_F(DBTest, FlushEmptyColumnFamily) {
// Block flush thread and disable compaction thread
@ -2487,6 +2465,7 @@ TEST_F(DBTest, FlushEmptyColumnFamily) {
sleeping_task_low.WaitUntilDone();
}
#ifndef ROCKSDB_LITE
TEST_F(DBTest, GetProperty) {
// Set sizes to both background thread pool to be 1 and block them.
env_->SetBackgroundThreads(1, Env::HIGH);
@ -2792,6 +2771,7 @@ TEST_F(DBTest, EstimatePendingCompBytes) {
"rocksdb.estimate-pending-compaction-bytes", &int_num));
ASSERT_EQ(int_num, 0U);
}
#endif // ROCKSDB_LITE
TEST_F(DBTest, FLUSH) {
do {
@ -2851,7 +2831,7 @@ TEST_F(DBTest, RecoveryWithEmptyLog) {
} while (ChangeOptions());
}
#ifndef ROCKSDB_LITE
TEST_F(DBTest, FlushSchedule) {
Options options = CurrentOptions();
options.disable_auto_compactions = true;
@ -2892,7 +2872,7 @@ TEST_F(DBTest, FlushSchedule) {
ASSERT_LE(pikachu_tables, static_cast<uint64_t>(10));
ASSERT_GT(pikachu_tables, static_cast<uint64_t>(0));
}
#endif // ROCKSDB_LITE
TEST_F(DBTest, ManifestRollOver) {
do {
@ -2940,6 +2920,7 @@ TEST_F(DBTest, IdentityAcrossRestarts) {
} while (ChangeCompactOptions());
}
#ifndef ROCKSDB_LITE
TEST_F(DBTest, RecoverWithLargeLog) {
do {
{
@ -2966,6 +2947,7 @@ TEST_F(DBTest, RecoverWithLargeLog) {
ASSERT_GT(NumTableFilesAtLevel(0, 1), 1);
} while (ChangeCompactOptions());
}
#endif // ROCKSDB_LITE
namespace {
class KeepFilter : public CompactionFilter {
@ -3030,6 +3012,7 @@ class DelayFilterFactory : public CompactionFilterFactory {
};
} // namespace
#ifndef ROCKSDB_LITE
TEST_F(DBTest, CompressedCache) {
if (!Snappy_Supported()) {
return;
@ -3153,6 +3136,7 @@ static std::string CompressibleString(Random* rnd, int len) {
test::CompressibleString(rnd, 0.8, len, &r);
return r;
}
#endif // ROCKSDB_LITE
TEST_F(DBTest, FailMoreDbPaths) {
Options options = CurrentOptions();
@ -3183,6 +3167,7 @@ void CheckColumnFamilyMeta(const ColumnFamilyMetaData& cf_meta) {
ASSERT_EQ(cf_meta.size, cf_size);
}
#ifndef ROCKSDB_LITE
TEST_F(DBTest, ColumnFamilyMetaDataTest) {
Options options = CurrentOptions();
options.create_if_missing = true;
@ -3329,6 +3314,7 @@ TEST_F(DBTest, RepeatedWritesToSameKey) {
}
} while (ChangeCompactOptions());
}
#endif // ROCKSDB_LITE
TEST_F(DBTest, SparseMerge) {
do {
@ -3375,6 +3361,7 @@ TEST_F(DBTest, SparseMerge) {
} while (ChangeCompactOptions());
}
#ifndef ROCKSDB_LITE
static bool Between(uint64_t val, uint64_t low, uint64_t high) {
bool result = (val >= low) && (val <= high);
if (!result) {
@ -3587,6 +3574,7 @@ TEST_F(DBTest, ApproximateSizes_MixOfSmallAndLarge) {
// ApproximateOffsetOf() is not yet implemented in plain table format.
} while (ChangeOptions(kSkipPlainTable));
}
#endif // ROCKSDB_LITE
TEST_F(DBTest, IteratorPinsRef) {
do {
@ -3614,6 +3602,7 @@ TEST_F(DBTest, IteratorPinsRef) {
} while (ChangeCompactOptions());
}
#ifndef ROCKSDB_LITE
TEST_F(DBTest, Snapshot) {
anon::OptionsOverride options_override;
options_override.skip_policy = kSkipNoSnapshot;
@ -3716,6 +3705,7 @@ TEST_F(DBTest, HiddenValuesAreRemoved) {
} while (ChangeOptions(kSkipUniversalCompaction | kSkipFIFOCompaction |
kSkipPlainTable | kSkipHashCuckoo));
}
#endif // ROCKSDB_LITE
TEST_F(DBTest, CompactBetweenSnapshots) {
anon::OptionsOverride options_override;
@ -3821,6 +3811,7 @@ TEST_F(DBTest, UnremovableSingleDelete) {
kSkipUniversalCompaction | kSkipMergePut));
}
#ifndef ROCKSDB_LITE
TEST_F(DBTest, DeletionMarkers1) {
Options options = CurrentOptions();
options.max_background_flushes = 0;
@ -3930,6 +3921,7 @@ TEST_F(DBTest, OverlapInLevel0) {
ASSERT_EQ("NOT_FOUND", Get(1, "600"));
} while (ChangeOptions(kSkipUniversalCompaction | kSkipFIFOCompaction));
}
#endif // ROCKSDB_LITE
TEST_F(DBTest, ComparatorCheck) {
class NewComparator : public Comparator {
@ -4122,6 +4114,7 @@ TEST_F(DBTest, DestroyDBMetaDatabase) {
ASSERT_TRUE(!(DB::Open(options, metametadbname, &db)).ok());
}
#ifndef ROCKSDB_LITE
// Check that number of files does not grow when writes are dropped
TEST_F(DBTest, DropWrites) {
do {
@ -4192,6 +4185,7 @@ TEST_F(DBTest, DropWritesFlush) {
env_->drop_writes_.store(false, std::memory_order_release);
} while (ChangeCompactOptions());
}
#endif // ROCKSDB_LITE
// Check that CompactRange() returns failure if there is not enough space left
// on device
@ -4241,6 +4235,7 @@ TEST_F(DBTest, NonWritableFileSystem) {
} while (ChangeCompactOptions());
}
#ifndef ROCKSDB_LITE
TEST_F(DBTest, ManifestWriteError) {
// Test for the following problem:
// (a) Compaction produces file F
@ -4282,6 +4277,7 @@ TEST_F(DBTest, ManifestWriteError) {
ASSERT_EQ("bar", Get("foo"));
}
}
#endif // ROCKSDB_LITE
TEST_F(DBTest, PutFailsParanoid) {
// Test the following:
@ -4540,6 +4536,7 @@ TEST_F(DBTest, BloomFilterWrapper) {
ASSERT_EQ(2U * maxKey, policy->GetCounter());
}
#ifndef ROCKSDB_LITE
TEST_F(DBTest, SnapshotFiles) {
do {
Options options = CurrentOptions();
@ -4667,6 +4664,7 @@ TEST_F(DBTest, SnapshotFiles) {
dbfull()->DisableFileDeletions();
} while (ChangeCompactOptions());
}
#endif
TEST_F(DBTest, CompactOnFlush) {
anon::OptionsOverride options_override;
@ -4806,6 +4804,7 @@ TEST_F(DBTest, FlushOneColumnFamily) {
}
}
#ifndef ROCKSDB_LITE
// In https://reviews.facebook.net/D20661 we change
// recovery behavior: previously for each log file each column family
// memtable was flushed, even it was empty. Now it's changed:
@ -5011,6 +5010,7 @@ TEST_F(DBTest, SharedWriteBuffer) {
static_cast<uint64_t>(4));
}
}
#endif // ROCKSDB_LITE
TEST_F(DBTest, PurgeInfoLogs) {
Options options = CurrentOptions();
@ -5083,6 +5083,7 @@ TEST_F(DBTest, SyncMultipleLogs) {
ASSERT_OK(dbfull()->SyncWAL());
}
#ifndef ROCKSDB_LITE
//
// Test WAL recovery for the various modes available
//
@ -5365,7 +5366,6 @@ TEST_F(DBTest, kSkipAnyCorruptedRecords) {
}
}
// Multi-threaded test:
namespace {
@ -5532,6 +5532,7 @@ TEST_P(MultiThreadedDBTest, MultiThreaded) {
INSTANTIATE_TEST_CASE_P(
MultiThreaded, MultiThreadedDBTest,
::testing::ValuesIn(MultiThreadedDBTest::GenerateOptionConfigs()));
#endif // ROCKSDB_LITE
// Group commit test:
namespace {
@ -5672,6 +5673,7 @@ class ModelDB: public DB {
return s;
}
#ifndef ROCKSDB_LITE
using DB::AddFile;
virtual Status AddFile(ColumnFamilyHandle* column_family,
const ExternalSstFileInfo* file_path,
@ -5690,6 +5692,7 @@ class ModelDB: public DB {
TablePropertiesCollection* props) override {
return Status();
}
#endif // ROCKSDB_LITE
using DB::KeyMayExist;
virtual bool KeyMayExist(const ReadOptions& options,
@ -5834,7 +5837,9 @@ class ModelDB: public DB {
return Status::OK();
}
#ifndef ROCKSDB_LITE
virtual Status DisableFileDeletions() override { return Status::OK(); }
virtual Status EnableFileDeletions(bool force) override {
return Status::OK();
}
@ -5849,11 +5854,6 @@ class ModelDB: public DB {
virtual Status DeleteFile(std::string name) override { return Status::OK(); }
virtual Status GetDbIdentity(std::string& identity) const override {
return Status::OK();
}
virtual SequenceNumber GetLatestSequenceNumber() const override { return 0; }
virtual Status GetUpdatesSince(
rocksdb::SequenceNumber, unique_ptr<rocksdb::TransactionLogIterator>*,
const TransactionLogIterator::ReadOptions&
@ -5861,13 +5861,20 @@ class ModelDB: public DB {
return Status::NotSupported("Not supported in Model DB");
}
virtual ColumnFamilyHandle* DefaultColumnFamily() const override {
return nullptr;
}
virtual void GetColumnFamilyMetaData(
ColumnFamilyHandle* column_family,
ColumnFamilyMetaData* metadata) override {}
#endif // ROCKSDB_LITE
virtual Status GetDbIdentity(std::string& identity) const override {
return Status::OK();
}
virtual SequenceNumber GetLatestSequenceNumber() const override { return 0; }
virtual ColumnFamilyHandle* DefaultColumnFamily() const override {
return nullptr;
}
private:
class ModelIter: public Iterator {
@ -6128,6 +6135,7 @@ TEST_F(DBTest, MultiGetEmpty) {
} while (ChangeCompactOptions());
}
#ifndef ROCKSDB_LITE
namespace {
void PrefixScanInit(DBTest *dbtest) {
char buf[100];
@ -6230,6 +6238,7 @@ TEST_F(DBTest, PrefixScan) {
} // end of while
XFUNC_TEST("", "dbtest_prefix", prefix_skip1, XFuncPoint::SetSkip, 0);
}
#endif // ROCKSDB_LITE
TEST_F(DBTest, BlockBasedTablePrefixIndexTest) {
// create a DB with block prefix index
@ -6291,6 +6300,7 @@ TEST_F(DBTest, ChecksumTest) {
ASSERT_EQ("h", Get("g"));
}
#ifndef ROCKSDB_LITE
TEST_P(DBTestWithParam, FIFOCompactionTest) {
for (int iter = 0; iter < 2; ++iter) {
// first iteration -- auto compaction
@ -6330,6 +6340,7 @@ TEST_P(DBTestWithParam, FIFOCompactionTest) {
}
}
}
#endif // ROCKSDB_LITE
// verify that we correctly deprecated timeout_hint_us
TEST_F(DBTest, SimpleWriteTimeoutTest) {
@ -6340,6 +6351,7 @@ TEST_F(DBTest, SimpleWriteTimeoutTest) {
ASSERT_NOK(Put(Key(1), Key(1) + std::string(100, 'v'), write_opt));
}
#ifndef ROCKSDB_LITE
/*
* This test is not reliable enough as it heavily depends on disk behavior.
*/
@ -6432,6 +6444,7 @@ TEST_F(DBTest, TableOptionsSanitizeTest) {
options.prefix_extractor.reset(NewFixedPrefixTransform(1));
ASSERT_OK(TryReopen(options));
}
#endif // ROCKSDB_LITE
TEST_F(DBTest, SanitizeNumThreads) {
for (int attempt = 0; attempt < 2; attempt++) {
@ -6677,6 +6690,7 @@ TEST_F(DBTest, DisableDataSyncTest) {
}
}
#ifndef ROCKSDB_LITE
TEST_F(DBTest, DynamicMemtableOptions) {
const uint64_t k64KB = 1 << 16;
const uint64_t k128KB = 1 << 17;
@ -6818,6 +6832,7 @@ TEST_F(DBTest, DynamicMemtableOptions) {
rocksdb::SyncPoint::GetInstance()->DisableProcessing();
}
#endif // ROCKSDB_LITE
#if ROCKSDB_USING_THREAD_STATUS
namespace {
@ -7243,6 +7258,7 @@ TEST_P(DBTestWithParam, PreShutdownCompactionMiddle) {
#endif // ROCKSDB_USING_THREAD_STATUS
#ifndef ROCKSDB_LITE
TEST_F(DBTest, FlushOnDestroy) {
WriteOptions wo;
wo.disableWAL = true;
@ -7696,6 +7712,7 @@ TEST_F(DBTest, DynamicCompactionOptions) {
rocksdb::SyncPoint::GetInstance()->DisableProcessing();
}
#endif // ROCKSDB_LITE
TEST_F(DBTest, FileCreationRandomFailure) {
Options options;
@ -7757,6 +7774,7 @@ TEST_F(DBTest, FileCreationRandomFailure) {
}
}
#ifndef ROCKSDB_LITE
TEST_F(DBTest, DynamicMiscOptions) {
// Test max_sequential_skip_in_iterations
Options options;
@ -7806,6 +7824,7 @@ TEST_F(DBTest, DynamicMiscOptions) {
// No reseek
assert_reseek_count(300, 1);
}
#endif // ROCKSDB_LITE
TEST_F(DBTest, DontDeletePendingOutputs) {
Options options;
@ -7839,6 +7858,7 @@ TEST_F(DBTest, DontDeletePendingOutputs) {
Compact("a", "b");
}
#ifndef ROCKSDB_LITE
TEST_F(DBTest, DontDeleteMovedFile) {
// This test triggers move compaction and verifies that the file is not
// deleted when it's part of move compaction
@ -7941,6 +7961,7 @@ TEST_F(DBTest, OptimizeFiltersForHits) {
ASSERT_EQ(Get(1, Key(i)), "val");
}
}
#endif // ROCKSDB_LITE
TEST_F(DBTest, L0L1L2AndUpHitCounter) {
Options options = CurrentOptions();
@ -8040,6 +8061,7 @@ TEST_F(DBTest, MutexWaitStats) {
ThreadStatus::STATE_MUTEX_WAIT, 0);
}
#ifndef ROCKSDB_LITE
// This reproduces a bug where we don't delete a file because when it was
// supposed to be deleted, it was blocked by pending_outputs
// Consider:
@ -8135,6 +8157,7 @@ TEST_F(DBTest, DeleteObsoleteFilesPendingOutputs) {
ASSERT_EQ(Status::NotFound(), env_->FileExists(dbname_ + file_on_L2));
listener->VerifyMatchedCount(1);
}
#endif // ROCKSDB_LITE
TEST_F(DBTest, CloseSpeedup) {
Options options = CurrentOptions();
@ -8256,6 +8279,7 @@ TEST_F(DBTest, MergeTestTime) {
#endif // ROCKSDB_USING_THREAD_STATUS
}
#ifndef ROCKSDB_LITE
TEST_P(DBTestWithParam, MergeCompactionTimeTest) {
SetPerfLevel(kEnableTime);
Options options;
@ -8306,6 +8330,7 @@ TEST_P(DBTestWithParam, FilterCompactionTimeTest) {
ASSERT_NE(TestGetTickerCount(options, FILTER_OPERATION_TOTAL_TIME), 0);
delete itr;
}
#endif // ROCKSDB_LITE
TEST_F(DBTest, TestLogCleanup) {
Options options = CurrentOptions();
@ -8322,6 +8347,7 @@ TEST_F(DBTest, TestLogCleanup) {
}
}
#ifndef ROCKSDB_LITE
TEST_F(DBTest, EmptyCompactedDB) {
Options options;
options.max_open_files = -1;
@ -8332,6 +8358,7 @@ TEST_F(DBTest, EmptyCompactedDB) {
ASSERT_TRUE(s.IsNotSupported());
Close();
}
#endif // ROCKSDB_LITE
class CountingDeleteTabPropCollector : public TablePropertiesCollector {
public:
@ -8373,6 +8400,7 @@ class CountingDeleteTabPropCollectorFactory
}
};
#ifndef ROCKSDB_LITE
TEST_F(DBTest, TablePropertiesNeedCompactTest) {
Random rnd(301);
@ -8671,6 +8699,7 @@ TEST_F(DBTest, PromoteL0Failure) {
status = experimental::PromoteL0(db_, db_->DefaultColumnFamily());
ASSERT_TRUE(status.IsInvalidArgument());
}
#endif // ROCKSDB_LITE
// Github issue #596
TEST_F(DBTest, HugeNumberOfLevels) {
@ -8877,6 +8906,7 @@ TEST_F(DBTest, HardLimit) {
rocksdb::SyncPoint::GetInstance()->DisableProcessing();
}
#ifndef ROCKSDB_LITE
TEST_F(DBTest, SoftLimit) {
Options options;
options.env = env_;
@ -8996,6 +9026,7 @@ TEST_F(DBTest, SoftLimit) {
ASSERT_EQ(sleep_count.load(), 0);
rocksdb::SyncPoint::GetInstance()->DisableProcessing();
}
#endif // ROCKSDB_LITE
TEST_F(DBTest, FailWhenCompressionNotSupportedTest) {
CompressionType compressions[] = {kZlibCompression, kBZip2Compression,
@ -9017,6 +9048,7 @@ TEST_F(DBTest, FailWhenCompressionNotSupportedTest) {
}
}
#ifndef ROCKSDB_LITE
TEST_F(DBTest, RowCache) {
Options options = CurrentOptions();
options.statistics = rocksdb::CreateDBStatistics();
@ -9035,6 +9067,7 @@ TEST_F(DBTest, RowCache) {
ASSERT_EQ(TestGetTickerCount(options, ROW_CACHE_HIT), 1);
ASSERT_EQ(TestGetTickerCount(options, ROW_CACHE_MISS), 1);
}
#endif // ROCKSDB_LITE
// TODO(3.13): fix the issue of Seek() + Prev() which might not necessary
// return the biggest key which is smaller than the seek key.
@ -9095,6 +9128,7 @@ TEST_F(DBTest, DeletingOldWalAfterDrop) {
EXPECT_GT(lognum2, lognum1);
}
#ifndef ROCKSDB_LITE
TEST_F(DBTest, RateLimitedDelete) {
rocksdb::SyncPoint::GetInstance()->LoadDependency({
{"DBTest::RateLimitedDelete:1",
@ -9263,6 +9297,7 @@ TEST_F(DBTest, DestroyDBWithRateLimitedDelete) {
// We have deleted the 4 sst files in the delete_scheduler
ASSERT_EQ(bg_delete_file, 4);
}
#endif // ROCKSDB_LITE
TEST_F(DBTest, UnsupportedManualSync) {
DestroyAndReopen(CurrentOptions());
@ -9271,6 +9306,7 @@ TEST_F(DBTest, UnsupportedManualSync) {
ASSERT_TRUE(s.IsNotSupported());
}
#ifndef ROCKSDB_LITE
TEST_F(DBTest, OpenDBWithInfiniteMaxOpenFiles) {
// Open DB with infinite max open files
// - First iteration use 1 thread to open files
@ -9815,6 +9851,7 @@ TEST_F(DBTest, AddExternalSstFileMultiThreaded) {
} while (ChangeOptions(kSkipPlainTable | kSkipUniversalCompaction |
kSkipFIFOCompaction));
}
#endif // ROCKSDB_LITE
// 1 Create some SST files by inserting K-V pairs into DB
// 2 Close DB and change suffix from ".sst" to ".ldb" for every other SST file
@ -9887,6 +9924,47 @@ TEST_F(DBTest, PauseBackgroundWorkTest) {
ASSERT_EQ(true, done.load());
}
#ifndef ROCKSDB_LITE
class BloomStatsTestWithParam
: public DBTest,
public testing::WithParamInterface<std::tuple<bool, bool>> {
public:
BloomStatsTestWithParam() {
use_block_table_ = std::get<0>(GetParam());
use_block_based_builder_ = std::get<1>(GetParam());
options_.create_if_missing = true;
options_.prefix_extractor.reset(rocksdb::NewFixedPrefixTransform(4));
options_.memtable_prefix_bloom_bits = 8 * 1024;
if (use_block_table_) {
BlockBasedTableOptions table_options;
table_options.hash_index_allow_collision = false;
table_options.filter_policy.reset(
NewBloomFilterPolicy(10, use_block_based_builder_));
options_.table_factory.reset(NewBlockBasedTableFactory(table_options));
} else {
PlainTableOptions table_options;
options_.table_factory.reset(NewPlainTableFactory(table_options));
}
perf_context.Reset();
DestroyAndReopen(options_);
}
~BloomStatsTestWithParam() {
perf_context.Reset();
Destroy(options_);
}
// Required if inheriting from testing::WithParamInterface<>
static void SetUpTestCase() {}
static void TearDownTestCase() {}
bool use_block_table_;
bool use_block_based_builder_;
Options options_;
};
// 1 Insert 2 K-V pairs into DB
// 2 Call Get() for both keys - expext memtable bloom hit stat to be 2
// 3 Call Get() for nonexisting key - expect memtable bloom miss stat to be 1
@ -10000,6 +10078,7 @@ INSTANTIATE_TEST_CASE_P(BloomStatsTestWithParam, BloomStatsTestWithParam,
::testing::Values(std::make_tuple(true, true),
std::make_tuple(true, false),
std::make_tuple(false, false)));
#endif // ROCKSDB_LITE
} // namespace rocksdb
#endif

@ -86,7 +86,11 @@ bool DBTestBase::ChangeOptions(int skip_mask) {
option_config_ == kPlainTableCappedPrefixNonMmap ||
option_config_ == kPlainTableAllBytesPrefix ||
option_config_ == kVectorRep || option_config_ == kHashLinkList ||
option_config_ == kHashCuckoo) {
option_config_ == kHashCuckoo ||
option_config_ == kUniversalCompaction ||
option_config_ == kUniversalCompactionMultiLevel ||
option_config_ == kUniversalSubcompactions ||
option_config_ == kFIFOCompaction) {
continue;
}
#endif

Loading…
Cancel
Save