Skip to content

Commit

Permalink
TableMock + framework for mock classes
Browse files Browse the repository at this point in the history
Summary:
This diff replaces BlockBasedTable in flush_job_test with TableMock, making it depend on less things and making it closer to an unit test than integration test.

It also introduces a framework to compile mock classes -- Any file named *mock.cc will not be compiled into the build. It will only get compiled into the tests. What way we can mock out most other classes, Version, VersionSet, DBImpl, etc.

Test Plan: flush_job_test

Reviewers: ljin, rven, yhchiang, sdong

Reviewed By: sdong

Subscribers: dhruba, leveldb

Differential Revision: https://reviews.facebook.net/D27681
  • Loading branch information
igorcanadi committed Oct 29, 2014
1 parent fb3f8ff commit abac3d6
Show file tree
Hide file tree
Showing 6 changed files with 286 additions and 12 deletions.
4 changes: 2 additions & 2 deletions Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -76,11 +76,11 @@ CXXFLAGS += $(WARNING_FLAGS) -I. -I./include $(PLATFORM_CXXFLAGS) $(OPT) -Woverl
LDFLAGS += $(PLATFORM_LDFLAGS)

LIBOBJECTS = $(SOURCES:.cc=.o)
LIBOBJECTS += $(SOURCESCPP:.cpp=.o)
MEMENVOBJECTS = $(MEMENV_SOURCES:.cc=.o)
MOCKOBJECTS = $(MOCK_SOURCES:.cc=.o)

TESTUTIL = ./util/testutil.o
TESTHARNESS = ./util/testharness.o $(TESTUTIL)
TESTHARNESS = ./util/testharness.o $(TESTUTIL) $(MOCKOBJECTS)
BENCHHARNESS = ./util/benchharness.o
VALGRIND_ERROR = 2
VALGRIND_DIR = build_tools/VALGRIND_LOGS
Expand Down
7 changes: 4 additions & 3 deletions build_tools/build_detect_platform
Original file line number Diff line number Diff line change
Expand Up @@ -173,14 +173,15 @@ DIRS="util db table utilities"
set -f # temporarily disable globbing so that our patterns arent expanded
PRUNE_TEST="-name *test*.cc -prune"
PRUNE_BENCH="-name *bench*.cc -prune"
PORTABLE_FILES=`cd "$ROCKSDB_ROOT"; find $DIRS $PRUNE_TEST -o $PRUNE_BENCH -o -name '*.cc' -print | sort | tr "\n" " "`
PORTABLE_CPP=`cd "$ROCKSDB_ROOT"; find $DIRS $PRUNE_TEST -o $PRUNE_BENCH -o -name '*.cpp' -print | sort | tr "\n" " "`
PRUNE_MOCK="-name *mock*.cc -prune"
PORTABLE_FILES=`cd "$ROCKSDB_ROOT"; find $DIRS $PRUNE_TEST -o $PRUNE_BENCH -o $PRUNE_MOCK -o -name '*.cc' -print | sort | tr "\n" " "`
MOCK_SOURCES=`cd "$ROCKSDB_ROOT"; find $DIRS -name '*mock.cc' -print | sort | tr "\n" " "`
set +f # re-enable globbing

# The sources consist of the portable files, plus the platform-specific port
# file.
echo "SOURCES=$PORTABLE_FILES $GENERIC_PORT_FILES $PORT_FILES" >> "$OUTPUT"
echo "SOURCESCPP=$PORTABLE_CPP" >> "$OUTPUT"
echo "MOCK_SOURCES=$MOCK_SOURCES" >> "$OUTPUT"
echo "MEMENV_SOURCES=helpers/memenv/memenv.cc" >> "$OUTPUT"

if [ "$CROSS_COMPILE" = "true" -o "$FBCODE_BUILD" = "true" ]; then
Expand Down
20 changes: 14 additions & 6 deletions db/flush_job_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -3,19 +3,22 @@
// 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 <map>
#include <string>

#include "db/flush_job.h"
#include "db/column_family.h"
#include "db/version_set.h"
#include "rocksdb/cache.h"
#include "util/testharness.h"
#include "util/testutil.h"
#include "table/mock_table.h"

namespace rocksdb {

// TODO(icanadi) Mock out everything else:
// 1. VersionSet
// 2. TableBuilder
// 3. Memtable
// 2. Memtable
class FlushJobTest {
public:
FlushJobTest()
Expand All @@ -24,14 +27,16 @@ class FlushJobTest {
table_cache_(NewLRUCache(50000, 16, 8)),
versions_(new VersionSet(dbname_, &db_options_, env_options_,
table_cache_.get(), &write_controller_)),
shutting_down_(false) {
shutting_down_(false),
mock_table_factory_(new MockTableFactory()) {
ASSERT_OK(env_->CreateDirIfMissing(dbname_));
db_options_.db_paths.emplace_back(dbname_,
std::numeric_limits<uint64_t>::max());
// TODO(icanadi) Remove this once we mock out VersionSet
NewDB();
std::vector<ColumnFamilyDescriptor> column_families;
column_families.emplace_back();
cf_options_.table_factory = mock_table_factory_;
column_families.emplace_back(kDefaultColumnFamilyName, cf_options_);

ASSERT_OK(versions_->Recover(column_families, false));
}
Expand Down Expand Up @@ -69,6 +74,7 @@ class FlushJobTest {
port::Mutex mutex_;
std::atomic<bool> shutting_down_;
FileNumToPathIdMap pending_outputs_;
std::shared_ptr<MockTableFactory> mock_table_factory_;
};

TEST(FlushJobTest, Empty) {
Expand All @@ -89,10 +95,13 @@ TEST(FlushJobTest, NonEmpty) {
auto new_mem = new MemTable(cfd->internal_comparator(), *cfd->ioptions(),
*cfd->GetLatestMutableCFOptions());
new_mem->Ref();
std::map<std::string, std::string> inserted_keys;
for (int i = 1; i < 10000; ++i) {
std::string key(std::to_string(i));
std::string value("value" + std::to_string(i));
new_mem->Add(SequenceNumber(i), kTypeValue, key, value);
InternalKey internal_key(key, SequenceNumber(i), kTypeValue);
inserted_keys.insert({internal_key.Encode().ToString(), value});
}
cfd->imm()->Add(new_mem);

Expand All @@ -104,8 +113,7 @@ TEST(FlushJobTest, NonEmpty) {
mutex_.Lock();
ASSERT_OK(flush_job.Run());
mutex_.Unlock();
// TODO(icanadi) once you have TableMock, verify that key-values are as
// expected
mock_table_factory_->AssertSingleFile(inserted_keys);
}

} // namespace rocksdb
Expand Down
1 change: 0 additions & 1 deletion db/table_cache.cc
Original file line number Diff line number Diff line change
Expand Up @@ -88,7 +88,6 @@ Status TableCache::FindTable(const EnvOptions& env_options,
// We do not cache error results so that if the error is transient,
// or somebody repairs the file, we recover automatically.
} else {
assert(file.get() == nullptr);
*handle = cache_->Insert(key, table_reader.release(), 1, &DeleteEntry);
}
}
Expand Down
95 changes: 95 additions & 0 deletions table/mock_table.cc
Original file line number Diff line number Diff line change
@@ -0,0 +1,95 @@
// Use of this source code is governed by a BSD-style license that can be
// found in the LICENSE file. See the AUTHORS file for names of contributors.
// Copyright (c) 2013, 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 "rocksdb/table_properties.h"
#include "table/mock_table.h"
#include "table/get_context.h"
#include "db/dbformat.h"
#include "port/port.h"
#include "util/coding.h"

namespace rocksdb {

Iterator* MockTableReader::NewIterator(const ReadOptions&, Arena* arena) {
return new MockTableIterator(table_);
}

Status MockTableReader::Get(const ReadOptions&, const Slice& key,
GetContext* get_context) {
std::unique_ptr<MockTableIterator> iter(new MockTableIterator(table_));
for (iter->Seek(key); iter->Valid(); iter->Next()) {
ParsedInternalKey parsed_key;
if (!ParseInternalKey(iter->key(), &parsed_key)) {
return Status::Corruption(Slice());
}

if (!get_context->SaveValue(parsed_key, iter->value())) {
break;
}
}
return Status::OK();
}

std::shared_ptr<const TableProperties> MockTableReader::GetTableProperties()
const {
return std::shared_ptr<const TableProperties>(new TableProperties());
}

MockTableFactory::MockTableFactory() : next_id_(1) {}

Status MockTableFactory::NewTableReader(
const ImmutableCFOptions& ioptions, const EnvOptions& env_options,
const InternalKeyComparator& internal_key,
unique_ptr<RandomAccessFile>&& file, uint64_t file_size,
unique_ptr<TableReader>* table_reader) const {
uint32_t id = GetIDFromFile(file.get());

MutexLock lock_guard(&file_system_.mutex);

auto it = file_system_.files.find(id);
if (it == file_system_.files.end()) {
return Status::IOError("Mock file not found");
}

table_reader->reset(new MockTableReader(it->second));

return Status::OK();
}

TableBuilder* MockTableFactory::NewTableBuilder(
const ImmutableCFOptions& ioptions,
const InternalKeyComparator& internal_key, WritableFile* file,
const CompressionType compression_type,
const CompressionOptions& compression_opts) const {
uint32_t id = GetAndWriteNextID(file);

return new MockTableBuilder(id, &file_system_);
}

uint32_t MockTableFactory::GetAndWriteNextID(WritableFile* file) const {
uint32_t next_id = next_id_.fetch_add(1);
char buf[4];
EncodeFixed32(buf, next_id);
file->Append(Slice(buf, 4));
return next_id;
}

uint32_t MockTableFactory::GetIDFromFile(RandomAccessFile* file) const {
char buf[4];
Slice result;
file->Read(0, 4, &result, buf);
assert(result.size() == 4);
return DecodeFixed32(buf);
}

void MockTableFactory::AssertSingleFile(
const std::map<std::string, std::string>& file_contents) {
ASSERT_EQ(file_system_.files.size(), 1U);
ASSERT_TRUE(file_contents == file_system_.files.begin()->second);
}

} // namespace rocksdb
171 changes: 171 additions & 0 deletions table/mock_table.h
Original file line number Diff line number Diff line change
@@ -0,0 +1,171 @@
// Use of this source code is governed by a BSD-style license that can be
// found in the LICENSE file. See the AUTHORS file for names of contributors.
// Copyright (c) 2013, 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 <algorithm>
#include <set>
#include <memory>
#include <map>
#include <string>

#include "rocksdb/table.h"
#include "table/table_reader.h"
#include "table/table_builder.h"
#include "port/port.h"
#include "util/mutexlock.h"
#include "util/testharness.h"
#include "util/testutil.h"

namespace rocksdb {

// NOTE this currently only supports bitwise comparator

struct MockTableFileSystem {
port::Mutex mutex;
std::map<uint32_t, std::map<std::string, std::string>> files;
};

class MockTableReader : public TableReader {
public:
MockTableReader(const std::map<std::string, std::string>& table)
: table_(table) {}

Iterator* NewIterator(const ReadOptions&, Arena* arena) override;

Status Get(const ReadOptions&, const Slice& key,
GetContext* get_context) override;

uint64_t ApproximateOffsetOf(const Slice& key) override { return 0; }

virtual size_t ApproximateMemoryUsage() const override { return 0; }

void SetupForCompaction() override {}

std::shared_ptr<const TableProperties> GetTableProperties() const override;

~MockTableReader() {}

private:
const std::map<std::string, std::string>& table_;
};

class MockTableIterator : public Iterator {
public:
explicit MockTableIterator(const std::map<std::string, std::string>& table)
: table_(table) {
itr_ = table_.end();
}

bool Valid() const { return itr_ == table_.end(); }

void SeekToFirst() { itr_ = table_.begin(); }

void SeekToLast() {
itr_ = table_.end();
--itr_;
}

void Seek(const Slice& target) {
std::string str_target(target.data(), target.size());
itr_ = table_.lower_bound(str_target);
}

void Next() { ++itr_; }

void Prev() {
if (itr_ == table_.begin()) {
itr_ = table_.end();
} else {
--itr_;
}
}

Slice key() const { return Slice(itr_->first); }

Slice value() const { return Slice(itr_->second); }

Status status() const { return Status::OK(); }

private:
const std::map<std::string, std::string>& table_;
std::map<std::string, std::string>::const_iterator itr_;
};

class MockTableBuilder : public TableBuilder {
public:
MockTableBuilder(uint32_t id, MockTableFileSystem* file_system)
: id_(id), file_system_(file_system) {}

// REQUIRES: Either Finish() or Abandon() has been called.
~MockTableBuilder() {}

// Add key,value to the table being constructed.
// REQUIRES: key is after any previously added key according to comparator.
// REQUIRES: Finish(), Abandon() have not been called
void Add(const Slice& key, const Slice& value) override {
table_.insert({key.ToString(), value.ToString()});
}

// Return non-ok iff some error has been detected.
Status status() const override { return Status::OK(); }

Status Finish() override {
MutexLock lock_guard(&file_system_->mutex);
file_system_->files.insert({id_, table_});
return Status::OK();
}

void Abandon() override {}

uint64_t NumEntries() const override { return table_.size(); }

uint64_t FileSize() const override { return table_.size(); }

private:
uint32_t id_;
MockTableFileSystem* file_system_;
std::map<std::string, std::string> table_;
};

class MockTableFactory : public TableFactory {
public:
MockTableFactory();
const char* Name() const override { return "MockTable"; }
Status NewTableReader(const ImmutableCFOptions& ioptions,
const EnvOptions& env_options,
const InternalKeyComparator& internal_key,
unique_ptr<RandomAccessFile>&& file, uint64_t file_size,
unique_ptr<TableReader>* table_reader) const;

TableBuilder* NewTableBuilder(
const ImmutableCFOptions& ioptions,
const InternalKeyComparator& internal_key, WritableFile* file,
const CompressionType compression_type,
const CompressionOptions& compression_opts) const;

virtual Status SanitizeOptions(const DBOptions& db_opts,
const ColumnFamilyOptions& cf_opts) const {
return Status::OK();
}

virtual std::string GetPrintableTableOptions() const override {
return std::string();
}

// This function will assert that only a single file exists and that the
// contents are equal to file_contents
void AssertSingleFile(
const std::map<std::string, std::string>& file_contents);

private:
uint32_t GetAndWriteNextID(WritableFile* file) const;
uint32_t GetIDFromFile(RandomAccessFile* file) const;

mutable MockTableFileSystem file_system_;
mutable std::atomic<uint32_t> next_id_;
};

} // namespace rocksdb

0 comments on commit abac3d6

Please sign in to comment.