mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Reduce dependencies on ASTIdentifier.h
Goes from rebuilding 483 objects to 165 when it's modified
This commit is contained in:
parent
d7939a3b61
commit
051dddd8df
@ -1,8 +1,9 @@
|
|||||||
#include "Internals.h"
|
#include "Internals.h"
|
||||||
#include <Storages/MergeTree/MergeTreeData.h>
|
#include <Parsers/ASTIdentifier.h>
|
||||||
#include <Storages/extractKeyExpressionList.h>
|
|
||||||
#include <Processors/Executors/PullingPipelineExecutor.h>
|
#include <Processors/Executors/PullingPipelineExecutor.h>
|
||||||
#include <Processors/Transforms/SquashingChunksTransform.h>
|
#include <Processors/Transforms/SquashingChunksTransform.h>
|
||||||
|
#include <Storages/MergeTree/MergeTreeData.h>
|
||||||
|
#include <Storages/extractKeyExpressionList.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
@ -1,12 +1,13 @@
|
|||||||
#include <Backups/renameInCreateQuery.h>
|
|
||||||
#include <Backups/BackupRenamingConfig.h>
|
#include <Backups/BackupRenamingConfig.h>
|
||||||
|
#include <Backups/renameInCreateQuery.h>
|
||||||
|
#include <Interpreters/InDepthNodeVisitor.h>
|
||||||
|
#include <Interpreters/evaluateConstantExpression.h>
|
||||||
#include <Parsers/ASTCreateQuery.h>
|
#include <Parsers/ASTCreateQuery.h>
|
||||||
#include <Parsers/ASTFunction.h>
|
#include <Parsers/ASTFunction.h>
|
||||||
|
#include <Parsers/ASTIdentifier.h>
|
||||||
#include <Parsers/ASTLiteral.h>
|
#include <Parsers/ASTLiteral.h>
|
||||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||||
#include <TableFunctions/TableFunctionFactory.h>
|
#include <TableFunctions/TableFunctionFactory.h>
|
||||||
#include <Interpreters/InDepthNodeVisitor.h>
|
|
||||||
#include <Interpreters/evaluateConstantExpression.h>
|
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
|
@ -7,7 +7,6 @@
|
|||||||
#include <Compression/CompressionFactory.h>
|
#include <Compression/CompressionFactory.h>
|
||||||
#include <base/unaligned.h>
|
#include <base/unaligned.h>
|
||||||
#include <Parsers/IAST_fwd.h>
|
#include <Parsers/IAST_fwd.h>
|
||||||
#include <Parsers/ASTIdentifier.h>
|
|
||||||
#include <IO/WriteHelpers.h>
|
#include <IO/WriteHelpers.h>
|
||||||
#include <IO/ReadBufferFromMemory.h>
|
#include <IO/ReadBufferFromMemory.h>
|
||||||
#include <IO/BitHelpers.h>
|
#include <IO/BitHelpers.h>
|
||||||
|
@ -1,7 +1,6 @@
|
|||||||
#include <Compression/CompressionCodecNone.h>
|
#include <Compression/CompressionCodecNone.h>
|
||||||
#include <Compression/CompressionInfo.h>
|
#include <Compression/CompressionInfo.h>
|
||||||
#include <Compression/CompressionFactory.h>
|
#include <Compression/CompressionFactory.h>
|
||||||
#include <Parsers/ASTIdentifier.h>
|
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
|
@ -5,7 +5,6 @@
|
|||||||
#include <base/unaligned.h>
|
#include <base/unaligned.h>
|
||||||
#include <Parsers/IAST.h>
|
#include <Parsers/IAST.h>
|
||||||
#include <Parsers/ASTLiteral.h>
|
#include <Parsers/ASTLiteral.h>
|
||||||
#include <Parsers/ASTIdentifier.h>
|
|
||||||
#include <Parsers/ASTFunction.h>
|
#include <Parsers/ASTFunction.h>
|
||||||
#include <IO/WriteHelpers.h>
|
#include <IO/WriteHelpers.h>
|
||||||
#include <Core/Types.h>
|
#include <Core/Types.h>
|
||||||
|
@ -15,8 +15,8 @@
|
|||||||
|
|
||||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||||
#include <Parsers/ASTFunction.h>
|
#include <Parsers/ASTFunction.h>
|
||||||
|
#include <Parsers/ASTIdentifier_fwd.h>
|
||||||
#include <Parsers/ASTLiteral.h>
|
#include <Parsers/ASTLiteral.h>
|
||||||
#include <Parsers/ASTIdentifier.h>
|
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
|
@ -5,7 +5,6 @@
|
|||||||
#include <Core/MultiEnum.h>
|
#include <Core/MultiEnum.h>
|
||||||
#include <Core/SettingsEnums.h>
|
#include <Core/SettingsEnums.h>
|
||||||
#include <Parsers/ASTFunction.h>
|
#include <Parsers/ASTFunction.h>
|
||||||
#include <Parsers/ASTIdentifier.h>
|
|
||||||
#include <Parsers/IAST.h>
|
#include <Parsers/IAST.h>
|
||||||
#include "DataTypeDate.h"
|
#include "DataTypeDate.h"
|
||||||
#include "DataTypeDateTime.h"
|
#include "DataTypeDateTime.h"
|
||||||
|
@ -11,9 +11,7 @@
|
|||||||
#include <Interpreters/evaluateConstantExpression.h>
|
#include <Interpreters/evaluateConstantExpression.h>
|
||||||
#include <Parsers/ASTCreateQuery.h>
|
#include <Parsers/ASTCreateQuery.h>
|
||||||
#include <Parsers/ASTFunction.h>
|
#include <Parsers/ASTFunction.h>
|
||||||
#include <Parsers/ASTIdentifier.h>
|
|
||||||
#include <Parsers/ASTLiteral.h>
|
#include <Parsers/ASTLiteral.h>
|
||||||
#include <Parsers/formatAST.h>
|
|
||||||
#include <Parsers/queryToString.h>
|
#include <Parsers/queryToString.h>
|
||||||
#include <Storages/ExternalDataSourceConfiguration.h>
|
#include <Storages/ExternalDataSourceConfiguration.h>
|
||||||
#include <Common/Macros.h>
|
#include <Common/Macros.h>
|
||||||
|
@ -1,5 +1,7 @@
|
|||||||
#include <Databases/PostgreSQL/DatabasePostgreSQL.h>
|
#include <Databases/PostgreSQL/DatabasePostgreSQL.h>
|
||||||
|
|
||||||
|
#include <Parsers/ASTIdentifier.h>
|
||||||
|
|
||||||
#if USE_LIBPQXX
|
#if USE_LIBPQXX
|
||||||
|
|
||||||
#include <DataTypes/DataTypeNullable.h>
|
#include <DataTypes/DataTypeNullable.h>
|
||||||
|
@ -1,7 +1,7 @@
|
|||||||
#include <Functions/JSONPath/ASTs/ASTJSONPathMemberAccess.h>
|
#include <Functions/JSONPath/ASTs/ASTJSONPathMemberAccess.h>
|
||||||
#include <Functions/JSONPath/Parsers/ParserJSONPathMemberAccess.h>
|
#include <Functions/JSONPath/Parsers/ParserJSONPathMemberAccess.h>
|
||||||
|
|
||||||
#include <Parsers/ASTIdentifier.h>
|
#include <Parsers/ASTIdentifier_fwd.h>
|
||||||
#include <Parsers/ExpressionElementParsers.h>
|
#include <Parsers/ExpressionElementParsers.h>
|
||||||
#include <Parsers/Lexer.h>
|
#include <Parsers/Lexer.h>
|
||||||
|
|
||||||
|
@ -3,6 +3,7 @@
|
|||||||
#include <Interpreters/StorageID.h>
|
#include <Interpreters/StorageID.h>
|
||||||
#include <Interpreters/misc.h>
|
#include <Interpreters/misc.h>
|
||||||
#include <Parsers/ASTFunction.h>
|
#include <Parsers/ASTFunction.h>
|
||||||
|
#include <Parsers/ASTIdentifier.h>
|
||||||
#include <Parsers/ASTSelectQuery.h>
|
#include <Parsers/ASTSelectQuery.h>
|
||||||
#include <Parsers/ASTSubquery.h>
|
#include <Parsers/ASTSubquery.h>
|
||||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||||
|
@ -1,3 +1,4 @@
|
|||||||
|
#include <Parsers/ASTIdentifier.h>
|
||||||
#include <Parsers/queryToString.h>
|
#include <Parsers/queryToString.h>
|
||||||
|
|
||||||
#include <Interpreters/CollectJoinOnKeysVisitor.h>
|
#include <Interpreters/CollectJoinOnKeysVisitor.h>
|
||||||
|
@ -2,6 +2,8 @@
|
|||||||
#include <Functions/FunctionFactory.h>
|
#include <Functions/FunctionFactory.h>
|
||||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||||
#include <Interpreters/IdentifierSemantic.h>
|
#include <Interpreters/IdentifierSemantic.h>
|
||||||
|
#include <Parsers/ASTFunction.h>
|
||||||
|
#include <Parsers/ASTIdentifier.h>
|
||||||
#include <Parsers/ASTSubquery.h>
|
#include <Parsers/ASTSubquery.h>
|
||||||
|
|
||||||
|
|
||||||
|
@ -3,13 +3,14 @@
|
|||||||
#include <Interpreters/Context_fwd.h>
|
#include <Interpreters/Context_fwd.h>
|
||||||
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
||||||
#include <Interpreters/InDepthNodeVisitor.h>
|
#include <Interpreters/InDepthNodeVisitor.h>
|
||||||
#include <Parsers/ASTFunction.h>
|
|
||||||
#include <Parsers/ASTIdentifier.h>
|
|
||||||
#include <Parsers/IAST_fwd.h>
|
#include <Parsers/IAST_fwd.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
|
class ASTFunction;
|
||||||
|
class ASTIdentifier;
|
||||||
|
|
||||||
|
|
||||||
struct ExpressionInfoMatcher
|
struct ExpressionInfoMatcher
|
||||||
{
|
{
|
||||||
|
@ -10,7 +10,6 @@
|
|||||||
#include <Interpreters/interpretSubquery.h>
|
#include <Interpreters/interpretSubquery.h>
|
||||||
#include <Interpreters/SubqueryForSet.h>
|
#include <Interpreters/SubqueryForSet.h>
|
||||||
#include <Parsers/ASTFunction.h>
|
#include <Parsers/ASTFunction.h>
|
||||||
#include <Parsers/ASTIdentifier.h>
|
|
||||||
#include <Parsers/ASTLiteral.h>
|
#include <Parsers/ASTLiteral.h>
|
||||||
#include <Parsers/ASTSelectQuery.h>
|
#include <Parsers/ASTSelectQuery.h>
|
||||||
#include <Parsers/ASTSubquery.h>
|
#include <Parsers/ASTSubquery.h>
|
||||||
|
@ -6,6 +6,7 @@
|
|||||||
#include <Interpreters/StorageID.h>
|
#include <Interpreters/StorageID.h>
|
||||||
|
|
||||||
#include <Parsers/ASTFunction.h>
|
#include <Parsers/ASTFunction.h>
|
||||||
|
#include <Parsers/ASTIdentifier.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
@ -6,12 +6,13 @@
|
|||||||
#include <Interpreters/QueryAliasesVisitor.h>
|
#include <Interpreters/QueryAliasesVisitor.h>
|
||||||
#include <Interpreters/getHeaderForProcessingStage.h>
|
#include <Interpreters/getHeaderForProcessingStage.h>
|
||||||
#include <Interpreters/getTableExpressions.h>
|
#include <Interpreters/getTableExpressions.h>
|
||||||
#include <Parsers/ASTIdentifier.h>
|
|
||||||
#include <Parsers/ASTSelectQuery.h>
|
#include <Parsers/ASTSelectQuery.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
|
class ASTIdentifier;
|
||||||
|
|
||||||
struct IdentifierSemanticImpl
|
struct IdentifierSemanticImpl
|
||||||
{
|
{
|
||||||
bool special = false; /// for now it's 'not a column': tables, subselects and some special stuff like FORMAT
|
bool special = false; /// for now it's 'not a column': tables, subselects and some special stuff like FORMAT
|
||||||
|
@ -11,6 +11,7 @@
|
|||||||
#include <Interpreters/executeDDLQueryOnCluster.h>
|
#include <Interpreters/executeDDLQueryOnCluster.h>
|
||||||
#include <Parsers/ASTAlterQuery.h>
|
#include <Parsers/ASTAlterQuery.h>
|
||||||
#include <Parsers/ASTAssignment.h>
|
#include <Parsers/ASTAssignment.h>
|
||||||
|
#include <Parsers/ASTIdentifier_fwd.h>
|
||||||
#include <Storages/AlterCommands.h>
|
#include <Storages/AlterCommands.h>
|
||||||
#include <Storages/IStorage.h>
|
#include <Storages/IStorage.h>
|
||||||
#include <Storages/LiveView/LiveViewCommands.h>
|
#include <Storages/LiveView/LiveViewCommands.h>
|
||||||
|
@ -4,6 +4,7 @@
|
|||||||
#include <Interpreters/IdentifierSemantic.h>
|
#include <Interpreters/IdentifierSemantic.h>
|
||||||
#include <Interpreters/misc.h>
|
#include <Interpreters/misc.h>
|
||||||
#include <Parsers/ASTFunction.h>
|
#include <Parsers/ASTFunction.h>
|
||||||
|
#include <Parsers/ASTIdentifier.h>
|
||||||
#include <Parsers/ASTSelectQuery.h>
|
#include <Parsers/ASTSelectQuery.h>
|
||||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||||
|
|
||||||
|
@ -2,7 +2,6 @@
|
|||||||
|
|
||||||
#include <Interpreters/InDepthNodeVisitor.h>
|
#include <Interpreters/InDepthNodeVisitor.h>
|
||||||
#include <Parsers/ASTFunction.h>
|
#include <Parsers/ASTFunction.h>
|
||||||
#include <Parsers/ASTIdentifier.h>
|
|
||||||
#include <Parsers/ASTOrderByElement.h>
|
#include <Parsers/ASTOrderByElement.h>
|
||||||
#include <Parsers/ASTSelectQuery.h>
|
#include <Parsers/ASTSelectQuery.h>
|
||||||
#include <Parsers/ASTExpressionList.h>
|
#include <Parsers/ASTExpressionList.h>
|
||||||
@ -10,6 +9,8 @@
|
|||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
|
class ASTIdentifier;
|
||||||
|
|
||||||
class RedundantFunctionsInOrderByMatcher
|
class RedundantFunctionsInOrderByMatcher
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
@ -1,5 +1,6 @@
|
|||||||
#include <Interpreters/RenameColumnVisitor.h>
|
|
||||||
#include <Interpreters/IdentifierSemantic.h>
|
#include <Interpreters/IdentifierSemantic.h>
|
||||||
|
#include <Interpreters/RenameColumnVisitor.h>
|
||||||
|
#include <Parsers/ASTIdentifier.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
@ -1,10 +1,12 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <Interpreters/InDepthNodeVisitor.h>
|
#include <Interpreters/InDepthNodeVisitor.h>
|
||||||
#include <Parsers/ASTIdentifier.h>
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
|
class ASTIdentifier;
|
||||||
|
|
||||||
/// Data for RenameColumnVisitor which traverse tree and rename all columns with
|
/// Data for RenameColumnVisitor which traverse tree and rename all columns with
|
||||||
/// name column_name to rename_to
|
/// name column_name to rename_to
|
||||||
struct RenameColumnData
|
struct RenameColumnData
|
||||||
|
@ -1,6 +1,5 @@
|
|||||||
#include <Interpreters/RewriteCountVariantsVisitor.h>
|
#include <Interpreters/RewriteCountVariantsVisitor.h>
|
||||||
#include <Parsers/ASTFunction.h>
|
#include <Parsers/ASTFunction.h>
|
||||||
#include <Parsers/ASTIdentifier.h>
|
|
||||||
#include <Parsers/ASTLiteral.h>
|
#include <Parsers/ASTLiteral.h>
|
||||||
#include <Parsers/ASTSubquery.h>
|
#include <Parsers/ASTSubquery.h>
|
||||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||||
|
@ -6,6 +6,7 @@
|
|||||||
#include <Parsers/ASTQueryWithOutput.h>
|
#include <Parsers/ASTQueryWithOutput.h>
|
||||||
#include <Parsers/ASTQueryWithOnCluster.h>
|
#include <Parsers/ASTQueryWithOnCluster.h>
|
||||||
#include <Parsers/ASTAlterQuery.h>
|
#include <Parsers/ASTAlterQuery.h>
|
||||||
|
#include <Parsers/ASTIdentifier.h>
|
||||||
#include <Parsers/queryToString.h>
|
#include <Parsers/queryToString.h>
|
||||||
#include <Access/Common/AccessRightsElement.h>
|
#include <Access/Common/AccessRightsElement.h>
|
||||||
#include <Access/ContextAccess.h>
|
#include <Access/ContextAccess.h>
|
||||||
|
@ -3,6 +3,7 @@
|
|||||||
#include <Interpreters/TreeRewriter.h>
|
#include <Interpreters/TreeRewriter.h>
|
||||||
#include <Interpreters/IdentifierSemantic.h>
|
#include <Interpreters/IdentifierSemantic.h>
|
||||||
#include <Storages/IStorage.h>
|
#include <Storages/IStorage.h>
|
||||||
|
#include <Parsers/ASTIdentifier.h>
|
||||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||||
#include <Processors/Sources/SourceFromSingleChunk.h>
|
#include <Processors/Sources/SourceFromSingleChunk.h>
|
||||||
|
|
||||||
|
@ -4,7 +4,6 @@
|
|||||||
#include <Parsers/ASTQueryWithOnCluster.h>
|
#include <Parsers/ASTQueryWithOnCluster.h>
|
||||||
#include <Parsers/ASTDictionary.h>
|
#include <Parsers/ASTDictionary.h>
|
||||||
#include <Parsers/ASTDictionaryAttributeDeclaration.h>
|
#include <Parsers/ASTDictionaryAttributeDeclaration.h>
|
||||||
#include <Parsers/ASTIdentifier.h>
|
|
||||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||||
#include <Interpreters/StorageID.h>
|
#include <Interpreters/StorageID.h>
|
||||||
|
|
||||||
|
@ -6,13 +6,9 @@
|
|||||||
#include <Common/FieldVisitorToString.h>
|
#include <Common/FieldVisitorToString.h>
|
||||||
#include <Common/SipHash.h>
|
#include <Common/SipHash.h>
|
||||||
#include <Common/typeid_cast.h>
|
#include <Common/typeid_cast.h>
|
||||||
#include <DataTypes/IDataType.h>
|
|
||||||
#include <DataTypes/NumberTraits.h>
|
|
||||||
#include <IO/Operators.h>
|
#include <IO/Operators.h>
|
||||||
#include <IO/WriteBufferFromString.h>
|
#include <IO/WriteBufferFromString.h>
|
||||||
#include <IO/WriteHelpers.h>
|
#include <IO/WriteHelpers.h>
|
||||||
#include <Parsers/ASTExpressionList.h>
|
|
||||||
#include <Parsers/ASTIdentifier.h>
|
|
||||||
#include <Parsers/ASTLiteral.h>
|
#include <Parsers/ASTLiteral.h>
|
||||||
#include <Parsers/ASTSubquery.h>
|
#include <Parsers/ASTSubquery.h>
|
||||||
#include <Parsers/ASTWithAlias.h>
|
#include <Parsers/ASTWithAlias.h>
|
||||||
|
@ -1,14 +1,14 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <Parsers/ASTWithAlias.h>
|
|
||||||
#include <Parsers/ASTExpressionList.h>
|
#include <Parsers/ASTExpressionList.h>
|
||||||
|
#include <Parsers/ASTIdentifier_fwd.h>
|
||||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||||
|
#include <Parsers/ASTWithAlias.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
class ASTIdentifier;
|
|
||||||
|
|
||||||
/** AST for function application or operator.
|
/** AST for function application or operator.
|
||||||
*/
|
*/
|
||||||
|
@ -1,11 +1,10 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <Core/UUID.h>
|
#include <Core/UUID.h>
|
||||||
|
#include <Parsers/ASTIdentifier_fwd.h>
|
||||||
#include <Parsers/ASTQueryParameter.h>
|
#include <Parsers/ASTQueryParameter.h>
|
||||||
#include <Parsers/ASTWithAlias.h>
|
#include <Parsers/ASTWithAlias.h>
|
||||||
|
|
||||||
#include <optional>
|
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
@ -93,17 +92,4 @@ public:
|
|||||||
void updateTreeHashImpl(SipHash & hash_state) const override;
|
void updateTreeHashImpl(SipHash & hash_state) const override;
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
/// ASTIdentifier Helpers: hide casts and semantic.
|
|
||||||
|
|
||||||
void setIdentifierSpecial(ASTPtr & ast);
|
|
||||||
|
|
||||||
String getIdentifierName(const IAST * ast);
|
|
||||||
std::optional<String> tryGetIdentifierName(const IAST * ast);
|
|
||||||
bool tryGetIdentifierNameInto(const IAST * ast, String & name);
|
|
||||||
|
|
||||||
inline String getIdentifierName(const ASTPtr & ast) { return getIdentifierName(ast.get()); }
|
|
||||||
inline std::optional<String> tryGetIdentifierName(const ASTPtr & ast) { return tryGetIdentifierName(ast.get()); }
|
|
||||||
inline bool tryGetIdentifierNameInto(const ASTPtr & ast, String & name) { return tryGetIdentifierNameInto(ast.get(), name); }
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
36
src/Parsers/ASTIdentifier_fwd.h
Normal file
36
src/Parsers/ASTIdentifier_fwd.h
Normal file
@ -0,0 +1,36 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <Parsers/IAST_fwd.h>
|
||||||
|
#include <base/types.h>
|
||||||
|
|
||||||
|
#include <optional>
|
||||||
|
#include <vector>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
class ASTIdentifier;
|
||||||
|
class ASTTableIdentifier;
|
||||||
|
|
||||||
|
/// ASTIdentifier Helpers: hide casts and semantic.
|
||||||
|
|
||||||
|
void setIdentifierSpecial(ASTPtr & ast);
|
||||||
|
|
||||||
|
String getIdentifierName(const IAST * ast);
|
||||||
|
std::optional<String> tryGetIdentifierName(const IAST * ast);
|
||||||
|
bool tryGetIdentifierNameInto(const IAST * ast, String & name);
|
||||||
|
|
||||||
|
inline String getIdentifierName(const ASTPtr & ast)
|
||||||
|
{
|
||||||
|
return getIdentifierName(ast.get());
|
||||||
|
}
|
||||||
|
inline std::optional<String> tryGetIdentifierName(const ASTPtr & ast)
|
||||||
|
{
|
||||||
|
return tryGetIdentifierName(ast.get());
|
||||||
|
}
|
||||||
|
inline bool tryGetIdentifierNameInto(const ASTPtr & ast, String & name)
|
||||||
|
{
|
||||||
|
return tryGetIdentifierNameInto(ast.get(), name);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -2,7 +2,7 @@
|
|||||||
#include <Parsers/Access/ASTCreateQuotaQuery.h>
|
#include <Parsers/Access/ASTCreateQuotaQuery.h>
|
||||||
#include <Parsers/Access/ASTRolesOrUsersSet.h>
|
#include <Parsers/Access/ASTRolesOrUsersSet.h>
|
||||||
#include <Parsers/Access/ParserRolesOrUsersSet.h>
|
#include <Parsers/Access/ParserRolesOrUsersSet.h>
|
||||||
#include <Parsers/ASTIdentifier.h>
|
#include <Parsers/ASTIdentifier_fwd.h>
|
||||||
#include <Parsers/ASTLiteral.h>
|
#include <Parsers/ASTLiteral.h>
|
||||||
#include <Parsers/CommonParsers.h>
|
#include <Parsers/CommonParsers.h>
|
||||||
#include <Parsers/ExpressionElementParsers.h>
|
#include <Parsers/ExpressionElementParsers.h>
|
||||||
|
@ -1,9 +1,9 @@
|
|||||||
#include <Parsers/Access/ParserGrantQuery.h>
|
#include <Parsers/ASTIdentifier_fwd.h>
|
||||||
|
#include <Parsers/ASTLiteral.h>
|
||||||
#include <Parsers/Access/ASTGrantQuery.h>
|
#include <Parsers/Access/ASTGrantQuery.h>
|
||||||
#include <Parsers/Access/ASTRolesOrUsersSet.h>
|
#include <Parsers/Access/ASTRolesOrUsersSet.h>
|
||||||
|
#include <Parsers/Access/ParserGrantQuery.h>
|
||||||
#include <Parsers/Access/ParserRolesOrUsersSet.h>
|
#include <Parsers/Access/ParserRolesOrUsersSet.h>
|
||||||
#include <Parsers/ASTLiteral.h>
|
|
||||||
#include <Parsers/ASTIdentifier.h>
|
|
||||||
#include <Parsers/ExpressionElementParsers.h>
|
#include <Parsers/ExpressionElementParsers.h>
|
||||||
#include <Parsers/ExpressionListParsers.h>
|
#include <Parsers/ExpressionListParsers.h>
|
||||||
#include <Parsers/parseDatabaseAndTableName.h>
|
#include <Parsers/parseDatabaseAndTableName.h>
|
||||||
|
@ -1,10 +1,10 @@
|
|||||||
#include <Parsers/Access/ParserSettingsProfileElement.h>
|
#include <Parsers/ASTIdentifier_fwd.h>
|
||||||
#include <Parsers/Access/ASTSettingsProfileElement.h>
|
|
||||||
#include <Parsers/ASTLiteral.h>
|
#include <Parsers/ASTLiteral.h>
|
||||||
#include <Parsers/ASTIdentifier.h>
|
#include <Parsers/Access/ASTSettingsProfileElement.h>
|
||||||
|
#include <Parsers/Access/ParserSettingsProfileElement.h>
|
||||||
#include <Parsers/CommonParsers.h>
|
#include <Parsers/CommonParsers.h>
|
||||||
#include <Parsers/ExpressionListParsers.h>
|
|
||||||
#include <Parsers/ExpressionElementParsers.h>
|
#include <Parsers/ExpressionElementParsers.h>
|
||||||
|
#include <Parsers/ExpressionListParsers.h>
|
||||||
#include <Parsers/parseIdentifierOrStringLiteral.h>
|
#include <Parsers/parseIdentifierOrStringLiteral.h>
|
||||||
#include <boost/algorithm/string/predicate.hpp>
|
#include <boost/algorithm/string/predicate.hpp>
|
||||||
|
|
||||||
|
@ -6,12 +6,13 @@
|
|||||||
#include <Parsers/ASTExpressionList.h>
|
#include <Parsers/ASTExpressionList.h>
|
||||||
#include <Parsers/ASTFunction.h>
|
#include <Parsers/ASTFunction.h>
|
||||||
#include <Parsers/ASTFunctionWithKeyValueArguments.h>
|
#include <Parsers/ASTFunctionWithKeyValueArguments.h>
|
||||||
|
#include <Parsers/ASTIdentifier.h>
|
||||||
#include <Parsers/ASTSelectQuery.h>
|
#include <Parsers/ASTSelectQuery.h>
|
||||||
#include <Parsers/ASTSubquery.h>
|
#include <Parsers/ASTSubquery.h>
|
||||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||||
#include <Parsers/ParserCreateQuery.h>
|
#include <Parsers/ParserCreateQuery.h>
|
||||||
#include <Parsers/parseIntervalKind.h>
|
|
||||||
#include <Parsers/ParserUnionQueryElement.h>
|
#include <Parsers/ParserUnionQueryElement.h>
|
||||||
|
#include <Parsers/parseIntervalKind.h>
|
||||||
#include <Common/StringUtils/StringUtils.h>
|
#include <Common/StringUtils/StringUtils.h>
|
||||||
|
|
||||||
using namespace std::literals;
|
using namespace std::literals;
|
||||||
|
@ -1,12 +1,12 @@
|
|||||||
#include <Parsers/MySQL/ASTDeclareColumn.h>
|
#include <Parsers/MySQL/ASTDeclareColumn.h>
|
||||||
|
|
||||||
#include <Parsers/ASTIdentifier.h>
|
#include <Parsers/ASTIdentifier_fwd.h>
|
||||||
#include <Parsers/ParserDataType.h>
|
|
||||||
#include <Parsers/ExpressionListParsers.h>
|
|
||||||
#include <Parsers/ExpressionElementParsers.h>
|
#include <Parsers/ExpressionElementParsers.h>
|
||||||
|
#include <Parsers/ExpressionListParsers.h>
|
||||||
|
#include <Parsers/MySQL/ASTDeclareConstraint.h>
|
||||||
#include <Parsers/MySQL/ASTDeclareOption.h>
|
#include <Parsers/MySQL/ASTDeclareOption.h>
|
||||||
#include <Parsers/MySQL/ASTDeclareReference.h>
|
#include <Parsers/MySQL/ASTDeclareReference.h>
|
||||||
#include <Parsers/MySQL/ASTDeclareConstraint.h>
|
#include <Parsers/ParserDataType.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
@ -2,7 +2,7 @@
|
|||||||
#include <Parsers/parseQuery.h>
|
#include <Parsers/parseQuery.h>
|
||||||
#include <Parsers/ASTLiteral.h>
|
#include <Parsers/ASTLiteral.h>
|
||||||
#include <Parsers/ASTFunction.h>
|
#include <Parsers/ASTFunction.h>
|
||||||
#include <Parsers/ASTIdentifier.h>
|
#include <Parsers/ASTIdentifier_fwd.h>
|
||||||
#include <Parsers/MySQL/ASTAlterCommand.h>
|
#include <Parsers/MySQL/ASTAlterCommand.h>
|
||||||
#include <Parsers/MySQL/ASTDeclareOption.h>
|
#include <Parsers/MySQL/ASTDeclareOption.h>
|
||||||
|
|
||||||
|
@ -1,6 +1,6 @@
|
|||||||
#include <Parsers/ParserBackupQuery.h>
|
#include <Parsers/ParserBackupQuery.h>
|
||||||
#include <Parsers/ASTBackupQuery.h>
|
#include <Parsers/ASTBackupQuery.h>
|
||||||
#include <Parsers/ASTIdentifier.h>
|
#include <Parsers/ASTIdentifier_fwd.h>
|
||||||
#include <Parsers/ASTLiteral.h>
|
#include <Parsers/ASTLiteral.h>
|
||||||
#include <Parsers/CommonParsers.h>
|
#include <Parsers/CommonParsers.h>
|
||||||
#include <Parsers/ExpressionElementParsers.h>
|
#include <Parsers/ExpressionElementParsers.h>
|
||||||
|
@ -1,6 +1,5 @@
|
|||||||
#include <Parsers/ParserCheckQuery.h>
|
#include <Parsers/ParserCheckQuery.h>
|
||||||
#include <Parsers/CommonParsers.h>
|
#include <Parsers/CommonParsers.h>
|
||||||
#include <Parsers/ASTIdentifier.h>
|
|
||||||
#include <Parsers/ExpressionElementParsers.h>
|
#include <Parsers/ExpressionElementParsers.h>
|
||||||
#include <Parsers/ASTCheckQuery.h>
|
#include <Parsers/ASTCheckQuery.h>
|
||||||
#include <Parsers/ParserPartition.h>
|
#include <Parsers/ParserPartition.h>
|
||||||
|
@ -5,7 +5,7 @@
|
|||||||
#include <Parsers/ExpressionListParsers.h>
|
#include <Parsers/ExpressionListParsers.h>
|
||||||
#include <Parsers/ASTNameTypePair.h>
|
#include <Parsers/ASTNameTypePair.h>
|
||||||
#include <Parsers/ASTColumnDeclaration.h>
|
#include <Parsers/ASTColumnDeclaration.h>
|
||||||
#include <Parsers/ASTIdentifier.h>
|
#include <Parsers/ASTIdentifier_fwd.h>
|
||||||
#include <Parsers/ASTLiteral.h>
|
#include <Parsers/ASTLiteral.h>
|
||||||
#include <Parsers/CommonParsers.h>
|
#include <Parsers/CommonParsers.h>
|
||||||
#include <Parsers/ParserDataType.h>
|
#include <Parsers/ParserDataType.h>
|
||||||
|
@ -1,7 +1,7 @@
|
|||||||
#include <Parsers/ParserDataType.h>
|
#include <Parsers/ParserDataType.h>
|
||||||
|
|
||||||
#include <Parsers/ASTFunction.h>
|
#include <Parsers/ASTFunction.h>
|
||||||
#include <Parsers/ASTIdentifier.h>
|
#include <Parsers/ASTIdentifier_fwd.h>
|
||||||
#include <Parsers/CommonParsers.h>
|
#include <Parsers/CommonParsers.h>
|
||||||
#include <Parsers/ExpressionElementParsers.h>
|
#include <Parsers/ExpressionElementParsers.h>
|
||||||
#include <Parsers/ParserCreateQuery.h>
|
#include <Parsers/ParserCreateQuery.h>
|
||||||
|
@ -1,5 +1,6 @@
|
|||||||
#include <Parsers/ParserDictionaryAttributeDeclaration.h>
|
#include <Parsers/ParserDictionaryAttributeDeclaration.h>
|
||||||
|
|
||||||
|
#include <Parsers/ASTIdentifier_fwd.h>
|
||||||
#include <Parsers/ExpressionElementParsers.h>
|
#include <Parsers/ExpressionElementParsers.h>
|
||||||
#include <Parsers/ExpressionListParsers.h>
|
#include <Parsers/ExpressionListParsers.h>
|
||||||
#include <Parsers/ParserDataType.h>
|
#include <Parsers/ParserDataType.h>
|
||||||
|
@ -4,7 +4,6 @@
|
|||||||
#include <Parsers/IAST_fwd.h>
|
#include <Parsers/IAST_fwd.h>
|
||||||
#include <Parsers/CommonParsers.h>
|
#include <Parsers/CommonParsers.h>
|
||||||
#include <Parsers/ASTDictionaryAttributeDeclaration.h>
|
#include <Parsers/ASTDictionaryAttributeDeclaration.h>
|
||||||
#include <Parsers/ASTIdentifier.h>
|
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
|
@ -1,4 +1,3 @@
|
|||||||
#include <Parsers/ASTIdentifier.h>
|
|
||||||
#include <Parsers/ASTDropQuery.h>
|
#include <Parsers/ASTDropQuery.h>
|
||||||
|
|
||||||
#include <Parsers/CommonParsers.h>
|
#include <Parsers/CommonParsers.h>
|
||||||
|
@ -1,6 +1,6 @@
|
|||||||
#include <Parsers/ASTIdentifier.h>
|
#include <Parsers/ASTIdentifier_fwd.h>
|
||||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
|
||||||
#include <Parsers/ASTInsertQuery.h>
|
#include <Parsers/ASTInsertQuery.h>
|
||||||
|
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||||
|
|
||||||
#include <Parsers/CommonParsers.h>
|
#include <Parsers/CommonParsers.h>
|
||||||
#include <Parsers/ExpressionElementParsers.h>
|
#include <Parsers/ExpressionElementParsers.h>
|
||||||
|
@ -3,7 +3,6 @@
|
|||||||
#include <Parsers/CommonParsers.h>
|
#include <Parsers/CommonParsers.h>
|
||||||
|
|
||||||
#include <Parsers/ASTOptimizeQuery.h>
|
#include <Parsers/ASTOptimizeQuery.h>
|
||||||
#include <Parsers/ASTIdentifier.h>
|
|
||||||
#include <Parsers/ExpressionListParsers.h>
|
#include <Parsers/ExpressionListParsers.h>
|
||||||
|
|
||||||
|
|
||||||
|
@ -1,4 +1,4 @@
|
|||||||
#include <Parsers/ASTIdentifier.h>
|
#include <Parsers/ASTIdentifier_fwd.h>
|
||||||
#include <Parsers/ASTRenameQuery.h>
|
#include <Parsers/ASTRenameQuery.h>
|
||||||
|
|
||||||
#include <Parsers/CommonParsers.h>
|
#include <Parsers/CommonParsers.h>
|
||||||
|
@ -1,4 +1,4 @@
|
|||||||
#include <Parsers/ASTIdentifier.h>
|
#include <Parsers/ASTIdentifier_fwd.h>
|
||||||
#include <Parsers/ASTLiteral.h>
|
#include <Parsers/ASTLiteral.h>
|
||||||
#include <Parsers/ASTSetQuery.h>
|
#include <Parsers/ASTSetQuery.h>
|
||||||
|
|
||||||
|
@ -1,5 +1,5 @@
|
|||||||
|
#include <Parsers/ASTIdentifier_fwd.h>
|
||||||
#include <Parsers/ASTLiteral.h>
|
#include <Parsers/ASTLiteral.h>
|
||||||
#include <Parsers/ASTIdentifier.h>
|
|
||||||
#include <Parsers/ASTShowTablesQuery.h>
|
#include <Parsers/ASTShowTablesQuery.h>
|
||||||
|
|
||||||
#include <Parsers/CommonParsers.h>
|
#include <Parsers/CommonParsers.h>
|
||||||
|
@ -1,4 +1,3 @@
|
|||||||
#include <Parsers/ASTIdentifier.h>
|
|
||||||
#include <Parsers/TablePropertiesQueriesASTs.h>
|
#include <Parsers/TablePropertiesQueriesASTs.h>
|
||||||
|
|
||||||
#include <Parsers/CommonParsers.h>
|
#include <Parsers/CommonParsers.h>
|
||||||
|
@ -1,5 +1,5 @@
|
|||||||
#include <Parsers/ParserUseQuery.h>
|
#include <Parsers/ParserUseQuery.h>
|
||||||
#include <Parsers/ASTIdentifier.h>
|
#include <Parsers/ASTIdentifier_fwd.h>
|
||||||
#include <Parsers/ExpressionElementParsers.h>
|
#include <Parsers/ExpressionElementParsers.h>
|
||||||
#include <Parsers/CommonParsers.h>
|
#include <Parsers/CommonParsers.h>
|
||||||
#include <Parsers/ASTUseQuery.h>
|
#include <Parsers/ASTUseQuery.h>
|
||||||
|
@ -10,7 +10,6 @@ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|||||||
See the License for the specific language governing permissions and
|
See the License for the specific language governing permissions and
|
||||||
limitations under the License. */
|
limitations under the License. */
|
||||||
#include <Parsers/ASTLiteral.h>
|
#include <Parsers/ASTLiteral.h>
|
||||||
#include <Parsers/ASTIdentifier.h>
|
|
||||||
#include <Parsers/ASTWatchQuery.h>
|
#include <Parsers/ASTWatchQuery.h>
|
||||||
#include <Parsers/CommonParsers.h>
|
#include <Parsers/CommonParsers.h>
|
||||||
#include <Parsers/ParserWatchQuery.h>
|
#include <Parsers/ParserWatchQuery.h>
|
||||||
|
@ -1,5 +1,5 @@
|
|||||||
#include <Parsers/ASTExpressionList.h>
|
#include <Parsers/ASTExpressionList.h>
|
||||||
#include <Parsers/ASTIdentifier.h>
|
#include <Parsers/ASTIdentifier_fwd.h>
|
||||||
#include <Parsers/ASTSubquery.h>
|
#include <Parsers/ASTSubquery.h>
|
||||||
#include <Parsers/ASTWithElement.h>
|
#include <Parsers/ASTWithElement.h>
|
||||||
#include <Parsers/CommonParsers.h>
|
#include <Parsers/CommonParsers.h>
|
||||||
|
@ -1,7 +1,7 @@
|
|||||||
#include "parseDatabaseAndTableName.h"
|
#include "parseDatabaseAndTableName.h"
|
||||||
#include <Parsers/ExpressionElementParsers.h>
|
#include <Parsers/ASTIdentifier_fwd.h>
|
||||||
#include <Parsers/ASTIdentifier.h>
|
|
||||||
#include <Parsers/CommonParsers.h>
|
#include <Parsers/CommonParsers.h>
|
||||||
|
#include <Parsers/ExpressionElementParsers.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
|
@ -1,8 +1,8 @@
|
|||||||
#include "parseIdentifierOrStringLiteral.h"
|
#include "parseIdentifierOrStringLiteral.h"
|
||||||
|
|
||||||
#include "ExpressionElementParsers.h"
|
#include <Parsers/ExpressionElementParsers.h>
|
||||||
#include "ASTLiteral.h"
|
#include <Parsers/ASTLiteral.h>
|
||||||
#include "ASTIdentifier.h"
|
#include <Parsers/ASTIdentifier_fwd.h>
|
||||||
#include <Parsers/CommonParsers.h>
|
#include <Parsers/CommonParsers.h>
|
||||||
#include <Parsers/ExpressionListParsers.h>
|
#include <Parsers/ExpressionListParsers.h>
|
||||||
#include <Common/typeid_cast.h>
|
#include <Common/typeid_cast.h>
|
||||||
|
@ -1,6 +1,7 @@
|
|||||||
#include <base/types.h>
|
#include <base/types.h>
|
||||||
#include <Parsers/ASTCreateQuery.h>
|
#include <Parsers/ASTCreateQuery.h>
|
||||||
#include <Parsers/ASTDropQuery.h>
|
#include <Parsers/ASTDropQuery.h>
|
||||||
|
#include <Parsers/ASTIdentifier.h>
|
||||||
#include <Parsers/ParserCreateQuery.h>
|
#include <Parsers/ParserCreateQuery.h>
|
||||||
#include <Parsers/ParserDropQuery.h>
|
#include <Parsers/ParserDropQuery.h>
|
||||||
#include <Parsers/formatAST.h>
|
#include <Parsers/formatAST.h>
|
||||||
|
@ -20,7 +20,7 @@
|
|||||||
#include <IO/ReadBufferFromString.h>
|
#include <IO/ReadBufferFromString.h>
|
||||||
#include <IO/ReadHelpers.h>
|
#include <IO/ReadHelpers.h>
|
||||||
#include <Parsers/parseQuery.h>
|
#include <Parsers/parseQuery.h>
|
||||||
#include <Parsers/ASTIdentifier.h>
|
#include <Parsers/ASTIdentifier_fwd.h>
|
||||||
#include <Parsers/ASTInsertQuery.h>
|
#include <Parsers/ASTInsertQuery.h>
|
||||||
#include <Parsers/ASTQueryWithOutput.h>
|
#include <Parsers/ASTQueryWithOutput.h>
|
||||||
#include <Parsers/ParserQuery.h>
|
#include <Parsers/ParserQuery.h>
|
||||||
|
@ -1,6 +1,5 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <Parsers/ASTIdentifier.h>
|
|
||||||
#include <Interpreters/Context.h>
|
#include <Interpreters/Context.h>
|
||||||
#include <Poco/Util/AbstractConfiguration.h>
|
#include <Poco/Util/AbstractConfiguration.h>
|
||||||
|
|
||||||
|
@ -12,6 +12,7 @@
|
|||||||
#include <Interpreters/evaluateConstantExpression.h>
|
#include <Interpreters/evaluateConstantExpression.h>
|
||||||
#include <Parsers/ASTCreateQuery.h>
|
#include <Parsers/ASTCreateQuery.h>
|
||||||
#include <Parsers/ASTExpressionList.h>
|
#include <Parsers/ASTExpressionList.h>
|
||||||
|
#include <Parsers/ASTIdentifier.h>
|
||||||
#include <Parsers/ASTInsertQuery.h>
|
#include <Parsers/ASTInsertQuery.h>
|
||||||
#include <Parsers/ASTLiteral.h>
|
#include <Parsers/ASTLiteral.h>
|
||||||
#include <Processors/Executors/CompletedPipelineExecutor.h>
|
#include <Processors/Executors/CompletedPipelineExecutor.h>
|
||||||
|
@ -5,7 +5,6 @@
|
|||||||
#include <Interpreters/ExpressionAnalyzer.h>
|
#include <Interpreters/ExpressionAnalyzer.h>
|
||||||
#include <Interpreters/TreeRewriter.h>
|
#include <Interpreters/TreeRewriter.h>
|
||||||
|
|
||||||
#include <Parsers/ASTIdentifier.h>
|
|
||||||
#include <Parsers/ASTFunction.h>
|
#include <Parsers/ASTFunction.h>
|
||||||
#include <Parsers/ASTLiteral.h>
|
#include <Parsers/ASTLiteral.h>
|
||||||
|
|
||||||
|
@ -2,7 +2,6 @@
|
|||||||
#include <Storages/IStorage.h>
|
#include <Storages/IStorage.h>
|
||||||
#include <Storages/DataDestinationType.h>
|
#include <Storages/DataDestinationType.h>
|
||||||
#include <Parsers/ASTAlterQuery.h>
|
#include <Parsers/ASTAlterQuery.h>
|
||||||
#include <Parsers/ASTIdentifier.h>
|
|
||||||
#include <Core/ColumnWithTypeAndName.h>
|
#include <Core/ColumnWithTypeAndName.h>
|
||||||
#include <DataTypes/DataTypeString.h>
|
#include <DataTypes/DataTypeString.h>
|
||||||
#include <Processors/Chunk.h>
|
#include <Processors/Chunk.h>
|
||||||
|
@ -6,6 +6,7 @@
|
|||||||
#include <DataTypes/DataTypeNullable.h>
|
#include <DataTypes/DataTypeNullable.h>
|
||||||
#include <Interpreters/Context.h>
|
#include <Interpreters/Context.h>
|
||||||
#include <Interpreters/InterpreterInsertQuery.h>
|
#include <Interpreters/InterpreterInsertQuery.h>
|
||||||
|
#include <Parsers/ASTIdentifier.h>
|
||||||
#include <Processors/Sources/SourceFromSingleChunk.h>
|
#include <Processors/Sources/SourceFromSingleChunk.h>
|
||||||
#include <Processors/Executors/CompletedPipelineExecutor.h>
|
#include <Processors/Executors/CompletedPipelineExecutor.h>
|
||||||
#include <Common/SettingsChanges.h>
|
#include <Common/SettingsChanges.h>
|
||||||
|
@ -1,5 +1,7 @@
|
|||||||
#include "StorageMaterializedPostgreSQL.h"
|
#include "StorageMaterializedPostgreSQL.h"
|
||||||
|
|
||||||
|
#include <Parsers/ASTIdentifier.h>
|
||||||
|
|
||||||
#if USE_LIBPQXX
|
#if USE_LIBPQXX
|
||||||
#include <base/logger_useful.h>
|
#include <base/logger_useful.h>
|
||||||
#include <Common/Macros.h>
|
#include <Common/Macros.h>
|
||||||
|
@ -9,7 +9,6 @@
|
|||||||
#include <Parsers/IAST.h>
|
#include <Parsers/IAST.h>
|
||||||
#include <Parsers/ASTLiteral.h>
|
#include <Parsers/ASTLiteral.h>
|
||||||
#include <Parsers/ASTFunction.h>
|
#include <Parsers/ASTFunction.h>
|
||||||
#include <Parsers/ASTIdentifier.h>
|
|
||||||
#include <Parsers/ASTCreateQuery.h>
|
#include <Parsers/ASTCreateQuery.h>
|
||||||
#include <Parsers/ASTColumnDeclaration.h>
|
#include <Parsers/ASTColumnDeclaration.h>
|
||||||
#include <Interpreters/evaluateConstantExpression.h>
|
#include <Interpreters/evaluateConstantExpression.h>
|
||||||
|
@ -2,15 +2,15 @@
|
|||||||
#include <Interpreters/TreeRewriter.h>
|
#include <Interpreters/TreeRewriter.h>
|
||||||
#include <Storages/ProjectionsDescription.h>
|
#include <Storages/ProjectionsDescription.h>
|
||||||
|
|
||||||
|
#include <Parsers/ASTIdentifier.h>
|
||||||
#include <Parsers/ASTProjectionDeclaration.h>
|
#include <Parsers/ASTProjectionDeclaration.h>
|
||||||
|
#include <Parsers/ASTProjectionSelectQuery.h>
|
||||||
#include <Parsers/ParserCreateQuery.h>
|
#include <Parsers/ParserCreateQuery.h>
|
||||||
#include <Parsers/parseQuery.h>
|
#include <Parsers/parseQuery.h>
|
||||||
#include <Parsers/queryToString.h>
|
#include <Parsers/queryToString.h>
|
||||||
|
|
||||||
#include <Core/Defines.h>
|
#include <Core/Defines.h>
|
||||||
#include <Interpreters/InterpreterSelectQuery.h>
|
#include <Interpreters/InterpreterSelectQuery.h>
|
||||||
#include <Parsers/ASTProjectionSelectQuery.h>
|
|
||||||
#include <Parsers/ASTSubquery.h>
|
|
||||||
#include <QueryPipeline/Pipe.h>
|
#include <QueryPipeline/Pipe.h>
|
||||||
#include <Processors/Sources/SourceFromSingleChunk.h>
|
#include <Processors/Sources/SourceFromSingleChunk.h>
|
||||||
#include <Processors/Transforms/SquashingChunksTransform.h>
|
#include <Processors/Transforms/SquashingChunksTransform.h>
|
||||||
|
@ -5,9 +5,9 @@
|
|||||||
#include <Interpreters/evaluateConstantExpression.h>
|
#include <Interpreters/evaluateConstantExpression.h>
|
||||||
|
|
||||||
#include <Parsers/ASTCreateQuery.h>
|
#include <Parsers/ASTCreateQuery.h>
|
||||||
#include <Parsers/ASTLiteral.h>
|
#include <Parsers/ASTIdentifier_fwd.h>
|
||||||
#include <Parsers/ASTIdentifier.h>
|
|
||||||
#include <Parsers/ASTInsertQuery.h>
|
#include <Parsers/ASTInsertQuery.h>
|
||||||
|
#include <Parsers/ASTLiteral.h>
|
||||||
|
|
||||||
#include <IO/ReadBufferFromFile.h>
|
#include <IO/ReadBufferFromFile.h>
|
||||||
#include <IO/ReadBufferFromFileDescriptor.h>
|
#include <IO/ReadBufferFromFileDescriptor.h>
|
||||||
|
@ -5,7 +5,7 @@
|
|||||||
#include <Interpreters/HashJoin.h>
|
#include <Interpreters/HashJoin.h>
|
||||||
#include <Interpreters/Context.h>
|
#include <Interpreters/Context.h>
|
||||||
#include <Parsers/ASTCreateQuery.h>
|
#include <Parsers/ASTCreateQuery.h>
|
||||||
#include <Parsers/ASTIdentifier.h>
|
#include <Parsers/ASTIdentifier_fwd.h>
|
||||||
#include <Core/ColumnNumbers.h>
|
#include <Core/ColumnNumbers.h>
|
||||||
#include <DataTypes/NestedUtils.h>
|
#include <DataTypes/NestedUtils.h>
|
||||||
#include <Interpreters/joinDispatch.h>
|
#include <Interpreters/joinDispatch.h>
|
||||||
|
@ -4,9 +4,10 @@
|
|||||||
#include <DataTypes/DataTypeLowCardinality.h>
|
#include <DataTypes/DataTypeLowCardinality.h>
|
||||||
|
|
||||||
#include <Parsers/ASTCreateQuery.h>
|
#include <Parsers/ASTCreateQuery.h>
|
||||||
|
#include <Parsers/ASTIdentifier.h>
|
||||||
|
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||||
#include <Parsers/ASTSubquery.h>
|
#include <Parsers/ASTSubquery.h>
|
||||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
|
||||||
|
|
||||||
#include <Storages/StorageView.h>
|
#include <Storages/StorageView.h>
|
||||||
#include <Storages/StorageFactory.h>
|
#include <Storages/StorageFactory.h>
|
||||||
|
@ -2,7 +2,7 @@
|
|||||||
|
|
||||||
#include <Storages/getStructureOfRemoteTable.h>
|
#include <Storages/getStructureOfRemoteTable.h>
|
||||||
#include <Storages/StorageDistributed.h>
|
#include <Storages/StorageDistributed.h>
|
||||||
#include <Parsers/ASTIdentifier.h>
|
#include <Parsers/ASTIdentifier_fwd.h>
|
||||||
#include <Parsers/ASTLiteral.h>
|
#include <Parsers/ASTLiteral.h>
|
||||||
#include <Parsers/ASTFunction.h>
|
#include <Parsers/ASTFunction.h>
|
||||||
#include <Parsers/ASTExpressionList.h>
|
#include <Parsers/ASTExpressionList.h>
|
||||||
|
Loading…
Reference in New Issue
Block a user