ClickHouse/src/Storages/StorageS3.cpp

1011 lines
35 KiB
C++
Raw Normal View History

2019-12-06 14:37:21 +00:00
#include <Common/config.h>
2021-08-23 19:05:28 +00:00
#include "Parsers/ASTCreateQuery.h"
2019-12-06 14:37:21 +00:00
#if USE_AWS_S3
#include <Common/isValidUTF8.h>
2021-05-31 08:46:28 +00:00
2021-06-01 12:41:23 +00:00
#include <Functions/FunctionsConversion.h>
2019-12-03 16:23:24 +00:00
#include <IO/S3Common.h>
2019-05-23 09:03:39 +00:00
2021-05-25 06:45:30 +00:00
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/TreeRewriter.h>
2019-05-23 09:03:39 +00:00
#include <Interpreters/evaluateConstantExpression.h>
2021-05-25 06:45:30 +00:00
2021-06-01 12:41:23 +00:00
#include <Parsers/ASTFunction.h>
2021-05-25 06:45:30 +00:00
#include <Parsers/ASTInsertQuery.h>
2019-05-23 09:03:39 +00:00
#include <Parsers/ASTLiteral.h>
2021-06-01 12:41:23 +00:00
#include <Storages/StorageFactory.h>
#include <Storages/StorageS3.h>
#include <Storages/StorageS3Settings.h>
2021-10-26 09:31:01 +00:00
#include <Storages/PartitionedSink.h>
2019-05-23 09:03:39 +00:00
2019-06-01 21:18:20 +00:00
#include <IO/ReadBufferFromS3.h>
#include <IO/WriteBufferFromS3.h>
2019-05-23 09:03:39 +00:00
#include <Formats/FormatFactory.h>
#include <Formats/ReadSchemaUtils.h>
2019-05-31 07:27:14 +00:00
2021-07-21 16:13:17 +00:00
#include <Processors/Transforms/AddingDefaultsTransform.h>
2021-10-11 16:11:50 +00:00
#include <Processors/Formats/IOutputFormat.h>
2021-10-13 18:22:02 +00:00
#include <Processors/Formats/IInputFormat.h>
2021-10-15 20:18:20 +00:00
#include <QueryPipeline/narrowBlockInputStreams.h>
2019-05-23 09:03:39 +00:00
2021-10-16 14:03:50 +00:00
#include <QueryPipeline/QueryPipelineBuilder.h>
2021-07-20 18:18:43 +00:00
#include <Processors/Executors/PullingPipelineExecutor.h>
2020-01-26 14:05:51 +00:00
#include <DataTypes/DataTypeString.h>
#include <aws/core/auth/AWSCredentials.h>
2019-12-03 16:23:24 +00:00
#include <aws/s3/S3Client.h>
#include <aws/s3/model/ListObjectsV2Request.h>
2021-06-21 15:44:36 +00:00
#include <aws/s3/model/CopyObjectRequest.h>
#include <aws/s3/model/DeleteObjectsRequest.h>
2020-01-26 13:03:47 +00:00
#include <Common/parseGlobs.h>
2020-02-17 19:28:25 +00:00
#include <Common/quoteString.h>
2020-01-26 13:03:47 +00:00
#include <re2/re2.h>
2019-05-23 09:03:39 +00:00
#include <Processors/Sources/SourceWithProgress.h>
2021-07-23 19:33:59 +00:00
#include <Processors/Sinks/SinkToStorage.h>
2021-10-16 14:03:50 +00:00
#include <QueryPipeline/Pipe.h>
2021-05-08 10:59:55 +00:00
#include <filesystem>
2019-05-23 09:03:39 +00:00
2021-05-08 10:59:55 +00:00
namespace fs = std::filesystem;
2019-05-31 07:27:14 +00:00
2021-05-25 06:45:30 +00:00
static const String PARTITION_ID_WILDCARD = "{_partition_id}";
2019-05-23 09:03:39 +00:00
namespace DB
{
2021-07-18 11:03:45 +00:00
2019-05-23 09:03:39 +00:00
namespace ErrorCodes
{
2021-07-18 11:03:45 +00:00
extern const int CANNOT_PARSE_TEXT;
2021-08-20 13:12:30 +00:00
extern const int BAD_ARGUMENTS;
2019-05-23 09:03:39 +00:00
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
2020-01-30 06:17:55 +00:00
extern const int S3_ERROR;
2021-07-18 11:03:45 +00:00
extern const int UNEXPECTED_EXPRESSION;
extern const int DATABASE_ACCESS_DENIED;
extern const int CANNOT_EXTRACT_TABLE_STRUCTURE;
2019-05-23 09:03:39 +00:00
}
2021-10-11 16:11:50 +00:00
class IOutputFormat;
using OutputFormatPtr = std::shared_ptr<IOutputFormat>;
2021-04-06 19:18:45 +00:00
class StorageS3Source::DisclosedGlobIterator::Impl
{
public:
Impl(Aws::S3::S3Client & client_, const S3::URI & globbed_uri_)
2021-04-12 17:07:01 +00:00
: client(client_), globbed_uri(globbed_uri_)
{
2021-04-06 19:18:45 +00:00
if (globbed_uri.bucket.find_first_of("*?{") != globbed_uri.bucket.npos)
throw Exception("Expression can not have wildcards inside bucket name", ErrorCodes::UNEXPECTED_EXPRESSION);
const String key_prefix = globbed_uri.key.substr(0, globbed_uri.key.find_first_of("*?{"));
2021-06-28 17:02:22 +00:00
/// We don't have to list bucket, because there is no asterisks.
2021-04-06 19:18:45 +00:00
if (key_prefix.size() == globbed_uri.key.size())
2021-04-12 16:51:52 +00:00
{
2021-04-06 19:18:45 +00:00
buffer.emplace_back(globbed_uri.key);
2021-04-12 16:51:52 +00:00
buffer_iter = buffer.begin();
is_finished = true;
return;
}
2021-04-06 19:18:45 +00:00
request.SetBucket(globbed_uri.bucket);
request.SetPrefix(key_prefix);
matcher = std::make_unique<re2::RE2>(makeRegexpPatternFromGlobs(globbed_uri.key));
2021-04-10 14:58:29 +00:00
fillInternalBufferAssumeLocked();
2021-04-06 19:18:45 +00:00
}
2021-04-13 10:59:02 +00:00
String next()
2021-04-10 02:21:18 +00:00
{
std::lock_guard lock(mutex);
return nextAssumeLocked();
}
private:
2021-04-13 10:59:02 +00:00
String nextAssumeLocked()
2021-04-06 19:18:45 +00:00
{
if (buffer_iter != buffer.end())
{
auto answer = *buffer_iter;
++buffer_iter;
return answer;
}
if (is_finished)
2021-04-13 10:59:02 +00:00
return {};
2021-04-06 19:18:45 +00:00
2021-04-10 02:21:18 +00:00
fillInternalBufferAssumeLocked();
2021-04-06 19:18:45 +00:00
2021-04-10 02:21:18 +00:00
return nextAssumeLocked();
2021-04-06 19:18:45 +00:00
}
2021-04-10 02:21:18 +00:00
void fillInternalBufferAssumeLocked()
2021-04-06 19:18:45 +00:00
{
buffer.clear();
outcome = client.ListObjectsV2(request);
if (!outcome.IsSuccess())
throw Exception(ErrorCodes::S3_ERROR, "Could not list objects in bucket {} with prefix {}, S3 exception: {}, message: {}",
quoteString(request.GetBucket()), quoteString(request.GetPrefix()),
backQuote(outcome.GetError().GetExceptionName()), quoteString(outcome.GetError().GetMessage()));
const auto & result_batch = outcome.GetResult().GetContents();
buffer.reserve(result_batch.size());
for (const auto & row : result_batch)
{
String key = row.GetKey();
if (re2::RE2::FullMatch(key, *matcher))
buffer.emplace_back(std::move(key));
}
/// Set iterator only after the whole batch is processed
buffer_iter = buffer.begin();
request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken());
2021-04-12 17:07:01 +00:00
2021-04-06 19:18:45 +00:00
/// It returns false when all objects were returned
is_finished = !outcome.GetResult().GetIsTruncated();
}
2021-04-10 02:21:18 +00:00
std::mutex mutex;
2021-04-06 19:18:45 +00:00
Strings buffer;
Strings::iterator buffer_iter;
Aws::S3::S3Client client;
S3::URI globbed_uri;
Aws::S3::Model::ListObjectsV2Request request;
Aws::S3::Model::ListObjectsV2Outcome outcome;
std::unique_ptr<re2::RE2> matcher;
bool is_finished{false};
};
StorageS3Source::DisclosedGlobIterator::DisclosedGlobIterator(Aws::S3::S3Client & client_, const S3::URI & globbed_uri_)
2021-04-10 02:21:18 +00:00
: pimpl(std::make_shared<StorageS3Source::DisclosedGlobIterator::Impl>(client_, globbed_uri_)) {}
2021-04-06 19:18:45 +00:00
2021-04-13 10:59:02 +00:00
String StorageS3Source::DisclosedGlobIterator::next()
2021-04-06 19:18:45 +00:00
{
return pimpl->next();
}
class StorageS3Source::KeysIterator::Impl
{
public:
2022-01-18 19:26:13 +00:00
explicit Impl(const std::vector<String> & keys_) : keys(keys_), keys_iter(keys.begin())
{
}
String next()
{
2022-01-18 19:26:13 +00:00
std::lock_guard lock(mutex);
if (keys_iter == keys.end())
return "";
2022-01-18 19:26:13 +00:00
auto key = *keys_iter;
++keys_iter;
return key;
}
private:
2022-01-18 19:26:13 +00:00
std::mutex mutex;
Strings keys;
Strings::iterator keys_iter;
};
StorageS3Source::KeysIterator::KeysIterator(const std::vector<String> & keys_) : pimpl(std::make_shared<StorageS3Source::KeysIterator::Impl>(keys_))
{
}
String StorageS3Source::KeysIterator::next()
{
return pimpl->next();
}
2019-12-03 16:23:24 +00:00
2021-03-22 17:12:31 +00:00
Block StorageS3Source::getHeader(Block sample_block, bool with_path_column, bool with_file_column)
2019-05-23 09:03:39 +00:00
{
2021-03-22 17:12:31 +00:00
if (with_path_column)
2022-02-05 08:10:28 +00:00
sample_block.insert(
{DataTypeLowCardinality{std::make_shared<DataTypeString>()}.createColumn(),
std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>()),
"_path"});
2021-03-22 17:12:31 +00:00
if (with_file_column)
2022-02-05 08:10:28 +00:00
sample_block.insert(
{DataTypeLowCardinality{std::make_shared<DataTypeString>()}.createColumn(),
std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>()),
"_file"});
2020-02-14 17:47:39 +00:00
2021-03-22 17:12:31 +00:00
return sample_block;
}
2020-02-14 17:47:39 +00:00
2021-03-22 17:12:31 +00:00
StorageS3Source::StorageS3Source(
bool need_path,
bool need_file,
2021-04-08 00:09:15 +00:00
const String & format_,
2021-03-22 17:12:31 +00:00
String name_,
2021-04-08 00:09:15 +00:00
const Block & sample_block_,
2021-04-12 19:35:26 +00:00
ContextPtr context_,
2021-08-23 19:05:28 +00:00
std::optional<FormatSettings> format_settings_,
2021-04-08 00:09:15 +00:00
const ColumnsDescription & columns_,
UInt64 max_block_size_,
2021-05-19 21:42:25 +00:00
UInt64 max_single_read_retries_,
2021-06-01 14:23:46 +00:00
const String compression_hint_,
2021-04-08 00:09:15 +00:00
const std::shared_ptr<Aws::S3::S3Client> & client_,
const String & bucket_,
2021-04-10 02:21:18 +00:00
std::shared_ptr<IteratorWrapper> file_iterator_)
2021-04-08 00:09:15 +00:00
: SourceWithProgress(getHeader(sample_block_, need_path, need_file))
2021-04-12 19:35:26 +00:00
, WithContext(context_)
2021-03-22 17:12:31 +00:00
, name(std::move(name_))
2021-04-08 00:09:15 +00:00
, bucket(bucket_)
, format(format_)
, columns_desc(columns_)
, max_block_size(max_block_size_)
2021-05-19 21:42:25 +00:00
, max_single_read_retries(max_single_read_retries_)
2021-04-08 00:09:15 +00:00
, compression_hint(compression_hint_)
, client(client_)
, sample_block(sample_block_)
2021-08-23 19:05:28 +00:00
, format_settings(format_settings_)
2021-03-22 17:12:31 +00:00
, with_file_column(need_file)
, with_path_column(need_path)
2021-04-08 00:09:15 +00:00
, file_iterator(file_iterator_)
{
initialize();
}
2021-12-27 19:42:56 +00:00
void StorageS3Source::onCancel()
{
std::lock_guard lock(reader_mutex);
2021-12-27 19:42:56 +00:00
if (reader)
reader->cancel();
}
2021-04-08 00:09:15 +00:00
bool StorageS3Source::initialize()
2021-03-22 17:12:31 +00:00
{
2021-04-13 10:59:02 +00:00
String current_key = (*file_iterator)();
if (current_key.empty())
2021-04-08 00:09:15 +00:00
return false;
2021-04-13 10:59:02 +00:00
2021-05-08 10:59:55 +00:00
file_path = fs::path(bucket) / current_key;
2021-04-08 00:09:15 +00:00
read_buf = wrapReadBufferWithCompressionMethod(
2021-10-10 22:53:51 +00:00
std::make_unique<ReadBufferFromS3>(client, bucket, current_key, max_single_read_retries, getContext()->getReadSettings()),
chooseCompressionMethod(current_key, compression_hint));
2021-10-11 16:11:50 +00:00
auto input_format = getContext()->getInputFormat(format, *read_buf, sample_block, max_block_size, format_settings);
2021-09-16 17:40:42 +00:00
QueryPipelineBuilder builder;
builder.init(Pipe(input_format));
2020-02-14 17:47:39 +00:00
2021-04-08 00:09:15 +00:00
if (columns_desc.hasDefaults())
2021-07-20 18:18:43 +00:00
{
2021-09-16 17:40:42 +00:00
builder.addSimpleTransform([&](const Block & header)
2021-07-20 18:18:43 +00:00
{
return std::make_shared<AddingDefaultsTransform>(header, columns_desc, *input_format, getContext());
});
}
2021-09-16 17:40:42 +00:00
pipeline = std::make_unique<QueryPipeline>(QueryPipelineBuilder::getPipeline(std::move(builder)));
2021-07-20 18:18:43 +00:00
reader = std::make_unique<PullingPipelineExecutor>(*pipeline);
2021-04-08 00:09:15 +00:00
initialized = false;
return true;
2021-03-22 17:12:31 +00:00
}
2021-03-22 17:12:31 +00:00
String StorageS3Source::getName() const
{
return name;
}
2019-05-23 09:03:39 +00:00
2021-03-22 17:12:31 +00:00
Chunk StorageS3Source::generate()
{
if (!reader)
return {};
2021-07-20 18:18:43 +00:00
Chunk chunk;
if (reader->pull(chunk))
2021-03-22 17:12:31 +00:00
{
2021-07-20 18:18:43 +00:00
UInt64 num_rows = chunk.getNumRows();
2021-03-22 17:12:31 +00:00
if (with_path_column)
2022-02-05 08:10:28 +00:00
chunk.addColumn(DataTypeLowCardinality{std::make_shared<DataTypeString>()}
.createColumnConst(num_rows, file_path)
->convertToFullColumnIfConst());
2021-03-22 17:12:31 +00:00
if (with_file_column)
2019-05-31 07:27:14 +00:00
{
2021-03-22 17:12:31 +00:00
size_t last_slash_pos = file_path.find_last_of('/');
2022-02-05 08:10:28 +00:00
chunk.addColumn(DataTypeLowCardinality{std::make_shared<DataTypeString>()}
.createColumnConst(num_rows, file_path.substr(last_slash_pos + 1))
->convertToFullColumnIfConst());
2019-05-31 07:27:14 +00:00
}
2021-07-20 18:18:43 +00:00
return chunk;
2021-03-22 17:12:31 +00:00
}
2022-01-31 11:03:06 +00:00
{
std::lock_guard lock(reader_mutex);
reader.reset();
pipeline.reset();
read_buf.reset();
2021-03-22 17:12:31 +00:00
2022-01-31 11:03:06 +00:00
if (!initialize())
return {};
}
2021-04-08 00:09:15 +00:00
return generate();
2021-03-22 17:12:31 +00:00
}
static bool checkIfObjectExists(const std::shared_ptr<Aws::S3::S3Client> & client, const String & bucket, const String & key)
{
bool is_finished = false;
Aws::S3::Model::ListObjectsV2Request request;
Aws::S3::Model::ListObjectsV2Outcome outcome;
request.SetBucket(bucket);
request.SetPrefix(key);
while (!is_finished)
{
outcome = client->ListObjectsV2(request);
if (!outcome.IsSuccess())
throw Exception(
ErrorCodes::S3_ERROR,
"Could not list objects in bucket {} with key {}, S3 exception: {}, message: {}",
quoteString(bucket),
quoteString(key),
backQuote(outcome.GetError().GetExceptionName()),
quoteString(outcome.GetError().GetMessage()));
const auto & result_batch = outcome.GetResult().GetContents();
for (const auto & obj : result_batch)
{
if (obj.GetKey() == key)
return true;
}
request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken());
is_finished = !outcome.GetResult().GetIsTruncated();
}
return false;
}
2021-04-12 19:35:26 +00:00
2021-07-23 19:33:59 +00:00
class StorageS3Sink : public SinkToStorage
2021-03-22 17:12:31 +00:00
{
2021-04-12 17:07:01 +00:00
public:
2021-07-23 19:33:59 +00:00
StorageS3Sink(
2021-04-12 17:07:01 +00:00
const String & format,
const Block & sample_block_,
ContextPtr context,
2021-08-23 19:05:28 +00:00
std::optional<FormatSettings> format_settings_,
2021-04-12 17:07:01 +00:00
const CompressionMethod compression_method,
const std::shared_ptr<Aws::S3::S3Client> & client,
const String & bucket,
const String & key,
size_t min_upload_part_size,
size_t upload_part_size_multiply_factor,
size_t upload_part_size_multiply_parts_count_threshold,
2021-04-12 17:07:01 +00:00
size_t max_single_part_upload_size)
2021-07-23 19:33:59 +00:00
: SinkToStorage(sample_block_)
, sample_block(sample_block_)
2021-08-23 19:05:28 +00:00
, format_settings(format_settings_)
2019-05-23 09:03:39 +00:00
{
2021-04-12 17:07:01 +00:00
write_buf = wrapWriteBufferWithCompressionMethod(
std::make_unique<WriteBufferFromS3>(
client, bucket, key, min_upload_part_size,
upload_part_size_multiply_factor, upload_part_size_multiply_parts_count_threshold,
max_single_part_upload_size), compression_method, 3);
2021-10-11 16:11:50 +00:00
writer = FormatFactory::instance().getOutputFormatParallelIfPossible(format, *write_buf, sample_block, context, {}, format_settings);
2021-04-12 17:07:01 +00:00
}
2019-05-31 07:27:14 +00:00
2021-07-23 19:33:59 +00:00
String getName() const override { return "StorageS3Sink"; }
2019-05-31 07:27:14 +00:00
2021-07-23 19:33:59 +00:00
void consume(Chunk chunk) override
2021-04-12 17:07:01 +00:00
{
writer->write(getHeader().cloneWithColumns(chunk.detachColumns()));
2021-04-12 17:07:01 +00:00
}
2019-05-31 07:27:14 +00:00
2021-07-23 19:33:59 +00:00
void onFinish() override
2021-04-12 17:07:01 +00:00
{
try
{
2021-11-11 18:09:21 +00:00
writer->finalize();
writer->flush();
write_buf->finalize();
}
catch (...)
{
/// Stop ParallelFormattingOutputFormat correctly.
writer.reset();
throw;
}
2021-04-12 17:07:01 +00:00
}
2019-05-31 07:27:14 +00:00
2021-04-12 17:07:01 +00:00
private:
Block sample_block;
2021-08-23 19:05:28 +00:00
std::optional<FormatSettings> format_settings;
2021-04-12 17:07:01 +00:00
std::unique_ptr<WriteBuffer> write_buf;
2021-10-11 16:11:50 +00:00
OutputFormatPtr writer;
2021-04-12 17:07:01 +00:00
};
2019-05-23 09:03:39 +00:00
2021-10-26 09:31:01 +00:00
class PartitionedStorageS3Sink : public PartitionedSink
2021-05-25 06:45:30 +00:00
{
public:
PartitionedStorageS3Sink(
const ASTPtr & partition_by,
const String & format_,
const Block & sample_block_,
ContextPtr context_,
2021-08-25 15:37:17 +00:00
std::optional<FormatSettings> format_settings_,
2021-05-25 06:45:30 +00:00
const CompressionMethod compression_method_,
const std::shared_ptr<Aws::S3::S3Client> & client_,
const String & bucket_,
const String & key_,
size_t min_upload_part_size_,
size_t upload_part_size_multiply_factor_,
size_t upload_part_size_multiply_parts_count_threshold_,
2021-05-25 06:45:30 +00:00
size_t max_single_part_upload_size_)
2021-10-26 09:31:01 +00:00
: PartitionedSink(partition_by, context_, sample_block_)
2021-05-25 06:45:30 +00:00
, format(format_)
, sample_block(sample_block_)
, context(context_)
, compression_method(compression_method_)
, client(client_)
, bucket(bucket_)
, key(key_)
, min_upload_part_size(min_upload_part_size_)
, upload_part_size_multiply_factor(upload_part_size_multiply_factor_)
, upload_part_size_multiply_parts_count_threshold(upload_part_size_multiply_parts_count_threshold_)
2021-05-25 06:45:30 +00:00
, max_single_part_upload_size(max_single_part_upload_size_)
2021-08-25 15:37:17 +00:00
, format_settings(format_settings_)
2021-05-25 06:45:30 +00:00
{
}
2021-10-26 09:31:01 +00:00
SinkPtr createSinkForPartition(const String & partition_id) override
2021-05-25 06:45:30 +00:00
{
2021-10-26 09:31:01 +00:00
auto partition_bucket = replaceWildcards(bucket, partition_id);
validateBucket(partition_bucket);
2021-05-25 06:45:30 +00:00
2021-10-26 09:31:01 +00:00
auto partition_key = replaceWildcards(key, partition_id);
validateKey(partition_key);
2021-05-25 06:45:30 +00:00
2021-10-26 09:31:01 +00:00
return std::make_shared<StorageS3Sink>(
format,
sample_block,
context,
format_settings,
compression_method,
client,
partition_bucket,
partition_key,
min_upload_part_size,
upload_part_size_multiply_factor,
upload_part_size_multiply_parts_count_threshold,
2021-10-26 09:31:01 +00:00
max_single_part_upload_size
);
2021-05-25 06:45:30 +00:00
}
private:
const String format;
const Block sample_block;
ContextPtr context;
const CompressionMethod compression_method;
2021-10-26 09:31:01 +00:00
2021-05-25 06:45:30 +00:00
std::shared_ptr<Aws::S3::S3Client> client;
const String bucket;
const String key;
size_t min_upload_part_size;
size_t upload_part_size_multiply_factor;
size_t upload_part_size_multiply_parts_count_threshold;
2021-05-25 06:45:30 +00:00
size_t max_single_part_upload_size;
2021-08-25 15:37:17 +00:00
std::optional<FormatSettings> format_settings;
2021-05-25 06:45:30 +00:00
ExpressionActionsPtr partition_by_expr;
2021-08-20 13:12:30 +00:00
static void validateBucket(const String & str)
{
2021-08-20 13:12:30 +00:00
S3::URI::validateBucket(str, {});
2021-08-20 13:12:30 +00:00
if (!DB::UTF8::isValidUTF8(reinterpret_cast<const UInt8 *>(str.data()), str.size()))
throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Incorrect non-UTF8 sequence in bucket name");
validatePartitionKey(str, false);
}
2021-08-20 13:12:30 +00:00
static void validateKey(const String & str)
{
/// See:
/// - https://docs.aws.amazon.com/AmazonS3/latest/userguide/object-keys.html
/// - https://cloud.ibm.com/apidocs/cos/cos-compatibility#putobject
2021-08-20 13:12:30 +00:00
if (str.empty() || str.size() > 1024)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Incorrect key length (not empty, max 1023 characters), got: {}", str.size());
2021-08-20 13:12:30 +00:00
if (!DB::UTF8::isValidUTF8(reinterpret_cast<const UInt8 *>(str.data()), str.size()))
throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Incorrect non-UTF8 sequence in key");
validatePartitionKey(str, true);
}
2021-05-25 06:45:30 +00:00
};
2019-12-04 16:06:55 +00:00
StorageS3::StorageS3(
const S3::URI & uri_,
const String & access_key_id_,
const String & secret_access_key_,
2019-12-04 16:06:55 +00:00
const StorageID & table_id_,
2019-09-22 22:13:42 +00:00
const String & format_name_,
2021-05-19 21:42:25 +00:00
UInt64 max_single_read_retries_,
UInt64 min_upload_part_size_,
UInt64 upload_part_size_multiply_factor_,
UInt64 upload_part_size_multiply_parts_count_threshold_,
UInt64 max_single_part_upload_size_,
UInt64 max_connections_,
2019-09-22 22:13:42 +00:00
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
2021-04-23 12:18:23 +00:00
const String & comment,
ContextPtr context_,
2021-08-23 19:05:28 +00:00
std::optional<FormatSettings> format_settings_,
2021-04-13 20:17:25 +00:00
const String & compression_method_,
2021-10-26 12:22:13 +00:00
bool distributed_processing_,
ASTPtr partition_by_)
2020-04-27 13:55:30 +00:00
: IStorage(table_id_)
2021-03-16 18:41:29 +00:00
, client_auth{uri_, access_key_id_, secret_access_key_, max_connections_, {}, {}} /// Client and settings will be updated later
, keys({uri_.key})
2019-09-22 22:13:42 +00:00
, format_name(format_name_)
2021-05-19 21:42:25 +00:00
, max_single_read_retries(max_single_read_retries_)
, min_upload_part_size(min_upload_part_size_)
, upload_part_size_multiply_factor(upload_part_size_multiply_factor_)
, upload_part_size_multiply_parts_count_threshold(upload_part_size_multiply_parts_count_threshold_)
, max_single_part_upload_size(max_single_part_upload_size_)
, compression_method(compression_method_)
, name(uri_.storage_name)
2021-04-13 20:17:25 +00:00
, distributed_processing(distributed_processing_)
2021-08-23 19:05:28 +00:00
, format_settings(format_settings_)
2021-10-26 12:22:13 +00:00
, partition_by(partition_by_)
, is_key_with_globs(uri_.key.find_first_of("*?{") != std::string::npos)
2019-09-22 22:13:42 +00:00
{
2021-03-16 18:41:29 +00:00
context_->getGlobalContext()->getRemoteHostFilter().checkURL(uri_.uri);
2020-06-19 15:39:41 +00:00
StorageInMemoryMetadata storage_metadata;
updateClientAndAuthSettings(context_, client_auth);
if (columns_.empty())
{
auto columns = getTableStructureFromDataImpl(format_name, client_auth, max_single_read_retries_, compression_method, distributed_processing_, is_key_with_globs, format_settings, context_);
storage_metadata.setColumns(columns);
}
else
storage_metadata.setColumns(columns_);
2020-06-19 15:39:41 +00:00
storage_metadata.setConstraints(constraints_);
2021-04-23 12:18:23 +00:00
storage_metadata.setComment(comment);
2020-06-19 15:39:41 +00:00
setInMemoryMetadata(storage_metadata);
2019-09-22 22:13:42 +00:00
}
std::shared_ptr<StorageS3Source::IteratorWrapper> StorageS3::createFileIterator(const ClientAuthentication & client_auth, const std::vector<String> & keys, bool is_key_with_globs, bool distributed_processing, ContextPtr local_context)
{
if (distributed_processing)
{
return std::make_shared<StorageS3Source::IteratorWrapper>(
[callback = local_context->getReadTaskCallback()]() -> String {
return callback();
});
}
else if (is_key_with_globs)
{
/// Iterate through disclosed globs and make a source for each file
auto glob_iterator = std::make_shared<StorageS3Source::DisclosedGlobIterator>(*client_auth.client, client_auth.uri);
2022-01-14 15:16:18 +00:00
return std::make_shared<StorageS3Source::IteratorWrapper>([glob_iterator]()
{
return glob_iterator->next();
});
}
else
{
auto keys_iterator = std::make_shared<StorageS3Source::KeysIterator>(keys);
2022-01-14 15:16:18 +00:00
return std::make_shared<StorageS3Source::IteratorWrapper>([keys_iterator]()
{
return keys_iterator->next();
});
}
}
2019-09-22 22:13:42 +00:00
2020-08-03 13:54:14 +00:00
Pipe StorageS3::read(
2019-09-22 22:13:42 +00:00
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
SelectQueryInfo & /*query_info*/,
ContextPtr local_context,
2019-06-01 21:18:20 +00:00
QueryProcessingStage::Enum /*processed_stage*/,
2019-05-23 09:03:39 +00:00
size_t max_block_size,
unsigned num_streams)
2019-05-23 09:03:39 +00:00
{
2021-03-16 18:41:29 +00:00
updateClientAndAuthSettings(local_context, client_auth);
Pipes pipes;
2020-01-26 14:05:51 +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;
}
std::shared_ptr<StorageS3Source::IteratorWrapper> iterator_wrapper = createFileIterator(client_auth, keys, is_key_with_globs, distributed_processing, local_context);
2021-04-13 20:19:04 +00:00
2021-04-08 00:09:15 +00:00
for (size_t i = 0; i < num_streams; ++i)
2021-04-06 19:18:45 +00:00
{
2020-02-14 17:47:39 +00:00
pipes.emplace_back(std::make_shared<StorageS3Source>(
2020-01-26 14:05:51 +00:00
need_path_column,
need_file_column,
format_name,
getName(),
metadata_snapshot->getSampleBlock(),
local_context,
2021-08-23 19:05:28 +00:00
format_settings,
2020-10-02 12:38:50 +00:00
metadata_snapshot->getColumns(),
max_block_size,
2021-05-19 21:42:25 +00:00
max_single_read_retries,
2021-04-08 00:09:15 +00:00
compression_method,
2021-03-16 18:41:29 +00:00
client_auth.client,
client_auth.uri.bucket,
2021-04-10 02:21:18 +00:00
iterator_wrapper));
2021-04-06 19:18:45 +00:00
}
2021-04-10 14:58:29 +00:00
auto pipe = Pipe::unitePipes(std::move(pipes));
narrowPipe(pipe, num_streams);
return pipe;
2019-05-23 09:03:39 +00:00
}
2021-05-25 06:45:30 +00:00
SinkToStoragePtr StorageS3::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context)
2019-05-23 09:03:39 +00:00
{
2021-03-16 18:41:29 +00:00
updateClientAndAuthSettings(local_context, client_auth);
2021-05-25 06:45:30 +00:00
auto sample_block = metadata_snapshot->getSampleBlock();
auto chosen_compression_method = chooseCompressionMethod(keys.back(), compression_method);
bool has_wildcards = client_auth.uri.bucket.find(PARTITION_ID_WILDCARD) != String::npos || keys.back().find(PARTITION_ID_WILDCARD) != String::npos;
2021-07-14 08:49:05 +00:00
auto insert_query = std::dynamic_pointer_cast<ASTInsertQuery>(query);
2021-10-26 12:22:13 +00:00
auto partition_by_ast = insert_query ? (insert_query->partition_by ? insert_query->partition_by : partition_by) : nullptr;
bool is_partitioned_implementation = partition_by_ast && has_wildcards;
2021-07-14 08:49:05 +00:00
if (is_partitioned_implementation)
2021-05-25 06:45:30 +00:00
{
return std::make_shared<PartitionedStorageS3Sink>(
2021-10-26 12:22:13 +00:00
partition_by_ast,
2021-05-25 06:45:30 +00:00
format_name,
sample_block,
local_context,
2021-08-25 15:37:17 +00:00
format_settings,
2021-05-25 06:45:30 +00:00
chosen_compression_method,
client_auth.client,
client_auth.uri.bucket,
keys.back(),
2021-05-25 06:45:30 +00:00
min_upload_part_size,
upload_part_size_multiply_factor,
upload_part_size_multiply_parts_count_threshold,
2021-05-25 06:45:30 +00:00
max_single_part_upload_size);
}
else
{
2022-01-14 18:18:16 +00:00
if (is_key_with_globs)
throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, "S3 key '{}' contains globs, so the table is in readonly mode", client_auth.uri.key);
bool truncate_in_insert = local_context->getSettingsRef().s3_truncate_on_insert;
if (!truncate_in_insert && checkIfObjectExists(client_auth.client, client_auth.uri.bucket, keys.back()))
{
if (local_context->getSettingsRef().s3_create_new_file_on_insert)
{
size_t index = keys.size();
auto pos = keys[0].find_first_of('.');
String new_key;
do
{
new_key = keys[0].substr(0, pos) + "." + std::to_string(index) + (pos == std::string::npos ? "" : keys[0].substr(pos));
++index;
}
while (checkIfObjectExists(client_auth.client, client_auth.uri.bucket, new_key));
keys.push_back(new_key);
}
else
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Object in bucket {} with key {} already exists. If you want to overwrite it, enable setting s3_truncate_on_insert, if you "
"want to create a new file on each insert, enable setting s3_create_new_file_on_insert",
client_auth.uri.bucket,
keys.back());
}
2021-05-25 06:45:30 +00:00
return std::make_shared<StorageS3Sink>(
format_name,
sample_block,
local_context,
2021-08-25 15:37:17 +00:00
format_settings,
2021-05-25 06:45:30 +00:00
chosen_compression_method,
client_auth.client,
client_auth.uri.bucket,
keys.back(),
2021-05-25 06:45:30 +00:00
min_upload_part_size,
upload_part_size_multiply_factor,
upload_part_size_multiply_parts_count_threshold,
2021-05-25 06:45:30 +00:00
max_single_part_upload_size);
}
2019-05-23 09:03:39 +00:00
}
2021-06-21 15:44:36 +00:00
void StorageS3::truncate(const ASTPtr & /* query */, const StorageMetadataPtr &, ContextPtr local_context, TableExclusiveLockHolder &)
{
updateClientAndAuthSettings(local_context, client_auth);
if (is_key_with_globs)
throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, "S3 key '{}' contains globs, so the table is in readonly mode", client_auth.uri.key);
2021-06-21 15:44:36 +00:00
Aws::S3::Model::Delete delkeys;
for (const auto & key : keys)
{
Aws::S3::Model::ObjectIdentifier obj;
obj.SetKey(key);
delkeys.AddObjects(std::move(obj));
}
2021-06-21 15:44:36 +00:00
Aws::S3::Model::DeleteObjectsRequest request;
request.SetBucket(client_auth.uri.bucket);
request.SetDelete(delkeys);
auto response = client_auth.client->DeleteObjects(request);
if (!response.IsSuccess())
{
const auto & err = response.GetError();
throw Exception(std::to_string(static_cast<int>(err.GetErrorType())) + ": " + err.GetMessage(), ErrorCodes::S3_ERROR);
}
}
2021-06-28 17:02:22 +00:00
void StorageS3::updateClientAndAuthSettings(ContextPtr ctx, StorageS3::ClientAuthentication & upd)
{
2021-03-16 18:41:29 +00:00
auto settings = ctx->getStorageS3Settings().getSettings(upd.uri.uri.toString());
if (upd.client && (!upd.access_key_id.empty() || settings == upd.auth_settings))
return;
2021-03-16 18:41:29 +00:00
Aws::Auth::AWSCredentials credentials(upd.access_key_id, upd.secret_access_key);
HeaderCollection headers;
2021-03-16 18:41:29 +00:00
if (upd.access_key_id.empty())
{
credentials = Aws::Auth::AWSCredentials(settings.access_key_id, settings.secret_access_key);
headers = settings.headers;
}
S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration(
settings.region,
2021-03-16 18:41:29 +00:00
ctx->getRemoteHostFilter(), ctx->getGlobalContext()->getSettingsRef().s3_max_redirects);
2021-03-16 18:41:29 +00:00
client_configuration.endpointOverride = upd.uri.endpoint;
client_configuration.maxConnections = upd.max_connections;
2021-03-16 18:41:29 +00:00
upd.client = S3::ClientFactory::instance().create(
client_configuration,
2021-03-16 18:41:29 +00:00
upd.uri.is_virtual_hosted_style,
credentials.GetAWSAccessKeyId(),
credentials.GetAWSSecretKey(),
settings.server_side_encryption_customer_key_base64,
std::move(headers),
settings.use_environment_credentials.value_or(ctx->getConfigRef().getBool("s3.use_environment_credentials", false)),
2021-03-30 00:32:26 +00:00
settings.use_insecure_imds_request.value_or(ctx->getConfigRef().getBool("s3.use_insecure_imds_request", false)));
2021-03-16 18:41:29 +00:00
upd.auth_settings = std::move(settings);
}
2021-09-07 11:17:25 +00:00
StorageS3Configuration StorageS3::getConfiguration(ASTs & engine_args, ContextPtr local_context)
2019-05-23 09:03:39 +00:00
{
2021-09-15 22:45:43 +00:00
StorageS3Configuration configuration;
2019-06-17 00:06:14 +00:00
2021-09-15 22:45:43 +00:00
if (auto named_collection = getURLBasedDataSourceConfiguration(engine_args, local_context))
2019-06-17 00:06:14 +00:00
{
2021-09-15 22:45:43 +00:00
auto [common_configuration, storage_specific_args] = named_collection.value();
configuration.set(common_configuration);
2019-06-17 00:06:14 +00:00
2021-09-07 11:17:25 +00:00
for (const auto & [arg_name, arg_value] : storage_specific_args)
{
if (arg_name == "access_key_id")
2021-10-23 15:20:31 +00:00
configuration.access_key_id = arg_value->as<ASTLiteral>()->value.safeGet<String>();
2021-09-07 11:17:25 +00:00
else if (arg_name == "secret_access_key")
2021-10-23 15:20:31 +00:00
configuration.secret_access_key = arg_value->as<ASTLiteral>()->value.safeGet<String>();
2021-09-07 11:17:25 +00:00
else
2021-09-09 09:18:08 +00:00
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Unknown key-value argument `{}` for StorageS3, expected: url, [access_key_id, secret_access_key], name of used format and [compression_method].",
arg_name);
2021-09-07 11:17:25 +00:00
}
}
else
{
2022-01-14 13:27:57 +00:00
if (engine_args.empty() || engine_args.size() > 5)
2019-06-17 00:06:14 +00:00
throw Exception(
"Storage S3 requires 1 to 5 arguments: url, [access_key_id, secret_access_key], name of used format and [compression_method].",
2021-04-08 00:09:15 +00:00
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
2019-06-17 00:06:14 +00:00
2020-03-09 01:22:33 +00:00
for (auto & engine_arg : engine_args)
2021-09-07 11:17:25 +00:00
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, local_context);
2021-09-09 09:18:08 +00:00
configuration.url = engine_args[0]->as<ASTLiteral &>().value.safeGet<String>();
2021-09-07 11:17:25 +00:00
if (engine_args.size() >= 4)
{
configuration.access_key_id = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
configuration.secret_access_key = engine_args[2]->as<ASTLiteral &>().value.safeGet<String>();
}
if (engine_args.size() == 3 || engine_args.size() == 5)
{
configuration.compression_method = engine_args.back()->as<ASTLiteral &>().value.safeGet<String>();
configuration.format = engine_args[engine_args.size() - 2]->as<ASTLiteral &>().value.safeGet<String>();
}
else if (engine_args.size() != 1)
2021-09-07 11:17:25 +00:00
{
configuration.compression_method = "auto";
configuration.format = engine_args.back()->as<ASTLiteral &>().value.safeGet<String>();
}
}
2019-06-17 00:06:14 +00:00
if (configuration.format == "auto")
configuration.format = FormatFactory::instance().getFormatFromFileName(configuration.url, true);
2021-09-07 11:17:25 +00:00
return configuration;
}
ColumnsDescription StorageS3::getTableStructureFromData(
const String & format,
const S3::URI & uri,
const String & access_key_id,
const String & secret_access_key,
UInt64 max_connections,
UInt64 max_single_read_retries,
const String & compression_method,
bool distributed_processing,
const std::optional<FormatSettings> & format_settings,
ContextPtr ctx)
{
ClientAuthentication client_auth{uri, access_key_id, secret_access_key, max_connections, {}, {}};
updateClientAndAuthSettings(ctx, client_auth);
return getTableStructureFromDataImpl(format, client_auth, max_single_read_retries, compression_method, distributed_processing, uri.key.find_first_of("*?{") != std::string::npos, format_settings, ctx);
}
ColumnsDescription StorageS3::getTableStructureFromDataImpl(
const String & format,
const ClientAuthentication & client_auth,
UInt64 max_single_read_retries,
const String & compression_method,
bool distributed_processing,
bool is_key_with_globs,
const std::optional<FormatSettings> & format_settings,
ContextPtr ctx)
{
2022-01-18 19:26:13 +00:00
std::vector<String> keys = {client_auth.uri.key};
auto read_buffer_creator = [&]()
{
2022-01-18 19:26:13 +00:00
auto file_iterator = createFileIterator(client_auth, keys, is_key_with_globs, distributed_processing, ctx);
String current_key = (*file_iterator)();
if (current_key.empty())
throw Exception(
ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE,
"Cannot extract table structure from {} format file, because there are no files with provided path in S3. You must specify "
"table structure manually",
format);
return wrapReadBufferWithCompressionMethod(
2021-12-17 15:34:13 +00:00
std::make_unique<ReadBufferFromS3>(client_auth.client, client_auth.uri.bucket, current_key, max_single_read_retries, ctx->getReadSettings()),
chooseCompressionMethod(current_key, compression_method));
};
return readSchemaFromFormat(format, format_settings, read_buffer_creator, ctx);
}
2019-06-17 00:06:14 +00:00
2021-09-07 11:17:25 +00:00
void registerStorageS3Impl(const String & name, StorageFactory & factory)
{
factory.registerStorage(name, [](const StorageFactory::Arguments & args)
{
auto & engine_args = args.engine_args;
if (engine_args.empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments");
2021-09-15 18:11:49 +00:00
auto configuration = StorageS3::getConfiguration(engine_args, args.getLocalContext());
2021-08-23 19:05:28 +00:00
// Use format settings from global server context + settings from
// the SETTINGS clause of the create query. Settings from current
// session and user are ignored.
std::optional<FormatSettings> format_settings;
if (args.storage_def->settings)
{
FormatFactorySettings user_format_settings;
// Apply changed settings from global context, but ignore the
// unknown ones, because we only have the format settings here.
const auto & changes = args.getContext()->getSettingsRef().changes();
for (const auto & change : changes)
{
if (user_format_settings.has(change.name))
user_format_settings.set(change.name, change.value);
}
// Apply changes from SETTINGS clause, with validation.
user_format_settings.applyChanges(args.storage_def->settings->changes);
format_settings = getFormatSettings(args.getContext(), user_format_settings);
}
else
{
format_settings = getFormatSettings(args.getContext());
}
2021-09-07 11:17:25 +00:00
S3::URI s3_uri(Poco::URI(configuration.url));
auto max_single_read_retries = args.getLocalContext()->getSettingsRef().s3_max_single_read_retries;
auto min_upload_part_size = args.getLocalContext()->getSettingsRef().s3_min_upload_part_size;
auto upload_part_size_multiply_factor = args.getLocalContext()->getSettingsRef().s3_upload_part_size_multiply_factor;
auto upload_part_size_multiply_parts_count_threshold = args.getLocalContext()->getSettingsRef().s3_upload_part_size_multiply_parts_count_threshold;
2021-09-07 11:17:25 +00:00
auto max_single_part_upload_size = args.getLocalContext()->getSettingsRef().s3_max_single_part_upload_size;
2021-09-07 11:17:25 +00:00
auto max_connections = args.getLocalContext()->getSettingsRef().s3_max_connections;
2021-10-26 12:22:13 +00:00
ASTPtr partition_by;
if (args.storage_def->partition_by)
partition_by = args.storage_def->partition_by->clone();
return StorageS3::create(
s3_uri,
2021-09-07 11:17:25 +00:00
configuration.access_key_id,
configuration.secret_access_key,
args.table_id,
2021-09-07 11:17:25 +00:00
configuration.format,
2021-05-19 21:42:25 +00:00
max_single_read_retries,
min_upload_part_size,
upload_part_size_multiply_factor,
upload_part_size_multiply_parts_count_threshold,
max_single_part_upload_size,
max_connections,
args.columns,
args.constraints,
2021-04-23 12:18:23 +00:00
args.comment,
args.getContext(),
2021-08-23 19:05:28 +00:00
format_settings,
2021-10-26 12:22:13 +00:00
configuration.compression_method,
/* distributed_processing_ */false,
partition_by);
},
{
2021-08-23 19:05:28 +00:00
.supports_settings = true,
2021-10-26 12:22:13 +00:00
.supports_sort_order = true, // for partition by
2021-12-17 15:34:13 +00:00
.supports_schema_inference = true,
.source_access_type = AccessType::S3,
2019-06-17 00:06:14 +00:00
});
2019-05-23 09:03:39 +00:00
}
2019-12-11 14:21:48 +00:00
void registerStorageS3(StorageFactory & factory)
{
return registerStorageS3Impl("S3", factory);
}
void registerStorageCOS(StorageFactory & factory)
{
return registerStorageS3Impl("COSN", factory);
}
NamesAndTypesList StorageS3::getVirtuals() const
2020-04-27 13:55:30 +00:00
{
return NamesAndTypesList{
2022-02-05 08:10:28 +00:00
{"_path", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())},
{"_file", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())}};
2020-04-27 13:55:30 +00:00
}
2021-07-14 08:49:05 +00:00
bool StorageS3::supportsPartitionBy() const
{
return true;
}
2019-05-23 09:03:39 +00:00
}
2019-12-11 14:21:48 +00:00
#endif