Continue: ContextPtr -> const ContextPtr & in parameters

This commit is contained in:
Igor Nikonov 2022-07-06 12:59:25 +00:00
parent 059c5c0f0a
commit 9ca08f8fda
4 changed files with 21 additions and 21 deletions

View File

@ -206,7 +206,7 @@ InterpreterSelectQuery::~InterpreterSelectQuery() = default;
/** There are no limits on the maximum size of the result for the subquery. /** 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. * 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); auto subquery_context = Context::createCopy(context);
Settings subquery_settings = context->getSettings(); 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. /// Checks that the current user has the SELECT privilege.
static void checkAccessRightsForSelect( static void checkAccessRightsForSelect(
ContextPtr context, const ContextPtr & context,
const StorageID & table_id, const StorageID & table_id,
const StorageMetadataPtr & table_metadata, const StorageMetadataPtr & table_metadata,
const TreeRewriterResult & syntax_analyzer_result) const TreeRewriterResult & syntax_analyzer_result)
@ -816,7 +816,7 @@ Block InterpreterSelectQuery::getSampleBlockImpl()
return analysis_result.final_projection->getResultColumns(); 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); auto [field, type] = evaluateConstantExpression(node, context);
@ -826,7 +826,7 @@ static Field getWithFillFieldValue(const ASTPtr & node, ContextPtr context)
return field; 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); 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); 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; FillColumnDescription descr;
@ -886,7 +886,7 @@ static FillColumnDescription getWithFillDescription(const ASTOrderByElement & or
return descr; return descr;
} }
SortDescription InterpreterSelectQuery::getSortDescription(const ASTSelectQuery & query, ContextPtr context_) SortDescription InterpreterSelectQuery::getSortDescription(const ASTSelectQuery & query, const ContextPtr & context_)
{ {
SortDescription order_descr; SortDescription order_descr;
order_descr.reserve(query.orderBy()->children.size()); order_descr.reserve(query.orderBy()->children.size());
@ -998,7 +998,7 @@ static SortDescription getSortDescriptionFromGroupBy(const ASTSelectQuery & quer
return order_descr; 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); 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 length = 0;
UInt64 offset = 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. /// 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()) if (!query.distinct && !query.limitBy() && !query.limit_with_ties && !query.arrayJoinExpressionList().first && query.limitLength())
@ -1631,7 +1631,7 @@ static void executeMergeAggregatedImpl(
} }
void InterpreterSelectQuery::addEmptySourceToQueryPlan( 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)); Pipe pipe(std::make_shared<NullSource>(source_header));

View File

@ -112,7 +112,7 @@ public:
bool hasAggregation() const { return query_analyzer->hasAggregation(); } bool hasAggregation() const { return query_analyzer->hasAggregation(); }
static void addEmptySourceToQueryPlan( 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; } Names getRequiredColumns() { return required_columns; }
@ -128,8 +128,8 @@ public:
/// It will set shard_num and shard_count to the client_info /// It will set shard_num and shard_count to the client_info
void setProperClientInfo(size_t replica_num, size_t replica_count); void setProperClientInfo(size_t replica_num, size_t replica_count);
static SortDescription getSortDescription(const ASTSelectQuery & query, ContextPtr context); static SortDescription getSortDescription(const ASTSelectQuery & query, const ContextPtr & context);
static UInt64 getLimitForSorting(const ASTSelectQuery & query, ContextPtr context); static UInt64 getLimitForSorting(const ASTSelectQuery & query, const ContextPtr & context);
private: private:
InterpreterSelectQuery( InterpreterSelectQuery(

View File

@ -42,7 +42,7 @@ static std::pair<Field, std::shared_ptr<const IDataType>> getFieldAndDataTypeFro
return {res, type}; 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>()) if (ASTLiteral * literal = node->as<ASTLiteral>())
return getFieldAndDataTypeFromLiteral(literal); 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 it's already a literal.
if (node->as<ASTLiteral>()) if (node->as<ASTLiteral>())
@ -113,7 +113,7 @@ ASTPtr evaluateConstantExpressionAsLiteral(const ASTPtr & node, ContextPtr conte
return std::make_shared<ASTLiteral>(evaluateConstantExpression(node, context).first); 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>()) if (const auto * id = node->as<ASTIdentifier>())
return std::make_shared<ASTLiteral>(id->name()); return std::make_shared<ASTLiteral>(id->name());
@ -121,7 +121,7 @@ ASTPtr evaluateConstantExpressionOrIdentifierAsLiteral(const ASTPtr & node, Cont
return evaluateConstantExpressionAsLiteral(node, context); return evaluateConstantExpressionAsLiteral(node, context);
} }
ASTPtr evaluateConstantExpressionForDatabaseName(const ASTPtr & node, ContextPtr context) ASTPtr evaluateConstantExpressionForDatabaseName(const ASTPtr & node, const ContextPtr & context)
{ {
ASTPtr res = evaluateConstantExpressionOrIdentifierAsLiteral(node, context); ASTPtr res = evaluateConstantExpressionOrIdentifierAsLiteral(node, context);
auto & literal = res->as<ASTLiteral &>(); 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. * Throws exception if it's not a constant expression.
* Quite suboptimal. * 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. /** 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. /** Evaluate constant expression and returns ASTLiteral with its value.
* Also, if AST is identifier, then return string literal with its name. * 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. * 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(...), /** The same as evaluateConstantExpressionOrIdentifierAsLiteral(...),
* but if result is an empty string, replace it with current database name * but if result is an empty string, replace it with current database name
* or default 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. /** Try to fold condition to countable set of constant values.
* @param node a condition that we try to fold. * @param node a condition that we try to fold.