Fixes for -Wshorten-64-to-32

- lots of static_cast
- add safe_cast
- types adjustments
  - config
  - IStorage::read/watch
  - ...
- some TODO's (to convert types in future)

P.S. That was quite a journey...

v2: fixes after rebase
v3: fix conflicts after #42308 merged
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
This commit is contained in:
Azat Khuzhin 2022-10-07 12:46:45 +02:00
parent 19715f1542
commit 4e76629aaf
411 changed files with 1209 additions and 965 deletions

View File

@ -151,7 +151,7 @@ public:
{ {
size_t dot_pos = path.rfind('.'); size_t dot_pos = path.rfind('.');
if (dot_pos != std::string::npos) if (dot_pos != std::string::npos)
fd = ::mkstemps(path.data(), path.size() - dot_pos); fd = ::mkstemps(path.data(), static_cast<int>(path.size() - dot_pos));
else else
fd = ::mkstemp(path.data()); fd = ::mkstemp(path.data());
@ -408,7 +408,7 @@ ReplxxLineReader::ReplxxLineReader(
// In a simplest case use simple comment. // In a simplest case use simple comment.
commented_line = fmt::format("-- {}", state.text()); commented_line = fmt::format("-- {}", state.text());
} }
rx.set_state(replxx::Replxx::State(commented_line.c_str(), commented_line.size())); rx.set_state(replxx::Replxx::State(commented_line.c_str(), static_cast<int>(commented_line.size())));
return rx.invoke(Replxx::ACTION::COMMIT_LINE, code); return rx.invoke(Replxx::ACTION::COMMIT_LINE, code);
}; };
@ -480,7 +480,7 @@ void ReplxxLineReader::openEditor()
if (executeCommand(argv) == 0) if (executeCommand(argv) == 0)
{ {
const std::string & new_query = readFile(editor_file.getPath()); const std::string & new_query = readFile(editor_file.getPath());
rx.set_state(replxx::Replxx::State(new_query.c_str(), new_query.size())); rx.set_state(replxx::Replxx::State(new_query.c_str(), static_cast<int>(new_query.size())));
} }
} }
catch (const std::runtime_error & e) catch (const std::runtime_error & e)
@ -526,7 +526,7 @@ void ReplxxLineReader::openInteractiveHistorySearch()
{ {
std::string new_query = readFile(output_file.getPath()); std::string new_query = readFile(output_file.getPath());
rightTrim(new_query); rightTrim(new_query);
rx.set_state(replxx::Replxx::State(new_query.c_str(), new_query.size())); rx.set_state(replxx::Replxx::State(new_query.c_str(), static_cast<int>(new_query.size())));
} }
} }
catch (const std::runtime_error & e) catch (const std::runtime_error & e)

View File

@ -265,7 +265,7 @@ inline size_t hashLessThan16(const char * data, size_t size)
struct CRC32Hash struct CRC32Hash
{ {
size_t operator() (StringRef x) const unsigned operator() (StringRef x) const
{ {
const char * pos = x.data; const char * pos = x.data;
size_t size = x.size; size_t size = x.size;
@ -275,22 +275,22 @@ struct CRC32Hash
if (size < 8) if (size < 8)
{ {
return hashLessThan8(x.data, x.size); return static_cast<unsigned>(hashLessThan8(x.data, x.size));
} }
const char * end = pos + size; const char * end = pos + size;
size_t res = -1ULL; unsigned res = -1U;
do do
{ {
UInt64 word = unalignedLoad<UInt64>(pos); UInt64 word = unalignedLoad<UInt64>(pos);
res = CRC_INT(res, word); res = static_cast<unsigned>(CRC_INT(res, word));
pos += 8; pos += 8;
} while (pos + 8 < end); } while (pos + 8 < end);
UInt64 word = unalignedLoad<UInt64>(end - 8); /// I'm not sure if this is normal. UInt64 word = unalignedLoad<UInt64>(end - 8); /// I'm not sure if this is normal.
res = CRC_INT(res, word); res = static_cast<unsigned>(CRC_INT(res, word));
return res; return res;
} }

View File

@ -122,7 +122,7 @@ QuotientAndRemainder<N> static inline split(UnsignedOfSize<N> value)
constexpr DivisionBy10PowN<N> division; constexpr DivisionBy10PowN<N> division;
UnsignedOfSize<N> quotient = (division.multiplier * (UnsignedOfSize<2 * N>(value) + division.add)) >> division.shift; UnsignedOfSize<N> quotient = (division.multiplier * (UnsignedOfSize<2 * N>(value) + division.add)) >> division.shift;
UnsignedOfSize<N / 2> remainder = value - quotient * pow10<UnsignedOfSize<N / 2>>(N); UnsignedOfSize<N / 2> remainder = static_cast<UnsignedOfSize<2 * N>>(value - quotient * pow10<UnsignedOfSize<N / 2>>(N));
return {quotient, remainder}; return {quotient, remainder};
} }

View File

@ -1108,15 +1108,21 @@ void Client::processConfig()
else else
format = config().getString("format", is_interactive ? "PrettyCompact" : "TabSeparated"); format = config().getString("format", is_interactive ? "PrettyCompact" : "TabSeparated");
format_max_block_size = config().getInt("format_max_block_size", global_context->getSettingsRef().max_block_size); format_max_block_size = config().getUInt64("format_max_block_size",
global_context->getSettingsRef().max_block_size);
insert_format = "Values"; insert_format = "Values";
/// Setting value from cmd arg overrides one from config /// Setting value from cmd arg overrides one from config
if (global_context->getSettingsRef().max_insert_block_size.changed) if (global_context->getSettingsRef().max_insert_block_size.changed)
{
insert_format_max_block_size = global_context->getSettingsRef().max_insert_block_size; insert_format_max_block_size = global_context->getSettingsRef().max_insert_block_size;
}
else else
insert_format_max_block_size = config().getInt("insert_format_max_block_size", global_context->getSettingsRef().max_insert_block_size); {
insert_format_max_block_size = config().getUInt64("insert_format_max_block_size",
global_context->getSettingsRef().max_insert_block_size);
}
ClientInfo & client_info = global_context->getClientInfo(); ClientInfo & client_info = global_context->getClientInfo();
client_info.setInitialQuery(); client_info.setInitialQuery();

View File

@ -47,8 +47,8 @@ public:
WrappingUInt32 epoch; WrappingUInt32 epoch;
WrappingUInt32 counter; WrappingUInt32 counter;
explicit Zxid(UInt64 _zxid) explicit Zxid(UInt64 _zxid)
: epoch(_zxid >> 32) : epoch(static_cast<UInt32>(_zxid >> 32))
, counter(_zxid) , counter(static_cast<UInt32>(_zxid))
{} {}
bool operator<=(const Zxid & other) const bool operator<=(const Zxid & other) const

View File

@ -893,7 +893,7 @@ namespace
if (fs::exists(pid_file)) if (fs::exists(pid_file))
{ {
ReadBufferFromFile in(pid_file.string()); ReadBufferFromFile in(pid_file.string());
UInt64 pid; Int32 pid;
if (tryReadIntText(pid, in)) if (tryReadIntText(pid, in))
{ {
fmt::print("{} file exists and contains pid = {}.\n", pid_file.string(), pid); fmt::print("{} file exists and contains pid = {}.\n", pid_file.string(), pid);
@ -982,9 +982,9 @@ namespace
return 0; return 0;
} }
UInt64 isRunning(const fs::path & pid_file) int isRunning(const fs::path & pid_file)
{ {
UInt64 pid = 0; int pid = 0;
if (fs::exists(pid_file)) if (fs::exists(pid_file))
{ {
@ -1057,7 +1057,7 @@ namespace
if (force && do_not_kill) if (force && do_not_kill)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Specified flags are incompatible"); throw Exception(ErrorCodes::BAD_ARGUMENTS, "Specified flags are incompatible");
UInt64 pid = isRunning(pid_file); int pid = isRunning(pid_file);
if (!pid) if (!pid)
return 0; return 0;

View File

@ -68,12 +68,12 @@ namespace ErrorCodes
namespace namespace
{ {
int waitServersToFinish(std::vector<DB::ProtocolServerAdapter> & servers, size_t seconds_to_wait) size_t waitServersToFinish(std::vector<DB::ProtocolServerAdapter> & servers, size_t seconds_to_wait)
{ {
const int sleep_max_ms = 1000 * seconds_to_wait; const size_t sleep_max_ms = 1000 * seconds_to_wait;
const int sleep_one_ms = 100; const size_t sleep_one_ms = 100;
int sleep_current_ms = 0; size_t sleep_current_ms = 0;
int current_connections = 0; size_t current_connections = 0;
for (;;) for (;;)
{ {
current_connections = 0; current_connections = 0;
@ -441,7 +441,7 @@ int Keeper::main(const std::vector<std::string> & /*args*/)
main_config_reloader.reset(); main_config_reloader.reset();
LOG_DEBUG(log, "Waiting for current connections to Keeper to finish."); LOG_DEBUG(log, "Waiting for current connections to Keeper to finish.");
int current_connections = 0; size_t current_connections = 0;
for (auto & server : *servers) for (auto & server : *servers)
{ {
server.stop(); server.stop();

View File

@ -546,9 +546,14 @@ void LocalServer::processConfig()
/// Setting value from cmd arg overrides one from config /// Setting value from cmd arg overrides one from config
if (global_context->getSettingsRef().max_insert_block_size.changed) if (global_context->getSettingsRef().max_insert_block_size.changed)
{
insert_format_max_block_size = global_context->getSettingsRef().max_insert_block_size; insert_format_max_block_size = global_context->getSettingsRef().max_insert_block_size;
}
else else
insert_format_max_block_size = config().getInt("insert_format_max_block_size", global_context->getSettingsRef().max_insert_block_size); {
insert_format_max_block_size = config().getUInt64("insert_format_max_block_size",
global_context->getSettingsRef().max_insert_block_size);
}
/// Sets external authenticators config (LDAP, Kerberos). /// Sets external authenticators config (LDAP, Kerberos).
global_context->setExternalAuthenticatorsConfig(config()); global_context->setExternalAuthenticatorsConfig(config());

View File

@ -279,7 +279,7 @@ Float transformFloatMantissa(Float x, UInt64 seed)
constexpr size_t mantissa_num_bits = std::is_same_v<Float, Float32> ? 23 : 52; constexpr size_t mantissa_num_bits = std::is_same_v<Float, Float32> ? 23 : 52;
UInt x_uint = bit_cast<UInt>(x); UInt x_uint = bit_cast<UInt>(x);
x_uint = feistelNetwork(x_uint, mantissa_num_bits, seed); x_uint = static_cast<UInt>(feistelNetwork(x_uint, mantissa_num_bits, seed));
return bit_cast<Float>(x_uint); return bit_cast<Float>(x_uint);
} }
@ -511,13 +511,13 @@ public:
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < size; ++i)
{ {
UInt32 src_datetime = src_data[i]; UInt32 src_datetime = src_data[i];
UInt32 src_date = date_lut.toDate(src_datetime); UInt32 src_date = static_cast<UInt32>(date_lut.toDate(src_datetime));
Int32 src_diff = src_datetime - src_prev_value; Int32 src_diff = src_datetime - src_prev_value;
Int32 res_diff = transformSigned(src_diff, seed); Int32 res_diff = static_cast<Int32>(transformSigned(src_diff, seed));
UInt32 new_datetime = res_prev_value + res_diff; UInt32 new_datetime = res_prev_value + res_diff;
UInt32 new_time = new_datetime - date_lut.toDate(new_datetime); UInt32 new_time = new_datetime - static_cast<UInt32>(date_lut.toDate(new_datetime));
res_data[i] = src_date + new_time; res_data[i] = src_date + new_time;
src_prev_value = src_datetime; src_prev_value = src_datetime;

View File

@ -183,7 +183,10 @@ void ODBCColumnsInfoHandler::handleRequest(HTTPServerRequest & request, HTTPServ
if (columns.empty()) if (columns.empty())
throw Exception("Columns definition was not returned", ErrorCodes::LOGICAL_ERROR); throw Exception("Columns definition was not returned", ErrorCodes::LOGICAL_ERROR);
WriteBufferFromHTTPServerResponse out(response, request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD, keep_alive_timeout); WriteBufferFromHTTPServerResponse out(
response,
request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD,
static_cast<unsigned>(keep_alive_timeout));
try try
{ {
writeStringBinary(columns.toString(), out); writeStringBinary(columns.toString(), out);

View File

@ -139,7 +139,7 @@ void ODBCSource::insertValue(
readDateTimeText(time, in, assert_cast<const DataTypeDateTime *>(data_type.get())->getTimeZone()); readDateTimeText(time, in, assert_cast<const DataTypeDateTime *>(data_type.get())->getTimeZone());
if (time < 0) if (time < 0)
time = 0; time = 0;
assert_cast<ColumnUInt32 &>(column).insertValue(time); assert_cast<ColumnUInt32 &>(column).insertValue(static_cast<UInt32>(time));
break; break;
} }
case ValueType::vtDateTime64: case ValueType::vtDateTime64:

View File

@ -228,12 +228,12 @@ catch (...)
path)); path));
} }
int waitServersToFinish(std::vector<DB::ProtocolServerAdapter> & servers, size_t seconds_to_wait) size_t waitServersToFinish(std::vector<DB::ProtocolServerAdapter> & servers, size_t seconds_to_wait)
{ {
const int sleep_max_ms = 1000 * seconds_to_wait; const size_t sleep_max_ms = 1000 * seconds_to_wait;
const int sleep_one_ms = 100; const size_t sleep_one_ms = 100;
int sleep_current_ms = 0; size_t sleep_current_ms = 0;
int current_connections = 0; size_t current_connections = 0;
for (;;) for (;;)
{ {
current_connections = 0; current_connections = 0;
@ -933,7 +933,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
else else
{ {
rlim_t old = rlim.rlim_cur; rlim_t old = rlim.rlim_cur;
rlim.rlim_cur = config().getUInt("max_open_files", rlim.rlim_max); rlim.rlim_cur = config().getUInt("max_open_files", static_cast<unsigned>(rlim.rlim_max));
int rc = setrlimit(RLIMIT_NOFILE, &rlim); int rc = setrlimit(RLIMIT_NOFILE, &rlim);
if (rc != 0) if (rc != 0)
LOG_WARNING(log, "Cannot set max number of file descriptors to {}. Try to specify max_open_files according to your system limits. error: {}", rlim.rlim_cur, errnoToString()); LOG_WARNING(log, "Cannot set max number of file descriptors to {}. Try to specify max_open_files according to your system limits. error: {}", rlim.rlim_cur, errnoToString());
@ -1507,7 +1507,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
if (!servers_to_start_before_tables.empty()) if (!servers_to_start_before_tables.empty())
{ {
LOG_DEBUG(log, "Waiting for current connections to servers for tables to finish."); LOG_DEBUG(log, "Waiting for current connections to servers for tables to finish.");
int current_connections = 0; size_t current_connections = 0;
for (auto & server : servers_to_start_before_tables) for (auto & server : servers_to_start_before_tables)
{ {
server.stop(); server.stop();
@ -1793,7 +1793,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
is_cancelled = true; is_cancelled = true;
int current_connections = 0; size_t current_connections = 0;
{ {
std::lock_guard lock(servers_lock); std::lock_guard lock(servers_lock);
for (auto & server : servers) for (auto & server : servers)

View File

@ -236,7 +236,7 @@ void AllowedClientHosts::IPSubnet::set(const IPAddress & prefix_, const IPAddres
void AllowedClientHosts::IPSubnet::set(const IPAddress & prefix_, size_t num_prefix_bits) void AllowedClientHosts::IPSubnet::set(const IPAddress & prefix_, size_t num_prefix_bits)
{ {
set(prefix_, IPAddress(num_prefix_bits, prefix_.family())); set(prefix_, IPAddress(static_cast<unsigned>(num_prefix_bits), prefix_.family()));
} }
void AllowedClientHosts::IPSubnet::set(const IPAddress & address) void AllowedClientHosts::IPSubnet::set(const IPAddress & address)

View File

@ -254,7 +254,7 @@ bool LDAPClient::openConnection()
#endif #endif
{ {
const int search_timeout = params.search_timeout.count(); const int search_timeout = static_cast<int>(params.search_timeout.count());
diag(ldap_set_option(handle, LDAP_OPT_TIMELIMIT, &search_timeout)); diag(ldap_set_option(handle, LDAP_OPT_TIMELIMIT, &search_timeout));
} }

View File

@ -45,7 +45,8 @@ public:
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{ {
assert_cast<ColumnVector<T> &>(to).getData().push_back(this->data(place).rbs.size()); assert_cast<ColumnVector<T> &>(to).getData().push_back(
static_cast<T>(this->data(place).rbs.size()));
} }
}; };
@ -142,7 +143,8 @@ public:
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{ {
assert_cast<ColumnVector<T> &>(to).getData().push_back(this->data(place).rbs.size()); assert_cast<ColumnVector<T> &>(to).getData().push_back(
static_cast<T>(this->data(place).rbs.size()));
} }
}; };

View File

@ -426,7 +426,7 @@ public:
return 0; return 0;
if (isSmall()) if (isSmall())
return small.find(x) != small.end(); return small.find(static_cast<T>(x)) != small.end();
else else
return rb->contains(static_cast<Value>(x)); return rb->contains(static_cast<Value>(x));
} }

View File

@ -136,8 +136,8 @@ private:
for (size_t i = 0; i <= size; ++i) for (size_t i = 0; i <= size; ++i)
{ {
previous[i] = i - 1; previous[i] = static_cast<UInt32>(i - 1);
next[i] = i + 1; next[i] = static_cast<UInt32>(i + 1);
} }
next[size] = 0; next[size] = 0;
@ -157,7 +157,7 @@ private:
auto quality = [&](UInt32 i) { return points[next[i]].mean - points[i].mean; }; auto quality = [&](UInt32 i) { return points[next[i]].mean - points[i].mean; };
for (size_t i = 0; i + 1 < size; ++i) for (size_t i = 0; i + 1 < size; ++i)
queue.push({quality(i), i}); queue.push({quality(static_cast<UInt32>(i)), i});
while (new_size > max_bins && !queue.empty()) while (new_size > max_bins && !queue.empty())
{ {
@ -217,7 +217,7 @@ private:
points[left] = points[right]; points[left] = points[right];
} }
} }
size = left + 1; size = static_cast<UInt32>(left + 1);
} }
public: public:

View File

@ -540,7 +540,7 @@ public:
/// Assuming to.has() /// Assuming to.has()
void changeImpl(StringRef value, Arena * arena) void changeImpl(StringRef value, Arena * arena)
{ {
Int32 value_size = value.size; Int32 value_size = static_cast<Int32>(value.size);
if (value_size <= MAX_SMALL_STRING_SIZE) if (value_size <= MAX_SMALL_STRING_SIZE)
{ {
@ -555,7 +555,7 @@ public:
if (capacity < value_size) if (capacity < value_size)
{ {
/// Don't free large_data here. /// Don't free large_data here.
capacity = roundUpToPowerOfTwoOrZero(value_size); capacity = static_cast<Int32>(roundUpToPowerOfTwoOrZero(value_size));
large_data = arena->alloc(capacity); large_data = arena->alloc(capacity);
} }

View File

@ -44,7 +44,7 @@ struct AggregateFunctionRetentionData
void serialize(WriteBuffer & buf) const void serialize(WriteBuffer & buf) const
{ {
UInt32 event_value = events.to_ulong(); UInt32 event_value = static_cast<UInt32>(events.to_ulong());
writeBinary(event_value, buf); writeBinary(event_value, buf);
} }

View File

@ -272,7 +272,7 @@ private:
actions.emplace_back(PatternActionType::SpecificEvent, event_number - 1); actions.emplace_back(PatternActionType::SpecificEvent, event_number - 1);
dfa_states.back().transition = DFATransition::SpecificEvent; dfa_states.back().transition = DFATransition::SpecificEvent;
dfa_states.back().event = event_number - 1; dfa_states.back().event = static_cast<uint32_t>(event_number - 1);
dfa_states.emplace_back(); dfa_states.emplace_back();
conditions_in_pattern.set(event_number - 1); conditions_in_pattern.set(event_number - 1);
} }

View File

@ -226,7 +226,7 @@ public:
for (UInt8 i = 0; i < events_size; ++i) for (UInt8 i = 0; i < events_size; ++i)
if (assert_cast<const ColumnVector<UInt8> *>(columns[min_required_args + i])->getData()[row_num]) if (assert_cast<const ColumnVector<UInt8> *>(columns[min_required_args + i])->getData()[row_num])
node->events_bitset.set(i); node->events_bitset.set(i);
node->event_time = timestamp; node->event_time = static_cast<DataTypeDateTime::FieldType>(timestamp);
node->can_be_base = assert_cast<const ColumnVector<UInt8> *>(columns[base_cond_column_idx])->getData()[row_num]; node->can_be_base = assert_cast<const ColumnVector<UInt8> *>(columns[base_cond_column_idx])->getData()[row_num];
@ -365,7 +365,7 @@ public:
/// The first matched event is 0x00000001, the second one is 0x00000002, the third one is 0x00000004, and so on. /// The first matched event is 0x00000001, the second one is 0x00000002, the third one is 0x00000004, and so on.
UInt32 getNextNodeIndex(Data & data) const UInt32 getNextNodeIndex(Data & data) const
{ {
const UInt32 unmatched_idx = data.value.size(); const UInt32 unmatched_idx = static_cast<UInt32>(data.value.size());
if (data.value.size() <= events_size) if (data.value.size() <= events_size)
return unmatched_idx; return unmatched_idx;

View File

@ -165,7 +165,7 @@ private:
{ {
for (size_t i = 0; i <= diff_x; ++i) for (size_t i = 0; i <= diff_x; ++i)
{ {
auto it = data.points.find(min_x_local + i); auto it = data.points.find(static_cast<X>(min_x_local + i));
bool found = it != data.points.end(); bool found = it != data.points.end();
value += getBar(found ? std::round(((it->getMapped() - min_y) / diff_y) * 7) + 1 : 0.0); value += getBar(found ? std::round(((it->getMapped() - min_y) / diff_y) * 7) + 1 : 0.0);
} }
@ -173,7 +173,7 @@ private:
else else
{ {
for (size_t i = 0; i <= diff_x; ++i) for (size_t i = 0; i <= diff_x; ++i)
value += getBar(data.points.has(min_x_local + i) ? 1 : 0); value += getBar(data.points.has(min_x_local + static_cast<X>(i)) ? 1 : 0);
} }
} }
else else
@ -202,7 +202,7 @@ private:
if (i == bound.first) // is bound if (i == bound.first) // is bound
{ {
Float64 proportion = bound.second - bound.first; Float64 proportion = bound.second - bound.first;
auto it = data.points.find(min_x_local + i); auto it = data.points.find(min_x_local + static_cast<X>(i));
bool found = (it != data.points.end()); bool found = (it != data.points.end());
if (found && proportion > 0) if (found && proportion > 0)
new_y = new_y.value_or(0) + it->getMapped() * proportion; new_y = new_y.value_or(0) + it->getMapped() * proportion;
@ -229,7 +229,7 @@ private:
} }
else else
{ {
auto it = data.points.find(min_x_local + i); auto it = data.points.find(min_x_local + static_cast<X>(i));
if (it != data.points.end()) if (it != data.points.end())
new_y = new_y.value_or(0) + it->getMapped(); new_y = new_y.value_or(0) + it->getMapped();
} }
@ -267,8 +267,8 @@ public:
if (params.size() == 3) if (params.size() == 3)
{ {
specified_min_max_x = true; specified_min_max_x = true;
min_x = params.at(1).safeGet<X>(); min_x = static_cast<X>(params.at(1).safeGet<X>());
max_x = params.at(2).safeGet<X>(); max_x = static_cast<X>(params.at(2).safeGet<X>());
} }
else else
{ {

View File

@ -175,8 +175,9 @@ struct OneAdder
{ {
if constexpr (!std::is_same_v<T, String>) if constexpr (!std::is_same_v<T, String>)
{ {
using ValueType = typename decltype(data.set)::value_type;
const auto & value = assert_cast<const ColumnVector<T> &>(column).getElement(row_num); const auto & value = assert_cast<const ColumnVector<T> &>(column).getElement(row_num);
data.set.insert(AggregateFunctionUniqTraits<T>::hash(value)); data.set.insert(static_cast<ValueType>(AggregateFunctionUniqTraits<T>::hash(value)));
} }
else else
{ {

View File

@ -43,7 +43,7 @@ namespace detail
{ {
static Ret hash(UInt128 x) static Ret hash(UInt128 x)
{ {
return sipHash64(x); return static_cast<Ret>(sipHash64(x));
} }
}; };

View File

@ -84,7 +84,7 @@ public:
if (isNaN(v)) if (isNaN(v))
return; return;
UInt32 hash = intHash64(determinator); UInt32 hash = static_cast<UInt32>(intHash64(determinator));
insertImpl(v, hash); insertImpl(v, hash);
sorted = false; sorted = false;
++total_values; ++total_values;

View File

@ -118,7 +118,7 @@ private:
HashValue hash(Value key) const HashValue hash(Value key) const
{ {
return Hash()(key); return static_cast<HashValue>(Hash()(key));
} }
/// Delete all values whose hashes do not divide by 2 ^ skip_degree /// Delete all values whose hashes do not divide by 2 ^ skip_degree

View File

@ -44,12 +44,12 @@ namespace
S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration( S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration(
settings.auth_settings.region, settings.auth_settings.region,
context->getRemoteHostFilter(), context->getRemoteHostFilter(),
context->getGlobalContext()->getSettingsRef().s3_max_redirects, static_cast<unsigned>(context->getGlobalContext()->getSettingsRef().s3_max_redirects),
context->getGlobalContext()->getSettingsRef().enable_s3_requests_logging, context->getGlobalContext()->getSettingsRef().enable_s3_requests_logging,
/* for_disk_s3 = */ false); /* for_disk_s3 = */ false);
client_configuration.endpointOverride = s3_uri.endpoint; client_configuration.endpointOverride = s3_uri.endpoint;
client_configuration.maxConnections = context->getSettingsRef().s3_max_connections; client_configuration.maxConnections = static_cast<unsigned>(context->getSettingsRef().s3_max_connections);
/// Increase connect timeout /// Increase connect timeout
client_configuration.connectTimeoutMs = 10 * 1000; client_configuration.connectTimeoutMs = 10 * 1000;
/// Requests in backups can be extremely long, set to one hour /// Requests in backups can be extremely long, set to one hour
@ -221,7 +221,7 @@ void BackupWriterS3::copyObjectMultipartImpl(
part_request.SetBucket(dst_bucket); part_request.SetBucket(dst_bucket);
part_request.SetKey(dst_key); part_request.SetKey(dst_key);
part_request.SetUploadId(multipart_upload_id); part_request.SetUploadId(multipart_upload_id);
part_request.SetPartNumber(part_number); part_request.SetPartNumber(static_cast<int>(part_number));
part_request.SetCopySourceRange(fmt::format("bytes={}-{}", position, std::min(size, position + upload_part_size) - 1)); part_request.SetCopySourceRange(fmt::format("bytes={}-{}", position, std::min(size, position + upload_part_size) - 1));
auto outcome = client->UploadPartCopy(part_request); auto outcome = client->UploadPartCopy(part_request);
@ -251,7 +251,7 @@ void BackupWriterS3::copyObjectMultipartImpl(
for (size_t i = 0; i < part_tags.size(); ++i) for (size_t i = 0; i < part_tags.size(); ++i)
{ {
Aws::S3::Model::CompletedPart part; Aws::S3::Model::CompletedPart part;
multipart_upload.AddParts(part.WithETag(part_tags[i]).WithPartNumber(i + 1)); multipart_upload.AddParts(part.WithETag(part_tags[i]).WithPartNumber(static_cast<int>(i) + 1));
} }
req.SetMultipartUpload(multipart_upload); req.SetMultipartUpload(multipart_upload);

View File

@ -62,7 +62,6 @@ namespace
#define LIST_OF_BACKUP_SETTINGS(M) \ #define LIST_OF_BACKUP_SETTINGS(M) \
M(String, id) \ M(String, id) \
M(String, compression_method) \ M(String, compression_method) \
M(Int64, compression_level) \
M(String, password) \ M(String, password) \
M(Bool, structure_only) \ M(Bool, structure_only) \
M(Bool, async) \ M(Bool, async) \
@ -72,6 +71,7 @@ namespace
M(String, host_id) \ M(String, host_id) \
M(String, coordination_zk_path) \ M(String, coordination_zk_path) \
M(OptionalUUID, backup_uuid) M(OptionalUUID, backup_uuid)
/// M(Int64, compression_level)
BackupSettings BackupSettings::fromBackupQuery(const ASTBackupQuery & query) BackupSettings BackupSettings::fromBackupQuery(const ASTBackupQuery & query)
{ {
@ -82,6 +82,9 @@ BackupSettings BackupSettings::fromBackupQuery(const ASTBackupQuery & query)
const auto & settings = query.settings->as<const ASTSetQuery &>().changes; const auto & settings = query.settings->as<const ASTSetQuery &>().changes;
for (const auto & setting : settings) for (const auto & setting : settings)
{ {
if (setting.name == "compression_level")
res.compression_level = static_cast<int>(SettingFieldInt64{setting.value}.value);
else
#define GET_SETTINGS_FROM_BACKUP_QUERY_HELPER(TYPE, NAME) \ #define GET_SETTINGS_FROM_BACKUP_QUERY_HELPER(TYPE, NAME) \
if (setting.name == #NAME) \ if (setting.name == #NAME) \
res.NAME = SettingField##TYPE{setting.value}.value; \ res.NAME = SettingField##TYPE{setting.value}.value; \

View File

@ -43,7 +43,7 @@ private:
std::string hostname; std::string hostname;
size_t port; size_t port;
std::string log_level; std::string log_level;
size_t max_server_connections; unsigned max_server_connections;
size_t http_timeout; size_t http_timeout;
Poco::Logger * log; Poco::Logger * log;

View File

@ -43,7 +43,7 @@ protected:
virtual String serviceFileName() const = 0; virtual String serviceFileName() const = 0;
virtual size_t getDefaultPort() const = 0; virtual unsigned getDefaultPort() const = 0;
virtual bool startBridgeManually() const = 0; virtual bool startBridgeManually() const = 0;

View File

@ -23,7 +23,7 @@ protected:
String serviceFileName() const override { return serviceAlias(); } String serviceFileName() const override { return serviceAlias(); }
size_t getDefaultPort() const override { return DEFAULT_PORT; } unsigned getDefaultPort() const override { return DEFAULT_PORT; }
bool startBridgeManually() const override { return false; } bool startBridgeManually() const override { return false; }

View File

@ -109,7 +109,7 @@ protected:
String getName() const override { return BridgeHelperMixin::getName(); } String getName() const override { return BridgeHelperMixin::getName(); }
size_t getDefaultPort() const override { return DEFAULT_PORT; } unsigned getDefaultPort() const override { return DEFAULT_PORT; }
String serviceAlias() const override { return BridgeHelperMixin::serviceAlias(); } String serviceAlias() const override { return BridgeHelperMixin::serviceAlias(); }

View File

@ -550,7 +550,7 @@ try
out_file_buf = wrapWriteBufferWithCompressionMethod( out_file_buf = wrapWriteBufferWithCompressionMethod(
std::make_unique<WriteBufferFromFile>(out_file, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_EXCL | O_CREAT), std::make_unique<WriteBufferFromFile>(out_file, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_EXCL | O_CREAT),
compression_method, compression_method,
compression_level static_cast<int>(compression_level)
); );
if (query_with_output->is_into_outfile_with_stdout) if (query_with_output->is_into_outfile_with_stdout)
@ -1602,6 +1602,8 @@ MultiQueryProcessingStage ClientBase::analyzeMultiQueryText(
if (this_query_begin >= all_queries_end) if (this_query_begin >= all_queries_end)
return MultiQueryProcessingStage::QUERIES_END; return MultiQueryProcessingStage::QUERIES_END;
unsigned max_parser_depth = static_cast<unsigned>(global_context->getSettingsRef().max_parser_depth);
// If there are only comments left until the end of file, we just // If there are only comments left until the end of file, we just
// stop. The parser can't handle this situation because it always // stop. The parser can't handle this situation because it always
// expects that there is some query that it can parse. // expects that there is some query that it can parse.
@ -1611,7 +1613,7 @@ MultiQueryProcessingStage ClientBase::analyzeMultiQueryText(
// and it makes more sense to treat them as such. // and it makes more sense to treat them as such.
{ {
Tokens tokens(this_query_begin, all_queries_end); Tokens tokens(this_query_begin, all_queries_end);
IParser::Pos token_iterator(tokens, global_context->getSettingsRef().max_parser_depth); IParser::Pos token_iterator(tokens, max_parser_depth);
if (!token_iterator.isValid()) if (!token_iterator.isValid())
return MultiQueryProcessingStage::QUERIES_END; return MultiQueryProcessingStage::QUERIES_END;
} }
@ -1632,7 +1634,7 @@ MultiQueryProcessingStage ClientBase::analyzeMultiQueryText(
if (ignore_error) if (ignore_error)
{ {
Tokens tokens(this_query_begin, all_queries_end); Tokens tokens(this_query_begin, all_queries_end);
IParser::Pos token_iterator(tokens, global_context->getSettingsRef().max_parser_depth); IParser::Pos token_iterator(tokens, max_parser_depth);
while (token_iterator->type != TokenType::Semicolon && token_iterator.isValid()) while (token_iterator->type != TokenType::Semicolon && token_iterator.isValid())
++token_iterator; ++token_iterator;
this_query_begin = token_iterator->end; this_query_begin = token_iterator->end;
@ -1672,7 +1674,7 @@ MultiQueryProcessingStage ClientBase::analyzeMultiQueryText(
// after we have processed the query. But even this guess is // after we have processed the query. But even this guess is
// beneficial so that we see proper trailing comments in "echo" and // beneficial so that we see proper trailing comments in "echo" and
// server log. // server log.
adjustQueryEnd(this_query_end, all_queries_end, global_context->getSettingsRef().max_parser_depth); adjustQueryEnd(this_query_end, all_queries_end, max_parser_depth);
return MultiQueryProcessingStage::EXECUTE_QUERY; return MultiQueryProcessingStage::EXECUTE_QUERY;
} }
@ -1866,7 +1868,9 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text)
if (insert_ast && isSyncInsertWithData(*insert_ast, global_context)) if (insert_ast && isSyncInsertWithData(*insert_ast, global_context))
{ {
this_query_end = insert_ast->end; this_query_end = insert_ast->end;
adjustQueryEnd(this_query_end, all_queries_end, global_context->getSettingsRef().max_parser_depth); adjustQueryEnd(
this_query_end, all_queries_end,
static_cast<unsigned>(global_context->getSettingsRef().max_parser_depth));
} }
// Report error. // Report error.
@ -2347,7 +2351,7 @@ void ClientBase::init(int argc, char ** argv)
if (options.count("print-profile-events")) if (options.count("print-profile-events"))
config().setBool("print-profile-events", true); config().setBool("print-profile-events", true);
if (options.count("profile-events-delay-ms")) if (options.count("profile-events-delay-ms"))
config().setInt("profile-events-delay-ms", options["profile-events-delay-ms"].as<UInt64>()); config().setUInt64("profile-events-delay-ms", options["profile-events-delay-ms"].as<UInt64>());
if (options.count("progress")) if (options.count("progress"))
config().setBool("progress", true); config().setBool("progress", true);
if (options.count("echo")) if (options.count("echo"))

View File

@ -338,7 +338,7 @@ HedgedConnections::ReplicaLocation HedgedConnections::getReadyReplicaLocation(As
offset_states[location.offset].replicas[location.index].change_replica_timeout.reset(); offset_states[location.offset].replicas[location.index].change_replica_timeout.reset();
offset_states[location.offset].replicas[location.index].is_change_replica_timeout_expired = true; offset_states[location.offset].replicas[location.index].is_change_replica_timeout_expired = true;
offset_states[location.offset].next_replica_in_process = true; offset_states[location.offset].next_replica_in_process = true;
offsets_queue.push(location.offset); offsets_queue.push(static_cast<int>(location.offset));
ProfileEvents::increment(ProfileEvents::HedgedRequestsChangeReplica); ProfileEvents::increment(ProfileEvents::HedgedRequestsChangeReplica);
startNewReplica(); startNewReplica();
} }

View File

@ -362,7 +362,7 @@ void HedgedConnectionsFactory::removeReplicaFromEpoll(int index, int fd)
timeout_fd_to_replica_index.erase(replicas[index].change_replica_timeout.getDescriptor()); timeout_fd_to_replica_index.erase(replicas[index].change_replica_timeout.getDescriptor());
} }
int HedgedConnectionsFactory::numberOfProcessingReplicas() const size_t HedgedConnectionsFactory::numberOfProcessingReplicas() const
{ {
if (epoll.empty()) if (epoll.empty())
return 0; return 0;
@ -381,7 +381,7 @@ HedgedConnectionsFactory::State HedgedConnectionsFactory::setBestUsableReplica(C
&& result.is_usable && result.is_usable
&& !replicas[i].is_ready && !replicas[i].is_ready
&& (!skip_replicas_with_two_level_aggregation_incompatibility || !isTwoLevelAggregationIncompatible(&*result.entry))) && (!skip_replicas_with_two_level_aggregation_incompatibility || !isTwoLevelAggregationIncompatible(&*result.entry)))
indexes.push_back(i); indexes.push_back(static_cast<int>(i));
} }
if (indexes.empty()) if (indexes.empty())

View File

@ -70,7 +70,7 @@ public:
const ConnectionTimeouts & getConnectionTimeouts() const { return timeouts; } const ConnectionTimeouts & getConnectionTimeouts() const { return timeouts; }
int numberOfProcessingReplicas() const; size_t numberOfProcessingReplicas() const;
/// Tell Factory to not return connections with two level aggregation incompatibility. /// Tell Factory to not return connections with two level aggregation incompatibility.
void skipReplicasWithTwoLevelAggregationIncompatibility() { skip_replicas_with_two_level_aggregation_incompatibility = true; } void skipReplicasWithTwoLevelAggregationIncompatibility() { skip_replicas_with_two_level_aggregation_incompatibility = true; }

View File

@ -81,9 +81,9 @@ Field QueryFuzzer::getRandomField(int type)
{ {
static constexpr UInt64 scales[] = {0, 1, 2, 10}; static constexpr UInt64 scales[] = {0, 1, 2, 10};
return DecimalField<Decimal64>( return DecimalField<Decimal64>(
bad_int64_values[fuzz_rand() % (sizeof(bad_int64_values) bad_int64_values[fuzz_rand() % (sizeof(bad_int64_values) / sizeof(*bad_int64_values))],
/ sizeof(*bad_int64_values))], static_cast<UInt32>(scales[fuzz_rand() % (sizeof(scales) / sizeof(*scales))])
scales[fuzz_rand() % (sizeof(scales) / sizeof(*scales))]); );
} }
default: default:
assert(false); assert(false);

View File

@ -569,8 +569,8 @@ void ColumnArray::expand(const IColumn::Filter & mask, bool inverted)
if (mask.size() < offsets_data.size()) if (mask.size() < offsets_data.size())
throw Exception("Mask size should be no less than data size.", ErrorCodes::LOGICAL_ERROR); throw Exception("Mask size should be no less than data size.", ErrorCodes::LOGICAL_ERROR);
int index = mask.size() - 1; ssize_t index = mask.size() - 1;
int from = offsets_data.size() - 1; ssize_t from = offsets_data.size() - 1;
offsets_data.resize(mask.size()); offsets_data.resize(mask.size());
UInt64 last_offset = offsets_data[from]; UInt64 last_offset = offsets_data[from];
while (index >= 0) while (index >= 0)

View File

@ -27,8 +27,8 @@ std::shared_ptr<Memory<>> ColumnCompressed::compressBuffer(const void * data, si
auto compressed_size = LZ4_compress_default( auto compressed_size = LZ4_compress_default(
reinterpret_cast<const char *>(data), reinterpret_cast<const char *>(data),
compressed.data(), compressed.data(),
data_size, static_cast<UInt32>(data_size),
max_dest_size); static_cast<UInt32>(max_dest_size));
if (compressed_size <= 0) if (compressed_size <= 0)
throw Exception(ErrorCodes::CANNOT_COMPRESS, "Cannot compress column"); throw Exception(ErrorCodes::CANNOT_COMPRESS, "Cannot compress column");
@ -51,8 +51,8 @@ void ColumnCompressed::decompressBuffer(
auto processed_size = LZ4_decompress_safe( auto processed_size = LZ4_decompress_safe(
reinterpret_cast<const char *>(compressed_data), reinterpret_cast<const char *>(compressed_data),
reinterpret_cast<char *>(decompressed_data), reinterpret_cast<char *>(decompressed_data),
compressed_size, static_cast<UInt32>(compressed_size),
decompressed_size); static_cast<UInt32>(decompressed_size));
if (processed_size <= 0) if (processed_size <= 0)
throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress column"); throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress column");

View File

@ -277,8 +277,8 @@ void ColumnFixedString::expand(const IColumn::Filter & mask, bool inverted)
if (mask.size() < size()) if (mask.size() < size())
throw Exception("Mask size should be no less than data size.", ErrorCodes::LOGICAL_ERROR); throw Exception("Mask size should be no less than data size.", ErrorCodes::LOGICAL_ERROR);
int index = mask.size() - 1; ssize_t index = mask.size() - 1;
int from = size() - 1; ssize_t from = size() - 1;
chars.resize_fill(mask.size() * n, 0); chars.resize_fill(mask.size() * n, 0);
while (index >= 0) while (index >= 0)
{ {

View File

@ -46,7 +46,7 @@ namespace
HashMap<T, T> hash_map; HashMap<T, T> hash_map;
for (auto val : index) for (auto val : index)
hash_map.insert({val, hash_map.size()}); hash_map.insert({val, static_cast<unsigned>(hash_map.size())});
auto res_col = ColumnVector<T>::create(); auto res_col = ColumnVector<T>::create();
auto & data = res_col->getData(); auto & data = res_col->getData();
@ -632,7 +632,7 @@ void ColumnLowCardinality::Index::convertPositions()
/// TODO: Optimize with SSE? /// TODO: Optimize with SSE?
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < size; ++i)
new_data[i] = data[i]; new_data[i] = static_cast<CurIndexType>(data[i]);
positions = std::move(new_positions); positions = std::move(new_positions);
size_of_type = sizeof(IndexType); size_of_type = sizeof(IndexType);
@ -717,7 +717,7 @@ void ColumnLowCardinality::Index::insertPositionsRange(const IColumn & column, U
positions_data.resize(size + limit); positions_data.resize(size + limit);
for (UInt64 i = 0; i < limit; ++i) for (UInt64 i = 0; i < limit; ++i)
positions_data[size + i] = column_data[offset + i]; positions_data[size + i] = static_cast<CurIndexType>(column_data[offset + i]);
}; };
callForType(std::move(copy), size_of_type); callForType(std::move(copy), size_of_type);

View File

@ -168,8 +168,8 @@ void ColumnString::expand(const IColumn::Filter & mask, bool inverted)
/// We cannot change only offsets, because each string should end with terminating zero byte. /// We cannot change only offsets, because each string should end with terminating zero byte.
/// So, we will insert one zero byte when mask value is zero. /// So, we will insert one zero byte when mask value is zero.
int index = mask.size() - 1; ssize_t index = mask.size() - 1;
int from = offsets_data.size() - 1; ssize_t from = offsets_data.size() - 1;
/// mask.size() - offsets_data.size() should be equal to the number of zeros in mask /// mask.size() - offsets_data.size() should be equal to the number of zeros in mask
/// (if not, one of exceptions below will throw) and we can calculate the resulting chars size. /// (if not, one of exceptions below will throw) and we can calculate the resulting chars size.
UInt64 last_offset = offsets_data[from] + (mask.size() - offsets_data.size()); UInt64 last_offset = offsets_data[from] + (mask.size() - offsets_data.size());

View File

@ -550,7 +550,7 @@ MutableColumnPtr ColumnUnique<ColumnType>::uniqueInsertRangeImpl(
auto insert_key = [&](StringRef ref, ReverseIndex<UInt64, ColumnType> & cur_index) -> MutableColumnPtr auto insert_key = [&](StringRef ref, ReverseIndex<UInt64, ColumnType> & cur_index) -> MutableColumnPtr
{ {
auto inserted_pos = cur_index.insert(ref); auto inserted_pos = cur_index.insert(ref);
positions[num_added_rows] = inserted_pos; positions[num_added_rows] = static_cast<IndexType>(inserted_pos);
if (inserted_pos == next_position) if (inserted_pos == next_position)
return update_position(next_position); return update_position(next_position);
@ -562,9 +562,9 @@ MutableColumnPtr ColumnUnique<ColumnType>::uniqueInsertRangeImpl(
auto row = start + num_added_rows; auto row = start + num_added_rows;
if (null_map && (*null_map)[row]) if (null_map && (*null_map)[row])
positions[num_added_rows] = getNullValueIndex(); positions[num_added_rows] = static_cast<IndexType>(getNullValueIndex());
else if (column->compareAt(getNestedTypeDefaultValueIndex(), row, *src_column, 1) == 0) else if (column->compareAt(getNestedTypeDefaultValueIndex(), row, *src_column, 1) == 0)
positions[num_added_rows] = getNestedTypeDefaultValueIndex(); positions[num_added_rows] = static_cast<IndexType>(getNestedTypeDefaultValueIndex());
else else
{ {
auto ref = src_column->getDataAt(row); auto ref = src_column->getDataAt(row);
@ -576,7 +576,7 @@ MutableColumnPtr ColumnUnique<ColumnType>::uniqueInsertRangeImpl(
if (insertion_point == reverse_index.lastInsertionPoint()) if (insertion_point == reverse_index.lastInsertionPoint())
res = insert_key(ref, *secondary_index); res = insert_key(ref, *secondary_index);
else else
positions[num_added_rows] = insertion_point; positions[num_added_rows] = static_cast<IndexType>(insertion_point);
} }
else else
res = insert_key(ref, reverse_index); res = insert_key(ref, reverse_index);

View File

@ -22,8 +22,8 @@ void expandDataByMask(PaddedPODArray<T> & data, const PaddedPODArray<UInt8> & ma
if (mask.size() < data.size()) if (mask.size() < data.size())
throw Exception("Mask size should be no less than data size.", ErrorCodes::LOGICAL_ERROR); throw Exception("Mask size should be no less than data size.", ErrorCodes::LOGICAL_ERROR);
int from = data.size() - 1; ssize_t from = data.size() - 1;
int index = mask.size() - 1; ssize_t index = mask.size() - 1;
data.resize(mask.size()); data.resize(mask.size());
while (index >= 0) while (index >= 0)
{ {
@ -317,7 +317,7 @@ int checkShortCircuitArguments(const ColumnsWithTypeAndName & arguments)
for (size_t i = 0; i != arguments.size(); ++i) for (size_t i = 0; i != arguments.size(); ++i)
{ {
if (checkAndGetShortCircuitArgument(arguments[i].column)) if (checkAndGetShortCircuitArgument(arguments[i].column))
last_short_circuit_argument_index = i; last_short_circuit_argument_index = static_cast<int>(i);
} }
return last_short_circuit_argument_index; return last_short_circuit_argument_index;

View File

@ -164,7 +164,7 @@ TEST(WeakHash32, ColumnVectorU32)
for (int idx [[maybe_unused]] : {1, 2}) for (int idx [[maybe_unused]] : {1, 2})
{ {
for (uint64_t i = 0; i < 65536; ++i) for (uint32_t i = 0; i < 65536; ++i)
data.push_back(i << 16u); data.push_back(i << 16u);
} }
@ -181,7 +181,7 @@ TEST(WeakHash32, ColumnVectorI32)
for (int idx [[maybe_unused]] : {1, 2}) for (int idx [[maybe_unused]] : {1, 2})
{ {
for (int64_t i = -32768; i < 32768; ++i) for (int32_t i = -32768; i < 32768; ++i)
data.push_back(i << 16); //-V610 data.push_back(i << 16); //-V610
} }
@ -240,7 +240,7 @@ TEST(WeakHash32, ColumnVectorU128)
val.items[0] = i << 32u; val.items[0] = i << 32u;
val.items[1] = i << 32u; val.items[1] = i << 32u;
data.push_back(val); data.push_back(val);
eq_data.push_back(i); eq_data.push_back(static_cast<UInt32>(i));
} }
} }
@ -274,7 +274,7 @@ TEST(WeakHash32, ColumnDecimal32)
for (int idx [[maybe_unused]] : {1, 2}) for (int idx [[maybe_unused]] : {1, 2})
{ {
for (int64_t i = -32768; i < 32768; ++i) for (int32_t i = -32768; i < 32768; ++i)
data.push_back(i << 16); //-V610 data.push_back(i << 16); //-V610
} }
@ -326,7 +326,7 @@ TEST(WeakHash32, ColumnString1)
for (int idx [[maybe_unused]] : {1, 2}) for (int idx [[maybe_unused]] : {1, 2})
{ {
for (int64_t i = 0; i < 65536; ++i) for (int32_t i = 0; i < 65536; ++i)
{ {
data.push_back(i); data.push_back(i);
auto str = std::to_string(i); auto str = std::to_string(i);
@ -359,7 +359,7 @@ TEST(WeakHash32, ColumnString2)
{ {
size_t max_size = 3000; size_t max_size = 3000;
char letter = 'a'; char letter = 'a';
for (int64_t i = 0; i < 65536; ++i) for (int32_t i = 0; i < 65536; ++i)
{ {
data.push_back(i); data.push_back(i);
size_t s = (i % max_size) + 1; size_t s = (i % max_size) + 1;
@ -401,7 +401,7 @@ TEST(WeakHash32, ColumnString3)
char letter = 'a'; char letter = 'a';
for (int64_t i = 0; i < 65536; ++i) for (int64_t i = 0; i < 65536; ++i)
{ {
data.push_back(i); data.push_back(static_cast<UInt32>(i));
size_t s = (i % max_size) + 1; size_t s = (i % max_size) + 1;
std::string str(s,'\0'); std::string str(s,'\0');
str[0] = letter; str[0] = letter;
@ -430,7 +430,7 @@ TEST(WeakHash32, ColumnFixedString)
char letter = 'a'; char letter = 'a';
for (int64_t i = 0; i < 65536; ++i) for (int64_t i = 0; i < 65536; ++i)
{ {
data.push_back(i); data.push_back(static_cast<UInt32>(i));
size_t s = (i % max_size) + 1; size_t s = (i % max_size) + 1;
std::string str(s, letter); std::string str(s, letter);
col->insertData(str.data(), str.size()); col->insertData(str.data(), str.size());
@ -471,7 +471,7 @@ TEST(WeakHash32, ColumnArray)
UInt32 cur = 0; UInt32 cur = 0;
for (int64_t i = 0; i < 65536; ++i) for (int64_t i = 0; i < 65536; ++i)
{ {
eq_data.push_back(i); eq_data.push_back(static_cast<UInt32>(i));
size_t s = (i % max_size) + 1; size_t s = (i % max_size) + 1;
cur_off += s; cur_off += s;
@ -505,9 +505,9 @@ TEST(WeakHash32, ColumnArray2)
UInt64 cur_off = 0; UInt64 cur_off = 0;
for (int idx [[maybe_unused]] : {1, 2}) for (int idx [[maybe_unused]] : {1, 2})
{ {
for (int64_t i = 0; i < 1000; ++i) for (int32_t i = 0; i < 1000; ++i)
{ {
for (size_t j = 0; j < 1000; ++j) for (uint32_t j = 0; j < 1000; ++j)
{ {
eq_data.push_back(i * 1000 + j); eq_data.push_back(i * 1000 + j);
@ -556,7 +556,7 @@ TEST(WeakHash32, ColumnArrayArray)
UInt32 cur = 1; UInt32 cur = 1;
for (int64_t i = 0; i < 3000; ++i) for (int64_t i = 0; i < 3000; ++i)
{ {
eq_data.push_back(i); eq_data.push_back(static_cast<UInt32>(i));
size_t s = (i % max_size) + 1; size_t s = (i % max_size) + 1;
cur_off2 += s; cur_off2 += s;
@ -667,7 +667,7 @@ TEST(WeakHash32, ColumnTupleUInt64UInt64)
{ {
data1.push_back(l); data1.push_back(l);
data2.push_back(i << 32u); data2.push_back(i << 32u);
eq.push_back(l * 65536 + i); eq.push_back(static_cast<UInt32>(l * 65536 + i));
} }
} }
@ -695,7 +695,7 @@ TEST(WeakHash32, ColumnTupleUInt64String)
size_t max_size = 3000; size_t max_size = 3000;
char letter = 'a'; char letter = 'a';
for (int64_t i = 0; i < 65536; ++i) for (int32_t i = 0; i < 65536; ++i)
{ {
data1.push_back(l); data1.push_back(l);
eq.push_back(l * 65536 + i); eq.push_back(l * 65536 + i);
@ -737,7 +737,7 @@ TEST(WeakHash32, ColumnTupleUInt64FixedString)
for (int64_t i = 0; i < 65536; ++i) for (int64_t i = 0; i < 65536; ++i)
{ {
data1.push_back(l); data1.push_back(l);
eq.push_back(l * 65536 + i); eq.push_back(static_cast<Int32>(l * 65536 + i));
size_t s = (i % max_size) + 1; size_t s = (i % max_size) + 1;
std::string str(s, letter); std::string str(s, letter);
@ -778,7 +778,7 @@ TEST(WeakHash32, ColumnTupleUInt64Array)
auto l = idx % 2; auto l = idx % 2;
UInt32 cur = 0; UInt32 cur = 0;
for (int64_t i = 0; i < 65536; ++i) for (int32_t i = 0; i < 65536; ++i)
{ {
data1.push_back(l); data1.push_back(l);
eq_data.push_back(l * 65536 + i); eq_data.push_back(l * 65536 + i);

View File

@ -65,7 +65,7 @@ public:
private: private:
using Small = SmallSet<Key, small_set_size_max>; using Small = SmallSet<Key, small_set_size_max>;
using Medium = HashContainer; using Medium = HashContainer;
using Large = HyperLogLogCounter<K, Hash, HashValueType, DenominatorType, BiasEstimator, mode>; using Large = HyperLogLogCounter<K, Key, Hash, HashValueType, DenominatorType, BiasEstimator, mode>;
public: public:
CombinedCardinalityEstimator() CombinedCardinalityEstimator()

View File

@ -149,9 +149,9 @@ DateLUTImpl::DateLUTImpl(const std::string & time_zone_)
/// Fill lookup table for years and months. /// Fill lookup table for years and months.
size_t year_months_lut_index = 0; size_t year_months_lut_index = 0;
size_t first_day_of_last_month = 0; unsigned first_day_of_last_month = 0;
for (size_t day = 0; day < DATE_LUT_SIZE; ++day) for (unsigned day = 0; day < DATE_LUT_SIZE; ++day)
{ {
const Values & values = lut[day]; const Values & values = lut[day];

View File

@ -73,7 +73,7 @@ private:
return LUTIndex(0); return LUTIndex(0);
if (index >= DATE_LUT_SIZE) if (index >= DATE_LUT_SIZE)
return LUTIndex(DATE_LUT_SIZE - 1); return LUTIndex(DATE_LUT_SIZE - 1);
return LUTIndex{index}; return LUTIndex{static_cast<UInt32>(index)};
} }
template <typename T> template <typename T>
@ -229,12 +229,12 @@ private:
if (t >= lut[guess].date) if (t >= lut[guess].date)
{ {
if (guess + 1 >= DATE_LUT_SIZE || t < lut[guess + 1].date) if (guess + 1 >= DATE_LUT_SIZE || t < lut[guess + 1].date)
return LUTIndex(guess); return LUTIndex(static_cast<unsigned>(guess));
return LUTIndex(guess + 1); return LUTIndex(static_cast<unsigned>(guess) + 1);
} }
return LUTIndex(guess ? guess - 1 : 0); return LUTIndex(guess ? static_cast<unsigned>(guess) - 1 : 0);
} }
static inline LUTIndex toLUTIndex(DayNum d) static inline LUTIndex toLUTIndex(DayNum d)
@ -272,11 +272,11 @@ private:
if (likely(offset_is_whole_number_of_hours_during_epoch)) if (likely(offset_is_whole_number_of_hours_during_epoch))
{ {
if (likely(x >= 0)) if (likely(x >= 0))
return x / divisor * divisor; return static_cast<DateOrTime>(x / divisor * divisor);
/// Integer division for negative numbers rounds them towards zero (up). /// Integer division for negative numbers rounds them towards zero (up).
/// We will shift the number so it will be rounded towards -inf (down). /// We will shift the number so it will be rounded towards -inf (down).
return (x + 1 - divisor) / divisor * divisor; return static_cast<DateOrTime>((x + 1 - divisor) / divisor * divisor);
} }
Time date = find(x).date; Time date = find(x).date;
@ -285,7 +285,7 @@ private:
{ {
if (unlikely(res < 0)) if (unlikely(res < 0))
return 0; return 0;
return res; return static_cast<DateOrTime>(res);
} }
else else
return res; return res;
@ -509,7 +509,7 @@ public:
if (time >= lut[index].time_at_offset_change()) if (time >= lut[index].time_at_offset_change())
time += lut[index].amount_of_offset_change(); time += lut[index].amount_of_offset_change();
unsigned res = time / 3600; unsigned res = static_cast<unsigned>(time / 3600);
/// In case time was changed backwards at the start of next day, we will repeat the hour 23. /// In case time was changed backwards at the start of next day, we will repeat the hour 23.
return res <= 23 ? res : 23; return res <= 23 ? res : 23;
@ -548,8 +548,8 @@ public:
{ {
Time res = t % 60; Time res = t % 60;
if (likely(res >= 0)) if (likely(res >= 0))
return res; return static_cast<unsigned>(res);
return res + 60; return static_cast<unsigned>(res) + 60;
} }
LUTIndex index = findIndex(t); LUTIndex index = findIndex(t);
@ -973,7 +973,7 @@ public:
if constexpr (std::is_same_v<Date, DayNum>) if constexpr (std::is_same_v<Date, DayNum>)
return DayNum(4 + (d - 4) / days * days); return DayNum(4 + (d - 4) / days * days);
else else
return ExtendedDayNum(4 + (d - 4) / days * days); return ExtendedDayNum(static_cast<Int32>(4 + (d - 4) / days * days));
} }
template <typename Date> template <typename Date>
@ -983,9 +983,9 @@ public:
if (days == 1) if (days == 1)
return toDate(d); return toDate(d);
if constexpr (std::is_same_v<Date, DayNum>) if constexpr (std::is_same_v<Date, DayNum>)
return lut_saturated[toLUTIndex(ExtendedDayNum(d / days * days))].date; return lut_saturated[toLUTIndex(ExtendedDayNum(static_cast<Int32>(d / days * days)))].date;
else else
return lut[toLUTIndex(ExtendedDayNum(d / days * days))].date; return lut[toLUTIndex(ExtendedDayNum(static_cast<Int32>(d / days * days)))].date;
} }
template <typename DateOrTime> template <typename DateOrTime>
@ -1034,7 +1034,7 @@ public:
{ {
if (unlikely(res < 0)) if (unlikely(res < 0))
return 0; return 0;
return res; return static_cast<DateOrTime>(res);
} }
else else
return res; return res;
@ -1047,8 +1047,8 @@ public:
if (likely(offset_is_whole_number_of_minutes_during_epoch)) if (likely(offset_is_whole_number_of_minutes_during_epoch))
{ {
if (likely(t >= 0)) if (likely(t >= 0))
return t / divisor * divisor; return static_cast<DateOrTime>(t / divisor * divisor);
return (t + 1 - divisor) / divisor * divisor; return static_cast<DateOrTime>((t + 1 - divisor) / divisor * divisor);
} }
Time date = find(t).date; Time date = find(t).date;
@ -1057,7 +1057,7 @@ public:
{ {
if (unlikely(res < 0)) if (unlikely(res < 0))
return 0; return 0;
return res; return static_cast<UInt32>(res);
} }
else else
return res; return res;
@ -1071,7 +1071,7 @@ public:
if (seconds % 60 == 0) if (seconds % 60 == 0)
return toStartOfMinuteInterval(t, seconds / 60); return toStartOfMinuteInterval(t, seconds / 60);
return roundDown(t, seconds); return static_cast<DateOrTime>(roundDown(t, seconds));
} }
inline LUTIndex makeLUTIndex(Int16 year, UInt8 month, UInt8 day_of_month) const inline LUTIndex makeLUTIndex(Int16 year, UInt8 month, UInt8 day_of_month) const
@ -1249,9 +1249,9 @@ public:
return lut[new_index].date + time; return lut[new_index].date + time;
} }
inline NO_SANITIZE_UNDEFINED Time addWeeks(Time t, Int32 delta) const inline NO_SANITIZE_UNDEFINED Time addWeeks(Time t, Int64 delta) const
{ {
return addDays(t, static_cast<Int64>(delta) * 7); return addDays(t, delta * 7);
} }
inline UInt8 saturateDayOfMonth(Int16 year, UInt8 month, UInt8 day_of_month) const inline UInt8 saturateDayOfMonth(Int16 year, UInt8 month, UInt8 day_of_month) const
@ -1331,9 +1331,9 @@ public:
} }
template <typename DateOrTime> template <typename DateOrTime>
inline auto addQuarters(DateOrTime d, Int32 delta) const inline auto addQuarters(DateOrTime d, Int64 delta) const
{ {
return addMonths(d, static_cast<Int64>(delta) * 3); return addMonths(d, delta * 3);
} }
template <typename DateOrTime> template <typename DateOrTime>

View File

@ -218,7 +218,7 @@ private:
// Offset from start to first attribute // Offset from start to first attribute
uint8_t attr_offset; uint8_t attr_offset;
// Offset within debug info. // Offset within debug info.
uint32_t offset; uint64_t offset;
uint64_t code; uint64_t code;
DIEAbbreviation abbr; DIEAbbreviation abbr;
}; };
@ -252,10 +252,10 @@ private:
uint8_t unit_type = DW_UT_compile; // DW_UT_compile or DW_UT_skeleton uint8_t unit_type = DW_UT_compile; // DW_UT_compile or DW_UT_skeleton
uint8_t addr_size = 0; uint8_t addr_size = 0;
// Offset in .debug_info of this compilation unit. // Offset in .debug_info of this compilation unit.
uint32_t offset = 0; uint64_t offset = 0;
uint32_t size = 0; uint64_t size = 0;
// Offset in .debug_info for the first DIE in this compilation unit. // Offset in .debug_info for the first DIE in this compilation unit.
uint32_t first_die = 0; uint64_t first_die = 0;
uint64_t abbrev_offset = 0; uint64_t abbrev_offset = 0;
// The beginning of the CU's contribution to .debug_addr // The beginning of the CU's contribution to .debug_addr

View File

@ -48,33 +48,36 @@ inline DB::UInt64 intHash64(DB::UInt64 x)
#include <arm_acle.h> #include <arm_acle.h>
#endif #endif
inline DB::UInt64 intHashCRC32(DB::UInt64 x) /// NOTE: Intel intrinsic can be confusing.
/// - https://code.google.com/archive/p/sse-intrinsics/wikis/PmovIntrinsicBug.wiki
/// - https://stackoverflow.com/questions/15752770/mm-crc32-u64-poorly-defined
inline DB::UInt32 intHashCRC32(DB::UInt64 x)
{ {
#ifdef __SSE4_2__ #ifdef __SSE4_2__
return _mm_crc32_u64(-1ULL, x); return static_cast<UInt32>(_mm_crc32_u64(-1ULL, x));
#elif defined(__aarch64__) && defined(__ARM_FEATURE_CRC32) #elif defined(__aarch64__) && defined(__ARM_FEATURE_CRC32)
return __crc32cd(-1U, x); return static_cast<UInt32>(__crc32cd(-1U, x));
#else #else
/// On other platforms we do not have CRC32. NOTE This can be confusing. /// On other platforms we do not have CRC32. NOTE This can be confusing.
return intHash64(x); /// NOTE: consider using intHash32()
return static_cast<UInt32>(intHash64(x));
#endif #endif
} }
inline DB::UInt32 intHashCRC32(DB::UInt64 x, DB::UInt64 updated_value)
inline DB::UInt64 intHashCRC32(DB::UInt64 x, DB::UInt64 updated_value)
{ {
#ifdef __SSE4_2__ #ifdef __SSE4_2__
return _mm_crc32_u64(updated_value, x); return static_cast<UInt32>(_mm_crc32_u64(updated_value, x));
#elif defined(__aarch64__) && defined(__ARM_FEATURE_CRC32) #elif defined(__aarch64__) && defined(__ARM_FEATURE_CRC32)
return __crc32cd(updated_value, x); return __crc32cd(static_cast<UInt32>(updated_value), x);
#else #else
/// On other platforms we do not have CRC32. NOTE This can be confusing. /// On other platforms we do not have CRC32. NOTE This can be confusing.
return intHash64(x) ^ updated_value; return static_cast<UInt32>(intHash64(x) ^ updated_value);
#endif #endif
} }
template <typename T> template <typename T>
requires (sizeof(T) > sizeof(DB::UInt64)) requires (sizeof(T) > sizeof(DB::UInt64))
inline DB::UInt64 intHashCRC32(const T & x, DB::UInt64 updated_value) inline DB::UInt32 intHashCRC32(const T & x, DB::UInt64 updated_value)
{ {
const auto * begin = reinterpret_cast<const char *>(&x); const auto * begin = reinterpret_cast<const char *>(&x);
for (size_t i = 0; i < sizeof(T); i += sizeof(UInt64)) for (size_t i = 0; i < sizeof(T); i += sizeof(UInt64))
@ -83,7 +86,7 @@ inline DB::UInt64 intHashCRC32(const T & x, DB::UInt64 updated_value)
begin += sizeof(DB::UInt64); begin += sizeof(DB::UInt64);
} }
return updated_value; return static_cast<UInt32>(updated_value);
} }
@ -219,7 +222,7 @@ template <typename T> struct HashCRC32;
template <typename T> template <typename T>
requires (sizeof(T) <= sizeof(UInt64)) requires (sizeof(T) <= sizeof(UInt64))
inline size_t hashCRC32(T key, DB::UInt64 updated_value = -1) inline UInt32 hashCRC32(T key, DB::UInt64 updated_value = -1)
{ {
union union
{ {
@ -233,7 +236,7 @@ inline size_t hashCRC32(T key, DB::UInt64 updated_value = -1)
template <typename T> template <typename T>
requires (sizeof(T) > sizeof(UInt64)) requires (sizeof(T) > sizeof(UInt64))
inline size_t hashCRC32(T key, DB::UInt64 updated_value = -1) inline UInt32 hashCRC32(T key, DB::UInt64 updated_value = -1)
{ {
return intHashCRC32(key, updated_value); return intHashCRC32(key, updated_value);
} }
@ -241,7 +244,7 @@ inline size_t hashCRC32(T key, DB::UInt64 updated_value = -1)
#define DEFINE_HASH(T) \ #define DEFINE_HASH(T) \
template <> struct HashCRC32<T>\ template <> struct HashCRC32<T>\
{\ {\
size_t operator() (T key) const\ UInt32 operator() (T key) const\
{\ {\
return hashCRC32<T>(key);\ return hashCRC32<T>(key);\
}\ }\
@ -302,8 +305,8 @@ struct UInt128HashCRC32
size_t operator()(UInt128 x) const size_t operator()(UInt128 x) const
{ {
UInt64 crc = -1ULL; UInt64 crc = -1ULL;
crc = __crc32cd(crc, x.items[0]); crc = __crc32cd(static_cast<UInt32>(crc), x.items[0]);
crc = __crc32cd(crc, x.items[1]); crc = __crc32cd(static_cast<UInt32>(crc), x.items[1]);
return crc; return crc;
} }
}; };
@ -358,10 +361,10 @@ struct UInt256HashCRC32
size_t operator()(UInt256 x) const size_t operator()(UInt256 x) const
{ {
UInt64 crc = -1ULL; UInt64 crc = -1ULL;
crc = __crc32cd(crc, x.items[0]); crc = __crc32cd(static_cast<UInt32>(crc), x.items[0]);
crc = __crc32cd(crc, x.items[1]); crc = __crc32cd(static_cast<UInt32>(crc), x.items[1]);
crc = __crc32cd(crc, x.items[2]); crc = __crc32cd(static_cast<UInt32>(crc), x.items[2]);
crc = __crc32cd(crc, x.items[3]); crc = __crc32cd(static_cast<UInt32>(crc), x.items[3]);
return crc; return crc;
} }
}; };
@ -423,7 +426,7 @@ inline DB::UInt32 intHash32(DB::UInt64 key)
key = key + (key << 6); key = key + (key << 6);
key = key ^ ((key >> 22) | (key << 42)); key = key ^ ((key >> 22) | (key << 42));
return key; return static_cast<UInt32>(key);
} }

View File

@ -44,8 +44,8 @@ protected:
public: public:
using Impl = ImplTable; using Impl = ImplTable;
static constexpr size_t NUM_BUCKETS = 1ULL << BITS_FOR_BUCKET; static constexpr UInt32 NUM_BUCKETS = 1ULL << BITS_FOR_BUCKET;
static constexpr size_t MAX_BUCKET = NUM_BUCKETS - 1; static constexpr UInt32 MAX_BUCKET = NUM_BUCKETS - 1;
size_t hash(const Key & x) const { return Hash::operator()(x); } size_t hash(const Key & x) const { return Hash::operator()(x); }
@ -286,13 +286,13 @@ public:
void write(DB::WriteBuffer & wb) const void write(DB::WriteBuffer & wb) const
{ {
for (size_t i = 0; i < NUM_BUCKETS; ++i) for (UInt32 i = 0; i < NUM_BUCKETS; ++i)
impls[i].write(wb); impls[i].write(wb);
} }
void writeText(DB::WriteBuffer & wb) const void writeText(DB::WriteBuffer & wb) const
{ {
for (size_t i = 0; i < NUM_BUCKETS; ++i) for (UInt32 i = 0; i < NUM_BUCKETS; ++i)
{ {
if (i != 0) if (i != 0)
DB::writeChar(',', wb); DB::writeChar(',', wb);
@ -302,13 +302,13 @@ public:
void read(DB::ReadBuffer & rb) void read(DB::ReadBuffer & rb)
{ {
for (size_t i = 0; i < NUM_BUCKETS; ++i) for (UInt32 i = 0; i < NUM_BUCKETS; ++i)
impls[i].read(rb); impls[i].read(rb);
} }
void readText(DB::ReadBuffer & rb) void readText(DB::ReadBuffer & rb)
{ {
for (size_t i = 0; i < NUM_BUCKETS; ++i) for (UInt32 i = 0; i < NUM_BUCKETS; ++i)
{ {
if (i != 0) if (i != 0)
DB::assertChar(',', rb); DB::assertChar(',', rb);
@ -320,7 +320,7 @@ public:
size_t size() const size_t size() const
{ {
size_t res = 0; size_t res = 0;
for (size_t i = 0; i < NUM_BUCKETS; ++i) for (UInt32 i = 0; i < NUM_BUCKETS; ++i)
res += impls[i].size(); res += impls[i].size();
return res; return res;
@ -328,7 +328,7 @@ public:
bool empty() const bool empty() const
{ {
for (size_t i = 0; i < NUM_BUCKETS; ++i) for (UInt32 i = 0; i < NUM_BUCKETS; ++i)
if (!impls[i].empty()) if (!impls[i].empty())
return false; return false;
@ -338,7 +338,7 @@ public:
size_t getBufferSizeInBytes() const size_t getBufferSizeInBytes() const
{ {
size_t res = 0; size_t res = 0;
for (size_t i = 0; i < NUM_BUCKETS; ++i) for (UInt32 i = 0; i < NUM_BUCKETS; ++i)
res += impls[i].getBufferSizeInBytes(); res += impls[i].getBufferSizeInBytes();
return res; return res;

View File

@ -13,8 +13,8 @@ public:
using Key = StringRef; using Key = StringRef;
using Impl = ImplTable; using Impl = ImplTable;
static constexpr size_t NUM_BUCKETS = 1ULL << BITS_FOR_BUCKET; static constexpr UInt32 NUM_BUCKETS = 1ULL << BITS_FOR_BUCKET;
static constexpr size_t MAX_BUCKET = NUM_BUCKETS - 1; static constexpr UInt32 MAX_BUCKET = NUM_BUCKETS - 1;
// TODO: currently hashing contains redundant computations when doing distributed or external aggregations // TODO: currently hashing contains redundant computations when doing distributed or external aggregations
size_t hash(const Key & x) const size_t hash(const Key & x) const
@ -175,13 +175,13 @@ public:
void write(DB::WriteBuffer & wb) const void write(DB::WriteBuffer & wb) const
{ {
for (size_t i = 0; i < NUM_BUCKETS; ++i) for (UInt32 i = 0; i < NUM_BUCKETS; ++i)
impls[i].write(wb); impls[i].write(wb);
} }
void writeText(DB::WriteBuffer & wb) const void writeText(DB::WriteBuffer & wb) const
{ {
for (size_t i = 0; i < NUM_BUCKETS; ++i) for (UInt32 i = 0; i < NUM_BUCKETS; ++i)
{ {
if (i != 0) if (i != 0)
DB::writeChar(',', wb); DB::writeChar(',', wb);
@ -191,13 +191,13 @@ public:
void read(DB::ReadBuffer & rb) void read(DB::ReadBuffer & rb)
{ {
for (size_t i = 0; i < NUM_BUCKETS; ++i) for (UInt32 i = 0; i < NUM_BUCKETS; ++i)
impls[i].read(rb); impls[i].read(rb);
} }
void readText(DB::ReadBuffer & rb) void readText(DB::ReadBuffer & rb)
{ {
for (size_t i = 0; i < NUM_BUCKETS; ++i) for (UInt32 i = 0; i < NUM_BUCKETS; ++i)
{ {
if (i != 0) if (i != 0)
DB::assertChar(',', rb); DB::assertChar(',', rb);
@ -208,7 +208,7 @@ public:
size_t size() const size_t size() const
{ {
size_t res = 0; size_t res = 0;
for (size_t i = 0; i < NUM_BUCKETS; ++i) for (UInt32 i = 0; i < NUM_BUCKETS; ++i)
res += impls[i].size(); res += impls[i].size();
return res; return res;
@ -216,7 +216,7 @@ public:
bool empty() const bool empty() const
{ {
for (size_t i = 0; i < NUM_BUCKETS; ++i) for (UInt32 i = 0; i < NUM_BUCKETS; ++i)
if (!impls[i].empty()) if (!impls[i].empty())
return false; return false;
@ -226,7 +226,7 @@ public:
size_t getBufferSizeInBytes() const size_t getBufferSizeInBytes() const
{ {
size_t res = 0; size_t res = 0;
for (size_t i = 0; i < NUM_BUCKETS; ++i) for (UInt32 i = 0; i < NUM_BUCKETS; ++i)
res += impls[i].getBufferSizeInBytes(); res += impls[i].getBufferSizeInBytes();
return res; return res;

View File

@ -264,7 +264,8 @@ enum class HyperLogLogMode
/// of Algorithms). /// of Algorithms).
template < template <
UInt8 precision, UInt8 precision,
typename Hash = IntHash32<UInt64>, typename Key = UInt64,
typename Hash = IntHash32<Key>,
typename HashValueType = UInt32, typename HashValueType = UInt32,
typename DenominatorType = double, typename DenominatorType = double,
typename BiasEstimator = TrivialBiasEstimator, typename BiasEstimator = TrivialBiasEstimator,
@ -409,7 +410,9 @@ private:
inline HashValueType getHash(Value key) const inline HashValueType getHash(Value key) const
{ {
return Hash::operator()(key); /// NOTE: this should be OK, since value is the same as key for HLL.
return static_cast<HashValueType>(
Hash::operator()(static_cast<Key>(key)));
} }
/// Update maximum rank for current bucket. /// Update maximum rank for current bucket.
@ -532,6 +535,7 @@ private:
template template
< <
UInt8 precision, UInt8 precision,
typename Key,
typename Hash, typename Hash,
typename HashValueType, typename HashValueType,
typename DenominatorType, typename DenominatorType,
@ -542,6 +546,7 @@ template
details::LogLUT<precision> HyperLogLogCounter details::LogLUT<precision> HyperLogLogCounter
< <
precision, precision,
Key,
Hash, Hash,
HashValueType, HashValueType,
DenominatorType, DenominatorType,
@ -555,6 +560,7 @@ details::LogLUT<precision> HyperLogLogCounter
/// Serialization format must not be changed. /// Serialization format must not be changed.
using HLL12 = HyperLogLogCounter< using HLL12 = HyperLogLogCounter<
12, 12,
UInt64,
IntHash32<UInt64>, IntHash32<UInt64>,
UInt32, UInt32,
double, double,

View File

@ -26,7 +26,7 @@ class HyperLogLogWithSmallSetOptimization : private boost::noncopyable
{ {
private: private:
using Small = SmallSet<Key, small_set_size>; using Small = SmallSet<Key, small_set_size>;
using Large = HyperLogLogCounter<K, Hash, UInt32, DenominatorType>; using Large = HyperLogLogCounter<K, Key, Hash, UInt32, DenominatorType>;
using LargeValueType = typename Large::value_type; using LargeValueType = typename Large::value_type;
Small small; Small small;

View File

@ -506,8 +506,16 @@ unsigned OptimizedRegularExpressionImpl<thread_safe>::match(const char * subject
DB::PODArrayWithStackMemory<StringPieceType, 128> pieces(limit); DB::PODArrayWithStackMemory<StringPieceType, 128> pieces(limit);
if (!re2->Match(StringPieceType(subject, subject_size), 0, subject_size, RegexType::UNANCHORED, pieces.data(), pieces.size())) if (!re2->Match(
StringPieceType(subject, subject_size),
0,
subject_size,
RegexType::UNANCHORED,
pieces.data(),
static_cast<int>(pieces.size())))
{
return 0; return 0;
}
else else
{ {
matches.resize(limit); matches.resize(limit);

View File

@ -339,7 +339,7 @@ struct PoolWithFailoverBase<TNestedPool>::PoolState
Int64 config_priority = 1; Int64 config_priority = 1;
/// Priority from the GetPriorityFunc. /// Priority from the GetPriorityFunc.
Int64 priority = 0; Int64 priority = 0;
UInt32 random = 0; UInt64 random = 0;
void randomize() void randomize()
{ {

View File

@ -132,11 +132,11 @@ QueryProfilerBase<ProfilerImpl>::QueryProfilerBase(UInt64 thread_id, int clock_t
sev.sigev_signo = pause_signal; sev.sigev_signo = pause_signal;
#if defined(OS_FREEBSD) #if defined(OS_FREEBSD)
sev._sigev_un._threadid = thread_id; sev._sigev_un._threadid = static_cast<pid_t>(thread_id);
#elif defined(USE_MUSL) #elif defined(USE_MUSL)
sev.sigev_notify_thread_id = thread_id; sev.sigev_notify_thread_id = static_cast<pid_t>(thread_id);
#else #else
sev._sigev_un._tid = thread_id; sev._sigev_un._tid = static_cast<pid_t>(thread_id);
#endif #endif
timer_t local_timer_id; timer_t local_timer_id;
if (timer_create(clock_type, &sev, &local_timer_id)) if (timer_create(clock_type, &sev, &local_timer_id))

View File

@ -273,13 +273,13 @@ private:
{ {
/// Replace the histograms with the accumulated sums: the value in position i is the sum of the previous positions minus one. /// Replace the histograms with the accumulated sums: the value in position i is the sum of the previous positions minus one.
size_t sums[NUM_PASSES] = {0}; CountType sums[NUM_PASSES] = {0};
for (size_t i = 0; i < HISTOGRAM_SIZE; ++i) for (size_t i = 0; i < HISTOGRAM_SIZE; ++i)
{ {
for (size_t pass = 0; pass < NUM_PASSES; ++pass) for (size_t pass = 0; pass < NUM_PASSES; ++pass)
{ {
size_t tmp = histograms[pass * HISTOGRAM_SIZE + i] + sums[pass]; CountType tmp = histograms[pass * HISTOGRAM_SIZE + i] + sums[pass];
histograms[pass * HISTOGRAM_SIZE + i] = sums[pass] - 1; histograms[pass * HISTOGRAM_SIZE + i] = sums[pass] - 1;
sums[pass] = tmp; sums[pass] = tmp;
} }

View File

@ -40,7 +40,7 @@ public:
#ifdef __SSE2__ #ifdef __SSE2__
protected: protected:
static constexpr auto n = sizeof(__m128i); static constexpr auto n = sizeof(__m128i);
const int page_size = ::getPageSize(); const Int64 page_size = ::getPageSize();
bool pageSafe(const void * const ptr) const bool pageSafe(const void * const ptr) const
{ {

View File

@ -87,7 +87,7 @@ void ThreadPoolImpl<Thread>::setQueueSize(size_t value)
template <typename Thread> template <typename Thread>
template <typename ReturnType> template <typename ReturnType>
ReturnType ThreadPoolImpl<Thread>::scheduleImpl(Job job, int priority, std::optional<uint64_t> wait_microseconds, bool propagate_opentelemetry_tracing_context) ReturnType ThreadPoolImpl<Thread>::scheduleImpl(Job job, ssize_t priority, std::optional<uint64_t> wait_microseconds, bool propagate_opentelemetry_tracing_context)
{ {
auto on_error = [&](const std::string & reason) auto on_error = [&](const std::string & reason)
{ {
@ -163,19 +163,19 @@ ReturnType ThreadPoolImpl<Thread>::scheduleImpl(Job job, int priority, std::opti
} }
template <typename Thread> template <typename Thread>
void ThreadPoolImpl<Thread>::scheduleOrThrowOnError(Job job, int priority) void ThreadPoolImpl<Thread>::scheduleOrThrowOnError(Job job, ssize_t priority)
{ {
scheduleImpl<void>(std::move(job), priority, std::nullopt); scheduleImpl<void>(std::move(job), priority, std::nullopt);
} }
template <typename Thread> template <typename Thread>
bool ThreadPoolImpl<Thread>::trySchedule(Job job, int priority, uint64_t wait_microseconds) noexcept bool ThreadPoolImpl<Thread>::trySchedule(Job job, ssize_t priority, uint64_t wait_microseconds) noexcept
{ {
return scheduleImpl<bool>(std::move(job), priority, wait_microseconds); return scheduleImpl<bool>(std::move(job), priority, wait_microseconds);
} }
template <typename Thread> template <typename Thread>
void ThreadPoolImpl<Thread>::scheduleOrThrow(Job job, int priority, uint64_t wait_microseconds, bool propagate_opentelemetry_tracing_context) void ThreadPoolImpl<Thread>::scheduleOrThrow(Job job, ssize_t priority, uint64_t wait_microseconds, bool propagate_opentelemetry_tracing_context)
{ {
scheduleImpl<void>(std::move(job), priority, wait_microseconds, propagate_opentelemetry_tracing_context); scheduleImpl<void>(std::move(job), priority, wait_microseconds, propagate_opentelemetry_tracing_context);
} }

View File

@ -50,13 +50,13 @@ public:
/// NOTE: Probably you should call wait() if exception was thrown. If some previously scheduled jobs are using some objects, /// NOTE: Probably you should call wait() if exception was thrown. If some previously scheduled jobs are using some objects,
/// located on stack of current thread, the stack must not be unwinded until all jobs finished. However, /// located on stack of current thread, the stack must not be unwinded until all jobs finished. However,
/// if ThreadPool is a local object, it will wait for all scheduled jobs in own destructor. /// if ThreadPool is a local object, it will wait for all scheduled jobs in own destructor.
void scheduleOrThrowOnError(Job job, int priority = 0); void scheduleOrThrowOnError(Job job, ssize_t priority = 0);
/// Similar to scheduleOrThrowOnError(...). Wait for specified amount of time and schedule a job or return false. /// Similar to scheduleOrThrowOnError(...). Wait for specified amount of time and schedule a job or return false.
bool trySchedule(Job job, int priority = 0, uint64_t wait_microseconds = 0) noexcept; bool trySchedule(Job job, ssize_t priority = 0, uint64_t wait_microseconds = 0) noexcept;
/// Similar to scheduleOrThrowOnError(...). Wait for specified amount of time and schedule a job or throw an exception. /// Similar to scheduleOrThrowOnError(...). Wait for specified amount of time and schedule a job or throw an exception.
void scheduleOrThrow(Job job, int priority = 0, uint64_t wait_microseconds = 0, bool propagate_opentelemetry_tracing_context = true); void scheduleOrThrow(Job job, ssize_t priority = 0, uint64_t wait_microseconds = 0, bool propagate_opentelemetry_tracing_context = true);
/// Wait for all currently active jobs to be done. /// Wait for all currently active jobs to be done.
/// You may call schedule and wait many times in arbitrary order. /// You may call schedule and wait many times in arbitrary order.
@ -96,10 +96,10 @@ private:
struct JobWithPriority struct JobWithPriority
{ {
Job job; Job job;
int priority; ssize_t priority;
DB::OpenTelemetry::TracingContextOnThread thread_trace_context; DB::OpenTelemetry::TracingContextOnThread thread_trace_context;
JobWithPriority(Job job_, int priority_, const DB::OpenTelemetry::TracingContextOnThread& thread_trace_context_) JobWithPriority(Job job_, ssize_t priority_, const DB::OpenTelemetry::TracingContextOnThread& thread_trace_context_)
: job(job_), priority(priority_), thread_trace_context(thread_trace_context_) {} : job(job_), priority(priority_), thread_trace_context(thread_trace_context_) {}
bool operator< (const JobWithPriority & rhs) const bool operator< (const JobWithPriority & rhs) const
@ -113,7 +113,7 @@ private:
std::exception_ptr first_exception; std::exception_ptr first_exception;
template <typename ReturnType> template <typename ReturnType>
ReturnType scheduleImpl(Job job, int priority, std::optional<uint64_t> wait_microseconds, bool propagate_opentelemetry_tracing_context = true); ReturnType scheduleImpl(Job job, ssize_t priority, std::optional<uint64_t> wait_microseconds, bool propagate_opentelemetry_tracing_context = true);
void worker(typename std::list<Thread>::iterator thread_it); void worker(typename std::list<Thread>::iterator thread_it);

View File

@ -121,7 +121,7 @@ TasksStatsCounters::TasksStatsCounters(const UInt64 tid, const MetricsProvider p
stats_getter = [metrics_provider = std::make_shared<TaskStatsInfoGetter>(), tid]() stats_getter = [metrics_provider = std::make_shared<TaskStatsInfoGetter>(), tid]()
{ {
::taskstats result{}; ::taskstats result{};
metrics_provider->getStat(result, tid); metrics_provider->getStat(result, static_cast<pid_t>(tid));
return result; return result;
}; };
break; break;
@ -526,7 +526,7 @@ void PerfEventsCounters::finalizeProfileEvents(ProfileEvents::Counters & profile
continue; continue;
constexpr ssize_t bytes_to_read = sizeof(current_values[0]); constexpr ssize_t bytes_to_read = sizeof(current_values[0]);
const int bytes_read = read(fd, &current_values[i], bytes_to_read); const ssize_t bytes_read = read(fd, &current_values[i], bytes_to_read);
if (bytes_read != bytes_to_read) if (bytes_read != bytes_to_read)
{ {

View File

@ -99,7 +99,10 @@ requires (sizeof(CharT) == 1)
size_t convertCodePointToUTF8(int code_point, CharT * out_bytes, size_t out_length) size_t convertCodePointToUTF8(int code_point, CharT * out_bytes, size_t out_length)
{ {
static const Poco::UTF8Encoding utf8; static const Poco::UTF8Encoding utf8;
int res = utf8.convert(code_point, reinterpret_cast<uint8_t *>(out_bytes), out_length); int res = utf8.convert(
code_point,
reinterpret_cast<uint8_t *>(out_bytes),
static_cast<int>(out_length));
assert(res >= 0); assert(res >= 0);
return res; return res;
} }
@ -109,7 +112,9 @@ requires (sizeof(CharT) == 1)
std::optional<uint32_t> convertUTF8ToCodePoint(const CharT * in_bytes, size_t in_length) std::optional<uint32_t> convertUTF8ToCodePoint(const CharT * in_bytes, size_t in_length)
{ {
static const Poco::UTF8Encoding utf8; static const Poco::UTF8Encoding utf8;
int res = utf8.queryConvert(reinterpret_cast<const uint8_t *>(in_bytes), in_length); int res = utf8.queryConvert(
reinterpret_cast<const uint8_t *>(in_bytes),
static_cast<int>(in_length));
if (res >= 0) if (res >= 0)
return res; return res;

View File

@ -404,7 +404,8 @@ public:
/// And also adding from the end guarantees that we will find first occurrence because we will lookup bigger offsets first. /// And also adding from the end guarantees that we will find first occurrence because we will lookup bigger offsets first.
for (auto i = static_cast<ssize_t>(needle_size - sizeof(VolnitskyTraits::Ngram)); i >= 0; --i) for (auto i = static_cast<ssize_t>(needle_size - sizeof(VolnitskyTraits::Ngram)); i >= 0; --i)
{ {
bool ok = VolnitskyTraits::putNGram<CaseSensitive, ASCII>(needle + i, i + 1, needle, needle_size, callback); bool ok = VolnitskyTraits::putNGram<CaseSensitive, ASCII>(
needle + i, static_cast<int>(i + 1), needle, needle_size, callback);
/** `putNGramUTF8CaseInsensitive` does not work if characters with lower and upper cases /** `putNGramUTF8CaseInsensitive` does not work if characters with lower and upper cases
* are represented by different number of bytes or code points. * are represented by different number of bytes or code points.

View File

@ -218,7 +218,7 @@ std::pair<ResponsePtr, Undo> TestKeeperCreateRequest::process(TestKeeper::Contai
created_node.stat.ctime = std::chrono::system_clock::now().time_since_epoch() / std::chrono::milliseconds(1); created_node.stat.ctime = std::chrono::system_clock::now().time_since_epoch() / std::chrono::milliseconds(1);
created_node.stat.mtime = created_node.stat.ctime; created_node.stat.mtime = created_node.stat.ctime;
created_node.stat.numChildren = 0; created_node.stat.numChildren = 0;
created_node.stat.dataLength = data.length(); created_node.stat.dataLength = static_cast<int>(data.length());
created_node.data = data; created_node.data = data;
created_node.is_ephemeral = is_ephemeral; created_node.is_ephemeral = is_ephemeral;
created_node.is_sequental = is_sequential; created_node.is_sequental = is_sequential;

View File

@ -45,7 +45,7 @@ struct ShuffleHost
{ {
String host; String host;
Int64 priority = 0; Int64 priority = 0;
UInt32 random = 0; UInt64 random = 0;
void randomize() void randomize()
{ {

View File

@ -724,7 +724,7 @@ void ZooKeeperRequest::createLogElements(LogElements & elems) const
elem.has_watch = has_watch; elem.has_watch = has_watch;
elem.op_num = static_cast<uint32_t>(getOpNum()); elem.op_num = static_cast<uint32_t>(getOpNum());
elem.path = getPath(); elem.path = getPath();
elem.request_idx = elems.size() - 1; elem.request_idx = static_cast<uint32_t>(elems.size()) - 1;
} }
@ -762,7 +762,7 @@ void ZooKeeperCheckRequest::createLogElements(LogElements & elems) const
void ZooKeeperMultiRequest::createLogElements(LogElements & elems) const void ZooKeeperMultiRequest::createLogElements(LogElements & elems) const
{ {
ZooKeeperRequest::createLogElements(elems); ZooKeeperRequest::createLogElements(elems);
elems.back().requests_size = requests.size(); elems.back().requests_size = static_cast<uint32_t>(requests.size());
for (const auto & request : requests) for (const auto & request : requests)
{ {
auto & req = dynamic_cast<ZooKeeperRequest &>(*request); auto & req = dynamic_cast<ZooKeeperRequest &>(*request);

View File

@ -546,7 +546,7 @@ void ZooKeeper::sendAuth(const String & scheme, const String & data)
if (read_xid != AUTH_XID) if (read_xid != AUTH_XID)
throw Exception(Error::ZMARSHALLINGERROR, "Unexpected event received in reply to auth request: {}", read_xid); throw Exception(Error::ZMARSHALLINGERROR, "Unexpected event received in reply to auth request: {}", read_xid);
int32_t actual_length = in->count() - count_before_event; int32_t actual_length = static_cast<int32_t>(in->count() - count_before_event);
if (length != actual_length) if (length != actual_length)
throw Exception(Error::ZMARSHALLINGERROR, "Response length doesn't match. Expected: {}, actual: {}", length, actual_length); throw Exception(Error::ZMARSHALLINGERROR, "Response length doesn't match. Expected: {}, actual: {}", length, actual_length);
@ -821,7 +821,7 @@ void ZooKeeper::receiveEvent()
} }
} }
int32_t actual_length = in->count() - count_before_event; int32_t actual_length = static_cast<int32_t>(in->count() - count_before_event);
if (length != actual_length) if (length != actual_length)
throw Exception(Error::ZMARSHALLINGERROR, "Response length doesn't match. Expected: {}, actual: {}", length, actual_length); throw Exception(Error::ZMARSHALLINGERROR, "Response length doesn't match. Expected: {}, actual: {}", length, actual_length);

View File

@ -492,7 +492,7 @@ int main(int argc, char ** argv)
watch.restart(); watch.restart();
for (size_t i = 0; i < MapTwoLevel::NUM_BUCKETS; ++i) for (unsigned i = 0; i < MapTwoLevel::NUM_BUCKETS; ++i)
pool.scheduleOrThrowOnError([&] { merge2(maps.data(), num_threads, i); }); pool.scheduleOrThrowOnError([&] { merge2(maps.data(), num_threads, i); });
pool.wait(); pool.wait();
@ -545,7 +545,7 @@ int main(int argc, char ** argv)
watch.restart(); watch.restart();
for (size_t i = 0; i < MapTwoLevel::NUM_BUCKETS; ++i) for (unsigned i = 0; i < MapTwoLevel::NUM_BUCKETS; ++i)
pool.scheduleOrThrowOnError([&] { merge2(maps.data(), num_threads, i); }); pool.scheduleOrThrowOnError([&] { merge2(maps.data(), num_threads, i); });
pool.wait(); pool.wait();

View File

@ -258,7 +258,7 @@ size_t getSizeFromFileDescriptor(int fd, const String & file_name)
return buf.st_size; return buf.st_size;
} }
int getINodeNumberFromPath(const String & path) Int64 getINodeNumberFromPath(const String & path)
{ {
struct stat file_stat; struct stat file_stat;
if (stat(path.data(), &file_stat)) if (stat(path.data(), &file_stat))

View File

@ -74,7 +74,7 @@ std::optional<size_t> tryGetSizeFromFilePath(const String & path);
/// Get inode number for a file path. /// Get inode number for a file path.
/// Will not work correctly if filesystem does not support inodes. /// Will not work correctly if filesystem does not support inodes.
int getINodeNumberFromPath(const String & path); Int64 getINodeNumberFromPath(const String & path);
} }

View File

@ -80,7 +80,7 @@ static void printInteger(char *& out, T value)
void formatIPv6(const unsigned char * src, char *& dst, uint8_t zeroed_tail_bytes_count) void formatIPv6(const unsigned char * src, char *& dst, uint8_t zeroed_tail_bytes_count)
{ {
struct { int base, len; } best{-1, 0}, cur{-1, 0}; struct { Int64 base, len; } best{-1, 0}, cur{-1, 0};
std::array<UInt16, IPV6_BINARY_LENGTH / sizeof(UInt16)> words{}; std::array<UInt16, IPV6_BINARY_LENGTH / sizeof(UInt16)> words{};
/** Preprocess: /** Preprocess:
@ -122,15 +122,19 @@ void formatIPv6(const unsigned char * src, char *& dst, uint8_t zeroed_tail_byte
best.base = -1; best.base = -1;
/// Format the result. /// Format the result.
for (const int i : collections::range(0, words.size())) for (const size_t i : collections::range(0, words.size()))
{ {
/// Are we inside the best run of 0x00's? /// Are we inside the best run of 0x00's?
if (best.base != -1 && i >= best.base && i < (best.base + best.len)) if (best.base != -1)
{ {
if (i == best.base) size_t best_base = static_cast<size_t>(best.base);
if (i >= best_base && i < (best_base + best.len))
{
if (i == best_base)
*dst++ = ':'; *dst++ = ':';
continue; continue;
} }
}
/// Are we following an initial run of 0x00s or any real hex? /// Are we following an initial run of 0x00s or any real hex?
if (i != 0) if (i != 0)

View File

@ -7,10 +7,10 @@
#include <filesystem> #include <filesystem>
int getCurrentProcessFDCount() Int64 getCurrentProcessFDCount()
{ {
namespace fs = std::filesystem; namespace fs = std::filesystem;
int result = -1; Int64 result = -1;
#if defined(OS_LINUX) || defined(OS_DARWIN) #if defined(OS_LINUX) || defined(OS_DARWIN)
using namespace DB; using namespace DB;

View File

@ -1,5 +1,7 @@
#pragma once #pragma once
#include <base/types.h>
/// Get current process file descriptor count /// Get current process file descriptor count
/// @return -1 os doesn't support "lsof" command or some error occurs. /// @return -1 os doesn't support "lsof" command or some error occurs.
int getCurrentProcessFDCount(); Int64 getCurrentProcessFDCount();

View File

@ -232,7 +232,7 @@ template <typename T>
constexpr inline T intExp10OfSize(int x) constexpr inline T intExp10OfSize(int x)
{ {
if constexpr (sizeof(T) <= 8) if constexpr (sizeof(T) <= 8)
return intExp10(x); return static_cast<T>(intExp10(x));
else if constexpr (sizeof(T) <= 16) else if constexpr (sizeof(T) <= 16)
return common::exp10_i128(x); return common::exp10_i128(x);
else else

View File

@ -124,7 +124,7 @@ double Value::readFloatText(const char * buf, size_t length) const
case 'E': case 'E':
{ {
++buf; ++buf;
Int32 exponent = readIntText(buf, end - buf); Int32 exponent = static_cast<Int32>(readIntText(buf, end - buf));
x *= preciseExp10(exponent); x *= preciseExp10(exponent);
if (negative) if (negative)
x = -x; x = -x;

View File

@ -44,9 +44,9 @@ public:
/** Получить значение по индексу. /** Получить значение по индексу.
* Здесь используется int, а не unsigned, чтобы не было неоднозначности с тем же методом, принимающим const char *. * Здесь используется int, а не unsigned, чтобы не было неоднозначности с тем же методом, принимающим const char *.
*/ */
Value operator[] (int n) const Value operator[] (size_t n) const
{ {
if (unlikely(static_cast<size_t>(n) >= res->getNumFields())) if (unlikely(n >= res->getNumFields()))
throw Exception("Index of column is out of range."); throw Exception("Index of column is out of range.");
return Value(row[n], lengths[n], res); return Value(row[n], lengths[n], res);
} }

View File

@ -242,8 +242,8 @@ template <> inline unsigned char Value::get<unsigned char >() cons
template <> inline char8_t Value::get<char8_t >() const { return getUInt(); } template <> inline char8_t Value::get<char8_t >() const { return getUInt(); }
template <> inline short Value::get<short >() const { return getInt(); } /// NOLINT template <> inline short Value::get<short >() const { return getInt(); } /// NOLINT
template <> inline unsigned short Value::get<unsigned short >() const { return getUInt(); } /// NOLINT template <> inline unsigned short Value::get<unsigned short >() const { return getUInt(); } /// NOLINT
template <> inline int Value::get<int >() const { return getInt(); } template <> inline int Value::get<int >() const { return static_cast<int>(getInt()); }
template <> inline unsigned int Value::get<unsigned int >() const { return getUInt(); } template <> inline unsigned int Value::get<unsigned int >() const { return static_cast<unsigned int>(getUInt()); }
template <> inline long Value::get<long >() const { return getInt(); } /// NOLINT template <> inline long Value::get<long >() const { return getInt(); } /// NOLINT
template <> inline unsigned long Value::get<unsigned long >() const { return getUInt(); } /// NOLINT template <> inline unsigned long Value::get<unsigned long >() const { return getUInt(); } /// NOLINT
template <> inline long long Value::get<long long >() const { return getInt(); } /// NOLINT template <> inline long long Value::get<long long >() const { return getInt(); } /// NOLINT

View File

@ -68,14 +68,14 @@ std::string makeRegexpPatternFromGlobs(const std::string & initial_str_with_glob
output_width = std::max(range_begin_width, range_end_width); output_width = std::max(range_begin_width, range_end_width);
if (leading_zeros) if (leading_zeros)
oss_for_replacing << std::setfill('0') << std::setw(output_width); oss_for_replacing << std::setfill('0') << std::setw(static_cast<int>(output_width));
oss_for_replacing << range_begin; oss_for_replacing << range_begin;
for (size_t i = range_begin + 1; i <= range_end; ++i) for (size_t i = range_begin + 1; i <= range_end; ++i)
{ {
oss_for_replacing << '|'; oss_for_replacing << '|';
if (leading_zeros) if (leading_zeros)
oss_for_replacing << std::setfill('0') << std::setw(output_width); oss_for_replacing << std::setfill('0') << std::setw(static_cast<int>(output_width));
oss_for_replacing << i; oss_for_replacing << i;
} }
} }

View File

@ -82,8 +82,8 @@ std::vector<String> parseRemoteDescription(const String & description, size_t l,
/// Either the numeric interval (8..10) or equivalent expression in brackets /// Either the numeric interval (8..10) or equivalent expression in brackets
if (description[i] == '{') if (description[i] == '{')
{ {
int cnt = 1; ssize_t cnt = 1;
int last_dot = -1; /// The rightmost pair of points, remember the index of the right of the two ssize_t last_dot = -1; /// The rightmost pair of points, remember the index of the right of the two
size_t m; size_t m;
std::vector<String> buffer; std::vector<String> buffer;
bool have_splitter = false; bool have_splitter = false;

22
src/Common/safe_cast.h Normal file
View File

@ -0,0 +1,22 @@
#pragma once
#include <Common/Exception.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
template <class To, class From>
To safe_cast(From from)
{
constexpr auto max = std::numeric_limits<To>::max();
if (from > max)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Overflow ({} > {})", from, max);
return static_cast<To>(from);
}
}

View File

@ -58,7 +58,7 @@ FailuresCount countFailures(const ::testing::TestResult & test_result)
const size_t count = test_result.total_part_count(); const size_t count = test_result.total_part_count();
for (size_t i = 0; i < count; ++i) for (size_t i = 0; i < count; ++i)
{ {
const auto & part = test_result.GetTestPartResult(i); const auto & part = test_result.GetTestPartResult(static_cast<int>(i));
if (part.nonfatally_failed()) if (part.nonfatally_failed())
{ {
++failures.non_fatal; ++failures.non_fatal;

View File

@ -216,27 +216,27 @@ TEST(HashTable, Erase)
using Cont = HashSet<int, DefaultHash<int>, HashTableGrowerWithPrecalculation<1>>; using Cont = HashSet<int, DefaultHash<int>, HashTableGrowerWithPrecalculation<1>>;
Cont cont; Cont cont;
for (size_t i = 0; i < 5000; ++i) for (int i = 0; i < 5000; ++i)
{ {
cont.insert(i); cont.insert(i);
} }
for (size_t i = 0; i < 2500; ++i) for (int i = 0; i < 2500; ++i)
{ {
cont.erase(i); cont.erase(i);
} }
for (size_t i = 5000; i < 10000; ++i) for (int i = 5000; i < 10000; ++i)
{ {
cont.insert(i); cont.insert(i);
} }
for (size_t i = 5000; i < 10000; ++i) for (int i = 5000; i < 10000; ++i)
{ {
cont.erase(i); cont.erase(i);
} }
for (size_t i = 2500; i < 5000; ++i) for (int i = 2500; i < 5000; ++i)
{ {
cont.erase(i); cont.erase(i);
} }

View File

@ -26,7 +26,7 @@ void testInsert(size_t elements_to_insert_size, size_t map_size)
std::vector<int> expected; std::vector<int> expected;
for (size_t i = 0; i < elements_to_insert_size; ++i) for (size_t i = 0; i < elements_to_insert_size; ++i)
map.insert(i, i); map.insert(static_cast<int>(i), static_cast<int>(i));
for (size_t i = elements_to_insert_size - map_size; i < elements_to_insert_size; ++i) for (size_t i = elements_to_insert_size - map_size; i < elements_to_insert_size; ++i)
expected.emplace_back(i); expected.emplace_back(i);

View File

@ -54,7 +54,7 @@ namespace DB
static int syscall_pidfd_open(pid_t pid) static int syscall_pidfd_open(pid_t pid)
{ {
return syscall(SYS_pidfd_open, pid, 0); return static_cast<int>(syscall(SYS_pidfd_open, pid, 0));
} }
static bool supportsPidFdOpen() static bool supportsPidFdOpen()
@ -170,7 +170,8 @@ bool waitForPid(pid_t pid, size_t timeout_in_seconds)
/// If timeout is positive try waitpid without block in loop until /// If timeout is positive try waitpid without block in loop until
/// process is normally terminated or waitpid return error /// process is normally terminated or waitpid return error
int timeout_in_ms = timeout_in_seconds * 1000; /// NOTE: timeout casted to int, since poll() accept int for timeout
int timeout_in_ms = static_cast<int>(timeout_in_seconds * 1000);
while (timeout_in_ms > 0) while (timeout_in_ms > 0)
{ {
int waitpid_res = HANDLE_EINTR(waitpid(pid, &status, WNOHANG)); int waitpid_res = HANDLE_EINTR(waitpid(pid, &status, WNOHANG));

View File

@ -279,7 +279,7 @@ static void readHeaderAndGetCodec(const char * compressed_buffer, size_t size_de
void CompressedReadBufferBase::decompressTo(char * to, size_t size_decompressed, size_t size_compressed_without_checksum) void CompressedReadBufferBase::decompressTo(char * to, size_t size_decompressed, size_t size_compressed_without_checksum)
{ {
readHeaderAndGetCodec(compressed_buffer, size_decompressed, codec, allow_different_codecs); readHeaderAndGetCodec(compressed_buffer, size_decompressed, codec, allow_different_codecs);
codec->decompress(compressed_buffer, size_compressed_without_checksum, to); codec->decompress(compressed_buffer, static_cast<UInt32>(size_compressed_without_checksum), to);
} }
void CompressedReadBufferBase::decompress(BufferBase::Buffer & to, size_t size_decompressed, size_t size_compressed_without_checksum) void CompressedReadBufferBase::decompress(BufferBase::Buffer & to, size_t size_decompressed, size_t size_compressed_without_checksum)
@ -300,7 +300,7 @@ void CompressedReadBufferBase::decompress(BufferBase::Buffer & to, size_t size_d
to = BufferBase::Buffer(compressed_buffer + header_size, compressed_buffer + size_compressed_without_checksum); to = BufferBase::Buffer(compressed_buffer + header_size, compressed_buffer + size_compressed_without_checksum);
} }
else else
codec->decompress(compressed_buffer, size_compressed_without_checksum, to.begin()); codec->decompress(compressed_buffer, static_cast<UInt32>(size_compressed_without_checksum), to.begin());
} }
void CompressedReadBufferBase::flushAsynchronousDecompressRequests() const void CompressedReadBufferBase::flushAsynchronousDecompressRequests() const

View File

@ -3,6 +3,7 @@
#include <base/types.h> #include <base/types.h>
#include <base/unaligned.h> #include <base/unaligned.h>
#include <base/defines.h>
#include <Compression/CompressionFactory.h> #include <Compression/CompressionFactory.h>
#include "CompressedWriteBuffer.h" #include "CompressedWriteBuffer.h"
@ -22,7 +23,8 @@ void CompressedWriteBuffer::nextImpl()
if (!offset()) if (!offset())
return; return;
size_t decompressed_size = offset(); chassert(offset() <= INT_MAX);
UInt32 decompressed_size = static_cast<UInt32>(offset());
UInt32 compressed_reserve_size = codec->getCompressedReserveSize(decompressed_size); UInt32 compressed_reserve_size = codec->getCompressedReserveSize(decompressed_size);
/** During compression we need buffer with capacity >= compressed_reserve_size + CHECKSUM_SIZE. /** During compression we need buffer with capacity >= compressed_reserve_size + CHECKSUM_SIZE.

View File

@ -353,7 +353,7 @@ UInt32 compressDataForType(const char * source, UInt32 source_size, char * dest)
writer.flush(); writer.flush();
return (dest - dest_start) + (writer.count() + 7) / 8; return static_cast<UInt32>((dest - dest_start) + (writer.count() + 7) / 8);
} }
template <typename ValueType> template <typename ValueType>
@ -414,7 +414,7 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest,
if (write_spec.data_bits != 0) if (write_spec.data_bits != 0)
{ {
const UInt8 sign = reader.readBit(); const UInt8 sign = reader.readBit();
double_delta = reader.readBits(write_spec.data_bits - 1) + 1; double_delta = static_cast<UnsignedDeltaType>(reader.readBits(write_spec.data_bits - 1) + 1);
if (sign) if (sign)
{ {
/// It's well defined for unsigned data types. /// It's well defined for unsigned data types.

View File

@ -7,6 +7,7 @@
#include <Compression/CompressionCodecEncrypted.h> #include <Compression/CompressionCodecEncrypted.h>
#include <Poco/Logger.h> #include <Poco/Logger.h>
#include <Common/logger_useful.h> #include <Common/logger_useful.h>
#include <Common/safe_cast.h>
// This depends on BoringSSL-specific API, notably <openssl/aead.h>. // This depends on BoringSSL-specific API, notably <openssl/aead.h>.
#if USE_SSL #if USE_SSL
@ -480,7 +481,8 @@ UInt32 CompressionCodecEncrypted::doCompressData(const char * source, UInt32 sou
if (out_len != source_size + tag_size) if (out_len != source_size + tag_size)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't encrypt data, length after encryption {} is wrong, expected {}", out_len, source_size + tag_size); throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't encrypt data, length after encryption {} is wrong, expected {}", out_len, source_size + tag_size);
return out_len + keyid_size + nonce_size; size_t out_size = out_len + keyid_size + nonce_size;
return safe_cast<UInt32>(out_size);
} }
void CompressionCodecEncrypted::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const void CompressionCodecEncrypted::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 uncompressed_size) const

View File

@ -453,9 +453,9 @@ UInt32 CompressionCodecFPC::doCompressData(const char * source, UInt32 source_si
switch (float_width) switch (float_width)
{ {
case sizeof(Float64): case sizeof(Float64):
return HEADER_SIZE + FPCOperation<UInt64>(destination, level).encode(src); return static_cast<UInt32>(HEADER_SIZE + FPCOperation<UInt64>(destination, level).encode(src));
case sizeof(Float32): case sizeof(Float32):
return HEADER_SIZE + FPCOperation<UInt32>(destination, level).encode(src); return static_cast<UInt32>(HEADER_SIZE + FPCOperation<UInt32>(destination, level).encode(src));
default: default:
break; break;
} }

View File

@ -259,7 +259,7 @@ UInt32 compressDataForType(const char * source, UInt32 source_size, char * dest,
writer.flush(); writer.flush();
return (dest - dest_start) + (writer.count() + 7) / 8; return static_cast<UInt32>((dest - dest_start) + (writer.count() + 7) / 8);
} }
template <typename T> template <typename T>
@ -320,7 +320,7 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest)
ErrorCodes::CANNOT_DECOMPRESS); ErrorCodes::CANNOT_DECOMPRESS);
} }
xored_data = reader.readBits(curr_xored_info.data_bits); xored_data = static_cast<T>(reader.readBits(curr_xored_info.data_bits));
xored_data <<= curr_xored_info.trailing_zero_bits; xored_data <<= curr_xored_info.trailing_zero_bits;
curr_value = prev_value ^ xored_data; curr_value = prev_value ^ xored_data;
} }

View File

@ -134,7 +134,7 @@ void registerCodecLZ4HC(CompressionCodecFactory & factory)
if (!literal) if (!literal)
throw Exception("LZ4HC codec argument must be integer", ErrorCodes::ILLEGAL_CODEC_PARAMETER); throw Exception("LZ4HC codec argument must be integer", ErrorCodes::ILLEGAL_CODEC_PARAMETER);
level = literal->value.safeGet<UInt64>(); level = static_cast<int>(literal->value.safeGet<UInt64>());
} }
return std::make_shared<CompressionCodecLZ4HC>(level); return std::make_shared<CompressionCodecLZ4HC>(level);

View File

@ -48,7 +48,7 @@ UInt32 CompressionCodecMultiple::getMaxCompressedDataSize(UInt32 uncompressed_si
compressed_size = codec->getCompressedReserveSize(compressed_size); compressed_size = codec->getCompressedReserveSize(compressed_size);
/// TotalCodecs ByteForEachCodec data /// TotalCodecs ByteForEachCodec data
return sizeof(UInt8) + codecs.size() + compressed_size; return static_cast<UInt32>(sizeof(UInt8) + codecs.size() + compressed_size);
} }
UInt32 CompressionCodecMultiple::doCompressData(const char * source, UInt32 source_size, char * dest) const UInt32 CompressionCodecMultiple::doCompressData(const char * source, UInt32 source_size, char * dest) const
@ -73,7 +73,7 @@ UInt32 CompressionCodecMultiple::doCompressData(const char * source, UInt32 sour
memcpy(&dest[1 + codecs.size()], uncompressed_buf.data(), source_size); memcpy(&dest[1 + codecs.size()], uncompressed_buf.data(), source_size);
return 1 + codecs.size() + source_size; return static_cast<UInt32>(1 + codecs.size() + source_size);
} }
void CompressionCodecMultiple::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 decompressed_size) const void CompressionCodecMultiple::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 decompressed_size) const

View File

@ -550,7 +550,7 @@ void decompressData(const char * src, UInt32 bytes_size, char * dst, UInt32 unco
UInt32 num_bits = getValuableBitsNumber(min, max); UInt32 num_bits = getValuableBitsNumber(min, max);
if (!num_bits) if (!num_bits)
{ {
T min_value = min; T min_value = static_cast<T>(min);
for (UInt32 i = 0; i < num_elements; ++i, dst += sizeof(T)) for (UInt32 i = 0; i < num_elements; ++i, dst += sizeof(T))
unalignedStore<T>(dst, min_value); unalignedStore<T>(dst, min_value);
return; return;
@ -572,14 +572,14 @@ void decompressData(const char * src, UInt32 bytes_size, char * dst, UInt32 unco
T upper_max [[maybe_unused]] = 0; T upper_max [[maybe_unused]] = 0;
T sign_bit [[maybe_unused]] = 0; T sign_bit [[maybe_unused]] = 0;
if (num_bits < 64) if (num_bits < 64)
upper_min = static_cast<UInt64>(min) >> num_bits << num_bits; upper_min = static_cast<T>(static_cast<UInt64>(min) >> num_bits << num_bits);
if constexpr (is_signed_v<T>) if constexpr (is_signed_v<T>)
{ {
if (min < 0 && max >= 0 && num_bits < 64) if (min < 0 && max >= 0 && num_bits < 64)
{ {
sign_bit = 1ull << (num_bits - 1); sign_bit = static_cast<T>(1ull << (num_bits - 1));
upper_max = static_cast<UInt64>(max) >> num_bits << num_bits; upper_max = static_cast<T>(static_cast<UInt64>(max) >> num_bits << num_bits);
} }
} }

View File

@ -65,7 +65,7 @@ void CompressionCodecZSTD::updateHash(SipHash & hash) const
UInt32 CompressionCodecZSTD::getMaxCompressedDataSize(UInt32 uncompressed_size) const UInt32 CompressionCodecZSTD::getMaxCompressedDataSize(UInt32 uncompressed_size) const
{ {
return ZSTD_compressBound(uncompressed_size); return static_cast<UInt32>(ZSTD_compressBound(uncompressed_size));
} }
@ -84,7 +84,7 @@ UInt32 CompressionCodecZSTD::doCompressData(const char * source, UInt32 source_s
if (ZSTD_isError(compressed_size)) if (ZSTD_isError(compressed_size))
throw Exception("Cannot compress block with ZSTD: " + std::string(ZSTD_getErrorName(compressed_size)), ErrorCodes::CANNOT_COMPRESS); throw Exception("Cannot compress block with ZSTD: " + std::string(ZSTD_getErrorName(compressed_size)), ErrorCodes::CANNOT_COMPRESS);
return compressed_size; return static_cast<UInt32>(compressed_size);
} }
@ -124,18 +124,20 @@ void registerCodecZSTD(CompressionCodecFactory & factory)
if (!literal) if (!literal)
throw Exception("ZSTD codec argument must be integer", ErrorCodes::ILLEGAL_CODEC_PARAMETER); throw Exception("ZSTD codec argument must be integer", ErrorCodes::ILLEGAL_CODEC_PARAMETER);
level = literal->value.safeGet<UInt64>(); level = static_cast<int>(literal->value.safeGet<UInt64>());
if (level > ZSTD_maxCLevel()) if (level > ZSTD_maxCLevel())
throw Exception( {
"ZSTD codec can't have level more than " + toString(ZSTD_maxCLevel()) + ", given " + toString(level), throw Exception(ErrorCodes::ILLEGAL_CODEC_PARAMETER,
ErrorCodes::ILLEGAL_CODEC_PARAMETER); "ZSTD codec can't have level more than {}, given {}",
ZSTD_maxCLevel(), level);
}
if (arguments->children.size() > 1) if (arguments->children.size() > 1)
{ {
const auto * window_literal = children[1]->as<ASTLiteral>(); const auto * window_literal = children[1]->as<ASTLiteral>();
if (!window_literal) if (!window_literal)
throw Exception("ZSTD codec second argument must be integer", ErrorCodes::ILLEGAL_CODEC_PARAMETER); throw Exception("ZSTD codec second argument must be integer", ErrorCodes::ILLEGAL_CODEC_PARAMETER);
const int window_log = window_literal->value.safeGet<UInt64>(); const int window_log = static_cast<int>(window_literal->value.safeGet<UInt64>());
ZSTD_bounds window_log_bounds = ZSTD_cParam_getBounds(ZSTD_c_windowLog); ZSTD_bounds window_log_bounds = ZSTD_cParam_getBounds(ZSTD_c_windowLog);
if (ZSTD_isError(window_log_bounds.error)) if (ZSTD_isError(window_log_bounds.error))

View File

@ -391,7 +391,7 @@ CodecTestSequence generateSeq(Generator gen, const char* gen_name, B Begin = 0,
for (auto i = Begin; i < End; i += direction) for (auto i = Begin; i < End; i += direction)
{ {
const T v = gen(static_cast<T>(i)); const T v = static_cast<T>(gen(i));
unalignedStoreLE<T>(write_pos, v); unalignedStoreLE<T>(write_pos, v);
write_pos += sizeof(v); write_pos += sizeof(v);
@ -464,13 +464,15 @@ void testTranscoding(Timer & timer, ICompressionCodec & codec, const CodecTestSe
{ {
const auto & source_data = test_sequence.serialized_data; const auto & source_data = test_sequence.serialized_data;
const UInt32 encoded_max_size = codec.getCompressedReserveSize(source_data.size()); const UInt32 encoded_max_size = codec.getCompressedReserveSize(
static_cast<UInt32>(source_data.size()));
PODArray<char> encoded(encoded_max_size); PODArray<char> encoded(encoded_max_size);
timer.start(); timer.start();
assert(source_data.data() != nullptr); // Codec assumes that source buffer is not null. assert(source_data.data() != nullptr); // Codec assumes that source buffer is not null.
const UInt32 encoded_size = codec.compress(source_data.data(), source_data.size(), encoded.data()); const UInt32 encoded_size = codec.compress(
source_data.data(), static_cast<UInt32>(source_data.size()), encoded.data());
timer.report("encoding"); timer.report("encoding");
encoded.resize(encoded_size); encoded.resize(encoded_size);
@ -478,7 +480,8 @@ void testTranscoding(Timer & timer, ICompressionCodec & codec, const CodecTestSe
PODArray<char> decoded(source_data.size()); PODArray<char> decoded(source_data.size());
timer.start(); timer.start();
const UInt32 decoded_size = codec.decompress(encoded.data(), encoded.size(), decoded.data()); const UInt32 decoded_size = codec.decompress(
encoded.data(), static_cast<UInt32>(encoded.size()), decoded.data());
timer.report("decoding"); timer.report("decoding");
decoded.resize(decoded_size); decoded.resize(decoded_size);
@ -542,10 +545,12 @@ TEST_P(CodecTestCompatibility, Encoding)
const auto & source_data = data_sequence.serialized_data; const auto & source_data = data_sequence.serialized_data;
// Just encode the data with codec // Just encode the data with codec
const UInt32 encoded_max_size = codec->getCompressedReserveSize(source_data.size()); const UInt32 encoded_max_size = codec->getCompressedReserveSize(
static_cast<UInt32>(source_data.size()));
PODArray<char> encoded(encoded_max_size); PODArray<char> encoded(encoded_max_size);
const UInt32 encoded_size = codec->compress(source_data.data(), source_data.size(), encoded.data()); const UInt32 encoded_size = codec->compress(
source_data.data(), static_cast<UInt32>(source_data.size()), encoded.data());
encoded.resize(encoded_size); encoded.resize(encoded_size);
SCOPED_TRACE(::testing::Message("encoded: ") << AsHexString(encoded)); SCOPED_TRACE(::testing::Message("encoded: ") << AsHexString(encoded));
@ -560,7 +565,8 @@ TEST_P(CodecTestCompatibility, Decoding)
const auto codec = makeCodec(codec_spec.codec_statement, expected.data_type); const auto codec = makeCodec(codec_spec.codec_statement, expected.data_type);
PODArray<char> decoded(expected.serialized_data.size()); PODArray<char> decoded(expected.serialized_data.size());
const UInt32 decoded_size = codec->decompress(encoded_data.c_str(), encoded_data.size(), decoded.data()); const UInt32 decoded_size = codec->decompress(
encoded_data.c_str(), static_cast<UInt32>(encoded_data.size()), decoded.data());
decoded.resize(decoded_size); decoded.resize(decoded_size);
ASSERT_TRUE(EqualByteContainers(expected.data_type->getSizeOfValueInMemory(), expected.serialized_data, decoded)); ASSERT_TRUE(EqualByteContainers(expected.data_type->getSizeOfValueInMemory(), expected.serialized_data, decoded));

View File

@ -284,8 +284,9 @@ void KeeperServer::launchRaftServer(const Poco::Util::AbstractConfiguration & co
params.client_req_timeout_ params.client_req_timeout_
= getValueOrMaxInt32AndLogWarning(coordination_settings->operation_timeout_ms.totalMilliseconds(), "operation_timeout_ms", log); = getValueOrMaxInt32AndLogWarning(coordination_settings->operation_timeout_ms.totalMilliseconds(), "operation_timeout_ms", log);
params.auto_forwarding_ = coordination_settings->auto_forwarding; params.auto_forwarding_ = coordination_settings->auto_forwarding;
params.auto_forwarding_req_timeout_ params.auto_forwarding_req_timeout_ = std::max<int32_t>(
= std::max<uint64_t>(coordination_settings->operation_timeout_ms.totalMilliseconds() * 2, std::numeric_limits<int32_t>::max()); static_cast<int32_t>(coordination_settings->operation_timeout_ms.totalMilliseconds() * 2),
std::numeric_limits<int32_t>::max());
params.auto_forwarding_req_timeout_ params.auto_forwarding_req_timeout_
= getValueOrMaxInt32AndLogWarning(coordination_settings->operation_timeout_ms.totalMilliseconds() * 2, "operation_timeout_ms", log); = getValueOrMaxInt32AndLogWarning(coordination_settings->operation_timeout_ms.totalMilliseconds() * 2, "operation_timeout_ms", log);
params.max_append_size_ params.max_append_size_

View File

@ -922,7 +922,7 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr
stat.version = 0; stat.version = 0;
stat.aversion = 0; stat.aversion = 0;
stat.cversion = 0; stat.cversion = 0;
stat.dataLength = request.data.length(); stat.dataLength = static_cast<UInt32>(request.data.length());
stat.ephemeralOwner = request.is_ephemeral ? session_id : 0; stat.ephemeralOwner = request.is_ephemeral ? session_id : 0;
new_deltas.emplace_back( new_deltas.emplace_back(
@ -1222,7 +1222,7 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce
value.stat.version++; value.stat.version++;
value.stat.mzxid = zxid; value.stat.mzxid = zxid;
value.stat.mtime = time; value.stat.mtime = time;
value.stat.dataLength = data.length(); value.stat.dataLength = static_cast<UInt32>(data.length());
value.setData(data); value.setData(data);
}, },
request.version}); request.version});

View File

@ -71,10 +71,10 @@ void SummingStateMachine::createSnapshotInternal(nuraft::snapshot & s)
snapshots[s.get_last_log_idx()] = ctx; snapshots[s.get_last_log_idx()] = ctx;
// Maintain last 3 snapshots only. // Maintain last 3 snapshots only.
int num = snapshots.size(); ssize_t num = snapshots.size();
auto entry = snapshots.begin(); auto entry = snapshots.begin();
for (int ii = 0; ii < num - MAX_SNAPSHOTS; ++ii) for (ssize_t ii = 0; ii < num - MAX_SNAPSHOTS; ++ii)
{ {
if (entry == snapshots.end()) if (entry == snapshots.end())
break; break;

View File

@ -119,7 +119,7 @@ int64_t deserializeStorageData(KeeperStorage & storage, ReadBuffer & in, Poco::L
Coordination::read(node.stat.pzxid, in); Coordination::read(node.stat.pzxid, in);
if (!path.empty()) if (!path.empty())
{ {
node.stat.dataLength = node.getData().length(); node.stat.dataLength = static_cast<UInt32>(node.getData().length());
node.seq_num = node.stat.cversion; node.seq_num = node.stat.cversion;
storage.container.insertOrReplace(path, node); storage.container.insertOrReplace(path, node);

Some files were not shown because too many files have changed in this diff Show More