ClickHouse/src/Storages/StorageS3Distributed.cpp

282 lines
8.7 KiB
C++
Raw Normal View History

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>
2021-04-06 19:18:45 +00:00
#include <Processors/Sources/RemoteSource.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>
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_,
NextTaskCallback read_task_callback_,
2021-03-24 18:36:31 +00:00
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-04-07 14:43:34 +00:00
, s3b(s3_builder_)
, cab(client_auth_builder_)
, read_task_callback(read_task_callback_)
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
{
return read_task_callback(initial_query_id);
2021-03-22 17:12:31 +00:00
}
catch (...)
{
2021-04-07 14:43:34 +00:00
tryLogCurrentException(&Poco::Logger::get(getName()));
2021-03-22 17:12:31 +00:00
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,
2021-04-07 14:43:34 +00:00
cab.access_key_id,
cab.secret_access_key,
cab.max_connections,
2021-03-24 18:36:31 +00:00
{}, {}};
2021-04-07 14:43:34 +00:00
StorageS3::updateClientAndAuthSettings(s3b.context, client_auth);
2021-03-22 17:12:31 +00:00
inner = std::make_unique<StorageS3Source>(
2021-04-07 14:43:34 +00:00
s3b.need_path, s3b.need_file, s3b.format, s3b.name,
s3b.sample_block, s3b.context, s3b.columns, s3b.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
String initial_query_id;
2021-04-07 14:43:34 +00:00
StorageS3SourceBuilder s3b;
ClientAuthentificationBuilder cab;
2021-03-22 17:12:31 +00:00
std::unique_ptr<StorageS3Source> inner;
NextTaskCallback read_task_callback;
2021-03-22 17:12:31 +00:00
};
2021-03-19 21:49:18 +00:00
StorageS3Distributed::StorageS3Distributed(
2021-04-06 19:18:45 +00:00
const String & 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-04-06 19:18:45 +00:00
, filename(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
{
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-04-07 14:43:34 +00:00
StorageS3SourceBuilder s3builder{
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>(
2021-04-07 14:43:34 +00:00
context.getInitialQueryId(), context.getNextTaskCallback(), cli_builder, s3builder));
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-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
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>(
2021-04-06 19:18:45 +00:00
node.host_name, node.port, context.getGlobalContext().getCurrentDatabase(),
node.user, node.password, node.cluster, node.cluster_secret,
2021-04-07 14:43:34 +00:00
"S3DistributedInititiator",
node.compression,
node.secure
2021-03-19 21:49:18 +00:00
));
2021-04-06 19:18:45 +00:00
auto remote_query_executor = std::make_shared<RemoteQueryExecutor>(
*connections.back(), queryToString(query_info.query), header, context, /*throttler=*/nullptr, scalars, Tables(), processed_stage);
2021-04-07 14:43:34 +00:00
pipes.emplace_back(std::make_shared<RemoteSource>(remote_query_executor, false, false));
2021-03-19 21:49:18 +00:00
}
}
metadata_snapshot->check(column_names, getVirtuals(), getStorageID());
return Pipe::unitePipes(std::move(pipes));
}
2021-04-07 14:43:34 +00:00
QueryProcessingStage::Enum StorageS3Distributed::getQueryProcessingStage(const Context & context, QueryProcessingStage::Enum /*to_stage*/, SelectQueryInfo &) const
{
/// Initiator executes query on remote node.
if (context.getClientInfo().query_kind == ClientInfo::QueryKind::INITIAL_QUERY) {
return QueryProcessingStage::Enum::WithMergeableState;
}
/// Follower just reads the data.
return QueryProcessingStage::Enum::FetchColumns;
}
2021-03-19 21:49:18 +00:00
}
2021-03-24 21:02:21 +00:00
#endif