mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
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:
parent
19715f1542
commit
4e76629aaf
@ -151,7 +151,7 @@ public:
|
||||
{
|
||||
size_t dot_pos = path.rfind('.');
|
||||
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
|
||||
fd = ::mkstemp(path.data());
|
||||
|
||||
@ -408,7 +408,7 @@ ReplxxLineReader::ReplxxLineReader(
|
||||
// In a simplest case use simple comment.
|
||||
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);
|
||||
};
|
||||
@ -480,7 +480,7 @@ void ReplxxLineReader::openEditor()
|
||||
if (executeCommand(argv) == 0)
|
||||
{
|
||||
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)
|
||||
@ -526,7 +526,7 @@ void ReplxxLineReader::openInteractiveHistorySearch()
|
||||
{
|
||||
std::string new_query = readFile(output_file.getPath());
|
||||
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)
|
||||
|
@ -265,7 +265,7 @@ inline size_t hashLessThan16(const char * data, size_t size)
|
||||
|
||||
struct CRC32Hash
|
||||
{
|
||||
size_t operator() (StringRef x) const
|
||||
unsigned operator() (StringRef x) const
|
||||
{
|
||||
const char * pos = x.data;
|
||||
size_t size = x.size;
|
||||
@ -275,22 +275,22 @@ struct CRC32Hash
|
||||
|
||||
if (size < 8)
|
||||
{
|
||||
return hashLessThan8(x.data, x.size);
|
||||
return static_cast<unsigned>(hashLessThan8(x.data, x.size));
|
||||
}
|
||||
|
||||
const char * end = pos + size;
|
||||
size_t res = -1ULL;
|
||||
unsigned res = -1U;
|
||||
|
||||
do
|
||||
{
|
||||
UInt64 word = unalignedLoad<UInt64>(pos);
|
||||
res = CRC_INT(res, word);
|
||||
res = static_cast<unsigned>(CRC_INT(res, word));
|
||||
|
||||
pos += 8;
|
||||
} while (pos + 8 < end);
|
||||
|
||||
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;
|
||||
}
|
||||
|
@ -122,7 +122,7 @@ QuotientAndRemainder<N> static inline split(UnsignedOfSize<N> value)
|
||||
constexpr DivisionBy10PowN<N> division;
|
||||
|
||||
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};
|
||||
}
|
||||
|
@ -1108,15 +1108,21 @@ void Client::processConfig()
|
||||
else
|
||||
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";
|
||||
|
||||
/// Setting value from cmd arg overrides one from config
|
||||
if (global_context->getSettingsRef().max_insert_block_size.changed)
|
||||
{
|
||||
insert_format_max_block_size = global_context->getSettingsRef().max_insert_block_size;
|
||||
}
|
||||
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();
|
||||
client_info.setInitialQuery();
|
||||
|
@ -47,8 +47,8 @@ public:
|
||||
WrappingUInt32 epoch;
|
||||
WrappingUInt32 counter;
|
||||
explicit Zxid(UInt64 _zxid)
|
||||
: epoch(_zxid >> 32)
|
||||
, counter(_zxid)
|
||||
: epoch(static_cast<UInt32>(_zxid >> 32))
|
||||
, counter(static_cast<UInt32>(_zxid))
|
||||
{}
|
||||
|
||||
bool operator<=(const Zxid & other) const
|
||||
|
@ -893,7 +893,7 @@ namespace
|
||||
if (fs::exists(pid_file))
|
||||
{
|
||||
ReadBufferFromFile in(pid_file.string());
|
||||
UInt64 pid;
|
||||
Int32 pid;
|
||||
if (tryReadIntText(pid, in))
|
||||
{
|
||||
fmt::print("{} file exists and contains pid = {}.\n", pid_file.string(), pid);
|
||||
@ -982,9 +982,9 @@ namespace
|
||||
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))
|
||||
{
|
||||
@ -1057,7 +1057,7 @@ namespace
|
||||
if (force && do_not_kill)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Specified flags are incompatible");
|
||||
|
||||
UInt64 pid = isRunning(pid_file);
|
||||
int pid = isRunning(pid_file);
|
||||
|
||||
if (!pid)
|
||||
return 0;
|
||||
|
@ -68,12 +68,12 @@ namespace ErrorCodes
|
||||
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 int sleep_one_ms = 100;
|
||||
int sleep_current_ms = 0;
|
||||
int current_connections = 0;
|
||||
const size_t sleep_max_ms = 1000 * seconds_to_wait;
|
||||
const size_t sleep_one_ms = 100;
|
||||
size_t sleep_current_ms = 0;
|
||||
size_t current_connections = 0;
|
||||
for (;;)
|
||||
{
|
||||
current_connections = 0;
|
||||
@ -441,7 +441,7 @@ int Keeper::main(const std::vector<std::string> & /*args*/)
|
||||
main_config_reloader.reset();
|
||||
|
||||
LOG_DEBUG(log, "Waiting for current connections to Keeper to finish.");
|
||||
int current_connections = 0;
|
||||
size_t current_connections = 0;
|
||||
for (auto & server : *servers)
|
||||
{
|
||||
server.stop();
|
||||
|
@ -546,9 +546,14 @@ void LocalServer::processConfig()
|
||||
|
||||
/// Setting value from cmd arg overrides one from config
|
||||
if (global_context->getSettingsRef().max_insert_block_size.changed)
|
||||
{
|
||||
insert_format_max_block_size = global_context->getSettingsRef().max_insert_block_size;
|
||||
}
|
||||
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).
|
||||
global_context->setExternalAuthenticatorsConfig(config());
|
||||
|
@ -279,7 +279,7 @@ Float transformFloatMantissa(Float x, UInt64 seed)
|
||||
constexpr size_t mantissa_num_bits = std::is_same_v<Float, Float32> ? 23 : 52;
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
@ -511,13 +511,13 @@ public:
|
||||
for (size_t i = 0; i < size; ++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 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_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;
|
||||
|
||||
src_prev_value = src_datetime;
|
||||
|
@ -183,7 +183,10 @@ void ODBCColumnsInfoHandler::handleRequest(HTTPServerRequest & request, HTTPServ
|
||||
if (columns.empty())
|
||||
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
|
||||
{
|
||||
writeStringBinary(columns.toString(), out);
|
||||
|
@ -139,7 +139,7 @@ void ODBCSource::insertValue(
|
||||
readDateTimeText(time, in, assert_cast<const DataTypeDateTime *>(data_type.get())->getTimeZone());
|
||||
if (time < 0)
|
||||
time = 0;
|
||||
assert_cast<ColumnUInt32 &>(column).insertValue(time);
|
||||
assert_cast<ColumnUInt32 &>(column).insertValue(static_cast<UInt32>(time));
|
||||
break;
|
||||
}
|
||||
case ValueType::vtDateTime64:
|
||||
|
@ -228,12 +228,12 @@ catch (...)
|
||||
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 int sleep_one_ms = 100;
|
||||
int sleep_current_ms = 0;
|
||||
int current_connections = 0;
|
||||
const size_t sleep_max_ms = 1000 * seconds_to_wait;
|
||||
const size_t sleep_one_ms = 100;
|
||||
size_t sleep_current_ms = 0;
|
||||
size_t current_connections = 0;
|
||||
for (;;)
|
||||
{
|
||||
current_connections = 0;
|
||||
@ -933,7 +933,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
else
|
||||
{
|
||||
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);
|
||||
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());
|
||||
@ -1507,7 +1507,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
if (!servers_to_start_before_tables.empty())
|
||||
{
|
||||
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)
|
||||
{
|
||||
server.stop();
|
||||
@ -1793,7 +1793,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
|
||||
is_cancelled = true;
|
||||
|
||||
int current_connections = 0;
|
||||
size_t current_connections = 0;
|
||||
{
|
||||
std::lock_guard lock(servers_lock);
|
||||
for (auto & server : servers)
|
||||
|
@ -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)
|
||||
{
|
||||
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)
|
||||
|
@ -254,7 +254,7 @@ bool LDAPClient::openConnection()
|
||||
#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));
|
||||
}
|
||||
|
||||
|
@ -45,7 +45,8 @@ public:
|
||||
|
||||
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
|
||||
{
|
||||
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()));
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -426,7 +426,7 @@ public:
|
||||
return 0;
|
||||
|
||||
if (isSmall())
|
||||
return small.find(x) != small.end();
|
||||
return small.find(static_cast<T>(x)) != small.end();
|
||||
else
|
||||
return rb->contains(static_cast<Value>(x));
|
||||
}
|
||||
|
@ -136,8 +136,8 @@ private:
|
||||
|
||||
for (size_t i = 0; i <= size; ++i)
|
||||
{
|
||||
previous[i] = i - 1;
|
||||
next[i] = i + 1;
|
||||
previous[i] = static_cast<UInt32>(i - 1);
|
||||
next[i] = static_cast<UInt32>(i + 1);
|
||||
}
|
||||
|
||||
next[size] = 0;
|
||||
@ -157,7 +157,7 @@ private:
|
||||
auto quality = [&](UInt32 i) { return points[next[i]].mean - points[i].mean; };
|
||||
|
||||
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())
|
||||
{
|
||||
@ -217,7 +217,7 @@ private:
|
||||
points[left] = points[right];
|
||||
}
|
||||
}
|
||||
size = left + 1;
|
||||
size = static_cast<UInt32>(left + 1);
|
||||
}
|
||||
|
||||
public:
|
||||
|
@ -540,7 +540,7 @@ public:
|
||||
/// Assuming to.has()
|
||||
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)
|
||||
{
|
||||
@ -555,7 +555,7 @@ public:
|
||||
if (capacity < value_size)
|
||||
{
|
||||
/// Don't free large_data here.
|
||||
capacity = roundUpToPowerOfTwoOrZero(value_size);
|
||||
capacity = static_cast<Int32>(roundUpToPowerOfTwoOrZero(value_size));
|
||||
large_data = arena->alloc(capacity);
|
||||
}
|
||||
|
||||
|
@ -44,7 +44,7 @@ struct AggregateFunctionRetentionData
|
||||
|
||||
void serialize(WriteBuffer & buf) const
|
||||
{
|
||||
UInt32 event_value = events.to_ulong();
|
||||
UInt32 event_value = static_cast<UInt32>(events.to_ulong());
|
||||
writeBinary(event_value, buf);
|
||||
}
|
||||
|
||||
|
@ -272,7 +272,7 @@ private:
|
||||
|
||||
actions.emplace_back(PatternActionType::SpecificEvent, event_number - 1);
|
||||
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();
|
||||
conditions_in_pattern.set(event_number - 1);
|
||||
}
|
||||
|
@ -226,7 +226,7 @@ public:
|
||||
for (UInt8 i = 0; i < events_size; ++i)
|
||||
if (assert_cast<const ColumnVector<UInt8> *>(columns[min_required_args + i])->getData()[row_num])
|
||||
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];
|
||||
|
||||
@ -365,7 +365,7 @@ public:
|
||||
/// The first matched event is 0x00000001, the second one is 0x00000002, the third one is 0x00000004, and so on.
|
||||
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)
|
||||
return unmatched_idx;
|
||||
|
@ -165,7 +165,7 @@ private:
|
||||
{
|
||||
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();
|
||||
value += getBar(found ? std::round(((it->getMapped() - min_y) / diff_y) * 7) + 1 : 0.0);
|
||||
}
|
||||
@ -173,7 +173,7 @@ private:
|
||||
else
|
||||
{
|
||||
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
|
||||
@ -202,7 +202,7 @@ private:
|
||||
if (i == bound.first) // is bound
|
||||
{
|
||||
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());
|
||||
if (found && proportion > 0)
|
||||
new_y = new_y.value_or(0) + it->getMapped() * proportion;
|
||||
@ -229,7 +229,7 @@ private:
|
||||
}
|
||||
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())
|
||||
new_y = new_y.value_or(0) + it->getMapped();
|
||||
}
|
||||
@ -267,8 +267,8 @@ public:
|
||||
if (params.size() == 3)
|
||||
{
|
||||
specified_min_max_x = true;
|
||||
min_x = params.at(1).safeGet<X>();
|
||||
max_x = params.at(2).safeGet<X>();
|
||||
min_x = static_cast<X>(params.at(1).safeGet<X>());
|
||||
max_x = static_cast<X>(params.at(2).safeGet<X>());
|
||||
}
|
||||
else
|
||||
{
|
||||
|
@ -175,8 +175,9 @@ struct OneAdder
|
||||
{
|
||||
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);
|
||||
data.set.insert(AggregateFunctionUniqTraits<T>::hash(value));
|
||||
data.set.insert(static_cast<ValueType>(AggregateFunctionUniqTraits<T>::hash(value)));
|
||||
}
|
||||
else
|
||||
{
|
||||
|
@ -43,7 +43,7 @@ namespace detail
|
||||
{
|
||||
static Ret hash(UInt128 x)
|
||||
{
|
||||
return sipHash64(x);
|
||||
return static_cast<Ret>(sipHash64(x));
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -84,7 +84,7 @@ public:
|
||||
if (isNaN(v))
|
||||
return;
|
||||
|
||||
UInt32 hash = intHash64(determinator);
|
||||
UInt32 hash = static_cast<UInt32>(intHash64(determinator));
|
||||
insertImpl(v, hash);
|
||||
sorted = false;
|
||||
++total_values;
|
||||
|
@ -118,7 +118,7 @@ private:
|
||||
|
||||
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
|
||||
|
@ -44,12 +44,12 @@ namespace
|
||||
S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration(
|
||||
settings.auth_settings.region,
|
||||
context->getRemoteHostFilter(),
|
||||
context->getGlobalContext()->getSettingsRef().s3_max_redirects,
|
||||
static_cast<unsigned>(context->getGlobalContext()->getSettingsRef().s3_max_redirects),
|
||||
context->getGlobalContext()->getSettingsRef().enable_s3_requests_logging,
|
||||
/* for_disk_s3 = */ false);
|
||||
|
||||
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
|
||||
client_configuration.connectTimeoutMs = 10 * 1000;
|
||||
/// 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.SetKey(dst_key);
|
||||
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));
|
||||
|
||||
auto outcome = client->UploadPartCopy(part_request);
|
||||
@ -251,7 +251,7 @@ void BackupWriterS3::copyObjectMultipartImpl(
|
||||
for (size_t i = 0; i < part_tags.size(); ++i)
|
||||
{
|
||||
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);
|
||||
|
@ -62,7 +62,6 @@ namespace
|
||||
#define LIST_OF_BACKUP_SETTINGS(M) \
|
||||
M(String, id) \
|
||||
M(String, compression_method) \
|
||||
M(Int64, compression_level) \
|
||||
M(String, password) \
|
||||
M(Bool, structure_only) \
|
||||
M(Bool, async) \
|
||||
@ -72,6 +71,7 @@ namespace
|
||||
M(String, host_id) \
|
||||
M(String, coordination_zk_path) \
|
||||
M(OptionalUUID, backup_uuid)
|
||||
/// M(Int64, compression_level)
|
||||
|
||||
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;
|
||||
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) \
|
||||
if (setting.name == #NAME) \
|
||||
res.NAME = SettingField##TYPE{setting.value}.value; \
|
||||
|
@ -43,7 +43,7 @@ private:
|
||||
std::string hostname;
|
||||
size_t port;
|
||||
std::string log_level;
|
||||
size_t max_server_connections;
|
||||
unsigned max_server_connections;
|
||||
size_t http_timeout;
|
||||
|
||||
Poco::Logger * log;
|
||||
|
@ -43,7 +43,7 @@ protected:
|
||||
|
||||
virtual String serviceFileName() const = 0;
|
||||
|
||||
virtual size_t getDefaultPort() const = 0;
|
||||
virtual unsigned getDefaultPort() const = 0;
|
||||
|
||||
virtual bool startBridgeManually() const = 0;
|
||||
|
||||
|
@ -23,7 +23,7 @@ protected:
|
||||
|
||||
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; }
|
||||
|
||||
|
@ -109,7 +109,7 @@ protected:
|
||||
|
||||
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(); }
|
||||
|
||||
|
@ -550,7 +550,7 @@ try
|
||||
out_file_buf = wrapWriteBufferWithCompressionMethod(
|
||||
std::make_unique<WriteBufferFromFile>(out_file, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_EXCL | O_CREAT),
|
||||
compression_method,
|
||||
compression_level
|
||||
static_cast<int>(compression_level)
|
||||
);
|
||||
|
||||
if (query_with_output->is_into_outfile_with_stdout)
|
||||
@ -1602,6 +1602,8 @@ MultiQueryProcessingStage ClientBase::analyzeMultiQueryText(
|
||||
if (this_query_begin >= all_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
|
||||
// stop. The parser can't handle this situation because it always
|
||||
// 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.
|
||||
{
|
||||
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())
|
||||
return MultiQueryProcessingStage::QUERIES_END;
|
||||
}
|
||||
@ -1632,7 +1634,7 @@ MultiQueryProcessingStage ClientBase::analyzeMultiQueryText(
|
||||
if (ignore_error)
|
||||
{
|
||||
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())
|
||||
++token_iterator;
|
||||
this_query_begin = token_iterator->end;
|
||||
@ -1672,7 +1674,7 @@ MultiQueryProcessingStage ClientBase::analyzeMultiQueryText(
|
||||
// after we have processed the query. But even this guess is
|
||||
// beneficial so that we see proper trailing comments in "echo" and
|
||||
// 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;
|
||||
}
|
||||
|
||||
@ -1866,7 +1868,9 @@ bool ClientBase::executeMultiQuery(const String & all_queries_text)
|
||||
if (insert_ast && isSyncInsertWithData(*insert_ast, global_context))
|
||||
{
|
||||
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.
|
||||
@ -2347,7 +2351,7 @@ void ClientBase::init(int argc, char ** argv)
|
||||
if (options.count("print-profile-events"))
|
||||
config().setBool("print-profile-events", true);
|
||||
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"))
|
||||
config().setBool("progress", true);
|
||||
if (options.count("echo"))
|
||||
|
@ -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].is_change_replica_timeout_expired = 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);
|
||||
startNewReplica();
|
||||
}
|
||||
|
@ -362,7 +362,7 @@ void HedgedConnectionsFactory::removeReplicaFromEpoll(int index, int fd)
|
||||
timeout_fd_to_replica_index.erase(replicas[index].change_replica_timeout.getDescriptor());
|
||||
}
|
||||
|
||||
int HedgedConnectionsFactory::numberOfProcessingReplicas() const
|
||||
size_t HedgedConnectionsFactory::numberOfProcessingReplicas() const
|
||||
{
|
||||
if (epoll.empty())
|
||||
return 0;
|
||||
@ -381,7 +381,7 @@ HedgedConnectionsFactory::State HedgedConnectionsFactory::setBestUsableReplica(C
|
||||
&& result.is_usable
|
||||
&& !replicas[i].is_ready
|
||||
&& (!skip_replicas_with_two_level_aggregation_incompatibility || !isTwoLevelAggregationIncompatible(&*result.entry)))
|
||||
indexes.push_back(i);
|
||||
indexes.push_back(static_cast<int>(i));
|
||||
}
|
||||
|
||||
if (indexes.empty())
|
||||
|
@ -70,7 +70,7 @@ public:
|
||||
|
||||
const ConnectionTimeouts & getConnectionTimeouts() const { return timeouts; }
|
||||
|
||||
int numberOfProcessingReplicas() const;
|
||||
size_t numberOfProcessingReplicas() const;
|
||||
|
||||
/// Tell Factory to not return connections with two level aggregation incompatibility.
|
||||
void skipReplicasWithTwoLevelAggregationIncompatibility() { skip_replicas_with_two_level_aggregation_incompatibility = true; }
|
||||
|
@ -81,9 +81,9 @@ Field QueryFuzzer::getRandomField(int type)
|
||||
{
|
||||
static constexpr UInt64 scales[] = {0, 1, 2, 10};
|
||||
return DecimalField<Decimal64>(
|
||||
bad_int64_values[fuzz_rand() % (sizeof(bad_int64_values)
|
||||
/ sizeof(*bad_int64_values))],
|
||||
scales[fuzz_rand() % (sizeof(scales) / sizeof(*scales))]);
|
||||
bad_int64_values[fuzz_rand() % (sizeof(bad_int64_values) / sizeof(*bad_int64_values))],
|
||||
static_cast<UInt32>(scales[fuzz_rand() % (sizeof(scales) / sizeof(*scales))])
|
||||
);
|
||||
}
|
||||
default:
|
||||
assert(false);
|
||||
|
@ -569,8 +569,8 @@ void ColumnArray::expand(const IColumn::Filter & mask, bool inverted)
|
||||
if (mask.size() < offsets_data.size())
|
||||
throw Exception("Mask size should be no less than data size.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
int index = mask.size() - 1;
|
||||
int from = offsets_data.size() - 1;
|
||||
ssize_t index = mask.size() - 1;
|
||||
ssize_t from = offsets_data.size() - 1;
|
||||
offsets_data.resize(mask.size());
|
||||
UInt64 last_offset = offsets_data[from];
|
||||
while (index >= 0)
|
||||
|
@ -27,8 +27,8 @@ std::shared_ptr<Memory<>> ColumnCompressed::compressBuffer(const void * data, si
|
||||
auto compressed_size = LZ4_compress_default(
|
||||
reinterpret_cast<const char *>(data),
|
||||
compressed.data(),
|
||||
data_size,
|
||||
max_dest_size);
|
||||
static_cast<UInt32>(data_size),
|
||||
static_cast<UInt32>(max_dest_size));
|
||||
|
||||
if (compressed_size <= 0)
|
||||
throw Exception(ErrorCodes::CANNOT_COMPRESS, "Cannot compress column");
|
||||
@ -51,8 +51,8 @@ void ColumnCompressed::decompressBuffer(
|
||||
auto processed_size = LZ4_decompress_safe(
|
||||
reinterpret_cast<const char *>(compressed_data),
|
||||
reinterpret_cast<char *>(decompressed_data),
|
||||
compressed_size,
|
||||
decompressed_size);
|
||||
static_cast<UInt32>(compressed_size),
|
||||
static_cast<UInt32>(decompressed_size));
|
||||
|
||||
if (processed_size <= 0)
|
||||
throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress column");
|
||||
|
@ -277,8 +277,8 @@ void ColumnFixedString::expand(const IColumn::Filter & mask, bool inverted)
|
||||
if (mask.size() < size())
|
||||
throw Exception("Mask size should be no less than data size.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
int index = mask.size() - 1;
|
||||
int from = size() - 1;
|
||||
ssize_t index = mask.size() - 1;
|
||||
ssize_t from = size() - 1;
|
||||
chars.resize_fill(mask.size() * n, 0);
|
||||
while (index >= 0)
|
||||
{
|
||||
|
@ -46,7 +46,7 @@ namespace
|
||||
|
||||
HashMap<T, T> hash_map;
|
||||
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 & data = res_col->getData();
|
||||
@ -632,7 +632,7 @@ void ColumnLowCardinality::Index::convertPositions()
|
||||
|
||||
/// TODO: Optimize with SSE?
|
||||
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);
|
||||
size_of_type = sizeof(IndexType);
|
||||
@ -717,7 +717,7 @@ void ColumnLowCardinality::Index::insertPositionsRange(const IColumn & column, U
|
||||
positions_data.resize(size + limit);
|
||||
|
||||
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);
|
||||
|
@ -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.
|
||||
/// So, we will insert one zero byte when mask value is zero.
|
||||
|
||||
int index = mask.size() - 1;
|
||||
int from = offsets_data.size() - 1;
|
||||
ssize_t index = mask.size() - 1;
|
||||
ssize_t from = offsets_data.size() - 1;
|
||||
/// 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.
|
||||
UInt64 last_offset = offsets_data[from] + (mask.size() - offsets_data.size());
|
||||
|
@ -550,7 +550,7 @@ MutableColumnPtr ColumnUnique<ColumnType>::uniqueInsertRangeImpl(
|
||||
auto insert_key = [&](StringRef ref, ReverseIndex<UInt64, ColumnType> & cur_index) -> MutableColumnPtr
|
||||
{
|
||||
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)
|
||||
return update_position(next_position);
|
||||
|
||||
@ -562,9 +562,9 @@ MutableColumnPtr ColumnUnique<ColumnType>::uniqueInsertRangeImpl(
|
||||
auto row = start + num_added_rows;
|
||||
|
||||
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)
|
||||
positions[num_added_rows] = getNestedTypeDefaultValueIndex();
|
||||
positions[num_added_rows] = static_cast<IndexType>(getNestedTypeDefaultValueIndex());
|
||||
else
|
||||
{
|
||||
auto ref = src_column->getDataAt(row);
|
||||
@ -576,7 +576,7 @@ MutableColumnPtr ColumnUnique<ColumnType>::uniqueInsertRangeImpl(
|
||||
if (insertion_point == reverse_index.lastInsertionPoint())
|
||||
res = insert_key(ref, *secondary_index);
|
||||
else
|
||||
positions[num_added_rows] = insertion_point;
|
||||
positions[num_added_rows] = static_cast<IndexType>(insertion_point);
|
||||
}
|
||||
else
|
||||
res = insert_key(ref, reverse_index);
|
||||
|
@ -22,8 +22,8 @@ void expandDataByMask(PaddedPODArray<T> & data, const PaddedPODArray<UInt8> & ma
|
||||
if (mask.size() < data.size())
|
||||
throw Exception("Mask size should be no less than data size.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
int from = data.size() - 1;
|
||||
int index = mask.size() - 1;
|
||||
ssize_t from = data.size() - 1;
|
||||
ssize_t index = mask.size() - 1;
|
||||
data.resize(mask.size());
|
||||
while (index >= 0)
|
||||
{
|
||||
@ -317,7 +317,7 @@ int checkShortCircuitArguments(const ColumnsWithTypeAndName & arguments)
|
||||
for (size_t i = 0; i != arguments.size(); ++i)
|
||||
{
|
||||
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;
|
||||
|
@ -164,7 +164,7 @@ TEST(WeakHash32, ColumnVectorU32)
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
@ -181,7 +181,7 @@ TEST(WeakHash32, ColumnVectorI32)
|
||||
|
||||
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
|
||||
}
|
||||
|
||||
@ -240,7 +240,7 @@ TEST(WeakHash32, ColumnVectorU128)
|
||||
val.items[0] = i << 32u;
|
||||
val.items[1] = i << 32u;
|
||||
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 (int64_t i = -32768; i < 32768; ++i)
|
||||
for (int32_t i = -32768; i < 32768; ++i)
|
||||
data.push_back(i << 16); //-V610
|
||||
}
|
||||
|
||||
@ -326,7 +326,7 @@ TEST(WeakHash32, ColumnString1)
|
||||
|
||||
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);
|
||||
auto str = std::to_string(i);
|
||||
@ -359,7 +359,7 @@ TEST(WeakHash32, ColumnString2)
|
||||
{
|
||||
size_t max_size = 3000;
|
||||
char letter = 'a';
|
||||
for (int64_t i = 0; i < 65536; ++i)
|
||||
for (int32_t i = 0; i < 65536; ++i)
|
||||
{
|
||||
data.push_back(i);
|
||||
size_t s = (i % max_size) + 1;
|
||||
@ -401,7 +401,7 @@ TEST(WeakHash32, ColumnString3)
|
||||
char letter = 'a';
|
||||
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;
|
||||
std::string str(s,'\0');
|
||||
str[0] = letter;
|
||||
@ -430,7 +430,7 @@ TEST(WeakHash32, ColumnFixedString)
|
||||
char letter = 'a';
|
||||
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;
|
||||
std::string str(s, letter);
|
||||
col->insertData(str.data(), str.size());
|
||||
@ -471,7 +471,7 @@ TEST(WeakHash32, ColumnArray)
|
||||
UInt32 cur = 0;
|
||||
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;
|
||||
|
||||
cur_off += s;
|
||||
@ -505,9 +505,9 @@ TEST(WeakHash32, ColumnArray2)
|
||||
UInt64 cur_off = 0;
|
||||
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);
|
||||
|
||||
@ -556,7 +556,7 @@ TEST(WeakHash32, ColumnArrayArray)
|
||||
UInt32 cur = 1;
|
||||
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;
|
||||
|
||||
cur_off2 += s;
|
||||
@ -667,7 +667,7 @@ TEST(WeakHash32, ColumnTupleUInt64UInt64)
|
||||
{
|
||||
data1.push_back(l);
|
||||
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;
|
||||
char letter = 'a';
|
||||
for (int64_t i = 0; i < 65536; ++i)
|
||||
for (int32_t i = 0; i < 65536; ++i)
|
||||
{
|
||||
data1.push_back(l);
|
||||
eq.push_back(l * 65536 + i);
|
||||
@ -737,7 +737,7 @@ TEST(WeakHash32, ColumnTupleUInt64FixedString)
|
||||
for (int64_t i = 0; i < 65536; ++i)
|
||||
{
|
||||
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;
|
||||
std::string str(s, letter);
|
||||
@ -778,7 +778,7 @@ TEST(WeakHash32, ColumnTupleUInt64Array)
|
||||
auto l = idx % 2;
|
||||
|
||||
UInt32 cur = 0;
|
||||
for (int64_t i = 0; i < 65536; ++i)
|
||||
for (int32_t i = 0; i < 65536; ++i)
|
||||
{
|
||||
data1.push_back(l);
|
||||
eq_data.push_back(l * 65536 + i);
|
||||
|
@ -65,7 +65,7 @@ public:
|
||||
private:
|
||||
using Small = SmallSet<Key, small_set_size_max>;
|
||||
using Medium = HashContainer;
|
||||
using Large = HyperLogLogCounter<K, Hash, HashValueType, DenominatorType, BiasEstimator, mode>;
|
||||
using Large = HyperLogLogCounter<K, Key, Hash, HashValueType, DenominatorType, BiasEstimator, mode>;
|
||||
|
||||
public:
|
||||
CombinedCardinalityEstimator()
|
||||
|
@ -149,9 +149,9 @@ DateLUTImpl::DateLUTImpl(const std::string & time_zone_)
|
||||
|
||||
/// Fill lookup table for years and months.
|
||||
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];
|
||||
|
||||
|
@ -73,7 +73,7 @@ private:
|
||||
return LUTIndex(0);
|
||||
if (index >= DATE_LUT_SIZE)
|
||||
return LUTIndex(DATE_LUT_SIZE - 1);
|
||||
return LUTIndex{index};
|
||||
return LUTIndex{static_cast<UInt32>(index)};
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
@ -229,12 +229,12 @@ private:
|
||||
if (t >= lut[guess].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)
|
||||
@ -272,11 +272,11 @@ private:
|
||||
if (likely(offset_is_whole_number_of_hours_during_epoch))
|
||||
{
|
||||
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).
|
||||
/// 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;
|
||||
@ -285,7 +285,7 @@ private:
|
||||
{
|
||||
if (unlikely(res < 0))
|
||||
return 0;
|
||||
return res;
|
||||
return static_cast<DateOrTime>(res);
|
||||
}
|
||||
else
|
||||
return res;
|
||||
@ -509,7 +509,7 @@ public:
|
||||
if (time >= lut[index].time_at_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.
|
||||
return res <= 23 ? res : 23;
|
||||
@ -548,8 +548,8 @@ public:
|
||||
{
|
||||
Time res = t % 60;
|
||||
if (likely(res >= 0))
|
||||
return res;
|
||||
return res + 60;
|
||||
return static_cast<unsigned>(res);
|
||||
return static_cast<unsigned>(res) + 60;
|
||||
}
|
||||
|
||||
LUTIndex index = findIndex(t);
|
||||
@ -973,7 +973,7 @@ public:
|
||||
if constexpr (std::is_same_v<Date, DayNum>)
|
||||
return DayNum(4 + (d - 4) / days * days);
|
||||
else
|
||||
return ExtendedDayNum(4 + (d - 4) / days * days);
|
||||
return ExtendedDayNum(static_cast<Int32>(4 + (d - 4) / days * days));
|
||||
}
|
||||
|
||||
template <typename Date>
|
||||
@ -983,9 +983,9 @@ public:
|
||||
if (days == 1)
|
||||
return toDate(d);
|
||||
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
|
||||
return lut[toLUTIndex(ExtendedDayNum(d / days * days))].date;
|
||||
return lut[toLUTIndex(ExtendedDayNum(static_cast<Int32>(d / days * days)))].date;
|
||||
}
|
||||
|
||||
template <typename DateOrTime>
|
||||
@ -1034,7 +1034,7 @@ public:
|
||||
{
|
||||
if (unlikely(res < 0))
|
||||
return 0;
|
||||
return res;
|
||||
return static_cast<DateOrTime>(res);
|
||||
}
|
||||
else
|
||||
return res;
|
||||
@ -1047,8 +1047,8 @@ public:
|
||||
if (likely(offset_is_whole_number_of_minutes_during_epoch))
|
||||
{
|
||||
if (likely(t >= 0))
|
||||
return t / divisor * divisor;
|
||||
return (t + 1 - divisor) / divisor * divisor;
|
||||
return static_cast<DateOrTime>(t / divisor * divisor);
|
||||
return static_cast<DateOrTime>((t + 1 - divisor) / divisor * divisor);
|
||||
}
|
||||
|
||||
Time date = find(t).date;
|
||||
@ -1057,7 +1057,7 @@ public:
|
||||
{
|
||||
if (unlikely(res < 0))
|
||||
return 0;
|
||||
return res;
|
||||
return static_cast<UInt32>(res);
|
||||
}
|
||||
else
|
||||
return res;
|
||||
@ -1071,7 +1071,7 @@ public:
|
||||
if (seconds % 60 == 0)
|
||||
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
|
||||
@ -1249,9 +1249,9 @@ public:
|
||||
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
|
||||
@ -1331,9 +1331,9 @@ public:
|
||||
}
|
||||
|
||||
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>
|
||||
|
@ -218,7 +218,7 @@ private:
|
||||
// Offset from start to first attribute
|
||||
uint8_t attr_offset;
|
||||
// Offset within debug info.
|
||||
uint32_t offset;
|
||||
uint64_t offset;
|
||||
uint64_t code;
|
||||
DIEAbbreviation abbr;
|
||||
};
|
||||
@ -252,10 +252,10 @@ private:
|
||||
uint8_t unit_type = DW_UT_compile; // DW_UT_compile or DW_UT_skeleton
|
||||
uint8_t addr_size = 0;
|
||||
// Offset in .debug_info of this compilation unit.
|
||||
uint32_t offset = 0;
|
||||
uint32_t size = 0;
|
||||
uint64_t offset = 0;
|
||||
uint64_t size = 0;
|
||||
// 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;
|
||||
|
||||
// The beginning of the CU's contribution to .debug_addr
|
||||
|
@ -48,33 +48,36 @@ inline DB::UInt64 intHash64(DB::UInt64 x)
|
||||
#include <arm_acle.h>
|
||||
#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__
|
||||
return _mm_crc32_u64(-1ULL, x);
|
||||
return static_cast<UInt32>(_mm_crc32_u64(-1ULL, x));
|
||||
#elif defined(__aarch64__) && defined(__ARM_FEATURE_CRC32)
|
||||
return __crc32cd(-1U, x);
|
||||
return static_cast<UInt32>(__crc32cd(-1U, x));
|
||||
#else
|
||||
/// 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
|
||||
}
|
||||
|
||||
inline DB::UInt64 intHashCRC32(DB::UInt64 x, DB::UInt64 updated_value)
|
||||
inline DB::UInt32 intHashCRC32(DB::UInt64 x, DB::UInt64 updated_value)
|
||||
{
|
||||
#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)
|
||||
return __crc32cd(updated_value, x);
|
||||
return __crc32cd(static_cast<UInt32>(updated_value), x);
|
||||
#else
|
||||
/// 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
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
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);
|
||||
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);
|
||||
}
|
||||
|
||||
return updated_value;
|
||||
return static_cast<UInt32>(updated_value);
|
||||
}
|
||||
|
||||
|
||||
@ -219,7 +222,7 @@ template <typename T> struct HashCRC32;
|
||||
|
||||
template <typename T>
|
||||
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
|
||||
{
|
||||
@ -233,7 +236,7 @@ inline size_t hashCRC32(T key, DB::UInt64 updated_value = -1)
|
||||
|
||||
template <typename T>
|
||||
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);
|
||||
}
|
||||
@ -241,7 +244,7 @@ inline size_t hashCRC32(T key, DB::UInt64 updated_value = -1)
|
||||
#define DEFINE_HASH(T) \
|
||||
template <> struct HashCRC32<T>\
|
||||
{\
|
||||
size_t operator() (T key) const\
|
||||
UInt32 operator() (T key) const\
|
||||
{\
|
||||
return hashCRC32<T>(key);\
|
||||
}\
|
||||
@ -302,8 +305,8 @@ struct UInt128HashCRC32
|
||||
size_t operator()(UInt128 x) const
|
||||
{
|
||||
UInt64 crc = -1ULL;
|
||||
crc = __crc32cd(crc, x.items[0]);
|
||||
crc = __crc32cd(crc, x.items[1]);
|
||||
crc = __crc32cd(static_cast<UInt32>(crc), x.items[0]);
|
||||
crc = __crc32cd(static_cast<UInt32>(crc), x.items[1]);
|
||||
return crc;
|
||||
}
|
||||
};
|
||||
@ -358,10 +361,10 @@ struct UInt256HashCRC32
|
||||
size_t operator()(UInt256 x) const
|
||||
{
|
||||
UInt64 crc = -1ULL;
|
||||
crc = __crc32cd(crc, x.items[0]);
|
||||
crc = __crc32cd(crc, x.items[1]);
|
||||
crc = __crc32cd(crc, x.items[2]);
|
||||
crc = __crc32cd(crc, x.items[3]);
|
||||
crc = __crc32cd(static_cast<UInt32>(crc), x.items[0]);
|
||||
crc = __crc32cd(static_cast<UInt32>(crc), x.items[1]);
|
||||
crc = __crc32cd(static_cast<UInt32>(crc), x.items[2]);
|
||||
crc = __crc32cd(static_cast<UInt32>(crc), x.items[3]);
|
||||
return crc;
|
||||
}
|
||||
};
|
||||
@ -423,7 +426,7 @@ inline DB::UInt32 intHash32(DB::UInt64 key)
|
||||
key = key + (key << 6);
|
||||
key = key ^ ((key >> 22) | (key << 42));
|
||||
|
||||
return key;
|
||||
return static_cast<UInt32>(key);
|
||||
}
|
||||
|
||||
|
||||
|
@ -44,8 +44,8 @@ protected:
|
||||
public:
|
||||
using Impl = ImplTable;
|
||||
|
||||
static constexpr size_t NUM_BUCKETS = 1ULL << BITS_FOR_BUCKET;
|
||||
static constexpr size_t MAX_BUCKET = NUM_BUCKETS - 1;
|
||||
static constexpr UInt32 NUM_BUCKETS = 1ULL << BITS_FOR_BUCKET;
|
||||
static constexpr UInt32 MAX_BUCKET = NUM_BUCKETS - 1;
|
||||
|
||||
size_t hash(const Key & x) const { return Hash::operator()(x); }
|
||||
|
||||
@ -286,13 +286,13 @@ public:
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
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)
|
||||
DB::writeChar(',', wb);
|
||||
@ -302,13 +302,13 @@ public:
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
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)
|
||||
DB::assertChar(',', rb);
|
||||
@ -320,7 +320,7 @@ public:
|
||||
size_t size() const
|
||||
{
|
||||
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();
|
||||
|
||||
return res;
|
||||
@ -328,7 +328,7 @@ public:
|
||||
|
||||
bool empty() const
|
||||
{
|
||||
for (size_t i = 0; i < NUM_BUCKETS; ++i)
|
||||
for (UInt32 i = 0; i < NUM_BUCKETS; ++i)
|
||||
if (!impls[i].empty())
|
||||
return false;
|
||||
|
||||
@ -338,7 +338,7 @@ public:
|
||||
size_t getBufferSizeInBytes() const
|
||||
{
|
||||
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();
|
||||
|
||||
return res;
|
||||
|
@ -13,8 +13,8 @@ public:
|
||||
using Key = StringRef;
|
||||
using Impl = ImplTable;
|
||||
|
||||
static constexpr size_t NUM_BUCKETS = 1ULL << BITS_FOR_BUCKET;
|
||||
static constexpr size_t MAX_BUCKET = NUM_BUCKETS - 1;
|
||||
static constexpr UInt32 NUM_BUCKETS = 1ULL << BITS_FOR_BUCKET;
|
||||
static constexpr UInt32 MAX_BUCKET = NUM_BUCKETS - 1;
|
||||
|
||||
// TODO: currently hashing contains redundant computations when doing distributed or external aggregations
|
||||
size_t hash(const Key & x) const
|
||||
@ -175,13 +175,13 @@ public:
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
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)
|
||||
DB::writeChar(',', wb);
|
||||
@ -191,13 +191,13 @@ public:
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
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)
|
||||
DB::assertChar(',', rb);
|
||||
@ -208,7 +208,7 @@ public:
|
||||
size_t size() const
|
||||
{
|
||||
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();
|
||||
|
||||
return res;
|
||||
@ -216,7 +216,7 @@ public:
|
||||
|
||||
bool empty() const
|
||||
{
|
||||
for (size_t i = 0; i < NUM_BUCKETS; ++i)
|
||||
for (UInt32 i = 0; i < NUM_BUCKETS; ++i)
|
||||
if (!impls[i].empty())
|
||||
return false;
|
||||
|
||||
@ -226,7 +226,7 @@ public:
|
||||
size_t getBufferSizeInBytes() const
|
||||
{
|
||||
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();
|
||||
|
||||
return res;
|
||||
|
@ -264,7 +264,8 @@ enum class HyperLogLogMode
|
||||
/// of Algorithms).
|
||||
template <
|
||||
UInt8 precision,
|
||||
typename Hash = IntHash32<UInt64>,
|
||||
typename Key = UInt64,
|
||||
typename Hash = IntHash32<Key>,
|
||||
typename HashValueType = UInt32,
|
||||
typename DenominatorType = double,
|
||||
typename BiasEstimator = TrivialBiasEstimator,
|
||||
@ -409,7 +410,9 @@ private:
|
||||
|
||||
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.
|
||||
@ -532,6 +535,7 @@ private:
|
||||
template
|
||||
<
|
||||
UInt8 precision,
|
||||
typename Key,
|
||||
typename Hash,
|
||||
typename HashValueType,
|
||||
typename DenominatorType,
|
||||
@ -542,6 +546,7 @@ template
|
||||
details::LogLUT<precision> HyperLogLogCounter
|
||||
<
|
||||
precision,
|
||||
Key,
|
||||
Hash,
|
||||
HashValueType,
|
||||
DenominatorType,
|
||||
@ -555,6 +560,7 @@ details::LogLUT<precision> HyperLogLogCounter
|
||||
/// Serialization format must not be changed.
|
||||
using HLL12 = HyperLogLogCounter<
|
||||
12,
|
||||
UInt64,
|
||||
IntHash32<UInt64>,
|
||||
UInt32,
|
||||
double,
|
||||
|
@ -26,7 +26,7 @@ class HyperLogLogWithSmallSetOptimization : private boost::noncopyable
|
||||
{
|
||||
private:
|
||||
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;
|
||||
|
||||
Small small;
|
||||
|
@ -506,8 +506,16 @@ unsigned OptimizedRegularExpressionImpl<thread_safe>::match(const char * subject
|
||||
|
||||
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;
|
||||
}
|
||||
else
|
||||
{
|
||||
matches.resize(limit);
|
||||
|
@ -339,7 +339,7 @@ struct PoolWithFailoverBase<TNestedPool>::PoolState
|
||||
Int64 config_priority = 1;
|
||||
/// Priority from the GetPriorityFunc.
|
||||
Int64 priority = 0;
|
||||
UInt32 random = 0;
|
||||
UInt64 random = 0;
|
||||
|
||||
void randomize()
|
||||
{
|
||||
|
@ -132,11 +132,11 @@ QueryProfilerBase<ProfilerImpl>::QueryProfilerBase(UInt64 thread_id, int clock_t
|
||||
sev.sigev_signo = pause_signal;
|
||||
|
||||
#if defined(OS_FREEBSD)
|
||||
sev._sigev_un._threadid = thread_id;
|
||||
sev._sigev_un._threadid = static_cast<pid_t>(thread_id);
|
||||
#elif defined(USE_MUSL)
|
||||
sev.sigev_notify_thread_id = thread_id;
|
||||
sev.sigev_notify_thread_id = static_cast<pid_t>(thread_id);
|
||||
#else
|
||||
sev._sigev_un._tid = thread_id;
|
||||
sev._sigev_un._tid = static_cast<pid_t>(thread_id);
|
||||
#endif
|
||||
timer_t local_timer_id;
|
||||
if (timer_create(clock_type, &sev, &local_timer_id))
|
||||
|
@ -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.
|
||||
size_t sums[NUM_PASSES] = {0};
|
||||
CountType sums[NUM_PASSES] = {0};
|
||||
|
||||
for (size_t i = 0; i < HISTOGRAM_SIZE; ++i)
|
||||
{
|
||||
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;
|
||||
sums[pass] = tmp;
|
||||
}
|
||||
|
@ -40,7 +40,7 @@ public:
|
||||
#ifdef __SSE2__
|
||||
protected:
|
||||
static constexpr auto n = sizeof(__m128i);
|
||||
const int page_size = ::getPageSize();
|
||||
const Int64 page_size = ::getPageSize();
|
||||
|
||||
bool pageSafe(const void * const ptr) const
|
||||
{
|
||||
|
@ -87,7 +87,7 @@ void ThreadPoolImpl<Thread>::setQueueSize(size_t value)
|
||||
|
||||
template <typename Thread>
|
||||
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)
|
||||
{
|
||||
@ -163,19 +163,19 @@ ReturnType ThreadPoolImpl<Thread>::scheduleImpl(Job job, int priority, std::opti
|
||||
}
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
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);
|
||||
}
|
||||
|
@ -50,13 +50,13 @@ public:
|
||||
/// 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,
|
||||
/// 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.
|
||||
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.
|
||||
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.
|
||||
/// You may call schedule and wait many times in arbitrary order.
|
||||
@ -96,10 +96,10 @@ private:
|
||||
struct JobWithPriority
|
||||
{
|
||||
Job job;
|
||||
int priority;
|
||||
ssize_t priority;
|
||||
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_) {}
|
||||
|
||||
bool operator< (const JobWithPriority & rhs) const
|
||||
@ -113,7 +113,7 @@ private:
|
||||
std::exception_ptr first_exception;
|
||||
|
||||
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);
|
||||
|
||||
|
@ -121,7 +121,7 @@ TasksStatsCounters::TasksStatsCounters(const UInt64 tid, const MetricsProvider p
|
||||
stats_getter = [metrics_provider = std::make_shared<TaskStatsInfoGetter>(), tid]()
|
||||
{
|
||||
::taskstats result{};
|
||||
metrics_provider->getStat(result, tid);
|
||||
metrics_provider->getStat(result, static_cast<pid_t>(tid));
|
||||
return result;
|
||||
};
|
||||
break;
|
||||
@ -526,7 +526,7 @@ void PerfEventsCounters::finalizeProfileEvents(ProfileEvents::Counters & profile
|
||||
continue;
|
||||
|
||||
constexpr ssize_t bytes_to_read = sizeof(current_values[0]);
|
||||
const int bytes_read = read(fd, ¤t_values[i], bytes_to_read);
|
||||
const ssize_t bytes_read = read(fd, ¤t_values[i], bytes_to_read);
|
||||
|
||||
if (bytes_read != bytes_to_read)
|
||||
{
|
||||
|
@ -99,7 +99,10 @@ requires (sizeof(CharT) == 1)
|
||||
size_t convertCodePointToUTF8(int code_point, CharT * out_bytes, size_t out_length)
|
||||
{
|
||||
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);
|
||||
return res;
|
||||
}
|
||||
@ -109,7 +112,9 @@ requires (sizeof(CharT) == 1)
|
||||
std::optional<uint32_t> convertUTF8ToCodePoint(const CharT * in_bytes, size_t in_length)
|
||||
{
|
||||
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)
|
||||
return res;
|
||||
|
@ -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.
|
||||
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
|
||||
* are represented by different number of bytes or code points.
|
||||
|
@ -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.mtime = created_node.stat.ctime;
|
||||
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.is_ephemeral = is_ephemeral;
|
||||
created_node.is_sequental = is_sequential;
|
||||
|
@ -45,7 +45,7 @@ struct ShuffleHost
|
||||
{
|
||||
String host;
|
||||
Int64 priority = 0;
|
||||
UInt32 random = 0;
|
||||
UInt64 random = 0;
|
||||
|
||||
void randomize()
|
||||
{
|
||||
|
@ -724,7 +724,7 @@ void ZooKeeperRequest::createLogElements(LogElements & elems) const
|
||||
elem.has_watch = has_watch;
|
||||
elem.op_num = static_cast<uint32_t>(getOpNum());
|
||||
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
|
||||
{
|
||||
ZooKeeperRequest::createLogElements(elems);
|
||||
elems.back().requests_size = requests.size();
|
||||
elems.back().requests_size = static_cast<uint32_t>(requests.size());
|
||||
for (const auto & request : requests)
|
||||
{
|
||||
auto & req = dynamic_cast<ZooKeeperRequest &>(*request);
|
||||
|
@ -546,7 +546,7 @@ void ZooKeeper::sendAuth(const String & scheme, const String & data)
|
||||
if (read_xid != AUTH_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)
|
||||
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)
|
||||
throw Exception(Error::ZMARSHALLINGERROR, "Response length doesn't match. Expected: {}, actual: {}", length, actual_length);
|
||||
|
||||
|
@ -492,7 +492,7 @@ int main(int argc, char ** argv)
|
||||
|
||||
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.wait();
|
||||
@ -545,7 +545,7 @@ int main(int argc, char ** argv)
|
||||
|
||||
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.wait();
|
||||
|
@ -258,7 +258,7 @@ size_t getSizeFromFileDescriptor(int fd, const String & file_name)
|
||||
return buf.st_size;
|
||||
}
|
||||
|
||||
int getINodeNumberFromPath(const String & path)
|
||||
Int64 getINodeNumberFromPath(const String & path)
|
||||
{
|
||||
struct stat file_stat;
|
||||
if (stat(path.data(), &file_stat))
|
||||
|
@ -74,7 +74,7 @@ std::optional<size_t> tryGetSizeFromFilePath(const String & path);
|
||||
|
||||
/// Get inode number for a file path.
|
||||
/// Will not work correctly if filesystem does not support inodes.
|
||||
int getINodeNumberFromPath(const String & path);
|
||||
Int64 getINodeNumberFromPath(const String & path);
|
||||
|
||||
}
|
||||
|
||||
|
@ -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)
|
||||
{
|
||||
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{};
|
||||
|
||||
/** Preprocess:
|
||||
@ -122,15 +122,19 @@ void formatIPv6(const unsigned char * src, char *& dst, uint8_t zeroed_tail_byte
|
||||
best.base = -1;
|
||||
|
||||
/// 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?
|
||||
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++ = ':';
|
||||
continue;
|
||||
}
|
||||
}
|
||||
|
||||
/// Are we following an initial run of 0x00s or any real hex?
|
||||
if (i != 0)
|
||||
|
@ -7,10 +7,10 @@
|
||||
#include <filesystem>
|
||||
|
||||
|
||||
int getCurrentProcessFDCount()
|
||||
Int64 getCurrentProcessFDCount()
|
||||
{
|
||||
namespace fs = std::filesystem;
|
||||
int result = -1;
|
||||
Int64 result = -1;
|
||||
#if defined(OS_LINUX) || defined(OS_DARWIN)
|
||||
using namespace DB;
|
||||
|
||||
|
@ -1,5 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <base/types.h>
|
||||
|
||||
/// Get current process file descriptor count
|
||||
/// @return -1 os doesn't support "lsof" command or some error occurs.
|
||||
int getCurrentProcessFDCount();
|
||||
Int64 getCurrentProcessFDCount();
|
||||
|
@ -232,7 +232,7 @@ template <typename T>
|
||||
constexpr inline T intExp10OfSize(int x)
|
||||
{
|
||||
if constexpr (sizeof(T) <= 8)
|
||||
return intExp10(x);
|
||||
return static_cast<T>(intExp10(x));
|
||||
else if constexpr (sizeof(T) <= 16)
|
||||
return common::exp10_i128(x);
|
||||
else
|
||||
|
@ -124,7 +124,7 @@ double Value::readFloatText(const char * buf, size_t length) const
|
||||
case 'E':
|
||||
{
|
||||
++buf;
|
||||
Int32 exponent = readIntText(buf, end - buf);
|
||||
Int32 exponent = static_cast<Int32>(readIntText(buf, end - buf));
|
||||
x *= preciseExp10(exponent);
|
||||
if (negative)
|
||||
x = -x;
|
||||
|
@ -44,9 +44,9 @@ public:
|
||||
/** Получить значение по индексу.
|
||||
* Здесь используется 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.");
|
||||
return Value(row[n], lengths[n], res);
|
||||
}
|
||||
|
@ -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 short Value::get<short >() const { return getInt(); } /// NOLINT
|
||||
template <> inline unsigned short Value::get<unsigned short >() const { return getUInt(); } /// NOLINT
|
||||
template <> inline int Value::get<int >() const { return getInt(); }
|
||||
template <> inline unsigned int Value::get<unsigned int >() const { return getUInt(); }
|
||||
template <> inline int Value::get<int >() const { return static_cast<int>(getInt()); }
|
||||
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 unsigned long Value::get<unsigned long >() const { return getUInt(); } /// NOLINT
|
||||
template <> inline long long Value::get<long long >() const { return getInt(); } /// NOLINT
|
||||
|
@ -68,14 +68,14 @@ std::string makeRegexpPatternFromGlobs(const std::string & initial_str_with_glob
|
||||
output_width = std::max(range_begin_width, range_end_width);
|
||||
|
||||
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;
|
||||
|
||||
for (size_t i = range_begin + 1; i <= range_end; ++i)
|
||||
{
|
||||
oss_for_replacing << '|';
|
||||
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;
|
||||
}
|
||||
}
|
||||
|
@ -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
|
||||
if (description[i] == '{')
|
||||
{
|
||||
int cnt = 1;
|
||||
int last_dot = -1; /// The rightmost pair of points, remember the index of the right of the two
|
||||
ssize_t cnt = 1;
|
||||
ssize_t last_dot = -1; /// The rightmost pair of points, remember the index of the right of the two
|
||||
size_t m;
|
||||
std::vector<String> buffer;
|
||||
bool have_splitter = false;
|
||||
|
22
src/Common/safe_cast.h
Normal file
22
src/Common/safe_cast.h
Normal 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);
|
||||
}
|
||||
|
||||
}
|
@ -58,7 +58,7 @@ FailuresCount countFailures(const ::testing::TestResult & test_result)
|
||||
const size_t count = test_result.total_part_count();
|
||||
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())
|
||||
{
|
||||
++failures.non_fatal;
|
||||
|
@ -216,27 +216,27 @@ TEST(HashTable, Erase)
|
||||
using Cont = HashSet<int, DefaultHash<int>, HashTableGrowerWithPrecalculation<1>>;
|
||||
Cont cont;
|
||||
|
||||
for (size_t i = 0; i < 5000; ++i)
|
||||
for (int i = 0; i < 5000; ++i)
|
||||
{
|
||||
cont.insert(i);
|
||||
}
|
||||
|
||||
for (size_t i = 0; i < 2500; ++i)
|
||||
for (int i = 0; i < 2500; ++i)
|
||||
{
|
||||
cont.erase(i);
|
||||
}
|
||||
|
||||
for (size_t i = 5000; i < 10000; ++i)
|
||||
for (int i = 5000; i < 10000; ++i)
|
||||
{
|
||||
cont.insert(i);
|
||||
}
|
||||
|
||||
for (size_t i = 5000; i < 10000; ++i)
|
||||
for (int i = 5000; i < 10000; ++i)
|
||||
{
|
||||
cont.erase(i);
|
||||
}
|
||||
|
||||
for (size_t i = 2500; i < 5000; ++i)
|
||||
for (int i = 2500; i < 5000; ++i)
|
||||
{
|
||||
cont.erase(i);
|
||||
}
|
||||
|
@ -26,7 +26,7 @@ void testInsert(size_t elements_to_insert_size, size_t map_size)
|
||||
std::vector<int> expected;
|
||||
|
||||
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)
|
||||
expected.emplace_back(i);
|
||||
|
@ -54,7 +54,7 @@ namespace DB
|
||||
|
||||
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()
|
||||
@ -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
|
||||
/// 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)
|
||||
{
|
||||
int waitpid_res = HANDLE_EINTR(waitpid(pid, &status, WNOHANG));
|
||||
|
@ -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)
|
||||
{
|
||||
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)
|
||||
@ -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);
|
||||
}
|
||||
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
|
||||
|
@ -3,6 +3,7 @@
|
||||
|
||||
#include <base/types.h>
|
||||
#include <base/unaligned.h>
|
||||
#include <base/defines.h>
|
||||
|
||||
#include <Compression/CompressionFactory.h>
|
||||
#include "CompressedWriteBuffer.h"
|
||||
@ -22,7 +23,8 @@ void CompressedWriteBuffer::nextImpl()
|
||||
if (!offset())
|
||||
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);
|
||||
|
||||
/** During compression we need buffer with capacity >= compressed_reserve_size + CHECKSUM_SIZE.
|
||||
|
@ -353,7 +353,7 @@ UInt32 compressDataForType(const char * source, UInt32 source_size, char * dest)
|
||||
|
||||
writer.flush();
|
||||
|
||||
return (dest - dest_start) + (writer.count() + 7) / 8;
|
||||
return static_cast<UInt32>((dest - dest_start) + (writer.count() + 7) / 8);
|
||||
}
|
||||
|
||||
template <typename ValueType>
|
||||
@ -414,7 +414,7 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest,
|
||||
if (write_spec.data_bits != 0)
|
||||
{
|
||||
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)
|
||||
{
|
||||
/// It's well defined for unsigned data types.
|
||||
|
@ -7,6 +7,7 @@
|
||||
#include <Compression/CompressionCodecEncrypted.h>
|
||||
#include <Poco/Logger.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/safe_cast.h>
|
||||
|
||||
// This depends on BoringSSL-specific API, notably <openssl/aead.h>.
|
||||
#if USE_SSL
|
||||
@ -480,7 +481,8 @@ UInt32 CompressionCodecEncrypted::doCompressData(const char * source, UInt32 sou
|
||||
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);
|
||||
|
||||
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
|
||||
|
@ -453,9 +453,9 @@ UInt32 CompressionCodecFPC::doCompressData(const char * source, UInt32 source_si
|
||||
switch (float_width)
|
||||
{
|
||||
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):
|
||||
return HEADER_SIZE + FPCOperation<UInt32>(destination, level).encode(src);
|
||||
return static_cast<UInt32>(HEADER_SIZE + FPCOperation<UInt32>(destination, level).encode(src));
|
||||
default:
|
||||
break;
|
||||
}
|
||||
|
@ -259,7 +259,7 @@ UInt32 compressDataForType(const char * source, UInt32 source_size, char * dest,
|
||||
|
||||
writer.flush();
|
||||
|
||||
return (dest - dest_start) + (writer.count() + 7) / 8;
|
||||
return static_cast<UInt32>((dest - dest_start) + (writer.count() + 7) / 8);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
@ -320,7 +320,7 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest)
|
||||
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;
|
||||
curr_value = prev_value ^ xored_data;
|
||||
}
|
||||
|
@ -134,7 +134,7 @@ void registerCodecLZ4HC(CompressionCodecFactory & factory)
|
||||
if (!literal)
|
||||
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);
|
||||
|
@ -48,7 +48,7 @@ UInt32 CompressionCodecMultiple::getMaxCompressedDataSize(UInt32 uncompressed_si
|
||||
compressed_size = codec->getCompressedReserveSize(compressed_size);
|
||||
|
||||
/// 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
|
||||
@ -73,7 +73,7 @@ UInt32 CompressionCodecMultiple::doCompressData(const char * source, UInt32 sour
|
||||
|
||||
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
|
||||
|
@ -550,7 +550,7 @@ void decompressData(const char * src, UInt32 bytes_size, char * dst, UInt32 unco
|
||||
UInt32 num_bits = getValuableBitsNumber(min, max);
|
||||
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))
|
||||
unalignedStore<T>(dst, min_value);
|
||||
return;
|
||||
@ -572,14 +572,14 @@ void decompressData(const char * src, UInt32 bytes_size, char * dst, UInt32 unco
|
||||
T upper_max [[maybe_unused]] = 0;
|
||||
T sign_bit [[maybe_unused]] = 0;
|
||||
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 (min < 0 && max >= 0 && num_bits < 64)
|
||||
{
|
||||
sign_bit = 1ull << (num_bits - 1);
|
||||
upper_max = static_cast<UInt64>(max) >> num_bits << num_bits;
|
||||
sign_bit = static_cast<T>(1ull << (num_bits - 1));
|
||||
upper_max = static_cast<T>(static_cast<UInt64>(max) >> num_bits << num_bits);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -65,7 +65,7 @@ void CompressionCodecZSTD::updateHash(SipHash & hash) 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))
|
||||
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)
|
||||
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())
|
||||
throw Exception(
|
||||
"ZSTD codec can't have level more than " + toString(ZSTD_maxCLevel()) + ", given " + toString(level),
|
||||
ErrorCodes::ILLEGAL_CODEC_PARAMETER);
|
||||
{
|
||||
throw Exception(ErrorCodes::ILLEGAL_CODEC_PARAMETER,
|
||||
"ZSTD codec can't have level more than {}, given {}",
|
||||
ZSTD_maxCLevel(), level);
|
||||
}
|
||||
if (arguments->children.size() > 1)
|
||||
{
|
||||
const auto * window_literal = children[1]->as<ASTLiteral>();
|
||||
if (!window_literal)
|
||||
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);
|
||||
if (ZSTD_isError(window_log_bounds.error))
|
||||
|
@ -391,7 +391,7 @@ CodecTestSequence generateSeq(Generator gen, const char* gen_name, B Begin = 0,
|
||||
|
||||
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);
|
||||
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 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);
|
||||
|
||||
timer.start();
|
||||
|
||||
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");
|
||||
|
||||
encoded.resize(encoded_size);
|
||||
@ -478,7 +480,8 @@ void testTranscoding(Timer & timer, ICompressionCodec & codec, const CodecTestSe
|
||||
PODArray<char> decoded(source_data.size());
|
||||
|
||||
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");
|
||||
|
||||
decoded.resize(decoded_size);
|
||||
@ -542,10 +545,12 @@ TEST_P(CodecTestCompatibility, Encoding)
|
||||
const auto & source_data = data_sequence.serialized_data;
|
||||
|
||||
// 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);
|
||||
|
||||
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);
|
||||
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);
|
||||
|
||||
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);
|
||||
|
||||
ASSERT_TRUE(EqualByteContainers(expected.data_type->getSizeOfValueInMemory(), expected.serialized_data, decoded));
|
||||
|
@ -284,8 +284,9 @@ void KeeperServer::launchRaftServer(const Poco::Util::AbstractConfiguration & co
|
||||
params.client_req_timeout_
|
||||
= getValueOrMaxInt32AndLogWarning(coordination_settings->operation_timeout_ms.totalMilliseconds(), "operation_timeout_ms", log);
|
||||
params.auto_forwarding_ = coordination_settings->auto_forwarding;
|
||||
params.auto_forwarding_req_timeout_
|
||||
= std::max<uint64_t>(coordination_settings->operation_timeout_ms.totalMilliseconds() * 2, std::numeric_limits<int32_t>::max());
|
||||
params.auto_forwarding_req_timeout_ = std::max<int32_t>(
|
||||
static_cast<int32_t>(coordination_settings->operation_timeout_ms.totalMilliseconds() * 2),
|
||||
std::numeric_limits<int32_t>::max());
|
||||
params.auto_forwarding_req_timeout_
|
||||
= getValueOrMaxInt32AndLogWarning(coordination_settings->operation_timeout_ms.totalMilliseconds() * 2, "operation_timeout_ms", log);
|
||||
params.max_append_size_
|
||||
|
@ -922,7 +922,7 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr
|
||||
stat.version = 0;
|
||||
stat.aversion = 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;
|
||||
|
||||
new_deltas.emplace_back(
|
||||
@ -1222,7 +1222,7 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce
|
||||
value.stat.version++;
|
||||
value.stat.mzxid = zxid;
|
||||
value.stat.mtime = time;
|
||||
value.stat.dataLength = data.length();
|
||||
value.stat.dataLength = static_cast<UInt32>(data.length());
|
||||
value.setData(data);
|
||||
},
|
||||
request.version});
|
||||
|
@ -71,10 +71,10 @@ void SummingStateMachine::createSnapshotInternal(nuraft::snapshot & s)
|
||||
snapshots[s.get_last_log_idx()] = ctx;
|
||||
|
||||
// Maintain last 3 snapshots only.
|
||||
int num = snapshots.size();
|
||||
ssize_t num = snapshots.size();
|
||||
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())
|
||||
break;
|
||||
|
@ -119,7 +119,7 @@ int64_t deserializeStorageData(KeeperStorage & storage, ReadBuffer & in, Poco::L
|
||||
Coordination::read(node.stat.pzxid, in);
|
||||
if (!path.empty())
|
||||
{
|
||||
node.stat.dataLength = node.getData().length();
|
||||
node.stat.dataLength = static_cast<UInt32>(node.getData().length());
|
||||
node.seq_num = node.stat.cversion;
|
||||
storage.container.insertOrReplace(path, node);
|
||||
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user