mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 00:52:02 +00:00
Merge pull request #23877 from ClickHouse/pvs-studio-fixes-5
Fix some warnings by PVS-Studio
This commit is contained in:
commit
c607291042
@ -50,7 +50,7 @@ struct QueryFuzzer
|
|||||||
// Some debug fields for detecting problematic ASTs with loops.
|
// Some debug fields for detecting problematic ASTs with loops.
|
||||||
// These are reset for each fuzzMain call.
|
// These are reset for each fuzzMain call.
|
||||||
std::unordered_set<const IAST *> debug_visited_nodes;
|
std::unordered_set<const IAST *> debug_visited_nodes;
|
||||||
ASTPtr * debug_top_ast;
|
ASTPtr * debug_top_ast = nullptr;
|
||||||
|
|
||||||
|
|
||||||
// This is the only function you have to call -- it will modify the passed
|
// This is the only function you have to call -- it will modify the passed
|
||||||
|
@ -1,6 +1,5 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
|
|
||||||
#include <Common/HashTable/HashTable.h>
|
#include <Common/HashTable/HashTable.h>
|
||||||
#include <Common/HashTable/HashTableKeyHolder.h>
|
#include <Common/HashTable/HashTableKeyHolder.h>
|
||||||
#include <Common/ColumnsHashingImpl.h>
|
#include <Common/ColumnsHashingImpl.h>
|
||||||
@ -15,6 +14,8 @@
|
|||||||
|
|
||||||
#include <Core/Defines.h>
|
#include <Core/Defines.h>
|
||||||
#include <memory>
|
#include <memory>
|
||||||
|
#include <cassert>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
@ -594,8 +595,11 @@ struct HashMethodKeysFixed
|
|||||||
return prepared_keys[row];
|
return prepared_keys[row];
|
||||||
|
|
||||||
#if defined(__SSSE3__) && !defined(MEMORY_SANITIZER)
|
#if defined(__SSSE3__) && !defined(MEMORY_SANITIZER)
|
||||||
if constexpr (!has_low_cardinality && !has_nullable_keys && sizeof(Key) <= 16)
|
if constexpr (sizeof(Key) <= 16)
|
||||||
|
{
|
||||||
|
assert(!has_low_cardinality && !has_nullable_keys);
|
||||||
return packFixedShuffle<Key>(columns_data.get(), keys_size, key_sizes.data(), row, masks.get());
|
return packFixedShuffle<Key>(columns_data.get(), keys_size, key_sizes.data(), row, masks.get());
|
||||||
|
}
|
||||||
#endif
|
#endif
|
||||||
return packFixed<Key>(row, keys_size, Base::getActualColumns(), key_sizes);
|
return packFixed<Key>(row, keys_size, Base::getActualColumns(), key_sizes);
|
||||||
}
|
}
|
||||||
|
@ -91,49 +91,38 @@ inline UInt32 updateWeakHash32(const DB::UInt8 * pos, size_t size, DB::UInt32 up
|
|||||||
{
|
{
|
||||||
if (size < 8)
|
if (size < 8)
|
||||||
{
|
{
|
||||||
DB::UInt64 value = 0;
|
UInt64 value = 0;
|
||||||
auto * value_ptr = reinterpret_cast<unsigned char *>(&value);
|
|
||||||
|
|
||||||
typedef __attribute__((__aligned__(1))) uint16_t uint16_unaligned_t;
|
|
||||||
typedef __attribute__((__aligned__(1))) uint32_t uint32_unaligned_t;
|
|
||||||
|
|
||||||
/// Adopted code from FastMemcpy.h (memcpy_tiny)
|
|
||||||
switch (size)
|
switch (size)
|
||||||
{
|
{
|
||||||
case 0:
|
case 0:
|
||||||
break;
|
break;
|
||||||
case 1:
|
case 1:
|
||||||
value_ptr[0] = pos[0];
|
__builtin_memcpy(&value, pos, 1);
|
||||||
break;
|
break;
|
||||||
case 2:
|
case 2:
|
||||||
*reinterpret_cast<uint16_t *>(value_ptr) = *reinterpret_cast<const uint16_unaligned_t *>(pos);
|
__builtin_memcpy(&value, pos, 2);
|
||||||
break;
|
break;
|
||||||
case 3:
|
case 3:
|
||||||
*reinterpret_cast<uint16_t *>(value_ptr) = *reinterpret_cast<const uint16_unaligned_t *>(pos);
|
__builtin_memcpy(&value, pos, 3);
|
||||||
value_ptr[2] = pos[2];
|
|
||||||
break;
|
break;
|
||||||
case 4:
|
case 4:
|
||||||
*reinterpret_cast<uint32_t *>(value_ptr) = *reinterpret_cast<const uint32_unaligned_t *>(pos);
|
__builtin_memcpy(&value, pos, 4);
|
||||||
break;
|
break;
|
||||||
case 5:
|
case 5:
|
||||||
*reinterpret_cast<uint32_t *>(value_ptr) = *reinterpret_cast<const uint32_unaligned_t *>(pos);
|
__builtin_memcpy(&value, pos, 5);
|
||||||
value_ptr[4] = pos[4];
|
|
||||||
break;
|
break;
|
||||||
case 6:
|
case 6:
|
||||||
*reinterpret_cast<uint32_t *>(value_ptr) = *reinterpret_cast<const uint32_unaligned_t *>(pos);
|
__builtin_memcpy(&value, pos, 6);
|
||||||
*reinterpret_cast<uint16_unaligned_t *>(value_ptr + 4) =
|
|
||||||
*reinterpret_cast<const uint16_unaligned_t *>(pos + 4);
|
|
||||||
break;
|
break;
|
||||||
case 7:
|
case 7:
|
||||||
*reinterpret_cast<uint32_t *>(value_ptr) = *reinterpret_cast<const uint32_unaligned_t *>(pos);
|
__builtin_memcpy(&value, pos, 7);
|
||||||
*reinterpret_cast<uint32_unaligned_t *>(value_ptr + 3) =
|
|
||||||
*reinterpret_cast<const uint32_unaligned_t *>(pos + 3);
|
|
||||||
break;
|
break;
|
||||||
default:
|
default:
|
||||||
__builtin_unreachable();
|
__builtin_unreachable();
|
||||||
}
|
}
|
||||||
|
|
||||||
value_ptr[7] = size;
|
reinterpret_cast<unsigned char *>(&value)[7] = size;
|
||||||
return intHashCRC32(value, updated_value);
|
return intHashCRC32(value, updated_value);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -85,7 +85,7 @@ public:
|
|||||||
|
|
||||||
struct Counter
|
struct Counter
|
||||||
{
|
{
|
||||||
Counter() {}
|
Counter() = default; //-V730
|
||||||
|
|
||||||
Counter(const TKey & k, UInt64 c = 0, UInt64 e = 0, size_t h = 0)
|
Counter(const TKey & k, UInt64 c = 0, UInt64 e = 0, size_t h = 0)
|
||||||
: key(k), slot(0), hash(h), count(c), error(e) {}
|
: key(k), slot(0), hash(h), count(c), error(e) {}
|
||||||
@ -148,7 +148,7 @@ public:
|
|||||||
// Increase weight of a key that already exists
|
// Increase weight of a key that already exists
|
||||||
auto hash = counter_map.hash(key);
|
auto hash = counter_map.hash(key);
|
||||||
|
|
||||||
if (auto counter = findCounter(key, hash); counter)
|
if (auto * counter = findCounter(key, hash); counter)
|
||||||
{
|
{
|
||||||
counter->count += increment;
|
counter->count += increment;
|
||||||
counter->error += error;
|
counter->error += error;
|
||||||
@ -159,12 +159,12 @@ public:
|
|||||||
// Key doesn't exist, but can fit in the top K
|
// Key doesn't exist, but can fit in the top K
|
||||||
if (unlikely(size() < capacity()))
|
if (unlikely(size() < capacity()))
|
||||||
{
|
{
|
||||||
auto c = new Counter(arena.emplace(key), increment, error, hash);
|
auto * c = new Counter(arena.emplace(key), increment, error, hash);
|
||||||
push(c);
|
push(c);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
auto min = counter_list.back();
|
auto * min = counter_list.back();
|
||||||
// The key doesn't exist and cannot fit in the current top K, but
|
// The key doesn't exist and cannot fit in the current top K, but
|
||||||
// the new key has a bigger weight and is virtually more present
|
// the new key has a bigger weight and is virtually more present
|
||||||
// compared to the element who is less present on the set. This part
|
// compared to the element who is less present on the set. This part
|
||||||
@ -218,7 +218,7 @@ public:
|
|||||||
*/
|
*/
|
||||||
if (m2 > 0)
|
if (m2 > 0)
|
||||||
{
|
{
|
||||||
for (auto counter : counter_list)
|
for (auto * counter : counter_list)
|
||||||
{
|
{
|
||||||
counter->count += m2;
|
counter->count += m2;
|
||||||
counter->error += m2;
|
counter->error += m2;
|
||||||
@ -226,10 +226,10 @@ public:
|
|||||||
}
|
}
|
||||||
|
|
||||||
// The list is sorted in descending order, we have to scan in reverse
|
// The list is sorted in descending order, we have to scan in reverse
|
||||||
for (auto counter : boost::adaptors::reverse(rhs.counter_list))
|
for (auto * counter : boost::adaptors::reverse(rhs.counter_list))
|
||||||
{
|
{
|
||||||
size_t hash = counter_map.hash(counter->key);
|
size_t hash = counter_map.hash(counter->key);
|
||||||
if (auto current = findCounter(counter->key, hash))
|
if (auto * current = findCounter(counter->key, hash))
|
||||||
{
|
{
|
||||||
// Subtract m2 previously added, guaranteed not negative
|
// Subtract m2 previously added, guaranteed not negative
|
||||||
current->count += (counter->count - m2);
|
current->count += (counter->count - m2);
|
||||||
@ -262,7 +262,7 @@ public:
|
|||||||
std::vector<Counter> topK(size_t k) const
|
std::vector<Counter> topK(size_t k) const
|
||||||
{
|
{
|
||||||
std::vector<Counter> res;
|
std::vector<Counter> res;
|
||||||
for (auto counter : counter_list)
|
for (auto * counter : counter_list)
|
||||||
{
|
{
|
||||||
res.push_back(*counter);
|
res.push_back(*counter);
|
||||||
if (res.size() == k)
|
if (res.size() == k)
|
||||||
@ -274,7 +274,7 @@ public:
|
|||||||
void write(WriteBuffer & wb) const
|
void write(WriteBuffer & wb) const
|
||||||
{
|
{
|
||||||
writeVarUInt(size(), wb);
|
writeVarUInt(size(), wb);
|
||||||
for (auto counter : counter_list)
|
for (auto * counter : counter_list)
|
||||||
counter->write(wb);
|
counter->write(wb);
|
||||||
|
|
||||||
writeVarUInt(alpha_map.size(), wb);
|
writeVarUInt(alpha_map.size(), wb);
|
||||||
@ -290,7 +290,7 @@ public:
|
|||||||
|
|
||||||
for (size_t i = 0; i < count; ++i)
|
for (size_t i = 0; i < count; ++i)
|
||||||
{
|
{
|
||||||
auto counter = new Counter();
|
auto * counter = new Counter();
|
||||||
counter->read(rb);
|
counter->read(rb);
|
||||||
counter->hash = counter_map.hash(counter->key);
|
counter->hash = counter_map.hash(counter->key);
|
||||||
push(counter);
|
push(counter);
|
||||||
@ -325,7 +325,7 @@ protected:
|
|||||||
{
|
{
|
||||||
while (counter->slot > 0)
|
while (counter->slot > 0)
|
||||||
{
|
{
|
||||||
auto next = counter_list[counter->slot - 1];
|
auto * next = counter_list[counter->slot - 1];
|
||||||
if (*counter > *next)
|
if (*counter > *next)
|
||||||
{
|
{
|
||||||
std::swap(next->slot, counter->slot);
|
std::swap(next->slot, counter->slot);
|
||||||
@ -339,7 +339,7 @@ protected:
|
|||||||
private:
|
private:
|
||||||
void destroyElements()
|
void destroyElements()
|
||||||
{
|
{
|
||||||
for (auto counter : counter_list)
|
for (auto * counter : counter_list)
|
||||||
{
|
{
|
||||||
arena.free(counter->key);
|
arena.free(counter->key);
|
||||||
delete counter;
|
delete counter;
|
||||||
@ -376,7 +376,7 @@ private:
|
|||||||
{
|
{
|
||||||
removed_keys = 0;
|
removed_keys = 0;
|
||||||
counter_map.clear();
|
counter_map.clear();
|
||||||
for (auto counter : counter_list)
|
for (auto * counter : counter_list)
|
||||||
counter_map[counter->key] = counter;
|
counter_map[counter->key] = counter;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -310,7 +310,7 @@ public:
|
|||||||
template <typename T, typename Z = void *>
|
template <typename T, typename Z = void *>
|
||||||
using enable_if_not_field_or_stringlike_t = std::enable_if_t<!std::is_same_v<std::decay_t<T>, Field> && !std::is_same_v<NearestFieldType<std::decay_t<T>>, String>, Z>;
|
using enable_if_not_field_or_stringlike_t = std::enable_if_t<!std::is_same_v<std::decay_t<T>, Field> && !std::is_same_v<NearestFieldType<std::decay_t<T>>, String>, Z>;
|
||||||
|
|
||||||
Field()
|
Field() //-V730
|
||||||
: which(Types::Null)
|
: which(Types::Null)
|
||||||
{
|
{
|
||||||
}
|
}
|
||||||
@ -851,7 +851,7 @@ decltype(auto) castToNearestFieldType(T && x)
|
|||||||
}
|
}
|
||||||
|
|
||||||
template <typename T>
|
template <typename T>
|
||||||
Field::Field(T && rhs, enable_if_not_field_or_stringlike_t<T>)
|
Field::Field(T && rhs, enable_if_not_field_or_stringlike_t<T>) //-V730
|
||||||
{
|
{
|
||||||
auto && val = castToNearestFieldType(std::forward<T>(rhs));
|
auto && val = castToNearestFieldType(std::forward<T>(rhs));
|
||||||
createConcrete(std::forward<decltype(val)>(val));
|
createConcrete(std::forward<decltype(val)>(val));
|
||||||
|
@ -137,7 +137,7 @@ ColumnPtr wrapInNullable(const ColumnPtr & src, const ColumnsWithTypeAndName & a
|
|||||||
if (const auto * nullable = checkAndGetColumn<ColumnNullable>(*elem.column))
|
if (const auto * nullable = checkAndGetColumn<ColumnNullable>(*elem.column))
|
||||||
{
|
{
|
||||||
const ColumnPtr & null_map_column = nullable->getNullMapColumnPtr();
|
const ColumnPtr & null_map_column = nullable->getNullMapColumnPtr();
|
||||||
if (!result_null_map_column)
|
if (!result_null_map_column) //-V1051
|
||||||
{
|
{
|
||||||
result_null_map_column = null_map_column;
|
result_null_map_column = null_map_column;
|
||||||
}
|
}
|
||||||
|
@ -1043,12 +1043,12 @@ private:
|
|||||||
*/
|
*/
|
||||||
struct AggregateFunctionInstruction
|
struct AggregateFunctionInstruction
|
||||||
{
|
{
|
||||||
const IAggregateFunction * that;
|
const IAggregateFunction * that{};
|
||||||
size_t state_offset;
|
size_t state_offset{};
|
||||||
const IColumn ** arguments;
|
const IColumn ** arguments{};
|
||||||
const IAggregateFunction * batch_that;
|
const IAggregateFunction * batch_that{};
|
||||||
const IColumn ** batch_arguments;
|
const IColumn ** batch_arguments{};
|
||||||
const UInt64 * offsets = nullptr;
|
const UInt64 * offsets{};
|
||||||
};
|
};
|
||||||
|
|
||||||
using AggregateFunctionInstructions = std::vector<AggregateFunctionInstruction>;
|
using AggregateFunctionInstructions = std::vector<AggregateFunctionInstruction>;
|
||||||
|
@ -3128,7 +3128,6 @@ StorageReplicatedMergeTree::CreateMergeEntryResult StorageReplicatedMergeTree::c
|
|||||||
entry.merge_type = merge_type;
|
entry.merge_type = merge_type;
|
||||||
entry.deduplicate = deduplicate;
|
entry.deduplicate = deduplicate;
|
||||||
entry.deduplicate_by_columns = deduplicate_by_columns;
|
entry.deduplicate_by_columns = deduplicate_by_columns;
|
||||||
entry.merge_type = merge_type;
|
|
||||||
entry.create_time = time(nullptr);
|
entry.create_time = time(nullptr);
|
||||||
|
|
||||||
for (const auto & part : parts)
|
for (const auto & part : parts)
|
||||||
@ -5215,11 +5214,7 @@ void StorageReplicatedMergeTree::getStatus(Status & res, bool with_zk_fields)
|
|||||||
{
|
{
|
||||||
auto log_entries = zookeeper->getChildren(zookeeper_path + "/log");
|
auto log_entries = zookeeper->getChildren(zookeeper_path + "/log");
|
||||||
|
|
||||||
if (log_entries.empty())
|
if (!log_entries.empty())
|
||||||
{
|
|
||||||
res.log_max_index = 0;
|
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
{
|
||||||
const String & last_log_entry = *std::max_element(log_entries.begin(), log_entries.end());
|
const String & last_log_entry = *std::max_element(log_entries.begin(), log_entries.end());
|
||||||
res.log_max_index = parse<UInt64>(last_log_entry.substr(strlen("log-")));
|
res.log_max_index = parse<UInt64>(last_log_entry.substr(strlen("log-")));
|
||||||
@ -5231,7 +5226,6 @@ void StorageReplicatedMergeTree::getStatus(Status & res, bool with_zk_fields)
|
|||||||
auto all_replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
|
auto all_replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
|
||||||
res.total_replicas = all_replicas.size();
|
res.total_replicas = all_replicas.size();
|
||||||
|
|
||||||
res.active_replicas = 0;
|
|
||||||
for (const String & replica : all_replicas)
|
for (const String & replica : all_replicas)
|
||||||
if (zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active"))
|
if (zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active"))
|
||||||
++res.active_replicas;
|
++res.active_replicas;
|
||||||
@ -5365,7 +5359,7 @@ void StorageReplicatedMergeTree::fetchPartition(
|
|||||||
ContextPtr query_context)
|
ContextPtr query_context)
|
||||||
{
|
{
|
||||||
Macros::MacroExpansionInfo info;
|
Macros::MacroExpansionInfo info;
|
||||||
info.expand_special_macros_only = false;
|
info.expand_special_macros_only = false; //-V1048
|
||||||
info.table_id = getStorageID();
|
info.table_id = getStorageID();
|
||||||
info.table_id.uuid = UUIDHelpers::Nil;
|
info.table_id.uuid = UUIDHelpers::Nil;
|
||||||
auto expand_from = query_context->getMacros()->expand(from_, info);
|
auto expand_from = query_context->getMacros()->expand(from_, info);
|
||||||
@ -6317,7 +6311,7 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta
|
|||||||
entry_delete.type = LogEntry::DROP_RANGE;
|
entry_delete.type = LogEntry::DROP_RANGE;
|
||||||
entry_delete.source_replica = replica_name;
|
entry_delete.source_replica = replica_name;
|
||||||
entry_delete.new_part_name = drop_range_fake_part_name;
|
entry_delete.new_part_name = drop_range_fake_part_name;
|
||||||
entry_delete.detach = false;
|
entry_delete.detach = false; //-V1048
|
||||||
entry_delete.create_time = time(nullptr);
|
entry_delete.create_time = time(nullptr);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
Loading…
Reference in New Issue
Block a user