From 2665724301cf5a3d870f84e4f93023ca79ddc4f8 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 13 Mar 2022 12:05:14 +0000 Subject: [PATCH] Fix clang-tidy warnings in Parsers, Processors, QueryPipeline folders --- src/Parsers/ASTAssignment.h | 1 + src/Parsers/ASTCreateQuery.h | 2 +- .../ASTFunctionWithKeyValueArguments.h | 2 -- src/Parsers/ASTHelpers.h | 2 +- src/Parsers/ASTProjectionSelectQuery.h | 8 ++--- src/Parsers/ASTQueryWithOnCluster.h | 2 +- src/Parsers/ASTSelectQuery.h | 30 +++++++++---------- src/Parsers/ASTTTLElement.h | 5 ++-- src/Parsers/ASTTableOverrides.cpp | 2 +- src/Parsers/ASTTableOverrides.h | 2 +- src/Parsers/ASTUseQuery.h | 1 - src/Parsers/Access/ASTUserNameWithHost.h | 4 +-- src/Parsers/CommonParsers.h | 3 +- src/Parsers/ExpressionListParsers.h | 2 +- src/Parsers/IAST.h | 2 +- src/Parsers/ParserExplainQuery.h | 2 +- src/Parsers/ParserQueryWithOutput.h | 2 +- src/Parsers/ParserTablesInSelectQuery.h | 2 +- src/Processors/Chunk.h | 2 +- src/Processors/Formats/ISchemaReader.h | 2 +- .../Formats/Impl/ArrowBufferedStreams.h | 2 +- .../Formats/Impl/AvroRowInputFormat.h | 2 +- .../Formats/Impl/CapnProtoRowOutputFormat.h | 2 +- .../Formats/Impl/ConstantExpressionTemplate.h | 2 ++ .../Formats/Impl/ORCBlockOutputFormat.h | 2 +- .../Formats/Impl/RegexpRowInputFormat.h | 2 +- .../Formats/Impl/TabSeparatedRowInputFormat.h | 2 +- .../Algorithms/FixedSizeDequeWithGaps.h | 2 ++ src/Processors/Merges/Algorithms/Graphite.cpp | 2 +- src/Processors/Merges/Algorithms/Graphite.h | 2 +- src/Processors/Merges/Algorithms/MergedData.h | 6 ++++ src/Processors/Port.h | 12 ++++---- src/Processors/QueueBuffer.h | 2 +- .../Transforms/AggregatingInOrderTransform.h | 2 +- .../Transforms/AggregatingTransform.h | 2 +- .../Transforms/ColumnGathererTransform.h | 2 +- .../Transforms/PostgreSQLSource.cpp | 4 +-- src/Processors/Transforms/PostgreSQLSource.h | 4 +-- src/Processors/Transforms/WindowTransform.h | 7 ++--- src/QueryPipeline/SizeLimits.h | 2 +- 40 files changed, 74 insertions(+), 67 deletions(-) diff --git a/src/Parsers/ASTAssignment.h b/src/Parsers/ASTAssignment.h index 88d4bb96c15..a37a31ae38e 100644 --- a/src/Parsers/ASTAssignment.h +++ b/src/Parsers/ASTAssignment.h @@ -28,6 +28,7 @@ public: protected: void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override { + settings.ostr << (settings.hilite ? hilite_identifier : ""); settings.writeIdentifier(column_name); settings.ostr << (settings.hilite ? hilite_none : ""); diff --git a/src/Parsers/ASTCreateQuery.h b/src/Parsers/ASTCreateQuery.h index a0070892b79..04755a02399 100644 --- a/src/Parsers/ASTCreateQuery.h +++ b/src/Parsers/ASTCreateQuery.h @@ -51,7 +51,7 @@ public: void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override; - bool empty() + bool empty() const { return (!columns || columns->children.empty()) && (!indices || indices->children.empty()) && (!constraints || constraints->children.empty()) && (!projections || projections->children.empty()); diff --git a/src/Parsers/ASTFunctionWithKeyValueArguments.h b/src/Parsers/ASTFunctionWithKeyValueArguments.h index 5820e8564ac..4b745e2c1a2 100644 --- a/src/Parsers/ASTFunctionWithKeyValueArguments.h +++ b/src/Parsers/ASTFunctionWithKeyValueArguments.h @@ -19,7 +19,6 @@ public: /// Value is closed in brackets (HOST '127.0.0.1') bool second_with_brackets; -public: explicit ASTPair(bool second_with_brackets_) : second_with_brackets(second_with_brackets_) { @@ -54,7 +53,6 @@ public: { } -public: String getID(char delim) const override; ASTPtr clone() const override; diff --git a/src/Parsers/ASTHelpers.h b/src/Parsers/ASTHelpers.h index 086b361bf85..0b3db8e02d5 100644 --- a/src/Parsers/ASTHelpers.h +++ b/src/Parsers/ASTHelpers.h @@ -6,7 +6,7 @@ namespace DB { -static inline bool isFunctionCast(const ASTFunction * function) +static inline bool isFunctionCast(const ASTFunction * function) /// NOLINT { if (function) return function->name == "CAST" || function->name == "_CAST"; diff --git a/src/Parsers/ASTProjectionSelectQuery.h b/src/Parsers/ASTProjectionSelectQuery.h index 71334c50868..d93c10b6e39 100644 --- a/src/Parsers/ASTProjectionSelectQuery.h +++ b/src/Parsers/ASTProjectionSelectQuery.h @@ -26,10 +26,10 @@ public: ASTPtr & refSelect() { return getExpression(Expression::SELECT); } - const ASTPtr with() const { return getExpression(Expression::WITH); } - const ASTPtr select() const { return getExpression(Expression::SELECT); } - const ASTPtr groupBy() const { return getExpression(Expression::GROUP_BY); } - const ASTPtr orderBy() const { return getExpression(Expression::ORDER_BY); } + ASTPtr with() const { return getExpression(Expression::WITH); } + ASTPtr select() const { return getExpression(Expression::SELECT); } + ASTPtr groupBy() const { return getExpression(Expression::GROUP_BY); } + ASTPtr orderBy() const { return getExpression(Expression::ORDER_BY); } /// Set/Reset/Remove expression. void setExpression(Expression expr, ASTPtr && ast); diff --git a/src/Parsers/ASTQueryWithOnCluster.h b/src/Parsers/ASTQueryWithOnCluster.h index b309ae5e847..c5daaa6ce37 100644 --- a/src/Parsers/ASTQueryWithOnCluster.h +++ b/src/Parsers/ASTQueryWithOnCluster.h @@ -17,7 +17,7 @@ public: /// new_database should be used by queries that refer to default db /// and default_database is specified for remote server - virtual ASTPtr getRewrittenASTWithoutOnCluster(const std::string & new_database = {}) const = 0; + virtual ASTPtr getRewrittenASTWithoutOnCluster(const std::string & new_database = {}) const = 0; /// NOLINT /// Returns a query prepared for execution on remote server std::string getRewrittenQueryWithoutOnCluster(const std::string & new_database = {}) const; diff --git a/src/Parsers/ASTSelectQuery.h b/src/Parsers/ASTSelectQuery.h index 1c631783fdb..9a8f1dbd2e7 100644 --- a/src/Parsers/ASTSelectQuery.h +++ b/src/Parsers/ASTSelectQuery.h @@ -91,21 +91,21 @@ public: ASTPtr & refWhere() { return getExpression(Expression::WHERE); } ASTPtr & refHaving() { return getExpression(Expression::HAVING); } - const ASTPtr with() const { return getExpression(Expression::WITH); } - const ASTPtr select() const { return getExpression(Expression::SELECT); } - const ASTPtr tables() const { return getExpression(Expression::TABLES); } - const ASTPtr prewhere() const { return getExpression(Expression::PREWHERE); } - const ASTPtr where() const { return getExpression(Expression::WHERE); } - const ASTPtr groupBy() const { return getExpression(Expression::GROUP_BY); } - const ASTPtr having() const { return getExpression(Expression::HAVING); } - const ASTPtr window() const { return getExpression(Expression::WINDOW); } - const ASTPtr orderBy() const { return getExpression(Expression::ORDER_BY); } - const ASTPtr limitByOffset() const { return getExpression(Expression::LIMIT_BY_OFFSET); } - const ASTPtr limitByLength() const { return getExpression(Expression::LIMIT_BY_LENGTH); } - const ASTPtr limitBy() const { return getExpression(Expression::LIMIT_BY); } - const ASTPtr limitOffset() const { return getExpression(Expression::LIMIT_OFFSET); } - const ASTPtr limitLength() const { return getExpression(Expression::LIMIT_LENGTH); } - const ASTPtr settings() const { return getExpression(Expression::SETTINGS); } + ASTPtr with() const { return getExpression(Expression::WITH); } + ASTPtr select() const { return getExpression(Expression::SELECT); } + ASTPtr tables() const { return getExpression(Expression::TABLES); } + ASTPtr prewhere() const { return getExpression(Expression::PREWHERE); } + ASTPtr where() const { return getExpression(Expression::WHERE); } + ASTPtr groupBy() const { return getExpression(Expression::GROUP_BY); } + ASTPtr having() const { return getExpression(Expression::HAVING); } + ASTPtr window() const { return getExpression(Expression::WINDOW); } + ASTPtr orderBy() const { return getExpression(Expression::ORDER_BY); } + ASTPtr limitByOffset() const { return getExpression(Expression::LIMIT_BY_OFFSET); } + ASTPtr limitByLength() const { return getExpression(Expression::LIMIT_BY_LENGTH); } + ASTPtr limitBy() const { return getExpression(Expression::LIMIT_BY); } + ASTPtr limitOffset() const { return getExpression(Expression::LIMIT_OFFSET); } + ASTPtr limitLength() const { return getExpression(Expression::LIMIT_LENGTH); } + ASTPtr settings() const { return getExpression(Expression::SETTINGS); } bool hasFiltration() const { return where() || prewhere() || having(); } diff --git a/src/Parsers/ASTTTLElement.h b/src/Parsers/ASTTTLElement.h index a396a4c54e0..9705cafbce3 100644 --- a/src/Parsers/ASTTTLElement.h +++ b/src/Parsers/ASTTTLElement.h @@ -37,8 +37,8 @@ public: ASTPtr clone() const override; - const ASTPtr ttl() const { return getExpression(ttl_expr_pos); } - const ASTPtr where() const { return getExpression(where_expr_pos); } + ASTPtr ttl() const { return getExpression(ttl_expr_pos); } + ASTPtr where() const { return getExpression(where_expr_pos); } void setTTL(ASTPtr && ast) { setExpression(ttl_expr_pos, std::forward(ast)); } void setWhere(ASTPtr && ast) { setExpression(where_expr_pos, std::forward(ast)); } @@ -50,7 +50,6 @@ private: int ttl_expr_pos; int where_expr_pos; -private: void setExpression(int & pos, ASTPtr && ast); ASTPtr getExpression(int pos, bool clone = false) const; }; diff --git a/src/Parsers/ASTTableOverrides.cpp b/src/Parsers/ASTTableOverrides.cpp index 8fc21db218f..0f34a9fb247 100644 --- a/src/Parsers/ASTTableOverrides.cpp +++ b/src/Parsers/ASTTableOverrides.cpp @@ -93,7 +93,7 @@ ASTPtr ASTTableOverrideList::tryGetTableOverride(const String & name) const return children[it->second]; } -void ASTTableOverrideList::setTableOverride(const String & name, const ASTPtr ast) +void ASTTableOverrideList::setTableOverride(const String & name, ASTPtr ast) { auto it = positions.find(name); if (it == positions.end()) diff --git a/src/Parsers/ASTTableOverrides.h b/src/Parsers/ASTTableOverrides.h index c0603f7a8e0..c47260789d8 100644 --- a/src/Parsers/ASTTableOverrides.h +++ b/src/Parsers/ASTTableOverrides.h @@ -40,7 +40,7 @@ public: String getID(char) const override { return "TableOverrideList"; } ASTPtr clone() const override; void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; - void setTableOverride(const String & name, const ASTPtr ast); + void setTableOverride(const String & name, ASTPtr ast); void removeTableOverride(const String & name); ASTPtr tryGetTableOverride(const String & name) const; bool hasOverride(const String & name) const; diff --git a/src/Parsers/ASTUseQuery.h b/src/Parsers/ASTUseQuery.h index 4e4a13c2a7f..16d449f905f 100644 --- a/src/Parsers/ASTUseQuery.h +++ b/src/Parsers/ASTUseQuery.h @@ -25,7 +25,6 @@ protected: void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override { settings.ostr << (settings.hilite ? hilite_keyword : "") << "USE " << (settings.hilite ? hilite_none : "") << backQuoteIfNeed(database); - return; } }; diff --git a/src/Parsers/Access/ASTUserNameWithHost.h b/src/Parsers/Access/ASTUserNameWithHost.h index ada9bfb0673..bd28b42b48a 100644 --- a/src/Parsers/Access/ASTUserNameWithHost.h +++ b/src/Parsers/Access/ASTUserNameWithHost.h @@ -23,7 +23,7 @@ public: void concatParts(); ASTUserNameWithHost() = default; - ASTUserNameWithHost(const String & name_) : base_name(name_) {} + explicit ASTUserNameWithHost(const String & name_) : base_name(name_) {} String getID(char) const override { return "UserNameWithHost"; } ASTPtr clone() const override { return std::make_shared(*this); } void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; @@ -39,7 +39,7 @@ public: auto begin() const { return names.begin(); } auto end() const { return names.end(); } auto front() const { return *begin(); } - void push_back(const String & name_) { names.push_back(std::make_shared(name_)); } + void push_back(const String & name_) { names.push_back(std::make_shared(name_)); } /// NOLINT Strings toStrings() const; void concatParts(); diff --git a/src/Parsers/CommonParsers.h b/src/Parsers/CommonParsers.h index 58fac2341cf..d2911754b24 100644 --- a/src/Parsers/CommonParsers.h +++ b/src/Parsers/CommonParsers.h @@ -33,7 +33,8 @@ class ParserToken : public IParserBase private: TokenType token_type; public: - ParserToken(TokenType token_type_) : token_type(token_type_) {} + ParserToken(TokenType token_type_) : token_type(token_type_) {} /// NOLINT + protected: const char * getName() const override { return "token"; } diff --git a/src/Parsers/ExpressionListParsers.h b/src/Parsers/ExpressionListParsers.h index 358fe778f91..86d0fd0f861 100644 --- a/src/Parsers/ExpressionListParsers.h +++ b/src/Parsers/ExpressionListParsers.h @@ -207,7 +207,7 @@ private: ParserPtr elem_parser; public: - ParserCastExpression(ParserPtr && elem_parser_) + explicit ParserCastExpression(ParserPtr && elem_parser_) : elem_parser(std::move(elem_parser_)) { } diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index fdf821c4a0b..bd8167c64fe 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -69,7 +69,7 @@ public: } /** Get the text that identifies this element. */ - virtual String getID(char delimiter = '_') const = 0; + virtual String getID(char delimiter = '_') const = 0; /// NOLINT ASTPtr ptr() { return shared_from_this(); } diff --git a/src/Parsers/ParserExplainQuery.h b/src/Parsers/ParserExplainQuery.h index a1865e30239..ba30e97a58f 100644 --- a/src/Parsers/ParserExplainQuery.h +++ b/src/Parsers/ParserExplainQuery.h @@ -14,7 +14,7 @@ protected: const char * getName() const override { return "EXPLAIN"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; public: - ParserExplainQuery(const char* end_) : end(end_) {} + explicit ParserExplainQuery(const char* end_) : end(end_) {} }; } diff --git a/src/Parsers/ParserQueryWithOutput.h b/src/Parsers/ParserQueryWithOutput.h index 854d5a74ffd..1fd7bec1eea 100644 --- a/src/Parsers/ParserQueryWithOutput.h +++ b/src/Parsers/ParserQueryWithOutput.h @@ -15,7 +15,7 @@ protected: const char * getName() const override { return "Query with output"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; public: - ParserQueryWithOutput(const char * end_) : end(end_) {} + explicit ParserQueryWithOutput(const char * end_) : end(end_) {} }; } diff --git a/src/Parsers/ParserTablesInSelectQuery.h b/src/Parsers/ParserTablesInSelectQuery.h index 9e5b591ccbe..772f1992f4d 100644 --- a/src/Parsers/ParserTablesInSelectQuery.h +++ b/src/Parsers/ParserTablesInSelectQuery.h @@ -21,7 +21,7 @@ protected: class ParserTablesInSelectQueryElement : public IParserBase { public: - ParserTablesInSelectQueryElement(bool is_first_) : is_first(is_first_) {} + explicit ParserTablesInSelectQueryElement(bool is_first_) : is_first(is_first_) {} protected: const char * getName() const override { return "table, table function, subquery or list of joined tables"; } diff --git a/src/Processors/Chunk.h b/src/Processors/Chunk.h index e70ba57a267..1c9240ba114 100644 --- a/src/Processors/Chunk.h +++ b/src/Processors/Chunk.h @@ -90,7 +90,7 @@ public: bool hasRows() const { return num_rows > 0; } bool hasColumns() const { return !columns.empty(); } bool empty() const { return !hasRows() && !hasColumns(); } - operator bool() const { return !empty(); } + operator bool() const { return !empty(); } /// NOLINT void addColumn(ColumnPtr column); void addColumn(size_t position, ColumnPtr column); diff --git a/src/Processors/Formats/ISchemaReader.h b/src/Processors/Formats/ISchemaReader.h index 67a8eb88d61..2d35809e26a 100644 --- a/src/Processors/Formats/ISchemaReader.h +++ b/src/Processors/Formats/ISchemaReader.h @@ -14,7 +14,7 @@ namespace DB class ISchemaReader { public: - ISchemaReader(ReadBuffer & in_) : in(in_) {} + explicit ISchemaReader(ReadBuffer & in_) : in(in_) {} virtual NamesAndTypesList readSchema() = 0; diff --git a/src/Processors/Formats/Impl/ArrowBufferedStreams.h b/src/Processors/Formats/Impl/ArrowBufferedStreams.h index d649c52557f..e06eab04f1b 100644 --- a/src/Processors/Formats/Impl/ArrowBufferedStreams.h +++ b/src/Processors/Formats/Impl/ArrowBufferedStreams.h @@ -44,7 +44,7 @@ class RandomAccessFileFromSeekableReadBuffer : public arrow::io::RandomAccessFil public: RandomAccessFileFromSeekableReadBuffer(SeekableReadBuffer & in_, off_t file_size_); - RandomAccessFileFromSeekableReadBuffer(SeekableReadBufferWithSize & in_); + explicit RandomAccessFileFromSeekableReadBuffer(SeekableReadBufferWithSize & in_); arrow::Result GetSize() override; diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.h b/src/Processors/Formats/Impl/AvroRowInputFormat.h index 1e8ee4aebb9..7a598de1f6a 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.h +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.h @@ -61,7 +61,7 @@ private: , target_column_idx(target_column_idx_) , deserialize_fn(deserialize_fn_) {} - Action(SkipFn skip_fn_) + explicit Action(SkipFn skip_fn_) : type(Skip) , skip_fn(skip_fn_) {} diff --git a/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.h b/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.h index 288b36508ce..12dc5eda2b3 100644 --- a/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.h +++ b/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.h @@ -15,7 +15,7 @@ namespace DB class CapnProtoOutputStream : public kj::OutputStream { public: - CapnProtoOutputStream(WriteBuffer & out_); + explicit CapnProtoOutputStream(WriteBuffer & out_); void write(const void * buffer, size_t size) override; diff --git a/src/Processors/Formats/Impl/ConstantExpressionTemplate.h b/src/Processors/Formats/Impl/ConstantExpressionTemplate.h index 6659243df63..c5d4f033258 100644 --- a/src/Processors/Formats/Impl/ConstantExpressionTemplate.h +++ b/src/Processors/Formats/Impl/ConstantExpressionTemplate.h @@ -3,6 +3,8 @@ #include #include #include +#include +#include namespace DB { diff --git a/src/Processors/Formats/Impl/ORCBlockOutputFormat.h b/src/Processors/Formats/Impl/ORCBlockOutputFormat.h index 2ffee597e8f..f69fd1c0aab 100644 --- a/src/Processors/Formats/Impl/ORCBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/ORCBlockOutputFormat.h @@ -17,7 +17,7 @@ class WriteBuffer; class ORCOutputStream : public orc::OutputStream { public: - ORCOutputStream(WriteBuffer & out_); + explicit ORCOutputStream(WriteBuffer & out_); uint64_t getLength() const override; uint64_t getNaturalWriteSize() const override; diff --git a/src/Processors/Formats/Impl/RegexpRowInputFormat.h b/src/Processors/Formats/Impl/RegexpRowInputFormat.h index 75c630d0607..04f24bbb3e4 100644 --- a/src/Processors/Formats/Impl/RegexpRowInputFormat.h +++ b/src/Processors/Formats/Impl/RegexpRowInputFormat.h @@ -22,7 +22,7 @@ class ReadBuffer; class RegexpFieldExtractor { public: - RegexpFieldExtractor(const FormatSettings & format_settings); + explicit RegexpFieldExtractor(const FormatSettings & format_settings); /// Return true if row was successfully parsed and row fields were extracted. bool parseRow(PeekableReadBuffer & buf); diff --git a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h index ed67a8256bc..abab5b02c96 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h +++ b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h @@ -53,7 +53,7 @@ public: bool parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) override; bool parseRowEndWithDiagnosticInfo(WriteBuffer & out) override; - FormatSettings::EscapingRule getEscapingRule() + FormatSettings::EscapingRule getEscapingRule() const { return is_raw ? FormatSettings::EscapingRule::Raw : FormatSettings::EscapingRule::Escaped; } diff --git a/src/Processors/Merges/Algorithms/FixedSizeDequeWithGaps.h b/src/Processors/Merges/Algorithms/FixedSizeDequeWithGaps.h index 35cfded4214..ff8f113d9a6 100644 --- a/src/Processors/Merges/Algorithms/FixedSizeDequeWithGaps.h +++ b/src/Processors/Merges/Algorithms/FixedSizeDequeWithGaps.h @@ -1,5 +1,7 @@ #pragma once +#include + namespace DB { diff --git a/src/Processors/Merges/Algorithms/Graphite.cpp b/src/Processors/Merges/Algorithms/Graphite.cpp index 2c6d08ed287..c0f595fa539 100644 --- a/src/Processors/Merges/Algorithms/Graphite.cpp +++ b/src/Processors/Merges/Algorithms/Graphite.cpp @@ -89,7 +89,7 @@ inline static const Patterns & selectPatternsForMetricType(const Graphite::Param Graphite::RollupRule selectPatternForPath( const Graphite::Params & params, - const StringRef path) + StringRef path) { const Graphite::Pattern * first_match = &undef_pattern; diff --git a/src/Processors/Merges/Algorithms/Graphite.h b/src/Processors/Merges/Algorithms/Graphite.h index dc39cb46386..05306ebe30f 100644 --- a/src/Processors/Merges/Algorithms/Graphite.h +++ b/src/Processors/Merges/Algorithms/Graphite.h @@ -147,7 +147,7 @@ struct Params using RollupRule = std::pair; -Graphite::RollupRule selectPatternForPath(const Graphite::Params & params, const StringRef path); +Graphite::RollupRule selectPatternForPath(const Graphite::Params & params, StringRef path); void setGraphitePatternsFromConfig(ContextPtr context, const String & config_element, Graphite::Params & params); diff --git a/src/Processors/Merges/Algorithms/MergedData.h b/src/Processors/Merges/Algorithms/MergedData.h index 9bf33d72f31..89da346980d 100644 --- a/src/Processors/Merges/Algorithms/MergedData.h +++ b/src/Processors/Merges/Algorithms/MergedData.h @@ -1,5 +1,11 @@ #pragma once +#include +#include +#include +#include + + namespace DB { diff --git a/src/Processors/Port.h b/src/Processors/Port.h index 9f27b440be5..7cb25f3930e 100644 --- a/src/Processors/Port.h +++ b/src/Processors/Port.h @@ -214,7 +214,7 @@ protected: public: using Data = State::Data; - Port(Block header_) : header(std::move(header_)) {} + Port(Block header_) : header(std::move(header_)) {} /// NOLINT Port(Block header_, IProcessor * processor_) : header(std::move(header_)), processor(processor_) {} void setUpdateInfo(UpdateInfo * info) { update_info = info; } @@ -303,12 +303,12 @@ public: Chunk ALWAYS_INLINE pull(bool set_not_needed = false) { - auto data_ = pullData(set_not_needed); + auto pull_data = pullData(set_not_needed); - if (data_.exception) - std::rethrow_exception(data_.exception); + if (pull_data.exception) + std::rethrow_exception(pull_data.exception); - return std::move(data_.chunk); + return std::move(pull_data.chunk); } bool ALWAYS_INLINE isFinished() const @@ -396,7 +396,7 @@ public: void ALWAYS_INLINE pushException(std::exception_ptr exception) { - pushData({.chunk = {}, .exception = std::move(exception)}); + pushData({.chunk = {}, .exception = exception}); } void ALWAYS_INLINE pushData(Data data_) diff --git a/src/Processors/QueueBuffer.h b/src/Processors/QueueBuffer.h index 826f4a22b8b..6856e214823 100644 --- a/src/Processors/QueueBuffer.h +++ b/src/Processors/QueueBuffer.h @@ -17,7 +17,7 @@ private: public: String getName() const override { return "QueueBuffer"; } - QueueBuffer(Block header) + explicit QueueBuffer(Block header) : IAccumulatingTransform(header, header) { } diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.h b/src/Processors/Transforms/AggregatingInOrderTransform.h index 929ab98d6e6..e4c217a8f81 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.h +++ b/src/Processors/Transforms/AggregatingInOrderTransform.h @@ -11,7 +11,7 @@ namespace DB struct ChunkInfoWithAllocatedBytes : public ChunkInfo { - ChunkInfoWithAllocatedBytes(Int64 allocated_bytes_) + explicit ChunkInfoWithAllocatedBytes(Int64 allocated_bytes_) : allocated_bytes(allocated_bytes_) {} Int64 allocated_bytes; }; diff --git a/src/Processors/Transforms/AggregatingTransform.h b/src/Processors/Transforms/AggregatingTransform.h index 01df264005b..d7917fc95a7 100644 --- a/src/Processors/Transforms/AggregatingTransform.h +++ b/src/Processors/Transforms/AggregatingTransform.h @@ -12,7 +12,7 @@ class AggregatedArenasChunkInfo : public ChunkInfo { public: Arenas arenas; - AggregatedArenasChunkInfo(Arenas arenas_) + explicit AggregatedArenasChunkInfo(Arenas arenas_) : arenas(std::move(arenas_)) {} }; diff --git a/src/Processors/Transforms/ColumnGathererTransform.h b/src/Processors/Transforms/ColumnGathererTransform.h index 2d013e596ce..da6dc877abf 100644 --- a/src/Processors/Transforms/ColumnGathererTransform.h +++ b/src/Processors/Transforms/ColumnGathererTransform.h @@ -20,7 +20,7 @@ struct RowSourcePart RowSourcePart() = default; - RowSourcePart(size_t source_num, bool skip_flag = false) + explicit RowSourcePart(size_t source_num, bool skip_flag = false) { static_assert(sizeof(*this) == 1, "Size of RowSourcePart is too big due to compiler settings"); setSourceNum(source_num); diff --git a/src/Processors/Transforms/PostgreSQLSource.cpp b/src/Processors/Transforms/PostgreSQLSource.cpp index 88f092a2533..a31cd879257 100644 --- a/src/Processors/Transforms/PostgreSQLSource.cpp +++ b/src/Processors/Transforms/PostgreSQLSource.cpp @@ -28,7 +28,7 @@ PostgreSQLSource::PostgreSQLSource( postgres::ConnectionHolderPtr connection_holder_, const std::string & query_str_, const Block & sample_block, - const UInt64 max_block_size_) + UInt64 max_block_size_) : SourceWithProgress(sample_block.cloneEmpty()) , query_str(query_str_) , max_block_size(max_block_size_) @@ -43,7 +43,7 @@ PostgreSQLSource::PostgreSQLSource( std::shared_ptr tx_, const std::string & query_str_, const Block & sample_block, - const UInt64 max_block_size_, + UInt64 max_block_size_, bool auto_commit_) : SourceWithProgress(sample_block.cloneEmpty()) , query_str(query_str_) diff --git a/src/Processors/Transforms/PostgreSQLSource.h b/src/Processors/Transforms/PostgreSQLSource.h index c7e55c09c32..bd6203042bb 100644 --- a/src/Processors/Transforms/PostgreSQLSource.h +++ b/src/Processors/Transforms/PostgreSQLSource.h @@ -24,7 +24,7 @@ public: postgres::ConnectionHolderPtr connection_holder_, const String & query_str_, const Block & sample_block, - const UInt64 max_block_size_); + UInt64 max_block_size_); String getName() const override { return "PostgreSQL"; } @@ -33,7 +33,7 @@ protected: std::shared_ptr tx_, const std::string & query_str_, const Block & sample_block, - const UInt64 max_block_size_, + UInt64 max_block_size_, bool auto_commit_); String query_str; diff --git a/src/Processors/Transforms/WindowTransform.h b/src/Processors/Transforms/WindowTransform.h index 077979e83b9..d536c8780d2 100644 --- a/src/Processors/Transforms/WindowTransform.h +++ b/src/Processors/Transforms/WindowTransform.h @@ -245,7 +245,6 @@ public: return RowNumber{first_block_number, 0}; } -public: /* * Data (formerly) inherited from ISimpleTransform, needed for the * implementation of the IProcessor interface. @@ -349,10 +348,10 @@ public: template <> struct fmt::formatter { - constexpr auto parse(format_parse_context & ctx) + static constexpr auto parse(format_parse_context & ctx) { - auto it = ctx.begin(); - auto end = ctx.end(); + const auto * it = ctx.begin(); + const auto * end = ctx.end(); /// Only support {}. if (it != end && *it != '}') diff --git a/src/QueryPipeline/SizeLimits.h b/src/QueryPipeline/SizeLimits.h index ce7e1795475..fc052714b0c 100644 --- a/src/QueryPipeline/SizeLimits.h +++ b/src/QueryPipeline/SizeLimits.h @@ -26,7 +26,7 @@ struct SizeLimits UInt64 max_bytes = 0; OverflowMode overflow_mode = OverflowMode::THROW; - SizeLimits() {} + SizeLimits() = default; SizeLimits(UInt64 max_rows_, UInt64 max_bytes_, OverflowMode overflow_mode_) : max_rows(max_rows_), max_bytes(max_bytes_), overflow_mode(overflow_mode_) {}