ClickHouse/src/Storages/StorageS3.cpp

1690 lines
60 KiB
C++
Raw Normal View History

#include "config.h"
2022-09-12 08:01:17 +00:00
#include <Common/ProfileEvents.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>
#include <IO/S3/Requests.h>
#include <IO/ParallelReadBuffer.h>
#include <IO/SharedThreadPools.h>
2019-05-23 09:03:39 +00:00
2021-05-25 06:45:30 +00:00
#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
2021-06-01 12:41:23 +00:00
#include <Storages/StorageFactory.h>
#include <Storages/StorageS3.h>
#include <Storages/StorageS3Settings.h>
#include <Storages/StorageSnapshot.h>
2021-10-26 09:31:01 +00:00
#include <Storages/PartitionedSink.h>
2022-05-19 11:18:58 +00:00
#include <Storages/VirtualColumnUtils.h>
2022-03-28 19:18:20 +00:00
#include <Storages/getVirtualsForStorage.h>
#include <Storages/checkAndGetLiteralArgument.h>
2022-09-13 13:07:43 +00:00
#include <Storages/StorageURL.h>
#include <Storages/NamedCollectionsHelpers.h>
#include <Storages/ReadFromStorageProgress.h>
2019-05-23 09:03:39 +00:00
2023-05-22 17:51:58 +00:00
#include <Disks/IO/AsynchronousBoundedReadBuffer.h>
2022-11-26 03:24:11 +00:00
#include <Disks/IO/ReadBufferFromRemoteFSGather.h>
#include <Disks/ObjectStorages/StoredObject.h>
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>
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
2020-01-26 14:05:51 +00:00
#include <DataTypes/DataTypeString.h>
#include <aws/core/auth/AWSCredentials.h>
2020-01-26 13:03:47 +00:00
#include <Common/NamedCollections/NamedCollections.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>
#include <Common/CurrentMetrics.h>
2020-01-26 13:03:47 +00:00
#include <re2/re2.h>
2019-05-23 09:03:39 +00:00
2022-05-20 19:49:31 +00:00
#include <Processors/ISource.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
#include <boost/algorithm/string.hpp>
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
namespace CurrentMetrics
{
extern const Metric StorageS3Threads;
extern const Metric StorageS3ThreadsActive;
}
2022-09-12 08:01:17 +00:00
namespace ProfileEvents
{
2022-09-19 17:23:22 +00:00
extern const Event S3DeleteObjects;
extern const Event S3ListObjects;
2022-09-12 08:01:17 +00:00
}
2019-05-23 09:03:39 +00:00
namespace DB
{
2021-07-18 11:03:45 +00:00
static const std::unordered_set<std::string_view> required_configuration_keys = {
"url",
};
static const std::unordered_set<std::string_view> optional_configuration_keys = {
"format",
"compression",
2022-12-15 08:53:44 +00:00
"compression_method",
"structure",
"access_key_id",
"secret_access_key",
"filename",
"use_environment_credentials",
"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",
2023-03-29 11:08:44 +00:00
"expiration_window_seconds",
"no_sign_request"
};
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;
2022-05-19 11:18:58 +00:00
extern const int NOT_IMPLEMENTED;
extern const int CANNOT_COMPILE_REGEXP;
extern const int FILE_DOESNT_EXIST;
2019-05-23 09:03:39 +00:00
}
2021-10-11 16:11:50 +00:00
class IOutputFormat;
using OutputFormatPtr = std::shared_ptr<IOutputFormat>;
2022-11-28 13:56:25 +00:00
static void addPathToVirtualColumns(Block & block, const String & path, size_t idx)
2021-04-06 19:18:45 +00:00
{
2022-11-28 13:56:25 +00:00
if (block.has("_path"))
block.getByName("_path").column->assumeMutableRef().insert(path);
if (block.has("_file"))
2022-12-05 14:43:41 +00:00
{
auto pos = path.find_last_of('/');
assert(pos != std::string::npos);
auto file = path.substr(pos + 1);
2022-11-28 13:56:25 +00:00
block.getByName("_file").column->assumeMutableRef().insert(file);
2022-12-05 14:43:41 +00:00
}
2022-11-28 13:56:25 +00:00
block.getByName("_idx").column->assumeMutableRef().insert(idx);
}
2021-04-06 19:18:45 +00:00
2022-11-28 13:56:25 +00:00
class StorageS3Source::DisclosedGlobIterator::Impl : WithContext
{
2021-04-06 19:18:45 +00:00
public:
Impl(
const S3::Client & client_,
const S3::URI & globbed_uri_,
ASTPtr & query_,
const Block & virtual_header_,
ContextPtr context_,
2023-03-29 22:09:17 +00:00
KeysWithInfo * read_keys_,
const S3Settings::RequestSettings & request_settings_)
: WithContext(context_)
, client(S3::Client::create(client_))
, globbed_uri(globbed_uri_)
, query(query_)
, virtual_header(virtual_header_)
, read_keys(read_keys_)
, request_settings(request_settings_)
, list_objects_pool(CurrentMetrics::StorageS3Threads, CurrentMetrics::StorageS3ThreadsActive, 1)
2022-12-05 14:43:41 +00:00
, list_objects_scheduler(threadPoolCallbackRunner<ListObjectsOutcome>(list_objects_pool, "ListObjects"))
2021-04-12 17:07:01 +00:00
{
2021-04-06 19:18:45 +00:00
if (globbed_uri.bucket.find_first_of("*?{") != globbed_uri.bucket.npos)
throw Exception(ErrorCodes::UNEXPECTED_EXPRESSION, "Expression can not have wildcards inside bucket name");
2021-04-06 19:18:45 +00:00
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
{
2022-11-26 03:24:11 +00:00
buffer.emplace_back(globbed_uri.key, std::nullopt);
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);
request.SetMaxKeys(static_cast<int>(request_settings.list_object_keys_size));
2022-11-26 03:24:11 +00:00
outcome_future = listObjectsAsync();
2021-04-06 19:18:45 +00:00
matcher = std::make_unique<re2::RE2>(makeRegexpPatternFromGlobs(globbed_uri.key));
if (!matcher->ok())
throw Exception(ErrorCodes::CANNOT_COMPILE_REGEXP,
"Cannot compile regex from glob ({}): {}", globbed_uri.key, matcher->error());
recursive = globbed_uri.key == "/**" ? true : false;
2021-04-10 14:58:29 +00:00
fillInternalBufferAssumeLocked();
2021-04-06 19:18:45 +00:00
}
2022-11-26 03:24:11 +00:00
KeyWithInfo next()
2021-04-10 02:21:18 +00:00
{
std::lock_guard lock(mutex);
return nextAssumeLocked();
}
size_t getTotalSize() const
{
2023-01-02 14:18:43 +00:00
return total_size.load(std::memory_order_relaxed);
}
2022-11-26 03:24:11 +00:00
~Impl()
{
list_objects_pool.wait();
}
2021-04-10 02:21:18 +00:00
private:
2022-12-05 14:43:41 +00:00
using ListObjectsOutcome = Aws::S3::Model::ListObjectsV2Outcome;
2021-04-10 02:21:18 +00:00
2022-11-26 03:24:11 +00:00
KeyWithInfo 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
try
{
fillInternalBufferAssumeLocked();
}
catch (...)
{
/// In case of exception thrown while listing new batch of files
/// iterator may be partially initialized and its further using may lead to UB.
/// Iterator is used by several processors from several threads and
/// it may take some time for threads to stop processors and they
/// may still use this iterator after exception is thrown.
/// To avoid this UB, reset the buffer and return defaults for further calls.
is_finished = true;
buffer.clear();
buffer_iter = buffer.begin();
throw;
}
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();
2022-11-26 03:24:11 +00:00
assert(outcome_future.valid());
auto outcome = outcome_future.get();
2021-04-06 19:18:45 +00:00
if (!outcome.IsSuccess())
{
2021-04-06 19:18:45 +00:00
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()));
}
2021-04-06 19:18:45 +00:00
const auto & result_batch = outcome.GetResult().GetContents();
2022-11-28 13:56:25 +00:00
/// It returns false when all objects were returned
2022-11-26 03:24:11 +00:00
is_finished = !outcome.GetResult().GetIsTruncated();
2022-05-19 11:18:58 +00:00
2022-11-26 03:24:11 +00:00
if (!is_finished)
{
/// Even if task is finished the thread may be not freed in pool.
/// So wait until it will be freed before scheduling a new task.
list_objects_pool.wait();
2022-11-26 03:24:11 +00:00
outcome_future = listObjectsAsync();
}
2022-05-19 11:18:58 +00:00
2023-02-04 14:11:14 +00:00
if (request_settings.throw_on_zero_files_match && result_batch.empty())
throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Can not match any files using prefix {}", request.GetPrefix());
2022-11-28 13:56:25 +00:00
KeysWithInfo temp_buffer;
temp_buffer.reserve(result_batch.size());
2022-05-19 11:18:58 +00:00
2022-11-28 13:56:25 +00:00
for (const auto & row : result_batch)
2021-04-06 19:18:45 +00:00
{
2022-11-28 13:56:25 +00:00
String key = row.GetKey();
if (recursive || re2::RE2::FullMatch(key, *matcher))
2022-05-19 11:18:58 +00:00
{
2022-11-28 13:56:25 +00:00
S3::ObjectInfo info =
2022-05-19 11:18:58 +00:00
{
2022-11-28 13:56:25 +00:00
.size = size_t(row.GetSize()),
.last_modification_time = row.GetLastModified().Millis() / 1000,
};
2022-05-19 11:18:58 +00:00
2022-11-28 13:56:25 +00:00
temp_buffer.emplace_back(std::move(key), std::move(info));
2022-12-05 17:32:56 +00:00
}
2021-04-06 19:18:45 +00:00
}
2022-05-19 11:18:58 +00:00
if (temp_buffer.empty())
2022-11-29 17:33:35 +00:00
{
buffer_iter = buffer.begin();
return;
2022-11-29 17:33:35 +00:00
}
2022-05-19 11:18:58 +00:00
2022-12-05 14:43:41 +00:00
if (!is_initialized)
{
createFilterAST(temp_buffer.front().key);
2022-12-05 14:43:41 +00:00
is_initialized = true;
}
2022-11-26 03:24:11 +00:00
2022-12-05 14:43:41 +00:00
if (filter_ast)
2021-04-06 19:18:45 +00:00
{
2022-11-28 13:56:25 +00:00
auto block = virtual_header.cloneEmpty();
for (size_t i = 0; i < temp_buffer.size(); ++i)
addPathToVirtualColumns(block, fs::path(globbed_uri.bucket) / temp_buffer[i].key, i);
2022-05-19 11:18:58 +00:00
2022-12-05 14:43:41 +00:00
VirtualColumnUtils::filterBlockWithQuery(query, block, getContext(), filter_ast);
2022-11-26 03:24:11 +00:00
const auto & idxs = typeid_cast<const ColumnUInt64 &>(*block.getByName("_idx").column);
2022-11-28 13:56:25 +00:00
buffer.reserve(block.rows());
for (UInt64 idx : idxs.getData())
2022-05-19 11:18:58 +00:00
{
2023-01-02 14:18:43 +00:00
total_size.fetch_add(temp_buffer[idx].info->size, std::memory_order_relaxed);
2022-11-26 03:24:11 +00:00
buffer.emplace_back(std::move(temp_buffer[idx]));
2022-05-19 11:18:58 +00:00
}
2021-04-06 19:18:45 +00:00
}
2022-05-19 11:18:58 +00:00
else
2021-04-06 19:18:45 +00:00
{
2022-11-28 13:56:25 +00:00
buffer = std::move(temp_buffer);
for (const auto & [_, info] : buffer)
2023-01-02 14:18:43 +00:00
total_size.fetch_add(info->size, std::memory_order_relaxed);
2021-04-06 19:18:45 +00:00
}
2022-05-19 11:18:58 +00:00
2021-04-06 19:18:45 +00:00
/// Set iterator only after the whole batch is processed
buffer_iter = buffer.begin();
2022-11-26 03:24:11 +00:00
if (read_keys)
2023-03-29 22:09:17 +00:00
read_keys->insert(read_keys->end(), buffer.begin(), buffer.end());
2022-11-26 03:24:11 +00:00
}
2021-04-12 17:07:01 +00:00
2022-11-28 13:56:25 +00:00
void createFilterAST(const String & any_key)
{
if (!query || !virtual_header)
return;
2022-11-28 13:56:25 +00:00
/// Create a virtual block with one row to construct filter
/// Append "idx" column as the filter result
virtual_header.insert({ColumnUInt64::create(), std::make_shared<DataTypeUInt64>(), "_idx"});
auto block = virtual_header.cloneEmpty();
addPathToVirtualColumns(block, fs::path(globbed_uri.bucket) / any_key, 0);
2022-12-05 14:43:41 +00:00
VirtualColumnUtils::prepareFilterBlockWithQuery(query, getContext(), block, filter_ast);
2022-11-28 13:56:25 +00:00
}
2022-12-05 14:43:41 +00:00
std::future<ListObjectsOutcome> listObjectsAsync()
2022-11-26 03:24:11 +00:00
{
return list_objects_scheduler([this]
{
ProfileEvents::increment(ProfileEvents::S3ListObjects);
auto outcome = client->ListObjectsV2(request);
2022-11-26 03:24:11 +00:00
/// Outcome failure will be handled on the caller side.
if (outcome.IsSuccess())
request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken());
return outcome;
}, Priority{});
2021-04-06 19:18:45 +00:00
}
2021-04-10 02:21:18 +00:00
std::mutex mutex;
2022-11-26 03:24:11 +00:00
KeysWithInfo buffer;
KeysWithInfo::iterator buffer_iter;
std::unique_ptr<S3::Client> client;
2021-04-06 19:18:45 +00:00
S3::URI globbed_uri;
2022-05-19 11:18:58 +00:00
ASTPtr query;
Block virtual_header;
2022-12-05 14:43:41 +00:00
bool is_initialized{false};
2022-05-19 11:18:58 +00:00
ASTPtr filter_ast;
2021-04-06 19:18:45 +00:00
std::unique_ptr<re2::RE2> matcher;
bool recursive{false};
2021-04-06 19:18:45 +00:00
bool is_finished{false};
2023-03-29 22:09:17 +00:00
KeysWithInfo * read_keys;
2022-11-26 03:24:11 +00:00
S3::ListObjectsV2Request request;
S3Settings::RequestSettings request_settings;
2022-11-26 03:24:11 +00:00
ThreadPool list_objects_pool;
2022-12-05 14:43:41 +00:00
ThreadPoolCallbackRunner<ListObjectsOutcome> list_objects_scheduler;
std::future<ListObjectsOutcome> outcome_future;
2023-01-02 14:18:43 +00:00
std::atomic<size_t> total_size = 0;
2021-04-06 19:18:45 +00:00
};
2022-05-19 11:18:58 +00:00
StorageS3Source::DisclosedGlobIterator::DisclosedGlobIterator(
const S3::Client & client_,
2022-05-25 19:47:05 +00:00
const S3::URI & globbed_uri_,
ASTPtr query,
const Block & virtual_header,
ContextPtr context,
2023-03-29 22:09:17 +00:00
KeysWithInfo * read_keys_,
const S3Settings::RequestSettings & request_settings_)
2023-03-29 22:09:17 +00:00
: pimpl(std::make_shared<StorageS3Source::DisclosedGlobIterator::Impl>(client_, globbed_uri_, query, virtual_header, context, read_keys_, request_settings_))
2022-05-19 11:18:58 +00:00
{
}
2021-04-06 19:18:45 +00:00
2022-11-26 03:24:11 +00:00
StorageS3Source::KeyWithInfo StorageS3Source::DisclosedGlobIterator::next()
2021-04-06 19:18:45 +00:00
{
return pimpl->next();
}
size_t StorageS3Source::DisclosedGlobIterator::getTotalSize() const
{
return pimpl->getTotalSize();
}
2022-05-19 11:18:58 +00:00
class StorageS3Source::KeysIterator::Impl : WithContext
{
public:
2022-05-19 11:18:58 +00:00
explicit Impl(
const S3::Client & client_,
const std::string & version_id_,
const std::vector<String> & keys_,
const String & bucket_,
const S3Settings::RequestSettings & request_settings_,
ASTPtr query_,
const Block & virtual_header_,
ContextPtr context_,
bool need_total_size,
2023-03-29 22:09:17 +00:00
KeysWithInfo * read_keys_)
: WithContext(context_)
, bucket(bucket_)
, query(query_)
, virtual_header(virtual_header_)
{
2022-12-14 13:04:24 +00:00
Strings all_keys = keys_;
2022-05-19 11:18:58 +00:00
/// Create a virtual block with one row to construct filter
if (query && virtual_header && !all_keys.empty())
2022-05-19 11:18:58 +00:00
{
2022-11-28 13:56:25 +00:00
/// Append "idx" column as the filter result
virtual_header.insert({ColumnUInt64::create(), std::make_shared<DataTypeUInt64>(), "_idx"});
2022-05-19 11:18:58 +00:00
auto block = virtual_header.cloneEmpty();
addPathToVirtualColumns(block, fs::path(bucket) / all_keys.front(), 0);
2022-05-19 11:18:58 +00:00
ASTPtr filter_ast;
VirtualColumnUtils::prepareFilterBlockWithQuery(query, getContext(), block, filter_ast);
if (filter_ast)
{
block = virtual_header.cloneEmpty();
for (size_t i = 0; i < all_keys.size(); ++i)
addPathToVirtualColumns(block, fs::path(bucket) / all_keys[i], i);
2022-05-19 11:18:58 +00:00
VirtualColumnUtils::filterBlockWithQuery(query, block, getContext(), filter_ast);
2022-11-28 13:56:25 +00:00
const auto & idxs = typeid_cast<const ColumnUInt64 &>(*block.getByName("_idx").column);
2022-05-19 11:18:58 +00:00
Strings filtered_keys;
2022-11-28 13:56:25 +00:00
filtered_keys.reserve(block.rows());
for (UInt64 idx : idxs.getData())
filtered_keys.emplace_back(std::move(all_keys[idx]));
2022-05-19 11:18:58 +00:00
all_keys = std::move(filtered_keys);
2022-05-19 11:18:58 +00:00
}
}
2022-05-19 11:18:58 +00:00
for (auto && key : all_keys)
{
std::optional<S3::ObjectInfo> info;
2023-06-15 08:22:02 +00:00
/// In case all_keys.size() > 1, avoid getting object info now
/// (it will be done anyway eventually, but with delay and in parallel).
/// But progress bar will not work in this case.
2023-06-14 10:59:06 +00:00
if (need_total_size && all_keys.size() == 1)
{
info = S3::getObjectInfo(client_, bucket, key, version_id_, request_settings_);
total_size += info->size;
}
keys.emplace_back(std::move(key), std::move(info));
2022-05-19 11:18:58 +00:00
}
2023-03-29 22:09:17 +00:00
if (read_keys_)
*read_keys_ = keys;
}
2022-11-26 03:24:11 +00:00
KeyWithInfo next()
{
2022-05-02 12:09:54 +00:00
size_t current_index = index.fetch_add(1, std::memory_order_relaxed);
2022-04-19 18:23:04 +00:00
if (current_index >= keys.size())
2022-11-26 03:24:11 +00:00
return {};
2022-04-19 18:23:04 +00:00
return keys[current_index];
}
size_t getTotalSize() const
{
return total_size;
}
private:
KeysWithInfo keys;
2022-04-19 18:23:04 +00:00
std::atomic_size_t index = 0;
2022-05-19 11:18:58 +00:00
String bucket;
ASTPtr query;
Block virtual_header;
size_t total_size = 0;
};
2022-05-19 11:18:58 +00:00
StorageS3Source::KeysIterator::KeysIterator(
const S3::Client & client_,
const std::string & version_id_,
const std::vector<String> & keys_,
const String & bucket_,
const S3Settings::RequestSettings & request_settings_,
ASTPtr query,
const Block & virtual_header,
ContextPtr context,
bool need_total_size,
2023-03-29 22:09:17 +00:00
KeysWithInfo * read_keys)
: pimpl(std::make_shared<StorageS3Source::KeysIterator::Impl>(
client_, version_id_, keys_, bucket_, request_settings_,
query, virtual_header, context, need_total_size, read_keys))
{
}
2022-11-26 03:24:11 +00:00
StorageS3Source::KeyWithInfo StorageS3Source::KeysIterator::next()
{
return pimpl->next();
}
2019-12-03 16:23:24 +00:00
size_t StorageS3Source::KeysIterator::getTotalSize() const
{
return pimpl->getTotalSize();
}
2022-03-28 19:18:20 +00:00
Block StorageS3Source::getHeader(Block sample_block, const std::vector<NameAndTypePair> & requested_virtual_columns)
2019-05-23 09:03:39 +00:00
{
2022-03-28 19:18:20 +00:00
for (const auto & virtual_column : requested_virtual_columns)
sample_block.insert({virtual_column.type->createColumn(), virtual_column.type, virtual_column.name});
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(
2022-03-28 19:18:20 +00:00
const std::vector<NameAndTypePair> & requested_virtual_columns_,
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_,
const S3Settings::RequestSettings & request_settings_,
String compression_hint_,
const std::shared_ptr<const S3::Client> & client_,
2021-04-08 00:09:15 +00:00
const String & bucket_,
const String & version_id_,
std::shared_ptr<IIterator> file_iterator_,
const size_t download_thread_num_)
2022-05-20 19:49:31 +00:00
: ISource(getHeader(sample_block_, requested_virtual_columns_))
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_)
, version_id(version_id_)
2021-04-08 00:09:15 +00:00
, format(format_)
, columns_desc(columns_)
, max_block_size(max_block_size_)
, request_settings(request_settings_)
, compression_hint(std::move(compression_hint_))
2021-04-08 00:09:15 +00:00
, client(client_)
, sample_block(sample_block_)
2021-08-23 19:05:28 +00:00
, format_settings(format_settings_)
2022-03-28 19:18:20 +00:00
, requested_virtual_columns(requested_virtual_columns_)
2021-04-08 00:09:15 +00:00
, file_iterator(file_iterator_)
2022-03-23 08:40:00 +00:00
, download_thread_num(download_thread_num_)
, create_reader_pool(CurrentMetrics::StorageS3Threads, CurrentMetrics::StorageS3ThreadsActive, 1)
2022-11-26 03:24:11 +00:00
, create_reader_scheduler(threadPoolCallbackRunner<ReaderHolder>(create_reader_pool, "CreateS3Reader"))
2021-04-08 00:09:15 +00:00
{
2022-11-26 03:24:11 +00:00
reader = createReader();
if (reader)
{
total_objects_size = tryGetFileSizeFromReadBuffer(*reader.getReadBuffer()).value_or(0);
2022-11-26 03:24:11 +00:00
reader_future = createReaderAsync();
}
2021-04-08 00:09:15 +00:00
}
2022-11-26 03:24:11 +00:00
StorageS3Source::ReaderHolder StorageS3Source::createReader()
2021-03-22 17:12:31 +00:00
{
2023-06-13 14:43:50 +00:00
KeyWithInfo key_with_info;
size_t object_size;
do
{
key_with_info = (*file_iterator)();
if (key_with_info.key.empty())
return {};
2023-06-13 14:43:50 +00:00
object_size = key_with_info.info ? key_with_info.info->size : S3::getObjectSize(*client, bucket, key_with_info.key, version_id, request_settings);
}
while (getContext()->getSettingsRef().s3_skip_empty_files && object_size == 0);
2021-04-13 10:59:02 +00:00
2023-06-13 14:43:50 +00:00
auto compression_method = chooseCompressionMethod(key_with_info.key, compression_hint);
2021-04-08 00:09:15 +00:00
2023-06-13 14:43:50 +00:00
auto read_buf = createS3ReadBuffer(key_with_info.key, object_size);
auto input_format = FormatFactory::instance().getInput(
format, *read_buf, sample_block, getContext(), max_block_size,
format_settings, std::nullopt, std::nullopt,
/* is_remote_fs */ true, compression_method);
2022-03-21 14:52:26 +00:00
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
{
2022-03-23 08:40:00 +00:00
builder.addSimpleTransform(
[&](const Block & header)
{ return std::make_shared<AddingDefaultsTransform>(header, columns_desc, *input_format, getContext()); });
2021-07-20 18:18:43 +00:00
}
2022-11-26 03:24:11 +00:00
auto pipeline = std::make_unique<QueryPipeline>(QueryPipelineBuilder::getPipeline(std::move(builder)));
auto current_reader = std::make_unique<PullingPipelineExecutor>(*pipeline);
2021-04-08 00:09:15 +00:00
return ReaderHolder{fs::path(bucket) / key_with_info.key, std::move(read_buf), input_format, std::move(pipeline), std::move(current_reader)};
2021-03-22 17:12:31 +00:00
}
2022-11-26 03:24:11 +00:00
std::future<StorageS3Source::ReaderHolder> StorageS3Source::createReaderAsync()
2022-03-23 08:40:00 +00:00
{
return create_reader_scheduler([this] { return createReader(); }, Priority{});
2022-11-26 03:24:11 +00:00
}
std::unique_ptr<ReadBuffer> StorageS3Source::createS3ReadBuffer(const String & key, size_t object_size)
2022-11-26 03:24:11 +00:00
{
auto read_settings = getContext()->getReadSettings().adjustBufferSize(object_size);
read_settings.enable_filesystem_cache = false;
2022-03-23 08:40:00 +00:00
auto download_buffer_size = getContext()->getSettings().max_download_buffer_size;
2023-03-28 20:28:28 +00:00
const bool object_too_small = object_size <= 2 * download_buffer_size;
2022-11-26 03:24:11 +00:00
2023-03-28 20:28:28 +00:00
// Create a read buffer that will prefetch the first ~1 MB of the file.
// When reading lots of tiny files, this prefetching almost doubles the throughput.
// For bigger files, parallel reading is more useful.
if (object_too_small && read_settings.remote_fs_method == RemoteFSReadMethod::threadpool)
2022-03-23 08:40:00 +00:00
{
2023-03-28 20:28:28 +00:00
LOG_TRACE(log, "Downloading object of size {} from S3 with initial prefetch", object_size);
return createAsyncS3ReadBuffer(key, read_settings, object_size);
2022-03-23 08:40:00 +00:00
}
return std::make_unique<ReadBufferFromS3>(
client, bucket, key, version_id, request_settings, read_settings,
/*use_external_buffer*/ false, /*offset_*/ 0, /*read_until_position_*/ 0,
/*restricted_seek_*/ false, object_size);
2022-03-23 08:40:00 +00:00
}
2022-11-26 03:24:11 +00:00
std::unique_ptr<ReadBuffer> StorageS3Source::createAsyncS3ReadBuffer(
const String & key, const ReadSettings & read_settings, size_t object_size)
{
auto context = getContext();
2022-11-26 03:24:11 +00:00
auto read_buffer_creator =
2023-03-28 20:28:28 +00:00
[this, read_settings, object_size]
2023-04-17 14:41:21 +00:00
(const std::string & path, size_t read_until_position) -> std::unique_ptr<ReadBufferFromFileBase>
2022-11-26 03:24:11 +00:00
{
2023-04-17 14:41:21 +00:00
return std::make_unique<ReadBufferFromS3>(
2022-11-26 03:24:11 +00:00
client,
bucket,
path,
version_id,
request_settings,
read_settings,
/* use_external_buffer */true,
/* offset */0,
read_until_position,
2023-03-28 20:28:28 +00:00
/* restricted_seek */true,
object_size);
2022-11-26 03:24:11 +00:00
};
auto s3_impl = std::make_unique<ReadBufferFromRemoteFSGather>(
std::move(read_buffer_creator),
StoredObjects{StoredObject{key, object_size}},
read_settings,
2023-06-08 11:39:05 +00:00
/* cache_log */nullptr, /* use_external_buffer */true);
2023-05-03 11:16:08 +00:00
auto modified_settings{read_settings};
/// FIXME: Changing this setting to default value breaks something around parquet reading
modified_settings.remote_read_min_bytes_for_seek = modified_settings.remote_fs_buffer_size;
2022-11-26 03:24:11 +00:00
auto & pool_reader = context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER);
2023-05-22 17:51:58 +00:00
auto async_reader = std::make_unique<AsynchronousBoundedReadBuffer>(
std::move(s3_impl), pool_reader, modified_settings,
context->getAsyncReadCounters(), context->getFilesystemReadPrefetchesLog());
2022-11-26 03:24:11 +00:00
async_reader->setReadUntilEnd();
if (read_settings.remote_fs_prefetch)
2023-02-07 17:50:31 +00:00
async_reader->prefetch(DEFAULT_PREFETCH_PRIORITY);
2022-11-26 03:24:11 +00:00
return async_reader;
}
StorageS3Source::~StorageS3Source()
{
create_reader_pool.wait();
}
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()
{
2022-02-11 18:14:55 +00:00
while (true)
2021-03-22 17:12:31 +00:00
{
if (isCancelled() || !reader)
{
if (reader)
reader->cancel();
2022-02-11 18:14:55 +00:00
break;
}
2021-03-22 17:12:31 +00:00
2022-02-11 18:14:55 +00:00
Chunk chunk;
if (reader->pull(chunk))
2019-05-31 07:27:14 +00:00
{
2022-02-11 18:14:55 +00:00
UInt64 num_rows = chunk.getNumRows();
2022-11-26 03:24:11 +00:00
const auto & file_path = reader.getPath();
if (num_rows && total_objects_size)
{
size_t chunk_size = reader.getFormat()->getApproxBytesReadForChunk();
if (!chunk_size)
chunk_size = chunk.bytes();
updateRowsProgressApprox(*this, num_rows, chunk_size, total_objects_size, total_rows_approx_accumulated, total_rows_count_times, total_rows_approx_max);
}
for (const auto & virtual_column : requested_virtual_columns)
{
if (virtual_column.name == "_path")
{
chunk.addColumn(virtual_column.type->createColumnConst(num_rows, file_path)->convertToFullColumnIfConst());
}
else if (virtual_column.name == "_file")
{
size_t last_slash_pos = file_path.find_last_of('/');
auto column = virtual_column.type->createColumnConst(num_rows, file_path.substr(last_slash_pos + 1));
chunk.addColumn(column->convertToFullColumnIfConst());
}
}
2022-02-11 18:14:55 +00:00
return chunk;
2019-05-31 07:27:14 +00:00
}
2021-03-22 17:12:31 +00:00
assert(reader_future.valid());
reader = reader_future.get();
if (!reader)
break;
size_t object_size = tryGetFileSizeFromReadBuffer(*reader.getReadBuffer()).value_or(0);
/// Adjust total_rows_approx_accumulated with new total size.
if (total_objects_size)
total_rows_approx_accumulated = static_cast<size_t>(
std::ceil(static_cast<double>(total_objects_size + object_size) / total_objects_size * total_rows_approx_accumulated));
total_objects_size += object_size;
/// Even if task is finished the thread may be not freed in pool.
/// So wait until it will be freed before scheduling a new task.
create_reader_pool.wait();
reader_future = createReaderAsync();
}
2022-11-26 03:24:11 +00:00
return {};
}
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,
2023-03-28 13:39:59 +00:00
const StorageS3::Configuration & configuration_,
2021-04-12 17:07:01 +00:00
const String & bucket,
2022-04-03 22:33:59 +00:00
const String & key)
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>(
2023-03-28 13:39:59 +00:00
configuration_.client,
2022-03-30 08:15:20 +00:00
bucket,
key,
DBMS_DEFAULT_BUFFER_SIZE,
2023-03-28 13:39:59 +00:00
configuration_.request_settings,
2022-03-30 08:15:20 +00:00
std::nullopt,
threadPoolCallbackRunner<void>(getIOThreadPool().get(), "S3ParallelWrite"),
2022-07-13 17:48:57 +00:00
context->getWriteSettings()),
2022-03-30 08:15:20 +00:00
compression_method,
3);
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
{
std::lock_guard lock(cancel_mutex);
if (cancelled)
return;
writer->write(getHeader().cloneWithColumns(chunk.detachColumns()));
2021-04-12 17:07:01 +00:00
}
2019-05-31 07:27:14 +00:00
void onCancel() override
{
std::lock_guard lock(cancel_mutex);
finalize();
cancelled = true;
}
void onException() override
{
std::lock_guard lock(cancel_mutex);
2023-06-03 18:59:04 +00:00
release();
}
2021-07-23 19:33:59 +00:00
void onFinish() override
2021-04-12 17:07:01 +00:00
{
std::lock_guard lock(cancel_mutex);
finalize();
}
private:
void finalize()
{
if (!writer)
return;
try
{
2021-11-11 18:09:21 +00:00
writer->finalize();
writer->flush();
write_buf->finalize();
}
catch (...)
{
/// Stop ParallelFormattingOutputFormat correctly.
writer.reset();
2023-04-24 15:11:36 +00:00
write_buf->finalize();
throw;
}
2021-04-12 17:07:01 +00:00
}
2019-05-31 07:27:14 +00:00
2023-06-03 18:59:04 +00:00
void release()
{
if (!writer)
return;
writer.reset();
write_buf.reset();
}
2021-04-12 17:07:01 +00:00
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;
bool cancelled = false;
std::mutex cancel_mutex;
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_,
2023-03-28 13:39:59 +00:00
const StorageS3::Configuration & configuration_,
2021-05-25 06:45:30 +00:00
const String & bucket_,
2022-04-03 22:33:59 +00:00
const String & key_)
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_)
2023-03-28 13:39:59 +00:00
, configuration(configuration_)
2021-05-25 06:45:30 +00:00
, bucket(bucket_)
, key(key_)
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,
2023-03-28 13:39:59 +00:00
configuration,
2021-10-26 09:31:01 +00:00
partition_bucket,
2022-04-03 22:33:59 +00:00
partition_key
2021-10-26 09:31:01 +00:00
);
2021-05-25 06:45:30 +00:00
}
private:
const String format;
const Block sample_block;
2023-03-31 14:08:28 +00:00
const ContextPtr context;
2021-05-25 06:45:30 +00:00
const CompressionMethod compression_method;
const StorageS3::Configuration configuration;
2021-05-25 06:45:30 +00:00
const String bucket;
const String key;
2023-03-31 14:08:28 +00:00
const 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(
2023-03-28 13:39:59 +00:00
const Configuration & configuration_,
2023-03-24 21:35:12 +00:00
ContextPtr context_,
2019-12-04 16:06:55 +00:00
const StorageID & table_id_,
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,
2021-08-23 19:05:28 +00:00
std::optional<FormatSettings> format_settings_,
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_)
2023-03-28 13:39:59 +00:00
, configuration(configuration_)
, name(configuration.url.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_)
2019-09-22 22:13:42 +00:00
{
2023-04-03 17:53:34 +00:00
updateConfiguration(context_);
2023-03-28 13:39:59 +00:00
FormatFactory::instance().checkFormatName(configuration.format);
context_->getGlobalContext()->getRemoteHostFilter().checkURL(configuration.url.uri);
StorageInMemoryMetadata storage_metadata;
if (columns_.empty())
{
auto columns = getTableStructureFromDataImpl(configuration, 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);
2022-03-28 19:18:20 +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);
2022-05-19 11:18:58 +00:00
for (const auto & column : virtual_columns)
virtual_block.insert({column.type->createColumn(), column.type, column.name});
2019-09-22 22:13:42 +00:00
}
std::shared_ptr<StorageS3Source::IIterator> StorageS3::createFileIterator(
2023-03-28 13:39:59 +00:00
const Configuration & configuration,
2022-04-19 18:23:04 +00:00
bool distributed_processing,
ContextPtr local_context,
2022-05-19 11:18:58 +00:00
ASTPtr query,
const Block & virtual_block,
bool need_total_size,
2023-03-29 22:09:17 +00:00
KeysWithInfo * read_keys)
{
if (distributed_processing)
{
return std::make_shared<StorageS3Source::ReadTaskIterator>(local_context->getReadTaskCallback());
}
2023-03-28 13:39:59 +00:00
else if (configuration.withGlobs())
{
/// Iterate through disclosed globs and make a source for each file
return std::make_shared<StorageS3Source::DisclosedGlobIterator>(
2023-03-28 13:39:59 +00:00
*configuration.client, configuration.url, query, virtual_block,
local_context, read_keys, configuration.request_settings);
}
else
{
return std::make_shared<StorageS3Source::KeysIterator>(
2023-04-03 17:53:34 +00:00
*configuration.client, configuration.url.version_id, configuration.keys,
configuration.url.bucket, configuration.request_settings, query,
virtual_block, local_context, need_total_size, read_keys);
}
}
2019-09-22 22:13:42 +00:00
bool StorageS3::supportsSubcolumns() const
{
2023-03-28 13:39:59 +00:00
return FormatFactory::instance().checkIfFormatSupportsSubcolumns(configuration.format);
}
2022-05-13 18:39:19 +00:00
bool StorageS3::supportsSubsetOfColumns() const
2022-02-23 19:31:16 +00:00
{
2023-03-28 13:39:59 +00:00
return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration.format);
2022-02-23 19:31:16 +00:00
}
bool StorageS3::prefersLargeBlocks() const
{
return FormatFactory::instance().checkIfOutputFormatPrefersLargeBlocks(configuration.format);
}
bool StorageS3::parallelizeOutputAfterReading(ContextPtr context) const
{
return FormatFactory::instance().checkParallelizeOutputAfterReading(configuration.format, context);
}
2020-08-03 13:54:14 +00:00
Pipe StorageS3::read(
2019-09-22 22:13:42 +00:00
const Names & column_names,
const StorageSnapshotPtr & storage_snapshot,
2022-05-19 11:18:58 +00:00
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,
size_t num_streams)
2019-05-23 09:03:39 +00:00
{
2023-04-03 17:53:34 +00:00
auto query_configuration = updateConfigurationAndGetCopy(local_context);
2023-03-30 21:06:53 +00:00
2023-04-03 17:53:34 +00:00
if (partition_by && query_configuration.withWildcard())
2022-05-19 11:18:58 +00:00
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Reading from a partitioned S3 storage is not implemented yet");
Pipes pipes;
2022-03-28 19:18:20 +00:00
std::unordered_set<String> column_names_set(column_names.begin(), column_names.end());
std::vector<NameAndTypePair> requested_virtual_columns;
for (const auto & virtual_column : getVirtuals())
2020-01-26 14:05:51 +00:00
{
2022-03-28 19:18:20 +00:00
if (column_names_set.contains(virtual_column.name))
requested_virtual_columns.push_back(virtual_column);
2020-01-26 14:05:51 +00:00
}
std::shared_ptr<StorageS3Source::IIterator> iterator_wrapper = createFileIterator(
query_configuration, distributed_processing, local_context, query_info.query, virtual_block);
2021-04-13 20:19:04 +00:00
2022-02-23 19:31:16 +00:00
ColumnsDescription columns_description;
Block block_for_format;
if (supportsSubsetOfColumns())
2022-02-23 19:31:16 +00:00
{
auto fetch_columns = column_names;
const auto & virtuals = getVirtuals();
std::erase_if(
fetch_columns,
[&](const String & col)
{ return std::any_of(virtuals.begin(), virtuals.end(), [&](const NameAndTypePair & virtual_col){ return col == virtual_col.name; }); });
if (fetch_columns.empty())
2023-01-10 11:22:33 +00:00
fetch_columns.push_back(ExpressionActions::getSmallestColumn(storage_snapshot->metadata->getColumns().getAllPhysical()).name);
columns_description = storage_snapshot->getDescriptionForColumns(fetch_columns);
block_for_format = storage_snapshot->getSampleBlockForColumns(columns_description.getNamesOfPhysical());
}
else
{
columns_description = storage_snapshot->metadata->getColumns();
block_for_format = storage_snapshot->metadata->getSampleBlock();
2022-02-23 19:31:16 +00:00
}
2022-03-23 08:40:00 +00:00
const size_t max_download_threads = local_context->getSettingsRef().max_download_threads;
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>(
2022-03-28 19:18:20 +00:00
requested_virtual_columns,
query_configuration.format,
getName(),
2022-02-23 19:31:16 +00:00
block_for_format,
local_context,
2021-08-23 19:05:28 +00:00
format_settings,
2022-02-23 19:31:16 +00:00
columns_description,
max_block_size,
query_configuration.request_settings,
query_configuration.compression_method,
query_configuration.client,
query_configuration.url.bucket,
query_configuration.url.version_id,
2022-03-23 08:40:00 +00:00
iterator_wrapper,
max_download_threads));
2021-04-06 19:18:45 +00:00
}
2023-04-18 22:41:28 +00:00
return Pipe::unitePipes(std::move(pipes));
2019-05-23 09:03:39 +00:00
}
SinkToStoragePtr StorageS3::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/)
2019-05-23 09:03:39 +00:00
{
2023-04-03 17:53:34 +00:00
auto query_configuration = updateConfigurationAndGetCopy(local_context);
2021-05-25 06:45:30 +00:00
auto sample_block = metadata_snapshot->getSampleBlock();
2023-04-03 17:53:34 +00:00
auto chosen_compression_method = chooseCompressionMethod(query_configuration.keys.back(), query_configuration.compression_method);
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;
2023-04-03 17:53:34 +00:00
bool is_partitioned_implementation = partition_by_ast && query_configuration.withWildcard();
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,
2023-04-03 17:53:34 +00:00
query_configuration.format,
2021-05-25 06:45:30 +00:00
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,
query_configuration,
query_configuration.url.bucket,
query_configuration.keys.back());
2021-05-25 06:45:30 +00:00
}
else
{
2023-04-03 17:53:34 +00:00
if (query_configuration.withGlobs())
throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED,
"S3 key '{}' contains globs, so the table is in readonly mode", query_configuration.url.key);
2022-01-14 18:18:16 +00:00
bool truncate_in_insert = local_context->getSettingsRef().s3_truncate_on_insert;
if (!truncate_in_insert && S3::objectExists(*query_configuration.client, query_configuration.url.bucket, query_configuration.keys.back(), query_configuration.url.version_id, query_configuration.request_settings))
2022-01-14 18:18:16 +00:00
{
if (local_context->getSettingsRef().s3_create_new_file_on_insert)
{
2023-04-03 17:53:34 +00:00
size_t index = query_configuration.keys.size();
const auto & first_key = query_configuration.keys[0];
2023-03-28 13:39:59 +00:00
auto pos = first_key.find_first_of('.');
2022-01-14 18:18:16 +00:00
String new_key;
do
{
2023-03-28 13:39:59 +00:00
new_key = first_key.substr(0, pos) + "." + std::to_string(index) + (pos == std::string::npos ? "" : first_key.substr(pos));
2022-01-14 18:18:16 +00:00
++index;
}
while (S3::objectExists(*query_configuration.client, query_configuration.url.bucket, new_key, query_configuration.url.version_id, query_configuration.request_settings));
2023-04-05 12:13:46 +00:00
2023-04-03 17:53:34 +00:00
query_configuration.keys.push_back(new_key);
2023-04-05 12:13:46 +00:00
configuration.keys.push_back(new_key);
2022-01-14 18:18:16 +00:00
}
else
2023-03-28 13:39:59 +00:00
{
2022-01-14 18:18:16 +00:00
throw Exception(
2023-03-28 13:39:59 +00:00
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",
query_configuration.url.bucket, query_configuration.keys.back());
2023-03-28 13:39:59 +00:00
}
2022-01-14 18:18:16 +00:00
}
2021-05-25 06:45:30 +00:00
return std::make_shared<StorageS3Sink>(
2023-04-03 17:53:34 +00:00
query_configuration.format,
2021-05-25 06:45:30 +00:00
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,
query_configuration,
query_configuration.url.bucket,
query_configuration.keys.back());
2021-05-25 06:45:30 +00:00
}
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 &)
{
2023-04-03 17:53:34 +00:00
auto query_configuration = updateConfigurationAndGetCopy(local_context);
2021-06-21 15:44:36 +00:00
2023-04-03 17:53:34 +00:00
if (query_configuration.withGlobs())
2023-03-24 21:35:12 +00:00
{
throw Exception(
ErrorCodes::DATABASE_ACCESS_DENIED,
"S3 key '{}' contains globs, so the table is in readonly mode",
query_configuration.url.key);
2023-03-24 21:35:12 +00:00
}
2021-06-21 15:44:36 +00:00
Aws::S3::Model::Delete delkeys;
2023-04-03 17:53:34 +00:00
for (const auto & key : query_configuration.keys)
{
Aws::S3::Model::ObjectIdentifier obj;
obj.SetKey(key);
delkeys.AddObjects(std::move(obj));
}
2021-06-21 15:44:36 +00:00
2022-09-19 17:23:22 +00:00
ProfileEvents::increment(ProfileEvents::S3DeleteObjects);
S3::DeleteObjectsRequest request;
request.SetBucket(query_configuration.url.bucket);
2021-06-21 15:44:36 +00:00
request.SetDelete(delkeys);
auto response = query_configuration.client->DeleteObjects(request);
2021-06-21 15:44:36 +00:00
if (!response.IsSuccess())
{
const auto & err = response.GetError();
throw Exception(ErrorCodes::S3_ERROR, "{}: {}", std::to_string(static_cast<int>(err.GetErrorType())), err.GetMessage());
2021-06-21 15:44:36 +00:00
}
for (const auto & error : response.GetResult().GetErrors())
LOG_WARNING(&Poco::Logger::get("StorageS3"), "Failed to delete {}, error: {}", error.GetKey(), error.GetMessage());
2021-06-21 15:44:36 +00:00
}
2023-04-03 17:53:34 +00:00
StorageS3::Configuration StorageS3::updateConfigurationAndGetCopy(ContextPtr local_context)
{
std::lock_guard lock(configuration_update_mutex);
configuration.update(local_context);
return configuration;
}
void StorageS3::updateConfiguration(ContextPtr local_context)
2023-03-28 13:39:59 +00:00
{
2023-04-03 17:53:34 +00:00
std::lock_guard lock(configuration_update_mutex);
configuration.update(local_context);
2023-03-28 13:39:59 +00:00
}
void StorageS3::useConfiguration(const Configuration & new_configuration)
{
2023-04-03 17:53:34 +00:00
std::lock_guard lock(configuration_update_mutex);
2023-03-28 13:39:59 +00:00
configuration = new_configuration;
}
2021-06-21 15:44:36 +00:00
2023-04-03 17:53:34 +00:00
const StorageS3::Configuration & StorageS3::getConfiguration()
{
std::lock_guard lock(configuration_update_mutex);
return configuration;
}
2023-03-24 21:35:12 +00:00
bool StorageS3::Configuration::update(ContextPtr context)
{
2023-03-28 13:39:59 +00:00
auto s3_settings = context->getStorageS3Settings().getSettings(url.uri.toString());
request_settings = s3_settings.request_settings;
request_settings.updateFromSettings(context->getSettings());
2022-09-13 13:07:43 +00:00
2023-03-28 13:39:59 +00:00
if (client && (static_configuration || s3_settings.auth_settings == auth_settings))
return false;
2023-03-28 13:39:59 +00:00
auth_settings.updateFrom(s3_settings.auth_settings);
2023-04-05 12:13:46 +00:00
keys[0] = url.key;
2023-03-28 13:39:59 +00:00
connect(context);
return true;
}
2022-09-13 13:07:43 +00:00
2023-03-28 13:39:59 +00:00
void StorageS3::Configuration::connect(ContextPtr context)
{
S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration(
2023-03-24 21:35:12 +00:00
auth_settings.region,
context->getRemoteHostFilter(),
static_cast<unsigned>(context->getGlobalContext()->getSettingsRef().s3_max_redirects),
context->getGlobalContext()->getSettingsRef().enable_s3_requests_logging,
/* for_disk_s3 = */ false,
2023-03-24 21:35:12 +00:00
request_settings.get_request_throttler,
request_settings.put_request_throttler);
2023-03-24 21:35:12 +00:00
client_configuration.endpointOverride = url.endpoint;
client_configuration.maxConnections = static_cast<unsigned>(request_settings.max_connections);
auto headers = auth_settings.headers;
if (!headers_from_ast.empty())
headers.insert(headers.end(), headers_from_ast.begin(), headers_from_ast.end());
2022-09-21 14:36:47 +00:00
2023-05-21 10:42:28 +00:00
client_configuration.requestTimeoutMs = request_settings.request_timeout_ms;
client_configuration.retryStrategy
= std::make_shared<Aws::Client::DefaultRetryStrategy>(request_settings.retry_attempts);
2023-03-28 13:39:59 +00:00
auto credentials = Aws::Auth::AWSCredentials(auth_settings.access_key_id, auth_settings.secret_access_key);
2023-03-24 21:35:12 +00:00
client = S3::ClientFactory::instance().create(
client_configuration,
2023-03-24 21:35:12 +00:00
url.is_virtual_hosted_style,
credentials.GetAWSAccessKeyId(),
credentials.GetAWSSecretKey(),
2023-03-24 21:35:12 +00:00
auth_settings.server_side_encryption_customer_key_base64,
auth_settings.server_side_encryption_kms_config,
std::move(headers),
S3::CredentialsConfiguration{
2023-05-21 10:42:28 +00:00
auth_settings.use_environment_credentials.value_or(context->getConfigRef().getBool("s3.use_environment_credentials", true)),
auth_settings.use_insecure_imds_request.value_or(context->getConfigRef().getBool("s3.use_insecure_imds_request", false)),
auth_settings.expiration_window_seconds.value_or(
context->getConfigRef().getUInt64("s3.expiration_window_seconds", S3::DEFAULT_EXPIRATION_WINDOW_SECONDS)),
auth_settings.no_sign_request.value_or(context->getConfigRef().getBool("s3.no_sign_request", false)),
});
2022-04-03 22:33:59 +00:00
}
void StorageS3::processNamedCollectionResult(StorageS3::Configuration & configuration, const NamedCollection & collection)
2022-04-03 22:33:59 +00:00
{
validateNamedCollection(collection, required_configuration_keys, optional_configuration_keys);
2022-12-20 21:33:54 +00:00
auto filename = collection.getOrDefault<String>("filename", "");
if (!filename.empty())
configuration.url = S3::URI(std::filesystem::path(collection.get<String>("url")) / filename);
else
configuration.url = S3::URI(collection.get<String>("url"));
configuration.auth_settings.access_key_id = collection.getOrDefault<String>("access_key_id", "");
configuration.auth_settings.secret_access_key = collection.getOrDefault<String>("secret_access_key", "");
configuration.auth_settings.use_environment_credentials = collection.getOrDefault<UInt64>("use_environment_credentials", 1);
2023-03-30 06:41:14 +00:00
configuration.auth_settings.no_sign_request = collection.getOrDefault<bool>("no_sign_request", false);
2023-03-29 11:08:44 +00:00
configuration.auth_settings.expiration_window_seconds = collection.getOrDefault<UInt64>("expiration_window_seconds", S3::DEFAULT_EXPIRATION_WINDOW_SECONDS);
configuration.format = collection.getOrDefault<String>("format", configuration.format);
configuration.compression_method = collection.getOrDefault<String>("compression_method", collection.getOrDefault<String>("compression", "auto"));
configuration.structure = collection.getOrDefault<String>("structure", "auto");
configuration.request_settings = S3Settings::RequestSettings(collection);
}
2023-03-28 13:39:59 +00:00
StorageS3::Configuration StorageS3::getConfiguration(ASTs & engine_args, ContextPtr local_context, bool get_format_from_file)
2019-05-23 09:03:39 +00:00
{
2023-03-28 13:39:59 +00:00
StorageS3::Configuration configuration;
2023-02-13 14:45:12 +00:00
2023-03-28 13:39:59 +00:00
if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, local_context))
2019-06-17 00:06:14 +00:00
{
2023-03-28 13:39:59 +00:00
processNamedCollectionResult(configuration, *named_collection);
2021-09-07 11:17:25 +00:00
}
else
{
2023-02-13 14:45:12 +00:00
/// Supported signatures:
///
/// S3('url')
/// S3('url', 'format')
/// S3('url', 'format', 'compression')
/// S3('url', NOSIGN)
/// S3('url', NOSIGN, 'format')
/// S3('url', NOSIGN, 'format', 'compression')
2023-02-13 14:45:12 +00:00
/// S3('url', 'aws_access_key_id', 'aws_secret_access_key')
/// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format')
/// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression')
/// with optional headers() function
if (engine_args.empty() || engine_args.size() > 5)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Storage S3 requires 1 to 5 arguments: "
"url, [NOSIGN | access_key_id, secret_access_key], name of used format and [compression_method]");
2023-02-13 14:45:12 +00:00
2023-03-28 13:39:59 +00:00
auto * header_it = StorageURL::collectHeaders(engine_args, configuration.headers_from_ast, local_context);
2023-02-13 14:45:12 +00:00
if (header_it != engine_args.end())
engine_args.erase(header_it);
for (auto & engine_arg : engine_args)
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, local_context);
2023-02-13 12:27:49 +00:00
/// Size -> argument indexes
2023-02-13 14:45:12 +00:00
static std::unordered_map<size_t, std::unordered_map<std::string_view, size_t>> size_to_engine_args
2021-09-07 11:17:25 +00:00
{
2023-02-13 12:27:49 +00:00
{1, {{}}},
2023-02-13 14:45:12 +00:00
{5, {{"access_key_id", 1}, {"secret_access_key", 2}, {"format", 3}, {"compression_method", 4}}}
2023-02-13 12:27:49 +00:00
};
2023-02-13 14:45:12 +00:00
std::unordered_map<std::string_view, size_t> engine_args_to_idx;
bool no_sign_request = false;
/// For 2 arguments we support 2 possible variants:
/// - s3(source, format)
/// - s3(source, NOSIGN)
/// We can distinguish them by looking at the 2-nd argument: check if it's NOSIGN or not.
if (engine_args.size() == 2)
{
auto second_arg = checkAndGetLiteralArgument<String>(engine_args[1], "format/NOSIGN");
if (boost::iequals(second_arg, "NOSIGN"))
no_sign_request = true;
else
engine_args_to_idx = {{"format", 1}};
}
2023-02-13 12:27:49 +00:00
/// For 3 arguments we support 2 possible variants:
/// - s3(source, format, compression_method)
/// - s3(source, access_key_id, access_key_id)
/// - s3(source, NOSIGN, format)
/// We can distinguish them by looking at the 2-nd argument: check if it's NOSIGN or format name.
else if (engine_args.size() == 3)
2021-09-07 11:17:25 +00:00
{
auto second_arg = checkAndGetLiteralArgument<String>(engine_args[1], "format/access_key_id/NOSIGN");
if (boost::iequals(second_arg, "NOSIGN"))
{
no_sign_request = true;
engine_args_to_idx = {{"format", 2}};
}
else if (second_arg == "auto" || FormatFactory::instance().getAllFormats().contains(second_arg))
2023-02-13 14:45:12 +00:00
engine_args_to_idx = {{"format", 1}, {"compression_method", 2}};
2023-02-13 12:27:49 +00:00
else
2023-02-13 14:45:12 +00:00
engine_args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}};
2021-09-07 11:17:25 +00:00
}
/// For 4 arguments we support 2 possible variants:
/// - s3(source, access_key_id, secret_access_key, format)
/// - s3(source, NOSIGN, format, compression_method)
/// We can distinguish them by looking at the 2-nd argument: check if it's a NOSIGN or not.
else if (engine_args.size() == 4)
{
auto second_arg = checkAndGetLiteralArgument<String>(engine_args[1], "access_key_id/NOSIGN");
if (boost::iequals(second_arg, "NOSIGN"))
{
no_sign_request = true;
engine_args_to_idx = {{"format", 2}, {"compression_method", 3}};
}
else
engine_args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"format", 3}};
}
2023-02-13 12:27:49 +00:00
else
2021-09-07 11:17:25 +00:00
{
2023-02-13 14:45:12 +00:00
engine_args_to_idx = size_to_engine_args[engine_args.size()];
2021-09-07 11:17:25 +00:00
}
2023-02-13 12:27:49 +00:00
/// This argument is always the first
2023-03-28 13:39:59 +00:00
configuration.url = S3::URI(checkAndGetLiteralArgument<String>(engine_args[0], "url"));
2023-02-13 12:27:49 +00:00
2023-02-13 14:45:12 +00:00
if (engine_args_to_idx.contains("format"))
2023-03-28 13:39:59 +00:00
configuration.format = checkAndGetLiteralArgument<String>(engine_args[engine_args_to_idx["format"]], "format");
2023-02-13 12:27:49 +00:00
2023-02-13 14:45:12 +00:00
if (engine_args_to_idx.contains("compression_method"))
2023-03-28 13:39:59 +00:00
configuration.compression_method = checkAndGetLiteralArgument<String>(engine_args[engine_args_to_idx["compression_method"]], "compression_method");
2023-02-13 12:27:49 +00:00
2023-02-13 14:45:12 +00:00
if (engine_args_to_idx.contains("access_key_id"))
2023-03-28 13:39:59 +00:00
configuration.auth_settings.access_key_id = checkAndGetLiteralArgument<String>(engine_args[engine_args_to_idx["access_key_id"]], "access_key_id");
2023-02-13 12:27:49 +00:00
2023-02-13 14:45:12 +00:00
if (engine_args_to_idx.contains("secret_access_key"))
2023-03-28 13:39:59 +00:00
configuration.auth_settings.secret_access_key = checkAndGetLiteralArgument<String>(engine_args[engine_args_to_idx["secret_access_key"]], "secret_access_key");
configuration.auth_settings.no_sign_request = no_sign_request;
2021-09-07 11:17:25 +00:00
}
2019-06-17 00:06:14 +00:00
2023-03-28 13:39:59 +00:00
configuration.static_configuration = !configuration.auth_settings.access_key_id.empty();
configuration.keys = {configuration.url.key};
2023-02-13 14:45:12 +00:00
2023-03-28 13:39:59 +00:00
if (configuration.format == "auto" && get_format_from_file)
configuration.format = FormatFactory::instance().getFormatFromFileName(configuration.url.key, true);
2021-09-07 11:17:25 +00:00
return configuration;
}
ColumnsDescription StorageS3::getTableStructureFromData(
2023-03-30 21:06:53 +00:00
const StorageS3::Configuration & configuration,
const std::optional<FormatSettings> & format_settings,
2023-03-29 22:09:17 +00:00
ContextPtr ctx)
{
2023-04-03 17:53:34 +00:00
return getTableStructureFromDataImpl(configuration, format_settings, ctx);
}
ColumnsDescription StorageS3::getTableStructureFromDataImpl(
2023-03-28 13:39:59 +00:00
const Configuration & configuration,
const std::optional<FormatSettings> & format_settings,
2023-03-29 22:09:17 +00:00
ContextPtr ctx)
{
2023-03-29 22:09:17 +00:00
KeysWithInfo read_keys;
auto file_iterator = createFileIterator(configuration, false, ctx, nullptr, {}, false, &read_keys);
2022-02-09 16:14:14 +00:00
std::optional<ColumnsDescription> columns_from_cache;
size_t prev_read_keys_size = read_keys.size();
2022-08-15 12:33:08 +00:00
if (ctx->getSettingsRef().schema_inference_use_cache_for_s3)
2023-04-03 17:53:34 +00:00
columns_from_cache = tryGetColumnsFromCache(read_keys.begin(), read_keys.end(), configuration, format_settings, ctx);
ReadBufferIterator read_buffer_iterator = [&, first = true](ColumnsDescription & cached_columns) mutable -> std::unique_ptr<ReadBuffer>
{
2023-06-13 14:43:50 +00:00
while (true)
2022-02-09 16:14:14 +00:00
{
2023-06-15 12:59:46 +00:00
auto key_with_info = (*file_iterator)();
2023-06-13 14:43:50 +00:00
if (key_with_info.key.empty())
{
2023-06-13 14:43:50 +00:00
if (first)
throw Exception(
ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE,
"Cannot extract table structure from {} format file, because there are no files with provided path "
"in S3 or all files are empty. You must specify table structure manually",
configuration.format);
return nullptr;
}
2023-06-13 14:43:50 +00:00
/// S3 file iterator could get new keys after new iteration, check them in schema cache.
if (ctx->getSettingsRef().schema_inference_use_cache_for_s3 && read_keys.size() > prev_read_keys_size)
{
2023-06-13 14:43:50 +00:00
columns_from_cache = tryGetColumnsFromCache(read_keys.begin() + prev_read_keys_size, read_keys.end(), configuration, format_settings, ctx);
prev_read_keys_size = read_keys.size();
if (columns_from_cache)
{
cached_columns = *columns_from_cache;
return nullptr;
}
}
2023-06-13 14:43:50 +00:00
if (ctx->getSettingsRef().s3_skip_empty_files && key_with_info.info && key_with_info.info->size == 0)
continue;
int zstd_window_log_max = static_cast<int>(ctx->getSettingsRef().zstd_window_log_max);
2023-06-15 12:59:46 +00:00
auto impl = std::make_unique<ReadBufferFromS3>(configuration.client, configuration.url.bucket, key_with_info.key, configuration.url.version_id, configuration.request_settings, ctx->getReadSettings());
if (!ctx->getSettingsRef().s3_skip_empty_files || !impl->eof())
2023-06-13 14:43:50 +00:00
{
first = false;
2023-06-15 12:59:46 +00:00
return wrapReadBufferWithCompressionMethod(std::move(impl), chooseCompressionMethod(key_with_info.key, configuration.compression_method), zstd_window_log_max);
2023-06-13 14:43:50 +00:00
}
}
};
ColumnsDescription columns;
if (columns_from_cache)
columns = *columns_from_cache;
else
2023-03-28 13:39:59 +00:00
columns = readSchemaFromFormat(configuration.format, format_settings, read_buffer_iterator, configuration.withGlobs(), ctx);
2022-08-15 12:33:08 +00:00
if (ctx->getSettingsRef().schema_inference_use_cache_for_s3)
2023-03-28 13:39:59 +00:00
addColumnsToCache(read_keys, configuration, columns, configuration.format, format_settings, ctx);
return columns;
}
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-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 std::make_shared<StorageS3>(
2023-03-24 21:35:12 +00:00
std::move(configuration),
args.getContext(),
args.table_id,
args.columns,
args.constraints,
2021-04-23 12:18:23 +00:00
args.comment,
2021-08-23 19:05:28 +00:00
format_settings,
2021-10-26 12:22:13 +00:00
/* 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);
}
2022-11-11 08:40:10 +00:00
void registerStorageOSS(StorageFactory & factory)
{
return registerStorageS3Impl("OSS", factory);
}
NamesAndTypesList StorageS3::getVirtuals() const
2020-04-27 13:55:30 +00:00
{
2022-03-28 19:18:20 +00:00
return virtual_columns;
2020-04-27 13:55:30 +00:00
}
2021-07-14 08:49:05 +00:00
bool StorageS3::supportsPartitionBy() const
{
return true;
}
SchemaCache & StorageS3::getSchemaCache(const ContextPtr & ctx)
{
static SchemaCache schema_cache(ctx->getConfigRef().getUInt("schema_inference_cache_max_elements_for_s3", DEFAULT_SCHEMA_CACHE_ELEMENTS));
return schema_cache;
}
std::optional<ColumnsDescription> StorageS3::tryGetColumnsFromCache(
2023-03-29 22:09:17 +00:00
const KeysWithInfo::const_iterator & begin,
const KeysWithInfo::const_iterator & end,
2023-03-28 13:39:59 +00:00
const Configuration & configuration,
const std::optional<FormatSettings> & format_settings,
const ContextPtr & ctx)
{
auto & schema_cache = getSchemaCache(ctx);
for (auto it = begin; it < end; ++it)
{
2023-03-29 22:09:17 +00:00
auto get_last_mod_time = [&]
{
2023-03-29 22:09:17 +00:00
time_t last_modification_time = 0;
if (it->info)
{
last_modification_time = it->info->last_modification_time;
}
else
{
/// Note that in case of exception in getObjectInfo returned info will be empty,
/// but schema cache will handle this case and won't return columns from cache
/// because we can't say that it's valid without last modification time.
2023-03-29 22:09:17 +00:00
last_modification_time = S3::getObjectInfo(
*configuration.client,
2023-04-03 17:53:34 +00:00
configuration.url.bucket,
2023-03-29 22:09:17 +00:00
it->key,
configuration.url.version_id,
configuration.request_settings,
2023-03-29 22:09:17 +00:00
/*with_metadata=*/ false,
/*for_disk_s3=*/ false,
/*throw_on_error= */ false).last_modification_time;
}
2023-03-29 22:09:17 +00:00
return last_modification_time ? std::make_optional(last_modification_time) : std::nullopt;
};
String path = fs::path(configuration.url.bucket) / it->key;
2023-03-28 13:39:59 +00:00
String source = fs::path(configuration.url.uri.getHost() + std::to_string(configuration.url.uri.getPort())) / path;
2023-04-03 17:53:34 +00:00
auto cache_key = getKeyForSchemaCache(source, configuration.format, format_settings, ctx);
auto columns = schema_cache.tryGet(cache_key, get_last_mod_time);
if (columns)
return columns;
}
return std::nullopt;
}
void StorageS3::addColumnsToCache(
2023-03-29 22:09:17 +00:00
const KeysWithInfo & keys,
2023-03-28 13:39:59 +00:00
const Configuration & configuration,
const ColumnsDescription & columns,
const String & format_name,
const std::optional<FormatSettings> & format_settings,
const ContextPtr & ctx)
{
2023-03-28 13:39:59 +00:00
auto host_and_bucket = fs::path(configuration.url.uri.getHost() + std::to_string(configuration.url.uri.getPort())) / configuration.url.bucket;
Strings sources;
sources.reserve(keys.size());
2023-03-29 22:09:17 +00:00
std::transform(keys.begin(), keys.end(), std::back_inserter(sources), [&](const auto & elem){ return host_and_bucket / elem.key; });
2022-08-19 16:42:23 +00:00
auto cache_keys = getKeysForSchemaCache(sources, format_name, format_settings, ctx);
auto & schema_cache = getSchemaCache(ctx);
schema_cache.addMany(cache_keys, columns);
}
2019-05-23 09:03:39 +00:00
}
2019-12-11 14:21:48 +00:00
#endif