mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Continue: ContextPtr -> const ContextPtr & in parameters
This commit is contained in:
parent
059c5c0f0a
commit
9ca08f8fda
@ -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)
|
||||
@ -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));
|
||||
|
||||
|
@ -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,8 +128,8 @@ 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(
|
||||
|
@ -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 &>();
|
||||
|
@ -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.
|
||||
|
Loading…
Reference in New Issue
Block a user