@ -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