More fixes for PVS-Studio

This commit is contained in:
Alexey Milovidov 2021-05-08 18:20:40 +03:00
parent 022667a190
commit e905883c75
37 changed files with 86 additions and 65 deletions

View File

@ -23,6 +23,8 @@ public:
SharedLibraryHandler(const SharedLibraryHandler & other);
SharedLibraryHandler & operator=(const SharedLibraryHandler & other) = delete;
~SharedLibraryHandler();
BlockInputStreamPtr loadAll();

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -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_)
{

View File

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

View File

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

View File

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

View File

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

View File

@ -79,7 +79,7 @@ struct StringHashTableHash
};
template <typename Cell>
struct StringHashTableEmpty
struct StringHashTableEmpty //-V730
{
using Self = StringHashTableEmpty;

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -73,7 +73,7 @@ public:
{
NodeRawConstPtrs children;
ActionType type;
ActionType type{};
std::string result_name;
DataTypePtr result_type;

View File

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

View File

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

View File

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

View File

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

View File

@ -17,7 +17,7 @@ class PollingQueue
public:
struct TaskData
{
size_t thread_num;
size_t thread_num = 0;
void * data = nullptr;
int fd = -1;

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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