2021-04-12 21:42:52 +00:00
|
|
|
#include "Storages/StorageS3Cluster.h"
|
2021-03-19 21:49:18 +00:00
|
|
|
|
2022-09-28 08:45:15 +00:00
|
|
|
#include "config.h"
|
2021-03-24 21:02:21 +00:00
|
|
|
|
|
|
|
#if USE_AWS_S3
|
|
|
|
|
2021-03-22 17:12:31 +00:00
|
|
|
#include "Common/Exception.h"
|
2021-03-19 21:49:18 +00:00
|
|
|
#include "Client/Connection.h"
|
2021-03-22 23:06:10 +00:00
|
|
|
#include "Core/QueryProcessingStage.h"
|
2021-03-22 17:12:31 +00:00
|
|
|
#include <DataTypes/DataTypeString.h>
|
|
|
|
#include <IO/WriteBufferFromS3.h>
|
|
|
|
#include <IO/WriteHelpers.h>
|
2021-03-26 15:33:14 +00:00
|
|
|
#include <Interpreters/Context.h>
|
|
|
|
#include <Interpreters/getHeaderForProcessingStage.h>
|
|
|
|
#include <Interpreters/SelectQueryOptions.h>
|
|
|
|
#include <Interpreters/InterpreterSelectQuery.h>
|
|
|
|
#include <Interpreters/getTableExpressions.h>
|
2021-07-21 16:13:17 +00:00
|
|
|
#include <Processors/Transforms/AddingDefaultsTransform.h>
|
2022-04-24 10:33:48 +00:00
|
|
|
#include <QueryPipeline/narrowPipe.h>
|
2021-10-16 14:03:50 +00:00
|
|
|
#include <QueryPipeline/Pipe.h>
|
2021-04-06 19:18:45 +00:00
|
|
|
#include <Processors/Sources/RemoteSource.h>
|
2021-10-15 20:18:20 +00:00
|
|
|
#include <QueryPipeline/RemoteQueryExecutor.h>
|
2021-03-19 21:49:18 +00:00
|
|
|
#include <Parsers/queryToString.h>
|
2021-03-24 18:36:31 +00:00
|
|
|
#include <Parsers/ASTTablesInSelectQuery.h>
|
2022-09-30 16:59:17 +00:00
|
|
|
#include <Parsers/ASTLiteral.h>
|
2021-03-26 15:33:14 +00:00
|
|
|
#include <Storages/IStorage.h>
|
|
|
|
#include <Storages/SelectQueryInfo.h>
|
2022-05-19 11:18:58 +00:00
|
|
|
#include <Storages/getVirtualsForStorage.h>
|
2022-09-30 16:59:17 +00:00
|
|
|
#include <Storages/StorageDictionary.h>
|
2022-04-27 15:05:45 +00:00
|
|
|
#include <Common/logger_useful.h>
|
2021-03-19 21:49:18 +00:00
|
|
|
|
2021-03-26 15:33:14 +00:00
|
|
|
#include <aws/core/auth/AWSCredentials.h>
|
|
|
|
#include <aws/s3/S3Client.h>
|
|
|
|
#include <aws/s3/model/ListObjectsV2Request.h>
|
2021-03-22 17:12:31 +00:00
|
|
|
|
2021-03-19 21:49:18 +00:00
|
|
|
#include <memory>
|
|
|
|
#include <string>
|
2021-03-22 17:12:31 +00:00
|
|
|
|
2021-03-19 21:49:18 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
2022-09-30 16:59:17 +00:00
|
|
|
|
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int LOGICAL_ERROR;
|
|
|
|
}
|
|
|
|
|
|
|
|
static ASTPtr addColumnsStructureToQuery(const ASTPtr & query, const String & structure)
|
|
|
|
{
|
|
|
|
/// Add argument with table structure to s3Cluster table function in select query.
|
|
|
|
auto result_query = query->clone();
|
|
|
|
ASTExpressionList * expression_list = extractTableFunctionArgumentsFromSelectQuery(result_query);
|
|
|
|
if (!expression_list)
|
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected SELECT query from table function from s3Cluster, got '{}'", queryToString(query));
|
|
|
|
auto structure_literal = std::make_shared<ASTLiteral>(structure);
|
|
|
|
|
|
|
|
if (expression_list->children.size() < 2 || expression_list->children.size() > 5)
|
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected 2 to 5 arguments in s3Cluster table functions, got {}", expression_list->children.size());
|
|
|
|
|
|
|
|
if (expression_list->children.size() == 2 || expression_list->children.size() == 4)
|
|
|
|
{
|
|
|
|
auto format_literal = std::make_shared<ASTLiteral>("auto");
|
|
|
|
expression_list->children.push_back(format_literal);
|
|
|
|
}
|
|
|
|
|
|
|
|
expression_list->children.push_back(structure_literal);
|
|
|
|
return result_query;
|
|
|
|
}
|
|
|
|
|
2021-04-12 21:42:52 +00:00
|
|
|
StorageS3Cluster::StorageS3Cluster(
|
2022-09-13 13:07:43 +00:00
|
|
|
const StorageS3ClusterConfiguration & configuration_,
|
2021-03-22 17:12:31 +00:00
|
|
|
const StorageID & table_id_,
|
|
|
|
const ColumnsDescription & columns_,
|
|
|
|
const ConstraintsDescription & constraints_,
|
2022-09-13 13:07:43 +00:00
|
|
|
ContextPtr context_)
|
2022-08-24 12:17:15 +00:00
|
|
|
: IStorage(table_id_)
|
2022-09-13 13:07:43 +00:00
|
|
|
, s3_configuration{configuration_.url, configuration_.auth_settings, configuration_.rw_settings, configuration_.headers}
|
|
|
|
, filename(configuration_.url)
|
|
|
|
, cluster_name(configuration_.cluster_name)
|
|
|
|
, format_name(configuration_.format)
|
|
|
|
, compression_method(configuration_.compression_method)
|
2021-03-19 21:49:18 +00:00
|
|
|
{
|
2022-03-28 22:46:35 +00:00
|
|
|
context_->getGlobalContext()->getRemoteHostFilter().checkURL(Poco::URI{filename});
|
2021-03-19 21:49:18 +00:00
|
|
|
StorageInMemoryMetadata storage_metadata;
|
2022-04-03 22:33:59 +00:00
|
|
|
StorageS3::updateS3Configuration(context_, s3_configuration);
|
2022-03-28 22:46:35 +00:00
|
|
|
|
|
|
|
if (columns_.empty())
|
|
|
|
{
|
|
|
|
const bool is_key_with_globs = filename.find_first_of("*?{") != std::string::npos;
|
|
|
|
|
|
|
|
/// `distributed_processing` is set to false, because this code is executed on the initiator, so there is no callback set
|
|
|
|
/// for asking for the next tasks.
|
|
|
|
/// `format_settings` is set to std::nullopt, because StorageS3Cluster is used only as table function
|
2022-04-03 22:33:59 +00:00
|
|
|
auto columns = StorageS3::getTableStructureFromDataImpl(format_name, s3_configuration, compression_method,
|
2022-03-28 22:46:35 +00:00
|
|
|
/*distributed_processing_*/false, is_key_with_globs, /*format_settings=*/std::nullopt, context_);
|
|
|
|
storage_metadata.setColumns(columns);
|
2022-09-30 16:59:17 +00:00
|
|
|
need_to_add_structure_to_query = true;
|
2022-03-28 22:46:35 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
storage_metadata.setColumns(columns_);
|
|
|
|
|
2021-03-22 17:12:31 +00:00
|
|
|
storage_metadata.setConstraints(constraints_);
|
2021-03-19 21:49:18 +00:00
|
|
|
setInMemoryMetadata(storage_metadata);
|
2022-05-19 11:18:58 +00:00
|
|
|
|
|
|
|
auto default_virtuals = NamesAndTypesList{
|
|
|
|
{"_path", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())},
|
|
|
|
{"_file", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())}};
|
|
|
|
|
|
|
|
auto columns = storage_metadata.getSampleBlock().getNamesAndTypesList();
|
|
|
|
virtual_columns = getVirtualsForStorage(columns, default_virtuals);
|
|
|
|
for (const auto & column : virtual_columns)
|
|
|
|
virtual_block.insert({column.type->createColumn(), column.type, column.name});
|
2021-03-19 21:49:18 +00:00
|
|
|
}
|
|
|
|
|
2021-04-13 20:17:25 +00:00
|
|
|
/// The code executes on initiator
|
2021-04-12 21:42:52 +00:00
|
|
|
Pipe StorageS3Cluster::read(
|
2021-03-19 21:49:18 +00:00
|
|
|
const Names & column_names,
|
2021-07-09 03:15:41 +00:00
|
|
|
const StorageSnapshotPtr & storage_snapshot,
|
2021-03-19 21:49:18 +00:00
|
|
|
SelectQueryInfo & query_info,
|
2021-04-12 19:35:26 +00:00
|
|
|
ContextPtr context,
|
2021-03-23 17:58:29 +00:00
|
|
|
QueryProcessingStage::Enum processed_stage,
|
2021-04-13 20:17:25 +00:00
|
|
|
size_t /*max_block_size*/,
|
|
|
|
unsigned /*num_streams*/)
|
2021-03-19 21:49:18 +00:00
|
|
|
{
|
2022-04-03 22:33:59 +00:00
|
|
|
StorageS3::updateS3Configuration(context, s3_configuration);
|
2022-08-24 12:17:15 +00:00
|
|
|
|
|
|
|
auto cluster = context->getCluster(cluster_name)->getClusterWithReplicasAsShards(context->getSettingsRef());
|
|
|
|
|
|
|
|
auto iterator = std::make_shared<StorageS3Source::DisclosedGlobIterator>(
|
|
|
|
*s3_configuration.client, s3_configuration.uri, query_info.query, virtual_block, context);
|
|
|
|
auto callback = std::make_shared<StorageS3Source::IteratorWrapper>([iterator]() mutable -> String { return iterator->next(); });
|
2021-03-19 21:49:18 +00:00
|
|
|
|
2021-03-26 15:33:14 +00:00
|
|
|
/// Calculate the header. This is significant, because some columns could be thrown away in some cases like query with count(*)
|
2021-03-23 17:58:29 +00:00
|
|
|
Block header =
|
2021-04-06 11:51:16 +00:00
|
|
|
InterpreterSelectQuery(query_info.query, context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock();
|
2021-03-23 17:58:29 +00:00
|
|
|
|
2021-04-12 19:35:26 +00:00
|
|
|
const Scalars & scalars = context->hasQueryContext() ? context->getQueryContext()->getScalars() : Scalars{};
|
2021-03-19 21:49:18 +00:00
|
|
|
|
2021-03-24 18:36:31 +00:00
|
|
|
Pipes pipes;
|
|
|
|
|
2021-04-13 10:59:02 +00:00
|
|
|
const bool add_agg_info = processed_stage == QueryProcessingStage::WithMergeableState;
|
|
|
|
|
2022-09-30 16:59:17 +00:00
|
|
|
ASTPtr query_to_send = query_info.original_query;
|
|
|
|
if (need_to_add_structure_to_query)
|
|
|
|
query_to_send = addColumnsStructureToQuery(
|
|
|
|
query_to_send, StorageDictionary::generateNamesAndTypesDescription(storage_snapshot->metadata->getColumns().getAll()));
|
|
|
|
|
2021-04-12 17:07:01 +00:00
|
|
|
for (const auto & replicas : cluster->getShardsAddresses())
|
|
|
|
{
|
2021-03-19 21:49:18 +00:00
|
|
|
/// There will be only one replica, because we consider each replica as a shard
|
|
|
|
for (const auto & node : replicas)
|
|
|
|
{
|
2021-07-23 07:40:03 +00:00
|
|
|
auto connection = std::make_shared<Connection>(
|
2021-04-12 19:35:26 +00:00
|
|
|
node.host_name, node.port, context->getGlobalContext()->getCurrentDatabase(),
|
2022-08-03 19:44:08 +00:00
|
|
|
node.user, node.password, node.quota_key, node.cluster, node.cluster_secret,
|
2021-04-12 21:42:52 +00:00
|
|
|
"S3ClusterInititiator",
|
2021-04-07 14:43:34 +00:00
|
|
|
node.compression,
|
|
|
|
node.secure
|
2021-07-23 07:40:03 +00:00
|
|
|
);
|
|
|
|
|
2021-04-06 19:18:45 +00:00
|
|
|
|
2021-04-08 14:22:19 +00:00
|
|
|
/// For unknown reason global context is passed to IStorage::read() method
|
|
|
|
/// So, task_identifier is passed as constructor argument. It is more obvious.
|
2021-04-06 19:18:45 +00:00
|
|
|
auto remote_query_executor = std::make_shared<RemoteQueryExecutor>(
|
2021-07-23 07:40:03 +00:00
|
|
|
connection,
|
2022-09-30 16:59:17 +00:00
|
|
|
queryToString(query_to_send),
|
2021-07-14 13:17:30 +00:00
|
|
|
header,
|
|
|
|
context,
|
|
|
|
/*throttler=*/nullptr,
|
|
|
|
scalars,
|
|
|
|
Tables(),
|
|
|
|
processed_stage,
|
2021-12-09 10:39:28 +00:00
|
|
|
RemoteQueryExecutor::Extension{.task_iterator = callback});
|
2021-04-06 19:18:45 +00:00
|
|
|
|
2021-04-13 10:59:02 +00:00
|
|
|
pipes.emplace_back(std::make_shared<RemoteSource>(remote_query_executor, add_agg_info, false));
|
2021-03-19 21:49:18 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-07-09 03:15:41 +00:00
|
|
|
storage_snapshot->check(column_names);
|
2021-03-19 21:49:18 +00:00
|
|
|
return Pipe::unitePipes(std::move(pipes));
|
|
|
|
}
|
2021-04-07 14:43:34 +00:00
|
|
|
|
2021-04-12 21:42:52 +00:00
|
|
|
QueryProcessingStage::Enum StorageS3Cluster::getQueryProcessingStage(
|
2021-07-09 03:15:41 +00:00
|
|
|
ContextPtr context, QueryProcessingStage::Enum to_stage, const StorageSnapshotPtr &, SelectQueryInfo &) const
|
2021-04-07 14:43:34 +00:00
|
|
|
{
|
|
|
|
/// Initiator executes query on remote node.
|
2021-04-12 19:35:26 +00:00
|
|
|
if (context->getClientInfo().query_kind == ClientInfo::QueryKind::INITIAL_QUERY)
|
2021-04-12 17:48:16 +00:00
|
|
|
if (to_stage >= QueryProcessingStage::Enum::WithMergeableState)
|
|
|
|
return QueryProcessingStage::Enum::WithMergeableState;
|
2021-04-12 17:07:01 +00:00
|
|
|
|
2021-04-07 14:43:34 +00:00
|
|
|
/// Follower just reads the data.
|
|
|
|
return QueryProcessingStage::Enum::FetchColumns;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2021-04-12 21:42:52 +00:00
|
|
|
NamesAndTypesList StorageS3Cluster::getVirtuals() const
|
2021-04-12 17:33:55 +00:00
|
|
|
{
|
2022-05-19 11:18:58 +00:00
|
|
|
return virtual_columns;
|
2021-04-12 17:33:55 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2021-03-19 21:49:18 +00:00
|
|
|
}
|
|
|
|
|
2021-03-24 21:02:21 +00:00
|
|
|
#endif
|