Merge pull request #23877 from ClickHouse/pvs-studio-fixes-5

Fix some warnings by PVS-Studio
This commit is contained in:
alexey-milovidov 2021-05-04 22:57:19 +03:00 committed by GitHub
commit c607291042
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
8 changed files with 41 additions and 54 deletions

View File

@ -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

View File

@ -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);
} }

View File

@ -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);
} }

View File

@ -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;
} }

View File

@ -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));

View File

@ -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;
} }

View File

@ -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>;

View File

@ -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);
} }