mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Forward declaration for Context as much as possible.
Now after changing Context.h 488 modules will be recompiled instead of 582.
This commit is contained in:
parent
07924f0ae1
commit
d93b9a57f6
@ -5,6 +5,7 @@
|
||||
#include <Client/Connection.h>
|
||||
#include <IO/ConnectionTimeouts.h>
|
||||
#include <common/LineReader.h>
|
||||
#include <thread>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -8,7 +8,6 @@
|
||||
#include <Poco/NullChannel.h>
|
||||
#include <Databases/DatabaseMemory.h>
|
||||
#include <Storages/System/attachSystemTables.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/ProcessList.h>
|
||||
#include <Interpreters/executeQuery.h>
|
||||
#include <Interpreters/loadMetadata.h>
|
||||
|
@ -4,13 +4,12 @@
|
||||
#include <Poco/Util/Application.h>
|
||||
#include <memory>
|
||||
#include <loggers/Loggers.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Context;
|
||||
|
||||
/// Lightweight Application for clickhouse-local
|
||||
/// No networking, no extra configs and working directories, no pid and status files, no dictionaries, no logging.
|
||||
/// Quiet mode by default
|
||||
|
@ -50,6 +50,8 @@ class Connection;
|
||||
using ConnectionPtr = std::shared_ptr<Connection>;
|
||||
using Connections = std::vector<ConnectionPtr>;
|
||||
|
||||
using Scalars = std::map<String, Block>;
|
||||
|
||||
|
||||
/// Packet that could be received from server.
|
||||
struct Packet
|
||||
|
@ -3,12 +3,12 @@
|
||||
#include <DataStreams/IBlockOutputStream.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Storages/ColumnDefault.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Context;
|
||||
|
||||
/** This stream adds three types of columns into block
|
||||
* 1. Columns, that are missed inside request, but present in table without defaults (missed columns)
|
||||
|
@ -2,12 +2,13 @@
|
||||
|
||||
#include <DataStreams/IBlockInputStream.h>
|
||||
#include <Storages/ColumnDefault.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Context;
|
||||
|
||||
/// Adds defaults to columns using BlockDelayedDefaults bitmask attached to Block by child InputStream.
|
||||
class AddingDefaultsBlockInputStream : public IBlockInputStream
|
||||
{
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <Interpreters/Set.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <DataStreams/materializeBlock.h>
|
||||
#include <DataStreams/IBlockOutputStream.h>
|
||||
#include <DataStreams/CreatingSetsBlockInputStream.h>
|
||||
|
@ -8,7 +8,6 @@
|
||||
#include <IO/ReadBuffer.h>
|
||||
#include <Processors/Formats/IRowInputFormat.h>
|
||||
#include <Processors/Formats/InputStreamFromInputFormat.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <DataTypes/NestedUtils.h>
|
||||
#include <Interpreters/InterpreterSelectQuery.h>
|
||||
#include <Interpreters/InterpreterInsertQuery.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Parsers/ASTInsertQuery.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Common/setThreadName.h>
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <DataStreams/IBlockOutputStream.h>
|
||||
#include <Common/Throttler.h>
|
||||
#include <IO/ConnectionTimeouts.h>
|
||||
#include <Interpreters/ClientInfo.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -8,6 +8,7 @@
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <Common/renameat2.h>
|
||||
#include <Storages/StorageMaterializedView.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <filesystem>
|
||||
|
||||
|
||||
|
@ -1,7 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <Databases/DatabaseOnDisk.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
|
||||
|
||||
@ -10,6 +9,7 @@ namespace DB
|
||||
|
||||
|
||||
class DatabaseLazyIterator;
|
||||
class Context;
|
||||
|
||||
/** Lazy engine of databases.
|
||||
* Works like DatabaseOrdinary, but stores in memory only cache.
|
||||
|
@ -5,14 +5,16 @@
|
||||
|
||||
#include <mysqlxx/Pool.h>
|
||||
#include <Databases/DatabasesCommon.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <memory>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Context;
|
||||
|
||||
/** Real-time access to table list and table structure from remote MySQL
|
||||
* It doesn't make any manipulations with filesystem.
|
||||
* All tables are created by calling code after real-time pull-out structure from remote MySQL
|
||||
|
@ -3,7 +3,6 @@
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <Databases/DatabasesCommon.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Storages/IStorage.h>
|
||||
|
||||
@ -11,6 +10,8 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Context;
|
||||
|
||||
std::pair<String, StoragePtr> createTableFromAST(
|
||||
ASTCreateQuery ast_create_query,
|
||||
const String & database_name,
|
||||
|
@ -5,6 +5,9 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Context;
|
||||
class ExternalDictionariesLoader;
|
||||
|
||||
|
||||
class DatabaseWithDictionaries : public DatabaseOnDisk
|
||||
{
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <common/logger_useful.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
#include <set>
|
||||
|
||||
|
@ -1,6 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Disks/DiskFactory.h>
|
||||
#include <Disks/IDisk.h>
|
||||
|
||||
@ -10,6 +9,7 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class Context;
|
||||
class DiskSelector;
|
||||
using DiskSelectorPtr = std::shared_ptr<const DiskSelector>;
|
||||
|
||||
|
@ -18,6 +18,7 @@
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
#include <string>
|
||||
#include <memory>
|
||||
|
@ -14,6 +14,7 @@
|
||||
#include <Poco/URI.h>
|
||||
#include <Poco/Version.h>
|
||||
#include <Common/DNSResolver.h>
|
||||
#include <Common/RemoteHostFilter.h>
|
||||
#include <common/logger_useful.h>
|
||||
#include <Poco/URIStreamFactory.h>
|
||||
|
||||
|
@ -32,6 +32,7 @@
|
||||
#include <Parsers/ASTSubquery.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Interpreters/misc.h>
|
||||
#include <Interpreters/ActionsVisitor.h>
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Interpreters/AsynchronousMetrics.h>
|
||||
#include <Interpreters/ExpressionJIT.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/setThreadName.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
|
@ -2,11 +2,14 @@
|
||||
|
||||
#include <Core/QueryProcessingStage.h>
|
||||
#include <Interpreters/ClusterProxy/IStreamFactory.h>
|
||||
#include <Interpreters/StorageID.h>
|
||||
#include <Storages/IStorage_fwd.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using Scalars = std::map<String, Block>;
|
||||
|
||||
namespace ClusterProxy
|
||||
{
|
||||
|
||||
|
@ -17,6 +17,7 @@
|
||||
#include <Interpreters/executeQuery.h>
|
||||
#include <Interpreters/Cluster.h>
|
||||
#include <Interpreters/AddDefaultDatabaseVisitor.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Access/AccessRightsElement.h>
|
||||
#include <Common/DNSResolver.h>
|
||||
#include <Common/Macros.h>
|
||||
|
@ -1,5 +1,5 @@
|
||||
#pragma once
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
#include <Interpreters/Cluster.h>
|
||||
#include <DataStreams/BlockIO.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
@ -13,9 +13,15 @@
|
||||
#include <mutex>
|
||||
#include <thread>
|
||||
|
||||
namespace zkutil
|
||||
{
|
||||
class ZooKeeper;
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Context;
|
||||
class ASTAlterQuery;
|
||||
class AccessRightsElements;
|
||||
struct DDLLogEntry;
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Interpreters/ExpressionJIT.h>
|
||||
#include <Interpreters/TableJoin.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
@ -509,6 +510,33 @@ std::string ExpressionAction::toString() const
|
||||
return ss.str();
|
||||
}
|
||||
|
||||
ExpressionActions::ExpressionActions(const NamesAndTypesList & input_columns_, const Context & context_)
|
||||
: input_columns(input_columns_), settings(context_.getSettingsRef())
|
||||
{
|
||||
for (const auto & input_elem : input_columns)
|
||||
sample_block.insert(ColumnWithTypeAndName(nullptr, input_elem.type, input_elem.name));
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
compilation_cache = context_.getCompiledExpressionCache();
|
||||
#endif
|
||||
}
|
||||
|
||||
/// For constant columns the columns themselves can be contained in `input_columns_`.
|
||||
ExpressionActions::ExpressionActions(const ColumnsWithTypeAndName & input_columns_, const Context & context_)
|
||||
: settings(context_.getSettingsRef())
|
||||
{
|
||||
for (const auto & input_elem : input_columns_)
|
||||
{
|
||||
input_columns.emplace_back(input_elem.name, input_elem.type);
|
||||
sample_block.insert(input_elem);
|
||||
}
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
compilation_cache = context_.getCompiledExpressionCache();
|
||||
#endif
|
||||
}
|
||||
|
||||
ExpressionActions::~ExpressionActions() = default;
|
||||
|
||||
void ExpressionActions::checkLimits(Block & block) const
|
||||
{
|
||||
if (settings.max_temporary_columns && block.columns() > settings.max_temporary_columns)
|
||||
|
@ -4,7 +4,6 @@
|
||||
#include <Core/ColumnWithTypeAndName.h>
|
||||
#include <Core/Names.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <Common/UInt128.h>
|
||||
#include <unordered_map>
|
||||
@ -25,6 +24,7 @@ namespace ErrorCodes
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
class Context;
|
||||
class TableJoin;
|
||||
class IJoin;
|
||||
using JoinPtr = std::shared_ptr<IJoin>;
|
||||
@ -42,6 +42,7 @@ class IDataType;
|
||||
using DataTypePtr = std::shared_ptr<const IDataType>;
|
||||
|
||||
class ExpressionActions;
|
||||
class CompiledExpressionCache;
|
||||
|
||||
/** Action on the block.
|
||||
*/
|
||||
@ -155,30 +156,12 @@ class ExpressionActions
|
||||
public:
|
||||
using Actions = std::vector<ExpressionAction>;
|
||||
|
||||
ExpressionActions(const NamesAndTypesList & input_columns_, const Context & context_)
|
||||
: input_columns(input_columns_), settings(context_.getSettingsRef())
|
||||
{
|
||||
for (const auto & input_elem : input_columns)
|
||||
sample_block.insert(ColumnWithTypeAndName(nullptr, input_elem.type, input_elem.name));
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
compilation_cache = context_.getCompiledExpressionCache();
|
||||
#endif
|
||||
}
|
||||
ExpressionActions(const NamesAndTypesList & input_columns_, const Context & context_);
|
||||
|
||||
/// For constant columns the columns themselves can be contained in `input_columns_`.
|
||||
ExpressionActions(const ColumnsWithTypeAndName & input_columns_, const Context & context_)
|
||||
: settings(context_.getSettingsRef())
|
||||
{
|
||||
for (const auto & input_elem : input_columns_)
|
||||
{
|
||||
input_columns.emplace_back(input_elem.name, input_elem.type);
|
||||
sample_block.insert(input_elem);
|
||||
}
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
compilation_cache = context_.getCompiledExpressionCache();
|
||||
#endif
|
||||
}
|
||||
ExpressionActions(const ColumnsWithTypeAndName & input_columns_, const Context & context_);
|
||||
|
||||
~ExpressionActions();
|
||||
|
||||
/// Add the input column.
|
||||
/// The name of the column must not match the names of the intermediate columns that occur when evaluating the expression.
|
||||
|
@ -32,6 +32,7 @@
|
||||
#include <Interpreters/HashJoin.h>
|
||||
#include <Interpreters/MergeJoin.h>
|
||||
#include <Interpreters/DictionaryReader.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <AggregateFunctions/parseAggregateFunctionParameters.h>
|
||||
|
@ -7,7 +7,6 @@
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/interpretSubquery.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Core/Block.h>
|
||||
@ -17,6 +16,7 @@
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/InDepthNodeVisitor.h>
|
||||
#include <Interpreters/IdentifierSemantic.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -7,6 +7,7 @@
|
||||
#include <Interpreters/IdentifierSemantic.h>
|
||||
#include <Interpreters/getTableExpressions.h>
|
||||
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Parsers/DumpASTNode.h>
|
||||
#include <Parsers/ASTSubquery.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
|
@ -1,6 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/IInterpreter.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
|
||||
@ -8,6 +7,8 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Context;
|
||||
|
||||
/// Returns single row with explain results
|
||||
class InterpreterExplainQuery : public IInterpreter
|
||||
{
|
||||
|
@ -61,6 +61,7 @@
|
||||
#include <Interpreters/InterpreterUseQuery.h>
|
||||
#include <Interpreters/InterpreterWatchQuery.h>
|
||||
#include <Interpreters/InterpreterGrantQuery.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
#include <Parsers/ASTSystemQuery.h>
|
||||
|
||||
|
@ -2,13 +2,14 @@
|
||||
|
||||
#include <DataStreams/IBlockOutputStream.h>
|
||||
#include <DataStreams/BlockIO.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/IInterpreter.h>
|
||||
#include <Parsers/ASTInsertQuery.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Context;
|
||||
|
||||
|
||||
/** Interprets the INSERT query.
|
||||
*/
|
||||
|
@ -5,7 +5,6 @@
|
||||
#include <Core/QueryProcessingStage.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <DataStreams/IBlockStream_fwd.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Interpreters/ExpressionAnalyzer.h>
|
||||
#include <Interpreters/IInterpreter.h>
|
||||
@ -25,6 +24,7 @@ namespace DB
|
||||
|
||||
struct SubqueryForSet;
|
||||
class InterpreterSelectWithUnionQuery;
|
||||
class Context;
|
||||
|
||||
struct SyntaxAnalyzerResult;
|
||||
using SyntaxAnalyzerResultPtr = std::shared_ptr<const SyntaxAnalyzerResult>;
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
||||
#include <Interpreters/InterpreterSelectQuery.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <DataStreams/UnionBlockInputStream.h>
|
||||
|
@ -1,15 +1,16 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/QueryProcessingStage.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/IInterpreter.h>
|
||||
#include <Interpreters/SelectQueryOptions.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
|
||||
#include <Processors/QueryPipeline.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Context;
|
||||
class InterpreterSelectQuery;
|
||||
|
||||
|
||||
|
@ -6,6 +6,8 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Context;
|
||||
class ASTSetRoleQuery;
|
||||
struct ExtendedRoleSet;
|
||||
struct User;
|
||||
|
@ -13,6 +13,7 @@ limitations under the License. */
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Parsers/ASTWatchQuery.h>
|
||||
#include <Interpreters/InterpreterWatchQuery.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Access/AccessFlags.h>
|
||||
#include <DataStreams/IBlockInputStream.h>
|
||||
#include <DataStreams/OneBlockInputStream.h>
|
||||
|
@ -18,11 +18,11 @@ limitations under the License. */
|
||||
#include <Interpreters/IInterpreter.h>
|
||||
#include <Storages/SelectQueryInfo.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Context;
|
||||
class IAST;
|
||||
using ASTPtr = std::shared_ptr<IAST>;
|
||||
using StoragePtr = std::shared_ptr<IStorage>;
|
||||
|
@ -1,6 +1,5 @@
|
||||
#include <Interpreters/JoinedTables.h>
|
||||
#include <Interpreters/TableJoin.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/getTableExpressions.h>
|
||||
#include <Interpreters/InJoinSubqueriesPreprocessor.h>
|
||||
#include <Interpreters/IdentifierSemantic.h>
|
||||
|
@ -3,13 +3,14 @@
|
||||
#include <Core/NamesAndTypes.h>
|
||||
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
||||
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/StorageID.h>
|
||||
#include <Storages/IStorage_fwd.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ASTSelectQuery;
|
||||
class Context;
|
||||
class TableJoin;
|
||||
struct SelectQueryOptions;
|
||||
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Interpreters/ExpressionAnalyzer.h>
|
||||
#include <Interpreters/InterpreterSelectQuery.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Storages/IStorage_fwd.h>
|
||||
#include <Storages/MutationCommands.h>
|
||||
|
||||
|
@ -10,6 +10,7 @@
|
||||
#include <Storages/MergeTree/IMergeTreeDataPart.h>
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
#include <Interpreters/PartLog.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <Interpreters/SystemLog.h>
|
||||
#include <Interpreters/ClientInfo.h>
|
||||
#include <Core/SettingsCollection.h>
|
||||
|
||||
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <Interpreters/SystemLog.h>
|
||||
#include <Interpreters/ClientInfo.h>
|
||||
|
||||
|
||||
namespace ProfileEvents
|
||||
|
@ -23,6 +23,7 @@
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Interpreters/NullableUtils.h>
|
||||
#include <Interpreters/sortBlock.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
#include <Storages/MergeTree/KeyCondition.h>
|
||||
|
||||
|
@ -5,7 +5,6 @@
|
||||
#include <DataStreams/SizeLimits.h>
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <Interpreters/SetVariants.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Storages/MergeTree/BoolMask.h>
|
||||
|
||||
@ -17,6 +16,7 @@ namespace DB
|
||||
|
||||
struct Range;
|
||||
|
||||
class Context;
|
||||
class IFunctionBase;
|
||||
using FunctionBasePtr = std::shared_ptr<IFunctionBase>;
|
||||
|
||||
|
@ -11,7 +11,6 @@
|
||||
#include <Core/Types.h>
|
||||
#include <Core/Defines.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Common/Stopwatch.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
@ -65,6 +64,12 @@ namespace ErrorCodes
|
||||
|
||||
|
||||
class Context;
|
||||
class QueryLog;
|
||||
class QueryThreadLog;
|
||||
class PartLog;
|
||||
class TextLog;
|
||||
class TraceLog;
|
||||
class MetricLog;
|
||||
|
||||
|
||||
class ISystemLog
|
||||
|
@ -31,6 +31,7 @@
|
||||
#include <Interpreters/InterpreterSetQuery.h>
|
||||
#include <Interpreters/ReplaceQueryParameterVisitor.h>
|
||||
#include <Interpreters/executeQuery.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
|
||||
#include <Interpreters/DNSCacheUpdater.h>
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <Interpreters/getTableExpressions.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
|
@ -12,6 +12,7 @@
|
||||
|
||||
#include <Interpreters/interpretSubquery.h>
|
||||
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -8,6 +8,8 @@
|
||||
#include <Interpreters/ExpressionAnalyzer.h>
|
||||
#include <Interpreters/ReplaceQueryParameterVisitor.h>
|
||||
#include <Interpreters/SyntaxAnalyzer.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/convertFieldToType.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
@ -17,7 +19,6 @@
|
||||
#include <Parsers/CommonParsers.h>
|
||||
#include <Processors/Formats/Impl/ConstantExpressionTemplate.h>
|
||||
#include <Parsers/ExpressionElementParsers.h>
|
||||
#include <Interpreters/convertFieldToType.h>
|
||||
#include <boost/functional/hash.hpp>
|
||||
|
||||
|
||||
|
@ -81,13 +81,13 @@ StorageDistributedDirectoryMonitor::StorageDistributedDirectoryMonitor(
|
||||
: storage(storage_)
|
||||
, pool(std::move(pool_))
|
||||
, path{path_ + '/'}
|
||||
, should_batch_inserts(storage.global_context.getSettingsRef().distributed_directory_monitor_batch_inserts)
|
||||
, min_batched_block_size_rows(storage.global_context.getSettingsRef().min_insert_block_size_rows)
|
||||
, min_batched_block_size_bytes(storage.global_context.getSettingsRef().min_insert_block_size_bytes)
|
||||
, should_batch_inserts(storage.global_context->getSettingsRef().distributed_directory_monitor_batch_inserts)
|
||||
, min_batched_block_size_rows(storage.global_context->getSettingsRef().min_insert_block_size_rows)
|
||||
, min_batched_block_size_bytes(storage.global_context->getSettingsRef().min_insert_block_size_bytes)
|
||||
, current_batch_file_path{path + "current_batch.txt"}
|
||||
, default_sleep_time{storage.global_context.getSettingsRef().distributed_directory_monitor_sleep_time_ms.totalMilliseconds()}
|
||||
, default_sleep_time{storage.global_context->getSettingsRef().distributed_directory_monitor_sleep_time_ms.totalMilliseconds()}
|
||||
, sleep_time{default_sleep_time}
|
||||
, max_sleep_time{storage.global_context.getSettingsRef().distributed_directory_monitor_max_sleep_time_ms.totalMilliseconds()}
|
||||
, max_sleep_time{storage.global_context->getSettingsRef().distributed_directory_monitor_max_sleep_time_ms.totalMilliseconds()}
|
||||
, log{&Logger::get(getLoggerName())}
|
||||
, monitor_blocker(monitor_blocker_)
|
||||
, bg_pool(bg_pool_)
|
||||
@ -214,7 +214,7 @@ ConnectionPoolPtr StorageDistributedDirectoryMonitor::createPool(const std::stri
|
||||
|
||||
auto pools = createPoolsForAddresses(name, pool_factory);
|
||||
|
||||
const auto settings = storage.global_context.getSettings();
|
||||
const auto settings = storage.global_context->getSettings();
|
||||
return pools.size() == 1 ? pools.front() : std::make_shared<ConnectionPoolWithFailover>(pools,
|
||||
settings.load_balancing,
|
||||
settings.distributed_replica_error_half_life.totalSeconds(),
|
||||
@ -262,7 +262,7 @@ bool StorageDistributedDirectoryMonitor::processFiles()
|
||||
void StorageDistributedDirectoryMonitor::processFile(const std::string & file_path)
|
||||
{
|
||||
LOG_TRACE(log, "Started processing `" << file_path << '`');
|
||||
auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(storage.global_context.getSettingsRef());
|
||||
auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(storage.global_context->getSettingsRef());
|
||||
auto connection = pool->get(timeouts);
|
||||
|
||||
try
|
||||
@ -437,7 +437,7 @@ struct StorageDistributedDirectoryMonitor::Batch
|
||||
|
||||
Poco::File{tmp_file}.renameTo(parent.current_batch_file_path);
|
||||
}
|
||||
auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(parent.storage.global_context.getSettingsRef());
|
||||
auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(parent.storage.global_context->getSettingsRef());
|
||||
auto connection = parent.pool->get(timeouts);
|
||||
|
||||
bool batch_broken = false;
|
||||
|
@ -17,6 +17,7 @@
|
||||
#include <Interpreters/InterpreterInsertQuery.h>
|
||||
#include <Interpreters/createBlockSelector.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
|
@ -11,7 +11,6 @@
|
||||
#include <chrono>
|
||||
#include <optional>
|
||||
#include <Interpreters/Cluster.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
|
||||
namespace Poco
|
||||
@ -22,6 +21,7 @@ namespace Poco
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Context;
|
||||
class StorageDistributed;
|
||||
|
||||
/** If insert_sync_ is true, the write is synchronous. Uses insert_timeout_ if it is not zero.
|
||||
|
@ -36,6 +36,7 @@ using VolumePtr = std::shared_ptr<IVolume>;
|
||||
|
||||
class IMergeTreeReader;
|
||||
class IMergeTreeDataPartWriter;
|
||||
class MarkCache;
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
|
@ -3,7 +3,6 @@
|
||||
#include <sstream>
|
||||
#include <optional>
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/Set.h>
|
||||
#include <Core/SortDescription.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
@ -15,10 +14,9 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
class Context;
|
||||
class IFunction;
|
||||
using FunctionBasePtr = std::shared_ptr<IFunctionBase>;
|
||||
|
||||
class ExpressionActions;
|
||||
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
|
||||
|
||||
|
@ -3,7 +3,6 @@
|
||||
#include <Common/Stopwatch.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
#include <Common/MemoryTracker.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
#include <memory>
|
||||
#include <list>
|
||||
|
@ -18,6 +18,7 @@
|
||||
#include <Interpreters/ExpressionAnalyzer.h>
|
||||
#include <Interpreters/PartLog.h>
|
||||
#include <Interpreters/SyntaxAnalyzer.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTNameTypePair.h>
|
||||
|
@ -1,7 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/SimpleIncrement.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Common/MultiVersion.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Storages/MergeTree/MergeTreeIndices.h>
|
||||
#include <Storages/MergeTree/MergeTreePartInfo.h>
|
||||
@ -34,6 +34,7 @@ class MergeListEntry;
|
||||
class AlterCommands;
|
||||
class MergeTreePartsMover;
|
||||
class MutationCommands;
|
||||
class Context;
|
||||
|
||||
class ExpressionActions;
|
||||
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
|
||||
|
@ -26,6 +26,7 @@
|
||||
#include <Processors/Transforms/MaterializingTransform.h>
|
||||
#include <Processors/Executors/TreeExecutorBlockInputStream.h>
|
||||
#include <Interpreters/MutationsInterpreter.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Common/SimpleIncrement.h>
|
||||
#include <Common/interpolate.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <Storages/MergeTree/MergeTreeDataPartWriterWide.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -18,6 +18,7 @@
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTSampleRatio.h>
|
||||
#include <Interpreters/ExpressionAnalyzer.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
/// Allow to use __uint128_t as a template parameter for boost::rational.
|
||||
// https://stackoverflow.com/questions/41198673/uint128-t-not-working-with-clang-and-libstdc
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Common/Exception.h>
|
||||
#include <Disks/createVolume.h>
|
||||
#include <Interpreters/AggregationCommon.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <IO/HashingWriteBuffer.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
|
@ -9,7 +9,6 @@
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
|
||||
#include <Interpreters/sortBlock.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Storages/MergeTree/MergeTreeReverseSelectProcessor.h>
|
||||
#include <Storages/MergeTree/MergeTreeBaseSelectProcessor.h>
|
||||
#include <Storages/MergeTree/IMergeTreeReader.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Storages/MergeTree/MergeTreeSelectProcessor.h>
|
||||
#include <Storages/MergeTree/MergeTreeBaseSelectProcessor.h>
|
||||
#include <Storages/MergeTree/IMergeTreeReader.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Storages/MergeTree/MergeTreeSequentialSource.h>
|
||||
#include <Storages/MergeTree/MergeTreeBlockReadUtils.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Storages/MergeTree/IMergeTreeReader.h>
|
||||
#include <Storages/MergeTree/MergeTreeReadPool.h>
|
||||
#include <Storages/MergeTree/MergeTreeThreadSelectBlockInputProcessor.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <Storages/MergeTree/MergedBlockOutputStream.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Poco/File.h>
|
||||
|
||||
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <Storages/MergeTree/MergedColumnOnlyOutputStream.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -3,7 +3,6 @@
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Core/Block.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
@ -13,6 +12,7 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class Context;
|
||||
|
||||
/// Builds reverse polish notation
|
||||
template <typename RPNElement>
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h>
|
||||
#include <Storages/StorageReplicatedMergeTree.h>
|
||||
#include <Poco/Timestamp.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
#include <random>
|
||||
#include <unordered_set>
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <Storages/MergeTree/checkDataPart.h>
|
||||
#include <Storages/MergeTree/ReplicatedMergeTreePartHeader.h>
|
||||
#include <Storages/StorageReplicatedMergeTree.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
|
||||
namespace ProfileEvents
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h>
|
||||
#include <Storages/MergeTree/ReplicatedMergeTreeQuorumEntry.h>
|
||||
#include <Storages/MergeTree/ReplicatedMergeTreeAddress.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Common/ZooKeeper/KeeperException.h>
|
||||
#include <Common/randomSeed.h>
|
||||
|
||||
|
@ -17,6 +17,8 @@
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <AggregateFunctions/parseAggregateFunctionParameters.h>
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -8,6 +8,7 @@
|
||||
#include <Storages/IStorage.h>
|
||||
#include <DataStreams/IBlockOutputStream.h>
|
||||
#include <Poco/Event.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
|
||||
namespace Poco { class Logger; }
|
||||
@ -16,8 +17,6 @@ namespace Poco { class Logger; }
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Context;
|
||||
|
||||
|
||||
/** During insertion, buffers the data in the RAM until certain thresholds are exceeded.
|
||||
* When thresholds are exceeded, flushes the data to another table.
|
||||
|
@ -40,6 +40,7 @@
|
||||
#include <Interpreters/InterpreterSelectQuery.h>
|
||||
#include <Interpreters/TranslateQualifiedNamesVisitor.h>
|
||||
#include <Interpreters/SyntaxAnalyzer.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/createBlockSelector.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Interpreters/getClusterName.h>
|
||||
@ -276,8 +277,8 @@ StorageDistributed::StorageDistributed(
|
||||
: IStorage(id_)
|
||||
, remote_database(remote_database_)
|
||||
, remote_table(remote_table_)
|
||||
, global_context(context_)
|
||||
, cluster_name(global_context.getMacros()->expand(cluster_name_))
|
||||
, global_context(std::make_unique<Context>(context_))
|
||||
, cluster_name(global_context->getMacros()->expand(cluster_name_))
|
||||
, has_sharding_key(sharding_key_)
|
||||
, storage_policy(storage_policy_)
|
||||
, relative_data_path(relative_data_path_)
|
||||
@ -287,7 +288,7 @@ StorageDistributed::StorageDistributed(
|
||||
|
||||
if (sharding_key_)
|
||||
{
|
||||
sharding_key_expr = buildShardingKeyExpression(sharding_key_, global_context, getColumns().getAllPhysical(), false);
|
||||
sharding_key_expr = buildShardingKeyExpression(sharding_key_, *global_context, getColumns().getAllPhysical(), false);
|
||||
sharding_key_column_name = sharding_key_->getColumnName();
|
||||
}
|
||||
|
||||
@ -297,7 +298,7 @@ StorageDistributed::StorageDistributed(
|
||||
/// Sanity check. Skip check if the table is already created to allow the server to start.
|
||||
if (!attach_ && !cluster_name.empty())
|
||||
{
|
||||
size_t num_local_shards = global_context.getCluster(cluster_name)->getLocalShardCount();
|
||||
size_t num_local_shards = global_context->getCluster(cluster_name)->getLocalShardCount();
|
||||
if (num_local_shards && remote_database == id_.database_name && remote_table == id_.table_name)
|
||||
throw Exception("Distributed table " + id_.table_name + " looks at itself", ErrorCodes::INFINITE_LOOP);
|
||||
}
|
||||
@ -325,7 +326,7 @@ void StorageDistributed::createStorage()
|
||||
/// Create default policy with the relative_data_path_
|
||||
if (storage_policy.empty())
|
||||
{
|
||||
std::string path(global_context.getPath());
|
||||
std::string path(global_context->getPath());
|
||||
/// Disk must ends with '/'
|
||||
if (!path.ends_with('/'))
|
||||
path += '/';
|
||||
@ -334,7 +335,7 @@ void StorageDistributed::createStorage()
|
||||
}
|
||||
else
|
||||
{
|
||||
auto policy = global_context.getStoragePolicySelector()->get(storage_policy);
|
||||
auto policy = global_context->getStoragePolicySelector()->get(storage_policy);
|
||||
if (policy->getVolumes().size() != 1)
|
||||
throw Exception("Policy for Distributed table, should have exactly one volume", ErrorCodes::BAD_ARGUMENTS);
|
||||
volume = policy->getVolume(0);
|
||||
@ -628,7 +629,7 @@ StoragePolicyPtr StorageDistributed::getStoragePolicy() const
|
||||
{
|
||||
if (storage_policy.empty())
|
||||
return {};
|
||||
return global_context.getStoragePolicySelector()->get(storage_policy);
|
||||
return global_context->getStoragePolicySelector()->get(storage_policy);
|
||||
}
|
||||
|
||||
void StorageDistributed::createDirectoryMonitors(const std::string & disk)
|
||||
@ -655,7 +656,7 @@ StorageDistributedDirectoryMonitor& StorageDistributed::requireDirectoryMonitor(
|
||||
{
|
||||
node_data.conneciton_pool = StorageDistributedDirectoryMonitor::createPool(name, *this);
|
||||
node_data.directory_monitor = std::make_unique<StorageDistributedDirectoryMonitor>(
|
||||
*this, path, node_data.conneciton_pool, monitors_blocker, global_context.getDistributedSchedulePool());
|
||||
*this, path, node_data.conneciton_pool, monitors_blocker, global_context->getDistributedSchedulePool());
|
||||
}
|
||||
return *node_data.directory_monitor;
|
||||
}
|
||||
@ -672,7 +673,7 @@ std::pair<const std::string &, const std::string &> StorageDistributed::getPath(
|
||||
|
||||
ClusterPtr StorageDistributed::getCluster() const
|
||||
{
|
||||
return owned_cluster ? owned_cluster : global_context.getCluster(cluster_name);
|
||||
return owned_cluster ? owned_cluster : global_context->getCluster(cluster_name);
|
||||
}
|
||||
|
||||
ClusterPtr StorageDistributed::getOptimizedCluster(const Context & context, const ASTPtr & query_ptr) const
|
||||
|
@ -127,7 +127,7 @@ public:
|
||||
String remote_table;
|
||||
ASTPtr remote_table_function_ptr;
|
||||
|
||||
Context global_context;
|
||||
std::unique_ptr<Context> global_context;
|
||||
Logger * log = &Logger::get("StorageDistributed");
|
||||
|
||||
/// Used to implement TableFunctionRemote.
|
||||
|
@ -7,6 +7,8 @@
|
||||
#include <Common/ThreadPool.h>
|
||||
#include <Interpreters/InterpreterAlterQuery.h>
|
||||
#include <Interpreters/PartLog.h>
|
||||
#include <Interpreters/MutationsInterpreter.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Parsers/ASTCheckQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
@ -21,9 +23,8 @@
|
||||
#include <Disks/StoragePolicy.h>
|
||||
#include <Storages/MergeTree/MergeList.h>
|
||||
#include <Storages/MergeTree/checkDataPart.h>
|
||||
#include <optional>
|
||||
#include <Interpreters/MutationsInterpreter.h>
|
||||
#include <Processors/Pipe.h>
|
||||
#include <optional>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -44,6 +44,7 @@
|
||||
|
||||
#include <Interpreters/InterpreterAlterQuery.h>
|
||||
#include <Interpreters/PartLog.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
#include <DataStreams/RemoteBlockInputStream.h>
|
||||
#include <DataStreams/NullBlockOutputStream.h>
|
||||
|
@ -9,6 +9,7 @@
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Interpreters/Set.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Poco/DirectoryIterator.h>
|
||||
|
||||
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Interpreters/InterpreterSelectQuery.h>
|
||||
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
||||
#include <Interpreters/PredicateExpressionsOptimizer.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/getTableExpressions.h>
|
||||
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
|
@ -12,6 +12,7 @@
|
||||
#include <Access/ContextAccess.h>
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <Processors/Sources/NullSource.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <DataStreams/OneBlockInputStream.h>
|
||||
#include <Storages/System/StorageSystemDisks.h>
|
||||
#include <Processors/Sources/SourceFromSingleChunk.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -9,6 +9,7 @@
|
||||
#include <Storages/VirtualColumnUtils.h>
|
||||
#include <Access/ContextAccess.h>
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -14,6 +14,7 @@
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Processors/Sources/SourceFromSingleChunk.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Storages/System/StorageSystemStoragePolicies.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <Processors/Sources/SourceFromSingleChunk.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,11 +1,11 @@
|
||||
#pragma once
|
||||
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class ColumnsDescription;
|
||||
class Context;
|
||||
|
||||
/*
|
||||
* function(source, format, structure) - creates a temporary storage from formated source
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Access/AccessFlags.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <TableFunctions/TableFunctionFile.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include "registerTableFunctions.h"
|
||||
|
||||
namespace DB
|
||||
|
@ -1,11 +1,11 @@
|
||||
#pragma once
|
||||
|
||||
#include <TableFunctions/ITableFunctionFileLike.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class Context;
|
||||
/* file(path, format, structure) - creates a temporary storage from file
|
||||
*
|
||||
*
|
||||
|
@ -5,11 +5,13 @@
|
||||
#if USE_HDFS
|
||||
|
||||
#include <TableFunctions/ITableFunctionFileLike.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Context;
|
||||
|
||||
/* hdfs(name_node_ip:name_node_port, format, structure) - creates a temporary storage from hdfs file
|
||||
*
|
||||
*/
|
||||
@ -27,6 +29,7 @@ private:
|
||||
const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name, const String & compression_method) const override;
|
||||
const char * getStorageTypeName() const override { return "HDFS"; }
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
#endif
|
||||
|
@ -8,7 +8,6 @@
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Storages/StorageInput.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <boost/algorithm/string.hpp>
|
||||
#include "registerTableFunctions.h"
|
||||
|
@ -1,11 +1,13 @@
|
||||
#pragma once
|
||||
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Context;
|
||||
|
||||
/* input(structure) - allows to make INSERT SELECT from incoming stream of data
|
||||
*/
|
||||
class TableFunctionInput : public ITableFunction
|
||||
@ -18,4 +20,5 @@ private:
|
||||
StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override;
|
||||
const char * getStorageTypeName() const override { return "Input"; }
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -1,11 +1,13 @@
|
||||
#pragma once
|
||||
|
||||
#include <TableFunctions/ITableFunctionFileLike.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Context;
|
||||
|
||||
/* url(source, format, structure) - creates a temporary storage from url
|
||||
*/
|
||||
class TableFunctionURL : public ITableFunctionFileLike
|
||||
@ -22,4 +24,5 @@ private:
|
||||
const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name, const String & compression_method) const override;
|
||||
const char * getStorageTypeName() const override { return "URL"; }
|
||||
};
|
||||
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user