2023-04-19 18:15:24 +00:00
|
|
|
#pragma once
|
|
|
|
|
|
|
|
#include "config.h"
|
|
|
|
|
|
|
|
#if USE_AWS_S3
|
|
|
|
|
2023-04-29 15:48:37 +00:00
|
|
|
# include <Core/Types.h>
|
|
|
|
|
|
|
|
# include <Compression/CompressionInfo.h>
|
|
|
|
# include <Common/ZooKeeper/ZooKeeper.h>
|
|
|
|
|
|
|
|
# include <Core/BackgroundSchedulePool.h>
|
|
|
|
# include <Storages/IStorage.h>
|
|
|
|
# include <Storages/S3Queue/S3QueueSettings.h>
|
|
|
|
# include <Storages/S3Queue/S3QueueSource.h>
|
|
|
|
# include <Storages/StorageS3Settings.h>
|
|
|
|
|
|
|
|
# include <IO/CompressionMethod.h>
|
|
|
|
# include <IO/S3/getObjectInfo.h>
|
|
|
|
# include <Interpreters/Context.h>
|
|
|
|
# include <Interpreters/threadPoolCallbackRunner.h>
|
|
|
|
# include <Processors/Executors/PullingPipelineExecutor.h>
|
|
|
|
# include <Processors/ISource.h>
|
|
|
|
# include <Storages/Cache/SchemaCache.h>
|
|
|
|
# include <Storages/StorageConfiguration.h>
|
|
|
|
# include <Storages/StorageS3.h>
|
|
|
|
# include <Poco/URI.h>
|
|
|
|
# include <Common/logger_useful.h>
|
2023-04-19 18:15:24 +00:00
|
|
|
|
|
|
|
namespace Aws::S3
|
|
|
|
{
|
|
|
|
class Client;
|
|
|
|
}
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
|
|
|
|
class StorageS3Queue : public IStorage, WithContext
|
|
|
|
{
|
|
|
|
public:
|
|
|
|
using Configuration = typename StorageS3::Configuration;
|
|
|
|
StorageS3Queue(
|
2023-04-29 15:48:37 +00:00
|
|
|
std::unique_ptr<S3QueueSettings> s3queue_settings_,
|
2023-04-19 18:15:24 +00:00
|
|
|
const Configuration & configuration_,
|
|
|
|
const StorageID & table_id_,
|
|
|
|
const ColumnsDescription & columns_,
|
|
|
|
const ConstraintsDescription & constraints_,
|
|
|
|
const String & comment,
|
|
|
|
ContextPtr context_,
|
|
|
|
std::optional<FormatSettings> format_settings_,
|
|
|
|
bool distributed_processing_ = false,
|
|
|
|
ASTPtr partition_by_ = nullptr);
|
|
|
|
|
|
|
|
String getName() const override { return "S3Queue"; }
|
|
|
|
|
|
|
|
Pipe read(
|
|
|
|
const Names & column_names,
|
|
|
|
const StorageSnapshotPtr & storage_snapshot,
|
|
|
|
SelectQueryInfo & query_info,
|
|
|
|
ContextPtr context,
|
|
|
|
QueryProcessingStage::Enum processed_stage,
|
|
|
|
size_t max_block_size,
|
|
|
|
size_t num_streams) override;
|
|
|
|
|
2023-04-29 15:48:37 +00:00
|
|
|
SinkToStoragePtr write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr /*context*/) override
|
|
|
|
{
|
2023-04-19 18:15:24 +00:00
|
|
|
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Write is not supported by storage {}", getName());
|
|
|
|
}
|
|
|
|
|
2023-04-29 15:48:37 +00:00
|
|
|
void truncate(
|
|
|
|
const ASTPtr & /*query*/,
|
|
|
|
const StorageMetadataPtr & /*metadata_snapshot*/,
|
|
|
|
ContextPtr /*local_context*/,
|
|
|
|
TableExclusiveLockHolder &) override
|
|
|
|
{
|
2023-04-19 18:15:24 +00:00
|
|
|
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Truncate is not supported by storage {}", getName());
|
|
|
|
}
|
|
|
|
|
|
|
|
NamesAndTypesList getVirtuals() const override;
|
|
|
|
|
|
|
|
bool supportsPartitionBy() const override;
|
|
|
|
|
|
|
|
const auto & getFormatName() const { return format_name; }
|
|
|
|
|
|
|
|
private:
|
2023-04-29 15:48:37 +00:00
|
|
|
std::unique_ptr<S3QueueSettings> s3queue_settings;
|
2023-04-19 18:15:24 +00:00
|
|
|
Configuration s3_configuration;
|
|
|
|
std::vector<String> keys;
|
|
|
|
NamesAndTypesList virtual_columns;
|
|
|
|
Block virtual_block;
|
|
|
|
uint64_t milliseconds_to_wait = 10000;
|
|
|
|
|
|
|
|
String format_name;
|
|
|
|
String compression_method;
|
|
|
|
String name;
|
|
|
|
const bool distributed_processing;
|
|
|
|
std::optional<FormatSettings> format_settings;
|
|
|
|
ASTPtr partition_by;
|
|
|
|
bool is_key_with_globs = false;
|
|
|
|
|
|
|
|
bool supportsSubcolumns() const override;
|
|
|
|
|
|
|
|
void threadFunc();
|
|
|
|
size_t getTableDependentCount() const;
|
|
|
|
std::atomic<bool> mv_attached = false;
|
|
|
|
bool hasDependencies(const StorageID & table_id);
|
|
|
|
std::atomic<bool> shutdown_called{false};
|
|
|
|
Poco::Logger * log;
|
|
|
|
|
|
|
|
|
|
|
|
void startup() override;
|
|
|
|
void shutdown() override;
|
|
|
|
|
|
|
|
struct TaskContext
|
|
|
|
{
|
|
|
|
BackgroundSchedulePool::TaskHolder holder;
|
2023-04-29 15:48:37 +00:00
|
|
|
std::atomic<bool> stream_cancelled{false};
|
|
|
|
explicit TaskContext(BackgroundSchedulePool::TaskHolder && task_) : holder(std::move(task_)) { }
|
2023-04-19 18:15:24 +00:00
|
|
|
};
|
|
|
|
std::shared_ptr<TaskContext> task;
|
|
|
|
|
|
|
|
bool supportsSubsetOfColumns() const override;
|
|
|
|
static Names getVirtualColumnNames();
|
|
|
|
|
2023-04-29 15:48:37 +00:00
|
|
|
String zookeeper_path;
|
2023-04-19 18:15:24 +00:00
|
|
|
|
|
|
|
zkutil::ZooKeeperPtr current_zookeeper;
|
|
|
|
mutable std::mutex current_zookeeper_mutex;
|
2023-04-27 11:55:23 +00:00
|
|
|
mutable std::mutex sync_mutex;
|
2023-04-19 18:15:24 +00:00
|
|
|
|
|
|
|
void setZooKeeper();
|
|
|
|
zkutil::ZooKeeperPtr tryGetZooKeeper() const;
|
|
|
|
zkutil::ZooKeeperPtr getZooKeeper() const;
|
|
|
|
bool createTableIfNotExists(const StorageMetadataPtr & metadata_snapshot);
|
|
|
|
// Return default or custom zookeeper name for table
|
|
|
|
const String & getZooKeeperPath() const { return zookeeper_path; }
|
|
|
|
|
2023-04-27 11:55:23 +00:00
|
|
|
using KeysWithInfo = StorageS3QueueSource::KeysWithInfo;
|
|
|
|
|
2023-04-29 15:48:37 +00:00
|
|
|
std::shared_ptr<StorageS3QueueSource::IIterator>
|
|
|
|
createFileIterator(ContextPtr local_context, ASTPtr query, KeysWithInfo * read_keys = nullptr);
|
2023-04-27 11:55:23 +00:00
|
|
|
|
|
|
|
static std::unordered_set<String> parseCollection(String & files);
|
|
|
|
std::unordered_set<String> getExcludedFiles();
|
|
|
|
|
|
|
|
bool streamToViews();
|
2023-04-29 15:48:37 +00:00
|
|
|
|
|
|
|
Configuration updateConfigurationAndGetCopy(ContextPtr local_context);
|
2023-04-19 18:15:24 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
#endif
|