From b4123145f54162d1e2f0c48d3e81cc7f611c8a7e Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Wed, 19 Apr 2023 21:15:24 +0300 Subject: [PATCH 01/34] wip add zookeeper add settings --- src/CMakeLists.txt | 1 + src/Storages/S3Queue/ReadBufferFromS3.cpp | 175 +++++++ src/Storages/S3Queue/ReadBufferFromS3.h | 86 ++++ src/Storages/S3Queue/S3QueueSettings.cpp | 41 ++ src/Storages/S3Queue/S3QueueSettings.h | 28 ++ src/Storages/S3Queue/S3QueueSource.cpp | 132 +++++ src/Storages/S3Queue/S3QueueSource.h | 55 ++ src/Storages/S3Queue/StorageS3Queue.cpp | 578 ++++++++++++++++++++++ src/Storages/S3Queue/StorageS3Queue.h | 152 ++++++ src/Storages/StorageS3.h | 1 + src/Storages/registerStorages.cpp | 2 + 11 files changed, 1251 insertions(+) create mode 100644 src/Storages/S3Queue/ReadBufferFromS3.cpp create mode 100644 src/Storages/S3Queue/ReadBufferFromS3.h create mode 100644 src/Storages/S3Queue/S3QueueSettings.cpp create mode 100644 src/Storages/S3Queue/S3QueueSettings.h create mode 100644 src/Storages/S3Queue/S3QueueSource.cpp create mode 100644 src/Storages/S3Queue/S3QueueSource.h create mode 100644 src/Storages/S3Queue/StorageS3Queue.cpp create mode 100644 src/Storages/S3Queue/StorageS3Queue.h diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index f6727bad0e8..319a77727d7 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -250,6 +250,7 @@ add_object_library(clickhouse_storages_distributed Storages/Distributed) add_object_library(clickhouse_storages_mergetree Storages/MergeTree) add_object_library(clickhouse_storages_liveview Storages/LiveView) add_object_library(clickhouse_storages_windowview Storages/WindowView) +add_object_library(clickhouse_storages_s3queue Storages/S3Queue) add_object_library(clickhouse_client Client) add_object_library(clickhouse_bridge BridgeHelper) add_object_library(clickhouse_server Server) diff --git a/src/Storages/S3Queue/ReadBufferFromS3.cpp b/src/Storages/S3Queue/ReadBufferFromS3.cpp new file mode 100644 index 00000000000..c91c710bc9b --- /dev/null +++ b/src/Storages/S3Queue/ReadBufferFromS3.cpp @@ -0,0 +1,175 @@ +//#include +//#include +//#include +// +//#include +// +//#include +//#include +// +//namespace DB +//{ +//namespace ErrorCodes +//{ +// extern const int CANNOT_READ_ALL_DATA; +//} +// +//ReadBufferFromS3::ReadBufferFromS3( +// StorageS3Queue & storage_, +// size_t max_batch_size, +// size_t poll_timeout_, +// ContextPtr context_, +// size_t stream_number_, +// size_t max_streams_number_) +// : ReadBuffer(nullptr, 0) +// , log(&Poco::Logger::get("ReadBufferFromS3 " + toString(stream_number_))) +// , storage(storage_) +// , batch_size(max_batch_size) +// , poll_timeout(poll_timeout_) +// , context(context_) +// , stream_number(stream_number_) +// , max_streams_number(max_streams_number_) +//{ +// current = records.begin(); +// allowed = false; +//} +// +//bool ReadBufferFromS3::poll() +//{ +// if (hasMorePolledRecords()) +// { +// allowed = true; +// return true; +// } +// +// auto new_records = pollBatch(batch_size); +// if (new_records.empty()) +// { +// buffer_status = BufferStatus::NO_RECORD_RETURNED; +// LOG_TRACE(log, "No new records to read"); +// return false; +// } +// else +// { +// records = std::move(new_records); +// current = records.begin(); +// +// LOG_TRACE(log, "Polled batch of {} records. ", records.size()); +// +// buffer_status = BufferStatus::POLLED_OK; +// allowed = true; +// return true; +// } +//} +// +//ReadBufferFromS3::Records ReadBufferFromS3::pollBatch(size_t batch_size_) +//{ +// Records new_records; +// new_records.reserve(batch_size_); +// +// readNewRecords(new_records, batch_size); +// if (new_records.size() == batch_size_ || stream_out) +// return new_records; +// +// Stopwatch watch; +// while (watch.elapsedMilliseconds() < poll_timeout && new_records.size() != batch_size_) +// { +// readNewRecords(new_records, batch_size); +// /// All ifstrem reach end, no need to wait for timeout, +// /// since file status can not be updated during a streamToViews +// if (stream_out) +// break; +// } +// +// return new_records; +//} +// +//void ReadBufferFromS3::readNewRecords(ReadBufferFromS3::Records & /*new_records*/, size_t /*batch_size_*/) +//{ +//// size_t need_records_size = batch_size_ - new_records.size(); +//// size_t read_records_size = 0; +//// +//// auto & file_infos = storage.getFileInfos(); +//// +//// size_t files_per_stream = file_infos.file_names.size() / max_streams_number; +//// size_t start = stream_number * files_per_stream; +//// size_t end = stream_number == max_streams_number - 1 ? file_infos.file_names.size() : (stream_number + 1) * files_per_stream; +//// +//// for (size_t i = start; i < end; ++i) +//// { +//// const auto & file_name = file_infos.file_names[i]; +//// +//// auto & file_ctx = StorageFileLog::findInMap(file_infos.context_by_name, file_name); +//// if (file_ctx.status == StorageFileLog::FileStatus::NO_CHANGE) +//// continue; +//// +//// auto & file_meta = StorageFileLog::findInMap(file_infos.meta_by_inode, file_ctx.inode); +//// +//// if (!file_ctx.reader) +//// throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Ifstream for file {} is not initialized", file_meta.file_name); +//// +//// auto & reader = file_ctx.reader.value(); +//// StorageFileLog::assertStreamGood(reader); +//// +//// Record record; +//// while (read_records_size < need_records_size) +//// { +//// /// Need to get offset before reading record from stream +//// auto offset = reader.tellg(); +//// if (static_cast(offset) >= file_meta.last_open_end) +//// break; +//// record.offset = offset; +//// StorageFileLog::assertStreamGood(reader); +//// +//// record.file_name = file_name; +//// +//// +//// std::getline(reader, record.data); +//// StorageFileLog::assertStreamGood(reader); +//// +//// new_records.emplace_back(record); +//// ++read_records_size; +//// } +//// +//// UInt64 current_position = reader.tellg(); +//// StorageFileLog::assertStreamGood(reader); +//// +//// file_meta.last_writen_position = current_position; +//// +//// /// stream reach to end +//// if (current_position == file_meta.last_open_end) +//// { +//// file_ctx.status = StorageFileLog::FileStatus::NO_CHANGE; +//// } +//// +//// /// All ifstream reach end +//// if (i == end - 1 && (file_ctx.status == StorageFileLog::FileStatus::NO_CHANGE)) +//// { +//// stream_out = true; +//// } +//// +//// if (read_records_size == need_records_size) +//// { +//// break; +//// } +//// } +//} +// +//bool ReadBufferFromS3::nextImpl() +//{ +// if (!allowed || !hasMorePolledRecords()) +// return false; +// +// auto * new_position = const_cast(current->data.data()); +// BufferBase::set(new_position, current->data.size(), 0); +// allowed = false; +// +// current_file = current->file_name; +// current_offset = current->offset; +// +// ++current; +// +// return true; +//} +// +//} diff --git a/src/Storages/S3Queue/ReadBufferFromS3.h b/src/Storages/S3Queue/ReadBufferFromS3.h new file mode 100644 index 00000000000..3cf1d7711da --- /dev/null +++ b/src/Storages/S3Queue/ReadBufferFromS3.h @@ -0,0 +1,86 @@ +//#pragma once +// +//#include +//#include +//#include +// +//#include +//#include +// +//namespace DB +//{ +//class ReadBufferFromS3 : public ReadBuffer +//{ +//public: +// ReadBufferFromS3( +// StorageS3Queue & storage_, +// size_t max_batch_size, +// size_t poll_timeout_, +// ContextPtr context_, +// size_t stream_number_, +// size_t max_streams_number_); +// +// ~ReadBufferFromS3() override = default; +// +// auto pollTimeout() const { return poll_timeout; } +// +// bool hasMorePolledRecords() const { return current != records.end(); } +// +// bool poll(); +// +// bool noRecords() { return buffer_status == BufferStatus::NO_RECORD_RETURNED; } +// +// auto getFileName() const { return current_file; } +// auto getOffset() const { return current_offset; } +// +//private: +// enum class BufferStatus +// { +// INIT, +// NO_RECORD_RETURNED, +// POLLED_OK, +// }; +// +// BufferStatus buffer_status = BufferStatus::INIT; +// +// Poco::Logger * log; +// +// StorageS3Queue & storage; +// +// bool stream_out = false; +// +// size_t batch_size; +// size_t poll_timeout; +// +// ContextPtr context; +// +// size_t stream_number; +// size_t max_streams_number; +// +// bool allowed = true; +// +// using RecordData = std::string; +// struct Record +// { +// RecordData data; +// std::string file_name; +// /// Offset is the start of a row, which is needed for virtual columns. +// UInt64 offset; +// }; +// using Records = std::vector; +// +// Records records; +// Records::const_iterator current; +// +// String current_file; +// UInt64 current_offset = 0; +// +// using TaskThread = BackgroundSchedulePool::TaskHolder; +// +// Records pollBatch(size_t batch_size_); +// +// void readNewRecords(Records & new_records, size_t batch_size_); +// +// bool nextImpl() override; +//}; +//} diff --git a/src/Storages/S3Queue/S3QueueSettings.cpp b/src/Storages/S3Queue/S3QueueSettings.cpp new file mode 100644 index 00000000000..72bf5fb6742 --- /dev/null +++ b/src/Storages/S3Queue/S3QueueSettings.cpp @@ -0,0 +1,41 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNKNOWN_SETTING; +} + +IMPLEMENT_SETTINGS_TRAITS(S3QueueSettingsTraits, LIST_OF_S3QUEUE_SETTINGS) + +void S3QueueSettings::loadFromQuery(ASTStorage & storage_def) +{ + if (storage_def.settings) + { + try + { + applyChanges(storage_def.settings->changes); + } + catch (Exception & e) + { + if (e.code() == ErrorCodes::UNKNOWN_SETTING) + e.addMessage("for storage " + storage_def.engine->name); + throw; + } + } + else + { + auto settings_ast = std::make_shared(); + settings_ast->is_standalone = false; + storage_def.set(storage_def.settings, settings_ast); + } +} + +} diff --git a/src/Storages/S3Queue/S3QueueSettings.h b/src/Storages/S3Queue/S3QueueSettings.h new file mode 100644 index 00000000000..8494ae47a90 --- /dev/null +++ b/src/Storages/S3Queue/S3QueueSettings.h @@ -0,0 +1,28 @@ +#pragma once + +#include +#include + + +namespace DB +{ +class ASTStorage; + + +#define S3QUEUE_RELATED_SETTINGS(M, ALIAS) \ + M(String, mode, "unordered", "With unordered mode, the set of all already processed files is tracked with persistent nodes in ZooKepeer", 0) \ + M(String, keeper_path, "/", "Zookeeper node path", 0) \ + +#define LIST_OF_S3QUEUE_SETTINGS(M, ALIAS) \ + S3QUEUE_RELATED_SETTINGS(M, ALIAS) \ + FORMAT_FACTORY_SETTINGS(M, ALIAS) + +DECLARE_SETTINGS_TRAITS(S3QueueSettingsTraits, LIST_OF_S3QUEUE_SETTINGS) + + +struct S3QueueSettings : public BaseSettings +{ + void loadFromQuery(ASTStorage & storage_def); +}; + +} diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp new file mode 100644 index 00000000000..8e0d970729b --- /dev/null +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -0,0 +1,132 @@ +//#include +//#include +//#include +//#include +//#include +//#include +//#include +// +//namespace DB +//{ +//static constexpr auto MAX_FAILED_POLL_ATTEMPTS = 10; +// +//S3QueueSource::S3QueueSource( +// StorageS3Queue & storage_, +// const StorageSnapshotPtr & storage_snapshot_, +// const ContextPtr & context_, +// const Names & columns, +// size_t max_block_size_, +// size_t poll_time_out_, +// size_t stream_number_, +// size_t max_streams_number_) +// : ISource(storage_snapshot_->getSampleBlockForColumns(columns)) +// , storage(storage_) +// , storage_snapshot(storage_snapshot_) +// , context(context_) +// , column_names(columns) +// , max_block_size(max_block_size_) +// , poll_time_out(poll_time_out_) +// , stream_number(stream_number_) +// , max_streams_number(max_streams_number_) +// , non_virtual_header(storage_snapshot->metadata->getSampleBlockNonMaterialized()) +// , virtual_header(storage_snapshot->getSampleBlockForColumns(storage.getVirtualColumnNames())) +//{ +// buffer = std::make_unique(storage, max_block_size, poll_time_out, context, stream_number_, max_streams_number_); +//} +// +//S3QueueSource::~S3QueueSource() +//{ +// try +// { +// if (!finished) +// onFinish(); +// } +// catch (...) +// { +// tryLogCurrentException(__PRETTY_FUNCTION__); +// } +//} +// +//void S3QueueSource::onFinish() +//{ +// finished = true; +//} +// +//Chunk S3QueueSource::generate() +//{ +// /// Store metas of last written chunk into disk +// // storage.storeMetas(start, end); +// +// if (!buffer || buffer->noRecords()) +// { +// /// There is no onFinish for ISource, we call it +// /// when no records return to close files +// onFinish(); +// return {}; +// } +// +// MutableColumns virtual_columns = virtual_header.cloneEmptyColumns(); +// +// auto input_format +// = FormatFactory::instance().getInputFormat(storage.getFormatName(), *buffer, non_virtual_header, context, max_block_size); +// +// StreamingFormatExecutor executor(non_virtual_header, input_format); +// +// size_t total_rows = 0; +// size_t failed_poll_attempts = 0; +// +// Stopwatch watch; +// while (true) +// { +// size_t new_rows = 0; +// if (buffer->poll()) +// new_rows = executor.execute(); +// +// if (new_rows) +// { +// auto file_name = buffer->getFileName(); +// auto offset = buffer->getOffset(); +// for (size_t i = 0; i < new_rows; ++i) +// { +// virtual_columns[0]->insert(file_name); +// virtual_columns[1]->insert(offset); +// } +// total_rows = total_rows + new_rows; +// } +// else /// poll succeed, but parse failed +// { +// ++failed_poll_attempts; +// } +// +// if (!buffer->hasMorePolledRecords() +// && ((total_rows >= max_block_size) || watch.elapsedMilliseconds() > poll_time_out +// || failed_poll_attempts >= MAX_FAILED_POLL_ATTEMPTS)) +// { +// break; +// } +// } +// +// if (total_rows == 0) +// { +// onFinish(); +// return {}; +// } +// +// auto result_block = non_virtual_header.cloneWithColumns(executor.getResultColumns()); +// auto virtual_block = virtual_header.cloneWithColumns(std::move(virtual_columns)); +// +// for (const auto & column : virtual_block.getColumnsWithTypeAndName()) +// result_block.insert(column); +// +// auto converting_dag = ActionsDAG::makeConvertingActions( +// result_block.cloneEmpty().getColumnsWithTypeAndName(), +// getPort().getHeader().getColumnsWithTypeAndName(), +// ActionsDAG::MatchColumnsMode::Name); +// +// auto converting_actions = std::make_shared(std::move(converting_dag)); +// converting_actions->execute(result_block); +// +// return Chunk(result_block.getColumns(), result_block.rows()); +//} +// +//} diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h new file mode 100644 index 00000000000..4714e23f748 --- /dev/null +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -0,0 +1,55 @@ +//#pragma once +// +//#include +//#include +//#include +// +//namespace Poco +//{ +//class Logger; +//} +//namespace DB +//{ +//class S3QueueSource : public ISource +//{ +//public: +// S3QueueSource( +// StorageS3Queue & storage_, +// const StorageSnapshotPtr & storage_snapshot_, +// const ContextPtr & context_, +// const Names & columns, +// size_t max_block_size_, +// size_t poll_time_out_, +// size_t stream_number_, +// size_t max_streams_number_); +// +// String getName() const override { return "S3Queue"; } +// +// bool noRecords() { return !buffer || buffer->noRecords(); } +// +// void onFinish(); +// +// virtual ~S3QueueSource() override; +// +//protected: +// Chunk generate() override; +// +//private: +// StorageS3Queue & storage; +// StorageSnapshotPtr storage_snapshot; +// ContextPtr context; +// Names column_names; +// UInt64 max_block_size; +// +// size_t poll_time_out; +// +// size_t stream_number; +// size_t max_streams_number; +// +// std::unique_ptr buffer; +// +// Block non_virtual_header; +// Block virtual_header; +//}; +// +//} diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp new file mode 100644 index 00000000000..ae162b90906 --- /dev/null +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -0,0 +1,578 @@ +#include "config.h" +#include +#include "IO/ParallelReadBuffer.h" +#include "IO/IOThreadPool.h" +#include "Parsers/ASTCreateQuery.h" + +#if USE_AWS_S3 + +#include +#include + +#include + +#include +#include + +#include +#include + +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +#include +#include +#include + +#include +#include + +#include +#include + +#include +#include +#include +#include + +#include + +#include + +#include + +#include +#include +#include + +#include +#include +#include +#include + +namespace fs = std::filesystem; + + +namespace ProfileEvents +{ +extern const Event S3DeleteObjects; +extern const Event S3ListObjects; +} + +namespace DB +{ + +static const String PARTITION_ID_WILDCARD = "{_partition_id}"; + +static const std::unordered_set required_configuration_keys = { + "url", +}; +static const std::unordered_set optional_configuration_keys = { + "format", + "compression", + "compression_method", + "structure", + "access_key_id", + "secret_access_key", + "filename", + "use_environment_credentials", + "max_single_read_retries", + "min_upload_part_size", + "upload_part_size_multiply_factor", + "upload_part_size_multiply_parts_count_threshold", + "max_single_part_upload_size", + "max_connections", +}; + +namespace ErrorCodes +{ + extern const int CANNOT_PARSE_TEXT; + extern const int BAD_ARGUMENTS; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int S3_ERROR; + extern const int UNEXPECTED_EXPRESSION; + extern const int DATABASE_ACCESS_DENIED; + extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; + extern const int NOT_IMPLEMENTED; + extern const int CANNOT_COMPILE_REGEXP; + extern const int FILE_DOESNT_EXIST; + extern const int QUERY_NOT_ALLOWED; + extern const int NO_ZOOKEEPER; + extern const int REPLICA_ALREADY_EXISTS; +} + +class IOutputFormat; +using OutputFormatPtr = std::shared_ptr; + +const String StorageS3Queue::default_zookeeper_name = "default"; + + +StorageS3Queue::StorageS3Queue( + const String & zookeeper_path_, + const String & mode_, + const StorageS3::Configuration & configuration_, + const StorageID & table_id_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + const String & comment, + ContextPtr context_, + std::optional format_settings_, + bool distributed_processing_, + ASTPtr partition_by_) + : IStorage(table_id_) + , WithContext(context_) + , s3_configuration{configuration_} + , keys({s3_configuration.url.key}) + , format_name(configuration_.format) + , compression_method(configuration_.compression_method) + , name(s3_configuration.url.storage_name) + , distributed_processing(distributed_processing_) + , format_settings(format_settings_) + , partition_by(partition_by_) + , is_key_with_globs(s3_configuration.url.key.find_first_of("*?{") != std::string::npos) + , log(&Poco::Logger::get("StorageS3Queue (" + table_id_.table_name + ")")) + , mode(mode_) + , zookeeper_name(zkutil::extractZooKeeperName(zookeeper_path_)) + , zookeeper_path(zkutil::extractZooKeeperPath(zookeeper_path_, /* check_starts_with_slash */ true, log)) +{ + FormatFactory::instance().checkFormatName(format_name); + context_->getGlobalContext()->getRemoteHostFilter().checkURL(s3_configuration.url.uri); + + StorageInMemoryMetadata storage_metadata; + + StorageS3::updateConfiguration(context_, s3_configuration); + if (columns_.empty()) + { + auto columns = StorageS3::getTableStructureFromDataImpl( + format_name, + s3_configuration, + compression_method, + is_key_with_globs, + format_settings, + context_); + + storage_metadata.setColumns(columns); + } + else + storage_metadata.setColumns(columns_); + + storage_metadata.setConstraints(constraints_); + storage_metadata.setComment(comment); + setInMemoryMetadata(storage_metadata); + + auto default_virtuals = NamesAndTypesList{ + {"_path", std::make_shared(std::make_shared())}, + {"_file", std::make_shared(std::make_shared())}}; + + auto columns = storage_metadata.getSampleBlock().getNamesAndTypesList(); + virtual_columns = getVirtualsForStorage(columns, default_virtuals); + for (const auto & column : virtual_columns) + virtual_block.insert({column.type->createColumn(), column.type, column.name}); + + auto thread = context_->getSchedulePool().createTask("S3QueueStreamingTask", [this] { threadFunc(); }); + setZooKeeper(); + auto metadata_snapshot = getInMemoryMetadataPtr(); + createTableIfNotExists(metadata_snapshot); + task = std::make_shared(std::move(thread)); +} + + +bool StorageS3Queue::supportsSubcolumns() const +{ + return FormatFactory::instance().checkIfFormatSupportsSubcolumns(format_name); +} + +bool StorageS3Queue::supportsSubsetOfColumns() const +{ + return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(format_name); +} + +Pipe StorageS3Queue::read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr local_context, + QueryProcessingStage::Enum /*processed_stage*/, + size_t max_block_size, + size_t num_streams) +{ +// if (!local_context->getSettingsRef().stream_like_engine_allow_direct_select) +// throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, +// "Direct select is not allowed. To enable use setting `stream_like_engine_allow_direct_select`"); + + if (mv_attached) + throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "Cannot read from StorageRabbitMQ with attached materialized views"); + + auto query_s3_configuration = StorageS3::copyAndUpdateConfiguration(local_context, s3_configuration); + + bool has_wildcards = + query_s3_configuration.url.bucket.find(PARTITION_ID_WILDCARD) != String::npos + || keys.back().find(PARTITION_ID_WILDCARD) != String::npos; + + if (partition_by && has_wildcards) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Reading from a partitioned S3 storage is not implemented yet"); + + Pipes pipes; + + std::unordered_set column_names_set(column_names.begin(), column_names.end()); + std::vector requested_virtual_columns; + + for (const auto & virtual_column : getVirtuals()) + { + if (column_names_set.contains(virtual_column.name)) + requested_virtual_columns.push_back(virtual_column); + } + + std::shared_ptr iterator_wrapper = StorageS3::createFileIterator( + query_s3_configuration, + keys, + is_key_with_globs, + distributed_processing, + local_context, + query_info.query, + virtual_block); + + ColumnsDescription columns_description; + Block block_for_format; + if (supportsSubsetOfColumns()) + { + auto fetch_columns = column_names; + const auto & virtuals = getVirtuals(); + std::erase_if( + fetch_columns, + [&](const String & col) + { return std::any_of(virtuals.begin(), virtuals.end(), [&](const NameAndTypePair & virtual_col){ return col == virtual_col.name; }); }); + + if (fetch_columns.empty()) + fetch_columns.push_back(ExpressionActions::getSmallestColumn(storage_snapshot->metadata->getColumns().getAllPhysical()).name); + + columns_description = storage_snapshot->getDescriptionForColumns(fetch_columns); + block_for_format = storage_snapshot->getSampleBlockForColumns(columns_description.getNamesOfPhysical()); + } + else + { + columns_description = storage_snapshot->metadata->getColumns(); + block_for_format = storage_snapshot->metadata->getSampleBlock(); + } + + const size_t max_download_threads = local_context->getSettingsRef().max_download_threads; + // const size_t max_download_threads = 1; + LOG_WARNING(log, "num_streams"); + + for (size_t i = 0; i < num_streams; ++i) + { + pipes.emplace_back(std::make_shared( + requested_virtual_columns, + format_name, + getName(), + block_for_format, + local_context, + format_settings, + columns_description, + max_block_size, + query_s3_configuration.request_settings, + compression_method, + query_s3_configuration.client, + query_s3_configuration.url.bucket, + query_s3_configuration.url.version_id, + iterator_wrapper, + max_download_threads)); + } + + auto pipe = Pipe::unitePipes(std::move(pipes)); + LOG_WARNING(log, "unitePipes"); + + narrowPipe(pipe, num_streams); + LOG_WARNING(log, "narrowPipe"); + + return pipe; +} + +NamesAndTypesList StorageS3Queue::getVirtuals() const +{ + return virtual_columns; +} + +Names StorageS3Queue::getVirtualColumnNames() +{ + return {"_path", "_file"}; +} + +bool StorageS3Queue::supportsPartitionBy() const +{ + return true; +} + +void StorageS3Queue::startup() +{ + if (task) + task->holder->activateAndSchedule(); +} + +void StorageS3Queue::shutdown() +{ + shutdown_called = true; + LOG_TRACE(log, "Deactivating background tasks"); + + if (task) + { + task->stream_cancelled = true; + + /// Reader thread may wait for wake up +// wakeUp(); + + LOG_TRACE(log, "Waiting for cleanup"); + task->holder->deactivate(); + /// If no reading call and threadFunc, the log files will never + /// be opened, also just leave the work of close files and + /// store meta to streams. because if we close files in here, + /// may result in data race with unfinishing reading pipeline + } +} + +size_t StorageS3Queue::getTableDependentCount() const +{ + auto table_id = getStorageID(); + // Check if at least one direct dependency is attached + return DatabaseCatalog::instance().getDependentViews(table_id).size(); +} + +bool StorageS3Queue::hasDependencies(const StorageID & table_id) +{ + // Check if all dependencies are attached + auto view_ids = DatabaseCatalog::instance().getDependentViews(table_id); + LOG_TEST(log, "Number of attached views {} for {}", view_ids.size(), table_id.getNameForLogs()); + + if (view_ids.empty()) + return false; + + // Check the dependencies are ready? + for (const auto & view_id : view_ids) + { + auto view = DatabaseCatalog::instance().tryGetTable(view_id, getContext()); + if (!view) + return false; + + // If it materialized view, check it's target table + auto * materialized_view = dynamic_cast(view.get()); + if (materialized_view && !materialized_view->tryGetTargetTable()) + return false; + } + + return true; +} + +void StorageS3Queue::threadFunc() +{ + bool reschedule = true; + try + { + auto table_id = getStorageID(); + + auto dependencies_count = getTableDependentCount(); + + if (dependencies_count) + { + // auto start_time = std::chrono::steady_clock::now(); + + mv_attached.store(true); + // Keep streaming as long as there are attached views and streaming is not cancelled + while (!task->stream_cancelled) + { + if (!hasDependencies(table_id)) + { + /// For this case, we can not wait for watch thread to wake up + reschedule = true; + break; + } + + LOG_DEBUG(log, "Started streaming to {} attached views", dependencies_count); + + // if (streamToViews()) + // { + // LOG_TRACE(log, "Stream stalled. Reschedule."); + // break; + // } + + // auto ts = std::chrono::steady_clock::now(); + // auto duration = std::chrono::duration_cast(ts-start_time); + // if (duration.count() > MAX_THREAD_WORK_DURATION_MS) + // { + // LOG_TRACE(log, "Thread work duration limit exceeded. Reschedule."); + // reschedule = true; + // break; + // } + } + } + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + + mv_attached.store(false); + + // Wait for attached views + if (reschedule && !shutdown_called) + { + LOG_TRACE(log, "Reschedule S3 Queue thread func."); + /// Reschedule with backoff. + task->holder->scheduleAfter(milliseconds_to_wait); + } +} + + +void StorageS3Queue::setZooKeeper() +{ + std::lock_guard lock(current_zookeeper_mutex); + LOG_WARNING(log, "zookeeper name {}", zookeeper_name); + if (zookeeper_name == default_zookeeper_name) + { + current_zookeeper = getContext()->getZooKeeper(); + } + else + { + current_zookeeper = getContext()->getAuxiliaryZooKeeper(zookeeper_name); + } +} + +zkutil::ZooKeeperPtr StorageS3Queue::tryGetZooKeeper() const +{ + std::lock_guard lock(current_zookeeper_mutex); + return current_zookeeper; +} + +zkutil::ZooKeeperPtr StorageS3Queue::getZooKeeper() const +{ + auto res = tryGetZooKeeper(); + if (!res) + throw Exception(ErrorCodes::NO_ZOOKEEPER, "Cannot get ZooKeeper"); + return res; +} + + +bool StorageS3Queue::createTableIfNotExists(const StorageMetadataPtr & metadata_snapshot) +{ + auto zookeeper = getZooKeeper(); + zookeeper->createAncestors(zookeeper_path); + + for (size_t i = 0; i < 1000; ++i) + { + if (zookeeper->exists(zookeeper_path + "")) + { + LOG_DEBUG(log, "This table {} is already created, will add new replica", zookeeper_path); + return false; + } + /// We write metadata of table so that the replicas can check table parameters with them. + // String metadata_str = ReplicatedMergeTreeTableMetadata(*this, metadata_snapshot).toString(); + + Coordination::Requests ops; + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path, "", zkutil::CreateMode::Persistent)); + + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processed", "", + zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/failed", "", + zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processing", "", + zkutil::CreateMode::Ephemeral)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/columns", metadata_snapshot->getColumns().toString(), + zkutil::CreateMode::Persistent)); +// ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/metadata", metadata_str, +// zkutil::CreateMode::Persistent)); + + Coordination::Responses responses; + auto code = zookeeper->tryMulti(ops, responses); + if (code == Coordination::Error::ZNODEEXISTS) + { + LOG_INFO(log, "It looks like the table {} was created by another server at the same moment, will retry", zookeeper_path); + continue; + } + else if (code != Coordination::Error::ZOK) + { + zkutil::KeeperMultiException::check(code, ops, responses); + } + + return true; + } + + /// Do not use LOGICAL_ERROR code, because it may happen if user has specified wrong zookeeper_path + throw Exception(ErrorCodes::REPLICA_ALREADY_EXISTS, + "Cannot create table, because it is created concurrently every time or because " + "of wrong zookeeper_path or because of logical error"); +} + +void registerStorageS3QueueImpl(const String & name, StorageFactory & factory) +{ + factory.registerStorage(name, [](const StorageFactory::Arguments & args) + { + auto & engine_args = args.engine_args; + if (engine_args.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments"); + + auto configuration = StorageS3::getConfiguration(engine_args, args.getLocalContext()); + // Use format settings from global server context + settings from + // the SETTINGS clause of the create query. Settings from current + // session and user are ignored. + auto s3queue_settings = std::make_unique(); + std::optional format_settings; + + if (args.storage_def->settings) + { + s3queue_settings->loadFromQuery(*args.storage_def); + + } + format_settings = getFormatSettings(args.getContext()); + + ASTPtr partition_by; + if (args.storage_def->partition_by) + partition_by = args.storage_def->partition_by->clone(); + + String keeper_path = s3queue_settings->keeper_path; + String mode = s3queue_settings->mode; + + return std::make_shared( + keeper_path, + mode, + configuration, + args.table_id, + args.columns, + args.constraints, + args.comment, + args.getContext(), + format_settings, + /* distributed_processing_ */false, + partition_by); + }, + { + .supports_settings = true, + .supports_sort_order = true, // for partition by + .supports_schema_inference = true, + .source_access_type = AccessType::S3, + }); +} + +void registerStorageS3Queue(StorageFactory & factory) +{ + return registerStorageS3QueueImpl("S3Queue", factory); +} + +} + + +#endif diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h new file mode 100644 index 00000000000..9c1390c24a8 --- /dev/null +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -0,0 +1,152 @@ +#pragma once + +#include "config.h" + +#if USE_AWS_S3 + +#include + +#include +#include + +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace Aws::S3 +{ +class Client; +} + +namespace DB +{ + + + +class StorageS3Queue : public IStorage, WithContext +{ +public: + using Configuration = typename StorageS3::Configuration; + StorageS3Queue( + const String & zookeper_path_, + const String & mode_, + const Configuration & configuration_, + const StorageID & table_id_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + const String & comment, + ContextPtr context_, + std::optional 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; + + SinkToStoragePtr write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr /*context*/) override { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Write is not supported by storage {}", getName()); + } + + void truncate(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr /*local_context*/, TableExclusiveLockHolder &) override { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Truncate is not supported by storage {}", getName()); + } + + NamesAndTypesList getVirtuals() const override; + + bool supportsPartitionBy() const override; + + static ColumnsDescription getTableStructureFromData( + Configuration & configuration, const std::optional & format_settings, ContextPtr ctx) + { + return StorageS3::getTableStructureFromData(configuration, format_settings, ctx); + } + + const auto & getFormatName() const { return format_name; } + +private: + + Configuration s3_configuration; + std::vector 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 format_settings; + ASTPtr partition_by; + bool is_key_with_globs = false; + + bool supportsSubcolumns() const override; + + void threadFunc(); + size_t getTableDependentCount() const; + std::atomic mv_attached = false; + bool hasDependencies(const StorageID & table_id); + std::atomic shutdown_called{false}; + Poco::Logger * log; + + + void startup() override; + void shutdown() override; + + struct TaskContext + { + BackgroundSchedulePool::TaskHolder holder; + std::atomic stream_cancelled {false}; + explicit TaskContext(BackgroundSchedulePool::TaskHolder&& task_) : holder(std::move(task_)) + { + } + }; + std::shared_ptr task; + + bool supportsSubsetOfColumns() const override; + static Names getVirtualColumnNames(); + + const String mode; + + static const String default_zookeeper_name; + const String zookeeper_name; + const String zookeeper_path; + const String replica_name; + const String replica_path; + + zkutil::ZooKeeperPtr current_zookeeper; + mutable std::mutex current_zookeeper_mutex; + + 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 & getZooKeeperName() const { return zookeeper_name; } + const String & getZooKeeperPath() const { return zookeeper_path; } + +}; + +} + +#endif diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index f4d915e9c55..131649ff5fd 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -304,6 +304,7 @@ protected: private: friend class StorageS3Cluster; friend class TableFunctionS3Cluster; + friend class StorageS3Queue; Configuration s3_configuration; std::vector keys; diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index e690189c365..61c415a3f9d 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -35,6 +35,7 @@ void registerStorageCOS(StorageFactory & factory); void registerStorageOSS(StorageFactory & factory); void registerStorageHudi(StorageFactory & factory); void registerStorageDeltaLake(StorageFactory & factory); +void registerStorageS3Queue(StorageFactory & factory); #if USE_AVRO void registerStorageIceberg(StorageFactory & factory); #endif @@ -127,6 +128,7 @@ void registerStorages() registerStorageOSS(factory); registerStorageHudi(factory); registerStorageDeltaLake(factory); + registerStorageS3Queue(factory); #if USE_AVRO registerStorageIceberg(factory); From 2b7a403752b523e5384cfe42d76907a3c097386a Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Thu, 27 Apr 2023 14:55:23 +0300 Subject: [PATCH 02/34] wip: select and streaming to views --- src/Storages/S3Queue/ReadBufferFromS3.cpp | 175 ------- src/Storages/S3Queue/ReadBufferFromS3.h | 86 --- src/Storages/S3Queue/S3QueueSource.cpp | 608 +++++++++++++++++----- src/Storages/S3Queue/S3QueueSource.h | 201 +++++-- src/Storages/S3Queue/StorageS3Queue.cpp | 277 ++++++++-- src/Storages/S3Queue/StorageS3Queue.h | 15 +- src/Storages/StorageS3.h | 2 + 7 files changed, 871 insertions(+), 493 deletions(-) delete mode 100644 src/Storages/S3Queue/ReadBufferFromS3.cpp delete mode 100644 src/Storages/S3Queue/ReadBufferFromS3.h diff --git a/src/Storages/S3Queue/ReadBufferFromS3.cpp b/src/Storages/S3Queue/ReadBufferFromS3.cpp deleted file mode 100644 index c91c710bc9b..00000000000 --- a/src/Storages/S3Queue/ReadBufferFromS3.cpp +++ /dev/null @@ -1,175 +0,0 @@ -//#include -//#include -//#include -// -//#include -// -//#include -//#include -// -//namespace DB -//{ -//namespace ErrorCodes -//{ -// extern const int CANNOT_READ_ALL_DATA; -//} -// -//ReadBufferFromS3::ReadBufferFromS3( -// StorageS3Queue & storage_, -// size_t max_batch_size, -// size_t poll_timeout_, -// ContextPtr context_, -// size_t stream_number_, -// size_t max_streams_number_) -// : ReadBuffer(nullptr, 0) -// , log(&Poco::Logger::get("ReadBufferFromS3 " + toString(stream_number_))) -// , storage(storage_) -// , batch_size(max_batch_size) -// , poll_timeout(poll_timeout_) -// , context(context_) -// , stream_number(stream_number_) -// , max_streams_number(max_streams_number_) -//{ -// current = records.begin(); -// allowed = false; -//} -// -//bool ReadBufferFromS3::poll() -//{ -// if (hasMorePolledRecords()) -// { -// allowed = true; -// return true; -// } -// -// auto new_records = pollBatch(batch_size); -// if (new_records.empty()) -// { -// buffer_status = BufferStatus::NO_RECORD_RETURNED; -// LOG_TRACE(log, "No new records to read"); -// return false; -// } -// else -// { -// records = std::move(new_records); -// current = records.begin(); -// -// LOG_TRACE(log, "Polled batch of {} records. ", records.size()); -// -// buffer_status = BufferStatus::POLLED_OK; -// allowed = true; -// return true; -// } -//} -// -//ReadBufferFromS3::Records ReadBufferFromS3::pollBatch(size_t batch_size_) -//{ -// Records new_records; -// new_records.reserve(batch_size_); -// -// readNewRecords(new_records, batch_size); -// if (new_records.size() == batch_size_ || stream_out) -// return new_records; -// -// Stopwatch watch; -// while (watch.elapsedMilliseconds() < poll_timeout && new_records.size() != batch_size_) -// { -// readNewRecords(new_records, batch_size); -// /// All ifstrem reach end, no need to wait for timeout, -// /// since file status can not be updated during a streamToViews -// if (stream_out) -// break; -// } -// -// return new_records; -//} -// -//void ReadBufferFromS3::readNewRecords(ReadBufferFromS3::Records & /*new_records*/, size_t /*batch_size_*/) -//{ -//// size_t need_records_size = batch_size_ - new_records.size(); -//// size_t read_records_size = 0; -//// -//// auto & file_infos = storage.getFileInfos(); -//// -//// size_t files_per_stream = file_infos.file_names.size() / max_streams_number; -//// size_t start = stream_number * files_per_stream; -//// size_t end = stream_number == max_streams_number - 1 ? file_infos.file_names.size() : (stream_number + 1) * files_per_stream; -//// -//// for (size_t i = start; i < end; ++i) -//// { -//// const auto & file_name = file_infos.file_names[i]; -//// -//// auto & file_ctx = StorageFileLog::findInMap(file_infos.context_by_name, file_name); -//// if (file_ctx.status == StorageFileLog::FileStatus::NO_CHANGE) -//// continue; -//// -//// auto & file_meta = StorageFileLog::findInMap(file_infos.meta_by_inode, file_ctx.inode); -//// -//// if (!file_ctx.reader) -//// throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Ifstream for file {} is not initialized", file_meta.file_name); -//// -//// auto & reader = file_ctx.reader.value(); -//// StorageFileLog::assertStreamGood(reader); -//// -//// Record record; -//// while (read_records_size < need_records_size) -//// { -//// /// Need to get offset before reading record from stream -//// auto offset = reader.tellg(); -//// if (static_cast(offset) >= file_meta.last_open_end) -//// break; -//// record.offset = offset; -//// StorageFileLog::assertStreamGood(reader); -//// -//// record.file_name = file_name; -//// -//// -//// std::getline(reader, record.data); -//// StorageFileLog::assertStreamGood(reader); -//// -//// new_records.emplace_back(record); -//// ++read_records_size; -//// } -//// -//// UInt64 current_position = reader.tellg(); -//// StorageFileLog::assertStreamGood(reader); -//// -//// file_meta.last_writen_position = current_position; -//// -//// /// stream reach to end -//// if (current_position == file_meta.last_open_end) -//// { -//// file_ctx.status = StorageFileLog::FileStatus::NO_CHANGE; -//// } -//// -//// /// All ifstream reach end -//// if (i == end - 1 && (file_ctx.status == StorageFileLog::FileStatus::NO_CHANGE)) -//// { -//// stream_out = true; -//// } -//// -//// if (read_records_size == need_records_size) -//// { -//// break; -//// } -//// } -//} -// -//bool ReadBufferFromS3::nextImpl() -//{ -// if (!allowed || !hasMorePolledRecords()) -// return false; -// -// auto * new_position = const_cast(current->data.data()); -// BufferBase::set(new_position, current->data.size(), 0); -// allowed = false; -// -// current_file = current->file_name; -// current_offset = current->offset; -// -// ++current; -// -// return true; -//} -// -//} diff --git a/src/Storages/S3Queue/ReadBufferFromS3.h b/src/Storages/S3Queue/ReadBufferFromS3.h deleted file mode 100644 index 3cf1d7711da..00000000000 --- a/src/Storages/S3Queue/ReadBufferFromS3.h +++ /dev/null @@ -1,86 +0,0 @@ -//#pragma once -// -//#include -//#include -//#include -// -//#include -//#include -// -//namespace DB -//{ -//class ReadBufferFromS3 : public ReadBuffer -//{ -//public: -// ReadBufferFromS3( -// StorageS3Queue & storage_, -// size_t max_batch_size, -// size_t poll_timeout_, -// ContextPtr context_, -// size_t stream_number_, -// size_t max_streams_number_); -// -// ~ReadBufferFromS3() override = default; -// -// auto pollTimeout() const { return poll_timeout; } -// -// bool hasMorePolledRecords() const { return current != records.end(); } -// -// bool poll(); -// -// bool noRecords() { return buffer_status == BufferStatus::NO_RECORD_RETURNED; } -// -// auto getFileName() const { return current_file; } -// auto getOffset() const { return current_offset; } -// -//private: -// enum class BufferStatus -// { -// INIT, -// NO_RECORD_RETURNED, -// POLLED_OK, -// }; -// -// BufferStatus buffer_status = BufferStatus::INIT; -// -// Poco::Logger * log; -// -// StorageS3Queue & storage; -// -// bool stream_out = false; -// -// size_t batch_size; -// size_t poll_timeout; -// -// ContextPtr context; -// -// size_t stream_number; -// size_t max_streams_number; -// -// bool allowed = true; -// -// using RecordData = std::string; -// struct Record -// { -// RecordData data; -// std::string file_name; -// /// Offset is the start of a row, which is needed for virtual columns. -// UInt64 offset; -// }; -// using Records = std::vector; -// -// Records records; -// Records::const_iterator current; -// -// String current_file; -// UInt64 current_offset = 0; -// -// using TaskThread = BackgroundSchedulePool::TaskHolder; -// -// Records pollBatch(size_t batch_size_); -// -// void readNewRecords(Records & new_records, size_t batch_size_); -// -// bool nextImpl() override; -//}; -//} diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 8e0d970729b..9272d4f4c02 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -1,132 +1,476 @@ -//#include -//#include -//#include -//#include -//#include -//#include -//#include -// -//namespace DB -//{ -//static constexpr auto MAX_FAILED_POLL_ATTEMPTS = 10; -// -//S3QueueSource::S3QueueSource( -// StorageS3Queue & storage_, -// const StorageSnapshotPtr & storage_snapshot_, -// const ContextPtr & context_, -// const Names & columns, -// size_t max_block_size_, -// size_t poll_time_out_, -// size_t stream_number_, -// size_t max_streams_number_) -// : ISource(storage_snapshot_->getSampleBlockForColumns(columns)) -// , storage(storage_) -// , storage_snapshot(storage_snapshot_) -// , context(context_) -// , column_names(columns) -// , max_block_size(max_block_size_) -// , poll_time_out(poll_time_out_) -// , stream_number(stream_number_) -// , max_streams_number(max_streams_number_) -// , non_virtual_header(storage_snapshot->metadata->getSampleBlockNonMaterialized()) -// , virtual_header(storage_snapshot->getSampleBlockForColumns(storage.getVirtualColumnNames())) -//{ -// buffer = std::make_unique(storage, max_block_size, poll_time_out, context, stream_number_, max_streams_number_); -//} -// -//S3QueueSource::~S3QueueSource() -//{ -// try -// { -// if (!finished) -// onFinish(); -// } -// catch (...) -// { -// tryLogCurrentException(__PRETTY_FUNCTION__); -// } -//} -// -//void S3QueueSource::onFinish() -//{ -// finished = true; -//} -// -//Chunk S3QueueSource::generate() -//{ -// /// Store metas of last written chunk into disk -// // storage.storeMetas(start, end); -// -// if (!buffer || buffer->noRecords()) -// { -// /// There is no onFinish for ISource, we call it -// /// when no records return to close files -// onFinish(); -// return {}; -// } -// -// MutableColumns virtual_columns = virtual_header.cloneEmptyColumns(); -// -// auto input_format -// = FormatFactory::instance().getInputFormat(storage.getFormatName(), *buffer, non_virtual_header, context, max_block_size); -// -// StreamingFormatExecutor executor(non_virtual_header, input_format); -// -// size_t total_rows = 0; -// size_t failed_poll_attempts = 0; -// -// Stopwatch watch; -// while (true) -// { -// size_t new_rows = 0; -// if (buffer->poll()) -// new_rows = executor.execute(); -// -// if (new_rows) -// { -// auto file_name = buffer->getFileName(); -// auto offset = buffer->getOffset(); -// for (size_t i = 0; i < new_rows; ++i) -// { -// virtual_columns[0]->insert(file_name); -// virtual_columns[1]->insert(offset); -// } -// total_rows = total_rows + new_rows; -// } -// else /// poll succeed, but parse failed -// { -// ++failed_poll_attempts; -// } -// -// if (!buffer->hasMorePolledRecords() -// && ((total_rows >= max_block_size) || watch.elapsedMilliseconds() > poll_time_out -// || failed_poll_attempts >= MAX_FAILED_POLL_ATTEMPTS)) -// { -// break; -// } -// } -// -// if (total_rows == 0) -// { -// onFinish(); -// return {}; -// } -// -// auto result_block = non_virtual_header.cloneWithColumns(executor.getResultColumns()); -// auto virtual_block = virtual_header.cloneWithColumns(std::move(virtual_columns)); -// -// for (const auto & column : virtual_block.getColumnsWithTypeAndName()) -// result_block.insert(column); -// -// auto converting_dag = ActionsDAG::makeConvertingActions( -// result_block.cloneEmpty().getColumnsWithTypeAndName(), -// getPort().getHeader().getColumnsWithTypeAndName(), -// ActionsDAG::MatchColumnsMode::Name); -// -// auto converting_actions = std::make_shared(std::move(converting_dag)); -// converting_actions->execute(result_block); -// -// return Chunk(result_block.getColumns(), result_block.rows()); -//} -// -//} +#include "config.h" +#include +#include "IO/ParallelReadBuffer.h" +#include "IO/IOThreadPool.h" +#include "Parsers/ASTCreateQuery.h" +#include +#include + +#if USE_AWS_S3 + +#include + +#include + +#include +#include + +#include +#include + +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include + +#include +#include + +#include +#include + +#include +#include +#include +#include + +#include + +#include + +#include + +#include +#include +#include +#include +#include + +#include +#include +#include +#include + +#include + +namespace fs = std::filesystem; + + +namespace CurrentMetrics +{ +extern const Metric StorageS3Threads; +extern const Metric StorageS3ThreadsActive; +} + +namespace ProfileEvents +{ +extern const Event S3DeleteObjects; +extern const Event S3ListObjects; +} + +namespace DB +{ + +static const String PARTITION_ID_WILDCARD = "{_partition_id}"; + +static const std::unordered_set required_configuration_keys = { + "url", +}; +static const std::unordered_set optional_configuration_keys = { + "format", + "compression", + "compression_method", + "structure", + "access_key_id", + "secret_access_key", + "filename", + "use_environment_credentials", + "max_single_read_retries", + "min_upload_part_size", + "upload_part_size_multiply_factor", + "upload_part_size_multiply_parts_count_threshold", + "max_single_part_upload_size", + "max_connections", + "expiration_window_seconds", + "no_sign_request" +}; + +namespace ErrorCodes +{ + extern const int CANNOT_PARSE_TEXT; + extern const int BAD_ARGUMENTS; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int S3_ERROR; + extern const int UNEXPECTED_EXPRESSION; + extern const int DATABASE_ACCESS_DENIED; + extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; + extern const int NOT_IMPLEMENTED; + extern const int CANNOT_COMPILE_REGEXP; + extern const int FILE_DOESNT_EXIST; +} + +class IOutputFormat; +using OutputFormatPtr = std::shared_ptr; + + + +StorageS3QueueSource::QueueGlobIterator::QueueGlobIterator( + const S3::Client & client_, + const S3::URI & globbed_uri_, + ASTPtr query, + const Block & virtual_header, + ContextPtr context, + StorageS3QueueSource::KeysWithInfo * read_keys_, + const S3Settings::RequestSettings & request_settings_) + : bucket(globbed_uri_.bucket), glob_iterator(std::make_unique(client_, globbed_uri_, query, virtual_header, context, read_keys_, request_settings_)) +{ + while (true) { + KeyWithInfo val = glob_iterator->next(); + if (val.key.empty()) { + break; + } + keys_buf.push_back(val); + } +} + +Strings StorageS3QueueSource::QueueGlobIterator::setProcessing(String & mode, std::unordered_set & exclude_keys) { + for (KeyWithInfo val: keys_buf) { + if (exclude_keys.find(bucket + '/' + val.key) != exclude_keys.end()) { + LOG_INFO(log, "Found in exclude keys {}", val.key); + continue; + } + if (processing_keys.size() < max_poll_size) { + processing_keys.push_back(val); + } else { + break; + } + } + + if (mode == "ordered") { + std::sort(processing_keys.begin( ), processing_keys.end( ), [ ]( const KeyWithInfo& lhs, const KeyWithInfo& rhs ) + { + return lhs.key < rhs.key; + }); + } + + Strings keys; + for (auto v: processing_keys) { + keys.push_back(bucket + '/' + v.key); + } + processing_keys.push_back(KeyWithInfo()); + + processing_iterator = processing_keys.begin(); + return keys; +} + + +StorageS3QueueSource::KeyWithInfo StorageS3QueueSource::QueueGlobIterator::next() { + std::lock_guard lock(mutex); + if (processing_iterator != processing_keys.end()) { + return *processing_iterator++; + } + + return KeyWithInfo(); +} + +size_t StorageS3QueueSource::QueueGlobIterator::getTotalSize() const +{ + return glob_iterator->getTotalSize(); +} + + +Block StorageS3QueueSource::getHeader(Block sample_block, const std::vector & requested_virtual_columns) +{ + for (const auto & virtual_column : requested_virtual_columns) + sample_block.insert({virtual_column.type->createColumn(), virtual_column.type, virtual_column.name}); + + return sample_block; +} + +StorageS3QueueSource::StorageS3QueueSource( + const std::vector & requested_virtual_columns_, + const String & format_, + String name_, + const Block & sample_block_, + ContextPtr context_, + std::optional format_settings_, + const ColumnsDescription & columns_, + UInt64 max_block_size_, + const S3Settings::RequestSettings & request_settings_, + String compression_hint_, + const std::shared_ptr & client_, + const String & bucket_, + const String & version_id_, + std::shared_ptr file_iterator_, + zkutil::ZooKeeperPtr current_zookeeper, + const String & zookeeper_path_, + const size_t download_thread_num_) + : ISource(getHeader(sample_block_, requested_virtual_columns_)) + , WithContext(context_) + , name(std::move(name_)) + , bucket(bucket_) + , version_id(version_id_) + , format(format_) + , columns_desc(columns_) + , max_block_size(max_block_size_) + , request_settings(request_settings_) + , compression_hint(std::move(compression_hint_)) + , client(client_) + , sample_block(sample_block_) + , format_settings(format_settings_) + , requested_virtual_columns(requested_virtual_columns_) + , file_iterator(file_iterator_) + , download_thread_num(download_thread_num_) + , zookeeper(current_zookeeper) + , zookeeper_path(zookeeper_path_) + , create_reader_pool(CurrentMetrics::StorageS3Threads, CurrentMetrics::StorageS3ThreadsActive, 1) + , create_reader_scheduler(threadPoolCallbackRunner(create_reader_pool, "CreateS3QueueReader")) +{ + reader = createReader(); + if (reader) + reader_future = createReaderAsync(); +} + +StorageS3QueueSource::ReaderHolder StorageS3QueueSource::createReader() +{ + auto [current_key, info] = (*file_iterator)(); + if (current_key.empty()) + return {}; + + size_t object_size = info ? info->size : S3::getObjectSize(*client, bucket, current_key, version_id, request_settings); + + int zstd_window_log_max = static_cast(getContext()->getSettingsRef().zstd_window_log_max); + auto read_buf = wrapReadBufferWithCompressionMethod( + createS3ReadBuffer(current_key, object_size), + chooseCompressionMethod(current_key, compression_hint), + zstd_window_log_max); + + auto input_format = getContext()->getInputFormat(format, *read_buf, sample_block, max_block_size, format_settings); + QueryPipelineBuilder builder; + builder.init(Pipe(input_format)); + + if (columns_desc.hasDefaults()) + { + builder.addSimpleTransform( + [&](const Block & header) + { return std::make_shared(header, columns_desc, *input_format, getContext()); }); + } + + auto pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); + auto current_reader = std::make_unique(*pipeline); + + return ReaderHolder{fs::path(bucket) / current_key, std::move(read_buf), std::move(pipeline), std::move(current_reader)}; +} + +std::future StorageS3QueueSource::createReaderAsync() +{ + return create_reader_scheduler([this] { return createReader(); }, 0); +} + +std::unique_ptr StorageS3QueueSource::createS3ReadBuffer(const String & key, size_t object_size) +{ + auto read_settings = getContext()->getReadSettings().adjustBufferSize(object_size); + read_settings.enable_filesystem_cache = false; + + auto download_buffer_size = getContext()->getSettings().max_download_buffer_size; + const bool use_parallel_download = download_buffer_size > 0 && download_thread_num > 1; + const bool object_too_small = object_size < download_thread_num * download_buffer_size; + + if (!use_parallel_download || object_too_small) + { + LOG_TRACE(log, "Downloading object of size {} from S3 in single thread", object_size); + if (read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) + return createAsyncS3ReadBuffer(key, read_settings, object_size); + + return std::make_unique(client, bucket, key, version_id, request_settings, read_settings); + } + + assert(object_size > 0); + if (download_buffer_size < DBMS_DEFAULT_BUFFER_SIZE) + { + LOG_WARNING(log, "Downloading buffer {} bytes too small, set at least {} bytes", download_buffer_size, DBMS_DEFAULT_BUFFER_SIZE); + download_buffer_size = DBMS_DEFAULT_BUFFER_SIZE; + } + + auto factory = std::make_unique( + client, bucket, key, version_id, download_buffer_size, object_size, request_settings, read_settings); + + LOG_TRACE(log, + "Downloading from S3 in {} threads. Object size: {}, Range size: {}.", + download_thread_num, object_size, download_buffer_size); + + return std::make_unique(std::move(factory), threadPoolCallbackRunner(IOThreadPool::get(), "S3ParallelRead"), download_thread_num); +} + +std::unique_ptr StorageS3QueueSource::createAsyncS3ReadBuffer( + const String & key, const ReadSettings & read_settings, size_t object_size) +{ + auto read_buffer_creator = + [this, read_settings] + (const std::string & path, size_t read_until_position) -> std::shared_ptr + { + return std::make_shared( + client, + bucket, + path, + version_id, + request_settings, + read_settings, + /* use_external_buffer */true, + /* offset */0, + read_until_position, + /* restricted_seek */true); + }; + + auto s3_impl = std::make_unique( + std::move(read_buffer_creator), + StoredObjects{StoredObject{key, object_size}}, + read_settings); + + auto & pool_reader = getContext()->getThreadPoolReader(Context::FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); + auto async_reader = std::make_unique(pool_reader, read_settings, std::move(s3_impl)); + + async_reader->setReadUntilEnd(); + if (read_settings.remote_fs_prefetch) + async_reader->prefetch(DEFAULT_PREFETCH_PRIORITY); + + return async_reader; +} + +StorageS3QueueSource::~StorageS3QueueSource() +{ + create_reader_pool.wait(); +} + +String StorageS3QueueSource::getName() const +{ + return name; +} + +Chunk StorageS3QueueSource::generate() +{ + while (true) + { + if (isCancelled() || !reader) + { + if (reader) + reader->cancel(); + break; + } + + Chunk chunk; + LOG_WARNING(log, "Try to pull new chunk"); + try { + if (reader->pull(chunk)) + { + LOG_WARNING(log, "Success in pulling!"); + UInt64 num_rows = chunk.getNumRows(); + + const auto & file_path = reader.getPath(); + size_t total_size = file_iterator->getTotalSize(); + if (num_rows && total_size) + { + updateRowsProgressApprox( + *this, chunk, total_size, total_rows_approx_accumulated, total_rows_count_times, total_rows_approx_max); + } + + for (const auto & virtual_column : requested_virtual_columns) + { + if (virtual_column.name == "_path") + { + chunk.addColumn(virtual_column.type->createColumnConst(num_rows, file_path)->convertToFullColumnIfConst()); + } + else if (virtual_column.name == "_file") + { + size_t last_slash_pos = file_path.find_last_of('/'); + auto column = virtual_column.type->createColumnConst(num_rows, file_path.substr(last_slash_pos + 1)); + chunk.addColumn(column->convertToFullColumnIfConst()); + } + } + LOG_WARNING(log, "Set processed: {}", file_path); + setFileProcessed(file_path); + // TODO: Set processed + return chunk; + } + } catch (const Exception & e) { + LOG_ERROR(log, "Exception: {} ", e.displayText()); + const auto & failed_file_path = reader.getPath(); + LOG_WARNING(log, "Set failed: {}", failed_file_path); + setFileFailed(failed_file_path); + } + + + + assert(reader_future.valid()); + reader = reader_future.get(); + + if (!reader) + break; + + /// Even if task is finished the thread may be not freed in pool. + /// So wait until it will be freed before scheduling a new task. + create_reader_pool.wait(); + reader_future = createReaderAsync(); + } + + return {}; +} + +void StorageS3QueueSource::setFileProcessed(const String & file_path) { + std::lock_guard lock(mutex); + String processed_files = zookeeper->get(zookeeper_path + "/processed"); + std::unordered_set processed = parseCollection(processed_files); + + processed.insert(file_path); + Strings set_processed; + set_processed.insert(set_processed.end(), processed.begin(), processed.end()); + + zookeeper->set(zookeeper_path + "/processed", toString(set_processed)); +} + + +void StorageS3QueueSource::setFileFailed(const String & file_path) { + std::lock_guard lock(mutex); + String processed_files = zookeeper->get(zookeeper_path + "/failed"); + std::unordered_set processed = parseCollection(processed_files); + + processed.insert(file_path); + Strings set_failed; + set_failed.insert(set_failed.end(), processed.begin(), processed.end()); + + zookeeper->set(zookeeper_path + "/failed", toString(set_failed)); +} + +std::unordered_set StorageS3QueueSource::parseCollection(String & files) { + ReadBuffer rb(const_cast(reinterpret_cast(files.data())), files.length(), 0); + Strings deserialized; + try { + readQuoted(deserialized, rb); + } catch (...) { + deserialized = {}; + } + + std::unordered_set processed(deserialized.begin(), deserialized.end()); + + return processed; +} + + +} + +#endif diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index 4714e23f748..c9bc9660a34 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -1,55 +1,146 @@ -//#pragma once -// -//#include -//#include -//#include -// -//namespace Poco -//{ -//class Logger; -//} -//namespace DB -//{ -//class S3QueueSource : public ISource -//{ -//public: -// S3QueueSource( -// StorageS3Queue & storage_, -// const StorageSnapshotPtr & storage_snapshot_, -// const ContextPtr & context_, -// const Names & columns, -// size_t max_block_size_, -// size_t poll_time_out_, -// size_t stream_number_, -// size_t max_streams_number_); -// -// String getName() const override { return "S3Queue"; } -// -// bool noRecords() { return !buffer || buffer->noRecords(); } -// -// void onFinish(); -// -// virtual ~S3QueueSource() override; -// -//protected: -// Chunk generate() override; -// -//private: -// StorageS3Queue & storage; -// StorageSnapshotPtr storage_snapshot; -// ContextPtr context; -// Names column_names; -// UInt64 max_block_size; -// -// size_t poll_time_out; -// -// size_t stream_number; -// size_t max_streams_number; -// -// std::unique_ptr buffer; -// -// Block non_virtual_header; -// Block virtual_header; -//}; -// -//} +#pragma once + +#include "config.h" + +#if USE_AWS_S3 + +#include + +#include + +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + + +class StorageS3QueueSource : public ISource, WithContext +{ +public: + using IIterator = StorageS3Source::IIterator; + using DisclosedGlobIterator = StorageS3Source::DisclosedGlobIterator; + using KeysWithInfo = StorageS3Source::KeysWithInfo; + using KeyWithInfo = StorageS3Source::KeyWithInfo; + + class QueueGlobIterator : public IIterator + { + public: + QueueGlobIterator( + const S3::Client & client_, + const S3::URI & globbed_uri_, + ASTPtr query, + const Block & virtual_header, + ContextPtr context, + KeysWithInfo * read_keys_ = nullptr, + const S3Settings::RequestSettings & request_settings_ = {}); + + KeyWithInfo next() override; + size_t getTotalSize() const override; + + Strings setProcessing(String & mode, std::unordered_set & exclude_keys); + private: + size_t max_poll_size = 10; + const String bucket; + KeysWithInfo keys_buf; + KeysWithInfo processing_keys; + mutable std::mutex mutex; + std::unique_ptr glob_iterator; + std::vector::iterator processing_iterator; + + Poco::Logger * log = &Poco::Logger::get("StorageS3QueueSourceIterator"); + }; + + static Block getHeader(Block sample_block, const std::vector & requested_virtual_columns); + + StorageS3QueueSource( + const std::vector & requested_virtual_columns_, + const String & format, + String name_, + const Block & sample_block, + ContextPtr context_, + std::optional format_settings_, + const ColumnsDescription & columns_, + UInt64 max_block_size_, + const S3Settings::RequestSettings & request_settings_, + String compression_hint_, + const std::shared_ptr & client_, + const String & bucket, + const String & version_id, + std::shared_ptr file_iterator_, + zkutil::ZooKeeperPtr current_zookeeper, + const String & zookeeper_path_, + size_t download_thread_num); + + ~StorageS3QueueSource() override; + + String getName() const override; + + Chunk generate() override; + + static std::unordered_set parseCollection(String & files); + + +private: + String name; + String bucket; + String version_id; + String format; + ColumnsDescription columns_desc; + UInt64 max_block_size; + S3Settings::RequestSettings request_settings; + String compression_hint; + std::shared_ptr client; + Block sample_block; + std::optional format_settings; + + using ReaderHolder = StorageS3Source::ReaderHolder; + ReaderHolder reader; + + std::vector requested_virtual_columns; + std::shared_ptr file_iterator; + size_t download_thread_num = 1; + + Poco::Logger * log = &Poco::Logger::get("StorageS3QueueSource"); + + zkutil::ZooKeeperPtr zookeeper; + const String zookeeper_path; + + ThreadPool create_reader_pool; + ThreadPoolCallbackRunner create_reader_scheduler; + std::future reader_future; + + UInt64 total_rows_approx_max = 0; + size_t total_rows_count_times = 0; + UInt64 total_rows_approx_accumulated = 0; + + mutable std::mutex mutex; + + + ReaderHolder createReader(); + std::future createReaderAsync(); + + std::unique_ptr createS3ReadBuffer(const String & key, size_t object_size); + std::unique_ptr createAsyncS3ReadBuffer(const String & key, const ReadSettings & read_settings, size_t object_size); + + void setFileProcessed(const String & file_path); + void setFileFailed(const String & file_path); + +}; + +} +#endif diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index ae162b90906..972302300ec 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -3,6 +3,14 @@ #include "IO/ParallelReadBuffer.h" #include "IO/IOThreadPool.h" #include "Parsers/ASTCreateQuery.h" +#include +#include +#include +#include + +#include +#include + #if USE_AWS_S3 @@ -23,6 +31,7 @@ #include #include #include +#include #include #include #include @@ -68,11 +77,16 @@ namespace fs = std::filesystem; +//namespace CurrentMetrics +//{ +//extern const Metric S3QueueBackgroundReads; +//} namespace ProfileEvents { extern const Event S3DeleteObjects; extern const Event S3ListObjects; +extern const Event S3QueueBackgroundReads; } namespace DB @@ -151,6 +165,9 @@ StorageS3Queue::StorageS3Queue( , zookeeper_name(zkutil::extractZooKeeperName(zookeeper_path_)) , zookeeper_path(zkutil::extractZooKeeperPath(zookeeper_path_, /* check_starts_with_slash */ true, log)) { + if (!is_key_with_globs) { + throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "S3Queue engine can read only from key with globs"); + } FormatFactory::instance().checkFormatName(format_name); context_->getGlobalContext()->getRemoteHostFilter().checkURL(s3_configuration.url.uri); @@ -212,6 +229,7 @@ Pipe StorageS3Queue::read( size_t max_block_size, size_t num_streams) { + // if (!local_context->getSettingsRef().stream_like_engine_allow_direct_select) // throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, // "Direct select is not allowed. To enable use setting `stream_like_engine_allow_direct_select`"); @@ -239,14 +257,9 @@ Pipe StorageS3Queue::read( requested_virtual_columns.push_back(virtual_column); } - std::shared_ptr iterator_wrapper = StorageS3::createFileIterator( - query_s3_configuration, - keys, - is_key_with_globs, - distributed_processing, + std::shared_ptr iterator_wrapper = createFileIterator( local_context, - query_info.query, - virtual_block); + query_info.query); ColumnsDescription columns_description; Block block_for_format; @@ -273,11 +286,12 @@ Pipe StorageS3Queue::read( const size_t max_download_threads = local_context->getSettingsRef().max_download_threads; // const size_t max_download_threads = 1; - LOG_WARNING(log, "num_streams"); + + auto zookeeper = getZooKeeper(); for (size_t i = 0; i < num_streams; ++i) { - pipes.emplace_back(std::make_shared( + pipes.emplace_back(std::make_shared( requested_virtual_columns, format_name, getName(), @@ -292,14 +306,13 @@ Pipe StorageS3Queue::read( query_s3_configuration.url.bucket, query_s3_configuration.url.version_id, iterator_wrapper, + zookeeper, + zookeeper_path, max_download_threads)); } auto pipe = Pipe::unitePipes(std::move(pipes)); - LOG_WARNING(log, "unitePipes"); - narrowPipe(pipe, num_streams); - LOG_WARNING(log, "narrowPipe"); return pipe; } @@ -386,10 +399,10 @@ void StorageS3Queue::threadFunc() auto table_id = getStorageID(); auto dependencies_count = getTableDependentCount(); - + LOG_TRACE(log, "dependencies_count {}", toString(dependencies_count)); if (dependencies_count) { - // auto start_time = std::chrono::steady_clock::now(); + auto start_time = std::chrono::steady_clock::now(); mv_attached.store(true); // Keep streaming as long as there are attached views and streaming is not cancelled @@ -404,20 +417,20 @@ void StorageS3Queue::threadFunc() LOG_DEBUG(log, "Started streaming to {} attached views", dependencies_count); - // if (streamToViews()) - // { - // LOG_TRACE(log, "Stream stalled. Reschedule."); - // break; - // } + if (streamToViews()) + { + LOG_TRACE(log, "Stream stalled. Reschedule."); + break; + } - // auto ts = std::chrono::steady_clock::now(); - // auto duration = std::chrono::duration_cast(ts-start_time); - // if (duration.count() > MAX_THREAD_WORK_DURATION_MS) - // { - // LOG_TRACE(log, "Thread work duration limit exceeded. Reschedule."); - // reschedule = true; - // break; - // } + auto ts = std::chrono::steady_clock::now(); + auto duration = std::chrono::duration_cast(ts-start_time); + if (duration.count() > 600000) + { + LOG_TRACE(log, "Thread work duration limit exceeded. Reschedule."); + reschedule = true; + break; + } } } } @@ -438,6 +451,125 @@ void StorageS3Queue::threadFunc() } +bool StorageS3Queue::streamToViews() +{ + LOG_TRACE(log, "streamToViews"); + + Stopwatch watch; + + auto table_id = getStorageID(); + auto table = DatabaseCatalog::instance().getTable(table_id, getContext()); + if (!table) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Engine table {} doesn't exist.", table_id.getNameForLogs()); + + // CurrentMetrics::Increment metric_increment{CurrentMetrics::KafkaBackgroundReads}; + // ProfileEvents::increment(ProfileEvents::S3QueueBackgroundReads); + + auto storage_snapshot = getStorageSnapshot(getInMemoryMetadataPtr(), getContext()); + + // Create an INSERT query for streaming data + auto insert = std::make_shared(); + insert->table_id = table_id; + + size_t block_size = 100; + + auto s3queue_context = Context::createCopy(getContext()); + s3queue_context->makeQueryContext(); + auto query_s3_configuration = StorageS3::copyAndUpdateConfiguration(s3queue_context, s3_configuration); + + // s3queue_context->applySettingsChanges(settings_adjustments); + + // Create a stream for each consumer and join them in a union stream + // Only insert into dependent views and expect that input blocks contain virtual columns + InterpreterInsertQuery interpreter(insert, s3queue_context, false, true, true); + auto block_io = interpreter.execute(); + auto column_names = block_io.pipeline.getHeader().getNames(); + + // Create a stream for each consumer and join them in a union stream + std::vector requested_virtual_columns; + + for (const auto & virtual_column : getVirtuals()) + { + requested_virtual_columns.push_back(virtual_column); + } + + std::shared_ptr iterator_wrapper = createFileIterator( + s3queue_context, + nullptr); + ColumnsDescription columns_description; + Block block_for_format; + if (supportsSubsetOfColumns()) + { + auto fetch_columns = column_names; + const auto & virtuals = getVirtuals(); + std::erase_if( + fetch_columns, + [&](const String & col) + { return std::any_of(virtuals.begin(), virtuals.end(), [&](const NameAndTypePair & virtual_col){ return col == virtual_col.name; }); }); + + if (fetch_columns.empty()) + fetch_columns.push_back(ExpressionActions::getSmallestColumn(storage_snapshot->metadata->getColumns().getAllPhysical()).name); + + columns_description = storage_snapshot->getDescriptionForColumns(fetch_columns); + block_for_format = storage_snapshot->getSampleBlockForColumns(columns_description.getNamesOfPhysical()); + } + else + { + columns_description = storage_snapshot->metadata->getColumns(); + block_for_format = storage_snapshot->metadata->getSampleBlock(); + } + + const size_t max_download_threads = s3queue_context->getSettingsRef().max_download_threads; + + Pipes pipes; + + auto zookeeper = getZooKeeper(); + size_t num_streams = 1; + for (size_t i = 0; i < num_streams; ++i) + { + pipes.emplace_back(std::make_shared( + requested_virtual_columns, + format_name, + getName(), + block_for_format, + s3queue_context, + format_settings, + columns_description, + block_size, + query_s3_configuration.request_settings, + compression_method, + query_s3_configuration.client, + query_s3_configuration.url.bucket, + query_s3_configuration.url.version_id, + iterator_wrapper, + zookeeper, + zookeeper_path, + max_download_threads)); + } + + auto pipe = Pipe::unitePipes(std::move(pipes)); + + // We can't cancel during copyData, as it's not aware of commits and other kafka-related stuff. + // It will be cancelled on underlying layer (kafka buffer) + + std::atomic_size_t rows = 0; + { + block_io.pipeline.complete(std::move(pipe)); + + // we need to read all consumers in parallel (sequential read may lead to situation + // when some of consumers are not used, and will break some Kafka consumer invariants) + block_io.pipeline.setNumThreads(num_streams); + + block_io.pipeline.setProgressCallback([&](const Progress & progress) { rows += progress.read_rows.load(); }); + CompletedPipelineExecutor executor(block_io.pipeline); + executor.execute(); + } + + bool some_stream_is_stalled = false; + return some_stream_is_stalled; +} + + void StorageS3Queue::setZooKeeper() { std::lock_guard lock(current_zookeeper_mutex); @@ -474,28 +606,31 @@ bool StorageS3Queue::createTableIfNotExists(const StorageMetadataPtr & metadata_ for (size_t i = 0; i < 1000; ++i) { + Coordination::Requests ops; + auto table_uuid = getStorageID().uuid; + if (zookeeper->exists(zookeeper_path + "")) { + ops.emplace_back(zkutil::makeCreateRequest(fs::path(zookeeper_path) / "processing" / toString(table_uuid), "{}", + zkutil::CreateMode::Ephemeral)); LOG_DEBUG(log, "This table {} is already created, will add new replica", zookeeper_path); - return false; } - /// We write metadata of table so that the replicas can check table parameters with them. - // String metadata_str = ReplicatedMergeTreeTableMetadata(*this, metadata_snapshot).toString(); - - Coordination::Requests ops; - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path, "", zkutil::CreateMode::Persistent)); - - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processed", "", - zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/failed", "", - zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processing", "", - zkutil::CreateMode::Ephemeral)); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/columns", metadata_snapshot->getColumns().toString(), - zkutil::CreateMode::Persistent)); -// ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/metadata", metadata_str, -// zkutil::CreateMode::Persistent)); + else + { + /// We write metadata of table so that the replicas can check table parameters with them. + // String metadata_str = ReplicatedMergeTreeTableMetadata(*this, metadata_snapshot).toString(); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path, "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processed", "[]", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/failed", "[]", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processing", "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest( + fs::path(zookeeper_path) / "processing" / toString(table_uuid), "[]", zkutil::CreateMode::Ephemeral)); + ops.emplace_back(zkutil::makeCreateRequest( + zookeeper_path + "/columns", metadata_snapshot->getColumns().toString(), zkutil::CreateMode::Persistent)); + // ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/metadata", metadata_str, + // zkutil::CreateMode::Persistent)); + } Coordination::Responses responses; auto code = zookeeper->tryMulti(ops, responses); if (code == Coordination::Error::ZNODEEXISTS) @@ -517,6 +652,60 @@ bool StorageS3Queue::createTableIfNotExists(const StorageMetadataPtr & metadata_ "of wrong zookeeper_path or because of logical error"); } + +std::shared_ptr StorageS3Queue::createFileIterator( + ContextPtr local_context, + ASTPtr query, + KeysWithInfo * read_keys) +{ + /// Iterate through disclosed globs and make a source for each file + auto it = std::make_shared( + *s3_configuration.client, s3_configuration.url, query, virtual_block, + local_context, read_keys, s3_configuration.request_settings); + mode = "unordered"; + + std::lock_guard lock{sync_mutex}; + std::unordered_set exclude = getExcludedFiles(); + + auto zookeeper = getZooKeeper(); + auto table_uuid = getStorageID().uuid; + Strings processing = it->setProcessing(mode, exclude); + zookeeper->set(fs::path(zookeeper_path) / "processing" / toString(table_uuid), toString(processing)); + + return it; +} + +std::unordered_set StorageS3Queue::getExcludedFiles() { + auto zookeeper = getZooKeeper(); + std::unordered_set exclude_files; + + String failed = zookeeper->get(zookeeper_path + "/failed"); + std::unordered_set failed_files = StorageS3QueueSource::parseCollection(failed); + + LOG_DEBUG(log, "failed_files {}", failed_files.size()); + String processed = zookeeper->get(zookeeper_path + "/processed"); + std::unordered_set processed_files = StorageS3QueueSource::parseCollection(processed); + LOG_DEBUG(log, "processed_files {}", processed_files.size()); + + exclude_files.merge(failed_files); + exclude_files.merge(processed_files); + + Strings consumer_table_uuids; + zookeeper->tryGetChildren(zookeeper_path + "/processing", consumer_table_uuids); + + for (const auto & uuid : consumer_table_uuids) + { + String processing = zookeeper->get(fs::path(zookeeper_path) / "processing" / toString(uuid)); + std::unordered_set processing_files = StorageS3QueueSource::parseCollection(processing); + LOG_DEBUG(log, "processing {}", processing_files.size()); + exclude_files.merge(processing_files); + } + + return exclude_files; +} + + + void registerStorageS3QueueImpl(const String & name, StorageFactory & factory) { factory.registerStorage(name, [](const StorageFactory::Arguments & args) diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index 9c1390c24a8..24d99960f17 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -13,6 +13,7 @@ #include #include #include +#include #include #include @@ -126,7 +127,7 @@ private: bool supportsSubsetOfColumns() const override; static Names getVirtualColumnNames(); - const String mode; + String mode; static const String default_zookeeper_name; const String zookeeper_name; @@ -136,6 +137,7 @@ private: zkutil::ZooKeeperPtr current_zookeeper; mutable std::mutex current_zookeeper_mutex; + mutable std::mutex sync_mutex; void setZooKeeper(); zkutil::ZooKeeperPtr tryGetZooKeeper() const; @@ -145,6 +147,17 @@ private: const String & getZooKeeperName() const { return zookeeper_name; } const String & getZooKeeperPath() const { return zookeeper_path; } + using KeysWithInfo = StorageS3QueueSource::KeysWithInfo; + + std::shared_ptr createFileIterator( + ContextPtr local_context, + ASTPtr query, + KeysWithInfo * read_keys = nullptr); + + static std::unordered_set parseCollection(String & files); + std::unordered_set getExcludedFiles(); + + bool streamToViews(); }; } diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 131649ff5fd..80510e84b78 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -145,6 +145,8 @@ public: Chunk generate() override; private: + friend class StorageS3QueueSource; + String name; String bucket; String version_id; From 60d7e7f57445f208e0b501e3b78faf4c5d4abe5e Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Sat, 29 Apr 2023 18:48:37 +0300 Subject: [PATCH 03/34] update settings --- src/Core/SettingsEnums.cpp | 4 + src/Core/SettingsEnums.h | 8 + src/Storages/S3Queue/S3QueueSettings.cpp | 4 +- src/Storages/S3Queue/S3QueueSettings.h | 14 +- src/Storages/S3Queue/S3QueueSource.cpp | 291 ++++++++++--------- src/Storages/S3Queue/S3QueueSource.h | 38 +-- src/Storages/S3Queue/StorageS3Queue.cpp | 341 ++++++++++++----------- src/Storages/S3Queue/StorageS3Queue.h | 83 +++--- 8 files changed, 420 insertions(+), 363 deletions(-) diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index e0f16ea00db..fdda4ec8bc6 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -202,4 +202,8 @@ IMPLEMENT_SETTING_ENUM(ORCCompression, ErrorCodes::BAD_ARGUMENTS, {"zlib", FormatSettings::ORCCompression::ZLIB}, {"lz4", FormatSettings::ORCCompression::LZ4}}) +IMPLEMENT_SETTING_ENUM(S3QueueMode, ErrorCodes::BAD_ARGUMENTS, + {{"ordered", S3QueueMode::ORDERED}, + {"unordered", S3QueueMode::UNORDERED}}) + } diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 3ae7bfaa673..b8912e5b9b4 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -218,4 +218,12 @@ enum class ParallelReplicasCustomKeyFilterType : uint8_t DECLARE_SETTING_ENUM(ParallelReplicasCustomKeyFilterType) DECLARE_SETTING_ENUM(LocalFSReadMethod) + +enum class S3QueueMode +{ + ORDERED, + UNORDERED, +}; + +DECLARE_SETTING_ENUM(S3QueueMode) } diff --git a/src/Storages/S3Queue/S3QueueSettings.cpp b/src/Storages/S3Queue/S3QueueSettings.cpp index 72bf5fb6742..b74cf8d39bb 100644 --- a/src/Storages/S3Queue/S3QueueSettings.cpp +++ b/src/Storages/S3Queue/S3QueueSettings.cpp @@ -1,7 +1,7 @@ -#include #include -#include #include +#include +#include #include diff --git a/src/Storages/S3Queue/S3QueueSettings.h b/src/Storages/S3Queue/S3QueueSettings.h index 8494ae47a90..3860a236a9b 100644 --- a/src/Storages/S3Queue/S3QueueSettings.h +++ b/src/Storages/S3Queue/S3QueueSettings.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB @@ -10,8 +11,17 @@ class ASTStorage; #define S3QUEUE_RELATED_SETTINGS(M, ALIAS) \ - M(String, mode, "unordered", "With unordered mode, the set of all already processed files is tracked with persistent nodes in ZooKepeer", 0) \ - M(String, keeper_path, "/", "Zookeeper node path", 0) \ + M(S3QueueMode, \ + mode, \ + S3QueueMode::ORDERED, \ + "With unordered mode, the set of all already processed files is tracked with persistent nodes in ZooKepeer", \ + 0) \ + M(String, keeper_path, "", "Zookeeper node path", 0) \ + M(String, processed_action, "keep", "Keep, delete or move file after successful loading", 0) \ + M(UInt64, s3queue_max_retries, 0, "Retry loading up to specified number of times", 0) \ + M(UInt64, s3queue_polling_min_timeout, 1000, "Minimal timeout before next polling", 0) \ + M(UInt64, s3queue_polling_max_timeout, 10000, "Maximum timeout before next polling", 0) \ + M(UInt64, s3queue_polling_backoff, 0, "Retry loading up to specified number of times", 0) #define LIST_OF_S3QUEUE_SETTINGS(M, ALIAS) \ S3QUEUE_RELATED_SETTINGS(M, ALIAS) \ diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 9272d4f4c02..92056c79d8f 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -1,72 +1,72 @@ -#include "config.h" -#include -#include "IO/ParallelReadBuffer.h" -#include "IO/IOThreadPool.h" -#include "Parsers/ASTCreateQuery.h" -#include #include +#include +#include +#include "IO/IOThreadPool.h" +#include "IO/ParallelReadBuffer.h" +#include "Parsers/ASTCreateQuery.h" +#include "config.h" #if USE_AWS_S3 -#include +# include -#include +# include -#include -#include +# include +# include -#include -#include +# include +# include -#include -#include +# include +# include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include -#include -#include -#include +# include +# include +# include -#include -#include +# include +# include -#include -#include +# include +# include -#include -#include -#include -#include +# include +# include +# include +# include -#include +# include -#include +# include -#include +# include -#include -#include -#include -#include -#include +# include +# include +# include +# include +# include -#include -#include -#include -#include +# include +# include +# include +# include -#include +# include namespace fs = std::filesystem; @@ -91,24 +91,23 @@ static const String PARTITION_ID_WILDCARD = "{_partition_id}"; static const std::unordered_set required_configuration_keys = { "url", }; -static const std::unordered_set optional_configuration_keys = { - "format", - "compression", - "compression_method", - "structure", - "access_key_id", - "secret_access_key", - "filename", - "use_environment_credentials", - "max_single_read_retries", - "min_upload_part_size", - "upload_part_size_multiply_factor", - "upload_part_size_multiply_parts_count_threshold", - "max_single_part_upload_size", - "max_connections", - "expiration_window_seconds", - "no_sign_request" -}; +static const std::unordered_set optional_configuration_keys + = {"format", + "compression", + "compression_method", + "structure", + "access_key_id", + "secret_access_key", + "filename", + "use_environment_credentials", + "max_single_read_retries", + "min_upload_part_size", + "upload_part_size_multiply_factor", + "upload_part_size_multiply_parts_count_threshold", + "max_single_part_upload_size", + "max_connections", + "expiration_window_seconds", + "no_sign_request"}; namespace ErrorCodes { @@ -128,7 +127,6 @@ class IOutputFormat; using OutputFormatPtr = std::shared_ptr; - StorageS3QueueSource::QueueGlobIterator::QueueGlobIterator( const S3::Client & client_, const S3::URI & globbed_uri_, @@ -137,39 +135,51 @@ StorageS3QueueSource::QueueGlobIterator::QueueGlobIterator( ContextPtr context, StorageS3QueueSource::KeysWithInfo * read_keys_, const S3Settings::RequestSettings & request_settings_) - : bucket(globbed_uri_.bucket), glob_iterator(std::make_unique(client_, globbed_uri_, query, virtual_header, context, read_keys_, request_settings_)) + : bucket(globbed_uri_.bucket) + , glob_iterator(std::make_unique( + client_, globbed_uri_, query, virtual_header, context, read_keys_, request_settings_)) { - while (true) { + while (true) + { KeyWithInfo val = glob_iterator->next(); - if (val.key.empty()) { + if (val.key.empty()) + { break; } keys_buf.push_back(val); } } -Strings StorageS3QueueSource::QueueGlobIterator::setProcessing(String & mode, std::unordered_set & exclude_keys) { - for (KeyWithInfo val: keys_buf) { - if (exclude_keys.find(bucket + '/' + val.key) != exclude_keys.end()) { +Strings StorageS3QueueSource::QueueGlobIterator::setProcessing(String & mode, std::unordered_set & exclude_keys) +{ + for (KeyWithInfo val : keys_buf) + { + if (exclude_keys.find(bucket + '/' + val.key) != exclude_keys.end()) + { LOG_INFO(log, "Found in exclude keys {}", val.key); continue; } - if (processing_keys.size() < max_poll_size) { + if (processing_keys.size() < max_poll_size) + { processing_keys.push_back(val); - } else { + } + else + { break; } } - if (mode == "ordered") { - std::sort(processing_keys.begin( ), processing_keys.end( ), [ ]( const KeyWithInfo& lhs, const KeyWithInfo& rhs ) - { - return lhs.key < rhs.key; - }); + if (mode == "ordered") + { + std::sort( + processing_keys.begin(), + processing_keys.end(), + [](const KeyWithInfo & lhs, const KeyWithInfo & rhs) { return lhs.key < rhs.key; }); } Strings keys; - for (auto v: processing_keys) { + for (auto v : processing_keys) + { keys.push_back(bucket + '/' + v.key); } processing_keys.push_back(KeyWithInfo()); @@ -179,9 +189,11 @@ Strings StorageS3QueueSource::QueueGlobIterator::setProcessing(String & mode, st } -StorageS3QueueSource::KeyWithInfo StorageS3QueueSource::QueueGlobIterator::next() { +StorageS3QueueSource::KeyWithInfo StorageS3QueueSource::QueueGlobIterator::next() +{ std::lock_guard lock(mutex); - if (processing_iterator != processing_keys.end()) { + if (processing_iterator != processing_keys.end()) + { return *processing_iterator++; } @@ -253,14 +265,32 @@ StorageS3QueueSource::ReaderHolder StorageS3QueueSource::createReader() return {}; size_t object_size = info ? info->size : S3::getObjectSize(*client, bucket, current_key, version_id, request_settings); + auto compression_method = chooseCompressionMethod(current_key, compression_hint); - int zstd_window_log_max = static_cast(getContext()->getSettingsRef().zstd_window_log_max); - auto read_buf = wrapReadBufferWithCompressionMethod( - createS3ReadBuffer(current_key, object_size), - chooseCompressionMethod(current_key, compression_hint), - zstd_window_log_max); + InputFormatPtr input_format; + std::unique_ptr owned_read_buf; + + auto read_buf_or_factory = createS3ReadBuffer(current_key, object_size); + if (read_buf_or_factory.buf_factory) + { + input_format = FormatFactory::instance().getInputRandomAccess( + format, + std::move(read_buf_or_factory.buf_factory), + sample_block, + getContext(), + max_block_size, + /* is_remote_fs */ true, + compression_method, + format_settings); + } + else + { + owned_read_buf = wrapReadBufferWithCompressionMethod( + std::move(read_buf_or_factory.buf), compression_method, static_cast(getContext()->getSettingsRef().zstd_window_log_max)); + input_format + = FormatFactory::instance().getInput(format, *owned_read_buf, sample_block, getContext(), max_block_size, format_settings); + } - auto input_format = getContext()->getInputFormat(format, *read_buf, sample_block, max_block_size, format_settings); QueryPipelineBuilder builder; builder.init(Pipe(input_format)); @@ -274,7 +304,7 @@ StorageS3QueueSource::ReaderHolder StorageS3QueueSource::createReader() auto pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); auto current_reader = std::make_unique(*pipeline); - return ReaderHolder{fs::path(bucket) / current_key, std::move(read_buf), std::move(pipeline), std::move(current_reader)}; + return ReaderHolder{fs::path(bucket) / current_key, std::move(owned_read_buf), std::move(pipeline), std::move(current_reader)}; } std::future StorageS3QueueSource::createReaderAsync() @@ -282,47 +312,31 @@ std::future StorageS3QueueSource::createRead return create_reader_scheduler([this] { return createReader(); }, 0); } -std::unique_ptr StorageS3QueueSource::createS3ReadBuffer(const String & key, size_t object_size) +StorageS3QueueSource::ReadBufferOrFactory StorageS3QueueSource::createS3ReadBuffer(const String & key, size_t object_size) { auto read_settings = getContext()->getReadSettings().adjustBufferSize(object_size); read_settings.enable_filesystem_cache = false; - auto download_buffer_size = getContext()->getSettings().max_download_buffer_size; - const bool use_parallel_download = download_buffer_size > 0 && download_thread_num > 1; - const bool object_too_small = object_size < download_thread_num * download_buffer_size; + const bool object_too_small = object_size <= 2 * download_buffer_size; - if (!use_parallel_download || object_too_small) + // Create a read buffer that will prefetch the first ~1 MB of the file. + // When reading lots of tiny files, this prefetching almost doubles the throughput. + // For bigger files, parallel reading is more useful. + if (object_too_small && read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) { - LOG_TRACE(log, "Downloading object of size {} from S3 in single thread", object_size); - if (read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) - return createAsyncS3ReadBuffer(key, read_settings, object_size); - - return std::make_unique(client, bucket, key, version_id, request_settings, read_settings); + LOG_TRACE(log, "Downloading object of size {} from S3 with initial prefetch", object_size); + return {.buf = createAsyncS3ReadBuffer(key, read_settings, object_size)}; } - assert(object_size > 0); - if (download_buffer_size < DBMS_DEFAULT_BUFFER_SIZE) - { - LOG_WARNING(log, "Downloading buffer {} bytes too small, set at least {} bytes", download_buffer_size, DBMS_DEFAULT_BUFFER_SIZE); - download_buffer_size = DBMS_DEFAULT_BUFFER_SIZE; - } - - auto factory = std::make_unique( - client, bucket, key, version_id, download_buffer_size, object_size, request_settings, read_settings); - - LOG_TRACE(log, - "Downloading from S3 in {} threads. Object size: {}, Range size: {}.", - download_thread_num, object_size, download_buffer_size); - - return std::make_unique(std::move(factory), threadPoolCallbackRunner(IOThreadPool::get(), "S3ParallelRead"), download_thread_num); + auto factory = std::make_unique(client, bucket, key, version_id, object_size, request_settings, read_settings); + return {.buf_factory = std::move(factory)}; } -std::unique_ptr StorageS3QueueSource::createAsyncS3ReadBuffer( - const String & key, const ReadSettings & read_settings, size_t object_size) +std::unique_ptr +StorageS3QueueSource::createAsyncS3ReadBuffer(const String & key, const ReadSettings & read_settings, size_t object_size) { auto read_buffer_creator = - [this, read_settings] - (const std::string & path, size_t read_until_position) -> std::shared_ptr + [this, read_settings, object_size](const std::string & path, size_t read_until_position) -> std::shared_ptr { return std::make_shared( client, @@ -331,16 +345,15 @@ std::unique_ptr StorageS3QueueSource::createAsyncS3ReadBuffer( version_id, request_settings, read_settings, - /* use_external_buffer */true, - /* offset */0, + /* use_external_buffer */ true, + /* offset */ 0, read_until_position, - /* restricted_seek */true); + /* restricted_seek */ true, + object_size); }; auto s3_impl = std::make_unique( - std::move(read_buffer_creator), - StoredObjects{StoredObject{key, object_size}}, - read_settings); + std::move(read_buffer_creator), StoredObjects{StoredObject{key, object_size}}, read_settings); auto & pool_reader = getContext()->getThreadPoolReader(Context::FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); auto async_reader = std::make_unique(pool_reader, read_settings, std::move(s3_impl)); @@ -375,7 +388,8 @@ Chunk StorageS3QueueSource::generate() Chunk chunk; LOG_WARNING(log, "Try to pull new chunk"); - try { + try + { if (reader->pull(chunk)) { LOG_WARNING(log, "Success in pulling!"); @@ -407,7 +421,9 @@ Chunk StorageS3QueueSource::generate() // TODO: Set processed return chunk; } - } catch (const Exception & e) { + } + catch (const Exception & e) + { LOG_ERROR(log, "Exception: {} ", e.displayText()); const auto & failed_file_path = reader.getPath(); LOG_WARNING(log, "Set failed: {}", failed_file_path); @@ -415,7 +431,6 @@ Chunk StorageS3QueueSource::generate() } - assert(reader_future.valid()); reader = reader_future.get(); @@ -431,7 +446,8 @@ Chunk StorageS3QueueSource::generate() return {}; } -void StorageS3QueueSource::setFileProcessed(const String & file_path) { +void StorageS3QueueSource::setFileProcessed(const String & file_path) +{ std::lock_guard lock(mutex); String processed_files = zookeeper->get(zookeeper_path + "/processed"); std::unordered_set processed = parseCollection(processed_files); @@ -444,7 +460,8 @@ void StorageS3QueueSource::setFileProcessed(const String & file_path) { } -void StorageS3QueueSource::setFileFailed(const String & file_path) { +void StorageS3QueueSource::setFileFailed(const String & file_path) +{ std::lock_guard lock(mutex); String processed_files = zookeeper->get(zookeeper_path + "/failed"); std::unordered_set processed = parseCollection(processed_files); @@ -456,12 +473,16 @@ void StorageS3QueueSource::setFileFailed(const String & file_path) { zookeeper->set(zookeeper_path + "/failed", toString(set_failed)); } -std::unordered_set StorageS3QueueSource::parseCollection(String & files) { +std::unordered_set StorageS3QueueSource::parseCollection(String & files) +{ ReadBuffer rb(const_cast(reinterpret_cast(files.data())), files.length(), 0); Strings deserialized; - try { + try + { readQuoted(deserialized, rb); - } catch (...) { + } + catch (...) + { deserialized = {}; } diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index c9bc9660a34..3abb388cd2d 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -4,25 +4,25 @@ #if USE_AWS_S3 -#include +# include -#include +# include -#include -#include -#include +# include +# include +# include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include namespace DB @@ -36,7 +36,7 @@ public: using DisclosedGlobIterator = StorageS3Source::DisclosedGlobIterator; using KeysWithInfo = StorageS3Source::KeysWithInfo; using KeyWithInfo = StorageS3Source::KeyWithInfo; - + using ReadBufferOrFactory = StorageS3Source::ReadBufferOrFactory; class QueueGlobIterator : public IIterator { public: @@ -53,6 +53,7 @@ public: size_t getTotalSize() const override; Strings setProcessing(String & mode, std::unordered_set & exclude_keys); + private: size_t max_poll_size = 10; const String bucket; @@ -134,12 +135,11 @@ private: ReaderHolder createReader(); std::future createReaderAsync(); - std::unique_ptr createS3ReadBuffer(const String & key, size_t object_size); + ReadBufferOrFactory createS3ReadBuffer(const String & key, size_t object_size); std::unique_ptr createAsyncS3ReadBuffer(const String & key, const ReadSettings & read_settings, size_t object_size); void setFileProcessed(const String & file_path); void setFileFailed(const String & file_path); - }; } diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 972302300ec..1cb04350635 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -1,79 +1,79 @@ -#include "config.h" -#include -#include "IO/ParallelReadBuffer.h" -#include "IO/IOThreadPool.h" -#include "Parsers/ASTCreateQuery.h" -#include +#include +#include #include +#include +#include #include #include - -#include -#include +#include +#include "IO/IOThreadPool.h" +#include "IO/ParallelReadBuffer.h" +#include "Parsers/ASTCreateQuery.h" +#include "config.h" #if USE_AWS_S3 -#include -#include +# include +# include -#include +# include -#include -#include +# include +# include -#include -#include +# include +# include -#include -#include +# include +# include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include -#include -#include -#include +# include +# include +# include -#include -#include +# include +# include -#include -#include +# include +# include -#include -#include -#include -#include +# include +# include +# include +# include -#include +# include -#include +# include -#include +# include -#include -#include -#include +# include +# include +# include -#include -#include -#include -#include +# include +# include +# include +# include namespace fs = std::filesystem; @@ -134,12 +134,8 @@ namespace ErrorCodes class IOutputFormat; using OutputFormatPtr = std::shared_ptr; -const String StorageS3Queue::default_zookeeper_name = "default"; - - StorageS3Queue::StorageS3Queue( - const String & zookeeper_path_, - const String & mode_, + std::unique_ptr s3queue_settings_, const StorageS3::Configuration & configuration_, const StorageID & table_id_, const ColumnsDescription & columns_, @@ -151,6 +147,7 @@ StorageS3Queue::StorageS3Queue( ASTPtr partition_by_) : IStorage(table_id_) , WithContext(context_) + , s3queue_settings(std::move(s3queue_settings_)) , s3_configuration{configuration_} , keys({s3_configuration.url.key}) , format_name(configuration_.format) @@ -161,29 +158,43 @@ StorageS3Queue::StorageS3Queue( , partition_by(partition_by_) , is_key_with_globs(s3_configuration.url.key.find_first_of("*?{") != std::string::npos) , log(&Poco::Logger::get("StorageS3Queue (" + table_id_.table_name + ")")) - , mode(mode_) - , zookeeper_name(zkutil::extractZooKeeperName(zookeeper_path_)) - , zookeeper_path(zkutil::extractZooKeeperPath(zookeeper_path_, /* check_starts_with_slash */ true, log)) { - if (!is_key_with_globs) { + String setting_zookeeper_path = s3queue_settings->keeper_path; + if (setting_zookeeper_path == "") + { + auto table_id = getStorageID(); + auto database = DatabaseCatalog::instance().getDatabase(table_id.database_name); + bool is_in_replicated_database = database->getEngineName() == "Replicated"; + if (is_in_replicated_database) + { + LOG_INFO(log, "S3Queue engine keeper_path not specified. Use replicated database zookeeper path"); + String base_zookeeper_path = assert_cast(database.get())->getZooKeeperPath(); + zookeeper_path = zkutil::extractZooKeeperPath( + fs::path(base_zookeeper_path) / toString(table_id.uuid), /* check_starts_with_slash */ true, log); + } + else + { + throw Exception(ErrorCodes::NO_ZOOKEEPER, "S3Queue zookeeper path not specified and table not in replicated database."); + } + } + else + { + zookeeper_path = zkutil::extractZooKeeperPath(s3queue_settings->keeper_path, /* check_starts_with_slash */ true, log); + } + LOG_INFO(log, "Storage S3Queue zookeeper_path= {} with mode", zookeeper_path); + + if (!is_key_with_globs) + { throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "S3Queue engine can read only from key with globs"); } FormatFactory::instance().checkFormatName(format_name); context_->getGlobalContext()->getRemoteHostFilter().checkURL(s3_configuration.url.uri); - StorageInMemoryMetadata storage_metadata; + s3_configuration.update(context_); - StorageS3::updateConfiguration(context_, s3_configuration); if (columns_.empty()) { - auto columns = StorageS3::getTableStructureFromDataImpl( - format_name, - s3_configuration, - compression_method, - is_key_with_globs, - format_settings, - context_); - + auto columns = StorageS3::getTableStructureFromDataImpl(s3_configuration, format_settings, context_); storage_metadata.setColumns(columns); } else @@ -207,6 +218,7 @@ StorageS3Queue::StorageS3Queue( auto metadata_snapshot = getInMemoryMetadataPtr(); createTableIfNotExists(metadata_snapshot); task = std::make_shared(std::move(thread)); + LOG_TRACE(log, "Complete"); } @@ -229,18 +241,16 @@ Pipe StorageS3Queue::read( size_t max_block_size, size_t num_streams) { - -// if (!local_context->getSettingsRef().stream_like_engine_allow_direct_select) -// throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, -// "Direct select is not allowed. To enable use setting `stream_like_engine_allow_direct_select`"); + // if (!local_context->getSettingsRef().stream_like_engine_allow_direct_select) + // throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, + // "Direct select is not allowed. To enable use setting `stream_like_engine_allow_direct_select`"); if (mv_attached) throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "Cannot read from StorageRabbitMQ with attached materialized views"); - auto query_s3_configuration = StorageS3::copyAndUpdateConfiguration(local_context, s3_configuration); + auto query_s3_configuration = updateConfigurationAndGetCopy(local_context); - bool has_wildcards = - query_s3_configuration.url.bucket.find(PARTITION_ID_WILDCARD) != String::npos + bool has_wildcards = query_s3_configuration.url.bucket.find(PARTITION_ID_WILDCARD) != String::npos || keys.back().find(PARTITION_ID_WILDCARD) != String::npos; if (partition_by && has_wildcards) @@ -257,9 +267,7 @@ Pipe StorageS3Queue::read( requested_virtual_columns.push_back(virtual_column); } - std::shared_ptr iterator_wrapper = createFileIterator( - local_context, - query_info.query); + std::shared_ptr iterator_wrapper = createFileIterator(local_context, query_info.query); ColumnsDescription columns_description; Block block_for_format; @@ -269,8 +277,10 @@ Pipe StorageS3Queue::read( const auto & virtuals = getVirtuals(); std::erase_if( fetch_columns, - [&](const String & col) - { return std::any_of(virtuals.begin(), virtuals.end(), [&](const NameAndTypePair & virtual_col){ return col == virtual_col.name; }); }); + [&](const String & col) { + return std::any_of( + virtuals.begin(), virtuals.end(), [&](const NameAndTypePair & virtual_col) { return col == virtual_col.name; }); + }); if (fetch_columns.empty()) fetch_columns.push_back(ExpressionActions::getSmallestColumn(storage_snapshot->metadata->getColumns().getAllPhysical()).name); @@ -348,7 +358,7 @@ void StorageS3Queue::shutdown() task->stream_cancelled = true; /// Reader thread may wait for wake up -// wakeUp(); + // wakeUp(); LOG_TRACE(log, "Waiting for cleanup"); task->holder->deactivate(); @@ -424,7 +434,7 @@ void StorageS3Queue::threadFunc() } auto ts = std::chrono::steady_clock::now(); - auto duration = std::chrono::duration_cast(ts-start_time); + auto duration = std::chrono::duration_cast(ts - start_time); if (duration.count() > 600000) { LOG_TRACE(log, "Thread work duration limit exceeded. Reschedule."); @@ -475,7 +485,7 @@ bool StorageS3Queue::streamToViews() auto s3queue_context = Context::createCopy(getContext()); s3queue_context->makeQueryContext(); - auto query_s3_configuration = StorageS3::copyAndUpdateConfiguration(s3queue_context, s3_configuration); + auto query_s3_configuration = updateConfigurationAndGetCopy(s3queue_context); // s3queue_context->applySettingsChanges(settings_adjustments); @@ -493,9 +503,7 @@ bool StorageS3Queue::streamToViews() requested_virtual_columns.push_back(virtual_column); } - std::shared_ptr iterator_wrapper = createFileIterator( - s3queue_context, - nullptr); + std::shared_ptr iterator_wrapper = createFileIterator(s3queue_context, nullptr); ColumnsDescription columns_description; Block block_for_format; if (supportsSubsetOfColumns()) @@ -504,8 +512,10 @@ bool StorageS3Queue::streamToViews() const auto & virtuals = getVirtuals(); std::erase_if( fetch_columns, - [&](const String & col) - { return std::any_of(virtuals.begin(), virtuals.end(), [&](const NameAndTypePair & virtual_col){ return col == virtual_col.name; }); }); + [&](const String & col) { + return std::any_of( + virtuals.begin(), virtuals.end(), [&](const NameAndTypePair & virtual_col) { return col == virtual_col.name; }); + }); if (fetch_columns.empty()) fetch_columns.push_back(ExpressionActions::getSmallestColumn(storage_snapshot->metadata->getColumns().getAllPhysical()).name); @@ -569,19 +579,16 @@ bool StorageS3Queue::streamToViews() return some_stream_is_stalled; } +StorageS3Queue::Configuration StorageS3Queue::updateConfigurationAndGetCopy(ContextPtr local_context) +{ + s3_configuration.update(local_context); + return s3_configuration; +} void StorageS3Queue::setZooKeeper() { std::lock_guard lock(current_zookeeper_mutex); - LOG_WARNING(log, "zookeeper name {}", zookeeper_name); - if (zookeeper_name == default_zookeeper_name) - { - current_zookeeper = getContext()->getZooKeeper(); - } - else - { - current_zookeeper = getContext()->getAuxiliaryZooKeeper(zookeeper_name); - } + current_zookeeper = getContext()->getZooKeeper(); } zkutil::ZooKeeperPtr StorageS3Queue::tryGetZooKeeper() const @@ -611,8 +618,8 @@ bool StorageS3Queue::createTableIfNotExists(const StorageMetadataPtr & metadata_ if (zookeeper->exists(zookeeper_path + "")) { - ops.emplace_back(zkutil::makeCreateRequest(fs::path(zookeeper_path) / "processing" / toString(table_uuid), "{}", - zkutil::CreateMode::Ephemeral)); + ops.emplace_back(zkutil::makeCreateRequest( + fs::path(zookeeper_path) / "processing" / toString(table_uuid), "{}", zkutil::CreateMode::Ephemeral)); LOG_DEBUG(log, "This table {} is already created, will add new replica", zookeeper_path); } else @@ -647,35 +654,34 @@ bool StorageS3Queue::createTableIfNotExists(const StorageMetadataPtr & metadata_ } /// Do not use LOGICAL_ERROR code, because it may happen if user has specified wrong zookeeper_path - throw Exception(ErrorCodes::REPLICA_ALREADY_EXISTS, - "Cannot create table, because it is created concurrently every time or because " - "of wrong zookeeper_path or because of logical error"); + throw Exception( + ErrorCodes::REPLICA_ALREADY_EXISTS, + "Cannot create table, because it is created concurrently every time or because " + "of wrong zookeeper_path or because of logical error"); } -std::shared_ptr StorageS3Queue::createFileIterator( - ContextPtr local_context, - ASTPtr query, - KeysWithInfo * read_keys) +std::shared_ptr +StorageS3Queue::createFileIterator(ContextPtr local_context, ASTPtr query, KeysWithInfo * read_keys) { /// Iterate through disclosed globs and make a source for each file - auto it = std::make_shared( - *s3_configuration.client, s3_configuration.url, query, virtual_block, - local_context, read_keys, s3_configuration.request_settings); - mode = "unordered"; + auto it = std::make_shared( + *s3_configuration.client, s3_configuration.url, query, virtual_block, local_context, read_keys, s3_configuration.request_settings); + String cur_mode = "unordered"; std::lock_guard lock{sync_mutex}; std::unordered_set exclude = getExcludedFiles(); auto zookeeper = getZooKeeper(); auto table_uuid = getStorageID().uuid; - Strings processing = it->setProcessing(mode, exclude); + Strings processing = it->setProcessing(cur_mode, exclude); zookeeper->set(fs::path(zookeeper_path) / "processing" / toString(table_uuid), toString(processing)); return it; } -std::unordered_set StorageS3Queue::getExcludedFiles() { +std::unordered_set StorageS3Queue::getExcludedFiles() +{ auto zookeeper = getZooKeeper(); std::unordered_set exclude_files; @@ -705,55 +711,74 @@ std::unordered_set StorageS3Queue::getExcludedFiles() { } - void registerStorageS3QueueImpl(const String & name, StorageFactory & factory) { - factory.registerStorage(name, [](const StorageFactory::Arguments & args) - { - auto & engine_args = args.engine_args; - if (engine_args.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments"); + factory.registerStorage( + name, + [](const StorageFactory::Arguments & args) + { + auto & engine_args = args.engine_args; + if (engine_args.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments"); + auto configuration = StorageS3::getConfiguration(engine_args, args.getLocalContext()); - auto configuration = StorageS3::getConfiguration(engine_args, args.getLocalContext()); - // Use format settings from global server context + settings from - // the SETTINGS clause of the create query. Settings from current - // session and user are ignored. - auto s3queue_settings = std::make_unique(); - std::optional format_settings; + // Use format settings from global server context + settings from + // the SETTINGS clause of the create query. Settings from current + // session and user are ignored. + std::optional format_settings; - if (args.storage_def->settings) - { - s3queue_settings->loadFromQuery(*args.storage_def); + auto s3queue_settings = std::make_unique(); + if (args.storage_def->settings) + { + s3queue_settings->loadFromQuery(*args.storage_def); + FormatFactorySettings user_format_settings; - } - format_settings = getFormatSettings(args.getContext()); + // Apply changed settings from global context, but ignore the + // unknown ones, because we only have the format settings here. + const auto & changes = args.getContext()->getSettingsRef().changes(); + for (const auto & change : changes) + { + if (user_format_settings.has(change.name)) + user_format_settings.set(change.name, change.value); + else + LOG_TRACE(&Poco::Logger::get("StorageS3"), "Remove: {}", change.name); + args.storage_def->settings->changes.removeSetting(change.name); + } - ASTPtr partition_by; - if (args.storage_def->partition_by) - partition_by = args.storage_def->partition_by->clone(); + for (const auto & change : args.storage_def->settings->changes) + { + if (user_format_settings.has(change.name)) + user_format_settings.applyChange(change); + } + format_settings = getFormatSettings(args.getContext(), user_format_settings); + } + else + { + format_settings = getFormatSettings(args.getContext()); + } - String keeper_path = s3queue_settings->keeper_path; - String mode = s3queue_settings->mode; + ASTPtr partition_by; + if (args.storage_def->partition_by) + partition_by = args.storage_def->partition_by->clone(); - return std::make_shared( - keeper_path, - mode, - configuration, - args.table_id, - args.columns, - args.constraints, - args.comment, - args.getContext(), - format_settings, - /* distributed_processing_ */false, - partition_by); - }, - { - .supports_settings = true, - .supports_sort_order = true, // for partition by - .supports_schema_inference = true, - .source_access_type = AccessType::S3, - }); + return std::make_shared( + std::move(s3queue_settings), + std::move(configuration), + args.table_id, + args.columns, + args.constraints, + args.comment, + args.getContext(), + format_settings, + /* distributed_processing_ */ false, + partition_by); + }, + { + .supports_settings = true, + .supports_sort_order = true, // for partition by + .supports_schema_inference = true, + .source_access_type = AccessType::S3, + }); } void registerStorageS3Queue(StorageFactory & factory) diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index 24d99960f17..1763e882d59 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -4,28 +4,28 @@ #if USE_AWS_S3 -#include +# include -#include -#include +# include +# include -#include -#include -#include -#include -#include +# include +# include +# include +# include +# include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include namespace Aws::S3 { @@ -36,14 +36,12 @@ namespace DB { - class StorageS3Queue : public IStorage, WithContext { public: using Configuration = typename StorageS3::Configuration; StorageS3Queue( - const String & zookeper_path_, - const String & mode_, + std::unique_ptr s3queue_settings_, const Configuration & configuration_, const StorageID & table_id_, const ColumnsDescription & columns_, @@ -65,11 +63,17 @@ public: size_t max_block_size, size_t num_streams) override; - SinkToStoragePtr write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr /*context*/) override { + SinkToStoragePtr write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr /*context*/) override + { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Write is not supported by storage {}", getName()); } - void truncate(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr /*local_context*/, TableExclusiveLockHolder &) override { + void truncate( + const ASTPtr & /*query*/, + const StorageMetadataPtr & /*metadata_snapshot*/, + ContextPtr /*local_context*/, + TableExclusiveLockHolder &) override + { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Truncate is not supported by storage {}", getName()); } @@ -77,16 +81,10 @@ public: bool supportsPartitionBy() const override; - static ColumnsDescription getTableStructureFromData( - Configuration & configuration, const std::optional & format_settings, ContextPtr ctx) - { - return StorageS3::getTableStructureFromData(configuration, format_settings, ctx); - } - const auto & getFormatName() const { return format_name; } private: - + std::unique_ptr s3queue_settings; Configuration s3_configuration; std::vector keys; NamesAndTypesList virtual_columns; @@ -117,23 +115,15 @@ private: struct TaskContext { BackgroundSchedulePool::TaskHolder holder; - std::atomic stream_cancelled {false}; - explicit TaskContext(BackgroundSchedulePool::TaskHolder&& task_) : holder(std::move(task_)) - { - } + std::atomic stream_cancelled{false}; + explicit TaskContext(BackgroundSchedulePool::TaskHolder && task_) : holder(std::move(task_)) { } }; std::shared_ptr task; bool supportsSubsetOfColumns() const override; static Names getVirtualColumnNames(); - String mode; - - static const String default_zookeeper_name; - const String zookeeper_name; - const String zookeeper_path; - const String replica_name; - const String replica_path; + String zookeeper_path; zkutil::ZooKeeperPtr current_zookeeper; mutable std::mutex current_zookeeper_mutex; @@ -144,20 +134,19 @@ private: zkutil::ZooKeeperPtr getZooKeeper() const; bool createTableIfNotExists(const StorageMetadataPtr & metadata_snapshot); // Return default or custom zookeeper name for table - const String & getZooKeeperName() const { return zookeeper_name; } const String & getZooKeeperPath() const { return zookeeper_path; } using KeysWithInfo = StorageS3QueueSource::KeysWithInfo; - std::shared_ptr createFileIterator( - ContextPtr local_context, - ASTPtr query, - KeysWithInfo * read_keys = nullptr); + std::shared_ptr + createFileIterator(ContextPtr local_context, ASTPtr query, KeysWithInfo * read_keys = nullptr); static std::unordered_set parseCollection(String & files); std::unordered_set getExcludedFiles(); bool streamToViews(); + + Configuration updateConfigurationAndGetCopy(ContextPtr local_context); }; } From dd5e95e2e36fa4c781e7a12cef2264dee67c5c97 Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Mon, 1 May 2023 08:23:33 +0300 Subject: [PATCH 04/34] add ordered mode --- src/Core/SettingsEnums.cpp | 3 + src/Core/SettingsEnums.h | 9 ++ src/Storages/S3Queue/S3QueueSettings.h | 11 +- src/Storages/S3Queue/S3QueueSource.cpp | 49 ++++-- src/Storages/S3Queue/S3QueueSource.h | 7 +- src/Storages/S3Queue/StorageS3Queue.cpp | 190 ++++++++++-------------- src/Storages/S3Queue/StorageS3Queue.h | 11 +- 7 files changed, 153 insertions(+), 127 deletions(-) diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index fdda4ec8bc6..303a05ce2e0 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -206,4 +206,7 @@ IMPLEMENT_SETTING_ENUM(S3QueueMode, ErrorCodes::BAD_ARGUMENTS, {{"ordered", S3QueueMode::ORDERED}, {"unordered", S3QueueMode::UNORDERED}}) +IMPLEMENT_SETTING_ENUM(S3QueueAction, ErrorCodes::BAD_ARGUMENTS, + {{"keep", S3QueueAction::KEEP}, + {"delete", S3QueueAction::DELETE}}) } diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index b8912e5b9b4..8ac727170cf 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -226,4 +226,13 @@ enum class S3QueueMode }; DECLARE_SETTING_ENUM(S3QueueMode) + +enum class S3QueueAction +{ + KEEP, + DELETE, +}; + +DECLARE_SETTING_ENUM(S3QueueAction) + } diff --git a/src/Storages/S3Queue/S3QueueSettings.h b/src/Storages/S3Queue/S3QueueSettings.h index 3860a236a9b..d556a768a15 100644 --- a/src/Storages/S3Queue/S3QueueSettings.h +++ b/src/Storages/S3Queue/S3QueueSettings.h @@ -15,13 +15,14 @@ class ASTStorage; mode, \ S3QueueMode::ORDERED, \ "With unordered mode, the set of all already processed files is tracked with persistent nodes in ZooKepeer", \ - 0) \ - M(String, keeper_path, "", "Zookeeper node path", 0) \ + 0) \ + M(S3QueueAction, after_processing, S3QueueAction::KEEP , "Delete, keep or move file in S3 after processing", 0) \ + M(String, keeper_path, "", "Zookeeper node path", 0) \ M(String, processed_action, "keep", "Keep, delete or move file after successful loading", 0) \ M(UInt64, s3queue_max_retries, 0, "Retry loading up to specified number of times", 0) \ - M(UInt64, s3queue_polling_min_timeout, 1000, "Minimal timeout before next polling", 0) \ - M(UInt64, s3queue_polling_max_timeout, 10000, "Maximum timeout before next polling", 0) \ - M(UInt64, s3queue_polling_backoff, 0, "Retry loading up to specified number of times", 0) + M(UInt64, s3queue_polling_min_timeout_ms, 1000, "Minimal timeout before next polling", 0) \ + M(UInt64, s3queue_polling_max_timeout_ms, 10000, "Maximum timeout before next polling", 0) \ + M(UInt64, s3queue_polling_backoff_ms, 0, "Retry loading up to specified number of times", 0) #define LIST_OF_S3QUEUE_SETTINGS(M, ALIAS) \ S3QUEUE_RELATED_SETTINGS(M, ALIAS) \ diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 92056c79d8f..80478d2f326 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -150,15 +150,19 @@ StorageS3QueueSource::QueueGlobIterator::QueueGlobIterator( } } -Strings StorageS3QueueSource::QueueGlobIterator::setProcessing(String & mode, std::unordered_set & exclude_keys) +Strings StorageS3QueueSource::QueueGlobIterator::setProcessing(S3QueueMode & engine_mode, std::unordered_set & exclude_keys, const String & max_file) { for (KeyWithInfo val : keys_buf) { - if (exclude_keys.find(bucket + '/' + val.key) != exclude_keys.end()) + auto full_path = bucket + '/' + val.key; + if (exclude_keys.find(full_path) != exclude_keys.end()) { LOG_INFO(log, "Found in exclude keys {}", val.key); continue; } + if (engine_mode == S3QueueMode::ORDERED && full_path.compare(max_file) <= 0) { + continue; + } if (processing_keys.size() < max_poll_size) { processing_keys.push_back(val); @@ -169,7 +173,7 @@ Strings StorageS3QueueSource::QueueGlobIterator::setProcessing(String & mode, st } } - if (mode == "ordered") + if (engine_mode == S3QueueMode::ORDERED) { std::sort( processing_keys.begin(), @@ -229,6 +233,8 @@ StorageS3QueueSource::StorageS3QueueSource( const String & bucket_, const String & version_id_, std::shared_ptr file_iterator_, + const S3QueueMode & mode_, + const S3QueueAction & action_, zkutil::ZooKeeperPtr current_zookeeper, const String & zookeeper_path_, const size_t download_thread_num_) @@ -247,6 +253,8 @@ StorageS3QueueSource::StorageS3QueueSource( , format_settings(format_settings_) , requested_virtual_columns(requested_virtual_columns_) , file_iterator(file_iterator_) + , mode(mode_) + , action(action_) , download_thread_num(download_thread_num_) , zookeeper(current_zookeeper) , zookeeper_path(zookeeper_path_) @@ -418,7 +426,7 @@ Chunk StorageS3QueueSource::generate() } LOG_WARNING(log, "Set processed: {}", file_path); setFileProcessed(file_path); - // TODO: Set processed + applyActionAfterProcessing(file_path); return chunk; } } @@ -449,14 +457,18 @@ Chunk StorageS3QueueSource::generate() void StorageS3QueueSource::setFileProcessed(const String & file_path) { std::lock_guard lock(mutex); - String processed_files = zookeeper->get(zookeeper_path + "/processed"); - std::unordered_set processed = parseCollection(processed_files); + if (mode == S3QueueMode::UNORDERED) { + String processed_files = zookeeper->get(zookeeper_path + "/processed"); + std::unordered_set processed = parseCollection(processed_files); - processed.insert(file_path); - Strings set_processed; - set_processed.insert(set_processed.end(), processed.begin(), processed.end()); + processed.insert(file_path); + Strings set_processed; + set_processed.insert(set_processed.end(), processed.begin(), processed.end()); - zookeeper->set(zookeeper_path + "/processed", toString(set_processed)); + zookeeper->set(zookeeper_path + "/processed", toString(set_processed)); + } else { + zookeeper->set(zookeeper_path + "/processed", file_path); + } } @@ -473,6 +485,23 @@ void StorageS3QueueSource::setFileFailed(const String & file_path) zookeeper->set(zookeeper_path + "/failed", toString(set_failed)); } + +void StorageS3QueueSource::applyActionAfterProcessing(const String & file_path) +{ + LOG_WARNING(log, "Delete {} Bucke {}", file_path, bucket); + S3::DeleteObjectRequest request; + request.SetBucket(bucket); + request.SetKey(file_path); + auto outcome = client->DeleteObject(request); + if (!outcome.IsSuccess() && !S3::isNotFoundError(outcome.GetError().GetErrorType())) + { + const auto & err = outcome.GetError(); + LOG_ERROR(log, "{} (Code: {})", err.GetMessage(), static_cast(err.GetErrorType())); + } else { + LOG_TRACE(log, "Object with path {} was removed from S3", file_path); + } +} + std::unordered_set StorageS3QueueSource::parseCollection(String & files) { ReadBuffer rb(const_cast(reinterpret_cast(files.data())), files.length(), 0); diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index 3abb388cd2d..b5160588bab 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -52,7 +52,7 @@ public: KeyWithInfo next() override; size_t getTotalSize() const override; - Strings setProcessing(String & mode, std::unordered_set & exclude_keys); + Strings setProcessing(S3QueueMode & engine_mode, std::unordered_set & exclude_keys, const String & max_file = ""); private: size_t max_poll_size = 10; @@ -83,6 +83,8 @@ public: const String & bucket, const String & version_id, std::shared_ptr file_iterator_, + const S3QueueMode & mode_, + const S3QueueAction & action_, zkutil::ZooKeeperPtr current_zookeeper, const String & zookeeper_path_, size_t download_thread_num); @@ -114,6 +116,8 @@ private: std::vector requested_virtual_columns; std::shared_ptr file_iterator; + const S3QueueMode mode; + const S3QueueAction action; size_t download_thread_num = 1; Poco::Logger * log = &Poco::Logger::get("StorageS3QueueSource"); @@ -140,6 +144,7 @@ private: void setFileProcessed(const String & file_path); void setFileFailed(const String & file_path); + void applyActionAfterProcessing(const String & file_path); }; } diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 1cb04350635..9febd6b2c9f 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -86,33 +86,13 @@ namespace ProfileEvents { extern const Event S3DeleteObjects; extern const Event S3ListObjects; -extern const Event S3QueueBackgroundReads; } namespace DB { static const String PARTITION_ID_WILDCARD = "{_partition_id}"; - -static const std::unordered_set required_configuration_keys = { - "url", -}; -static const std::unordered_set optional_configuration_keys = { - "format", - "compression", - "compression_method", - "structure", - "access_key_id", - "secret_access_key", - "filename", - "use_environment_credentials", - "max_single_read_retries", - "min_upload_part_size", - "upload_part_size_multiply_factor", - "upload_part_size_multiply_parts_count_threshold", - "max_single_part_upload_size", - "max_connections", -}; +static const auto MAX_THREAD_WORK_DURATION_MS = 60000; namespace ErrorCodes { @@ -131,8 +111,6 @@ namespace ErrorCodes extern const int REPLICA_ALREADY_EXISTS; } -class IOutputFormat; -using OutputFormatPtr = std::shared_ptr; StorageS3Queue::StorageS3Queue( std::unique_ptr s3queue_settings_, @@ -150,6 +128,9 @@ StorageS3Queue::StorageS3Queue( , s3queue_settings(std::move(s3queue_settings_)) , s3_configuration{configuration_} , keys({s3_configuration.url.key}) + , mode(s3queue_settings->mode) + , after_processing(s3queue_settings->after_processing) + , milliseconds_to_wait(s3queue_settings->s3queue_polling_min_timeout_ms) , format_name(configuration_.format) , compression_method(configuration_.compression_method) , name(s3_configuration.url.storage_name) @@ -159,7 +140,15 @@ StorageS3Queue::StorageS3Queue( , is_key_with_globs(s3_configuration.url.key.find_first_of("*?{") != std::string::npos) , log(&Poco::Logger::get("StorageS3Queue (" + table_id_.table_name + ")")) { + LOG_INFO(log, "Init engine"); + + if (!is_key_with_globs) + { + throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "S3Queue engine can read only from url with globs"); + } + String setting_zookeeper_path = s3queue_settings->keeper_path; + LOG_INFO(log, "Setttings zookeeper_path={}", setting_zookeeper_path); if (setting_zookeeper_path == "") { auto table_id = getStorageID(); @@ -181,12 +170,8 @@ StorageS3Queue::StorageS3Queue( { zookeeper_path = zkutil::extractZooKeeperPath(s3queue_settings->keeper_path, /* check_starts_with_slash */ true, log); } - LOG_INFO(log, "Storage S3Queue zookeeper_path= {} with mode", zookeeper_path); + LOG_INFO(log, "Set zookeeper_path={}", zookeeper_path); - if (!is_key_with_globs) - { - throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "S3Queue engine can read only from key with globs"); - } FormatFactory::instance().checkFormatName(format_name); context_->getGlobalContext()->getRemoteHostFilter().checkURL(s3_configuration.url.uri); StorageInMemoryMetadata storage_metadata; @@ -213,12 +198,12 @@ StorageS3Queue::StorageS3Queue( for (const auto & column : virtual_columns) virtual_block.insert({column.type->createColumn(), column.type, column.name}); - auto thread = context_->getSchedulePool().createTask("S3QueueStreamingTask", [this] { threadFunc(); }); setZooKeeper(); auto metadata_snapshot = getInMemoryMetadataPtr(); createTableIfNotExists(metadata_snapshot); - task = std::make_shared(std::move(thread)); - LOG_TRACE(log, "Complete"); + + auto poll_thread = context_->getSchedulePool().createTask("S3QueueStreamingTask", [this] { threadFunc(); }); + task = std::make_shared(std::move(poll_thread)); } @@ -241,21 +226,15 @@ Pipe StorageS3Queue::read( size_t max_block_size, size_t num_streams) { - // if (!local_context->getSettingsRef().stream_like_engine_allow_direct_select) - // throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, - // "Direct select is not allowed. To enable use setting `stream_like_engine_allow_direct_select`"); + if (!local_context->getSettingsRef().stream_like_engine_allow_direct_select) + throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, + "Direct select is not allowed. To enable use setting `stream_like_engine_allow_direct_select`"); if (mv_attached) throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "Cannot read from StorageRabbitMQ with attached materialized views"); auto query_s3_configuration = updateConfigurationAndGetCopy(local_context); - bool has_wildcards = query_s3_configuration.url.bucket.find(PARTITION_ID_WILDCARD) != String::npos - || keys.back().find(PARTITION_ID_WILDCARD) != String::npos; - - if (partition_by && has_wildcards) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Reading from a partitioned S3 storage is not implemented yet"); - Pipes pipes; std::unordered_set column_names_set(column_names.begin(), column_names.end()); @@ -295,9 +274,6 @@ Pipe StorageS3Queue::read( } const size_t max_download_threads = local_context->getSettingsRef().max_download_threads; - // const size_t max_download_threads = 1; - - auto zookeeper = getZooKeeper(); for (size_t i = 0; i < num_streams; ++i) { @@ -316,6 +292,8 @@ Pipe StorageS3Queue::read( query_s3_configuration.url.bucket, query_s3_configuration.url.version_id, iterator_wrapper, + mode, + after_processing, zookeeper, zookeeper_path, max_download_threads)); @@ -332,11 +310,6 @@ NamesAndTypesList StorageS3Queue::getVirtuals() const return virtual_columns; } -Names StorageS3Queue::getVirtualColumnNames() -{ - return {"_path", "_file"}; -} - bool StorageS3Queue::supportsPartitionBy() const { return true; @@ -351,21 +324,10 @@ void StorageS3Queue::startup() void StorageS3Queue::shutdown() { shutdown_called = true; - LOG_TRACE(log, "Deactivating background tasks"); - if (task) { task->stream_cancelled = true; - - /// Reader thread may wait for wake up - // wakeUp(); - - LOG_TRACE(log, "Waiting for cleanup"); task->holder->deactivate(); - /// If no reading call and threadFunc, the log files will never - /// be opened, also just leave the work of close files and - /// store meta to streams. because if we close files in here, - /// may result in data race with unfinishing reading pipeline } } @@ -409,7 +371,6 @@ void StorageS3Queue::threadFunc() auto table_id = getStorageID(); auto dependencies_count = getTableDependentCount(); - LOG_TRACE(log, "dependencies_count {}", toString(dependencies_count)); if (dependencies_count) { auto start_time = std::chrono::steady_clock::now(); @@ -426,21 +387,18 @@ void StorageS3Queue::threadFunc() } LOG_DEBUG(log, "Started streaming to {} attached views", dependencies_count); - - if (streamToViews()) - { - LOG_TRACE(log, "Stream stalled. Reschedule."); - break; - } + streamToViews(); auto ts = std::chrono::steady_clock::now(); auto duration = std::chrono::duration_cast(ts - start_time); - if (duration.count() > 600000) + if (duration.count() > MAX_THREAD_WORK_DURATION_MS) { LOG_TRACE(log, "Thread work duration limit exceeded. Reschedule."); reschedule = true; break; } + + milliseconds_to_wait = s3queue_settings->s3queue_polling_min_timeout_ms; } } } @@ -451,30 +409,24 @@ void StorageS3Queue::threadFunc() mv_attached.store(false); - // Wait for attached views if (reschedule && !shutdown_called) { LOG_TRACE(log, "Reschedule S3 Queue thread func."); /// Reschedule with backoff. + if (milliseconds_to_wait < s3queue_settings->s3queue_polling_max_timeout_ms) + milliseconds_to_wait += s3queue_settings->s3queue_polling_backoff_ms; task->holder->scheduleAfter(milliseconds_to_wait); } } -bool StorageS3Queue::streamToViews() +void StorageS3Queue::streamToViews() { - LOG_TRACE(log, "streamToViews"); - - Stopwatch watch; - auto table_id = getStorageID(); auto table = DatabaseCatalog::instance().getTable(table_id, getContext()); if (!table) throw Exception(ErrorCodes::LOGICAL_ERROR, "Engine table {} doesn't exist.", table_id.getNameForLogs()); - // CurrentMetrics::Increment metric_increment{CurrentMetrics::KafkaBackgroundReads}; - // ProfileEvents::increment(ProfileEvents::S3QueueBackgroundReads); - auto storage_snapshot = getStorageSnapshot(getInMemoryMetadataPtr(), getContext()); // Create an INSERT query for streaming data @@ -487,8 +439,6 @@ bool StorageS3Queue::streamToViews() s3queue_context->makeQueryContext(); auto query_s3_configuration = updateConfigurationAndGetCopy(s3queue_context); - // s3queue_context->applySettingsChanges(settings_adjustments); - // Create a stream for each consumer and join them in a union stream // Only insert into dependent views and expect that input blocks contain virtual columns InterpreterInsertQuery interpreter(insert, s3queue_context, false, true, true); @@ -552,6 +502,8 @@ bool StorageS3Queue::streamToViews() query_s3_configuration.url.bucket, query_s3_configuration.url.version_id, iterator_wrapper, + mode, + after_processing, zookeeper, zookeeper_path, max_download_threads)); @@ -559,24 +511,15 @@ bool StorageS3Queue::streamToViews() auto pipe = Pipe::unitePipes(std::move(pipes)); - // We can't cancel during copyData, as it's not aware of commits and other kafka-related stuff. - // It will be cancelled on underlying layer (kafka buffer) - std::atomic_size_t rows = 0; { block_io.pipeline.complete(std::move(pipe)); - - // we need to read all consumers in parallel (sequential read may lead to situation - // when some of consumers are not used, and will break some Kafka consumer invariants) block_io.pipeline.setNumThreads(num_streams); block_io.pipeline.setProgressCallback([&](const Progress & progress) { rows += progress.read_rows.load(); }); CompletedPipelineExecutor executor(block_io.pipeline); executor.execute(); } - - bool some_stream_is_stalled = false; - return some_stream_is_stalled; } StorageS3Queue::Configuration StorageS3Queue::updateConfigurationAndGetCopy(ContextPtr local_context) @@ -620,23 +563,18 @@ bool StorageS3Queue::createTableIfNotExists(const StorageMetadataPtr & metadata_ { ops.emplace_back(zkutil::makeCreateRequest( fs::path(zookeeper_path) / "processing" / toString(table_uuid), "{}", zkutil::CreateMode::Ephemeral)); - LOG_DEBUG(log, "This table {} is already created, will add new replica", zookeeper_path); } else { - /// We write metadata of table so that the replicas can check table parameters with them. - // String metadata_str = ReplicatedMergeTreeTableMetadata(*this, metadata_snapshot).toString(); + String deafult_processed = mode == S3QueueMode::ORDERED ? "" : "[]"; ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path, "", zkutil::CreateMode::Persistent)); - - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processed", "[]", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processed", deafult_processed, zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/failed", "[]", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processing", "", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest( fs::path(zookeeper_path) / "processing" / toString(table_uuid), "[]", zkutil::CreateMode::Ephemeral)); ops.emplace_back(zkutil::makeCreateRequest( zookeeper_path + "/columns", metadata_snapshot->getColumns().toString(), zkutil::CreateMode::Persistent)); - // ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/metadata", metadata_str, - // zkutil::CreateMode::Persistent)); } Coordination::Responses responses; auto code = zookeeper->tryMulti(ops, responses); @@ -653,7 +591,6 @@ bool StorageS3Queue::createTableIfNotExists(const StorageMetadataPtr & metadata_ return true; } - /// Do not use LOGICAL_ERROR code, because it may happen if user has specified wrong zookeeper_path throw Exception( ErrorCodes::REPLICA_ALREADY_EXISTS, "Cannot create table, because it is created concurrently every time or because " @@ -667,45 +604,82 @@ StorageS3Queue::createFileIterator(ContextPtr local_context, ASTPtr query, KeysW /// Iterate through disclosed globs and make a source for each file auto it = std::make_shared( *s3_configuration.client, s3_configuration.url, query, virtual_block, local_context, read_keys, s3_configuration.request_settings); - String cur_mode = "unordered"; std::lock_guard lock{sync_mutex}; std::unordered_set exclude = getExcludedFiles(); + Strings processing; + if (mode == S3QueueMode::UNORDERED) { + processing = it->setProcessing(mode, exclude); + } else { + String max_processed_file = getMaxProcessedFile(); + processing = it->setProcessing(mode, exclude, max_processed_file); + } + auto zookeeper = getZooKeeper(); auto table_uuid = getStorageID().uuid; - Strings processing = it->setProcessing(cur_mode, exclude); zookeeper->set(fs::path(zookeeper_path) / "processing" / toString(table_uuid), toString(processing)); return it; } -std::unordered_set StorageS3Queue::getExcludedFiles() -{ +std::unordered_set StorageS3Queue::getFailedFiles() { auto zookeeper = getZooKeeper(); - std::unordered_set exclude_files; String failed = zookeeper->get(zookeeper_path + "/failed"); std::unordered_set failed_files = StorageS3QueueSource::parseCollection(failed); - LOG_DEBUG(log, "failed_files {}", failed_files.size()); + return failed_files; +} + +std::unordered_set StorageS3Queue::getProcessedFiles() { + auto zookeeper = getZooKeeper(); + String processed = zookeeper->get(zookeeper_path + "/processed"); std::unordered_set processed_files = StorageS3QueueSource::parseCollection(processed); - LOG_DEBUG(log, "processed_files {}", processed_files.size()); - exclude_files.merge(failed_files); - exclude_files.merge(processed_files); + return processed_files; +} + +String StorageS3Queue::getMaxProcessedFile() { + auto zookeeper = getZooKeeper(); + + String processed = zookeeper->get(zookeeper_path + "/processed"); + return processed; +} + +std::unordered_set StorageS3Queue::getProcessingFiles() { + auto zookeeper = getZooKeeper(); Strings consumer_table_uuids; zookeeper->tryGetChildren(zookeeper_path + "/processing", consumer_table_uuids); - + std::unordered_set processing_files; for (const auto & uuid : consumer_table_uuids) { String processing = zookeeper->get(fs::path(zookeeper_path) / "processing" / toString(uuid)); - std::unordered_set processing_files = StorageS3QueueSource::parseCollection(processing); - LOG_DEBUG(log, "processing {}", processing_files.size()); - exclude_files.merge(processing_files); + std::unordered_set cur_processing_files = StorageS3QueueSource::parseCollection(processing); + processing_files.merge(cur_processing_files); } + return processing_files; +} + +std::unordered_set StorageS3Queue::getExcludedFiles() +{ + std::unordered_set exclude_files; + + std::unordered_set failed_files = getFailedFiles(); + LOG_DEBUG(log, "failed_files {}", failed_files.size()); + exclude_files.merge(failed_files); + + if (mode != S3QueueMode::ORDERED) { + std::unordered_set processed_files = getProcessedFiles(); + LOG_DEBUG(log, "processed_files {}", processed_files.size()); + exclude_files.merge(processed_files); + } + + std::unordered_set processing_files = getProcessingFiles(); + LOG_DEBUG(log, "processing {}", processing_files.size()); + exclude_files.merge(processing_files); return exclude_files; } diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index 1763e882d59..257cb23f303 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -89,6 +89,8 @@ private: std::vector keys; NamesAndTypesList virtual_columns; Block virtual_block; + S3QueueMode mode; + S3QueueAction after_processing; uint64_t milliseconds_to_wait = 10000; String format_name; @@ -121,8 +123,6 @@ private: std::shared_ptr task; bool supportsSubsetOfColumns() const override; - static Names getVirtualColumnNames(); - String zookeeper_path; zkutil::ZooKeeperPtr current_zookeeper; @@ -142,9 +142,14 @@ private: createFileIterator(ContextPtr local_context, ASTPtr query, KeysWithInfo * read_keys = nullptr); static std::unordered_set parseCollection(String & files); + + std::unordered_set getFailedFiles(); + std::unordered_set getProcessedFiles(); + String getMaxProcessedFile(); + std::unordered_set getProcessingFiles(); std::unordered_set getExcludedFiles(); - bool streamToViews(); + void streamToViews(); Configuration updateConfigurationAndGetCopy(ContextPtr local_context); }; From 624d846feae7bcc73b0d5683d52d02c187fb7ea3 Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Mon, 1 May 2023 15:50:20 +0300 Subject: [PATCH 05/34] add template for tests --- src/Storages/S3Queue/S3QueueSource.cpp | 2 +- .../test_storage_s3_queue/__init__.py | 0 .../configs/defaultS3.xml | 11 ++ .../configs/named_collections.xml | 43 ++++++ .../test_storage_s3_queue/configs/users.xml | 7 + .../integration/test_storage_s3_queue/test.py | 123 ++++++++++++++++++ 6 files changed, 185 insertions(+), 1 deletion(-) create mode 100644 tests/integration/test_storage_s3_queue/__init__.py create mode 100644 tests/integration/test_storage_s3_queue/configs/defaultS3.xml create mode 100644 tests/integration/test_storage_s3_queue/configs/named_collections.xml create mode 100644 tests/integration/test_storage_s3_queue/configs/users.xml create mode 100644 tests/integration/test_storage_s3_queue/test.py diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 80478d2f326..e117ec52e0e 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -259,7 +259,7 @@ StorageS3QueueSource::StorageS3QueueSource( , zookeeper(current_zookeeper) , zookeeper_path(zookeeper_path_) , create_reader_pool(CurrentMetrics::StorageS3Threads, CurrentMetrics::StorageS3ThreadsActive, 1) - , create_reader_scheduler(threadPoolCallbackRunner(create_reader_pool, "CreateS3QueueReader")) + , create_reader_scheduler(threadPoolCallbackRunner(create_reader_pool, "CreateS3QReader")) { reader = createReader(); if (reader) diff --git a/tests/integration/test_storage_s3_queue/__init__.py b/tests/integration/test_storage_s3_queue/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_storage_s3_queue/configs/defaultS3.xml b/tests/integration/test_storage_s3_queue/configs/defaultS3.xml new file mode 100644 index 00000000000..7dac6d9fbb5 --- /dev/null +++ b/tests/integration/test_storage_s3_queue/configs/defaultS3.xml @@ -0,0 +1,11 @@ + + + + http://resolver:8080 +
Authorization: Bearer TOKEN
+
+ + http://resolver:8080/root-with-auth/restricteddirectory/ + +
+
diff --git a/tests/integration/test_storage_s3_queue/configs/named_collections.xml b/tests/integration/test_storage_s3_queue/configs/named_collections.xml new file mode 100644 index 00000000000..64674e2a3e3 --- /dev/null +++ b/tests/integration/test_storage_s3_queue/configs/named_collections.xml @@ -0,0 +1,43 @@ + + + + http://minio1:9001/root/test_table + minio + minio123 + + + http://minio1:9001/root/test_parquet + minio + minio123 + + + http://minio1:9001/root/test_parquet_gz + minio + minio123 + + + http://minio1:9001/root/test_orc + minio + minio123 + + + http://minio1:9001/root/test_native + minio + minio123 + + + http://minio1:9001/root/test.arrow + minio + minio123 + + + http://minio1:9001/root/test.parquet + minio + minio123 + + + http://minio1:9001/root/test_cache4.jsonl + true + + + diff --git a/tests/integration/test_storage_s3_queue/configs/users.xml b/tests/integration/test_storage_s3_queue/configs/users.xml new file mode 100644 index 00000000000..2cef0a6de3c --- /dev/null +++ b/tests/integration/test_storage_s3_queue/configs/users.xml @@ -0,0 +1,7 @@ + + + + 1 + + + diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py new file mode 100644 index 00000000000..92f7955c9a4 --- /dev/null +++ b/tests/integration/test_storage_s3_queue/test.py @@ -0,0 +1,123 @@ +import gzip +import json +import logging +import os +import io +import random +import threading +import time + +import helpers.client +import pytest +from helpers.cluster import ClickHouseCluster, ClickHouseInstance +from helpers.network import PartitionManager +from helpers.mock_servers import start_mock_servers +from helpers.test_tools import exec_query_with_retry +from helpers.s3_tools import prepare_s3_bucket + +cluster = ClickHouseCluster(__file__) +instance = cluster.add_instance( + "instance", + user_configs=["configs/users.xml"], + with_minio=True, + with_zookeeper=True, +) + + +MINIO_INTERNAL_PORT = 9001 + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + + +def put_s3_file_content(started_cluster, bucket, filename, data): + buf = io.BytesIO(data) + started_cluster.minio_client.put_object(bucket, filename, buf, len(data)) + + +# Returns content of given S3 file as string. +def get_s3_file_content(started_cluster, bucket, filename, decode=True): + # type: (ClickHouseCluster, str, str, bool) -> str + + data = started_cluster.minio_client.get_object(bucket, filename) + data_str = b"" + for chunk in data.stream(): + data_str += chunk + if decode: + return data_str.decode() + return data_str + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster = ClickHouseCluster(__file__) + cluster.add_instance( + "instance", + user_configs=["configs/users.xml"], + with_minio=True, + with_zookeeper=True, + main_configs=[ + "configs/defaultS3.xml", + "configs/named_collections.xml" + ], + ) + + logging.info("Starting cluster...") + cluster.start() + logging.info("Cluster started") + + prepare_s3_bucket(cluster) + # logging.info("S3 bucket created") + # run_s3_mocks(cluster) + + yield cluster + finally: + cluster.shutdown() + + +def run_query(instance, query, stdin=None, settings=None): + # type: (ClickHouseInstance, str, object, dict) -> str + + logging.info("Running query '{}'...".format(query)) + result = instance.query(query, stdin=stdin, settings=settings) + logging.info("Query finished") + + return result + + +def test_get_file(started_cluster): + auth = "'minio','minio123'," + bucket = started_cluster.minio_restricted_bucket + instance = started_cluster.instances["instance"] + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + values = [ + [12549, 2463, 19893], + [64021, 38652, 66703], + [81611, 39650, 83516], + [11079, 59507, 61546], + [51764, 69952, 6876], + [41165, 90293, 29095], + [40167, 78432, 48309], + [81629, 81327, 11855], + [55852, 21643, 98507], + [6738, 54643, 41155], + ] + values_csv = ( + "\n".join((",".join(map(str, row)) for row in values)) + "\n" + ).encode() + filename = f"test.csv" + put_s3_file_content(started_cluster, bucket, filename, values_csv) + + instance.query( + f"create table test ({table_format}) engine=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/*', {auth}'CSV') SETTINGS mode = 'unordered', keeper_path = '/clickhouse/testing'" + ) + + get_query = f"SELECT * FROM test" + assert [ + list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + ] == values + + get_query = f"SELECT * FROM test" + assert [ + list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + ] == [] \ No newline at end of file From f21c838f111aa4715727f5a275a5b8ba2e36e2ca Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Tue, 2 May 2023 17:27:53 +0300 Subject: [PATCH 06/34] add some tests --- .../integration/test_storage_s3_queue/test.py | 303 +++++++++++++++--- 1 file changed, 265 insertions(+), 38 deletions(-) diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index 92f7955c9a4..8cdf7e8dda0 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -1,30 +1,26 @@ -import gzip -import json +import io import logging import os -import io import random -import threading import time -import helpers.client import pytest from helpers.cluster import ClickHouseCluster, ClickHouseInstance -from helpers.network import PartitionManager -from helpers.mock_servers import start_mock_servers -from helpers.test_tools import exec_query_with_retry from helpers.s3_tools import prepare_s3_bucket -cluster = ClickHouseCluster(__file__) -instance = cluster.add_instance( - "instance", - user_configs=["configs/users.xml"], - with_minio=True, - with_zookeeper=True, -) + +@pytest.fixture(autouse=True) +def s3_queue_setup_teardown(started_cluster): + instance = started_cluster.instances["instance"] + instance.query("DROP DATABASE IF EXISTS test; CREATE DATABASE test;") + # logging.debug("kafka is available - running test") + yield # run test MINIO_INTERNAL_PORT = 9001 +AVAILABLE_MODES = ["ordered", "unordered"] +AUTH = "'minio','minio123'," + SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) @@ -34,6 +30,24 @@ def put_s3_file_content(started_cluster, bucket, filename, data): started_cluster.minio_client.put_object(bucket, filename, buf, len(data)) +def generate_random_files( + count, prefix, cluster, bucket, column_num=3, row_num=10, start_ind=0 +): + total_values = [] + for i in range(start_ind, start_ind + count): + print(i) + rand_values = [ + [random.randint(0, 50) for _ in range(column_num)] for _ in range(row_num) + ] + total_values += rand_values + values_csv = ( + "\n".join((",".join(map(str, row)) for row in rand_values)) + "\n" + ).encode() + filename = f"{prefix}/test_{i}.csv" + put_s3_file_content(cluster, bucket, filename, values_csv) + return total_values + + # Returns content of given S3 file as string. def get_s3_file_content(started_cluster, bucket, filename, decode=True): # type: (ClickHouseCluster, str, str, bool) -> str @@ -56,10 +70,7 @@ def started_cluster(): user_configs=["configs/users.xml"], with_minio=True, with_zookeeper=True, - main_configs=[ - "configs/defaultS3.xml", - "configs/named_collections.xml" - ], + main_configs=["configs/defaultS3.xml", "configs/named_collections.xml"], ) logging.info("Starting cluster...") @@ -67,9 +78,6 @@ def started_cluster(): logging.info("Cluster started") prepare_s3_bucket(cluster) - # logging.info("S3 bucket created") - # run_s3_mocks(cluster) - yield cluster finally: cluster.shutdown() @@ -85,7 +93,8 @@ def run_query(instance, query, stdin=None, settings=None): return result -def test_get_file(started_cluster): +@pytest.mark.parametrize("mode", AVAILABLE_MODES) +def test_direct_select_file(started_cluster, mode): auth = "'minio','minio123'," bucket = started_cluster.minio_restricted_bucket instance = started_cluster.instances["instance"] @@ -94,30 +103,248 @@ def test_get_file(started_cluster): [12549, 2463, 19893], [64021, 38652, 66703], [81611, 39650, 83516], - [11079, 59507, 61546], - [51764, 69952, 6876], - [41165, 90293, 29095], - [40167, 78432, 48309], - [81629, 81327, 11855], - [55852, 21643, 98507], - [6738, 54643, 41155], ] values_csv = ( - "\n".join((",".join(map(str, row)) for row in values)) + "\n" + "\n".join((",".join(map(str, row)) for row in values)) + "\n" ).encode() filename = f"test.csv" put_s3_file_content(started_cluster, bucket, filename, values_csv) + instance.query("drop table if exists test.s3_queue") + instance.query("drop table if exists test.s3_queue_2") + instance.query("drop table if exists test.s3_queue_3") instance.query( - f"create table test ({table_format}) engine=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/*', {auth}'CSV') SETTINGS mode = 'unordered', keeper_path = '/clickhouse/testing'" + f""" + CREATE TABLE test.s3_queue ({table_format}) + ENGINE = S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/*', {auth}'CSV') + SETTINGS + mode = '{mode}', + keeper_path = '/clickhouse/select_{mode}' + """ ) - get_query = f"SELECT * FROM test" + get_query = f"SELECT * FROM test.s3_queue" assert [ - list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() - ] == values + list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + ] == values - get_query = f"SELECT * FROM test" + instance.query( + f""" + CREATE TABLE test.s3_queue_2 ({table_format}) + ENGINE = S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/*', {auth}'CSV') + SETTINGS + mode = '{mode}', + keeper_path = '/clickhouse/select_{mode}' + """ + ) + + get_query = f"SELECT * FROM test.s3_queue" assert [ - list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() - ] == [] \ No newline at end of file + list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + ] == [] + # New table with same zookeeper path + get_query = f"SELECT * FROM test.s3_queue_2" + assert [ + list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + ] == [] + # New table with different zookeeper path + instance.query( + f""" + CREATE TABLE test.s3_queue_3 ({table_format}) + ENGINE = S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/*', {auth}'CSV') + SETTINGS + mode = '{mode}', + keeper_path='/clickhouse/select_{mode}_2' + """ + ) + get_query = f"SELECT * FROM test.s3_queue_3" + assert [ + list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + ] == values + + values = [ + [1, 1, 1], + ] + values_csv = ( + "\n".join((",".join(map(str, row)) for row in values)) + "\n" + ).encode() + filename = f"t.csv" + put_s3_file_content(started_cluster, bucket, filename, values_csv) + + get_query = f"SELECT * FROM test.s3_queue_3" + if mode == "unordered": + assert [ + list(map(int, l.split())) + for l in run_query(instance, get_query).splitlines() + ] == values + elif mode == "ordered": + assert [ + list(map(int, l.split())) + for l in run_query(instance, get_query).splitlines() + ] == [] + + +@pytest.mark.parametrize("mode", AVAILABLE_MODES) +def test_direct_select_multiple_files(started_cluster, mode): + prefix = f"multiple_files_{mode}" + bucket = started_cluster.minio_restricted_bucket + instance = started_cluster.instances["instance"] + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + instance.query("drop table if exists test.s3_queue") + instance.query( + f""" + CREATE TABLE test.s3_queue ({table_format}) + ENGINE = S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + SETTINGS + mode = '{mode}', + keeper_path = '/clickhouse/select_multiple_{mode}' + """ + ) + + for i in range(10): + print(i) + rand_values = [[random.randint(0, 50) for _ in range(3)] for _ in range(10)] + + values_csv = ( + "\n".join((",".join(map(str, row)) for row in rand_values)) + "\n" + ).encode() + filename = f"{prefix}/test_{i}.csv" + put_s3_file_content(started_cluster, bucket, filename, values_csv) + + get_query = f"SELECT * FROM test.s3_queue" + assert [ + list(map(int, l.split())) + for l in run_query(instance, get_query).splitlines() + ] == rand_values + + total_values = generate_random_files( + 5, prefix, started_cluster, bucket, start_ind=10 + ) + get_query = f"SELECT * FROM test.s3_queue" + assert { + tuple(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + } == set([tuple(i) for i in total_values]) + + +@pytest.mark.parametrize("mode", AVAILABLE_MODES) +def test_streaming_to_view_(started_cluster, mode): + prefix = f"streaming_files_{mode}" + bucket = started_cluster.minio_restricted_bucket + instance = started_cluster.instances["instance"] + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + + total_values = generate_random_files(10, prefix, started_cluster, bucket) + instance.query( + f""" + DROP TABLE IF EXISTS test.s3_queue_persistent; + DROP TABLE IF EXISTS test.s3_queue; + DROP TABLE IF EXISTS test.persistent_s3_queue_mv; + + CREATE TABLE test.s3_queue_persistent ({table_format}) + ENGINE = MergeTree() + ORDER BY column1; + + CREATE TABLE test.s3_queue ({table_format}) + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + SETTINGS + mode = '{mode}', + keeper_path = '/clickhouse/select_multiple_{mode}'; + + CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv TO test.s3_queue_persistent AS + SELECT + * + FROM test.s3_queue; + """ + ) + expected_values = set([tuple(i) for i in total_values]) + for i in range(10): + get_query = f"SELECT * FROM test.persistent_s3_queue_mv" + + selected_values = { + tuple(map(int, l.split())) + for l in run_query(instance, get_query).splitlines() + } + if selected_values != expected_values: + time.sleep(1) + else: + break + + assert selected_values == expected_values + + +@pytest.mark.parametrize("mode", AVAILABLE_MODES) +def test_streaming_to_many_views(started_cluster, mode): + prefix = f"streaming_files_{mode}" + bucket = started_cluster.minio_restricted_bucket + instance = started_cluster.instances["instance"] + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + retry_cnt = 10 + + instance.query( + f""" + DROP TABLE IF EXISTS test.s3_queue_persistent; + DROP TABLE IF EXISTS test.s3_queue_persistent_2; + DROP TABLE IF EXISTS test.s3_queue_persistent_3; + DROP TABLE IF EXISTS test.s3_queue; + DROP TABLE IF EXISTS test.persistent_s3_queue_mv; + DROP TABLE IF EXISTS test.persistent_s3_queue_mv_2; + DROP TABLE IF EXISTS test.persistent_s3_queue_mv_3; + + + CREATE TABLE test.s3_queue_persistent ({table_format}) + ENGINE = MergeTree() + ORDER BY column1; + + CREATE TABLE test.s3_queue_persistent_2 ({table_format}) + ENGINE = MergeTree() + ORDER BY column1; + + CREATE TABLE test.s3_queue_persistent_3 ({table_format}) + ENGINE = MergeTree() + ORDER BY column1; + + CREATE TABLE test.s3_queue ({table_format}) + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + SETTINGS + mode = '{mode}', + keeper_path = '/clickhouse/select_multiple_{mode}'; + + CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv TO test.s3_queue_persistent AS + SELECT + * + FROM test.s3_queue; + + CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv_2 TO test.s3_queue_persistent_2 AS + SELECT + * + FROM test.s3_queue; + + CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv_3 TO test.s3_queue_persistent_3 AS + SELECT + * + FROM test.s3_queue; + """ + ) + total_values = generate_random_files(10, prefix, started_cluster, bucket) + expected_values = set([tuple(i) for i in total_values]) + + for i in range(retry_cnt): + retry = False + for get_query in [ + f"SELECT * FROM test.s3_queue_persistent", + f"SELECT * FROM test.s3_queue_persistent_2", + f"SELECT * FROM test.s3_queue_persistent_3", + ]: + selected_values = { + tuple(map(int, l.split())) + for l in run_query(instance, get_query).splitlines() + } + if i == retry_cnt - 1: + assert selected_values == expected_values + if selected_values != expected_values: + retry = True + break + if retry: + time.sleep(1) + else: + break From 559e8fa6e5e1ed2f58d2acb738a8dc26f1bdee0a Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Tue, 2 May 2023 18:10:57 +0300 Subject: [PATCH 07/34] fix codestyle --- src/Storages/S3Queue/S3QueueSource.cpp | 28 +++++--------- src/Storages/S3Queue/StorageS3Queue.cpp | 50 +++++++++++++++++-------- src/Storages/S3Queue/StorageS3Queue.h | 10 +---- 3 files changed, 46 insertions(+), 42 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index e117ec52e0e..e77680c0d36 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -109,20 +109,6 @@ static const std::unordered_set optional_configuration_keys "expiration_window_seconds", "no_sign_request"}; -namespace ErrorCodes -{ - extern const int CANNOT_PARSE_TEXT; - extern const int BAD_ARGUMENTS; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int S3_ERROR; - extern const int UNEXPECTED_EXPRESSION; - extern const int DATABASE_ACCESS_DENIED; - extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; - extern const int NOT_IMPLEMENTED; - extern const int CANNOT_COMPILE_REGEXP; - extern const int FILE_DOESNT_EXIST; -} - class IOutputFormat; using OutputFormatPtr = std::shared_ptr; @@ -160,7 +146,8 @@ Strings StorageS3QueueSource::QueueGlobIterator::setProcessing(S3QueueMode & eng LOG_INFO(log, "Found in exclude keys {}", val.key); continue; } - if (engine_mode == S3QueueMode::ORDERED && full_path.compare(max_file) <= 0) { + if (engine_mode == S3QueueMode::ORDERED && full_path.compare(max_file) <= 0) + { continue; } if (processing_keys.size() < max_poll_size) @@ -457,7 +444,8 @@ Chunk StorageS3QueueSource::generate() void StorageS3QueueSource::setFileProcessed(const String & file_path) { std::lock_guard lock(mutex); - if (mode == S3QueueMode::UNORDERED) { + if (mode == S3QueueMode::UNORDERED) + { String processed_files = zookeeper->get(zookeeper_path + "/processed"); std::unordered_set processed = parseCollection(processed_files); @@ -466,7 +454,9 @@ void StorageS3QueueSource::setFileProcessed(const String & file_path) set_processed.insert(set_processed.end(), processed.begin(), processed.end()); zookeeper->set(zookeeper_path + "/processed", toString(set_processed)); - } else { + } + else + { zookeeper->set(zookeeper_path + "/processed", file_path); } } @@ -497,7 +487,9 @@ void StorageS3QueueSource::applyActionAfterProcessing(const String & file_path) { const auto & err = outcome.GetError(); LOG_ERROR(log, "{} (Code: {})", err.GetMessage(), static_cast(err.GetErrorType())); - } else { + } + else + { LOG_TRACE(log, "Object with path {} was removed from S3", file_path); } } diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 9febd6b2c9f..ac9300ac4f4 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -96,16 +96,10 @@ static const auto MAX_THREAD_WORK_DURATION_MS = 60000; namespace ErrorCodes { - extern const int CANNOT_PARSE_TEXT; + extern const int LOGICAL_ERROR; extern const int BAD_ARGUMENTS; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int S3_ERROR; - extern const int UNEXPECTED_EXPRESSION; - extern const int DATABASE_ACCESS_DENIED; - extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; extern const int NOT_IMPLEMENTED; - extern const int CANNOT_COMPILE_REGEXP; - extern const int FILE_DOESNT_EXIST; extern const int QUERY_NOT_ALLOWED; extern const int NO_ZOOKEEPER; extern const int REPLICA_ALREADY_EXISTS; @@ -256,7 +250,8 @@ Pipe StorageS3Queue::read( const auto & virtuals = getVirtuals(); std::erase_if( fetch_columns, - [&](const String & col) { + [&](const String & col) + { return std::any_of( virtuals.begin(), virtuals.end(), [&](const NameAndTypePair & virtual_col) { return col == virtual_col.name; }); }); @@ -305,6 +300,20 @@ Pipe StorageS3Queue::read( return pipe; } +SinkToStoragePtr StorageS3Queue::write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr /*context*/) +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Write is not supported by storage {}", getName()); +} + +void StorageS3Queue::truncate( + const ASTPtr & /*query*/, + const StorageMetadataPtr & /*metadata_snapshot*/, + ContextPtr /*local_context*/, + TableExclusiveLockHolder &) +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Truncate is not supported by storage {}", getName()); +} + NamesAndTypesList StorageS3Queue::getVirtuals() const { return virtual_columns; @@ -462,7 +471,8 @@ void StorageS3Queue::streamToViews() const auto & virtuals = getVirtuals(); std::erase_if( fetch_columns, - [&](const String & col) { + [&](const String & col) + { return std::any_of( virtuals.begin(), virtuals.end(), [&](const NameAndTypePair & virtual_col) { return col == virtual_col.name; }); }); @@ -609,9 +619,12 @@ StorageS3Queue::createFileIterator(ContextPtr local_context, ASTPtr query, KeysW std::unordered_set exclude = getExcludedFiles(); Strings processing; - if (mode == S3QueueMode::UNORDERED) { + if (mode == S3QueueMode::UNORDERED) + { processing = it->setProcessing(mode, exclude); - } else { + } + else + { String max_processed_file = getMaxProcessedFile(); processing = it->setProcessing(mode, exclude, max_processed_file); } @@ -623,7 +636,8 @@ StorageS3Queue::createFileIterator(ContextPtr local_context, ASTPtr query, KeysW return it; } -std::unordered_set StorageS3Queue::getFailedFiles() { +std::unordered_set StorageS3Queue::getFailedFiles() +{ auto zookeeper = getZooKeeper(); String failed = zookeeper->get(zookeeper_path + "/failed"); @@ -632,7 +646,8 @@ std::unordered_set StorageS3Queue::getFailedFiles() { return failed_files; } -std::unordered_set StorageS3Queue::getProcessedFiles() { +std::unordered_set StorageS3Queue::getProcessedFiles() +{ auto zookeeper = getZooKeeper(); String processed = zookeeper->get(zookeeper_path + "/processed"); @@ -641,14 +656,16 @@ std::unordered_set StorageS3Queue::getProcessedFiles() { return processed_files; } -String StorageS3Queue::getMaxProcessedFile() { +String StorageS3Queue::getMaxProcessedFile() +{ auto zookeeper = getZooKeeper(); String processed = zookeeper->get(zookeeper_path + "/processed"); return processed; } -std::unordered_set StorageS3Queue::getProcessingFiles() { +std::unordered_set StorageS3Queue::getProcessingFiles() +{ auto zookeeper = getZooKeeper(); Strings consumer_table_uuids; @@ -671,7 +688,8 @@ std::unordered_set StorageS3Queue::getExcludedFiles() LOG_DEBUG(log, "failed_files {}", failed_files.size()); exclude_files.merge(failed_files); - if (mode != S3QueueMode::ORDERED) { + if (mode != S3QueueMode::ORDERED) + { std::unordered_set processed_files = getProcessedFiles(); LOG_DEBUG(log, "processed_files {}", processed_files.size()); exclude_files.merge(processed_files); diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index 257cb23f303..4ca6ba73271 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -63,19 +63,13 @@ public: size_t max_block_size, size_t num_streams) override; - SinkToStoragePtr write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr /*context*/) override - { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Write is not supported by storage {}", getName()); - } + SinkToStoragePtr write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr /*context*/) override; void truncate( const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr /*local_context*/, - TableExclusiveLockHolder &) override - { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Truncate is not supported by storage {}", getName()); - } + TableExclusiveLockHolder &) override; NamesAndTypesList getVirtuals() const override; From 7073d377eb5d5e564f8b3a24fdc2814fd56766d9 Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Tue, 2 May 2023 18:18:49 +0300 Subject: [PATCH 08/34] fix typo --- src/Storages/S3Queue/StorageS3Queue.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index ac9300ac4f4..6d2bec7a0d3 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -142,7 +142,7 @@ StorageS3Queue::StorageS3Queue( } String setting_zookeeper_path = s3queue_settings->keeper_path; - LOG_INFO(log, "Setttings zookeeper_path={}", setting_zookeeper_path); + LOG_INFO(log, "Settings zookeeper_path={}", setting_zookeeper_path); if (setting_zookeeper_path == "") { auto table_id = getStorageID(); From 7498ed23b1d88a8cf539690813877a689158dd4e Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Thu, 4 May 2023 10:04:04 +0300 Subject: [PATCH 09/34] check meta, reformat code, add tests --- src/Storages/S3Queue/S3QueueHolder.cpp | 186 +++++++++ src/Storages/S3Queue/S3QueueHolder.h | 49 +++ src/Storages/S3Queue/S3QueueSettings.h | 11 +- src/Storages/S3Queue/S3QueueSource.cpp | 139 ++----- src/Storages/S3Queue/S3QueueSource.h | 38 +- src/Storages/S3Queue/S3QueueTableMetadata.cpp | 115 ++++++ src/Storages/S3Queue/S3QueueTableMetadata.h | 45 +++ src/Storages/S3Queue/StorageS3Queue.cpp | 365 ++++++++---------- src/Storages/S3Queue/StorageS3Queue.h | 12 +- .../integration/test_storage_s3_queue/test.py | 113 +++++- 10 files changed, 725 insertions(+), 348 deletions(-) create mode 100644 src/Storages/S3Queue/S3QueueHolder.cpp create mode 100644 src/Storages/S3Queue/S3QueueHolder.h create mode 100644 src/Storages/S3Queue/S3QueueTableMetadata.cpp create mode 100644 src/Storages/S3Queue/S3QueueTableMetadata.h diff --git a/src/Storages/S3Queue/S3QueueHolder.cpp b/src/Storages/S3Queue/S3QueueHolder.cpp new file mode 100644 index 00000000000..119cbe8e6e4 --- /dev/null +++ b/src/Storages/S3Queue/S3QueueHolder.cpp @@ -0,0 +1,186 @@ +#include "config.h" + +#if USE_AWS_S3 +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; + extern const int NO_ZOOKEEPER; +} + +S3QueueHolder::S3QueueHolder(const String & zookeeper_path_, const S3QueueMode & mode_, const UUID & table_uuid_, ContextPtr context_) + : WithContext(context_) + , zookeeper_path(zookeeper_path_) + , zookeeper_failed_path(zookeeper_path_ + "/failed") + , zookeeper_processing_path(zookeeper_path_ + "/processing") + , zookeeper_processed_path(zookeeper_path_ + "/processed") + , mode(mode_) + , table_uuid(table_uuid_) + , log(&Poco::Logger::get("S3QueueHolder")) +{ + current_zookeeper = getContext()->getZooKeeper(); +} + + +zkutil::ZooKeeperPtr S3QueueHolder::tryGetZooKeeper() const +{ + std::lock_guard lock(current_zookeeper_mutex); + return current_zookeeper; +} + +zkutil::ZooKeeperPtr S3QueueHolder::getZooKeeper() const +{ + auto res = tryGetZooKeeper(); + if (!res) + throw Exception(ErrorCodes::NO_ZOOKEEPER, "Cannot get ZooKeeper"); + return res; +} + + +void S3QueueHolder::setFileProcessed(const String & file_path) +{ + auto zookeeper = getZooKeeper(); + std::lock_guard lock(mutex); + + switch (mode) + { + case S3QueueMode::UNORDERED: { + String processed_files = zookeeper->get(zookeeper_processed_path); + S3FilesCollection processed = parseCollection(processed_files); + processed.insert(file_path); + Strings set_processed; + set_processed.insert(set_processed.end(), processed.begin(), processed.end()); + zookeeper->set(zookeeper_processed_path, toString(set_processed)); + break; + } + case S3QueueMode::ORDERED: { + zookeeper->set(zookeeper_processed_path, file_path); + } + } +} + + +void S3QueueHolder::setFileFailed(const String & file_path) +{ + auto zookeeper = getZooKeeper(); + + std::lock_guard lock(mutex); + String failed_files = zookeeper->get(zookeeper_failed_path); + S3FilesCollection failed = parseCollection(failed_files); + + failed.insert(file_path); + Strings set_failed; + set_failed.insert(set_failed.end(), failed.begin(), failed.end()); + + zookeeper->set(zookeeper_failed_path, toString(set_failed)); +} + +S3QueueHolder::S3FilesCollection S3QueueHolder::parseCollection(String & files) +{ + ReadBuffer rb(const_cast(reinterpret_cast(files.data())), files.length(), 0); + Strings deserialized; + try + { + readQuoted(deserialized, rb); + } + catch (...) + { + deserialized = {}; + } + + std::unordered_set processed(deserialized.begin(), deserialized.end()); + + return processed; +} + +S3QueueHolder::S3FilesCollection S3QueueHolder::getExcludedFiles() +{ + std::unordered_set exclude_files; + + std::unordered_set failed_files = getFailedFiles(); + LOG_DEBUG(log, "failed_files {}", failed_files.size()); + exclude_files.merge(failed_files); + + if (mode != S3QueueMode::ORDERED) + { + std::unordered_set processed_files = getProcessedFiles(); + LOG_DEBUG(log, "processed_files {}", processed_files.size()); + exclude_files.merge(processed_files); + } + + std::unordered_set processing_files = getProcessingFiles(); + LOG_DEBUG(log, "processing {}", processing_files.size()); + exclude_files.merge(processing_files); + + return exclude_files; +} + +String S3QueueHolder::getMaxProcessedFile() +{ + if (mode != S3QueueMode::ORDERED) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "getMaxProcessedFile not implemented for unordered mode"); + + auto zookeeper = getZooKeeper(); + std::lock_guard lock(mutex); + String processed = zookeeper->get(zookeeper_path + "/processed"); + return processed; +} + +void S3QueueHolder::setFilesProcessing(Strings & file_paths) +{ + auto zookeeper = getZooKeeper(); + + std::lock_guard lock(mutex); + String node_data; + if (zookeeper->tryGet(fs::path(zookeeper_processing_path), node_data)) + { + S3FilesCollection processing_files = parseCollection(node_data); + for (auto x : processing_files) + { + if (!std::count(file_paths.begin(), file_paths.end(), x)) + { + file_paths.push_back(x); + } + } + } + zookeeper->set(fs::path(zookeeper_processing_path), toString(file_paths)); +} + +S3QueueHolder::S3FilesCollection S3QueueHolder::getFailedFiles() +{ + auto zookeeper = getZooKeeper(); + std::lock_guard lock(mutex); + + String failed = zookeeper->get(zookeeper_failed_path); + return parseCollection(failed); +} + +S3QueueHolder::S3FilesCollection S3QueueHolder::getProcessedFiles() +{ + auto zookeeper = getZooKeeper(); + + std::lock_guard lock(mutex); + String processed = zookeeper->get(zookeeper_processed_path); + return parseCollection(processed); +} + +S3QueueHolder::S3FilesCollection S3QueueHolder::getProcessingFiles() +{ + auto zookeeper = getZooKeeper(); + + std::lock_guard lock(mutex); + String processing = zookeeper->get(fs::path(zookeeper_processing_path)); + return parseCollection(processing); +} + +} + +#endif diff --git a/src/Storages/S3Queue/S3QueueHolder.h b/src/Storages/S3Queue/S3QueueHolder.h new file mode 100644 index 00000000000..686b5fc1ddf --- /dev/null +++ b/src/Storages/S3Queue/S3QueueHolder.h @@ -0,0 +1,49 @@ +#pragma once + +#if USE_AWS_S3 + +#include +#include +#include +#include + +namespace DB +{ +class S3QueueHolder : public WithContext +{ +public: + using S3FilesCollection = std::unordered_set; + S3QueueHolder(const String & zookeeper_path_, const S3QueueMode & mode_, const UUID & table_uuid_, ContextPtr context_); + + void setFileProcessed(const String & file_path); + void setFileFailed(const String & file_path); + void setFilesProcessing(Strings & file_paths); + static S3FilesCollection parseCollection(String & files); + + S3FilesCollection getExcludedFiles(); + String getMaxProcessedFile(); + S3FilesCollection getFailedFiles(); + S3FilesCollection getProcessedFiles(); + S3FilesCollection getProcessingFiles(); + +private: + zkutil::ZooKeeperPtr current_zookeeper; + mutable std::mutex current_zookeeper_mutex; + mutable std::mutex mutex; + const String zookeeper_path; + const String zookeeper_failed_path; + const String zookeeper_processing_path; + const String zookeeper_processed_path; + const S3QueueMode mode; + const UUID table_uuid; + Poco::Logger * log; + + zkutil::ZooKeeperPtr tryGetZooKeeper() const; + zkutil::ZooKeeperPtr getZooKeeper() const; +}; + + +} + + +#endif diff --git a/src/Storages/S3Queue/S3QueueSettings.h b/src/Storages/S3Queue/S3QueueSettings.h index d556a768a15..a2798a42263 100644 --- a/src/Storages/S3Queue/S3QueueSettings.h +++ b/src/Storages/S3Queue/S3QueueSettings.h @@ -15,14 +15,15 @@ class ASTStorage; mode, \ S3QueueMode::ORDERED, \ "With unordered mode, the set of all already processed files is tracked with persistent nodes in ZooKepeer", \ - 0) \ - M(S3QueueAction, after_processing, S3QueueAction::KEEP , "Delete, keep or move file in S3 after processing", 0) \ - M(String, keeper_path, "", "Zookeeper node path", 0) \ - M(String, processed_action, "keep", "Keep, delete or move file after successful loading", 0) \ + 0) \ + M(S3QueueAction, after_processing, S3QueueAction::KEEP, "Delete, keep or move file in S3 after processing", 0) \ + M(String, keeper_path, "", "Zookeeper node path", 0) \ M(UInt64, s3queue_max_retries, 0, "Retry loading up to specified number of times", 0) \ M(UInt64, s3queue_polling_min_timeout_ms, 1000, "Minimal timeout before next polling", 0) \ M(UInt64, s3queue_polling_max_timeout_ms, 10000, "Maximum timeout before next polling", 0) \ - M(UInt64, s3queue_polling_backoff_ms, 0, "Retry loading up to specified number of times", 0) + M(UInt64, s3queue_polling_backoff_ms, 0, "Retry loading up to specified number of times", 0) \ + M(UInt64, s3queue_max_set_size, 1000, "Max set size for tracking processed files in unordered mode in ZooKeeper", 0) \ + M(UInt64, s3queue_max_set_age_s, 0, "Maximum number of seconds to store processed files in ZooKeeper node (forever default)", 0) #define LIST_OF_S3QUEUE_SETTINGS(M, ALIAS) \ S3QUEUE_RELATED_SETTINGS(M, ALIAS) \ diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index e77680c0d36..c2ee5154d75 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -8,65 +8,51 @@ #if USE_AWS_S3 -# include +#include -# include +#include -# include -# include +#include +#include -# include -# include +#include -# include -# include +#include +#include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include +#include +#include +#include +#include +#include +#include +#include +#include -# include -# include -# include +#include +#include +#include -# include -# include +#include -# include -# include +#include -# include -# include -# include -# include +#include +#include +#include -# include +#include -# include +#include -# include +#include +#include +#include -# include -# include -# include -# include -# include - -# include -# include -# include -# include - -# include +#include +#include +#include +#include namespace fs = std::filesystem; @@ -220,7 +206,7 @@ StorageS3QueueSource::StorageS3QueueSource( const String & bucket_, const String & version_id_, std::shared_ptr file_iterator_, - const S3QueueMode & mode_, + std::shared_ptr queue_holder_, const S3QueueAction & action_, zkutil::ZooKeeperPtr current_zookeeper, const String & zookeeper_path_, @@ -238,9 +224,9 @@ StorageS3QueueSource::StorageS3QueueSource( , client(client_) , sample_block(sample_block_) , format_settings(format_settings_) + , queue_holder(queue_holder_) , requested_virtual_columns(requested_virtual_columns_) , file_iterator(file_iterator_) - , mode(mode_) , action(action_) , download_thread_num(download_thread_num_) , zookeeper(current_zookeeper) @@ -412,7 +398,7 @@ Chunk StorageS3QueueSource::generate() } } LOG_WARNING(log, "Set processed: {}", file_path); - setFileProcessed(file_path); + queue_holder->setFileProcessed(file_path); applyActionAfterProcessing(file_path); return chunk; } @@ -422,7 +408,7 @@ Chunk StorageS3QueueSource::generate() LOG_ERROR(log, "Exception: {} ", e.displayText()); const auto & failed_file_path = reader.getPath(); LOG_WARNING(log, "Set failed: {}", failed_file_path); - setFileFailed(failed_file_path); + queue_holder->setFileFailed(failed_file_path); } @@ -441,41 +427,6 @@ Chunk StorageS3QueueSource::generate() return {}; } -void StorageS3QueueSource::setFileProcessed(const String & file_path) -{ - std::lock_guard lock(mutex); - if (mode == S3QueueMode::UNORDERED) - { - String processed_files = zookeeper->get(zookeeper_path + "/processed"); - std::unordered_set processed = parseCollection(processed_files); - - processed.insert(file_path); - Strings set_processed; - set_processed.insert(set_processed.end(), processed.begin(), processed.end()); - - zookeeper->set(zookeeper_path + "/processed", toString(set_processed)); - } - else - { - zookeeper->set(zookeeper_path + "/processed", file_path); - } -} - - -void StorageS3QueueSource::setFileFailed(const String & file_path) -{ - std::lock_guard lock(mutex); - String processed_files = zookeeper->get(zookeeper_path + "/failed"); - std::unordered_set processed = parseCollection(processed_files); - - processed.insert(file_path); - Strings set_failed; - set_failed.insert(set_failed.end(), processed.begin(), processed.end()); - - zookeeper->set(zookeeper_path + "/failed", toString(set_failed)); -} - - void StorageS3QueueSource::applyActionAfterProcessing(const String & file_path) { LOG_WARNING(log, "Delete {} Bucke {}", file_path, bucket); @@ -494,24 +445,6 @@ void StorageS3QueueSource::applyActionAfterProcessing(const String & file_path) } } -std::unordered_set StorageS3QueueSource::parseCollection(String & files) -{ - ReadBuffer rb(const_cast(reinterpret_cast(files.data())), files.length(), 0); - Strings deserialized; - try - { - readQuoted(deserialized, rb); - } - catch (...) - { - deserialized = {}; - } - - std::unordered_set processed(deserialized.begin(), deserialized.end()); - - return processed; -} - } diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index b5160588bab..0c36499d516 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -1,28 +1,28 @@ #pragma once - #include "config.h" #if USE_AWS_S3 -# include +#include -# include +#include -# include -# include -# include +#include +#include +#include +#include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include namespace DB @@ -83,7 +83,7 @@ public: const String & bucket, const String & version_id, std::shared_ptr file_iterator_, - const S3QueueMode & mode_, + std::shared_ptr queue_holder_, const S3QueueAction & action_, zkutil::ZooKeeperPtr current_zookeeper, const String & zookeeper_path_, @@ -111,12 +111,12 @@ private: Block sample_block; std::optional format_settings; + std::shared_ptr queue_holder; using ReaderHolder = StorageS3Source::ReaderHolder; ReaderHolder reader; std::vector requested_virtual_columns; std::shared_ptr file_iterator; - const S3QueueMode mode; const S3QueueAction action; size_t download_thread_num = 1; diff --git a/src/Storages/S3Queue/S3QueueTableMetadata.cpp b/src/Storages/S3Queue/S3QueueTableMetadata.cpp new file mode 100644 index 00000000000..40d29e26b68 --- /dev/null +++ b/src/Storages/S3Queue/S3QueueTableMetadata.cpp @@ -0,0 +1,115 @@ +#include + +#if USE_AWS_S3 + +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int METADATA_MISMATCH; +} + +S3QueueTableMetadata::S3QueueTableMetadata(const StorageS3::Configuration & configuration, const S3QueueSettings & engine_settings) +{ + format_name = configuration.format; + after_processing = engine_settings.after_processing.toString(); + mode = engine_settings.mode.toString(); + s3queue_max_set_size = engine_settings.s3queue_max_set_size; + s3queue_max_set_age_s = engine_settings.s3queue_max_set_age_s; +} + +void S3QueueTableMetadata::write(WriteBuffer & out) const +{ + out << "metadata format version: 1\n" + << "after processing: " << after_processing << "\n" + << "mode: " << mode << "\n" + << "s3queue_max_set_size: " << s3queue_max_set_size << "\n" + << "s3queue_max_set_age_s: " << s3queue_max_set_age_s << "\n" + << "format name: " << format_name << "\n"; +} + +String S3QueueTableMetadata::toString() const +{ + WriteBufferFromOwnString out; + write(out); + return out.str(); +} + +void S3QueueTableMetadata::read(ReadBuffer & in) +{ + in >> "metadata format version: 1\n"; + in >> "after processing: " >> after_processing >> "\n"; + in >> "mode: " >> mode >> "\n"; + in >> "s3queue_max_set_size: " >> s3queue_max_set_size >> "\n"; + in >> "s3queue_max_set_age_s: " >> s3queue_max_set_age_s >> "\n"; + in >> "format name: " >> format_name >> "\n"; +} + +S3QueueTableMetadata S3QueueTableMetadata::parse(const String & s) +{ + S3QueueTableMetadata metadata; + ReadBufferFromString buf(s); + metadata.read(buf); + return metadata; +} + + +void S3QueueTableMetadata::checkImmutableFieldsEquals(const S3QueueTableMetadata & from_zk) const +{ + if (after_processing != from_zk.after_processing) + throw Exception( + ErrorCodes::METADATA_MISMATCH, + "Existing table metadata in ZooKeeper differs " + "in action after processing. Stored in ZooKeeper: {}, local: {}", + DB::toString(from_zk.after_processing), + DB::toString(after_processing)); + + if (mode != from_zk.mode) + throw Exception( + ErrorCodes::METADATA_MISMATCH, + "Existing table metadata in ZooKeeper differs in engine mode. " + "Stored in ZooKeeper: {}, local: {}", + DB::toString(from_zk.after_processing), + DB::toString(after_processing)); + + if (s3queue_max_set_size != from_zk.s3queue_max_set_size) + throw Exception( + ErrorCodes::METADATA_MISMATCH, + "Existing table metadata in ZooKeeper differs in max set size. " + "Stored in ZooKeeper: {}, local: {}", + from_zk.s3queue_max_set_size, + s3queue_max_set_size); + + if (s3queue_max_set_age_s != from_zk.s3queue_max_set_age_s) + throw Exception( + ErrorCodes::METADATA_MISMATCH, + "Existing table metadata in ZooKeeper differs in max set age. " + "Stored in ZooKeeper: {}, local: {}", + from_zk.s3queue_max_set_age_s, + s3queue_max_set_age_s); + + if (format_name != from_zk.format_name) + throw Exception( + ErrorCodes::METADATA_MISMATCH, + "Existing table metadata in ZooKeeper differs in format name. " + "Stored in ZooKeeper: {}, local: {}", + from_zk.format_name, + format_name); +} + +void S3QueueTableMetadata::checkEquals(const S3QueueTableMetadata & from_zk) const +{ + checkImmutableFieldsEquals(from_zk); +} + +} + +#endif diff --git a/src/Storages/S3Queue/S3QueueTableMetadata.h b/src/Storages/S3Queue/S3QueueTableMetadata.h new file mode 100644 index 00000000000..85df226a02f --- /dev/null +++ b/src/Storages/S3Queue/S3QueueTableMetadata.h @@ -0,0 +1,45 @@ +#pragma once + +#if USE_AWS_S3 + +#include +#include +#include + +namespace DB +{ + +class MergeTreeData; +class WriteBuffer; +class ReadBuffer; + +/** The basic parameters of S3Queue table engine for saving in ZooKeeper. + * Lets you verify that they match local ones. + */ +struct S3QueueTableMetadata +{ + String format_name; + String after_processing; + String mode; + UInt64 s3queue_max_set_size; + UInt64 s3queue_max_set_age_s; + + S3QueueTableMetadata() = default; + S3QueueTableMetadata(const StorageS3::Configuration & configuration, const S3QueueSettings & engine_settings); + + void read(ReadBuffer & in); + static S3QueueTableMetadata parse(const String & s); + + void write(WriteBuffer & out) const; + String toString() const; + + void checkEquals(const S3QueueTableMetadata & from_zk) const; + +private: + void checkImmutableFieldsEquals(const S3QueueTableMetadata & from_zk) const; +}; + + +} + +#endif diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 6d2bec7a0d3..7c0348347d5 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -1,3 +1,8 @@ +#include "config.h" + + +#if USE_AWS_S3 + #include #include #include @@ -6,74 +11,70 @@ #include #include #include +#include +#include #include "IO/IOThreadPool.h" #include "IO/ParallelReadBuffer.h" #include "Parsers/ASTCreateQuery.h" -#include "config.h" + +#include + +#include +#include + +#include +#include + +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include -#if USE_AWS_S3 +#include +#include +#include -# include -# include +#include +#include -# include +#include +#include -# include -# include +#include +#include +#include +#include -# include -# include +#include -# include -# include +#include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include -# include +#include +#include +#include +#include -# include -# include -# include - -# include -# include - -# include -# include - -# include -# include -# include -# include - -# include - -# include - -# include - -# include -# include -# include - -# include -# include -# include -# include +#include +#include +#include +#include namespace fs = std::filesystem; @@ -103,6 +104,7 @@ namespace ErrorCodes extern const int QUERY_NOT_ALLOWED; extern const int NO_ZOOKEEPER; extern const int REPLICA_ALREADY_EXISTS; + extern const int INCOMPATIBLE_COLUMNS; } @@ -183,6 +185,18 @@ StorageS3Queue::StorageS3Queue( storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); + setZooKeeper(); + auto metadata_snapshot = getInMemoryMetadataPtr(); + const bool is_first_replica = createTableIfNotExists(metadata_snapshot); + + if (!is_first_replica) + { + checkTableStructure(zookeeper_path, metadata_snapshot); + } + + auto table_uuid = getStorageID().uuid; + queue_holder = std::make_unique(zookeeper_path, mode, table_uuid, getContext()); + auto default_virtuals = NamesAndTypesList{ {"_path", std::make_shared(std::make_shared())}, {"_file", std::make_shared(std::make_shared())}}; @@ -192,10 +206,6 @@ StorageS3Queue::StorageS3Queue( for (const auto & column : virtual_columns) virtual_block.insert({column.type->createColumn(), column.type, column.name}); - setZooKeeper(); - auto metadata_snapshot = getInMemoryMetadataPtr(); - createTableIfNotExists(metadata_snapshot); - auto poll_thread = context_->getSchedulePool().createTask("S3QueueStreamingTask", [this] { threadFunc(); }); task = std::make_shared(std::move(poll_thread)); } @@ -218,11 +228,11 @@ Pipe StorageS3Queue::read( ContextPtr local_context, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, - size_t num_streams) + size_t /* num_streams */) { if (!local_context->getSettingsRef().stream_like_engine_allow_direct_select) - throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, - "Direct select is not allowed. To enable use setting `stream_like_engine_allow_direct_select`"); + throw Exception( + ErrorCodes::QUERY_NOT_ALLOWED, "Direct select is not allowed. To enable use setting `stream_like_engine_allow_direct_select`"); if (mv_attached) throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "Cannot read from StorageRabbitMQ with attached materialized views"); @@ -270,34 +280,27 @@ Pipe StorageS3Queue::read( const size_t max_download_threads = local_context->getSettingsRef().max_download_threads; auto zookeeper = getZooKeeper(); - for (size_t i = 0; i < num_streams; ++i) - { - pipes.emplace_back(std::make_shared( - requested_virtual_columns, - format_name, - getName(), - block_for_format, - local_context, - format_settings, - columns_description, - max_block_size, - query_s3_configuration.request_settings, - compression_method, - query_s3_configuration.client, - query_s3_configuration.url.bucket, - query_s3_configuration.url.version_id, - iterator_wrapper, - mode, - after_processing, - zookeeper, - zookeeper_path, - max_download_threads)); - } - auto pipe = Pipe::unitePipes(std::move(pipes)); - narrowPipe(pipe, num_streams); - - return pipe; + return Pipe(std::make_shared( + requested_virtual_columns, + format_name, + getName(), + block_for_format, + local_context, + format_settings, + columns_description, + max_block_size, + query_s3_configuration.request_settings, + compression_method, + query_s3_configuration.client, + query_s3_configuration.url.bucket, + query_s3_configuration.url.version_id, + iterator_wrapper, + queue_holder, + after_processing, + zookeeper, + zookeeper_path, + max_download_threads)); } SinkToStoragePtr StorageS3Queue::write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr /*context*/) @@ -306,10 +309,7 @@ SinkToStoragePtr StorageS3Queue::write(const ASTPtr & /*query*/, const StorageMe } void StorageS3Queue::truncate( - const ASTPtr & /*query*/, - const StorageMetadataPtr & /*metadata_snapshot*/, - ContextPtr /*local_context*/, - TableExclusiveLockHolder &) + const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr /*local_context*/, TableExclusiveLockHolder &) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Truncate is not supported by storage {}", getName()); } @@ -494,37 +494,32 @@ void StorageS3Queue::streamToViews() Pipes pipes; auto zookeeper = getZooKeeper(); - size_t num_streams = 1; - for (size_t i = 0; i < num_streams; ++i) - { - pipes.emplace_back(std::make_shared( - requested_virtual_columns, - format_name, - getName(), - block_for_format, - s3queue_context, - format_settings, - columns_description, - block_size, - query_s3_configuration.request_settings, - compression_method, - query_s3_configuration.client, - query_s3_configuration.url.bucket, - query_s3_configuration.url.version_id, - iterator_wrapper, - mode, - after_processing, - zookeeper, - zookeeper_path, - max_download_threads)); - } + auto pipe = Pipe(std::make_shared( + requested_virtual_columns, + format_name, + getName(), + block_for_format, + s3queue_context, + format_settings, + columns_description, + block_size, + query_s3_configuration.request_settings, + compression_method, + query_s3_configuration.client, + query_s3_configuration.url.bucket, + query_s3_configuration.url.version_id, + iterator_wrapper, + queue_holder, + after_processing, + zookeeper, + zookeeper_path, + max_download_threads)); - auto pipe = Pipe::unitePipes(std::move(pipes)); std::atomic_size_t rows = 0; { block_io.pipeline.complete(std::move(pipe)); - block_io.pipeline.setNumThreads(num_streams); + block_io.pipeline.setNumThreads(1); block_io.pipeline.setProgressCallback([&](const Progress & progress) { rows += progress.read_rows.load(); }); CompletedPipelineExecutor executor(block_io.pipeline); @@ -569,23 +564,29 @@ bool StorageS3Queue::createTableIfNotExists(const StorageMetadataPtr & metadata_ Coordination::Requests ops; auto table_uuid = getStorageID().uuid; - if (zookeeper->exists(zookeeper_path + "")) + if (zookeeper->exists(zookeeper_path + "/metadata")) { - ops.emplace_back(zkutil::makeCreateRequest( - fs::path(zookeeper_path) / "processing" / toString(table_uuid), "{}", zkutil::CreateMode::Ephemeral)); + LOG_DEBUG(log, "This table {} is already created, will use existing metadata for checking engine settings", zookeeper_path); + return false; } else { - String deafult_processed = mode == S3QueueMode::ORDERED ? "" : "[]"; + String metadata_str = S3QueueTableMetadata(s3_configuration, *s3queue_settings).toString(); + String default_processed = mode == S3QueueMode::ORDERED ? "" : "[]"; + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path, "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processed", deafult_processed, zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processed", default_processed, zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/failed", "[]", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processing", "", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest( fs::path(zookeeper_path) / "processing" / toString(table_uuid), "[]", zkutil::CreateMode::Ephemeral)); + ops.emplace_back(zkutil::makeCreateRequest( zookeeper_path + "/columns", metadata_snapshot->getColumns().toString(), zkutil::CreateMode::Persistent)); + + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/metadata", metadata_str, zkutil::CreateMode::Persistent)); } + Coordination::Responses responses; auto code = zookeeper->tryMulti(ops, responses); if (code == Coordination::Error::ZNODEEXISTS) @@ -608,101 +609,61 @@ bool StorageS3Queue::createTableIfNotExists(const StorageMetadataPtr & metadata_ } +/** Verify that list of columns and table settings match those specified in ZK (/metadata). + * If not, throw an exception. + */ +void StorageS3Queue::checkTableStructure(const String & zookeeper_prefix, const StorageMetadataPtr & metadata_snapshot) +{ + auto zookeeper = getZooKeeper(); + + S3QueueTableMetadata old_metadata(s3_configuration, *s3queue_settings); + + Coordination::Stat metadata_stat; + String metadata_str = zookeeper->get(fs::path(zookeeper_prefix) / "metadata", &metadata_stat); + auto metadata_from_zk = S3QueueTableMetadata::parse(metadata_str); + old_metadata.checkEquals(metadata_from_zk); + + Coordination::Stat columns_stat; + auto columns_from_zk = ColumnsDescription::parse(zookeeper->get(fs::path(zookeeper_prefix) / "columns", &columns_stat)); + + const ColumnsDescription & old_columns = metadata_snapshot->getColumns(); + if (columns_from_zk != old_columns) + { + throw Exception( + ErrorCodes::INCOMPATIBLE_COLUMNS, + "Table columns structure in ZooKeeper is different from local table structure. Local columns:\n" + "{}\nZookeeper columns:\n{}", + old_columns.toString(), + columns_from_zk.toString()); + } +} + + std::shared_ptr StorageS3Queue::createFileIterator(ContextPtr local_context, ASTPtr query, KeysWithInfo * read_keys) { /// Iterate through disclosed globs and make a source for each file + std::lock_guard lock{sync_mutex}; + auto it = std::make_shared( *s3_configuration.client, s3_configuration.url, query, virtual_block, local_context, read_keys, s3_configuration.request_settings); - std::lock_guard lock{sync_mutex}; - std::unordered_set exclude = getExcludedFiles(); + S3QueueHolder::S3FilesCollection exclude = queue_holder->getExcludedFiles(); - Strings processing; + Strings processing_files; if (mode == S3QueueMode::UNORDERED) { - processing = it->setProcessing(mode, exclude); + processing_files = it->setProcessing(mode, exclude); } else { - String max_processed_file = getMaxProcessedFile(); - processing = it->setProcessing(mode, exclude, max_processed_file); + String max_processed_file = queue_holder->getMaxProcessedFile(); + processing_files = it->setProcessing(mode, exclude, max_processed_file); } - - auto zookeeper = getZooKeeper(); - auto table_uuid = getStorageID().uuid; - zookeeper->set(fs::path(zookeeper_path) / "processing" / toString(table_uuid), toString(processing)); - + queue_holder->setFilesProcessing(processing_files); return it; } -std::unordered_set StorageS3Queue::getFailedFiles() -{ - auto zookeeper = getZooKeeper(); - - String failed = zookeeper->get(zookeeper_path + "/failed"); - std::unordered_set failed_files = StorageS3QueueSource::parseCollection(failed); - - return failed_files; -} - -std::unordered_set StorageS3Queue::getProcessedFiles() -{ - auto zookeeper = getZooKeeper(); - - String processed = zookeeper->get(zookeeper_path + "/processed"); - std::unordered_set processed_files = StorageS3QueueSource::parseCollection(processed); - - return processed_files; -} - -String StorageS3Queue::getMaxProcessedFile() -{ - auto zookeeper = getZooKeeper(); - - String processed = zookeeper->get(zookeeper_path + "/processed"); - return processed; -} - -std::unordered_set StorageS3Queue::getProcessingFiles() -{ - auto zookeeper = getZooKeeper(); - - Strings consumer_table_uuids; - zookeeper->tryGetChildren(zookeeper_path + "/processing", consumer_table_uuids); - std::unordered_set processing_files; - for (const auto & uuid : consumer_table_uuids) - { - String processing = zookeeper->get(fs::path(zookeeper_path) / "processing" / toString(uuid)); - std::unordered_set cur_processing_files = StorageS3QueueSource::parseCollection(processing); - processing_files.merge(cur_processing_files); - } - return processing_files; -} - -std::unordered_set StorageS3Queue::getExcludedFiles() -{ - std::unordered_set exclude_files; - - std::unordered_set failed_files = getFailedFiles(); - LOG_DEBUG(log, "failed_files {}", failed_files.size()); - exclude_files.merge(failed_files); - - if (mode != S3QueueMode::ORDERED) - { - std::unordered_set processed_files = getProcessedFiles(); - LOG_DEBUG(log, "processed_files {}", processed_files.size()); - exclude_files.merge(processed_files); - } - - std::unordered_set processing_files = getProcessingFiles(); - LOG_DEBUG(log, "processing {}", processing_files.size()); - exclude_files.merge(processing_files); - - return exclude_files; -} - - void registerStorageS3QueueImpl(const String & name, StorageFactory & factory) { factory.registerStorage( diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index 4ca6ba73271..ac098cdda99 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -13,6 +13,7 @@ # include # include # include +# include # include # include @@ -79,6 +80,7 @@ public: private: std::unique_ptr s3queue_settings; + std::shared_ptr queue_holder; Configuration s3_configuration; std::vector keys; NamesAndTypesList virtual_columns; @@ -127,6 +129,7 @@ private: zkutil::ZooKeeperPtr tryGetZooKeeper() const; zkutil::ZooKeeperPtr getZooKeeper() const; bool createTableIfNotExists(const StorageMetadataPtr & metadata_snapshot); + void checkTableStructure(const String & zookeeper_prefix, const StorageMetadataPtr & metadata_snapshot); // Return default or custom zookeeper name for table const String & getZooKeeperPath() const { return zookeeper_path; } @@ -135,16 +138,7 @@ private: std::shared_ptr createFileIterator(ContextPtr local_context, ASTPtr query, KeysWithInfo * read_keys = nullptr); - static std::unordered_set parseCollection(String & files); - - std::unordered_set getFailedFiles(); - std::unordered_set getProcessedFiles(); - String getMaxProcessedFile(); - std::unordered_set getProcessingFiles(); - std::unordered_set getExcludedFiles(); - void streamToViews(); - Configuration updateConfigurationAndGetCopy(ContextPtr local_context); }; diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index 8cdf7e8dda0..5d5d43347cb 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -5,6 +5,7 @@ import random import time import pytest +from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster, ClickHouseInstance from helpers.s3_tools import prepare_s3_bucket @@ -13,7 +14,13 @@ from helpers.s3_tools import prepare_s3_bucket def s3_queue_setup_teardown(started_cluster): instance = started_cluster.instances["instance"] instance.query("DROP DATABASE IF EXISTS test; CREATE DATABASE test;") - # logging.debug("kafka is available - running test") + + minio = started_cluster.minio_client + objects = list( + minio.list_objects(started_cluster.minio_restricted_bucket, recursive=True) + ) + for obj in objects: + minio.remove_object(started_cluster.minio_restricted_bucket, obj.object_name) yield # run test @@ -109,9 +116,13 @@ def test_direct_select_file(started_cluster, mode): ).encode() filename = f"test.csv" put_s3_file_content(started_cluster, bucket, filename, values_csv) - instance.query("drop table if exists test.s3_queue") - instance.query("drop table if exists test.s3_queue_2") - instance.query("drop table if exists test.s3_queue_3") + instance.query( + """ + DROP TABLE IF EXISTS test.s3_queue; + DROP TABLE IF EXISTS test.s3_queue_2; + DROP TABLE IF EXISTS test.s3_queue_3; + """ + ) instance.query( f""" @@ -201,8 +212,7 @@ def test_direct_select_multiple_files(started_cluster, mode): """ ) - for i in range(10): - print(i) + for i in range(5): rand_values = [[random.randint(0, 50) for _ in range(3)] for _ in range(10)] values_csv = ( @@ -218,7 +228,7 @@ def test_direct_select_multiple_files(started_cluster, mode): ] == rand_values total_values = generate_random_files( - 5, prefix, started_cluster, bucket, start_ind=10 + 4, prefix, started_cluster, bucket, start_ind=5 ) get_query = f"SELECT * FROM test.s3_queue" assert { @@ -248,7 +258,7 @@ def test_streaming_to_view_(started_cluster, mode): ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') SETTINGS mode = '{mode}', - keeper_path = '/clickhouse/select_multiple_{mode}'; + keeper_path = '/clickhouse/view_{mode}'; CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv TO test.s3_queue_persistent AS SELECT @@ -307,7 +317,7 @@ def test_streaming_to_many_views(started_cluster, mode): ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') SETTINGS mode = '{mode}', - keeper_path = '/clickhouse/select_multiple_{mode}'; + keeper_path = '/clickhouse/multiple_view_{mode}'; CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv TO test.s3_queue_persistent AS SELECT @@ -325,7 +335,7 @@ def test_streaming_to_many_views(started_cluster, mode): FROM test.s3_queue; """ ) - total_values = generate_random_files(10, prefix, started_cluster, bucket) + total_values = generate_random_files(5, prefix, started_cluster, bucket) expected_values = set([tuple(i) for i in total_values]) for i in range(retry_cnt): @@ -348,3 +358,86 @@ def test_streaming_to_many_views(started_cluster, mode): time.sleep(1) else: break + + +def test_multiple_tables_meta_mismatch(started_cluster): + prefix = f"test_meta" + bucket = started_cluster.minio_restricted_bucket + instance = started_cluster.instances["instance"] + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + + instance.query( + f""" + DROP TABLE IF EXISTS test.s3_queue; + + CREATE TABLE test.s3_queue ({table_format}) + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + SETTINGS + mode = 'ordered', + keeper_path = '/clickhouse/test_meta'; + """ + ) + # check mode + failed = False + try: + instance.query( + f""" + CREATE TABLE test.s3_queue_copy ({table_format}) + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + SETTINGS + mode = 'unordered', + keeper_path = '/clickhouse/test_meta'; + """ + ) + except QueryRuntimeException as e: + assert "Existing table metadata in ZooKeeper differs in engine mode" in str(e) + failed = True + assert failed is True + + # check columns + table_format_copy = table_format + ", column4 UInt32" + try: + instance.query( + f""" + CREATE TABLE test.s3_queue_copy ({table_format_copy}) + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + SETTINGS + mode = 'ordered', + keeper_path = '/clickhouse/test_meta'; + """ + ) + except QueryRuntimeException as e: + assert ( + "Table columns structure in ZooKeeper is different from local table structure" + in str(e) + ) + failed = True + + assert failed is True + + # check format + try: + instance.query( + f""" + CREATE TABLE test.s3_queue_copy ({table_format}) + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'TSV') + SETTINGS + mode = 'ordered', + keeper_path = '/clickhouse/test_meta'; + """ + ) + except QueryRuntimeException as e: + assert "Existing table metadata in ZooKeeper differs in format name" in str(e) + failed = True + assert failed is True + + # create working engine + instance.query( + f""" + CREATE TABLE test.s3_queue_copy ({table_format}) + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + SETTINGS + mode = 'ordered', + keeper_path = '/clickhouse/test_meta'; + """ + ) From c24ec8f83f732eb6752ae08f324e925f3aa92cd4 Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Thu, 4 May 2023 11:58:09 +0300 Subject: [PATCH 10/34] fix --- src/Storages/S3Queue/S3QueueSource.cpp | 1 - src/Storages/S3Queue/StorageS3Queue.cpp | 19 ------------------- .../integration/test_storage_s3_queue/test.py | 4 ++-- 3 files changed, 2 insertions(+), 22 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index c2ee5154d75..3da725cee71 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -1,7 +1,6 @@ #include #include #include -#include "IO/IOThreadPool.h" #include "IO/ParallelReadBuffer.h" #include "Parsers/ASTCreateQuery.h" #include "config.h" diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 7c0348347d5..f91ca11b491 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -4,8 +4,6 @@ #if USE_AWS_S3 #include -#include -#include #include #include #include @@ -13,17 +11,13 @@ #include #include #include -#include "IO/IOThreadPool.h" #include "IO/ParallelReadBuffer.h" -#include "Parsers/ASTCreateQuery.h" #include -#include #include #include -#include #include #include @@ -37,39 +31,26 @@ #include #include #include -#include #include -#include #include -#include #include #include #include #include -#include - -#include -#include #include -#include #include #include #include -#include #include #include -#include - -#include #include -#include #include #include diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index 5d5d43347cb..b8d0ec3d5da 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -408,8 +408,8 @@ def test_multiple_tables_meta_mismatch(started_cluster): ) except QueryRuntimeException as e: assert ( - "Table columns structure in ZooKeeper is different from local table structure" - in str(e) + "Table columns structure in ZooKeeper is different from local table structure" + in str(e) ) failed = True From a8d56b2290cf0ada1a3598f295ed45575778afe4 Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Mon, 8 May 2023 10:26:52 +0300 Subject: [PATCH 11/34] fix ordered mode + more tests --- src/Storages/S3Queue/S3QueueHolder.cpp | 206 +++++++++++--- src/Storages/S3Queue/S3QueueHolder.h | 36 ++- src/Storages/S3Queue/S3QueueSettings.h | 3 +- src/Storages/S3Queue/S3QueueSource.cpp | 85 +++--- src/Storages/S3Queue/S3QueueSource.h | 45 ++- src/Storages/S3Queue/S3QueueTableMetadata.cpp | 10 +- src/Storages/S3Queue/S3QueueTableMetadata.h | 7 +- src/Storages/S3Queue/StorageS3Queue.cpp | 108 ++++---- .../integration/test_storage_s3_queue/test.py | 262 +++++++++++++++++- 9 files changed, 593 insertions(+), 169 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueHolder.cpp b/src/Storages/S3Queue/S3QueueHolder.cpp index 119cbe8e6e4..21b2b7a45ce 100644 --- a/src/Storages/S3Queue/S3QueueHolder.cpp +++ b/src/Storages/S3Queue/S3QueueHolder.cpp @@ -1,11 +1,15 @@ #include "config.h" #if USE_AWS_S3 -#include -#include -#include -#include -#include +# include +# include +# include +# include +# include +# include +# include +# include + namespace DB { @@ -14,16 +18,99 @@ namespace ErrorCodes { extern const int NOT_IMPLEMENTED; extern const int NO_ZOOKEEPER; + extern const int TIMEOUT_EXCEEDED; } -S3QueueHolder::S3QueueHolder(const String & zookeeper_path_, const S3QueueMode & mode_, const UUID & table_uuid_, ContextPtr context_) +S3QueueHolder::ProcessedCollection::ProcessedCollection(const UInt64 & max_size_, const UInt64 & max_age_) + : max_size(max_size_), max_age(max_age_) +{ +} + +void S3QueueHolder::ProcessedCollection::read(ReadBuffer & in) +{ + files = {}; + in >> "processed_files\n"; + while (!in.eof()) + { + String file_name; + Int64 timestamp; + in >> file_name >> "\n"; + in >> timestamp >> "\n"; + auto pair = std::make_pair(file_name, timestamp); + files.push_back(pair); + } +} + +void S3QueueHolder::ProcessedCollection::write(WriteBuffer & out) const +{ + out << "processed_files\n"; + for (const auto & processed_file : files) + { + out << processed_file.first << "\n"; + out << processed_file.second << "\n"; + } +} + +void S3QueueHolder::ProcessedCollection::parse(const String & s) +{ + ReadBufferFromString buf(s); + read(buf); + // Remove old items + if (max_age > 0) + { + Int64 timestamp = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); + Int64 max_seconds_diff = max_age; + auto new_end = std::remove_if( + files.begin(), + files.end(), + [×tamp, &max_seconds_diff](std::pair processed_file) + { return (timestamp - processed_file.second) > max_seconds_diff; }); + files.erase(new_end, files.end()); + } +} + +String S3QueueHolder::ProcessedCollection::toString() const +{ + WriteBufferFromOwnString out; + write(out); + return out.str(); +} + +void S3QueueHolder::ProcessedCollection::add(const String & file_name) +{ + Int64 timestamp = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); + auto pair = std::make_pair(file_name, timestamp); + files.push_back(pair); + + // Check set size + if (files.size() > max_size) + { + files.erase(files.begin(), files.begin() + (files.size() - max_size)); + } +} + +S3QueueHolder::S3FilesCollection S3QueueHolder::ProcessedCollection::getFileNames() +{ + S3FilesCollection keys = {}; + for (auto & pair : files) + { + keys.insert(pair.first); + } + return keys; +} + + +S3QueueHolder::S3QueueHolder( + const String & zookeeper_path_, const S3QueueMode & mode_, ContextPtr context_, UInt64 & max_set_size_, UInt64 & max_set_age_s_) : WithContext(context_) + , max_set_size(max_set_size_) + , max_set_age_s(max_set_age_s_) , zookeeper_path(zookeeper_path_) , zookeeper_failed_path(zookeeper_path_ + "/failed") , zookeeper_processing_path(zookeeper_path_ + "/processing") , zookeeper_processed_path(zookeeper_path_ + "/processed") + , zookeeper_lock_path(zookeeper_path_ + "/lock") , mode(mode_) - , table_uuid(table_uuid_) , log(&Poco::Logger::get("S3QueueHolder")) { current_zookeeper = getContext()->getZooKeeper(); @@ -48,31 +135,47 @@ zkutil::ZooKeeperPtr S3QueueHolder::getZooKeeper() const void S3QueueHolder::setFileProcessed(const String & file_path) { auto zookeeper = getZooKeeper(); - std::lock_guard lock(mutex); + auto lock = AcquireLock(); - switch (mode) + if (mode == S3QueueMode::UNORDERED) { - case S3QueueMode::UNORDERED: { - String processed_files = zookeeper->get(zookeeper_processed_path); - S3FilesCollection processed = parseCollection(processed_files); - processed.insert(file_path); - Strings set_processed; - set_processed.insert(set_processed.end(), processed.begin(), processed.end()); - zookeeper->set(zookeeper_processed_path, toString(set_processed)); - break; - } - case S3QueueMode::ORDERED: { + String processed_files = zookeeper->get(zookeeper_processed_path); + auto processed = ProcessedCollection(max_set_size, max_set_age_s); + processed.parse(processed_files); + processed.add(file_path); + zookeeper->set(zookeeper_processed_path, processed.toString()); + } + else + { + String max_file = getMaxProcessedFile(); + if (max_file.compare(file_path) <= 0) + { zookeeper->set(zookeeper_processed_path, file_path); } } + + String node_data; + Strings file_paths; + if (zookeeper->tryGet(fs::path(zookeeper_processing_path), node_data)) + { + S3FilesCollection processing_files = parseCollection(node_data); + for (auto x : processing_files) + { + if (x != file_path) + { + file_paths.push_back(x); + } + } + } + zookeeper->set(fs::path(zookeeper_processing_path), toString(file_paths)); } void S3QueueHolder::setFileFailed(const String & file_path) { auto zookeeper = getZooKeeper(); + auto lock = AcquireLock(); - std::lock_guard lock(mutex); String failed_files = zookeeper->get(zookeeper_failed_path); S3FilesCollection failed = parseCollection(failed_files); @@ -104,20 +207,27 @@ S3QueueHolder::S3FilesCollection S3QueueHolder::parseCollection(String & files) S3QueueHolder::S3FilesCollection S3QueueHolder::getExcludedFiles() { std::unordered_set exclude_files; + auto zookeeper = getZooKeeper(); - std::unordered_set failed_files = getFailedFiles(); - LOG_DEBUG(log, "failed_files {}", failed_files.size()); + String failed = zookeeper->get(zookeeper_failed_path); + S3FilesCollection failed_files = parseCollection(failed); exclude_files.merge(failed_files); + String processed = zookeeper->get(zookeeper_processed_path); if (mode != S3QueueMode::ORDERED) { - std::unordered_set processed_files = getProcessedFiles(); - LOG_DEBUG(log, "processed_files {}", processed_files.size()); + auto collection = ProcessedCollection(max_set_size, max_set_age_s); + collection.parse(processed); + S3FilesCollection processed_files = collection.getFileNames(); exclude_files.merge(processed_files); } + else + { + exclude_files.insert(processed); + } - std::unordered_set processing_files = getProcessingFiles(); - LOG_DEBUG(log, "processing {}", processing_files.size()); + String processing = zookeeper->get(fs::path(zookeeper_processing_path)); + S3FilesCollection processing_files = parseCollection(processing); exclude_files.merge(processing_files); return exclude_files; @@ -129,7 +239,6 @@ String S3QueueHolder::getMaxProcessedFile() throw Exception(ErrorCodes::NOT_IMPLEMENTED, "getMaxProcessedFile not implemented for unordered mode"); auto zookeeper = getZooKeeper(); - std::lock_guard lock(mutex); String processed = zookeeper->get(zookeeper_path + "/processed"); return processed; } @@ -138,7 +247,6 @@ void S3QueueHolder::setFilesProcessing(Strings & file_paths) { auto zookeeper = getZooKeeper(); - std::lock_guard lock(mutex); String node_data; if (zookeeper->tryGet(fs::path(zookeeper_processing_path), node_data)) { @@ -157,8 +265,8 @@ void S3QueueHolder::setFilesProcessing(Strings & file_paths) S3QueueHolder::S3FilesCollection S3QueueHolder::getFailedFiles() { auto zookeeper = getZooKeeper(); - std::lock_guard lock(mutex); + auto lock = AcquireLock(); String failed = zookeeper->get(zookeeper_failed_path); return parseCollection(failed); } @@ -167,20 +275,54 @@ S3QueueHolder::S3FilesCollection S3QueueHolder::getProcessedFiles() { auto zookeeper = getZooKeeper(); - std::lock_guard lock(mutex); + auto lock = AcquireLock(); String processed = zookeeper->get(zookeeper_processed_path); - return parseCollection(processed); + auto collection = ProcessedCollection(max_set_size, max_set_age_s); + collection.parse(processed); + return collection.getFileNames(); } S3QueueHolder::S3FilesCollection S3QueueHolder::getProcessingFiles() { auto zookeeper = getZooKeeper(); - std::lock_guard lock(mutex); + auto lock = AcquireLock(); String processing = zookeeper->get(fs::path(zookeeper_processing_path)); return parseCollection(processing); } + +std::shared_ptr S3QueueHolder::AcquireLock() +{ + auto zookeeper = getZooKeeper(); + UInt32 retry_count = 200; + UInt32 sleep_ms = 100; + + UInt32 retries = 0; + while (true) + { + Coordination::Error code = zookeeper->tryCreate(zookeeper_lock_path, "", zkutil::CreateMode::Ephemeral); + if (code == Coordination::Error::ZNONODE || code == Coordination::Error::ZNODEEXISTS) + { + retries++; + if (retries >= retry_count) + { + throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Can't acquire zookeeper lock"); + } + sleepForMilliseconds(sleep_ms); + } + else if (code != Coordination::Error::ZOK) + { + throw Coordination::Exception(code, zookeeper_lock_path); + } + else + { + return zkutil::EphemeralNodeHolder::existing(zookeeper_lock_path, *zookeeper); + } + } +} + + } #endif diff --git a/src/Storages/S3Queue/S3QueueHolder.h b/src/Storages/S3Queue/S3QueueHolder.h index 686b5fc1ddf..081e58a9ea2 100644 --- a/src/Storages/S3Queue/S3QueueHolder.h +++ b/src/Storages/S3Queue/S3QueueHolder.h @@ -2,10 +2,10 @@ #if USE_AWS_S3 -#include -#include -#include -#include +# include +# include +# include +# include namespace DB { @@ -13,7 +13,10 @@ class S3QueueHolder : public WithContext { public: using S3FilesCollection = std::unordered_set; - S3QueueHolder(const String & zookeeper_path_, const S3QueueMode & mode_, const UUID & table_uuid_, ContextPtr context_); + using ProcessedFiles = std::vector>; + + S3QueueHolder( + const String & zookeeper_path_, const S3QueueMode & mode_, ContextPtr context_, UInt64 & max_set_size_, UInt64 & max_set_age_s_); void setFileProcessed(const String & file_path); void setFileFailed(const String & file_path); @@ -25,6 +28,28 @@ public: S3FilesCollection getFailedFiles(); S3FilesCollection getProcessedFiles(); S3FilesCollection getProcessingFiles(); + std::shared_ptr AcquireLock(); + + struct ProcessedCollection + { + ProcessedCollection(const UInt64 & max_size_, const UInt64 & max_age_); + + void parse(const String & s); + + String toString() const; + + void add(const String & file_name); + S3FilesCollection getFileNames(); + const UInt64 max_size; + const UInt64 max_age; + + void read(ReadBuffer & in); + void write(WriteBuffer & out) const; + ProcessedFiles files; + }; + + const UInt64 max_set_size; + const UInt64 max_set_age_s; private: zkutil::ZooKeeperPtr current_zookeeper; @@ -34,6 +59,7 @@ private: const String zookeeper_failed_path; const String zookeeper_processing_path; const String zookeeper_processed_path; + const String zookeeper_lock_path; const S3QueueMode mode; const UUID table_uuid; Poco::Logger * log; diff --git a/src/Storages/S3Queue/S3QueueSettings.h b/src/Storages/S3Queue/S3QueueSettings.h index a2798a42263..c2b8e51a1f8 100644 --- a/src/Storages/S3Queue/S3QueueSettings.h +++ b/src/Storages/S3Queue/S3QueueSettings.h @@ -23,7 +23,8 @@ class ASTStorage; M(UInt64, s3queue_polling_max_timeout_ms, 10000, "Maximum timeout before next polling", 0) \ M(UInt64, s3queue_polling_backoff_ms, 0, "Retry loading up to specified number of times", 0) \ M(UInt64, s3queue_max_set_size, 1000, "Max set size for tracking processed files in unordered mode in ZooKeeper", 0) \ - M(UInt64, s3queue_max_set_age_s, 0, "Maximum number of seconds to store processed files in ZooKeeper node (forever default)", 0) + M(UInt64, s3queue_max_set_age_s, 0, "Maximum number of seconds to store processed files in ZooKeeper node (forever default)", 0) \ + M(UInt64, s3queue_polling_size, 50, "Maximum files to fetch from S3 with SELECT", 0) #define LIST_OF_S3QUEUE_SETTINGS(M, ALIAS) \ S3QUEUE_RELATED_SETTINGS(M, ALIAS) \ diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 3da725cee71..b934f78c8df 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -7,51 +7,51 @@ #if USE_AWS_S3 -#include +# include -#include +# include -#include -#include +# include +# include -#include +# include -#include -#include +# include +# include -#include -#include -#include -#include -#include -#include -#include -#include +# include +# include +# include +# include +# include +# include +# include +# include -#include -#include -#include +# include +# include +# include -#include +# include -#include +# include -#include -#include -#include +# include +# include +# include -#include +# include -#include +# include -#include -#include -#include +# include +# include +# include -#include -#include -#include -#include +# include +# include +# include +# include namespace fs = std::filesystem; @@ -104,9 +104,11 @@ StorageS3QueueSource::QueueGlobIterator::QueueGlobIterator( ASTPtr query, const Block & virtual_header, ContextPtr context, + UInt64 & max_poll_size_, StorageS3QueueSource::KeysWithInfo * read_keys_, const S3Settings::RequestSettings & request_settings_) - : bucket(globbed_uri_.bucket) + : max_poll_size(max_poll_size_) + , bucket(globbed_uri_.bucket) , glob_iterator(std::make_unique( client_, globbed_uri_, query, virtual_header, context, read_keys_, request_settings_)) { @@ -121,7 +123,8 @@ StorageS3QueueSource::QueueGlobIterator::QueueGlobIterator( } } -Strings StorageS3QueueSource::QueueGlobIterator::setProcessing(S3QueueMode & engine_mode, std::unordered_set & exclude_keys, const String & max_file) +Strings StorageS3QueueSource::QueueGlobIterator::filterProcessingFiles( + const S3QueueMode & engine_mode, std::unordered_set & exclude_keys, const String & max_file) { for (KeyWithInfo val : keys_buf) { @@ -131,11 +134,11 @@ Strings StorageS3QueueSource::QueueGlobIterator::setProcessing(S3QueueMode & eng LOG_INFO(log, "Found in exclude keys {}", val.key); continue; } - if (engine_mode == S3QueueMode::ORDERED && full_path.compare(max_file) <= 0) + if ((engine_mode == S3QueueMode::ORDERED) && (full_path.compare(max_file) <= 0)) { continue; } - if (processing_keys.size() < max_poll_size) + if ((processing_keys.size() < max_poll_size) || (engine_mode == S3QueueMode::ORDERED)) { processing_keys.push_back(val); } @@ -150,7 +153,11 @@ Strings StorageS3QueueSource::QueueGlobIterator::setProcessing(S3QueueMode & eng std::sort( processing_keys.begin(), processing_keys.end(), - [](const KeyWithInfo & lhs, const KeyWithInfo & rhs) { return lhs.key < rhs.key; }); + [](const KeyWithInfo & lhs, const KeyWithInfo & rhs) { return lhs.key.compare(rhs.key) < 0; }); + if (processing_keys.size() > max_poll_size) + { + processing_keys.erase(processing_keys.begin() + max_poll_size, processing_keys.end()); + } } Strings keys; @@ -207,8 +214,6 @@ StorageS3QueueSource::StorageS3QueueSource( std::shared_ptr file_iterator_, std::shared_ptr queue_holder_, const S3QueueAction & action_, - zkutil::ZooKeeperPtr current_zookeeper, - const String & zookeeper_path_, const size_t download_thread_num_) : ISource(getHeader(sample_block_, requested_virtual_columns_)) , WithContext(context_) @@ -228,8 +233,6 @@ StorageS3QueueSource::StorageS3QueueSource( , file_iterator(file_iterator_) , action(action_) , download_thread_num(download_thread_num_) - , zookeeper(current_zookeeper) - , zookeeper_path(zookeeper_path_) , create_reader_pool(CurrentMetrics::StorageS3Threads, CurrentMetrics::StorageS3ThreadsActive, 1) , create_reader_scheduler(threadPoolCallbackRunner(create_reader_pool, "CreateS3QReader")) { diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index 0c36499d516..2891a5946af 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -3,26 +3,26 @@ #if USE_AWS_S3 -#include +# include -#include +# include -#include -#include -#include -#include +# include +# include +# include +# include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include namespace DB @@ -46,16 +46,18 @@ public: ASTPtr query, const Block & virtual_header, ContextPtr context, + UInt64 & max_poll_size_, KeysWithInfo * read_keys_ = nullptr, const S3Settings::RequestSettings & request_settings_ = {}); KeyWithInfo next() override; size_t getTotalSize() const override; - Strings setProcessing(S3QueueMode & engine_mode, std::unordered_set & exclude_keys, const String & max_file = ""); + Strings + filterProcessingFiles(const S3QueueMode & engine_mode, std::unordered_set & exclude_keys, const String & max_file = ""); private: - size_t max_poll_size = 10; + UInt64 max_poll_size; const String bucket; KeysWithInfo keys_buf; KeysWithInfo processing_keys; @@ -85,8 +87,6 @@ public: std::shared_ptr file_iterator_, std::shared_ptr queue_holder_, const S3QueueAction & action_, - zkutil::ZooKeeperPtr current_zookeeper, - const String & zookeeper_path_, size_t download_thread_num); ~StorageS3QueueSource() override; @@ -122,9 +122,6 @@ private: Poco::Logger * log = &Poco::Logger::get("StorageS3QueueSource"); - zkutil::ZooKeeperPtr zookeeper; - const String zookeeper_path; - ThreadPool create_reader_pool; ThreadPoolCallbackRunner create_reader_scheduler; std::future reader_future; diff --git a/src/Storages/S3Queue/S3QueueTableMetadata.cpp b/src/Storages/S3Queue/S3QueueTableMetadata.cpp index 40d29e26b68..340890f75de 100644 --- a/src/Storages/S3Queue/S3QueueTableMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueTableMetadata.cpp @@ -2,11 +2,11 @@ #if USE_AWS_S3 -#include -#include -#include -#include -#include +# include +# include +# include +# include +# include namespace DB diff --git a/src/Storages/S3Queue/S3QueueTableMetadata.h b/src/Storages/S3Queue/S3QueueTableMetadata.h index 85df226a02f..6e87528db37 100644 --- a/src/Storages/S3Queue/S3QueueTableMetadata.h +++ b/src/Storages/S3Queue/S3QueueTableMetadata.h @@ -2,14 +2,13 @@ #if USE_AWS_S3 -#include -#include -#include +# include +# include +# include namespace DB { -class MergeTreeData; class WriteBuffer; class ReadBuffer; diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index f91ca11b491..a5dd5b421c8 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -3,59 +3,59 @@ #if USE_AWS_S3 -#include -#include -#include -#include -#include -#include -#include -#include -#include "IO/ParallelReadBuffer.h" +# include +# include +# include +# include +# include +# include +# include +# include +# include "IO/ParallelReadBuffer.h" -#include +# include -#include +# include -#include +# include -#include -#include +# include +# include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include -#include -#include +# include +# include -#include +# include -#include -#include -#include +# include +# include +# include -#include +# include -#include +# include -#include +# include -#include -#include -#include -#include +# include +# include +# include +# include namespace fs = std::filesystem; @@ -175,8 +175,8 @@ StorageS3Queue::StorageS3Queue( checkTableStructure(zookeeper_path, metadata_snapshot); } - auto table_uuid = getStorageID().uuid; - queue_holder = std::make_unique(zookeeper_path, mode, table_uuid, getContext()); + queue_holder = std::make_unique( + zookeeper_path, mode, getContext(), s3queue_settings->s3queue_max_set_size.value, s3queue_settings->s3queue_max_set_age_s.value); auto default_virtuals = NamesAndTypesList{ {"_path", std::make_shared(std::make_shared())}, @@ -216,7 +216,7 @@ Pipe StorageS3Queue::read( ErrorCodes::QUERY_NOT_ALLOWED, "Direct select is not allowed. To enable use setting `stream_like_engine_allow_direct_select`"); if (mv_attached) - throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "Cannot read from StorageRabbitMQ with attached materialized views"); + throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "Cannot read from StorageS3Queue with attached materialized views"); auto query_s3_configuration = updateConfigurationAndGetCopy(local_context); @@ -279,8 +279,6 @@ Pipe StorageS3Queue::read( iterator_wrapper, queue_holder, after_processing, - zookeeper, - zookeeper_path, max_download_threads)); } @@ -492,8 +490,6 @@ void StorageS3Queue::streamToViews() iterator_wrapper, queue_holder, after_processing, - zookeeper, - zookeeper_path, max_download_threads)); @@ -553,10 +549,8 @@ bool StorageS3Queue::createTableIfNotExists(const StorageMetadataPtr & metadata_ else { String metadata_str = S3QueueTableMetadata(s3_configuration, *s3queue_settings).toString(); - String default_processed = mode == S3QueueMode::ORDERED ? "" : "[]"; - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path, "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processed", default_processed, zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processed", "processed_files\n", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/failed", "[]", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processing", "", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest( @@ -627,19 +621,27 @@ StorageS3Queue::createFileIterator(ContextPtr local_context, ASTPtr query, KeysW std::lock_guard lock{sync_mutex}; auto it = std::make_shared( - *s3_configuration.client, s3_configuration.url, query, virtual_block, local_context, read_keys, s3_configuration.request_settings); + *s3_configuration.client, + s3_configuration.url, + query, + virtual_block, + local_context, + s3queue_settings->s3queue_polling_size.value, + read_keys, + s3_configuration.request_settings); + auto zookeeper_lock = queue_holder->AcquireLock(); S3QueueHolder::S3FilesCollection exclude = queue_holder->getExcludedFiles(); Strings processing_files; if (mode == S3QueueMode::UNORDERED) { - processing_files = it->setProcessing(mode, exclude); + processing_files = it->filterProcessingFiles(mode, exclude); } else { String max_processed_file = queue_holder->getMaxProcessedFile(); - processing_files = it->setProcessing(mode, exclude, max_processed_file); + processing_files = it->filterProcessingFiles(mode, exclude, max_processed_file); } queue_holder->setFilesProcessing(processing_files); return it; diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index b8d0ec3d5da..3276c221014 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -13,7 +13,10 @@ from helpers.s3_tools import prepare_s3_bucket @pytest.fixture(autouse=True) def s3_queue_setup_teardown(started_cluster): instance = started_cluster.instances["instance"] + instance_2 = started_cluster.instances["instance2"] + instance.query("DROP DATABASE IF EXISTS test; CREATE DATABASE test;") + instance_2.query("DROP DATABASE IF EXISTS test; CREATE DATABASE test;") minio = started_cluster.minio_client objects = list( @@ -28,7 +31,6 @@ MINIO_INTERNAL_PORT = 9001 AVAILABLE_MODES = ["ordered", "unordered"] AUTH = "'minio','minio123'," - SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) @@ -41,8 +43,12 @@ def generate_random_files( count, prefix, cluster, bucket, column_num=3, row_num=10, start_ind=0 ): total_values = [] - for i in range(start_ind, start_ind + count): - print(i) + to_generate = [ + (f"{prefix}/test_{i}.csv", i) for i in range(start_ind, start_ind + count) + ] + to_generate.sort(key=lambda x: x[0]) + + for filename, i in to_generate: rand_values = [ [random.randint(0, 50) for _ in range(column_num)] for _ in range(row_num) ] @@ -50,7 +56,6 @@ def generate_random_files( values_csv = ( "\n".join((",".join(map(str, row)) for row in rand_values)) + "\n" ).encode() - filename = f"{prefix}/test_{i}.csv" put_s3_file_content(cluster, bucket, filename, values_csv) return total_values @@ -79,6 +84,13 @@ def started_cluster(): with_zookeeper=True, main_configs=["configs/defaultS3.xml", "configs/named_collections.xml"], ) + cluster.add_instance( + "instance2", + user_configs=["configs/users.xml"], + with_minio=True, + with_zookeeper=True, + main_configs=["configs/defaultS3.xml", "configs/named_collections.xml"], + ) logging.info("Starting cluster...") cluster.start() @@ -441,3 +453,245 @@ def test_multiple_tables_meta_mismatch(started_cluster): keeper_path = '/clickhouse/test_meta'; """ ) + + +def test_max_set_age(started_cluster): + files_to_generate = 10 + max_age = 1 + prefix = f"test_multiple" + bucket = started_cluster.minio_restricted_bucket + instance = started_cluster.instances["instance"] + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + + instance.query( + f""" + DROP TABLE IF EXISTS test.s3_queue; + + CREATE TABLE test.s3_queue ({table_format}) + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + SETTINGS + mode = 'unordered', + keeper_path = '/clickhouse/test_set_age', + s3queue_max_set_size = 10, + s3queue_max_set_age_s = {max_age}; + """ + ) + + total_values = generate_random_files( + files_to_generate, prefix, started_cluster, bucket, row_num=1 + ) + get_query = f"SELECT * FROM test.s3_queue" + res1 = [ + list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + ] + assert res1 == total_values + time.sleep(max_age + 1) + + get_query = f"SELECT * FROM test.s3_queue" + res1 = [ + list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + ] + assert res1 == total_values + + +@pytest.mark.parametrize("mode", AVAILABLE_MODES) +def test_multiple_tables_streaming_sync(started_cluster, mode): + files_to_generate = 300 + poll_size = 30 + prefix = f"test_multiple_{mode}" + bucket = started_cluster.minio_restricted_bucket + instance = started_cluster.instances["instance"] + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + + instance.query( + f""" + DROP TABLE IF EXISTS test.s3_queue; + DROP TABLE IF EXISTS test.s3_queue_copy; + DROP TABLE IF EXISTS test.s3_queue_copy_2; + + DROP TABLE IF EXISTS test.s3_queue_persistent; + DROP TABLE IF EXISTS test.s3_queue_persistent_copy; + DROP TABLE IF EXISTS test.s3_queue_persistent_copy_2; + + DROP TABLE IF EXISTS test.persistent_s3_queue_mv; + DROP TABLE IF EXISTS test.persistent_s3_queue_mv_copy; + DROP TABLE IF EXISTS test.persistent_s3_queue_mv_copy_2; + + CREATE TABLE test.s3_queue ({table_format}) + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + SETTINGS + mode = '{mode}', + keeper_path = '/clickhouse/test_multiple_consumers_sync_{mode}', + s3queue_polling_size = {poll_size}; + + CREATE TABLE test.s3_queue_copy ({table_format}) + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + SETTINGS + mode = '{mode}', + keeper_path = '/clickhouse/test_multiple_consumers_sync_{mode}', + s3queue_polling_size = {poll_size}; + + CREATE TABLE test.s3_queue_copy_2 ({table_format}) + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + SETTINGS + mode = '{mode}', + keeper_path = '/clickhouse/test_multiple_consumers_sync_{mode}', + s3queue_polling_size = {poll_size}; + + CREATE TABLE test.s3_queue_persistent ({table_format}) + ENGINE = MergeTree() + ORDER BY column1; + + CREATE TABLE test.s3_queue_persistent_copy ({table_format}) + ENGINE = MergeTree() + ORDER BY column1; + + CREATE TABLE test.s3_queue_persistent_copy_2 ({table_format}) + ENGINE = MergeTree() + ORDER BY column1; + + CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv TO test.s3_queue_persistent AS + SELECT + * + FROM test.s3_queue; + + CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv_copy TO test.s3_queue_persistent_copy AS + SELECT + * + FROM test.s3_queue_copy; + + CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv_copy_2 TO test.s3_queue_persistent_copy_2 AS + SELECT + * + FROM test.s3_queue_copy_2; + """ + ) + total_values = generate_random_files( + files_to_generate, prefix, started_cluster, bucket, row_num=1 + ) + time.sleep((files_to_generate // poll_size) + 10) + + get_query = f"SELECT * FROM test.s3_queue_persistent" + res1 = [ + list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + ] + get_query_copy = f"SELECT * FROM test.s3_queue_persistent_copy" + res2 = [ + list(map(int, l.split())) + for l in run_query(instance, get_query_copy).splitlines() + ] + get_query_copy_2 = f"SELECT * FROM test.s3_queue_persistent_copy_2" + res3 = [ + list(map(int, l.split())) + for l in run_query(instance, get_query_copy_2).splitlines() + ] + + # Checking that all engines have made progress + assert len(res1) > 0 + assert len(res2) > 0 + assert len(res3) > 0 + + # Checking that all files were processed only once + assert len(res1) + len(res2) + len(res3) == files_to_generate + assert {tuple(v) for v in res1 + res2 + res3} == set( + [tuple(i) for i in total_values] + ) + + +@pytest.mark.parametrize("mode", AVAILABLE_MODES) +def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): + files_to_generate = 100 + poll_size = 10 + prefix = f"test_multiple_{mode}" + bucket = started_cluster.minio_restricted_bucket + instance = started_cluster.instances["instance"] + instance_2 = started_cluster.instances["instance2"] + + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + for inst in [instance, instance_2]: + inst.query( + f""" + DROP TABLE IF EXISTS test.s3_queue; + DROP TABLE IF EXISTS test.s3_queue_persistent; + DROP TABLE IF EXISTS test.persistent_s3_queue_mv; + + CREATE TABLE test.s3_queue ({table_format}) + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + SETTINGS + mode = '{mode}', + keeper_path = '/clickhouse/test_multiple_consumers_{mode}', + s3queue_polling_size = {poll_size}; + + CREATE TABLE test.s3_queue_persistent ({table_format}) + ENGINE = MergeTree() + ORDER BY column1; + + CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv TO test.s3_queue_persistent AS + SELECT + * + FROM test.s3_queue; + """ + ) + + total_values = generate_random_files( + files_to_generate, prefix, started_cluster, bucket, row_num=1 + ) + + time.sleep((files_to_generate // poll_size) + 10) + get_query = f"SELECT * FROM test.s3_queue_persistent" + res1 = [ + list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + ] + res2 = [ + list(map(int, l.split())) for l in run_query(instance_2, get_query).splitlines() + ] + + # Checking that all engines have made progress + assert len(res1) > 0 + assert len(res2) > 0 + + # Checking that all files were processed only once + assert len(res1) + len(res2) == files_to_generate + assert {tuple(v) for v in res1 + res2} == set([tuple(i) for i in total_values]) + + +def test_max_set_size(started_cluster): + files_to_generate = 10 + prefix = f"test_multiple" + bucket = started_cluster.minio_restricted_bucket + instance = started_cluster.instances["instance"] + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + + instance.query( + f""" + DROP TABLE IF EXISTS test.s3_queue; + + CREATE TABLE test.s3_queue ({table_format}) + ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + SETTINGS + mode = 'unordered', + keeper_path = '/clickhouse/test_set_size', + s3queue_max_set_size = {files_to_generate - 1}; + """ + ) + + total_values = generate_random_files( + files_to_generate, prefix, started_cluster, bucket, start_ind=0, row_num=1 + ) + get_query = f"SELECT * FROM test.s3_queue" + res1 = [ + list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + ] + assert res1 == total_values + + get_query = f"SELECT * FROM test.s3_queue" + res1 = [ + list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + ] + assert res1 == [total_values[0]] + + get_query = f"SELECT * FROM test.s3_queue" + res1 = [ + list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + ] + assert res1 == [total_values[1]] From 751337fad0c64c2db0ca401a9d36106bc50c5346 Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Mon, 8 May 2023 16:31:24 +0300 Subject: [PATCH 12/34] reformat code --- src/Storages/S3Queue/S3QueueSource.cpp | 175 ++++-------------------- src/Storages/S3Queue/S3QueueSource.h | 20 +-- src/Storages/S3Queue/StorageS3Queue.cpp | 7 +- 3 files changed, 25 insertions(+), 177 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index b934f78c8df..0e0b00d2d3f 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -71,32 +71,6 @@ extern const Event S3ListObjects; namespace DB { -static const String PARTITION_ID_WILDCARD = "{_partition_id}"; - -static const std::unordered_set required_configuration_keys = { - "url", -}; -static const std::unordered_set optional_configuration_keys - = {"format", - "compression", - "compression_method", - "structure", - "access_key_id", - "secret_access_key", - "filename", - "use_environment_credentials", - "max_single_read_retries", - "min_upload_part_size", - "upload_part_size_multiply_factor", - "upload_part_size_multiply_parts_count_threshold", - "max_single_part_upload_size", - "max_connections", - "expiration_window_seconds", - "no_sign_request"}; - -class IOutputFormat; -using OutputFormatPtr = std::shared_ptr; - StorageS3QueueSource::QueueGlobIterator::QueueGlobIterator( const S3::Client & client_, @@ -222,135 +196,37 @@ StorageS3QueueSource::StorageS3QueueSource( , version_id(version_id_) , format(format_) , columns_desc(columns_) - , max_block_size(max_block_size_) , request_settings(request_settings_) - , compression_hint(std::move(compression_hint_)) , client(client_) - , sample_block(sample_block_) - , format_settings(format_settings_) , queue_holder(queue_holder_) , requested_virtual_columns(requested_virtual_columns_) , file_iterator(file_iterator_) , action(action_) - , download_thread_num(download_thread_num_) - , create_reader_pool(CurrentMetrics::StorageS3Threads, CurrentMetrics::StorageS3ThreadsActive, 1) - , create_reader_scheduler(threadPoolCallbackRunner(create_reader_pool, "CreateS3QReader")) { - reader = createReader(); + internal_source = std::make_shared( + requested_virtual_columns_, + format_, + name_, + sample_block_, + context_, + format_settings_, + columns_, + max_block_size_, + request_settings_, + compression_hint_, + client_, + bucket_, + version_id_, + file_iterator, + download_thread_num_); + reader = std::move(internal_source->reader); if (reader) - reader_future = createReaderAsync(); -} - -StorageS3QueueSource::ReaderHolder StorageS3QueueSource::createReader() -{ - auto [current_key, info] = (*file_iterator)(); - if (current_key.empty()) - return {}; - - size_t object_size = info ? info->size : S3::getObjectSize(*client, bucket, current_key, version_id, request_settings); - auto compression_method = chooseCompressionMethod(current_key, compression_hint); - - InputFormatPtr input_format; - std::unique_ptr owned_read_buf; - - auto read_buf_or_factory = createS3ReadBuffer(current_key, object_size); - if (read_buf_or_factory.buf_factory) - { - input_format = FormatFactory::instance().getInputRandomAccess( - format, - std::move(read_buf_or_factory.buf_factory), - sample_block, - getContext(), - max_block_size, - /* is_remote_fs */ true, - compression_method, - format_settings); - } - else - { - owned_read_buf = wrapReadBufferWithCompressionMethod( - std::move(read_buf_or_factory.buf), compression_method, static_cast(getContext()->getSettingsRef().zstd_window_log_max)); - input_format - = FormatFactory::instance().getInput(format, *owned_read_buf, sample_block, getContext(), max_block_size, format_settings); - } - - QueryPipelineBuilder builder; - builder.init(Pipe(input_format)); - - if (columns_desc.hasDefaults()) - { - builder.addSimpleTransform( - [&](const Block & header) - { return std::make_shared(header, columns_desc, *input_format, getContext()); }); - } - - auto pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); - auto current_reader = std::make_unique(*pipeline); - - return ReaderHolder{fs::path(bucket) / current_key, std::move(owned_read_buf), std::move(pipeline), std::move(current_reader)}; -} - -std::future StorageS3QueueSource::createReaderAsync() -{ - return create_reader_scheduler([this] { return createReader(); }, 0); -} - -StorageS3QueueSource::ReadBufferOrFactory StorageS3QueueSource::createS3ReadBuffer(const String & key, size_t object_size) -{ - auto read_settings = getContext()->getReadSettings().adjustBufferSize(object_size); - read_settings.enable_filesystem_cache = false; - auto download_buffer_size = getContext()->getSettings().max_download_buffer_size; - const bool object_too_small = object_size <= 2 * download_buffer_size; - - // Create a read buffer that will prefetch the first ~1 MB of the file. - // When reading lots of tiny files, this prefetching almost doubles the throughput. - // For bigger files, parallel reading is more useful. - if (object_too_small && read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) - { - LOG_TRACE(log, "Downloading object of size {} from S3 with initial prefetch", object_size); - return {.buf = createAsyncS3ReadBuffer(key, read_settings, object_size)}; - } - - auto factory = std::make_unique(client, bucket, key, version_id, object_size, request_settings, read_settings); - return {.buf_factory = std::move(factory)}; -} - -std::unique_ptr -StorageS3QueueSource::createAsyncS3ReadBuffer(const String & key, const ReadSettings & read_settings, size_t object_size) -{ - auto read_buffer_creator = - [this, read_settings, object_size](const std::string & path, size_t read_until_position) -> std::shared_ptr - { - return std::make_shared( - client, - bucket, - path, - version_id, - request_settings, - read_settings, - /* use_external_buffer */ true, - /* offset */ 0, - read_until_position, - /* restricted_seek */ true, - object_size); - }; - - auto s3_impl = std::make_unique( - std::move(read_buffer_creator), StoredObjects{StoredObject{key, object_size}}, read_settings); - - auto & pool_reader = getContext()->getThreadPoolReader(Context::FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); - auto async_reader = std::make_unique(pool_reader, read_settings, std::move(s3_impl)); - - async_reader->setReadUntilEnd(); - if (read_settings.remote_fs_prefetch) - async_reader->prefetch(DEFAULT_PREFETCH_PRIORITY); - - return async_reader; + reader_future = std::move(internal_source->reader_future); } StorageS3QueueSource::~StorageS3QueueSource() { - create_reader_pool.wait(); + internal_source->create_reader_pool.wait(); } String StorageS3QueueSource::getName() const @@ -370,12 +246,10 @@ Chunk StorageS3QueueSource::generate() } Chunk chunk; - LOG_WARNING(log, "Try to pull new chunk"); try { if (reader->pull(chunk)) { - LOG_WARNING(log, "Success in pulling!"); UInt64 num_rows = chunk.getNumRows(); const auto & file_path = reader.getPath(); @@ -399,7 +273,6 @@ Chunk StorageS3QueueSource::generate() chunk.addColumn(column->convertToFullColumnIfConst()); } } - LOG_WARNING(log, "Set processed: {}", file_path); queue_holder->setFileProcessed(file_path); applyActionAfterProcessing(file_path); return chunk; @@ -407,9 +280,8 @@ Chunk StorageS3QueueSource::generate() } catch (const Exception & e) { - LOG_ERROR(log, "Exception: {} ", e.displayText()); + LOG_ERROR(log, "Exception in chunk pulling: {} ", e.displayText()); const auto & failed_file_path = reader.getPath(); - LOG_WARNING(log, "Set failed: {}", failed_file_path); queue_holder->setFileFailed(failed_file_path); } @@ -419,11 +291,10 @@ Chunk StorageS3QueueSource::generate() if (!reader) break; - /// Even if task is finished the thread may be not freed in pool. /// So wait until it will be freed before scheduling a new task. - create_reader_pool.wait(); - reader_future = createReaderAsync(); + internal_source->create_reader_pool.wait(); + reader_future = internal_source->createReaderAsync(); } return {}; @@ -431,7 +302,7 @@ Chunk StorageS3QueueSource::generate() void StorageS3QueueSource::applyActionAfterProcessing(const String & file_path) { - LOG_WARNING(log, "Delete {} Bucke {}", file_path, bucket); + LOG_WARNING(log, "Delete {} Bucket {}", file_path, bucket); S3::DeleteObjectRequest request; request.SetBucket(bucket); request.SetKey(file_path); diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index 2891a5946af..fc7ce3606b0 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -36,7 +36,6 @@ public: using DisclosedGlobIterator = StorageS3Source::DisclosedGlobIterator; using KeysWithInfo = StorageS3Source::KeysWithInfo; using KeyWithInfo = StorageS3Source::KeyWithInfo; - using ReadBufferOrFactory = StorageS3Source::ReadBufferOrFactory; class QueueGlobIterator : public IIterator { public: @@ -95,8 +94,6 @@ public: Chunk generate() override; - static std::unordered_set parseCollection(String & files); - private: String name; @@ -104,12 +101,8 @@ private: String version_id; String format; ColumnsDescription columns_desc; - UInt64 max_block_size; S3Settings::RequestSettings request_settings; - String compression_hint; std::shared_ptr client; - Block sample_block; - std::optional format_settings; std::shared_ptr queue_holder; using ReaderHolder = StorageS3Source::ReaderHolder; @@ -118,12 +111,9 @@ private: std::vector requested_virtual_columns; std::shared_ptr file_iterator; const S3QueueAction action; - size_t download_thread_num = 1; Poco::Logger * log = &Poco::Logger::get("StorageS3QueueSource"); - ThreadPool create_reader_pool; - ThreadPoolCallbackRunner create_reader_scheduler; std::future reader_future; UInt64 total_rows_approx_max = 0; @@ -132,15 +122,7 @@ private: mutable std::mutex mutex; - - ReaderHolder createReader(); - std::future createReaderAsync(); - - ReadBufferOrFactory createS3ReadBuffer(const String & key, size_t object_size); - std::unique_ptr createAsyncS3ReadBuffer(const String & key, const ReadSettings & read_settings, size_t object_size); - - void setFileProcessed(const String & file_path); - void setFileFailed(const String & file_path); + std::shared_ptr internal_source; void applyActionAfterProcessing(const String & file_path); }; diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index a5dd5b421c8..09dc094eae5 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -539,8 +539,6 @@ bool StorageS3Queue::createTableIfNotExists(const StorageMetadataPtr & metadata_ for (size_t i = 0; i < 1000; ++i) { Coordination::Requests ops; - auto table_uuid = getStorageID().uuid; - if (zookeeper->exists(zookeeper_path + "/metadata")) { LOG_DEBUG(log, "This table {} is already created, will use existing metadata for checking engine settings", zookeeper_path); @@ -552,10 +550,7 @@ bool StorageS3Queue::createTableIfNotExists(const StorageMetadataPtr & metadata_ ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path, "", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processed", "processed_files\n", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/failed", "[]", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processing", "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest( - fs::path(zookeeper_path) / "processing" / toString(table_uuid), "[]", zkutil::CreateMode::Ephemeral)); - + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processing", "", zkutil::CreateMode::Ephemeral)); ops.emplace_back(zkutil::makeCreateRequest( zookeeper_path + "/columns", metadata_snapshot->getColumns().toString(), zkutil::CreateMode::Persistent)); From a443c7f28984e861a11407eb82d2f4da023a6f30 Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Tue, 9 May 2023 06:58:29 +0300 Subject: [PATCH 13/34] fix --- src/Storages/S3Queue/S3QueueHolder.cpp | 4 ++-- src/Storages/S3Queue/S3QueueSource.cpp | 2 -- src/Storages/S3Queue/StorageS3Queue.cpp | 7 +++++-- src/Storages/S3Queue/StorageS3Queue.h | 1 - 4 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueHolder.cpp b/src/Storages/S3Queue/S3QueueHolder.cpp index 21b2b7a45ce..1b708154bcd 100644 --- a/src/Storages/S3Queue/S3QueueHolder.cpp +++ b/src/Storages/S3Queue/S3QueueHolder.cpp @@ -159,7 +159,7 @@ void S3QueueHolder::setFileProcessed(const String & file_path) if (zookeeper->tryGet(fs::path(zookeeper_processing_path), node_data)) { S3FilesCollection processing_files = parseCollection(node_data); - for (auto x : processing_files) + for (const auto & x : processing_files) { if (x != file_path) { @@ -251,7 +251,7 @@ void S3QueueHolder::setFilesProcessing(Strings & file_paths) if (zookeeper->tryGet(fs::path(zookeeper_processing_path), node_data)) { S3FilesCollection processing_files = parseCollection(node_data); - for (auto x : processing_files) + for (const auto & x : processing_files) { if (!std::count(file_paths.begin(), file_paths.end(), x)) { diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 0e0b00d2d3f..eb49db8e84d 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -48,10 +48,8 @@ # include # include -# include # include # include -# include namespace fs = std::filesystem; diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 09dc094eae5..5a011c9b51a 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -539,10 +539,13 @@ bool StorageS3Queue::createTableIfNotExists(const StorageMetadataPtr & metadata_ for (size_t i = 0; i < 1000; ++i) { Coordination::Requests ops; + bool is_first_replica = true; if (zookeeper->exists(zookeeper_path + "/metadata")) { + if (!zookeeper->exists(zookeeper_path + "/processing")) + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processing", "", zkutil::CreateMode::Ephemeral)); LOG_DEBUG(log, "This table {} is already created, will use existing metadata for checking engine settings", zookeeper_path); - return false; + is_first_replica = false; } else { @@ -569,7 +572,7 @@ bool StorageS3Queue::createTableIfNotExists(const StorageMetadataPtr & metadata_ zkutil::KeeperMultiException::check(code, ops, responses); } - return true; + return is_first_replica; } throw Exception( diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index ac098cdda99..73f8075b7d1 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -130,7 +130,6 @@ private: zkutil::ZooKeeperPtr getZooKeeper() const; bool createTableIfNotExists(const StorageMetadataPtr & metadata_snapshot); void checkTableStructure(const String & zookeeper_prefix, const StorageMetadataPtr & metadata_snapshot); - // Return default or custom zookeeper name for table const String & getZooKeeperPath() const { return zookeeper_path; } using KeysWithInfo = StorageS3QueueSource::KeysWithInfo; From d37622434f5c5d22263120466a99d3a95b5465dc Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Tue, 9 May 2023 07:04:36 +0300 Subject: [PATCH 14/34] fix tests --- tests/integration/test_storage_s3_queue/test.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index 3276c221014..bc92e809f5d 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -28,7 +28,7 @@ def s3_queue_setup_teardown(started_cluster): MINIO_INTERNAL_PORT = 9001 -AVAILABLE_MODES = ["ordered", "unordered"] +AVAILABLE_MODES = ["unordered", "ordered"] AUTH = "'minio','minio123'," SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) @@ -569,7 +569,7 @@ def test_multiple_tables_streaming_sync(started_cluster, mode): total_values = generate_random_files( files_to_generate, prefix, started_cluster, bucket, row_num=1 ) - time.sleep((files_to_generate // poll_size) + 10) + time.sleep((files_to_generate // poll_size) * 2) get_query = f"SELECT * FROM test.s3_queue_persistent" res1 = [ @@ -637,7 +637,7 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): files_to_generate, prefix, started_cluster, bucket, row_num=1 ) - time.sleep((files_to_generate // poll_size) + 10) + time.sleep((files_to_generate // poll_size) * 2) get_query = f"SELECT * FROM test.s3_queue_persistent" res1 = [ list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() From 446cf3c847eb77608fd01a1020fb16bda8f4ccf5 Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Wed, 10 May 2023 18:17:26 +0300 Subject: [PATCH 15/34] handle failed files --- src/Core/Settings.h | 1 + src/Storages/S3Queue/S3QueueHolder.cpp | 269 ++++++++++-------- src/Storages/S3Queue/S3QueueHolder.h | 74 +++-- src/Storages/S3Queue/S3QueueSettings.h | 15 +- src/Storages/S3Queue/S3QueueSource.cpp | 45 ++- src/Storages/S3Queue/S3QueueSource.h | 1 + src/Storages/S3Queue/StorageS3Queue.cpp | 32 ++- .../integration/test_storage_s3_queue/test.py | 151 +++++++++- 8 files changed, 425 insertions(+), 163 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 1bea2c26392..1e0e0615f66 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -94,6 +94,7 @@ class IColumn; M(Bool, s3_allow_parallel_part_upload, true, "Use multiple threads for s3 multipart upload. It may lead to slightly higher memory usage", 0) \ M(Bool, s3_throw_on_zero_files_match, false, "Throw an error, when ListObjects request cannot match any files", 0) \ M(Bool, enable_s3_requests_logging, false, "Enable very explicit logging of S3 requests. Makes sense for debug only.", 0) \ + M(String, s3queue_default_zookeeper_path, "", "Default zookeeper path prefix for S3Queue engine", 0) \ M(UInt64, hdfs_replication, 0, "The actual number of replications can be specified when the hdfs file is created.", 0) \ M(Bool, hdfs_truncate_on_insert, false, "Enables or disables truncate before insert in s3 engine tables", 0) \ M(Bool, hdfs_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in hdfs engine tables", 0) \ diff --git a/src/Storages/S3Queue/S3QueueHolder.cpp b/src/Storages/S3Queue/S3QueueHolder.cpp index 1b708154bcd..91bf82001bb 100644 --- a/src/Storages/S3Queue/S3QueueHolder.cpp +++ b/src/Storages/S3Queue/S3QueueHolder.cpp @@ -16,20 +16,14 @@ namespace DB namespace ErrorCodes { - extern const int NOT_IMPLEMENTED; extern const int NO_ZOOKEEPER; extern const int TIMEOUT_EXCEEDED; } -S3QueueHolder::ProcessedCollection::ProcessedCollection(const UInt64 & max_size_, const UInt64 & max_age_) - : max_size(max_size_), max_age(max_age_) -{ -} - -void S3QueueHolder::ProcessedCollection::read(ReadBuffer & in) +void S3QueueHolder::S3QueueCollection::read(ReadBuffer & in) { files = {}; - in >> "processed_files\n"; + in >> "collection:\n"; while (!in.eof()) { String file_name; @@ -41,9 +35,9 @@ void S3QueueHolder::ProcessedCollection::read(ReadBuffer & in) } } -void S3QueueHolder::ProcessedCollection::write(WriteBuffer & out) const +void S3QueueHolder::S3QueueCollection::write(WriteBuffer & out) const { - out << "processed_files\n"; + out << "collection:\n"; for (const auto & processed_file : files) { out << processed_file.first << "\n"; @@ -51,7 +45,30 @@ void S3QueueHolder::ProcessedCollection::write(WriteBuffer & out) const } } -void S3QueueHolder::ProcessedCollection::parse(const String & s) +String S3QueueHolder::S3QueueCollection::toString() const +{ + WriteBufferFromOwnString out; + write(out); + return out.str(); +} + +S3QueueHolder::S3FilesCollection S3QueueHolder::S3QueueCollection::getFileNames() +{ + S3FilesCollection keys = {}; + for (const auto & pair : files) + { + keys.insert(pair.first); + } + return keys; +} + + +S3QueueHolder::S3QueueProcessedCollection::S3QueueProcessedCollection(const UInt64 & max_size_, const UInt64 & max_age_) + : max_size(max_size_), max_age(max_age_) +{ +} + +void S3QueueHolder::S3QueueProcessedCollection::parse(const String & s) { ReadBufferFromString buf(s); read(buf); @@ -69,14 +86,8 @@ void S3QueueHolder::ProcessedCollection::parse(const String & s) } } -String S3QueueHolder::ProcessedCollection::toString() const -{ - WriteBufferFromOwnString out; - write(out); - return out.str(); -} -void S3QueueHolder::ProcessedCollection::add(const String & file_name) +void S3QueueHolder::S3QueueProcessedCollection::add(const String & file_name) { Int64 timestamp = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); auto pair = std::make_pair(file_name, timestamp); @@ -89,22 +100,62 @@ void S3QueueHolder::ProcessedCollection::add(const String & file_name) } } -S3QueueHolder::S3FilesCollection S3QueueHolder::ProcessedCollection::getFileNames() + +S3QueueHolder::S3QueueFailedCollection::S3QueueFailedCollection(const UInt64 & max_retries_count_) : max_retries_count(max_retries_count_) { - S3FilesCollection keys = {}; - for (auto & pair : files) - { - keys.insert(pair.first); - } - return keys; +} + +void S3QueueHolder::S3QueueFailedCollection::parse(const String & s) +{ + ReadBufferFromString buf(s); + read(buf); } +bool S3QueueHolder::S3QueueFailedCollection::add(const String & file_name) +{ + auto failed_it + = std::find_if(files.begin(), files.end(), [&file_name](const std::pair & s) { return s.first == file_name; }); + if (failed_it != files.end()) + { + failed_it->second--; + if (failed_it->second == 0) + { + return false; + } + } + else + { + auto pair = std::make_pair(file_name, max_retries_count); + files.push_back(pair); + } + return true; +} + +S3QueueHolder::S3FilesCollection S3QueueHolder::S3QueueFailedCollection::getFilesWithoutRetries() +{ + S3FilesCollection failed_keys; + for (const auto & pair : files) + { + if (pair.second <= 0) + { + failed_keys.insert(pair.first); + } + } + return failed_keys; +} + S3QueueHolder::S3QueueHolder( - const String & zookeeper_path_, const S3QueueMode & mode_, ContextPtr context_, UInt64 & max_set_size_, UInt64 & max_set_age_s_) + const String & zookeeper_path_, + const S3QueueMode & mode_, + ContextPtr context_, + UInt64 & max_set_size_, + UInt64 & max_set_age_s_, + UInt64 & max_loading_retries_) : WithContext(context_) , max_set_size(max_set_size_) , max_set_age_s(max_set_age_s_) + , max_loading_retries(max_loading_retries_) , zookeeper_path(zookeeper_path_) , zookeeper_failed_path(zookeeper_path_ + "/failed") , zookeeper_processing_path(zookeeper_path_ + "/processing") @@ -140,7 +191,7 @@ void S3QueueHolder::setFileProcessed(const String & file_path) if (mode == S3QueueMode::UNORDERED) { String processed_files = zookeeper->get(zookeeper_processed_path); - auto processed = ProcessedCollection(max_set_size, max_set_age_s); + auto processed = S3QueueProcessedCollection(max_set_size, max_set_age_s); processed.parse(processed_files); processed.add(file_path); zookeeper->set(zookeeper_processed_path, processed.toString()); @@ -153,144 +204,111 @@ void S3QueueHolder::setFileProcessed(const String & file_path) zookeeper->set(zookeeper_processed_path, file_path); } } - - String node_data; - Strings file_paths; - if (zookeeper->tryGet(fs::path(zookeeper_processing_path), node_data)) - { - S3FilesCollection processing_files = parseCollection(node_data); - for (const auto & x : processing_files) - { - if (x != file_path) - { - file_paths.push_back(x); - } - } - } - zookeeper->set(fs::path(zookeeper_processing_path), toString(file_paths)); + removeProcessingFile(file_path); } -void S3QueueHolder::setFileFailed(const String & file_path) +bool S3QueueHolder::markFailedAndCheckRetry(const String & file_path) { auto zookeeper = getZooKeeper(); auto lock = AcquireLock(); String failed_files = zookeeper->get(zookeeper_failed_path); - S3FilesCollection failed = parseCollection(failed_files); + auto failed_collection = S3QueueFailedCollection(max_loading_retries); + failed_collection.parse(failed_files); + bool retry_later = failed_collection.add(file_path); - failed.insert(file_path); - Strings set_failed; - set_failed.insert(set_failed.end(), failed.begin(), failed.end()); + zookeeper->set(zookeeper_failed_path, failed_collection.toString()); + removeProcessingFile(file_path); - zookeeper->set(zookeeper_failed_path, toString(set_failed)); + return retry_later; } -S3QueueHolder::S3FilesCollection S3QueueHolder::parseCollection(String & files) +S3QueueHolder::S3FilesCollection S3QueueHolder::getFailedFiles() { - ReadBuffer rb(const_cast(reinterpret_cast(files.data())), files.length(), 0); - Strings deserialized; - try - { - readQuoted(deserialized, rb); - } - catch (...) - { - deserialized = {}; - } - - std::unordered_set processed(deserialized.begin(), deserialized.end()); - - return processed; -} - -S3QueueHolder::S3FilesCollection S3QueueHolder::getExcludedFiles() -{ - std::unordered_set exclude_files; auto zookeeper = getZooKeeper(); + String failed_files = zookeeper->get(zookeeper_failed_path); - String failed = zookeeper->get(zookeeper_failed_path); - S3FilesCollection failed_files = parseCollection(failed); - exclude_files.merge(failed_files); + auto failed_collection = S3QueueFailedCollection(max_loading_retries); + failed_collection.parse(failed_files); - String processed = zookeeper->get(zookeeper_processed_path); - if (mode != S3QueueMode::ORDERED) - { - auto collection = ProcessedCollection(max_set_size, max_set_age_s); - collection.parse(processed); - S3FilesCollection processed_files = collection.getFileNames(); - exclude_files.merge(processed_files); - } - else - { - exclude_files.insert(processed); - } - - String processing = zookeeper->get(fs::path(zookeeper_processing_path)); - S3FilesCollection processing_files = parseCollection(processing); - exclude_files.merge(processing_files); - - return exclude_files; + return failed_collection.getFilesWithoutRetries(); } String S3QueueHolder::getMaxProcessedFile() { - if (mode != S3QueueMode::ORDERED) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "getMaxProcessedFile not implemented for unordered mode"); - auto zookeeper = getZooKeeper(); - String processed = zookeeper->get(zookeeper_path + "/processed"); + String processed = zookeeper->get(zookeeper_processed_path); return processed; } +S3QueueHolder::S3FilesCollection S3QueueHolder::getProcessingFiles() +{ + auto zookeeper = getZooKeeper(); + String processing = zookeeper->get(fs::path(zookeeper_processing_path)); + return parseCollection(processing); +} + void S3QueueHolder::setFilesProcessing(Strings & file_paths) { auto zookeeper = getZooKeeper(); - String node_data; - if (zookeeper->tryGet(fs::path(zookeeper_processing_path), node_data)) + for (const auto & x : getProcessingFiles()) { - S3FilesCollection processing_files = parseCollection(node_data); - for (const auto & x : processing_files) + if (!std::count(file_paths.begin(), file_paths.end(), x)) { - if (!std::count(file_paths.begin(), file_paths.end(), x)) - { - file_paths.push_back(x); - } + file_paths.push_back(x); } } zookeeper->set(fs::path(zookeeper_processing_path), toString(file_paths)); } -S3QueueHolder::S3FilesCollection S3QueueHolder::getFailedFiles() +S3QueueHolder::S3FilesCollection S3QueueHolder::getUnorderedProcessedFiles() { auto zookeeper = getZooKeeper(); - auto lock = AcquireLock(); - String failed = zookeeper->get(zookeeper_failed_path); - return parseCollection(failed); -} - -S3QueueHolder::S3FilesCollection S3QueueHolder::getProcessedFiles() -{ - auto zookeeper = getZooKeeper(); - - auto lock = AcquireLock(); String processed = zookeeper->get(zookeeper_processed_path); - auto collection = ProcessedCollection(max_set_size, max_set_age_s); + auto collection = S3QueueProcessedCollection(max_set_size, max_set_age_s); collection.parse(processed); + return collection.getFileNames(); } -S3QueueHolder::S3FilesCollection S3QueueHolder::getProcessingFiles() +S3QueueHolder::S3FilesCollection S3QueueHolder::getExcludedFiles() { auto zookeeper = getZooKeeper(); - auto lock = AcquireLock(); - String processing = zookeeper->get(fs::path(zookeeper_processing_path)); - return parseCollection(processing); + S3FilesCollection exclude_files = getFailedFiles(); + + if (mode == S3QueueMode::UNORDERED) + { + S3FilesCollection processed_files = getUnorderedProcessedFiles(); + exclude_files.merge(processed_files); + } + else + { + String processed = getMaxProcessedFile(); + exclude_files.insert(processed); + } + + S3FilesCollection processing_files = getProcessingFiles(); + exclude_files.merge(processing_files); + + return exclude_files; } +void S3QueueHolder::removeProcessingFile(const String & file_path) +{ + auto zookeeper = getZooKeeper(); + String node_data; + Strings file_paths; + String processing = zookeeper->get(zookeeper_processing_path); + S3FilesCollection processing_files = parseCollection(processing); + file_paths.insert(file_paths.end(), processing_files.begin(), processing_files.end()); + + file_paths.erase(std::remove(file_paths.begin(), file_paths.end(), file_path), file_paths.end()); + zookeeper->set(fs::path(zookeeper_processing_path), toString(file_paths)); +} std::shared_ptr S3QueueHolder::AcquireLock() { @@ -322,6 +340,23 @@ std::shared_ptr S3QueueHolder::AcquireLock() } } +S3QueueHolder::S3FilesCollection S3QueueHolder::parseCollection(String & files) +{ + ReadBuffer rb(const_cast(reinterpret_cast(files.data())), files.length(), 0); + Strings deserialized; + try + { + readQuoted(deserialized, rb); + } + catch (...) + { + deserialized = {}; + } + + std::unordered_set processed(deserialized.begin(), deserialized.end()); + + return processed; +} } diff --git a/src/Storages/S3Queue/S3QueueHolder.h b/src/Storages/S3Queue/S3QueueHolder.h index 081e58a9ea2..f7fa0461a3a 100644 --- a/src/Storages/S3Queue/S3QueueHolder.h +++ b/src/Storages/S3Queue/S3QueueHolder.h @@ -14,44 +14,73 @@ class S3QueueHolder : public WithContext public: using S3FilesCollection = std::unordered_set; using ProcessedFiles = std::vector>; + using FailedFiles = std::vector>; S3QueueHolder( - const String & zookeeper_path_, const S3QueueMode & mode_, ContextPtr context_, UInt64 & max_set_size_, UInt64 & max_set_age_s_); + const String & zookeeper_path_, + const S3QueueMode & mode_, + ContextPtr context_, + UInt64 & max_set_size_, + UInt64 & max_set_age_s_, + UInt64 & max_loading_retries_); void setFileProcessed(const String & file_path); - void setFileFailed(const String & file_path); + bool markFailedAndCheckRetry(const String & file_path); void setFilesProcessing(Strings & file_paths); - static S3FilesCollection parseCollection(String & files); - S3FilesCollection getExcludedFiles(); String getMaxProcessedFile(); - S3FilesCollection getFailedFiles(); - S3FilesCollection getProcessedFiles(); - S3FilesCollection getProcessingFiles(); + std::shared_ptr AcquireLock(); - struct ProcessedCollection + struct S3QueueCollection { - ProcessedCollection(const UInt64 & max_size_, const UInt64 & max_age_); - - void parse(const String & s); - + public: + virtual ~S3QueueCollection() = default; String toString() const; - - void add(const String & file_name); S3FilesCollection getFileNames(); - const UInt64 max_size; - const UInt64 max_age; + + virtual void parse(const String & s) = 0; + + protected: + ProcessedFiles files; void read(ReadBuffer & in); void write(WriteBuffer & out) const; - ProcessedFiles files; }; - const UInt64 max_set_size; - const UInt64 max_set_age_s; + struct S3QueueProcessedCollection : public S3QueueCollection + { + public: + S3QueueProcessedCollection(const UInt64 & max_size_, const UInt64 & max_age_); + + void parse(const String & s) override; + void add(const String & file_name); + + private: + const UInt64 max_size; + const UInt64 max_age; + }; + + struct S3QueueFailedCollection : S3QueueCollection + { + public: + S3QueueFailedCollection(const UInt64 & max_retries_count_); + + void parse(const String & s) override; + bool add(const String & file_name); + + S3FilesCollection getFilesWithoutRetries(); + + private: + const UInt64 max_retries_count; + }; + private: + const UInt64 max_set_size; + const UInt64 max_set_age_s; + const UInt64 max_loading_retries; + zkutil::ZooKeeperPtr current_zookeeper; mutable std::mutex current_zookeeper_mutex; mutable std::mutex mutex; @@ -66,6 +95,13 @@ private: zkutil::ZooKeeperPtr tryGetZooKeeper() const; zkutil::ZooKeeperPtr getZooKeeper() const; + + S3FilesCollection getFailedFiles(); + S3FilesCollection getProcessingFiles(); + S3FilesCollection getUnorderedProcessedFiles(); + void removeProcessingFile(const String & file_path); + + static S3FilesCollection parseCollection(String & files); }; diff --git a/src/Storages/S3Queue/S3QueueSettings.h b/src/Storages/S3Queue/S3QueueSettings.h index c2b8e51a1f8..3b4ea475887 100644 --- a/src/Storages/S3Queue/S3QueueSettings.h +++ b/src/Storages/S3Queue/S3QueueSettings.h @@ -14,16 +14,21 @@ class ASTStorage; M(S3QueueMode, \ mode, \ S3QueueMode::ORDERED, \ - "With unordered mode, the set of all already processed files is tracked with persistent nodes in ZooKepeer", \ + "With unordered mode, the set of all already processed files is tracked with persistent nodes in ZooKepeer." \ + "With ordered mode, only the max name of the successfully consumed file stored.", \ 0) \ - M(S3QueueAction, after_processing, S3QueueAction::KEEP, "Delete, keep or move file in S3 after processing", 0) \ + M(S3QueueAction, after_processing, S3QueueAction::KEEP, "Delete or keep file in S3 after successful processing", 0) \ M(String, keeper_path, "", "Zookeeper node path", 0) \ - M(UInt64, s3queue_max_retries, 0, "Retry loading up to specified number of times", 0) \ + M(UInt64, s3queue_loading_retries, 0, "Retry loading up to specified number of times", 0) \ M(UInt64, s3queue_polling_min_timeout_ms, 1000, "Minimal timeout before next polling", 0) \ M(UInt64, s3queue_polling_max_timeout_ms, 10000, "Maximum timeout before next polling", 0) \ - M(UInt64, s3queue_polling_backoff_ms, 0, "Retry loading up to specified number of times", 0) \ + M(UInt64, s3queue_polling_backoff_ms, 0, "Polling backoff", 0) \ M(UInt64, s3queue_max_set_size, 1000, "Max set size for tracking processed files in unordered mode in ZooKeeper", 0) \ - M(UInt64, s3queue_max_set_age_s, 0, "Maximum number of seconds to store processed files in ZooKeeper node (forever default)", 0) \ + M(UInt64, \ + s3queue_max_set_age_s, \ + 0, \ + "Maximum number of seconds to store processed files in ZooKeeper node (store forever by default)", \ + 0) \ M(UInt64, s3queue_polling_size, 50, "Maximum files to fetch from S3 with SELECT", 0) #define LIST_OF_S3QUEUE_SETTINGS(M, ALIAS) \ diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index eb49db8e84d..27fcdcc892c 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -69,6 +69,11 @@ extern const Event S3ListObjects; namespace DB { +namespace ErrorCodes +{ + extern const int S3_ERROR; +} + StorageS3QueueSource::QueueGlobIterator::QueueGlobIterator( const S3::Client & client_, @@ -244,13 +249,15 @@ Chunk StorageS3QueueSource::generate() } Chunk chunk; + bool success_in_pulling = false; + String file_path; try { if (reader->pull(chunk)) { UInt64 num_rows = chunk.getNumRows(); - const auto & file_path = reader.getPath(); + file_path = reader.getPath(); size_t total_size = file_iterator->getTotalSize(); if (num_rows && total_size) { @@ -271,16 +278,21 @@ Chunk StorageS3QueueSource::generate() chunk.addColumn(column->convertToFullColumnIfConst()); } } - queue_holder->setFileProcessed(file_path); - applyActionAfterProcessing(file_path); - return chunk; + success_in_pulling = true; } } catch (const Exception & e) { LOG_ERROR(log, "Exception in chunk pulling: {} ", e.displayText()); const auto & failed_file_path = reader.getPath(); - queue_holder->setFileFailed(failed_file_path); + queue_holder->markFailedAndCheckRetry(failed_file_path); + success_in_pulling = false; + } + if (success_in_pulling) + { + applyActionAfterProcessing(file_path); + queue_holder->setFileProcessed(file_path); + return chunk; } @@ -298,14 +310,28 @@ Chunk StorageS3QueueSource::generate() return {}; } + void StorageS3QueueSource::applyActionAfterProcessing(const String & file_path) { - LOG_WARNING(log, "Delete {} Bucket {}", file_path, bucket); + switch (action) + { + case S3QueueAction::DELETE: + deleteProcessedObject(file_path); + break; + case S3QueueAction::KEEP: + break; + } +} + +void StorageS3QueueSource::deleteProcessedObject(const String & file_path) +{ + LOG_WARNING(log, "Delete processed file {} from bucket {}", file_path, bucket); S3::DeleteObjectRequest request; - request.SetBucket(bucket); - request.SetKey(file_path); + String delete_key = file_path.substr(bucket.length() + 1); + + request.WithKey(delete_key).WithBucket(bucket); auto outcome = client->DeleteObject(request); - if (!outcome.IsSuccess() && !S3::isNotFoundError(outcome.GetError().GetErrorType())) + if (!outcome.IsSuccess()) { const auto & err = outcome.GetError(); LOG_ERROR(log, "{} (Code: {})", err.GetMessage(), static_cast(err.GetErrorType())); @@ -316,7 +342,6 @@ void StorageS3QueueSource::applyActionAfterProcessing(const String & file_path) } } - } #endif diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index fc7ce3606b0..a55dd2cbe40 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -123,6 +123,7 @@ private: mutable std::mutex mutex; std::shared_ptr internal_source; + void deleteProcessedObject(const String & file_path); void applyActionAfterProcessing(const String & file_path); }; diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 5a011c9b51a..113abec4b09 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -59,11 +59,6 @@ namespace fs = std::filesystem; -//namespace CurrentMetrics -//{ -//extern const Metric S3QueueBackgroundReads; -//} - namespace ProfileEvents { extern const Event S3DeleteObjects; @@ -131,7 +126,14 @@ StorageS3Queue::StorageS3Queue( auto table_id = getStorageID(); auto database = DatabaseCatalog::instance().getDatabase(table_id.database_name); bool is_in_replicated_database = database->getEngineName() == "Replicated"; - if (is_in_replicated_database) + + auto default_path = getContext()->getSettingsRef().s3queue_default_zookeeper_path.value; + if (default_path != "") + { + zookeeper_path + = zkutil::extractZooKeeperPath(fs::path(default_path) / toString(table_id.uuid), /* check_starts_with_slash */ true, log); + } + else if (is_in_replicated_database) { LOG_INFO(log, "S3Queue engine keeper_path not specified. Use replicated database zookeeper path"); String base_zookeeper_path = assert_cast(database.get())->getZooKeeperPath(); @@ -140,7 +142,10 @@ StorageS3Queue::StorageS3Queue( } else { - throw Exception(ErrorCodes::NO_ZOOKEEPER, "S3Queue zookeeper path not specified and table not in replicated database."); + throw Exception( + ErrorCodes::NO_ZOOKEEPER, + "S3Queue keeper_path engine setting not specified, s3queue_default_zookeeper_path_prefix not specified and table not in " + "replicated database."); } } else @@ -176,7 +181,12 @@ StorageS3Queue::StorageS3Queue( } queue_holder = std::make_unique( - zookeeper_path, mode, getContext(), s3queue_settings->s3queue_max_set_size.value, s3queue_settings->s3queue_max_set_age_s.value); + zookeeper_path, + mode, + getContext(), + s3queue_settings->s3queue_max_set_size.value, + s3queue_settings->s3queue_max_set_age_s.value, + s3queue_settings->s3queue_loading_retries.value); auto default_virtuals = NamesAndTypesList{ {"_path", std::make_shared(std::make_shared())}, @@ -187,7 +197,7 @@ StorageS3Queue::StorageS3Queue( for (const auto & column : virtual_columns) virtual_block.insert({column.type->createColumn(), column.type, column.name}); - auto poll_thread = context_->getSchedulePool().createTask("S3QueueStreamingTask", [this] { threadFunc(); }); + auto poll_thread = getContext()->getSchedulePool().createTask("S3QueueStreamingTask", [this] { threadFunc(); }); task = std::make_shared(std::move(poll_thread)); } @@ -551,8 +561,8 @@ bool StorageS3Queue::createTableIfNotExists(const StorageMetadataPtr & metadata_ { String metadata_str = S3QueueTableMetadata(s3_configuration, *s3queue_settings).toString(); ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path, "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processed", "processed_files\n", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/failed", "[]", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processed", "collection:\n", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/failed", "collection:\n", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processing", "", zkutil::CreateMode::Ephemeral)); ops.emplace_back(zkutil::makeCreateRequest( zookeeper_path + "/columns", metadata_snapshot->getColumns().toString(), zkutil::CreateMode::Persistent)); diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index bc92e809f5d..085f92035d0 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -7,7 +7,72 @@ import time import pytest from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster, ClickHouseInstance -from helpers.s3_tools import prepare_s3_bucket +import json + +""" +export CLICKHOUSE_TESTS_SERVER_BIN_PATH=/home/sergey/vkr/ClickHouse/build/programs/clickhouse-server +export CLICKHOUSE_TESTS_CLIENT_BIN_PATH=/home/sergey/vkr/ClickHouse/build/programs/clickhouse-client +export CLICKHOUSE_TESTS_ODBC_BRIDGE_BIN_PATH=/home/sergey/vkr/ClickHouse/build/programs/clickhouse-odbc-bridge +export CLICKHOUSE_TESTS_BASE_CONFIG_DIR=/home/sergey/vkr/ClickHouse/programs/server + +""" + + +def prepare_s3_bucket(started_cluster): + # Allows read-write access for bucket without authorization. + bucket_read_write_policy = { + "Version": "2012-10-17", + "Statement": [ + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:GetBucketLocation", + "Resource": "arn:aws:s3:::root", + }, + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:ListBucket", + "Resource": "arn:aws:s3:::root", + }, + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:GetObject", + "Resource": "arn:aws:s3:::root/*", + }, + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:PutObject", + "Resource": "arn:aws:s3:::root/*", + }, + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:DeleteObject", + "Resource": "arn:aws:s3:::root/*", + }, + ], + } + + minio_client = started_cluster.minio_client + minio_client.set_bucket_policy( + started_cluster.minio_bucket, json.dumps(bucket_read_write_policy) + ) + + started_cluster.minio_restricted_bucket = "{}-with-auth".format( + started_cluster.minio_bucket + ) + if minio_client.bucket_exists(started_cluster.minio_restricted_bucket): + minio_client.remove_bucket(started_cluster.minio_restricted_bucket) + + minio_client.make_bucket(started_cluster.minio_restricted_bucket) @pytest.fixture(autouse=True) @@ -112,6 +177,90 @@ def run_query(instance, query, stdin=None, settings=None): return result +@pytest.mark.parametrize("mode", AVAILABLE_MODES) +def test_delete_after_processing(started_cluster, mode): + prefix = "delete" + bucket = started_cluster.minio_bucket + instance = started_cluster.instances["instance"] + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + + total_values = generate_random_files(5, prefix, started_cluster, bucket) + instance.query( + f""" + DROP TABLE IF EXISTS test.s3_queue; + CREATE TABLE test.s3_queue ({table_format}) + ENGINE = S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') + SETTINGS + mode = '{mode}', + keeper_path = '/clickhouse/test_delete_{mode}', + s3queue_loading_retries = 3, + after_processing='delete'; + """ + ) + + get_query = f"SELECT * FROM test.s3_queue" + assert [ + list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + ] == total_values + minio = started_cluster.minio_client + objects = list(minio.list_objects(started_cluster.minio_bucket, recursive=True)) + assert len(objects) == 0 + + +@pytest.mark.parametrize("mode", AVAILABLE_MODES) +def test_failed_retry(started_cluster, mode): + bucket = started_cluster.minio_restricted_bucket + instance = started_cluster.instances["instance"] + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + + values = [ + ["failed", 1, 1], + ] + values_csv = ( + "\n".join((",".join(map(str, row)) for row in values)) + "\n" + ).encode() + filename = f"test.csv" + put_s3_file_content(started_cluster, bucket, filename, values_csv) + + instance.query( + f""" + DROP TABLE IF EXISTS test.s3_queue; + CREATE TABLE test.s3_queue ({table_format}) + ENGINE = S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/*', {AUTH}'CSV') + SETTINGS + mode = '{mode}', + keeper_path = '/clickhouse/select_failed_retry_{mode}', + s3queue_loading_retries = 3; + """ + ) + + # first try + get_query = f"SELECT * FROM test.s3_queue" + assert [ + list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + ] == [] + # second try + assert [ + list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + ] == [] + # upload correct file + values = [ + [1, 1, 1], + ] + values_csv = ( + "\n".join((",".join(map(str, row)) for row in values)) + "\n" + ).encode() + put_s3_file_content(started_cluster, bucket, filename, values_csv) + + assert [ + list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + ] == values + + assert [ + list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() + ] == [] + + @pytest.mark.parametrize("mode", AVAILABLE_MODES) def test_direct_select_file(started_cluster, mode): auth = "'minio','minio123'," From abf1d459b6d76536855ba6ad62b23bd84fe06709 Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Mon, 15 May 2023 09:38:59 +0300 Subject: [PATCH 16/34] fix build --- src/Storages/S3Queue/StorageS3Queue.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 113abec4b09..3da15fc62b9 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -121,14 +121,14 @@ StorageS3Queue::StorageS3Queue( String setting_zookeeper_path = s3queue_settings->keeper_path; LOG_INFO(log, "Settings zookeeper_path={}", setting_zookeeper_path); - if (setting_zookeeper_path == "") + if (setting_zookeeper_path.empty()) { auto table_id = getStorageID(); auto database = DatabaseCatalog::instance().getDatabase(table_id.database_name); bool is_in_replicated_database = database->getEngineName() == "Replicated"; auto default_path = getContext()->getSettingsRef().s3queue_default_zookeeper_path.value; - if (default_path != "") + if (!default_path.empty()) { zookeeper_path = zkutil::extractZooKeeperPath(fs::path(default_path) / toString(table_id.uuid), /* check_starts_with_slash */ true, log); From 1880dedacac75432f7854a4b1a01db8b2192d42c Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Mon, 15 May 2023 11:19:50 +0300 Subject: [PATCH 17/34] fix build --- src/Storages/S3Queue/S3QueueSource.cpp | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 27fcdcc892c..9a9472d9578 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -51,8 +51,6 @@ # include # include -namespace fs = std::filesystem; - namespace CurrentMetrics { @@ -103,7 +101,7 @@ StorageS3QueueSource::QueueGlobIterator::QueueGlobIterator( Strings StorageS3QueueSource::QueueGlobIterator::filterProcessingFiles( const S3QueueMode & engine_mode, std::unordered_set & exclude_keys, const String & max_file) { - for (KeyWithInfo val : keys_buf) + for (const KeyWithInfo & val : keys_buf) { auto full_path = bucket + '/' + val.key; if (exclude_keys.find(full_path) != exclude_keys.end()) @@ -138,7 +136,7 @@ Strings StorageS3QueueSource::QueueGlobIterator::filterProcessingFiles( } Strings keys; - for (auto v : processing_keys) + for (const auto & v : processing_keys) { keys.push_back(bucket + '/' + v.key); } From ba55c11d0a7090abf84f480189d01fa324c5d7ec Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Mon, 15 May 2023 15:26:44 +0300 Subject: [PATCH 18/34] add documentation --- .../table-engines/integrations/s3queue.md | 194 ++++++++++++++++++ 1 file changed, 194 insertions(+) create mode 100644 docs/en/engines/table-engines/integrations/s3queue.md diff --git a/docs/en/engines/table-engines/integrations/s3queue.md b/docs/en/engines/table-engines/integrations/s3queue.md new file mode 100644 index 00000000000..46cecd35a48 --- /dev/null +++ b/docs/en/engines/table-engines/integrations/s3queue.md @@ -0,0 +1,194 @@ +--- +slug: /en/engines/table-engines/integrations/s3queue +sidebar_position: 7 +sidebar_label: S3Queue +--- + +# S3Queue Table Engine +This engine provides integration with [Amazon S3](https://aws.amazon.com/s3/) ecosystem and allows streaming import. This engine is similar to the [Kafka](../../../engines/table-engines/integrations/kafka.md), [RabbitMQ](../../../engines/table-engines/integrations/rabbitmq.md) engines, but provides S3-specific features. + +## Create Table {#creating-a-table} + +``` sql +CREATE TABLE s3_queue_engine_table (name String, value UInt32) + ENGINE = S3Queue(path [, NOSIGN | aws_access_key_id, aws_secret_access_key,] format, [compression]) + [SETTINGS] + [mode = 'unordered',] + [after_processing = 'keep',] + [keeper_path = '',] + [s3queue_loading_retries = 0,] + [s3queue_polling_min_timeout_ms = 1000,] + [s3queue_polling_max_timeout_ms = 10000,] + [s3queue_polling_backoff_ms = 0,] + [s3queue_max_set_size = 1000,] + [s3queue_max_set_age_s = 0,] + [s3queue_polling_size = 50,] +``` + +**Engine parameters** + +- `path` — Bucket url with path to file. Supports following wildcards in readonly mode: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc'`, `'def'` — strings. For more information see [below](#wildcards-in-path). +- `NOSIGN` - If this keyword is provided in place of credentials, all the requests will not be signed. +- `format` — The [format](../../../interfaces/formats.md#formats) of the file. +- `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file. For more information see [Using S3 for Data Storage](../mergetree-family/mergetree.md#table_engine-mergetree-s3). +- `compression` — Compression type. Supported values: `none`, `gzip/gz`, `brotli/br`, `xz/LZMA`, `zstd/zst`. Parameter is optional. By default, it will autodetect compression by file extension. + + +## Settings {#s3queue-settings} + +### mode {#mode} + +Allows to automatically close the connection after query execution, i.e. disable connection reuse. + +Possible values: + +- unordered — With unordered mode, the set of all already processed files is tracked with persistent nodes in ZooKepeer. +- ordered — With ordered mode, only the max name of the successfully consumed file, and the names of files that will be retried after unsuccessful loading attempt are being stored in ZooKeeper. + +Default value: `unordered`. + +### after_processing {#after_processing} + +Delete or keep file after successful processing. +Possible values: + +- keep. +- delete. + +Default value: `keep`. + +### keeper_path {#keeper_path} + +The path in ZooKeeper can be specified as a table engine setting or default path can be formed from the global configuration-provided path and table UUID. +Possible values: + +- String. + +Default value: ``. + +### s3queue_loading_retries {#s3queue_loading_retries} + +Retry file loading up to specified number of times. By default, there are no retries. +Possible values: + +- Positive integer. + +Default value: `0`. + +### s3queue_polling_min_timeout_ms {#s3queue_polling_min_timeout_ms} + +Minimal timeout before next polling (in milliseconds). + +Possible values: + +- Positive integer. + +Default value: `1000`. + +### s3queue_polling_max_timeout_ms {#s3queue_polling_max_timeout_ms} + +Maximum timeout before next polling (in milliseconds). + +Possible values: + +- Positive integer. + +Default value: `10000`. + +### s3queue_polling_backoff_ms {#s3queue_polling_backoff_ms} + +Polling backoff (in milliseconds). + +Possible values: + +- Positive integer. + +Default value: `0`. + +### s3queue_max_set_size {#s3queue_max_set_size} + +Max set size for tracking processed files in unordered mode in ZooKeeper. + +Possible values: + +- Positive integer. + +Default value: `1000`. + +### s3queue_max_set_age_s {#s3queue_max_set_age_s} + +Maximum number of seconds to store processed files in ZooKeeper node (store forever by default). + +Possible values: + +- Positive integer. + +Default value: `0`. + +### s3queue_polling_size {#s3queue_polling_size} + +Maximum files to fetch from S3 with SELECT or in background task. + +Possible values: + +- Positive integer. + +Default value: `50`. + + +## S3-related Settings {#s3-settings} + +Engine supports all s3 related settings. For more information about S3 settings see [here](../../../engines/table-engines/integrations/s3.md). + + +## Description {#description} + +`SELECT` is not particularly useful for streaming import (except for debugging), because each file can be imported only once. It is more practical to create real-time threads using [materialized views](../../../sql-reference/statements/create/view.md). To do this: + +1. Use the engine to create a table for consuming from specified path in S3 and consider it a data stream. +2. Create a table with the desired structure. +3. Create a materialized view that converts data from the engine and puts it into a previously created table. + +When the `MATERIALIZED VIEW` joins the engine, it starts collecting data in the background. + +Example: + +``` sql + CREATE TABLE s3queue_engine_table (name String, value UInt32) + ENGINE=S3Queue('https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/*', 'CSV', 'gzip') + SETTINGS + mode = 'unordred', + keeper_path = '/clickhouse/s3queue/'; + + CREATE TABLE stats (name String, value UInt32) + ENGINE = MergeTree() ORDER BY name; + + CREATE MATERIALIZED VIEW consumer TO stats + AS SELECT name, value FROM s3queue_engine_table; + + SELECT * FROM stats ORDER BY name; +``` + +## Virtual columns {#virtual-columns} + +- `_path` — Path to the file. +- `_file` — Name of the file. + +For more information about virtual columns see [here](../../../engines/table-engines/index.md#table_engines-virtual_columns). + + +## Wildcards In Path {#wildcards-in-path} + +`path` argument can specify multiple files using bash-like wildcards. For being processed file should exist and match to the whole path pattern. Listing of files is determined during `SELECT` (not at `CREATE` moment). + +- `*` — Substitutes any number of any characters except `/` including empty string. +- `?` — Substitutes any single character. +- `{some_string,another_string,yet_another_one}` — Substitutes any of strings `'some_string', 'another_string', 'yet_another_one'`. +- `{N..M}` — Substitutes any number in range from N to M including both borders. N and M can have leading zeroes e.g. `000..078`. + +Constructions with `{}` are similar to the [remote](../../../sql-reference/table-functions/remote.md) table function. + +:::note +If the listing of files contains number ranges with leading zeros, use the construction with braces for each digit separately or use `?`. +::: + From d7b4d9b6c0bcb5bd5d33d11cc9471f40031e25e7 Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Mon, 15 May 2023 15:27:44 +0300 Subject: [PATCH 19/34] add new table engine to index --- docs/en/engines/table-engines/index.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/engines/table-engines/index.md b/docs/en/engines/table-engines/index.md index d7c582164de..990e6986cbf 100644 --- a/docs/en/engines/table-engines/index.md +++ b/docs/en/engines/table-engines/index.md @@ -59,6 +59,7 @@ Engines in the family: - [EmbeddedRocksDB](../../engines/table-engines/integrations/embedded-rocksdb.md) - [RabbitMQ](../../engines/table-engines/integrations/rabbitmq.md) - [PostgreSQL](../../engines/table-engines/integrations/postgresql.md) +- [S3Queue](../../engines/table-engines/integrations/s3queue.md) ### Special Engines {#special-engines} From bc7c67e33f20a07602715788a1426492c5a26349 Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Thu, 25 May 2023 23:29:16 +0300 Subject: [PATCH 20/34] code review fixes --- .../table-engines/integrations/s3queue.md | 49 +++++++-- src/Core/Settings.h | 2 +- src/Storages/S3Queue/S3QueueHolder.cpp | 99 +++++++++---------- src/Storages/S3Queue/S3QueueHolder.h | 19 ++-- src/Storages/S3Queue/S3QueueSettings.h | 4 +- src/Storages/S3Queue/S3QueueTableMetadata.cpp | 62 ++++++------ src/Storages/S3Queue/S3QueueTableMetadata.h | 7 +- src/Storages/S3Queue/StorageS3Queue.cpp | 14 +-- src/Storages/S3Queue/StorageS3Queue.h | 4 +- .../integration/test_storage_s3_queue/test.py | 6 +- 10 files changed, 149 insertions(+), 117 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/s3queue.md b/docs/en/engines/table-engines/integrations/s3queue.md index 46cecd35a48..9fd6c88a966 100644 --- a/docs/en/engines/table-engines/integrations/s3queue.md +++ b/docs/en/engines/table-engines/integrations/s3queue.md @@ -20,8 +20,8 @@ CREATE TABLE s3_queue_engine_table (name String, value UInt32) [s3queue_polling_min_timeout_ms = 1000,] [s3queue_polling_max_timeout_ms = 10000,] [s3queue_polling_backoff_ms = 0,] - [s3queue_max_set_size = 1000,] - [s3queue_max_set_age_s = 0,] + [s3queue_tracked_files_limit = 1000,] + [s3queue_tracked_file_ttl_sec = 0,] [s3queue_polling_size = 50,] ``` @@ -33,13 +33,40 @@ CREATE TABLE s3_queue_engine_table (name String, value UInt32) - `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file. For more information see [Using S3 for Data Storage](../mergetree-family/mergetree.md#table_engine-mergetree-s3). - `compression` — Compression type. Supported values: `none`, `gzip/gz`, `brotli/br`, `xz/LZMA`, `zstd/zst`. Parameter is optional. By default, it will autodetect compression by file extension. +**Example** + +```sql +CREATE TABLE s3queue_engine_table (name String, value UInt32) +ENGINE=S3Queue('https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/*', 'CSV', 'gzip') +SETTINGS + mode = 'ordred'; +``` + +Using named collections: + +``` xml + + + + 'https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/* + test + test + + + +``` + +```sql +CREATE TABLE s3queue_engine_table (name String, value UInt32) +ENGINE=S3Queue(s3queue_conf, 'CSV', 'gzip') +SETTINGS + mode = 'ordred'; +``` ## Settings {#s3queue-settings} ### mode {#mode} -Allows to automatically close the connection after query execution, i.e. disable connection reuse. - Possible values: - unordered — With unordered mode, the set of all already processed files is tracked with persistent nodes in ZooKepeer. @@ -64,7 +91,7 @@ Possible values: - String. -Default value: ``. +Default value: `/`. ### s3queue_loading_retries {#s3queue_loading_retries} @@ -105,9 +132,10 @@ Possible values: Default value: `0`. -### s3queue_max_set_size {#s3queue_max_set_size} +### s3queue_tracked_files_limit {#s3queue_tracked_files_limit} -Max set size for tracking processed files in unordered mode in ZooKeeper. +Allows to limit the number of Zookeeper nodes if the 'unordered' mode is used, does nothing for 'ordered' mode. +If limit reached the oldest processed files will be deleted from ZooKeeper node and processed again. Possible values: @@ -115,9 +143,10 @@ Possible values: Default value: `1000`. -### s3queue_max_set_age_s {#s3queue_max_set_age_s} +### s3queue_tracked_file_ttl_sec {#s3queue_tracked_file_ttl_sec} -Maximum number of seconds to store processed files in ZooKeeper node (store forever by default). +Maximum number of seconds to store processed files in ZooKeeper node (store forever by default) for 'unordered' mode, does nothing for 'ordered' mode. +After the specified number of seconds, the file will be re-imported. Possible values: @@ -128,6 +157,8 @@ Default value: `0`. ### s3queue_polling_size {#s3queue_polling_size} Maximum files to fetch from S3 with SELECT or in background task. +Engine takes files for processing from S3 in batches. +We limit the batch size to increase concurrency if multiple table engines with the same `keeper_path` consume files from the same path. Possible values: diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 53648633630..18bbfa5d358 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -94,7 +94,7 @@ class IColumn; M(Bool, s3_allow_parallel_part_upload, true, "Use multiple threads for s3 multipart upload. It may lead to slightly higher memory usage", 0) \ M(Bool, s3_throw_on_zero_files_match, false, "Throw an error, when ListObjects request cannot match any files", 0) \ M(Bool, enable_s3_requests_logging, false, "Enable very explicit logging of S3 requests. Makes sense for debug only.", 0) \ - M(String, s3queue_default_zookeeper_path, "", "Default zookeeper path prefix for S3Queue engine", 0) \ + M(String, s3queue_default_zookeeper_path, "/", "Default zookeeper path prefix for S3Queue engine", 0) \ M(UInt64, hdfs_replication, 0, "The actual number of replications can be specified when the hdfs file is created.", 0) \ M(Bool, hdfs_truncate_on_insert, false, "Enables or disables truncate before insert in s3 engine tables", 0) \ M(Bool, hdfs_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in hdfs engine tables", 0) \ diff --git a/src/Storages/S3Queue/S3QueueHolder.cpp b/src/Storages/S3Queue/S3QueueHolder.cpp index 91bf82001bb..03efe6ef097 100644 --- a/src/Storages/S3Queue/S3QueueHolder.cpp +++ b/src/Storages/S3Queue/S3QueueHolder.cpp @@ -4,6 +4,7 @@ # include # include # include +# include # include # include # include @@ -22,16 +23,20 @@ namespace ErrorCodes void S3QueueHolder::S3QueueCollection::read(ReadBuffer & in) { + assertString("collection:\n", in); + files = {}; - in >> "collection:\n"; while (!in.eof()) { String file_name; Int64 timestamp; + UInt64 retries_count; in >> file_name >> "\n"; in >> timestamp >> "\n"; - auto pair = std::make_pair(file_name, timestamp); - files.push_back(pair); + in >> retries_count >> "\n"; + + TrackedCollectionItem item = {.file_path=file_name, .timestamp=timestamp, .retries_count=retries_count}; + files.push_back(item); } } @@ -40,8 +45,9 @@ void S3QueueHolder::S3QueueCollection::write(WriteBuffer & out) const out << "collection:\n"; for (const auto & processed_file : files) { - out << processed_file.first << "\n"; - out << processed_file.second << "\n"; + out << processed_file.file_path << "\n"; + out << processed_file.timestamp << "\n"; + out << processed_file.retries_count << "\n"; } } @@ -57,7 +63,7 @@ S3QueueHolder::S3FilesCollection S3QueueHolder::S3QueueCollection::getFileNames( S3FilesCollection keys = {}; for (const auto & pair : files) { - keys.insert(pair.first); + keys.insert(pair.file_path); } return keys; } @@ -80,8 +86,8 @@ void S3QueueHolder::S3QueueProcessedCollection::parse(const String & s) auto new_end = std::remove_if( files.begin(), files.end(), - [×tamp, &max_seconds_diff](std::pair processed_file) - { return (timestamp - processed_file.second) > max_seconds_diff; }); + [×tamp, &max_seconds_diff](TrackedCollectionItem processed_file) + { return (timestamp - processed_file.timestamp) > max_seconds_diff; }); files.erase(new_end, files.end()); } } @@ -90,8 +96,9 @@ void S3QueueHolder::S3QueueProcessedCollection::parse(const String & s) void S3QueueHolder::S3QueueProcessedCollection::add(const String & file_name) { Int64 timestamp = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); - auto pair = std::make_pair(file_name, timestamp); - files.push_back(pair); + + TrackedCollectionItem processed_file = {.file_path=file_name, .timestamp=timestamp}; + files.push_back(processed_file); // Check set size if (files.size() > max_size) @@ -115,19 +122,19 @@ void S3QueueHolder::S3QueueFailedCollection::parse(const String & s) bool S3QueueHolder::S3QueueFailedCollection::add(const String & file_name) { auto failed_it - = std::find_if(files.begin(), files.end(), [&file_name](const std::pair & s) { return s.first == file_name; }); + = std::find_if(files.begin(), files.end(), [&file_name](const TrackedCollectionItem & s) { return s.file_path == file_name; }); if (failed_it != files.end()) { - failed_it->second--; - if (failed_it->second == 0) + failed_it->retries_count--; + if (failed_it->retries_count == 0) { return false; } } else { - auto pair = std::make_pair(file_name, max_retries_count); - files.push_back(pair); + TrackedCollectionItem failed_file = {.file_path=file_name, .retries_count=max_retries_count}; + files.push_back(failed_file); } return true; } @@ -137,9 +144,9 @@ S3QueueHolder::S3FilesCollection S3QueueHolder::S3QueueFailedCollection::getFile S3FilesCollection failed_keys; for (const auto & pair : files) { - if (pair.second <= 0) + if (pair.retries_count <= 0) { - failed_keys.insert(pair.first); + failed_keys.insert(pair.file_path); } } return failed_keys; @@ -157,36 +164,29 @@ S3QueueHolder::S3QueueHolder( , max_set_age_s(max_set_age_s_) , max_loading_retries(max_loading_retries_) , zookeeper_path(zookeeper_path_) - , zookeeper_failed_path(zookeeper_path_ + "/failed") - , zookeeper_processing_path(zookeeper_path_ + "/processing") - , zookeeper_processed_path(zookeeper_path_ + "/processed") - , zookeeper_lock_path(zookeeper_path_ + "/lock") + , zookeeper_failed_path(fs::path(zookeeper_path_) / "failed") + , zookeeper_processing_path(fs::path(zookeeper_path_) / "processing") + , zookeeper_processed_path(fs::path(zookeeper_path_) / "processed") + , zookeeper_lock_path(fs::path(zookeeper_path_) / "lock") , mode(mode_) , log(&Poco::Logger::get("S3QueueHolder")) { current_zookeeper = getContext()->getZooKeeper(); + if (!current_zookeeper) + throw Exception(ErrorCodes::NO_ZOOKEEPER, "Cannot get ZooKeeper"); } -zkutil::ZooKeeperPtr S3QueueHolder::tryGetZooKeeper() const +zkutil::ZooKeeperPtr S3QueueHolder::getZooKeeper() const { std::lock_guard lock(current_zookeeper_mutex); return current_zookeeper; } -zkutil::ZooKeeperPtr S3QueueHolder::getZooKeeper() const -{ - auto res = tryGetZooKeeper(); - if (!res) - throw Exception(ErrorCodes::NO_ZOOKEEPER, "Cannot get ZooKeeper"); - return res; -} - - void S3QueueHolder::setFileProcessed(const String & file_path) { auto zookeeper = getZooKeeper(); - auto lock = AcquireLock(); + auto lock = acquireLock(); if (mode == S3QueueMode::UNORDERED) { @@ -196,9 +196,11 @@ void S3QueueHolder::setFileProcessed(const String & file_path) processed.add(file_path); zookeeper->set(zookeeper_processed_path, processed.toString()); } - else + else if (mode == S3QueueMode::ORDERED) { String max_file = getMaxProcessedFile(); + // Check that we set in ZooKeeper node only maximum processed file path. + // This check can be useful, when multiple table engines consume in ordered mode. if (max_file.compare(file_path) <= 0) { zookeeper->set(zookeeper_processed_path, file_path); @@ -211,7 +213,7 @@ void S3QueueHolder::setFileProcessed(const String & file_path) bool S3QueueHolder::markFailedAndCheckRetry(const String & file_path) { auto zookeeper = getZooKeeper(); - auto lock = AcquireLock(); + auto lock = acquireLock(); String failed_files = zookeeper->get(zookeeper_failed_path); auto failed_collection = S3QueueFailedCollection(max_loading_retries); @@ -245,22 +247,18 @@ String S3QueueHolder::getMaxProcessedFile() S3QueueHolder::S3FilesCollection S3QueueHolder::getProcessingFiles() { auto zookeeper = getZooKeeper(); - String processing = zookeeper->get(fs::path(zookeeper_processing_path)); + String processing = zookeeper->get(zookeeper_processing_path); return parseCollection(processing); } void S3QueueHolder::setFilesProcessing(Strings & file_paths) { auto zookeeper = getZooKeeper(); + std::unordered_set processing_files(file_paths.begin(), file_paths.end()); + processing_files.merge(getProcessingFiles()); - for (const auto & x : getProcessingFiles()) - { - if (!std::count(file_paths.begin(), file_paths.end(), x)) - { - file_paths.push_back(x); - } - } - zookeeper->set(fs::path(zookeeper_processing_path), toString(file_paths)); + Strings processing_file_paths(processing_files.begin(), processing_files.end()); + zookeeper->set(fs::path(zookeeper_processing_path), toString(processing_file_paths)); } S3QueueHolder::S3FilesCollection S3QueueHolder::getUnorderedProcessedFiles() @@ -301,16 +299,16 @@ void S3QueueHolder::removeProcessingFile(const String & file_path) { auto zookeeper = getZooKeeper(); String node_data; - Strings file_paths; String processing = zookeeper->get(zookeeper_processing_path); S3FilesCollection processing_files = parseCollection(processing); - file_paths.insert(file_paths.end(), processing_files.begin(), processing_files.end()); - file_paths.erase(std::remove(file_paths.begin(), file_paths.end(), file_path), file_paths.end()); + processing_files.erase(file_path); + + Strings file_paths(processing_files.begin(), processing_files.end()); zookeeper->set(fs::path(zookeeper_processing_path), toString(file_paths)); } -std::shared_ptr S3QueueHolder::AcquireLock() +std::shared_ptr S3QueueHolder::acquireLock() { auto zookeeper = getZooKeeper(); UInt32 retry_count = 200; @@ -323,7 +321,7 @@ std::shared_ptr S3QueueHolder::AcquireLock() if (code == Coordination::Error::ZNONODE || code == Coordination::Error::ZNODEEXISTS) { retries++; - if (retries >= retry_count) + if (retries > retry_count) { throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Can't acquire zookeeper lock"); } @@ -342,14 +340,15 @@ std::shared_ptr S3QueueHolder::AcquireLock() S3QueueHolder::S3FilesCollection S3QueueHolder::parseCollection(String & files) { - ReadBuffer rb(const_cast(reinterpret_cast(files.data())), files.length(), 0); + ReadBufferFromString rb(files); Strings deserialized; try { readQuoted(deserialized, rb); } - catch (...) + catch (const Exception & e) { + LOG_WARNING(log, "Can't parse collection from ZooKeeper node: {}", e.displayText()); deserialized = {}; } diff --git a/src/Storages/S3Queue/S3QueueHolder.h b/src/Storages/S3Queue/S3QueueHolder.h index f7fa0461a3a..77719c8b941 100644 --- a/src/Storages/S3Queue/S3QueueHolder.h +++ b/src/Storages/S3Queue/S3QueueHolder.h @@ -12,9 +12,15 @@ namespace DB class S3QueueHolder : public WithContext { public: + struct TrackedCollectionItem + { + String file_path; + Int64 timestamp = 0; + UInt64 retries_count = 0; + }; + using S3FilesCollection = std::unordered_set; - using ProcessedFiles = std::vector>; - using FailedFiles = std::vector>; + using TrackedFiles = std::vector; S3QueueHolder( const String & zookeeper_path_, @@ -30,7 +36,7 @@ public: S3FilesCollection getExcludedFiles(); String getMaxProcessedFile(); - std::shared_ptr AcquireLock(); + std::shared_ptr acquireLock(); struct S3QueueCollection { @@ -42,7 +48,7 @@ public: virtual void parse(const String & s) = 0; protected: - ProcessedFiles files; + TrackedFiles files; void read(ReadBuffer & in); void write(WriteBuffer & out) const; @@ -72,7 +78,7 @@ public: S3FilesCollection getFilesWithoutRetries(); private: - const UInt64 max_retries_count; + UInt64 max_retries_count; }; @@ -93,7 +99,6 @@ private: const UUID table_uuid; Poco::Logger * log; - zkutil::ZooKeeperPtr tryGetZooKeeper() const; zkutil::ZooKeeperPtr getZooKeeper() const; S3FilesCollection getFailedFiles(); @@ -101,7 +106,7 @@ private: S3FilesCollection getUnorderedProcessedFiles(); void removeProcessingFile(const String & file_path); - static S3FilesCollection parseCollection(String & files); + S3FilesCollection parseCollection(String & files); }; diff --git a/src/Storages/S3Queue/S3QueueSettings.h b/src/Storages/S3Queue/S3QueueSettings.h index 3b4ea475887..75defc4a57f 100644 --- a/src/Storages/S3Queue/S3QueueSettings.h +++ b/src/Storages/S3Queue/S3QueueSettings.h @@ -23,9 +23,9 @@ class ASTStorage; M(UInt64, s3queue_polling_min_timeout_ms, 1000, "Minimal timeout before next polling", 0) \ M(UInt64, s3queue_polling_max_timeout_ms, 10000, "Maximum timeout before next polling", 0) \ M(UInt64, s3queue_polling_backoff_ms, 0, "Polling backoff", 0) \ - M(UInt64, s3queue_max_set_size, 1000, "Max set size for tracking processed files in unordered mode in ZooKeeper", 0) \ + M(UInt64, s3queue_tracked_files_limit, 1000, "Max set size for tracking processed files in unordered mode in ZooKeeper", 0) \ M(UInt64, \ - s3queue_max_set_age_s, \ + s3queue_tracked_file_ttl_sec, \ 0, \ "Maximum number of seconds to store processed files in ZooKeeper node (store forever by default)", \ 0) \ diff --git a/src/Storages/S3Queue/S3QueueTableMetadata.cpp b/src/Storages/S3Queue/S3QueueTableMetadata.cpp index 340890f75de..5cfd1ee2d35 100644 --- a/src/Storages/S3Queue/S3QueueTableMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueTableMetadata.cpp @@ -2,8 +2,9 @@ #if USE_AWS_S3 -# include -# include +# include +# include +# include # include # include # include @@ -22,42 +23,41 @@ S3QueueTableMetadata::S3QueueTableMetadata(const StorageS3::Configuration & conf format_name = configuration.format; after_processing = engine_settings.after_processing.toString(); mode = engine_settings.mode.toString(); - s3queue_max_set_size = engine_settings.s3queue_max_set_size; - s3queue_max_set_age_s = engine_settings.s3queue_max_set_age_s; + s3queue_tracked_files_limit = engine_settings.s3queue_tracked_files_limit; + s3queue_tracked_file_ttl_sec = engine_settings.s3queue_tracked_file_ttl_sec; } -void S3QueueTableMetadata::write(WriteBuffer & out) const -{ - out << "metadata format version: 1\n" - << "after processing: " << after_processing << "\n" - << "mode: " << mode << "\n" - << "s3queue_max_set_size: " << s3queue_max_set_size << "\n" - << "s3queue_max_set_age_s: " << s3queue_max_set_age_s << "\n" - << "format name: " << format_name << "\n"; -} String S3QueueTableMetadata::toString() const { - WriteBufferFromOwnString out; - write(out); - return out.str(); + Poco::JSON::Object json; + json.set("after_processing", after_processing); + json.set("mode", mode); + json.set("s3queue_tracked_files_limit", s3queue_tracked_files_limit); + json.set("s3queue_tracked_file_ttl_sec", s3queue_tracked_file_ttl_sec); + json.set("format_name", format_name); + + std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + oss.exceptions(std::ios::failbit); + Poco::JSON::Stringifier::stringify(json, oss); + return oss.str(); } -void S3QueueTableMetadata::read(ReadBuffer & in) +void S3QueueTableMetadata::read(const String & s) { - in >> "metadata format version: 1\n"; - in >> "after processing: " >> after_processing >> "\n"; - in >> "mode: " >> mode >> "\n"; - in >> "s3queue_max_set_size: " >> s3queue_max_set_size >> "\n"; - in >> "s3queue_max_set_age_s: " >> s3queue_max_set_age_s >> "\n"; - in >> "format name: " >> format_name >> "\n"; + Poco::JSON::Parser parser; + auto json = parser.parse(s).extract(); + after_processing = json->getValue("after_processing"); + mode = json->getValue("mode"); + s3queue_tracked_files_limit = json->getValue("s3queue_tracked_files_limit"); + s3queue_tracked_file_ttl_sec = json->getValue("s3queue_tracked_file_ttl_sec"); + format_name = json->getValue("format_name"); } S3QueueTableMetadata S3QueueTableMetadata::parse(const String & s) { S3QueueTableMetadata metadata; - ReadBufferFromString buf(s); - metadata.read(buf); + metadata.read(s); return metadata; } @@ -80,21 +80,21 @@ void S3QueueTableMetadata::checkImmutableFieldsEquals(const S3QueueTableMetadata DB::toString(from_zk.after_processing), DB::toString(after_processing)); - if (s3queue_max_set_size != from_zk.s3queue_max_set_size) + if (s3queue_tracked_files_limit != from_zk.s3queue_tracked_files_limit) throw Exception( ErrorCodes::METADATA_MISMATCH, "Existing table metadata in ZooKeeper differs in max set size. " "Stored in ZooKeeper: {}, local: {}", - from_zk.s3queue_max_set_size, - s3queue_max_set_size); + from_zk.s3queue_tracked_files_limit, + s3queue_tracked_files_limit); - if (s3queue_max_set_age_s != from_zk.s3queue_max_set_age_s) + if (s3queue_tracked_file_ttl_sec != from_zk.s3queue_tracked_file_ttl_sec) throw Exception( ErrorCodes::METADATA_MISMATCH, "Existing table metadata in ZooKeeper differs in max set age. " "Stored in ZooKeeper: {}, local: {}", - from_zk.s3queue_max_set_age_s, - s3queue_max_set_age_s); + from_zk.s3queue_tracked_file_ttl_sec, + s3queue_tracked_file_ttl_sec); if (format_name != from_zk.format_name) throw Exception( diff --git a/src/Storages/S3Queue/S3QueueTableMetadata.h b/src/Storages/S3Queue/S3QueueTableMetadata.h index 6e87528db37..a0a768b64f2 100644 --- a/src/Storages/S3Queue/S3QueueTableMetadata.h +++ b/src/Storages/S3Queue/S3QueueTableMetadata.h @@ -20,16 +20,15 @@ struct S3QueueTableMetadata String format_name; String after_processing; String mode; - UInt64 s3queue_max_set_size; - UInt64 s3queue_max_set_age_s; + UInt64 s3queue_tracked_files_limit; + UInt64 s3queue_tracked_file_ttl_sec; S3QueueTableMetadata() = default; S3QueueTableMetadata(const StorageS3::Configuration & configuration, const S3QueueSettings & engine_settings); - void read(ReadBuffer & in); + void read(const String & in); static S3QueueTableMetadata parse(const String & s); - void write(WriteBuffer & out) const; String toString() const; void checkEquals(const S3QueueTableMetadata & from_zk) const; diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 3da15fc62b9..df99167f180 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -112,8 +112,6 @@ StorageS3Queue::StorageS3Queue( , is_key_with_globs(s3_configuration.url.key.find_first_of("*?{") != std::string::npos) , log(&Poco::Logger::get("StorageS3Queue (" + table_id_.table_name + ")")) { - LOG_INFO(log, "Init engine"); - if (!is_key_with_globs) { throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "S3Queue engine can read only from url with globs"); @@ -138,7 +136,7 @@ StorageS3Queue::StorageS3Queue( LOG_INFO(log, "S3Queue engine keeper_path not specified. Use replicated database zookeeper path"); String base_zookeeper_path = assert_cast(database.get())->getZooKeeperPath(); zookeeper_path = zkutil::extractZooKeeperPath( - fs::path(base_zookeeper_path) / toString(table_id.uuid), /* check_starts_with_slash */ true, log); + fs::path(base_zookeeper_path) / "s3queue" / toString(table_id.uuid), /* check_starts_with_slash */ true, log); } else { @@ -184,8 +182,8 @@ StorageS3Queue::StorageS3Queue( zookeeper_path, mode, getContext(), - s3queue_settings->s3queue_max_set_size.value, - s3queue_settings->s3queue_max_set_age_s.value, + s3queue_settings->s3queue_tracked_files_limit.value, + s3queue_settings->s3queue_tracked_file_ttl_sec.value, s3queue_settings->s3queue_loading_retries.value); auto default_virtuals = NamesAndTypesList{ @@ -506,8 +504,6 @@ void StorageS3Queue::streamToViews() std::atomic_size_t rows = 0; { block_io.pipeline.complete(std::move(pipe)); - block_io.pipeline.setNumThreads(1); - block_io.pipeline.setProgressCallback([&](const Progress & progress) { rows += progress.read_rows.load(); }); CompletedPipelineExecutor executor(block_io.pipeline); executor.execute(); @@ -546,7 +542,7 @@ bool StorageS3Queue::createTableIfNotExists(const StorageMetadataPtr & metadata_ auto zookeeper = getZooKeeper(); zookeeper->createAncestors(zookeeper_path); - for (size_t i = 0; i < 1000; ++i) + for (size_t i = 0; i < zk_create_table_retries; ++i) { Coordination::Requests ops; bool is_first_replica = true; @@ -638,7 +634,7 @@ StorageS3Queue::createFileIterator(ContextPtr local_context, ASTPtr query, KeysW read_keys, s3_configuration.request_settings); - auto zookeeper_lock = queue_holder->AcquireLock(); + auto zookeeper_lock = queue_holder->acquireLock(); S3QueueHolder::S3FilesCollection exclude = queue_holder->getExcludedFiles(); Strings processing_files; diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index 73f8075b7d1..b9aeac95f68 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -11,9 +11,9 @@ # include # include +# include # include # include -# include # include # include @@ -128,6 +128,8 @@ private: void setZooKeeper(); zkutil::ZooKeeperPtr tryGetZooKeeper() const; zkutil::ZooKeeperPtr getZooKeeper() const; + + const UInt32 zk_create_table_retries = 1000; bool createTableIfNotExists(const StorageMetadataPtr & metadata_snapshot); void checkTableStructure(const String & zookeeper_prefix, const StorageMetadataPtr & metadata_snapshot); const String & getZooKeeperPath() const { return zookeeper_path; } diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index 085f92035d0..798c5952682 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -621,8 +621,8 @@ def test_max_set_age(started_cluster): SETTINGS mode = 'unordered', keeper_path = '/clickhouse/test_set_age', - s3queue_max_set_size = 10, - s3queue_max_set_age_s = {max_age}; + s3queue_tracked_files_limit = 10, + s3queue_tracked_file_ttl_sec = {max_age}; """ ) @@ -820,7 +820,7 @@ def test_max_set_size(started_cluster): SETTINGS mode = 'unordered', keeper_path = '/clickhouse/test_set_size', - s3queue_max_set_size = {files_to_generate - 1}; + s3queue_tracked_files_limit = {files_to_generate - 1}; """ ) From 528fa9ae04ce5894b48d45dfabf089427205079a Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Thu, 25 May 2023 23:40:26 +0300 Subject: [PATCH 21/34] fix --- src/Storages/S3Queue/StorageS3Queue.cpp | 3 +-- src/Storages/S3Queue/StorageS3Queue.h | 2 +- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index df99167f180..593d5637ad9 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -109,10 +109,9 @@ StorageS3Queue::StorageS3Queue( , distributed_processing(distributed_processing_) , format_settings(format_settings_) , partition_by(partition_by_) - , is_key_with_globs(s3_configuration.url.key.find_first_of("*?{") != std::string::npos) , log(&Poco::Logger::get("StorageS3Queue (" + table_id_.table_name + ")")) { - if (!is_key_with_globs) + if (!withGlobs()) { throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "S3Queue engine can read only from url with globs"); } diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index b9aeac95f68..60857b6edad 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -95,9 +95,9 @@ private: const bool distributed_processing; std::optional format_settings; ASTPtr partition_by; - bool is_key_with_globs = false; bool supportsSubcolumns() const override; + bool withGlobs() const { return s3_configuration.url.key.find_first_of("*?{") != std::string::npos; } void threadFunc(); size_t getTableDependentCount() const; From 3629e0c980f967b41f6ddb1411d6954a147cad65 Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Fri, 26 May 2023 10:09:49 +0300 Subject: [PATCH 22/34] fix --- src/Storages/S3Queue/S3QueueSource.cpp | 6 ------ src/Storages/S3Queue/StorageS3Queue.cpp | 6 ------ src/Storages/S3Queue/StorageS3Queue.h | 3 +-- 3 files changed, 1 insertion(+), 14 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 9a9472d9578..85a1db2ad35 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -28,12 +28,6 @@ # include # include -# include -# include -# include - -# include - # include # include diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 593d5637ad9..69c6601365f 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -37,9 +37,6 @@ # include -# include -# include - # include # include @@ -93,7 +90,6 @@ StorageS3Queue::StorageS3Queue( const String & comment, ContextPtr context_, std::optional format_settings_, - bool distributed_processing_, ASTPtr partition_by_) : IStorage(table_id_) , WithContext(context_) @@ -106,7 +102,6 @@ StorageS3Queue::StorageS3Queue( , format_name(configuration_.format) , compression_method(configuration_.compression_method) , name(s3_configuration.url.storage_name) - , distributed_processing(distributed_processing_) , format_settings(format_settings_) , partition_by(partition_by_) , log(&Poco::Logger::get("StorageS3Queue (" + table_id_.table_name + ")")) @@ -709,7 +704,6 @@ void registerStorageS3QueueImpl(const String & name, StorageFactory & factory) args.comment, args.getContext(), format_settings, - /* distributed_processing_ */ false, partition_by); }, { diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index 60857b6edad..9dd1c6029f5 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -50,7 +50,6 @@ public: const String & comment, ContextPtr context_, std::optional format_settings_, - bool distributed_processing_ = false, ASTPtr partition_by_ = nullptr); String getName() const override { return "S3Queue"; } @@ -92,7 +91,7 @@ private: String format_name; String compression_method; String name; - const bool distributed_processing; + std::optional format_settings; ASTPtr partition_by; From 3e39ce79cf47c4f721a1cd066c9a29e5c6c66def Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Fri, 26 May 2023 21:02:32 +0300 Subject: [PATCH 23/34] fix build --- src/Storages/S3Queue/S3QueueTableMetadata.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/S3Queue/S3QueueTableMetadata.h b/src/Storages/S3Queue/S3QueueTableMetadata.h index a0a768b64f2..d11b3cad00a 100644 --- a/src/Storages/S3Queue/S3QueueTableMetadata.h +++ b/src/Storages/S3Queue/S3QueueTableMetadata.h @@ -26,7 +26,7 @@ struct S3QueueTableMetadata S3QueueTableMetadata() = default; S3QueueTableMetadata(const StorageS3::Configuration & configuration, const S3QueueSettings & engine_settings); - void read(const String & in); + void read(const String & s); static S3QueueTableMetadata parse(const String & s); String toString() const; From a347408e203eee4f0b03341eacc6780003027c27 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 17 Jul 2023 14:13:50 +0200 Subject: [PATCH 24/34] Fix style check --- docs/en/engines/table-engines/integrations/s3queue.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/s3queue.md b/docs/en/engines/table-engines/integrations/s3queue.md index 9fd6c88a966..f857df8b1c8 100644 --- a/docs/en/engines/table-engines/integrations/s3queue.md +++ b/docs/en/engines/table-engines/integrations/s3queue.md @@ -69,7 +69,7 @@ SETTINGS Possible values: -- unordered — With unordered mode, the set of all already processed files is tracked with persistent nodes in ZooKepeer. +- unordered — With unordered mode, the set of all already processed files is tracked with persistent nodes in ZooKeeper. - ordered — With ordered mode, only the max name of the successfully consumed file, and the names of files that will be retried after unsuccessful loading attempt are being stored in ZooKeeper. Default value: `unordered`. @@ -222,4 +222,3 @@ Constructions with `{}` are similar to the [remote](../../../sql-reference/table :::note If the listing of files contains number ranges with leading zeros, use the construction with braces for each digit separately or use `?`. ::: - From fe53cd3d7464f960c8b824e88ec98183f03b54e8 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 17 Jul 2023 16:11:58 +0200 Subject: [PATCH 25/34] Minor changes --- .../table-engines/integrations/s3queue.md | 2 +- src/Storages/S3Queue/S3QueueHolder.cpp | 19 +++++++++---------- src/Storages/S3Queue/S3QueueHolder.h | 6 +++--- src/Storages/S3Queue/S3QueueSource.cpp | 12 +++++++----- src/Storages/S3Queue/StorageS3Queue.cpp | 5 ++--- src/Storages/S3Queue/StorageS3Queue.h | 6 +++++- 6 files changed, 27 insertions(+), 23 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/s3queue.md b/docs/en/engines/table-engines/integrations/s3queue.md index f857df8b1c8..3e6cf88312f 100644 --- a/docs/en/engines/table-engines/integrations/s3queue.md +++ b/docs/en/engines/table-engines/integrations/s3queue.md @@ -58,7 +58,7 @@ Using named collections: ```sql CREATE TABLE s3queue_engine_table (name String, value UInt32) -ENGINE=S3Queue(s3queue_conf, 'CSV', 'gzip') +ENGINE=S3Queue(s3queue_conf, format = 'CSV', compression_method = 'gzip') SETTINGS mode = 'ordred'; ``` diff --git a/src/Storages/S3Queue/S3QueueHolder.cpp b/src/Storages/S3Queue/S3QueueHolder.cpp index 03efe6ef097..7a984cd418f 100644 --- a/src/Storages/S3Queue/S3QueueHolder.cpp +++ b/src/Storages/S3Queue/S3QueueHolder.cpp @@ -29,7 +29,7 @@ void S3QueueHolder::S3QueueCollection::read(ReadBuffer & in) while (!in.eof()) { String file_name; - Int64 timestamp; + UInt64 timestamp; UInt64 retries_count; in >> file_name >> "\n"; in >> timestamp >> "\n"; @@ -81,11 +81,10 @@ void S3QueueHolder::S3QueueProcessedCollection::parse(const String & s) // Remove old items if (max_age > 0) { - Int64 timestamp = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); - Int64 max_seconds_diff = max_age; + UInt64 timestamp = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); + UInt64 max_seconds_diff = max_age; auto new_end = std::remove_if( - files.begin(), - files.end(), + files.begin(), files.end(), [×tamp, &max_seconds_diff](TrackedCollectionItem processed_file) { return (timestamp - processed_file.timestamp) > max_seconds_diff; }); files.erase(new_end, files.end()); @@ -95,7 +94,7 @@ void S3QueueHolder::S3QueueProcessedCollection::parse(const String & s) void S3QueueHolder::S3QueueProcessedCollection::add(const String & file_name) { - Int64 timestamp = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); + UInt64 timestamp = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); TrackedCollectionItem processed_file = {.file_path=file_name, .timestamp=timestamp}; files.push_back(processed_file); @@ -157,11 +156,11 @@ S3QueueHolder::S3QueueHolder( const S3QueueMode & mode_, ContextPtr context_, UInt64 & max_set_size_, - UInt64 & max_set_age_s_, + UInt64 & max_set_age_sec_, UInt64 & max_loading_retries_) : WithContext(context_) , max_set_size(max_set_size_) - , max_set_age_s(max_set_age_s_) + , max_set_age_sec(max_set_age_sec_) , max_loading_retries(max_loading_retries_) , zookeeper_path(zookeeper_path_) , zookeeper_failed_path(fs::path(zookeeper_path_) / "failed") @@ -191,7 +190,7 @@ void S3QueueHolder::setFileProcessed(const String & file_path) if (mode == S3QueueMode::UNORDERED) { String processed_files = zookeeper->get(zookeeper_processed_path); - auto processed = S3QueueProcessedCollection(max_set_size, max_set_age_s); + auto processed = S3QueueProcessedCollection(max_set_size, max_set_age_sec); processed.parse(processed_files); processed.add(file_path); zookeeper->set(zookeeper_processed_path, processed.toString()); @@ -266,7 +265,7 @@ S3QueueHolder::S3FilesCollection S3QueueHolder::getUnorderedProcessedFiles() auto zookeeper = getZooKeeper(); String processed = zookeeper->get(zookeeper_processed_path); - auto collection = S3QueueProcessedCollection(max_set_size, max_set_age_s); + auto collection = S3QueueProcessedCollection(max_set_size, max_set_age_sec); collection.parse(processed); return collection.getFileNames(); diff --git a/src/Storages/S3Queue/S3QueueHolder.h b/src/Storages/S3Queue/S3QueueHolder.h index 77719c8b941..1292ebadc60 100644 --- a/src/Storages/S3Queue/S3QueueHolder.h +++ b/src/Storages/S3Queue/S3QueueHolder.h @@ -15,7 +15,7 @@ public: struct TrackedCollectionItem { String file_path; - Int64 timestamp = 0; + UInt64 timestamp = 0; UInt64 retries_count = 0; }; @@ -27,7 +27,7 @@ public: const S3QueueMode & mode_, ContextPtr context_, UInt64 & max_set_size_, - UInt64 & max_set_age_s_, + UInt64 & max_set_age_sec_, UInt64 & max_loading_retries_); void setFileProcessed(const String & file_path); @@ -84,7 +84,7 @@ public: private: const UInt64 max_set_size; - const UInt64 max_set_age_s; + const UInt64 max_set_age_sec; const UInt64 max_loading_retries; zkutil::ZooKeeperPtr current_zookeeper; diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 85a1db2ad35..151b63b8051 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -100,7 +100,7 @@ Strings StorageS3QueueSource::QueueGlobIterator::filterProcessingFiles( auto full_path = bucket + '/' + val.key; if (exclude_keys.find(full_path) != exclude_keys.end()) { - LOG_INFO(log, "Found in exclude keys {}", val.key); + LOG_TRACE(log, "Found in exclude keys {}", val.key); continue; } if ((engine_mode == S3QueueMode::ORDERED) && (full_path.compare(max_file) <= 0)) @@ -130,9 +130,9 @@ Strings StorageS3QueueSource::QueueGlobIterator::filterProcessingFiles( } Strings keys; - for (const auto & v : processing_keys) + for (const auto & key_info : processing_keys) { - keys.push_back(bucket + '/' + v.key); + keys.push_back(bucket + '/' + key_info.key); } processing_keys.push_back(KeyWithInfo()); @@ -253,8 +253,10 @@ Chunk StorageS3QueueSource::generate() size_t total_size = file_iterator->getTotalSize(); if (num_rows && total_size) { - updateRowsProgressApprox( - *this, chunk, total_size, total_rows_approx_accumulated, total_rows_count_times, total_rows_approx_max); + size_t chunk_size = reader.getFormat()->getApproxBytesReadForChunk(); + if (!chunk_size) + chunk_size = chunk.bytes(); + updateRowsProgressApprox(*this, num_rows, chunk_size, total_size, total_rows_approx_accumulated, total_rows_count_times, total_rows_approx_max); } for (const auto & virtual_column : requested_virtual_columns) diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 69c6601365f..71a93884896 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -284,13 +284,12 @@ Pipe StorageS3Queue::read( max_download_threads)); } -SinkToStoragePtr StorageS3Queue::write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr /*context*/) +SinkToStoragePtr StorageS3Queue::write(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, bool) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Write is not supported by storage {}", getName()); } -void StorageS3Queue::truncate( - const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr /*local_context*/, TableExclusiveLockHolder &) +void StorageS3Queue::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Truncate is not supported by storage {}", getName()); } diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index 9dd1c6029f5..f9500371489 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -63,7 +63,11 @@ public: size_t max_block_size, size_t num_streams) override; - SinkToStoragePtr write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr /*context*/) override; + SinkToStoragePtr write( + const ASTPtr & query, + const StorageMetadataPtr & metadata_snapshot, + ContextPtr context, + bool async_insert) override; void truncate( const ASTPtr & /*query*/, From d2195cff1101f5035148e9c0f7672bfb83eeb693 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 20 Jul 2023 18:21:37 +0200 Subject: [PATCH 26/34] Small fixes --- src/Storages/S3Queue/S3QueueHolder.cpp | 15 +++++++-------- src/Storages/S3Queue/S3QueueHolder.h | 8 ++++---- src/Storages/S3Queue/S3QueueTableMetadata.cpp | 8 ++++---- src/Storages/S3Queue/S3QueueTableMetadata.h | 4 ++-- 4 files changed, 17 insertions(+), 18 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueHolder.cpp b/src/Storages/S3Queue/S3QueueHolder.cpp index 7a984cd418f..0900c84a2e8 100644 --- a/src/Storages/S3Queue/S3QueueHolder.cpp +++ b/src/Storages/S3Queue/S3QueueHolder.cpp @@ -74,9 +74,9 @@ S3QueueHolder::S3QueueProcessedCollection::S3QueueProcessedCollection(const UInt { } -void S3QueueHolder::S3QueueProcessedCollection::parse(const String & s) +void S3QueueHolder::S3QueueProcessedCollection::parse(const String & collection_str) { - ReadBufferFromString buf(s); + ReadBufferFromString buf(collection_str); read(buf); // Remove old items if (max_age > 0) @@ -111,9 +111,9 @@ S3QueueHolder::S3QueueFailedCollection::S3QueueFailedCollection(const UInt64 & m { } -void S3QueueHolder::S3QueueFailedCollection::parse(const String & s) +void S3QueueHolder::S3QueueFailedCollection::parse(const String & collection_str) { - ReadBufferFromString buf(s); + ReadBufferFromString buf(collection_str); read(buf); } @@ -124,8 +124,7 @@ bool S3QueueHolder::S3QueueFailedCollection::add(const String & file_name) = std::find_if(files.begin(), files.end(), [&file_name](const TrackedCollectionItem & s) { return s.file_path == file_name; }); if (failed_it != files.end()) { - failed_it->retries_count--; - if (failed_it->retries_count == 0) + if (failed_it->retries_count == 0 || --failed_it->retries_count == 0) { return false; } @@ -138,7 +137,7 @@ bool S3QueueHolder::S3QueueFailedCollection::add(const String & file_name) return true; } -S3QueueHolder::S3FilesCollection S3QueueHolder::S3QueueFailedCollection::getFilesWithoutRetries() +S3QueueHolder::S3FilesCollection S3QueueHolder::S3QueueFailedCollection::getFileNames() { S3FilesCollection failed_keys; for (const auto & pair : files) @@ -233,7 +232,7 @@ S3QueueHolder::S3FilesCollection S3QueueHolder::getFailedFiles() auto failed_collection = S3QueueFailedCollection(max_loading_retries); failed_collection.parse(failed_files); - return failed_collection.getFilesWithoutRetries(); + return failed_collection.getFileNames(); } String S3QueueHolder::getMaxProcessedFile() diff --git a/src/Storages/S3Queue/S3QueueHolder.h b/src/Storages/S3Queue/S3QueueHolder.h index 1292ebadc60..83edd8a2e74 100644 --- a/src/Storages/S3Queue/S3QueueHolder.h +++ b/src/Storages/S3Queue/S3QueueHolder.h @@ -45,7 +45,7 @@ public: String toString() const; S3FilesCollection getFileNames(); - virtual void parse(const String & s) = 0; + virtual void parse(const String & collection_str) = 0; protected: TrackedFiles files; @@ -59,7 +59,7 @@ public: public: S3QueueProcessedCollection(const UInt64 & max_size_, const UInt64 & max_age_); - void parse(const String & s) override; + void parse(const String & collection_str) override; void add(const String & file_name); private: @@ -72,10 +72,10 @@ public: public: S3QueueFailedCollection(const UInt64 & max_retries_count_); - void parse(const String & s) override; + void parse(const String & collection_str) override; bool add(const String & file_name); - S3FilesCollection getFilesWithoutRetries(); + S3FilesCollection getFileNames(); private: UInt64 max_retries_count; diff --git a/src/Storages/S3Queue/S3QueueTableMetadata.cpp b/src/Storages/S3Queue/S3QueueTableMetadata.cpp index 5cfd1ee2d35..23eebb6ded9 100644 --- a/src/Storages/S3Queue/S3QueueTableMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueTableMetadata.cpp @@ -43,10 +43,10 @@ String S3QueueTableMetadata::toString() const return oss.str(); } -void S3QueueTableMetadata::read(const String & s) +void S3QueueTableMetadata::read(const String & metadata_str) { Poco::JSON::Parser parser; - auto json = parser.parse(s).extract(); + auto json = parser.parse(metadata_str).extract(); after_processing = json->getValue("after_processing"); mode = json->getValue("mode"); s3queue_tracked_files_limit = json->getValue("s3queue_tracked_files_limit"); @@ -54,10 +54,10 @@ void S3QueueTableMetadata::read(const String & s) format_name = json->getValue("format_name"); } -S3QueueTableMetadata S3QueueTableMetadata::parse(const String & s) +S3QueueTableMetadata S3QueueTableMetadata::parse(const String & metadata_str) { S3QueueTableMetadata metadata; - metadata.read(s); + metadata.read(metadata_str); return metadata; } diff --git a/src/Storages/S3Queue/S3QueueTableMetadata.h b/src/Storages/S3Queue/S3QueueTableMetadata.h index d11b3cad00a..4b6fbc54825 100644 --- a/src/Storages/S3Queue/S3QueueTableMetadata.h +++ b/src/Storages/S3Queue/S3QueueTableMetadata.h @@ -26,8 +26,8 @@ struct S3QueueTableMetadata S3QueueTableMetadata() = default; S3QueueTableMetadata(const StorageS3::Configuration & configuration, const S3QueueSettings & engine_settings); - void read(const String & s); - static S3QueueTableMetadata parse(const String & s); + void read(const String & metadata_str); + static S3QueueTableMetadata parse(const String & metadata_str); String toString() const; From f82364d2c9bb9d9484e1f5c1648ef9f1284e3633 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 21 Jul 2023 17:03:30 +0200 Subject: [PATCH 27/34] Fix flaky test --- .../integration/test_storage_s3_queue/test.py | 61 +++++++++++-------- 1 file changed, 36 insertions(+), 25 deletions(-) diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index 798c5952682..f06772df9be 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -410,7 +410,7 @@ def test_streaming_to_view_(started_cluster, mode): DROP TABLE IF EXISTS test.s3_queue_persistent; DROP TABLE IF EXISTS test.s3_queue; DROP TABLE IF EXISTS test.persistent_s3_queue_mv; - + CREATE TABLE test.s3_queue_persistent ({table_format}) ENGINE = MergeTree() ORDER BY column1; @@ -461,15 +461,15 @@ def test_streaming_to_many_views(started_cluster, mode): DROP TABLE IF EXISTS test.persistent_s3_queue_mv_2; DROP TABLE IF EXISTS test.persistent_s3_queue_mv_3; - + CREATE TABLE test.s3_queue_persistent ({table_format}) ENGINE = MergeTree() ORDER BY column1; - + CREATE TABLE test.s3_queue_persistent_2 ({table_format}) ENGINE = MergeTree() ORDER BY column1; - + CREATE TABLE test.s3_queue_persistent_3 ({table_format}) ENGINE = MergeTree() ORDER BY column1; @@ -484,12 +484,12 @@ def test_streaming_to_many_views(started_cluster, mode): SELECT * FROM test.s3_queue; - + CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv_2 TO test.s3_queue_persistent_2 AS SELECT * FROM test.s3_queue; - + CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv_3 TO test.s3_queue_persistent_3 AS SELECT * @@ -530,7 +530,7 @@ def test_multiple_tables_meta_mismatch(started_cluster): instance.query( f""" DROP TABLE IF EXISTS test.s3_queue; - + CREATE TABLE test.s3_queue ({table_format}) ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') SETTINGS @@ -615,7 +615,7 @@ def test_max_set_age(started_cluster): instance.query( f""" DROP TABLE IF EXISTS test.s3_queue; - + CREATE TABLE test.s3_queue ({table_format}) ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') SETTINGS @@ -672,14 +672,14 @@ def test_multiple_tables_streaming_sync(started_cluster, mode): mode = '{mode}', keeper_path = '/clickhouse/test_multiple_consumers_sync_{mode}', s3queue_polling_size = {poll_size}; - + CREATE TABLE test.s3_queue_copy ({table_format}) ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') SETTINGS mode = '{mode}', keeper_path = '/clickhouse/test_multiple_consumers_sync_{mode}', s3queue_polling_size = {poll_size}; - + CREATE TABLE test.s3_queue_copy_2 ({table_format}) ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') SETTINGS @@ -690,20 +690,20 @@ def test_multiple_tables_streaming_sync(started_cluster, mode): CREATE TABLE test.s3_queue_persistent ({table_format}) ENGINE = MergeTree() ORDER BY column1; - + CREATE TABLE test.s3_queue_persistent_copy ({table_format}) ENGINE = MergeTree() ORDER BY column1; - + CREATE TABLE test.s3_queue_persistent_copy_2 ({table_format}) ENGINE = MergeTree() ORDER BY column1; - + CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv TO test.s3_queue_persistent AS SELECT * FROM test.s3_queue; - + CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv_copy TO test.s3_queue_persistent_copy AS SELECT * @@ -718,7 +718,18 @@ def test_multiple_tables_streaming_sync(started_cluster, mode): total_values = generate_random_files( files_to_generate, prefix, started_cluster, bucket, row_num=1 ) - time.sleep((files_to_generate // poll_size) * 2) + + def get_count(table_name): + return int(run_query(instance, f"SELECT count() FROM {table_name}")) + + for _ in range(100): + if ( + get_count("test.s3_queue_persistent") + + get_count("test.s3_queue_persistent_copy") + + get_count("test.s3_queue_persistent_copy_2") + ) == files_to_generate: + break + time.sleep(1) get_query = f"SELECT * FROM test.s3_queue_persistent" res1 = [ @@ -734,18 +745,18 @@ def test_multiple_tables_streaming_sync(started_cluster, mode): list(map(int, l.split())) for l in run_query(instance, get_query_copy_2).splitlines() ] - - # Checking that all engines have made progress - assert len(res1) > 0 - assert len(res2) > 0 - assert len(res3) > 0 - - # Checking that all files were processed only once - assert len(res1) + len(res2) + len(res3) == files_to_generate assert {tuple(v) for v in res1 + res2 + res3} == set( [tuple(i) for i in total_values] ) + # Checking that all files were processed only once + time.sleep(10) + assert ( + get_count("test.s3_queue_persistent") + + get_count("test.s3_queue_persistent_copy") + + get_count("test.s3_queue_persistent_copy_2") + ) == files_to_generate + @pytest.mark.parametrize("mode", AVAILABLE_MODES) def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): @@ -774,7 +785,7 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): CREATE TABLE test.s3_queue_persistent ({table_format}) ENGINE = MergeTree() ORDER BY column1; - + CREATE MATERIALIZED VIEW test.persistent_s3_queue_mv TO test.s3_queue_persistent AS SELECT * @@ -814,7 +825,7 @@ def test_max_set_size(started_cluster): instance.query( f""" DROP TABLE IF EXISTS test.s3_queue; - + CREATE TABLE test.s3_queue ({table_format}) ENGINE=S3Queue('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{prefix}/*', {AUTH}'CSV') SETTINGS From f207e5bf1184e75c2769b687b9f5c08f85104384 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 24 Jul 2023 00:31:37 +0200 Subject: [PATCH 28/34] Add TODO --- src/Storages/S3Queue/S3QueueHolder.cpp | 3 +++ src/Storages/S3Queue/S3QueueSource.cpp | 2 ++ 2 files changed, 5 insertions(+) diff --git a/src/Storages/S3Queue/S3QueueHolder.cpp b/src/Storages/S3Queue/S3QueueHolder.cpp index 0900c84a2e8..7997fd2e051 100644 --- a/src/Storages/S3Queue/S3QueueHolder.cpp +++ b/src/Storages/S3Queue/S3QueueHolder.cpp @@ -49,6 +49,7 @@ void S3QueueHolder::S3QueueCollection::write(WriteBuffer & out) const out << processed_file.timestamp << "\n"; out << processed_file.retries_count << "\n"; } + /// todo(kssenii): use a more flexible format? } String S3QueueHolder::S3QueueCollection::toString() const @@ -104,6 +105,7 @@ void S3QueueHolder::S3QueueProcessedCollection::add(const String & file_name) { files.erase(files.begin(), files.begin() + (files.size() - max_size)); } + /// todo(kssenii): use deque here } @@ -177,6 +179,7 @@ S3QueueHolder::S3QueueHolder( zkutil::ZooKeeperPtr S3QueueHolder::getZooKeeper() const { + /// todo(kssenii): current_zookeeper is not updated at all apart from in constructor, remove the lock? std::lock_guard lock(current_zookeeper_mutex); return current_zookeeper; } diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 151b63b8051..31eda8db31f 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -81,6 +81,7 @@ StorageS3QueueSource::QueueGlobIterator::QueueGlobIterator( , glob_iterator(std::make_unique( client_, globbed_uri_, query, virtual_header, context, read_keys_, request_settings_)) { + /// todo(kssenii): remove this loop, it should not be here while (true) { KeyWithInfo val = glob_iterator->next(); @@ -321,6 +322,7 @@ void StorageS3QueueSource::deleteProcessedObject(const String & file_path) { LOG_WARNING(log, "Delete processed file {} from bucket {}", file_path, bucket); S3::DeleteObjectRequest request; + /// todo(kssenii): looks incorrect String delete_key = file_path.substr(bucket.length() + 1); request.WithKey(delete_key).WithBucket(bucket); From 6e99e5c4537c087abb735fa9312b238ad7389dda Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 30 Jul 2023 13:27:01 +0200 Subject: [PATCH 29/34] Fix another flaky test --- .../integration/test_storage_s3_queue/test.py | 20 +++++++++++++++++-- 1 file changed, 18 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index f06772df9be..77e32e2922c 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -797,7 +797,17 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): files_to_generate, prefix, started_cluster, bucket, row_num=1 ) - time.sleep((files_to_generate // poll_size) * 2) + def get_count(node, table_name): + return int(run_query(node, f"SELECT count() FROM {table_name}")) + + for _ in range(100): + if ( + get_count(instance, "test.s3_queue_persistent") + + get_count(instance_2, "test.s3_queue_persistent") + ) == files_to_generate: + break + time.sleep(1) + get_query = f"SELECT * FROM test.s3_queue_persistent" res1 = [ list(map(int, l.split())) for l in run_query(instance, get_query).splitlines() @@ -810,10 +820,16 @@ def test_multiple_tables_streaming_sync_distributed(started_cluster, mode): assert len(res1) > 0 assert len(res2) > 0 - # Checking that all files were processed only once assert len(res1) + len(res2) == files_to_generate assert {tuple(v) for v in res1 + res2} == set([tuple(i) for i in total_values]) + # Checking that all files were processed only once + time.sleep(10) + assert ( + get_count(instance, "test.s3_queue_persistent") + + get_count(instance_2, "test.s3_queue_persistent") + ) == files_to_generate + def test_max_set_size(started_cluster): files_to_generate = 10 From 870a506a0bb477cd17b65c06b12a73ef4320958c Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 31 Jul 2023 20:07:23 +0200 Subject: [PATCH 30/34] Some fixes --- src/Core/Settings.h | 2 +- src/Storages/S3Queue/S3QueueHolder.cpp | 135 ++++++++++-------------- src/Storages/S3Queue/S3QueueHolder.h | 15 ++- src/Storages/S3Queue/S3QueueSource.cpp | 58 ++++------ src/Storages/S3Queue/S3QueueSource.h | 7 -- src/Storages/S3Queue/StorageS3Queue.cpp | 68 ++++++------ src/Storages/S3Queue/StorageS3Queue.h | 5 +- src/Storages/StorageS3.cpp | 2 +- src/Storages/StorageS3.h | 15 ++- 9 files changed, 131 insertions(+), 176 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index e6c756ec4ae..923576b5f24 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -104,7 +104,7 @@ class IColumn; M(UInt64, s3_retry_attempts, 10, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries", 0) \ M(UInt64, s3_request_timeout_ms, 3000, "Idleness timeout for sending and receiving data to/from S3. Fail if a single TCP read or write call blocks for this long.", 0) \ M(Bool, enable_s3_requests_logging, false, "Enable very explicit logging of S3 requests. Makes sense for debug only.", 0) \ - M(String, s3queue_default_zookeeper_path, "/", "Default zookeeper path prefix for S3Queue engine", 0) \ + M(String, s3queue_default_zookeeper_path, "/s3queue/", "Default zookeeper path prefix for S3Queue engine", 0) \ M(UInt64, hdfs_replication, 0, "The actual number of replications can be specified when the hdfs file is created.", 0) \ M(Bool, hdfs_truncate_on_insert, false, "Enables or disables truncate before insert in s3 engine tables", 0) \ M(Bool, hdfs_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in hdfs engine tables", 0) \ diff --git a/src/Storages/S3Queue/S3QueueHolder.cpp b/src/Storages/S3Queue/S3QueueHolder.cpp index 7997fd2e051..66f540ba24e 100644 --- a/src/Storages/S3Queue/S3QueueHolder.cpp +++ b/src/Storages/S3Queue/S3QueueHolder.cpp @@ -1,3 +1,4 @@ +#include "IO/VarInt.h" #include "config.h" #if USE_AWS_S3 @@ -23,33 +24,34 @@ namespace ErrorCodes void S3QueueHolder::S3QueueCollection::read(ReadBuffer & in) { - assertString("collection:\n", in); - files = {}; - while (!in.eof()) - { - String file_name; - UInt64 timestamp; - UInt64 retries_count; - in >> file_name >> "\n"; - in >> timestamp >> "\n"; - in >> retries_count >> "\n"; - TrackedCollectionItem item = {.file_path=file_name, .timestamp=timestamp, .retries_count=retries_count}; + if (in.eof()) + return; + + size_t files_num; + in >> files_num >> "\n"; + while (files_num--) + { + TrackedCollectionItem item; + in >> item.file_path >> "\n"; + in >> item.timestamp >> "\n"; + in >> item.retries_count >> "\n"; + in >> item.last_exception >> "\n"; files.push_back(item); } } void S3QueueHolder::S3QueueCollection::write(WriteBuffer & out) const { - out << "collection:\n"; + out << files.size() << "\n"; for (const auto & processed_file : files) { out << processed_file.file_path << "\n"; out << processed_file.timestamp << "\n"; out << processed_file.retries_count << "\n"; + out << processed_file.last_exception << "\n"; } - /// todo(kssenii): use a more flexible format? } String S3QueueHolder::S3QueueCollection::toString() const @@ -79,16 +81,14 @@ void S3QueueHolder::S3QueueProcessedCollection::parse(const String & collection_ { ReadBufferFromString buf(collection_str); read(buf); - // Remove old items - if (max_age > 0) + if (max_age > 0) // Remove old items { UInt64 timestamp = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); UInt64 max_seconds_diff = max_age; - auto new_end = std::remove_if( - files.begin(), files.end(), - [×tamp, &max_seconds_diff](TrackedCollectionItem processed_file) + std::erase_if( + files, + [×tamp, &max_seconds_diff](const TrackedCollectionItem & processed_file) { return (timestamp - processed_file.timestamp) > max_seconds_diff; }); - files.erase(new_end, files.end()); } } @@ -100,12 +100,10 @@ void S3QueueHolder::S3QueueProcessedCollection::add(const String & file_name) TrackedCollectionItem processed_file = {.file_path=file_name, .timestamp=timestamp}; files.push_back(processed_file); - // Check set size - if (files.size() > max_size) + while (files.size() > max_size) { - files.erase(files.begin(), files.begin() + (files.size() - max_size)); + files.pop_front(); } - /// todo(kssenii): use deque here } @@ -120,7 +118,7 @@ void S3QueueHolder::S3QueueFailedCollection::parse(const String & collection_str } -bool S3QueueHolder::S3QueueFailedCollection::add(const String & file_name) +bool S3QueueHolder::S3QueueFailedCollection::add(const String & file_name, const String & exception_message) { auto failed_it = std::find_if(files.begin(), files.end(), [&file_name](const TrackedCollectionItem & s) { return s.file_path == file_name; }); @@ -133,7 +131,7 @@ bool S3QueueHolder::S3QueueFailedCollection::add(const String & file_name) } else { - TrackedCollectionItem failed_file = {.file_path=file_name, .retries_count=max_retries_count}; + TrackedCollectionItem failed_file = { .file_path=file_name, .retries_count=max_retries_count, .last_exception = exception_message }; files.push_back(failed_file); } return true; @@ -163,6 +161,7 @@ S3QueueHolder::S3QueueHolder( , max_set_size(max_set_size_) , max_set_age_sec(max_set_age_sec_) , max_loading_retries(max_loading_retries_) + , zk_client(getContext()->getZooKeeper()) , zookeeper_path(zookeeper_path_) , zookeeper_failed_path(fs::path(zookeeper_path_) / "failed") , zookeeper_processing_path(fs::path(zookeeper_path_) / "processing") @@ -171,31 +170,20 @@ S3QueueHolder::S3QueueHolder( , mode(mode_) , log(&Poco::Logger::get("S3QueueHolder")) { - current_zookeeper = getContext()->getZooKeeper(); - if (!current_zookeeper) - throw Exception(ErrorCodes::NO_ZOOKEEPER, "Cannot get ZooKeeper"); } -zkutil::ZooKeeperPtr S3QueueHolder::getZooKeeper() const -{ - /// todo(kssenii): current_zookeeper is not updated at all apart from in constructor, remove the lock? - std::lock_guard lock(current_zookeeper_mutex); - return current_zookeeper; -} - void S3QueueHolder::setFileProcessed(const String & file_path) { - auto zookeeper = getZooKeeper(); auto lock = acquireLock(); if (mode == S3QueueMode::UNORDERED) { - String processed_files = zookeeper->get(zookeeper_processed_path); + String processed_files = zk_client->get(zookeeper_processed_path); auto processed = S3QueueProcessedCollection(max_set_size, max_set_age_sec); processed.parse(processed_files); processed.add(file_path); - zookeeper->set(zookeeper_processed_path, processed.toString()); + zk_client->set(zookeeper_processed_path, processed.toString()); } else if (mode == S3QueueMode::ORDERED) { @@ -204,24 +192,22 @@ void S3QueueHolder::setFileProcessed(const String & file_path) // This check can be useful, when multiple table engines consume in ordered mode. if (max_file.compare(file_path) <= 0) { - zookeeper->set(zookeeper_processed_path, file_path); + zk_client->set(zookeeper_processed_path, file_path); } } removeProcessingFile(file_path); } -bool S3QueueHolder::markFailedAndCheckRetry(const String & file_path) +bool S3QueueHolder::setFileFailed(const String & file_path, const String & exception_message) { - auto zookeeper = getZooKeeper(); auto lock = acquireLock(); - String failed_files = zookeeper->get(zookeeper_failed_path); auto failed_collection = S3QueueFailedCollection(max_loading_retries); - failed_collection.parse(failed_files); - bool retry_later = failed_collection.add(file_path); + failed_collection.parse(zk_client->get(zookeeper_failed_path)); + bool retry_later = failed_collection.add(file_path, exception_message); - zookeeper->set(zookeeper_failed_path, failed_collection.toString()); + zk_client->set(zookeeper_failed_path, failed_collection.toString()); removeProcessingFile(file_path); return retry_later; @@ -229,8 +215,7 @@ bool S3QueueHolder::markFailedAndCheckRetry(const String & file_path) S3QueueHolder::S3FilesCollection S3QueueHolder::getFailedFiles() { - auto zookeeper = getZooKeeper(); - String failed_files = zookeeper->get(zookeeper_failed_path); + String failed_files = zk_client->get(zookeeper_failed_path); auto failed_collection = S3QueueFailedCollection(max_loading_retries); failed_collection.parse(failed_files); @@ -240,85 +225,79 @@ S3QueueHolder::S3FilesCollection S3QueueHolder::getFailedFiles() String S3QueueHolder::getMaxProcessedFile() { - auto zookeeper = getZooKeeper(); - String processed = zookeeper->get(zookeeper_processed_path); + String processed = zk_client->get(zookeeper_processed_path); return processed; } S3QueueHolder::S3FilesCollection S3QueueHolder::getProcessingFiles() { - auto zookeeper = getZooKeeper(); - String processing = zookeeper->get(zookeeper_processing_path); - return parseCollection(processing); + String processing_files; + if (!zk_client->tryGet(zookeeper_processing_path, processing_files)) + return {}; + return parseCollection(processing_files); } void S3QueueHolder::setFilesProcessing(Strings & file_paths) { - auto zookeeper = getZooKeeper(); std::unordered_set processing_files(file_paths.begin(), file_paths.end()); processing_files.merge(getProcessingFiles()); + String processing_files_str = toString(Strings(processing_files.begin(), processing_files.end())); - Strings processing_file_paths(processing_files.begin(), processing_files.end()); - zookeeper->set(fs::path(zookeeper_processing_path), toString(processing_file_paths)); + if (zk_client->exists(zookeeper_processing_path)) + zk_client->set(fs::path(zookeeper_processing_path), processing_files_str); + else + zk_client->create(fs::path(zookeeper_processing_path), processing_files_str, zkutil::CreateMode::Ephemeral); } S3QueueHolder::S3FilesCollection S3QueueHolder::getUnorderedProcessedFiles() { - auto zookeeper = getZooKeeper(); - - String processed = zookeeper->get(zookeeper_processed_path); + String processed = zk_client->get(zookeeper_processed_path); auto collection = S3QueueProcessedCollection(max_set_size, max_set_age_sec); collection.parse(processed); - return collection.getFileNames(); } -S3QueueHolder::S3FilesCollection S3QueueHolder::getExcludedFiles() +S3QueueHolder::S3FilesCollection S3QueueHolder::getProcessedAndFailedFiles() { - auto zookeeper = getZooKeeper(); - - S3FilesCollection exclude_files = getFailedFiles(); + S3FilesCollection processed_and_failed_files = getFailedFiles(); if (mode == S3QueueMode::UNORDERED) { - S3FilesCollection processed_files = getUnorderedProcessedFiles(); - exclude_files.merge(processed_files); + processed_and_failed_files.merge(getUnorderedProcessedFiles()); } else { String processed = getMaxProcessedFile(); - exclude_files.insert(processed); + processed_and_failed_files.insert(processed); } S3FilesCollection processing_files = getProcessingFiles(); - exclude_files.merge(processing_files); + processed_and_failed_files.merge(processing_files); - return exclude_files; + return processed_and_failed_files; } void S3QueueHolder::removeProcessingFile(const String & file_path) { - auto zookeeper = getZooKeeper(); String node_data; - String processing = zookeeper->get(zookeeper_processing_path); + String processing = zk_client->get(zookeeper_processing_path); S3FilesCollection processing_files = parseCollection(processing); processing_files.erase(file_path); Strings file_paths(processing_files.begin(), processing_files.end()); - zookeeper->set(fs::path(zookeeper_processing_path), toString(file_paths)); + zk_client->set(fs::path(zookeeper_processing_path), toString(file_paths)); } std::shared_ptr S3QueueHolder::acquireLock() { - auto zookeeper = getZooKeeper(); UInt32 retry_count = 200; UInt32 sleep_ms = 100; UInt32 retries = 0; while (true) { - Coordination::Error code = zookeeper->tryCreate(zookeeper_lock_path, "", zkutil::CreateMode::Ephemeral); + Coordination::Error code = zk_client->tryCreate(zookeeper_lock_path, "", zkutil::CreateMode::Ephemeral); if (code == Coordination::Error::ZNONODE || code == Coordination::Error::ZNODEEXISTS) { retries++; @@ -334,14 +313,14 @@ std::shared_ptr S3QueueHolder::acquireLock() } else { - return zkutil::EphemeralNodeHolder::existing(zookeeper_lock_path, *zookeeper); + return zkutil::EphemeralNodeHolder::existing(zookeeper_lock_path, *zk_client); } } } -S3QueueHolder::S3FilesCollection S3QueueHolder::parseCollection(String & files) +S3QueueHolder::S3FilesCollection S3QueueHolder::parseCollection(const String & collection_str) { - ReadBufferFromString rb(files); + ReadBufferFromString rb(collection_str); Strings deserialized; try { @@ -353,9 +332,7 @@ S3QueueHolder::S3FilesCollection S3QueueHolder::parseCollection(String & files) deserialized = {}; } - std::unordered_set processed(deserialized.begin(), deserialized.end()); - - return processed; + return std::unordered_set(deserialized.begin(), deserialized.end()); } } diff --git a/src/Storages/S3Queue/S3QueueHolder.h b/src/Storages/S3Queue/S3QueueHolder.h index 83edd8a2e74..de7f1f56f9a 100644 --- a/src/Storages/S3Queue/S3QueueHolder.h +++ b/src/Storages/S3Queue/S3QueueHolder.h @@ -17,10 +17,11 @@ public: String file_path; UInt64 timestamp = 0; UInt64 retries_count = 0; + String last_exception; }; using S3FilesCollection = std::unordered_set; - using TrackedFiles = std::vector; + using TrackedFiles = std::deque; S3QueueHolder( const String & zookeeper_path_, @@ -31,9 +32,9 @@ public: UInt64 & max_loading_retries_); void setFileProcessed(const String & file_path); - bool markFailedAndCheckRetry(const String & file_path); + bool setFileFailed(const String & file_path, const String & exception_message); void setFilesProcessing(Strings & file_paths); - S3FilesCollection getExcludedFiles(); + S3FilesCollection getProcessedAndFailedFiles(); String getMaxProcessedFile(); std::shared_ptr acquireLock(); @@ -73,7 +74,7 @@ public: S3QueueFailedCollection(const UInt64 & max_retries_count_); void parse(const String & collection_str) override; - bool add(const String & file_name); + bool add(const String & file_name, const String & exception_message); S3FilesCollection getFileNames(); @@ -87,7 +88,7 @@ private: const UInt64 max_set_age_sec; const UInt64 max_loading_retries; - zkutil::ZooKeeperPtr current_zookeeper; + zkutil::ZooKeeperPtr zk_client; mutable std::mutex current_zookeeper_mutex; mutable std::mutex mutex; const String zookeeper_path; @@ -99,14 +100,12 @@ private: const UUID table_uuid; Poco::Logger * log; - zkutil::ZooKeeperPtr getZooKeeper() const; - S3FilesCollection getFailedFiles(); S3FilesCollection getProcessingFiles(); S3FilesCollection getUnorderedProcessedFiles(); void removeProcessingFile(const String & file_path); - S3FilesCollection parseCollection(String & files); + S3FilesCollection parseCollection(const String & collection_str); }; diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 31eda8db31f..4785a131f60 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -21,7 +21,6 @@ # include # include -# include # include # include # include @@ -74,21 +73,17 @@ StorageS3QueueSource::QueueGlobIterator::QueueGlobIterator( const Block & virtual_header, ContextPtr context, UInt64 & max_poll_size_, - StorageS3QueueSource::KeysWithInfo * read_keys_, const S3Settings::RequestSettings & request_settings_) : max_poll_size(max_poll_size_) - , bucket(globbed_uri_.bucket) , glob_iterator(std::make_unique( - client_, globbed_uri_, query, virtual_header, context, read_keys_, request_settings_)) + client_, globbed_uri_, query, virtual_header, context, nullptr, request_settings_)) { /// todo(kssenii): remove this loop, it should not be here while (true) { KeyWithInfo val = glob_iterator->next(); if (val.key.empty()) - { break; - } keys_buf.push_back(val); } } @@ -98,16 +93,17 @@ Strings StorageS3QueueSource::QueueGlobIterator::filterProcessingFiles( { for (const KeyWithInfo & val : keys_buf) { - auto full_path = bucket + '/' + val.key; + auto full_path = val.key; if (exclude_keys.find(full_path) != exclude_keys.end()) { - LOG_TRACE(log, "Found in exclude keys {}", val.key); + LOG_TEST(log, "File {} will be skipped, because it was found in exclude files list " + "(either already processed or failed to be processed)", val.key); continue; } + if ((engine_mode == S3QueueMode::ORDERED) && (full_path.compare(max_file) <= 0)) - { continue; - } + if ((processing_keys.size() < max_poll_size) || (engine_mode == S3QueueMode::ORDERED)) { processing_keys.push_back(val); @@ -124,6 +120,7 @@ Strings StorageS3QueueSource::QueueGlobIterator::filterProcessingFiles( processing_keys.begin(), processing_keys.end(), [](const KeyWithInfo & lhs, const KeyWithInfo & rhs) { return lhs.key.compare(rhs.key) < 0; }); + if (processing_keys.size() > max_poll_size) { processing_keys.erase(processing_keys.begin() + max_poll_size, processing_keys.end()); @@ -132,11 +129,9 @@ Strings StorageS3QueueSource::QueueGlobIterator::filterProcessingFiles( Strings keys; for (const auto & key_info : processing_keys) - { - keys.push_back(bucket + '/' + key_info.key); - } - processing_keys.push_back(KeyWithInfo()); + keys.push_back(key_info.key); + processing_keys.push_back(KeyWithInfo()); processing_iterator = processing_keys.begin(); return keys; } @@ -153,12 +148,6 @@ StorageS3QueueSource::KeyWithInfo StorageS3QueueSource::QueueGlobIterator::next( return KeyWithInfo(); } -size_t StorageS3QueueSource::QueueGlobIterator::getTotalSize() const -{ - return glob_iterator->getTotalSize(); -} - - Block StorageS3QueueSource::getHeader(Block sample_block, const std::vector & requested_virtual_columns) { for (const auto & virtual_column : requested_virtual_columns) @@ -232,6 +221,7 @@ String StorageS3QueueSource::getName() const Chunk StorageS3QueueSource::generate() { + auto file_progress = getContext()->getFileProgressCallback(); while (true) { if (isCancelled() || !reader) @@ -243,22 +233,12 @@ Chunk StorageS3QueueSource::generate() Chunk chunk; bool success_in_pulling = false; - String file_path; try { if (reader->pull(chunk)) { UInt64 num_rows = chunk.getNumRows(); - - file_path = reader.getPath(); - size_t total_size = file_iterator->getTotalSize(); - if (num_rows && total_size) - { - size_t chunk_size = reader.getFormat()->getApproxBytesReadForChunk(); - if (!chunk_size) - chunk_size = chunk.bytes(); - updateRowsProgressApprox(*this, num_rows, chunk_size, total_size, total_rows_approx_accumulated, total_rows_count_times, total_rows_approx_max); - } + auto file_path = reader.getPath(); for (const auto & virtual_column : requested_virtual_columns) { @@ -279,14 +259,13 @@ Chunk StorageS3QueueSource::generate() catch (const Exception & e) { LOG_ERROR(log, "Exception in chunk pulling: {} ", e.displayText()); - const auto & failed_file_path = reader.getPath(); - queue_holder->markFailedAndCheckRetry(failed_file_path); + queue_holder->setFileFailed(reader.getFile(), e.message()); success_in_pulling = false; } if (success_in_pulling) { - applyActionAfterProcessing(file_path); - queue_holder->setFileProcessed(file_path); + applyActionAfterProcessing(reader.getFile()); + queue_holder->setFileProcessed(reader.getFile()); return chunk; } @@ -296,6 +275,7 @@ Chunk StorageS3QueueSource::generate() if (!reader) break; + /// Even if task is finished the thread may be not freed in pool. /// So wait until it will be freed before scheduling a new task. internal_source->create_reader_pool.wait(); @@ -320,12 +300,10 @@ void StorageS3QueueSource::applyActionAfterProcessing(const String & file_path) void StorageS3QueueSource::deleteProcessedObject(const String & file_path) { - LOG_WARNING(log, "Delete processed file {} from bucket {}", file_path, bucket); - S3::DeleteObjectRequest request; - /// todo(kssenii): looks incorrect - String delete_key = file_path.substr(bucket.length() + 1); + LOG_INFO(log, "Delete processed file {} from bucket {}", file_path, bucket); - request.WithKey(delete_key).WithBucket(bucket); + S3::DeleteObjectRequest request; + request.WithKey(file_path).WithBucket(bucket); auto outcome = client->DeleteObject(request); if (!outcome.IsSuccess()) { diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index a55dd2cbe40..e2e472b5007 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -46,18 +46,15 @@ public: const Block & virtual_header, ContextPtr context, UInt64 & max_poll_size_, - KeysWithInfo * read_keys_ = nullptr, const S3Settings::RequestSettings & request_settings_ = {}); KeyWithInfo next() override; - size_t getTotalSize() const override; Strings filterProcessingFiles(const S3QueueMode & engine_mode, std::unordered_set & exclude_keys, const String & max_file = ""); private: UInt64 max_poll_size; - const String bucket; KeysWithInfo keys_buf; KeysWithInfo processing_keys; mutable std::mutex mutex; @@ -116,10 +113,6 @@ private: std::future reader_future; - UInt64 total_rows_approx_max = 0; - size_t total_rows_count_times = 0; - UInt64 total_rows_approx_accumulated = 0; - mutable std::mutex mutex; std::shared_ptr internal_source; diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 71a93884896..d60c9adb4c8 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -24,7 +24,6 @@ # include # include -# include # include # include # include @@ -107,44 +106,43 @@ StorageS3Queue::StorageS3Queue( , log(&Poco::Logger::get("StorageS3Queue (" + table_id_.table_name + ")")) { if (!withGlobs()) - { throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "S3Queue engine can read only from url with globs"); - } - String setting_zookeeper_path = s3queue_settings->keeper_path; - LOG_INFO(log, "Settings zookeeper_path={}", setting_zookeeper_path); + std::string setting_zookeeper_path = s3queue_settings->keeper_path; + std::string zk_path_prefix; if (setting_zookeeper_path.empty()) { - auto table_id = getStorageID(); - auto database = DatabaseCatalog::instance().getDatabase(table_id.database_name); + auto database = DatabaseCatalog::instance().getDatabase(table_id_.database_name); bool is_in_replicated_database = database->getEngineName() == "Replicated"; auto default_path = getContext()->getSettingsRef().s3queue_default_zookeeper_path.value; if (!default_path.empty()) { - zookeeper_path - = zkutil::extractZooKeeperPath(fs::path(default_path) / toString(table_id.uuid), /* check_starts_with_slash */ true, log); + zk_path_prefix = default_path; } else if (is_in_replicated_database) { - LOG_INFO(log, "S3Queue engine keeper_path not specified. Use replicated database zookeeper path"); - String base_zookeeper_path = assert_cast(database.get())->getZooKeeperPath(); - zookeeper_path = zkutil::extractZooKeeperPath( - fs::path(base_zookeeper_path) / "s3queue" / toString(table_id.uuid), /* check_starts_with_slash */ true, log); + LOG_INFO(log, "S3Queue engine zookeeper path is not specified. " + "Using replicated database zookeeper path"); + + zk_path_prefix = fs::path(assert_cast(database.get())->getZooKeeperPath()) / "s3queue"; } else { - throw Exception( - ErrorCodes::NO_ZOOKEEPER, - "S3Queue keeper_path engine setting not specified, s3queue_default_zookeeper_path_prefix not specified and table not in " - "replicated database."); + throw Exception(ErrorCodes::NO_ZOOKEEPER, + "S3Queue keeper_path engine setting not specified, " + "s3queue_default_zookeeper_path_prefix not specified"); } } else { - zookeeper_path = zkutil::extractZooKeeperPath(s3queue_settings->keeper_path, /* check_starts_with_slash */ true, log); + zk_path_prefix = s3queue_settings->keeper_path.value; } - LOG_INFO(log, "Set zookeeper_path={}", zookeeper_path); + + zookeeper_path = zkutil::extractZooKeeperPath( + fs::path(zk_path_prefix) / toString(table_id_.uuid), /* check_starts_with_slash */ true, log); + + LOG_INFO(log, "Using zookeeper path: {}", zookeeper_path); FormatFactory::instance().checkFormatName(format_name); context_->getGlobalContext()->getRemoteHostFilter().checkURL(s3_configuration.url.uri); @@ -550,8 +548,8 @@ bool StorageS3Queue::createTableIfNotExists(const StorageMetadataPtr & metadata_ { String metadata_str = S3QueueTableMetadata(s3_configuration, *s3queue_settings).toString(); ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path, "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processed", "collection:\n", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/failed", "collection:\n", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processed", "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/failed", "", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processing", "", zkutil::CreateMode::Ephemeral)); ops.emplace_back(zkutil::makeCreateRequest( zookeeper_path + "/columns", metadata_snapshot->getColumns().toString(), zkutil::CreateMode::Persistent)); @@ -612,11 +610,8 @@ void StorageS3Queue::checkTableStructure(const String & zookeeper_prefix, const std::shared_ptr -StorageS3Queue::createFileIterator(ContextPtr local_context, ASTPtr query, KeysWithInfo * read_keys) +StorageS3Queue::createFileIterator(ContextPtr local_context, ASTPtr query) { - /// Iterate through disclosed globs and make a source for each file - std::lock_guard lock{sync_mutex}; - auto it = std::make_shared( *s3_configuration.client, s3_configuration.url, @@ -624,26 +619,35 @@ StorageS3Queue::createFileIterator(ContextPtr local_context, ASTPtr query, KeysW virtual_block, local_context, s3queue_settings->s3queue_polling_size.value, - read_keys, s3_configuration.request_settings); - auto zookeeper_lock = queue_holder->acquireLock(); - S3QueueHolder::S3FilesCollection exclude = queue_holder->getExcludedFiles(); + auto lock = queue_holder->acquireLock(); + S3QueueHolder::S3FilesCollection files_to_skip = queue_holder->getProcessedAndFailedFiles(); - Strings processing_files; + Strings files_to_process; if (mode == S3QueueMode::UNORDERED) { - processing_files = it->filterProcessingFiles(mode, exclude); + files_to_process = it->filterProcessingFiles(mode, files_to_skip); } else { String max_processed_file = queue_holder->getMaxProcessedFile(); - processing_files = it->filterProcessingFiles(mode, exclude, max_processed_file); + files_to_process = it->filterProcessingFiles(mode, files_to_skip, max_processed_file); } - queue_holder->setFilesProcessing(processing_files); + + LOG_TEST(log, "Found files to process: {}", fmt::join(files_to_process, ", ")); + + queue_holder->setFilesProcessing(files_to_process); return it; } +void StorageS3Queue::drop() +{ + auto zk_client = getZooKeeper(); + if (zk_client->exists(zookeeper_path)) + zk_client->removeRecursive(zookeeper_path); +} + void registerStorageS3QueueImpl(const String & name, StorageFactory & factory) { factory.registerStorage( diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index f9500371489..6af38058dd8 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -109,9 +109,9 @@ private: std::atomic shutdown_called{false}; Poco::Logger * log; - void startup() override; void shutdown() override; + void drop() override; struct TaskContext { @@ -126,7 +126,6 @@ private: zkutil::ZooKeeperPtr current_zookeeper; mutable std::mutex current_zookeeper_mutex; - mutable std::mutex sync_mutex; void setZooKeeper(); zkutil::ZooKeeperPtr tryGetZooKeeper() const; @@ -140,7 +139,7 @@ private: using KeysWithInfo = StorageS3QueueSource::KeysWithInfo; std::shared_ptr - createFileIterator(ContextPtr local_context, ASTPtr query, KeysWithInfo * read_keys = nullptr); + createFileIterator(ContextPtr local_context, ASTPtr query); void streamToViews(); Configuration updateConfigurationAndGetCopy(ContextPtr local_context); diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index b52150250b8..7525edfc718 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -596,7 +596,7 @@ StorageS3Source::ReaderHolder StorageS3Source::createReader() auto pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); auto current_reader = std::make_unique(*pipeline); - return ReaderHolder{fs::path(bucket) / key_with_info.key, std::move(read_buf), std::move(input_format), std::move(pipeline), std::move(current_reader)}; + return ReaderHolder{key_with_info.key, bucket, std::move(read_buf), std::move(input_format), std::move(pipeline), std::move(current_reader)}; } std::future StorageS3Source::createReaderAsync() diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index dd2a750f2b2..e08c01d0c91 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -159,12 +159,14 @@ private: { public: ReaderHolder( - String path_, + String key_, + String bucket_, std::unique_ptr read_buf_, std::shared_ptr input_format_, std::unique_ptr pipeline_, std::unique_ptr reader_) - : path(std::move(path_)) + : key(std::move(key_)) + , bucket(std::move(bucket_)) , read_buf(std::move(read_buf_)) , input_format(std::move(input_format_)) , pipeline(std::move(pipeline_)) @@ -189,19 +191,22 @@ private: pipeline = std::move(other.pipeline); input_format = std::move(other.input_format); read_buf = std::move(other.read_buf); - path = std::move(other.path); + key = std::move(other.key); + bucket = std::move(other.bucket); 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; } + String getPath() const { return fs::path(bucket) / key; } + const String & getFile() const { return key; } const IInputFormat * getInputFormat() const { return input_format.get(); } private: - String path; + String key; + String bucket; std::unique_ptr read_buf; std::shared_ptr input_format; std::unique_ptr pipeline; From 75f6a50079dda28dc88b038b618175bfd8178ff3 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 31 Jul 2023 20:08:31 +0200 Subject: [PATCH 31/34] Add todo --- src/Storages/S3Queue/S3QueueHolder.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Storages/S3Queue/S3QueueHolder.cpp b/src/Storages/S3Queue/S3QueueHolder.cpp index 66f540ba24e..24bb31517a5 100644 --- a/src/Storages/S3Queue/S3QueueHolder.cpp +++ b/src/Storages/S3Queue/S3QueueHolder.cpp @@ -22,6 +22,8 @@ namespace ErrorCodes extern const int TIMEOUT_EXCEEDED; } +/// TODO: update zk session if expired + void S3QueueHolder::S3QueueCollection::read(ReadBuffer & in) { files = {}; From be458fd44b7eca63080fc821e9bf88c968a4eb28 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 1 Aug 2023 12:06:25 +0200 Subject: [PATCH 32/34] Fix style check --- src/Storages/S3Queue/S3QueueHolder.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/S3Queue/S3QueueHolder.cpp b/src/Storages/S3Queue/S3QueueHolder.cpp index 24bb31517a5..860484da671 100644 --- a/src/Storages/S3Queue/S3QueueHolder.cpp +++ b/src/Storages/S3Queue/S3QueueHolder.cpp @@ -18,7 +18,6 @@ namespace DB namespace ErrorCodes { - extern const int NO_ZOOKEEPER; extern const int TIMEOUT_EXCEEDED; } From 08f5ebf3e86528b900341fe5c09cc222a3584230 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 1 Aug 2023 12:19:15 +0200 Subject: [PATCH 33/34] Fix test --- src/Storages/S3Queue/StorageS3Queue.cpp | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index d60c9adb4c8..baacb3397c7 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -108,14 +108,15 @@ StorageS3Queue::StorageS3Queue( if (!withGlobs()) throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "S3Queue engine can read only from url with globs"); - std::string setting_zookeeper_path = s3queue_settings->keeper_path; - std::string zk_path_prefix; + String setting_zookeeper_path = s3queue_settings->keeper_path; if (setting_zookeeper_path.empty()) { auto database = DatabaseCatalog::instance().getDatabase(table_id_.database_name); bool is_in_replicated_database = database->getEngineName() == "Replicated"; auto default_path = getContext()->getSettingsRef().s3queue_default_zookeeper_path.value; + String zk_path_prefix; + if (!default_path.empty()) { zk_path_prefix = default_path; @@ -133,15 +134,16 @@ StorageS3Queue::StorageS3Queue( "S3Queue keeper_path engine setting not specified, " "s3queue_default_zookeeper_path_prefix not specified"); } + + zookeeper_path = zkutil::extractZooKeeperPath( + fs::path(zk_path_prefix) / toString(table_id_.uuid), /* check_starts_with_slash */ true, log); } else { - zk_path_prefix = s3queue_settings->keeper_path.value; + /// We do not add table uuid here on purpose. + zookeeper_path = zkutil::extractZooKeeperPath(s3queue_settings->keeper_path.value, /* check_starts_with_slash */ true, log); } - zookeeper_path = zkutil::extractZooKeeperPath( - fs::path(zk_path_prefix) / toString(table_id_.uuid), /* check_starts_with_slash */ true, log); - LOG_INFO(log, "Using zookeeper path: {}", zookeeper_path); FormatFactory::instance().checkFormatName(format_name); From a14a6b56b3b99ea7aa96078f51d93e6e250723fb Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 1 Aug 2023 14:31:05 +0200 Subject: [PATCH 34/34] Better --- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 351 ++++++++++++++++++ ...S3QueueHolder.h => S3QueueFilesMetadata.h} | 53 +-- src/Storages/S3Queue/S3QueueHolder.cpp | 341 ----------------- src/Storages/S3Queue/S3QueueSource.cpp | 8 +- src/Storages/S3Queue/S3QueueSource.h | 6 +- src/Storages/S3Queue/StorageS3Queue.cpp | 175 ++++----- src/Storages/S3Queue/StorageS3Queue.h | 48 ++- 7 files changed, 489 insertions(+), 493 deletions(-) create mode 100644 src/Storages/S3Queue/S3QueueFilesMetadata.cpp rename src/Storages/S3Queue/{S3QueueHolder.h => S3QueueFilesMetadata.h} (73%) delete mode 100644 src/Storages/S3Queue/S3QueueHolder.cpp diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp new file mode 100644 index 00000000000..514baeb7b07 --- /dev/null +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -0,0 +1,351 @@ +#include "IO/VarInt.h" +#include "config.h" + +#if USE_AWS_S3 +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int TIMEOUT_EXCEEDED; +} + +namespace +{ + UInt64 getCurrentTime() + { + return std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); + } +} + +void S3QueueFilesMetadata::S3QueueCollection::read(ReadBuffer & in) +{ + files = {}; + if (in.eof()) + return; + + size_t files_num; + in >> files_num >> "\n"; + while (files_num--) + { + TrackedCollectionItem item; + in >> item.file_path >> "\n"; + in >> item.timestamp >> "\n"; + in >> item.retries_count >> "\n"; + in >> item.last_exception >> "\n"; + files.push_back(item); + } +} + +void S3QueueFilesMetadata::S3QueueCollection::write(WriteBuffer & out) const +{ + out << files.size() << "\n"; + for (const auto & processed_file : files) + { + out << processed_file.file_path << "\n"; + out << processed_file.timestamp << "\n"; + out << processed_file.retries_count << "\n"; + out << processed_file.last_exception << "\n"; + } +} + +String S3QueueFilesMetadata::S3QueueCollection::toString() const +{ + WriteBufferFromOwnString out; + write(out); + return out.str(); +} + +S3QueueFilesMetadata::S3FilesCollection S3QueueFilesMetadata::S3QueueCollection::getFileNames() +{ + S3FilesCollection keys = {}; + for (const auto & pair : files) + keys.insert(pair.file_path); + return keys; +} + + +S3QueueFilesMetadata::S3QueueProcessedCollection::S3QueueProcessedCollection(const UInt64 & max_size_, const UInt64 & max_age_) + : max_size(max_size_), max_age(max_age_) +{ +} + +void S3QueueFilesMetadata::S3QueueProcessedCollection::parse(const String & collection_str) +{ + ReadBufferFromString buf(collection_str); + read(buf); + if (max_age > 0) // Remove old items + { + std::erase_if( + files, + [timestamp = getCurrentTime(), this](const TrackedCollectionItem & processed_file) + { return (timestamp - processed_file.timestamp) > max_age; }); + } +} + + +void S3QueueFilesMetadata::S3QueueProcessedCollection::add(const String & file_name) +{ + TrackedCollectionItem processed_file = { .file_path=file_name, .timestamp = getCurrentTime() }; + files.push_back(processed_file); + + /// TODO: it is strange that in parse() we take into account only max_age, but here only max_size. + while (files.size() > max_size) + { + files.pop_front(); + } +} + + +S3QueueFilesMetadata::S3QueueFailedCollection::S3QueueFailedCollection(const UInt64 & max_retries_count_) + : max_retries_count(max_retries_count_) +{ +} + +void S3QueueFilesMetadata::S3QueueFailedCollection::parse(const String & collection_str) +{ + ReadBufferFromString buf(collection_str); + read(buf); +} + + +bool S3QueueFilesMetadata::S3QueueFailedCollection::add(const String & file_name, const String & exception_message) +{ + auto failed_it = std::find_if( + files.begin(), files.end(), + [&file_name](const TrackedCollectionItem & s) { return s.file_path == file_name; }); + + if (failed_it == files.end()) + { + files.emplace_back(file_name, 0, max_retries_count, exception_message); + } + else if (failed_it->retries_count == 0 || --failed_it->retries_count == 0) + { + return false; + } + return true; +} + +S3QueueFilesMetadata::S3FilesCollection S3QueueFilesMetadata::S3QueueFailedCollection::getFileNames() +{ + S3FilesCollection failed_keys; + for (const auto & pair : files) + { + if (pair.retries_count == 0) + failed_keys.insert(pair.file_path); + } + return failed_keys; +} + +void S3QueueFilesMetadata::S3QueueProcessingCollection::parse(const String & collection_str) +{ + ReadBufferFromString rb(collection_str); + Strings result; + readQuoted(result, rb); + files = S3FilesCollection(result.begin(), result.end()); +} + +void S3QueueFilesMetadata::S3QueueProcessingCollection::add(const Strings & file_names) +{ + files.insert(file_names.begin(), file_names.end()); +} + +void S3QueueFilesMetadata::S3QueueProcessingCollection::remove(const String & file_name) +{ + files.erase(file_name); +} + +String S3QueueFilesMetadata::S3QueueProcessingCollection::toString() const +{ + return DB::toString(Strings(files.begin(), files.end())); +} + + +S3QueueFilesMetadata::S3QueueFilesMetadata( + const StorageS3Queue * storage_, + const S3QueueSettings & settings_) + : storage(storage_) + , mode(settings_.mode) + , max_set_size(settings_.s3queue_tracked_files_limit.value) + , max_set_age_sec(settings_.s3queue_tracked_file_ttl_sec.value) + , max_loading_retries(settings_.s3queue_loading_retries.value) + , zookeeper_processing_path(fs::path(storage->getZooKeeperPath()) / "processing") + , zookeeper_processed_path(fs::path(storage->getZooKeeperPath()) / "processed") + , zookeeper_failed_path(fs::path(storage->getZooKeeperPath()) / "failed") + , zookeeper_lock_path(fs::path(storage->getZooKeeperPath()) / "lock") + , log(&Poco::Logger::get("S3QueueFilesMetadata")) +{ +} + +void S3QueueFilesMetadata::setFileProcessed(const String & file_path) +{ + auto zookeeper = storage->getZooKeeper(); + auto lock = acquireLock(zookeeper); + + switch (mode) + { + case S3QueueMode::UNORDERED: + { + S3QueueProcessedCollection processed_files(max_set_size, max_set_age_sec); + processed_files.parse(zookeeper->get(zookeeper_processed_path)); + processed_files.add(file_path); + zookeeper->set(zookeeper_processed_path, processed_files.toString()); + break; + } + case S3QueueMode::ORDERED: + { + // Check that we set in ZooKeeper node only maximum processed file path. + // This check can be useful, when multiple table engines consume in ordered mode. + String max_file = getMaxProcessedFile(); + if (max_file.compare(file_path) <= 0) + zookeeper->set(zookeeper_processed_path, file_path); + break; + } + } + removeProcessingFile(file_path); +} + + +bool S3QueueFilesMetadata::setFileFailed(const String & file_path, const String & exception_message) +{ + auto zookeeper = storage->getZooKeeper(); + auto lock = acquireLock(zookeeper); + + S3QueueFailedCollection failed_collection(max_loading_retries); + failed_collection.parse(zookeeper->get(zookeeper_failed_path)); + const bool can_be_retried = failed_collection.add(file_path, exception_message); + zookeeper->set(zookeeper_failed_path, failed_collection.toString()); + removeProcessingFile(file_path); + return can_be_retried; +} + +S3QueueFilesMetadata::S3FilesCollection S3QueueFilesMetadata::getFailedFiles() +{ + auto zookeeper = storage->getZooKeeper(); + String failed_files = zookeeper->get(zookeeper_failed_path); + + S3QueueFailedCollection failed_collection(max_loading_retries); + failed_collection.parse(failed_files); + return failed_collection.getFileNames(); +} + +String S3QueueFilesMetadata::getMaxProcessedFile() +{ + auto zookeeper = storage->getZooKeeper(); + return zookeeper->get(zookeeper_processed_path); +} + +S3QueueFilesMetadata::S3FilesCollection S3QueueFilesMetadata::getProcessingFiles() +{ + auto zookeeper = storage->getZooKeeper(); + String processing_files; + if (!zookeeper->tryGet(zookeeper_processing_path, processing_files)) + return {}; + + S3QueueProcessingCollection processing_collection; + if (!processing_files.empty()) + processing_collection.parse(processing_files); + return processing_collection.getFileNames(); +} + +void S3QueueFilesMetadata::setFilesProcessing(const Strings & file_paths) +{ + auto zookeeper = storage->getZooKeeper(); + String processing_files; + zookeeper->tryGet(zookeeper_processing_path, processing_files); + + S3QueueProcessingCollection processing_collection; + if (!processing_files.empty()) + processing_collection.parse(processing_files); + processing_collection.add(file_paths); + + if (zookeeper->exists(zookeeper_processing_path)) + zookeeper->set(zookeeper_processing_path, processing_collection.toString()); + else + zookeeper->create(zookeeper_processing_path, processing_collection.toString(), zkutil::CreateMode::Ephemeral); +} + +void S3QueueFilesMetadata::removeProcessingFile(const String & file_path) +{ + auto zookeeper = storage->getZooKeeper(); + String processing_files; + zookeeper->tryGet(zookeeper_processing_path, processing_files); + + S3QueueProcessingCollection processing_collection; + processing_collection.parse(processing_files); + processing_collection.remove(file_path); + zookeeper->set(zookeeper_processing_path, processing_collection.toString()); +} + +S3QueueFilesMetadata::S3FilesCollection S3QueueFilesMetadata::getUnorderedProcessedFiles() +{ + auto zookeeper = storage->getZooKeeper(); + S3QueueProcessedCollection processed_collection(max_set_size, max_set_age_sec); + processed_collection.parse(zookeeper->get(zookeeper_processed_path)); + return processed_collection.getFileNames(); +} + +S3QueueFilesMetadata::S3FilesCollection S3QueueFilesMetadata::getProcessedFailedAndProcessingFiles() +{ + S3FilesCollection processed_and_failed_files = getFailedFiles(); + switch (mode) + { + case S3QueueMode::UNORDERED: + { + processed_and_failed_files.merge(getUnorderedProcessedFiles()); + break; + } + case S3QueueMode::ORDERED: + { + processed_and_failed_files.insert(getMaxProcessedFile()); + break; + } + } + processed_and_failed_files.merge(getProcessingFiles()); + return processed_and_failed_files; +} + +std::shared_ptr S3QueueFilesMetadata::acquireLock(zkutil::ZooKeeperPtr zookeeper) +{ + UInt32 retry_count = 200; + UInt32 sleep_ms = 100; + UInt32 retries = 0; + + while (true) + { + Coordination::Error code = zookeeper->tryCreate(zookeeper_lock_path, "", zkutil::CreateMode::Ephemeral); + if (code == Coordination::Error::ZNONODE || code == Coordination::Error::ZNODEEXISTS) + { + retries++; + if (retries > retry_count) + { + throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Can't acquire zookeeper lock"); + } + sleepForMilliseconds(sleep_ms); + } + else if (code != Coordination::Error::ZOK) + { + throw Coordination::Exception(code, zookeeper_lock_path); + } + else + { + return zkutil::EphemeralNodeHolder::existing(zookeeper_lock_path, *zookeeper); + } + } +} + +} + +#endif diff --git a/src/Storages/S3Queue/S3QueueHolder.h b/src/Storages/S3Queue/S3QueueFilesMetadata.h similarity index 73% rename from src/Storages/S3Queue/S3QueueHolder.h rename to src/Storages/S3Queue/S3QueueFilesMetadata.h index de7f1f56f9a..c436de946ff 100644 --- a/src/Storages/S3Queue/S3QueueHolder.h +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.h @@ -9,7 +9,10 @@ namespace DB { -class S3QueueHolder : public WithContext +class StorageS3Queue; +struct S3QueueSettings; + +class S3QueueFilesMetadata { public: struct TrackedCollectionItem @@ -23,27 +26,21 @@ public: using S3FilesCollection = std::unordered_set; using TrackedFiles = std::deque; - S3QueueHolder( - const String & zookeeper_path_, - const S3QueueMode & mode_, - ContextPtr context_, - UInt64 & max_set_size_, - UInt64 & max_set_age_sec_, - UInt64 & max_loading_retries_); + S3QueueFilesMetadata(const StorageS3Queue * storage_, const S3QueueSettings & settings_); + void setFilesProcessing(const Strings & file_paths); void setFileProcessed(const String & file_path); bool setFileFailed(const String & file_path, const String & exception_message); - void setFilesProcessing(Strings & file_paths); - S3FilesCollection getProcessedAndFailedFiles(); - String getMaxProcessedFile(); - std::shared_ptr acquireLock(); + S3FilesCollection getProcessedFailedAndProcessingFiles(); + String getMaxProcessedFile(); + std::shared_ptr acquireLock(zkutil::ZooKeeperPtr zookeeper); struct S3QueueCollection { public: virtual ~S3QueueCollection() = default; - String toString() const; + virtual String toString() const; S3FilesCollection getFileNames(); virtual void parse(const String & collection_str) = 0; @@ -82,30 +79,42 @@ public: UInt64 max_retries_count; }; + struct S3QueueProcessingCollection + { + public: + S3QueueProcessingCollection() = default; + + void parse(const String & collection_str); + void add(const Strings & file_names); + void remove(const String & file_name); + + String toString() const; + const S3FilesCollection & getFileNames() const { return files; } + + private: + S3FilesCollection files; + }; private: + const StorageS3Queue * storage; + const S3QueueMode mode; const UInt64 max_set_size; const UInt64 max_set_age_sec; const UInt64 max_loading_retries; - zkutil::ZooKeeperPtr zk_client; - mutable std::mutex current_zookeeper_mutex; - mutable std::mutex mutex; - const String zookeeper_path; - const String zookeeper_failed_path; const String zookeeper_processing_path; const String zookeeper_processed_path; + const String zookeeper_failed_path; const String zookeeper_lock_path; - const S3QueueMode mode; - const UUID table_uuid; + + mutable std::mutex mutex; Poco::Logger * log; S3FilesCollection getFailedFiles(); S3FilesCollection getProcessingFiles(); S3FilesCollection getUnorderedProcessedFiles(); - void removeProcessingFile(const String & file_path); - S3FilesCollection parseCollection(const String & collection_str); + void removeProcessingFile(const String & file_path); }; diff --git a/src/Storages/S3Queue/S3QueueHolder.cpp b/src/Storages/S3Queue/S3QueueHolder.cpp deleted file mode 100644 index 860484da671..00000000000 --- a/src/Storages/S3Queue/S3QueueHolder.cpp +++ /dev/null @@ -1,341 +0,0 @@ -#include "IO/VarInt.h" -#include "config.h" - -#if USE_AWS_S3 -# include -# include -# include -# include -# include -# include -# include -# include -# include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int TIMEOUT_EXCEEDED; -} - -/// TODO: update zk session if expired - -void S3QueueHolder::S3QueueCollection::read(ReadBuffer & in) -{ - files = {}; - - if (in.eof()) - return; - - size_t files_num; - in >> files_num >> "\n"; - while (files_num--) - { - TrackedCollectionItem item; - in >> item.file_path >> "\n"; - in >> item.timestamp >> "\n"; - in >> item.retries_count >> "\n"; - in >> item.last_exception >> "\n"; - files.push_back(item); - } -} - -void S3QueueHolder::S3QueueCollection::write(WriteBuffer & out) const -{ - out << files.size() << "\n"; - for (const auto & processed_file : files) - { - out << processed_file.file_path << "\n"; - out << processed_file.timestamp << "\n"; - out << processed_file.retries_count << "\n"; - out << processed_file.last_exception << "\n"; - } -} - -String S3QueueHolder::S3QueueCollection::toString() const -{ - WriteBufferFromOwnString out; - write(out); - return out.str(); -} - -S3QueueHolder::S3FilesCollection S3QueueHolder::S3QueueCollection::getFileNames() -{ - S3FilesCollection keys = {}; - for (const auto & pair : files) - { - keys.insert(pair.file_path); - } - return keys; -} - - -S3QueueHolder::S3QueueProcessedCollection::S3QueueProcessedCollection(const UInt64 & max_size_, const UInt64 & max_age_) - : max_size(max_size_), max_age(max_age_) -{ -} - -void S3QueueHolder::S3QueueProcessedCollection::parse(const String & collection_str) -{ - ReadBufferFromString buf(collection_str); - read(buf); - if (max_age > 0) // Remove old items - { - UInt64 timestamp = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); - UInt64 max_seconds_diff = max_age; - std::erase_if( - files, - [×tamp, &max_seconds_diff](const TrackedCollectionItem & processed_file) - { return (timestamp - processed_file.timestamp) > max_seconds_diff; }); - } -} - - -void S3QueueHolder::S3QueueProcessedCollection::add(const String & file_name) -{ - UInt64 timestamp = std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); - - TrackedCollectionItem processed_file = {.file_path=file_name, .timestamp=timestamp}; - files.push_back(processed_file); - - while (files.size() > max_size) - { - files.pop_front(); - } -} - - -S3QueueHolder::S3QueueFailedCollection::S3QueueFailedCollection(const UInt64 & max_retries_count_) : max_retries_count(max_retries_count_) -{ -} - -void S3QueueHolder::S3QueueFailedCollection::parse(const String & collection_str) -{ - ReadBufferFromString buf(collection_str); - read(buf); -} - - -bool S3QueueHolder::S3QueueFailedCollection::add(const String & file_name, const String & exception_message) -{ - auto failed_it - = std::find_if(files.begin(), files.end(), [&file_name](const TrackedCollectionItem & s) { return s.file_path == file_name; }); - if (failed_it != files.end()) - { - if (failed_it->retries_count == 0 || --failed_it->retries_count == 0) - { - return false; - } - } - else - { - TrackedCollectionItem failed_file = { .file_path=file_name, .retries_count=max_retries_count, .last_exception = exception_message }; - files.push_back(failed_file); - } - return true; -} - -S3QueueHolder::S3FilesCollection S3QueueHolder::S3QueueFailedCollection::getFileNames() -{ - S3FilesCollection failed_keys; - for (const auto & pair : files) - { - if (pair.retries_count <= 0) - { - failed_keys.insert(pair.file_path); - } - } - return failed_keys; -} - -S3QueueHolder::S3QueueHolder( - const String & zookeeper_path_, - const S3QueueMode & mode_, - ContextPtr context_, - UInt64 & max_set_size_, - UInt64 & max_set_age_sec_, - UInt64 & max_loading_retries_) - : WithContext(context_) - , max_set_size(max_set_size_) - , max_set_age_sec(max_set_age_sec_) - , max_loading_retries(max_loading_retries_) - , zk_client(getContext()->getZooKeeper()) - , zookeeper_path(zookeeper_path_) - , zookeeper_failed_path(fs::path(zookeeper_path_) / "failed") - , zookeeper_processing_path(fs::path(zookeeper_path_) / "processing") - , zookeeper_processed_path(fs::path(zookeeper_path_) / "processed") - , zookeeper_lock_path(fs::path(zookeeper_path_) / "lock") - , mode(mode_) - , log(&Poco::Logger::get("S3QueueHolder")) -{ -} - - -void S3QueueHolder::setFileProcessed(const String & file_path) -{ - auto lock = acquireLock(); - - if (mode == S3QueueMode::UNORDERED) - { - String processed_files = zk_client->get(zookeeper_processed_path); - auto processed = S3QueueProcessedCollection(max_set_size, max_set_age_sec); - processed.parse(processed_files); - processed.add(file_path); - zk_client->set(zookeeper_processed_path, processed.toString()); - } - else if (mode == S3QueueMode::ORDERED) - { - String max_file = getMaxProcessedFile(); - // Check that we set in ZooKeeper node only maximum processed file path. - // This check can be useful, when multiple table engines consume in ordered mode. - if (max_file.compare(file_path) <= 0) - { - zk_client->set(zookeeper_processed_path, file_path); - } - } - removeProcessingFile(file_path); -} - - -bool S3QueueHolder::setFileFailed(const String & file_path, const String & exception_message) -{ - auto lock = acquireLock(); - - auto failed_collection = S3QueueFailedCollection(max_loading_retries); - failed_collection.parse(zk_client->get(zookeeper_failed_path)); - bool retry_later = failed_collection.add(file_path, exception_message); - - zk_client->set(zookeeper_failed_path, failed_collection.toString()); - removeProcessingFile(file_path); - - return retry_later; -} - -S3QueueHolder::S3FilesCollection S3QueueHolder::getFailedFiles() -{ - String failed_files = zk_client->get(zookeeper_failed_path); - - auto failed_collection = S3QueueFailedCollection(max_loading_retries); - failed_collection.parse(failed_files); - - return failed_collection.getFileNames(); -} - -String S3QueueHolder::getMaxProcessedFile() -{ - String processed = zk_client->get(zookeeper_processed_path); - return processed; -} - -S3QueueHolder::S3FilesCollection S3QueueHolder::getProcessingFiles() -{ - String processing_files; - if (!zk_client->tryGet(zookeeper_processing_path, processing_files)) - return {}; - return parseCollection(processing_files); -} - -void S3QueueHolder::setFilesProcessing(Strings & file_paths) -{ - std::unordered_set processing_files(file_paths.begin(), file_paths.end()); - processing_files.merge(getProcessingFiles()); - String processing_files_str = toString(Strings(processing_files.begin(), processing_files.end())); - - if (zk_client->exists(zookeeper_processing_path)) - zk_client->set(fs::path(zookeeper_processing_path), processing_files_str); - else - zk_client->create(fs::path(zookeeper_processing_path), processing_files_str, zkutil::CreateMode::Ephemeral); -} - -S3QueueHolder::S3FilesCollection S3QueueHolder::getUnorderedProcessedFiles() -{ - String processed = zk_client->get(zookeeper_processed_path); - auto collection = S3QueueProcessedCollection(max_set_size, max_set_age_sec); - collection.parse(processed); - return collection.getFileNames(); -} - -S3QueueHolder::S3FilesCollection S3QueueHolder::getProcessedAndFailedFiles() -{ - S3FilesCollection processed_and_failed_files = getFailedFiles(); - - if (mode == S3QueueMode::UNORDERED) - { - processed_and_failed_files.merge(getUnorderedProcessedFiles()); - } - else - { - String processed = getMaxProcessedFile(); - processed_and_failed_files.insert(processed); - } - - S3FilesCollection processing_files = getProcessingFiles(); - processed_and_failed_files.merge(processing_files); - - return processed_and_failed_files; -} - -void S3QueueHolder::removeProcessingFile(const String & file_path) -{ - String node_data; - String processing = zk_client->get(zookeeper_processing_path); - S3FilesCollection processing_files = parseCollection(processing); - - processing_files.erase(file_path); - - Strings file_paths(processing_files.begin(), processing_files.end()); - zk_client->set(fs::path(zookeeper_processing_path), toString(file_paths)); -} - -std::shared_ptr S3QueueHolder::acquireLock() -{ - UInt32 retry_count = 200; - UInt32 sleep_ms = 100; - - UInt32 retries = 0; - while (true) - { - Coordination::Error code = zk_client->tryCreate(zookeeper_lock_path, "", zkutil::CreateMode::Ephemeral); - if (code == Coordination::Error::ZNONODE || code == Coordination::Error::ZNODEEXISTS) - { - retries++; - if (retries > retry_count) - { - throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Can't acquire zookeeper lock"); - } - sleepForMilliseconds(sleep_ms); - } - else if (code != Coordination::Error::ZOK) - { - throw Coordination::Exception(code, zookeeper_lock_path); - } - else - { - return zkutil::EphemeralNodeHolder::existing(zookeeper_lock_path, *zk_client); - } - } -} - -S3QueueHolder::S3FilesCollection S3QueueHolder::parseCollection(const String & collection_str) -{ - ReadBufferFromString rb(collection_str); - Strings deserialized; - try - { - readQuoted(deserialized, rb); - } - catch (const Exception & e) - { - LOG_WARNING(log, "Can't parse collection from ZooKeeper node: {}", e.displayText()); - deserialized = {}; - } - - return std::unordered_set(deserialized.begin(), deserialized.end()); -} - -} - -#endif diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 4785a131f60..57d2d6304b0 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -171,7 +171,7 @@ StorageS3QueueSource::StorageS3QueueSource( const String & bucket_, const String & version_id_, std::shared_ptr file_iterator_, - std::shared_ptr queue_holder_, + std::shared_ptr files_metadata_, const S3QueueAction & action_, const size_t download_thread_num_) : ISource(getHeader(sample_block_, requested_virtual_columns_)) @@ -183,7 +183,7 @@ StorageS3QueueSource::StorageS3QueueSource( , columns_desc(columns_) , request_settings(request_settings_) , client(client_) - , queue_holder(queue_holder_) + , files_metadata(files_metadata_) , requested_virtual_columns(requested_virtual_columns_) , file_iterator(file_iterator_) , action(action_) @@ -259,13 +259,13 @@ Chunk StorageS3QueueSource::generate() catch (const Exception & e) { LOG_ERROR(log, "Exception in chunk pulling: {} ", e.displayText()); - queue_holder->setFileFailed(reader.getFile(), e.message()); + files_metadata->setFileFailed(reader.getFile(), e.message()); success_in_pulling = false; } if (success_in_pulling) { applyActionAfterProcessing(reader.getFile()); - queue_holder->setFileProcessed(reader.getFile()); + files_metadata->setFileProcessed(reader.getFile()); return chunk; } diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index e2e472b5007..a85fce46ad8 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -8,7 +8,7 @@ # include # include -# include +# include # include # include @@ -81,7 +81,7 @@ public: const String & bucket, const String & version_id, std::shared_ptr file_iterator_, - std::shared_ptr queue_holder_, + std::shared_ptr files_metadata_, const S3QueueAction & action_, size_t download_thread_num); @@ -101,7 +101,7 @@ private: S3Settings::RequestSettings request_settings; std::shared_ptr client; - std::shared_ptr queue_holder; + std::shared_ptr files_metadata; using ReaderHolder = StorageS3Source::ReaderHolder; ReaderHolder reader; diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index baacb3397c7..87bff398172 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -93,23 +93,21 @@ StorageS3Queue::StorageS3Queue( : IStorage(table_id_) , WithContext(context_) , s3queue_settings(std::move(s3queue_settings_)) - , s3_configuration{configuration_} - , keys({s3_configuration.url.key}) - , mode(s3queue_settings->mode) , after_processing(s3queue_settings->after_processing) - , milliseconds_to_wait(s3queue_settings->s3queue_polling_min_timeout_ms) - , format_name(configuration_.format) - , compression_method(configuration_.compression_method) - , name(s3_configuration.url.storage_name) + , configuration{configuration_} + , reschedule_processing_interval_ms(s3queue_settings->s3queue_polling_min_timeout_ms) , format_settings(format_settings_) , partition_by(partition_by_) , log(&Poco::Logger::get("StorageS3Queue (" + table_id_.table_name + ")")) { - if (!withGlobs()) - throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "S3Queue engine can read only from url with globs"); + if (configuration.url.key.ends_with('/')) + configuration.url.key += '*'; - String setting_zookeeper_path = s3queue_settings->keeper_path; - if (setting_zookeeper_path.empty()) + if (!withGlobs()) + throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "S3Queue url must either end with '/' or contain globs"); + + String setting_zk_path = s3queue_settings->keeper_path; + if (setting_zk_path.empty()) { auto database = DatabaseCatalog::instance().getDatabase(table_id_.database_name); bool is_in_replicated_database = database->getEngineName() == "Replicated"; @@ -135,25 +133,25 @@ StorageS3Queue::StorageS3Queue( "s3queue_default_zookeeper_path_prefix not specified"); } - zookeeper_path = zkutil::extractZooKeeperPath( + zk_path = zkutil::extractZooKeeperPath( fs::path(zk_path_prefix) / toString(table_id_.uuid), /* check_starts_with_slash */ true, log); } else { /// We do not add table uuid here on purpose. - zookeeper_path = zkutil::extractZooKeeperPath(s3queue_settings->keeper_path.value, /* check_starts_with_slash */ true, log); + zk_path = zkutil::extractZooKeeperPath(s3queue_settings->keeper_path.value, /* check_starts_with_slash */ true, log); } - LOG_INFO(log, "Using zookeeper path: {}", zookeeper_path); + LOG_INFO(log, "Using zookeeper path: {}", zk_path); - FormatFactory::instance().checkFormatName(format_name); - context_->getGlobalContext()->getRemoteHostFilter().checkURL(s3_configuration.url.uri); + FormatFactory::instance().checkFormatName(configuration.format); + context_->getGlobalContext()->getRemoteHostFilter().checkURL(configuration.url.uri); StorageInMemoryMetadata storage_metadata; - s3_configuration.update(context_); + configuration.update(context_); if (columns_.empty()) { - auto columns = StorageS3::getTableStructureFromDataImpl(s3_configuration, format_settings, context_); + auto columns = StorageS3::getTableStructureFromDataImpl(configuration, format_settings, context_); storage_metadata.setColumns(columns); } else @@ -163,22 +161,15 @@ StorageS3Queue::StorageS3Queue( storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); - setZooKeeper(); auto metadata_snapshot = getInMemoryMetadataPtr(); const bool is_first_replica = createTableIfNotExists(metadata_snapshot); if (!is_first_replica) { - checkTableStructure(zookeeper_path, metadata_snapshot); + checkTableStructure(zk_path, metadata_snapshot); } - queue_holder = std::make_unique( - zookeeper_path, - mode, - getContext(), - s3queue_settings->s3queue_tracked_files_limit.value, - s3queue_settings->s3queue_tracked_file_ttl_sec.value, - s3queue_settings->s3queue_loading_retries.value); + files_metadata = std::make_shared(this, *s3queue_settings); auto default_virtuals = NamesAndTypesList{ {"_path", std::make_shared(std::make_shared())}, @@ -196,12 +187,12 @@ StorageS3Queue::StorageS3Queue( bool StorageS3Queue::supportsSubcolumns() const { - return FormatFactory::instance().checkIfFormatSupportsSubcolumns(format_name); + return FormatFactory::instance().checkIfFormatSupportsSubcolumns(configuration.format); } bool StorageS3Queue::supportsSubsetOfColumns() const { - return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(format_name); + return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration.format); } Pipe StorageS3Queue::read( @@ -220,7 +211,7 @@ Pipe StorageS3Queue::read( if (mv_attached) throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "Cannot read from StorageS3Queue with attached materialized views"); - auto query_s3_configuration = updateConfigurationAndGetCopy(local_context); + auto query_configuration = updateConfigurationAndGetCopy(local_context); Pipes pipes; @@ -262,24 +253,23 @@ Pipe StorageS3Queue::read( } const size_t max_download_threads = local_context->getSettingsRef().max_download_threads; - auto zookeeper = getZooKeeper(); return Pipe(std::make_shared( requested_virtual_columns, - format_name, + configuration.format, getName(), block_for_format, local_context, format_settings, columns_description, max_block_size, - query_s3_configuration.request_settings, - compression_method, - query_s3_configuration.client, - query_s3_configuration.url.bucket, - query_s3_configuration.url.version_id, + query_configuration.request_settings, + configuration.compression_method, + query_configuration.client, + query_configuration.url.bucket, + query_configuration.url.version_id, iterator_wrapper, - queue_holder, + files_metadata, after_processing, max_download_threads)); } @@ -387,7 +377,7 @@ void StorageS3Queue::threadFunc() break; } - milliseconds_to_wait = s3queue_settings->s3queue_polling_min_timeout_ms; + reschedule_processing_interval_ms = s3queue_settings->s3queue_polling_min_timeout_ms; } } } @@ -402,9 +392,9 @@ void StorageS3Queue::threadFunc() { LOG_TRACE(log, "Reschedule S3 Queue thread func."); /// Reschedule with backoff. - if (milliseconds_to_wait < s3queue_settings->s3queue_polling_max_timeout_ms) - milliseconds_to_wait += s3queue_settings->s3queue_polling_backoff_ms; - task->holder->scheduleAfter(milliseconds_to_wait); + if (reschedule_processing_interval_ms < s3queue_settings->s3queue_polling_max_timeout_ms) + reschedule_processing_interval_ms += s3queue_settings->s3queue_polling_backoff_ms; + task->holder->scheduleAfter(reschedule_processing_interval_ms); } } @@ -426,7 +416,7 @@ void StorageS3Queue::streamToViews() auto s3queue_context = Context::createCopy(getContext()); s3queue_context->makeQueryContext(); - auto query_s3_configuration = updateConfigurationAndGetCopy(s3queue_context); + auto query_configuration = updateConfigurationAndGetCopy(s3queue_context); // Create a stream for each consumer and join them in a union stream // Only insert into dependent views and expect that input blocks contain virtual columns @@ -473,23 +463,22 @@ void StorageS3Queue::streamToViews() Pipes pipes; - auto zookeeper = getZooKeeper(); auto pipe = Pipe(std::make_shared( requested_virtual_columns, - format_name, + configuration.format, getName(), block_for_format, s3queue_context, format_settings, columns_description, block_size, - query_s3_configuration.request_settings, - compression_method, - query_s3_configuration.client, - query_s3_configuration.url.bucket, - query_s3_configuration.url.version_id, + query_configuration.request_settings, + configuration.compression_method, + query_configuration.client, + query_configuration.url.bucket, + query_configuration.url.version_id, iterator_wrapper, - queue_holder, + files_metadata, after_processing, max_download_threads)); @@ -505,65 +494,56 @@ void StorageS3Queue::streamToViews() StorageS3Queue::Configuration StorageS3Queue::updateConfigurationAndGetCopy(ContextPtr local_context) { - s3_configuration.update(local_context); - return s3_configuration; -} - -void StorageS3Queue::setZooKeeper() -{ - std::lock_guard lock(current_zookeeper_mutex); - current_zookeeper = getContext()->getZooKeeper(); -} - -zkutil::ZooKeeperPtr StorageS3Queue::tryGetZooKeeper() const -{ - std::lock_guard lock(current_zookeeper_mutex); - return current_zookeeper; + configuration.update(local_context); + return configuration; } zkutil::ZooKeeperPtr StorageS3Queue::getZooKeeper() const { - auto res = tryGetZooKeeper(); - if (!res) - throw Exception(ErrorCodes::NO_ZOOKEEPER, "Cannot get ZooKeeper"); - return res; + std::lock_guard lock{zk_mutex}; + if (!zk_client || zk_client->expired()) + { + zk_client = getContext()->getZooKeeper(); + zk_client->sync(zk_path); + } + return zk_client; } bool StorageS3Queue::createTableIfNotExists(const StorageMetadataPtr & metadata_snapshot) { auto zookeeper = getZooKeeper(); - zookeeper->createAncestors(zookeeper_path); + zookeeper->createAncestors(zk_path); for (size_t i = 0; i < zk_create_table_retries; ++i) { Coordination::Requests ops; bool is_first_replica = true; - if (zookeeper->exists(zookeeper_path + "/metadata")) + if (zookeeper->exists(zk_path + "/metadata")) { - if (!zookeeper->exists(zookeeper_path + "/processing")) - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processing", "", zkutil::CreateMode::Ephemeral)); - LOG_DEBUG(log, "This table {} is already created, will use existing metadata for checking engine settings", zookeeper_path); + if (!zookeeper->exists(zk_path + "/processing")) + ops.emplace_back(zkutil::makeCreateRequest(zk_path + "/processing", "", zkutil::CreateMode::Ephemeral)); + LOG_DEBUG(log, "This table {} is already created, will use existing metadata for checking engine settings", zk_path); is_first_replica = false; } else { - String metadata_str = S3QueueTableMetadata(s3_configuration, *s3queue_settings).toString(); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path, "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processed", "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/failed", "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/processing", "", zkutil::CreateMode::Ephemeral)); + String metadata_str = S3QueueTableMetadata(configuration, *s3queue_settings).toString(); + ops.emplace_back(zkutil::makeCreateRequest(zk_path, "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zk_path + "/processed", "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zk_path + "/failed", "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zk_path + "/processing", "", zkutil::CreateMode::Ephemeral)); ops.emplace_back(zkutil::makeCreateRequest( - zookeeper_path + "/columns", metadata_snapshot->getColumns().toString(), zkutil::CreateMode::Persistent)); + zk_path + "/columns", metadata_snapshot->getColumns().toString(), zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/metadata", metadata_str, zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zk_path + "/metadata", metadata_str, zkutil::CreateMode::Persistent)); } Coordination::Responses responses; auto code = zookeeper->tryMulti(ops, responses); if (code == Coordination::Error::ZNODEEXISTS) { - LOG_INFO(log, "It looks like the table {} was created by another server at the same moment, will retry", zookeeper_path); + LOG_INFO(log, "It looks like the table {} was created by another server at the same moment, will retry", zk_path); continue; } else if (code != Coordination::Error::ZOK) @@ -577,7 +557,7 @@ bool StorageS3Queue::createTableIfNotExists(const StorageMetadataPtr & metadata_ throw Exception( ErrorCodes::REPLICA_ALREADY_EXISTS, "Cannot create table, because it is created concurrently every time or because " - "of wrong zookeeper_path or because of logical error"); + "of wrong zk_path or because of logical error"); } @@ -588,7 +568,7 @@ void StorageS3Queue::checkTableStructure(const String & zookeeper_prefix, const { auto zookeeper = getZooKeeper(); - S3QueueTableMetadata old_metadata(s3_configuration, *s3queue_settings); + S3QueueTableMetadata old_metadata(configuration, *s3queue_settings); Coordination::Stat metadata_stat; String metadata_str = zookeeper->get(fs::path(zookeeper_prefix) / "metadata", &metadata_stat); @@ -615,39 +595,40 @@ std::shared_ptr StorageS3Queue::createFileIterator(ContextPtr local_context, ASTPtr query) { auto it = std::make_shared( - *s3_configuration.client, - s3_configuration.url, + *configuration.client, + configuration.url, query, virtual_block, local_context, s3queue_settings->s3queue_polling_size.value, - s3_configuration.request_settings); + configuration.request_settings); - auto lock = queue_holder->acquireLock(); - S3QueueHolder::S3FilesCollection files_to_skip = queue_holder->getProcessedAndFailedFiles(); + auto zookeeper = getZooKeeper(); + auto lock = files_metadata->acquireLock(zookeeper); + S3QueueFilesMetadata::S3FilesCollection files_to_skip = files_metadata->getProcessedFailedAndProcessingFiles(); Strings files_to_process; - if (mode == S3QueueMode::UNORDERED) + if (s3queue_settings->mode == S3QueueMode::UNORDERED) { - files_to_process = it->filterProcessingFiles(mode, files_to_skip); + files_to_process = it->filterProcessingFiles(s3queue_settings->mode, files_to_skip); } else { - String max_processed_file = queue_holder->getMaxProcessedFile(); - files_to_process = it->filterProcessingFiles(mode, files_to_skip, max_processed_file); + String max_processed_file = files_metadata->getMaxProcessedFile(); + files_to_process = it->filterProcessingFiles(s3queue_settings->mode, files_to_skip, max_processed_file); } LOG_TEST(log, "Found files to process: {}", fmt::join(files_to_process, ", ")); - queue_holder->setFilesProcessing(files_to_process); + files_metadata->setFilesProcessing(files_to_process); return it; } void StorageS3Queue::drop() { - auto zk_client = getZooKeeper(); - if (zk_client->exists(zookeeper_path)) - zk_client->removeRecursive(zookeeper_path); + auto zookeeper = getZooKeeper(); + if (zookeeper->exists(zk_path)) + zookeeper->removeRecursive(zk_path); } void registerStorageS3QueueImpl(const String & name, StorageFactory & factory) diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index 6af38058dd8..9737d5fcefa 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -11,7 +11,7 @@ # include # include -# include +# include # include # include # include @@ -41,6 +41,7 @@ class StorageS3Queue : public IStorage, WithContext { public: using Configuration = typename StorageS3::Configuration; + StorageS3Queue( std::unique_ptr s3queue_settings_, const Configuration & configuration_, @@ -79,35 +80,39 @@ public: bool supportsPartitionBy() const override; - const auto & getFormatName() const { return format_name; } + const auto & getFormatName() const { return configuration.format; } + + const String & getZooKeeperPath() const { return zk_path; } + + zkutil::ZooKeeperPtr getZooKeeper() const; private: - std::unique_ptr s3queue_settings; - std::shared_ptr queue_holder; - Configuration s3_configuration; - std::vector keys; + const std::unique_ptr s3queue_settings; + const S3QueueAction after_processing; + + std::shared_ptr files_metadata; + Configuration configuration; NamesAndTypesList virtual_columns; Block virtual_block; - S3QueueMode mode; - S3QueueAction after_processing; - uint64_t milliseconds_to_wait = 10000; - - String format_name; - String compression_method; - String name; + UInt64 reschedule_processing_interval_ms; std::optional format_settings; ASTPtr partition_by; + String zk_path; + mutable zkutil::ZooKeeperPtr zk_client; + mutable std::mutex zk_mutex; + + std::atomic mv_attached = false; + std::atomic shutdown_called{false}; + Poco::Logger * log; + bool supportsSubcolumns() const override; - bool withGlobs() const { return s3_configuration.url.key.find_first_of("*?{") != std::string::npos; } + bool withGlobs() const { return configuration.url.key.find_first_of("*?{") != std::string::npos; } void threadFunc(); size_t getTableDependentCount() const; - std::atomic mv_attached = false; bool hasDependencies(const StorageID & table_id); - std::atomic shutdown_called{false}; - Poco::Logger * log; void startup() override; void shutdown() override; @@ -122,19 +127,10 @@ private: std::shared_ptr task; bool supportsSubsetOfColumns() const override; - String zookeeper_path; - - zkutil::ZooKeeperPtr current_zookeeper; - mutable std::mutex current_zookeeper_mutex; - - void setZooKeeper(); - zkutil::ZooKeeperPtr tryGetZooKeeper() const; - zkutil::ZooKeeperPtr getZooKeeper() const; const UInt32 zk_create_table_retries = 1000; bool createTableIfNotExists(const StorageMetadataPtr & metadata_snapshot); void checkTableStructure(const String & zookeeper_prefix, const StorageMetadataPtr & metadata_snapshot); - const String & getZooKeeperPath() const { return zookeeper_path; } using KeysWithInfo = StorageS3QueueSource::KeysWithInfo;