Replace table function engines with their -Cluster alternatives

This commit is contained in:
Konstantin Bogdanov 2024-11-13 16:13:14 +01:00
parent 5445f55e62
commit 31eddf6073
No known key found for this signature in database
11 changed files with 178 additions and 38 deletions

View File

@ -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.

View File

@ -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",

View File

@ -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

View File

@ -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

View File

@ -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>();
}
}

View File

@ -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);
}

View File

@ -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:

View File

@ -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;

View File

@ -6,8 +6,10 @@
#include <Storages/ObjectStorage/StorageObjectStorage.h>
#include <Storages/VirtualColumnUtils.h>
#include <TableFunctions/ITableFunction.h>
#include "config.h"
namespace DB
{

View File

@ -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),

View File

@ -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)