ClickHouse/src/Storages/StorageAzureBlob.h

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

345 lines
11 KiB
C++
Raw Normal View History

2023-06-02 14:51:09 +00:00
#pragma once
#include "config.h"
#if USE_AZURE_BLOB_STORAGE
2023-06-05 23:28:04 +00:00
#include <re2/re2.h>
2023-06-02 14:51:09 +00:00
#include <Storages/IStorage.h>
#include <Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h>
#include <Storages/Cache/SchemaCache.h>
#include <Storages/StorageConfiguration.h>
#include <Processors/Executors/PullingPipelineExecutor.h>
2023-06-06 16:48:20 +00:00
#include <Storages/NamedCollectionsHelpers.h>
#include <Storages/prepareReadingFromFormat.h>
2023-06-02 14:51:09 +00:00
namespace DB
{
2023-06-06 19:58:54 +00:00
class StorageAzureBlob : public IStorage
2023-06-02 14:51:09 +00:00
{
public:
2023-06-06 16:48:20 +00:00
2023-06-02 14:51:09 +00:00
using AzureClient = Azure::Storage::Blobs::BlobContainerClient;
using AzureClientPtr = std::unique_ptr<Azure::Storage::Blobs::BlobContainerClient>;
struct Configuration : public StatelessTableEngineConfiguration
{
Configuration() = default;
String getPath() const { return blob_path; }
bool update(ContextPtr context);
void connect(ContextPtr context);
bool withGlobs() const { return blob_path.find_first_of("*?{") != std::string::npos; }
bool withWildcard() const
{
static const String PARTITION_ID_WILDCARD = "{_partition_id}";
return blobs_paths.back().find(PARTITION_ID_WILDCARD) != String::npos;
}
Poco::URI getConnectionURL() const;
2023-06-02 14:51:09 +00:00
std::string connection_url;
bool is_connection_string;
std::optional<std::string> account_name;
std::optional<std::string> account_key;
std::string container;
std::string blob_path;
std::vector<String> blobs_paths;
};
2023-06-06 19:58:54 +00:00
StorageAzureBlob(
2023-06-02 14:51:09 +00:00
const Configuration & configuration_,
std::unique_ptr<AzureObjectStorage> && object_storage_,
ContextPtr context_,
const StorageID & table_id_,
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
const String & comment,
std::optional<FormatSettings> format_settings_,
bool distributed_processing_,
2023-06-02 14:51:09 +00:00
ASTPtr partition_by_);
2023-06-08 14:11:27 +00:00
static StorageAzureBlob::Configuration getConfiguration(ASTs & engine_args, ContextPtr local_context);
static AzureClientPtr createClient(StorageAzureBlob::Configuration configuration, bool is_read_only);
2023-06-06 16:48:20 +00:00
static AzureObjectStorage::SettingsPtr createSettings(ContextPtr local_context);
2023-06-06 19:58:54 +00:00
static void processNamedCollectionResult(StorageAzureBlob::Configuration & configuration, const NamedCollection & collection);
2023-06-02 14:51:09 +00:00
String getName() const override
{
return name;
}
Pipe read(
2023-06-03 21:11:03 +00:00
const Names &,
const StorageSnapshotPtr &,
SelectQueryInfo &,
ContextPtr,
QueryProcessingStage::Enum,
size_t,
2023-06-05 23:28:04 +00:00
size_t) override;
2023-06-02 14:51:09 +00:00
2023-06-08 15:06:15 +00:00
SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /* metadata_snapshot */, ContextPtr context, bool /*async_insert*/) override;
2023-06-02 14:51:09 +00:00
void truncate(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, TableExclusiveLockHolder &) override;
NamesAndTypesList getVirtuals() const override;
bool supportsPartitionBy() const override;
bool supportsSubcolumns() const override { return true; }
2023-06-05 12:46:52 +00:00
bool supportsSubsetOfColumns() const override;
bool prefersLargeBlocks() const override;
bool parallelizeOutputAfterReading(ContextPtr context) const override;
2023-06-02 14:51:09 +00:00
static SchemaCache & getSchemaCache(const ContextPtr & ctx);
2023-06-06 16:48:20 +00:00
static ColumnsDescription getTableStructureFromData(
AzureObjectStorage * object_storage,
const Configuration & configuration,
const std::optional<FormatSettings> & format_settings,
ContextPtr ctx,
bool distributed_processing = false);
2023-06-06 16:48:20 +00:00
2023-06-02 14:51:09 +00:00
private:
std::string name;
Configuration configuration;
std::unique_ptr<AzureObjectStorage> object_storage;
NamesAndTypesList virtual_columns;
Block virtual_block;
const bool distributed_processing;
std::optional<FormatSettings> format_settings;
ASTPtr partition_by;
2023-06-06 14:57:51 +00:00
2023-06-06 16:48:20 +00:00
static std::optional<ColumnsDescription> tryGetColumnsFromCache(
2023-06-06 14:57:51 +00:00
const RelativePathsWithMetadata::const_iterator & begin,
const RelativePathsWithMetadata::const_iterator & end,
2023-06-06 19:58:54 +00:00
const StorageAzureBlob::Configuration & configuration,
2023-06-06 16:48:20 +00:00
const std::optional<FormatSettings> & format_settings,
2023-06-06 14:57:51 +00:00
const ContextPtr & ctx);
2023-06-06 16:48:20 +00:00
static void addColumnsToCache(
2023-06-06 14:57:51 +00:00
const RelativePathsWithMetadata & keys,
const ColumnsDescription & columns,
2023-06-06 16:48:20 +00:00
const Configuration & configuration,
const std::optional<FormatSettings> & format_settings,
2023-06-06 14:57:51 +00:00
const String & format_name,
const ContextPtr & ctx);
};
2023-06-06 19:58:54 +00:00
class StorageAzureBlobSource : public ISource, WithContext
{
public:
class IIterator : public WithContext
{
public:
IIterator(ContextPtr context_):WithContext(context_) {}
virtual ~IIterator() = default;
virtual RelativePathWithMetadata next() = 0;
RelativePathWithMetadata operator ()() { return next(); }
};
class GlobIterator : public IIterator
2023-06-05 23:28:04 +00:00
{
public:
GlobIterator(
2023-06-05 23:28:04 +00:00
AzureObjectStorage * object_storage_,
const std::string & container_,
String blob_path_with_globs_,
2023-06-05 23:28:04 +00:00
ASTPtr query_,
const Block & virtual_header_,
2023-06-06 14:57:51 +00:00
ContextPtr context_,
RelativePathsWithMetadata * outer_blobs_,
std::function<void(FileProgress)> file_progress_callback_ = {});
2023-06-05 23:28:04 +00:00
RelativePathWithMetadata next() override;
~GlobIterator() override = default;
2023-06-06 14:57:51 +00:00
private:
2023-06-05 23:28:04 +00:00
AzureObjectStorage * object_storage;
std::string container;
String blob_path_with_globs;
2023-06-05 23:28:04 +00:00
ASTPtr query;
ASTPtr filter_ast;
Block virtual_header;
size_t index = 0;
2023-06-05 23:28:04 +00:00
RelativePathsWithMetadata blobs_with_metadata;
2023-06-06 14:57:51 +00:00
RelativePathsWithMetadata * outer_blobs;
2023-06-05 23:28:04 +00:00
ObjectStorageIteratorPtr object_storage_iterator;
bool recursive{false};
std::unique_ptr<re2::RE2> matcher;
void createFilterAST(const String & any_key);
bool is_finished = false;
bool is_initialized = false;
2023-06-08 14:11:27 +00:00
std::mutex next_mutex;
std::function<void(FileProgress)> file_progress_callback;
2023-06-05 23:28:04 +00:00
};
class ReadIterator : public IIterator
{
public:
explicit ReadIterator(ContextPtr context_,
const ReadTaskCallback & callback_)
: IIterator(context_), callback(callback_) { }
RelativePathWithMetadata next() override
{
return {callback(), {}};
}
private:
ReadTaskCallback callback;
};
class KeysIterator : public IIterator
{
public:
KeysIterator(
AzureObjectStorage * object_storage_,
const std::string & container_,
2023-06-29 11:59:09 +00:00
const Strings & keys_,
ASTPtr query_,
const Block & virtual_header_,
ContextPtr context_,
2023-06-29 11:59:09 +00:00
RelativePathsWithMetadata * outer_blobs,
std::function<void(FileProgress)> file_progress_callback = {});
RelativePathWithMetadata next() override;
~KeysIterator() override = default;
private:
AzureObjectStorage * object_storage;
std::string container;
RelativePathsWithMetadata keys;
ASTPtr query;
ASTPtr filter_ast;
Block virtual_header;
std::atomic<size_t> index = 0;
};
2023-06-06 19:58:54 +00:00
StorageAzureBlobSource(
const ReadFromFormatInfo & info,
2023-06-05 23:28:04 +00:00
const String & format_,
String name_,
ContextPtr context_,
std::optional<FormatSettings> format_settings_,
UInt64 max_block_size_,
String compression_hint_,
AzureObjectStorage * object_storage_,
const String & container_,
std::shared_ptr<IIterator> file_iterator_);
2023-06-05 23:28:04 +00:00
2023-06-06 19:58:54 +00:00
~StorageAzureBlobSource() override;
Chunk generate() override;
2023-06-05 23:28:04 +00:00
String getName() const override;
private:
NamesAndTypesList requested_columns;
NamesAndTypesList requested_virtual_columns;
String format;
String name;
Block sample_block;
std::optional<FormatSettings> format_settings;
2023-06-05 21:15:13 +00:00
ColumnsDescription columns_desc;
2023-06-05 23:28:04 +00:00
UInt64 max_block_size;
String compression_hint;
AzureObjectStorage * object_storage;
String container;
std::shared_ptr<IIterator> file_iterator;
2023-06-05 23:28:04 +00:00
struct ReaderHolder
{
public:
ReaderHolder(
String path_,
std::unique_ptr<ReadBuffer> read_buf_,
std::shared_ptr<IInputFormat> input_format_,
2023-06-05 23:28:04 +00:00
std::unique_ptr<QueryPipeline> pipeline_,
std::unique_ptr<PullingPipelineExecutor> reader_)
: path(std::move(path_))
, read_buf(std::move(read_buf_))
2023-06-23 13:43:40 +00:00
, input_format(std::move(input_format_))
2023-06-05 23:28:04 +00:00
, pipeline(std::move(pipeline_))
, reader(std::move(reader_))
{
}
ReaderHolder() = default;
ReaderHolder(const ReaderHolder & other) = delete;
ReaderHolder & operator=(const ReaderHolder & other) = delete;
ReaderHolder(ReaderHolder && other) noexcept
{
*this = std::move(other);
}
ReaderHolder & operator=(ReaderHolder && other) noexcept
{
/// The order of destruction is important.
/// reader uses pipeline, pipeline uses read_buf.
reader = std::move(other.reader);
pipeline = std::move(other.pipeline);
input_format = std::move(other.input_format);
2023-06-05 23:28:04 +00:00
read_buf = std::move(other.read_buf);
path = std::move(other.path);
return *this;
}
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; }
2023-06-23 13:43:40 +00:00
const IInputFormat * getInputFormat() const { return input_format.get(); }
2023-06-05 23:28:04 +00:00
private:
String path;
std::unique_ptr<ReadBuffer> read_buf;
std::shared_ptr<IInputFormat> input_format;
2023-06-05 23:28:04 +00:00
std::unique_ptr<QueryPipeline> pipeline;
std::unique_ptr<PullingPipelineExecutor> reader;
};
ReaderHolder reader;
Poco::Logger * log = &Poco::Logger::get("StorageAzureBlobSource");
ThreadPool create_reader_pool;
ThreadPoolCallbackRunner<ReaderHolder> create_reader_scheduler;
std::future<ReaderHolder> reader_future;
/// Recreate ReadBuffer and Pipeline for each file.
ReaderHolder createReader();
std::future<ReaderHolder> createReaderAsync();
2023-06-05 23:28:04 +00:00
std::unique_ptr<ReadBuffer> createAzureReadBuffer(const String & key, size_t object_size);
2023-06-06 14:16:49 +00:00
std::unique_ptr<ReadBuffer> createAsyncAzureReadBuffer(
const String & key, const ReadSettings & read_settings, size_t object_size);
2023-06-02 14:51:09 +00:00
};
}
#endif