mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Reduce dependencies on ASTFunction.h
481 -> 230
This commit is contained in:
parent
91bf938075
commit
b2cfa70541
@ -8,6 +8,7 @@
|
||||
#include <Common/setThreadName.h>
|
||||
#include <IO/ConnectionTimeoutsContext.h>
|
||||
#include <Interpreters/InterpreterInsertQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Processors/Transforms/ExpressionTransform.h>
|
||||
#include <QueryPipeline/QueryPipelineBuilder.h>
|
||||
#include <QueryPipeline/Chain.h>
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include "Internals.h"
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Processors/Executors/PullingPipelineExecutor.h>
|
||||
#include <Processors/Transforms/SquashingChunksTransform.h>
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include "ClusterPartition.h"
|
||||
|
||||
#include <Core/Defines.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
|
||||
#include <base/map.h>
|
||||
#include <boost/algorithm/string/join.hpp>
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Databases/DDLDependencyVisitor.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <filesystem>
|
||||
|
||||
|
@ -7,6 +7,7 @@
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/InterpreterCreateQuery.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ParserCreateQuery.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
|
@ -18,6 +18,7 @@
|
||||
#include <base/getFQDNOrHostName.h>
|
||||
#include <Parsers/ASTAlterQuery.h>
|
||||
#include <Parsers/ASTDropQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ParserCreateQuery.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Interpreters/InterpreterCreateQuery.h>
|
||||
|
@ -1,7 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <base/types.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Storages/IStorage_fwd.h>
|
||||
#include <Databases/IDatabase.h>
|
||||
|
@ -8,6 +8,7 @@
|
||||
#include <Databases/SQLite/fetchSQLiteTableStructure.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTColumnDeclaration.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Storages/StorageSQLite.h>
|
||||
#include <Databases/SQLite/SQLiteUtils.h>
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <Parsers/ASTColumnDeclaration.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,7 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <string>
|
||||
#include <Interpreters/GatherFunctionQuantileVisitor.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <base/types.h>
|
||||
|
||||
|
@ -1,13 +1,14 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Interpreters/InDepthNodeVisitor.h>
|
||||
#include <AggregateFunctions/AggregateFunctionQuantile.h>
|
||||
#include <Interpreters/InDepthNodeVisitor.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ASTFunction;
|
||||
|
||||
/// Gather all the `quantile*` functions
|
||||
class GatherFunctionQuantileData
|
||||
{
|
||||
|
@ -1,15 +1,16 @@
|
||||
#include <Interpreters/InterpreterCreateFunctionQuery.h>
|
||||
|
||||
#include <Access/ContextAccess.h>
|
||||
#include <Parsers/ASTCreateFunctionQuery.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Interpreters/ExpressionAnalyzer.h>
|
||||
#include <Interpreters/FunctionNameNormalizer.h>
|
||||
#include <Interpreters/UserDefinedSQLObjectsLoader.h>
|
||||
#include <Interpreters/UserDefinedSQLFunctionFactory.h>
|
||||
#include <Interpreters/UserDefinedSQLObjectsLoader.h>
|
||||
#include <Interpreters/executeDDLQueryOnCluster.h>
|
||||
#include <Parsers/ASTCreateFunctionQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,12 +1,13 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Interpreters/InDepthNodeVisitor.h>
|
||||
#include <Storages/StorageInMemoryMetadata.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ASTFunction;
|
||||
|
||||
/// Rewrites functions to subcolumns, if possible, to reduce amount of read data.
|
||||
/// E.g. 'length(arr)' -> 'arr.size0', 'col IS NULL' -> 'col.null'
|
||||
class RewriteFunctionToSubcolumnData
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Interpreters/TreeRewriter.h>
|
||||
#include <Interpreters/IdentifierSemantic.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Processors/Sources/SourceFromSingleChunk.h>
|
||||
|
@ -1,3 +1,4 @@
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTProjectionDeclaration.h>
|
||||
#include <Common/quoteString.h>
|
||||
|
||||
|
@ -1,6 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/IAST.h>
|
||||
|
||||
|
||||
|
@ -1,3 +1,4 @@
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Processors/QueryPlan/ReadFromMergeTree.h>
|
||||
#include <QueryPipeline/QueryPipelineBuilder.h>
|
||||
#include <Processors/ConcatProcessor.h>
|
||||
|
@ -17,7 +17,6 @@
|
||||
#include <Parsers/ASTColumnDeclaration.h>
|
||||
#include <Parsers/ASTConstraintDeclaration.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTIndexDeclaration.h>
|
||||
#include <Parsers/ASTProjectionDeclaration.h>
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <Storages/StorageDistributed.h>
|
||||
#include <Disks/StoragePolicy.h>
|
||||
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTInsertQuery.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
|
@ -6,13 +6,13 @@
|
||||
#include <Core/SortDescription.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Storages/SelectQueryInfo.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ASTFunction;
|
||||
class Context;
|
||||
class IFunction;
|
||||
using FunctionBasePtr = std::shared_ptr<IFunctionBase>;
|
||||
|
@ -8,6 +8,7 @@
|
||||
#include <Storages/MergeTree/MergeTreeIndexConditionBloomFilter.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Interpreters/BloomFilterHash.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -4,6 +4,8 @@
|
||||
#include <Interpreters/ExpressionAnalyzer.h>
|
||||
#include <Interpreters/TreeRewriter.h>
|
||||
|
||||
#include <Parsers/ASTFunction.h>
|
||||
|
||||
#include <Poco/Logger.h>
|
||||
#include <Common/FieldVisitorsAccurateComparison.h>
|
||||
|
||||
|
@ -1,7 +1,5 @@
|
||||
#include "StorageMaterializedPostgreSQL.h"
|
||||
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
|
||||
#if USE_LIBPQXX
|
||||
#include <base/logger_useful.h>
|
||||
#include <Common/Macros.h>
|
||||
@ -15,6 +13,8 @@
|
||||
#include <Formats/FormatFactory.h>
|
||||
#include <Formats/FormatSettings.h>
|
||||
#include <Processors/Transforms/FilterTransform.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <QueryPipeline/Pipe.h>
|
||||
#include <Interpreters/executeQuery.h>
|
||||
|
@ -7,7 +7,6 @@
|
||||
#include "MaterializedPostgreSQLSettings.h"
|
||||
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTColumnDeclaration.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <Interpreters/TreeRewriter.h>
|
||||
#include <Storages/ProjectionsDescription.h>
|
||||
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTProjectionDeclaration.h>
|
||||
#include <Parsers/ASTProjectionSelectQuery.h>
|
||||
|
@ -26,6 +26,7 @@
|
||||
|
||||
#include <Parsers/ASTDropQuery.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTInsertQuery.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
|
@ -8,7 +8,6 @@
|
||||
#include <Common/SimpleIncrement.h>
|
||||
#include <Client/ConnectionPool.h>
|
||||
#include <Client/ConnectionPoolWithFailover.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <base/logger_useful.h>
|
||||
#include <Common/ActionBlocker.h>
|
||||
#include <Interpreters/Cluster.h>
|
||||
|
@ -13,6 +13,7 @@
|
||||
#include <Interpreters/getHeaderForProcessingStage.h>
|
||||
#include <Interpreters/addTypeConversionToAST.h>
|
||||
#include <Interpreters/replaceAliasColumnsInQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
|
@ -39,6 +39,7 @@
|
||||
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <Parsers/ASTDropQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTOptimizeQuery.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
#include <Parsers/ASTSubquery.h>
|
||||
|
Loading…
Reference in New Issue
Block a user