Skip to content

Commit

Permalink
Fix iOS compile with -Wshorten-64-to-32
Browse files Browse the repository at this point in the history
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
  • Loading branch information
igorcanadi committed Nov 13, 2014
1 parent fa50abb commit 25f2730
Show file tree
Hide file tree
Showing 33 changed files with 158 additions and 84 deletions.
3 changes: 3 additions & 0 deletions db/column_family.cc
Original file line number Diff line number Diff line change
Expand Up @@ -524,13 +524,16 @@ 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(
db, GetName(), file_path,
// Use path 0 as fulled memtables are first flushed into path 0.
triggered_flush_slowdown, triggered_flush_stop);
}
#endif // ROCKSDB_LITE
}

SuperVersion* ColumnFamilyData::InstallSuperVersion(
Expand Down
4 changes: 2 additions & 2 deletions db/compaction_job.cc
Original file line number Diff line number Diff line change
Expand Up @@ -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(
Expand Down
3 changes: 2 additions & 1 deletion db/db_impl.cc
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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;
}
Expand Down
4 changes: 0 additions & 4 deletions db/db_impl.h
Original file line number Diff line number Diff line change
Expand Up @@ -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
2 changes: 1 addition & 1 deletion db/log_reader.cc
Original file line number Diff line number Diff line change
Expand Up @@ -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;
}
}
Expand Down
2 changes: 1 addition & 1 deletion db/version_set.cc
Original file line number Diff line number Diff line change
Expand Up @@ -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();
Expand Down
2 changes: 2 additions & 0 deletions include/rocksdb/immutable_options.h
Original file line number Diff line number Diff line change
Expand Up @@ -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
7 changes: 6 additions & 1 deletion include/rocksdb/options.h
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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(
Expand All @@ -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
Expand Down Expand Up @@ -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();
Expand All @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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(
Expand All @@ -36,5 +37,6 @@ bool GetDBOptionsFromString(
const DBOptions& base_options,
const std::string& opts_str,
DBOptions* new_options);
#endif // ROCKSDB_LITE

} // namespace rocksdb
4 changes: 2 additions & 2 deletions java/rocksjni/writebatchhandlerjnicallback.cc
Original file line number Diff line number Diff line change
Expand Up @@ -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;
}
Expand Down
4 changes: 2 additions & 2 deletions port/port_posix.cc
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down
7 changes: 6 additions & 1 deletion table/block_based_table_builder.cc
Original file line number Diff line number Diff line change
Expand Up @@ -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) {
Expand Down Expand Up @@ -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.
Expand Down Expand Up @@ -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();
}
Expand Down
1 change: 1 addition & 0 deletions table/block_based_table_reader.h
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down
12 changes: 9 additions & 3 deletions table/format.cc
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -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
Expand Down Expand Up @@ -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,
Expand Down
37 changes: 17 additions & 20 deletions table/merger.cc
Original file line number Diff line number Diff line change
Expand Up @@ -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;

Expand All @@ -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]);
Expand Down Expand Up @@ -271,8 +268,8 @@ class MergingIterator : public Iterator {
kReverse
};
Direction direction_;
merger::MaxIterHeap maxHeap_;
merger::MinIterHeap minHeap_;
MergerMaxIterHeap maxHeap_;
MergerMinIterHeap minHeap_;
};

void MergingIterator::FindSmallest() {
Expand All @@ -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,
Expand Down
1 change: 1 addition & 0 deletions table/meta_blocks.cc
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down
4 changes: 0 additions & 4 deletions table/meta_blocks.h
Original file line number Diff line number Diff line change
Expand Up @@ -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,
Expand Down
6 changes: 2 additions & 4 deletions table/plain_table_index.cc
Original file line number Diff line number Diff line change
Expand Up @@ -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;
}
Expand Down
2 changes: 1 addition & 1 deletion table/plain_table_index.h
Original file line number Diff line number Diff line change
Expand Up @@ -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 +
Expand Down
1 change: 1 addition & 0 deletions table/table_properties.cc
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down
18 changes: 18 additions & 0 deletions table/table_properties_internal.h
Original file line number Diff line number Diff line change
@@ -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
Loading

0 comments on commit 25f2730

Please sign in to comment.