ClickHouse/src/Storages/StorageS3.h

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

384 lines
12 KiB
C++
Raw Normal View History

2019-05-23 09:03:39 +00:00
#pragma once
#include "config.h"
2019-12-06 14:37:21 +00:00
#if USE_AWS_S3
2021-03-22 17:12:31 +00:00
#include <Core/Types.h>
#include <Compression/CompressionInfo.h>
2019-05-23 09:03:39 +00:00
#include <Storages/IStorage.h>
#include <Storages/StorageS3Settings.h>
2021-03-22 17:12:31 +00:00
2022-05-20 19:49:31 +00:00
#include <Processors/ISource.h>
2022-11-26 03:24:11 +00:00
#include <Processors/Executors/PullingPipelineExecutor.h>
2019-05-29 12:54:31 +00:00
#include <Poco/URI.h>
2022-04-27 15:05:45 +00:00
#include <Common/logger_useful.h>
2021-03-16 18:41:29 +00:00
#include <IO/S3Common.h>
2021-03-22 17:12:31 +00:00
#include <IO/CompressionMethod.h>
2021-04-08 00:09:15 +00:00
#include <Interpreters/Context.h>
2022-11-26 03:24:11 +00:00
#include <Interpreters/threadPoolCallbackRunner.h>
2021-09-07 11:17:25 +00:00
#include <Storages/ExternalDataSourceConfiguration.h>
#include <Storages/Cache/SchemaCache.h>
2019-12-11 14:21:48 +00:00
namespace Aws::S3
{
class S3Client;
}
2019-09-22 22:13:42 +00:00
2019-05-23 09:03:39 +00:00
namespace DB
{
2019-12-03 16:23:24 +00:00
2021-07-20 18:18:43 +00:00
class PullingPipelineExecutor;
2021-03-22 17:12:31 +00:00
class StorageS3SequentialSource;
class NamedCollection;
2022-05-20 19:49:31 +00:00
class StorageS3Source : public ISource, WithContext
2021-03-22 17:12:31 +00:00
{
public:
2022-11-26 03:24:11 +00:00
struct KeyWithInfo
{
KeyWithInfo() = default;
KeyWithInfo(String key_, std::optional<S3::ObjectInfo> info_)
: key(std::move(key_)), info(std::move(info_))
{
}
String key;
std::optional<S3::ObjectInfo> info;
};
using KeysWithInfo = std::vector<KeyWithInfo>;
using ObjectInfos = std::unordered_map<String, S3::ObjectInfo>;
class IIterator
{
public:
virtual ~IIterator() = default;
virtual KeyWithInfo next() = 0;
virtual size_t getTotalSize() const = 0;
KeyWithInfo operator ()() { return next(); }
};
class DisclosedGlobIterator : public IIterator
2021-04-06 19:18:45 +00:00
{
2022-05-19 11:18:58 +00:00
public:
DisclosedGlobIterator(
2022-05-25 19:47:05 +00:00
const Aws::S3::S3Client & client_,
const S3::URI & globbed_uri_,
ASTPtr query,
const Block & virtual_header,
ContextPtr context,
ObjectInfos * object_infos = nullptr,
Strings * read_keys_ = nullptr,
const S3Settings::RequestSettings & request_settings_ = {});
2022-05-25 19:47:05 +00:00
KeyWithInfo next() override;
size_t getTotalSize() const override;
2022-05-19 11:18:58 +00:00
private:
class Impl;
/// shared_ptr to have copy constructor
std::shared_ptr<Impl> pimpl;
2021-04-06 19:18:45 +00:00
};
class KeysIterator : public IIterator
{
2022-05-19 11:18:58 +00:00
public:
explicit KeysIterator(
const Aws::S3::S3Client & client_,
const std::string & version_id_,
const std::vector<String> & keys_,
const String & bucket_,
ASTPtr query,
const Block & virtual_header,
ContextPtr context,
ObjectInfos * object_infos = nullptr,
Strings * read_keys = nullptr);
KeyWithInfo next() override;
size_t getTotalSize() const override;
2022-05-19 11:18:58 +00:00
private:
class Impl;
/// shared_ptr to have copy constructor
std::shared_ptr<Impl> pimpl;
};
class ReadTaskIterator : public IIterator
{
public:
explicit ReadTaskIterator(const ReadTaskCallback & callback_) : callback(callback_) {}
KeyWithInfo next() override { return {callback(), {}}; }
size_t getTotalSize() const override { return 0; }
private:
ReadTaskCallback callback;
};
2021-04-08 00:09:15 +00:00
2022-03-28 19:18:20 +00:00
static Block getHeader(Block sample_block, const std::vector<NameAndTypePair> & requested_virtual_columns);
2021-03-22 17:12:31 +00:00
StorageS3Source(
2022-03-28 19:18:20 +00:00
const std::vector<NameAndTypePair> & requested_virtual_columns_,
2021-03-22 17:12:31 +00:00
const String & format,
String name_,
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_,
2022-05-11 22:04:54 +00:00
const std::shared_ptr<const Aws::S3::S3Client> & client_,
2021-03-22 17:12:31 +00:00
const String & bucket,
const String & version_id,
std::shared_ptr<IIterator> file_iterator_,
size_t download_thread_num);
2022-11-26 03:24:11 +00:00
~StorageS3Source() override;
2021-03-22 17:12:31 +00:00
String getName() const override;
Chunk generate() override;
private:
String name;
2021-04-08 00:09:15 +00:00
String bucket;
String version_id;
2021-04-08 00:09:15 +00:00
String format;
ColumnsDescription columns_desc;
UInt64 max_block_size;
S3Settings::RequestSettings request_settings;
2021-04-08 00:09:15 +00:00
String compression_hint;
2022-05-11 22:04:54 +00:00
std::shared_ptr<const Aws::S3::S3Client> client;
2021-04-08 00:09:15 +00:00
Block sample_block;
2021-08-23 19:05:28 +00:00
std::optional<FormatSettings> format_settings;
2021-04-08 00:09:15 +00:00
2022-11-26 03:24:11 +00:00
struct ReaderHolder
{
public:
ReaderHolder(
String path_,
std::unique_ptr<ReadBuffer> read_buf_,
std::unique_ptr<QueryPipeline> pipeline_,
std::unique_ptr<PullingPipelineExecutor> reader_)
: path(std::move(path_))
, read_buf(std::move(read_buf_))
, pipeline(std::move(pipeline_))
, reader(std::move(reader_))
{
}
ReaderHolder() = default;
ReaderHolder(const ReaderHolder & other) = delete;
ReaderHolder & operator=(const ReaderHolder & other) = delete;
2023-01-13 16:33:11 +00:00
ReaderHolder(ReaderHolder && other) noexcept
{
*this = std::move(other);
}
2023-01-13 16:33:11 +00:00
ReaderHolder & operator=(ReaderHolder && other) noexcept
{
2023-01-13 16:33:11 +00:00
/// The order of destruction is important.
/// reader uses pipeline, pipeline uses read_buf.
reader = std::move(other.reader);
pipeline = std::move(other.pipeline);
read_buf = std::move(other.read_buf);
2023-01-16 15:47:04 +00:00
path = std::move(other.path);
return *this;
}
2022-11-26 03:24:11 +00:00
explicit operator bool() const { return reader != nullptr; }
PullingPipelineExecutor * operator->() { return reader.get(); }
const PullingPipelineExecutor * operator->() const { return reader.get(); }
const String & getPath() const { return path; }
private:
String path;
std::unique_ptr<ReadBuffer> read_buf;
std::unique_ptr<QueryPipeline> pipeline;
std::unique_ptr<PullingPipelineExecutor> reader;
};
ReaderHolder reader;
2021-04-08 00:09:15 +00:00
2022-03-28 19:18:20 +00:00
std::vector<NameAndTypePair> requested_virtual_columns;
std::shared_ptr<IIterator> file_iterator;
2022-03-23 08:40:00 +00:00
size_t download_thread_num = 1;
2021-04-08 00:09:15 +00:00
2022-03-28 08:19:23 +00:00
Poco::Logger * log = &Poco::Logger::get("StorageS3Source");
2021-04-08 00:09:15 +00:00
2022-11-26 03:24:11 +00:00
ThreadPool create_reader_pool;
ThreadPoolCallbackRunner<ReaderHolder> create_reader_scheduler;
std::future<ReaderHolder> reader_future;
UInt64 total_rows_approx_max = 0;
size_t total_rows_count_times = 0;
UInt64 total_rows_approx_accumulated = 0;
2022-05-09 19:13:02 +00:00
/// Recreate ReadBuffer and Pipeline for each file.
2022-11-26 03:24:11 +00:00
ReaderHolder createReader();
std::future<ReaderHolder> createReaderAsync();
2022-03-23 08:40:00 +00:00
2022-11-26 03:24:11 +00:00
std::unique_ptr<ReadBuffer> createS3ReadBuffer(const String & key, size_t object_size);
std::unique_ptr<ReadBuffer> createAsyncS3ReadBuffer(const String & key, const ReadSettings & read_settings, size_t object_size);
2021-03-22 17:12:31 +00:00
};
2019-05-31 07:27:14 +00:00
/**
2019-06-01 21:18:20 +00:00
* This class represents table engine for external S3 urls.
2019-05-31 07:27:14 +00:00
* It sends HTTP GET to server when select is called and
2019-06-01 21:18:20 +00:00
* HTTP PUT when insert is called.
2019-05-31 07:27:14 +00:00
*/
class StorageS3 : public IStorage, WithContext
2019-05-23 09:03:39 +00:00
{
public:
2021-04-23 12:18:23 +00:00
StorageS3(
2022-09-13 13:07:43 +00:00
const StorageS3Configuration & configuration_,
2019-12-04 16:06:55 +00:00
const StorageID & table_id_,
2019-06-01 21:18:20 +00:00
const ColumnsDescription & columns_,
2019-09-22 22:13:42 +00:00
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-10-26 12:22:13 +00:00
bool distributed_processing_ = false,
ASTPtr partition_by_ = nullptr);
2019-06-01 21:18:20 +00:00
String getName() const override
{
return name;
2019-06-01 21:18:20 +00:00
}
2020-08-03 13:54:14 +00:00
Pipe read(
2019-09-22 22:13:42 +00:00
const Names & column_names,
const StorageSnapshotPtr & storage_snapshot,
SelectQueryInfo & query_info,
ContextPtr context,
2019-05-23 09:03:39 +00:00
QueryProcessingStage::Enum processed_stage,
size_t max_block_size,
size_t num_streams) override;
2019-05-23 09:03:39 +00:00
2021-07-23 19:33:59 +00:00
SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override;
2019-05-23 09:03:39 +00:00
2021-06-21 15:44:36 +00:00
void truncate(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, TableExclusiveLockHolder &) override;
NamesAndTypesList getVirtuals() const override;
2020-04-27 13:55:30 +00:00
2021-07-14 08:49:05 +00:00
bool supportsPartitionBy() const override;
2021-09-07 11:17:25 +00:00
static StorageS3Configuration getConfiguration(ASTs & engine_args, ContextPtr local_context);
using ObjectInfos = StorageS3Source::ObjectInfos;
static ColumnsDescription getTableStructureFromData(
2022-09-13 13:07:43 +00:00
const StorageS3Configuration & configuration,
bool distributed_processing,
const std::optional<FormatSettings> & format_settings,
ContextPtr ctx,
ObjectInfos * object_infos = nullptr);
2021-03-22 17:12:31 +00:00
static void processNamedCollectionResult(StorageS3Configuration & configuration, const NamedCollection & collection);
2021-03-25 13:49:07 +00:00
2022-04-03 22:33:59 +00:00
struct S3Configuration
2021-03-16 18:41:29 +00:00
{
const S3::URI uri;
2022-05-11 22:04:54 +00:00
std::shared_ptr<const Aws::S3::S3Client> client;
2022-09-13 13:07:43 +00:00
S3::AuthSettings auth_settings;
S3Settings::RequestSettings request_settings;
2022-09-13 13:07:43 +00:00
/// If s3 configuration was passed from ast, then it is static.
/// If from config - it can be changed with config reload.
2022-09-13 13:07:43 +00:00
bool static_configuration = true;
2022-09-13 14:30:28 +00:00
/// Headers from ast is a part of static configuration.
HTTPHeaderEntries headers_from_ast;
2022-09-13 13:07:43 +00:00
S3Configuration(
const String & url_,
const S3::AuthSettings & auth_settings_,
const S3Settings::RequestSettings & request_settings_,
const HTTPHeaderEntries & headers_from_ast_)
2022-09-13 13:07:43 +00:00
: uri(S3::URI(url_))
, auth_settings(auth_settings_)
, request_settings(request_settings_)
2022-09-13 13:07:43 +00:00
, static_configuration(!auth_settings_.access_key_id.empty())
, headers_from_ast(headers_from_ast_) {}
2021-03-16 18:41:29 +00:00
};
static SchemaCache & getSchemaCache(const ContextPtr & ctx);
2022-04-03 22:33:59 +00:00
private:
friend class StorageS3Cluster;
friend class TableFunctionS3Cluster;
2022-09-28 11:03:43 +00:00
friend class StorageHudi;
friend class StorageDeltaLake;
2022-04-03 22:33:59 +00:00
S3Configuration s3_configuration;
std::vector<String> keys;
2022-03-28 19:18:20 +00:00
NamesAndTypesList virtual_columns;
2022-05-19 11:18:58 +00:00
Block virtual_block;
2019-05-23 09:03:39 +00:00
2019-05-31 07:27:14 +00:00
String format_name;
String compression_method;
String name;
2021-04-13 20:17:25 +00:00
const bool distributed_processing;
2021-08-23 19:05:28 +00:00
std::optional<FormatSettings> format_settings;
2021-10-26 12:22:13 +00:00
ASTPtr partition_by;
bool is_key_with_globs = false;
ObjectInfos object_infos;
2022-04-06 20:27:38 +00:00
static void updateS3Configuration(ContextPtr, S3Configuration &);
static std::shared_ptr<StorageS3Source::IIterator> createFileIterator(
const S3Configuration & s3_configuration,
2022-04-19 18:23:04 +00:00
const std::vector<String> & keys,
bool is_key_with_globs,
bool distributed_processing,
ContextPtr local_context,
2022-05-19 11:18:58 +00:00
ASTPtr query,
const Block & virtual_block,
ObjectInfos * object_infos = nullptr,
Strings * read_keys = nullptr);
static ColumnsDescription getTableStructureFromDataImpl(
const String & format,
2022-04-03 22:33:59 +00:00
const S3Configuration & s3_configuration,
const String & compression_method,
bool distributed_processing,
bool is_key_with_globs,
const std::optional<FormatSettings> & format_settings,
2022-04-19 18:23:04 +00:00
ContextPtr ctx,
ObjectInfos * object_infos = nullptr);
2022-02-23 19:31:16 +00:00
bool supportsSubcolumns() const override;
2022-05-13 18:39:19 +00:00
bool supportsSubsetOfColumns() const override;
static std::optional<ColumnsDescription> tryGetColumnsFromCache(
const Strings::const_iterator & begin,
const Strings::const_iterator & end,
const S3Configuration & s3_configuration,
ObjectInfos * object_infos,
const String & format_name,
const std::optional<FormatSettings> & format_settings,
const ContextPtr & ctx);
static void addColumnsToCache(
const Strings & keys,
const S3Configuration & s3_configuration,
const ColumnsDescription & columns,
const String & format_name,
const std::optional<FormatSettings> & format_settings,
const ContextPtr & ctx);
2019-05-23 09:03:39 +00:00
};
2019-12-11 14:21:48 +00:00
2019-05-23 09:03:39 +00:00
}
2019-12-11 14:21:48 +00:00
#endif