2019-05-23 09:03:39 +00:00
|
|
|
#pragma once
|
|
|
|
|
2022-09-28 08:45:15 +00:00
|
|
|
#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>
|
2021-03-04 15:56:55 +00:00
|
|
|
#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>
|
2023-01-27 14:09:38 +00:00
|
|
|
#include <IO/S3/getObjectInfo.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>
|
2022-06-21 13:02:48 +00:00
|
|
|
#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;
|
2022-12-13 21:07:30 +00:00
|
|
|
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>;
|
2022-12-13 16:33:21 +00:00
|
|
|
using ObjectInfos = std::unordered_map<String, S3::ObjectInfo>;
|
2022-11-21 16:52:15 +00:00
|
|
|
class IIterator
|
|
|
|
{
|
|
|
|
public:
|
|
|
|
virtual ~IIterator() = default;
|
2022-12-13 16:33:21 +00:00
|
|
|
virtual KeyWithInfo next() = 0;
|
2022-11-21 16:52:15 +00:00
|
|
|
virtual size_t getTotalSize() const = 0;
|
|
|
|
|
2022-12-13 16:33:21 +00:00
|
|
|
KeyWithInfo operator ()() { return next(); }
|
2022-11-21 16:52:15 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
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,
|
2022-06-21 13:02:48 +00:00
|
|
|
ContextPtr context,
|
2022-12-13 16:33:21 +00:00
|
|
|
ObjectInfos * object_infos = nullptr,
|
2022-11-17 16:35:04 +00:00
|
|
|
Strings * read_keys_ = nullptr,
|
|
|
|
const S3Settings::RequestSettings & request_settings_ = {});
|
2022-05-25 19:47:05 +00:00
|
|
|
|
2022-12-13 16:33:21 +00:00
|
|
|
KeyWithInfo next() override;
|
2022-11-21 16:52:15 +00:00
|
|
|
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
|
|
|
};
|
|
|
|
|
2022-11-21 16:52:15 +00:00
|
|
|
class KeysIterator : public IIterator
|
2021-12-29 18:03:15 +00:00
|
|
|
{
|
2022-05-19 11:18:58 +00:00
|
|
|
public:
|
|
|
|
explicit KeysIterator(
|
2022-11-21 16:52:15 +00:00
|
|
|
const Aws::S3::S3Client & client_,
|
|
|
|
const std::string & version_id_,
|
|
|
|
const std::vector<String> & keys_,
|
|
|
|
const String & bucket_,
|
2023-01-27 13:08:19 +00:00
|
|
|
const S3Settings::RequestSettings & request_settings_,
|
2022-11-21 16:52:15 +00:00
|
|
|
ASTPtr query,
|
|
|
|
const Block & virtual_header,
|
|
|
|
ContextPtr context,
|
2022-12-13 16:33:21 +00:00
|
|
|
ObjectInfos * object_infos = nullptr,
|
|
|
|
Strings * read_keys = nullptr);
|
2022-11-21 16:52:15 +00:00
|
|
|
|
2022-12-13 17:18:17 +00:00
|
|
|
KeyWithInfo next() override;
|
2022-11-21 16:52:15 +00:00
|
|
|
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-12-29 18:03:15 +00:00
|
|
|
};
|
|
|
|
|
2022-11-21 16:52:15 +00:00
|
|
|
class ReadTaskIterator : public IIterator
|
|
|
|
{
|
|
|
|
public:
|
|
|
|
explicit ReadTaskIterator(const ReadTaskCallback & callback_) : callback(callback_) {}
|
|
|
|
|
2022-12-13 16:33:21 +00:00
|
|
|
KeyWithInfo next() override { return {callback(), {}}; }
|
2022-11-21 16:52:15 +00:00
|
|
|
|
|
|
|
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_,
|
2022-11-17 16:35:04 +00:00
|
|
|
const S3Settings::RequestSettings & request_settings_,
|
2022-03-13 12:23:51 +00:00
|
|
|
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,
|
2022-03-15 02:25:40 +00:00
|
|
|
const String & version_id,
|
2022-11-21 16:52:15 +00:00
|
|
|
std::shared_ptr<IIterator> file_iterator_,
|
2023-01-05 15:38:20 +00:00
|
|
|
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;
|
2022-03-15 02:03:58 +00:00
|
|
|
String version_id;
|
2021-04-08 00:09:15 +00:00
|
|
|
String format;
|
|
|
|
ColumnsDescription columns_desc;
|
|
|
|
UInt64 max_block_size;
|
2022-11-17 16:35:04 +00:00
|
|
|
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;
|
2023-01-13 16:31:30 +00:00
|
|
|
ReaderHolder(const ReaderHolder & other) = delete;
|
|
|
|
ReaderHolder & operator=(const ReaderHolder & other) = delete;
|
|
|
|
|
2023-01-13 16:33:11 +00:00
|
|
|
ReaderHolder(ReaderHolder && other) noexcept
|
2023-01-13 16:31:30 +00:00
|
|
|
{
|
|
|
|
*this = std::move(other);
|
|
|
|
}
|
|
|
|
|
2023-01-13 16:33:11 +00:00
|
|
|
ReaderHolder & operator=(ReaderHolder && other) noexcept
|
2023-01-13 16:31:30 +00:00
|
|
|
{
|
2023-01-13 16:33:11 +00:00
|
|
|
/// The order of destruction is important.
|
2023-01-13 16:31:30 +00:00
|
|
|
/// 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);
|
2023-01-13 16:31:30 +00:00
|
|
|
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;
|
2022-11-21 16:52:15 +00:00
|
|
|
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;
|
2022-06-21 13:02:48 +00:00
|
|
|
|
2022-11-21 16:52:15 +00:00
|
|
|
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
|
|
|
*/
|
2022-05-03 06:43:28 +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,
|
2021-04-10 23:33:54 +00:00
|
|
|
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
|
|
|
|
{
|
2020-07-13 14:13:30 +00:00
|
|
|
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,
|
2021-07-09 03:15:41 +00:00
|
|
|
const StorageSnapshotPtr & storage_snapshot,
|
2020-09-20 17:52:17 +00:00
|
|
|
SelectQueryInfo & query_info,
|
2021-04-10 23:33:54 +00:00
|
|
|
ContextPtr context,
|
2019-05-23 09:03:39 +00:00
|
|
|
QueryProcessingStage::Enum processed_stage,
|
|
|
|
size_t max_block_size,
|
2022-10-07 10:46:45 +00:00
|
|
|
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;
|
|
|
|
|
2020-04-28 10:38:57 +00:00
|
|
|
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);
|
|
|
|
|
2022-12-13 16:33:21 +00:00
|
|
|
using ObjectInfos = StorageS3Source::ObjectInfos;
|
|
|
|
|
2021-12-15 11:30:57 +00:00
|
|
|
static ColumnsDescription getTableStructureFromData(
|
2022-09-13 13:07:43 +00:00
|
|
|
const StorageS3Configuration & configuration,
|
2021-12-15 11:30:57 +00:00
|
|
|
bool distributed_processing,
|
|
|
|
const std::optional<FormatSettings> & format_settings,
|
2022-06-21 13:02:48 +00:00
|
|
|
ContextPtr ctx,
|
2022-12-13 16:33:21 +00:00
|
|
|
ObjectInfos * object_infos = nullptr);
|
2021-03-22 17:12:31 +00:00
|
|
|
|
2022-12-13 21:07:30 +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
|
|
|
|
2022-09-15 07:45:28 +00:00
|
|
|
S3::AuthSettings auth_settings;
|
2022-11-17 16:35:04 +00:00
|
|
|
S3Settings::RequestSettings request_settings;
|
2022-09-13 13:07:43 +00:00
|
|
|
|
|
|
|
/// If s3 configuration was passed from ast, then it is static.
|
2022-09-22 15:59:26 +00:00
|
|
|
/// 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.
|
2022-12-16 22:57:09 +00:00
|
|
|
HTTPHeaderEntries headers_from_ast;
|
2022-09-13 13:07:43 +00:00
|
|
|
|
|
|
|
S3Configuration(
|
|
|
|
const String & url_,
|
2022-09-27 07:28:26 +00:00
|
|
|
const S3::AuthSettings & auth_settings_,
|
2022-11-17 16:35:04 +00:00
|
|
|
const S3Settings::RequestSettings & request_settings_,
|
2022-12-16 22:57:09 +00:00
|
|
|
const HTTPHeaderEntries & headers_from_ast_)
|
2022-09-13 13:07:43 +00:00
|
|
|
: uri(S3::URI(url_))
|
|
|
|
, auth_settings(auth_settings_)
|
2022-11-17 16:35:04 +00:00
|
|
|
, 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
|
|
|
};
|
|
|
|
|
2022-08-05 16:20:15 +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;
|
2022-11-21 08:23:33 +00:00
|
|
|
friend class StorageDeltaLake;
|
2022-04-03 22:33:59 +00:00
|
|
|
|
|
|
|
S3Configuration s3_configuration;
|
2021-12-29 18:03:15 +00:00
|
|
|
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;
|
2019-11-19 12:46:07 +00:00
|
|
|
String compression_method;
|
2020-07-13 14:13:30 +00:00
|
|
|
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;
|
2021-12-29 18:03:15 +00:00
|
|
|
bool is_key_with_globs = false;
|
2021-03-04 15:56:55 +00:00
|
|
|
|
2022-12-13 16:33:21 +00:00
|
|
|
ObjectInfos object_infos;
|
2022-06-21 13:02:48 +00:00
|
|
|
|
2022-04-06 20:27:38 +00:00
|
|
|
static void updateS3Configuration(ContextPtr, S3Configuration &);
|
2021-12-15 11:30:57 +00:00
|
|
|
|
2022-11-21 16:52:15 +00:00
|
|
|
static std::shared_ptr<StorageS3Source::IIterator> createFileIterator(
|
2022-04-21 11:39:52 +00:00
|
|
|
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,
|
2022-04-21 11:39:52 +00:00
|
|
|
ContextPtr local_context,
|
2022-05-19 11:18:58 +00:00
|
|
|
ASTPtr query,
|
|
|
|
const Block & virtual_block,
|
2022-12-13 16:33:21 +00:00
|
|
|
ObjectInfos * object_infos = nullptr,
|
2022-06-21 13:02:48 +00:00
|
|
|
Strings * read_keys = nullptr);
|
2021-12-15 11:30:57 +00:00
|
|
|
|
|
|
|
static ColumnsDescription getTableStructureFromDataImpl(
|
|
|
|
const String & format,
|
2022-04-03 22:33:59 +00:00
|
|
|
const S3Configuration & s3_configuration,
|
2021-12-15 11:30:57 +00:00
|
|
|
const String & compression_method,
|
|
|
|
bool distributed_processing,
|
2021-12-29 18:03:15 +00:00
|
|
|
bool is_key_with_globs,
|
2021-12-15 11:30:57 +00:00
|
|
|
const std::optional<FormatSettings> & format_settings,
|
2022-04-19 18:23:04 +00:00
|
|
|
ContextPtr ctx,
|
2022-12-13 16:33:21 +00:00
|
|
|
ObjectInfos * object_infos = nullptr);
|
2022-02-23 19:31:16 +00:00
|
|
|
|
2022-11-23 15:36:12 +00:00
|
|
|
bool supportsSubcolumns() const override;
|
|
|
|
|
2022-05-13 18:39:19 +00:00
|
|
|
bool supportsSubsetOfColumns() const override;
|
2022-06-21 13:02:48 +00:00
|
|
|
|
2022-06-27 12:43:24 +00:00
|
|
|
static std::optional<ColumnsDescription> tryGetColumnsFromCache(
|
|
|
|
const Strings::const_iterator & begin,
|
|
|
|
const Strings::const_iterator & end,
|
|
|
|
const S3Configuration & s3_configuration,
|
2022-12-13 16:33:21 +00:00
|
|
|
ObjectInfos * object_infos,
|
2022-06-27 12:43:24 +00:00
|
|
|
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
|
|
|
|
2019-12-09 12:36:06 +00:00
|
|
|
#endif
|