Fix clang-tidy warnings in Parsers, Processors, QueryPipeline folders

This commit is contained in:
Maksim Kita 2022-03-13 12:05:14 +00:00
parent e30117a3d6
commit 2665724301
40 changed files with 74 additions and 67 deletions

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -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<ASTPtr>(ast)); }
void setWhere(ASTPtr && ast) { setExpression(where_expr_pos, std::forward<ASTPtr>(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;
};

View File

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

View File

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

View File

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

View File

@ -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<ASTUserNameWithHost>(*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<ASTUserNameWithHost>(name_)); }
void push_back(const String & name_) { names.push_back(std::make_shared<ASTUserNameWithHost>(name_)); } /// NOLINT
Strings toStrings() const;
void concatParts();

View File

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

View File

@ -207,7 +207,7 @@ private:
ParserPtr elem_parser;
public:
ParserCastExpression(ParserPtr && elem_parser_)
explicit ParserCastExpression(ParserPtr && elem_parser_)
: elem_parser(std::move(elem_parser_))
{
}

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -14,7 +14,7 @@ namespace DB
class ISchemaReader
{
public:
ISchemaReader(ReadBuffer & in_) : in(in_) {}
explicit ISchemaReader(ReadBuffer & in_) : in(in_) {}
virtual NamesAndTypesList readSchema() = 0;

View File

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

View File

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

View File

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

View File

@ -3,6 +3,8 @@
#include <Core/Block.h>
#include <Formats/FormatSettings.h>
#include <Parsers/TokenIterator.h>
#include <Parsers/IAST.h>
#include <Interpreters/Context_fwd.h>
namespace DB
{

View File

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

View File

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

View File

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

View File

@ -1,5 +1,7 @@
#pragma once
#include <Common/PODArray.h>
namespace DB
{

View File

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

View File

@ -147,7 +147,7 @@ struct Params
using RollupRule = std::pair<const RetentionPattern *, const AggregationPattern *>;
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);

View File

@ -1,5 +1,11 @@
#pragma once
#include <Common/Exception.h>
#include <Core/Types.h>
#include <Columns/IColumn.h>
#include <Processors/Chunk.h>
namespace DB
{

View File

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

View File

@ -17,7 +17,7 @@ private:
public:
String getName() const override { return "QueueBuffer"; }
QueueBuffer(Block header)
explicit QueueBuffer(Block header)
: IAccumulatingTransform(header, header)
{
}

View File

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

View File

@ -12,7 +12,7 @@ class AggregatedArenasChunkInfo : public ChunkInfo
{
public:
Arenas arenas;
AggregatedArenasChunkInfo(Arenas arenas_)
explicit AggregatedArenasChunkInfo(Arenas arenas_)
: arenas(std::move(arenas_))
{}
};

View File

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

View File

@ -28,7 +28,7 @@ PostgreSQLSource<T>::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<T>::PostgreSQLSource(
std::shared_ptr<T> 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_)

View File

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

View File

@ -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<DB::RowNumber>
{
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 != '}')

View File

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