mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 17:12:03 +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.
|
/** 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));
|
||||||
|
|
||||||
|
@ -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(
|
||||||
|
@ -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 &>();
|
||||||
|
@ -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.
|
||||||
|
Loading…
Reference in New Issue
Block a user