提交 588bca20 编写于 作者: I Igor Canadi

RocksDBLite

Summary:
Introducing RocksDBLite! Removes all the non-essential features and reduces the binary size. This effort should help our adoption on mobile.

Binary size when compiling for IOS (`TARGET_OS=IOS m static_lib`) is down to 9MB from 15MB (without stripping)

Test Plan: compiles :)

Reviewers: dhruba, haobo, ljin, sdong, yhchiang

Reviewed By: yhchiang

CC: leveldb

Differential Revision: https://reviews.facebook.net/D17835
上级 23c8f89b
...@@ -15,7 +15,12 @@ endif ...@@ -15,7 +15,12 @@ endif
ifeq ($(MAKECMDGOALS),shared_lib) ifeq ($(MAKECMDGOALS),shared_lib)
PLATFORM_SHARED_LDFLAGS=-fPIC PLATFORM_SHARED_LDFLAGS=-fPIC
OPT += -DNDEBUG
endif
ifeq ($(MAKECMDGOALS),static_lib)
OPT += -DNDEBUG
endif endif
#----------------------------------------------- #-----------------------------------------------
# detect what platform we're building on # detect what platform we're building on
......
# RocksDBLite
RocksDBLite is a project focused on mobile use cases, which don't need a lot of fancy things we've built for server workloads and they are very sensitive to binary size. For that reason, we added a compile flag ROCKSDB_LITE that comments out a lot of the nonessential code and keeps the binary lean.
Some examples of the features disabled by ROCKSDB_LITE:
* compiled-in support for LDB tool
* No backupable DB
* No support for replication (which we provide in form of TrasactionalIterator)
* No advanced monitoring tools
* No special-purpose memtables that are highly optimized for specific use cases
When adding a new big feature to RocksDB, please add ROCKSDB_LITE compile guard if:
* Nobody from mobile really needs your feature,
* Your feature is adding a lot of weight to the binary.
Don't add ROCKSDB_LITE compile guard if:
* It would introduce a lot of code complexity. Compile guards make code harder to read. It's a trade-off.
* Your feature is not adding a lot of weight.
If unsure, ask. :)
...@@ -100,7 +100,7 @@ case "$TARGET_OS" in ...@@ -100,7 +100,7 @@ case "$TARGET_OS" in
;; ;;
IOS) IOS)
PLATFORM=IOS PLATFORM=IOS
COMMON_FLAGS="$COMMON_FLAGS -DOS_MACOSX -DIOS_CROSS_COMPILE" COMMON_FLAGS="$COMMON_FLAGS -DOS_MACOSX -DIOS_CROSS_COMPILE -DROCKSDB_LITE"
PLATFORM_SHARED_EXT=dylib PLATFORM_SHARED_EXT=dylib
PLATFORM_SHARED_LDFLAGS="-dynamiclib -install_name " PLATFORM_SHARED_LDFLAGS="-dynamiclib -install_name "
CROSS_COMPILE=true CROSS_COMPILE=true
......
...@@ -7,6 +7,8 @@ ...@@ -7,6 +7,8 @@
// Use of this source code is governed by a BSD-style license that can be // 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. // found in the LICENSE file. See the AUTHORS file for names of contributors.
#ifndef ROCKSDB_LITE
#include "rocksdb/c.h" #include "rocksdb/c.h"
#include <stdlib.h> #include <stdlib.h>
...@@ -1467,3 +1469,5 @@ extern void rocksdb_livefiles_destroy( ...@@ -1467,3 +1469,5 @@ extern void rocksdb_livefiles_destroy(
} }
} // end extern "C" } // end extern "C"
#endif // ROCKSDB_LITE
...@@ -7,6 +7,8 @@ ...@@ -7,6 +7,8 @@
// Use of this source code is governed by a BSD-style license that can be // Use of this source code is governed by a BSD-style license that can be
// found in the LICENSE file. // found in the LICENSE file.
#ifndef ROCKSDB_LITE
#define __STDC_FORMAT_MACROS #define __STDC_FORMAT_MACROS
#include <inttypes.h> #include <inttypes.h>
#include <algorithm> #include <algorithm>
...@@ -166,3 +168,5 @@ Status DBImpl::GetSortedWalFiles(VectorLogPtr& files) { ...@@ -166,3 +168,5 @@ Status DBImpl::GetSortedWalFiles(VectorLogPtr& files) {
} }
} }
#endif // ROCKSDB_LITE
此差异已折叠。
...@@ -115,6 +115,10 @@ class DBImpl : public DB { ...@@ -115,6 +115,10 @@ class DBImpl : public DB {
using DB::Flush; using DB::Flush;
virtual Status Flush(const FlushOptions& options, virtual Status Flush(const FlushOptions& options,
ColumnFamilyHandle* column_family); ColumnFamilyHandle* column_family);
virtual SequenceNumber GetLatestSequenceNumber() const;
#ifndef ROCKSDB_LITE
virtual Status DisableFileDeletions(); virtual Status DisableFileDeletions();
virtual Status EnableFileDeletions(bool force); virtual Status EnableFileDeletions(bool force);
// All the returned filenames start with "/" // All the returned filenames start with "/"
...@@ -122,7 +126,7 @@ class DBImpl : public DB { ...@@ -122,7 +126,7 @@ class DBImpl : public DB {
uint64_t* manifest_file_size, uint64_t* manifest_file_size,
bool flush_memtable = true); bool flush_memtable = true);
virtual Status GetSortedWalFiles(VectorLogPtr& files); virtual Status GetSortedWalFiles(VectorLogPtr& files);
virtual SequenceNumber GetLatestSequenceNumber() const;
virtual Status GetUpdatesSince( virtual Status GetUpdatesSince(
SequenceNumber seq_number, unique_ptr<TransactionLogIterator>* iter, SequenceNumber seq_number, unique_ptr<TransactionLogIterator>* iter,
const TransactionLogIterator::ReadOptions& const TransactionLogIterator::ReadOptions&
...@@ -130,6 +134,7 @@ class DBImpl : public DB { ...@@ -130,6 +134,7 @@ class DBImpl : public DB {
virtual Status DeleteFile(std::string name); virtual Status DeleteFile(std::string name);
virtual void GetLiveFilesMetaData(std::vector<LiveFileMetaData>* metadata); virtual void GetLiveFilesMetaData(std::vector<LiveFileMetaData>* metadata);
#endif // ROCKSDB_LITE
// checks if all live files exist on file system and that their file sizes // checks if all live files exist on file system and that their file sizes
// match to our in-memory records // match to our in-memory records
...@@ -141,7 +146,9 @@ class DBImpl : public DB { ...@@ -141,7 +146,9 @@ class DBImpl : public DB {
int output_level, const Slice* begin, int output_level, const Slice* begin,
const Slice* end); const Slice* end);
#ifndef NDEBUG
// Extra methods (for testing) that are not in the public DB interface // Extra methods (for testing) that are not in the public DB interface
// Implemented in db_impl_debug.cc
// Compact any files in the named level that overlap [*begin, *end] // Compact any files in the named level that overlap [*begin, *end]
Status TEST_CompactRange(int level, const Slice* begin, const Slice* end, Status TEST_CompactRange(int level, const Slice* begin, const Slice* end,
...@@ -184,6 +191,8 @@ class DBImpl : public DB { ...@@ -184,6 +191,8 @@ class DBImpl : public DB {
void TEST_GetFilesMetaData(ColumnFamilyHandle* column_family, void TEST_GetFilesMetaData(ColumnFamilyHandle* column_family,
std::vector<std::vector<FileMetaData>>* metadata); std::vector<std::vector<FileMetaData>>* metadata);
#endif // NDEBUG
// needed for CleanupIteratorState // needed for CleanupIteratorState
struct DeletionState { struct DeletionState {
inline bool HaveSomethingToDelete() const { inline bool HaveSomethingToDelete() const {
...@@ -270,7 +279,9 @@ class DBImpl : public DB { ...@@ -270,7 +279,9 @@ class DBImpl : public DB {
private: private:
friend class DB; friend class DB;
friend class InternalStats; friend class InternalStats;
#ifndef ROCKSDB_LITE
friend class TailingIterator; friend class TailingIterator;
#endif
friend struct SuperVersion; friend struct SuperVersion;
struct CompactionState; struct CompactionState;
struct Writer; struct Writer;
...@@ -326,8 +337,11 @@ class DBImpl : public DB { ...@@ -326,8 +337,11 @@ class DBImpl : public DB {
Status WaitForFlushMemTable(ColumnFamilyData* cfd); Status WaitForFlushMemTable(ColumnFamilyData* cfd);
void MaybeScheduleLogDBDeployStats(); void MaybeScheduleLogDBDeployStats();
#ifndef ROCKSDB_LITE
static void BGLogDBDeployStats(void* db); static void BGLogDBDeployStats(void* db);
void LogDBDeployStats(); void LogDBDeployStats();
#endif // ROCKSDB_LITE
void MaybeScheduleFlushOrCompaction(); void MaybeScheduleFlushOrCompaction();
static void BGWorkCompaction(void* db); static void BGWorkCompaction(void* db);
...@@ -375,6 +389,12 @@ class DBImpl : public DB { ...@@ -375,6 +389,12 @@ class DBImpl : public DB {
void AllocateCompactionOutputFileNumbers(CompactionState* compact); void AllocateCompactionOutputFileNumbers(CompactionState* compact);
void ReleaseCompactionUnusedFileNumbers(CompactionState* compact); void ReleaseCompactionUnusedFileNumbers(CompactionState* compact);
#ifdef ROCKSDB_LITE
void PurgeObsoleteWALFiles() {
// this function is used for archiving WAL files. we don't need this in
// ROCKSDB_LITE
}
#else
void PurgeObsoleteWALFiles(); void PurgeObsoleteWALFiles();
Status GetSortedWalsOfType(const std::string& path, Status GetSortedWalsOfType(const std::string& path,
...@@ -394,6 +414,7 @@ class DBImpl : public DB { ...@@ -394,6 +414,7 @@ class DBImpl : public DB {
WriteBatch* const result); WriteBatch* const result);
Status ReadFirstLine(const std::string& fname, WriteBatch* const batch); Status ReadFirstLine(const std::string& fname, WriteBatch* const batch);
#endif // ROCKSDB_LITE
void PrintStatistics(); void PrintStatistics();
...@@ -540,10 +561,12 @@ class DBImpl : public DB { ...@@ -540,10 +561,12 @@ class DBImpl : public DB {
void InstallSuperVersion(ColumnFamilyData* cfd, void InstallSuperVersion(ColumnFamilyData* cfd,
DeletionState& deletion_state); DeletionState& deletion_state);
#ifndef ROCKSDB_LITE
using DB::GetPropertiesOfAllTables; using DB::GetPropertiesOfAllTables;
virtual Status GetPropertiesOfAllTables(ColumnFamilyHandle* column_family, virtual Status GetPropertiesOfAllTables(ColumnFamilyHandle* column_family,
TablePropertiesCollection* props) TablePropertiesCollection* props)
override; override;
#endif // ROCKSDB_LITE
// Function that Get and KeyMayExist call with no_io true or false // Function that Get and KeyMayExist call with no_io true or false
// Note: 'value_found' from KeyMayExist propagates here // Note: 'value_found' from KeyMayExist propagates here
......
// 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.
//
// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
// 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.
#ifndef NDEBUG
#include "db/db_impl.h"
void DBImpl::TEST_PurgeObsoleteteWAL() { PurgeObsoleteWALFiles(); }
uint64_t DBImpl::TEST_GetLevel0TotalSize() {
MutexLock l(&mutex_);
return default_cf_handle_->cfd()->current()->NumLevelBytes(0);
}
Iterator* DBImpl::TEST_NewInternalIterator(ColumnFamilyHandle* column_family) {
ColumnFamilyData* cfd;
if (column_family == nullptr) {
cfd = default_cf_handle_->cfd();
} else {
auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
cfd = cfh->cfd();
}
mutex_.Lock();
SuperVersion* super_version = cfd->GetSuperVersion()->Ref();
mutex_.Unlock();
ReadOptions roptions;
roptions.prefix_seek = true;
return NewInternalIterator(roptions, cfd, super_version);
}
int64_t DBImpl::TEST_MaxNextLevelOverlappingBytes(
ColumnFamilyHandle* column_family) {
ColumnFamilyData* cfd;
if (column_family == nullptr) {
cfd = default_cf_handle_->cfd();
} else {
auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
cfd = cfh->cfd();
}
MutexLock l(&mutex_);
return cfd->current()->MaxNextLevelOverlappingBytes();
}
void DBImpl::TEST_GetFilesMetaData(
ColumnFamilyHandle* column_family,
std::vector<std::vector<FileMetaData>>* metadata) {
auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
auto cfd = cfh->cfd();
MutexLock l(&mutex_);
metadata->resize(NumberLevels());
for (int level = 0; level < NumberLevels(); level++) {
const std::vector<FileMetaData*>& files = cfd->current()->files_[level];
(*metadata)[level].clear();
for (const auto& f : files) {
(*metadata)[level].push_back(*f);
}
}
}
uint64_t DBImpl::TEST_Current_Manifest_FileNo() {
return versions_->ManifestFileNumber();
}
Status DBImpl::TEST_CompactRange(int level, const Slice* begin,
const Slice* end,
ColumnFamilyHandle* column_family) {
ColumnFamilyData* cfd;
if (column_family == nullptr) {
cfd = default_cf_handle_->cfd();
} else {
auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
cfd = cfh->cfd();
}
int output_level =
(cfd->options()->compaction_style == kCompactionStyleUniversal)
? level
: level + 1;
return RunManualCompaction(cfd, level, output_level, begin, end);
}
Status DBImpl::TEST_FlushMemTable(bool wait) {
FlushOptions fo;
fo.wait = wait;
return FlushMemTable(default_cf_handle_->cfd(), fo);
}
Status DBImpl::TEST_WaitForFlushMemTable(ColumnFamilyHandle* column_family) {
ColumnFamilyData* cfd;
if (column_family == nullptr) {
cfd = default_cf_handle_->cfd();
} else {
auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
cfd = cfh->cfd();
}
return WaitForFlushMemTable(cfd);
}
Status DBImpl::TEST_WaitForCompact() {
// Wait until the compaction completes
// TODO: a bug here. This function actually does not necessarily
// wait for compact. It actually waits for scheduled compaction
// OR flush to finish.
MutexLock l(&mutex_);
while ((bg_compaction_scheduled_ || bg_flush_scheduled_) && bg_error_.ok()) {
bg_cv_.Wait();
}
return bg_error_;
}
#endif // NDEBUG
...@@ -20,7 +20,8 @@ ...@@ -20,7 +20,8 @@
namespace rocksdb { namespace rocksdb {
void DBImpl::MaybeScheduleLogDBDeployStats() { void DBImpl::MaybeScheduleLogDBDeployStats() {
// we did say maybe
#ifndef ROCKSDB_LITE
// There is a lock in the actual logger. // There is a lock in the actual logger.
if (!logger_ || options_.db_stats_log_interval < 0 if (!logger_ || options_.db_stats_log_interval < 0
|| host_name_.empty()) { || host_name_.empty()) {
...@@ -89,6 +90,6 @@ void DBImpl::LogDBDeployStats() { ...@@ -89,6 +90,6 @@ void DBImpl::LogDBDeployStats() {
bg_logstats_scheduled_ = false; bg_logstats_scheduled_ = false;
bg_cv_.SignalAll(); bg_cv_.SignalAll();
mutex_.Unlock(); mutex_.Unlock();
#endif
} }
} }
...@@ -29,6 +29,8 @@ ...@@ -29,6 +29,8 @@
// Store per-table metadata (smallest, largest, largest-seq#, ...) // Store per-table metadata (smallest, largest, largest-seq#, ...)
// in the table's meta section to speed up ScanTable. // in the table's meta section to speed up ScanTable.
#ifndef ROCKSDB_LITE
#include "db/builder.h" #include "db/builder.h"
#include "db/db_impl.h" #include "db/db_impl.h"
#include "db/dbformat.h" #include "db/dbformat.h"
...@@ -396,3 +398,5 @@ Status RepairDB(const std::string& dbname, const Options& options) { ...@@ -396,3 +398,5 @@ Status RepairDB(const std::string& dbname, const Options& options) {
} }
} // namespace rocksdb } // namespace rocksdb
#endif // ROCKSDB_LITE
...@@ -3,6 +3,7 @@ ...@@ -3,6 +3,7 @@
// LICENSE file in the root directory of this source tree. An additional grant // 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. // of patent rights can be found in the PATENTS file in the same directory.
#ifndef ROCKSDB_LITE
#include "db/tailing_iter.h" #include "db/tailing_iter.h"
#include <string> #include <string>
...@@ -217,3 +218,4 @@ void TailingIterator::SeekImmutable(const Slice& target) { ...@@ -217,3 +218,4 @@ void TailingIterator::SeekImmutable(const Slice& target) {
} }
} // namespace rocksdb } // namespace rocksdb
#endif // ROCKSDB_LITE
...@@ -2,6 +2,7 @@ ...@@ -2,6 +2,7 @@
// This source code is licensed under the BSD-style license found in the // 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 // 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. // of patent rights can be found in the PATENTS file in the same directory.
#ifndef ROCKSDB_LITE
#pragma once #pragma once
#include <string> #include <string>
...@@ -92,3 +93,4 @@ class TailingIterator : public Iterator { ...@@ -92,3 +93,4 @@ class TailingIterator : public Iterator {
}; };
} // namespace rocksdb } // namespace rocksdb
#endif // ROCKSDB_LITE
...@@ -2,7 +2,8 @@ ...@@ -2,7 +2,8 @@
// This source code is licensed under the BSD-style license found in the // 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 // 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. // of patent rights can be found in the PATENTS file in the same directory.
//
#ifndef ROCKSDB_LITE
#include "db/transaction_log_impl.h" #include "db/transaction_log_impl.h"
#include "db/write_batch_internal.h" #include "db/write_batch_internal.h"
...@@ -257,3 +258,4 @@ Status TransactionLogIteratorImpl::OpenLogReader(const LogFile* logFile) { ...@@ -257,3 +258,4 @@ Status TransactionLogIteratorImpl::OpenLogReader(const LogFile* logFile) {
return Status::OK(); return Status::OK();
} }
} // namespace rocksdb } // namespace rocksdb
#endif // ROCKSDB_LITE
...@@ -2,7 +2,8 @@ ...@@ -2,7 +2,8 @@
// This source code is licensed under the BSD-style license found in the // 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 // 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. // of patent rights can be found in the PATENTS file in the same directory.
//
#ifndef ROCKSDB_LITE
#pragma once #pragma once
#include <vector> #include <vector>
...@@ -116,3 +117,4 @@ class TransactionLogIteratorImpl : public TransactionLogIterator { ...@@ -116,3 +117,4 @@ class TransactionLogIteratorImpl : public TransactionLogIterator {
Status OpenLogReader(const LogFile* file); Status OpenLogReader(const LogFile* file);
}; };
} // namespace rocksdb } // namespace rocksdb
#endif // ROCKSDB_LITE
...@@ -2170,6 +2170,7 @@ Status VersionSet::ListColumnFamilies(std::vector<std::string>* column_families, ...@@ -2170,6 +2170,7 @@ Status VersionSet::ListColumnFamilies(std::vector<std::string>* column_families,
return s; return s;
} }
#ifndef ROCKSDB_LITE
Status VersionSet::ReduceNumberOfLevels(const std::string& dbname, Status VersionSet::ReduceNumberOfLevels(const std::string& dbname,
const Options* options, const Options* options,
const EnvOptions& storage_options, const EnvOptions& storage_options,
...@@ -2430,6 +2431,7 @@ Status VersionSet::DumpManifest(Options& options, std::string& dscname, ...@@ -2430,6 +2431,7 @@ Status VersionSet::DumpManifest(Options& options, std::string& dscname,
return s; return s;
} }
#endif // ROCKSDB_LITE
void VersionSet::MarkFileNumberUsed(uint64_t number) { void VersionSet::MarkFileNumberUsed(uint64_t number) {
if (next_file_number_ <= number) { if (next_file_number_ <= number) {
......
...@@ -319,6 +319,7 @@ class VersionSet { ...@@ -319,6 +319,7 @@ class VersionSet {
static Status ListColumnFamilies(std::vector<std::string>* column_families, static Status ListColumnFamilies(std::vector<std::string>* column_families,
const std::string& dbname, Env* env); const std::string& dbname, Env* env);
#ifndef ROCKSDB_LITE
// Try to reduce the number of levels. This call is valid when // Try to reduce the number of levels. This call is valid when
// only one level from the new max level to the old // only one level from the new max level to the old
// max level containing files. // max level containing files.
...@@ -333,6 +334,12 @@ class VersionSet { ...@@ -333,6 +334,12 @@ class VersionSet {
const EnvOptions& storage_options, const EnvOptions& storage_options,
int new_levels); int new_levels);
// printf contents (for debugging)
Status DumpManifest(Options& options, std::string& manifestFileName,
bool verbose, bool hex = false);
#endif // ROCKSDB_LITE
// Return the current manifest file number // Return the current manifest file number
uint64_t ManifestFileNumber() const { return manifest_file_number_; } uint64_t ManifestFileNumber() const { return manifest_file_number_; }
...@@ -393,10 +400,6 @@ class VersionSet { ...@@ -393,10 +400,6 @@ class VersionSet {
// "key" as of version "v". // "key" as of version "v".
uint64_t ApproximateOffsetOf(Version* v, const InternalKey& key); uint64_t ApproximateOffsetOf(Version* v, const InternalKey& key);
// printf contents (for debugging)
Status DumpManifest(Options& options, std::string& manifestFileName,
bool verbose, bool hex = false);
// Return the size of the current manifest file // Return the size of the current manifest file
uint64_t ManifestFileSize() const { return manifest_file_size_; } uint64_t ManifestFileSize() const { return manifest_file_size_; }
......
...@@ -381,6 +381,11 @@ class DB { ...@@ -381,6 +381,11 @@ class DB {
return Flush(options, DefaultColumnFamily()); return Flush(options, DefaultColumnFamily());
} }
// The sequence number of the most recent transaction.
virtual SequenceNumber GetLatestSequenceNumber() const = 0;
#ifndef ROCKSDB_LITE
// Prevent file deletions. Compactions will continue to occur, // Prevent file deletions. Compactions will continue to occur,
// but no obsolete files will be deleted. Calling this multiple // but no obsolete files will be deleted. Calling this multiple
// times have the same effect as calling it once. // times have the same effect as calling it once.
...@@ -422,9 +427,6 @@ class DB { ...@@ -422,9 +427,6 @@ class DB {
// Retrieve the sorted list of all wal files with earliest file first // Retrieve the sorted list of all wal files with earliest file first
virtual Status GetSortedWalFiles(VectorLogPtr& files) = 0; virtual Status GetSortedWalFiles(VectorLogPtr& files) = 0;
// The sequence number of the most recent transaction.
virtual SequenceNumber GetLatestSequenceNumber() const = 0;
// Sets iter to an iterator that is positioned at a write-batch containing // Sets iter to an iterator that is positioned at a write-batch containing
// seq_number. If the sequence number is non existent, it returns an iterator // seq_number. If the sequence number is non existent, it returns an iterator
// at the first available seq_no after the requested seq_no // at the first available seq_no after the requested seq_no
...@@ -447,6 +449,8 @@ class DB { ...@@ -447,6 +449,8 @@ class DB {
// and end key // and end key
virtual void GetLiveFilesMetaData(std::vector<LiveFileMetaData>* metadata) {} virtual void GetLiveFilesMetaData(std::vector<LiveFileMetaData>* metadata) {}
#endif // ROCKSDB_LITE
// Sets the globally unique ID created at database creation time by invoking // Sets the globally unique ID created at database creation time by invoking
// Env::GenerateUniqueId(), in identity. Returns Status::OK if identity could // Env::GenerateUniqueId(), in identity. Returns Status::OK if identity could
// be set properly // be set properly
...@@ -455,11 +459,13 @@ class DB { ...@@ -455,11 +459,13 @@ class DB {
// Returns default column family handle // Returns default column family handle
virtual ColumnFamilyHandle* DefaultColumnFamily() const = 0; virtual ColumnFamilyHandle* DefaultColumnFamily() const = 0;
#ifndef ROCKSDB_LITE
virtual Status GetPropertiesOfAllTables(ColumnFamilyHandle* column_family, virtual Status GetPropertiesOfAllTables(ColumnFamilyHandle* column_family,
TablePropertiesCollection* props) = 0; TablePropertiesCollection* props) = 0;
Status GetPropertiesOfAllTables(TablePropertiesCollection* props) { Status GetPropertiesOfAllTables(TablePropertiesCollection* props) {
return GetPropertiesOfAllTables(DefaultColumnFamily(), props); return GetPropertiesOfAllTables(DefaultColumnFamily(), props);
} }
#endif // ROCKSDB_LITE
private: private:
// No copying allowed // No copying allowed
...@@ -471,11 +477,13 @@ class DB { ...@@ -471,11 +477,13 @@ class DB {
// Be very careful using this method. // Be very careful using this method.
Status DestroyDB(const std::string& name, const Options& options); Status DestroyDB(const std::string& name, const Options& options);
#ifndef ROCKSDB_LITE
// If a DB cannot be opened, you may attempt to call this method to // If a DB cannot be opened, you may attempt to call this method to
// resurrect as much of the contents of the database as possible. // resurrect as much of the contents of the database as possible.
// Some data may be lost, so be careful when calling this function // Some data may be lost, so be careful when calling this function
// on a database that contains important information. // on a database that contains important information.
Status RepairDB(const std::string& dbname, const Options& options); Status RepairDB(const std::string& dbname, const Options& options);
#endif
} // namespace rocksdb } // namespace rocksdb
......
...@@ -2,8 +2,8 @@ ...@@ -2,8 +2,8 @@
// This source code is licensed under the BSD-style license found in the // 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 // 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. // of patent rights can be found in the PATENTS file in the same directory.
#ifndef STORAGE_ROCKSDB_INCLUDE_LDB_TOOL_H #ifndef ROCKSDB_LITE
#define STORAGE_ROCKSDB_INCLUDE_LDB_TOOL_H #pragma once
#include "rocksdb/options.h" #include "rocksdb/options.h"
namespace rocksdb { namespace rocksdb {
...@@ -15,4 +15,4 @@ class LDBTool { ...@@ -15,4 +15,4 @@ class LDBTool {
} // namespace rocksdb } // namespace rocksdb
#endif // STORAGE_ROCKSDB_INCLUDE_LDB_TOOL_H #endif // ROCKSDB_LITE
...@@ -177,6 +177,16 @@ class MemTableRepFactory { ...@@ -177,6 +177,16 @@ class MemTableRepFactory {
virtual const char* Name() const = 0; virtual const char* Name() const = 0;
}; };
// This uses a skip list to store keys. It is the default.
class SkipListFactory : public MemTableRepFactory {
public:
virtual MemTableRep* CreateMemTableRep(const MemTableRep::KeyComparator&,
Arena*,
const SliceTransform*) override;
virtual const char* Name() const override { return "SkipListFactory"; }
};
#ifndef ROCKSDB_LITE
// This creates MemTableReps that are backed by an std::vector. On iteration, // This creates MemTableReps that are backed by an std::vector. On iteration,
// the vector is sorted. This is useful for workloads where iteration is very // the vector is sorted. This is useful for workloads where iteration is very
// rare and writes are generally not issued after reads begin. // rare and writes are generally not issued after reads begin.
...@@ -198,17 +208,6 @@ class VectorRepFactory : public MemTableRepFactory { ...@@ -198,17 +208,6 @@ class VectorRepFactory : public MemTableRepFactory {
} }
}; };
// This uses a skip list to store keys. It is the default.
class SkipListFactory : public MemTableRepFactory {
public:
virtual MemTableRep* CreateMemTableRep(
const MemTableRep::KeyComparator&, Arena*,
const SliceTransform*) override;
virtual const char* Name() const override {
return "SkipListFactory";
}
};
// This class contains a fixed array of buckets, each // This class contains a fixed array of buckets, each
// pointing to a skiplist (null if the bucket is empty). // pointing to a skiplist (null if the bucket is empty).
// bucket_count: number of fixed array buckets // bucket_count: number of fixed array buckets
...@@ -227,4 +226,6 @@ extern MemTableRepFactory* NewHashSkipListRepFactory( ...@@ -227,4 +226,6 @@ extern MemTableRepFactory* NewHashSkipListRepFactory(
extern MemTableRepFactory* NewHashLinkListRepFactory( extern MemTableRepFactory* NewHashLinkListRepFactory(
size_t bucket_count = 50000); size_t bucket_count = 50000);
#endif // ROCKSDB_LITE
} // namespace rocksdb } // namespace rocksdb
...@@ -850,6 +850,7 @@ struct ReadOptions { ...@@ -850,6 +850,7 @@ struct ReadOptions {
// added data) and is optimized for sequential reads. It will return records // added data) and is optimized for sequential reads. It will return records
// that were inserted into the database after the creation of the iterator. // that were inserted into the database after the creation of the iterator.
// Default: false // Default: false
// Not supported in ROCKSDB_LITE mode!
bool tailing; bool tailing;
ReadOptions() ReadOptions()
......
...@@ -81,6 +81,7 @@ struct BlockBasedTablePropertyNames { ...@@ -81,6 +81,7 @@ struct BlockBasedTablePropertyNames {
extern TableFactory* NewBlockBasedTableFactory( extern TableFactory* NewBlockBasedTableFactory(
const BlockBasedTableOptions& table_options = BlockBasedTableOptions()); const BlockBasedTableOptions& table_options = BlockBasedTableOptions());
#ifndef ROCKSDB_LITE
// -- Plain Table with prefix-only seek // -- Plain Table with prefix-only seek
// For this factory, you need to set Options.prefix_extrator properly to make it // For this factory, you need to set Options.prefix_extrator properly to make it
// work. Look-up will starts with prefix hash lookup for key prefix. Inside the // work. Look-up will starts with prefix hash lookup for key prefix. Inside the
...@@ -120,6 +121,8 @@ extern TableFactory* NewTotalOrderPlainTableFactory( ...@@ -120,6 +121,8 @@ extern TableFactory* NewTotalOrderPlainTableFactory(
uint32_t user_key_len = kPlainTableVariableLength, uint32_t user_key_len = kPlainTableVariableLength,
int bloom_bits_per_key = 0, size_t index_sparseness = 16); int bloom_bits_per_key = 0, size_t index_sparseness = 16);
#endif // ROCKSDB_LITE
// A base class for table factories. // A base class for table factories.
class TableFactory { class TableFactory {
public: public:
......
...@@ -7,6 +7,7 @@ ...@@ -7,6 +7,7 @@
// Use of this source code is governed by a BSD-style license that can be // 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. // found in the LICENSE file. See the AUTHORS file for names of contributors.
#ifndef ROCKSDB_LITE
#pragma once #pragma once
#include "utilities/stackable_db.h" #include "utilities/stackable_db.h"
#include "rocksdb/env.h" #include "rocksdb/env.h"
...@@ -210,3 +211,4 @@ class RestoreBackupableDB { ...@@ -210,3 +211,4 @@ class RestoreBackupableDB {
}; };
} // rocksdb namespace } // rocksdb namespace
#endif // ROCKSDB_LITE
...@@ -4,6 +4,7 @@ ...@@ -4,6 +4,7 @@
// of patent rights can be found in the PATENTS file in the same directory. // of patent rights can be found in the PATENTS file in the same directory.
// //
#ifndef ROCKSDB_LITE
#pragma once #pragma once
#include <string> #include <string>
#include <vector> #include <vector>
...@@ -101,3 +102,4 @@ class GeoDB : public StackableDB { ...@@ -101,3 +102,4 @@ class GeoDB : public StackableDB {
}; };
} // namespace rocksdb } // namespace rocksdb
#endif // ROCKSDB_LITE
...@@ -2,6 +2,7 @@ ...@@ -2,6 +2,7 @@
// Use of this source code is governed by a BSD-style license that can be // 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. // found in the LICENSE file. See the AUTHORS file for names of contributors.
#ifndef ROCKSDB_LITE
#pragma once #pragma once
#include "stackable_db.h" #include "stackable_db.h"
...@@ -48,3 +49,4 @@ class UtilityDB { ...@@ -48,3 +49,4 @@ class UtilityDB {
}; };
} // namespace rocksdb } // namespace rocksdb
#endif // ROCKSDB_LITE
...@@ -2,6 +2,7 @@ ...@@ -2,6 +2,7 @@
// Use of this source code is governed by a BSD-style license that can be // 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. // found in the LICENSE file. See the AUTHORS file for names of contributors.
#ifndef ROCKSDB_LITE
#include "table/plain_table_builder.h" #include "table/plain_table_builder.h"
#include <assert.h> #include <assert.h>
...@@ -205,3 +206,4 @@ uint64_t PlainTableBuilder::FileSize() const { ...@@ -205,3 +206,4 @@ uint64_t PlainTableBuilder::FileSize() const {
} }
} // namespace rocksdb } // namespace rocksdb
#endif // ROCKSDB_LITE
...@@ -5,6 +5,7 @@ ...@@ -5,6 +5,7 @@
// IndexedTable is a simple table format for UNIT TEST ONLY. It is not built // IndexedTable is a simple table format for UNIT TEST ONLY. It is not built
// as production quality. // as production quality.
#ifndef ROCKSDB_LITE
#pragma once #pragma once
#include <stdint.h> #include <stdint.h>
#include "rocksdb/options.h" #include "rocksdb/options.h"
...@@ -80,3 +81,4 @@ private: ...@@ -80,3 +81,4 @@ private:
} // namespace rocksdb } // namespace rocksdb
#endif // ROCKSDB_LITE
...@@ -2,6 +2,7 @@ ...@@ -2,6 +2,7 @@
// Use of this source code is governed by a BSD-style license that can be // 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. // found in the LICENSE file. See the AUTHORS file for names of contributors.
#ifndef ROCKSDB_LITE
#include "table/plain_table_factory.h" #include "table/plain_table_factory.h"
#include <memory> #include <memory>
...@@ -46,3 +47,4 @@ extern TableFactory* NewTotalOrderPlainTableFactory(uint32_t user_key_len, ...@@ -46,3 +47,4 @@ extern TableFactory* NewTotalOrderPlainTableFactory(uint32_t user_key_len,
} }
} // namespace rocksdb } // namespace rocksdb
#endif // ROCKSDB_LITE
...@@ -2,6 +2,7 @@ ...@@ -2,6 +2,7 @@
// Use of this source code is governed by a BSD-style license that can be // 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. // found in the LICENSE file. See the AUTHORS file for names of contributors.
#ifndef ROCKSDB_LITE
#pragma once #pragma once
#include <memory> #include <memory>
#include <stdint.h> #include <stdint.h>
...@@ -83,3 +84,4 @@ class PlainTableFactory : public TableFactory { ...@@ -83,3 +84,4 @@ class PlainTableFactory : public TableFactory {
}; };
} // namespace rocksdb } // namespace rocksdb
#endif // ROCKSDB_LITE
...@@ -2,6 +2,7 @@ ...@@ -2,6 +2,7 @@
// Use of this source code is governed by a BSD-style license that can be // 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. // found in the LICENSE file. See the AUTHORS file for names of contributors.
#ifndef ROCKSDB_LITE
#include "table/plain_table_reader.h" #include "table/plain_table_reader.h"
#include <string> #include <string>
...@@ -745,3 +746,4 @@ Status PlainTableIterator::status() const { ...@@ -745,3 +746,4 @@ Status PlainTableIterator::status() const {
} }
} // namespace rocksdb } // namespace rocksdb
#endif // ROCKSDB_LITE
...@@ -2,6 +2,7 @@ ...@@ -2,6 +2,7 @@
// Use of this source code is governed by a BSD-style license that can be // 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. // found in the LICENSE file. See the AUTHORS file for names of contributors.
#ifndef ROCKSDB_LITE
#pragma once #pragma once
#include <unordered_map> #include <unordered_map>
#include <memory> #include <memory>
...@@ -255,3 +256,4 @@ class PlainTableReader: public TableReader { ...@@ -255,3 +256,4 @@ class PlainTableReader: public TableReader {
void operator=(const TableReader&) = delete; void operator=(const TableReader&) = delete;
}; };
} // namespace rocksdb } // namespace rocksdb
#endif // ROCKSDB_LITE
...@@ -12,6 +12,10 @@ ...@@ -12,6 +12,10 @@
namespace rocksdb { namespace rocksdb {
#ifdef ROCKSDB_LITE
template <class T, size_t kSize = 8>
class autovector : public std::vector<T> {};
#else
// A vector that leverages pre-allocated stack-based array to achieve better // A vector that leverages pre-allocated stack-based array to achieve better
// performance for array with small amount of items. // performance for array with small amount of items.
// //
...@@ -299,5 +303,5 @@ autovector<T, kSize>& autovector<T, kSize>::assign(const autovector& other) { ...@@ -299,5 +303,5 @@ autovector<T, kSize>& autovector<T, kSize>::assign(const autovector& other) {
return *this; return *this;
} }
#endif // ROCKSDB_LITE
} // rocksdb } // namespace rocksdb
...@@ -2,6 +2,7 @@ ...@@ -2,6 +2,7 @@
// Use of this source code is governed by a BSD-style license that can be // 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. // found in the LICENSE file. See the AUTHORS file for names of contributors.
#ifndef ROCKSDB_LITE
#include "util/blob_store.h" #include "util/blob_store.h"
namespace rocksdb { namespace rocksdb {
...@@ -266,3 +267,4 @@ Status BlobStore::CreateNewBucket() { ...@@ -266,3 +267,4 @@ Status BlobStore::CreateNewBucket() {
} }
} // namespace rocksdb } // namespace rocksdb
#endif // ROCKSDB_LITE
...@@ -2,6 +2,7 @@ ...@@ -2,6 +2,7 @@
// Use of this source code is governed by a BSD-style license that can be // 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. // found in the LICENSE file. See the AUTHORS file for names of contributors.
#ifndef ROCKSDB_LITE
#pragma once #pragma once
#include "rocksdb/env.h" #include "rocksdb/env.h"
#include "rocksdb/status.h" #include "rocksdb/status.h"
...@@ -159,3 +160,4 @@ class BlobStore { ...@@ -159,3 +160,4 @@ class BlobStore {
}; };
} // namespace rocksdb } // namespace rocksdb
#endif // ROCKSDB_LITE
...@@ -4,6 +4,7 @@ ...@@ -4,6 +4,7 @@
// of patent rights can be found in the PATENTS file in the same directory. // of patent rights can be found in the PATENTS file in the same directory.
// //
#ifndef ROCKSDB_LITE
#include "util/hash_linklist_rep.h" #include "util/hash_linklist_rep.h"
#include "rocksdb/memtablerep.h" #include "rocksdb/memtablerep.h"
...@@ -484,3 +485,4 @@ MemTableRepFactory* NewHashLinkListRepFactory(size_t bucket_count) { ...@@ -484,3 +485,4 @@ MemTableRepFactory* NewHashLinkListRepFactory(size_t bucket_count) {
} }
} // namespace rocksdb } // namespace rocksdb
#endif // ROCKSDB_LITE
...@@ -6,6 +6,7 @@ ...@@ -6,6 +6,7 @@
// Use of this source code is governed by a BSD-style license that can be // 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. // found in the LICENSE file. See the AUTHORS file for names of contributors.
#ifndef ROCKSDB_LITE
#pragma once #pragma once
#include "rocksdb/slice_transform.h" #include "rocksdb/slice_transform.h"
#include "rocksdb/memtablerep.h" #include "rocksdb/memtablerep.h"
...@@ -32,3 +33,4 @@ class HashLinkListRepFactory : public MemTableRepFactory { ...@@ -32,3 +33,4 @@ class HashLinkListRepFactory : public MemTableRepFactory {
}; };
} }
#endif // ROCKSDB_LITE
...@@ -4,6 +4,7 @@ ...@@ -4,6 +4,7 @@
// of patent rights can be found in the PATENTS file in the same directory. // of patent rights can be found in the PATENTS file in the same directory.
// //
#ifndef ROCKSDB_LITE
#include "util/hash_skiplist_rep.h" #include "util/hash_skiplist_rep.h"
#include "rocksdb/memtablerep.h" #include "rocksdb/memtablerep.h"
...@@ -339,3 +340,4 @@ MemTableRepFactory* NewHashSkipListRepFactory( ...@@ -339,3 +340,4 @@ MemTableRepFactory* NewHashSkipListRepFactory(
} }
} // namespace rocksdb } // namespace rocksdb
#endif // ROCKSDB_LITE
...@@ -6,6 +6,7 @@ ...@@ -6,6 +6,7 @@
// Use of this source code is governed by a BSD-style license that can be // 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. // found in the LICENSE file. See the AUTHORS file for names of contributors.
#ifndef ROCKSDB_LITE
#pragma once #pragma once
#include "rocksdb/slice_transform.h" #include "rocksdb/slice_transform.h"
#include "rocksdb/memtablerep.h" #include "rocksdb/memtablerep.h"
...@@ -39,3 +40,4 @@ class HashSkipListRepFactory : public MemTableRepFactory { ...@@ -39,3 +40,4 @@ class HashSkipListRepFactory : public MemTableRepFactory {
}; };
} }
#endif // ROCKSDB_LITE
...@@ -3,6 +3,7 @@ ...@@ -3,6 +3,7 @@
// LICENSE file in the root directory of this source tree. An additional grant // 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. // of patent rights can be found in the PATENTS file in the same directory.
// //
#ifndef ROCKSDB_LITE
#include "util/ldb_cmd.h" #include "util/ldb_cmd.h"
#include "db/dbformat.h" #include "db/dbformat.h"
...@@ -1834,3 +1835,4 @@ void CheckConsistencyCommand::DoCommand() { ...@@ -1834,3 +1835,4 @@ void CheckConsistencyCommand::DoCommand() {
} }
} // namespace rocksdb } // namespace rocksdb
#endif // ROCKSDB_LITE
...@@ -3,6 +3,7 @@ ...@@ -3,6 +3,7 @@
// LICENSE file in the root directory of this source tree. An additional grant // 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. // of patent rights can be found in the PATENTS file in the same directory.
// //
#ifndef ROCKSDB_LITE
#include "rocksdb/ldb_tool.h" #include "rocksdb/ldb_tool.h"
#include "util/ldb_cmd.h" #include "util/ldb_cmd.h"
...@@ -103,3 +104,4 @@ void LDBTool::Run(int argc, char** argv, Options options) { ...@@ -103,3 +104,4 @@ void LDBTool::Run(int argc, char** argv, Options options) {
} }
} // namespace rocksdb } // namespace rocksdb
#endif // ROCKSDB_LITE
...@@ -50,15 +50,6 @@ std::string EscapeString(const Slice& value) { ...@@ -50,15 +50,6 @@ std::string EscapeString(const Slice& value) {
return r; return r;
} }
bool ConsumeChar(Slice* in, char c) {
if (!in->empty() && (*in)[0] == c) {
in->remove_prefix(1);
return true;
} else {
return false;
}
}
bool ConsumeDecimalNumber(Slice* in, uint64_t* val) { bool ConsumeDecimalNumber(Slice* in, uint64_t* val) {
uint64_t v = 0; uint64_t v = 0;
int digits = 0; int digits = 0;
......
...@@ -35,10 +35,6 @@ extern std::string NumberToString(uint64_t num); ...@@ -35,10 +35,6 @@ extern std::string NumberToString(uint64_t num);
// Escapes any non-printable characters found in "value". // Escapes any non-printable characters found in "value".
extern std::string EscapeString(const Slice& value); extern std::string EscapeString(const Slice& value);
// If *in starts with "c", advances *in past the first character and
// returns true. Otherwise, returns false.
extern bool ConsumeChar(Slice* in, char c);
// Parse a human-readable number from "*in" into *value. On success, // Parse a human-readable number from "*in" into *value. On success,
// advances "*in" past the consumed number and sets "*val" to the // advances "*in" past the consumed number and sets "*val" to the
// numeric value. Otherwise, returns false and leaves *in in an // numeric value. Otherwise, returns false and leaves *in in an
......
...@@ -3,6 +3,7 @@ ...@@ -3,6 +3,7 @@
// LICENSE file in the root directory of this source tree. An additional grant // 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. // of patent rights can be found in the PATENTS file in the same directory.
// //
#ifndef ROCKSDB_LITE
#include "rocksdb/memtablerep.h" #include "rocksdb/memtablerep.h"
#include <unordered_set> #include <unordered_set>
...@@ -278,3 +279,4 @@ MemTableRep* VectorRepFactory::CreateMemTableRep( ...@@ -278,3 +279,4 @@ MemTableRep* VectorRepFactory::CreateMemTableRep(
return new VectorRep(compare, arena, count_); return new VectorRep(compare, arena, count_);
} }
} // namespace rocksdb } // namespace rocksdb
#endif // ROCKSDB_LITE
...@@ -7,6 +7,8 @@ ...@@ -7,6 +7,8 @@
// Use of this source code is governed by a BSD-style license that can be // 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. // found in the LICENSE file. See the AUTHORS file for names of contributors.
#ifndef ROCKSDB_LITE
#include "utilities/backupable_db.h" #include "utilities/backupable_db.h"
#include "db/filename.h" #include "db/filename.h"
#include "util/coding.h" #include "util/coding.h"
...@@ -1173,3 +1175,5 @@ Status RestoreBackupableDB::DeleteBackup(BackupID backup_id) { ...@@ -1173,3 +1175,5 @@ Status RestoreBackupableDB::DeleteBackup(BackupID backup_id) {
} }
} // namespace rocksdb } // namespace rocksdb
#endif // ROCKSDB_LITE
...@@ -3,6 +3,8 @@ ...@@ -3,6 +3,8 @@
// LICENSE file in the root directory of this source tree. An additional grant // 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. // of patent rights can be found in the PATENTS file in the same directory.
// //
#ifndef ROCKSDB_LITE
#include "utilities/geodb/geodb_impl.h" #include "utilities/geodb/geodb_impl.h"
#define __STDC_FORMAT_MACROS #define __STDC_FORMAT_MACROS
...@@ -425,3 +427,5 @@ void GeoDBImpl::QuadKeyToTile(std::string quadkey, Tile* tile, ...@@ -425,3 +427,5 @@ void GeoDBImpl::QuadKeyToTile(std::string quadkey, Tile* tile,
} }
} }
} // namespace rocksdb } // namespace rocksdb
#endif // ROCKSDB_LITE
...@@ -4,6 +4,8 @@ ...@@ -4,6 +4,8 @@
// of patent rights can be found in the PATENTS file in the same directory. // of patent rights can be found in the PATENTS file in the same directory.
// //
#ifndef ROCKSDB_LITE
#pragma once #pragma once
#include <algorithm> #include <algorithm>
#include <cmath> #include <cmath>
...@@ -185,3 +187,5 @@ class GeoDBImpl : public GeoDB { ...@@ -185,3 +187,5 @@ class GeoDBImpl : public GeoDB {
}; };
} // namespace rocksdb } // namespace rocksdb
#endif // ROCKSDB_LITE
...@@ -5,6 +5,7 @@ ...@@ -5,6 +5,7 @@
* Copyright 2013 Facebook * Copyright 2013 Facebook
*/ */
#ifndef ROCKSDB_LITE
#pragma once #pragma once
#include <exception> #include <exception>
...@@ -18,3 +19,4 @@ class RedisListException: public std::exception { ...@@ -18,3 +19,4 @@ class RedisListException: public std::exception {
}; };
} // namespace rocksdb } // namespace rocksdb
#endif
// Copyright 2013 Facebook
/** /**
* RedisListIterator: * RedisListIterator:
* An abstraction over the "list" concept (e.g.: for redis lists). * An abstraction over the "list" concept (e.g.: for redis lists).
...@@ -34,9 +35,9 @@ ...@@ -34,9 +35,9 @@
* - n bytes of data: the actual data. * - n bytes of data: the actual data.
* *
* @author Deon Nicholas (dnicholas@fb.com) * @author Deon Nicholas (dnicholas@fb.com)
* Copyright 2013 Facebook
*/ */
#ifndef ROCKSDB_LITE
#pragma once #pragma once
#include <string> #include <string>
...@@ -306,3 +307,4 @@ class RedisListIterator { ...@@ -306,3 +307,4 @@ class RedisListIterator {
}; };
} // namespace rocksdb } // namespace rocksdb
#endif // ROCKSDB_LITE
// Copyright 2013 Facebook
/** /**
* A (persistent) Redis API built using the rocksdb backend. * A (persistent) Redis API built using the rocksdb backend.
* Implements Redis Lists as described on: http://redis.io/commands#list * Implements Redis Lists as described on: http://redis.io/commands#list
...@@ -18,9 +19,9 @@ ...@@ -18,9 +19,9 @@
* wouldn't have to read and re-write the entire list. * wouldn't have to read and re-write the entire list.
* *
* @author Deon Nicholas (dnicholas@fb.com) * @author Deon Nicholas (dnicholas@fb.com)
* Copyright 2013 Facebook
*/ */
#ifndef ROCKSDB_LITE
#include "redis_lists.h" #include "redis_lists.h"
#include <iostream> #include <iostream>
...@@ -547,5 +548,5 @@ int RedisLists::Insert(const std::string& key, const std::string& pivot, ...@@ -547,5 +548,5 @@ int RedisLists::Insert(const std::string& key, const std::string& pivot,
return it.Length(); return it.Length();
} }
} // namespace rocksdb
} #endif // ROCKSDB_LITE
...@@ -8,6 +8,7 @@ ...@@ -8,6 +8,7 @@
* Copyright 2013 Facebook * Copyright 2013 Facebook
*/ */
#ifndef ROCKSDB_LITE
#pragma once #pragma once
#include <string> #include <string>
...@@ -104,3 +105,4 @@ class RedisLists { ...@@ -104,3 +105,4 @@ class RedisLists {
}; };
} // namespace rocksdb } // namespace rocksdb
#endif // ROCKSDB_LITE
// Copyright (c) 2011 The LevelDB Authors. All rights reserved. // Copyright (c) 2011 The LevelDB Authors. All rights reserved.
// Use of this source code is governed by a BSD-style license that can be // 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. // found in the LICENSE file. See the AUTHORS file for names of contributors.
#ifndef ROCKSDB_LITE
#include "utilities/ttl/db_ttl.h" #include "utilities/ttl/db_ttl.h"
#include "db/filename.h" #include "db/filename.h"
...@@ -215,3 +216,4 @@ Iterator* DBWithTTL::NewIterator(const ReadOptions& opts, ...@@ -215,3 +216,4 @@ Iterator* DBWithTTL::NewIterator(const ReadOptions& opts,
} }
} // namespace rocksdb } // namespace rocksdb
#endif // ROCKSDB_LITE
...@@ -2,6 +2,7 @@ ...@@ -2,6 +2,7 @@
// Use of this source code is governed by a BSD-style license that can be // 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. // found in the LICENSE file. See the AUTHORS file for names of contributors.
#ifndef ROCKSDB_LITE
#pragma once #pragma once
#include <deque> #include <deque>
#include <string> #include <string>
...@@ -328,3 +329,4 @@ class TtlMergeOperator : public MergeOperator { ...@@ -328,3 +329,4 @@ class TtlMergeOperator : public MergeOperator {
}; };
} }
#endif // ROCKSDB_LITE
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册