mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Fix clang-tidy warnings in Parsers, Processors, QueryPipeline folders
This commit is contained in:
parent
e30117a3d6
commit
2665724301
@ -28,6 +28,7 @@ public:
|
|||||||
protected:
|
protected:
|
||||||
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
|
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
|
||||||
{
|
{
|
||||||
|
|
||||||
settings.ostr << (settings.hilite ? hilite_identifier : "");
|
settings.ostr << (settings.hilite ? hilite_identifier : "");
|
||||||
settings.writeIdentifier(column_name);
|
settings.writeIdentifier(column_name);
|
||||||
settings.ostr << (settings.hilite ? hilite_none : "");
|
settings.ostr << (settings.hilite ? hilite_none : "");
|
||||||
|
@ -51,7 +51,7 @@ public:
|
|||||||
|
|
||||||
void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override;
|
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())
|
return (!columns || columns->children.empty()) && (!indices || indices->children.empty()) && (!constraints || constraints->children.empty())
|
||||||
&& (!projections || projections->children.empty());
|
&& (!projections || projections->children.empty());
|
||||||
|
@ -19,7 +19,6 @@ public:
|
|||||||
/// Value is closed in brackets (HOST '127.0.0.1')
|
/// Value is closed in brackets (HOST '127.0.0.1')
|
||||||
bool second_with_brackets;
|
bool second_with_brackets;
|
||||||
|
|
||||||
public:
|
|
||||||
explicit ASTPair(bool second_with_brackets_)
|
explicit ASTPair(bool second_with_brackets_)
|
||||||
: second_with_brackets(second_with_brackets_)
|
: second_with_brackets(second_with_brackets_)
|
||||||
{
|
{
|
||||||
@ -54,7 +53,6 @@ public:
|
|||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
public:
|
|
||||||
String getID(char delim) const override;
|
String getID(char delim) const override;
|
||||||
|
|
||||||
ASTPtr clone() const override;
|
ASTPtr clone() const override;
|
||||||
|
@ -6,7 +6,7 @@
|
|||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
static inline bool isFunctionCast(const ASTFunction * function)
|
static inline bool isFunctionCast(const ASTFunction * function) /// NOLINT
|
||||||
{
|
{
|
||||||
if (function)
|
if (function)
|
||||||
return function->name == "CAST" || function->name == "_CAST";
|
return function->name == "CAST" || function->name == "_CAST";
|
||||||
|
@ -26,10 +26,10 @@ public:
|
|||||||
|
|
||||||
ASTPtr & refSelect() { return getExpression(Expression::SELECT); }
|
ASTPtr & refSelect() { return getExpression(Expression::SELECT); }
|
||||||
|
|
||||||
const ASTPtr with() const { return getExpression(Expression::WITH); }
|
ASTPtr with() const { return getExpression(Expression::WITH); }
|
||||||
const ASTPtr select() const { return getExpression(Expression::SELECT); }
|
ASTPtr select() const { return getExpression(Expression::SELECT); }
|
||||||
const ASTPtr groupBy() const { return getExpression(Expression::GROUP_BY); }
|
ASTPtr groupBy() const { return getExpression(Expression::GROUP_BY); }
|
||||||
const ASTPtr orderBy() const { return getExpression(Expression::ORDER_BY); }
|
ASTPtr orderBy() const { return getExpression(Expression::ORDER_BY); }
|
||||||
|
|
||||||
/// Set/Reset/Remove expression.
|
/// Set/Reset/Remove expression.
|
||||||
void setExpression(Expression expr, ASTPtr && ast);
|
void setExpression(Expression expr, ASTPtr && ast);
|
||||||
|
@ -17,7 +17,7 @@ public:
|
|||||||
|
|
||||||
/// new_database should be used by queries that refer to default db
|
/// new_database should be used by queries that refer to default db
|
||||||
/// and default_database is specified for remote server
|
/// 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
|
/// Returns a query prepared for execution on remote server
|
||||||
std::string getRewrittenQueryWithoutOnCluster(const std::string & new_database = {}) const;
|
std::string getRewrittenQueryWithoutOnCluster(const std::string & new_database = {}) const;
|
||||||
|
@ -91,21 +91,21 @@ public:
|
|||||||
ASTPtr & refWhere() { return getExpression(Expression::WHERE); }
|
ASTPtr & refWhere() { return getExpression(Expression::WHERE); }
|
||||||
ASTPtr & refHaving() { return getExpression(Expression::HAVING); }
|
ASTPtr & refHaving() { return getExpression(Expression::HAVING); }
|
||||||
|
|
||||||
const ASTPtr with() const { return getExpression(Expression::WITH); }
|
ASTPtr with() const { return getExpression(Expression::WITH); }
|
||||||
const ASTPtr select() const { return getExpression(Expression::SELECT); }
|
ASTPtr select() const { return getExpression(Expression::SELECT); }
|
||||||
const ASTPtr tables() const { return getExpression(Expression::TABLES); }
|
ASTPtr tables() const { return getExpression(Expression::TABLES); }
|
||||||
const ASTPtr prewhere() const { return getExpression(Expression::PREWHERE); }
|
ASTPtr prewhere() const { return getExpression(Expression::PREWHERE); }
|
||||||
const ASTPtr where() const { return getExpression(Expression::WHERE); }
|
ASTPtr where() const { return getExpression(Expression::WHERE); }
|
||||||
const ASTPtr groupBy() const { return getExpression(Expression::GROUP_BY); }
|
ASTPtr groupBy() const { return getExpression(Expression::GROUP_BY); }
|
||||||
const ASTPtr having() const { return getExpression(Expression::HAVING); }
|
ASTPtr having() const { return getExpression(Expression::HAVING); }
|
||||||
const ASTPtr window() const { return getExpression(Expression::WINDOW); }
|
ASTPtr window() const { return getExpression(Expression::WINDOW); }
|
||||||
const ASTPtr orderBy() const { return getExpression(Expression::ORDER_BY); }
|
ASTPtr orderBy() const { return getExpression(Expression::ORDER_BY); }
|
||||||
const ASTPtr limitByOffset() const { return getExpression(Expression::LIMIT_BY_OFFSET); }
|
ASTPtr limitByOffset() const { return getExpression(Expression::LIMIT_BY_OFFSET); }
|
||||||
const ASTPtr limitByLength() const { return getExpression(Expression::LIMIT_BY_LENGTH); }
|
ASTPtr limitByLength() const { return getExpression(Expression::LIMIT_BY_LENGTH); }
|
||||||
const ASTPtr limitBy() const { return getExpression(Expression::LIMIT_BY); }
|
ASTPtr limitBy() const { return getExpression(Expression::LIMIT_BY); }
|
||||||
const ASTPtr limitOffset() const { return getExpression(Expression::LIMIT_OFFSET); }
|
ASTPtr limitOffset() const { return getExpression(Expression::LIMIT_OFFSET); }
|
||||||
const ASTPtr limitLength() const { return getExpression(Expression::LIMIT_LENGTH); }
|
ASTPtr limitLength() const { return getExpression(Expression::LIMIT_LENGTH); }
|
||||||
const ASTPtr settings() const { return getExpression(Expression::SETTINGS); }
|
ASTPtr settings() const { return getExpression(Expression::SETTINGS); }
|
||||||
|
|
||||||
bool hasFiltration() const { return where() || prewhere() || having(); }
|
bool hasFiltration() const { return where() || prewhere() || having(); }
|
||||||
|
|
||||||
|
@ -37,8 +37,8 @@ public:
|
|||||||
|
|
||||||
ASTPtr clone() const override;
|
ASTPtr clone() const override;
|
||||||
|
|
||||||
const ASTPtr ttl() const { return getExpression(ttl_expr_pos); }
|
ASTPtr ttl() const { return getExpression(ttl_expr_pos); }
|
||||||
const ASTPtr where() const { return getExpression(where_expr_pos); }
|
ASTPtr where() const { return getExpression(where_expr_pos); }
|
||||||
|
|
||||||
void setTTL(ASTPtr && ast) { setExpression(ttl_expr_pos, std::forward<ASTPtr>(ast)); }
|
void setTTL(ASTPtr && ast) { setExpression(ttl_expr_pos, std::forward<ASTPtr>(ast)); }
|
||||||
void setWhere(ASTPtr && ast) { setExpression(where_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 ttl_expr_pos;
|
||||||
int where_expr_pos;
|
int where_expr_pos;
|
||||||
|
|
||||||
private:
|
|
||||||
void setExpression(int & pos, ASTPtr && ast);
|
void setExpression(int & pos, ASTPtr && ast);
|
||||||
ASTPtr getExpression(int pos, bool clone = false) const;
|
ASTPtr getExpression(int pos, bool clone = false) const;
|
||||||
};
|
};
|
||||||
|
@ -93,7 +93,7 @@ ASTPtr ASTTableOverrideList::tryGetTableOverride(const String & name) const
|
|||||||
return children[it->second];
|
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);
|
auto it = positions.find(name);
|
||||||
if (it == positions.end())
|
if (it == positions.end())
|
||||||
|
@ -40,7 +40,7 @@ public:
|
|||||||
String getID(char) const override { return "TableOverrideList"; }
|
String getID(char) const override { return "TableOverrideList"; }
|
||||||
ASTPtr clone() const override;
|
ASTPtr clone() const override;
|
||||||
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) 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);
|
void removeTableOverride(const String & name);
|
||||||
ASTPtr tryGetTableOverride(const String & name) const;
|
ASTPtr tryGetTableOverride(const String & name) const;
|
||||||
bool hasOverride(const String & name) const;
|
bool hasOverride(const String & name) const;
|
||||||
|
@ -25,7 +25,6 @@ protected:
|
|||||||
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override
|
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override
|
||||||
{
|
{
|
||||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << "USE " << (settings.hilite ? hilite_none : "") << backQuoteIfNeed(database);
|
settings.ostr << (settings.hilite ? hilite_keyword : "") << "USE " << (settings.hilite ? hilite_none : "") << backQuoteIfNeed(database);
|
||||||
return;
|
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
@ -23,7 +23,7 @@ public:
|
|||||||
void concatParts();
|
void concatParts();
|
||||||
|
|
||||||
ASTUserNameWithHost() = default;
|
ASTUserNameWithHost() = default;
|
||||||
ASTUserNameWithHost(const String & name_) : base_name(name_) {}
|
explicit ASTUserNameWithHost(const String & name_) : base_name(name_) {}
|
||||||
String getID(char) const override { return "UserNameWithHost"; }
|
String getID(char) const override { return "UserNameWithHost"; }
|
||||||
ASTPtr clone() const override { return std::make_shared<ASTUserNameWithHost>(*this); }
|
ASTPtr clone() const override { return std::make_shared<ASTUserNameWithHost>(*this); }
|
||||||
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
|
void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
|
||||||
@ -39,7 +39,7 @@ public:
|
|||||||
auto begin() const { return names.begin(); }
|
auto begin() const { return names.begin(); }
|
||||||
auto end() const { return names.end(); }
|
auto end() const { return names.end(); }
|
||||||
auto front() const { return *begin(); }
|
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;
|
Strings toStrings() const;
|
||||||
void concatParts();
|
void concatParts();
|
||||||
|
@ -33,7 +33,8 @@ class ParserToken : public IParserBase
|
|||||||
private:
|
private:
|
||||||
TokenType token_type;
|
TokenType token_type;
|
||||||
public:
|
public:
|
||||||
ParserToken(TokenType token_type_) : token_type(token_type_) {}
|
ParserToken(TokenType token_type_) : token_type(token_type_) {} /// NOLINT
|
||||||
|
|
||||||
protected:
|
protected:
|
||||||
const char * getName() const override { return "token"; }
|
const char * getName() const override { return "token"; }
|
||||||
|
|
||||||
|
@ -207,7 +207,7 @@ private:
|
|||||||
ParserPtr elem_parser;
|
ParserPtr elem_parser;
|
||||||
|
|
||||||
public:
|
public:
|
||||||
ParserCastExpression(ParserPtr && elem_parser_)
|
explicit ParserCastExpression(ParserPtr && elem_parser_)
|
||||||
: elem_parser(std::move(elem_parser_))
|
: elem_parser(std::move(elem_parser_))
|
||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
@ -69,7 +69,7 @@ public:
|
|||||||
}
|
}
|
||||||
|
|
||||||
/** Get the text that identifies this element. */
|
/** 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(); }
|
ASTPtr ptr() { return shared_from_this(); }
|
||||||
|
|
||||||
|
@ -14,7 +14,7 @@ protected:
|
|||||||
const char * getName() const override { return "EXPLAIN"; }
|
const char * getName() const override { return "EXPLAIN"; }
|
||||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||||
public:
|
public:
|
||||||
ParserExplainQuery(const char* end_) : end(end_) {}
|
explicit ParserExplainQuery(const char* end_) : end(end_) {}
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -15,7 +15,7 @@ protected:
|
|||||||
const char * getName() const override { return "Query with output"; }
|
const char * getName() const override { return "Query with output"; }
|
||||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||||
public:
|
public:
|
||||||
ParserQueryWithOutput(const char * end_) : end(end_) {}
|
explicit ParserQueryWithOutput(const char * end_) : end(end_) {}
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -21,7 +21,7 @@ protected:
|
|||||||
class ParserTablesInSelectQueryElement : public IParserBase
|
class ParserTablesInSelectQueryElement : public IParserBase
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
ParserTablesInSelectQueryElement(bool is_first_) : is_first(is_first_) {}
|
explicit ParserTablesInSelectQueryElement(bool is_first_) : is_first(is_first_) {}
|
||||||
|
|
||||||
protected:
|
protected:
|
||||||
const char * getName() const override { return "table, table function, subquery or list of joined tables"; }
|
const char * getName() const override { return "table, table function, subquery or list of joined tables"; }
|
||||||
|
@ -90,7 +90,7 @@ public:
|
|||||||
bool hasRows() const { return num_rows > 0; }
|
bool hasRows() const { return num_rows > 0; }
|
||||||
bool hasColumns() const { return !columns.empty(); }
|
bool hasColumns() const { return !columns.empty(); }
|
||||||
bool empty() const { return !hasRows() && !hasColumns(); }
|
bool empty() const { return !hasRows() && !hasColumns(); }
|
||||||
operator bool() const { return !empty(); }
|
operator bool() const { return !empty(); } /// NOLINT
|
||||||
|
|
||||||
void addColumn(ColumnPtr column);
|
void addColumn(ColumnPtr column);
|
||||||
void addColumn(size_t position, ColumnPtr column);
|
void addColumn(size_t position, ColumnPtr column);
|
||||||
|
@ -14,7 +14,7 @@ namespace DB
|
|||||||
class ISchemaReader
|
class ISchemaReader
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
ISchemaReader(ReadBuffer & in_) : in(in_) {}
|
explicit ISchemaReader(ReadBuffer & in_) : in(in_) {}
|
||||||
|
|
||||||
virtual NamesAndTypesList readSchema() = 0;
|
virtual NamesAndTypesList readSchema() = 0;
|
||||||
|
|
||||||
|
@ -44,7 +44,7 @@ class RandomAccessFileFromSeekableReadBuffer : public arrow::io::RandomAccessFil
|
|||||||
public:
|
public:
|
||||||
RandomAccessFileFromSeekableReadBuffer(SeekableReadBuffer & in_, off_t file_size_);
|
RandomAccessFileFromSeekableReadBuffer(SeekableReadBuffer & in_, off_t file_size_);
|
||||||
|
|
||||||
RandomAccessFileFromSeekableReadBuffer(SeekableReadBufferWithSize & in_);
|
explicit RandomAccessFileFromSeekableReadBuffer(SeekableReadBufferWithSize & in_);
|
||||||
|
|
||||||
arrow::Result<int64_t> GetSize() override;
|
arrow::Result<int64_t> GetSize() override;
|
||||||
|
|
||||||
|
@ -61,7 +61,7 @@ private:
|
|||||||
, target_column_idx(target_column_idx_)
|
, target_column_idx(target_column_idx_)
|
||||||
, deserialize_fn(deserialize_fn_) {}
|
, deserialize_fn(deserialize_fn_) {}
|
||||||
|
|
||||||
Action(SkipFn skip_fn_)
|
explicit Action(SkipFn skip_fn_)
|
||||||
: type(Skip)
|
: type(Skip)
|
||||||
, skip_fn(skip_fn_) {}
|
, skip_fn(skip_fn_) {}
|
||||||
|
|
||||||
|
@ -15,7 +15,7 @@ namespace DB
|
|||||||
class CapnProtoOutputStream : public kj::OutputStream
|
class CapnProtoOutputStream : public kj::OutputStream
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
CapnProtoOutputStream(WriteBuffer & out_);
|
explicit CapnProtoOutputStream(WriteBuffer & out_);
|
||||||
|
|
||||||
void write(const void * buffer, size_t size) override;
|
void write(const void * buffer, size_t size) override;
|
||||||
|
|
||||||
|
@ -3,6 +3,8 @@
|
|||||||
#include <Core/Block.h>
|
#include <Core/Block.h>
|
||||||
#include <Formats/FormatSettings.h>
|
#include <Formats/FormatSettings.h>
|
||||||
#include <Parsers/TokenIterator.h>
|
#include <Parsers/TokenIterator.h>
|
||||||
|
#include <Parsers/IAST.h>
|
||||||
|
#include <Interpreters/Context_fwd.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
@ -17,7 +17,7 @@ class WriteBuffer;
|
|||||||
class ORCOutputStream : public orc::OutputStream
|
class ORCOutputStream : public orc::OutputStream
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
ORCOutputStream(WriteBuffer & out_);
|
explicit ORCOutputStream(WriteBuffer & out_);
|
||||||
|
|
||||||
uint64_t getLength() const override;
|
uint64_t getLength() const override;
|
||||||
uint64_t getNaturalWriteSize() const override;
|
uint64_t getNaturalWriteSize() const override;
|
||||||
|
@ -22,7 +22,7 @@ class ReadBuffer;
|
|||||||
class RegexpFieldExtractor
|
class RegexpFieldExtractor
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
RegexpFieldExtractor(const FormatSettings & format_settings);
|
explicit RegexpFieldExtractor(const FormatSettings & format_settings);
|
||||||
|
|
||||||
/// Return true if row was successfully parsed and row fields were extracted.
|
/// Return true if row was successfully parsed and row fields were extracted.
|
||||||
bool parseRow(PeekableReadBuffer & buf);
|
bool parseRow(PeekableReadBuffer & buf);
|
||||||
|
@ -53,7 +53,7 @@ public:
|
|||||||
|
|
||||||
bool parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) override;
|
bool parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) override;
|
||||||
bool parseRowEndWithDiagnosticInfo(WriteBuffer & out) override;
|
bool parseRowEndWithDiagnosticInfo(WriteBuffer & out) override;
|
||||||
FormatSettings::EscapingRule getEscapingRule()
|
FormatSettings::EscapingRule getEscapingRule() const
|
||||||
{
|
{
|
||||||
return is_raw ? FormatSettings::EscapingRule::Raw : FormatSettings::EscapingRule::Escaped;
|
return is_raw ? FormatSettings::EscapingRule::Raw : FormatSettings::EscapingRule::Escaped;
|
||||||
}
|
}
|
||||||
|
@ -1,5 +1,7 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
|
#include <Common/PODArray.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
|
@ -89,7 +89,7 @@ inline static const Patterns & selectPatternsForMetricType(const Graphite::Param
|
|||||||
|
|
||||||
Graphite::RollupRule selectPatternForPath(
|
Graphite::RollupRule selectPatternForPath(
|
||||||
const Graphite::Params & params,
|
const Graphite::Params & params,
|
||||||
const StringRef path)
|
StringRef path)
|
||||||
{
|
{
|
||||||
const Graphite::Pattern * first_match = &undef_pattern;
|
const Graphite::Pattern * first_match = &undef_pattern;
|
||||||
|
|
||||||
|
@ -147,7 +147,7 @@ struct Params
|
|||||||
|
|
||||||
using RollupRule = std::pair<const RetentionPattern *, const AggregationPattern *>;
|
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);
|
void setGraphitePatternsFromConfig(ContextPtr context, const String & config_element, Graphite::Params & params);
|
||||||
|
|
||||||
|
@ -1,5 +1,11 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
|
#include <Common/Exception.h>
|
||||||
|
#include <Core/Types.h>
|
||||||
|
#include <Columns/IColumn.h>
|
||||||
|
#include <Processors/Chunk.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
|
@ -214,7 +214,7 @@ protected:
|
|||||||
public:
|
public:
|
||||||
using Data = State::Data;
|
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_) {}
|
Port(Block header_, IProcessor * processor_) : header(std::move(header_)), processor(processor_) {}
|
||||||
|
|
||||||
void setUpdateInfo(UpdateInfo * info) { update_info = info; }
|
void setUpdateInfo(UpdateInfo * info) { update_info = info; }
|
||||||
@ -303,12 +303,12 @@ public:
|
|||||||
|
|
||||||
Chunk ALWAYS_INLINE pull(bool set_not_needed = false)
|
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)
|
if (pull_data.exception)
|
||||||
std::rethrow_exception(data_.exception);
|
std::rethrow_exception(pull_data.exception);
|
||||||
|
|
||||||
return std::move(data_.chunk);
|
return std::move(pull_data.chunk);
|
||||||
}
|
}
|
||||||
|
|
||||||
bool ALWAYS_INLINE isFinished() const
|
bool ALWAYS_INLINE isFinished() const
|
||||||
@ -396,7 +396,7 @@ public:
|
|||||||
|
|
||||||
void ALWAYS_INLINE pushException(std::exception_ptr exception)
|
void ALWAYS_INLINE pushException(std::exception_ptr exception)
|
||||||
{
|
{
|
||||||
pushData({.chunk = {}, .exception = std::move(exception)});
|
pushData({.chunk = {}, .exception = exception});
|
||||||
}
|
}
|
||||||
|
|
||||||
void ALWAYS_INLINE pushData(Data data_)
|
void ALWAYS_INLINE pushData(Data data_)
|
||||||
|
@ -17,7 +17,7 @@ private:
|
|||||||
public:
|
public:
|
||||||
String getName() const override { return "QueueBuffer"; }
|
String getName() const override { return "QueueBuffer"; }
|
||||||
|
|
||||||
QueueBuffer(Block header)
|
explicit QueueBuffer(Block header)
|
||||||
: IAccumulatingTransform(header, header)
|
: IAccumulatingTransform(header, header)
|
||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
@ -11,7 +11,7 @@ namespace DB
|
|||||||
|
|
||||||
struct ChunkInfoWithAllocatedBytes : public ChunkInfo
|
struct ChunkInfoWithAllocatedBytes : public ChunkInfo
|
||||||
{
|
{
|
||||||
ChunkInfoWithAllocatedBytes(Int64 allocated_bytes_)
|
explicit ChunkInfoWithAllocatedBytes(Int64 allocated_bytes_)
|
||||||
: allocated_bytes(allocated_bytes_) {}
|
: allocated_bytes(allocated_bytes_) {}
|
||||||
Int64 allocated_bytes;
|
Int64 allocated_bytes;
|
||||||
};
|
};
|
||||||
|
@ -12,7 +12,7 @@ class AggregatedArenasChunkInfo : public ChunkInfo
|
|||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
Arenas arenas;
|
Arenas arenas;
|
||||||
AggregatedArenasChunkInfo(Arenas arenas_)
|
explicit AggregatedArenasChunkInfo(Arenas arenas_)
|
||||||
: arenas(std::move(arenas_))
|
: arenas(std::move(arenas_))
|
||||||
{}
|
{}
|
||||||
};
|
};
|
||||||
|
@ -20,7 +20,7 @@ struct RowSourcePart
|
|||||||
|
|
||||||
RowSourcePart() = default;
|
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");
|
static_assert(sizeof(*this) == 1, "Size of RowSourcePart is too big due to compiler settings");
|
||||||
setSourceNum(source_num);
|
setSourceNum(source_num);
|
||||||
|
@ -28,7 +28,7 @@ PostgreSQLSource<T>::PostgreSQLSource(
|
|||||||
postgres::ConnectionHolderPtr connection_holder_,
|
postgres::ConnectionHolderPtr connection_holder_,
|
||||||
const std::string & query_str_,
|
const std::string & query_str_,
|
||||||
const Block & sample_block,
|
const Block & sample_block,
|
||||||
const UInt64 max_block_size_)
|
UInt64 max_block_size_)
|
||||||
: SourceWithProgress(sample_block.cloneEmpty())
|
: SourceWithProgress(sample_block.cloneEmpty())
|
||||||
, query_str(query_str_)
|
, query_str(query_str_)
|
||||||
, max_block_size(max_block_size_)
|
, max_block_size(max_block_size_)
|
||||||
@ -43,7 +43,7 @@ PostgreSQLSource<T>::PostgreSQLSource(
|
|||||||
std::shared_ptr<T> tx_,
|
std::shared_ptr<T> tx_,
|
||||||
const std::string & query_str_,
|
const std::string & query_str_,
|
||||||
const Block & sample_block,
|
const Block & sample_block,
|
||||||
const UInt64 max_block_size_,
|
UInt64 max_block_size_,
|
||||||
bool auto_commit_)
|
bool auto_commit_)
|
||||||
: SourceWithProgress(sample_block.cloneEmpty())
|
: SourceWithProgress(sample_block.cloneEmpty())
|
||||||
, query_str(query_str_)
|
, query_str(query_str_)
|
||||||
|
@ -24,7 +24,7 @@ public:
|
|||||||
postgres::ConnectionHolderPtr connection_holder_,
|
postgres::ConnectionHolderPtr connection_holder_,
|
||||||
const String & query_str_,
|
const String & query_str_,
|
||||||
const Block & sample_block,
|
const Block & sample_block,
|
||||||
const UInt64 max_block_size_);
|
UInt64 max_block_size_);
|
||||||
|
|
||||||
String getName() const override { return "PostgreSQL"; }
|
String getName() const override { return "PostgreSQL"; }
|
||||||
|
|
||||||
@ -33,7 +33,7 @@ protected:
|
|||||||
std::shared_ptr<T> tx_,
|
std::shared_ptr<T> tx_,
|
||||||
const std::string & query_str_,
|
const std::string & query_str_,
|
||||||
const Block & sample_block,
|
const Block & sample_block,
|
||||||
const UInt64 max_block_size_,
|
UInt64 max_block_size_,
|
||||||
bool auto_commit_);
|
bool auto_commit_);
|
||||||
|
|
||||||
String query_str;
|
String query_str;
|
||||||
|
@ -245,7 +245,6 @@ public:
|
|||||||
return RowNumber{first_block_number, 0};
|
return RowNumber{first_block_number, 0};
|
||||||
}
|
}
|
||||||
|
|
||||||
public:
|
|
||||||
/*
|
/*
|
||||||
* Data (formerly) inherited from ISimpleTransform, needed for the
|
* Data (formerly) inherited from ISimpleTransform, needed for the
|
||||||
* implementation of the IProcessor interface.
|
* implementation of the IProcessor interface.
|
||||||
@ -349,10 +348,10 @@ public:
|
|||||||
template <>
|
template <>
|
||||||
struct fmt::formatter<DB::RowNumber>
|
struct fmt::formatter<DB::RowNumber>
|
||||||
{
|
{
|
||||||
constexpr auto parse(format_parse_context & ctx)
|
static constexpr auto parse(format_parse_context & ctx)
|
||||||
{
|
{
|
||||||
auto it = ctx.begin();
|
const auto * it = ctx.begin();
|
||||||
auto end = ctx.end();
|
const auto * end = ctx.end();
|
||||||
|
|
||||||
/// Only support {}.
|
/// Only support {}.
|
||||||
if (it != end && *it != '}')
|
if (it != end && *it != '}')
|
||||||
|
@ -26,7 +26,7 @@ struct SizeLimits
|
|||||||
UInt64 max_bytes = 0;
|
UInt64 max_bytes = 0;
|
||||||
OverflowMode overflow_mode = OverflowMode::THROW;
|
OverflowMode overflow_mode = OverflowMode::THROW;
|
||||||
|
|
||||||
SizeLimits() {}
|
SizeLimits() = default;
|
||||||
SizeLimits(UInt64 max_rows_, UInt64 max_bytes_, OverflowMode overflow_mode_)
|
SizeLimits(UInt64 max_rows_, UInt64 max_bytes_, OverflowMode overflow_mode_)
|
||||||
: max_rows(max_rows_), max_bytes(max_bytes_), overflow_mode(overflow_mode_) {}
|
: max_rows(max_rows_), max_bytes(max_bytes_), overflow_mode(overflow_mode_) {}
|
||||||
|
|
||||||
|
Loading…
Reference in New Issue
Block a user