Fix iOS compile with -Wshorten-64-to-32

Summary: So iOS size_t is 32-bit, so we need to static_cast<size_t> any uint64_t :(

Test Plan: TARGET_OS=IOS make static_lib

Reviewers: dhruba, ljin, yhchiang, rven, sdong

Reviewed By: sdong

Subscribers: dhruba, leveldb

Differential Revision: https://reviews.facebook.net/D28743
This commit is contained in:
Igor Canadi 2014-11-13 14:39:30 -05:00
parent fa50abb726
commit 25f273027b
33 changed files with 158 additions and 84 deletions

View File

@ -524,6 +524,8 @@ void ColumnFamilyData::NotifyOnFlushCompleted(
DB* db, const std::string& file_path,
bool triggered_flush_slowdown,
bool triggered_flush_stop) {
#ifndef ROCKSDB_LITE
auto listeners = ioptions()->listeners;
for (auto listener : listeners) {
listener->OnFlushCompleted(
@ -531,6 +533,7 @@ void ColumnFamilyData::NotifyOnFlushCompleted(
// Use path 0 as fulled memtables are first flushed into path 0.
triggered_flush_slowdown, triggered_flush_stop);
}
#endif // ROCKSDB_LITE
}
SuperVersion* ColumnFamilyData::InstallSuperVersion(

View File

@ -1044,8 +1044,8 @@ Status CompactionJob::OpenCompactionOutputFile() {
compact_->outputs.push_back(out);
compact_->outfile->SetIOPriority(Env::IO_LOW);
compact_->outfile->SetPreallocationBlockSize(
compact_->compaction->OutputFilePreallocationSize(mutable_cf_options_));
compact_->outfile->SetPreallocationBlockSize(static_cast<size_t>(
compact_->compaction->OutputFilePreallocationSize(mutable_cf_options_)));
ColumnFamilyData* cfd = compact_->compaction->column_family_data();
compact_->builder.reset(NewTableBuilder(

View File

@ -64,6 +64,7 @@
#include "util/autovector.h"
#include "util/build_version.h"
#include "util/coding.h"
#include "util/db_info_dumper.h"
#include "util/hash_skiplist_rep.h"
#include "util/hash_linklist_rep.h"
#include "util/logging.h"
@ -3362,7 +3363,7 @@ Status DBImpl::GetDbIdentity(std::string& identity) {
}
char buffer[file_size];
Slice id;
s = idfile->Read(file_size, &id, buffer);
s = idfile->Read(static_cast<size_t>(file_size), &id, buffer);
if (!s.ok()) {
return s;
}

View File

@ -585,8 +585,4 @@ static void ClipToRange(T* ptr, V minvalue, V maxvalue) {
if (static_cast<V>(*ptr) < minvalue) *ptr = minvalue;
}
// Dump db file summary, implemented in util/
extern void DumpDBFileSummary(const DBOptions& options,
const std::string& dbname);
} // namespace rocksdb

View File

@ -54,7 +54,7 @@ bool Reader::SkipToInitialBlock() {
if (block_start_location > 0) {
Status skip_status = file_->Skip(block_start_location);
if (!skip_status.ok()) {
ReportDrop(block_start_location, skip_status);
ReportDrop(static_cast<size_t>(block_start_location), skip_status);
return false;
}
}

View File

@ -2635,7 +2635,7 @@ void VersionSet::AddLiveFiles(std::vector<FileDescriptor>* live_list) {
}
// just one time extension to the right size
live_list->reserve(live_list->size() + total_files);
live_list->reserve(live_list->size() + static_cast<size_t>(total_files));
for (auto cfd : *column_family_set_) {
Version* dummy_versions = cfd->dummy_versions();

View File

@ -91,9 +91,11 @@ struct ImmutableCFOptions {
int num_levels;
#ifndef ROCKSDB_LITE
// A vector of EventListeners which call-back functions will be called
// when specific RocksDB event happens.
std::vector<std::shared_ptr<EventListener>> listeners;
#endif // ROCKSDB_LITE
};
} // namespace rocksdb

View File

@ -63,7 +63,6 @@ enum CompactionStyle : char {
// jobs are submitted via CompactFiles()
};
struct CompactionOptionsFIFO {
// once the total sum of table files reaches this, we will delete the oldest
// table file
@ -102,6 +101,7 @@ struct Options;
struct ColumnFamilyOptions {
// Some functions that make it easier to optimize RocksDB
#ifndef ROCKSDB_LITE
// Use this if you don't need to keep the data sorted, i.e. you'll never use
// an iterator, only Put() and Get() API calls
ColumnFamilyOptions* OptimizeForPointLookup(
@ -125,6 +125,7 @@ struct ColumnFamilyOptions {
uint64_t memtable_memory_budget = 512 * 1024 * 1024);
ColumnFamilyOptions* OptimizeUniversalStyleCompaction(
uint64_t memtable_memory_budget = 512 * 1024 * 1024);
#endif // ROCKSDB_LITE
// -------------------
// Parameters that affect behavior
@ -591,9 +592,11 @@ struct ColumnFamilyOptions {
// Default: 2
uint32_t min_partial_merge_operands;
#ifndef ROCKSDB_LITE
// A vector of EventListeners which call-back functions will be called
// when specific RocksDB event happens.
std::vector<std::shared_ptr<EventListener>> listeners;
#endif // ROCKSDB_LITE
// Create ColumnFamilyOptions with default values for all fields
ColumnFamilyOptions();
@ -606,12 +609,14 @@ struct ColumnFamilyOptions {
struct DBOptions {
// Some functions that make it easier to optimize RocksDB
#ifndef ROCKSDB_LITE
// By default, RocksDB uses only one background thread for flush and
// compaction. Calling this function will set it up such that total of
// `total_threads` is used. Good value for `total_threads` is the number of
// cores. You almost definitely want to call this function if your system is
// bottlenecked by RocksDB.
DBOptions* IncreaseParallelism(int total_threads = 16);
#endif // ROCKSDB_LITE
// If true, the database will be created if it is missing.
// Default: false

View File

@ -11,6 +11,7 @@
namespace rocksdb {
#ifndef ROCKSDB_LITE
// Take a map of option name and option value, apply them into the
// base_options, and return the new options as a result
bool GetColumnFamilyOptionsFromMap(
@ -36,5 +37,6 @@ bool GetDBOptionsFromString(
const DBOptions& base_options,
const std::string& opts_str,
DBOptions* new_options);
#endif // ROCKSDB_LITE
} // namespace rocksdb

View File

@ -91,9 +91,9 @@ bool WriteBatchHandlerJniCallback::Continue() {
* on the result after you have finished with it
*/
jbyteArray WriteBatchHandlerJniCallback::sliceToJArray(const Slice& s) {
jbyteArray ja = m_env->NewByteArray(s.size());
jbyteArray ja = m_env->NewByteArray(static_cast<jsize>(s.size()));
m_env->SetByteArrayRegion(
ja, 0, s.size(),
ja, 0, static_cast<jsize>(s.size()),
reinterpret_cast<const jbyte*>(s.data()));
return ja;
}

View File

@ -88,8 +88,8 @@ void CondVar::Wait() {
bool CondVar::TimedWait(uint64_t abs_time_us) {
struct timespec ts;
ts.tv_sec = abs_time_us / 1000000;
ts.tv_nsec = (abs_time_us % 1000000) * 1000;
ts.tv_sec = static_cast<time_t>(abs_time_us / 1000000);
ts.tv_nsec = static_cast<suseconds_t>((abs_time_us % 1000000) * 1000);
#ifndef NDEBUG
mu_->locked_ = false;

View File

@ -258,6 +258,9 @@ class HashIndexBuilder : public IndexBuilder {
uint64_t current_restart_index_ = 0;
};
// Without anonymous namespace here, we fail the warning -Wmissing-prototypes
namespace {
// Create a index builder based on its type.
IndexBuilder* CreateIndexBuilder(IndexType type, const Comparator* comparator,
const SliceTransform* prefix_extractor) {
@ -352,6 +355,8 @@ Slice CompressBlock(const Slice& raw,
return raw;
}
} // namespace
// kBlockBasedTableMagicNumber was picked by running
// echo rocksdb.table.block_based | sha1sum
// and taking the leading 64 bits.
@ -660,7 +665,7 @@ Status BlockBasedTableBuilder::InsertBlockInCache(const Slice& block_contents,
block_cache_compressed->Release(cache_handle);
// Invalidate OS cache.
r->file->InvalidateCache(r->offset, size);
r->file->InvalidateCache(static_cast<size_t>(r->offset), size);
}
return Status::OK();
}

View File

@ -19,6 +19,7 @@
#include "rocksdb/status.h"
#include "rocksdb/table.h"
#include "table/table_reader.h"
#include "table/table_properties_internal.h"
#include "util/coding.h"
namespace rocksdb {

View File

@ -188,9 +188,10 @@ Status ReadFooterFromFile(RandomAccessFile* file,
char footer_space[Footer::kMaxEncodedLength];
Slice footer_input;
size_t read_offset = (file_size > Footer::kMaxEncodedLength)
? (file_size - Footer::kMaxEncodedLength)
: 0;
size_t read_offset =
(file_size > Footer::kMaxEncodedLength)
? static_cast<size_t>(file_size - Footer::kMaxEncodedLength)
: 0;
Status s = file->Read(read_offset, Footer::kMaxEncodedLength, &footer_input,
footer_space);
if (!s.ok()) return s;
@ -204,6 +205,9 @@ Status ReadFooterFromFile(RandomAccessFile* file,
return footer->DecodeFrom(&footer_input);
}
// Without anonymous namespace here, we fail the warning -Wmissing-prototypes
namespace {
// Read a block and check its CRC
// contents is the result of reading.
// According to the implementation of file->Read, contents may not point to buf
@ -255,6 +259,8 @@ Status ReadBlock(RandomAccessFile* file, const Footer& footer,
return s;
}
} // namespace
Status ReadBlockContents(RandomAccessFile* file, const Footer& footer,
const ReadOptions& options, const BlockHandle& handle,
BlockContents* contents, Env* env,

View File

@ -23,27 +23,24 @@
#include "util/autovector.h"
namespace rocksdb {
namespace merger {
typedef std::priority_queue<
IteratorWrapper*,
std::vector<IteratorWrapper*>,
MaxIteratorComparator> MaxIterHeap;
// Without anonymous namespace here, we fail the warning -Wmissing-prototypes
namespace {
typedef std::priority_queue<IteratorWrapper*, std::vector<IteratorWrapper*>,
MaxIteratorComparator> MergerMaxIterHeap;
typedef std::priority_queue<
IteratorWrapper*,
std::vector<IteratorWrapper*>,
MinIteratorComparator> MinIterHeap;
typedef std::priority_queue<IteratorWrapper*, std::vector<IteratorWrapper*>,
MinIteratorComparator> MergerMinIterHeap;
// Return's a new MaxHeap of IteratorWrapper's using the provided Comparator.
MaxIterHeap NewMaxIterHeap(const Comparator* comparator) {
return MaxIterHeap(MaxIteratorComparator(comparator));
MergerMaxIterHeap NewMergerMaxIterHeap(const Comparator* comparator) {
return MergerMaxIterHeap(MaxIteratorComparator(comparator));
}
// Return's a new MinHeap of IteratorWrapper's using the provided Comparator.
MinIterHeap NewMinIterHeap(const Comparator* comparator) {
return MinIterHeap(MinIteratorComparator(comparator));
MergerMinIterHeap NewMergerMinIterHeap(const Comparator* comparator) {
return MergerMinIterHeap(MinIteratorComparator(comparator));
}
} // namespace merger
} // namespace
const size_t kNumIterReserve = 4;
@ -56,8 +53,8 @@ class MergingIterator : public Iterator {
current_(nullptr),
use_heap_(true),
direction_(kForward),
maxHeap_(merger::NewMaxIterHeap(comparator_)),
minHeap_(merger::NewMinIterHeap(comparator_)) {
maxHeap_(NewMergerMaxIterHeap(comparator_)),
minHeap_(NewMergerMinIterHeap(comparator_)) {
children_.resize(n);
for (int i = 0; i < n; i++) {
children_[i].Set(children[i]);
@ -271,8 +268,8 @@ class MergingIterator : public Iterator {
kReverse
};
Direction direction_;
merger::MaxIterHeap maxHeap_;
merger::MinIterHeap minHeap_;
MergerMaxIterHeap maxHeap_;
MergerMinIterHeap minHeap_;
};
void MergingIterator::FindSmallest() {
@ -299,8 +296,8 @@ void MergingIterator::FindLargest() {
void MergingIterator::ClearHeaps() {
use_heap_ = true;
maxHeap_ = merger::NewMaxIterHeap(comparator_);
minHeap_ = merger::NewMinIterHeap(comparator_);
maxHeap_ = NewMergerMaxIterHeap(comparator_);
minHeap_ = NewMergerMinIterHeap(comparator_);
}
Iterator* NewMergingIterator(const Comparator* cmp, Iterator** list, int n,

View File

@ -11,6 +11,7 @@
#include "rocksdb/table_properties.h"
#include "table/block.h"
#include "table/format.h"
#include "table/table_properties_internal.h"
#include "util/coding.h"
namespace rocksdb {

View File

@ -119,10 +119,6 @@ Status ReadTableProperties(RandomAccessFile* file, uint64_t file_size,
uint64_t table_magic_number, Env* env,
Logger* info_log, TableProperties** properties);
// Seek to the properties block.
// If it successfully seeks to the properties block, "is_found" will be
// set to true.
extern Status SeekToPropertiesBlock(Iterator* meta_iter, bool* is_found);
// Find the meta block from the meta index block.
Status FindMetaBlock(Iterator* meta_index_iter,

View File

@ -55,16 +55,14 @@ PlainTableIndex::IndexSearchResult PlainTableIndex::GetOffset(
}
}
void PlainTableIndexBuilder::IndexRecordList::AddRecord(murmur_t hash,
void PlainTableIndexBuilder::IndexRecordList::AddRecord(uint32_t hash,
uint32_t offset) {
if (num_records_in_current_group_ == kNumRecordsPerGroup) {
current_group_ = AllocateNewGroup();
num_records_in_current_group_ = 0;
}
auto& new_record = current_group_[num_records_in_current_group_++];
// TODO(sdong) -- check if this is OK -- murmur_t is uint64_t, while we only
// use 32 bits here
new_record.hash = static_cast<uint32_t>(hash);
new_record.hash = hash;
new_record.offset = offset;
new_record.next = nullptr;
}

View File

@ -156,7 +156,7 @@ class PlainTableIndexBuilder {
}
}
void AddRecord(murmur_t hash, uint32_t offset);
void AddRecord(uint32_t hash, uint32_t offset);
size_t GetNumRecords() const {
return (groups_.size() - 1) * kNumRecordsPerGroup +

View File

@ -3,6 +3,7 @@
// 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 "table/table_properties_internal.h"
#include "rocksdb/table_properties.h"
#include "rocksdb/iterator.h"
#include "rocksdb/env.h"

View File

@ -0,0 +1,18 @@
// 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 "rocksdb/status.h"
#include "rocksdb/iterator.h"
namespace rocksdb {
// Seek to the properties block.
// If it successfully seeks to the properties block, "is_found" will be
// set to true.
Status SeekToPropertiesBlock(Iterator* meta_iter, bool* is_found);
} // namespace rocksdb

View File

@ -17,6 +17,8 @@
#include <utility>
#include <vector>
#include <gflags/gflags.h>
using std::function;
using std::get;
using std::make_pair;
@ -28,6 +30,12 @@ using std::string;
using std::tuple;
using std::vector;
#ifndef GFLAGS
bool FLAGS_benchmark = false;
uint64_t FLAGS_bm_min_usec = 100;
int64_t FLAGS_bm_min_iter = 1;
int32_t FLAGS_bm_max_secs = 1;
#else
DEFINE_bool(benchmark, false, "Run benchmarks.");
DEFINE_uint64(bm_min_usec, 100,
@ -38,7 +46,7 @@ DEFINE_int64(bm_min_iters, 1,
DEFINE_int32(bm_max_secs, 1,
"Maximum # of seconds we'll spend on each benchmark.");
#endif // GFLAGS
namespace rocksdb {
namespace benchmark {

View File

@ -9,8 +9,6 @@
#pragma once
#include <gflags/gflags.h>
#include <cassert>
#include <functional>
#include <limits>

View File

@ -2,9 +2,6 @@
// 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.
//
// Must not be included from any .h files to avoid polluting the namespace
// with macros.
#ifndef __STDC_FORMAT_MACROS
#define __STDC_FORMAT_MACROS
@ -16,9 +13,10 @@
#include <algorithm>
#include <vector>
#include "db/filename.h"
#include "rocksdb/options.h"
#include "rocksdb/env.h"
#include "db/filename.h"
#include "util/db_info_dumper.h"
namespace rocksdb {

13
util/db_info_dumper.h Normal file
View File

@ -0,0 +1,13 @@
// 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 <string>
#include "rocksdb/options.h"
namespace rocksdb {
void DumpDBFileSummary(const DBOptions& options, const std::string& dbname);
} // namespace rocksdb

View File

@ -9,8 +9,8 @@
#include "rocksdb/slice.h"
#include <util/arena.h>
#include <port/port_posix.h>
#include "util/arena.h"
#include "port/port_posix.h"
#include <atomic>
#include <memory>

View File

@ -201,7 +201,7 @@ class PosixSequentialFile: public SequentialFile {
}
virtual Status Skip(uint64_t n) {
if (fseek(file_, n, SEEK_CUR)) {
if (fseek(file_, static_cast<long int>(n), SEEK_CUR)) {
return IOError(filename_, errno);
}
return Status::OK();
@ -486,7 +486,7 @@ class PosixMmapFile : public WritableFile {
const char* src = data.data();
size_t left = data.size();
TEST_KILL_RANDOM(rocksdb_kill_odds * REDUCE_ODDS);
PrepareWrite(GetFileSize(), left);
PrepareWrite(static_cast<size_t>(GetFileSize()), left);
while (left > 0) {
assert(base_ <= dst_);
assert(dst_ <= limit_);
@ -683,7 +683,7 @@ class PosixWritableFile : public WritableFile {
TEST_KILL_RANDOM(rocksdb_kill_odds * REDUCE_ODDS2);
PrepareWrite(GetFileSize(), left);
PrepareWrite(static_cast<size_t>(GetFileSize()), left);
// if there is no space in the cache, then flush
if (cursize_ + left > capacity_) {
s = Flush();
@ -1380,7 +1380,7 @@ class PosixEnv : public Env {
}
virtual Status GetHostName(char* name, uint64_t len) {
int ret = gethostname(name, len);
int ret = gethostname(name, static_cast<size_t>(len));
if (ret < 0) {
if (errno == EFAULT || errno == EINVAL)
return Status::InvalidArgument(strerror(errno));

View File

@ -60,7 +60,7 @@ size_t HistogramBucketMapper::IndexForValue(const uint64_t value) const {
std::map<uint64_t, uint64_t>::const_iterator lowerBound =
valueIndexMap_.lower_bound(value);
if (lowerBound != valueIndexMap_.end()) {
return lowerBound->second;
return static_cast<size_t>(lowerBound->second);
} else {
return 0;
}

View File

@ -38,7 +38,7 @@ class HistogramBucketMapper {
return minBucketValue_;
}
uint64_t BucketLimit(const uint64_t bucketNumber) const {
uint64_t BucketLimit(const size_t bucketNumber) const {
assert(bucketNumber < BucketCount());
return bucketValues_[bucketNumber];
}

View File

@ -4,6 +4,9 @@
// of patent rights can be found in the PATENTS file in the same directory.
//
#pragma once
#ifndef ROCKSDB_LITE
#include <string>
#include <iostream>
#include <sstream>
@ -730,3 +733,5 @@ public:
};
} // namespace rocksdb
#endif // ROCKSDB_LITE

View File

@ -64,8 +64,12 @@ ImmutableCFOptions::ImmutableCFOptions(const Options& options)
compression_per_level(options.compression_per_level),
compression_opts(options.compression_opts),
access_hint_on_compaction_start(options.access_hint_on_compaction_start),
num_levels(options.num_levels),
listeners(options.listeners) {}
num_levels(options.num_levels)
#ifndef ROCKSDB_LITE
, listeners(options.listeners) {}
#else // ROCKSDB_LITE
{}
#endif // ROCKSDB_LITE
ColumnFamilyOptions::ColumnFamilyOptions()
: comparator(BytewiseComparator()),
@ -113,8 +117,12 @@ ColumnFamilyOptions::ColumnFamilyOptions()
memtable_prefix_bloom_huge_page_tlb_size(0),
bloom_locality(0),
max_successive_merges(0),
min_partial_merge_operands(2),
listeners() {
min_partial_merge_operands(2)
#ifndef ROCKSDB_LITE
, listeners() {
#else // ROCKSDB_LITE
{
#endif // ROCKSDB_LITE
assert(memtable_factory.get() != nullptr);
}
@ -174,8 +182,12 @@ ColumnFamilyOptions::ColumnFamilyOptions(const Options& options)
options.memtable_prefix_bloom_huge_page_tlb_size),
bloom_locality(options.bloom_locality),
max_successive_merges(options.max_successive_merges),
min_partial_merge_operands(options.min_partial_merge_operands),
listeners(options.listeners) {
min_partial_merge_operands(options.min_partial_merge_operands)
#ifndef ROCKSDB_LITE
, listeners(options.listeners) {
#else // ROCKSDB_LITE
{
#endif // ROCKSDB_LITE
assert(memtable_factory.get() != nullptr);
if (max_bytes_for_level_multiplier_additional.size() <
static_cast<unsigned int>(num_levels)) {
@ -496,6 +508,7 @@ Options::PrepareForBulkLoad()
return this;
}
#ifndef ROCKSDB_LITE
// Optimization functions
ColumnFamilyOptions* ColumnFamilyOptions::OptimizeForPointLookup(
uint64_t block_cache_size_mb) {
@ -504,17 +517,15 @@ ColumnFamilyOptions* ColumnFamilyOptions::OptimizeForPointLookup(
block_based_options.index_type = BlockBasedTableOptions::kHashSearch;
block_based_options.filter_policy.reset(NewBloomFilterPolicy(10));
block_based_options.block_cache =
NewLRUCache(block_cache_size_mb * 1024 * 1024);
NewLRUCache(static_cast<size_t>(block_cache_size_mb * 1024 * 1024));
table_factory.reset(new BlockBasedTableFactory(block_based_options));
#ifndef ROCKSDB_LITE
memtable_factory.reset(NewHashLinkListRepFactory());
#endif
return this;
}
ColumnFamilyOptions* ColumnFamilyOptions::OptimizeLevelStyleCompaction(
uint64_t memtable_memory_budget) {
write_buffer_size = memtable_memory_budget / 4;
write_buffer_size = static_cast<size_t>(memtable_memory_budget / 4);
// merge two memtables when flushing to L0
min_write_buffer_number_to_merge = 2;
// this means we'll use 50% extra memory in the worst case, but will reduce
@ -546,7 +557,7 @@ ColumnFamilyOptions* ColumnFamilyOptions::OptimizeLevelStyleCompaction(
ColumnFamilyOptions* ColumnFamilyOptions::OptimizeUniversalStyleCompaction(
uint64_t memtable_memory_budget) {
write_buffer_size = memtable_memory_budget / 4;
write_buffer_size = static_cast<size_t>(memtable_memory_budget / 4);
// merge two memtables when flushing to L0
min_write_buffer_number_to_merge = 2;
// this means we'll use 50% extra memory in the worst case, but will reduce
@ -565,5 +576,6 @@ DBOptions* DBOptions::IncreaseParallelism(int total_threads) {
env->SetBackgroundThreads(1, Env::HIGH);
return this;
}
#endif // ROCKSDB_LITE
} // namespace rocksdb

View File

@ -7,10 +7,13 @@
#include <cctype>
#include <unordered_set>
#include "rocksdb/options.h"
#include "rocksdb/utilities/convenience.h"
#include "util/options_helper.h"
namespace rocksdb {
#ifndef ROCKSDB_LITE
namespace {
CompressionType ParseCompressionType(const std::string& type) {
if (type == "kNoCompression") {
@ -50,8 +53,8 @@ uint64_t ParseUint64(const std::string& value) {
return std::stoull(value);
}
int64_t ParseInt64(const std::string& value) {
return std::stol(value);
size_t ParseSizeT(const std::string& value) {
return static_cast<size_t>(ParseUint64(value));
}
double ParseDouble(const std::string& value) {
@ -76,24 +79,24 @@ template<typename OptionsType>
bool ParseMemtableOptions(const std::string& name, const std::string& value,
OptionsType* new_options) {
if (name == "write_buffer_size") {
new_options->write_buffer_size = ParseInt64(value);
new_options->write_buffer_size = ParseSizeT(value);
} else if (name == "arena_block_size") {
new_options->arena_block_size = ParseInt64(value);
new_options->arena_block_size = ParseSizeT(value);
} else if (name == "memtable_prefix_bloom_bits") {
new_options->memtable_prefix_bloom_bits = ParseUint32(value);
} else if (name == "memtable_prefix_bloom_probes") {
new_options->memtable_prefix_bloom_probes = ParseUint32(value);
} else if (name == "memtable_prefix_bloom_huge_page_tlb_size") {
new_options->memtable_prefix_bloom_huge_page_tlb_size =
ParseInt64(value);
ParseSizeT(value);
} else if (name == "max_successive_merges") {
new_options->max_successive_merges = ParseInt64(value);
new_options->max_successive_merges = ParseSizeT(value);
} else if (name == "filter_deletes") {
new_options->filter_deletes = ParseBoolean(name, value);
} else if (name == "max_write_buffer_number") {
new_options->max_write_buffer_number = ParseInt(value);
} else if (name == "inplace_update_num_locks") {
new_options->inplace_update_num_locks = ParseInt64(value);
new_options->inplace_update_num_locks = ParseSizeT(value);
} else {
return false;
}
@ -367,11 +370,11 @@ bool GetDBOptionsFromMap(
} else if (o.first == "max_background_flushes") {
new_options->max_background_flushes = ParseInt(o.second);
} else if (o.first == "max_log_file_size") {
new_options->max_log_file_size = ParseInt64(o.second);
new_options->max_log_file_size = ParseSizeT(o.second);
} else if (o.first == "log_file_time_to_roll") {
new_options->log_file_time_to_roll = ParseInt64(o.second);
new_options->log_file_time_to_roll = ParseSizeT(o.second);
} else if (o.first == "keep_log_file_num") {
new_options->keep_log_file_num = ParseInt64(o.second);
new_options->keep_log_file_num = ParseSizeT(o.second);
} else if (o.first == "max_manifest_file_size") {
new_options->max_manifest_file_size = ParseUint64(o.second);
} else if (o.first == "table_cache_numshardbits") {
@ -383,7 +386,7 @@ bool GetDBOptionsFromMap(
} else if (o.first == "WAL_size_limit_MB") {
new_options->WAL_size_limit_MB = ParseUint64(o.second);
} else if (o.first == "manifest_preallocation_size") {
new_options->manifest_preallocation_size = ParseInt64(o.second);
new_options->manifest_preallocation_size = ParseSizeT(o.second);
} else if (o.first == "allow_os_buffer") {
new_options->allow_os_buffer = ParseBoolean(o.first, o.second);
} else if (o.first == "allow_mmap_reads") {
@ -424,4 +427,5 @@ bool GetDBOptionsFromString(
return GetDBOptionsFromMap(base_options, opts_map, new_options);
}
#endif // ROCKSDB_LITE
} // namespace rocksdb

View File

@ -6,6 +6,8 @@
#include "rocksdb/sst_dump_tool.h"
#ifndef ROCKSDB_LITE
#ifndef __STDC_FORMAT_MACROS
#define __STDC_FORMAT_MACROS
#endif
@ -428,3 +430,5 @@ void SSTDumpTool::Run(int argc, char** argv) {
}
}
} // namespace rocksdb
#endif // ROCKSDB_LITE