ClickHouse/src/TableFunctions/TableFunctionS3Cluster.cpp

160 lines
5.5 KiB
C++
Raw Normal View History

2021-04-12 17:33:55 +00:00
#if !defined(ARCADIA_BUILD)
2021-03-16 18:41:29 +00:00
#include <Common/config.h>
2021-04-12 17:33:55 +00:00
#endif
2021-03-16 18:41:29 +00:00
#if USE_AWS_S3
2021-04-12 21:42:52 +00:00
#include <Storages/StorageS3Cluster.h>
2021-04-06 19:18:45 +00:00
#include <DataTypes/DataTypeString.h>
2021-04-12 17:07:01 +00:00
#include <DataStreams/RemoteBlockInputStream.h>
2021-03-16 18:41:29 +00:00
#include <IO/S3Common.h>
#include <Storages/StorageS3.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Interpreters/Context.h>
2021-04-12 17:07:01 +00:00
#include <Interpreters/ClientInfo.h>
2021-03-16 18:41:29 +00:00
#include <TableFunctions/TableFunctionFactory.h>
#include <TableFunctions/TableFunctionS3.h>
2021-04-12 21:42:52 +00:00
#include <TableFunctions/TableFunctionS3Cluster.h>
2021-03-16 18:41:29 +00:00
#include <TableFunctions/parseColumnsListForTableFunction.h>
#include <Parsers/ASTLiteral.h>
2021-04-12 17:07:01 +00:00
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/IAST_fwd.h>
#include <Processors/Sources/SourceFromInputStream.h>
2021-04-12 17:33:55 +00:00
#include "registerTableFunctions.h"
2021-04-12 17:07:01 +00:00
#include <memory>
#include <thread>
2021-03-16 18:41:29 +00:00
2021-03-16 18:41:29 +00:00
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
2021-04-06 19:18:45 +00:00
2021-04-12 21:42:52 +00:00
void TableFunctionS3Cluster::parseArguments(const ASTPtr & ast_function, ContextPtr context)
2021-03-16 18:41:29 +00:00
{
/// Parse args
ASTs & args_func = ast_function->children;
if (args_func.size() != 1)
2021-04-12 17:33:55 +00:00
throw Exception("Table function '" + getName() + "' must have arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
2021-03-16 18:41:29 +00:00
ASTs & args = args_func.at(0)->children;
2021-04-06 19:18:45 +00:00
const auto message = fmt::format(
"The signature of table function {} could be the following:\n" \
" - cluster, url, format, structure\n" \
" - cluster, url, format, structure, compression_method\n" \
" - cluster, url, access_key_id, secret_access_key, format, structure\n" \
" - cluster, url, access_key_id, secret_access_key, format, structure, compression_method",
getName());
2021-03-19 21:49:18 +00:00
if (args.size() < 4 || args.size() > 7)
2021-04-06 19:18:45 +00:00
throw Exception(message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
2021-03-16 18:41:29 +00:00
for (auto & arg : args)
arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context);
/// This arguments are always the first
2021-03-19 21:49:18 +00:00
cluster_name = args[0]->as<ASTLiteral &>().value.safeGet<String>();
2021-04-06 19:18:45 +00:00
filename = args[1]->as<ASTLiteral &>().value.safeGet<String>();
2021-03-16 18:41:29 +00:00
/// Size -> argument indexes
static auto size_to_args = std::map<size_t, std::map<String, size_t>>
2021-04-06 19:18:45 +00:00
{
{4, {{"format", 2}, {"structure", 3}}},
{5, {{"format", 2}, {"structure", 3}, {"compression_method", 4}}},
{6, {{"access_key_id", 2}, {"secret_access_key", 3}, {"format", 4}, {"structure", 5}}},
{7, {{"access_key_id", 2}, {"secret_access_key", 3}, {"format", 4}, {"structure", 5}, {"compression_method", 6}}}
};
auto & args_to_idx = size_to_args[args.size()];
if (args_to_idx.contains("format"))
format = args[args_to_idx["format"]]->as<ASTLiteral &>().value.safeGet<String>();
if (args_to_idx.contains("structure"))
structure = args[args_to_idx["structure"]]->as<ASTLiteral &>().value.safeGet<String>();
if (args_to_idx.contains("compression_method"))
compression_method = args[args_to_idx["compression_method"]]->as<ASTLiteral &>().value.safeGet<String>();
if (args_to_idx.contains("access_key_id"))
access_key_id = args[args_to_idx["access_key_id"]]->as<ASTLiteral &>().value.safeGet<String>();
if (args_to_idx.contains("secret_access_key"))
secret_access_key = args[args_to_idx["secret_access_key"]]->as<ASTLiteral &>().value.safeGet<String>();
2021-03-16 18:41:29 +00:00
}
2021-04-12 21:42:52 +00:00
ColumnsDescription TableFunctionS3Cluster::getActualTableStructure(ContextPtr context) const
2021-03-16 18:41:29 +00:00
{
return parseColumnsListFromString(structure, context);
}
2021-04-12 21:42:52 +00:00
StoragePtr TableFunctionS3Cluster::executeImpl(
2021-04-12 19:35:26 +00:00
const ASTPtr & /*function*/, ContextPtr context,
2021-03-19 21:49:18 +00:00
const std::string & table_name, ColumnsDescription /*cached_columns*/) const
2021-03-16 18:41:29 +00:00
{
2021-04-13 20:17:25 +00:00
StoragePtr storage;
if (context->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY)
{
/// On worker node this filename won't contains globs
Poco::URI uri (filename);
S3::URI s3_uri (uri);
/// Actually this parameters are not used
2021-05-19 21:42:25 +00:00
UInt64 max_single_read_retries = context->getSettingsRef().s3_max_single_read_retries;
2021-04-13 20:17:25 +00:00
UInt64 min_upload_part_size = context->getSettingsRef().s3_min_upload_part_size;
UInt64 max_single_part_upload_size = context->getSettingsRef().s3_max_single_part_upload_size;
UInt64 max_connections = context->getSettingsRef().s3_max_connections;
storage = StorageS3::create(
2021-04-23 12:18:23 +00:00
s3_uri,
access_key_id,
secret_access_key,
StorageID(getDatabaseName(), table_name),
format,
2021-05-19 21:42:25 +00:00
max_single_read_retries,
2021-04-23 12:18:23 +00:00
min_upload_part_size,
max_single_part_upload_size,
max_connections,
getActualTableStructure(context),
ConstraintsDescription{},
String{},
context,
2021-08-23 19:05:28 +00:00
// No format_settings for S3Cluster, but maybe we could get them from current context ?
std::nullopt,
2021-04-23 12:18:23 +00:00
compression_method,
/*distributed_processing=*/true);
2021-04-13 20:17:25 +00:00
}
2021-04-13 20:19:04 +00:00
else
{
2021-04-13 20:17:25 +00:00
storage = StorageS3Cluster::create(
filename, access_key_id, secret_access_key, StorageID(getDatabaseName(), table_name),
cluster_name, format, context->getSettingsRef().s3_max_connections,
getActualTableStructure(context), ConstraintsDescription{},
context, compression_method);
}
2021-03-16 18:41:29 +00:00
storage->startup();
return storage;
}
2021-04-12 21:42:52 +00:00
void registerTableFunctionS3Cluster(TableFunctionFactory & factory)
2021-03-16 18:41:29 +00:00
{
2021-04-12 21:42:52 +00:00
factory.registerFunction<TableFunctionS3Cluster>();
2021-03-16 18:41:29 +00:00
}
2021-04-06 19:18:45 +00:00
2021-03-16 18:41:29 +00:00
}
#endif