mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
More fixes for PVS-Studio
This commit is contained in:
parent
022667a190
commit
e905883c75
@ -23,6 +23,8 @@ public:
|
||||
|
||||
SharedLibraryHandler(const SharedLibraryHandler & other);
|
||||
|
||||
SharedLibraryHandler & operator=(const SharedLibraryHandler & other) = delete;
|
||||
|
||||
~SharedLibraryHandler();
|
||||
|
||||
BlockInputStreamPtr loadAll();
|
||||
|
@ -45,7 +45,7 @@ struct Quota : public IAccessEntity
|
||||
|
||||
struct ResourceTypeInfo
|
||||
{
|
||||
const char * const raw_name;
|
||||
const char * const raw_name = "";
|
||||
const String name; /// Lowercased with underscores, e.g. "result_rows".
|
||||
const String keyword; /// Uppercased with spaces, e.g. "RESULT ROWS".
|
||||
const bool output_as_float = false;
|
||||
|
@ -220,7 +220,7 @@ private:
|
||||
}
|
||||
|
||||
public:
|
||||
AggregateFunctionHistogramData()
|
||||
AggregateFunctionHistogramData() //-V730
|
||||
: size(0)
|
||||
, lower_bound(std::numeric_limits<Mean>::max())
|
||||
, upper_bound(std::numeric_limits<Mean>::lowest())
|
||||
|
@ -181,7 +181,7 @@ public:
|
||||
/** For strings. Short strings are stored in the object itself, and long strings are allocated separately.
|
||||
* NOTE It could also be suitable for arrays of numbers.
|
||||
*/
|
||||
struct SingleValueDataString
|
||||
struct SingleValueDataString //-V730
|
||||
{
|
||||
private:
|
||||
using Self = SingleValueDataString;
|
||||
|
@ -159,12 +159,12 @@ private:
|
||||
/// The number of bytes read.
|
||||
size_t read_count = 0;
|
||||
/// The content in the current position.
|
||||
UInt8 value_l;
|
||||
UInt8 value_r;
|
||||
UInt8 value_l = 0;
|
||||
UInt8 value_r = 0;
|
||||
///
|
||||
bool is_eof = false;
|
||||
/// Does the cell fully fit into one byte?
|
||||
bool fits_in_byte;
|
||||
bool fits_in_byte = false;
|
||||
};
|
||||
|
||||
/** TODO This code looks very suboptimal.
|
||||
|
@ -29,7 +29,7 @@ namespace ErrorCodes
|
||||
struct Error
|
||||
{
|
||||
/// Number of times Exception with this ErrorCode had been throw.
|
||||
Value count;
|
||||
Value count = 0;
|
||||
/// Time of the last error.
|
||||
UInt64 error_time_ms = 0;
|
||||
/// Message for the last error.
|
||||
|
@ -44,7 +44,7 @@ struct ClearableHashTableCell : public BaseCell
|
||||
/// Do I need to store the zero key separately (that is, can a zero key be inserted into the hash table).
|
||||
static constexpr bool need_zero_value_storage = false;
|
||||
|
||||
ClearableHashTableCell() {}
|
||||
ClearableHashTableCell() {} //-V730
|
||||
ClearableHashTableCell(const Key & key_, const State & state) : BaseCell(key_, state), version(state.version) {}
|
||||
};
|
||||
|
||||
|
@ -13,7 +13,7 @@ struct FixedClearableHashTableCell
|
||||
using mapped_type = VoidMapped;
|
||||
UInt32 version;
|
||||
|
||||
FixedClearableHashTableCell() {}
|
||||
FixedClearableHashTableCell() {} //-V730
|
||||
FixedClearableHashTableCell(const Key &, const State & state) : version(state.version) {}
|
||||
|
||||
const VoidKey getKey() const { return {}; }
|
||||
|
@ -16,7 +16,7 @@ struct FixedHashMapCell
|
||||
bool full;
|
||||
Mapped mapped;
|
||||
|
||||
FixedHashMapCell() {}
|
||||
FixedHashMapCell() {} //-V730
|
||||
FixedHashMapCell(const Key &, const State &) : full(true) {}
|
||||
FixedHashMapCell(const value_type & value_, const State &) : full(true), mapped(value_.second) {}
|
||||
|
||||
@ -31,7 +31,7 @@ struct FixedHashMapCell
|
||||
/// Note that we have to assemble a continuous layout for the value_type on each call of getValue().
|
||||
struct CellExt
|
||||
{
|
||||
CellExt() {}
|
||||
CellExt() {} //-V730
|
||||
CellExt(Key && key_, const FixedHashMapCell * ptr_) : key(key_), ptr(const_cast<FixedHashMapCell *>(ptr_)) {}
|
||||
void update(Key && key_, const FixedHashMapCell * ptr_)
|
||||
{
|
||||
@ -76,7 +76,7 @@ struct FixedHashMapImplicitZeroCell
|
||||
/// Note that we have to assemble a continuous layout for the value_type on each call of getValue().
|
||||
struct CellExt
|
||||
{
|
||||
CellExt() {}
|
||||
CellExt() {} //-V730
|
||||
CellExt(Key && key_, const FixedHashMapImplicitZeroCell * ptr_) : key(key_), ptr(const_cast<FixedHashMapImplicitZeroCell *>(ptr_)) {}
|
||||
void update(Key && key_, const FixedHashMapImplicitZeroCell * ptr_)
|
||||
{
|
||||
|
@ -19,7 +19,7 @@ struct FixedHashTableCell
|
||||
using mapped_type = VoidMapped;
|
||||
bool full;
|
||||
|
||||
FixedHashTableCell() {}
|
||||
FixedHashTableCell() {} //-V730
|
||||
FixedHashTableCell(const Key &, const State &) : full(true) {}
|
||||
|
||||
const VoidKey getKey() const { return {}; }
|
||||
@ -267,7 +267,7 @@ public:
|
||||
DB::ReadBuffer & in;
|
||||
Cell cell;
|
||||
size_t read_count = 0;
|
||||
size_t size;
|
||||
size_t size = 0;
|
||||
bool is_eof = false;
|
||||
bool is_initialized = false;
|
||||
};
|
||||
|
@ -73,8 +73,8 @@ struct HashSetCellWithSavedHash : public HashTableCell<Key, Hash, TState>
|
||||
|
||||
size_t saved_hash;
|
||||
|
||||
HashSetCellWithSavedHash() : Base() {}
|
||||
HashSetCellWithSavedHash(const Key & key_, const typename Base::State & state) : Base(key_, state) {}
|
||||
HashSetCellWithSavedHash() : Base() {} //-V730
|
||||
HashSetCellWithSavedHash(const Key & key_, const typename Base::State & state) : Base(key_, state) {} //-V730
|
||||
|
||||
bool keyEquals(const Key & key_) const { return bitEquals(this->key, key_); }
|
||||
bool keyEquals(const Key & key_, size_t hash_) const { return saved_hash == hash_ && bitEquals(this->key, key_); }
|
||||
|
@ -305,7 +305,7 @@ template <bool need_zero_value_storage, typename Cell>
|
||||
struct ZeroValueStorage;
|
||||
|
||||
template <typename Cell>
|
||||
struct ZeroValueStorage<true, Cell>
|
||||
struct ZeroValueStorage<true, Cell> //-V308
|
||||
{
|
||||
private:
|
||||
bool has_zero = false;
|
||||
|
@ -80,7 +80,7 @@ public:
|
||||
{
|
||||
public:
|
||||
Reader(DB::ReadBuffer & in_)
|
||||
: in(in_)
|
||||
: in(in_)
|
||||
{
|
||||
}
|
||||
|
||||
@ -124,15 +124,15 @@ public:
|
||||
DB::ReadBuffer & in;
|
||||
Cell cell;
|
||||
size_t read_count = 0;
|
||||
size_t size;
|
||||
size_t size = 0;
|
||||
bool is_eof = false;
|
||||
bool is_initialized = false;
|
||||
};
|
||||
|
||||
class iterator
|
||||
{
|
||||
Self * container;
|
||||
Cell * ptr;
|
||||
Self * container = nullptr;
|
||||
Cell * ptr = nullptr;
|
||||
|
||||
friend class SmallTable;
|
||||
|
||||
@ -158,8 +158,8 @@ public:
|
||||
|
||||
class const_iterator
|
||||
{
|
||||
const Self * container;
|
||||
const Cell * ptr;
|
||||
const Self * container = nullptr;
|
||||
const Cell * ptr = nullptr;
|
||||
|
||||
friend class SmallTable;
|
||||
|
||||
@ -184,16 +184,16 @@ public:
|
||||
};
|
||||
|
||||
|
||||
const_iterator begin() const { return iteratorTo(buf); }
|
||||
iterator begin() { return iteratorTo(buf); }
|
||||
const_iterator begin() const { return iteratorTo(buf); }
|
||||
iterator begin() { return iteratorTo(buf); }
|
||||
|
||||
const_iterator end() const { return iteratorTo(buf + m_size); }
|
||||
iterator end() { return iteratorTo(buf + m_size); }
|
||||
const_iterator end() const { return iteratorTo(buf + m_size); }
|
||||
iterator end() { return iteratorTo(buf + m_size); }
|
||||
|
||||
|
||||
protected:
|
||||
const_iterator iteratorTo(const Cell * ptr) const { return const_iterator(this, ptr); }
|
||||
iterator iteratorTo(Cell * ptr) { return iterator(this, ptr); }
|
||||
const_iterator iteratorTo(const Cell * ptr) const { return const_iterator(this, ptr); }
|
||||
iterator iteratorTo(Cell * ptr) { return iterator(this, ptr); }
|
||||
|
||||
|
||||
public:
|
||||
|
@ -79,7 +79,7 @@ struct StringHashTableHash
|
||||
};
|
||||
|
||||
template <typename Cell>
|
||||
struct StringHashTableEmpty
|
||||
struct StringHashTableEmpty //-V730
|
||||
{
|
||||
using Self = StringHashTableEmpty;
|
||||
|
||||
|
@ -119,9 +119,9 @@ public:
|
||||
|
||||
class iterator
|
||||
{
|
||||
Self * container;
|
||||
size_t bucket;
|
||||
typename Impl::iterator current_it;
|
||||
Self * container{};
|
||||
size_t bucket{};
|
||||
typename Impl::iterator current_it{};
|
||||
|
||||
friend class TwoLevelHashTable;
|
||||
|
||||
@ -156,9 +156,9 @@ public:
|
||||
|
||||
class const_iterator
|
||||
{
|
||||
Self * container;
|
||||
size_t bucket;
|
||||
typename Impl::const_iterator current_it;
|
||||
Self * container{};
|
||||
size_t bucket{};
|
||||
typename Impl::const_iterator current_it{};
|
||||
|
||||
friend class TwoLevelHashTable;
|
||||
|
||||
|
@ -208,7 +208,7 @@ public:
|
||||
|
||||
static bool isBlocked(VariableContext current_level, bool fault_injection)
|
||||
{
|
||||
return counter > 0 && current_level >= level && (!fault_injection || (fault_injection && block_fault_injections));
|
||||
return counter > 0 && current_level >= level && (!fault_injection || block_fault_injections);
|
||||
}
|
||||
};
|
||||
};
|
||||
|
@ -44,7 +44,7 @@ using SharedBlockPtr = boost::intrusive_ptr<detail::SharedBlock>;
|
||||
struct SharedBlockRowRef
|
||||
{
|
||||
ColumnRawPtrs * columns = nullptr;
|
||||
size_t row_num;
|
||||
size_t row_num = 0;
|
||||
SharedBlockPtr shared_block;
|
||||
|
||||
void swap(SharedBlockRowRef & other)
|
||||
|
@ -71,7 +71,7 @@ public:
|
||||
LogsLevel client_logs_level = LogsLevel::none;
|
||||
|
||||
String query;
|
||||
UInt64 normalized_query_hash;
|
||||
UInt64 normalized_query_hash = 0;
|
||||
};
|
||||
|
||||
using ThreadGroupStatusPtr = std::shared_ptr<ThreadGroupStatus>;
|
||||
|
@ -25,7 +25,7 @@ namespace Coordination
|
||||
struct ZooKeeperResponse : virtual Response
|
||||
{
|
||||
XID xid = 0;
|
||||
int64_t zxid;
|
||||
int64_t zxid = 0;
|
||||
|
||||
virtual ~ZooKeeperResponse() override = default;
|
||||
virtual void readImpl(ReadBuffer &) = 0;
|
||||
|
@ -30,10 +30,10 @@ static constexpr auto CURRENT_CHANGELOG_VERSION = ChangelogVersion::V0;
|
||||
struct ChangelogRecordHeader
|
||||
{
|
||||
ChangelogVersion version = CURRENT_CHANGELOG_VERSION;
|
||||
uint64_t index; /// entry log number
|
||||
uint64_t term;
|
||||
uint64_t index = 0; /// entry log number
|
||||
uint64_t term = 0;
|
||||
nuraft::log_val_type value_type;
|
||||
uint64_t blob_size;
|
||||
uint64_t blob_size = 0;
|
||||
};
|
||||
|
||||
/// Changelog record on disk
|
||||
|
@ -257,6 +257,7 @@ class FirstMessage : public FrontMessage
|
||||
{
|
||||
public:
|
||||
Int32 payload_size;
|
||||
|
||||
FirstMessage() = delete;
|
||||
FirstMessage(int payload_size_) : payload_size(payload_size_) {}
|
||||
};
|
||||
@ -264,8 +265,9 @@ public:
|
||||
class CancelRequest : public FirstMessage
|
||||
{
|
||||
public:
|
||||
Int32 process_id;
|
||||
Int32 secret_key;
|
||||
Int32 process_id = 0;
|
||||
Int32 secret_key = 0;
|
||||
|
||||
CancelRequest(int payload_size_) : FirstMessage(payload_size_) {}
|
||||
|
||||
void deserialize(ReadBuffer & in) override
|
||||
|
@ -126,7 +126,7 @@ struct SortCursorImpl
|
||||
|
||||
/// Prevent using pos instead of getRow()
|
||||
private:
|
||||
size_t pos;
|
||||
size_t pos = 0;
|
||||
};
|
||||
|
||||
using SortCursorImpls = std::vector<SortCursorImpl>;
|
||||
|
@ -37,7 +37,7 @@ static auto typesFromString(const std::string & str)
|
||||
|
||||
struct TypesTestCase
|
||||
{
|
||||
const char * from_types;
|
||||
const char * from_types = nullptr;
|
||||
const char * expected_type = nullptr;
|
||||
};
|
||||
|
||||
|
@ -85,7 +85,7 @@ public:
|
||||
|
||||
auto result_column = ColumnUInt8::create();
|
||||
|
||||
auto call = [&](const auto & types) -> bool
|
||||
auto call = [&](const auto & types) -> bool //-V567
|
||||
{
|
||||
using Types = std::decay_t<decltype(types)>;
|
||||
using Type = typename Types::RightType;
|
||||
|
@ -73,7 +73,7 @@ public:
|
||||
{
|
||||
NodeRawConstPtrs children;
|
||||
|
||||
ActionType type;
|
||||
ActionType type{};
|
||||
|
||||
std::string result_name;
|
||||
DataTypePtr result_type;
|
||||
|
@ -126,8 +126,8 @@ private:
|
||||
String cluster_name;
|
||||
ClusterPtr cluster;
|
||||
Cluster::Address address_in_cluster;
|
||||
size_t host_shard_num;
|
||||
size_t host_replica_num;
|
||||
size_t host_shard_num = 0;
|
||||
size_t host_replica_num = 0;
|
||||
};
|
||||
|
||||
struct DatabaseReplicatedTask : public DDLTaskBase
|
||||
|
@ -234,7 +234,7 @@ private:
|
||||
StorageID table_id = StorageID::createEmpty();
|
||||
StoragePtr table;
|
||||
String metadata_path;
|
||||
time_t drop_time;
|
||||
time_t drop_time{};
|
||||
};
|
||||
using TablesMarkedAsDropped = std::list<TableMarkedAsDropped>;
|
||||
|
||||
|
@ -10,10 +10,10 @@ struct TextLogElement
|
||||
{
|
||||
time_t event_time{};
|
||||
Decimal64 event_time_microseconds{};
|
||||
UInt32 microseconds;
|
||||
UInt32 microseconds{};
|
||||
|
||||
String thread_name;
|
||||
UInt64 thread_id;
|
||||
UInt64 thread_id{};
|
||||
|
||||
Message::Priority level = Message::PRIO_TRACE;
|
||||
|
||||
@ -22,7 +22,7 @@ struct TextLogElement
|
||||
String message;
|
||||
|
||||
String source_file;
|
||||
UInt64 source_line;
|
||||
UInt64 source_line{};
|
||||
|
||||
static std::string name() { return "TextLog"; }
|
||||
static Block createBlock();
|
||||
|
@ -32,7 +32,7 @@ bool isAlreadySorted(const Block & block, const SortDescription & description);
|
||||
/// Column with description for sort
|
||||
struct ColumnWithSortDescription
|
||||
{
|
||||
const IColumn * column;
|
||||
const IColumn * column = nullptr;
|
||||
SortColumnDescription description;
|
||||
|
||||
/// It means, that this column is ColumnConst
|
||||
|
@ -17,7 +17,7 @@ class PollingQueue
|
||||
public:
|
||||
struct TaskData
|
||||
{
|
||||
size_t thread_num;
|
||||
size_t thread_num = 0;
|
||||
|
||||
void * data = nullptr;
|
||||
int fd = -1;
|
||||
|
@ -784,8 +784,10 @@ String alterTypeToString(const AlterCommand::Type type)
|
||||
return "RENAME COLUMN";
|
||||
case AlterCommand::Type::REMOVE_TTL:
|
||||
return "REMOVE TTL";
|
||||
default:
|
||||
throw Exception("Uninitialized ALTER command", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
__builtin_unreachable();
|
||||
|
||||
}
|
||||
|
||||
void AlterCommands::apply(StorageInMemoryMetadata & metadata, ContextPtr context) const
|
||||
|
@ -23,6 +23,7 @@ struct AlterCommand
|
||||
|
||||
enum Type
|
||||
{
|
||||
UNKNOWN,
|
||||
ADD_COLUMN,
|
||||
DROP_COLUMN,
|
||||
MODIFY_COLUMN,
|
||||
@ -55,7 +56,7 @@ struct AlterCommand
|
||||
TTL
|
||||
};
|
||||
|
||||
Type type;
|
||||
Type type = UNKNOWN;
|
||||
|
||||
String column_name;
|
||||
|
||||
|
@ -31,16 +31,16 @@ public:
|
||||
struct Part
|
||||
{
|
||||
/// Size of data part in bytes.
|
||||
size_t size;
|
||||
size_t size = 0;
|
||||
|
||||
/// How old this data part in seconds.
|
||||
time_t age;
|
||||
time_t age = 0;
|
||||
|
||||
/// Depth of tree of merges by which this part was created. New parts has zero level.
|
||||
unsigned level;
|
||||
unsigned level = 0;
|
||||
|
||||
/// Opaque pointer to avoid dependencies (it is not possible to do forward declaration of typedef).
|
||||
const void * data;
|
||||
const void * data = nullptr;
|
||||
|
||||
/// Information about different TTLs for part. Can be used by
|
||||
/// TTLSelector to assign merges with TTL.
|
||||
|
@ -3022,6 +3022,9 @@ Pipe MergeTreeData::alterPartition(
|
||||
}
|
||||
|
||||
break;
|
||||
|
||||
default:
|
||||
throw Exception("Uninitialized partition command", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
for (auto & command_result : current_command_results)
|
||||
command_result.command_type = command.typeToString();
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Common/interpolate.h>
|
||||
|
||||
#include <cmath>
|
||||
#include <cassert>
|
||||
#include <iostream>
|
||||
|
||||
|
||||
@ -168,6 +169,7 @@ void selectWithinPartition(
|
||||
|
||||
for (size_t end = begin + 2; end <= parts_count; ++end)
|
||||
{
|
||||
assert(end > begin);
|
||||
if (settings.max_parts_to_merge_at_once && end - begin > settings.max_parts_to_merge_at_once)
|
||||
break;
|
||||
|
||||
|
@ -13,6 +13,12 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
|
||||
std::optional<PartitionCommand> PartitionCommand::parse(const ASTAlterCommand * command_ast)
|
||||
{
|
||||
if (command_ast->type == ASTAlterCommand::DROP_PARTITION)
|
||||
@ -155,8 +161,9 @@ std::string PartitionCommand::typeToString() const
|
||||
return "UNFREEZE ALL";
|
||||
case PartitionCommand::Type::REPLACE_PARTITION:
|
||||
return "REPLACE PARTITION";
|
||||
default:
|
||||
throw Exception("Uninitialized partition command", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
__builtin_unreachable();
|
||||
}
|
||||
|
||||
Pipe convertCommandsResultToSource(const PartitionCommandsResultInfo & commands_result)
|
||||
|
@ -20,6 +20,8 @@ struct PartitionCommand
|
||||
{
|
||||
enum Type
|
||||
{
|
||||
UNKNOWN,
|
||||
|
||||
ATTACH_PARTITION,
|
||||
MOVE_PARTITION,
|
||||
DROP_PARTITION,
|
||||
@ -32,7 +34,7 @@ struct PartitionCommand
|
||||
REPLACE_PARTITION,
|
||||
};
|
||||
|
||||
Type type;
|
||||
Type type = UNKNOWN;
|
||||
|
||||
ASTPtr partition;
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user