2021-03-19 21:49:18 +00:00
|
|
|
#include "Storages/StorageS3Distributed.h"
|
|
|
|
|
2021-03-24 21:02:21 +00:00
|
|
|
#include <Common/config.h>
|
2021-03-26 15:33:14 +00:00
|
|
|
#include "Processors/Sources/SourceWithProgress.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 <Common/Throttler.h>
|
|
|
|
#include "Client/Connection.h"
|
2021-03-22 23:06:10 +00:00
|
|
|
#include "Core/QueryProcessingStage.h"
|
2021-03-19 21:49:18 +00:00
|
|
|
#include "DataStreams/RemoteBlockInputStream.h"
|
|
|
|
#include <Columns/ColumnsNumber.h>
|
|
|
|
#include <DataTypes/DataTypesNumber.h>
|
2021-03-22 17:12:31 +00:00
|
|
|
#include <DataTypes/DataTypeString.h>
|
|
|
|
#include <IO/ReadBufferFromS3.h>
|
|
|
|
#include <IO/ReadHelpers.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-03-22 17:12:31 +00:00
|
|
|
#include <Formats/FormatFactory.h>
|
|
|
|
#include <DataStreams/IBlockOutputStream.h>
|
|
|
|
#include <DataStreams/AddingDefaultsBlockInputStream.h>
|
|
|
|
#include <DataStreams/narrowBlockInputStreams.h>
|
|
|
|
#include <Processors/Formats/InputStreamFromInputFormat.h>
|
2021-03-19 21:49:18 +00:00
|
|
|
#include <Processors/Pipe.h>
|
|
|
|
#include <Processors/Sources/SourceFromInputStream.h>
|
|
|
|
#include <Parsers/queryToString.h>
|
2021-03-24 18:36:31 +00:00
|
|
|
#include <Parsers/ASTTablesInSelectQuery.h>
|
2021-03-26 15:33:14 +00:00
|
|
|
#include <Storages/IStorage.h>
|
|
|
|
#include <Storages/SelectQueryInfo.h>
|
|
|
|
#include <Storages/StorageS3.h>
|
|
|
|
#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-23 17:58:29 +00:00
|
|
|
#include <ios>
|
2021-03-19 21:49:18 +00:00
|
|
|
#include <memory>
|
|
|
|
#include <string>
|
|
|
|
#include <thread>
|
2021-03-22 17:12:31 +00:00
|
|
|
#include <cassert>
|
|
|
|
|
2021-03-19 21:49:18 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
|
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int LOGICAL_ERROR;
|
|
|
|
}
|
|
|
|
|
2021-03-24 18:36:31 +00:00
|
|
|
struct StorageS3SourceBuilder
|
|
|
|
{
|
|
|
|
bool need_path;
|
|
|
|
bool need_file;
|
|
|
|
String format;
|
|
|
|
String name;
|
|
|
|
Block sample_block;
|
|
|
|
const Context & context;
|
|
|
|
const ColumnsDescription & columns;
|
|
|
|
UInt64 max_block_size;
|
|
|
|
String compression_method;
|
|
|
|
};
|
|
|
|
|
2021-03-22 17:12:31 +00:00
|
|
|
class StorageS3SequentialSource : public SourceWithProgress
|
|
|
|
{
|
|
|
|
public:
|
|
|
|
|
|
|
|
static Block getHeader(Block sample_block, bool with_path_column, bool with_file_column)
|
|
|
|
{
|
|
|
|
if (with_path_column)
|
|
|
|
sample_block.insert({DataTypeString().createColumn(), std::make_shared<DataTypeString>(), "_path"});
|
|
|
|
if (with_file_column)
|
|
|
|
sample_block.insert({DataTypeString().createColumn(), std::make_shared<DataTypeString>(), "_file"});
|
|
|
|
|
|
|
|
return sample_block;
|
|
|
|
}
|
|
|
|
|
|
|
|
StorageS3SequentialSource(
|
|
|
|
String initial_query_id_,
|
2021-03-24 18:36:31 +00:00
|
|
|
Cluster::Address initiator,
|
|
|
|
const ClientAuthentificationBuilder & client_auth_builder_,
|
|
|
|
const StorageS3SourceBuilder & s3_builder_)
|
|
|
|
: SourceWithProgress(getHeader(s3_builder_.sample_block, s3_builder_.need_path, s3_builder_.need_file))
|
2021-03-22 17:12:31 +00:00
|
|
|
, initial_query_id(initial_query_id_)
|
2021-03-24 18:36:31 +00:00
|
|
|
, s3_source_builder(s3_builder_)
|
|
|
|
, cli_builder(client_auth_builder_)
|
2021-03-22 17:12:31 +00:00
|
|
|
{
|
2021-03-24 18:36:31 +00:00
|
|
|
connections = std::make_shared<ConnectionPool>(
|
|
|
|
/*max_connections*/3,
|
|
|
|
/*host*/initiator.host_name,
|
|
|
|
/*port*/initiator.port,
|
|
|
|
/*default_database=*/s3_builder_.context.getGlobalContext().getCurrentDatabase(),
|
|
|
|
/*user=*/s3_builder_.context.getClientInfo().initial_user,
|
|
|
|
/*password=*/initiator.password,
|
|
|
|
/*cluster=*/initiator.cluster,
|
|
|
|
/*cluster_secret=*/initiator.cluster_secret
|
2021-03-22 17:12:31 +00:00
|
|
|
);
|
|
|
|
|
|
|
|
createOrUpdateInnerSource();
|
|
|
|
}
|
|
|
|
|
|
|
|
String getName() const override
|
|
|
|
{
|
2021-03-24 18:36:31 +00:00
|
|
|
return "StorageS3SequentialSource";
|
2021-03-22 17:12:31 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
Chunk generate() override
|
|
|
|
{
|
2021-03-23 17:58:29 +00:00
|
|
|
if (!inner)
|
2021-03-22 17:12:31 +00:00
|
|
|
return {};
|
2021-03-23 17:58:29 +00:00
|
|
|
|
|
|
|
auto chunk = inner->generate();
|
|
|
|
if (!chunk)
|
|
|
|
{
|
|
|
|
if (!createOrUpdateInnerSource())
|
|
|
|
return {};
|
|
|
|
else
|
|
|
|
chunk = inner->generate();
|
|
|
|
}
|
|
|
|
return chunk;
|
2021-03-22 17:12:31 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
private:
|
|
|
|
|
|
|
|
String askAboutNextKey()
|
|
|
|
{
|
|
|
|
try
|
|
|
|
{
|
2021-03-24 18:36:31 +00:00
|
|
|
auto connection = connections->get(timeouts);
|
|
|
|
connection->sendNextTaskRequest(initial_query_id);
|
|
|
|
auto packet = connection->receivePacket();
|
2021-03-22 17:12:31 +00:00
|
|
|
assert(packet.type = Protocol::Server::NextTaskReply);
|
2021-03-23 17:58:29 +00:00
|
|
|
LOG_TRACE(&Poco::Logger::get("StorageS3SequentialSource"), "Got new task {}", packet.next_task);
|
2021-03-22 17:12:31 +00:00
|
|
|
return packet.next_task;
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
tryLogCurrentException(&Poco::Logger::get("StorageS3SequentialSource"));
|
|
|
|
throw;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
bool createOrUpdateInnerSource()
|
|
|
|
{
|
2021-03-23 17:58:29 +00:00
|
|
|
auto next_string = askAboutNextKey();
|
|
|
|
if (next_string.empty())
|
2021-03-22 17:12:31 +00:00
|
|
|
return false;
|
|
|
|
|
2021-03-23 17:58:29 +00:00
|
|
|
auto next_uri = S3::URI(Poco::URI(next_string));
|
|
|
|
|
2021-03-24 18:36:31 +00:00
|
|
|
auto client_auth = StorageS3::ClientAuthentificaiton{
|
|
|
|
next_uri,
|
|
|
|
cli_builder.access_key_id,
|
|
|
|
cli_builder.secret_access_key,
|
|
|
|
cli_builder.max_connections,
|
|
|
|
{}, {}};
|
|
|
|
StorageS3::updateClientAndAuthSettings(s3_source_builder.context, client_auth);
|
2021-03-22 17:12:31 +00:00
|
|
|
|
|
|
|
inner = std::make_unique<StorageS3Source>(
|
2021-03-24 18:36:31 +00:00
|
|
|
s3_source_builder.need_path,
|
|
|
|
s3_source_builder.need_file,
|
|
|
|
s3_source_builder.format,
|
|
|
|
s3_source_builder.name,
|
|
|
|
s3_source_builder.sample_block,
|
|
|
|
s3_source_builder.context,
|
|
|
|
s3_source_builder.columns,
|
|
|
|
s3_source_builder.max_block_size,
|
2021-03-26 15:33:14 +00:00
|
|
|
chooseCompressionMethod(client_auth.uri.key, ""),
|
2021-03-22 17:12:31 +00:00
|
|
|
client_auth.client,
|
|
|
|
client_auth.uri.bucket,
|
|
|
|
next_uri.key
|
|
|
|
);
|
2021-03-26 15:33:14 +00:00
|
|
|
|
2021-03-22 17:12:31 +00:00
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
2021-03-24 18:36:31 +00:00
|
|
|
/// This is used to ask about next task
|
|
|
|
String initial_query_id;
|
|
|
|
|
|
|
|
StorageS3SourceBuilder s3_source_builder;
|
|
|
|
ClientAuthentificationBuilder cli_builder;
|
2021-03-22 17:12:31 +00:00
|
|
|
|
|
|
|
std::unique_ptr<StorageS3Source> inner;
|
|
|
|
|
|
|
|
/// One second just in case
|
|
|
|
ConnectionTimeouts timeouts{{1, 0}, {1, 0}, {1, 0}};
|
2021-03-24 18:36:31 +00:00
|
|
|
std::shared_ptr<ConnectionPool> connections;
|
2021-03-22 17:12:31 +00:00
|
|
|
};
|
|
|
|
|
2021-03-19 21:49:18 +00:00
|
|
|
|
|
|
|
|
|
|
|
StorageS3Distributed::StorageS3Distributed(
|
2021-03-24 18:36:31 +00:00
|
|
|
IAST::Hash tree_hash_,
|
|
|
|
const String & address_hash_or_filename_,
|
2021-03-22 17:12:31 +00:00
|
|
|
const String & access_key_id_,
|
|
|
|
const String & secret_access_key_,
|
|
|
|
const StorageID & table_id_,
|
|
|
|
String cluster_name_,
|
|
|
|
const String & format_name_,
|
|
|
|
UInt64 max_connections_,
|
|
|
|
const ColumnsDescription & columns_,
|
|
|
|
const ConstraintsDescription & constraints_,
|
|
|
|
const Context & context_,
|
|
|
|
const String & compression_method_)
|
2021-03-19 21:49:18 +00:00
|
|
|
: IStorage(table_id_)
|
2021-03-24 18:36:31 +00:00
|
|
|
, tree_hash(tree_hash_)
|
|
|
|
, address_hash_or_filename(address_hash_or_filename_)
|
2021-03-19 21:49:18 +00:00
|
|
|
, cluster_name(cluster_name_)
|
2021-03-22 17:12:31 +00:00
|
|
|
, cluster(context_.getCluster(cluster_name)->getClusterWithReplicasAsShards(context_.getSettings()))
|
|
|
|
, format_name(format_name_)
|
|
|
|
, compression_method(compression_method_)
|
2021-03-24 18:36:31 +00:00
|
|
|
, cli_builder{access_key_id_, secret_access_key_, max_connections_}
|
2021-03-19 21:49:18 +00:00
|
|
|
{
|
|
|
|
StorageInMemoryMetadata storage_metadata;
|
2021-03-22 17:12:31 +00:00
|
|
|
storage_metadata.setColumns(columns_);
|
|
|
|
storage_metadata.setConstraints(constraints_);
|
2021-03-19 21:49:18 +00:00
|
|
|
setInMemoryMetadata(storage_metadata);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
Pipe StorageS3Distributed::read(
|
|
|
|
const Names & column_names,
|
|
|
|
const StorageMetadataPtr & metadata_snapshot,
|
|
|
|
SelectQueryInfo & query_info,
|
|
|
|
const Context & context,
|
2021-03-23 17:58:29 +00:00
|
|
|
QueryProcessingStage::Enum processed_stage,
|
2021-03-22 17:12:31 +00:00
|
|
|
size_t max_block_size,
|
2021-03-19 21:49:18 +00:00
|
|
|
unsigned /*num_streams*/)
|
|
|
|
{
|
|
|
|
/// Secondary query, need to read from S3
|
2021-03-26 15:33:14 +00:00
|
|
|
if (context.getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY)
|
2021-03-22 17:12:31 +00:00
|
|
|
{
|
2021-03-24 18:36:31 +00:00
|
|
|
/// Find initiator in cluster
|
|
|
|
Cluster::Address initiator;
|
2021-03-26 15:33:14 +00:00
|
|
|
[&]()
|
|
|
|
{
|
|
|
|
for (const auto & replicas : cluster->getShardsAddresses())
|
|
|
|
for (const auto & node : replicas)
|
|
|
|
if (node.getHash() == address_hash_or_filename)
|
|
|
|
{
|
|
|
|
initiator = node;
|
|
|
|
return;
|
|
|
|
}
|
|
|
|
}();
|
2021-03-24 18:36:31 +00:00
|
|
|
|
2021-03-22 17:12:31 +00:00
|
|
|
|
|
|
|
bool need_path_column = false;
|
|
|
|
bool need_file_column = false;
|
|
|
|
for (const auto & column : column_names)
|
|
|
|
{
|
|
|
|
if (column == "_path")
|
|
|
|
need_path_column = true;
|
|
|
|
if (column == "_file")
|
|
|
|
need_file_column = true;
|
|
|
|
}
|
2021-03-19 21:49:18 +00:00
|
|
|
|
2021-03-24 18:36:31 +00:00
|
|
|
StorageS3SourceBuilder s3builder
|
|
|
|
{
|
2021-03-22 17:12:31 +00:00
|
|
|
need_path_column,
|
|
|
|
need_file_column,
|
|
|
|
format_name,
|
|
|
|
getName(),
|
|
|
|
metadata_snapshot->getSampleBlock(),
|
|
|
|
context,
|
|
|
|
metadata_snapshot->getColumns(),
|
|
|
|
max_block_size,
|
2021-03-24 18:36:31 +00:00
|
|
|
compression_method
|
|
|
|
};
|
|
|
|
|
|
|
|
return Pipe(std::make_shared<StorageS3SequentialSource>(
|
|
|
|
context.getInitialQueryId(),
|
|
|
|
/*initiator*/initiator,
|
|
|
|
cli_builder,
|
|
|
|
s3builder
|
2021-03-22 17:12:31 +00:00
|
|
|
));
|
2021-03-19 21:49:18 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2021-03-26 15:33:14 +00:00
|
|
|
/// The code from here and below executes on initiator
|
2021-03-19 21:49:18 +00:00
|
|
|
|
2021-03-24 18:36:31 +00:00
|
|
|
String hash_of_address;
|
2021-03-26 15:33:14 +00:00
|
|
|
[&]()
|
|
|
|
{
|
|
|
|
for (const auto & replicas : cluster->getShardsAddresses())
|
|
|
|
for (const auto & node : replicas)
|
|
|
|
/// Finding ourselves in cluster
|
|
|
|
if (node.is_local && node.port == context.getTCPPort())
|
|
|
|
{
|
|
|
|
hash_of_address = node.getHash();
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
}();
|
2021-03-23 17:58:29 +00:00
|
|
|
|
2021-03-24 18:36:31 +00:00
|
|
|
if (hash_of_address.empty())
|
2021-03-26 15:33:14 +00:00
|
|
|
throw Exception(fmt::format("The initiator must be a part of a cluster {}", cluster_name), ErrorCodes::BAD_ARGUMENTS);
|
2021-03-24 18:36:31 +00:00
|
|
|
|
2021-03-26 15:33:14 +00:00
|
|
|
/// Our purpose to change some arguments of this function to store some relevant
|
|
|
|
/// information. Then we will send changed query to another hosts.
|
|
|
|
/// We got a pointer to table function representation in AST (a pointer to subtree)
|
|
|
|
/// as parameter of TableFunctionRemote::execute and saved its hash value.
|
|
|
|
/// Here we find it in the AST of whole query, change parameter and format it to string.
|
2021-03-24 18:36:31 +00:00
|
|
|
auto remote_query_ast = query_info.query->clone();
|
|
|
|
auto table_expressions_from_whole_query = getTableExpressions(remote_query_ast->as<ASTSelectQuery &>());
|
|
|
|
|
|
|
|
String remote_query;
|
|
|
|
for (const auto & table_expression : table_expressions_from_whole_query)
|
|
|
|
{
|
|
|
|
const auto & table_function_ast = table_expression->table_function;
|
|
|
|
if (table_function_ast->getTreeHash() == tree_hash)
|
|
|
|
{
|
|
|
|
auto & arguments = table_function_ast->children.at(0)->children;
|
|
|
|
auto & bucket = arguments[1]->as<ASTLiteral &>().value.safeGet<String>();
|
|
|
|
/// We rewrite query, and insert a port to connect as a first parameter
|
|
|
|
/// So, we write hash_of_address here as buckey name to find initiator node
|
|
|
|
/// in cluster from config on remote replica
|
|
|
|
bucket = hash_of_address;
|
|
|
|
remote_query = queryToString(remote_query_ast);
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if (remote_query.empty())
|
2021-03-26 15:33:14 +00:00
|
|
|
throw Exception(fmt::format("There is no table function with hash of AST equals to {}", hash_of_address), ErrorCodes::LOGICAL_ERROR);
|
2021-03-23 17:58:29 +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-03-24 18:36:31 +00:00
|
|
|
InterpreterSelectQuery(remote_query_ast, context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock();
|
2021-03-23 17:58:29 +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;
|
|
|
|
connections.reserve(cluster->getShardCount());
|
|
|
|
|
2021-03-19 21:49:18 +00:00
|
|
|
for (const auto & replicas : cluster->getShardsAddresses()) {
|
|
|
|
/// There will be only one replica, because we consider each replica as a shard
|
|
|
|
for (const auto & node : replicas)
|
|
|
|
{
|
|
|
|
connections.emplace_back(std::make_shared<Connection>(
|
|
|
|
/*host=*/node.host_name,
|
|
|
|
/*port=*/node.port,
|
|
|
|
/*default_database=*/context.getGlobalContext().getCurrentDatabase(),
|
|
|
|
/*user=*/node.user,
|
|
|
|
/*password=*/node.password,
|
|
|
|
/*cluster=*/node.cluster,
|
|
|
|
/*cluster_secret=*/node.cluster_secret
|
|
|
|
));
|
|
|
|
auto stream = std::make_shared<RemoteBlockInputStream>(
|
|
|
|
/*connection=*/*connections.back(),
|
2021-03-24 18:36:31 +00:00
|
|
|
/*query=*/remote_query,
|
2021-03-23 17:58:29 +00:00
|
|
|
/*header=*/header,
|
2021-03-22 23:06:10 +00:00
|
|
|
/*context=*/context,
|
2021-03-26 15:33:14 +00:00
|
|
|
/*throttler=*/nullptr,
|
|
|
|
/*scalars*/scalars,
|
|
|
|
/*external_tables*/Tables(),
|
|
|
|
/*stage*/processed_stage
|
2021-03-19 21:49:18 +00:00
|
|
|
);
|
|
|
|
pipes.emplace_back(std::make_shared<SourceFromInputStream>(std::move(stream)));
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
metadata_snapshot->check(column_names, getVirtuals(), getStorageID());
|
|
|
|
return Pipe::unitePipes(std::move(pipes));
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-03-24 21:02:21 +00:00
|
|
|
#endif
|