mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Replace table function engines with their -Cluster alternatives
This commit is contained in:
parent
5445f55e62
commit
31eddf6073
@ -5643,6 +5643,9 @@ Parts virtually divided into segments to be distributed between replicas for par
|
||||
DECLARE(Bool, parallel_replicas_local_plan, true, R"(
|
||||
Build local plan for local replica
|
||||
)", BETA) \
|
||||
DECLARE(Bool, parallel_replicas_for_cluster_engines, true, R"(
|
||||
Replace table function engines with their -Cluster alternatives
|
||||
)", EXPERIMENTAL) \
|
||||
\
|
||||
DECLARE(Bool, allow_experimental_analyzer, true, R"(
|
||||
Allow new query analyzer.
|
||||
|
@ -60,6 +60,7 @@ static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory
|
||||
{
|
||||
{"24.12",
|
||||
{
|
||||
{"parallel_replicas_for_cluster_engines", true, true, "New setting"},
|
||||
}
|
||||
},
|
||||
{"24.11",
|
||||
|
@ -66,12 +66,14 @@ StorageFileCluster::StorageFileCluster(
|
||||
|
||||
void StorageFileCluster::updateQueryToSendIfNeeded(DB::ASTPtr & query, const StorageSnapshotPtr & storage_snapshot, const DB::ContextPtr & context)
|
||||
{
|
||||
ASTExpressionList * expression_list = extractTableFunctionArgumentsFromSelectQuery(query);
|
||||
if (!expression_list)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected SELECT query from table function fileCluster, got '{}'", queryToString(query));
|
||||
auto * table_function = extractTableFunctionFromSelectQuery(query);
|
||||
|
||||
TableFunctionFileCluster::updateStructureAndFormatArgumentsIfNeeded(
|
||||
expression_list->children, storage_snapshot->metadata->getColumns().getAll().toNamesAndTypesDescription(), format_name, context);
|
||||
table_function,
|
||||
storage_snapshot->metadata->getColumns().getAll().toNamesAndTypesDescription(),
|
||||
format_name,
|
||||
context
|
||||
);
|
||||
}
|
||||
|
||||
RemoteQueryExecutor::Extension StorageFileCluster::getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context) const
|
||||
|
@ -86,12 +86,14 @@ StorageURLCluster::StorageURLCluster(
|
||||
|
||||
void StorageURLCluster::updateQueryToSendIfNeeded(ASTPtr & query, const StorageSnapshotPtr & storage_snapshot, const ContextPtr & context)
|
||||
{
|
||||
ASTExpressionList * expression_list = extractTableFunctionArgumentsFromSelectQuery(query);
|
||||
if (!expression_list)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected SELECT query from table function urlCluster, got '{}'", queryToString(query));
|
||||
auto * table_function = extractTableFunctionFromSelectQuery(query);
|
||||
|
||||
TableFunctionURLCluster::updateStructureAndFormatArgumentsIfNeeded(
|
||||
expression_list->children, storage_snapshot->metadata->getColumns().getAll().toNamesAndTypesDescription(), format_name, context);
|
||||
table_function,
|
||||
storage_snapshot->metadata->getColumns().getAll().toNamesAndTypesDescription(),
|
||||
format_name,
|
||||
context
|
||||
);
|
||||
}
|
||||
|
||||
RemoteQueryExecutor::Extension StorageURLCluster::getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context) const
|
||||
|
@ -1,7 +1,6 @@
|
||||
#include <Storages/extractTableFunctionArgumentsFromSelectQuery.h>
|
||||
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
@ -26,4 +25,18 @@ ASTExpressionList * extractTableFunctionArgumentsFromSelectQuery(ASTPtr & query)
|
||||
return table_function->arguments->as<ASTExpressionList>();
|
||||
}
|
||||
|
||||
ASTFunction * extractTableFunctionFromSelectQuery(ASTPtr & query)
|
||||
{
|
||||
auto * select_query = query->as<ASTSelectQuery>();
|
||||
if (!select_query || !select_query->tables())
|
||||
return nullptr;
|
||||
|
||||
auto * tables = select_query->tables()->as<ASTTablesInSelectQuery>();
|
||||
auto * table_expression = tables->children[0]->as<ASTTablesInSelectQueryElement>()->table_expression->as<ASTTableExpression>();
|
||||
if (!table_expression->table_function)
|
||||
return nullptr;
|
||||
|
||||
return table_expression->table_function->as<ASTFunction>();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -2,10 +2,12 @@
|
||||
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
ASTExpressionList * extractTableFunctionArgumentsFromSelectQuery(ASTPtr & query);
|
||||
ASTFunction * extractTableFunctionFromSelectQuery(ASTPtr & query);
|
||||
|
||||
}
|
||||
|
@ -2,10 +2,13 @@
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Storages/checkAndGetLiteralArgument.h>
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <TableFunctions/TableFunctionObjectStorage.h>
|
||||
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -24,15 +27,23 @@ class ITableFunctionCluster : public Base
|
||||
public:
|
||||
String getName() const override = 0;
|
||||
|
||||
static void updateStructureAndFormatArgumentsIfNeeded(ASTs & args, const String & structure_, const String & format_, const ContextPtr & context)
|
||||
static void updateStructureAndFormatArgumentsIfNeeded(ASTFunction * table_function, const String & structure_, const String & format_, const ContextPtr & context)
|
||||
{
|
||||
auto * expression_list = table_function->arguments->as<ASTExpressionList>();
|
||||
ASTs args = expression_list->children;
|
||||
|
||||
if (args.empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected empty list of arguments for {}Cluster table function", Base::name);
|
||||
|
||||
ASTPtr cluster_name_arg = args.front();
|
||||
args.erase(args.begin());
|
||||
Base::updateStructureAndFormatArgumentsIfNeeded(args, structure_, format_, context);
|
||||
args.insert(args.begin(), cluster_name_arg);
|
||||
if (table_function-> name == Base::name)
|
||||
Base::updateStructureAndFormatArgumentsIfNeeded(args, structure_, format_, context);
|
||||
else
|
||||
{
|
||||
ASTPtr cluster_name_arg = args.front();
|
||||
args.erase(args.begin());
|
||||
Base::updateStructureAndFormatArgumentsIfNeeded(args, structure_, format_, context);
|
||||
args.insert(args.begin(), cluster_name_arg);
|
||||
}
|
||||
}
|
||||
|
||||
protected:
|
||||
|
@ -1,5 +1,8 @@
|
||||
#include "config.h"
|
||||
|
||||
#include <Core/Settings.h>
|
||||
#include <Core/SettingsEnums.h>
|
||||
|
||||
#include <Access/Common/AccessFlags.h>
|
||||
#include <Analyzer/FunctionNode.h>
|
||||
#include <Analyzer/TableFunctionNode.h>
|
||||
@ -19,11 +22,20 @@
|
||||
#include <Storages/ObjectStorage/Local/Configuration.h>
|
||||
#include <Storages/ObjectStorage/S3/Configuration.h>
|
||||
#include <Storages/ObjectStorage/StorageObjectStorage.h>
|
||||
#include <Storages/ObjectStorage/StorageObjectStorageCluster.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace Setting
|
||||
{
|
||||
extern const SettingsBool use_parallel_replicas;
|
||||
extern const SettingsBool parallel_replicas_for_cluster_engines;
|
||||
extern const SettingsString cluster_for_parallel_replicas;
|
||||
extern const SettingsParallelReplicasMode parallel_replicas_mode;
|
||||
}
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
@ -100,8 +112,9 @@ StoragePtr TableFunctionObjectStorage<Definition, Configuration>::executeImpl(
|
||||
ColumnsDescription cached_columns,
|
||||
bool is_insert_query) const
|
||||
{
|
||||
ColumnsDescription columns;
|
||||
chassert(configuration);
|
||||
ColumnsDescription columns;
|
||||
|
||||
if (configuration->structure != "auto")
|
||||
columns = parseColumnsListFromString(configuration->structure, context);
|
||||
else if (!structure_hint.empty())
|
||||
@ -109,18 +122,61 @@ StoragePtr TableFunctionObjectStorage<Definition, Configuration>::executeImpl(
|
||||
else if (!cached_columns.empty())
|
||||
columns = cached_columns;
|
||||
|
||||
StoragePtr storage = std::make_shared<StorageObjectStorage>(
|
||||
StoragePtr storage;
|
||||
|
||||
if (context->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY)
|
||||
{
|
||||
storage = std::make_shared<StorageObjectStorage>(
|
||||
configuration,
|
||||
getObjectStorage(context, !is_insert_query),
|
||||
context,
|
||||
StorageID(getDatabaseName(), table_name),
|
||||
columns,
|
||||
ConstraintsDescription{},
|
||||
/* comment */ String{},
|
||||
/* format_settings */ std::nullopt,
|
||||
/* mode */ LoadingStrictnessLevel::CREATE,
|
||||
/* distributed_processing */ true,
|
||||
/* partition_by */ nullptr);
|
||||
|
||||
storage->startup();
|
||||
return storage;
|
||||
}
|
||||
|
||||
const auto & settings = context->getSettingsRef();
|
||||
auto parallel_replicas_cluster_name = settings[Setting::cluster_for_parallel_replicas].toString();
|
||||
auto can_use_parallel_replicas = settings[Setting::use_parallel_replicas]
|
||||
&& settings[Setting::parallel_replicas_for_cluster_engines]
|
||||
&& settings[Setting::parallel_replicas_mode] == ParallelReplicasMode::READ_TASKS
|
||||
&& !parallel_replicas_cluster_name.empty();
|
||||
|
||||
if (can_use_parallel_replicas)
|
||||
{
|
||||
storage = std::make_shared<StorageObjectStorageCluster>(
|
||||
parallel_replicas_cluster_name,
|
||||
configuration,
|
||||
getObjectStorage(context, !is_insert_query),
|
||||
StorageID(getDatabaseName(), table_name),
|
||||
columns,
|
||||
ConstraintsDescription{},
|
||||
context);
|
||||
|
||||
storage->startup();
|
||||
return storage;
|
||||
}
|
||||
|
||||
storage = std::make_shared<StorageObjectStorage>(
|
||||
configuration,
|
||||
getObjectStorage(context, !is_insert_query),
|
||||
context,
|
||||
StorageID(getDatabaseName(), table_name),
|
||||
columns,
|
||||
ConstraintsDescription{},
|
||||
String{},
|
||||
/* comment */ String{},
|
||||
/* format_settings */ std::nullopt,
|
||||
/* mode */ LoadingStrictnessLevel::CREATE,
|
||||
/* distributed_processing */ false,
|
||||
nullptr);
|
||||
/* partition_by */ nullptr);
|
||||
|
||||
storage->startup();
|
||||
return storage;
|
||||
|
@ -6,8 +6,10 @@
|
||||
#include <Storages/ObjectStorage/StorageObjectStorage.h>
|
||||
#include <Storages/VirtualColumnUtils.h>
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
|
||||
#include "config.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -2,22 +2,34 @@
|
||||
|
||||
#include "registerTableFunctions.h"
|
||||
#include <Access/Common/AccessFlags.h>
|
||||
#include <Analyzer/FunctionNode.h>
|
||||
#include <Analyzer/TableFunctionNode.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Formats/FormatFactory.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Interpreters/parseColumnsListForTableFunction.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Storages/ColumnsDescription.h>
|
||||
#include <Storages/NamedCollectionsHelpers.h>
|
||||
#include <Storages/StorageURLCluster.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <Analyzer/FunctionNode.h>
|
||||
#include <Analyzer/TableFunctionNode.h>
|
||||
#include <Interpreters/parseColumnsListForTableFunction.h>
|
||||
#include <Formats/FormatFactory.h>
|
||||
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/WriteBufferFromVector.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace Setting
|
||||
{
|
||||
extern const SettingsBool use_parallel_replicas;
|
||||
extern const SettingsBool parallel_replicas_for_cluster_engines;
|
||||
extern const SettingsString cluster_for_parallel_replicas;
|
||||
extern const SettingsParallelReplicasMode parallel_replicas_mode;
|
||||
}
|
||||
|
||||
std::vector<size_t> TableFunctionURL::skipAnalysisForArguments(const QueryTreeNodePtr & query_node_table_function, ContextPtr) const
|
||||
{
|
||||
auto & table_function_node = query_node_table_function->as<TableFunctionNode &>();
|
||||
@ -120,6 +132,46 @@ StoragePtr TableFunctionURL::getStorage(
|
||||
const String & source, const String & format_, const ColumnsDescription & columns, ContextPtr global_context,
|
||||
const std::string & table_name, const String & compression_method_) const
|
||||
{
|
||||
if (global_context->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY)
|
||||
{
|
||||
return std::make_shared<StorageURL>(
|
||||
source,
|
||||
StorageID(getDatabaseName(), table_name),
|
||||
format_,
|
||||
std::nullopt /*format settings*/,
|
||||
columns,
|
||||
ConstraintsDescription{},
|
||||
String{},
|
||||
global_context,
|
||||
compression_method_,
|
||||
configuration.headers,
|
||||
configuration.http_method,
|
||||
nullptr,
|
||||
/*distributed_processing*/ true);
|
||||
}
|
||||
|
||||
const auto & settings = global_context->getSettingsRef();
|
||||
auto parallel_replicas_cluster_name = settings[Setting::cluster_for_parallel_replicas].toString();
|
||||
auto can_use_parallel_replicas = settings[Setting::use_parallel_replicas]
|
||||
&& settings[Setting::parallel_replicas_for_cluster_engines]
|
||||
&& settings[Setting::parallel_replicas_mode] == ParallelReplicasMode::READ_TASKS
|
||||
&& !parallel_replicas_cluster_name.empty();
|
||||
|
||||
if (can_use_parallel_replicas)
|
||||
{
|
||||
LOG_DEBUG(&Poco::Logger::get("TableFunctionURL"), "TableFunctionURL::getStorage wrapped to cluster version");
|
||||
return std::make_shared<StorageURLCluster>(
|
||||
global_context,
|
||||
settings[Setting::cluster_for_parallel_replicas],
|
||||
filename,
|
||||
format,
|
||||
compression_method,
|
||||
StorageID(getDatabaseName(), table_name),
|
||||
getActualTableStructure(global_context, /* is_insert_query */ true),
|
||||
ConstraintsDescription{},
|
||||
configuration);
|
||||
}
|
||||
|
||||
return std::make_shared<StorageURL>(
|
||||
source,
|
||||
StorageID(getDatabaseName(), table_name),
|
||||
|
@ -10,11 +10,10 @@ StoragePtr TableFunctionURLCluster::getStorage(
|
||||
const String & /*source*/, const String & /*format_*/, const ColumnsDescription & columns, ContextPtr context,
|
||||
const std::string & table_name, const String & /*compression_method_*/) const
|
||||
{
|
||||
StoragePtr storage;
|
||||
if (context->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY)
|
||||
{
|
||||
//On worker node this uri won't contain globs
|
||||
storage = std::make_shared<StorageURL>(
|
||||
return std::make_shared<StorageURL>(
|
||||
filename,
|
||||
StorageID(getDatabaseName(), table_name),
|
||||
format,
|
||||
@ -29,20 +28,17 @@ StoragePtr TableFunctionURLCluster::getStorage(
|
||||
nullptr,
|
||||
/*distributed_processing=*/ true);
|
||||
}
|
||||
else
|
||||
{
|
||||
storage = std::make_shared<StorageURLCluster>(
|
||||
context,
|
||||
cluster_name,
|
||||
filename,
|
||||
format,
|
||||
compression_method,
|
||||
StorageID(getDatabaseName(), table_name),
|
||||
getActualTableStructure(context, /* is_insert_query */ true),
|
||||
ConstraintsDescription{},
|
||||
configuration);
|
||||
}
|
||||
return storage;
|
||||
|
||||
return std::make_shared<StorageURLCluster>(
|
||||
context,
|
||||
cluster_name,
|
||||
filename,
|
||||
format,
|
||||
compression_method,
|
||||
StorageID(getDatabaseName(), table_name),
|
||||
getActualTableStructure(context, /* is_insert_query */ true),
|
||||
ConstraintsDescription{},
|
||||
configuration);
|
||||
}
|
||||
|
||||
void registerTableFunctionURLCluster(TableFunctionFactory & factory)
|
||||
|
Loading…
Reference in New Issue
Block a user