mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 16:42:05 +00:00
Fixing more headers
This commit is contained in:
parent
c9cbe36b1f
commit
4ee1aa8c7c
@ -25,21 +25,21 @@ public:
|
||||
static const uint32_t bits = 128;
|
||||
|
||||
// Constructor initializes the same as Initialize()
|
||||
MetroHash128(const uint64_t seed=0);
|
||||
|
||||
explicit MetroHash128(const uint64_t seed=0);
|
||||
|
||||
// Initializes internal state for new hash with optional seed
|
||||
void Initialize(const uint64_t seed=0);
|
||||
|
||||
|
||||
// Update the hash state with a string of bytes. If the length
|
||||
// is sufficiently long, the implementation switches to a bulk
|
||||
// hashing algorithm directly on the argument buffer for speed.
|
||||
void Update(const uint8_t * buffer, const uint64_t length);
|
||||
|
||||
|
||||
// Constructs the final hash and writes it to the argument buffer.
|
||||
// After a hash is finalized, this instance must be Initialized()-ed
|
||||
// again or the behavior of Update() and Finalize() is undefined.
|
||||
void Finalize(uint8_t * const hash);
|
||||
|
||||
|
||||
// A non-incremental function implementation. This can be significantly
|
||||
// faster than the incremental implementation for some usage patterns.
|
||||
static void Hash(const uint8_t * buffer, const uint64_t length, uint8_t * const hash, const uint64_t seed=0);
|
||||
@ -57,7 +57,7 @@ private:
|
||||
static const uint64_t k1 = 0x8648DBDB;
|
||||
static const uint64_t k2 = 0x7BDEC03B;
|
||||
static const uint64_t k3 = 0x2F5870A5;
|
||||
|
||||
|
||||
struct { uint64_t v[4]; } state;
|
||||
struct { uint8_t b[32]; } input;
|
||||
uint64_t bytes;
|
||||
|
@ -25,21 +25,21 @@ public:
|
||||
static const uint32_t bits = 64;
|
||||
|
||||
// Constructor initializes the same as Initialize()
|
||||
MetroHash64(const uint64_t seed=0);
|
||||
|
||||
explicit MetroHash64(const uint64_t seed=0);
|
||||
|
||||
// Initializes internal state for new hash with optional seed
|
||||
void Initialize(const uint64_t seed=0);
|
||||
|
||||
|
||||
// Update the hash state with a string of bytes. If the length
|
||||
// is sufficiently long, the implementation switches to a bulk
|
||||
// hashing algorithm directly on the argument buffer for speed.
|
||||
void Update(const uint8_t * buffer, const uint64_t length);
|
||||
|
||||
|
||||
// Constructs the final hash and writes it to the argument buffer.
|
||||
// After a hash is finalized, this instance must be Initialized()-ed
|
||||
// again or the behavior of Update() and Finalize() is undefined.
|
||||
void Finalize(uint8_t * const hash);
|
||||
|
||||
|
||||
// A non-incremental function implementation. This can be significantly
|
||||
// faster than the incremental implementation for some usage patterns.
|
||||
static void Hash(const uint8_t * buffer, const uint64_t length, uint8_t * const hash, const uint64_t seed=0);
|
||||
@ -57,7 +57,7 @@ private:
|
||||
static const uint64_t k1 = 0xA2AA033B;
|
||||
static const uint64_t k2 = 0x62992FC1;
|
||||
static const uint64_t k3 = 0x30BC5B29;
|
||||
|
||||
|
||||
struct { uint64_t v[4]; } state;
|
||||
struct { uint8_t b[32]; } input;
|
||||
uint64_t bytes;
|
||||
|
@ -24,7 +24,7 @@ class HTTPAuthClient
|
||||
public:
|
||||
using Result = TResponseParser::Result;
|
||||
|
||||
HTTPAuthClient(const HTTPAuthClientParams & params, const TResponseParser & parser_ = TResponseParser{})
|
||||
explicit HTTPAuthClient(const HTTPAuthClientParams & params, const TResponseParser & parser_ = TResponseParser{})
|
||||
: timeouts{params.timeouts}
|
||||
, max_tries{params.max_tries}
|
||||
, retry_initial_backoff_ms{params.retry_initial_backoff_ms}
|
||||
|
@ -204,7 +204,7 @@ private:
|
||||
class Adam : public IWeightsUpdater
|
||||
{
|
||||
public:
|
||||
Adam(size_t num_params)
|
||||
explicit Adam(size_t num_params)
|
||||
{
|
||||
beta1_powered = beta1;
|
||||
beta2_powered = beta2;
|
||||
|
@ -19,7 +19,7 @@ class BackupCoordinationFileInfos
|
||||
public:
|
||||
/// plain_backup sets that we're writing a plain backup, which means all duplicates are written as is, and empty files are written as is.
|
||||
/// (For normal backups only the first file amongst duplicates is actually stored, and empty files are not stored).
|
||||
BackupCoordinationFileInfos(bool plain_backup_) : plain_backup(plain_backup_) {}
|
||||
explicit BackupCoordinationFileInfos(bool plain_backup_) : plain_backup(plain_backup_) {}
|
||||
|
||||
/// Adds file infos for the specified host.
|
||||
void addFileInfos(BackupFileInfos && file_infos, const String & host_id);
|
||||
|
@ -21,7 +21,7 @@ namespace DB
|
||||
class BackupCoordinationLocal : public IBackupCoordination
|
||||
{
|
||||
public:
|
||||
BackupCoordinationLocal(bool plain_backup_);
|
||||
explicit BackupCoordinationLocal(bool plain_backup_);
|
||||
~BackupCoordinationLocal() override;
|
||||
|
||||
void setStage(const String & new_stage, const String & message) override;
|
||||
|
@ -52,7 +52,7 @@ private:
|
||||
|
||||
struct Task : public AsyncTask
|
||||
{
|
||||
Task(PacketReceiver & receiver_) : receiver(receiver_) {}
|
||||
explicit Task(PacketReceiver & receiver_) : receiver(receiver_) {}
|
||||
|
||||
PacketReceiver & receiver;
|
||||
|
||||
|
@ -53,7 +53,7 @@ class TestHint
|
||||
{
|
||||
public:
|
||||
using ErrorVector = std::vector<int>;
|
||||
TestHint(const String & query_);
|
||||
explicit TestHint(const String & query_);
|
||||
|
||||
const auto & serverErrors() const { return server_errors; }
|
||||
const auto & clientErrors() const { return client_errors; }
|
||||
|
@ -70,7 +70,7 @@ private:
|
||||
return *this;
|
||||
}
|
||||
|
||||
MemoryChunk(size_t size_)
|
||||
explicit MemoryChunk(size_t size_)
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::ArenaAllocChunks);
|
||||
ProfileEvents::increment(ProfileEvents::ArenaAllocBytes, size_);
|
||||
|
@ -46,7 +46,7 @@ public:
|
||||
class AsyncTaskExecutor
|
||||
{
|
||||
public:
|
||||
AsyncTaskExecutor(std::unique_ptr<AsyncTask> task_);
|
||||
explicit AsyncTaskExecutor(std::unique_ptr<AsyncTask> task_);
|
||||
|
||||
/// Resume task execution. This method returns when task is completed or suspended.
|
||||
void resume();
|
||||
|
@ -11,7 +11,7 @@ namespace DB
|
||||
class EnvironmentProxyConfigurationResolver : public ProxyConfigurationResolver
|
||||
{
|
||||
public:
|
||||
EnvironmentProxyConfigurationResolver(Protocol request_protocol, bool disable_tunneling_for_https_requests_over_http_proxy_ = false);
|
||||
explicit EnvironmentProxyConfigurationResolver(Protocol request_protocol, bool disable_tunneling_for_https_requests_over_http_proxy_ = false);
|
||||
|
||||
ProxyConfiguration resolve() override;
|
||||
void errorReport(const ProxyConfiguration &) override {}
|
||||
|
@ -19,7 +19,7 @@ using DiskPtr = std::shared_ptr<IDisk>;
|
||||
class FileChecker
|
||||
{
|
||||
public:
|
||||
FileChecker(const String & file_info_path_);
|
||||
explicit FileChecker(const String & file_info_path_);
|
||||
FileChecker(DiskPtr disk_, const String & file_info_path_);
|
||||
|
||||
void setPath(const String & file_info_path_);
|
||||
@ -50,7 +50,7 @@ public:
|
||||
|
||||
struct DataValidationTasks
|
||||
{
|
||||
DataValidationTasks(const std::map<String, size_t> & map_)
|
||||
explicit DataValidationTasks(const std::map<String, size_t> & map_)
|
||||
: map(map_), it(map.begin())
|
||||
{}
|
||||
|
||||
|
@ -25,7 +25,7 @@ class FileRenamer
|
||||
public:
|
||||
FileRenamer();
|
||||
|
||||
FileRenamer(const String & renaming_rule);
|
||||
explicit FileRenamer(const String & renaming_rule);
|
||||
|
||||
String generateNewFilename(const String & filename) const;
|
||||
|
||||
|
@ -35,7 +35,7 @@ class PriorityPolicy : public ISchedulerNode
|
||||
};
|
||||
|
||||
public:
|
||||
PriorityPolicy(EventQueue * event_queue_, const Poco::Util::AbstractConfiguration & config = emptyConfig(), const String & config_prefix = {})
|
||||
explicit PriorityPolicy(EventQueue * event_queue_, const Poco::Util::AbstractConfiguration & config = emptyConfig(), const String & config_prefix = {})
|
||||
: ISchedulerNode(event_queue_, config, config_prefix)
|
||||
{}
|
||||
|
||||
|
@ -18,7 +18,7 @@ class SemaphoreConstraint : public ISchedulerConstraint
|
||||
static constexpr Int64 default_max_requests = std::numeric_limits<Int64>::max();
|
||||
static constexpr Int64 default_max_cost = std::numeric_limits<Int64>::max();
|
||||
public:
|
||||
SemaphoreConstraint(EventQueue * event_queue_, const Poco::Util::AbstractConfiguration & config = emptyConfig(), const String & config_prefix = {})
|
||||
explicit SemaphoreConstraint(EventQueue * event_queue_, const Poco::Util::AbstractConfiguration & config = emptyConfig(), const String & config_prefix = {})
|
||||
: ISchedulerConstraint(event_queue_, config, config_prefix)
|
||||
, max_requests(config.getInt64(config_prefix + ".max_requests", default_max_requests))
|
||||
, max_cost(config.getInt64(config_prefix + ".max_cost", config.getInt64(config_prefix + ".max_bytes", default_max_cost)))
|
||||
|
@ -20,7 +20,7 @@ class ThrottlerConstraint : public ISchedulerConstraint
|
||||
public:
|
||||
static constexpr double default_burst_seconds = 1.0;
|
||||
|
||||
ThrottlerConstraint(EventQueue * event_queue_, const Poco::Util::AbstractConfiguration & config = emptyConfig(), const String & config_prefix = {})
|
||||
explicit ThrottlerConstraint(EventQueue * event_queue_, const Poco::Util::AbstractConfiguration & config = emptyConfig(), const String & config_prefix = {})
|
||||
: ISchedulerConstraint(event_queue_, config, config_prefix)
|
||||
, max_speed(config.getDouble(config_prefix + ".max_speed", 0))
|
||||
, max_burst(config.getDouble(config_prefix + ".max_burst", default_burst_seconds * max_speed))
|
||||
|
@ -75,7 +75,7 @@ struct ResourceTestBase
|
||||
|
||||
struct ConstraintTest : public SemaphoreConstraint
|
||||
{
|
||||
ConstraintTest(EventQueue * event_queue_, const Poco::Util::AbstractConfiguration & config = emptyConfig(), const String & config_prefix = {})
|
||||
explicit ConstraintTest(EventQueue * event_queue_, const Poco::Util::AbstractConfiguration & config = emptyConfig(), const String & config_prefix = {})
|
||||
: SemaphoreConstraint(event_queue_, config, config_prefix)
|
||||
{}
|
||||
|
||||
|
@ -152,7 +152,7 @@ private:
|
||||
|
||||
struct ResponsesWithFutures
|
||||
{
|
||||
ResponsesWithFutures(FutureResponses future_responses_) : future_responses(std::move(future_responses_))
|
||||
ResponsesWithFutures(FutureResponses future_responses_) : future_responses(std::move(future_responses_)) /// NOLINT(google-explicit-constructor)
|
||||
{
|
||||
cached_responses.resize(future_responses.size());
|
||||
}
|
||||
|
@ -25,7 +25,7 @@ struct ZooKeeperArgs
|
||||
ZooKeeperArgs(const Poco::Util::AbstractConfiguration & config, const String & config_name);
|
||||
|
||||
/// hosts_string -- comma separated [secure://]host:port list
|
||||
ZooKeeperArgs(const String & hosts_string);
|
||||
ZooKeeperArgs(const String & hosts_string); /// NOLINT(google-explicit-constructor)
|
||||
ZooKeeperArgs() = default;
|
||||
bool operator == (const ZooKeeperArgs &) const = default;
|
||||
|
||||
|
@ -66,7 +66,7 @@ public:
|
||||
/// RET_ERROR stands for hardware codec fail, needs fallback to software codec.
|
||||
static constexpr Int32 RET_ERROR = -1;
|
||||
|
||||
HardwareCodecDeflateQpl(SoftwareCodecDeflateQpl & sw_codec_);
|
||||
explicit HardwareCodecDeflateQpl(SoftwareCodecDeflateQpl & sw_codec_);
|
||||
~HardwareCodecDeflateQpl();
|
||||
|
||||
Int32 doCompressData(const char * source, UInt32 source_size, char * dest, UInt32 dest_size) const;
|
||||
|
@ -210,7 +210,7 @@ namespace MySQLReplication
|
||||
public:
|
||||
EventHeader header;
|
||||
|
||||
EventBase(EventHeader && header_) : header(std::move(header_)) {}
|
||||
explicit EventBase(EventHeader && header_) : header(std::move(header_)) {}
|
||||
|
||||
virtual ~EventBase() = default;
|
||||
virtual void dump(WriteBuffer & out) const = 0;
|
||||
@ -224,7 +224,7 @@ namespace MySQLReplication
|
||||
class FormatDescriptionEvent : public EventBase
|
||||
{
|
||||
public:
|
||||
FormatDescriptionEvent(EventHeader && header_)
|
||||
explicit FormatDescriptionEvent(EventHeader && header_)
|
||||
: EventBase(std::move(header_)), binlog_version(0), create_timestamp(0), event_header_length(0)
|
||||
{
|
||||
}
|
||||
@ -249,7 +249,7 @@ namespace MySQLReplication
|
||||
UInt64 position;
|
||||
String next_binlog;
|
||||
|
||||
RotateEvent(EventHeader && header_) : EventBase(std::move(header_)), position(0) {}
|
||||
explicit RotateEvent(EventHeader && header_) : EventBase(std::move(header_)), position(0) {}
|
||||
void dump(WriteBuffer & out) const override;
|
||||
|
||||
protected:
|
||||
@ -280,7 +280,7 @@ namespace MySQLReplication
|
||||
QueryType typ = QUERY_EVENT_DDL;
|
||||
bool transaction_complete = true;
|
||||
|
||||
QueryEvent(EventHeader && header_)
|
||||
explicit QueryEvent(EventHeader && header_)
|
||||
: EventBase(std::move(header_)), thread_id(0), exec_time(0), schema_len(0), error_code(0), status_len(0)
|
||||
{
|
||||
}
|
||||
@ -295,7 +295,7 @@ namespace MySQLReplication
|
||||
class XIDEvent : public EventBase
|
||||
{
|
||||
public:
|
||||
XIDEvent(EventHeader && header_) : EventBase(std::move(header_)), xid(0) {}
|
||||
explicit XIDEvent(EventHeader && header_) : EventBase(std::move(header_)), xid(0) {}
|
||||
|
||||
protected:
|
||||
UInt64 xid;
|
||||
@ -417,7 +417,7 @@ namespace MySQLReplication
|
||||
UInt64 table_id;
|
||||
UInt16 flags;
|
||||
|
||||
RowsEventHeader(EventType type_) : type(type_), table_id(0), flags(0) {}
|
||||
explicit RowsEventHeader(EventType type_) : type(type_), table_id(0), flags(0) {}
|
||||
void parse(ReadBuffer & payload);
|
||||
};
|
||||
|
||||
@ -482,7 +482,7 @@ namespace MySQLReplication
|
||||
UInt8 commit_flag;
|
||||
GTID gtid;
|
||||
|
||||
GTIDEvent(EventHeader && header_) : EventBase(std::move(header_)), commit_flag(0) {}
|
||||
explicit GTIDEvent(EventHeader && header_) : EventBase(std::move(header_)), commit_flag(0) {}
|
||||
void dump(WriteBuffer & out) const override;
|
||||
|
||||
protected:
|
||||
@ -492,7 +492,7 @@ namespace MySQLReplication
|
||||
class DryRunEvent : public EventBase
|
||||
{
|
||||
public:
|
||||
DryRunEvent(EventHeader && header_) : EventBase(std::move(header_)) {}
|
||||
explicit DryRunEvent(EventHeader && header_) : EventBase(std::move(header_)) {}
|
||||
void dump(WriteBuffer & out) const override;
|
||||
|
||||
protected:
|
||||
|
@ -93,7 +93,7 @@ protected:
|
||||
void writePayloadImpl(WriteBuffer & buffer) const override;
|
||||
|
||||
public:
|
||||
OKPacket(uint32_t capabilities_);
|
||||
explicit OKPacket(uint32_t capabilities_);
|
||||
|
||||
OKPacket(uint8_t header_, uint32_t capabilities_, uint64_t affected_rows_,
|
||||
uint32_t status_flags_, int16_t warnings_, String session_state_changes_ = "", String info_ = "");
|
||||
@ -180,7 +180,7 @@ protected:
|
||||
void readPayloadImpl(ReadBuffer & payload) override;
|
||||
|
||||
public:
|
||||
ResponsePacket(UInt32 server_capability_flags_);
|
||||
explicit ResponsePacket(UInt32 server_capability_flags_);
|
||||
|
||||
ResponsePacket(UInt32 server_capability_flags_, bool is_handshake_);
|
||||
};
|
||||
|
@ -34,7 +34,7 @@ protected:
|
||||
void writePayloadImpl(WriteBuffer & buffer) const override;
|
||||
|
||||
public:
|
||||
RegisterSlave(UInt32 server_id_);
|
||||
explicit RegisterSlave(UInt32 server_id_);
|
||||
};
|
||||
|
||||
/// https://dev.mysql.com/doc/internals/en/com-binlog-dump-gtid.html
|
||||
|
@ -19,7 +19,7 @@ namespace ErrorCodes
|
||||
class ClickHouseVersion
|
||||
{
|
||||
public:
|
||||
ClickHouseVersion(const String & version)
|
||||
ClickHouseVersion(const String & version) /// NOLINT(google-explicit-constructor)
|
||||
{
|
||||
Strings split;
|
||||
boost::split(split, version, [](char c){ return c == '.'; });
|
||||
@ -37,7 +37,7 @@ public:
|
||||
}
|
||||
}
|
||||
|
||||
ClickHouseVersion(const char * version) : ClickHouseVersion(String(version)) {}
|
||||
ClickHouseVersion(const char * version) : ClickHouseVersion(String(version)) {} /// NOLINT(google-explicit-constructor)
|
||||
|
||||
String toString() const
|
||||
{
|
||||
|
@ -15,7 +15,7 @@
|
||||
class GraphiteWriter
|
||||
{
|
||||
public:
|
||||
GraphiteWriter(const std::string & config_name, const std::string & sub_path = "");
|
||||
explicit GraphiteWriter(const std::string & config_name, const std::string & sub_path = "");
|
||||
|
||||
template <typename T> using KeyValuePair = std::pair<std::string, T>;
|
||||
template <typename T> using KeyValueVector = std::vector<KeyValuePair<T>>;
|
||||
|
@ -94,7 +94,7 @@ using BinlogFactoryPtr = std::shared_ptr<IBinlogFactory>;
|
||||
class BinlogFromFileFactory : public IBinlogFactory
|
||||
{
|
||||
public:
|
||||
BinlogFromFileFactory(const String & filename_);
|
||||
explicit BinlogFromFileFactory(const String & filename_);
|
||||
BinlogPtr createBinlog(const String & executed_gtid_set) override;
|
||||
|
||||
private:
|
||||
|
@ -14,7 +14,7 @@ namespace DB::MySQLReplication
|
||||
class BinlogClient
|
||||
{
|
||||
public:
|
||||
BinlogClient(const BinlogFactoryPtr & factory,
|
||||
explicit BinlogClient(const BinlogFactoryPtr & factory,
|
||||
const String & name = {},
|
||||
UInt64 max_bytes_in_buffer_ = DBMS_DEFAULT_BUFFER_SIZE,
|
||||
UInt64 max_flush_ms_ = 1000);
|
||||
|
@ -18,7 +18,7 @@ class BinlogFromDispatcher;
|
||||
class BinlogEventsDispatcher final : boost::noncopyable
|
||||
{
|
||||
public:
|
||||
BinlogEventsDispatcher(const String & logger_name_ = "BinlogDispatcher", size_t max_bytes_in_buffer_ = 1_MiB, UInt64 max_flush_ms_ = 1000);
|
||||
explicit BinlogEventsDispatcher(const String & logger_name_ = "BinlogDispatcher", size_t max_bytes_in_buffer_ = 1_MiB, UInt64 max_flush_ms_ = 1000);
|
||||
~BinlogEventsDispatcher();
|
||||
|
||||
/// Moves all IBinlog objects to \a to if it has the same position
|
||||
|
@ -14,7 +14,7 @@ namespace ErrorCodes
|
||||
class NullDictionarySource final : public IDictionarySource
|
||||
{
|
||||
public:
|
||||
NullDictionarySource(Block & sample_block_);
|
||||
explicit NullDictionarySource(Block & sample_block_);
|
||||
|
||||
NullDictionarySource(const NullDictionarySource & other);
|
||||
|
||||
|
@ -76,7 +76,7 @@ private:
|
||||
const LoggerPtr log;
|
||||
|
||||
public:
|
||||
IOUringReader(uint32_t entries_);
|
||||
explicit IOUringReader(uint32_t entries_);
|
||||
|
||||
inline bool isSupported() { return is_supported; }
|
||||
std::future<Result> submit(Request request) override;
|
||||
|
@ -78,7 +78,7 @@ private:
|
||||
|
||||
std::vector<MetadataOperationPtr> operations;
|
||||
public:
|
||||
MetadataStorageFromPlainObjectStorageTransaction(const MetadataStorageFromPlainObjectStorage & metadata_storage_)
|
||||
explicit MetadataStorageFromPlainObjectStorageTransaction(const MetadataStorageFromPlainObjectStorage & metadata_storage_)
|
||||
: metadata_storage(metadata_storage_)
|
||||
{}
|
||||
|
||||
|
@ -63,7 +63,7 @@ class MarksInCompressedFile
|
||||
public:
|
||||
using PlainArray = PODArray<MarkInCompressedFile>;
|
||||
|
||||
MarksInCompressedFile(const PlainArray & marks);
|
||||
explicit MarksInCompressedFile(const PlainArray & marks);
|
||||
|
||||
MarkInCompressedFile get(size_t idx) const;
|
||||
|
||||
|
@ -141,7 +141,7 @@ private:
|
||||
const std::shared_ptr<typename DictGetter::Src> owned_dict;
|
||||
|
||||
public:
|
||||
FunctionTransformWithDictionary(const std::shared_ptr<typename DictGetter::Src> & owned_dict_)
|
||||
explicit FunctionTransformWithDictionary(const std::shared_ptr<typename DictGetter::Src> & owned_dict_)
|
||||
: owned_dict(owned_dict_)
|
||||
{
|
||||
if (!owned_dict)
|
||||
@ -232,7 +232,7 @@ private:
|
||||
const std::shared_ptr<typename DictGetter::Src> owned_dict;
|
||||
|
||||
public:
|
||||
FunctionIsInWithDictionary(const std::shared_ptr<typename DictGetter::Src> & owned_dict_)
|
||||
explicit FunctionIsInWithDictionary(const std::shared_ptr<typename DictGetter::Src> & owned_dict_)
|
||||
: owned_dict(owned_dict_)
|
||||
{
|
||||
if (!owned_dict)
|
||||
@ -365,7 +365,7 @@ private:
|
||||
const std::shared_ptr<typename DictGetter::Src> owned_dict;
|
||||
|
||||
public:
|
||||
FunctionHierarchyWithDictionary(const std::shared_ptr<typename DictGetter::Src> & owned_dict_)
|
||||
explicit FunctionHierarchyWithDictionary(const std::shared_ptr<typename DictGetter::Src> & owned_dict_)
|
||||
: owned_dict(owned_dict_)
|
||||
{
|
||||
if (!owned_dict)
|
||||
@ -563,7 +563,7 @@ private:
|
||||
const MultiVersion<RegionsNames>::Version owned_dict;
|
||||
|
||||
public:
|
||||
FunctionRegionToName(const MultiVersion<RegionsNames>::Version & owned_dict_)
|
||||
explicit FunctionRegionToName(const MultiVersion<RegionsNames>::Version & owned_dict_)
|
||||
: owned_dict(owned_dict_)
|
||||
{
|
||||
if (!owned_dict)
|
||||
|
@ -403,7 +403,7 @@ struct NoEscapingStateHandler : public StateHandlerImpl<false>
|
||||
};
|
||||
|
||||
template <typename ... Args>
|
||||
NoEscapingStateHandler(Args && ... args)
|
||||
explicit NoEscapingStateHandler(Args && ... args)
|
||||
: StateHandlerImpl<false>(std::forward<Args>(args)...) {}
|
||||
};
|
||||
|
||||
@ -465,7 +465,7 @@ struct InlineEscapingStateHandler : public StateHandlerImpl<true>
|
||||
};
|
||||
|
||||
template <typename ... Args>
|
||||
InlineEscapingStateHandler(Args && ... args)
|
||||
explicit InlineEscapingStateHandler(Args && ... args)
|
||||
: StateHandlerImpl<true>(std::forward<Args>(args)...) {}
|
||||
};
|
||||
|
||||
|
@ -21,7 +21,7 @@ class MockRequestHandler : public Poco::Net::HTTPRequestHandler
|
||||
Poco::Net::MessageHeader & last_request_header;
|
||||
|
||||
public:
|
||||
MockRequestHandler(Poco::Net::MessageHeader & last_request_header_)
|
||||
explicit MockRequestHandler(Poco::Net::MessageHeader & last_request_header_)
|
||||
: Poco::Net::HTTPRequestHandler(), last_request_header(last_request_header_)
|
||||
{
|
||||
}
|
||||
@ -44,7 +44,7 @@ class HTTPRequestHandlerFactory : public Poco::Net::HTTPRequestHandlerFactory
|
||||
}
|
||||
|
||||
public:
|
||||
HTTPRequestHandlerFactory(Poco::Net::MessageHeader & last_request_header_)
|
||||
explicit HTTPRequestHandlerFactory(Poco::Net::MessageHeader & last_request_header_)
|
||||
: Poco::Net::HTTPRequestHandlerFactory(), last_request_header(last_request_header_)
|
||||
{
|
||||
}
|
||||
|
@ -15,7 +15,7 @@ namespace DB
|
||||
struct BackupLogElement
|
||||
{
|
||||
BackupLogElement() = default;
|
||||
BackupLogElement(BackupOperationInfo info_);
|
||||
explicit BackupLogElement(BackupOperationInfo info_);
|
||||
BackupLogElement(const BackupLogElement &) = default;
|
||||
BackupLogElement & operator=(const BackupLogElement &) = default;
|
||||
BackupLogElement(BackupLogElement &&) = default;
|
||||
|
@ -12,7 +12,7 @@ namespace DB
|
||||
struct OpenTelemetrySpanLogElement : public OpenTelemetry::Span
|
||||
{
|
||||
OpenTelemetrySpanLogElement() = default;
|
||||
OpenTelemetrySpanLogElement(const OpenTelemetry::Span & span)
|
||||
explicit OpenTelemetrySpanLogElement(const OpenTelemetry::Span & span)
|
||||
: OpenTelemetry::Span(span) {}
|
||||
|
||||
static std::string name() { return "OpenTelemetrySpanLog"; }
|
||||
|
@ -127,7 +127,7 @@ protected:
|
||||
|
||||
struct ExecutorHolder
|
||||
{
|
||||
ExecutorHolder(PipelineExecutor * e) : executor(e) {}
|
||||
explicit ExecutorHolder(PipelineExecutor * e) : executor(e) {}
|
||||
|
||||
void cancel();
|
||||
|
||||
|
@ -56,7 +56,7 @@ struct SelectQueryOptions
|
||||
std::optional<UInt32> shard_num;
|
||||
std::optional<UInt32> shard_count;
|
||||
|
||||
SelectQueryOptions(
|
||||
SelectQueryOptions( /// NOLINT(google-explicit-constructor)
|
||||
QueryProcessingStage::Enum stage = QueryProcessingStage::Complete,
|
||||
size_t depth = 0,
|
||||
bool is_subquery_ = false,
|
||||
|
@ -26,7 +26,7 @@ class Loggers;
|
||||
class OwnJSONPatternFormatter : public OwnPatternFormatter
|
||||
{
|
||||
public:
|
||||
OwnJSONPatternFormatter(Poco::Util::AbstractConfiguration & config);
|
||||
explicit OwnJSONPatternFormatter(Poco::Util::AbstractConfiguration & config);
|
||||
|
||||
void format(const Poco::Message & msg, std::string & text) override;
|
||||
void formatExtended(const DB::ExtendedLogMessage & msg_ext, std::string & text) const override;
|
||||
|
@ -25,7 +25,7 @@ class Loggers;
|
||||
class OwnPatternFormatter : public Poco::PatternFormatter
|
||||
{
|
||||
public:
|
||||
OwnPatternFormatter(bool color_ = false);
|
||||
explicit OwnPatternFormatter(bool color_ = false);
|
||||
|
||||
void format(const Poco::Message & msg, std::string & text) override;
|
||||
virtual void formatExtended(const DB::ExtendedLogMessage & msg_ext, std::string & text) const;
|
||||
|
@ -79,7 +79,7 @@ protected:
|
||||
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
public:
|
||||
ParserDeclareOptionImpl(const std::vector<OptionDescribe> & options_collection_) : options_collection(options_collection_) {}
|
||||
explicit ParserDeclareOptionImpl(const std::vector<OptionDescribe> & options_collection_) : options_collection(options_collection_) {}
|
||||
};
|
||||
|
||||
using ParserDeclareOption = ParserDeclareOptionImpl<false>;
|
||||
|
@ -47,7 +47,7 @@ protected:
|
||||
public:
|
||||
ASTAlterQuery::AlterObjectType alter_object;
|
||||
|
||||
ParserAlterCommandList(ASTAlterQuery::AlterObjectType alter_object_ = ASTAlterQuery::AlterObjectType::TABLE)
|
||||
explicit ParserAlterCommandList(ASTAlterQuery::AlterObjectType alter_object_ = ASTAlterQuery::AlterObjectType::TABLE)
|
||||
: alter_object(alter_object_) {}
|
||||
};
|
||||
|
||||
|
@ -124,7 +124,7 @@ private:
|
||||
class DWARFSchemaReader : public ISchemaReader
|
||||
{
|
||||
public:
|
||||
DWARFSchemaReader(ReadBuffer & in_);
|
||||
explicit DWARFSchemaReader(ReadBuffer & in_);
|
||||
|
||||
NamesAndTypesList readSchema() override;
|
||||
};
|
||||
|
@ -69,7 +69,7 @@ public:
|
||||
class JSONAsObjectExternalSchemaReader : public IExternalSchemaReader
|
||||
{
|
||||
public:
|
||||
JSONAsObjectExternalSchemaReader(const FormatSettings & settings);
|
||||
explicit JSONAsObjectExternalSchemaReader(const FormatSettings & settings);
|
||||
|
||||
NamesAndTypesList readSchema() override
|
||||
{
|
||||
|
@ -15,7 +15,7 @@ namespace DB
|
||||
class JSONColumnsReader : public JSONColumnsReaderBase
|
||||
{
|
||||
public:
|
||||
JSONColumnsReader(ReadBuffer & in_);
|
||||
explicit JSONColumnsReader(ReadBuffer & in_);
|
||||
|
||||
void readChunkStart() override;
|
||||
std::optional<String> readColumnStart() override;
|
||||
|
@ -16,7 +16,7 @@ class ReadBuffer;
|
||||
class JSONColumnsReaderBase
|
||||
{
|
||||
public:
|
||||
JSONColumnsReaderBase(ReadBuffer & in_);
|
||||
explicit JSONColumnsReaderBase(ReadBuffer & in_);
|
||||
|
||||
virtual ~JSONColumnsReaderBase() = default;
|
||||
|
||||
|
@ -22,7 +22,7 @@ private:
|
||||
class JSONColumnsWithMetadataSchemaReader : public ISchemaReader
|
||||
{
|
||||
public:
|
||||
JSONColumnsWithMetadataSchemaReader(ReadBuffer & in_);
|
||||
explicit JSONColumnsWithMetadataSchemaReader(ReadBuffer & in_);
|
||||
|
||||
NamesAndTypesList readSchema() override;
|
||||
};
|
||||
|
@ -15,7 +15,7 @@ namespace DB
|
||||
class JSONCompactColumnsReader : public JSONColumnsReaderBase
|
||||
{
|
||||
public:
|
||||
JSONCompactColumnsReader(ReadBuffer & in_);
|
||||
explicit JSONCompactColumnsReader(ReadBuffer & in_);
|
||||
|
||||
void readChunkStart() override;
|
||||
std::optional<String> readColumnStart() override;
|
||||
|
@ -19,7 +19,7 @@ class ReadBuffer;
|
||||
class MsgPackVisitor : public msgpack::null_visitor
|
||||
{
|
||||
public:
|
||||
MsgPackVisitor(bool null_as_default_) : null_as_default(null_as_default_) {}
|
||||
explicit MsgPackVisitor(bool null_as_default_) : null_as_default(null_as_default_) {}
|
||||
|
||||
struct Info
|
||||
{
|
||||
|
@ -79,7 +79,7 @@ private:
|
||||
|
||||
MemoryToken mem;
|
||||
|
||||
ColumnChunk(ParquetBlockOutputFormat * p) : mem(p) {}
|
||||
explicit ColumnChunk(ParquetBlockOutputFormat * p) : mem(p) {}
|
||||
};
|
||||
|
||||
struct RowGroupState
|
||||
|
@ -83,7 +83,7 @@ private:
|
||||
class ParquetMetadataSchemaReader : public ISchemaReader
|
||||
{
|
||||
public:
|
||||
ParquetMetadataSchemaReader(ReadBuffer & in_);
|
||||
explicit ParquetMetadataSchemaReader(ReadBuffer & in_);
|
||||
|
||||
NamesAndTypesList readSchema() override;
|
||||
};
|
||||
|
@ -18,7 +18,7 @@ public:
|
||||
String raw_data;
|
||||
};
|
||||
|
||||
InputFormatErrorsLogger(const ContextPtr & context);
|
||||
explicit InputFormatErrorsLogger(const ContextPtr & context);
|
||||
|
||||
virtual ~InputFormatErrorsLogger();
|
||||
|
||||
@ -45,7 +45,7 @@ using InputFormatErrorsLoggerPtr = std::shared_ptr<InputFormatErrorsLogger>;
|
||||
class ParallelInputFormatErrorsLogger : public InputFormatErrorsLogger
|
||||
{
|
||||
public:
|
||||
ParallelInputFormatErrorsLogger(const ContextPtr & context) : InputFormatErrorsLogger(context) { }
|
||||
explicit ParallelInputFormatErrorsLogger(const ContextPtr & context) : InputFormatErrorsLogger(context) { }
|
||||
|
||||
~ParallelInputFormatErrorsLogger() override;
|
||||
|
||||
|
@ -20,7 +20,7 @@ struct StreamSettings
|
||||
bool fetch_by_name;
|
||||
size_t default_num_tries_on_connection_loss;
|
||||
|
||||
StreamSettings(const Settings & settings, bool auto_close_ = false, bool fetch_by_name_ = false, size_t max_retry_ = 5);
|
||||
explicit StreamSettings(const Settings & settings, bool auto_close_ = false, bool fetch_by_name_ = false, size_t max_retry_ = 5);
|
||||
|
||||
};
|
||||
|
||||
|
@ -53,7 +53,7 @@ private:
|
||||
|
||||
struct Task : public AsyncTask
|
||||
{
|
||||
Task(RemoteQueryExecutorReadContext & read_context_) : read_context(read_context_) {}
|
||||
explicit Task(RemoteQueryExecutorReadContext & read_context_) : read_context(read_context_) {}
|
||||
|
||||
RemoteQueryExecutorReadContext & read_context;
|
||||
|
||||
|
@ -9,7 +9,7 @@ namespace DB
|
||||
class NotFoundHandler : public HTTPRequestHandler
|
||||
{
|
||||
public:
|
||||
NotFoundHandler(std::vector<std::string> hints_) : hints(std::move(hints_)) {}
|
||||
explicit NotFoundHandler(std::vector<std::string> hints_) : hints(std::move(hints_)) {}
|
||||
void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override;
|
||||
private:
|
||||
std::vector<std::string> hints;
|
||||
|
@ -15,7 +15,7 @@ class PlayWebUIRequestHandler : public HTTPRequestHandler
|
||||
private:
|
||||
IServer & server;
|
||||
public:
|
||||
PlayWebUIRequestHandler(IServer & server_);
|
||||
explicit PlayWebUIRequestHandler(IServer & server_);
|
||||
void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override;
|
||||
};
|
||||
|
||||
@ -24,7 +24,7 @@ class DashboardWebUIRequestHandler : public HTTPRequestHandler
|
||||
private:
|
||||
IServer & server;
|
||||
public:
|
||||
DashboardWebUIRequestHandler(IServer & server_);
|
||||
explicit DashboardWebUIRequestHandler(IServer & server_);
|
||||
void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override;
|
||||
};
|
||||
|
||||
@ -33,7 +33,7 @@ class BinaryWebUIRequestHandler : public HTTPRequestHandler
|
||||
private:
|
||||
IServer & server;
|
||||
public:
|
||||
BinaryWebUIRequestHandler(IServer & server_);
|
||||
explicit BinaryWebUIRequestHandler(IServer & server_);
|
||||
void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override;
|
||||
};
|
||||
|
||||
@ -42,7 +42,7 @@ class JavaScriptWebUIRequestHandler : public HTTPRequestHandler
|
||||
private:
|
||||
IServer & server;
|
||||
public:
|
||||
JavaScriptWebUIRequestHandler(IServer & server_);
|
||||
explicit JavaScriptWebUIRequestHandler(IServer & server_);
|
||||
void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override;
|
||||
};
|
||||
|
||||
|
@ -22,7 +22,7 @@ const size_t DEFAULT_SCHEMA_CACHE_ELEMENTS = 4096;
|
||||
class SchemaCache
|
||||
{
|
||||
public:
|
||||
SchemaCache(size_t max_elements_);
|
||||
explicit SchemaCache(size_t max_elements_);
|
||||
|
||||
struct Key
|
||||
{
|
||||
|
@ -43,7 +43,7 @@ struct GetColumnsOptions
|
||||
All = AllPhysical | Aliases | Ephemeral,
|
||||
};
|
||||
|
||||
GetColumnsOptions(Kind kind_) : kind(kind_) {}
|
||||
GetColumnsOptions(Kind kind_) : kind(kind_) {} /// NOLINT(google-explicit-constructor)
|
||||
|
||||
GetColumnsOptions & withSubcolumns(bool value = true)
|
||||
{
|
||||
@ -113,7 +113,7 @@ public:
|
||||
|
||||
explicit ColumnsDescription(NamesAndTypesList ordinary);
|
||||
|
||||
explicit ColumnsDescription(std::initializer_list<ColumnDescription> ordinary);
|
||||
ColumnsDescription(std::initializer_list<ColumnDescription> ordinary);
|
||||
|
||||
explicit ColumnsDescription(NamesAndTypesList ordinary, NamesAndAliases aliases);
|
||||
|
||||
|
@ -32,7 +32,7 @@ public:
|
||||
class Unfreezer
|
||||
{
|
||||
public:
|
||||
Unfreezer(ContextPtr context);
|
||||
explicit Unfreezer(ContextPtr context);
|
||||
PartitionCommandsResultInfo unfreezePartitionsFromTableDirectory(MergeTreeData::MatcherFn matcher, const String & backup_name, const Disks & disks, const fs::path & table_directory);
|
||||
BlockIO systemUnfreeze(const String & backup_name);
|
||||
private:
|
||||
|
@ -1354,7 +1354,7 @@ protected:
|
||||
size_t max_postpone_time_ms;
|
||||
size_t max_postpone_power;
|
||||
|
||||
PartMutationInfo(size_t max_postpone_time_ms_)
|
||||
explicit PartMutationInfo(size_t max_postpone_time_ms_)
|
||||
: retry_count(0ull)
|
||||
, latest_fail_time_us(static_cast<size_t>(Poco::Timestamp().epochMicroseconds()))
|
||||
, max_postpone_time_ms(max_postpone_time_ms_)
|
||||
|
@ -18,7 +18,7 @@ class MergeTreeData;
|
||||
*/
|
||||
struct MarkType
|
||||
{
|
||||
MarkType(std::string_view extension);
|
||||
explicit MarkType(std::string_view extension);
|
||||
MarkType(bool adaptive_, bool compressed_, MergeTreeDataPartType::Value part_type_);
|
||||
|
||||
static bool isMarkFileExtension(std::string_view extension);
|
||||
|
@ -499,7 +499,7 @@ class BaseMergePredicate
|
||||
{
|
||||
public:
|
||||
BaseMergePredicate() = default;
|
||||
BaseMergePredicate(std::optional<PartitionIdsHint> && partition_ids_hint_) : partition_ids_hint(std::move(partition_ids_hint_)) {}
|
||||
explicit BaseMergePredicate(std::optional<PartitionIdsHint> && partition_ids_hint_) : partition_ids_hint(std::move(partition_ids_hint_)) {}
|
||||
|
||||
/// Depending on the existence of left part checks a merge predicate for two parts or for single part.
|
||||
bool operator()(const MergeTreeData::DataPartPtr & left,
|
||||
@ -550,7 +550,7 @@ protected:
|
||||
class LocalMergePredicate : public BaseMergePredicate<ActiveDataPartSet, ReplicatedMergeTreeQueue>
|
||||
{
|
||||
public:
|
||||
LocalMergePredicate(ReplicatedMergeTreeQueue & queue_);
|
||||
explicit LocalMergePredicate(ReplicatedMergeTreeQueue & queue_);
|
||||
};
|
||||
|
||||
class ReplicatedMergeTreeMergePredicate : public BaseMergePredicate<ActiveDataPartSet, ReplicatedMergeTreeQueue>
|
||||
|
@ -22,7 +22,7 @@ struct ReplicatedMergeTreeQuorumAddedParts
|
||||
|
||||
MergeTreeDataFormatVersion format_version;
|
||||
|
||||
ReplicatedMergeTreeQuorumAddedParts(const MergeTreeDataFormatVersion format_version_)
|
||||
explicit ReplicatedMergeTreeQuorumAddedParts(const MergeTreeDataFormatVersion format_version_)
|
||||
: format_version(format_version_)
|
||||
{}
|
||||
|
||||
|
@ -51,9 +51,9 @@ struct RedisEqualKeysSet
|
||||
template <typename EqualKeys> struct NamedCollectionValidateKey
|
||||
{
|
||||
NamedCollectionValidateKey() = default;
|
||||
NamedCollectionValidateKey(const char * value_) : value(value_) {}
|
||||
NamedCollectionValidateKey(std::string_view value_) : value(value_) {}
|
||||
NamedCollectionValidateKey(const String & value_) : value(value_) {}
|
||||
NamedCollectionValidateKey(const char * value_) : value(value_) {} /// NOLINT(google-explicit-constructor)
|
||||
NamedCollectionValidateKey(std::string_view value_) : value(value_) {} /// NOLINT(google-explicit-constructor)
|
||||
NamedCollectionValidateKey(const String & value_) : value(value_) {} /// NOLINT(google-explicit-constructor)
|
||||
|
||||
std::string_view value;
|
||||
|
||||
|
@ -149,7 +149,7 @@ public:
|
||||
class IIterator : public WithContext
|
||||
{
|
||||
public:
|
||||
IIterator(const ContextPtr & context_):WithContext(context_) {}
|
||||
explicit IIterator(const ContextPtr & context_):WithContext(context_) {}
|
||||
virtual ~IIterator() = default;
|
||||
virtual RelativePathWithMetadata next() = 0;
|
||||
|
||||
|
@ -37,7 +37,7 @@ struct StoragesInfo
|
||||
class StoragesInfoStreamBase
|
||||
{
|
||||
public:
|
||||
StoragesInfoStreamBase(ContextPtr context)
|
||||
explicit StoragesInfoStreamBase(ContextPtr context)
|
||||
: query_id(context->getCurrentQueryId()), settings(context->getSettingsRef()), next_row(0), rows(0)
|
||||
{}
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user