Remove XFunc tests

Summary:
Xfunc is hardly used. Remove it to keep the code simple.
Closes https://github.com/facebook/rocksdb/pull/1905

Differential Revision: D4603220

Pulled By: siying

fbshipit-source-id: 731f96d
main
Siying Dong 8 years ago committed by Facebook Github Bot
parent e7d902e693
commit 1ba2804b7f
  1. 2
      CMakeLists.txt
  2. 7
      Makefile
  3. 9
      db/column_family.cc
  4. 1
      db/compaction_job_stats_test.cc
  5. 3
      db/db_bloom_filter_test.cc
  6. 19
      db/db_impl.cc
  7. 6
      db/db_tailing_iter_test.cc
  8. 1
      db/db_test.cc
  9. 3
      db/db_test_util.cc
  10. 3
      db/db_test_util.h
  11. 8
      db/managed_iterator.cc
  12. 145
      db/xfunc_test_points.cc
  13. 33
      db/xfunc_test_points.h
  14. 17
      src.mk
  15. 5
      util/options.cc
  16. 50
      util/xfunc.cc
  17. 113
      util/xfunc.h
  18. 1
      utilities/checkpoint/checkpoint_test.cc

@ -314,7 +314,6 @@ set(SOURCES
db/write_batch_base.cc
db/write_controller.cc
db/write_thread.cc
db/xfunc_test_points.cc
memtable/hash_cuckoo_rep.cc
memtable/hash_linklist_rep.cc
memtable/hash_skiplist_rep.cc
@ -408,7 +407,6 @@ set(SOURCES
util/thread_status_util.cc
util/thread_status_util_debug.cc
util/transaction_test_util.cc
util/xfunc.cc
util/xxhash.cc
utilities/backupable/backupable_db.cc
utilities/blob_db/blob_db.cc

@ -1529,13 +1529,6 @@ commit_prereq: build_tools/rocksdb-lego-determinator \
J=$(J) build_tools/precommit_checker.py unit unit_481 clang_unit release release_481 clang_release tsan asan ubsan lite unit_non_shm
$(MAKE) clean && $(MAKE) jclean && $(MAKE) rocksdbjava;
xfunc:
for xftest in $(XFUNC_TESTS); do \
echo "===== Running xftest $$xftest"; \
make check ROCKSDB_XFUNC_TEST="$$xftest" tests-regexp="DBTest" ;\
done
# ---------------------------------------------------------------------------
# Platform-specific compilation
# ---------------------------------------------------------------------------

@ -32,7 +32,6 @@
#include "util/compression.h"
#include "util/options_helper.h"
#include "util/thread_status_util.h"
#include "util/xfunc.h"
namespace rocksdb {
@ -182,14 +181,6 @@ ColumnFamilyOptions SanitizeOptions(const ImmutableDBOptions& db_options,
} else if (result.memtable_prefix_bloom_size_ratio < 0) {
result.memtable_prefix_bloom_size_ratio = 0;
}
XFUNC_TEST("memtablelist_history", "transaction_xftest_SanitizeOptions",
xf_transaction_set_memtable_history1,
xf_transaction_set_memtable_history,
&result.max_write_buffer_number_to_maintain);
XFUNC_TEST("memtablelist_history_clear", "transaction_xftest_SanitizeOptions",
xf_transaction_clear_memtable_history1,
xf_transaction_clear_memtable_history,
&result.max_write_buffer_number_to_maintain);
if (!result.prefix_extractor) {
assert(result.memtable_factory);

@ -61,7 +61,6 @@
#include "util/testharness.h"
#include "util/testutil.h"
#include "util/thread_status_util.h"
#include "util/xfunc.h"
#include "utilities/merge_operators.h"
#if !defined(IOS_CROSS_COMPILE)

@ -788,8 +788,6 @@ void PrefixScanInit(DBBloomFilterTest* dbtest) {
} // namespace
TEST_F(DBBloomFilterTest, PrefixScan) {
XFUNC_TEST("", "dbtest_prefix", prefix_skip1, XFuncPoint::SetSkip,
kSkipNoPrefix);
while (ChangeFilterOptions()) {
int count;
Slice prefix;
@ -836,7 +834,6 @@ TEST_F(DBBloomFilterTest, PrefixScan) {
ASSERT_EQ(env_->random_read_counter_.Read(), 2);
Close();
} // end of while
XFUNC_TEST("", "dbtest_prefix", prefix_skip1, XFuncPoint::SetSkip, 0);
}
TEST_F(DBBloomFilterTest, OptimizeFiltersForHits) {

@ -58,7 +58,6 @@
#include "db/version_set.h"
#include "db/write_batch_internal.h"
#include "db/write_callback.h"
#include "db/xfunc_test_points.h"
#include "memtable/hash_linklist_rep.h"
#include "memtable/hash_skiplist_rep.h"
#include "port/likely.h"
@ -100,7 +99,6 @@
#include "util/sync_point.h"
#include "util/thread_status_updater.h"
#include "util/thread_status_util.h"
#include "util/xfunc.h"
namespace rocksdb {
@ -4393,10 +4391,6 @@ Iterator* DBImpl::NewIterator(const ReadOptions& read_options,
}
auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
auto cfd = cfh->cfd();
XFUNC_TEST("", "managed_new", managed_new1, xf_manage_new,
reinterpret_cast<DBImpl*>(this),
const_cast<ReadOptions*>(&read_options), is_snapshot_supported_);
if (read_options.managed) {
#ifdef ROCKSDB_LITE
// not supported in lite version
@ -4505,9 +4499,6 @@ Status DBImpl::NewIterators(
}
iterators->clear();
iterators->reserve(column_families.size());
XFUNC_TEST("", "managed_new", managed_new1, xf_manage_new,
reinterpret_cast<DBImpl*>(this),
const_cast<ReadOptions*>(&read_options), is_snapshot_supported_);
if (read_options.managed) {
#ifdef ROCKSDB_LITE
return Status::InvalidArgument(
@ -4650,16 +4641,6 @@ Status DBImpl::WriteImpl(const WriteOptions& write_options,
Status status;
bool xfunc_attempted_write = false;
XFUNC_TEST("transaction", "transaction_xftest_write_impl",
xf_transaction_write1, xf_transaction_write, write_options,
immutable_db_options_, my_batch, callback, this, &status,
&xfunc_attempted_write);
if (xfunc_attempted_write) {
// Test already did the write
return status;
}
PERF_TIMER_GUARD(write_pre_and_post_process_time);
WriteThread::Writer w;
w.batch = my_batch;

@ -294,8 +294,6 @@ TEST_F(DBTestTailingIterator, TailingIteratorDeletes) {
}
TEST_F(DBTestTailingIterator, TailingIteratorPrefixSeek) {
XFUNC_TEST("", "dbtest_prefix", prefix_skip1, XFuncPoint::SetSkip,
kSkipNoPrefix);
ReadOptions read_options;
read_options.tailing = true;
@ -326,7 +324,6 @@ TEST_F(DBTestTailingIterator, TailingIteratorPrefixSeek) {
iter->Next();
ASSERT_TRUE(!iter->Valid());
XFUNC_TEST("", "dbtest_prefix", prefix_skip1, XFuncPoint::SetSkip, 0);
}
TEST_F(DBTestTailingIterator, TailingIteratorIncomplete) {
@ -614,8 +611,6 @@ TEST_F(DBTestTailingIterator, ManagedTailingIteratorDeletes) {
}
TEST_F(DBTestTailingIterator, ManagedTailingIteratorPrefixSeek) {
XFUNC_TEST("", "dbtest_prefix", prefix_skip1, XFuncPoint::SetSkip,
kSkipNoPrefix);
ReadOptions read_options;
read_options.tailing = true;
read_options.managed = true;
@ -647,7 +642,6 @@ TEST_F(DBTestTailingIterator, ManagedTailingIteratorPrefixSeek) {
iter->Next();
ASSERT_TRUE(!iter->Valid());
XFUNC_TEST("", "dbtest_prefix", prefix_skip1, XFuncPoint::SetSkip, 0);
}
TEST_F(DBTestTailingIterator, ManagedTailingIteratorIncomplete) {

@ -67,7 +67,6 @@
#include "util/testharness.h"
#include "util/testutil.h"
#include "util/thread_status_util.h"
#include "util/xfunc.h"
#include "utilities/merge_operators.h"
namespace rocksdb {

@ -229,9 +229,6 @@ Options DBTestBase::CurrentOptions(
const anon::OptionsOverride& options_override) {
// this redundant copy is to minimize code change w/o having lint error.
Options options = defaultOptions;
XFUNC_TEST("", "dbtest_options", inplace_options1, GetXFTestOptions,
reinterpret_cast<Options*>(&options),
options_override.skip_policy);
BlockBasedTableOptions table_options;
bool set_block_based_table_factory = true;
switch (option_config_) {

@ -58,7 +58,6 @@
#endif // !(defined NDEBUG) || !defined(OS_WIN)
#include "util/testharness.h"
#include "util/testutil.h"
#include "util/xfunc.h"
#include "utilities/merge_operators.h"
namespace rocksdb {
@ -120,6 +119,8 @@ struct OptionsOverride {
} // namespace anon
enum SkipPolicy { kSkipNone = 0, kSkipNoSnapshot = 1, kSkipNoPrefix = 2 };
// A hacky skip list mem table that triggers flush after number of entries.
class SpecialMemTableRep : public MemTableRep {
public:

@ -15,12 +15,10 @@
#include "db/db_impl.h"
#include "db/db_iter.h"
#include "db/dbformat.h"
#include "db/xfunc_test_points.h"
#include "rocksdb/env.h"
#include "rocksdb/slice.h"
#include "rocksdb/slice_transform.h"
#include "table/merging_iterator.h"
#include "util/xfunc.h"
namespace rocksdb {
@ -42,8 +40,6 @@ class MILock {
}
~MILock() {
this->mu_->unlock();
XFUNC_TEST("managed_xftest_release", "managed_unlock", managed_unlock1,
xf_manage_release, mi_);
}
ManagedIterator* GetManagedIterator() { return mi_; }
@ -84,8 +80,6 @@ ManagedIterator::ManagedIterator(DBImpl* db, const ReadOptions& read_options,
}
cfh_.SetCFD(cfd);
mutable_iter_ = unique_ptr<Iterator>(db->NewIterator(read_options_, &cfh_));
XFUNC_TEST("managed_xftest_dropold", "managed_create", xf_managed_create1,
xf_manage_create, this);
}
ManagedIterator::~ManagedIterator() {
@ -261,8 +255,6 @@ bool ManagedIterator::TryLock() { return in_use_.try_lock(); }
void ManagedIterator::UnLock() {
in_use_.unlock();
XFUNC_TEST("managed_xftest_release", "managed_unlock", managed_unlock1,
xf_manage_release, this);
}
} // namespace rocksdb

@ -1,145 +0,0 @@
// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
// This source code is licensed under the BSD-style license found in the
// LICENSE file in the root directory of this source tree. An additional grant
// of patent rights can be found in the PATENTS file in the same directory.
#include "db/xfunc_test_points.h"
#include "util/xfunc.h"
namespace rocksdb {
#ifdef XFUNC
void xf_manage_release(ManagedIterator* iter) {
if (!(XFuncPoint::GetSkip() & kSkipNoPrefix)) {
iter->ReleaseIter(false);
}
}
void xf_manage_create(ManagedIterator* iter) { iter->SetDropOld(false); }
void xf_manage_new(DBImpl* db, ReadOptions* read_options,
bool is_snapshot_supported) {
if ((!XFuncPoint::Check("managed_xftest_dropold") &&
(!XFuncPoint::Check("managed_xftest_release"))) ||
(!read_options->managed)) {
return;
}
if ((!read_options->tailing) && (read_options->snapshot == nullptr) &&
(!is_snapshot_supported)) {
read_options->managed = false;
return;
}
if (db->GetOptions().prefix_extractor != nullptr) {
if (strcmp(db->GetOptions().table_factory.get()->Name(), "PlainTable")) {
if (!(XFuncPoint::GetSkip() & kSkipNoPrefix)) {
read_options->total_order_seek = true;
}
} else {
read_options->managed = false;
}
}
}
class XFTransactionWriteHandler : public WriteBatch::Handler {
public:
Transaction* txn_;
DBImpl* db_impl_;
XFTransactionWriteHandler(Transaction* txn, DBImpl* db_impl)
: txn_(txn), db_impl_(db_impl) {}
virtual Status PutCF(uint32_t column_family_id, const Slice& key,
const Slice& value) override {
InstrumentedMutexLock l(&db_impl_->mutex_);
ColumnFamilyHandle* cfh = db_impl_->GetColumnFamilyHandle(column_family_id);
if (cfh == nullptr) {
return Status::InvalidArgument(
"XFUNC test could not find column family "
"handle for id ",
ToString(column_family_id));
}
txn_->Put(cfh, key, value);
return Status::OK();
}
virtual Status MergeCF(uint32_t column_family_id, const Slice& key,
const Slice& value) override {
InstrumentedMutexLock l(&db_impl_->mutex_);
ColumnFamilyHandle* cfh = db_impl_->GetColumnFamilyHandle(column_family_id);
if (cfh == nullptr) {
return Status::InvalidArgument(
"XFUNC test could not find column family "
"handle for id ",
ToString(column_family_id));
}
txn_->Merge(cfh, key, value);
return Status::OK();
}
virtual Status DeleteCF(uint32_t column_family_id,
const Slice& key) override {
InstrumentedMutexLock l(&db_impl_->mutex_);
ColumnFamilyHandle* cfh = db_impl_->GetColumnFamilyHandle(column_family_id);
if (cfh == nullptr) {
return Status::InvalidArgument(
"XFUNC test could not find column family "
"handle for id ",
ToString(column_family_id));
}
txn_->Delete(cfh, key);
return Status::OK();
}
virtual void LogData(const Slice& blob) override { txn_->PutLogData(blob); }
};
// Whenever DBImpl::Write is called, create a transaction and do the write via
// the transaction.
void xf_transaction_write(const WriteOptions& write_options,
const DBOptions& db_options, WriteBatch* my_batch,
WriteCallback* callback, DBImpl* db_impl, Status* s,
bool* write_attempted) {
if (callback != nullptr) {
// We may already be in a transaction, don't force a transaction
*write_attempted = false;
return;
}
OptimisticTransactionDB* txn_db = new OptimisticTransactionDB(db_impl);
Transaction* txn = Transaction::BeginTransaction(txn_db, write_options);
XFTransactionWriteHandler handler(txn, db_impl);
*s = my_batch->Iterate(&handler);
if (!s->ok()) {
Log(InfoLogLevel::ERROR_LEVEL, db_options.info_log,
"XFUNC test could not iterate batch. status: $s\n",
s->ToString().c_str());
}
*s = txn->Commit();
if (!s->ok()) {
Log(InfoLogLevel::ERROR_LEVEL, db_options.info_log,
"XFUNC test could not commit transaction. status: $s\n",
s->ToString().c_str());
}
*write_attempted = true;
delete txn;
delete txn_db;
}
#endif // XFUNC
} // namespace rocksdb

@ -1,33 +0,0 @@
// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
// This source code is licensed under the BSD-style license found in the
// LICENSE file in the root directory of this source tree. An additional grant
// of patent rights can be found in the PATENTS file in the same directory.
#pragma once
#include "db/db_impl.h"
#include "db/managed_iterator.h"
#include "db/write_callback.h"
#include "rocksdb/options.h"
#include "rocksdb/write_batch.h"
#include "util/xfunc.h"
namespace rocksdb {
#ifdef XFUNC
// DB-specific test points for the cross-functional test framework (see
// util/xfunc.h).
void xf_manage_release(ManagedIterator* iter);
void xf_manage_create(ManagedIterator* iter);
void xf_manage_new(DBImpl* db, ReadOptions* readoptions,
bool is_snapshot_supported);
void xf_transaction_write(const WriteOptions& write_options,
const DBOptions& db_options,
class WriteBatch* my_batch,
class WriteCallback* callback, DBImpl* db_impl,
Status* success, bool* write_attempted);
#endif // XFUNC
} // namespace rocksdb

@ -49,7 +49,6 @@ LIB_SOURCES = \
db/write_batch_base.cc \
db/write_controller.cc \
db/write_thread.cc \
db/xfunc_test_points.cc \
memtable/hash_cuckoo_rep.cc \
memtable/hash_linklist_rep.cc \
memtable/hash_skiplist_rep.cc \
@ -142,7 +141,6 @@ LIB_SOURCES = \
util/thread_status_util_debug.cc \
util/threadpool_imp.cc \
util/transaction_test_util.cc \
util/xfunc.cc \
util/xxhash.cc \
utilities/backupable/backupable_db.cc \
utilities/blob_db/blob_db.cc \
@ -359,18 +357,3 @@ JNI_NATIVE_SOURCES = \
java/rocksjni/writebatchhandlerjnicallback.cc \
java/rocksjni/write_batch_test.cc \
java/rocksjni/write_batch_with_index.cc
# Currently, we do not generate dependencies for
# java/rocksjni/write_batch_test.cc, because its dependent,
# java/include/org_rocksdb_WriteBatch.h is generated.
# TODO/FIXME: fix the above. Otherwise, the current rules would fail:
# java/rocksjni/write_batch_test.cc:13:44: fatal error: include/org_rocksdb_WriteBatch.h: No such file or directory
# #include "include/org_rocksdb_WriteBatch.h"
# These are the xfunc tests run :
XFUNC_TESTS = \
"managed_new" \
"managed_xftest_dropold" \
"managed_xftest_release" \
"inplace_lock_test" \
"transaction"

@ -33,7 +33,6 @@
#include "util/db_options.h"
#include "util/options_helper.h"
#include "util/statistics.h"
#include "util/xfunc.h"
namespace rocksdb {
@ -567,8 +566,6 @@ ReadOptions::ReadOptions()
background_purge_on_iterator_cleanup(false),
readahead_size(0),
ignore_range_deletions(false) {
XFUNC_TEST("", "managed_options", managed_options, xf_manage_options,
reinterpret_cast<ReadOptions*>(this));
}
ReadOptions::ReadOptions(bool cksum, bool cache)
@ -585,8 +582,6 @@ ReadOptions::ReadOptions(bool cksum, bool cache)
background_purge_on_iterator_cleanup(false),
readahead_size(0),
ignore_range_deletions(false) {
XFUNC_TEST("", "managed_options", managed_options, xf_manage_options,
reinterpret_cast<ReadOptions*>(this));
}
} // namespace rocksdb

@ -1,50 +0,0 @@
// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
// This source code is licensed under the BSD-style license found in the
// LICENSE file in the root directory of this source tree. An additional grant
// of patent rights can be found in the PATENTS file in the same directory.
#ifdef XFUNC
#include "util/xfunc.h"
#include <string>
#include "rocksdb/db.h"
#include "rocksdb/options.h"
#include "rocksdb/utilities/optimistic_transaction_db.h"
#include "rocksdb/write_batch.h"
namespace rocksdb {
std::string XFuncPoint::xfunc_test_;
bool XFuncPoint::initialized_ = false;
bool XFuncPoint::enabled_ = false;
int XFuncPoint::skip_policy_ = 0;
void GetXFTestOptions(Options* options, int skip_policy) {
if (XFuncPoint::Check("inplace_lock_test") &&
(!(skip_policy & kSkipNoSnapshot))) {
options->inplace_update_support = true;
}
}
void xf_manage_options(ReadOptions* read_options) {
if (!XFuncPoint::Check("managed_xftest_dropold") &&
(!XFuncPoint::Check("managed_xftest_release"))) {
return;
}
read_options->managed = true;
}
void xf_transaction_set_memtable_history(
int32_t* max_write_buffer_number_to_maintain) {
*max_write_buffer_number_to_maintain = 10;
}
void xf_transaction_clear_memtable_history(
int32_t* max_write_buffer_number_to_maintain) {
*max_write_buffer_number_to_maintain = 0;
}
} // namespace rocksdb
#endif // XFUNC

@ -1,113 +0,0 @@
// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
// This source code is licensed under the BSD-style license found in the
// LICENSE file in the root directory of this source tree. An additional grant
// of patent rights can be found in the PATENTS file in the same directory.
#pragma once
#include <cstdlib>
#include <functional>
#include <string>
#include "rocksdb/options.h"
namespace rocksdb {
/*
* If ROCKSDB_XFTEST_FORCE has a value of 1, XFUNC is forced to be defined.
* If ROCKSDB_XFTEST_FORCE has a value other than 1,
* XFUNC is forced to be undefined.
* If ROCKSDB_XFTEST_FORCE is undefined, XFUNC is defined based on NDEBUG,
* with XFUNC only being set for debug builds.
*/
#if defined(ROCKSDB_XFTEST_FORCE)
#ifndef ROCKSDB_LITE
#if (ROCKSDB_XFTEST_FORCE == 1)
#define XFUNC
#endif // ROCKSDB_XFTEST_FORCE == 1
#elif defined(NDEBUG)
#else
#define XFUNC
#endif // defined(ROCKSDB_XFTEST_FORCE)
#endif // !ROCKSDB_LITE
#ifndef XFUNC
#define XFUNC_TEST(condition, location, lfname, fname, ...)
#else
void GetXFTestOptions(Options* options, int skip_policy);
void xf_manage_options(ReadOptions* read_options);
void xf_transaction_set_memtable_history(
int32_t* max_write_buffer_number_to_maintain);
void xf_transaction_clear_memtable_history(
int32_t* max_write_buffer_number_to_maintain);
// This class provides the facility to run custom code to test a specific
// feature typically with all existing unit tests.
// A developer could specify cross functional test points in the codebase
// via XFUNC_TEST.
// Each xfunc test represents a position in the execution stream of a thread.
// Whenever that particular piece of code is called, the given cross-functional
// test point is executed.
// eg. on DBOpen, a particular option can be set.
// on Get, a particular option can be set, or a specific check can be invoked.
// XFUNC_TEST(TestName, location, lfname, FunctionName, Args)
// Turn on a specific cross functional test by setting the environment variable
// ROCKSDB_XFUNC_TEST
class XFuncPoint {
public:
// call once at the beginning of a test to get the test name
static void Init() {
char* s = getenv("ROCKSDB_XFUNC_TEST");
if (s == nullptr) {
xfunc_test_ = "";
enabled_ = false;
} else {
xfunc_test_ = s;
enabled_ = true;
}
initialized_ = true;
}
static bool Initialized() { return initialized_; }
static bool Check(std::string test) {
return (enabled_ &&
((test.compare("") == 0) || (test.compare(xfunc_test_) == 0)));
}
static void SetSkip(int skip) { skip_policy_ = skip; }
static int GetSkip(void) { return skip_policy_; }
private:
static std::string xfunc_test_;
static bool initialized_;
static bool enabled_;
static int skip_policy_;
};
// Use XFUNC_TEST to specify cross functional test points inside the code base.
// By setting ROCKSDB_XFUNC_TEST, all XFUNC_TEST having that
// value in the condition field will be executed.
// The second argument specifies a string representing the calling location
// The third argument, lfname, is the name of the function which will be created
// and called.
// The fourth argument fname represents the function to be called
// The arguments following that are the arguments to fname
// See Options::Options in options.h for an example use case.
// XFUNC_TEST is no op in release build.
#define XFUNC_TEST(condition, location, lfname, fname, ...) \
{ \
if (!XFuncPoint::Initialized()) { \
XFuncPoint::Init(); \
} \
if (XFuncPoint::Check(condition)) { \
std::function<void()> lfname = std::bind(fname, __VA_ARGS__); \
lfname(); \
} \
}
#endif // XFUNC
enum SkipPolicy { kSkipNone = 0, kSkipNoSnapshot = 1, kSkipNoPrefix = 2 };
} // namespace rocksdb

@ -25,7 +25,6 @@
#include "rocksdb/utilities/transaction_db.h"
#include "util/sync_point.h"
#include "util/testharness.h"
#include "util/xfunc.h"
namespace rocksdb {
class CheckpointTest : public testing::Test {

Loading…
Cancel
Save