Merge pull request #38902 from ClickHouse/interpreter_cleanup_2

Interpreter cleanup: ContextPtr -> const ContextPtr & in parameters
This commit is contained in:
Igor Nikonov 2022-07-06 23:46:55 +02:00 committed by GitHub
commit 77317c65fd
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
5 changed files with 37 additions and 37 deletions

View File

@ -12,12 +12,12 @@ namespace DB
class IInterpreterUnionOrSelectQuery : public IInterpreter
{
public:
IInterpreterUnionOrSelectQuery(const ASTPtr & query_ptr_, ContextPtr context_, const SelectQueryOptions & options_)
IInterpreterUnionOrSelectQuery(const ASTPtr & query_ptr_, const ContextPtr & context_, const SelectQueryOptions & options_)
: IInterpreterUnionOrSelectQuery(query_ptr_, Context::createCopy(context_), options_)
{
}
IInterpreterUnionOrSelectQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_, const SelectQueryOptions & options_)
IInterpreterUnionOrSelectQuery(const ASTPtr & query_ptr_, const ContextMutablePtr & context_, const SelectQueryOptions & options_)
: query_ptr(query_ptr_)
, context(context_)
, options(options_)

View File

@ -159,7 +159,7 @@ String InterpreterSelectQuery::generateFilterActions(ActionsDAGPtr & actions, co
InterpreterSelectQuery::InterpreterSelectQuery(
const ASTPtr & query_ptr_,
ContextPtr context_,
const ContextPtr & context_,
const SelectQueryOptions & options_,
const Names & required_result_column_names_)
: InterpreterSelectQuery(query_ptr_, context_, std::nullopt, nullptr, options_, required_result_column_names_)
@ -167,7 +167,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
InterpreterSelectQuery::InterpreterSelectQuery(
const ASTPtr & query_ptr_,
ContextMutablePtr context_,
const ContextMutablePtr & context_,
const SelectQueryOptions & options_,
const Names & required_result_column_names_)
: InterpreterSelectQuery(query_ptr_, context_, std::nullopt, nullptr, options_, required_result_column_names_)
@ -175,7 +175,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
InterpreterSelectQuery::InterpreterSelectQuery(
const ASTPtr & query_ptr_,
ContextPtr context_,
const ContextPtr & context_,
Pipe input_pipe_,
const SelectQueryOptions & options_)
: InterpreterSelectQuery(query_ptr_, context_, std::move(input_pipe_), nullptr, options_.copy().noSubquery())
@ -183,7 +183,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
InterpreterSelectQuery::InterpreterSelectQuery(
const ASTPtr & query_ptr_,
ContextPtr context_,
const ContextPtr & context_,
const StoragePtr & storage_,
const StorageMetadataPtr & metadata_snapshot_,
const SelectQueryOptions & options_)
@ -192,7 +192,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
InterpreterSelectQuery::InterpreterSelectQuery(
const ASTPtr & query_ptr_,
ContextPtr context_,
const ContextPtr & context_,
const SelectQueryOptions & options_,
SubqueriesForSets subquery_for_sets_,
PreparedSets prepared_sets_)
@ -206,7 +206,7 @@ InterpreterSelectQuery::~InterpreterSelectQuery() = default;
/** There are no limits on the maximum size of the result for the subquery.
* Since the result of the query is not the result of the entire query.
*/
static ContextPtr getSubqueryContext(ContextPtr context)
static ContextPtr getSubqueryContext(const ContextPtr & context)
{
auto subquery_context = Context::createCopy(context);
Settings subquery_settings = context->getSettings();
@ -239,7 +239,7 @@ static void rewriteMultipleJoins(ASTPtr & query, const TablesWithColumns & table
/// Checks that the current user has the SELECT privilege.
static void checkAccessRightsForSelect(
ContextPtr context,
const ContextPtr & context,
const StorageID & table_id,
const StorageMetadataPtr & table_metadata,
const TreeRewriterResult & syntax_analyzer_result)
@ -282,7 +282,7 @@ static bool shouldIgnoreQuotaAndLimits(const StorageID & table_id)
InterpreterSelectQuery::InterpreterSelectQuery(
const ASTPtr & query_ptr_,
ContextPtr context_,
const ContextPtr & context_,
std::optional<Pipe> input_pipe_,
const StoragePtr & storage_,
const SelectQueryOptions & options_,
@ -304,7 +304,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
InterpreterSelectQuery::InterpreterSelectQuery(
const ASTPtr & query_ptr_,
ContextMutablePtr context_,
const ContextMutablePtr & context_,
std::optional<Pipe> input_pipe_,
const StoragePtr & storage_,
const SelectQueryOptions & options_,
@ -816,7 +816,7 @@ Block InterpreterSelectQuery::getSampleBlockImpl()
return analysis_result.final_projection->getResultColumns();
}
static Field getWithFillFieldValue(const ASTPtr & node, ContextPtr context)
static Field getWithFillFieldValue(const ASTPtr & node, const ContextPtr & context)
{
auto [field, type] = evaluateConstantExpression(node, context);
@ -826,7 +826,7 @@ static Field getWithFillFieldValue(const ASTPtr & node, ContextPtr context)
return field;
}
static std::pair<Field, std::optional<IntervalKind>> getWithFillStep(const ASTPtr & node, ContextPtr context)
static std::pair<Field, std::optional<IntervalKind>> getWithFillStep(const ASTPtr & node, const ContextPtr & context)
{
auto [field, type] = evaluateConstantExpression(node, context);
@ -839,7 +839,7 @@ static std::pair<Field, std::optional<IntervalKind>> getWithFillStep(const ASTPt
throw Exception("Illegal type " + type->getName() + " of WITH FILL expression, must be numeric type", ErrorCodes::INVALID_WITH_FILL_EXPRESSION);
}
static FillColumnDescription getWithFillDescription(const ASTOrderByElement & order_by_elem, ContextPtr context)
static FillColumnDescription getWithFillDescription(const ASTOrderByElement & order_by_elem, const ContextPtr & context)
{
FillColumnDescription descr;
@ -886,7 +886,7 @@ static FillColumnDescription getWithFillDescription(const ASTOrderByElement & or
return descr;
}
SortDescription InterpreterSelectQuery::getSortDescription(const ASTSelectQuery & query, ContextPtr context_)
SortDescription InterpreterSelectQuery::getSortDescription(const ASTSelectQuery & query, const ContextPtr & context_)
{
SortDescription order_descr;
order_descr.reserve(query.orderBy()->children.size());
@ -998,7 +998,7 @@ static SortDescription getSortDescriptionFromGroupBy(const ASTSelectQuery & quer
return order_descr;
}
static UInt64 getLimitUIntValue(const ASTPtr & node, ContextPtr context, const std::string & expr)
static UInt64 getLimitUIntValue(const ASTPtr & node, const ContextPtr & context, const std::string & expr)
{
const auto & [field, type] = evaluateConstantExpression(node, context);
@ -1016,7 +1016,7 @@ static UInt64 getLimitUIntValue(const ASTPtr & node, ContextPtr context, const s
}
static std::pair<UInt64, UInt64> getLimitLengthAndOffset(const ASTSelectQuery & query, ContextPtr context)
static std::pair<UInt64, UInt64> getLimitLengthAndOffset(const ASTSelectQuery & query, const ContextPtr & context)
{
UInt64 length = 0;
UInt64 offset = 0;
@ -1033,7 +1033,7 @@ static std::pair<UInt64, UInt64> getLimitLengthAndOffset(const ASTSelectQuery &
}
UInt64 InterpreterSelectQuery::getLimitForSorting(const ASTSelectQuery & query, ContextPtr context_)
UInt64 InterpreterSelectQuery::getLimitForSorting(const ASTSelectQuery & query, const ContextPtr & context_)
{
/// Partial sort can be done if there is LIMIT but no DISTINCT or LIMIT BY, neither ARRAY JOIN.
if (!query.distinct && !query.limitBy() && !query.limit_with_ties && !query.arrayJoinExpressionList().first && query.limitLength())
@ -1631,7 +1631,7 @@ static void executeMergeAggregatedImpl(
}
void InterpreterSelectQuery::addEmptySourceToQueryPlan(
QueryPlan & query_plan, const Block & source_header, const SelectQueryInfo & query_info, ContextPtr context_)
QueryPlan & query_plan, const Block & source_header, const SelectQueryInfo & query_info, const ContextPtr & context_)
{
Pipe pipe(std::make_shared<NullSource>(source_header));

View File

@ -52,27 +52,27 @@ public:
InterpreterSelectQuery(
const ASTPtr & query_ptr_,
ContextPtr context_,
const ContextPtr & context_,
const SelectQueryOptions &,
const Names & required_result_column_names_ = Names{});
InterpreterSelectQuery(
const ASTPtr & query_ptr_,
ContextMutablePtr context_,
const ContextMutablePtr & context_,
const SelectQueryOptions &,
const Names & required_result_column_names_ = Names{});
/// Read data not from the table specified in the query, but from the prepared pipe `input`.
InterpreterSelectQuery(
const ASTPtr & query_ptr_,
ContextPtr context_,
const ContextPtr & context_,
Pipe input_pipe_,
const SelectQueryOptions & = {});
/// Read data not from the table specified in the query, but from the specified `storage_`.
InterpreterSelectQuery(
const ASTPtr & query_ptr_,
ContextPtr context_,
const ContextPtr & context_,
const StoragePtr & storage_,
const StorageMetadataPtr & metadata_snapshot_ = nullptr,
const SelectQueryOptions & = {});
@ -81,7 +81,7 @@ public:
/// TODO: Find a general way of sharing sets among different interpreters, such as subqueries.
InterpreterSelectQuery(
const ASTPtr & query_ptr_,
ContextPtr context_,
const ContextPtr & context_,
const SelectQueryOptions &,
SubqueriesForSets subquery_for_sets_,
PreparedSets prepared_sets_);
@ -112,7 +112,7 @@ public:
bool hasAggregation() const { return query_analyzer->hasAggregation(); }
static void addEmptySourceToQueryPlan(
QueryPlan & query_plan, const Block & source_header, const SelectQueryInfo & query_info, ContextPtr context_);
QueryPlan & query_plan, const Block & source_header, const SelectQueryInfo & query_info, const ContextPtr & context_);
Names getRequiredColumns() { return required_columns; }
@ -128,13 +128,13 @@ public:
/// It will set shard_num and shard_count to the client_info
void setProperClientInfo(size_t replica_num, size_t replica_count);
static SortDescription getSortDescription(const ASTSelectQuery & query, ContextPtr context);
static UInt64 getLimitForSorting(const ASTSelectQuery & query, ContextPtr context);
static SortDescription getSortDescription(const ASTSelectQuery & query, const ContextPtr & context);
static UInt64 getLimitForSorting(const ASTSelectQuery & query, const ContextPtr & context);
private:
InterpreterSelectQuery(
const ASTPtr & query_ptr_,
ContextPtr context_,
const ContextPtr & context_,
std::optional<Pipe> input_pipe,
const StoragePtr & storage_,
const SelectQueryOptions &,
@ -145,7 +145,7 @@ private:
InterpreterSelectQuery(
const ASTPtr & query_ptr_,
ContextMutablePtr context_,
const ContextMutablePtr & context_,
std::optional<Pipe> input_pipe,
const StoragePtr & storage_,
const SelectQueryOptions &,

View File

@ -42,7 +42,7 @@ static std::pair<Field, std::shared_ptr<const IDataType>> getFieldAndDataTypeFro
return {res, type};
}
std::pair<Field, std::shared_ptr<const IDataType>> evaluateConstantExpression(const ASTPtr & node, ContextPtr context)
std::pair<Field, std::shared_ptr<const IDataType>> evaluateConstantExpression(const ASTPtr & node, const ContextPtr & context)
{
if (ASTLiteral * literal = node->as<ASTLiteral>())
return getFieldAndDataTypeFromLiteral(literal);
@ -105,7 +105,7 @@ std::pair<Field, std::shared_ptr<const IDataType>> evaluateConstantExpression(co
}
ASTPtr evaluateConstantExpressionAsLiteral(const ASTPtr & node, ContextPtr context)
ASTPtr evaluateConstantExpressionAsLiteral(const ASTPtr & node, const ContextPtr & context)
{
/// If it's already a literal.
if (node->as<ASTLiteral>())
@ -113,7 +113,7 @@ ASTPtr evaluateConstantExpressionAsLiteral(const ASTPtr & node, ContextPtr conte
return std::make_shared<ASTLiteral>(evaluateConstantExpression(node, context).first);
}
ASTPtr evaluateConstantExpressionOrIdentifierAsLiteral(const ASTPtr & node, ContextPtr context)
ASTPtr evaluateConstantExpressionOrIdentifierAsLiteral(const ASTPtr & node, const ContextPtr & context)
{
if (const auto * id = node->as<ASTIdentifier>())
return std::make_shared<ASTLiteral>(id->name());
@ -121,7 +121,7 @@ ASTPtr evaluateConstantExpressionOrIdentifierAsLiteral(const ASTPtr & node, Cont
return evaluateConstantExpressionAsLiteral(node, context);
}
ASTPtr evaluateConstantExpressionForDatabaseName(const ASTPtr & node, ContextPtr context)
ASTPtr evaluateConstantExpressionForDatabaseName(const ASTPtr & node, const ContextPtr & context)
{
ASTPtr res = evaluateConstantExpressionOrIdentifierAsLiteral(node, context);
auto & literal = res->as<ASTLiteral &>();

View File

@ -22,25 +22,25 @@ using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
* Throws exception if it's not a constant expression.
* Quite suboptimal.
*/
std::pair<Field, std::shared_ptr<const IDataType>> evaluateConstantExpression(const ASTPtr & node, ContextPtr context);
std::pair<Field, std::shared_ptr<const IDataType>> evaluateConstantExpression(const ASTPtr & node, const ContextPtr & context);
/** Evaluate constant expression and returns ASTLiteral with its value.
*/
ASTPtr evaluateConstantExpressionAsLiteral(const ASTPtr & node, ContextPtr context);
ASTPtr evaluateConstantExpressionAsLiteral(const ASTPtr & node, const ContextPtr & context);
/** Evaluate constant expression and returns ASTLiteral with its value.
* Also, if AST is identifier, then return string literal with its name.
* Useful in places where some name may be specified as identifier, or as result of a constant expression.
*/
ASTPtr evaluateConstantExpressionOrIdentifierAsLiteral(const ASTPtr & node, ContextPtr context);
ASTPtr evaluateConstantExpressionOrIdentifierAsLiteral(const ASTPtr & node, const ContextPtr & context);
/** The same as evaluateConstantExpressionOrIdentifierAsLiteral(...),
* but if result is an empty string, replace it with current database name
* or default database name.
*/
ASTPtr evaluateConstantExpressionForDatabaseName(const ASTPtr & node, ContextPtr context);
ASTPtr evaluateConstantExpressionForDatabaseName(const ASTPtr & node, const ContextPtr & context);
/** Try to fold condition to countable set of constant values.
* @param node a condition that we try to fold.