mirror of https://github.com/facebook/rocksdb.git
Merge pull request #846 from yuslepukhin/enble_c4244_lossofdata
Enable MS compiler warning c4244.
This commit is contained in:
commit
298ba27ae2
|
@ -60,7 +60,7 @@ add_custom_target(GenerateBuildVersion DEPENDS ${BUILD_VERSION_CC})
|
|||
|
||||
set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} /Zi /nologo /EHsc /GS /Gd /GR /GF /fp:precise /Zc:wchar_t /Zc:forScope /errorReport:queue")
|
||||
|
||||
set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} /FC /d2Zi+ /W3 /WX /wd4127 /wd4244 /wd4800 /wd4996")
|
||||
set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} /FC /d2Zi+ /W3 /WX /wd4127 /wd4800 /wd4996")
|
||||
|
||||
# Used to run CI build and tests so we can run faster
|
||||
set(OPTIMIZE_DEBUG_DEFAULT 0) # Debug build is unoptimized by default use -DOPTDBG=1 to optimize
|
||||
|
@ -244,7 +244,7 @@ set(SOURCES
|
|||
utilities/document/document_db.cc
|
||||
utilities/document/json_document.cc
|
||||
utilities/document/json_document_builder.cc
|
||||
utilities/env_mirror.cc
|
||||
utilities/env_mirror.cc
|
||||
utilities/flashcache/flashcache.cc
|
||||
utilities/geodb/geodb_impl.cc
|
||||
utilities/leveldb_options/leveldb_options.cc
|
||||
|
|
|
@ -436,7 +436,7 @@ uint64_t Compaction::OutputFilePreallocationSize() {
|
|||
}
|
||||
// Over-estimate slightly so we don't end up just barely crossing
|
||||
// the threshold
|
||||
return preallocation_size * 1.1;
|
||||
return preallocation_size + (preallocation_size / 10);
|
||||
}
|
||||
|
||||
std::unique_ptr<CompactionFilter> Compaction::CreateCompactionFilter() const {
|
||||
|
|
|
@ -415,9 +415,12 @@ void CompactionJob::GenSubcompactionBoundaries() {
|
|||
|
||||
// Group the ranges into subcompactions
|
||||
const double min_file_fill_percent = 4.0 / 5;
|
||||
uint64_t max_output_files = std::ceil(
|
||||
sum / min_file_fill_percent /
|
||||
cfd->GetCurrentMutableCFOptions()->MaxFileSizeForLevel(out_lvl));
|
||||
uint64_t max_output_files =
|
||||
static_cast<uint64_t>(
|
||||
std::ceil(
|
||||
sum / min_file_fill_percent /
|
||||
cfd->GetCurrentMutableCFOptions()->MaxFileSizeForLevel(out_lvl))
|
||||
);
|
||||
uint64_t subcompactions =
|
||||
std::min({static_cast<uint64_t>(ranges.size()),
|
||||
static_cast<uint64_t>(db_options_.max_subcompactions),
|
||||
|
|
|
@ -551,8 +551,9 @@ uint64_t EstimatedFileSize(
|
|||
const size_t kFooterSize = 512;
|
||||
|
||||
uint64_t data_size =
|
||||
static_cast<uint64_t>(
|
||||
num_records * (key_size + value_size * compression_ratio +
|
||||
kPerKeyOverhead);
|
||||
kPerKeyOverhead));
|
||||
|
||||
return data_size + kFooterSize
|
||||
+ num_records * bloom_bits_per_key / 8 // filter block
|
||||
|
|
|
@ -11,6 +11,7 @@
|
|||
|
||||
#include "rocksdb/db.h"
|
||||
|
||||
#include <inttypes.h>
|
||||
#include <errno.h>
|
||||
#include <fcntl.h>
|
||||
#include <sys/stat.h>
|
||||
|
@ -104,8 +105,8 @@ class CorruptionTest : public testing::Test {
|
|||
}
|
||||
|
||||
void Check(int min_expected, int max_expected) {
|
||||
unsigned int next_expected = 0;
|
||||
int missed = 0;
|
||||
uint64_t next_expected = 0;
|
||||
uint64_t missed = 0;
|
||||
int bad_keys = 0;
|
||||
int bad_values = 0;
|
||||
int correct = 0;
|
||||
|
@ -126,7 +127,7 @@ class CorruptionTest : public testing::Test {
|
|||
continue;
|
||||
}
|
||||
missed += (key - next_expected);
|
||||
next_expected = static_cast<unsigned int>(key + 1);
|
||||
next_expected = key + 1;
|
||||
if (iter->value() != Value(static_cast<int>(key), &value_space)) {
|
||||
bad_values++;
|
||||
} else {
|
||||
|
@ -136,8 +137,9 @@ class CorruptionTest : public testing::Test {
|
|||
delete iter;
|
||||
|
||||
fprintf(stderr,
|
||||
"expected=%d..%d; got=%d; bad_keys=%d; bad_values=%d; missed=%d\n",
|
||||
min_expected, max_expected, correct, bad_keys, bad_values, missed);
|
||||
"expected=%d..%d; got=%d; bad_keys=%d; bad_values=%d; missed=%llu\n",
|
||||
min_expected, max_expected, correct, bad_keys, bad_values,
|
||||
static_cast<unsigned long long>(missed));
|
||||
ASSERT_LE(min_expected, correct);
|
||||
ASSERT_GE(max_expected, correct);
|
||||
}
|
||||
|
|
|
@ -1194,15 +1194,15 @@ static std::unordered_map<OperationType, std::string, std::hash<unsigned char>>
|
|||
class Stats {
|
||||
private:
|
||||
int id_;
|
||||
double start_;
|
||||
double finish_;
|
||||
uint64_t start_;
|
||||
uint64_t finish_;
|
||||
double seconds_;
|
||||
int64_t done_;
|
||||
int64_t last_report_done_;
|
||||
int64_t next_report_;
|
||||
int64_t bytes_;
|
||||
double last_op_finish_;
|
||||
double last_report_finish_;
|
||||
uint64_t done_;
|
||||
uint64_t last_report_done_;
|
||||
uint64_t next_report_;
|
||||
uint64_t bytes_;
|
||||
uint64_t last_op_finish_;
|
||||
uint64_t last_report_finish_;
|
||||
std::unordered_map<OperationType, HistogramImpl,
|
||||
std::hash<unsigned char>> hist_;
|
||||
std::string message_;
|
||||
|
@ -1304,8 +1304,8 @@ class Stats {
|
|||
reporter_agent_->ReportFinishedOps(num_ops);
|
||||
}
|
||||
if (FLAGS_histogram) {
|
||||
double now = FLAGS_env->NowMicros();
|
||||
double micros = now - last_op_finish_;
|
||||
uint64_t now = FLAGS_env->NowMicros();
|
||||
uint64_t micros = now - last_op_finish_;
|
||||
|
||||
if (hist_.find(op_type) == hist_.end())
|
||||
{
|
||||
|
@ -1315,7 +1315,7 @@ class Stats {
|
|||
hist_[op_type].Add(micros);
|
||||
|
||||
if (micros > 20000 && !FLAGS_stats_interval) {
|
||||
fprintf(stderr, "long op: %.1f micros%30s\r", micros, "");
|
||||
fprintf(stderr, "long op: %" PRIu64 " micros%30s\r", micros, "");
|
||||
fflush(stderr);
|
||||
}
|
||||
last_op_finish_ = now;
|
||||
|
@ -1333,7 +1333,7 @@ class Stats {
|
|||
else next_report_ += 100000;
|
||||
fprintf(stderr, "... finished %" PRIu64 " ops%30s\r", done_, "");
|
||||
} else {
|
||||
double now = FLAGS_env->NowMicros();
|
||||
uint64_t now = FLAGS_env->NowMicros();
|
||||
int64_t usecs_since_last = now - last_report_finish_;
|
||||
|
||||
// Determine whether to print status where interval is either
|
||||
|
@ -1349,7 +1349,7 @@ class Stats {
|
|||
fprintf(stderr,
|
||||
"%s ... thread %d: (%" PRIu64 ",%" PRIu64 ") ops and "
|
||||
"(%.1f,%.1f) ops/second in (%.6f,%.6f) seconds\n",
|
||||
FLAGS_env->TimeToString((uint64_t) now/1000000).c_str(),
|
||||
FLAGS_env->TimeToString(now/1000000).c_str(),
|
||||
id_,
|
||||
done_ - last_report_done_, done_,
|
||||
(done_ - last_report_done_) /
|
||||
|
@ -1503,7 +1503,7 @@ struct ThreadState {
|
|||
|
||||
class Duration {
|
||||
public:
|
||||
Duration(int max_seconds, int64_t max_ops, int64_t ops_per_stage = 0) {
|
||||
Duration(uint64_t max_seconds, int64_t max_ops, int64_t ops_per_stage = 0) {
|
||||
max_seconds_ = max_seconds;
|
||||
max_ops_= max_ops;
|
||||
ops_per_stage_ = (ops_per_stage > 0) ? ops_per_stage : max_ops;
|
||||
|
@ -1520,8 +1520,8 @@ class Duration {
|
|||
if (max_seconds_) {
|
||||
// Recheck every appx 1000 ops (exact iff increment is factor of 1000)
|
||||
if ((ops_/1000) != ((ops_-increment)/1000)) {
|
||||
double now = FLAGS_env->NowMicros();
|
||||
return ((now - start_at_) / 1000000.0) >= max_seconds_;
|
||||
uint64_t now = FLAGS_env->NowMicros();
|
||||
return ((now - start_at_) / 1000000) >= max_seconds_;
|
||||
} else {
|
||||
return false;
|
||||
}
|
||||
|
@ -1531,11 +1531,11 @@ class Duration {
|
|||
}
|
||||
|
||||
private:
|
||||
int max_seconds_;
|
||||
uint64_t max_seconds_;
|
||||
int64_t max_ops_;
|
||||
int64_t ops_per_stage_;
|
||||
int64_t ops_;
|
||||
double start_at_;
|
||||
uint64_t start_at_;
|
||||
};
|
||||
|
||||
class Benchmark {
|
||||
|
@ -3234,7 +3234,7 @@ class Benchmark {
|
|||
void BGWriter(ThreadState* thread, enum OperationType write_merge) {
|
||||
// Special thread that keeps writing until other threads are done.
|
||||
RandomGenerator gen;
|
||||
double last = FLAGS_env->NowMicros();
|
||||
uint64_t last = FLAGS_env->NowMicros();
|
||||
int writes_per_second_by_10 = 0;
|
||||
int num_writes = 0;
|
||||
int64_t bytes = 0;
|
||||
|
@ -3279,14 +3279,14 @@ class Benchmark {
|
|||
|
||||
++num_writes;
|
||||
if (writes_per_second_by_10 && num_writes >= writes_per_second_by_10) {
|
||||
double now = FLAGS_env->NowMicros();
|
||||
double usecs_since_last = now - last;
|
||||
uint64_t now = FLAGS_env->NowMicros();
|
||||
uint64_t usecs_since_last = now - last;
|
||||
|
||||
num_writes = 0;
|
||||
last = now;
|
||||
|
||||
if (usecs_since_last < 100000.0) {
|
||||
FLAGS_env->SleepForMicroseconds(100000.0 - usecs_since_last);
|
||||
if (usecs_since_last < 100000) {
|
||||
FLAGS_env->SleepForMicroseconds(static_cast<int>(100000 - usecs_since_last));
|
||||
last = FLAGS_env->NowMicros();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -4466,8 +4466,9 @@ Status DBImpl::SwitchMemtable(ColumnFamilyData* cfd, WriteContext* context) {
|
|||
if (s.ok()) {
|
||||
// Our final size should be less than write_buffer_size
|
||||
// (compression, etc) but err on the side of caution.
|
||||
lfile->SetPreallocationBlockSize(1.1 *
|
||||
mutable_cf_options.write_buffer_size);
|
||||
lfile->SetPreallocationBlockSize(
|
||||
mutable_cf_options.write_buffer_size / 10 +
|
||||
mutable_cf_options.write_buffer_size);
|
||||
unique_ptr<WritableFileWriter> file_writer(
|
||||
new WritableFileWriter(std::move(lfile), opt_env_opt));
|
||||
new_log = new log::Writer(std::move(file_writer), new_log_number,
|
||||
|
@ -5144,7 +5145,7 @@ Status DB::Open(const DBOptions& db_options, const std::string& dbname,
|
|||
LogFileName(impl->db_options_.wal_dir, new_log_number),
|
||||
&lfile, opt_env_options);
|
||||
if (s.ok()) {
|
||||
lfile->SetPreallocationBlockSize(1.1 * max_write_buffer_size);
|
||||
lfile->SetPreallocationBlockSize((max_write_buffer_size / 10) + max_write_buffer_size);
|
||||
impl->logfile_number_ = new_log_number;
|
||||
unique_ptr<WritableFileWriter> file_writer(
|
||||
new WritableFileWriter(std::move(lfile), opt_env_options));
|
||||
|
|
|
@ -73,7 +73,7 @@
|
|||
|
||||
namespace rocksdb {
|
||||
|
||||
static long TestGetTickerCount(const Options& options, Tickers ticker_type) {
|
||||
static uint64_t TestGetTickerCount(const Options& options, Tickers ticker_type) {
|
||||
return options.statistics->getTickerCount(ticker_type);
|
||||
}
|
||||
|
||||
|
@ -1235,8 +1235,8 @@ TEST_F(DBTest, KeyMayExist) {
|
|||
ASSERT_OK(Flush(1));
|
||||
value.clear();
|
||||
|
||||
long numopen = TestGetTickerCount(options, NO_FILE_OPENS);
|
||||
long cache_added = TestGetTickerCount(options, BLOCK_CACHE_ADD);
|
||||
uint64_t numopen = TestGetTickerCount(options, NO_FILE_OPENS);
|
||||
uint64_t cache_added = TestGetTickerCount(options, BLOCK_CACHE_ADD);
|
||||
ASSERT_TRUE(
|
||||
db_->KeyMayExist(ropts, handles_[1], "a", &value, &value_found));
|
||||
ASSERT_TRUE(!value_found);
|
||||
|
@ -1304,8 +1304,8 @@ TEST_F(DBTest, NonBlockingIteration) {
|
|||
|
||||
// verify that a non-blocking iterator does not find any
|
||||
// kvs. Neither does it do any IOs to storage.
|
||||
long numopen = TestGetTickerCount(options, NO_FILE_OPENS);
|
||||
long cache_added = TestGetTickerCount(options, BLOCK_CACHE_ADD);
|
||||
uint64_t numopen = TestGetTickerCount(options, NO_FILE_OPENS);
|
||||
uint64_t cache_added = TestGetTickerCount(options, BLOCK_CACHE_ADD);
|
||||
iter = db_->NewIterator(non_blocking_opts, handles_[1]);
|
||||
count = 0;
|
||||
for (iter->SeekToFirst(); iter->Valid(); iter->Next()) {
|
||||
|
@ -5231,25 +5231,27 @@ class RecoveryTestHelper {
|
|||
test->Close();
|
||||
#endif
|
||||
if (trunc) {
|
||||
ASSERT_EQ(0, truncate(fname.c_str(), size * off));
|
||||
ASSERT_EQ(0, truncate(fname.c_str(),
|
||||
static_cast<int64_t>(size * off)));
|
||||
} else {
|
||||
InduceCorruption(fname, size * off, size * len);
|
||||
InduceCorruption(fname, static_cast<size_t>(size * off),
|
||||
static_cast<size_t>(size * len));
|
||||
}
|
||||
}
|
||||
|
||||
// Overwrite data with 'a' from offset for length len
|
||||
static void InduceCorruption(const std::string& filename, uint32_t offset,
|
||||
uint32_t len) {
|
||||
static void InduceCorruption(const std::string& filename, size_t offset,
|
||||
size_t len) {
|
||||
ASSERT_GT(len, 0U);
|
||||
|
||||
int fd = open(filename.c_str(), O_RDWR);
|
||||
|
||||
ASSERT_GT(fd, 0);
|
||||
ASSERT_EQ(offset, lseek(fd, offset, SEEK_SET));
|
||||
ASSERT_EQ(offset, lseek(fd, static_cast<long>(offset), SEEK_SET));
|
||||
|
||||
void* buf = alloca(len);
|
||||
memset(buf, 'a', len);
|
||||
ASSERT_EQ(len, write(fd, buf, len));
|
||||
ASSERT_EQ(len, write(fd, buf, static_cast<unsigned int>(len)));
|
||||
|
||||
close(fd);
|
||||
}
|
||||
|
@ -6445,7 +6447,7 @@ TEST_F(DBTest, RateLimitingTest) {
|
|||
RandomString(&rnd, (1 << 10) + 1), wo));
|
||||
}
|
||||
uint64_t elapsed = env_->NowMicros() - start;
|
||||
double raw_rate = env_->bytes_written_ * 1000000 / elapsed;
|
||||
double raw_rate = env_->bytes_written_ * 1000000.0 / elapsed;
|
||||
Close();
|
||||
|
||||
// # rate limiting with 0.7 x threshold
|
||||
|
@ -8238,7 +8240,7 @@ TEST_F(DBTest, MutexWaitStats) {
|
|||
options.create_if_missing = true;
|
||||
options.statistics = rocksdb::CreateDBStatistics();
|
||||
CreateAndReopenWithCF({"pikachu"}, options);
|
||||
const int64_t kMutexWaitDelay = 100;
|
||||
const uint64_t kMutexWaitDelay = 100;
|
||||
ThreadStatusUtil::TEST_SetStateDelay(
|
||||
ThreadStatus::STATE_MUTEX_WAIT, kMutexWaitDelay);
|
||||
ASSERT_OK(Put("hello", "rocksdb"));
|
||||
|
|
|
@ -8,7 +8,10 @@
|
|||
namespace rocksdb {
|
||||
|
||||
namespace {
|
||||
inline double SafeDivide(double a, double b) { return b == 0.0 ? 0 : a / b; }
|
||||
template<class T>
|
||||
inline T SafeDivide(T a, T b) {
|
||||
return b == 0 ? 0 : a / b;
|
||||
}
|
||||
} // namespace
|
||||
|
||||
void EventHelpers::AppendCurrentTime(JSONWriter* jwriter) {
|
||||
|
|
|
@ -43,9 +43,9 @@ void PrintLevelStatsHeader(char* buf, size_t len, const std::string& cf_name) {
|
|||
}
|
||||
|
||||
void PrintLevelStats(char* buf, size_t len, const std::string& name,
|
||||
int num_files, int being_compacted, double total_file_size,
|
||||
double score, double w_amp,
|
||||
const InternalStats::CompactionStats& stats) {
|
||||
int num_files, int being_compacted, double total_file_size,
|
||||
double score, double w_amp,
|
||||
const InternalStats::CompactionStats& stats) {
|
||||
uint64_t bytes_read =
|
||||
stats.bytes_read_non_output_levels + stats.bytes_read_output_level;
|
||||
int64_t bytes_new =
|
||||
|
@ -69,8 +69,8 @@ void PrintLevelStats(char* buf, size_t len, const std::string& name,
|
|||
"%9.0f " /* Comp(sec) */
|
||||
"%9d " /* Comp(cnt) */
|
||||
"%8.3f " /* Avg(sec) */
|
||||
"%7s " /* KeyIn */
|
||||
"%6s\n", /* KeyDrop */
|
||||
"%7s " /* KeyIn */
|
||||
"%6s\n", /* KeyDrop */
|
||||
name.c_str(),
|
||||
num_files, being_compacted, total_file_size / kMB, score,
|
||||
bytes_read / kGB, stats.bytes_read_non_output_levels / kGB,
|
||||
|
@ -687,7 +687,8 @@ void InternalStats::DumpCFStats(std::string* value) {
|
|||
comp_stats_[level].bytes_read_non_output_levels;
|
||||
PrintLevelStats(buf, sizeof(buf), "L" + ToString(level), files,
|
||||
files_being_compacted[level],
|
||||
vstorage->NumLevelBytes(level), compaction_score[level],
|
||||
static_cast<double>(vstorage->NumLevelBytes(level)),
|
||||
compaction_score[level],
|
||||
w_amp, comp_stats_[level]);
|
||||
value->append(buf);
|
||||
}
|
||||
|
|
|
@ -55,7 +55,8 @@ struct FileDescriptor {
|
|||
return packed_number_and_path_id & kFileNumberMask;
|
||||
}
|
||||
uint32_t GetPathId() const {
|
||||
return packed_number_and_path_id / (kFileNumberMask + 1);
|
||||
return static_cast<uint32_t>(
|
||||
packed_number_and_path_id / (kFileNumberMask + 1));
|
||||
}
|
||||
uint64_t GetFileSize() const { return file_size; }
|
||||
};
|
||||
|
|
|
@ -758,7 +758,10 @@ uint64_t VersionStorageInfo::GetEstimatedActiveKeys() const {
|
|||
|
||||
if (current_num_samples_ < file_count) {
|
||||
// casting to avoid overflowing
|
||||
return (est * static_cast<double>(file_count) / current_num_samples_);
|
||||
return
|
||||
static_cast<uint64_t>(
|
||||
(est * static_cast<double>(file_count) / current_num_samples_)
|
||||
);
|
||||
} else {
|
||||
return est;
|
||||
}
|
||||
|
|
|
@ -624,8 +624,9 @@ MemTableRep* HashCuckooRepFactory::CreateMemTableRep(
|
|||
size_t pointer_size = sizeof(std::atomic<const char*>);
|
||||
assert(write_buffer_size_ >= (average_data_size_ + pointer_size));
|
||||
size_t bucket_count =
|
||||
static_cast<size_t>(
|
||||
(write_buffer_size_ / (average_data_size_ + pointer_size)) / kFullness +
|
||||
1;
|
||||
1);
|
||||
unsigned int hash_function_count = hash_function_count_;
|
||||
if (hash_function_count < 2) {
|
||||
hash_function_count = 2;
|
||||
|
@ -635,7 +636,9 @@ MemTableRep* HashCuckooRepFactory::CreateMemTableRep(
|
|||
}
|
||||
return new HashCuckooRep(compare, allocator, bucket_count,
|
||||
hash_function_count,
|
||||
(average_data_size_ + pointer_size) / kFullness);
|
||||
static_cast<size_t>(
|
||||
(average_data_size_ + pointer_size) / kFullness)
|
||||
);
|
||||
}
|
||||
|
||||
MemTableRepFactory* NewHashCuckooRepFactory(size_t write_buffer_size,
|
||||
|
|
|
@ -1684,7 +1684,9 @@ class WinEnv : public Env {
|
|||
|
||||
virtual Status GetHostName(char* name, uint64_t len) override {
|
||||
Status s;
|
||||
DWORD nSize = len;
|
||||
DWORD nSize =
|
||||
static_cast<DWORD>(std::min<uint64_t>(len,
|
||||
std::numeric_limits<DWORD>::max()));
|
||||
|
||||
if (!::GetComputerNameA(name, &nSize)) {
|
||||
auto lastError = GetLastError();
|
||||
|
|
|
@ -39,8 +39,9 @@ void gettimeofday(struct timeval* tv, struct timezone* /* tz */) {
|
|||
|
||||
seconds secNow(duration_cast<seconds>(usNow));
|
||||
|
||||
tv->tv_sec = secNow.count();
|
||||
tv->tv_usec = usNow.count() - duration_cast<microseconds>(secNow).count();
|
||||
tv->tv_sec = static_cast<long>(secNow.count());
|
||||
tv->tv_usec = static_cast<long>(usNow.count() -
|
||||
duration_cast<microseconds>(secNow).count());
|
||||
}
|
||||
|
||||
Mutex::~Mutex() {}
|
||||
|
|
|
@ -246,7 +246,8 @@ Status CuckooTableBuilder::Finish() {
|
|||
if (num_entries_ > 0) {
|
||||
// Calculate the real hash size if module hash is enabled.
|
||||
if (use_module_hash_) {
|
||||
hash_table_size_ = num_entries_ / max_hash_table_ratio_;
|
||||
hash_table_size_ =
|
||||
static_cast<uint64_t>(num_entries_ / max_hash_table_ratio_);
|
||||
}
|
||||
s = MakeHashTable(&buckets);
|
||||
if (!s.ok()) {
|
||||
|
@ -402,7 +403,8 @@ uint64_t CuckooTableBuilder::FileSize() const {
|
|||
}
|
||||
|
||||
if (use_module_hash_) {
|
||||
return (key_size_ + value_size_) * num_entries_ / max_hash_table_ratio_;
|
||||
return static_cast<uint64_t>((key_size_ + value_size_) *
|
||||
num_entries_ / max_hash_table_ratio_);
|
||||
} else {
|
||||
// Account for buckets being a power of two.
|
||||
// As elements are added, file size remains constant for a while and
|
||||
|
|
|
@ -129,6 +129,11 @@ class CuckooBuilderTest : public testing::Test {
|
|||
return n;
|
||||
}
|
||||
|
||||
uint64_t GetExpectedTableSize(uint64_t num) {
|
||||
return NextPowOf2(static_cast<uint64_t>(num / kHashTableRatio));
|
||||
}
|
||||
|
||||
|
||||
Env* env_;
|
||||
EnvOptions env_options_;
|
||||
std::string fname;
|
||||
|
@ -169,7 +174,7 @@ TEST_F(CuckooBuilderTest, WriteSuccessNoCollisionFullKey) {
|
|||
for (auto& user_key : user_keys) {
|
||||
keys.push_back(GetInternalKey(user_key, false));
|
||||
}
|
||||
uint64_t expected_table_size = NextPowOf2(keys.size() / kHashTableRatio);
|
||||
uint64_t expected_table_size = GetExpectedTableSize(keys.size());
|
||||
|
||||
unique_ptr<WritableFile> writable_file;
|
||||
fname = test::TmpDir() + "/NoCollisionFullKey";
|
||||
|
@ -216,7 +221,7 @@ TEST_F(CuckooBuilderTest, WriteSuccessWithCollisionFullKey) {
|
|||
for (auto& user_key : user_keys) {
|
||||
keys.push_back(GetInternalKey(user_key, false));
|
||||
}
|
||||
uint64_t expected_table_size = NextPowOf2(keys.size() / kHashTableRatio);
|
||||
uint64_t expected_table_size = GetExpectedTableSize(keys.size());
|
||||
|
||||
unique_ptr<WritableFile> writable_file;
|
||||
fname = test::TmpDir() + "/WithCollisionFullKey";
|
||||
|
@ -263,7 +268,7 @@ TEST_F(CuckooBuilderTest, WriteSuccessWithCollisionAndCuckooBlock) {
|
|||
for (auto& user_key : user_keys) {
|
||||
keys.push_back(GetInternalKey(user_key, false));
|
||||
}
|
||||
uint64_t expected_table_size = NextPowOf2(keys.size() / kHashTableRatio);
|
||||
uint64_t expected_table_size = GetExpectedTableSize(keys.size());
|
||||
|
||||
unique_ptr<WritableFile> writable_file;
|
||||
uint32_t cuckoo_block_size = 2;
|
||||
|
@ -316,7 +321,7 @@ TEST_F(CuckooBuilderTest, WithCollisionPathFullKey) {
|
|||
for (auto& user_key : user_keys) {
|
||||
keys.push_back(GetInternalKey(user_key, false));
|
||||
}
|
||||
uint64_t expected_table_size = NextPowOf2(keys.size() / kHashTableRatio);
|
||||
uint64_t expected_table_size = GetExpectedTableSize(keys.size());
|
||||
|
||||
unique_ptr<WritableFile> writable_file;
|
||||
fname = test::TmpDir() + "/WithCollisionPathFullKey";
|
||||
|
@ -365,7 +370,7 @@ TEST_F(CuckooBuilderTest, WithCollisionPathFullKeyAndCuckooBlock) {
|
|||
for (auto& user_key : user_keys) {
|
||||
keys.push_back(GetInternalKey(user_key, false));
|
||||
}
|
||||
uint64_t expected_table_size = NextPowOf2(keys.size() / kHashTableRatio);
|
||||
uint64_t expected_table_size = GetExpectedTableSize(keys.size());
|
||||
|
||||
unique_ptr<WritableFile> writable_file;
|
||||
fname = test::TmpDir() + "/WithCollisionPathFullKeyAndCuckooBlock";
|
||||
|
@ -407,7 +412,7 @@ TEST_F(CuckooBuilderTest, WriteSuccessNoCollisionUserKey) {
|
|||
hash_map = std::move(hm);
|
||||
|
||||
std::vector<uint64_t> expected_locations = {0, 1, 2, 3};
|
||||
uint64_t expected_table_size = NextPowOf2(user_keys.size() / kHashTableRatio);
|
||||
uint64_t expected_table_size = GetExpectedTableSize(user_keys.size());
|
||||
|
||||
unique_ptr<WritableFile> writable_file;
|
||||
fname = test::TmpDir() + "/NoCollisionUserKey";
|
||||
|
@ -450,7 +455,7 @@ TEST_F(CuckooBuilderTest, WriteSuccessWithCollisionUserKey) {
|
|||
hash_map = std::move(hm);
|
||||
|
||||
std::vector<uint64_t> expected_locations = {0, 1, 2, 3};
|
||||
uint64_t expected_table_size = NextPowOf2(user_keys.size() / kHashTableRatio);
|
||||
uint64_t expected_table_size = GetExpectedTableSize(user_keys.size());
|
||||
|
||||
unique_ptr<WritableFile> writable_file;
|
||||
fname = test::TmpDir() + "/WithCollisionUserKey";
|
||||
|
@ -495,7 +500,7 @@ TEST_F(CuckooBuilderTest, WithCollisionPathUserKey) {
|
|||
hash_map = std::move(hm);
|
||||
|
||||
std::vector<uint64_t> expected_locations = {0, 1, 3, 4, 2};
|
||||
uint64_t expected_table_size = NextPowOf2(user_keys.size() / kHashTableRatio);
|
||||
uint64_t expected_table_size = GetExpectedTableSize(user_keys.size());
|
||||
|
||||
unique_ptr<WritableFile> writable_file;
|
||||
fname = test::TmpDir() + "/WithCollisionPathUserKey";
|
||||
|
|
|
@ -500,7 +500,7 @@ void ReadKeys(uint64_t num, uint32_t batch_size) {
|
|||
&get_context);
|
||||
}
|
||||
}
|
||||
float time_per_op = (env->NowMicros() - start_time) * 1.0 / num;
|
||||
float time_per_op = (env->NowMicros() - start_time) * 1.0f / num;
|
||||
fprintf(stderr,
|
||||
"Time taken per op is %.3fus (%.1f Mqps) with batch size of %u\n",
|
||||
time_per_op, 1.0 / time_per_op, batch_size);
|
||||
|
|
|
@ -117,7 +117,8 @@ void PlainTableIndexBuilder::AllocateIndex() {
|
|||
index_size_ = 1;
|
||||
} else {
|
||||
double hash_table_size_multipier = 1.0 / hash_table_ratio_;
|
||||
index_size_ = num_prefixes_ * hash_table_size_multipier + 1;
|
||||
index_size_ =
|
||||
static_cast<uint32_t>(num_prefixes_ * hash_table_size_multipier) + 1;
|
||||
assert(index_size_ > 0);
|
||||
}
|
||||
}
|
||||
|
@ -186,7 +187,7 @@ Slice PlainTableIndexBuilder::FillIndexes(
|
|||
index[i] = sub_index_offset | PlainTableIndex::kSubIndexMask;
|
||||
char* prev_ptr = &sub_index[sub_index_offset];
|
||||
char* cur_ptr = EncodeVarint32(prev_ptr, num_keys_for_bucket);
|
||||
sub_index_offset += (cur_ptr - prev_ptr);
|
||||
sub_index_offset += static_cast<uint32_t>(cur_ptr - prev_ptr);
|
||||
char* sub_index_pos = &sub_index[sub_index_offset];
|
||||
IndexRecord* record = hash_to_offsets[i];
|
||||
int j;
|
||||
|
|
|
@ -567,7 +567,7 @@ class FbsonJsonParserT {
|
|||
++precision;
|
||||
} else if (ch == '.') {
|
||||
// note we don't pop out '.'
|
||||
return parseDouble(in, val, precision, sign);
|
||||
return parseDouble(in, static_cast<double>(val), precision, sign);
|
||||
} else {
|
||||
err_ = FbsonErrType::E_INVALID_DECIMAL;
|
||||
return false;
|
||||
|
@ -578,7 +578,7 @@ class FbsonJsonParserT {
|
|||
// if the number overflows int64_t, first parse it as double iff we see a
|
||||
// decimal point later. Otherwise, will treat it as overflow
|
||||
if (val < 0 && val > std::numeric_limits<int64_t>::min()) {
|
||||
return parseDouble(in, (uint64_t)val, precision, sign);
|
||||
return parseDouble(in, static_cast<double>(val), precision, sign);
|
||||
}
|
||||
|
||||
ch = in.peek();
|
||||
|
|
|
@ -444,13 +444,13 @@ DEFINE_bool(use_merge, false, "On true, replaces all writes with a Merge "
|
|||
namespace rocksdb {
|
||||
|
||||
// convert long to a big-endian slice key
|
||||
static std::string Key(long val) {
|
||||
static std::string Key(int64_t val) {
|
||||
std::string little_endian_key;
|
||||
std::string big_endian_key;
|
||||
PutFixed64(&little_endian_key, val);
|
||||
assert(little_endian_key.size() == sizeof(val));
|
||||
big_endian_key.resize(sizeof(val));
|
||||
for (int i=0; i<(int)sizeof(val); i++) {
|
||||
for (size_t i = 0 ; i < sizeof(val); ++i) {
|
||||
big_endian_key[i] = little_endian_key[sizeof(val) - 1 - i];
|
||||
}
|
||||
return big_endian_key;
|
||||
|
@ -472,9 +472,9 @@ namespace {
|
|||
|
||||
class Stats {
|
||||
private:
|
||||
double start_;
|
||||
double finish_;
|
||||
double seconds_;
|
||||
uint64_t start_;
|
||||
uint64_t finish_;
|
||||
double seconds_;
|
||||
long done_;
|
||||
long gets_;
|
||||
long prefixes_;
|
||||
|
@ -487,7 +487,7 @@ class Stats {
|
|||
long errors_;
|
||||
int next_report_;
|
||||
size_t bytes_;
|
||||
double last_op_finish_;
|
||||
uint64_t last_op_finish_;
|
||||
HistogramImpl hist_;
|
||||
|
||||
public:
|
||||
|
@ -538,11 +538,11 @@ class Stats {
|
|||
|
||||
void FinishedSingleOp() {
|
||||
if (FLAGS_histogram) {
|
||||
double now = FLAGS_env->NowMicros();
|
||||
double micros = now - last_op_finish_;
|
||||
auto now = FLAGS_env->NowMicros();
|
||||
auto micros = now - last_op_finish_;
|
||||
hist_.Add(micros);
|
||||
if (micros > 20000) {
|
||||
fprintf(stdout, "long op: %.1f micros%30s\r", micros, "");
|
||||
fprintf(stdout, "long op: %" PRIu64 " micros%30s\r", micros, "");
|
||||
}
|
||||
last_op_finish_ = now;
|
||||
}
|
||||
|
@ -676,7 +676,7 @@ class SharedState {
|
|||
values_[i] = std::vector<uint32_t>(max_key_, SENTINEL);
|
||||
}
|
||||
|
||||
long num_locks = (max_key_ >> log2_keys_per_lock_);
|
||||
long num_locks = static_cast<long>(max_key_ >> log2_keys_per_lock_);
|
||||
if (max_key_ & ((1 << log2_keys_per_lock_) - 1)) {
|
||||
num_locks++;
|
||||
}
|
||||
|
@ -705,7 +705,7 @@ class SharedState {
|
|||
return stress_test_;
|
||||
}
|
||||
|
||||
long GetMaxKey() const {
|
||||
int64_t GetMaxKey() const {
|
||||
return max_key_;
|
||||
}
|
||||
|
||||
|
@ -785,21 +785,21 @@ class SharedState {
|
|||
std::fill(values_[cf].begin(), values_[cf].end(), SENTINEL);
|
||||
}
|
||||
|
||||
void Put(int cf, long key, uint32_t value_base) {
|
||||
void Put(int cf, int64_t key, uint32_t value_base) {
|
||||
values_[cf][key] = value_base;
|
||||
}
|
||||
|
||||
uint32_t Get(int cf, long key) const { return values_[cf][key]; }
|
||||
uint32_t Get(int cf, int64_t key) const { return values_[cf][key]; }
|
||||
|
||||
void Delete(int cf, long key) { values_[cf][key] = SENTINEL; }
|
||||
void Delete(int cf, int64_t key) { values_[cf][key] = SENTINEL; }
|
||||
|
||||
void SingleDelete(int cf, size_t key) { values_[cf][key] = SENTINEL; }
|
||||
void SingleDelete(int cf, int64_t key) { values_[cf][key] = SENTINEL; }
|
||||
|
||||
bool AllowsOverwrite(int cf, size_t key) {
|
||||
bool AllowsOverwrite(int cf, int64_t key) {
|
||||
return no_overwrite_ids_[cf].find(key) == no_overwrite_ids_[cf].end();
|
||||
}
|
||||
|
||||
bool Exists(int cf, size_t key) { return values_[cf][key] != SENTINEL; }
|
||||
bool Exists(int cf, int64_t key) { return values_[cf][key] != SENTINEL; }
|
||||
|
||||
uint32_t GetSeed() const { return seed_; }
|
||||
|
||||
|
@ -815,7 +815,7 @@ class SharedState {
|
|||
port::Mutex mu_;
|
||||
port::CondVar cv_;
|
||||
const uint32_t seed_;
|
||||
const long max_key_;
|
||||
const int64_t max_key_;
|
||||
const uint32_t log2_keys_per_lock_;
|
||||
const int num_threads_;
|
||||
long num_initialized_;
|
||||
|
@ -1122,9 +1122,9 @@ class StressTest {
|
|||
shared.GetCondVar()->Wait();
|
||||
}
|
||||
|
||||
double now = FLAGS_env->NowMicros();
|
||||
auto now = FLAGS_env->NowMicros();
|
||||
fprintf(stdout, "%s Starting database operations\n",
|
||||
FLAGS_env->TimeToString((uint64_t) now/1000000).c_str());
|
||||
FLAGS_env->TimeToString(now/1000000).c_str());
|
||||
|
||||
shared.SetStart();
|
||||
shared.GetCondVar()->SignalAll();
|
||||
|
@ -1157,10 +1157,10 @@ class StressTest {
|
|||
delete threads[i];
|
||||
threads[i] = nullptr;
|
||||
}
|
||||
double now = FLAGS_env->NowMicros();
|
||||
auto now = FLAGS_env->NowMicros();
|
||||
if (!FLAGS_test_batches_snapshots) {
|
||||
fprintf(stdout, "%s Verification successful\n",
|
||||
FLAGS_env->TimeToString((uint64_t) now/1000000).c_str());
|
||||
FLAGS_env->TimeToString(now/1000000).c_str());
|
||||
}
|
||||
PrintStatistics();
|
||||
|
||||
|
@ -1510,7 +1510,7 @@ class StressTest {
|
|||
WriteOptions write_opts;
|
||||
auto shared = thread->shared;
|
||||
char value[100];
|
||||
long max_key = thread->shared->GetMaxKey();
|
||||
auto max_key = thread->shared->GetMaxKey();
|
||||
std::string from_db;
|
||||
if (FLAGS_sync) {
|
||||
write_opts.sync = true;
|
||||
|
@ -1760,7 +1760,7 @@ class StressTest {
|
|||
unique_ptr<Iterator> iter(
|
||||
db_->NewIterator(options, column_families_[cf]));
|
||||
iter->Seek(Key(start));
|
||||
for (long i = start; i < end; i++) {
|
||||
for (auto i = start; i < end; i++) {
|
||||
if (thread->shared->HasVerificationFailedYet()) {
|
||||
break;
|
||||
}
|
||||
|
@ -1798,7 +1798,7 @@ class StressTest {
|
|||
}
|
||||
} else {
|
||||
// Use Get to verify this range
|
||||
for (long i = start; i < end; i++) {
|
||||
for (auto i = start; i < end; i++) {
|
||||
if (thread->shared->HasVerificationFailedYet()) {
|
||||
break;
|
||||
}
|
||||
|
@ -1818,13 +1818,13 @@ class StressTest {
|
|||
}
|
||||
|
||||
void VerificationAbort(SharedState* shared, std::string msg, int cf,
|
||||
long key) const {
|
||||
printf("Verification failed for column family %d key %ld: %s\n", cf, key,
|
||||
int64_t key) const {
|
||||
printf("Verification failed for column family %d key %" PRIi64 ": %s\n", cf, key,
|
||||
msg.c_str());
|
||||
shared->SetVerificationFailure();
|
||||
}
|
||||
|
||||
bool VerifyValue(int cf, long key, const ReadOptions& opts,
|
||||
bool VerifyValue(int cf, int64_t key, const ReadOptions& opts,
|
||||
SharedState* shared, const std::string& value_from_db,
|
||||
Status s, bool strict = false) const {
|
||||
if (shared->HasVerificationFailedYet()) {
|
||||
|
@ -1861,12 +1861,12 @@ class StressTest {
|
|||
return true;
|
||||
}
|
||||
|
||||
static void PrintKeyValue(int cf, uint32_t key, const char* value,
|
||||
static void PrintKeyValue(int cf, int64_t key, const char* value,
|
||||
size_t sz) {
|
||||
if (!FLAGS_verbose) {
|
||||
return;
|
||||
}
|
||||
fprintf(stdout, "[CF %d] %u ==> (%u) ", cf, key, (unsigned int)sz);
|
||||
fprintf(stdout, "[CF %d] %" PRIi64 " == > (%" ROCKSDB_PRIszt ") ", cf, key, sz);
|
||||
for (size_t i = 0; i < sz; i++) {
|
||||
fprintf(stdout, "%X", value[i]);
|
||||
}
|
||||
|
@ -2127,9 +2127,9 @@ class StressTest {
|
|||
db_ = nullptr;
|
||||
|
||||
num_times_reopened_++;
|
||||
double now = FLAGS_env->NowMicros();
|
||||
auto now = FLAGS_env->NowMicros();
|
||||
fprintf(stdout, "%s Reopening database for the %dth time\n",
|
||||
FLAGS_env->TimeToString((uint64_t) now/1000000).c_str(),
|
||||
FLAGS_env->TimeToString(now/1000000).c_str(),
|
||||
num_times_reopened_);
|
||||
Open();
|
||||
}
|
||||
|
|
|
@ -150,7 +150,7 @@ public:
|
|||
|
||||
if (i + 1 >= len) {
|
||||
// if odd number of chars than we just hit end of string
|
||||
parsed.push_back(p - hexas);
|
||||
parsed.push_back(static_cast<char>(p - hexas));
|
||||
break;
|
||||
}
|
||||
|
||||
|
@ -158,7 +158,7 @@ public:
|
|||
const char* q = std::lower_bound(hexas, hexas + 16, b);
|
||||
if (*q == b) {
|
||||
// pairwise compute decimal value from hex
|
||||
parsed.push_back(((p - hexas) << 4) | (q - hexas));
|
||||
parsed.push_back(static_cast<char>(((p - hexas) << 4) | (q - hexas)));
|
||||
} else {
|
||||
throw "Invalid hex value";
|
||||
}
|
||||
|
|
|
@ -206,8 +206,9 @@ class WriteStress {
|
|||
std::uniform_real_distribution<double> dist(0, 1);
|
||||
std::uniform_int_distribution<int> char_dist('a', 'z');
|
||||
while (!stop_.load(std::memory_order_relaxed)) {
|
||||
Env::Default()->SleepForMicroseconds(FLAGS_prefix_mutate_period_sec *
|
||||
1000 * 1000LL);
|
||||
Env::Default()->SleepForMicroseconds(static_cast<int>(
|
||||
FLAGS_prefix_mutate_period_sec *
|
||||
1000 * 1000LL));
|
||||
if (dist(rng) < FLAGS_first_char_mutate_probability) {
|
||||
key_prefix_[0].store(char_dist(rng), std::memory_order_relaxed);
|
||||
}
|
||||
|
|
|
@ -25,7 +25,7 @@ bool CheckMemoryAllocated(size_t allocated, size_t expected) {
|
|||
// The value returned by Arena::MemoryAllocatedBytes() may be greater than
|
||||
// the requested memory. We choose a somewhat arbitrary upper bound of
|
||||
// max_expected = expected * 1.1 to detect critical overallocation.
|
||||
size_t max_expected = expected * 1.1;
|
||||
size_t max_expected = expected + expected / 10;
|
||||
return allocated >= expected && allocated <= max_expected;
|
||||
}
|
||||
|
||||
|
|
|
@ -74,7 +74,7 @@ namespace {
|
|||
}
|
||||
|
||||
void HistogramImpl::Clear() {
|
||||
min_ = bucketMapper.LastValue();
|
||||
min_ = static_cast<double>(bucketMapper.LastValue());
|
||||
max_ = 0;
|
||||
num_ = 0;
|
||||
sum_ = 0;
|
||||
|
@ -87,8 +87,8 @@ bool HistogramImpl::Empty() { return num_ == 0; }
|
|||
void HistogramImpl::Add(uint64_t value) {
|
||||
const size_t index = bucketMapper.IndexForValue(value);
|
||||
buckets_[index] += 1;
|
||||
if (min_ > value) min_ = value;
|
||||
if (max_ < value) max_ = value;
|
||||
if (min_ > value) min_ = static_cast<double>(value);
|
||||
if (max_ < value) max_ = static_cast<double>(value);
|
||||
num_++;
|
||||
sum_ += value;
|
||||
sum_squares_ += (value * value);
|
||||
|
@ -116,8 +116,10 @@ double HistogramImpl::Percentile(double p) const {
|
|||
sum += buckets_[b];
|
||||
if (sum >= threshold) {
|
||||
// Scale linearly within this bucket
|
||||
double left_point = (b == 0) ? 0 : bucketMapper.BucketLimit(b-1);
|
||||
double right_point = bucketMapper.BucketLimit(b);
|
||||
double left_point =
|
||||
static_cast<double>((b == 0) ? 0 : bucketMapper.BucketLimit(b-1));
|
||||
double right_point =
|
||||
static_cast<double>(bucketMapper.BucketLimit(b));
|
||||
double left_sum = sum - buckets_[b];
|
||||
double right_sum = sum;
|
||||
double pos = 0;
|
||||
|
|
|
@ -60,7 +60,7 @@ class GenericRateLimiter : public RateLimiter {
|
|||
private:
|
||||
void Refill();
|
||||
int64_t CalculateRefillBytesPerPeriod(int64_t rate_bytes_per_sec) {
|
||||
return rate_bytes_per_sec * refill_period_us_ / 1000000.0;
|
||||
return rate_bytes_per_sec * refill_period_us_ / 1000000;
|
||||
}
|
||||
|
||||
// This mutex guard all internal states
|
||||
|
|
|
@ -352,8 +352,8 @@ Status GeoDBImpl::searchQuadIds(const GeoPosition& position,
|
|||
Pixel bottomRight = PositionToPixel(bottomRightPos, Detail);
|
||||
|
||||
// how many level of details to look for
|
||||
int numberOfTilesAtMaxDepth = floor((bottomRight.x - topLeft.x) / 256);
|
||||
int zoomLevelsToRise = floor(::log(numberOfTilesAtMaxDepth) / ::log(2));
|
||||
int numberOfTilesAtMaxDepth = static_cast<int>(floor((bottomRight.x - topLeft.x) / 256));
|
||||
int zoomLevelsToRise = static_cast<int>(floor(::log(numberOfTilesAtMaxDepth) / ::log(2)));
|
||||
zoomLevelsToRise++;
|
||||
int levels = std::max(0, Detail - zoomLevelsToRise);
|
||||
|
||||
|
@ -408,8 +408,8 @@ GeoPosition GeoDBImpl::PixelToPosition(const Pixel& pixel, int levelOfDetail) {
|
|||
|
||||
// Converts a Pixel to a Tile
|
||||
GeoDBImpl::Tile GeoDBImpl::PixelToTile(const Pixel& pixel) {
|
||||
unsigned int tileX = floor(pixel.x / 256);
|
||||
unsigned int tileY = floor(pixel.y / 256);
|
||||
unsigned int tileX = static_cast<unsigned int>(floor(pixel.x / 256));
|
||||
unsigned int tileY = static_cast<unsigned int>(floor(pixel.y / 256));
|
||||
return Tile(tileX, tileY);
|
||||
}
|
||||
|
||||
|
|
|
@ -18,8 +18,8 @@ inline uint64_t GetTileFromCoord(double x, double start, double end,
|
|||
if (x < start) {
|
||||
return 0;
|
||||
}
|
||||
uint64_t tiles = static_cast<uint64_t>(1) << tile_bits;
|
||||
uint64_t r = ((x - start) / (end - start)) * tiles;
|
||||
uint64_t tiles = 1ull << tile_bits;
|
||||
uint64_t r = static_cast<uint64_t>(((x - start) / (end - start)) * tiles);
|
||||
return std::min(r, tiles - 1);
|
||||
}
|
||||
|
||||
|
@ -27,7 +27,7 @@ inline uint64_t GetQuadKeyFromTile(uint64_t tile_x, uint64_t tile_y,
|
|||
uint32_t tile_bits) {
|
||||
uint64_t quad_key = 0;
|
||||
for (uint32_t i = 0; i < tile_bits; ++i) {
|
||||
uint64_t mask = static_cast<uint64_t>(1LL << i);
|
||||
uint64_t mask = (1ull << i);
|
||||
quad_key |= (tile_x & mask) << i;
|
||||
quad_key |= (tile_y & mask) << (i + 1);
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue