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;
|
|
|
|
}
|
|
|
|
|
2023-06-05 12:03:19 +00:00
|
|
|
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-05 18:36:17 +00:00
|
|
|
};
|
|
|
|
|
2023-06-06 19:58:54 +00:00
|
|
|
class StorageAzureBlobSource : public ISource, WithContext
|
2023-06-05 18:36:17 +00:00
|
|
|
{
|
|
|
|
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;
|
2023-06-05 18:36:17 +00:00
|
|
|
|
|
|
|
Chunk generate() override;
|
2023-06-05 23:28:04 +00:00
|
|
|
|
2023-06-05 18:36:17 +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);
|
2023-06-05 18:36:17 +00:00
|
|
|
|
|
|
|
private:
|
2023-06-05 23:28:04 +00:00
|
|
|
std::vector<NameAndTypePair> requested_virtual_columns;
|
2023-06-06 06:06:24 +00:00
|
|
|
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;
|
2023-06-06 06:06:24 +00:00
|
|
|
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::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;
|
|
|
|
|
|
|
|
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);
|
|
|
|
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; }
|
|
|
|
|
|
|
|
private:
|
|
|
|
String path;
|
|
|
|
std::unique_ptr<ReadBuffer> read_buf;
|
|
|
|
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;
|
|
|
|
|
|
|
|
/// Recreate ReadBuffer and Pipeline for each file.
|
|
|
|
ReaderHolder createReader();
|
|
|
|
std::future<ReaderHolder> createReaderAsync();
|
2023-06-05 18:36:17 +00:00
|
|
|
|
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
|