ClickHouse/src/Storages/StorageAzureBlob.h

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

302 lines
9.6 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>
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_,
ASTPtr partition_by_);
2023-06-08 14:11:27 +00:00
static StorageAzureBlob::Configuration getConfiguration(ASTs & engine_args, ContextPtr local_context);
2023-06-06 19:58:54 +00:00
static AzureClientPtr createClient(StorageAzureBlob::Configuration configuration);
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;
2023-06-05 12:46:52 +00:00
bool supportsSubcolumns() const override;
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);
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:
2023-06-05 23:28:04 +00:00
class Iterator : WithContext
{
public:
Iterator(
AzureObjectStorage * object_storage_,
const std::string & container_,
std::optional<Strings> keys_,
std::optional<String> blob_path_with_globs_,
ASTPtr query_,
const Block & virtual_header_,
2023-06-06 14:57:51 +00:00
ContextPtr context_,
RelativePathsWithMetadata * outer_blobs_);
2023-06-05 23:28:04 +00:00
RelativePathWithMetadata next();
size_t getTotalSize() const;
~Iterator() = default;
2023-06-06 14:57:51 +00:00
private:
2023-06-05 23:28:04 +00:00
AzureObjectStorage * object_storage;
std::string container;
std::optional<Strings> keys;
std::optional<String> blob_path_with_globs;
ASTPtr query;
ASTPtr filter_ast;
Block virtual_header;
std::atomic<size_t> index = 0;
std::atomic<size_t> total_size = 0;
std::optional<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);
2023-06-08 14:25:55 +00:00
std::atomic<bool> is_finished = false;
std::atomic<bool> is_initialized = false;
2023-06-08 14:11:27 +00:00
std::mutex next_mutex;
2023-06-05 23:28:04 +00:00
};
2023-06-06 19:58:54 +00:00
StorageAzureBlobSource(
2023-06-05 23:28:04 +00:00
const std::vector<NameAndTypePair> & requested_virtual_columns_,
const String & format_,
String name_,
const Block & sample_block_,
ContextPtr context_,
std::optional<FormatSettings> format_settings_,
const ColumnsDescription & columns_,
UInt64 max_block_size_,
String compression_hint_,
AzureObjectStorage * object_storage_,
const String & container_,
std::shared_ptr<Iterator> file_iterator_);
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;
2023-06-05 23:28:04 +00:00
static Block getHeader(Block sample_block, const std::vector<NameAndTypePair> & requested_virtual_columns);
private:
2023-06-05 23:28:04 +00:00
std::vector<NameAndTypePair> 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<Iterator> 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_))
, input_format(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; }
const std::unique_ptr<ReadBuffer> & getReadBuffer() const { return read_buf; }
const std::shared_ptr<IInputFormat> & getFormat() const { return input_format; }
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;
UInt64 total_rows_approx_max = 0;
size_t total_rows_count_times = 0;
UInt64 total_rows_approx_accumulated = 0;
size_t total_objects_size = 0;
2023-06-05 23:28:04 +00:00
/// 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