From f01d9890a982177599af6691a79c9bcf52353b13 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Tue, 3 Mar 2026 14:15:19 +0300 Subject: [PATCH 01/15] Merge pull request #1385 from Altinity/fp_antalya_26_1_parquet_metadata_cache Antalya 26.1 - Forward port of parquet metadata caching #1039 Source-PR: #1385 (https://github.com/Altinity/ClickHouse/pull/1385) --- programs/server/Server.cpp | 9 +++ src/Common/ProfileEvents.cpp | 3 +- src/Core/FormatFactorySettings.h | 3 +- src/Core/ServerSettings.cpp | 5 +- src/Core/SettingsChangesHistory.cpp | 1 + src/Interpreters/InterpreterSystemQuery.cpp | 15 ++++ src/Parsers/ASTSystemQuery.cpp | 1 + src/Parsers/ASTSystemQuery.h | 1 + src/Processors/Formats/IInputFormat.h | 4 ++ .../Formats/Impl/ParquetBlockInputFormat.cpp | 72 ++++++++++++++++++- .../Formats/Impl/ParquetBlockInputFormat.h | 15 ++++ .../Formats/Impl/ParquetFileMetaDataCache.cpp | 20 ++++++ .../Formats/Impl/ParquetFileMetaDataCache.h | 30 ++++++++ .../Impl/ParquetMetadataInputFormat.cpp | 53 ++++++++++++-- .../Formats/Impl/ParquetMetadataInputFormat.h | 10 +++ .../StorageObjectStorageSource.cpp | 8 +++ .../disable_parquet_metadata_caching.xml | 7 ++ tests/integration/test_storage_delta/test.py | 7 +- .../0_stateless/02995_settings_26_2_1.tsv | 1 + ...et_object_storage_metadata_cache.reference | 8 +++ ..._parquet_object_storage_metadata_cache.sql | 63 ++++++++++++++++ 21 files changed, 324 insertions(+), 12 deletions(-) create mode 100644 src/Processors/Formats/Impl/ParquetFileMetaDataCache.cpp create mode 100644 src/Processors/Formats/Impl/ParquetFileMetaDataCache.h create mode 100644 tests/integration/test_storage_delta/configs/users.d/disable_parquet_metadata_caching.xml create mode 100644 tests/queries/0_stateless/03299_parquet_object_storage_metadata_cache.reference create mode 100644 tests/queries/0_stateless/03299_parquet_object_storage_metadata_cache.sql diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 4e4ba58226cd..1e8b2bd35918 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -165,6 +165,10 @@ # include #endif +#if USE_PARQUET +# include +#endif + /// A minimal file used when the server is run without installation constexpr unsigned char resource_embedded_xml[] = @@ -424,6 +428,7 @@ namespace ServerSetting extern const ServerSettingsString hdfs_libhdfs3_conf; extern const ServerSettingsString config_file; extern const ServerSettingsString users_to_ignore_early_memory_limit_check; + extern const ServerSettingsUInt64 input_format_parquet_metadata_cache_max_size; } namespace ErrorCodes @@ -2792,6 +2797,10 @@ try auto replicas_reconnector = ReplicasReconnector::init(global_context); +#if USE_PARQUET + ParquetFileMetaDataCache::instance()->setMaxSizeInBytes(server_settings[ServerSetting::input_format_parquet_metadata_cache_max_size]); +#endif + /// Set current database name before loading tables and databases because /// system logs may copy global context. std::string default_database = server_settings[ServerSetting::default_database]; diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 9872e039abfa..3aa22a4fc217 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -1377,7 +1377,8 @@ The server successfully detected this situation and will download merged part fr M(RuntimeFilterRowsChecked, "Number of rows checked by JOIN Runtime Filters", ValueType::Number) \ M(RuntimeFilterRowsPassed, "Number of rows that passed (not filtered out by) JOIN Runtime Filters", ValueType::Number) \ M(RuntimeFilterRowsSkipped, "Number of rows in blocks that were skipped by JOIN Runtime Filters", ValueType::Number) \ - + M(ParquetMetaDataCacheHits, "Number of times the read from filesystem cache hit the cache.", ValueType::Number) \ + M(ParquetMetaDataCacheMisses, "Number of times the read from filesystem cache miss the cache.", ValueType::Number) \ #ifdef APPLY_FOR_EXTERNAL_EVENTS #define APPLY_FOR_EVENTS(M) APPLY_FOR_BUILTIN_EVENTS(M) APPLY_FOR_EXTERNAL_EVENTS(M) diff --git a/src/Core/FormatFactorySettings.h b/src/Core/FormatFactorySettings.h index 499c060d885a..728fd3b7aacf 100644 --- a/src/Core/FormatFactorySettings.h +++ b/src/Core/FormatFactorySettings.h @@ -1566,8 +1566,7 @@ Allow to write information about geo columns in parquet metadata and encode colu DECLARE(Bool, into_outfile_create_parent_directories, false, R"( Automatically create parent directories when using INTO OUTFILE if they do not already exists. )", 0) \ - - + DECLARE(Bool, input_format_parquet_use_metadata_cache, true, R"(Enable parquet file metadata caching)", 0) \ // End of FORMAT_FACTORY_SETTINGS #define OBSOLETE_FORMAT_SETTINGS(M, ALIAS) \ diff --git a/src/Core/ServerSettings.cpp b/src/Core/ServerSettings.cpp index 09520a9768d5..2471bf4ea8f2 100644 --- a/src/Core/ServerSettings.cpp +++ b/src/Core/ServerSettings.cpp @@ -1529,7 +1529,8 @@ The policy on how to perform a scheduling of CPU slots specified by `concurrent_ ```xml 1 ``` - )", 0) + )", 0) \ + DECLARE(UInt64, input_format_parquet_metadata_cache_max_size, 500000000, "Maximum size of parquet file metadata cache", 0) /// Settings with a path are server settings with at least one layer of nesting that have a fixed structure (no lists, lists, enumerations, repetitions, ...). #define LIST_OF_SERVER_SETTINGS_WITH_PATH(DECLARE, ALIAS) \ @@ -1602,7 +1603,7 @@ The policy on how to perform a scheduling of CPU slots specified by `concurrent_ DECLARE(UInt64, keeper_server_socket_receive_timeout_sec, DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC, R"(Keeper socket receive timeout.)", 0, "keeper_server.socket_receive_timeout_sec") \ DECLARE(UInt64, keeper_server_socket_send_timeout_sec, DBMS_DEFAULT_SEND_TIMEOUT_SEC, R"(Keeper socket send timeout.)", 0, "keeper_server.socket_send_timeout_sec") \ DECLARE(String, hdfs_libhdfs3_conf, "", R"(Points libhdfs3 to the right location for its config.)", 0, "hdfs.libhdfs3_conf") \ - DECLARE(String, config_file, "config.xml", R"(Points to the server config file.)", 0, "config-file") + DECLARE(String, config_file, "config.xml", R"(Points to the server config file.)", 0, "config-file") // clang-format on diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index ac6d42e957f6..26c60f2d9584 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -306,6 +306,7 @@ const VersionToSettingsChangesMap & getSettingsChangesHistory() // {"allow_database_unity_catalog", false, true, "Turned ON by default for Antalya (alias)."}, // {"allow_database_glue_catalog", false, true, "Turned ON by default for Antalya (alias)."}, // {"input_format_parquet_use_metadata_cache", true, true, "New setting, turned ON by default"}, // https://github.com/Altinity/ClickHouse/pull/586 + {"input_format_parquet_use_metadata_cache", true, true, "New setting, turned ON by default"}, // https://github.com/Altinity/ClickHouse/pull/586 // {"iceberg_timezone_for_timestamptz", "UTC", "UTC", "New setting."}, // {"object_storage_remote_initiator", false, false, "New setting."}, // {"allow_experimental_iceberg_read_optimization", true, true, "New setting."}, diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 8602cf1c36be..6083afdd58a9 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -83,6 +83,10 @@ #include #endif +#if USE_PARQUET +#include +#endif + #if USE_AWS_S3 #include #endif @@ -478,6 +482,16 @@ BlockIO InterpreterSystemQuery::execute() getContext()->clearQueryResultCache(query.query_result_cache_tag); break; } + case Type::DROP_PARQUET_METADATA_CACHE: + { +#if USE_PARQUET + getContext()->checkAccess(AccessType::SYSTEM_DROP_PARQUET_METADATA_CACHE); + ParquetFileMetaDataCache::instance()->clear(); + break; +#else + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "The server was compiled without the support for Parquet"); +#endif + } case Type::CLEAR_COMPILED_EXPRESSION_CACHE: #if USE_EMBEDDED_COMPILER getContext()->checkAccess(AccessType::SYSTEM_DROP_COMPILED_EXPRESSION_CACHE); @@ -2177,6 +2191,7 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster() case Type::CLEAR_PAGE_CACHE: case Type::CLEAR_SCHEMA_CACHE: case Type::CLEAR_FORMAT_SCHEMA_CACHE: + case Type::DROP_PARQUET_METADATA_CACHE: case Type::CLEAR_S3_CLIENT_CACHE: { required_access.emplace_back(AccessType::SYSTEM_DROP_CACHE); diff --git a/src/Parsers/ASTSystemQuery.cpp b/src/Parsers/ASTSystemQuery.cpp index 5fd99ca62aa5..6e97f8279598 100644 --- a/src/Parsers/ASTSystemQuery.cpp +++ b/src/Parsers/ASTSystemQuery.cpp @@ -597,6 +597,7 @@ void ASTSystemQuery::formatImpl(WriteBuffer & ostr, const FormatSettings & setti case Type::CLEAR_TEXT_INDEX_CACHES: case Type::CLEAR_COMPILED_EXPRESSION_CACHE: case Type::CLEAR_S3_CLIENT_CACHE: + case Type::DROP_PARQUET_METADATA_CACHE: case Type::CLEAR_ICEBERG_METADATA_CACHE: case Type::CLEAR_PARQUET_METADATA_CACHE: case Type::RESET_COVERAGE: diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index 8a062cbf1646..5b7682a80117 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -42,6 +42,7 @@ class ASTSystemQuery : public IAST, public ASTQueryWithOnCluster CLEAR_QUERY_CONDITION_CACHE, CLEAR_QUERY_CACHE, CLEAR_COMPILED_EXPRESSION_CACHE, + DROP_PARQUET_METADATA_CACHE, CLEAR_ICEBERG_METADATA_CACHE, CLEAR_PARQUET_METADATA_CACHE, CLEAR_FILESYSTEM_CACHE, diff --git a/src/Processors/Formats/IInputFormat.h b/src/Processors/Formats/IInputFormat.h index ab693ba7a212..de90836516c2 100644 --- a/src/Processors/Formats/IInputFormat.h +++ b/src/Processors/Formats/IInputFormat.h @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB @@ -128,6 +129,9 @@ class IInputFormat : public ISource void needOnlyCount() { need_only_count = true; } + /// Set additional info/key/id related to underlying storage of the ReadBuffer + virtual void setStorageRelatedUniqueKey(const Settings & /*settings*/, const String & /*key*/) {} + protected: ReadBuffer & getReadBuffer() const { chassert(in); return *in; } diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 9e5fe77e3d9f..b14721cc8cbd 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -10,6 +10,9 @@ #if USE_PARQUET +#include +#include +#include #include #include #include @@ -37,6 +40,7 @@ #include #include #include +#include #include #include #include @@ -48,6 +52,8 @@ namespace ProfileEvents extern const Event ParquetFetchWaitTimeMicroseconds; extern const Event ParquetReadRowGroups; extern const Event ParquetPrunedRowGroups; + extern const Event ParquetMetaDataCacheHits; + extern const Event ParquetMetaDataCacheMisses; } namespace CurrentMetrics @@ -67,6 +73,12 @@ namespace DB namespace Setting { extern const SettingsBool use_parquet_metadata_cache; + extern const SettingsBool input_format_parquet_use_metadata_cache; +} + +namespace ServerSetting +{ + extern const ServerSettingsUInt64 input_format_parquet_metadata_cache_max_size; } namespace ErrorCodes @@ -554,6 +566,49 @@ static std::vector getHyperrectangleForRowGroup(const parquet::FileMetaDa return hyperrectangle; } +std::shared_ptr ParquetBlockInputFormat::readMetadataFromFile() +{ + createArrowFileIfNotCreated(); + return parquet::ReadMetaData(arrow_file); +} + +std::shared_ptr ParquetBlockInputFormat::getFileMetaData() +{ + // in-memory cache is not implemented for local file operations, only for remote files + // there is a chance the user sets `input_format_parquet_use_metadata_cache=1` for a local file operation + // and the cache_key won't be set. Therefore, we also need to check for metadata_cache.key + if (!metadata_cache.use_cache || metadata_cache.key.empty()) + { + return readMetadataFromFile(); + } + + auto [parquet_file_metadata, loaded] = ParquetFileMetaDataCache::instance()->getOrSet( + metadata_cache.key, + [&]() + { + return readMetadataFromFile(); + } + ); + if (loaded) + ProfileEvents::increment(ProfileEvents::ParquetMetaDataCacheMisses); + else + ProfileEvents::increment(ProfileEvents::ParquetMetaDataCacheHits); + return parquet_file_metadata; +} + +void ParquetBlockInputFormat::createArrowFileIfNotCreated() +{ + if (arrow_file) + { + return; + } + + // Create arrow file adapter. + // TODO: Make the adapter do prefetching on IO threads, based on the full set of ranges that + // we'll need to read (which we know in advance). Use max_download_threads for that. + arrow_file = asArrowFile(*in, format_settings, is_stopped, "Parquet", PARQUET_MAGIC_BYTES, /* avoid_buffering */ true); +} + std::unordered_set getBloomFilterFilteringColumnKeys(const KeyCondition::RPN & rpn) { std::unordered_set column_keys; @@ -700,7 +755,7 @@ void ParquetBlockInputFormat::initializeIfNeeded() if (is_stopped) return; - metadata = parquet::ReadMetaData(arrow_file); + metadata = getFileMetaData(); if (buckets_to_read) { std::unordered_set set_to_read(buckets_to_read->row_group_ids.begin(), buckets_to_read->row_group_ids.end()); @@ -816,6 +871,8 @@ void ParquetBlockInputFormat::initializeIfNeeded() } } + bool has_row_groups_to_read = false; + auto skip_row_group_based_on_filters = [&](int row_group) { if (!format_settings.parquet.filter_push_down && !format_settings.parquet.bloom_filter_push_down) @@ -874,9 +931,22 @@ void ParquetBlockInputFormat::initializeIfNeeded() row_group_batches.back().total_bytes_compressed += row_group_size; auto rows = adaptive_chunk_size(row_group); row_group_batches.back().adaptive_chunk_size = rows ? rows : format_settings.parquet.max_block_size; + + has_row_groups_to_read = true; + } + + if (has_row_groups_to_read) + { + createArrowFileIfNotCreated(); } } +void ParquetBlockInputFormat::setStorageRelatedUniqueKey(const Settings & settings, const String & key_) +{ + metadata_cache.key = key_; + metadata_cache.use_cache = settings[Setting::input_format_parquet_use_metadata_cache]; +} + void ParquetBlockInputFormat::initializeRowGroupBatchReader(size_t row_group_batch_idx) { bool row_group_prefetch = io_pool != nullptr; diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h index 1fea5f0609c1..2c71a45f1751 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h @@ -95,6 +95,7 @@ class ParquetBlockInputFormat : public IInputFormat size_t getApproxBytesReadForChunk() const override { return previous_approx_bytes_read_for_chunk; } void setBucketsToRead(const FileBucketInfoPtr & buckets_to_read_) override; + void setStorageRelatedUniqueKey(const Settings & settings, const String & key_) override; private: Chunk read() override; @@ -114,6 +115,13 @@ class ParquetBlockInputFormat : public IInputFormat void threadFunction(size_t row_group_batch_idx); + void createArrowFileIfNotCreated(); + std::shared_ptr readMetadataFromFile(); + + std::shared_ptr getFileMetaData(); + + inline bool supportPrefetch() const; + // Data layout in the file: // // row group 0 @@ -362,6 +370,13 @@ class ParquetBlockInputFormat : public IInputFormat bool is_initialized = false; std::optional> parquet_names_to_clickhouse; std::optional> clickhouse_names_to_parquet; + struct Cache + { + String key; + bool use_cache = false; + }; + + Cache metadata_cache; }; class ArrowParquetSchemaReader : public ISchemaReader diff --git a/src/Processors/Formats/Impl/ParquetFileMetaDataCache.cpp b/src/Processors/Formats/Impl/ParquetFileMetaDataCache.cpp new file mode 100644 index 000000000000..da8ad825f505 --- /dev/null +++ b/src/Processors/Formats/Impl/ParquetFileMetaDataCache.cpp @@ -0,0 +1,20 @@ +#include + +#if USE_PARQUET + +namespace DB +{ + +ParquetFileMetaDataCache::ParquetFileMetaDataCache() + : CacheBase(CurrentMetrics::end(), CurrentMetrics::end(), 0) +{} + +ParquetFileMetaDataCache * ParquetFileMetaDataCache::instance() +{ + static ParquetFileMetaDataCache instance; + return &instance; +} + +} + +#endif diff --git a/src/Processors/Formats/Impl/ParquetFileMetaDataCache.h b/src/Processors/Formats/Impl/ParquetFileMetaDataCache.h new file mode 100644 index 000000000000..fb5fc1bb0217 --- /dev/null +++ b/src/Processors/Formats/Impl/ParquetFileMetaDataCache.h @@ -0,0 +1,30 @@ +#pragma once + +#include "config.h" + +#if USE_PARQUET + +namespace parquet +{ + +class FileMetaData; + +} + +#include + +namespace DB +{ + +class ParquetFileMetaDataCache : public CacheBase +{ +public: + static ParquetFileMetaDataCache * instance(); + +private: + ParquetFileMetaDataCache(); +}; + +} + +#endif diff --git a/src/Processors/Formats/Impl/ParquetMetadataInputFormat.cpp b/src/Processors/Formats/Impl/ParquetMetadataInputFormat.cpp index 02e5f79ee3f4..f7c2dab49db1 100644 --- a/src/Processors/Formats/Impl/ParquetMetadataInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetMetadataInputFormat.cpp @@ -22,8 +22,17 @@ #include #include #include +#include +#include +#include +namespace ProfileEvents +{ +extern const Event ParquetMetaDataCacheHits; +extern const Event ParquetMetaDataCacheMisses; +} + namespace DB { @@ -32,6 +41,11 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } +namespace Setting +{ +extern const SettingsBool input_format_parquet_use_metadata_cache; +} + static NamesAndTypesList getHeaderForParquetMetadata() { NamesAndTypesList names_and_types{ @@ -130,10 +144,35 @@ void checkHeader(const Block & header) static std::shared_ptr getFileMetadata( ReadBuffer & in, const FormatSettings & format_settings, - std::atomic & is_stopped) + std::atomic & is_stopped, + ParquetMetadataInputFormat::Cache metadata_cache) { - auto arrow_file = asArrowFile(in, format_settings, is_stopped, "Parquet", PARQUET_MAGIC_BYTES, /* avoid_buffering */ true); - return parquet::ReadMetaData(arrow_file); + // in-memory cache is not implemented for local file operations, only for remote files + // there is a chance the user sets `input_format_parquet_use_metadata_cache=1` for a local file operation + // and the cache_key won't be set. Therefore, we also need to check for metadata_cache.key + if (!metadata_cache.use_cache || metadata_cache.key.empty()) + { + auto arrow_file = asArrowFile(in, format_settings, is_stopped, "Parquet", PARQUET_MAGIC_BYTES, /* avoid_buffering */ true); + return parquet::ReadMetaData(arrow_file); + } + + auto [parquet_file_metadata, loaded] = ParquetFileMetaDataCache::instance()->getOrSet( + metadata_cache.key, + [&]() + { + auto arrow_file = asArrowFile(in, format_settings, is_stopped, "Parquet", PARQUET_MAGIC_BYTES, /* avoid_buffering */ true); + return parquet::ReadMetaData(arrow_file); + } + ); + + if (loaded) + ProfileEvents::increment(ProfileEvents::ParquetMetaDataCacheMisses); + else + ProfileEvents::increment(ProfileEvents::ParquetMetaDataCacheHits); + + return parquet_file_metadata; + + } ParquetMetadataInputFormat::ParquetMetadataInputFormat(ReadBuffer & in_, SharedHeader header_, const FormatSettings & format_settings_) @@ -148,7 +187,7 @@ Chunk ParquetMetadataInputFormat::read() if (done) return res; - auto metadata = getFileMetadata(*in, format_settings, is_stopped); + auto metadata = getFileMetadata(*in, format_settings, is_stopped, metadata_cache); const auto & header = getPort().getHeader(); auto names_and_types = getHeaderForParquetMetadata(); @@ -489,6 +528,12 @@ void ParquetMetadataInputFormat::resetParser() done = false; } +void ParquetMetadataInputFormat::setStorageRelatedUniqueKey(const Settings & settings, const String & key_) +{ + metadata_cache.key = key_; + metadata_cache.use_cache = settings[Setting::input_format_parquet_use_metadata_cache]; +} + ParquetMetadataSchemaReader::ParquetMetadataSchemaReader(ReadBuffer & in_) : ISchemaReader(in_) { diff --git a/src/Processors/Formats/Impl/ParquetMetadataInputFormat.h b/src/Processors/Formats/Impl/ParquetMetadataInputFormat.h index 81cf7890ee7e..6b667dcc5b1e 100644 --- a/src/Processors/Formats/Impl/ParquetMetadataInputFormat.h +++ b/src/Processors/Formats/Impl/ParquetMetadataInputFormat.h @@ -62,6 +62,14 @@ class ParquetMetadataInputFormat : public IInputFormat void resetParser() override; + void setStorageRelatedUniqueKey(const Settings & settings, const String & key_) override; + + struct Cache + { + String key; + bool use_cache = false; + }; + private: Chunk read() override; @@ -78,6 +86,8 @@ class ParquetMetadataInputFormat : public IInputFormat const FormatSettings format_settings; bool done = false; std::atomic is_stopped{0}; + + Cache metadata_cache; }; class ParquetMetadataSchemaReader : public ISchemaReader diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 2a13d2d09603..6820d54668ef 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -711,6 +711,14 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade if (need_only_count) input_format->needOnlyCount(); + if (!object_info->getPath().empty()) + { + if (const auto & metadata = object_info->relative_path_with_metadata.metadata) + { + input_format->setStorageRelatedUniqueKey(context_->getSettingsRef(), object_info->getPath() + ":" + metadata->etag); + } + } + builder.init(Pipe(input_format)); configuration->addDeleteTransformers(object_info, builder, format_settings, parser_shared_resources, context_); diff --git a/tests/integration/test_storage_delta/configs/users.d/disable_parquet_metadata_caching.xml b/tests/integration/test_storage_delta/configs/users.d/disable_parquet_metadata_caching.xml new file mode 100644 index 000000000000..bc34464e30da --- /dev/null +++ b/tests/integration/test_storage_delta/configs/users.d/disable_parquet_metadata_caching.xml @@ -0,0 +1,7 @@ + + + + 0 + + + diff --git a/tests/integration/test_storage_delta/test.py b/tests/integration/test_storage_delta/test.py index 728dc13fb2cd..b9191a4c3ef9 100644 --- a/tests/integration/test_storage_delta/test.py +++ b/tests/integration/test_storage_delta/test.py @@ -118,6 +118,7 @@ def started_cluster(): user_configs=[ "configs/users.d/users.xml", "configs/users.d/enable_writes.xml", + "configs/users.d/disable_parquet_metadata_caching.xml", ], env_variables={ "RUST_BACKTRACE": "1", @@ -139,6 +140,7 @@ def started_cluster(): user_configs=[ "configs/users.d/users.xml", "configs/users.d/enable_writes.xml", + "configs/users.d/disable_parquet_metadata_caching.xml", ], with_minio=True, stay_alive=True, @@ -189,6 +191,7 @@ def started_cluster(): user_configs=[ "configs/users.d/users.xml", "configs/users.d/disabled_delta_kernel.xml", + "configs/users.d/disable_parquet_metadata_caching.xml", ], with_minio=True, with_azurite=True, @@ -1449,7 +1452,7 @@ def test_session_token(started_cluster): parquet_data_path = create_initial_data_file( started_cluster, instance, - "SELECT toUInt64(number), toString(number) FROM numbers(100)", + "SELECT toUInt64(number), toString(number) FROM numbers(100) SETTINGS input_format_parquet_use_metadata_cache=0", TABLE_NAME, node_name=node_name, ) @@ -1462,7 +1465,7 @@ def test_session_token(started_cluster): f""" SELECT count() FROM deltaLake( 'http://{started_cluster.minio_host}:{started_cluster.minio_port}/{started_cluster.minio_bucket}/{TABLE_NAME}/', - SETTINGS allow_experimental_delta_kernel_rs=1) + SETTINGS allow_experimental_delta_kernel_rs=1, input_format_parquet_use_metadata_cache=0) """ ) ) diff --git a/tests/queries/0_stateless/02995_settings_26_2_1.tsv b/tests/queries/0_stateless/02995_settings_26_2_1.tsv index 17550d7ed430..5c088568fa48 100644 --- a/tests/queries/0_stateless/02995_settings_26_2_1.tsv +++ b/tests/queries/0_stateless/02995_settings_26_2_1.tsv @@ -685,6 +685,7 @@ input_format_parquet_case_insensitive_column_matching 0 input_format_parquet_enable_json_parsing 1 input_format_parquet_enable_row_group_prefetch 1 input_format_parquet_filter_push_down 1 +input_format_parquet_use_metadata_cache 1 input_format_parquet_import_nested 0 input_format_parquet_local_file_min_bytes_for_seek 8192 input_format_parquet_local_time_as_utc 1 diff --git a/tests/queries/0_stateless/03299_parquet_object_storage_metadata_cache.reference b/tests/queries/0_stateless/03299_parquet_object_storage_metadata_cache.reference new file mode 100644 index 000000000000..c87ad9008b60 --- /dev/null +++ b/tests/queries/0_stateless/03299_parquet_object_storage_metadata_cache.reference @@ -0,0 +1,8 @@ +10 +10 +10 +10 +10 +10 +0 +10 diff --git a/tests/queries/0_stateless/03299_parquet_object_storage_metadata_cache.sql b/tests/queries/0_stateless/03299_parquet_object_storage_metadata_cache.sql new file mode 100644 index 000000000000..c82b3f4dc0e3 --- /dev/null +++ b/tests/queries/0_stateless/03299_parquet_object_storage_metadata_cache.sql @@ -0,0 +1,63 @@ +-- Tags: no-parallel, no-fasttest, no-parallel-replicas + +SET input_format_parquet_use_native_reader_v3=0; + +DROP TABLE IF EXISTS t_parquet_03262; + +CREATE TABLE t_parquet_03262 (a UInt64) +ENGINE = S3(s3_conn, filename = 'test_03262_{_partition_id}', format = Parquet) +PARTITION BY a; + +INSERT INTO t_parquet_03262 SELECT number FROM numbers(10) SETTINGS s3_truncate_on_insert=1; + +SELECT COUNT(*) +FROM s3(s3_conn, filename = 'test_03262_*', format = Parquet) +SETTINGS input_format_parquet_use_metadata_cache=1, use_query_condition_cache=0,optimize_count_from_files=0; + +SELECT COUNT(*) +FROM s3(s3_conn, filename = 'test_03262_*', format = Parquet) +SETTINGS input_format_parquet_use_metadata_cache=1, use_query_condition_cache=0, optimize_count_from_files=0, log_comment='test_03262_parquet_metadata_cache'; + +SELECT COUNT(*) +FROM s3(s3_conn, filename = 'test_03262_*', format = ParquetMetadata) +SETTINGS input_format_parquet_use_metadata_cache=1, use_query_condition_cache=0, optimize_count_from_files=0, log_comment='test_03262_parquet_metadata_format_metadata_cache'; + +SYSTEM FLUSH LOGS; + +SELECT ProfileEvents['ParquetMetaDataCacheHits'] +FROM system.query_log +where log_comment = 'test_03262_parquet_metadata_cache' +AND type = 'QueryFinish' +ORDER BY event_time desc +LIMIT 1 SETTINGS use_query_condition_cache=0; + +SELECT ProfileEvents['ParquetMetaDataCacheHits'] +FROM system.query_log +where log_comment = 'test_03262_parquet_metadata_format_metadata_cache' +AND type = 'QueryFinish' +ORDER BY event_time desc +LIMIT 1 SETTINGS use_query_condition_cache=0; + +SYSTEM DROP PARQUET METADATA CACHE; + +SELECT COUNT(*) +FROM s3(s3_conn, filename = 'test_03262_*', format = Parquet) +SETTINGS input_format_parquet_use_metadata_cache=1, use_query_condition_cache=0, optimize_count_from_files=0, log_comment='test_03262_parquet_metadata_cache_cache_empty'; + +SYSTEM FLUSH LOGS; + +SELECT ProfileEvents['ParquetMetaDataCacheHits'] +FROM system.query_log +where log_comment = 'test_03262_parquet_metadata_cache_cache_empty' +AND type = 'QueryFinish' +ORDER BY event_time desc +LIMIT 1 SETTINGS use_query_condition_cache=0; + +SELECT ProfileEvents['ParquetMetaDataCacheMisses'] +FROM system.query_log +where log_comment = 'test_03262_parquet_metadata_cache_cache_empty' +AND type = 'QueryFinish' +ORDER BY event_time desc +LIMIT 1 SETTINGS use_query_condition_cache=0; + +DROP TABLE t_parquet_03262; From c0803b67be6a1c7f9e0ce9a5622116f8bf4f6e5d Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Wed, 4 Mar 2026 15:43:12 +0300 Subject: [PATCH 02/15] Merge pull request #1388 from Altinity/fp_antalya_26_1_export_part_partition Antalya 26.1 - Forward port of export part and partition Source-PR: #1388 (https://github.com/Altinity/ClickHouse/pull/1388) --- .../mergetree-family/part_export.md | 307 +++++ .../mergetree-family/partition_export.md | 182 +++ docs/en/operations/system-tables/exports.md | 56 + src/Access/Common/AccessType.h | 2 + src/CMakeLists.txt | 1 + src/Common/CurrentMetrics.cpp | 1 + src/Common/ErrorCodes.cpp | 3 +- src/Common/ProfileEvents.cpp | 6 + src/Common/setThreadName.h | 1 + src/Core/ServerSettings.cpp | 4 +- src/Core/Settings.cpp | 36 + src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.cpp | 24 + src/Core/SettingsEnums.cpp | 2 + src/Core/SettingsEnums.h | 10 + src/Databases/DatabaseReplicated.cpp | 2 +- .../ObjectStorages/IObjectStorage.h | 1 + src/Functions/generateSnowflakeID.cpp | 5 + src/Functions/generateSnowflakeID.h | 2 + src/Interpreters/Context.cpp | 24 + src/Interpreters/Context.h | 6 + src/Interpreters/DDLWorker.cpp | 3 +- src/Interpreters/InterpreterAlterQuery.cpp | 14 + .../InterpreterKillQueryQuery.cpp | 86 ++ src/Interpreters/PartLog.cpp | 11 +- src/Interpreters/PartLog.h | 2 + src/Interpreters/executeDDLQueryOnCluster.cpp | 2 + src/Parsers/ASTAlterQuery.cpp | 54 + src/Parsers/ASTAlterQuery.h | 7 + src/Parsers/ASTKillQueryQuery.cpp | 3 + src/Parsers/ASTKillQueryQuery.h | 1 + src/Parsers/CommonParsers.h | 2 + src/Parsers/ParserAlterQuery.cpp | 61 + src/Parsers/ParserKillQueryQuery.cpp | 3 + src/Storages/ColumnsDescription.cpp | 10 +- src/Storages/ColumnsDescription.h | 1 + ...portReplicatedMergeTreePartitionManifest.h | 191 +++ ...ortReplicatedMergeTreePartitionTaskEntry.h | 78 ++ src/Storages/IPartitionStrategy.cpp | 59 +- src/Storages/IPartitionStrategy.h | 11 +- src/Storages/IStorage.cpp | 5 + src/Storages/IStorage.h | 36 + src/Storages/MergeTree/ExportList.cpp | 74 ++ src/Storages/MergeTree/ExportList.h | 96 ++ src/Storages/MergeTree/ExportPartTask.cpp | 351 ++++++ src/Storages/MergeTree/ExportPartTask.h | 33 + .../ExportPartitionManifestUpdatingTask.cpp | 310 +++++ .../ExportPartitionManifestUpdatingTask.h | 42 + .../ExportPartitionTaskScheduler.cpp | 402 ++++++ .../MergeTree/ExportPartitionTaskScheduler.h | 66 + .../MergeTree/ExportPartitionUtils.cpp | 104 ++ src/Storages/MergeTree/ExportPartitionUtils.h | 28 + src/Storages/MergeTree/IMergeTreeDataPart.cpp | 36 + src/Storages/MergeTree/IMergeTreeDataPart.h | 2 + src/Storages/MergeTree/MergeTreeData.cpp | 318 ++++- src/Storages/MergeTree/MergeTreeData.h | 41 +- .../MergeTree/MergeTreeExportManifest.h | 50 + .../MergeTree/MergeTreePartExportManifest.h | 94 ++ .../MergeTree/MergeTreePartExportStatus.h | 20 + src/Storages/MergeTree/MergeTreePartition.cpp | 16 + src/Storages/MergeTree/MergeTreePartition.h | 2 + .../MergeTree/MergeTreeSequentialSource.cpp | 4 + .../MergeTree/MergeTreeSequentialSource.h | 1 + .../ReplicatedMergeTreeRestartingThread.cpp | 13 + .../tests/gtest_export_partition_ordering.cpp | 75 ++ .../MultiFileStorageObjectStorageSink.cpp | 153 +++ .../MultiFileStorageObjectStorageSink.h | 57 + .../ObjectStorageFilePathGenerator.h | 83 ++ .../ObjectStorage/StorageObjectStorage.cpp | 78 +- .../ObjectStorage/StorageObjectStorage.h | 20 + .../StorageObjectStorageCluster.cpp | 6 + .../StorageObjectStorageConfiguration.cpp | 36 +- .../StorageObjectStorageConfiguration.h | 9 +- .../StorageObjectStorageSink.cpp | 15 +- .../ObjectStorage/StorageObjectStorageSink.h | 6 +- src/Storages/PartitionCommands.cpp | 35 + src/Storages/PartitionCommands.h | 8 +- src/Storages/PartitionedSink.cpp | 4 +- src/Storages/PartitionedSink.h | 12 +- src/Storages/StorageFile.cpp | 20 +- src/Storages/StorageMergeTree.cpp | 14 +- src/Storages/StorageMergeTree.h | 2 - src/Storages/StorageReplicatedMergeTree.cpp | 539 ++++++++- src/Storages/StorageReplicatedMergeTree.h | 44 +- src/Storages/StorageURL.cpp | 12 +- src/Storages/System/StorageSystemExports.cpp | 72 ++ src/Storages/System/StorageSystemExports.h | 25 + src/Storages/System/StorageSystemMerges.cpp | 2 +- ...torageSystemReplicatedPartitionExports.cpp | 145 +++ .../StorageSystemReplicatedPartitionExports.h | 43 + src/Storages/System/attachSystemTables.cpp | 15 +- ...perimental_export_merge_tree_partition.xml | 3 + tests/config/install.sh | 1 + .../__init__.py | 0 .../configs/named_collections.xml | 9 + .../test.py | 286 +++++ .../__init__.py | 0 .../allow_experimental_export_partition.xml | 3 + .../disable_experimental_export_partition.xml | 3 + .../configs/named_collections.xml | 9 + .../configs/s3_retries.xml | 3 + .../configs/users.d/profile.xml | 8 + .../test.py | 1074 +++++++++++++++++ .../01271_show_privileges.reference | 2 + ...21_system_zookeeper_unrestricted.reference | 2 + ...stem_zookeeper_unrestricted_like.reference | 2 + ..._settings_cannot_be_enabled_by_default.sql | 2 +- ...572_export_merge_tree_part_basic.reference | 35 + .../03572_export_merge_tree_part_basic.sh | 85 ++ ..._part_limits_and_table_functions.reference | 18 + ...ge_tree_part_limits_and_table_functions.sh | 85 ++ ..._merge_tree_part_special_columns.reference | 39 + ..._export_merge_tree_part_special_columns.sh | 154 +++ ...ee_part_to_object_storage_simple.reference | 0 ...rge_tree_part_to_object_storage_simple.sql | 39 + ...erge_tree_part_to_object_storage.reference | 16 + ...cated_merge_tree_part_to_object_storage.sh | 44 + ...ee_part_to_object_storage_simple.reference | 0 ...rge_tree_part_to_object_storage_simple.sql | 22 + ...3604_export_merge_tree_partition.reference | 31 + .../03604_export_merge_tree_partition.sh | 57 + ..._system_background_schedule_pool.reference | 1 + 122 files changed, 6845 insertions(+), 137 deletions(-) create mode 100644 docs/en/engines/table-engines/mergetree-family/part_export.md create mode 100644 docs/en/engines/table-engines/mergetree-family/partition_export.md create mode 100644 docs/en/operations/system-tables/exports.md create mode 100644 src/Storages/ExportReplicatedMergeTreePartitionManifest.h create mode 100644 src/Storages/ExportReplicatedMergeTreePartitionTaskEntry.h create mode 100644 src/Storages/MergeTree/ExportList.cpp create mode 100644 src/Storages/MergeTree/ExportList.h create mode 100644 src/Storages/MergeTree/ExportPartTask.cpp create mode 100644 src/Storages/MergeTree/ExportPartTask.h create mode 100644 src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp create mode 100644 src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.h create mode 100644 src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp create mode 100644 src/Storages/MergeTree/ExportPartitionTaskScheduler.h create mode 100644 src/Storages/MergeTree/ExportPartitionUtils.cpp create mode 100644 src/Storages/MergeTree/ExportPartitionUtils.h create mode 100644 src/Storages/MergeTree/MergeTreeExportManifest.h create mode 100644 src/Storages/MergeTree/MergeTreePartExportManifest.h create mode 100644 src/Storages/MergeTree/MergeTreePartExportStatus.h create mode 100644 src/Storages/MergeTree/tests/gtest_export_partition_ordering.cpp create mode 100644 src/Storages/ObjectStorage/MultiFileStorageObjectStorageSink.cpp create mode 100644 src/Storages/ObjectStorage/MultiFileStorageObjectStorageSink.h create mode 100644 src/Storages/ObjectStorage/ObjectStorageFilePathGenerator.h create mode 100644 src/Storages/System/StorageSystemExports.cpp create mode 100644 src/Storages/System/StorageSystemExports.h create mode 100644 src/Storages/System/StorageSystemReplicatedPartitionExports.cpp create mode 100644 src/Storages/System/StorageSystemReplicatedPartitionExports.h create mode 100644 tests/config/config.d/enable_experimental_export_merge_tree_partition.xml create mode 100644 tests/integration/test_export_merge_tree_part_to_object_storage/__init__.py create mode 100644 tests/integration/test_export_merge_tree_part_to_object_storage/configs/named_collections.xml create mode 100644 tests/integration/test_export_merge_tree_part_to_object_storage/test.py create mode 100644 tests/integration/test_export_replicated_mt_partition_to_object_storage/__init__.py create mode 100644 tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/allow_experimental_export_partition.xml create mode 100644 tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/disable_experimental_export_partition.xml create mode 100644 tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/named_collections.xml create mode 100644 tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/s3_retries.xml create mode 100644 tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/users.d/profile.xml create mode 100644 tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py create mode 100644 tests/queries/0_stateless/03572_export_merge_tree_part_basic.reference create mode 100755 tests/queries/0_stateless/03572_export_merge_tree_part_basic.sh create mode 100644 tests/queries/0_stateless/03572_export_merge_tree_part_limits_and_table_functions.reference create mode 100755 tests/queries/0_stateless/03572_export_merge_tree_part_limits_and_table_functions.sh create mode 100644 tests/queries/0_stateless/03572_export_merge_tree_part_special_columns.reference create mode 100755 tests/queries/0_stateless/03572_export_merge_tree_part_special_columns.sh create mode 100644 tests/queries/0_stateless/03572_export_merge_tree_part_to_object_storage_simple.reference create mode 100644 tests/queries/0_stateless/03572_export_merge_tree_part_to_object_storage_simple.sql create mode 100644 tests/queries/0_stateless/03572_export_replicated_merge_tree_part_to_object_storage.reference create mode 100755 tests/queries/0_stateless/03572_export_replicated_merge_tree_part_to_object_storage.sh create mode 100644 tests/queries/0_stateless/03572_export_replicated_merge_tree_part_to_object_storage_simple.reference create mode 100644 tests/queries/0_stateless/03572_export_replicated_merge_tree_part_to_object_storage_simple.sql create mode 100644 tests/queries/0_stateless/03604_export_merge_tree_partition.reference create mode 100755 tests/queries/0_stateless/03604_export_merge_tree_partition.sh diff --git a/docs/en/engines/table-engines/mergetree-family/part_export.md b/docs/en/engines/table-engines/mergetree-family/part_export.md new file mode 100644 index 000000000000..287e0a17f3af --- /dev/null +++ b/docs/en/engines/table-engines/mergetree-family/part_export.md @@ -0,0 +1,307 @@ +# ALTER TABLE EXPORT PART + +## Overview + +The `ALTER TABLE EXPORT PART` command exports individual MergeTree data parts to object storage (S3, Azure Blob Storage, etc.), typically in Parquet format. A commit file is shipped to the same destination directory containing all data files exported within that transaction. + +**Key Characteristics:** +- **Experimental feature** - must be enabled via `allow_experimental_export_merge_tree_part` setting +- **Asynchronous** - executes in the background, returns immediately +- **Ephemeral** - no automatic retry mechanism; manual retry required on failure +- **Idempotent** - safe to re-export the same part (skips by default if file exists) +- **Preserves sort order** from the source table + +## Syntax + +```sql +ALTER TABLE [database.]table_name +EXPORT PART 'part_name' +TO TABLE [destination_database.]destination_table +SETTINGS allow_experimental_export_merge_tree_part = 1 + [, setting_name = value, ...] +``` + +## Syntax with table function + +```sql +ALTER TABLE [database.]table_name +EXPORT PART 'part_name' +TO TABLE FUNCTION s3(s3_conn, filename='table_function', partition_strategy...) +SETTINGS allow_experimental_export_merge_tree_part = 1 + [, setting_name = value, ...] +``` + +### Parameters + +- **`table_name`**: The source MergeTree table containing the part to export +- **`part_name`**: The exact name of the data part to export (e.g., `'2020_1_1_0'`, `'all_1_1_0'`) +- **`destination_table`**: The target table for the export (typically an S3, Azure, or other object storage table) + +## Requirements + +Source and destination tables must be 100% compatible: + +1. **Identical schemas** - same columns, types, and order +2. **Matching partition keys** - partition expressions must be identical + +In case a table function is used as the destination, the schema can be omitted and it will be inferred from the source table. + +## Settings + +### `allow_experimental_export_merge_tree_part` (Required) + +- **Type**: `Bool` +- **Default**: `false` +- **Description**: Must be set to `true` to enable the experimental feature. + +### `export_merge_tree_part_overwrite_file_if_exists` (Optional) + +- **Type**: `Bool` +- **Default**: `false` +- **Description**: If set to `true`, it will overwrite the file. Otherwise, fails with exception. + +### `export_merge_tree_part_max_bytes_per_file` (Optional) + +- **Type**: `UInt64` +- **Default**: `0` +- **Description**: Maximum number of bytes to write to a single file when exporting a merge tree part. 0 means no limit. This is not a hard limit, and it highly depends on the output format granularity and input source chunk size. Using this might break idempotency, use it with care. + +### `export_merge_tree_part_max_rows_per_file` (Optional) + +- **Type**: `UInt64` +- **Default**: `0` +- **Description**: Maximum number of rows to write to a single file when exporting a merge tree part. 0 means no limit. This is not a hard limit, and it highly depends on the output format granularity and input source chunk size. Using this might break idempotency, use it with care. + +### export_merge_tree_part_throw_on_pending_mutations + +- **Type**: `bool` +- **Default**: `true` +- **Description**: If set to true, throws if pending mutations exists for a given part. Note that by default mutations are applied to all parts, which means that if a mutation in practice would only affetct part/partition x, all the other parts/partition will throw upon export. The exception is when the `IN PARTITION` clause was used in the mutation command. Note the `IN PARTITION` clause is not properly implemented for plain MergeTree tables. + +### export_merge_tree_part_throw_on_pending_patch_parts + +- **Type**: `bool` +- **Default**: `true` +- **Description**: If set to true, throws if pending patch parts exists for a given part. Note that by default mutations are applied to all parts, which means that if a mutation in practice would only affetct part/partition x, all the other parts/partition will throw upon export. The exception is when the `IN PARTITION` clause was used in the mutation command. Note the `IN PARTITION` clause is not properly implemented for plain MergeTree tables. + +## Examples + +### Basic Export to S3 + +```sql +-- Create source and destination tables +CREATE TABLE mt_table (id UInt64, year UInt16) +ENGINE = MergeTree() PARTITION BY year ORDER BY tuple(); + +CREATE TABLE s3_table (id UInt64, year UInt16) +ENGINE = S3(s3_conn, filename='data', format=Parquet, partition_strategy='hive') +PARTITION BY year; + +-- Insert and export +INSERT INTO mt_table VALUES (1, 2020), (2, 2020), (3, 2021); + +ALTER TABLE mt_table EXPORT PART '2020_1_1_0' TO TABLE s3_table +SETTINGS allow_experimental_export_merge_tree_part = 1; + +ALTER TABLE mt_table EXPORT PART '2021_2_2_0' TO TABLE s3_table +SETTINGS allow_experimental_export_merge_tree_part = 1; +``` + +### Table function export + +```sql +-- Create source and destination tables +CREATE TABLE mt_table (id UInt64, year UInt16) +ENGINE = MergeTree() PARTITION BY year ORDER BY tuple(); + +-- Insert and export +INSERT INTO mt_table VALUES (1, 2020), (2, 2020), (3, 2021); + +ALTER TABLE mt_table EXPORT PART '2020_1_1_0' TO TABLE FUNCTION s3(s3_conn, filename='table_function', format=Parquet, partition_strategy='hive') PARTITION BY year +SETTINGS allow_experimental_export_merge_tree_part = 1; +``` + +## Monitoring + +### Active Exports + +Active exports can be found in the `system.exports` table. As of now, it only shows currently executing exports. It will not show pending or finished exports. + +```sql +arthur :) select * from system.exports; + +SELECT * +FROM system.exports + +Query id: 2026718c-d249-4208-891b-a271f1f93407 + +Row 1: +────── +source_database: default +source_table: source_mt_table +destination_database: default +destination_table: destination_table +create_time: 2025-11-19 09:09:11 +part_name: 20251016-365_1_1_0 +destination_file_paths: ['table_root/eventDate=2025-10-16/retention=365/20251016-365_1_1_0_17B2F6CD5D3C18E787C07AE3DAF16EB1.1.parquet'] +elapsed: 2.04845441 +rows_read: 1138688 -- 1.14 million +total_rows_to_read: 550961374 -- 550.96 million +total_size_bytes_compressed: 37619147120 -- 37.62 billion +total_size_bytes_uncompressed: 138166213721 -- 138.17 billion +bytes_read_uncompressed: 316892925 -- 316.89 million +memory_usage: 596006095 -- 596.01 million +peak_memory_usage: 601239033 -- 601.24 million +``` + +### Export History + +You can query succeeded or failed exports in `system.part_log`. For now, it only keeps track of completion events (either success or fails). + +```sql +arthur :) select * from system.part_log where event_type='ExportPart' and table = 'replicated_source' order by event_time desc limit 1; + +SELECT * +FROM system.part_log +WHERE (event_type = 'ExportPart') AND (`table` = 'replicated_source') +ORDER BY event_time DESC +LIMIT 1 + +Query id: ae1c1cd3-c20e-4f20-8b82-ed1f6af0237f + +Row 1: +────── +hostname: arthur +query_id: +event_type: ExportPart +merge_reason: NotAMerge +merge_algorithm: Undecided +event_date: 2025-11-19 +event_time: 2025-11-19 09:08:31 +event_time_microseconds: 2025-11-19 09:08:31.974701 +duration_ms: 4 +database: default +table: replicated_source +table_uuid: 78471c67-24f4-4398-9df5-ad0a6c3daf41 +part_name: 2021_0_0_0 +partition_id: 2021 +partition: 2021 +part_type: Compact +disk_name: default +path_on_disk: +remote_file_paths ['year=2021/2021_0_0_0_78C704B133D41CB0EF64DD2A9ED3B6BA.1.parquet'] +rows: 1 +size_in_bytes: 272 +merged_from: ['2021_0_0_0'] +bytes_uncompressed: 86 +read_rows: 1 +read_bytes: 6 +peak_memory_usage: 22 +error: 0 +exception: +ProfileEvents: {} +``` + +### Profile Events + +- `PartsExports` - Successful exports +- `PartsExportFailures` - Failed exports +- `PartsExportDuplicated` - Number of part exports that failed because target already exists. +- `PartsExportTotalMilliseconds` - Total time + +### Split large files + +```sql +alter table big_table export part '2025_0_32_3' to table replicated_big_destination SETTINGS export_merge_tree_part_max_bytes_per_file=10000000, output_format_parquet_row_group_size_bytes=5000000; + +arthur :) select * from system.exports; + +SELECT * +FROM system.exports + +Query id: d78d9ce5-cfbc-4957-b7dd-bc8129811634 + +Row 1: +────── +source_database: default +source_table: big_table +destination_database: default +destination_table: replicated_big_destination +create_time: 2025-12-15 13:12:48 +part_name: 2025_0_32_3 +destination_file_paths: ['replicated_big/year=2025/2025_0_32_3_E439C23833C39C6E5104F6F4D1048BE7.1.parquet','replicated_big/year=2025/2025_0_32_3_E439C23833C39C6E5104F6F4D1048BE7.2.parquet','replicated_big/year=2025/2025_0_32_3_E439C23833C39C6E5104F6F4D1048BE7.3.parquet','replicated_big/year=2025/2025_0_32_3_E439C23833C39C6E5104F6F4D1048BE7.4.parquet'] +elapsed: 14.360427274 +rows_read: 10256384 -- 10.26 million +total_rows_to_read: 10485760 -- 10.49 million +total_size_bytes_compressed: 83779395 -- 83.78 million +total_size_bytes_uncompressed: 10611691600 -- 10.61 billion +bytes_read_uncompressed: 10440998912 -- 10.44 billion +memory_usage: 89795477 -- 89.80 million +peak_memory_usage: 107362133 -- 107.36 million + +1 row in set. Elapsed: 0.014 sec. + +arthur :) select * from system.part_log where event_type = 'ExportPart' order by event_time desc limit 1 format Vertical; + +SELECT * +FROM system.part_log +WHERE event_type = 'ExportPart' +ORDER BY event_time DESC +LIMIT 1 +FORMAT Vertical + +Query id: 95128b01-b751-4726-8e3e-320728ac6af7 + +Row 1: +────── +hostname: arthur +query_id: +event_type: ExportPart +merge_reason: NotAMerge +merge_algorithm: Undecided +event_date: 2025-12-15 +event_time: 2025-12-15 13:13:03 +event_time_microseconds: 2025-12-15 13:13:03.197492 +duration_ms: 14673 +database: default +table: big_table +table_uuid: a3eeeea0-295c-41a3-84ef-6b5463dbbe8c +part_name: 2025_0_32_3 +partition_id: 2025 +partition: 2025 +part_type: Wide +disk_name: default +path_on_disk: ./store/a3e/a3eeeea0-295c-41a3-84ef-6b5463dbbe8c/2025_0_32_3/ +remote_file_paths: ['replicated_big/year=2025/2025_0_32_3_E439C23833C39C6E5104F6F4D1048BE7.1.parquet','replicated_big/year=2025/2025_0_32_3_E439C23833C39C6E5104F6F4D1048BE7.2.parquet','replicated_big/year=2025/2025_0_32_3_E439C23833C39C6E5104F6F4D1048BE7.3.parquet','replicated_big/year=2025/2025_0_32_3_E439C23833C39C6E5104F6F4D1048BE7.4.parquet'] +rows: 10485760 -- 10.49 million +size_in_bytes: 83779395 -- 83.78 million +merged_from: ['2025_0_32_3'] +bytes_uncompressed: 10611691600 -- 10.61 billion +read_rows: 10485760 -- 10.49 million +read_bytes: 10674503680 -- 10.67 billion +peak_memory_usage: 107362133 -- 107.36 million +error: 0 +exception: +ProfileEvents: {} + +1 row in set. Elapsed: 0.044 sec. + +arthur :) select _path, formatReadableSize(_size) as _size from s3(s3_conn, filename='**', format=One); + +SELECT + _path, + formatReadableSize(_size) AS _size +FROM s3(s3_conn, filename = '**', format = One) + +Query id: c48ae709-f590-4d1b-8158-191f8d628966 + + ┌─_path────────────────────────────────────────────────────────────────────────────────┬─_size─────┐ +1. │ test/replicated_big/year=2025/2025_0_32_3_E439C23833C39C6E5104F6F4D1048BE7.1.parquet │ 17.36 MiB │ +2. │ test/replicated_big/year=2025/2025_0_32_3_E439C23833C39C6E5104F6F4D1048BE7.2.parquet │ 17.32 MiB │ +3. │ test/replicated_big/year=2025/2025_0_32_3_E439C23833C39C6E5104F6F4D1048BE7.4.parquet │ 5.04 MiB │ +4. │ test/replicated_big/year=2025/2025_0_32_3_E439C23833C39C6E5104F6F4D1048BE7.3.parquet │ 17.40 MiB │ +5. │ test/replicated_big/year=2025/commit_2025_0_32_3_E439C23833C39C6E5104F6F4D1048BE7 │ 320.00 B │ + └──────────────────────────────────────────────────────────────────────────────────────┴───────────┘ + +5 rows in set. Elapsed: 0.072 sec. +``` diff --git a/docs/en/engines/table-engines/mergetree-family/partition_export.md b/docs/en/engines/table-engines/mergetree-family/partition_export.md new file mode 100644 index 000000000000..d91f226dbbf6 --- /dev/null +++ b/docs/en/engines/table-engines/mergetree-family/partition_export.md @@ -0,0 +1,182 @@ +# ALTER TABLE EXPORT PARTITION + +## Overview + +The `ALTER TABLE EXPORT PARTITION` command exports entire partitions from Replicated*MergeTree tables to object storage (S3, Azure Blob Storage, etc.), typically in Parquet format. This feature coordinates export part operations across all replicas using ZooKeeper. + +Each MergeTree part will become a separate file with the following name convention: `//_.`. To ensure atomicity, a commit file containing the relative paths of all exported parts is also shipped. A data file should only be considered part of the dataset if a commit file references it. The commit file will be named using the following convention: `/commit__`. + +The set of parts that are exported is based on the list of parts the replica that received the export command sees. The other replicas will assist in the export process if they have those parts locally. Otherwise they will ignore it. + +The partition export tasks can be observed through `system.replicated_partition_exports`. Querying this table results in a query to ZooKeeper, so it must be used with care. Individual part export progress can be observed as usual through `system.exports`. + +The same partition can not be exported to the same destination more than once. There are two ways to override this behavior: either by setting the `export_merge_tree_partition_force_export` setting or waiting for the task to expire. + +The export task can be killed by issuing the kill command: `KILL EXPORT PARTITION `. + +The task is persistent - it should be resumed after crashes, failures and etc. + +## Syntax + +```sql +ALTER TABLE [database.]table_name +EXPORT PARTITION ID 'partition_id' +TO TABLE [destination_database.]destination_table +[SETTINGS setting_name = value, ...] +``` + +### Parameters + +- **`table_name`**: The source Replicated*MergeTree table containing the partition to export +- **`partition_id`**: The partition identifier to export (e.g., `'2020'`, `'2021'`) +- **`destination_table`**: The target table for the export (typically an S3, Azure, or other object storage table) + +## Settings + +### Server Settings + +#### `enable_experimental_export_merge_tree_partition_feature` (Required) + +- **Type**: `Bool` +- **Default**: `false` +- **Description**: Enable export replicated merge tree partition feature. It is experimental and not yet ready for production use. + +### Query Settings + +#### `export_merge_tree_partition_force_export` (Optional) + +- **Type**: `Bool` +- **Default**: `false` +- **Description**: Ignore existing partition export and overwrite the ZooKeeper entry. Allows re-exporting a partition to the same destination before the manifest expires. + +#### `export_merge_tree_partition_max_retries` (Optional) + +- **Type**: `UInt64` +- **Default**: `3` +- **Description**: Maximum number of retries for exporting a merge tree part in an export partition task. If it exceeds, the entire task fails. + +#### `export_merge_tree_partition_manifest_ttl` (Optional) + +- **Type**: `UInt64` +- **Default**: `180` (seconds) +- **Description**: Determines how long the manifest will live in ZooKeeper. It prevents the same partition from being exported twice to the same destination. This setting does not affect or delete in-progress tasks; it only cleans up completed ones. + +#### `export_merge_tree_part_file_already_exists_policy` (Optional) + +- **Type**: `MergeTreePartExportFileAlreadyExistsPolicy` +- **Default**: `skip` +- **Description**: Policy for handling files that already exist during export. Possible values: + - `skip` - Skip the file if it already exists + - `error` - Throw an error if the file already exists + - `overwrite` - Overwrite the file + +### export_merge_tree_part_throw_on_pending_mutations + +- **Type**: `bool` +- **Default**: `true` +- **Description**: If set to true, throws if pending mutations exists for a given part. Note that by default mutations are applied to all parts, which means that if a mutation in practice would only affetct part/partition x, all the other parts/partition will throw upon export. The exception is when the `IN PARTITION` clause was used in the mutation command. Note the `IN PARTITION` clause is not properly implemented for plain MergeTree tables. + +### export_merge_tree_part_throw_on_pending_patch_parts + +- **Type**: `bool` +- **Default**: `true` +- **Description**: If set to true, throws if pending patch parts exists for a given part. Note that by default mutations are applied to all parts, which means that if a mutation in practice would only affetct part/partition x, all the other parts/partition will throw upon export. The exception is when the `IN PARTITION` clause was used in the mutation command. Note the `IN PARTITION` clause is not properly implemented for plain MergeTree tables. + +## Examples + +### Basic Export to S3 + +```sql +CREATE TABLE rmt_table (id UInt64, year UInt16) +ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/rmt_table', 'replica1') +PARTITION BY year ORDER BY tuple(); + +CREATE TABLE s3_table (id UInt64, year UInt16) +ENGINE = S3(s3_conn, filename='data', format=Parquet, partition_strategy='hive') +PARTITION BY year; + +INSERT INTO rmt_table VALUES (1, 2020), (2, 2020), (3, 2020), (4, 2021); + +ALTER TABLE rmt_table EXPORT PARTITION ID '2020' TO TABLE s3_table; + +## Killing Exports + +You can cancel in-progress partition exports using the `KILL EXPORT PARTITION` command: + +```sql +KILL EXPORT PARTITION +WHERE partition_id = '2020' + AND source_table = 'rmt_table' + AND destination_table = 's3_table' +``` + +The `WHERE` clause filters exports from the `system.replicated_partition_exports` table. You can use any columns from that table in the filter. + +## Monitoring + +### Active and Completed Exports + +Monitor partition exports using the `system.replicated_partition_exports` table: + +```sql +arthur :) select * from system.replicated_partition_exports Format Vertical; + +SELECT * +FROM system.replicated_partition_exports +FORMAT Vertical + +Query id: 9efc271a-a501-44d1-834f-bc4d20156164 + +Row 1: +────── +source_database: default +source_table: replicated_source +destination_database: default +destination_table: replicated_destination +create_time: 2025-11-21 18:21:51 +partition_id: 2022 +transaction_id: 7397746091717128192 +source_replica: r1 +parts: ['2022_0_0_0','2022_1_1_0','2022_2_2_0'] +parts_count: 3 +parts_to_do: 0 +status: COMPLETED +exception_replica: +last_exception: +exception_part: +exception_count: 0 + +Row 2: +────── +source_database: default +source_table: replicated_source +destination_database: default +destination_table: replicated_destination +create_time: 2025-11-21 18:20:35 +partition_id: 2021 +transaction_id: 7397745772618674176 +source_replica: r1 +parts: ['2021_0_0_0'] +parts_count: 1 +parts_to_do: 0 +status: COMPLETED +exception_replica: +last_exception: +exception_part: +exception_count: 0 + +2 rows in set. Elapsed: 0.019 sec. + +arthur :) +``` + +Status values include: +- `PENDING` - Export is queued / in progress +- `COMPLETED` - Export finished successfully +- `FAILED` - Export failed +- `KILLED` - Export was cancelled + +## Related Features + +- [ALTER TABLE EXPORT PART](/docs/en/engines/table-engines/mergetree-family/part_export.md) - Export individual parts (non-replicated) + diff --git a/docs/en/operations/system-tables/exports.md b/docs/en/operations/system-tables/exports.md new file mode 100644 index 000000000000..e26514364008 --- /dev/null +++ b/docs/en/operations/system-tables/exports.md @@ -0,0 +1,56 @@ +--- +description: 'System table containing information about in progress merge tree part exports' +keywords: ['system table', 'exports', 'merge tree', 'part'] +slug: /operations/system-tables/exports +title: 'system.exports' +--- + +Contains information about in progress merge tree part exports + +Columns: + +- `source_database` ([String](/docs/en/sql-reference/data-types/string.md)) — Name of the source database. +- `source_table` ([String](/docs/en/sql-reference/data-types/string.md)) — Name of the source table. +- `destination_database` ([String](/docs/en/sql-reference/data-types/string.md)) — Name of the destination database. +- `destination_table` ([String](/docs/en/sql-reference/data-types/string.md)) — Name of the destination table. +- `create_time` ([DateTime](/docs/en/sql-reference/data-types/datetime.md)) — Date and time when the export command was received in the server. +- `part_name` ([String](/docs/en/sql-reference/data-types/string.md)) — Name of the part. +- `destination_file_path` ([String](/docs/en/sql-reference/data-types/string.md)) — File path relative to where the part is being exported to. +- `elapsed` ([Float64](/docs/en/sql-reference/data-types/float.md)) — The time elapsed (in seconds) since the export started. +- `rows_read` ([UInt64](/docs/en/sql-reference/data-types/int-uint.md)) — The number of rows read from the exported part. +- `total_rows_to_read` ([UInt64](/docs/en/sql-reference/data-types/int-uint.md)) — The total number of rows to read from the exported part. +- `total_size_bytes_compressed` ([UInt64](/docs/en/sql-reference/data-types/int-uint.md)) — The total size of the compressed data in the exported part. +- `total_size_bytes_uncompressed` ([UInt64](/docs/en/sql-reference/data-types/int-uint.md)) — The total size of the uncompressed data in the exported part. +- `bytes_read_uncompressed` ([UInt64](/docs/en/sql-reference/data-types/int-uint.md)) — The number of uncompressed bytes read from the exported part. +- `memory_usage` ([UInt64](/docs/en/sql-reference/data-types/int-uint.md)) — Current memory usage in bytes for the export operation. +- `peak_memory_usage` ([UInt64](/docs/en/sql-reference/data-types/int-uint.md)) — Peak memory usage in bytes during the export operation. + +**Example** + +```sql +arthur :) select * from system.exports; + +SELECT * +FROM system.exports + +Query id: 2026718c-d249-4208-891b-a271f1f93407 + +Row 1: +────── +source_database: default +source_table: source_mt_table +destination_database: default +destination_table: destination_table +create_time: 2025-11-19 09:09:11 +part_name: 20251016-365_1_1_0 +destination_file_path: table_root/eventDate=2025-10-16/retention=365/20251016-365_1_1_0_17B2F6CD5D3C18E787C07AE3DAF16EB1.parquet +elapsed: 2.04845441 +rows_read: 1138688 -- 1.14 million +total_rows_to_read: 550961374 -- 550.96 million +total_size_bytes_compressed: 37619147120 -- 37.62 billion +total_size_bytes_uncompressed: 138166213721 -- 138.17 billion +bytes_read_uncompressed: 316892925 -- 316.89 million +memory_usage: 596006095 -- 596.01 million +peak_memory_usage: 601239033 -- 601.24 million +``` + diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index 3ae01fb44fce..77d8ed5f5070 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -213,6 +213,8 @@ enum class AccessType : uint8_t M(ALTER_REWRITE_PARTS, "REWRITE PARTS", TABLE, ALTER_TABLE) /* allows to execute ALTER REWRITE PARTS */\ M(ALTER_SETTINGS, "ALTER SETTING, ALTER MODIFY SETTING, MODIFY SETTING, RESET SETTING", TABLE, ALTER_TABLE) /* allows to execute ALTER MODIFY SETTING */\ M(ALTER_MOVE_PARTITION, "ALTER MOVE PART, MOVE PARTITION, MOVE PART", TABLE, ALTER_TABLE) \ + M(ALTER_EXPORT_PART, "ALTER EXPORT PART, EXPORT PART", TABLE, ALTER_TABLE) \ + M(ALTER_EXPORT_PARTITION, "ALTER EXPORT PARTITION, EXPORT PARTITION", TABLE, ALTER_TABLE) \ M(ALTER_FETCH_PARTITION, "ALTER FETCH PART, FETCH PARTITION", TABLE, ALTER_TABLE) \ M(ALTER_FREEZE_PARTITION, "FREEZE PARTITION, UNFREEZE", TABLE, ALTER_TABLE) \ M(ALTER_UNLOCK_SNAPSHOT, "UNLOCK SNAPSHOT", TABLE, ALTER_TABLE) \ diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 88dcaa1fa6ab..1c85cc4e0e0b 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -139,6 +139,7 @@ add_headers_and_sources(dbms Storages/ObjectStorage/Azure) add_headers_and_sources(dbms Storages/ObjectStorage/S3) add_headers_and_sources(dbms Storages/ObjectStorage/HDFS) add_headers_and_sources(dbms Storages/ObjectStorage/Local) +add_headers_and_sources(dbms Storages/ObjectStorage/MergeTree) add_headers_and_sources(dbms Storages/ObjectStorage/DataLakes) add_headers_and_sources(dbms Storages/ObjectStorage/DataLakes/Common) add_headers_and_sources(dbms Storages/ObjectStorage/DataLakes/Iceberg) diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 84d3b3c5bd0e..510ed321e0d7 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -12,6 +12,7 @@ M(Merge, "Number of executing background merges") \ M(MergeParts, "Number of source parts participating in current background merges") \ M(Move, "Number of currently executing moves") \ + M(Export, "Number of currently executing exports") \ M(PartMutation, "Number of mutations (ALTER DELETE/UPDATE)") \ M(ReplicatedFetch, "Number of data parts being fetched from replica") \ M(ReplicatedSend, "Number of data parts being sent to replicas") \ diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 6bfe68a6a24e..423ec04acb49 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -663,6 +663,7 @@ M(1002, UNKNOWN_EXCEPTION) \ M(1003, SSH_EXCEPTION) \ M(1004, STARTUP_SCRIPTS_ERROR) \ + M(1005, PENDING_MUTATIONS_NOT_ALLOWED) \ /* See END */ #ifdef APPLY_FOR_EXTERNAL_ERROR_CODES @@ -679,7 +680,7 @@ namespace ErrorCodes APPLY_FOR_ERROR_CODES(M) #undef M - constexpr ErrorCode END = 1004; + constexpr ErrorCode END = 1005; ErrorPairHolder values[END + 1]{}; struct ErrorCodesNames diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 3aa22a4fc217..2ffaa114d527 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -39,6 +39,10 @@ M(FailedInitialQuery, "Number of failed initial queries.", ValueType::Number) \ M(FailedInitialSelectQuery, "Same as FailedInitialQuery, but only for SELECT queries.", ValueType::Number) \ M(FailedQuery, "Number of total failed queries, both internal and user queries.", ValueType::Number) \ + M(PartsExports, "Number of successful part exports.", ValueType::Number) \ + M(PartsExportFailures, "Number of failed part exports.", ValueType::Number) \ + M(PartsExportDuplicated, "Number of part exports that failed because target already exists.", ValueType::Number) \ + M(PartsExportTotalMilliseconds, "Total time spent on part export operations.", ValueType::Milliseconds) \ M(FailedSelectQuery, "Same as FailedQuery, but only for SELECT queries.", ValueType::Number) \ M(FailedInsertQuery, "Same as FailedQuery, but only for INSERT queries.", ValueType::Number) \ M(FailedAsyncInsertQuery, "Number of failed ASYNC INSERT queries.", ValueType::Number) \ @@ -223,6 +227,8 @@ M(MergesThrottlerSleepMicroseconds, "Total time a query was sleeping to conform 'max_merges_bandwidth_for_server' throttling.", ValueType::Microseconds) \ M(MutationsThrottlerBytes, "Bytes passed through 'max_mutations_bandwidth_for_server' throttler.", ValueType::Bytes) \ M(MutationsThrottlerSleepMicroseconds, "Total time a query was sleeping to conform 'max_mutations_bandwidth_for_server' throttling.", ValueType::Microseconds) \ + M(ExportsThrottlerBytes, "Bytes passed through 'max_exports_bandwidth_for_server' throttler.", ValueType::Bytes) \ + M(ExportsThrottlerSleepMicroseconds, "Total time a query was sleeping to conform 'max_exports_bandwidth_for_server' throttling.", ValueType::Microseconds) \ M(QueryRemoteReadThrottlerBytes, "Bytes passed through 'max_remote_read_network_bandwidth' throttler.", ValueType::Bytes) \ M(QueryRemoteReadThrottlerSleepMicroseconds, "Total time a query was sleeping to conform 'max_remote_read_network_bandwidth' throttling.", ValueType::Microseconds) \ M(QueryRemoteWriteThrottlerBytes, "Bytes passed through 'max_remote_write_network_bandwidth' throttler.", ValueType::Bytes) \ diff --git a/src/Common/setThreadName.h b/src/Common/setThreadName.h index 9d94523958c5..daeb07a719cf 100644 --- a/src/Common/setThreadName.h +++ b/src/Common/setThreadName.h @@ -161,6 +161,7 @@ namespace DB M(ZOOKEEPER_SEND, "ZooKeeperSend") \ M(BLOB_KILLER_TASK, "BlobKillerTask") \ M(BLOB_COPIER_TASK, "BlobCopierTask") \ + M(EXPORT_PART, "ExportPart") \ enum class ThreadName : uint8_t diff --git a/src/Core/ServerSettings.cpp b/src/Core/ServerSettings.cpp index 2471bf4ea8f2..12a02b3d4bf8 100644 --- a/src/Core/ServerSettings.cpp +++ b/src/Core/ServerSettings.cpp @@ -148,6 +148,7 @@ namespace DECLARE(UInt64, max_unexpected_parts_loading_thread_pool_size, 8, R"(The number of threads to load inactive set of data parts (Unexpected ones) at startup.)", 0) \ DECLARE(UInt64, max_parts_cleaning_thread_pool_size, 128, R"(The number of threads for concurrent removal of inactive data parts.)", 0) \ DECLARE(UInt64, max_mutations_bandwidth_for_server, 0, R"(The maximum read speed of all mutations on server in bytes per second. Zero means unlimited.)", 0) \ + DECLARE(UInt64, max_exports_bandwidth_for_server, 0, R"(The maximum read speed of all exports on server in bytes per second. Zero means unlimited.)", 0) \ DECLARE(UInt64, max_merges_bandwidth_for_server, 0, R"(The maximum read speed of all merges on server in bytes per second. Zero means unlimited.)", 0) \ DECLARE(UInt64, max_replicated_fetches_network_bandwidth_for_server, 0, R"(The maximum speed of data exchange over the network in bytes per second for replicated fetches. Zero means unlimited.)", 0) \ DECLARE(UInt64, max_replicated_sends_network_bandwidth_for_server, 0, R"(The maximum speed of data exchange over the network in bytes per second for replicated sends. Zero means unlimited.)", 0) \ @@ -1530,6 +1531,7 @@ The policy on how to perform a scheduling of CPU slots specified by `concurrent_ 1 ``` )", 0) \ + DECLARE(Bool, enable_experimental_export_merge_tree_partition_feature, false, "Enable export replicated merge tree partition feature. It is experimental and not yet ready for production use.", 0) \ DECLARE(UInt64, input_format_parquet_metadata_cache_max_size, 500000000, "Maximum size of parquet file metadata cache", 0) /// Settings with a path are server settings with at least one layer of nesting that have a fixed structure (no lists, lists, enumerations, repetitions, ...). @@ -1603,7 +1605,7 @@ The policy on how to perform a scheduling of CPU slots specified by `concurrent_ DECLARE(UInt64, keeper_server_socket_receive_timeout_sec, DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC, R"(Keeper socket receive timeout.)", 0, "keeper_server.socket_receive_timeout_sec") \ DECLARE(UInt64, keeper_server_socket_send_timeout_sec, DBMS_DEFAULT_SEND_TIMEOUT_SEC, R"(Keeper socket send timeout.)", 0, "keeper_server.socket_send_timeout_sec") \ DECLARE(String, hdfs_libhdfs3_conf, "", R"(Points libhdfs3 to the right location for its config.)", 0, "hdfs.libhdfs3_conf") \ - DECLARE(String, config_file, "config.xml", R"(Points to the server config file.)", 0, "config-file") + DECLARE(String, config_file, "config.xml", R"(Points to the server config file.)", 0, "config-file") // clang-format on diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 0cbb8921adc8..43a9e93e4d2c 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -7480,6 +7480,39 @@ Always ignore ON CLUSTER clause for DDL queries with replicated databases. )", 0) \ DECLARE(UInt64, archive_adaptive_buffer_max_size_bytes, 8 * DBMS_DEFAULT_BUFFER_SIZE, R"( Limits the maximum size of the adaptive buffer used when writing to archive files (for example, tar archives)", 0) \ + DECLARE(Bool, export_merge_tree_part_overwrite_file_if_exists, false, R"( +Overwrite file if it already exists when exporting a merge tree part +)", 0) \ + DECLARE(Bool, export_merge_tree_partition_force_export, false, R"( +Ignore existing partition export and overwrite the zookeeper entry +)", 0) \ + DECLARE(UInt64, export_merge_tree_partition_max_retries, 3, R"( +Maximum number of retries for exporting a merge tree part in an export partition task +)", 0) \ + DECLARE(UInt64, export_merge_tree_partition_manifest_ttl, 180, R"( +Determines how long the manifest will live in ZooKeeper. It prevents the same partition from being exported twice to the same destination. +This setting does not affect / delete in progress tasks. It'll only cleanup the completed ones. +)", 0) \ + DECLARE(MergeTreePartExportFileAlreadyExistsPolicy, export_merge_tree_part_file_already_exists_policy, MergeTreePartExportFileAlreadyExistsPolicy::skip, R"( +Possible values: +- skip - Skip the file if it already exists. +- error - Throw an error if the file already exists. +- overwrite - Overwrite the file. +)", 0) \ + DECLARE(UInt64, export_merge_tree_part_max_bytes_per_file, 0, R"( +Maximum number of bytes to write to a single file when exporting a merge tree part. 0 means no limit. +This is not a hard limit, and it highly depends on the output format granularity and input source chunk size. +)", 0) \ + DECLARE(UInt64, export_merge_tree_part_max_rows_per_file, 0, R"( +Maximum number of rows to write to a single file when exporting a merge tree part. 0 means no limit. +This is not a hard limit, and it highly depends on the output format granularity and input source chunk size. +)", 0) \ + DECLARE(Bool, export_merge_tree_part_throw_on_pending_mutations, true, R"( +Throw an error if there are pending mutations when exporting a merge tree part. +)", 0) \ + DECLARE(Bool, export_merge_tree_part_throw_on_pending_patch_parts, true, R"( +Throw an error if there are pending patch parts when exporting a merge tree part. +)", 0) \ \ /* ####################################################### */ \ /* ########### START OF EXPERIMENTAL FEATURES ############ */ \ @@ -7718,6 +7751,9 @@ Rewrite expressions like 'x IN subquery' to JOIN. This might be useful for optim DECLARE_WITH_ALIAS(Bool, allow_experimental_time_series_aggregate_functions, false, R"( Experimental timeSeries* aggregate functions for Prometheus-like timeseries resampling, rate, delta calculation. )", EXPERIMENTAL, allow_experimental_ts_to_grid_aggregate_function) \ + DECLARE(Bool, allow_experimental_export_merge_tree_part, true, R"( +Experimental export merge tree part. +)", EXPERIMENTAL) \ \ DECLARE(String, promql_database, "", R"( Specifies the database name used by the 'promql' dialect. Empty string means the current database. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 77fd9cb452c2..c8211b7e6bba 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -83,6 +83,7 @@ class WriteBuffer; M(CLASS_NAME, LogsLevel) \ M(CLASS_NAME, Map) \ M(CLASS_NAME, MaxThreads) \ + M(CLASS_NAME, MergeTreePartExportFileAlreadyExistsPolicy) \ M(CLASS_NAME, Milliseconds) \ M(CLASS_NAME, MsgPackUUIDRepresentation) \ M(CLASS_NAME, MySQLDataTypesSupport) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 26c60f2d9584..88614a4d49d8 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -330,6 +330,30 @@ const VersionToSettingsChangesMap & getSettingsChangesHistory() // {"export_merge_tree_part_throw_on_pending_patch_parts", true, true, "New setting."}, // {"object_storage_cluster", "", "", "Antalya: New setting"}, // {"object_storage_max_nodes", 0, 0, "Antalya: New setting"}, + // {"allow_experimental_iceberg_read_optimization", true, true, "New setting."}, + // {"object_storage_cluster_join_mode", "allow", "allow", "New setting"}, + // {"lock_object_storage_task_distribution_ms", 500, 500, "Raised the value to 500 to avoid hoping tasks between executors."}, + // {"allow_retries_in_cluster_requests", false, false, "New setting"}, + // {"object_storage_remote_initiator", false, false, "New setting."}, + {"allow_experimental_export_merge_tree_part", false, true, "Turned ON by default for Antalya."}, + {"export_merge_tree_part_overwrite_file_if_exists", false, false, "New setting."}, + {"export_merge_tree_partition_force_export", false, false, "New setting."}, + {"export_merge_tree_partition_max_retries", 3, 3, "New setting."}, + {"export_merge_tree_partition_manifest_ttl", 180, 180, "New setting."}, + {"export_merge_tree_part_file_already_exists_policy", "skip", "skip", "New setting."}, + // {"iceberg_timezone_for_timestamptz", "UTC", "UTC", "New setting."}, + {"hybrid_table_auto_cast_columns", true, true, "New setting to automatically cast Hybrid table columns when segments disagree on types. Default enabled."}, + {"allow_experimental_hybrid_table", false, false, "Added new setting to allow the Hybrid table engine."}, + {"enable_alias_marker", true, true, "New setting."}, + // {"input_format_parquet_use_native_reader_v3", false, true, "Seems stable"}, + // {"input_format_parquet_verify_checksums", true, true, "New setting."}, + // {"output_format_parquet_write_checksums", false, true, "New setting."}, + {"export_merge_tree_part_max_bytes_per_file", 0, 0, "New setting."}, + {"export_merge_tree_part_max_rows_per_file", 0, 0, "New setting."}, + // {"cluster_table_function_split_granularity", "file", "file", "New setting."}, + // {"cluster_table_function_buckets_batch_size", 0, 0, "New setting."}, + {"export_merge_tree_part_throw_on_pending_mutations", true, true, "New setting."}, + {"export_merge_tree_part_throw_on_pending_patch_parts", true, true, "New setting."}, }); addSettingsChanges(settings_changes_history, "25.8", { diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 20322c02086e..da68db3093f8 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -475,4 +475,6 @@ IMPLEMENT_SETTING_ENUM(JemallocProfileFormat, ErrorCodes::BAD_ARGUMENTS, {"symbolized", JemallocProfileFormat::Symbolized}, {"collapsed", JemallocProfileFormat::Collapsed}}) +IMPLEMENT_SETTING_AUTO_ENUM(MergeTreePartExportFileAlreadyExistsPolicy, ErrorCodes::BAD_ARGUMENTS); + } diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index f4c710be2338..7de1d04eeef9 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -553,4 +553,14 @@ enum class JemallocProfileFormat : uint8_t }; DECLARE_SETTING_ENUM(JemallocProfileFormat) + +enum class MergeTreePartExportFileAlreadyExistsPolicy : uint8_t +{ + skip, + error, + overwrite, +}; + +DECLARE_SETTING_ENUM(MergeTreePartExportFileAlreadyExistsPolicy) + } diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 7598a4e414ef..58ede9310c57 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -2528,7 +2528,7 @@ bool DatabaseReplicated::shouldReplicateQuery(const ContextPtr & query_context, if (const auto * alter = query_ptr->as()) { if (alter->isAttachAlter() || alter->isFetchAlter() || alter->isDropPartitionAlter() || alter->isFreezeAlter() - || alter->isUnlockSnapshot()) + || alter->isUnlockSnapshot() || alter->isExportPartOrExportPartitionAlter()) return false; // Allowed ALTER operation on KeeperMap still should be replicated diff --git a/src/Disks/DiskObjectStorage/ObjectStorages/IObjectStorage.h b/src/Disks/DiskObjectStorage/ObjectStorages/IObjectStorage.h index 6610ee6874f9..c56fb3877313 100644 --- a/src/Disks/DiskObjectStorage/ObjectStorages/IObjectStorage.h +++ b/src/Disks/DiskObjectStorage/ObjectStorages/IObjectStorage.h @@ -131,6 +131,7 @@ struct RelativePathWithMetadata std::string getFileName() const { return std::filesystem::path(relative_path).filename(); } std::string getPath() const { return relative_path; } + std::string getFileNameWithoutExtension() const { return std::filesystem::path(relative_path).stem(); } }; struct ObjectKeyWithMetadata diff --git a/src/Functions/generateSnowflakeID.cpp b/src/Functions/generateSnowflakeID.cpp index 966088d4e52b..06832b3781db 100644 --- a/src/Functions/generateSnowflakeID.cpp +++ b/src/Functions/generateSnowflakeID.cpp @@ -154,6 +154,11 @@ uint64_t generateSnowflakeID() return fromSnowflakeId(snowflake_id); } +std::string generateSnowflakeIDString() +{ + return std::to_string(generateSnowflakeID()); +} + class FunctionGenerateSnowflakeID : public IFunction { public: diff --git a/src/Functions/generateSnowflakeID.h b/src/Functions/generateSnowflakeID.h index 38fa684a9b4b..4fc173dcf1be 100644 --- a/src/Functions/generateSnowflakeID.h +++ b/src/Functions/generateSnowflakeID.h @@ -7,4 +7,6 @@ namespace DB uint64_t generateSnowflakeID(); +std::string generateSnowflakeIDString(); + } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 92ae95d7e9c9..e3497fb6ec56 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -44,6 +44,7 @@ #include #include #include +#include #include #include #include @@ -170,6 +171,8 @@ namespace ProfileEvents extern const Event BackupThrottlerSleepMicroseconds; extern const Event MergesThrottlerBytes; extern const Event MergesThrottlerSleepMicroseconds; + extern const Event ExportsThrottlerBytes; + extern const Event ExportsThrottlerSleepMicroseconds; extern const Event MutationsThrottlerBytes; extern const Event MutationsThrottlerSleepMicroseconds; extern const Event QueryLocalReadThrottlerBytes; @@ -369,6 +372,7 @@ namespace ServerSetting extern const ServerSettingsUInt64 max_local_write_bandwidth_for_server; extern const ServerSettingsUInt64 max_merges_bandwidth_for_server; extern const ServerSettingsUInt64 max_mutations_bandwidth_for_server; + extern const ServerSettingsUInt64 max_exports_bandwidth_for_server; extern const ServerSettingsUInt64 max_remote_read_network_bandwidth_for_server; extern const ServerSettingsUInt64 max_remote_write_network_bandwidth_for_server; extern const ServerSettingsUInt64 max_replicated_fetches_network_bandwidth_for_server; @@ -569,6 +573,7 @@ struct ContextSharedPart : boost::noncopyable GlobalOvercommitTracker global_overcommit_tracker; MergeList merge_list; /// The list of executable merge (for (Replicated)?MergeTree) MovesList moves_list; /// The list of executing moves (for (Replicated)?MergeTree) + ExportsList exports_list; /// The list of executing exports (for (Replicated)?MergeTree) ReplicatedFetchList replicated_fetch_list; RefreshSet refresh_set; /// The list of active refreshes (for MaterializedView) ConfigurationPtr users_config TSA_GUARDED_BY(mutex); /// Config with the users, profiles and quotas sections. @@ -615,6 +620,8 @@ struct ContextSharedPart : boost::noncopyable mutable ThrottlerPtr distributed_cache_read_throttler; /// A server-wide throttler for distributed cache read mutable ThrottlerPtr distributed_cache_write_throttler; /// A server-wide throttler for distributed cache write + mutable ThrottlerPtr exports_throttler; /// A server-wide throttler for exports + MultiVersion macros; /// Substitutions extracted from config. std::unique_ptr ddl_worker TSA_GUARDED_BY(mutex); /// Process ddl commands from zk. LoadTaskPtr ddl_worker_startup_task; /// To postpone `ddl_worker->startup()` after all tables startup @@ -1164,6 +1171,9 @@ struct ContextSharedPart : boost::noncopyable if (auto bandwidth = server_settings[ServerSetting::max_merges_bandwidth_for_server]) merges_throttler = std::make_shared(bandwidth, ProfileEvents::MergesThrottlerBytes, ProfileEvents::MergesThrottlerSleepMicroseconds); + + if (auto bandwidth = server_settings[ServerSetting::max_exports_bandwidth_for_server]) + exports_throttler = std::make_shared(bandwidth, ProfileEvents::ExportsThrottlerBytes, ProfileEvents::ExportsThrottlerSleepMicroseconds); } }; @@ -1324,6 +1334,8 @@ MergeList & Context::getMergeList() { return shared->merge_list; } const MergeList & Context::getMergeList() const { return shared->merge_list; } MovesList & Context::getMovesList() { return shared->moves_list; } const MovesList & Context::getMovesList() const { return shared->moves_list; } +ExportsList & Context::getExportsList() { return shared->exports_list; } +const ExportsList & Context::getExportsList() const { return shared->exports_list; } ReplicatedFetchList & Context::getReplicatedFetchList() { return shared->replicated_fetch_list; } const ReplicatedFetchList & Context::getReplicatedFetchList() const { return shared->replicated_fetch_list; } RefreshSet & Context::getRefreshSet() { return shared->refresh_set; } @@ -3434,6 +3446,13 @@ void Context::makeQueryContextForMutate(const MergeTreeSettings & merge_tree_set = merge_tree_settings[MergeTreeSetting::mutation_workload].value.empty() ? getMutationWorkload() : merge_tree_settings[MergeTreeSetting::mutation_workload]; } +void Context::makeQueryContextForExportPart() +{ + makeQueryContext(); + classifier.reset(); // It is assumed that there are no active queries running using this classifier, otherwise this will lead to crashes + // Export part operations don't have a specific workload setting, so we leave the default workload +} + void Context::makeSessionContext() { session_context = shared_from_this(); @@ -4777,6 +4796,11 @@ ThrottlerPtr Context::getDistributedCacheWriteThrottler() const return shared->distributed_cache_write_throttler; } +ThrottlerPtr Context::getExportsThrottler() const +{ + return shared->exports_throttler; +} + void Context::reloadRemoteThrottlerConfig(size_t read_bandwidth, size_t write_bandwidth) const { if (read_bandwidth) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index b10d3a3f2ce4..1fe93b9ff861 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -96,6 +96,7 @@ class AsynchronousMetrics; class BackgroundSchedulePool; class MergeList; class MovesList; +class ExportsList; class ReplicatedFetchList; class RefreshSet; class Cluster; @@ -1249,6 +1250,7 @@ class Context: public ContextData, public std::enable_shared_from_this void makeQueryContext(); void makeQueryContextForMerge(const MergeTreeSettings & merge_tree_settings); void makeQueryContextForMutate(const MergeTreeSettings & merge_tree_settings); + void makeQueryContextForExportPart(); void makeSessionContext(); void makeGlobalContext(); void makeBackgroundContext(const Poco::Util::AbstractConfiguration & config); @@ -1285,6 +1287,9 @@ class Context: public ContextData, public std::enable_shared_from_this MovesList & getMovesList(); const MovesList & getMovesList() const; + ExportsList & getExportsList(); + const ExportsList & getExportsList() const; + ReplicatedFetchList & getReplicatedFetchList(); const ReplicatedFetchList & getReplicatedFetchList() const; @@ -1834,6 +1839,7 @@ class Context: public ContextData, public std::enable_shared_from_this ThrottlerPtr getMutationsThrottler() const; ThrottlerPtr getMergesThrottler() const; + ThrottlerPtr getExportsThrottler() const; ThrottlerPtr getDistributedCacheReadThrottler() const; ThrottlerPtr getDistributedCacheWriteThrottler() const; diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index 73f6fbd80aef..7c844224c5d3 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -811,7 +811,8 @@ bool DDLWorker::taskShouldBeExecutedOnLeader(const ASTPtr & ast_ddl, const Stora alter->isFreezeAlter() || alter->isUnlockSnapshot() || alter->isMovePartitionToDiskOrVolumeAlter() || - alter->isCommentAlter()) + alter->isCommentAlter() || + alter->isExportPartOrExportPartitionAlter()) return false; } diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index eb7223eed89f..f5fc47c6c3bb 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -657,6 +657,20 @@ AccessRightsElements InterpreterAlterQuery::getRequiredAccessForCommand(const AS required_access.emplace_back(AccessType::ALTER_DELETE | AccessType::INSERT, database, table); break; } + case ASTAlterCommand::EXPORT_PART: + { + required_access.emplace_back(AccessType::ALTER_EXPORT_PART, database, table); + /// For table functions, access control is handled by the table function itself + if (!command.to_table_function) + required_access.emplace_back(AccessType::INSERT, command.to_database, command.to_table); + break; + } + case ASTAlterCommand::EXPORT_PARTITION: + { + required_access.emplace_back(AccessType::ALTER_EXPORT_PARTITION, database, table); + required_access.emplace_back(AccessType::INSERT, command.to_database, command.to_table); + break; + } case ASTAlterCommand::FETCH_PARTITION: { required_access.emplace_back(AccessType::ALTER_FETCH_PARTITION, database, table); diff --git a/src/Interpreters/InterpreterKillQueryQuery.cpp b/src/Interpreters/InterpreterKillQueryQuery.cpp index 160e049f80e6..a8d182f73d4e 100644 --- a/src/Interpreters/InterpreterKillQueryQuery.cpp +++ b/src/Interpreters/InterpreterKillQueryQuery.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include @@ -37,11 +38,17 @@ namespace Setting extern const SettingsUInt64 max_parser_depth; } +namespace ServerSetting +{ + extern const ServerSettingsBool enable_experimental_export_merge_tree_partition_feature; +} + namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int ACCESS_DENIED; extern const int NOT_IMPLEMENTED; + extern const int SUPPORT_IS_DISABLED; } @@ -250,6 +257,82 @@ BlockIO InterpreterKillQueryQuery::execute() break; } + case ASTKillQueryQuery::Type::ExportPartition: + { + if (!getContext()->getServerSettings()[ServerSetting::enable_experimental_export_merge_tree_partition_feature]) + { + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "Exporting merge tree partition is experimental. Set the server setting `enable_experimental_export_merge_tree_partition_feature` to enable it"); + } + + Block exports_block = getSelectResult( + "source_database, source_table, transaction_id, destination_database, destination_table, partition_id", + "system.replicated_partition_exports"); + if (exports_block.empty()) + return res_io; + + const ColumnString & src_db_col = typeid_cast(*exports_block.getByName("source_database").column); + const ColumnString & src_table_col = typeid_cast(*exports_block.getByName("source_table").column); + const ColumnString & dst_db_col = typeid_cast(*exports_block.getByName("destination_database").column); + const ColumnString & dst_table_col = typeid_cast(*exports_block.getByName("destination_table").column); + const ColumnString & tx_col = typeid_cast(*exports_block.getByName("transaction_id").column); + + auto header = exports_block.cloneEmpty(); + header.insert(0, {ColumnString::create(), std::make_shared(), "kill_status"}); + + MutableColumns res_columns = header.cloneEmptyColumns(); + AccessRightsElements required_access_rights; + auto access = getContext()->getAccess(); + bool access_denied = false; + + for (size_t i = 0; i < exports_block.rows(); ++i) + { + const auto src_database = src_db_col.getDataAt(i); + const auto src_table = src_table_col.getDataAt(i); + const auto dst_database = dst_db_col.getDataAt(i); + const auto dst_table = dst_table_col.getDataAt(i); + + const auto table_id = StorageID{std::string{src_database}, std::string{src_table}}; + const auto transaction_id = tx_col.getDataAt(i); + + CancellationCode code = CancellationCode::Unknown; + if (!query.test) + { + auto storage = DatabaseCatalog::instance().tryGetTable(table_id, getContext()); + if (!storage) + code = CancellationCode::NotFound; + else + { + ASTAlterCommand alter_command{}; + alter_command.type = ASTAlterCommand::EXPORT_PARTITION; + alter_command.move_destination_type = DataDestinationType::TABLE; + alter_command.from_database = src_database; + alter_command.from_table = src_table; + alter_command.to_database = dst_database; + alter_command.to_table = dst_table; + + required_access_rights = InterpreterAlterQuery::getRequiredAccessForCommand( + alter_command, table_id.database_name, table_id.table_name); + if (!access->isGranted(required_access_rights)) + { + access_denied = true; + continue; + } + code = storage->killExportPartition(std::string{transaction_id}); + } + } + + insertResultRow(i, code, exports_block, header, res_columns); + } + + if (res_columns[0]->empty() && access_denied) + throw Exception(ErrorCodes::ACCESS_DENIED, "Not allowed to kill export partition. " + "To execute this query, it's necessary to have the grant {}", required_access_rights.toString()); + + res_io.pipeline = QueryPipeline(Pipe(std::make_shared(std::make_shared(header.cloneWithColumns(std::move(res_columns)))))); + + break; + } case ASTKillQueryQuery::Type::Mutation: { Block mutations_block = getSelectResult("database, table, mutation_id, command", "system.mutations"); @@ -471,6 +554,9 @@ AccessRightsElements InterpreterKillQueryQuery::getRequiredAccessForDDLOnCluster | AccessType::ALTER_MATERIALIZE_TTL | AccessType::ALTER_REWRITE_PARTS ); + /// todo arthur think about this + else if (query.type == ASTKillQueryQuery::Type::ExportPartition) + required_access.emplace_back(AccessType::ALTER_EXPORT_PARTITION); return required_access; } diff --git a/src/Interpreters/PartLog.cpp b/src/Interpreters/PartLog.cpp index 186d16c79ba7..5d14c0506f80 100644 --- a/src/Interpreters/PartLog.cpp +++ b/src/Interpreters/PartLog.cpp @@ -72,6 +72,7 @@ ColumnsDescription PartLogElement::getColumnsDescription() {"MovePart", static_cast(MOVE_PART)}, {"MergePartsStart", static_cast(MERGE_PARTS_START)}, {"MutatePartStart", static_cast(MUTATE_PART_START)}, + {"ExportPart", static_cast(EXPORT_PART)}, } ); @@ -113,7 +114,8 @@ ColumnsDescription PartLogElement::getColumnsDescription() "RemovePart — Removing or detaching a data part using [DETACH PARTITION](/sql-reference/statements/alter/partition#detach-partitionpart)." "MutatePartStart — Mutating of a data part has started, " "MutatePart — Mutating of a data part has finished, " - "MovePart — Moving the data part from the one disk to another one."}, + "MovePart — Moving the data part from the one disk to another one." + "ExportPart — Exporting the data part from a MergeTree table into a target table that represents external storage (e.g., object storage or a data lake).."}, {"merge_reason", std::move(merge_reason_datatype), "The reason for the event with type MERGE_PARTS. Can have one of the following values: " "NotAMerge — The current event has the type other than MERGE_PARTS, " @@ -137,6 +139,7 @@ ColumnsDescription PartLogElement::getColumnsDescription() {"part_storage_type", std::make_shared(), "The type of DataPartStorage. Possible values: Packed - all files are stored in a single blob, Full - a blob per file."}, {"disk_name", std::make_shared(), "The disk name data part lies on."}, {"path_on_disk", std::make_shared(), "Absolute path to the folder with data part files."}, + {"remote_file_paths", std::make_shared(std::make_shared()), "In case of an export operation to remote storages, the file paths a given export generated"}, {"rows", std::make_shared(), "The number of rows in the data part."}, {"size_in_bytes", std::make_shared(), "Size of the data part on disk in bytes."}, @@ -195,6 +198,12 @@ void PartLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insert(disk_name); columns[i++]->insert(path_on_disk); + Array remote_file_paths_array; + remote_file_paths_array.reserve(remote_file_paths.size()); + for (const auto & remote_file_path : remote_file_paths) + remote_file_paths_array.push_back(remote_file_path); + columns[i++]->insert(remote_file_paths_array); + columns[i++]->insert(rows); columns[i++]->insert(bytes_compressed_on_disk); diff --git a/src/Interpreters/PartLog.h b/src/Interpreters/PartLog.h index 5d01cd838a0e..f861b7e39b32 100644 --- a/src/Interpreters/PartLog.h +++ b/src/Interpreters/PartLog.h @@ -31,6 +31,7 @@ struct PartLogElement MOVE_PART = 6, MERGE_PARTS_START = 7, MUTATE_PART_START = 8, + EXPORT_PART = 9, }; /// Copy of MergeAlgorithm since values are written to disk. @@ -74,6 +75,7 @@ struct PartLogElement String disk_name; String path_on_disk; Strings deduplication_block_ids; + std::vector remote_file_paths; MergeTreeDataPartFormat part_format; diff --git a/src/Interpreters/executeDDLQueryOnCluster.cpp b/src/Interpreters/executeDDLQueryOnCluster.cpp index 514f390e93ff..26f36d403fc1 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.cpp +++ b/src/Interpreters/executeDDLQueryOnCluster.cpp @@ -54,6 +54,8 @@ bool isSupportedAlterTypeForOnClusterDDLQuery(int type) ASTAlterCommand::ATTACH_PARTITION, /// Usually followed by ATTACH PARTITION ASTAlterCommand::FETCH_PARTITION, + /// Data operation that should be executed locally on each replica + ASTAlterCommand::EXPORT_PART, /// Logical error ASTAlterCommand::NO_TYPE, }; diff --git a/src/Parsers/ASTAlterQuery.cpp b/src/Parsers/ASTAlterQuery.cpp index 3326c78bd639..abf11979b510 100644 --- a/src/Parsers/ASTAlterQuery.cpp +++ b/src/Parsers/ASTAlterQuery.cpp @@ -67,6 +67,10 @@ ASTPtr ASTAlterCommand::clone() const res->rename_to = res->children.emplace_back(rename_to->clone()).get(); if (execute_args) res->execute_args = res->children.emplace_back(execute_args->clone()).get(); + if (to_table_function) + res->to_table_function = res->children.emplace_back(to_table_function->clone()).get(); + if (partition_by_expr) + res->partition_by_expr = res->children.emplace_back(partition_by_expr->clone()).get(); return res; } @@ -370,6 +374,49 @@ void ASTAlterCommand::formatImpl(WriteBuffer & ostr, const FormatSettings & sett ostr << quoteString(move_destination_name); } } + else if (type == ASTAlterCommand::EXPORT_PART) + { + ostr << "EXPORT PART "; + partition->format(ostr, settings, state, frame); + ostr << " TO "; + switch (move_destination_type) + { + case DataDestinationType::TABLE: + ostr << "TABLE "; + if (to_table_function) + { + ostr << "FUNCTION "; + to_table_function->format(ostr, settings, state, frame); + if (partition_by_expr) + { + ostr << " PARTITION BY "; + partition_by_expr->format(ostr, settings, state, frame); + } + } + else + { + if (!to_database.empty()) + ostr << backQuoteIfNeed(to_database) << "."; + + ostr << backQuoteIfNeed(to_table); + } + return; + default: + break; + } + + } + else if (type == ASTAlterCommand::EXPORT_PARTITION) + { + ostr << "EXPORT PARTITION "; + partition->format(ostr, settings, state, frame); + ostr << " TO TABLE "; + if (!to_database.empty()) + { + ostr << backQuoteIfNeed(to_database) << "."; + } + ostr << backQuoteIfNeed(to_table); + } else if (type == ASTAlterCommand::REPLACE_PARTITION) { ostr << (replace ? "REPLACE" : "ATTACH") << " PARTITION " @@ -599,6 +646,8 @@ void ASTAlterCommand::forEachPointerToChild(std::functionmove_destination_name = ast_space_name->as().value.safeGet(); } + else if (s_export_part.ignore(pos, expected)) + { + if (!parser_string_and_substituion.parse(pos, command_partition, expected)) + return false; + + command->type = ASTAlterCommand::EXPORT_PART; + command->part = true; + + if (!s_to_table.ignore(pos, expected)) + { + return false; + } + + if (s_function.ignore(pos, expected)) + { + ParserFunction table_function_parser(/*allow_function_parameters=*/true, /*is_table_function=*/true); + + if (!table_function_parser.parse(pos, export_table_function, expected)) + return false; + + if (s_partition_by.ignore(pos, expected)) + if (!parser_exp_elem.parse(pos, export_table_function_partition_by_expr, expected)) + return false; + + command->to_table_function = export_table_function.get(); + command->partition_by_expr = export_table_function_partition_by_expr.get(); + command->move_destination_type = DataDestinationType::TABLE; + } + else + { + if (!parseDatabaseAndTableName(pos, expected, command->to_database, command->to_table)) + return false; + command->move_destination_type = DataDestinationType::TABLE; + } + } + else if (s_export_partition.ignore(pos, expected)) + { + if (!parser_partition.parse(pos, command_partition, expected)) + return false; + + command->type = ASTAlterCommand::EXPORT_PARTITION; + + if (!s_to_table.ignore(pos, expected)) + { + return false; + } + + if (!parseDatabaseAndTableName(pos, expected, command->to_database, command->to_table)) + return false; + command->move_destination_type = DataDestinationType::TABLE; + } else if (s_move_partition.ignore(pos, expected)) { if (!parser_partition.parse(pos, command_partition, expected)) @@ -1114,6 +1171,10 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected command->rename_to = command->children.emplace_back(std::move(command_rename_to)).get(); if (command_snapshot_desc) command->snapshot_desc = command->children.emplace_back(std::move(command_snapshot_desc)).get(); + if (export_table_function) + command->to_table_function = command->children.emplace_back(std::move(export_table_function)).get(); + if (export_table_function_partition_by_expr) + command->partition_by_expr = command->children.emplace_back(std::move(export_table_function_partition_by_expr)).get(); return true; } diff --git a/src/Parsers/ParserKillQueryQuery.cpp b/src/Parsers/ParserKillQueryQuery.cpp index 97e58566af67..99f2d6fd2d64 100644 --- a/src/Parsers/ParserKillQueryQuery.cpp +++ b/src/Parsers/ParserKillQueryQuery.cpp @@ -17,6 +17,7 @@ bool ParserKillQueryQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expect ParserKeyword p_kill{Keyword::KILL}; ParserKeyword p_query{Keyword::QUERY}; ParserKeyword p_mutation{Keyword::MUTATION}; + ParserKeyword p_export_partition{Keyword::EXPORT_PARTITION}; ParserKeyword p_part_move_to_shard{Keyword::PART_MOVE_TO_SHARD}; ParserKeyword p_transaction{Keyword::TRANSACTION}; ParserKeyword p_on{Keyword::ON}; @@ -33,6 +34,8 @@ bool ParserKillQueryQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expect query->type = ASTKillQueryQuery::Type::Query; else if (p_mutation.ignore(pos, expected)) query->type = ASTKillQueryQuery::Type::Mutation; + else if (p_export_partition.ignore(pos, expected)) + query->type = ASTKillQueryQuery::Type::ExportPartition; else if (p_part_move_to_shard.ignore(pos, expected)) query->type = ASTKillQueryQuery::Type::PartMoveToShard; else if (p_transaction.ignore(pos, expected)) diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index b1c2674f1306..45c32efa8d59 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -496,6 +496,15 @@ NamesAndTypesList ColumnsDescription::getInsertable() const return ret; } +NamesAndTypesList ColumnsDescription::getReadable() const +{ + NamesAndTypesList ret; + for (const auto & col : columns) + if (col.default_desc.kind != ColumnDefaultKind::Ephemeral) + ret.emplace_back(col.name, col.type); + return ret; +} + NamesAndTypesList ColumnsDescription::getMaterialized() const { NamesAndTypesList ret; @@ -869,7 +878,6 @@ std::optional ColumnsDescription::getDefault(const String & colum return {}; } - bool ColumnsDescription::hasCompressionCodec(const String & column_name) const { const auto it = columns.get<1>().find(column_name); diff --git a/src/Storages/ColumnsDescription.h b/src/Storages/ColumnsDescription.h index c60215bad489..b625532ff179 100644 --- a/src/Storages/ColumnsDescription.h +++ b/src/Storages/ColumnsDescription.h @@ -152,6 +152,7 @@ class ColumnsDescription : public IHints<> NamesAndTypesList getOrdinary() const; NamesAndTypesList getMaterialized() const; NamesAndTypesList getInsertable() const; /// ordinary + ephemeral + NamesAndTypesList getReadable() const; /// ordinary + materialized + aliases (no ephemeral) NamesAndTypesList getAliases() const; NamesAndTypesList getEphemeral() const; NamesAndTypesList getAllPhysical() const; /// ordinary + materialized. diff --git a/src/Storages/ExportReplicatedMergeTreePartitionManifest.h b/src/Storages/ExportReplicatedMergeTreePartitionManifest.h new file mode 100644 index 000000000000..7c18b8a881c2 --- /dev/null +++ b/src/Storages/ExportReplicatedMergeTreePartitionManifest.h @@ -0,0 +1,191 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +struct ExportReplicatedMergeTreePartitionProcessingPartEntry +{ + + enum class Status + { + PENDING, + COMPLETED, + FAILED + }; + + String part_name; + Status status; + size_t retry_count; + String finished_by; + + std::string toJsonString() const + { + Poco::JSON::Object json; + + json.set("part_name", part_name); + json.set("status", String(magic_enum::enum_name(status))); + json.set("retry_count", retry_count); + json.set("finished_by", finished_by); + std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + oss.exceptions(std::ios::failbit); + Poco::JSON::Stringifier::stringify(json, oss); + + return oss.str(); + } + + static ExportReplicatedMergeTreePartitionProcessingPartEntry fromJsonString(const std::string & json_string) + { + Poco::JSON::Parser parser; + auto json = parser.parse(json_string).extract(); + chassert(json); + + ExportReplicatedMergeTreePartitionProcessingPartEntry entry; + + entry.part_name = json->getValue("part_name"); + entry.status = magic_enum::enum_cast(json->getValue("status")).value(); + entry.retry_count = json->getValue("retry_count"); + if (json->has("finished_by")) + { + entry.finished_by = json->getValue("finished_by"); + } + return entry; + } +}; + +struct ExportReplicatedMergeTreePartitionProcessedPartEntry +{ + String part_name; + std::vector paths_in_destination; + String finished_by; + + std::string toJsonString() const + { + Poco::JSON::Object json; + json.set("part_name", part_name); + json.set("paths_in_destination", paths_in_destination); + json.set("finished_by", finished_by); + std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + oss.exceptions(std::ios::failbit); + Poco::JSON::Stringifier::stringify(json, oss); + return oss.str(); + } + + static ExportReplicatedMergeTreePartitionProcessedPartEntry fromJsonString(const std::string & json_string) + { + Poco::JSON::Parser parser; + auto json = parser.parse(json_string).extract(); + chassert(json); + + ExportReplicatedMergeTreePartitionProcessedPartEntry entry; + + entry.part_name = json->getValue("part_name"); + + const auto paths_in_destination_array = json->getArray("paths_in_destination"); + for (size_t i = 0; i < paths_in_destination_array->size(); ++i) + entry.paths_in_destination.emplace_back(paths_in_destination_array->getElement(static_cast(i))); + + entry.finished_by = json->getValue("finished_by"); + + return entry; + } +}; + +struct ExportReplicatedMergeTreePartitionManifest +{ + String transaction_id; + String query_id; + String partition_id; + String destination_database; + String destination_table; + String source_replica; + size_t number_of_parts; + std::vector parts; + time_t create_time; + size_t max_retries; + size_t ttl_seconds; + size_t max_threads; + bool parallel_formatting; + bool parquet_parallel_encoding; + size_t max_bytes_per_file; + size_t max_rows_per_file; + MergeTreePartExportManifest::FileAlreadyExistsPolicy file_already_exists_policy; + + std::string toJsonString() const + { + Poco::JSON::Object json; + json.set("transaction_id", transaction_id); + json.set("query_id", query_id); + json.set("partition_id", partition_id); + json.set("destination_database", destination_database); + json.set("destination_table", destination_table); + json.set("source_replica", source_replica); + json.set("number_of_parts", number_of_parts); + + Poco::JSON::Array::Ptr parts_array = new Poco::JSON::Array(); + for (const auto & part : parts) + parts_array->add(part); + json.set("parts", parts_array); + json.set("parallel_formatting", parallel_formatting); + json.set("max_threads", max_threads); + json.set("parquet_parallel_encoding", parquet_parallel_encoding); + json.set("max_bytes_per_file", max_bytes_per_file); + json.set("max_rows_per_file", max_rows_per_file); + json.set("file_already_exists_policy", String(magic_enum::enum_name(file_already_exists_policy))); + json.set("create_time", create_time); + json.set("max_retries", max_retries); + json.set("ttl_seconds", ttl_seconds); + std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + oss.exceptions(std::ios::failbit); + Poco::JSON::Stringifier::stringify(json, oss); + return oss.str(); + } + + static ExportReplicatedMergeTreePartitionManifest fromJsonString(const std::string & json_string) + { + Poco::JSON::Parser parser; + auto json = parser.parse(json_string).extract(); + chassert(json); + + ExportReplicatedMergeTreePartitionManifest manifest; + manifest.transaction_id = json->getValue("transaction_id"); + manifest.query_id = json->getValue("query_id"); + manifest.partition_id = json->getValue("partition_id"); + manifest.destination_database = json->getValue("destination_database"); + manifest.destination_table = json->getValue("destination_table"); + manifest.source_replica = json->getValue("source_replica"); + manifest.number_of_parts = json->getValue("number_of_parts"); + manifest.max_retries = json->getValue("max_retries"); + auto parts_array = json->getArray("parts"); + for (size_t i = 0; i < parts_array->size(); ++i) + manifest.parts.push_back(parts_array->getElement(static_cast(i))); + + manifest.create_time = json->getValue("create_time"); + manifest.ttl_seconds = json->getValue("ttl_seconds"); + manifest.max_threads = json->getValue("max_threads"); + manifest.parallel_formatting = json->getValue("parallel_formatting"); + manifest.parquet_parallel_encoding = json->getValue("parquet_parallel_encoding"); + manifest.max_bytes_per_file = json->getValue("max_bytes_per_file"); + manifest.max_rows_per_file = json->getValue("max_rows_per_file"); + if (json->has("file_already_exists_policy")) + { + const auto file_already_exists_policy = magic_enum::enum_cast(json->getValue("file_already_exists_policy")); + if (file_already_exists_policy) + { + manifest.file_already_exists_policy = file_already_exists_policy.value(); + } + + /// what to do if it's not a valid value? + } + + return manifest; + } +}; + +} diff --git a/src/Storages/ExportReplicatedMergeTreePartitionTaskEntry.h b/src/Storages/ExportReplicatedMergeTreePartitionTaskEntry.h new file mode 100644 index 000000000000..76674bfc4a92 --- /dev/null +++ b/src/Storages/ExportReplicatedMergeTreePartitionTaskEntry.h @@ -0,0 +1,78 @@ +#pragma once + +#include +#include +#include "Core/QualifiedTableName.h" +#include +#include +#include +#include + +namespace DB +{ +struct ExportReplicatedMergeTreePartitionTaskEntry +{ + using DataPartPtr = std::shared_ptr; + ExportReplicatedMergeTreePartitionManifest manifest; + + enum class Status + { + PENDING, + COMPLETED, + FAILED, + KILLED + }; + + /// Allows us to skip completed / failed entries during scheduling + mutable Status status; + + /// References to the parts that should be exported + /// This is used to prevent the parts from being deleted before finishing the export operation + /// It does not mean this replica will export all the parts + /// There is also a chance this replica does not contain a given part and it is totally ok. + std::vector part_references; + + std::string getCompositeKey() const + { + const auto qualified_table_name = QualifiedTableName {manifest.destination_database, manifest.destination_table}; + return manifest.partition_id + "_" + qualified_table_name.getFullName(); + } + + std::string getTransactionId() const + { + return manifest.transaction_id; + } + + /// Get create_time for sorted iteration + time_t getCreateTime() const + { + return manifest.create_time; + } +}; + +struct ExportPartitionTaskEntryTagByCompositeKey {}; +struct ExportPartitionTaskEntryTagByCreateTime {}; +struct ExportPartitionTaskEntryTagByTransactionId {}; + +// Multi-index container for export partition task entries +// - Index 0 (TagByCompositeKey): hashed_unique on composite key for O(1) lookup +// - Index 1 (TagByCreateTime): ordered_non_unique on create_time for sorted iteration +using ExportPartitionTaskEntriesContainer = boost::multi_index_container< + ExportReplicatedMergeTreePartitionTaskEntry, + boost::multi_index::indexed_by< + boost::multi_index::hashed_unique< + boost::multi_index::tag, + boost::multi_index::const_mem_fun + >, + boost::multi_index::ordered_non_unique< + boost::multi_index::tag, + boost::multi_index::const_mem_fun + >, + boost::multi_index::hashed_unique< + boost::multi_index::tag, + boost::multi_index::const_mem_fun + > + > +>; + +} diff --git a/src/Storages/IPartitionStrategy.cpp b/src/Storages/IPartitionStrategy.cpp index 9e3c12d2b6ae..9f6b40f79bed 100644 --- a/src/Storages/IPartitionStrategy.cpp +++ b/src/Storages/IPartitionStrategy.cpp @@ -281,17 +281,13 @@ ColumnPtr WildcardPartitionStrategy::computePartitionKey(const Chunk & chunk) return block_with_partition_by_expr.getByName(actions_with_column.column_name).column; } -std::string WildcardPartitionStrategy::getPathForRead( - const std::string & prefix) +ColumnPtr WildcardPartitionStrategy::computePartitionKey(Block & block) { - return prefix; -} - -std::string WildcardPartitionStrategy::getPathForWrite( - const std::string & prefix, - const std::string & partition_key) -{ - return PartitionedSink::replaceWildcards(prefix, partition_key); + ASTs arguments(1, partition_key_description.definition_ast); + ASTPtr partition_by_string = makeASTFunction("toString", std::move(arguments)); + auto actions_with_column = getPartitionExpressionActions(partition_by_string); + actions_with_column.actions->execute(block); + return block.getByName(actions_with_column.column_name).column; } HiveStylePartitionStrategy::HiveStylePartitionStrategy( @@ -313,41 +309,6 @@ HiveStylePartitionStrategy::HiveStylePartitionStrategy( block_without_partition_columns = buildBlockWithoutPartitionColumns(sample_block, partition_columns_name_set); } -std::string HiveStylePartitionStrategy::getPathForRead(const std::string & prefix) -{ - return prefix + "**." + Poco::toLower(file_format); -} - -std::string HiveStylePartitionStrategy::getPathForWrite( - const std::string & prefix, - const std::string & partition_key) -{ - std::string path; - - if (!prefix.empty()) - { - path += prefix; - if (path.back() != '/') - { - path += '/'; - } - } - - /// Not adding '/' because buildExpressionHive() always adds a trailing '/' - path += partition_key; - - /* - * File extension is toLower(format) - * This isn't ideal, but I guess multiple formats can be specified and introduced. - * So I think it is simpler to keep it this way. - * - * Or perhaps implement something like `IInputFormat::getFileExtension()` - */ - path += std::to_string(generateSnowflakeID()) + "." + Poco::toLower(file_format); - - return path; -} - ColumnPtr HiveStylePartitionStrategy::computePartitionKey(const Chunk & chunk) { auto hive_ast = buildHivePartitionAST(partition_key_description.definition_ast, getPartitionColumns()); @@ -360,6 +321,14 @@ ColumnPtr HiveStylePartitionStrategy::computePartitionKey(const Chunk & chunk) return block_with_partition_by_expr.getByName(actions_with_column.column_name).column; } +ColumnPtr HiveStylePartitionStrategy::computePartitionKey(Block & block) +{ + auto hive_ast = buildHivePartitionAST(partition_key_description.definition_ast, getPartitionColumns()); + auto actions_with_column = getPartitionExpressionActions(hive_ast); + actions_with_column.actions->execute(block); + return block.getByName(actions_with_column.column_name).column; +} + ColumnRawPtrs HiveStylePartitionStrategy::getFormatChunkColumns(const Chunk & chunk) { ColumnRawPtrs result; diff --git a/src/Storages/IPartitionStrategy.h b/src/Storages/IPartitionStrategy.h index a0f5f4cdf9d3..91397de2362d 100644 --- a/src/Storages/IPartitionStrategy.h +++ b/src/Storages/IPartitionStrategy.h @@ -29,8 +29,7 @@ struct IPartitionStrategy virtual ColumnPtr computePartitionKey(const Chunk & chunk) = 0; - virtual std::string getPathForRead(const std::string & prefix) = 0; - virtual std::string getPathForWrite(const std::string & prefix, const std::string & partition_key) = 0; + virtual ColumnPtr computePartitionKey(Block & block) = 0; virtual ColumnRawPtrs getFormatChunkColumns(const Chunk & chunk) { @@ -93,8 +92,8 @@ struct WildcardPartitionStrategy : IPartitionStrategy WildcardPartitionStrategy(KeyDescription partition_key_description_, const Block & sample_block_, ContextPtr context_); ColumnPtr computePartitionKey(const Chunk & chunk) override; - std::string getPathForRead(const std::string & prefix) override; - std::string getPathForWrite(const std::string & prefix, const std::string & partition_key) override; + + ColumnPtr computePartitionKey(Block & block) override; }; /* @@ -112,8 +111,8 @@ struct HiveStylePartitionStrategy : IPartitionStrategy bool partition_columns_in_data_file_); ColumnPtr computePartitionKey(const Chunk & chunk) override; - std::string getPathForRead(const std::string & prefix) override; - std::string getPathForWrite(const std::string & prefix, const std::string & partition_key) override; + + ColumnPtr computePartitionKey(Block & block) override; ColumnRawPtrs getFormatChunkColumns(const Chunk & chunk) override; Block getFormatHeader() override; diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index b7adabd74f2e..f974ee3db0ce 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -327,6 +327,11 @@ CancellationCode IStorage::killPartMoveToShard(const UUID & /*task_uuid*/) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Part moves between shards are not supported by storage {}", getName()); } +CancellationCode IStorage::killExportPartition(const String & /*transaction_id*/) +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Export partition is not supported by storage {}", getName()); +} + StorageID IStorage::getStorageID() const { std::lock_guard lock(id_mutex); diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index c166225ea058..fa284595e994 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -475,6 +475,39 @@ class IStorage : public std::enable_shared_from_this, public TypePromo ContextPtr /*context*/, bool /*async_insert*/); + virtual bool supportsImport() const + { + return false; + } + + /* +It is currently only implemented in StorageObjectStorage. + It is meant to be used to import merge tree data parts into object storage. It is similar to the write API, + but it won't re-partition the data and should allow the filename to be set by the caller. + */ + virtual SinkToStoragePtr import( + const std::string & /* file_name */, + Block & /* block_with_partition_values */, + const std::function & /* new_file_path_callback */, + bool /* overwrite_if_exists */, + std::size_t /* max_bytes_per_file */, + std::size_t /* max_rows_per_file */, + const std::optional & /* format_settings */, + ContextPtr /* context */) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Import is not implemented for storage {}", getName()); + } + + virtual void commitExportPartitionTransaction( + const String & /* transaction_id */, + const String & /* partition_id */, + const Strings & /* exported_paths */, + ContextPtr /* local_context */) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "commitExportPartitionTransaction is not implemented for storage type {}", getName()); + } + + /** Writes the data to a table in distributed manner. * It is supposed that implementation looks into SELECT part of the query and executes distributed * INSERT SELECT if it is possible with current storage as a receiver and query SELECT part as a producer. @@ -583,6 +616,9 @@ class IStorage : public std::enable_shared_from_this, public TypePromo virtual void setMutationCSN(const String & /*mutation_id*/, UInt64 /*csn*/); + /// Cancel a replicated partition export by transaction id. + virtual CancellationCode killExportPartition(const String & /*transaction_id*/); + /// Cancel a part move to shard. virtual CancellationCode killPartMoveToShard(const UUID & /*task_uuid*/); diff --git a/src/Storages/MergeTree/ExportList.cpp b/src/Storages/MergeTree/ExportList.cpp new file mode 100644 index 000000000000..018c1f091ef9 --- /dev/null +++ b/src/Storages/MergeTree/ExportList.cpp @@ -0,0 +1,74 @@ +#include + +namespace DB +{ + +ExportsListElement::ExportsListElement( + const StorageID & source_table_id_, + const StorageID & destination_table_id_, + UInt64 part_size_, + const String & part_name_, + const std::vector & destination_file_paths_, + UInt64 total_rows_to_read_, + UInt64 total_size_bytes_compressed_, + UInt64 total_size_bytes_uncompressed_, + time_t create_time_, + const String & query_id_, + const ContextPtr & context) +: source_table_id(source_table_id_) +, destination_table_id(destination_table_id_) +, part_size(part_size_) +, part_name(part_name_) +, destination_file_paths(destination_file_paths_) +, total_rows_to_read(total_rows_to_read_) +, total_size_bytes_compressed(total_size_bytes_compressed_) +, total_size_bytes_uncompressed(total_size_bytes_uncompressed_) +, create_time(create_time_) +, query_id(query_id_) +{ + thread_group = ThreadGroup::createForMergeMutate(context); +} + +ExportsListElement::~ExportsListElement() +{ + background_memory_tracker.adjustOnBackgroundTaskEnd(&thread_group->memory_tracker); +} + +ExportInfo ExportsListElement::getInfo() const +{ + ExportInfo res; + res.source_database = source_table_id.database_name; + res.source_table = source_table_id.table_name; + res.destination_database = destination_table_id.database_name; + res.destination_table = destination_table_id.table_name; + res.part_name = part_name; + + { + std::shared_lock lock(destination_file_paths_mutex); + res.destination_file_paths = destination_file_paths; + } + + res.rows_read = rows_read.load(std::memory_order_relaxed); + res.total_rows_to_read = total_rows_to_read; + res.total_size_bytes_compressed = total_size_bytes_compressed; + res.total_size_bytes_uncompressed = total_size_bytes_uncompressed; + res.bytes_read_uncompressed = bytes_read_uncompressed.load(std::memory_order_relaxed); + res.memory_usage = getMemoryUsage(); + res.peak_memory_usage = getPeakMemoryUsage(); + res.create_time = create_time; + res.elapsed = watch.elapsedSeconds(); + res.query_id = query_id; + return res; +} + +UInt64 ExportsListElement::getMemoryUsage() const +{ + return thread_group->memory_tracker.get(); +} + +UInt64 ExportsListElement::getPeakMemoryUsage() const +{ + return thread_group->memory_tracker.getPeak(); +} + +} diff --git a/src/Storages/MergeTree/ExportList.h b/src/Storages/MergeTree/ExportList.h new file mode 100644 index 000000000000..4a02826dfe44 --- /dev/null +++ b/src/Storages/MergeTree/ExportList.h @@ -0,0 +1,96 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include + +namespace CurrentMetrics +{ + extern const Metric Export; +} + +namespace DB +{ + +struct ExportInfo +{ + String source_database; + String source_table; + String destination_database; + String destination_table; + String part_name; + std::vector destination_file_paths; + UInt64 rows_read; + UInt64 total_rows_to_read; + UInt64 total_size_bytes_compressed; + UInt64 total_size_bytes_uncompressed; + UInt64 bytes_read_uncompressed; + UInt64 memory_usage; + UInt64 peak_memory_usage; + time_t create_time = 0; + Float64 elapsed; + String query_id; +}; + +struct ExportsListElement : private boost::noncopyable +{ + const StorageID source_table_id; + const StorageID destination_table_id; + const UInt64 part_size; + const String part_name; + + /// see destination_file_paths_mutex + std::vector destination_file_paths; + std::atomic rows_read {0}; + UInt64 total_rows_to_read {0}; + UInt64 total_size_bytes_compressed {0}; + UInt64 total_size_bytes_uncompressed {0}; + std::atomic bytes_read_uncompressed {0}; + time_t create_time {0}; + String query_id; + + Stopwatch watch; + ThreadGroupPtr thread_group; + mutable std::shared_mutex destination_file_paths_mutex; + + ExportsListElement( + const StorageID & source_table_id_, + const StorageID & destination_table_id_, + UInt64 part_size_, + const String & part_name_, + const std::vector & destination_file_paths_, + UInt64 total_rows_to_read_, + UInt64 total_size_bytes_compressed_, + UInt64 total_size_bytes_uncompressed_, + time_t create_time_, + const String & query_id_, + const ContextPtr & context); + + ~ExportsListElement(); + + ExportInfo getInfo() const; + + UInt64 getMemoryUsage() const; + UInt64 getPeakMemoryUsage() const; +}; + + +class ExportsList final : public BackgroundProcessList +{ +private: + using Parent = BackgroundProcessList; + +public: + ExportsList() + : Parent(CurrentMetrics::Export) + {} +}; + +using ExportsListEntry = BackgroundProcessListEntry; + +} diff --git a/src/Storages/MergeTree/ExportPartTask.cpp b/src/Storages/MergeTree/ExportPartTask.cpp new file mode 100644 index 000000000000..f4b504a27cb3 --- /dev/null +++ b/src/Storages/MergeTree/ExportPartTask.cpp @@ -0,0 +1,351 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include "Common/setThreadName.h" +#include +#include +#include +#include +#include +#include + +namespace ProfileEvents +{ + extern const Event PartsExportDuplicated; + extern const Event PartsExportFailures; + extern const Event PartsExports; + extern const Event PartsExportTotalMilliseconds; +} + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNKNOWN_TABLE; + extern const int FILE_ALREADY_EXISTS; + extern const int LOGICAL_ERROR; + extern const int QUERY_WAS_CANCELLED; +} + +namespace Setting +{ + extern const SettingsUInt64 min_bytes_to_use_direct_io; + extern const SettingsUInt64 export_merge_tree_part_max_bytes_per_file; + extern const SettingsUInt64 export_merge_tree_part_max_rows_per_file; + extern const SettingsBool allow_experimental_analyzer; +} + +namespace +{ + void materializeSpecialColumns( + const SharedHeader & header, + const StorageMetadataPtr & storage_metadata, + const ContextPtr & local_context, + QueryPlan & plan_for_part + ) + { + const auto readable_columns = storage_metadata->getColumns().getReadable(); + + // Enable all experimental settings for default expressions + // (same pattern as in IMergeTreeReader::evaluateMissingDefaults) + auto context_for_defaults = Context::createCopy(local_context); + enableAllExperimentalSettings(context_for_defaults); + + /// Copy the behavior of `IMergeTreeReader`, see https://github.com/ClickHouse/ClickHouse/blob/c45224e3f0a6dd9a9217e5d75723f378ffe0a86a/src/Storages/MergeTree/IMergeTreeReader.cpp#L215 + context_for_defaults->setSetting("enable_analyzer", local_context->getSettingsRef()[Setting::allow_experimental_analyzer].value); + + auto defaults_dag = evaluateMissingDefaults( + *header, + readable_columns, + storage_metadata->getColumns(), + context_for_defaults); + + if (defaults_dag) + { + ActionsDAG base_dag(header->getColumnsWithTypeAndName()); + + /// `evaluateMissingDefaults` has a new analyzer path since https://github.com/ClickHouse/ClickHouse/pull/87585 + /// which returns a DAG that does not contain all columns. We need to merge it with the base DAG to get all columns. + auto merged = ActionsDAG::merge(std::move(base_dag), std::move(*defaults_dag)); + + /// Ensure columns are in the correct order matching readable_columns + merged.removeUnusedActions(readable_columns.getNames(), false); + merged.addMaterializingOutputActions(/*materialize_sparse=*/ false); + + auto expression_step = std::make_unique( + header, + std::move(merged)); + expression_step->setStepDescription("Compute alias and default expressions for export"); + plan_for_part.addStep(std::move(expression_step)); + } + } +} + +ExportPartTask::ExportPartTask(MergeTreeData & storage_, const MergeTreePartExportManifest & manifest_) + : storage(storage_), + manifest(manifest_) +{ +} + +bool ExportPartTask::executeStep() +{ + auto local_context = Context::createCopy(storage.getContext()->getBackgroundContext()); + local_context->makeQueryContextForExportPart(); + local_context->setCurrentQueryId(manifest.query_id); + local_context->setSettings(manifest.settings); + + const auto & metadata_snapshot = manifest.metadata_snapshot; + + /// Read only physical columns from the part + const auto columns_to_read = metadata_snapshot->getColumns().getNamesOfPhysical(); + + MergeTreeSequentialSourceType read_type = MergeTreeSequentialSourceType::Export; + + Block block_with_partition_values; + if (metadata_snapshot->hasPartitionKey()) + { + /// todo arthur do I need to init minmax_idx? + block_with_partition_values = manifest.data_part->minmax_idx->getBlock(storage); + } + + const auto & destination_storage = manifest.destination_storage_ptr; + const auto destination_storage_id = destination_storage->getStorageID(); + + auto exports_list_entry = storage.getContext()->getExportsList().insert( + getStorageID(), + destination_storage_id, + manifest.data_part->getBytesOnDisk(), + manifest.data_part->name, + std::vector{}, + manifest.data_part->rows_count, + manifest.data_part->getBytesOnDisk(), + manifest.data_part->getBytesUncompressedOnDisk(), + manifest.create_time, + manifest.query_id, + local_context); + + SinkToStoragePtr sink; + + const auto new_file_path_callback = [&exports_list_entry](const std::string & file_path) + { + std::unique_lock lock((*exports_list_entry)->destination_file_paths_mutex); + (*exports_list_entry)->destination_file_paths.push_back(file_path); + }; + + try + { + sink = destination_storage->import( + manifest.data_part->name + "_" + manifest.data_part->checksums.getTotalChecksumHex(), + block_with_partition_values, + new_file_path_callback, + manifest.file_already_exists_policy == MergeTreePartExportManifest::FileAlreadyExistsPolicy::overwrite, + manifest.settings[Setting::export_merge_tree_part_max_bytes_per_file], + manifest.settings[Setting::export_merge_tree_part_max_rows_per_file], + getFormatSettings(local_context), + local_context); + + bool apply_deleted_mask = true; + bool read_with_direct_io = local_context->getSettingsRef()[Setting::min_bytes_to_use_direct_io] > manifest.data_part->getBytesOnDisk(); + bool prefetch = false; + + MergeTreeData::IMutationsSnapshot::Params mutations_snapshot_params + { + .metadata_version = metadata_snapshot->getMetadataVersion(), + .min_part_metadata_version = manifest.data_part->getMetadataVersion() + }; + + auto mutations_snapshot = storage.getMutationsSnapshot(mutations_snapshot_params); + auto alter_conversions = MergeTreeData::getAlterConversionsForPart( + manifest.data_part, + mutations_snapshot, + local_context); + + QueryPlan plan_for_part; + + createReadFromPartStep( + read_type, + plan_for_part, + storage, + storage.getStorageSnapshot(metadata_snapshot, local_context), + RangesInDataPart(manifest.data_part), + alter_conversions, + nullptr, + columns_to_read, + nullptr, + apply_deleted_mask, + std::nullopt, + read_with_direct_io, + prefetch, + local_context, + getLogger("ExportPartition")); + + ThreadGroupSwitcher switcher((*exports_list_entry)->thread_group, ThreadName::EXPORT_PART); + + /// We need to support exporting materialized and alias columns to object storage. For some reason, object storage engines don't support them. + /// This is a hack that materializes the columns before the export so they can be exported to tables that have matching columns + materializeSpecialColumns(plan_for_part.getCurrentHeader(), metadata_snapshot, local_context, plan_for_part); + + QueryPlanOptimizationSettings optimization_settings(local_context); + auto pipeline_settings = BuildQueryPipelineSettings(local_context); + auto builder = plan_for_part.buildQueryPipeline(optimization_settings, pipeline_settings); + + builder->setProgressCallback([&exports_list_entry](const Progress & progress) + { + (*exports_list_entry)->bytes_read_uncompressed += progress.read_bytes; + (*exports_list_entry)->rows_read += progress.read_rows; + }); + + pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); + + pipeline.complete(sink); + + CompletedPipelineExecutor exec(pipeline); + + auto is_cancelled_callback = [this]() + { + return isCancelled(); + }; + + exec.setCancelCallback(is_cancelled_callback, 100); + + exec.execute(); + + if (isCancelled()) + { + throw Exception(ErrorCodes::QUERY_WAS_CANCELLED, "Export part was cancelled"); + } + + std::lock_guard inner_lock(storage.export_manifests_mutex); + storage.writePartLog( + PartLogElement::Type::EXPORT_PART, + {}, + (*exports_list_entry)->watch.elapsed(), + manifest.data_part->name, + manifest.data_part, + {manifest.data_part}, + nullptr, + nullptr, + {}, + exports_list_entry.get()); + + storage.export_manifests.erase(manifest); + + ProfileEvents::increment(ProfileEvents::PartsExports); + ProfileEvents::increment(ProfileEvents::PartsExportTotalMilliseconds, (*exports_list_entry)->watch.elapsedMilliseconds()); + + if (manifest.completion_callback) + manifest.completion_callback(MergeTreePartExportManifest::CompletionCallbackResult::createSuccess((*exports_list_entry)->destination_file_paths)); + } + catch (const Exception & e) + { + /// If an exception is thrown before the pipeline is started, the sink will not be canceled and might leave buffers open. + /// Cancel it manually to ensure the buffers are closed. + if (sink) + { + sink->cancel(); + } + + if (e.code() == ErrorCodes::FILE_ALREADY_EXISTS) + { + ProfileEvents::increment(ProfileEvents::PartsExportDuplicated); + + /// File already exists and the policy is NO_OP, treat it as success. + if (manifest.file_already_exists_policy == MergeTreePartExportManifest::FileAlreadyExistsPolicy::skip) + { + storage.writePartLog( + PartLogElement::Type::EXPORT_PART, + {}, + (*exports_list_entry)->watch.elapsed(), + manifest.data_part->name, + manifest.data_part, + {manifest.data_part}, + nullptr, + nullptr, + {}, + exports_list_entry.get()); + + std::lock_guard inner_lock(storage.export_manifests_mutex); + storage.export_manifests.erase(manifest); + + ProfileEvents::increment(ProfileEvents::PartsExports); + ProfileEvents::increment(ProfileEvents::PartsExportTotalMilliseconds, (*exports_list_entry)->watch.elapsedMilliseconds()); + + if (manifest.completion_callback) + { + manifest.completion_callback(MergeTreePartExportManifest::CompletionCallbackResult::createSuccess((*exports_list_entry)->destination_file_paths)); + } + + return false; + } + } + + ProfileEvents::increment(ProfileEvents::PartsExportFailures); + + storage.writePartLog( + PartLogElement::Type::EXPORT_PART, + ExecutionStatus::fromCurrentException("", true), + (*exports_list_entry)->watch.elapsed(), + manifest.data_part->name, + manifest.data_part, + {manifest.data_part}, + nullptr, + nullptr, + {}, + exports_list_entry.get()); + + std::lock_guard inner_lock(storage.export_manifests_mutex); + storage.export_manifests.erase(manifest); + + if (manifest.completion_callback) + manifest.completion_callback(MergeTreePartExportManifest::CompletionCallbackResult::createFailure(e)); + return false; + } + + return false; +} + +void ExportPartTask::cancel() noexcept +{ + cancel_requested.store(true); + pipeline.cancel(); +} + +bool ExportPartTask::isCancelled() const +{ + return cancel_requested.load() || storage.parts_mover.moves_blocker.isCancelled(); +} + +void ExportPartTask::onCompleted() +{ +} + +StorageID ExportPartTask::getStorageID() const +{ + return storage.getStorageID(); +} + +Priority ExportPartTask::getPriority() const +{ + return Priority{}; +} + +String ExportPartTask::getQueryId() const +{ + return manifest.query_id; +} + +} diff --git a/src/Storages/MergeTree/ExportPartTask.h b/src/Storages/MergeTree/ExportPartTask.h new file mode 100644 index 000000000000..9c8fa6cc01dd --- /dev/null +++ b/src/Storages/MergeTree/ExportPartTask.h @@ -0,0 +1,33 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +class ExportPartTask : public IExecutableTask +{ +public: + explicit ExportPartTask( + MergeTreeData & storage_, + const MergeTreePartExportManifest & manifest_); + bool executeStep() override; + void onCompleted() override; + StorageID getStorageID() const override; + Priority getPriority() const override; + String getQueryId() const override; + + void cancel() noexcept override; + +private: + MergeTreeData & storage; + MergeTreePartExportManifest manifest; + QueryPipeline pipeline; + std::atomic cancel_requested = false; + + bool isCancelled() const; +}; + +} diff --git a/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp new file mode 100644 index 000000000000..79b92663b7bf --- /dev/null +++ b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp @@ -0,0 +1,310 @@ +#include +#include +#include +#include "Storages/MergeTree/ExportPartitionUtils.h" +#include "Common/logger_useful.h" +#include +#include +#include + +namespace DB +{ +namespace +{ + /* + Remove expired entries and fix non-committed exports that have already exported all parts. + + Return values: + - true: the cleanup was successful, the entry is removed from the entries_by_key container and the function returns true. Proceed to the next entry. + - false: the cleanup was not successful, the entry is not removed from the entries_by_key container and the function returns false. + */ + bool tryCleanup( + const zkutil::ZooKeeperPtr & zk, + const std::string & entry_path, + const LoggerPtr & log, + const ContextPtr & context, + const std::string & key, + const ExportReplicatedMergeTreePartitionManifest & metadata, + const time_t now, + const bool is_pending, + auto & entries_by_key + ) + { + bool has_expired = metadata.create_time < now - static_cast(metadata.ttl_seconds); + + if (has_expired && !is_pending) + { + zk->tryRemoveRecursive(fs::path(entry_path)); + auto it = entries_by_key.find(key); + if (it != entries_by_key.end()) + entries_by_key.erase(it); + LOG_INFO(log, "ExportPartition Manifest Updating Task: Removed {}: expired", key); + + return true; + } + else if (is_pending) + { + std::vector parts_in_processing_or_pending; + if (Coordination::Error::ZOK != zk->tryGetChildren(fs::path(entry_path) / "processing", parts_in_processing_or_pending)) + { + LOG_INFO(log, "ExportPartition Manifest Updating Task: Failed to get parts in processing or pending, skipping"); + return false; + } + + if (parts_in_processing_or_pending.empty()) + { + LOG_INFO(log, "ExportPartition Manifest Updating Task: Cleanup found PENDING for {} with all parts exported, try to fix it by committing the export", entry_path); + + const auto destination_storage_id = StorageID(QualifiedTableName {metadata.destination_database, metadata.destination_table}); + const auto destination_storage = DatabaseCatalog::instance().tryGetTable(destination_storage_id, context); + if (!destination_storage) + { + LOG_INFO(log, "ExportPartition Manifest Updating Task: Failed to reconstruct destination storage: {}, skipping", destination_storage_id.getNameForLogs()); + return false; + } + + /// it sounds like a replica exported the last part, but was not able to commit the export. Try to fix it + ExportPartitionUtils::commit(metadata, destination_storage, zk, log, entry_path, context); + + return true; + } + } + + return false; + } +} + +ExportPartitionManifestUpdatingTask::ExportPartitionManifestUpdatingTask(StorageReplicatedMergeTree & storage_) + : storage(storage_) +{ +} + +void ExportPartitionManifestUpdatingTask::poll() +{ + std::lock_guard lock(storage.export_merge_tree_partition_mutex); + + LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Polling for new entries for table {}. Current number of entries: {}", storage.getStorageID().getNameForLogs(), storage.export_merge_tree_partition_task_entries_by_key.size()); + + auto zk = storage.getZooKeeper(); + + const std::string exports_path = fs::path(storage.zookeeper_path) / "exports"; + const std::string cleanup_lock_path = fs::path(storage.zookeeper_path) / "exports_cleanup_lock"; + + auto cleanup_lock = zkutil::EphemeralNodeHolder::tryCreate(cleanup_lock_path, *zk, storage.replica_name); + if (cleanup_lock) + { + LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Cleanup lock acquired, will remove stale entries"); + } + + Coordination::Stat stat; + const auto children = zk->getChildrenWatch(exports_path, &stat, storage.export_merge_tree_partition_watch_callback); + const std::unordered_set zk_children(children.begin(), children.end()); + + const auto now = time(nullptr); + + auto & entries_by_key = storage.export_merge_tree_partition_task_entries_by_key; + + /// Load new entries + /// If we have the cleanup lock, also remove stale entries from zk and local + /// Upload dangling commit files if any + for (const auto & key : zk_children) + { + const std::string entry_path = fs::path(exports_path) / key; + + std::string metadata_json; + if (!zk->tryGet(fs::path(entry_path) / "metadata.json", metadata_json)) + { + LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Skipping {}: missing metadata.json", key); + continue; + } + + const auto metadata = ExportReplicatedMergeTreePartitionManifest::fromJsonString(metadata_json); + + const auto local_entry = entries_by_key.find(key); + + /// If the zk entry has been replaced with export_merge_tree_partition_force_export, checking only for the export key is not enough + /// we need to make sure it is the same transaction id. If it is not, it needs to be replaced. + bool has_local_entry_and_is_up_to_date = local_entry != entries_by_key.end() + && local_entry->manifest.transaction_id == metadata.transaction_id; + + /// If the entry is up to date and we don't have the cleanup lock, early exit, nothing to be done. + if (!cleanup_lock && has_local_entry_and_is_up_to_date) + continue; + + std::weak_ptr weak_manifest_updater = storage.export_merge_tree_partition_manifest_updater; + + auto status_watch_callback = std::make_shared([weak_manifest_updater, key](const Coordination::WatchResponse &) + { + /// If the table is dropped but the watch is not removed, we need to prevent use after free + /// below code assumes that if manifest updater is still alive, the status handling task is also alive + if (auto manifest_updater = weak_manifest_updater.lock()) + { + manifest_updater->addStatusChange(key); + manifest_updater->storage.export_merge_tree_partition_status_handling_task->schedule(); + } + }); + + std::string status; + if (!zk->tryGetWatch(fs::path(entry_path) / "status", status, nullptr, status_watch_callback)) + { + LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Skipping {}: missing status", key); + continue; + } + + bool is_pending = status == "PENDING"; + + /// if we have the cleanup lock, try to cleanup + /// if we successfully cleaned it up, early exit + if (cleanup_lock) + { + bool cleanup_successful = tryCleanup( + zk, + entry_path, + storage.log.load(), + storage.getContext(), + key, + metadata, + now, + is_pending, entries_by_key); + + if (cleanup_successful) + continue; + } + + if (!is_pending) + { + LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Skipping {}: status is not PENDING", key); + continue; + } + + if (has_local_entry_and_is_up_to_date) + { + LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Skipping {}: already exists", key); + continue; + } + + addTask(metadata, key, entries_by_key); + } + + /// Remove entries that were deleted by someone else + removeStaleEntries(zk_children, entries_by_key); + + LOG_INFO(storage.log, "ExportPartition Manifest Updating task: finished polling for new entries. Number of entries: {}", entries_by_key.size()); + + storage.export_merge_tree_partition_select_task->schedule(); +} + +void ExportPartitionManifestUpdatingTask::addTask( + const ExportReplicatedMergeTreePartitionManifest & metadata, + const std::string & key, + auto & entries_by_key +) +{ + std::vector part_references; + + for (const auto & part_name : metadata.parts) + { + if (const auto part = storage.getPartIfExists(part_name, {MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated})) + { + part_references.push_back(part); + } + } + + /// Insert or update entry. The multi_index container automatically maintains both indexes. + auto entry = ExportReplicatedMergeTreePartitionTaskEntry {metadata, ExportReplicatedMergeTreePartitionTaskEntry::Status::PENDING, std::move(part_references)}; + auto it = entries_by_key.find(key); + if (it != entries_by_key.end()) + entries_by_key.replace(it, entry); + else + entries_by_key.insert(entry); +} + +void ExportPartitionManifestUpdatingTask::removeStaleEntries( + const std::unordered_set & zk_children, + auto & entries_by_key +) +{ + for (auto it = entries_by_key.begin(); it != entries_by_key.end();) + { + const auto & key = it->getCompositeKey(); + if (zk_children.contains(key)) + { + ++it; + continue; + } + + const auto & transaction_id = it->manifest.transaction_id; + LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Export task {} was deleted, calling killExportPartition for transaction {}", key, transaction_id); + + try + { + storage.killExportPart(transaction_id); + } + catch (...) + { + tryLogCurrentException(storage.log, __PRETTY_FUNCTION__); + } + + it = entries_by_key.erase(it); + } +} + +void ExportPartitionManifestUpdatingTask::addStatusChange(const std::string & key) +{ + std::lock_guard lock(status_changes_mutex); + status_changes.emplace(key); +} + +void ExportPartitionManifestUpdatingTask::handleStatusChanges() +{ + std::lock_guard lock(status_changes_mutex); + std::lock_guard task_entries_lock(storage.export_merge_tree_partition_mutex); + auto zk = storage.getZooKeeper(); + + LOG_INFO(storage.log, "ExportPartition Manifest Updating task: handling status changes. Number of status changes: {}", status_changes.size()); + + while (!status_changes.empty()) + { + LOG_INFO(storage.log, "ExportPartition Manifest Updating task: handling status change for task {}", status_changes.front()); + const auto key = status_changes.front(); + status_changes.pop(); + + auto it = storage.export_merge_tree_partition_task_entries_by_key.find(key); + if (it == storage.export_merge_tree_partition_task_entries_by_key.end()) + continue; + + /// get new status from zk + std::string new_status_string; + if (!zk->tryGet(fs::path(storage.zookeeper_path) / "exports" / key / "status", new_status_string)) + { + LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Failed to get new status for task {}, skipping", key); + continue; + } + + const auto new_status = magic_enum::enum_cast(new_status_string); + if (!new_status) + { + LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Invalid status {} for task {}, skipping", new_status_string, key); + continue; + } + + LOG_INFO(storage.log, "ExportPartition Manifest Updating task: status changed for task {}. New status: {}", key, magic_enum::enum_name(*new_status).data()); + + /// If status changed to KILLED, cancel local export operations + if (*new_status == ExportReplicatedMergeTreePartitionTaskEntry::Status::KILLED) + { + try + { + storage.killExportPart(it->manifest.transaction_id); + } + catch (...) + { + tryLogCurrentException(storage.log, __PRETTY_FUNCTION__); + } + } + + it->status = *new_status; + } +} + +} diff --git a/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.h b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.h new file mode 100644 index 000000000000..ea52f679d654 --- /dev/null +++ b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.h @@ -0,0 +1,42 @@ +#pragma once + +#include +#include +#include +#include +namespace DB +{ + +class StorageReplicatedMergeTree; +struct ExportReplicatedMergeTreePartitionManifest; + +class ExportPartitionManifestUpdatingTask +{ +public: + ExportPartitionManifestUpdatingTask(StorageReplicatedMergeTree & storage); + + void poll(); + + void handleStatusChanges(); + + void addStatusChange(const std::string & key); + +private: + StorageReplicatedMergeTree & storage; + + void addTask( + const ExportReplicatedMergeTreePartitionManifest & metadata, + const std::string & key, + auto & entries_by_key + ); + + void removeStaleEntries( + const std::unordered_set & zk_children, + auto & entries_by_key + ); + + std::mutex status_changes_mutex; + std::queue status_changes; +}; + +} diff --git a/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp b/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp new file mode 100644 index 000000000000..925d7eafe412 --- /dev/null +++ b/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp @@ -0,0 +1,402 @@ +#include +#include +#include +#include +#include +#include +#include "Storages/MergeTree/ExportPartitionUtils.h" +#include "Storages/MergeTree/MergeTreePartExportManifest.h" + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int QUERY_WAS_CANCELLED; + extern const int LOGICAL_ERROR; +} + +namespace +{ + ContextPtr getContextCopyWithTaskSettings(const ContextPtr & context, const ExportReplicatedMergeTreePartitionManifest & manifest) + { + auto context_copy = Context::createCopy(context); + context_copy->makeQueryContextForExportPart(); + context_copy->setCurrentQueryId(manifest.query_id); + context_copy->setSetting("output_format_parallel_formatting", manifest.parallel_formatting); + context_copy->setSetting("output_format_parquet_parallel_encoding", manifest.parquet_parallel_encoding); + context_copy->setSetting("max_threads", manifest.max_threads); + context_copy->setSetting("export_merge_tree_part_file_already_exists_policy", String(magic_enum::enum_name(manifest.file_already_exists_policy))); + context_copy->setSetting("export_merge_tree_part_max_bytes_per_file", manifest.max_bytes_per_file); + context_copy->setSetting("export_merge_tree_part_max_rows_per_file", manifest.max_rows_per_file); + + /// always skip pending mutations and patch parts because we already validated the parts during query processing + context_copy->setSetting("export_merge_tree_part_throw_on_pending_mutations", false); + context_copy->setSetting("export_merge_tree_part_throw_on_pending_patch_parts", false); + + return context_copy; + } +} + +ExportPartitionTaskScheduler::ExportPartitionTaskScheduler(StorageReplicatedMergeTree & storage_) + : storage(storage_) +{ +} + +void ExportPartitionTaskScheduler::run() +{ + std::lock_guard lock(storage.export_merge_tree_partition_mutex); + + auto zk = storage.getZooKeeper(); + + // Iterate sorted by create_time + for (auto & entry : storage.export_merge_tree_partition_task_entries_by_create_time) + { + const auto & manifest = entry.manifest; + const auto key = entry.getCompositeKey(); + const auto database = storage.getContext()->resolveDatabase(manifest.destination_database); + const auto & table = manifest.destination_table; + + /// No need to query zk for status if the local one is not PENDING + if (entry.status != ExportReplicatedMergeTreePartitionTaskEntry::Status::PENDING) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Skipping... Local status is {}", magic_enum::enum_name(entry.status).data()); + continue; + } + + const auto destination_storage_id = StorageID(QualifiedTableName {database, table}); + + const auto destination_storage = DatabaseCatalog::instance().tryGetTable(destination_storage_id, storage.getContext()); + + if (!destination_storage) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to reconstruct destination storage: {}, skipping", destination_storage_id.getNameForLogs()); + continue; + } + + std::string status_in_zk_string; + if (!zk->tryGet(fs::path(storage.zookeeper_path) / "exports" / key / "status", status_in_zk_string)) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to get status, skipping"); + continue; + } + + const auto status_in_zk = magic_enum::enum_cast(status_in_zk_string); + + if (!status_in_zk) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to get status from zk, skipping"); + continue; + } + + if (status_in_zk.value() != ExportReplicatedMergeTreePartitionTaskEntry::Status::PENDING) + { + entry.status = status_in_zk.value(); + LOG_INFO(storage.log, "ExportPartition scheduler task: Skipping... Status from zk is {}", entry.status); + continue; + } + + std::vector parts_in_processing_or_pending; + + if (Coordination::Error::ZOK != zk->tryGetChildren(fs::path(storage.zookeeper_path) / "exports" / key / "processing", parts_in_processing_or_pending)) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to get parts in processing or pending, skipping"); + continue; + } + + if (parts_in_processing_or_pending.empty()) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: No parts in processing or pending, skipping"); + continue; + } + + std::vector locked_parts; + + if (Coordination::Error::ZOK != zk->tryGetChildren(fs::path(storage.zookeeper_path) / "exports" / key / "locks", locked_parts)) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to get locked parts, skipping"); + continue; + } + + std::unordered_set locked_parts_set(locked_parts.begin(), locked_parts.end()); + + for (const auto & zk_part_name : parts_in_processing_or_pending) + { + if (locked_parts_set.contains(zk_part_name)) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} is locked, skipping", zk_part_name); + continue; + } + + const auto part = storage.getPartIfExists(zk_part_name, {MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated}); + if (!part) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} not found locally, skipping", zk_part_name); + continue; + } + + if (Coordination::Error::ZOK != zk->tryCreate(fs::path(storage.zookeeper_path) / "exports" / key / "locks" / zk_part_name, storage.replica_name, zkutil::CreateMode::Ephemeral)) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to lock part {}, skipping", zk_part_name); + continue; + } + + try + { + storage.exportPartToTable( + part->name, + destination_storage_id, + manifest.transaction_id, + getContextCopyWithTaskSettings(storage.getContext(), manifest), + /*allow_outdated_parts*/ true, + [this, key, zk_part_name, manifest, destination_storage] + (MergeTreePartExportManifest::CompletionCallbackResult result) + { + handlePartExportCompletion(key, zk_part_name, manifest, destination_storage, result); + }); + } + catch (const Exception &) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + zk->tryRemove(fs::path(storage.zookeeper_path) / "exports" / key / "locks" / zk_part_name); + /// we should not increment retry_count because the node might just be full + } + } + } + + /// maybe we failed to schedule or failed to export, need to retry eventually + storage.export_merge_tree_partition_select_task->scheduleAfter(1000 * 5); +} + +void ExportPartitionTaskScheduler::handlePartExportCompletion( + const std::string & export_key, + const std::string & part_name, + const ExportReplicatedMergeTreePartitionManifest & manifest, + const StoragePtr & destination_storage, + const MergeTreePartExportManifest::CompletionCallbackResult & result) +{ + const auto export_path = fs::path(storage.zookeeper_path) / "exports" / export_key; + const auto processing_parts_path = export_path / "processing"; + const auto processed_part_path = export_path / "processed" / part_name; + const auto zk = storage.getZooKeeper(); + + if (result.success) + { + handlePartExportSuccess(manifest, destination_storage, processing_parts_path, processed_part_path, part_name, export_path, zk, result.relative_paths_in_destination_storage); + } + else + { + handlePartExportFailure(processing_parts_path, part_name, export_path, zk, result.exception, manifest.max_retries); + } +} + +void ExportPartitionTaskScheduler::handlePartExportSuccess( + const ExportReplicatedMergeTreePartitionManifest & manifest, + const StoragePtr & destination_storage, + const std::filesystem::path & processing_parts_path, + const std::filesystem::path & processed_part_path, + const std::string & part_name, + const std::filesystem::path & export_path, + const zkutil::ZooKeeperPtr & zk, + const std::vector & relative_paths_in_destination_storage +) +{ + LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} exported successfully, paths size: {}", part_name, relative_paths_in_destination_storage.size()); + + for (const auto & relative_path_in_destination_storage : relative_paths_in_destination_storage) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: {}", relative_path_in_destination_storage); + } + + if (!tryToMovePartToProcessed(export_path, processing_parts_path, processed_part_path, part_name, relative_paths_in_destination_storage, zk)) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to move part to processed, will not commit export partition"); + return; + } + + LOG_INFO(storage.log, "ExportPartition scheduler task: Marked part export {} as completed", part_name); + + if (!areAllPartsProcessed(export_path, zk)) + { + return; + } + + LOG_INFO(storage.log, "ExportPartition scheduler task: All parts are processed, will try to commit export partition"); + + ExportPartitionUtils::commit(manifest, destination_storage, zk, storage.log.load(), export_path, storage.getContext()); +} + +void ExportPartitionTaskScheduler::handlePartExportFailure( + const std::filesystem::path & processing_parts_path, + const std::string & part_name, + const std::filesystem::path & export_path, + const zkutil::ZooKeeperPtr & zk, + const std::optional & exception, + size_t max_retries +) +{ + if (!exception) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "ExportPartition scheduler task: No exception provided for error handling. Sounds like a bug"); + } + + /// Early exit if the query was cancelled - no need to increment error counts + if (exception->code() == ErrorCodes::QUERY_WAS_CANCELLED) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} export was cancelled, skipping error handling", part_name); + return; + } + + Coordination::Stat locked_by_stat; + std::string locked_by; + + if (!zk->tryGet(export_path / "locks" / part_name, locked_by, &locked_by_stat)) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} is not locked by any replica, will not increment error counts", part_name); + return; + } + + if (locked_by != storage.replica_name) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} is locked by another replica, will not increment error counts", part_name); + return; + } + + Coordination::Requests ops; + + const auto processing_part_path = processing_parts_path / part_name; + + std::string processing_part_string; + + if (!zk->tryGet(processing_part_path, processing_part_string)) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to get processing part, will not increment error counts"); + return; + } + + /// todo arthur could this have been cached? + auto processing_part_entry = ExportReplicatedMergeTreePartitionProcessingPartEntry::fromJsonString(processing_part_string); + + processing_part_entry.retry_count++; + + if (processing_part_entry.retry_count) + { + ops.emplace_back(zkutil::makeRemoveRequest(export_path / "locks" / part_name, locked_by_stat.version)); + ops.emplace_back(zkutil::makeSetRequest(processing_part_path, processing_part_entry.toJsonString(), -1)); + + if (processing_part_entry.retry_count >= max_retries) + { + /// just set status in processing_part_path and finished_by + processing_part_entry.status = ExportReplicatedMergeTreePartitionProcessingPartEntry::Status::FAILED; + processing_part_entry.finished_by = storage.replica_name; + + ops.emplace_back(zkutil::makeSetRequest(export_path / "status", String(magic_enum::enum_name(ExportReplicatedMergeTreePartitionTaskEntry::Status::FAILED)).data(), -1)); + LOG_INFO(storage.log, "ExportPartition scheduler task: Retry count limit exceeded for part {}, will try to fail the entire task", part_name); + } + + std::size_t num_exceptions = 0; + + const auto exceptions_per_replica_path = export_path / "exceptions_per_replica" / storage.replica_name; + const auto count_path = exceptions_per_replica_path / "count"; + const auto last_exception_path = exceptions_per_replica_path / "last_exception"; + + if (zk->exists(exceptions_per_replica_path)) + { + std::string num_exceptions_string; + zk->tryGet(count_path, num_exceptions_string); + num_exceptions = std::stoull(num_exceptions_string.c_str()); + + ops.emplace_back(zkutil::makeSetRequest(last_exception_path / "part", part_name, -1)); + ops.emplace_back(zkutil::makeSetRequest(last_exception_path / "exception", exception->message(), -1)); + } + else + { + ops.emplace_back(zkutil::makeCreateRequest(exceptions_per_replica_path, "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(count_path, "0", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(last_exception_path, "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(last_exception_path / "part", part_name, zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(last_exception_path / "exception", exception->message(), zkutil::CreateMode::Persistent)); + } + + num_exceptions++; + ops.emplace_back(zkutil::makeSetRequest(count_path, std::to_string(num_exceptions), -1)); + + Coordination::Responses responses; + if (Coordination::Error::ZOK != zk->tryMulti(ops, responses)) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: All failure mechanism failed, will not try to update it"); + return; + } + } +} + +bool ExportPartitionTaskScheduler::tryToMovePartToProcessed( + const std::filesystem::path & export_path, + const std::filesystem::path & processing_parts_path, + const std::filesystem::path & processed_part_path, + const std::string & part_name, + const std::vector & relative_paths_in_destination_storage, + const zkutil::ZooKeeperPtr & zk +) +{ + Coordination::Stat locked_by_stat; + std::string locked_by; + + if (!zk->tryGet(export_path / "locks" / part_name, locked_by, &locked_by_stat)) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} is not locked by any replica, will not commit or set it as completed", part_name); + return false; + } + + /// Is this a good idea? what if the file we just pushed to s3 ends up triggering an exception in the replica that actually locks the part and it does not commit? + /// I guess we should not throw if file already exists for export partition, hard coded. + if (locked_by != storage.replica_name) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} is locked by another replica, will not commit or set it as completed", part_name); + return false; + } + + Coordination::Requests requests; + + ExportReplicatedMergeTreePartitionProcessedPartEntry processed_part_entry; + processed_part_entry.part_name = part_name; + processed_part_entry.paths_in_destination = relative_paths_in_destination_storage; + processed_part_entry.finished_by = storage.replica_name; + + requests.emplace_back(zkutil::makeRemoveRequest(processing_parts_path / part_name, -1)); + requests.emplace_back(zkutil::makeCreateRequest(processed_part_path, processed_part_entry.toJsonString(), zkutil::CreateMode::Persistent)); + requests.emplace_back(zkutil::makeRemoveRequest(export_path / "locks" / part_name, locked_by_stat.version)); + + Coordination::Responses responses; + if (Coordination::Error::ZOK != zk->tryMulti(requests, responses)) + { + /// todo arthur remember what to do here + LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to update export path, skipping"); + return false; + } + + return true; +} + +bool ExportPartitionTaskScheduler::areAllPartsProcessed( + const std::filesystem::path & export_path, + const zkutil::ZooKeeperPtr & zk) +{ + Strings parts_in_processing_or_pending; + if (Coordination::Error::ZOK != zk->tryGetChildren(export_path / "processing", parts_in_processing_or_pending)) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to get parts in processing or pending, will not try to commit export partition"); + return false; + } + + if (!parts_in_processing_or_pending.empty()) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: There are still parts in processing or pending, will not try to commit export partition"); + return false; + } + + return true; +} + +} diff --git a/src/Storages/MergeTree/ExportPartitionTaskScheduler.h b/src/Storages/MergeTree/ExportPartitionTaskScheduler.h new file mode 100644 index 000000000000..29a41fde1cb9 --- /dev/null +++ b/src/Storages/MergeTree/ExportPartitionTaskScheduler.h @@ -0,0 +1,66 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class Exception; +class StorageReplicatedMergeTree; + +struct ExportReplicatedMergeTreePartitionManifest; + +/// todo arthur remember to add check(lock, version) when updating stuff because maybe if we believe we have the lock, we might not actually have it +class ExportPartitionTaskScheduler +{ +public: + ExportPartitionTaskScheduler(StorageReplicatedMergeTree & storage); + + void run(); +private: + StorageReplicatedMergeTree & storage; + + /// todo arthur maybe it is invalid to grab the manifst here + void handlePartExportCompletion( + const std::string & export_key, + const std::string & part_name, + const ExportReplicatedMergeTreePartitionManifest & manifest, + const StoragePtr & destination_storage, + const MergeTreePartExportManifest::CompletionCallbackResult & result); + + void handlePartExportSuccess( + const ExportReplicatedMergeTreePartitionManifest & manifest, + const StoragePtr & destination_storage, + const std::filesystem::path & processing_parts_path, + const std::filesystem::path & processed_part_path, + const std::string & part_name, + const std::filesystem::path & export_path, + const zkutil::ZooKeeperPtr & zk, + const std::vector & relative_paths_in_destination_storage + ); + + void handlePartExportFailure( + const std::filesystem::path & processing_parts_path, + const std::string & part_name, + const std::filesystem::path & export_path, + const zkutil::ZooKeeperPtr & zk, + const std::optional & exception, + size_t max_retries); + + bool tryToMovePartToProcessed( + const std::filesystem::path & export_path, + const std::filesystem::path & processing_parts_path, + const std::filesystem::path & processed_part_path, + const std::string & part_name, + const std::vector & relative_paths_in_destination_storage, + const zkutil::ZooKeeperPtr & zk + ); + + bool areAllPartsProcessed( + const std::filesystem::path & export_path, + const zkutil::ZooKeeperPtr & zk + ); +}; + +} diff --git a/src/Storages/MergeTree/ExportPartitionUtils.cpp b/src/Storages/MergeTree/ExportPartitionUtils.cpp new file mode 100644 index 000000000000..51ed72b7f6f6 --- /dev/null +++ b/src/Storages/MergeTree/ExportPartitionUtils.cpp @@ -0,0 +1,104 @@ +#include +#include +#include +#include "Storages/ExportReplicatedMergeTreePartitionManifest.h" +#include "Storages/ExportReplicatedMergeTreePartitionTaskEntry.h" +#include + +namespace DB +{ + +namespace fs = std::filesystem; + +namespace ExportPartitionUtils +{ + /// Collect all the exported paths from the processed parts + /// If multiRead is supported by the keeper implementation, it is done in a single request + /// Otherwise, multiple async requests are sent + std::vector getExportedPaths(const LoggerPtr & log, const zkutil::ZooKeeperPtr & zk, const std::string & export_path) + { + std::vector exported_paths; + + LOG_INFO(log, "ExportPartition: Getting exported paths for {}", export_path); + + const auto processed_parts_path = fs::path(export_path) / "processed"; + + std::vector processed_parts; + if (Coordination::Error::ZOK != zk->tryGetChildren(processed_parts_path, processed_parts)) + { + /// todo arthur do something here + LOG_INFO(log, "ExportPartition: Failed to get parts children, exiting"); + return {}; + } + + std::vector get_paths; + + for (const auto & processed_part : processed_parts) + { + get_paths.emplace_back(processed_parts_path / processed_part); + } + + auto responses = zk->tryGet(get_paths); + + responses.waitForResponses(); + + for (size_t i = 0; i < responses.size(); ++i) + { + if (responses[i].error != Coordination::Error::ZOK) + { + /// todo arthur what to do in this case? + /// It could be that zk is corrupt, in that case we should fail the task + /// but it can also be some temporary network issue? not sure + LOG_INFO(log, "ExportPartition: Failed to get exported path, exiting"); + return {}; + } + + const auto processed_part_entry = ExportReplicatedMergeTreePartitionProcessedPartEntry::fromJsonString(responses[i].data); + + for (const auto & path_in_destination : processed_part_entry.paths_in_destination) + { + exported_paths.emplace_back(path_in_destination); + } + } + + return exported_paths; + } + + void commit( + const ExportReplicatedMergeTreePartitionManifest & manifest, + const StoragePtr & destination_storage, + const zkutil::ZooKeeperPtr & zk, + const LoggerPtr & log, + const std::string & entry_path, + const ContextPtr & context) + { + const auto exported_paths = ExportPartitionUtils::getExportedPaths(log, zk, entry_path); + + if (exported_paths.empty()) + { + LOG_WARNING(log, "ExportPartition: No exported paths found, will not commit export. This might be a bug"); + return; + } + + //// not checking for an exact match because a single part might generate multiple files + if (exported_paths.size() < manifest.parts.size()) + { + LOG_WARNING(log, "ExportPartition: Reached the commit phase, but exported paths size is less than the number of parts, will not commit export. This might be a bug"); + return; + } + + destination_storage->commitExportPartitionTransaction(manifest.transaction_id, manifest.partition_id, exported_paths, context); + + LOG_INFO(log, "ExportPartition: Committed export, mark as completed"); + if (Coordination::Error::ZOK == zk->trySet(fs::path(entry_path) / "status", String(magic_enum::enum_name(ExportReplicatedMergeTreePartitionTaskEntry::Status::COMPLETED)).data(), -1)) + { + LOG_INFO(log, "ExportPartition: Marked export as completed"); + } + else + { + LOG_INFO(log, "ExportPartition: Failed to mark export as completed, will not try to fix it"); + } + } +} + +} diff --git a/src/Storages/MergeTree/ExportPartitionUtils.h b/src/Storages/MergeTree/ExportPartitionUtils.h new file mode 100644 index 000000000000..40fe04a5bfd3 --- /dev/null +++ b/src/Storages/MergeTree/ExportPartitionUtils.h @@ -0,0 +1,28 @@ +#pragma once + +#include +#include +#include +#include +#include "Storages/IStorage.h" + +namespace DB +{ + +struct ExportReplicatedMergeTreePartitionManifest; + +namespace ExportPartitionUtils +{ + std::vector getExportedPaths(const LoggerPtr & log, const zkutil::ZooKeeperPtr & zk, const std::string & export_path); + + void commit( + const ExportReplicatedMergeTreePartitionManifest & manifest, + const StoragePtr & destination_storage, + const zkutil::ZooKeeperPtr & zk, + const LoggerPtr & log, + const std::string & entry_path, + const ContextPtr & context + ); +} + +} diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 834f88941093..1d6ad1ef20a2 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -301,6 +301,42 @@ String IMergeTreeDataPart::MinMaxIndex::getFileColumnName(const String & column_ return stream_name; } +Block IMergeTreeDataPart::MinMaxIndex::getBlock(const MergeTreeData & data) const +{ + if (!initialized) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to get block from uninitialized MinMax index."); + + Block block; + + const auto metadata_snapshot = data.getInMemoryMetadataPtr(); + const auto & partition_key = metadata_snapshot->getPartitionKey(); + + const auto minmax_column_names = data.getMinMaxColumnsNames(partition_key); + const auto minmax_column_types = data.getMinMaxColumnsTypes(partition_key); + const auto minmax_idx_size = minmax_column_types.size(); + + for (size_t i = 0; i < minmax_idx_size; ++i) + { + const auto & data_type = minmax_column_types[i]; + const auto & column_name = minmax_column_names[i]; + + const auto column = data_type->createColumn(); + + auto range = hyperrectangle.at(i); + range.shrinkToIncludedIfPossible(); + + const auto & min_val = range.left; + const auto & max_val = range.right; + + column->insert(min_val); + column->insert(max_val); + + block.insert(ColumnWithTypeAndName(column->getPtr(), data_type, column_name)); + } + + return block; +} + void IMergeTreeDataPart::incrementStateMetric(MergeTreeDataPartState state_) const { switch (state_) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 9216ea55cc42..52d33fac8cd9 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -381,6 +381,8 @@ class IMergeTreeDataPart : public std::enable_shared_from_this; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index b7f36962b0b6..cacb45e4174f 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3,7 +3,9 @@ #include #include +#include #include +#include #include #include #include @@ -25,6 +27,11 @@ #include #include #include +#include "Storages/MergeTree/ExportPartTask.h" +#include +#include +#include +#include #include #include #include @@ -41,6 +48,7 @@ #include #include #include +#include #include #include #include @@ -96,6 +104,7 @@ #include #include #include +#include #include @@ -114,6 +123,7 @@ #include #include #include +#include #include #include @@ -160,6 +170,10 @@ namespace ProfileEvents extern const Event RestorePartsSkippedFiles; extern const Event RestorePartsSkippedBytes; extern const Event LoadedStatisticsMicroseconds; + extern const Event PartsExports; + extern const Event PartsExportTotalMilliseconds; + extern const Event PartsExportFailures; + extern const Event PartsExportDuplicated; } namespace CurrentMetrics @@ -211,6 +225,13 @@ namespace Setting extern const SettingsBool use_statistics; extern const SettingsBool use_statistics_cache; extern const SettingsBool use_partition_pruning; + extern const SettingsBool allow_experimental_export_merge_tree_part; + extern const SettingsUInt64 min_bytes_to_use_direct_io; + extern const SettingsMergeTreePartExportFileAlreadyExistsPolicy export_merge_tree_part_file_already_exists_policy; + extern const SettingsBool output_format_parallel_formatting; + extern const SettingsBool output_format_parquet_parallel_encoding; + extern const SettingsBool export_merge_tree_part_throw_on_pending_mutations; + extern const SettingsBool export_merge_tree_part_throw_on_pending_patch_parts; } namespace MergeTreeSetting @@ -341,6 +362,9 @@ namespace ErrorCodes extern const int CANNOT_FORGET_PARTITION; extern const int DATA_TYPE_CANNOT_BE_USED_IN_KEY; extern const int TOO_LARGE_LIGHTWEIGHT_UPDATES; + extern const int UNKNOWN_TABLE; + extern const int FILE_ALREADY_EXISTS; + extern const int PENDING_MUTATIONS_NOT_ALLOWED; } static String getPartNameFromAST(const ASTPtr & partition) @@ -4822,8 +4846,6 @@ void MergeTreeData::changeSettings( { if (new_settings) { - bool has_storage_policy_changed = false; - const auto & new_changes = new_settings->as().changes; StoragePolicyPtr new_storage_policy = nullptr; @@ -4862,8 +4884,6 @@ void MergeTreeData::changeSettings( disk->createDirectories(fs::path(relative_data_path) / DETACHED_DIR_NAME); } /// FIXME how would that be done while reloading configuration??? - - has_storage_policy_changed = true; } } } @@ -4897,9 +4917,6 @@ void MergeTreeData::changeSettings( setInMemoryMetadata(new_metadata); - if (has_storage_policy_changed) - startBackgroundMovesIfNeeded(); - if (has_refresh_statistics_interval_changed) { startStatisticsCache(); @@ -6583,6 +6600,189 @@ void MergeTreeData::movePartitionToTable(const PartitionCommand & command, Conte movePartitionToTable(dest_storage, command.partition, query_context); } +void MergeTreeData::exportPartToTable(const PartitionCommand & command, ContextPtr query_context) +{ + if (!query_context->getSettingsRef()[Setting::allow_experimental_export_merge_tree_part]) + { + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "Exporting merge tree part is experimental. Set `allow_experimental_export_merge_tree_part` to enable it"); + } + + const auto part_name = command.partition->as().value.safeGet(); + + if (!command.to_table_function) + { + const auto database_name = query_context->resolveDatabase(command.to_database); + exportPartToTable(part_name, StorageID{database_name, command.to_table}, generateSnowflakeIDString(), query_context); + + return; + } + + auto table_function_ast = command.to_table_function; + auto table_function_ptr = TableFunctionFactory::instance().get(command.to_table_function, query_context); + + if (table_function_ptr->needStructureHint()) + { + const auto source_metadata_ptr = getInMemoryMetadataPtr(); + + /// Grab only the readable columns from the source metadata to skip ephemeral columns + const auto readable_columns = ColumnsDescription(source_metadata_ptr->getColumns().getReadable()); + table_function_ptr->setStructureHint(readable_columns); + } + + if (command.partition_by_expr) + { + table_function_ptr->setPartitionBy(command.partition_by_expr); + } + + auto dest_storage = table_function_ptr->execute( + table_function_ast, + query_context, + table_function_ptr->getName(), + /* cached_columns */ {}, + /* use_global_context */ false, + /* is_insert_query */ true); + + if (!dest_storage) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Failed to reconstruct destination storage"); + } + + exportPartToTable(part_name, dest_storage, generateSnowflakeIDString(), query_context); +} + +void MergeTreeData::exportPartToTable( + const std::string & part_name, + const StorageID & destination_storage_id, + const String & transaction_id, + ContextPtr query_context, + bool allow_outdated_parts, + std::function completion_callback) +{ + auto dest_storage = DatabaseCatalog::instance().getTable(destination_storage_id, query_context); + + if (destination_storage_id == this->getStorageID()) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Exporting to the same table is not allowed"); + } + + exportPartToTable(part_name, dest_storage, transaction_id, query_context, allow_outdated_parts, completion_callback); +} + +void MergeTreeData::exportPartToTable( + const std::string & part_name, + const StoragePtr & dest_storage, + const String & transaction_id, + ContextPtr query_context, + bool allow_outdated_parts, + std::function completion_callback) +{ + if (!dest_storage->supportsImport()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Destination storage {} does not support MergeTree parts or uses unsupported partitioning", dest_storage->getName()); + + auto query_to_string = [] (const ASTPtr & ast) + { + return ast ? ast->formatWithSecretsOneLine() : ""; + }; + + auto source_metadata_ptr = getInMemoryMetadataPtr(); + auto destination_metadata_ptr = dest_storage->getInMemoryMetadataPtr(); + + const auto & source_columns = source_metadata_ptr->getColumns(); + + const auto & destination_columns = destination_metadata_ptr->getColumns(); + + /// compare all source readable columns with all destination insertable columns + /// this allows us to skip ephemeral columns + if (source_columns.getReadable().sizeOfDifference(destination_columns.getInsertable())) + throw Exception(ErrorCodes::INCOMPATIBLE_COLUMNS, "Tables have different structure"); + + if (query_to_string(source_metadata_ptr->getPartitionKeyAST()) != query_to_string(destination_metadata_ptr->getPartitionKeyAST())) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Tables have different partition key"); + + auto part = getPartIfExists(part_name, {MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated}); + + if (!part) + throw Exception(ErrorCodes::NO_SUCH_DATA_PART, "No such data part '{}' to export in table '{}'", + part_name, getStorageID().getFullTableName()); + + if (part->getState() == MergeTreeDataPartState::Outdated && !allow_outdated_parts) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Part {} is in the outdated state and cannot be exported", + part_name); + + const bool throw_on_pending_mutations = query_context->getSettingsRef()[Setting::export_merge_tree_part_throw_on_pending_mutations]; + const bool throw_on_pending_patch_parts = query_context->getSettingsRef()[Setting::export_merge_tree_part_throw_on_pending_patch_parts]; + + MergeTreeData::IMutationsSnapshot::Params mutations_snapshot_params + { + .metadata_version = source_metadata_ptr->getMetadataVersion(), + .min_part_metadata_version = part->getMetadataVersion(), + .need_data_mutations = throw_on_pending_mutations, + .need_alter_mutations = throw_on_pending_mutations || throw_on_pending_patch_parts, + .need_patch_parts = throw_on_pending_patch_parts, + }; + + const auto mutations_snapshot = getMutationsSnapshot(mutations_snapshot_params); + + const auto alter_conversions = getAlterConversionsForPart(part, mutations_snapshot, query_context); + + /// re-check `throw_on_pending_mutations` because `pending_mutations` might have been filled due to `throw_on_pending_patch_parts` + if (throw_on_pending_mutations && alter_conversions->hasMutations()) + { + throw Exception(ErrorCodes::PENDING_MUTATIONS_NOT_ALLOWED, + "Part {} can not be exported because there are pending mutations. Either wait for the mutations to be applied or set `export_merge_tree_part_throw_on_pending_mutations` to false", + part_name); + } + + if (alter_conversions->hasPatches()) + { + throw Exception(ErrorCodes::PENDING_MUTATIONS_NOT_ALLOWED, + "Part {} can not be exported because there are pending patch parts. Either wait for the patch parts to be applied or set `export_merge_tree_part_throw_on_pending_patch_parts` to false", + part_name); + } + + { + const auto format_settings = getFormatSettings(query_context); + MergeTreePartExportManifest manifest( + dest_storage, + part, + transaction_id, + query_context->getCurrentQueryId(), + query_context->getSettingsRef()[Setting::export_merge_tree_part_file_already_exists_policy].value, + query_context->getSettingsCopy(), + source_metadata_ptr, + completion_callback); + + std::lock_guard lock(export_manifests_mutex); + + if (!export_manifests.emplace(std::move(manifest)).second) + { + throw Exception(ErrorCodes::ABORTED, "Data part '{}' is already being exported", part_name); + } + } + + background_moves_assignee.trigger(); +} + +void MergeTreeData::killExportPart(const String & transaction_id) +{ + std::lock_guard lock(export_manifests_mutex); + + std::erase_if(export_manifests, [&](const auto & manifest) + { + if (manifest.transaction_id == transaction_id) + { + if (manifest.task) + manifest.task->cancel(); + + return true; + } + return false; + }); +} + void MergeTreeData::movePartitionToShard(const ASTPtr & /*partition*/, bool /*move_part*/, const String & /*to*/, ContextPtr /*query_context*/) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MOVE PARTITION TO SHARD is not supported by storage {}", getName()); @@ -6634,6 +6834,17 @@ Pipe MergeTreeData::alterPartition( } } break; + case PartitionCommand::EXPORT_PART: + { + exportPartToTable(command, query_context); + break; + } + + case PartitionCommand::EXPORT_PARTITION: + { + exportPartitionToTable(command, query_context); + break; + } case PartitionCommand::DROP_DETACHED_PARTITION: dropDetached(command.partition, command.part, query_context); @@ -9201,6 +9412,33 @@ std::pair MergeTreeData::cloneAn return std::make_pair(dst_data_part, std::move(temporary_directory_lock)); } +std::vector MergeTreeData::getExportsStatus() const +{ + std::lock_guard lock(export_manifests_mutex); + std::vector result; + + auto source_database = getStorageID().database_name; + auto source_table = getStorageID().table_name; + + for (const auto & manifest : export_manifests) + { + MergeTreeExportStatus status; + + status.source_database = source_database; + status.source_table = source_table; + const auto destination_storage_id = manifest.destination_storage_ptr->getStorageID(); + status.destination_database = destination_storage_id.database_name; + status.destination_table = destination_storage_id.table_name; + status.create_time = manifest.create_time; + status.part_name = manifest.data_part->name; + + result.emplace_back(std::move(status)); + } + + return result; +} + + bool MergeTreeData::canUseAdaptiveGranularity() const { const auto settings = getSettings(); @@ -9516,7 +9754,8 @@ void MergeTreeData::writePartLog( const DataPartsVector & source_parts, const MergeListEntry * merge_entry, std::shared_ptr profile_counters, - const Strings & mutation_ids) + const Strings & mutation_ids, + const ExportsListEntry * exports_entry) try { auto table_id = getStorageID(); @@ -9584,6 +9823,16 @@ try part_log_elem.rows = (*merge_entry)->rows_written; part_log_elem.peak_memory_usage = (*merge_entry)->getMemoryTracker().getPeak(); } + else if (exports_entry) + { + part_log_elem.rows_read = (*exports_entry)->rows_read; + part_log_elem.bytes_read_uncompressed = (*exports_entry)->bytes_read_uncompressed; + part_log_elem.peak_memory_usage = (*exports_entry)->getPeakMemoryUsage(); + part_log_elem.query_id = (*exports_entry)->query_id; + + /// no need to lock because at this point no one is writing to the destination file paths + part_log_elem.remote_file_paths = (*exports_entry)->destination_file_paths; + } if (profile_counters) { @@ -9627,21 +9876,46 @@ MergeTreeData::CurrentlyMovingPartsTagger::~CurrentlyMovingPartsTagger() bool MergeTreeData::scheduleDataMovingJob(BackgroundJobsAssignee & assignee) { - if (parts_mover.moves_blocker.isCancelled()) - return false; + if (!parts_mover.moves_blocker.isCancelled()) + { + auto moving_tagger = selectPartsForMove(); + if (!moving_tagger->parts_to_move.empty()) + { + assignee.scheduleMoveTask(std::make_shared( + [this, moving_tagger] () mutable + { + ReadSettings read_settings = Context::getGlobalContextInstance()->getReadSettings(); + WriteSettings write_settings = Context::getGlobalContextInstance()->getWriteSettings(); + return moveParts(moving_tagger, read_settings, write_settings, /* wait_for_move_if_zero_copy= */ false) == MovePartsOutcome::PartsMoved; + }, moves_assignee_trigger, getStorageID())); + return true; + } + } - auto moving_tagger = selectPartsForMove(); - if (moving_tagger->parts_to_move.empty()) - return false; + std::lock_guard lock(export_manifests_mutex); - assignee.scheduleMoveTask(std::make_shared( - [this, moving_tagger] () mutable + for (auto & manifest : export_manifests) + { + if (manifest.in_progress) { - ReadSettings read_settings = Context::getGlobalContextInstance()->getReadSettings(); - WriteSettings write_settings = Context::getGlobalContextInstance()->getWriteSettings(); - return moveParts(moving_tagger, read_settings, write_settings, /* wait_for_move_if_zero_copy= */ false) == MovePartsOutcome::PartsMoved; - }, moves_assignee_trigger, getStorageID())); - return true; + continue; + } + + auto task = std::make_shared(*this, manifest); + + manifest.in_progress = assignee.scheduleMoveTask(task); + + if (!manifest.in_progress) + { + continue; + } + + manifest.task = task; + + return true; + } + + return false; } bool MergeTreeData::areBackgroundMovesNeeded() const @@ -9859,6 +10133,10 @@ bool MergeTreeData::canUsePolymorphicParts() const return canUsePolymorphicParts(*getSettings(), unused); } +void MergeTreeData::startBackgroundMoves() +{ + background_moves_assignee.start(); +} void MergeTreeData::checkDropOrRenameCommandDoesntAffectInProgressMutations( const AlterCommand & command, const std::map & unfinished_mutations, ContextPtr local_context) const diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 85d3f5f7ae0d..0750c2b30693 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -21,6 +21,7 @@ #include #include #include +#include #include #include #include @@ -38,6 +39,8 @@ #include #include #include +#include +#include #include #include @@ -1041,6 +1044,31 @@ class MergeTreeData : public WithMutableContext, public IStorage, public IBackgr /// Moves partition to specified Table void movePartitionToTable(const PartitionCommand & command, ContextPtr query_context); + void exportPartToTable(const PartitionCommand & command, ContextPtr query_context); + + void exportPartToTable( + const std::string & part_name, + const StoragePtr & destination_storage, + const String & transaction_id, + ContextPtr query_context, + bool allow_outdated_parts = false, + std::function completion_callback = {}); + + void exportPartToTable( + const std::string & part_name, + const StorageID & destination_storage_id, + const String & transaction_id, + ContextPtr query_context, + bool allow_outdated_parts = false, + std::function completion_callback = {}); + + void killExportPart(const String & transaction_id); + + virtual void exportPartitionToTable(const PartitionCommand &, ContextPtr) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "EXPORT PARTITION is not implemented for engine {}", getName()); + } + /// Checks that Partition could be dropped right now /// Otherwise - throws an exception with detailed information. /// We do not use mutex because it is not very important that the size could change during the operation. @@ -1123,6 +1151,7 @@ class MergeTreeData : public WithMutableContext, public IStorage, public IBackgr const WriteSettings & write_settings); virtual std::vector getMutationsStatus() const = 0; + std::vector getExportsStatus() const; /// Returns true if table can create new parts with adaptive granularity /// Has additional constraint in replicated version @@ -1305,6 +1334,10 @@ class MergeTreeData : public WithMutableContext, public IStorage, public IBackgr /// Mutex for currently_moving_parts mutable std::mutex moving_parts_mutex; + mutable std::mutex export_manifests_mutex; + + std::set export_manifests; + PinnedPartUUIDsPtr getPinnedPartUUIDs() const; /// Schedules job to move parts between disks/volumes and so on. @@ -1397,6 +1430,7 @@ class MergeTreeData : public WithMutableContext, public IStorage, public IBackgr friend class IPartMetadataManager; friend class IMergedBlockOutputStream; // for access to log friend struct DataPartsLock; // for access to shared_parts_list/shared_ranges_in_parts + friend class ExportPartTask; bool require_part_metadata; @@ -1443,6 +1477,8 @@ class MergeTreeData : public WithMutableContext, public IStorage, public IBackgr size_t getColumnsDescriptionsCacheSize() const; protected: + void startBackgroundMoves(); + /// Engine-specific methods BrokenPartCallback broken_part_callback; @@ -1706,7 +1742,8 @@ class MergeTreeData : public WithMutableContext, public IStorage, public IBackgr const DataPartsVector & source_parts, const MergeListEntry * merge_entry, std::shared_ptr profile_counters, - const Strings & mutation_ids = {}); + const Strings & mutation_ids = {}, + const ExportsListEntry * exports_entry = nullptr); /// If part is assigned to merge or mutation (possibly replicated) /// Should be overridden by children, because they can have different @@ -1924,8 +1961,6 @@ class MergeTreeData : public WithMutableContext, public IStorage, public IBackgr bool canUsePolymorphicParts(const MergeTreeSettings & settings, String & out_reason) const; - virtual void startBackgroundMovesIfNeeded() = 0; - bool allow_nullable_key = false; bool allow_reverse_key = false; diff --git a/src/Storages/MergeTree/MergeTreeExportManifest.h b/src/Storages/MergeTree/MergeTreeExportManifest.h new file mode 100644 index 000000000000..05506ecb004a --- /dev/null +++ b/src/Storages/MergeTree/MergeTreeExportManifest.h @@ -0,0 +1,50 @@ +#include +#include + +namespace DB +{ + +struct MergeTreeExportManifest +{ + using DataPartPtr = std::shared_ptr; + + + MergeTreeExportManifest( + const StorageID & destination_storage_id_, + const DataPartPtr & data_part_, + bool overwrite_file_if_exists_, + const FormatSettings & format_settings_) + : destination_storage_id(destination_storage_id_), + data_part(data_part_), + overwrite_file_if_exists(overwrite_file_if_exists_), + format_settings(format_settings_), + create_time(time(nullptr)) {} + + StorageID destination_storage_id; + DataPartPtr data_part; + bool overwrite_file_if_exists; + FormatSettings format_settings; + + time_t create_time; + mutable bool in_progress = false; + + bool operator<(const MergeTreeExportManifest & rhs) const + { + // Lexicographic comparison: first compare destination storage, then part name + auto lhs_storage = destination_storage_id.getQualifiedName(); + auto rhs_storage = rhs.destination_storage_id.getQualifiedName(); + + if (lhs_storage != rhs_storage) + return lhs_storage < rhs_storage; + + return data_part->name < rhs.data_part->name; + } + + bool operator==(const MergeTreeExportManifest & rhs) const + { + return destination_storage_id.getQualifiedName() == rhs.destination_storage_id.getQualifiedName() + && data_part->name == rhs.data_part->name; + } +}; + +} diff --git a/src/Storages/MergeTree/MergeTreePartExportManifest.h b/src/Storages/MergeTree/MergeTreePartExportManifest.h new file mode 100644 index 000000000000..b9527d5140f6 --- /dev/null +++ b/src/Storages/MergeTree/MergeTreePartExportManifest.h @@ -0,0 +1,94 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +class Exception; + +class ExportPartTask; + +struct MergeTreePartExportManifest +{ + using FileAlreadyExistsPolicy = MergeTreePartExportFileAlreadyExistsPolicy; + + using DataPartPtr = std::shared_ptr; + + struct CompletionCallbackResult + { + private: + CompletionCallbackResult(bool success_, const std::vector & relative_paths_in_destination_storage_, std::optional exception_) + : success(success_), relative_paths_in_destination_storage(relative_paths_in_destination_storage_), exception(std::move(exception_)) {} + public: + + static CompletionCallbackResult createSuccess(const std::vector & relative_paths_in_destination_storage_) + { + return CompletionCallbackResult(true, relative_paths_in_destination_storage_, std::nullopt); + } + + static CompletionCallbackResult createFailure(Exception exception_) + { + return CompletionCallbackResult(false, {}, std::move(exception_)); + } + + bool success = false; + std::vector relative_paths_in_destination_storage; + std::optional exception; + }; + + MergeTreePartExportManifest( + const StoragePtr destination_storage_ptr_, + const DataPartPtr & data_part_, + const String & transaction_id_, + const String & query_id_, + FileAlreadyExistsPolicy file_already_exists_policy_, + const Settings & settings_, + const StorageMetadataPtr & metadata_snapshot_, + std::function completion_callback_ = {}) + : destination_storage_ptr(destination_storage_ptr_), + data_part(data_part_), + transaction_id(transaction_id_), + query_id(query_id_), + file_already_exists_policy(file_already_exists_policy_), + settings(settings_), + metadata_snapshot(metadata_snapshot_), + completion_callback(completion_callback_), + create_time(time(nullptr)) {} + + StoragePtr destination_storage_ptr; + DataPartPtr data_part; + /// Used for killing the export. + String transaction_id; + String query_id; + FileAlreadyExistsPolicy file_already_exists_policy; + Settings settings; + + /// Metadata snapshot captured at the time of query validation to prevent race conditions with mutations + /// Otherwise the export could fail if the schema changes between validation and execution + StorageMetadataPtr metadata_snapshot; + + std::function completion_callback; + + time_t create_time; + mutable bool in_progress = false; + mutable std::shared_ptr task = nullptr; + + bool operator<(const MergeTreePartExportManifest & rhs) const + { + return data_part->name < rhs.data_part->name; + } + + bool operator==(const MergeTreePartExportManifest & rhs) const + { + return data_part->name == rhs.data_part->name; + } +}; + +} diff --git a/src/Storages/MergeTree/MergeTreePartExportStatus.h b/src/Storages/MergeTree/MergeTreePartExportStatus.h new file mode 100644 index 000000000000..e71a2f15e6ed --- /dev/null +++ b/src/Storages/MergeTree/MergeTreePartExportStatus.h @@ -0,0 +1,20 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +struct MergeTreeExportStatus +{ + String source_database; + String source_table; + String destination_database; + String destination_table; + time_t create_time = 0; + std::string part_name; +}; + +} diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index ea16a5ff6602..6eed4c712e11 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -467,6 +467,22 @@ void MergeTreePartition::create(const StorageMetadataPtr & metadata_snapshot, Bl } } +Block MergeTreePartition::getBlockWithPartitionValues(const NamesAndTypesList & partition_columns) const +{ + chassert(partition_columns.size() == value.size()); + + Block result; + + std::size_t i = 0; + for (const auto & partition_column : partition_columns) + { + auto column = partition_column.type->createColumnConst(1, value[i++]); + result.insert({column, partition_column.type, partition_column.name}); + } + + return result; +} + NamesAndTypesList MergeTreePartition::executePartitionByExpression(const StorageMetadataPtr & metadata_snapshot, Block & block, ContextPtr context) { auto adjusted_partition_key = adjustPartitionKey(metadata_snapshot, context); diff --git a/src/Storages/MergeTree/MergeTreePartition.h b/src/Storages/MergeTree/MergeTreePartition.h index 4338b216cdb8..811cfdc2a90c 100644 --- a/src/Storages/MergeTree/MergeTreePartition.h +++ b/src/Storages/MergeTree/MergeTreePartition.h @@ -60,6 +60,8 @@ struct MergeTreePartition void create(const StorageMetadataPtr & metadata_snapshot, Block block, size_t row, ContextPtr context); + Block getBlockWithPartitionValues(const NamesAndTypesList & partition_columns) const; + /// Adjust partition key and execute its expression on block. Return sample block according to used expression. static NamesAndTypesList executePartitionByExpression(const StorageMetadataPtr & metadata_snapshot, Block & block, ContextPtr context); diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index ec217cab81c8..cb3e34ed7d2b 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -169,6 +169,10 @@ MergeTreeSequentialSource::MergeTreeSequentialSource( addThrottler(read_settings.remote_throttler, context->getMergesThrottler()); addThrottler(read_settings.local_throttler, context->getMergesThrottler()); break; + case Export: + addThrottler(read_settings.local_throttler, context->getExportsThrottler()); + addThrottler(read_settings.remote_throttler, context->getExportsThrottler()); + break; } MergeTreeReadTask::Extras extras = diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.h b/src/Storages/MergeTree/MergeTreeSequentialSource.h index abba230d9e79..a858adf33bb5 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.h +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.h @@ -15,6 +15,7 @@ enum MergeTreeSequentialSourceType { Mutation, Merge, + Export, }; /// Create stream for reading single part from MergeTree. diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index ed1913360591..8bef554cf999 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -34,6 +34,11 @@ namespace MergeTreeSetting extern const MergeTreeSettingsSeconds zookeeper_session_expiration_check_period; } +namespace ServerSetting +{ + extern const ServerSettingsBool enable_experimental_export_merge_tree_partition_feature; +} + namespace ErrorCodes { extern const int REPLICA_IS_ALREADY_ACTIVE; @@ -179,6 +184,14 @@ bool ReplicatedMergeTreeRestartingThread::runImpl() storage.mutations_updating_task->activateAndSchedule(); storage.mutations_finalizing_task->activateAndSchedule(); storage.merge_selecting_task->activateAndSchedule(); + + if (storage.getContext()->getServerSettings()[ServerSetting::enable_experimental_export_merge_tree_partition_feature]) + { + storage.export_merge_tree_partition_updating_task->activateAndSchedule(); + storage.export_merge_tree_partition_select_task->activateAndSchedule(); + storage.export_merge_tree_partition_status_handling_task->activateAndSchedule(); + } + storage.cleanup_thread.start(); storage.async_block_ids_cache.start(); storage.part_check_thread.start(); diff --git a/src/Storages/MergeTree/tests/gtest_export_partition_ordering.cpp b/src/Storages/MergeTree/tests/gtest_export_partition_ordering.cpp new file mode 100644 index 000000000000..c9e3ffd9eef9 --- /dev/null +++ b/src/Storages/MergeTree/tests/gtest_export_partition_ordering.cpp @@ -0,0 +1,75 @@ +#include +#include + +namespace DB +{ + +class ExportPartitionOrderingTest : public ::testing::Test +{ +protected: + ExportPartitionTaskEntriesContainer container; + ExportPartitionTaskEntriesContainer::index::type & by_key; + ExportPartitionTaskEntriesContainer::index::type & by_create_time; + + ExportPartitionOrderingTest() + : by_key(container.get()) + , by_create_time(container.get()) + { + } +}; + +TEST_F(ExportPartitionOrderingTest, IterationOrderMatchesCreateTime) +{ + time_t base_time = 1000; + + ExportReplicatedMergeTreePartitionManifest manifest1; + manifest1.partition_id = "2020"; + manifest1.destination_database = "db1"; + manifest1.destination_table = "table1"; + manifest1.transaction_id = "tx1"; + manifest1.create_time = base_time + 300; // Latest + + ExportReplicatedMergeTreePartitionManifest manifest2; + manifest2.partition_id = "2021"; + manifest2.destination_database = "db1"; + manifest2.destination_table = "table1"; + manifest2.transaction_id = "tx2"; + manifest2.create_time = base_time + 100; // Middle + + ExportReplicatedMergeTreePartitionManifest manifest3; + manifest3.partition_id = "2022"; + manifest3.destination_database = "db1"; + manifest3.destination_table = "table1"; + manifest3.transaction_id = "tx3"; + manifest3.create_time = base_time; // Oldest + + ExportReplicatedMergeTreePartitionTaskEntry entry1{manifest1, ExportReplicatedMergeTreePartitionTaskEntry::Status::PENDING, {}}; + ExportReplicatedMergeTreePartitionTaskEntry entry2{manifest2, ExportReplicatedMergeTreePartitionTaskEntry::Status::PENDING, {}}; + ExportReplicatedMergeTreePartitionTaskEntry entry3{manifest3, ExportReplicatedMergeTreePartitionTaskEntry::Status::PENDING, {}}; + + // Insert in reverse order + by_key.insert(entry1); + by_key.insert(entry2); + by_key.insert(entry3); + + // Verify iteration order matches create_time (ascending) + auto it = by_create_time.begin(); + ASSERT_NE(it, by_create_time.end()); + EXPECT_EQ(it->manifest.partition_id, "2022"); // Oldest first + EXPECT_EQ(it->manifest.create_time, base_time); + + ++it; + ASSERT_NE(it, by_create_time.end()); + EXPECT_EQ(it->manifest.partition_id, "2021"); + EXPECT_EQ(it->manifest.create_time, base_time + 100); + + ++it; + ASSERT_NE(it, by_create_time.end()); + EXPECT_EQ(it->manifest.partition_id, "2020"); + EXPECT_EQ(it->manifest.create_time, base_time + 300); + + ++it; + EXPECT_EQ(it, by_create_time.end()); +} + +} diff --git a/src/Storages/ObjectStorage/MultiFileStorageObjectStorageSink.cpp b/src/Storages/ObjectStorage/MultiFileStorageObjectStorageSink.cpp new file mode 100644 index 000000000000..f46985b9a52f --- /dev/null +++ b/src/Storages/ObjectStorage/MultiFileStorageObjectStorageSink.cpp @@ -0,0 +1,153 @@ +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int FILE_ALREADY_EXISTS; +} + +MultiFileStorageObjectStorageSink::MultiFileStorageObjectStorageSink( + const std::string & base_path_, + const String & transaction_id_, + ObjectStoragePtr object_storage_, + StorageObjectStorageConfigurationPtr configuration_, + std::size_t max_bytes_per_file_, + std::size_t max_rows_per_file_, + bool overwrite_if_exists_, + const std::function & new_file_path_callback_, + const std::optional & format_settings_, + SharedHeader sample_block_, + ContextPtr context_) + : SinkToStorage(sample_block_), + base_path(base_path_), + transaction_id(transaction_id_), + object_storage(object_storage_), + configuration(configuration_), + max_bytes_per_file(max_bytes_per_file_), + max_rows_per_file(max_rows_per_file_), + overwrite_if_exists(overwrite_if_exists_), + new_file_path_callback(new_file_path_callback_), + format_settings(format_settings_), + sample_block(sample_block_), + context(context_) +{ + current_sink = createNewSink(); +} + +MultiFileStorageObjectStorageSink::~MultiFileStorageObjectStorageSink() +{ + if (isCancelled()) + current_sink->cancel(); +} + +/// Adds a counter that represents file index to the file path. +/// Example: +/// Input is `table_root/year=2025/month=12/day=12/file.parquet` +/// Output is `table_root/year=2025/month=12/day=12/file.1.parquet` +std::string MultiFileStorageObjectStorageSink::generateNewFilePath() +{ + const auto file_format = Poco::toLower(configuration->format); + const auto index_string = std::to_string(file_paths.size() + 1); + std::size_t pos = base_path.rfind(file_format); + + /// normal case - path ends with the file format + if (pos != std::string::npos) + { + const auto path_without_extension = base_path.substr(0, pos); + const auto file_format_extension = "." + file_format; + + return path_without_extension + index_string + file_format_extension; + } + + /// if no extension is found, just append the index - I am not even sure this is possible + return base_path + "." + index_string; +} + +std::shared_ptr MultiFileStorageObjectStorageSink::createNewSink() +{ + auto new_path = generateNewFilePath(); + + /// todo + /// sounds like bad design, but callers might decide to ignore the exception, and if we throw it before the callback + /// they will not be able to grab the file path. + /// maybe I should consider moving the file already exists policy in here? + new_file_path_callback(new_path); + + file_paths.emplace_back(std::move(new_path)); + + if (!overwrite_if_exists && object_storage->exists(StoredObject(file_paths.back()))) + { + throw Exception(ErrorCodes::FILE_ALREADY_EXISTS, "File {} already exists", file_paths.back()); + } + + return std::make_shared( + file_paths.back(), + object_storage, + format_settings, + sample_block, + context, + configuration->format, + configuration->compression_method); +} + +void MultiFileStorageObjectStorageSink::consume(Chunk & chunk) +{ + if (isCancelled()) + { + current_sink->cancel(); + return; + } + + const auto written_bytes = current_sink->getWrittenBytes(); + + const bool exceeded_bytes_limit = max_bytes_per_file && written_bytes >= max_bytes_per_file; + const bool exceeded_rows_limit = max_rows_per_file && current_sink_written_rows >= max_rows_per_file; + + if (exceeded_bytes_limit || exceeded_rows_limit) + { + current_sink->onFinish(); + current_sink = createNewSink(); + current_sink_written_rows = 0; + } + + current_sink->consume(chunk); + current_sink_written_rows += chunk.getNumRows(); +} + +void MultiFileStorageObjectStorageSink::onFinish() +{ + current_sink->onFinish(); + commit(); +} + +void MultiFileStorageObjectStorageSink::commit() +{ + /// the commit file path should be in the same directory as the data files + const auto commit_file_path = fs::path(base_path).parent_path() / ("commit_" + transaction_id); + + if (!overwrite_if_exists && object_storage->exists(StoredObject(commit_file_path))) + { + throw Exception(ErrorCodes::FILE_ALREADY_EXISTS, "Commit file {} already exists, aborting {} export", commit_file_path, transaction_id); + } + + auto out = object_storage->writeObject( + StoredObject(commit_file_path), + WriteMode::Rewrite, /* attributes= */ + {}, DBMS_DEFAULT_BUFFER_SIZE, + context->getWriteSettings()); + + for (const auto & p : file_paths) + { + out->write(p.data(), p.size()); + out->write("\n", 1); + } + + out->finalize(); +} + +} diff --git a/src/Storages/ObjectStorage/MultiFileStorageObjectStorageSink.h b/src/Storages/ObjectStorage/MultiFileStorageObjectStorageSink.h new file mode 100644 index 000000000000..51f6b8094232 --- /dev/null +++ b/src/Storages/ObjectStorage/MultiFileStorageObjectStorageSink.h @@ -0,0 +1,57 @@ +#pragma once + +#include + +namespace DB +{ + +/// This is useful when the data is too large to fit into a single file. +/// It will create a new file when the current file exceeds the max bytes or max rows limit. +/// Ships a commit file including the list of data files to make it transactional +class MultiFileStorageObjectStorageSink : public SinkToStorage +{ +public: + MultiFileStorageObjectStorageSink( + const std::string & base_path_, + const String & transaction_id_, + ObjectStoragePtr object_storage_, + StorageObjectStorageConfigurationPtr configuration_, + std::size_t max_bytes_per_file_, + std::size_t max_rows_per_file_, + bool overwrite_if_exists_, + const std::function & new_file_path_callback_, + const std::optional & format_settings_, + SharedHeader sample_block_, + ContextPtr context_); + + ~MultiFileStorageObjectStorageSink() override; + + void consume(Chunk & chunk) override; + + void onFinish() override; + + String getName() const override { return "MultiFileStorageObjectStorageSink"; } + +private: + const std::string base_path; + const String transaction_id; + ObjectStoragePtr object_storage; + StorageObjectStorageConfigurationPtr configuration; + std::size_t max_bytes_per_file; + std::size_t max_rows_per_file; + bool overwrite_if_exists; + std::function new_file_path_callback; + const std::optional format_settings; + SharedHeader sample_block; + ContextPtr context; + + std::vector file_paths; + std::shared_ptr current_sink; + std::size_t current_sink_written_rows = 0; + + std::string generateNewFilePath(); + std::shared_ptr createNewSink(); + void commit(); +}; + +} diff --git a/src/Storages/ObjectStorage/ObjectStorageFilePathGenerator.h b/src/Storages/ObjectStorage/ObjectStorageFilePathGenerator.h new file mode 100644 index 000000000000..a1f21dc502d5 --- /dev/null +++ b/src/Storages/ObjectStorage/ObjectStorageFilePathGenerator.h @@ -0,0 +1,83 @@ +#pragma once + +#include +#include +#include +#include +#include + +namespace DB +{ + struct ObjectStorageFilePathGenerator + { + virtual ~ObjectStorageFilePathGenerator() = default; + std::string getPathForWrite(const std::string & partition_id) const { + return getPathForWrite(partition_id, ""); + } + virtual std::string getPathForWrite(const std::string & partition_id, const std::string & /* file_name_override */) const = 0; + virtual std::string getPathForRead() const = 0; + }; + + struct ObjectStorageWildcardFilePathGenerator : ObjectStorageFilePathGenerator + { + static constexpr const char * FILE_WILDCARD = "{_file}"; + explicit ObjectStorageWildcardFilePathGenerator(const std::string & raw_path_) : raw_path(raw_path_) {} + + using ObjectStorageFilePathGenerator::getPathForWrite; // Bring base class overloads into scope + std::string getPathForWrite(const std::string & partition_id, const std::string & file_name_override) const override + { + const auto partition_replaced_path = PartitionedSink::replaceWildcards(raw_path, partition_id); + const auto final_path = boost::replace_all_copy(partition_replaced_path, FILE_WILDCARD, file_name_override); + return final_path; + } + + std::string getPathForRead() const override + { + return raw_path; + } + + private: + std::string raw_path; + + }; + + struct ObjectStorageAppendFilePathGenerator : ObjectStorageFilePathGenerator + { + explicit ObjectStorageAppendFilePathGenerator( + const std::string & raw_path_, + const std::string & file_format_) + : raw_path(raw_path_), file_format(Poco::toLower(file_format_)){} + + using ObjectStorageFilePathGenerator::getPathForWrite; // Bring base class overloads into scope + std::string getPathForWrite(const std::string & partition_id, const std::string & file_name_override) const override + { + std::string result; + + result += raw_path; + + if (!result.empty() && result.back() != '/') + { + result += "/"; + } + + /// Not adding '/' because buildExpressionHive() always adds a trailing '/' + result += partition_id; + + const auto file_name = file_name_override.empty() ? std::to_string(generateSnowflakeID()) : file_name_override; + + result += file_name + "." + file_format; + + return result; + } + + std::string getPathForRead() const override + { + return raw_path + "**." + file_format; + } + + private: + std::string raw_path; + std::string file_format; + }; + +} diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 7a55d1ade4ad..89be0d011259 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -29,11 +30,13 @@ #include #include #include +#include #include #include #include #include #include +#include namespace DB @@ -53,6 +56,7 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; extern const int INCORRECT_DATA; extern const int BAD_ARGUMENTS; + extern const int FILE_ALREADY_EXISTS; } String StorageObjectStorage::getPathSample(ContextPtr context) @@ -563,7 +567,8 @@ SinkToStoragePtr StorageObjectStorage::write( if (configuration->partition_strategy) { - return std::make_shared(object_storage, configuration, format_settings, sample_block, local_context); + auto sink_creator = std::make_shared(object_storage, configuration, format_settings, sample_block, local_context); + return std::make_shared(configuration->partition_strategy, sink_creator, local_context, sample_block); } auto paths = configuration->getPaths(); @@ -596,6 +601,76 @@ bool StorageObjectStorage::optimize( return configuration->optimize(metadata_snapshot, context, format_settings); } +bool StorageObjectStorage::supportsImport() const +{ + if (!configuration->partition_strategy) + return false; + + if (configuration->partition_strategy_type == PartitionStrategyFactory::StrategyType::WILDCARD) + return configuration->getRawPath().hasExportFilenameWildcard(); + + return configuration->partition_strategy_type == PartitionStrategyFactory::StrategyType::HIVE; +} + + +SinkToStoragePtr StorageObjectStorage::import( + const std::string & file_name, + Block & block_with_partition_values, + const std::function & new_file_path_callback, + bool overwrite_if_exists, + std::size_t max_bytes_per_file, + std::size_t max_rows_per_file, + const std::optional & format_settings_, + ContextPtr local_context) +{ + std::string partition_key; + + if (configuration->partition_strategy) + { + const auto column_with_partition_key = configuration->partition_strategy->computePartitionKey(block_with_partition_values); + + if (!column_with_partition_key->empty()) + { + partition_key = column_with_partition_key->getDataAt(0); + } + } + + const auto base_path = configuration->getPathForWrite(partition_key, file_name).path; + + return std::make_shared( + base_path, + /* transaction_id= */ file_name, /// not pretty, but the sink needs some sort of id to generate the commit file name. Using the source part name should be enough + object_storage, + configuration, + max_bytes_per_file, + max_rows_per_file, + overwrite_if_exists, + new_file_path_callback, + format_settings_ ? format_settings_ : format_settings, + std::make_shared(getInMemoryMetadataPtr()->getSampleBlock()), + local_context); +} + +void StorageObjectStorage::commitExportPartitionTransaction(const String & transaction_id, const String & partition_id, const Strings & exported_paths, ContextPtr local_context) +{ + const String commit_object = configuration->getRawPath().path + "/commit_" + partition_id + "_" + transaction_id; + + /// if file already exists, nothing to be done + if (object_storage->exists(StoredObject(commit_object))) + { + LOG_DEBUG(getLogger("StorageObjectStorage"), "Commit file already exists, nothing to be done: {}", commit_object); + return; + } + + auto out = object_storage->writeObject(StoredObject(commit_object), WriteMode::Rewrite, /* attributes= */ {}, DBMS_DEFAULT_BUFFER_SIZE, local_context->getWriteSettings()); + for (const auto & p : exported_paths) + { + out->write(p.data(), p.size()); + out->write("\n", 1); + } + out->finalize(); +} + void StorageObjectStorage::truncate( const ASTPtr & /* query */, const StorageMetadataPtr & /* metadata_snapshot */, @@ -794,5 +869,4 @@ void StorageObjectStorage::checkAlterIsPossible(const AlterCommands & commands, configuration->checkAlterIsPossible(commands); } - } diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index 403bed005124..8a4600d6e075 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -7,6 +7,7 @@ #include #include #include +#include "Storages/ObjectStorage/ObjectStorageFilePathGenerator.h" #include #include #include @@ -73,6 +74,25 @@ class StorageObjectStorage : public IStorage ContextPtr context, bool async_insert) override; + + bool supportsImport() const override; + + SinkToStoragePtr import( + const std::string & /* file_name */, + Block & /* block_with_partition_values */, + const std::function & new_file_path_callback, + bool /* overwrite_if_exists */, + std::size_t /* max_bytes_per_file */, + std::size_t /* max_rows_per_file */, + const std::optional & /* format_settings_ */, + ContextPtr /* context */) override; + + void commitExportPartitionTransaction( + const String & transaction_id, + const String & partition_id, + const Strings & exported_paths, + ContextPtr local_context) override; + void truncate( const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp index 9af7bb7c6fdb..317387394e03 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -119,6 +120,11 @@ StorageObjectStorageCluster::StorageObjectStorageCluster( metadata.setConstraints(constraints_); + if (configuration->partition_strategy) + { + metadata.partition_key = configuration->partition_strategy->getPartitionKeyDescription(); + } + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage( metadata.columns, context_, diff --git a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp index fe768a7b037f..01559cedd8eb 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp @@ -148,10 +148,20 @@ void StorageObjectStorageConfiguration::initialize( else FormatFactory::instance().checkFormatName(configuration_to_initialize.format); - /// It might be changed on `StorageObjectStorageConfiguration::initPartitionStrategy` + if (configuration_to_initialize.partition_strategy_type == PartitionStrategyFactory::StrategyType::HIVE) + { + configuration_to_initialize.file_path_generator = std::make_shared( + configuration_to_initialize.getRawPath().path, + configuration_to_initialize.format); + } + else + { + configuration_to_initialize.file_path_generator = std::make_shared(configuration_to_initialize.getRawPath().path); + } + /// We shouldn't set path for disk setup because path prefix is already set in used object_storage. if (disk_name.empty()) - configuration_to_initialize.read_path = configuration_to_initialize.getRawPath(); + configuration_to_initialize.read_path = configuration_to_initialize.file_path_generator->getPathForRead(); configuration_to_initialize.initialized = true; } @@ -191,7 +201,6 @@ void StorageObjectStorageConfiguration::initPartitionStrategy(ASTPtr partition_b if (partition_strategy) { - read_path = partition_strategy->getPathForRead(getRawPath().path); LOG_DEBUG(getLogger("StorageObjectStorageConfiguration"), "Initialized partition strategy {}", magic_enum::enum_name(partition_strategy_type)); } } @@ -203,17 +212,15 @@ const StorageObjectStorageConfiguration::Path & StorageObjectStorageConfiguratio StorageObjectStorageConfiguration::Path StorageObjectStorageConfiguration::getPathForWrite(const std::string & partition_id) const { - auto raw_path = getRawPath(); + return getPathForWrite(partition_id, /* filename_override */ ""); +} +StorageObjectStorageConfiguration::Path StorageObjectStorageConfiguration::getPathForWrite(const std::string & partition_id, const std::string & filename_override) const +{ + auto path = file_path_generator->getPathForWrite(partition_id, filename_override); if (!schema_hash.empty()) - boost::replace_all(raw_path.path, SCHEMA_HASH_WILDCARD, schema_hash); - - if (!partition_strategy) - { - return raw_path; - } - - return Path {partition_strategy->getPathForWrite(raw_path.path, partition_id)}; + boost::replace_all(path, SCHEMA_HASH_WILDCARD, schema_hash); + return Path {std::move(path)}; } bool StorageObjectStorageConfiguration::Path::hasPartitionWildcard() const @@ -222,6 +229,11 @@ bool StorageObjectStorageConfiguration::Path::hasPartitionWildcard() const return path.find(PARTITION_ID_WILDCARD) != String::npos; } +bool StorageObjectStorageConfiguration::Path::hasExportFilenameWildcard() const +{ + return path.find(ObjectStorageWildcardFilePathGenerator::FILE_WILDCARD) != String::npos; +} + bool StorageObjectStorageConfiguration::Path::hasSchemaHashWildcard() const { return path.find(StorageObjectStorageConfiguration::SCHEMA_HASH_WILDCARD) != String::npos; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h index a1b95973a5c6..a8ef5fac2eda 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h @@ -17,6 +17,7 @@ #include #include #include +#include namespace DB { @@ -76,6 +77,7 @@ class StorageObjectStorageConfiguration bool hasPartitionWildcard() const; bool hasSchemaHashWildcard() const; bool hasGlobsIgnorePlaceholders() const; + bool hasExportFilenameWildcard() const; bool hasGlobs() const; std::string cutGlobs(bool supports_partial_prefix) const; }; @@ -108,8 +110,10 @@ class StorageObjectStorageConfiguration virtual const String & getRawURI() const = 0; const Path & getPathForRead() const; + // Path used for writing, it should not be globbed and might contain a partition key Path getPathForWrite(const std::string & partition_id = "") const; + Path getPathForWrite(const std::string & partition_id, const std::string & filename_override) const; void setPathForRead(const Path & path) { @@ -284,11 +288,12 @@ class StorageObjectStorageConfiguration String format = "auto"; String compression_method = "auto"; String structure = "auto"; + PartitionStrategyFactory::StrategyType partition_strategy_type = PartitionStrategyFactory::StrategyType::NONE; + std::shared_ptr partition_strategy; /// Whether partition column values are contained in the actual data. /// And alternative is with hive partitioning, when they are contained in file path. bool partition_columns_in_data_file = true; - std::shared_ptr partition_strategy; protected: void initializeFromParsedArguments(const StorageParsedArguments & parsed_arguments); @@ -308,6 +313,8 @@ class StorageObjectStorageConfiguration // Path used for reading, by default it is the same as `getRawPath` // When using `partition_strategy=hive`, a recursive reading pattern will be appended `'table_root/**.parquet' Path read_path; + + std::shared_ptr file_path_generator; }; using StorageObjectStorageConfigurationPtr = std::shared_ptr; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp index 2e4fee714f6c..4baa44e55d32 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp @@ -136,14 +136,20 @@ size_t StorageObjectStorageSink::getFileSize() const return *result_file_size; } +size_t StorageObjectStorageSink::getWrittenBytes() const +{ + if (!write_buf) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Buffer must be initialized before requesting written bytes"); + return write_buf->count(); +} + PartitionedStorageObjectStorageSink::PartitionedStorageObjectStorageSink( ObjectStoragePtr object_storage_, StorageObjectStorageConfigurationPtr configuration_, std::optional format_settings_, SharedHeader sample_block_, ContextPtr context_) - : PartitionedSink(configuration_->partition_strategy, context_, sample_block_) - , object_storage(object_storage_) + : object_storage(object_storage_) , configuration(configuration_) , query_settings(configuration_->getQuerySettings(context_)) , format_settings(format_settings_) @@ -175,10 +181,11 @@ SinkPtr PartitionedStorageObjectStorageSink::createSinkForPartition(const String file_path, object_storage, format_settings, - std::make_shared(partition_strategy->getFormatHeader()), + std::make_shared(configuration->partition_strategy->getFormatHeader()), context, configuration->format, - configuration->compression_method); + configuration->compression_method + ); } } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSink.h b/src/Storages/ObjectStorage/StorageObjectStorageSink.h index 823a6170a7e5..f2fc1fdd6d5a 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSink.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSink.h @@ -8,6 +8,8 @@ namespace DB { class StorageObjectStorageSink : public SinkToStorage { +friend class StorageObjectStorageImporterSink; + public: StorageObjectStorageSink( const std::string & path_, @@ -28,6 +30,8 @@ class StorageObjectStorageSink : public SinkToStorage const String & getPath() const { return path; } + size_t getWrittenBytes() const; + size_t getFileSize() const; private: @@ -42,7 +46,7 @@ class StorageObjectStorageSink : public SinkToStorage void cancelBuffers(); }; -class PartitionedStorageObjectStorageSink : public PartitionedSink +class PartitionedStorageObjectStorageSink : public PartitionedSink::SinkCreator { public: PartitionedStorageObjectStorageSink( diff --git a/src/Storages/PartitionCommands.cpp b/src/Storages/PartitionCommands.cpp index 593f4d895aad..c4d038f925c1 100644 --- a/src/Storages/PartitionCommands.cpp +++ b/src/Storages/PartitionCommands.cpp @@ -131,6 +131,37 @@ std::optional PartitionCommand::parse(const ASTAlterCommand * res.with_name = command_ast->with_name; return res; } + if (command_ast->type == ASTAlterCommand::EXPORT_PART) + { + PartitionCommand res; + res.type = EXPORT_PART; + res.partition = command_ast->partition->clone(); + res.part = command_ast->part; + res.to_database = command_ast->to_database; + res.to_table = command_ast->to_table; + if (command_ast->to_table_function) + { + res.to_table_function = command_ast->to_table_function->ptr(); + if (command_ast->partition_by_expr) + res.partition_by_expr = command_ast->partition_by_expr->clone(); + } + return res; + } + if (command_ast->type == ASTAlterCommand::EXPORT_PARTITION) + { + PartitionCommand res; + res.type = EXPORT_PARTITION; + res.partition = command_ast->partition->clone(); + res.to_database = command_ast->to_database; + res.to_table = command_ast->to_table; + if (command_ast->to_table_function) + { + res.to_table_function = command_ast->to_table_function->ptr(); + if (command_ast->partition_by_expr) + res.partition_by_expr = command_ast->partition_by_expr->clone(); + } + return res; + } return {}; } @@ -172,6 +203,10 @@ std::string PartitionCommand::typeToString() const return "UNFREEZE ALL"; case PartitionCommand::Type::REPLACE_PARTITION: return "REPLACE PARTITION"; + case PartitionCommand::Type::EXPORT_PART: + return "EXPORT PART"; + case PartitionCommand::Type::EXPORT_PARTITION: + return "EXPORT PARTITION"; default: throw Exception(ErrorCodes::LOGICAL_ERROR, "Uninitialized partition command"); } diff --git a/src/Storages/PartitionCommands.h b/src/Storages/PartitionCommands.h index 5c2b10097a83..ea5956aad3a0 100644 --- a/src/Storages/PartitionCommands.h +++ b/src/Storages/PartitionCommands.h @@ -33,6 +33,8 @@ struct PartitionCommand UNFREEZE_ALL_PARTITIONS, UNFREEZE_PARTITION, REPLACE_PARTITION, + EXPORT_PART, + EXPORT_PARTITION, }; Type type = UNKNOWN; @@ -50,10 +52,14 @@ struct PartitionCommand String from_table; bool replace = true; - /// For MOVE PARTITION + /// For MOVE PARTITION and EXPORT PART and EXPORT PARTITION String to_database; String to_table; + /// For EXPORT PART and EXPORT PARTITION with table functions + ASTPtr to_table_function; + ASTPtr partition_by_expr; + /// For FETCH PARTITION - path in ZK to the shard, from which to download the partition. String from_path; diff --git a/src/Storages/PartitionedSink.cpp b/src/Storages/PartitionedSink.cpp index 61789c3db19a..2be1ba5b93fc 100644 --- a/src/Storages/PartitionedSink.cpp +++ b/src/Storages/PartitionedSink.cpp @@ -26,10 +26,12 @@ namespace ErrorCodes PartitionedSink::PartitionedSink( std::shared_ptr partition_strategy_, + std::shared_ptr sink_creator_, ContextPtr context_, SharedHeader source_header_) : SinkToStorage(source_header_) , partition_strategy(partition_strategy_) + , sink_creator(sink_creator_) , context(context_) , source_header(source_header_) { @@ -41,7 +43,7 @@ SinkPtr PartitionedSink::getSinkForPartitionKey(std::string_view partition_key) auto it = partition_id_to_sink.find(partition_key); if (it == partition_id_to_sink.end()) { - auto sink = createSinkForPartition(std::string{partition_key}); + auto sink = sink_creator->createSinkForPartition(std::string{partition_key}); std::tie(it, std::ignore) = partition_id_to_sink.emplace(partition_key, sink); } diff --git a/src/Storages/PartitionedSink.h b/src/Storages/PartitionedSink.h index 3aa9cf6db16a..ee953296d068 100644 --- a/src/Storages/PartitionedSink.h +++ b/src/Storages/PartitionedSink.h @@ -17,10 +17,17 @@ namespace DB class PartitionedSink : public SinkToStorage { public: + struct SinkCreator + { + virtual ~SinkCreator() = default; + virtual SinkPtr createSinkForPartition(const String & partition_id) = 0; + }; + static constexpr auto PARTITION_ID_WILDCARD = "{_partition_id}"; PartitionedSink( std::shared_ptr partition_strategy_, + std::shared_ptr sink_creator_, ContextPtr context_, SharedHeader source_header_); @@ -34,16 +41,15 @@ class PartitionedSink : public SinkToStorage void onFinish() override; - virtual SinkPtr createSinkForPartition(const String & partition_id) = 0; - static void validatePartitionKey(const String & str, bool allow_slash); static String replaceWildcards(const String & haystack, const String & partition_id); + protected: std::shared_ptr partition_strategy; - private: + std::shared_ptr sink_creator; ContextPtr context; SharedHeader source_header; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index bdd7f5d248ed..f5e9b285e113 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -2088,7 +2088,7 @@ class StorageFileSink final : public SinkToStorage, WithContext std::unique_lock lock; }; -class PartitionedStorageFileSink : public PartitionedSink +class PartitionedStorageFileSink : public PartitionedSink::SinkCreator { public: PartitionedStorageFileSink( @@ -2103,7 +2103,7 @@ class PartitionedStorageFileSink : public PartitionedSink const String format_name_, ContextPtr context_, int flags_) - : PartitionedSink(partition_strategy_, context_, std::make_shared(metadata_snapshot_->getSampleBlock())) + : partition_strategy(partition_strategy_) , path(path_) , metadata_snapshot(metadata_snapshot_) , table_name_for_log(table_name_for_log_) @@ -2119,11 +2119,12 @@ class PartitionedStorageFileSink : public PartitionedSink SinkPtr createSinkForPartition(const String & partition_id) override { - std::string filepath = partition_strategy->getPathForWrite(path, partition_id); + const auto file_path_generator = std::make_shared(path); + std::string filepath = file_path_generator->getPathForWrite(partition_id); fs::create_directories(fs::path(filepath).parent_path()); - validatePartitionKey(filepath, true); + PartitionedSink::validatePartitionKey(filepath, true); checkCreationIsAllowed(context, context->getUserFilesPath(), filepath, /*can_be_directory=*/ true); return std::make_shared( metadata_snapshot, @@ -2140,6 +2141,7 @@ class PartitionedStorageFileSink : public PartitionedSink } private: + std::shared_ptr partition_strategy; const String path; StorageMetadataPtr metadata_snapshot; String table_name_for_log; @@ -2191,7 +2193,7 @@ SinkToStoragePtr StorageFile::write( has_wildcards, /* partition_columns_in_data_file */true); - return std::make_shared( + auto sink_creator = std::make_shared( partition_strategy, metadata_snapshot, getStorageID().getNameForLogs(), @@ -2203,6 +2205,13 @@ SinkToStoragePtr StorageFile::write( format_name, context, flags); + + return std::make_shared( + partition_strategy, + sink_creator, + context, + std::make_shared(metadata_snapshot->getSampleBlock()) + ); } String path; @@ -2228,6 +2237,7 @@ SinkToStoragePtr StorageFile::write( String new_path; do { + new_path = path.substr(0, pos) + "." + std::to_string(index) + (pos == std::string::npos ? "" : path.substr(pos)); ++index; } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 044057830d19..6b2885c5a25f 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -133,6 +133,7 @@ namespace ErrorCodes extern const int TOO_MANY_PARTS; extern const int PART_IS_LOCKED; extern const int PART_IS_TEMPORARILY_LOCKED; + extern const int INCOMPATIBLE_COLUMNS; } namespace ActionLocks @@ -232,7 +233,7 @@ void StorageMergeTree::startup() { cleanup_thread.start(); background_operations_assignee.start(); - startBackgroundMovesIfNeeded(); + startBackgroundMoves(); startOutdatedAndUnexpectedDataPartsLoadingTask(); startStatisticsCache(); } @@ -296,6 +297,11 @@ void StorageMergeTree::shutdown(bool) if (deduplication_log) deduplication_log->shutdown(); + + { + std::lock_guard lock(export_manifests_mutex); + export_manifests.clear(); + } } @@ -3069,12 +3075,6 @@ MutationCounters StorageMergeTree::getMutationCounters() const return mutation_counters; } -void StorageMergeTree::startBackgroundMovesIfNeeded() -{ - if (areBackgroundMovesNeeded()) - background_moves_assignee.start(); -} - std::unique_ptr StorageMergeTree::getDefaultSettings() const { return std::make_unique(getContext()->getMergeTreeSettings()); diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 2c01687618d1..f1882a1a7092 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -292,8 +292,6 @@ class StorageMergeTree final : public MergeTreeData std::unique_ptr fillNewPartName(MutableDataPartPtr & part, DataPartsLock & lock); std::unique_ptr fillNewPartNameAndResetLevel(MutableDataPartPtr & part, DataPartsLock & lock); - void startBackgroundMovesIfNeeded() override; - BackupEntries backupMutations(UInt64 version, const String & data_path_in_backup) const; /// Attaches restored parts to the storage. diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 290fed301ff6..bdbe0238e76e 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -7,6 +7,7 @@ #include #include +#include "Common/ZooKeeper/IKeeper.h" #include #include #include @@ -25,6 +26,7 @@ #include #include +#include #include #include @@ -68,6 +70,7 @@ #include #include #include +#include #include #include #include @@ -119,6 +122,13 @@ #include #include +#include "Functions/generateSnowflakeID.h" +#include "Interpreters/StorageID.h" +#include "QueryPipeline/QueryPlanResourceHolder.h" +#include "Storages/ExportReplicatedMergeTreePartitionManifest.h" +#include "Storages/ExportReplicatedMergeTreePartitionTaskEntry.h" +#include +#include #include #include @@ -186,6 +196,18 @@ namespace Setting extern const SettingsInt64 replication_wait_for_inactive_replica_timeout; extern const SettingsUInt64 select_sequential_consistency; extern const SettingsBool update_sequential_consistency; + extern const SettingsBool allow_experimental_export_merge_tree_part; + extern const SettingsBool export_merge_tree_partition_force_export; + extern const SettingsUInt64 export_merge_tree_partition_max_retries; + extern const SettingsUInt64 export_merge_tree_partition_manifest_ttl; + extern const SettingsBool output_format_parallel_formatting; + extern const SettingsBool output_format_parquet_parallel_encoding; + extern const SettingsMaxThreads max_threads; + extern const SettingsMergeTreePartExportFileAlreadyExistsPolicy export_merge_tree_part_file_already_exists_policy; + extern const SettingsUInt64 export_merge_tree_part_max_bytes_per_file; + extern const SettingsUInt64 export_merge_tree_part_max_rows_per_file; + extern const SettingsBool export_merge_tree_part_throw_on_pending_mutations; + extern const SettingsBool export_merge_tree_part_throw_on_pending_patch_parts; } @@ -295,6 +317,13 @@ namespace ErrorCodes extern const int FAULT_INJECTED; extern const int CANNOT_FORGET_PARTITION; extern const int TIMEOUT_EXCEEDED; + extern const int INVALID_SETTING_VALUE; + extern const int PENDING_MUTATIONS_NOT_ALLOWED; +} + +namespace ServerSetting +{ + extern const ServerSettingsBool enable_experimental_export_merge_tree_partition_feature; } namespace ActionLocks @@ -423,6 +452,9 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( , merge_strategy_picker(*this) , queue(*this, merge_strategy_picker) , fetcher(*this) + , export_merge_tree_partition_task_entries_by_key(export_merge_tree_partition_task_entries.get()) + , export_merge_tree_partition_task_entries_by_transaction_id(export_merge_tree_partition_task_entries.get()) + , export_merge_tree_partition_task_entries_by_create_time(export_merge_tree_partition_task_entries.get()) , cleanup_thread(*this) , deduplication_hashes_cache(*this, "deduplication_hashes") , async_block_ids_cache(*this, "async_blocks") @@ -468,6 +500,31 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( /// Will be activated by restarting thread. mutations_finalizing_task->deactivate(); + if (getContext()->getServerSettings()[ServerSetting::enable_experimental_export_merge_tree_partition_feature]) + { + export_merge_tree_partition_manifest_updater = std::make_shared(*this); + + export_merge_tree_partition_task_scheduler = std::make_shared(*this); + + export_merge_tree_partition_updating_task = getContext()->getSchedulePool().createTask( + getStorageID(), getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::export_merge_tree_partition_updating_task)", [this] { exportMergeTreePartitionUpdatingTask(); }); + + export_merge_tree_partition_updating_task->deactivate(); + + export_merge_tree_partition_status_handling_task = getContext()->getSchedulePool().createTask( + getStorageID(), getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::export_merge_tree_partition_status_handling_task)", [this] { exportMergeTreePartitionStatusHandlingTask(); }); + + export_merge_tree_partition_status_handling_task->deactivate(); + + export_merge_tree_partition_watch_callback = export_merge_tree_partition_updating_task->getWatchCallback(); + + export_merge_tree_partition_select_task = getContext()->getSchedulePool().createTask( + getStorageID(), getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::export_merge_tree_partition_select_task)", [this] { selectPartsToExport(); }); + + export_merge_tree_partition_select_task->deactivate(); + } + + bool has_zookeeper = getContext()->hasZooKeeper() || getContext()->hasAuxiliaryZooKeeper(zookeeper_info.zookeeper_name); auto component_guard = Coordination::setCurrentComponent("StorageReplicatedMergeTree::StorageReplicatedMergeTree"); if (has_zookeeper) @@ -903,6 +960,7 @@ void StorageReplicatedMergeTree::createNewZooKeeperNodesAttempt() const futures.push_back(zookeeper->asyncTryCreateNoThrow(zookeeper_path + "/quorum/last_part", String(), zkutil::CreateMode::Persistent)); futures.push_back(zookeeper->asyncTryCreateNoThrow(zookeeper_path + "/quorum/failed_parts", String(), zkutil::CreateMode::Persistent)); futures.push_back(zookeeper->asyncTryCreateNoThrow(zookeeper_path + "/mutations", String(), zkutil::CreateMode::Persistent)); + futures.push_back(zookeeper->asyncTryCreateNoThrow(zookeeper_path + "/exports", String(), zkutil::CreateMode::Persistent)); futures.push_back(zookeeper->asyncTryCreateNoThrow(zookeeper_path + "/quorum/parallel", String(), zkutil::CreateMode::Persistent)); @@ -1069,6 +1127,8 @@ bool StorageReplicatedMergeTree::createTableIfNotExistsAttempt(const StorageMeta zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/mutations", "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/exports", "", + zkutil::CreateMode::Persistent)); /// And create first replica atomically. See also "createReplica" method that is used to create not the first replicas. @@ -4491,6 +4551,160 @@ void StorageReplicatedMergeTree::mutationsFinalizingTask() } } +void StorageReplicatedMergeTree::exportMergeTreePartitionUpdatingTask() +{ + try + { + export_merge_tree_partition_manifest_updater->poll(); + } + catch (...) + { + tryLogCurrentException(log, __PRETTY_FUNCTION__); + } + + + export_merge_tree_partition_updating_task->scheduleAfter(30 * 1000); +} + +void StorageReplicatedMergeTree::selectPartsToExport() +{ + try + { + export_merge_tree_partition_task_scheduler->run(); + } + catch (...) + { + tryLogCurrentException(log, __PRETTY_FUNCTION__); + } + + export_merge_tree_partition_select_task->scheduleAfter(1000 * 5); +} + +void StorageReplicatedMergeTree::exportMergeTreePartitionStatusHandlingTask() +{ + try + { + export_merge_tree_partition_manifest_updater->handleStatusChanges(); + } + catch (...) + { + tryLogCurrentException(log, __PRETTY_FUNCTION__); + } +} + +std::vector StorageReplicatedMergeTree::getPartitionExportsInfo() const +{ + std::vector infos; + + const auto zk = getZooKeeper(); + const auto exports_path = fs::path(zookeeper_path) / "exports"; + std::vector children; + if (Coordination::Error::ZOK != zk->tryGetChildren(exports_path, children)) + { + LOG_INFO(log, "Failed to get children from exports path, returning empty export info list"); + return infos; + } + + for (const auto & child : children) + { + ReplicatedPartitionExportInfo info; + + const auto export_partition_path = fs::path(exports_path) / child; + std::string metadata_json; + if (!zk->tryGet(export_partition_path / "metadata.json", metadata_json)) + { + LOG_INFO(log, "Skipping {}: missing metadata.json", child); + continue; + } + + std::string status; + if (!zk->tryGet(export_partition_path / "status", status)) + { + LOG_INFO(log, "Skipping {}: missing status", child); + continue; + } + + std::vector processing_parts; + if (Coordination::Error::ZOK != zk->tryGetChildren(export_partition_path / "processing", processing_parts)) + { + LOG_INFO(log, "Skipping {}: missing processing parts", child); + continue; + } + + const auto parts_to_do = processing_parts.size(); + + std::string exception_replica; + std::string last_exception; + std::string exception_part; + std::size_t exception_count = 0; + + const auto exceptions_per_replica_path = export_partition_path / "exceptions_per_replica"; + + Strings exception_replicas; + if (Coordination::Error::ZOK != zk->tryGetChildren(exceptions_per_replica_path, exception_replicas)) + { + LOG_INFO(log, "Skipping {}: missing exceptions_per_replica", export_partition_path); + continue; + } + + for (const auto & replica : exception_replicas) + { + std::string exception_count_string; + if (!zk->tryGet(exceptions_per_replica_path / replica / "count", exception_count_string)) + { + LOG_INFO(log, "Skipping {}: missing count", replica); + continue; + } + + exception_count += std::stoull(exception_count_string.c_str()); + + if (last_exception.empty()) + { + const auto last_exception_path = exceptions_per_replica_path / replica / "last_exception"; + std::string last_exception_string; + if (!zk->tryGet(last_exception_path / "exception", last_exception_string)) + { + LOG_INFO(log, "Skipping {}: missing last_exception/exception", last_exception_path); + continue; + } + + std::string exception_part_zk; + if (!zk->tryGet(last_exception_path / "part", exception_part_zk)) + { + LOG_INFO(log, "Skipping {}: missing exception part", last_exception_path); + continue; + } + + exception_replica = replica; + last_exception = last_exception_string; + exception_part = exception_part_zk; + } + } + + const auto metadata = ExportReplicatedMergeTreePartitionManifest::fromJsonString(metadata_json); + + info.destination_database = metadata.destination_database; + info.destination_table = metadata.destination_table; + info.partition_id = metadata.partition_id; + info.transaction_id = metadata.transaction_id; + info.query_id = metadata.query_id; + info.create_time = metadata.create_time; + info.source_replica = metadata.source_replica; + info.parts_count = metadata.number_of_parts; + info.parts_to_do = parts_to_do; + info.parts = metadata.parts; + info.status = status; + info.exception_replica = exception_replica; + info.last_exception = last_exception; + info.exception_part = exception_part; + info.exception_count = exception_count; + + infos.emplace_back(std::move(info)); + } + + return infos; +} + StorageReplicatedMergeTree::CreateMergeEntryResult StorageReplicatedMergeTree::createLogEntryToMergeParts( zkutil::ZooKeeperPtr & zookeeper, @@ -5787,7 +6001,7 @@ void StorageReplicatedMergeTree::startupImpl(bool from_attach_thread, const ZooK restarting_thread.start(true); }); - startBackgroundMovesIfNeeded(); + startBackgroundMoves(); part_moves_between_shards_orchestrator.start(); @@ -5886,6 +6100,13 @@ void StorageReplicatedMergeTree::partialShutdown() mutations_updating_task->deactivate(); mutations_finalizing_task->deactivate(); + if (getContext()->getServerSettings()[ServerSetting::enable_experimental_export_merge_tree_partition_feature]) + { + export_merge_tree_partition_updating_task->deactivate(); + export_merge_tree_partition_select_task->deactivate(); + export_merge_tree_partition_status_handling_task->deactivate(); + } + cleanup_thread.stop(); deduplication_hashes_cache.stop(); async_block_ids_cache.stop(); @@ -5958,6 +6179,17 @@ void StorageReplicatedMergeTree::shutdown(bool) /// Wait for all of them std::lock_guard lock(data_parts_exchange_ptr->rwlock); } + + { + std::lock_guard lock(export_merge_tree_partition_mutex); + export_merge_tree_partition_task_entries.clear(); + } + + { + std::lock_guard lock(export_manifests_mutex); + export_manifests.clear(); + } + LOG_TRACE(log, "Shutdown finished"); } @@ -8098,6 +8330,221 @@ void StorageReplicatedMergeTree::fetchPartition( LOG_TRACE(log, "Fetch took {} sec. ({} tries)", watch.elapsedSeconds(), try_no); } +void StorageReplicatedMergeTree::exportPartitionToTable(const PartitionCommand & command, ContextPtr query_context) +{ + if (!query_context->getServerSettings()[ServerSetting::enable_experimental_export_merge_tree_partition_feature]) + { + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "Exporting merge tree partition is experimental. Set the server setting `enable_experimental_export_merge_tree_partition_feature` to enable it"); + } + + const auto dest_database = query_context->resolveDatabase(command.to_database); + const auto dest_table = command.to_table; + const auto dest_storage_id = StorageID(dest_database, dest_table); + auto dest_storage = DatabaseCatalog::instance().getTable({dest_database, dest_table}, query_context); + + if (dest_storage->getStorageID() == this->getStorageID()) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Exporting to the same table is not allowed"); + } + + if (!dest_storage->supportsImport()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Destination storage {} does not support MergeTree parts or uses unsupported partitioning", dest_storage->getName()); + + auto query_to_string = [] (const ASTPtr & ast) + { + return ast ? ast->formatWithSecretsOneLine() : ""; + }; + + auto src_snapshot = getInMemoryMetadataPtr(); + auto destination_snapshot = dest_storage->getInMemoryMetadataPtr(); + + /// compare all source readable columns with all destination insertable columns + /// this allows us to skip ephemeral columns + if (src_snapshot->getColumns().getReadable().sizeOfDifference(destination_snapshot->getColumns().getInsertable())) + throw Exception(ErrorCodes::INCOMPATIBLE_COLUMNS, "Tables have different structure"); + + if (query_to_string(src_snapshot->getPartitionKeyAST()) != query_to_string(destination_snapshot->getPartitionKeyAST())) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Tables have different partition key"); + + zkutil::ZooKeeperPtr zookeeper = getZooKeeperAndAssertNotReadonly(); + + const String partition_id = getPartitionIDFromQuery(command.partition, query_context); + + const auto exports_path = fs::path(zookeeper_path) / "exports"; + + const auto export_key = partition_id + "_" + dest_storage_id.getQualifiedName().getFullName(); + + const auto partition_exports_path = fs::path(exports_path) / export_key; + + /// check if entry already exists + if (zookeeper->exists(partition_exports_path)) + { + LOG_INFO(log, "Export with key {} is already exported or it is being exported. Checking if it has expired so that we can overwrite it", export_key); + + bool has_expired = false; + + if (zookeeper->exists(fs::path(partition_exports_path) / "metadata.json")) + { + std::string metadata_json; + if (zookeeper->tryGet(fs::path(partition_exports_path) / "metadata.json", metadata_json)) + { + const auto manifest = ExportReplicatedMergeTreePartitionManifest::fromJsonString(metadata_json); + + const auto now = time(nullptr); + const auto expiration_time = manifest.create_time + manifest.ttl_seconds; + + LOG_INFO(log, "Export with key {} has expiration time {}, now is {}", export_key, expiration_time, now); + + if (static_cast(expiration_time) < now) + { + has_expired = true; + } + } + } + + if (!has_expired && !query_context->getSettingsRef()[Setting::export_merge_tree_partition_force_export]) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Export with key {} already exported or it is being exported, and it has not expired. Set `export_merge_tree_partition_force_export` to overwrite it.", export_key); + } + + LOG_INFO(log, "Overwriting export with key {}", export_key); + + /// Not putting in ops (same transaction) because we can't construct a "tryRemoveRecursive" request. + /// It is possible that the zk being used does not support RemoveRecursive requests. + /// It is ok for this to be non transactional. Worst case scenario an on-going export is going to be killed and a new task won't be scheduled. + zookeeper->tryRemoveRecursive(partition_exports_path); + } + + Coordination::Requests ops; + + ops.emplace_back(zkutil::makeCreateRequest(partition_exports_path, "", zkutil::CreateMode::Persistent)); + + DataPartsVector parts; + + { + auto data_parts_lock = lockParts(); + parts = getDataPartsVectorInPartitionForInternalUsage(MergeTreeDataPartState::Active, partition_id, data_parts_lock); + } + + if (parts.empty()) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Partition {} doesn't exist", partition_id); + } + + const bool throw_on_pending_mutations = query_context->getSettingsRef()[Setting::export_merge_tree_part_throw_on_pending_mutations]; + const bool throw_on_pending_patch_parts = query_context->getSettingsRef()[Setting::export_merge_tree_part_throw_on_pending_patch_parts]; + + MergeTreeData::IMutationsSnapshot::Params mutations_snapshot_params + { + .metadata_version = getInMemoryMetadataPtr()->getMetadataVersion(), + .min_part_metadata_version = MergeTreeData::getMinMetadataVersion(parts), + .need_data_mutations = throw_on_pending_mutations, + .need_alter_mutations = throw_on_pending_mutations || throw_on_pending_patch_parts, + .need_patch_parts = throw_on_pending_patch_parts, + }; + + const auto mutations_snapshot = getMutationsSnapshot(mutations_snapshot_params); + + std::vector part_names; + for (const auto & part : parts) + { + const auto alter_conversions = getAlterConversionsForPart(part, mutations_snapshot, query_context); + + /// re-check `throw_on_pending_mutations` because `pending_mutations` might have been filled due to `throw_on_pending_patch_parts` + if (alter_conversions->hasMutations() && throw_on_pending_mutations) + { + throw Exception(ErrorCodes::PENDING_MUTATIONS_NOT_ALLOWED, + "Partition {} can not be exported because the part {} has pending mutations. Either wait for the mutations to be applied or set `export_merge_tree_part_throw_on_pending_mutations` to false", + partition_id, + part->name); + } + + if (alter_conversions->hasPatches()) + { + throw Exception(ErrorCodes::PENDING_MUTATIONS_NOT_ALLOWED, + "Partition {} can not be exported because the part {} has pending patch parts. Either wait for the patch parts to be applied or set `export_merge_tree_part_throw_on_pending_patch_parts` to false", + partition_id, + part->name); + } + + part_names.push_back(part->name); + } + + /// TODO arthur somehow check if the list of parts is updated "enough" + + ExportReplicatedMergeTreePartitionManifest manifest; + + manifest.transaction_id = generateSnowflakeIDString(); + manifest.query_id = query_context->getCurrentQueryId(); + manifest.partition_id = partition_id; + manifest.destination_database = dest_database; + manifest.destination_table = dest_table; + manifest.source_replica = replica_name; + manifest.number_of_parts = part_names.size(); + manifest.parts = part_names; + manifest.create_time = time(nullptr); + manifest.max_retries = query_context->getSettingsRef()[Setting::export_merge_tree_partition_max_retries]; + manifest.ttl_seconds = query_context->getSettingsRef()[Setting::export_merge_tree_partition_manifest_ttl]; + manifest.max_threads = query_context->getSettingsRef()[Setting::max_threads]; + manifest.parallel_formatting = query_context->getSettingsRef()[Setting::output_format_parallel_formatting]; + manifest.parquet_parallel_encoding = query_context->getSettingsRef()[Setting::output_format_parquet_parallel_encoding]; + manifest.max_bytes_per_file = query_context->getSettingsRef()[Setting::export_merge_tree_part_max_bytes_per_file]; + manifest.max_rows_per_file = query_context->getSettingsRef()[Setting::export_merge_tree_part_max_rows_per_file]; + + manifest.file_already_exists_policy = query_context->getSettingsRef()[Setting::export_merge_tree_part_file_already_exists_policy].value; + + ops.emplace_back(zkutil::makeCreateRequest( + fs::path(partition_exports_path) / "metadata.json", + manifest.toJsonString(), + zkutil::CreateMode::Persistent)); + + ops.emplace_back(zkutil::makeCreateRequest( + fs::path(partition_exports_path) / "exceptions_per_replica", + "", + zkutil::CreateMode::Persistent)); + + ops.emplace_back(zkutil::makeCreateRequest( + fs::path(partition_exports_path) / "processing", + "", + zkutil::CreateMode::Persistent)); + + for (const auto & part : part_names) + { + ExportReplicatedMergeTreePartitionProcessingPartEntry entry; + entry.status = ExportReplicatedMergeTreePartitionProcessingPartEntry::Status::PENDING; + entry.part_name = part; + entry.retry_count = 0; + + ops.emplace_back(zkutil::makeCreateRequest( + fs::path(partition_exports_path) / "processing" / part, + entry.toJsonString(), + zkutil::CreateMode::Persistent)); + } + + ops.emplace_back(zkutil::makeCreateRequest( + fs::path(partition_exports_path) / "processed", + "", + zkutil::CreateMode::Persistent)); + + ops.emplace_back(zkutil::makeCreateRequest( + fs::path(partition_exports_path) / "locks", + "", + zkutil::CreateMode::Persistent)); + + /// status: IN_PROGRESS, COMPLETED, FAILED + ops.emplace_back(zkutil::makeCreateRequest( + fs::path(partition_exports_path) / "status", + "PENDING", + zkutil::CreateMode::Persistent)); + + Coordination::Responses responses; + Coordination::Error code = zookeeper->tryMulti(ops, responses); + + if (code != Coordination::Error::ZOK) + throw zkutil::KeeperException::fromPath(code, partition_exports_path); +} + void StorageReplicatedMergeTree::forgetPartition(const ASTPtr & partition, ContextPtr query_context) { @@ -9541,6 +9988,89 @@ CancellationCode StorageReplicatedMergeTree::killPartMoveToShard(const UUID & ta return part_moves_between_shards_orchestrator.killPartMoveToShard(task_uuid); } +CancellationCode StorageReplicatedMergeTree::killExportPartition(const String & transaction_id) +{ + auto try_set_status_to_killed = [this](const zkutil::ZooKeeperPtr & zk, const std::string & status_path) + { + Coordination::Stat stat; + std::string status_from_zk_string; + + if (!zk->tryGet(status_path, status_from_zk_string, &stat)) + { + /// found entry locally, but not in zk. It might have been deleted by another replica and we did not have time to update the local entry. + LOG_INFO(log, "Export partition task not found in zk, can not cancel it"); + return CancellationCode::CancelCannotBeSent; + } + + const auto status_from_zk = magic_enum::enum_cast(status_from_zk_string); + + if (!status_from_zk) + { + LOG_INFO(log, "Export partition task status is invalid, can not cancel it"); + return CancellationCode::CancelCannotBeSent; + } + + if (status_from_zk.value() != ExportReplicatedMergeTreePartitionTaskEntry::Status::PENDING) + { + LOG_INFO(log, "Export partition task is {}, can not cancel it", String(magic_enum::enum_name(status_from_zk.value()))); + return CancellationCode::CancelCannotBeSent; + } + + if (zk->trySet(status_path, String(magic_enum::enum_name(ExportReplicatedMergeTreePartitionTaskEntry::Status::KILLED)), stat.version) != Coordination::Error::ZOK) + { + LOG_INFO(log, "Status has been updated while trying to kill the export partition task, can not cancel it"); + return CancellationCode::CancelCannotBeSent; + } + + return CancellationCode::CancelSent; + }; + + std::lock_guard lock(export_merge_tree_partition_mutex); + + const auto zk = getZooKeeper(); + + /// if we have the entry locally, no need to list from zk. we can save some requests. + const auto & entry = export_merge_tree_partition_task_entries_by_transaction_id.find(transaction_id); + if (entry != export_merge_tree_partition_task_entries_by_transaction_id.end()) + { + LOG_INFO(log, "Export partition task found locally, trying to cancel it"); + /// found locally, no need to get children on zk + if (entry->status != ExportReplicatedMergeTreePartitionTaskEntry::Status::PENDING) + { + LOG_INFO(log, "Export partition task is not pending, can not cancel it"); + return CancellationCode::CancelCannotBeSent; + } + + return try_set_status_to_killed(zk, fs::path(zookeeper_path) / "exports" / entry->getCompositeKey() / "status"); + } + else + { + LOG_INFO(log, "Export partition task not found locally, trying to find it on zk"); + /// for some reason, we don't have the entry locally. ls on zk to find the entry + const auto exports_path = fs::path(zookeeper_path) / "exports"; + + const auto export_keys = zk->getChildren(exports_path); + String export_key_to_be_cancelled; + + for (const auto & export_key : export_keys) + { + std::string metadata_json; + if (!zk->tryGet(fs::path(exports_path) / export_key / "metadata.json", metadata_json)) + continue; + const auto manifest = ExportReplicatedMergeTreePartitionManifest::fromJsonString(metadata_json); + if (manifest.transaction_id == transaction_id) + { + LOG_INFO(log, "Export partition task found on zk, trying to cancel it"); + return try_set_status_to_killed(zk, fs::path(exports_path) / export_key / "status"); + } + } + } + + LOG_INFO(log, "Export partition task not found, can not cancel it"); + + return CancellationCode::NotFound; +} + void StorageReplicatedMergeTree::getCommitPartOps( Coordination::Requests & ops, const DataPartPtr & part, @@ -10101,13 +10631,6 @@ MutationCounters StorageReplicatedMergeTree::getMutationCounters() const return queue.getMutationCounters(); } -void StorageReplicatedMergeTree::startBackgroundMovesIfNeeded() -{ - if (areBackgroundMovesNeeded()) - background_moves_assignee.start(); -} - - std::unique_ptr StorageReplicatedMergeTree::getDefaultSettings() const { return std::make_unique(getContext()->getReplicatedMergeTreeSettings()); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index ee2913b72691..bc9eced6c609 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -1,6 +1,10 @@ #pragma once #include +#include +#include +#include +#include #include #include #include @@ -96,6 +100,8 @@ namespace DB class ZooKeeperWithFaultInjection; using ZooKeeperWithFaultInjectionPtr = std::shared_ptr; +struct ReplicatedPartitionExportInfo; + class StorageReplicatedMergeTree final : public MergeTreeData { public: @@ -371,6 +377,8 @@ class StorageReplicatedMergeTree final : public MergeTreeData using ShutdownDeadline = std::chrono::time_point; void waitForUniquePartsToBeFetchedByOtherReplicas(ShutdownDeadline shutdown_deadline); + std::vector getPartitionExportsInfo() const; + private: std::atomic_bool are_restoring_replica {false}; @@ -395,6 +403,8 @@ class StorageReplicatedMergeTree final : public MergeTreeData friend class MergeFromLogEntryTask; friend class MutateFromLogEntryTask; friend class ReplicatedMergeMutateTaskBase; + friend class ExportPartitionManifestUpdatingTask; + friend class ExportPartitionTaskScheduler; using MergeStrategyPicker = ReplicatedMergeTreeMergeStrategyPicker; using LogEntry = ReplicatedMergeTreeLogEntry; @@ -507,6 +517,26 @@ class StorageReplicatedMergeTree final : public MergeTreeData /// A task that marks finished mutations as done. BackgroundSchedulePoolTaskHolder mutations_finalizing_task; + BackgroundSchedulePoolTaskHolder export_merge_tree_partition_updating_task; + + /// mostly handle kill operations + BackgroundSchedulePoolTaskHolder export_merge_tree_partition_status_handling_task; + std::shared_ptr export_merge_tree_partition_manifest_updater; + + std::shared_ptr export_merge_tree_partition_task_scheduler; + + Coordination::WatchCallbackPtr export_merge_tree_partition_watch_callback; + + std::mutex export_merge_tree_partition_mutex; + + BackgroundSchedulePoolTaskHolder export_merge_tree_partition_select_task; + + ExportPartitionTaskEntriesContainer export_merge_tree_partition_task_entries; + + // Convenience references to indexes + ExportPartitionTaskEntriesContainer::index::type & export_merge_tree_partition_task_entries_by_key; + ExportPartitionTaskEntriesContainer::index::type & export_merge_tree_partition_task_entries_by_transaction_id; + ExportPartitionTaskEntriesContainer::index::type & export_merge_tree_partition_task_entries_by_create_time; /// A thread that removes old parts, log entries, and blocks. ReplicatedMergeTreeCleanupThread cleanup_thread; @@ -739,6 +769,14 @@ class StorageReplicatedMergeTree final : public MergeTreeData /// Checks if some mutations are done and marks them as done. void mutationsFinalizingTask(); + void selectPartsToExport(); + + /// update in-memory list of partition exports + void exportMergeTreePartitionUpdatingTask(); + + /// handle status changes for export partition tasks + void exportMergeTreePartitionStatusHandlingTask(); + /** Write the selected parts to merge into the log, * Call when merge_selecting_mutex is locked. * Returns false if any part is not in ZK. @@ -921,6 +959,7 @@ class StorageReplicatedMergeTree final : public MergeTreeData void movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, ContextPtr query_context) override; void movePartitionToShard(const ASTPtr & partition, bool move_part, const String & to, ContextPtr query_context) override; CancellationCode killPartMoveToShard(const UUID & task_uuid) override; + CancellationCode killExportPartition(const String & transaction_id) override; void fetchPartition( const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, @@ -928,7 +967,8 @@ class StorageReplicatedMergeTree final : public MergeTreeData bool fetch_part, ContextPtr query_context) override; void forgetPartition(const ASTPtr & partition, ContextPtr query_context) override; - + + void exportPartitionToTable(const PartitionCommand &, ContextPtr) override; /// NOTE: there are no guarantees for concurrent merges. Dropping part can /// be concurrently merged into some covering part and dropPart will do @@ -960,8 +1000,6 @@ class StorageReplicatedMergeTree final : public MergeTreeData MutationsSnapshotPtr getMutationsSnapshot(const IMutationsSnapshot::Params & params) const override; - void startBackgroundMovesIfNeeded() override; - /// Attaches restored parts to the storage. void attachRestoredParts(MutableDataPartsVector && parts) override; diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index ab8c5089f6ab..81148b1514cb 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -732,7 +732,7 @@ void StorageURLSink::cancelBuffers() write_buf->cancel(); } -class PartitionedStorageURLSink : public PartitionedSink +class PartitionedStorageURLSink : public PartitionedSink::SinkCreator { public: PartitionedStorageURLSink( @@ -746,7 +746,7 @@ class PartitionedStorageURLSink : public PartitionedSink const CompressionMethod compression_method_, const HTTPHeaderEntries & headers_, const String & http_method_) - : PartitionedSink(partition_strategy_, context_, std::make_shared(sample_block_)) + : partition_strategy(partition_strategy_) , uri(uri_) , format(format_) , format_settings(format_settings_) @@ -761,7 +761,8 @@ class PartitionedStorageURLSink : public PartitionedSink SinkPtr createSinkForPartition(const String & partition_id) override { - std::string partition_path = partition_strategy->getPathForWrite(uri, partition_id); + const auto file_path_generator = std::make_shared(uri); + std::string partition_path = file_path_generator->getPathForWrite(partition_id); context->getRemoteHostFilter().checkURL(Poco::URI(partition_path)); return std::make_shared( @@ -769,6 +770,7 @@ class PartitionedStorageURLSink : public PartitionedSink } private: + std::shared_ptr partition_strategy; const String uri; const String format; const std::optional format_settings; @@ -1439,7 +1441,7 @@ SinkToStoragePtr IStorageURLBase::write(const ASTPtr & query, const StorageMetad has_wildcards, /* partition_columns_in_data_file */true); - return std::make_shared( + auto sink_creator = std::make_shared( partition_strategy, uri, format_name, @@ -1450,6 +1452,8 @@ SinkToStoragePtr IStorageURLBase::write(const ASTPtr & query, const StorageMetad compression_method, headers, http_method); + + return std::make_shared(partition_strategy, sink_creator, context, std::make_shared(metadata_snapshot->getSampleBlock())); } return std::make_shared( diff --git a/src/Storages/System/StorageSystemExports.cpp b/src/Storages/System/StorageSystemExports.cpp new file mode 100644 index 000000000000..fad8b43d0d2f --- /dev/null +++ b/src/Storages/System/StorageSystemExports.cpp @@ -0,0 +1,72 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +ColumnsDescription StorageSystemExports::getColumnsDescription() +{ + return ColumnsDescription + { + {"source_database", std::make_shared(), "Name of the source database."}, + {"source_table", std::make_shared(), "Name of the source table."}, + {"destination_database", std::make_shared(), "Name of the destination database."}, + {"destination_table", std::make_shared(), "Name of the destination table."}, + {"create_time", std::make_shared(), "Date and time when the export command was received in the server."}, + {"part_name", std::make_shared(), "Name of the part"}, + {"query_id", std::make_shared(), "Query ID of the export operation."}, + {"destination_file_paths", std::make_shared(std::make_shared()), "File paths where the part is being exported."}, + {"elapsed", std::make_shared(), "The time elapsed (in seconds) since the export started."}, + {"rows_read", std::make_shared(), "The number of rows read from the exported part."}, + {"total_rows_to_read", std::make_shared(), "The total number of rows to read from the exported part."}, + {"total_size_bytes_compressed", std::make_shared(), "The total size of the compressed data in the exported part."}, + {"total_size_bytes_uncompressed", std::make_shared(), "The total size of the uncompressed data in the exported part."}, + {"bytes_read_uncompressed", std::make_shared(), "The number of uncompressed bytes read from the exported part."}, + {"memory_usage", std::make_shared(), "Current memory usage in bytes for the export operation."}, + {"peak_memory_usage", std::make_shared(), "Peak memory usage in bytes during the export operation."}, + }; +} + +void StorageSystemExports::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const +{ + const auto access = context->getAccess(); + const bool check_access_for_tables = !access->isGranted(AccessType::SHOW_TABLES); + + for (const auto & export_info : context->getExportsList().get()) + { + if (check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, export_info.source_database, export_info.source_table)) + continue; + + size_t i = 0; + res_columns[i++]->insert(export_info.source_database); + res_columns[i++]->insert(export_info.source_table); + res_columns[i++]->insert(export_info.destination_database); + res_columns[i++]->insert(export_info.destination_table); + res_columns[i++]->insert(export_info.create_time); + res_columns[i++]->insert(export_info.part_name); + res_columns[i++]->insert(export_info.query_id); + Array destination_file_paths_array; + destination_file_paths_array.reserve(export_info.destination_file_paths.size()); + for (const auto & file_path : export_info.destination_file_paths) + destination_file_paths_array.push_back(file_path); + res_columns[i++]->insert(destination_file_paths_array); + res_columns[i++]->insert(export_info.elapsed); + res_columns[i++]->insert(export_info.rows_read); + res_columns[i++]->insert(export_info.total_rows_to_read); + res_columns[i++]->insert(export_info.total_size_bytes_compressed); + res_columns[i++]->insert(export_info.total_size_bytes_uncompressed); + res_columns[i++]->insert(export_info.bytes_read_uncompressed); + res_columns[i++]->insert(export_info.memory_usage); + res_columns[i++]->insert(export_info.peak_memory_usage); + } +} + +} diff --git a/src/Storages/System/StorageSystemExports.h b/src/Storages/System/StorageSystemExports.h new file mode 100644 index 000000000000..e13fbfa26aaa --- /dev/null +++ b/src/Storages/System/StorageSystemExports.h @@ -0,0 +1,25 @@ +#pragma once + +#include + + +namespace DB +{ + +class Context; + + +class StorageSystemExports final : public IStorageSystemOneBlock +{ +public: + std::string getName() const override { return "SystemExports"; } + + static ColumnsDescription getColumnsDescription(); + +protected: + using IStorageSystemOneBlock::IStorageSystemOneBlock; + + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; +}; + +} diff --git a/src/Storages/System/StorageSystemMerges.cpp b/src/Storages/System/StorageSystemMerges.cpp index 0fca5dc84a2b..c8c569ff4696 100644 --- a/src/Storages/System/StorageSystemMerges.cpp +++ b/src/Storages/System/StorageSystemMerges.cpp @@ -1,5 +1,5 @@ -#include #include +#include #include #include diff --git a/src/Storages/System/StorageSystemReplicatedPartitionExports.cpp b/src/Storages/System/StorageSystemReplicatedPartitionExports.cpp new file mode 100644 index 000000000000..ed4b2bc9f4c2 --- /dev/null +++ b/src/Storages/System/StorageSystemReplicatedPartitionExports.cpp @@ -0,0 +1,145 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include "Columns/ColumnString.h" +#include "Storages/VirtualColumnUtils.h" + + +namespace DB +{ + +ColumnsDescription StorageSystemReplicatedPartitionExports::getColumnsDescription() +{ + return ColumnsDescription + { + {"source_database", std::make_shared(), "Name of the source database."}, + {"source_table", std::make_shared(), "Name of the source table."}, + {"destination_database", std::make_shared(), "Name of the destination database."}, + {"destination_table", std::make_shared(), "Name of the destination table."}, + {"create_time", std::make_shared(), "Date and time when the export command was submitted"}, + {"partition_id", std::make_shared(), "ID of the partition"}, + {"transaction_id", std::make_shared(), "ID of the transaction."}, + {"query_id", std::make_shared(), "Query ID of the export operation."}, + {"source_replica", std::make_shared(), "Name of the source replica."}, + {"parts", std::make_shared(std::make_shared()), "List of part names to be exported."}, + {"parts_count", std::make_shared(), "Number of parts in the export."}, + {"parts_to_do", std::make_shared(), "Number of parts pending to be exported."}, + {"status", std::make_shared(), "Status of the export."}, + {"exception_replica", std::make_shared(), "Replica that caused the last exception"}, + {"last_exception", std::make_shared(), "Last exception message of any part (not necessarily the last global exception)"}, + {"exception_part", std::make_shared(), "Part that caused the last exception"}, + {"exception_count", std::make_shared(), "Number of global exceptions"}, + }; +} + +void StorageSystemReplicatedPartitionExports::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node * predicate, std::vector) const +{ + const auto access = context->getAccess(); + const bool check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES); + + std::map> replicated_merge_tree_tables; + for (const auto & db : DatabaseCatalog::instance().getDatabases(GetDatabasesOptions{.with_datalake_catalogs = false})) + { + /// skip data lakes + if (db.second->isExternal()) + continue; + + const bool check_access_for_tables = check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, db.first); + + for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next()) + { + const auto & table = iterator->table(); + if (!table) + continue; + + StorageReplicatedMergeTree * table_replicated = dynamic_cast(table.get()); + if (!table_replicated) + continue; + + if (check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, db.first, iterator->name())) + continue; + + replicated_merge_tree_tables[db.first][iterator->name()] = table; + } + } + + MutableColumnPtr col_database_mut = ColumnString::create(); + MutableColumnPtr col_table_mut = ColumnString::create(); + + for (auto & db : replicated_merge_tree_tables) + { + for (auto & table : db.second) + { + col_database_mut->insert(db.first); + col_table_mut->insert(table.first); + } + } + + ColumnPtr col_database = std::move(col_database_mut); + ColumnPtr col_table = std::move(col_table_mut); + + /// Determine what tables are needed by the conditions in the query. + { + Block filtered_block + { + { col_database, std::make_shared(), "database" }, + { col_table, std::make_shared(), "table" }, + }; + + VirtualColumnUtils::filterBlockWithPredicate(predicate, filtered_block, context); + + if (!filtered_block.rows()) + return; + + col_database = filtered_block.getByName("database").column; + col_table = filtered_block.getByName("table").column; + } + + for (size_t i_storage = 0; i_storage < col_database->size(); ++i_storage) + { + const auto database = (*col_database)[i_storage].safeGet(); + const auto table = (*col_table)[i_storage].safeGet(); + + std::vector partition_exports_info; + { + const IStorage * storage = replicated_merge_tree_tables[database][table].get(); + if (const auto * replicated_merge_tree = dynamic_cast(storage)) + partition_exports_info = replicated_merge_tree->getPartitionExportsInfo(); + } + + for (const ReplicatedPartitionExportInfo & info : partition_exports_info) + { + std::size_t i = 0; + res_columns[i++]->insert(database); + res_columns[i++]->insert(table); + res_columns[i++]->insert(info.destination_database); + res_columns[i++]->insert(info.destination_table); + res_columns[i++]->insert(info.create_time); + res_columns[i++]->insert(info.partition_id); + res_columns[i++]->insert(info.transaction_id); + res_columns[i++]->insert(info.query_id); + res_columns[i++]->insert(info.source_replica); + Array parts_array; + parts_array.reserve(info.parts.size()); + for (const auto & part : info.parts) + parts_array.push_back(part); + res_columns[i++]->insert(parts_array); + res_columns[i++]->insert(info.parts_count); + res_columns[i++]->insert(info.parts_to_do); + res_columns[i++]->insert(info.status); + res_columns[i++]->insert(info.exception_replica); + res_columns[i++]->insert(info.last_exception); + res_columns[i++]->insert(info.exception_part); + res_columns[i++]->insert(info.exception_count); + } + } +} + +} diff --git a/src/Storages/System/StorageSystemReplicatedPartitionExports.h b/src/Storages/System/StorageSystemReplicatedPartitionExports.h new file mode 100644 index 000000000000..d57844cbb34e --- /dev/null +++ b/src/Storages/System/StorageSystemReplicatedPartitionExports.h @@ -0,0 +1,43 @@ +#pragma once + +#include + +namespace DB +{ + +class Context; + +struct ReplicatedPartitionExportInfo +{ + String destination_database; + String destination_table; + String partition_id; + String transaction_id; + String query_id; + time_t create_time; + String source_replica; + size_t parts_count; + size_t parts_to_do; + std::vector parts; + String status; + std::string exception_replica; + std::string last_exception; + std::string exception_part; + size_t exception_count; +}; + +class StorageSystemReplicatedPartitionExports final : public IStorageSystemOneBlock +{ +public: + + std::string getName() const override { return "SystemReplicatedPartitionExports"; } + + static ColumnsDescription getColumnsDescription(); + +protected: + using IStorageSystemOneBlock::IStorageSystemOneBlock; + + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; +}; + +} diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index e9a8672c106e..fb69bdd1e154 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -1,11 +1,12 @@ #include +#include #include "config.h" #include #include #include #include - +#include #include #include #include @@ -35,6 +36,7 @@ #include #include #include +#include #include #include #include @@ -118,6 +120,7 @@ # include #endif #include +#include #include @@ -143,6 +146,11 @@ namespace DB { +namespace ServerSetting +{ + extern const ServerSettingsBool enable_experimental_export_merge_tree_partition_feature; +} + void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, bool has_zookeeper) { auto component_guard = Coordination::setCurrentComponent("attachSystemTablesServer"); @@ -230,6 +238,11 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b attach(context, system_database, "dimensional_metrics", "Contains dimensional metrics, which have multiple dimensions (labels) to provide more granular information. For example, counting failed merges by their error code. This table is always up to date."); attach(context, system_database, "merges", "Contains a list of merges currently executing merges of MergeTree tables and their progress. Each merge operation is represented by a single row."); attach(context, system_database, "moves", "Contains information about in-progress data part moves of MergeTree tables. Each data part movement is represented by a single row."); + attach(context, system_database, "exports", "Contains a list of exports currently executing exports of MergeTree tables and their progress. Each export operation is represented by a single row."); + if (context->getServerSettings()[ServerSetting::enable_experimental_export_merge_tree_partition_feature]) + { + attach(context, system_database, "replicated_partition_exports", "Contains a list of partition exports of ReplicatedMergeTree tables and their progress. Each export operation is represented by a single row."); + } attach(context, system_database, "mutations", "Contains a list of mutations and their progress. Each mutation command is represented by a single row."); attachNoDescription(context, system_database, "replicas", "Contains information and status of all table replicas on current server. Each replica is represented by a single row."); attachNoDescription(context, system_database, "database_replicas", "Contains information and status of all database replicas on current server. Each database replica is represented by a single row."); diff --git a/tests/config/config.d/enable_experimental_export_merge_tree_partition.xml b/tests/config/config.d/enable_experimental_export_merge_tree_partition.xml new file mode 100644 index 000000000000..72014c9de4db --- /dev/null +++ b/tests/config/config.d/enable_experimental_export_merge_tree_partition.xml @@ -0,0 +1,3 @@ + + 1 + diff --git a/tests/config/install.sh b/tests/config/install.sh index c9f2827b40f2..f3ded9587e2a 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -95,6 +95,7 @@ ln -sf $SRC_PATH/config.d/blob_storage_log.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/custom_settings_prefixes.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/database_catalog_drop_table_concurrency.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/enable_access_control_improvements.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/enable_experimental_export_merge_tree_partition.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/macros.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/secure_ports.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/clusters.xml $DEST_SERVER_PATH/config.d/ diff --git a/tests/integration/test_export_merge_tree_part_to_object_storage/__init__.py b/tests/integration/test_export_merge_tree_part_to_object_storage/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/integration/test_export_merge_tree_part_to_object_storage/configs/named_collections.xml b/tests/integration/test_export_merge_tree_part_to_object_storage/configs/named_collections.xml new file mode 100644 index 000000000000..d46920b7ba88 --- /dev/null +++ b/tests/integration/test_export_merge_tree_part_to_object_storage/configs/named_collections.xml @@ -0,0 +1,9 @@ + + + + http://minio1:9001/root/data + minio + ClickHouse_Minio_P@ssw0rd + + + \ No newline at end of file diff --git a/tests/integration/test_export_merge_tree_part_to_object_storage/test.py b/tests/integration/test_export_merge_tree_part_to_object_storage/test.py new file mode 100644 index 000000000000..af149e682049 --- /dev/null +++ b/tests/integration/test_export_merge_tree_part_to_object_storage/test.py @@ -0,0 +1,286 @@ +import logging +import pytest +import random +import string +import time +from typing import Optional +import uuid + +from helpers.cluster import ClickHouseCluster +from helpers.network import PartitionManager + +@pytest.fixture(scope="module") +def cluster(): + try: + cluster = ClickHouseCluster(__file__) + cluster.add_instance( + "node1", + main_configs=["configs/named_collections.xml"], + with_minio=True, + ) + logging.info("Starting cluster...") + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def create_s3_table(node, s3_table): + node.query(f"CREATE TABLE {s3_table} (id UInt64, year UInt16) ENGINE = S3(s3_conn, filename='{s3_table}', format=Parquet, partition_strategy='hive') PARTITION BY year") + + +def create_tables_and_insert_data(node, mt_table, s3_table): + node.query(f"CREATE TABLE {mt_table} (id UInt64, year UInt16) ENGINE = MergeTree() PARTITION BY year ORDER BY tuple() SETTINGS enable_block_number_column = 1, enable_block_offset_column = 1") + node.query(f"INSERT INTO {mt_table} VALUES (1, 2020), (2, 2020), (3, 2020), (4, 2021)") + + create_s3_table(node, s3_table) + + +def test_drop_column_during_export_snapshot(cluster): + node = cluster.instances["node1"] + + mt_table = "mutations_snapshot_mt_table" + s3_table = "mutations_snapshot_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table) + + # Block traffic to/from MinIO to force upload errors and retries, following existing S3 tests style + minio_ip = cluster.minio_ip + minio_port = cluster.minio_port + + # Ensure export sees a consistent snapshot at start time even if we mutate the source later + with PartitionManager() as pm: + # Block responses from MinIO (source_port matches MinIO service) + pm_rule_reject_responses = { + "instance": node, + "destination": node.ip_address, + "protocol": "tcp", + "source_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_responses) + + # Block requests to MinIO (destination: MinIO, destination_port: minio_port) + pm_rule_reject_requests = { + "instance": node, + "destination": minio_ip, + "protocol": "tcp", + "destination_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_requests) + + # Start export of 2020 + node.query( + f"ALTER TABLE {mt_table} EXPORT PART '2020_1_1_0' TO TABLE {s3_table};" + ) + + # Drop a column that is required for the export + node.query(f"ALTER TABLE {mt_table} DROP COLUMN id") + + time.sleep(3) + # assert the mutation has been applied AND the data has not been exported yet + assert "Unknown expression identifier `id`" in node.query_and_get_error(f"SELECT id FROM {mt_table}"), "Column id is not removed" + + # Wait for export to finish and then verify destination still reflects the original snapshot (3 rows) + time.sleep(5) + assert node.query(f"SELECT count() FROM {s3_table} WHERE id >= 0") == '3\n', "Export did not preserve snapshot at start time after source mutation" + + +def test_add_column_during_export(cluster): + node = cluster.instances["node1"] + + mt_table = "add_column_during_export_mt_table" + s3_table = "add_column_during_export_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table) + + # Block traffic to/from MinIO to force upload errors and retries, following existing S3 tests style + minio_ip = cluster.minio_ip + minio_port = cluster.minio_port + + # Ensure export sees a consistent snapshot at start time even if we mutate the source later + with PartitionManager() as pm: + # Block responses from MinIO (source_port matches MinIO service) + pm_rule_reject_responses = { + "instance": node, + "destination": node.ip_address, + "protocol": "tcp", + "source_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_responses) + + # Block requests to MinIO (destination: MinIO, destination_port: minio_port) + pm_rule_reject_requests = { + "instance": node, + "destination": minio_ip, + "protocol": "tcp", + "destination_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_requests) + + # Start export of 2020 + node.query( + f"ALTER TABLE {mt_table} EXPORT PART '2020_1_1_0' TO TABLE {s3_table};" + ) + + node.query(f"ALTER TABLE {mt_table} ADD COLUMN id2 UInt64") + + time.sleep(3) + + # assert the mutation has been applied AND the data has not been exported yet + assert node.query(f"SELECT count(id2) FROM {mt_table}") == '4\n', "Column id2 is not added" + + # Wait for export to finish and then verify destination still reflects the original snapshot (3 rows) + time.sleep(5) + assert node.query(f"SELECT count() FROM {s3_table} WHERE id >= 0") == '3\n', "Export did not preserve snapshot at start time after source mutation" + assert "Unknown expression identifier `id2`" in node.query_and_get_error(f"SELECT id2 FROM {s3_table}"), "Column id2 is present in the exported data" + + +def test_pending_mutations_throw_before_export(cluster): + """Test that pending mutations before export throw an error with default settings.""" + node = cluster.instances["node1"] + + mt_table = "pending_mutations_throw_mt_table" + s3_table = "pending_mutations_throw_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table) + + node.query(f"SYSTEM STOP MERGES {mt_table}") + + node.query(f"ALTER TABLE {mt_table} UPDATE id = id + 100 WHERE year = 2020") + + mutations = node.query(f"SELECT count() FROM system.mutations WHERE table = '{mt_table}' AND is_done = 0") + assert mutations.strip() != '0', "Mutation should be pending" + + error = node.query_and_get_error( + f"ALTER TABLE {mt_table} EXPORT PART '2020_1_1_0' TO TABLE {s3_table} SETTINGS export_merge_tree_part_throw_on_pending_mutations=true" + ) + + assert "PENDING_MUTATIONS_NOT_ALLOWED" in error, f"Expected error about pending mutations, got: {error}" + + +def test_pending_mutations_skip_before_export(cluster): + """Test that pending mutations before export are skipped with throw_on_pending_mutations=false.""" + node = cluster.instances["node1"] + + mt_table = "pending_mutations_skip_mt_table" + s3_table = "pending_mutations_skip_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table) + + node.query(f"SYSTEM STOP MERGES {mt_table}") + + node.query(f"ALTER TABLE {mt_table} UPDATE id = id + 100 WHERE year = 2020") + + mutations = node.query(f"SELECT count() FROM system.mutations WHERE table = '{mt_table}' AND is_done = 0") + assert mutations.strip() != '0', "Mutation should be pending" + + node.query( + f"ALTER TABLE {mt_table} EXPORT PART '2020_1_1_0' TO TABLE {s3_table} " + f"SETTINGS export_merge_tree_part_throw_on_pending_mutations=false" + ) + + time.sleep(5) + + result = node.query(f"SELECT id FROM {s3_table} WHERE year = 2020 ORDER BY id") + assert "101" not in result and "102" not in result and "103" not in result, \ + "Export should contain original data before mutation" + assert "1\n2\n3" in result, "Export should contain original data" + + +def test_data_mutations_after_export_started(cluster): + """Test that mutations applied after export starts don't affect the exported data.""" + node = cluster.instances["node1"] + + mt_table = "mutations_after_export_mt_table" + s3_table = "mutations_after_export_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table) + + # Block traffic to MinIO to delay export + minio_ip = cluster.minio_ip + minio_port = cluster.minio_port + + with PartitionManager() as pm: + pm_rule_reject_responses = { + "instance": node, + "destination": node.ip_address, + "protocol": "tcp", + "source_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_responses) + + pm_rule_reject_requests = { + "instance": node, + "destination": minio_ip, + "protocol": "tcp", + "destination_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_requests) + + node.query( + f"ALTER TABLE {mt_table} EXPORT PART '2020_1_1_0' TO TABLE {s3_table} " + f"SETTINGS export_merge_tree_part_throw_on_pending_mutations=true" + ) + + node.query(f"ALTER TABLE {mt_table} UPDATE id = id + 100 WHERE year = 2020") + + time.sleep(5) + + result = node.query(f"SELECT id FROM {s3_table} WHERE year = 2020 ORDER BY id") + assert "1\n2\n3" in result, "Export should contain original data before mutation" + assert "101" not in result, "Export should not contain mutated data" + + +def test_pending_patch_parts_throw_before_export(cluster): + """Test that pending patch parts before export throw an error with default settings.""" + node = cluster.instances["node1"] + + mt_table = "pending_patches_throw_mt_table" + s3_table = "pending_patches_throw_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table) + + node.query(f"SYSTEM STOP MERGES {mt_table}") + + node.query(f"UPDATE {mt_table} SET id = id + 100 WHERE year = 2020") + + error = node.query_and_get_error( + f"ALTER TABLE {mt_table} EXPORT PART '2020_1_1_0' TO TABLE {s3_table}" + ) + + node.query(f"DROP TABLE {mt_table}") + + assert "PENDING_MUTATIONS_NOT_ALLOWED" in error or "pending patch parts" in error.lower(), \ + f"Expected error about pending patch parts, got: {error}" + + +def test_pending_patch_parts_skip_before_export(cluster): + """Test that pending patch parts before export are skipped with throw_on_pending_patch_parts=false.""" + node = cluster.instances["node1"] + + mt_table = "pending_patches_skip_mt_table" + s3_table = "pending_patches_skip_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table) + + node.query(f"SYSTEM STOP MERGES {mt_table}") + + node.query(f"UPDATE {mt_table} SET id = id + 100 WHERE year = 2020") + + node.query( + f"ALTER TABLE {mt_table} EXPORT PART '2020_1_1_0' TO TABLE {s3_table} " + f"SETTINGS export_merge_tree_part_throw_on_pending_patch_parts=false" + ) + + time.sleep(5) + + result = node.query(f"SELECT id FROM {s3_table} WHERE year = 2020 ORDER BY id") + assert "1\n2\n3" in result, "Export should contain original data before patch" + + node.query(f"DROP TABLE {mt_table}") diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/__init__.py b/tests/integration/test_export_replicated_mt_partition_to_object_storage/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/allow_experimental_export_partition.xml b/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/allow_experimental_export_partition.xml new file mode 100644 index 000000000000..f8c5fab1a3be --- /dev/null +++ b/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/allow_experimental_export_partition.xml @@ -0,0 +1,3 @@ + + 1 + \ No newline at end of file diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/disable_experimental_export_partition.xml b/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/disable_experimental_export_partition.xml new file mode 100644 index 000000000000..ba6508ebd660 --- /dev/null +++ b/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/disable_experimental_export_partition.xml @@ -0,0 +1,3 @@ + + 0 + \ No newline at end of file diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/named_collections.xml b/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/named_collections.xml new file mode 100644 index 000000000000..d46920b7ba88 --- /dev/null +++ b/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/named_collections.xml @@ -0,0 +1,9 @@ + + + + http://minio1:9001/root/data + minio + ClickHouse_Minio_P@ssw0rd + + + \ No newline at end of file diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/s3_retries.xml b/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/s3_retries.xml new file mode 100644 index 000000000000..fb74c9bdf7de --- /dev/null +++ b/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/s3_retries.xml @@ -0,0 +1,3 @@ + + 3 + diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/users.d/profile.xml b/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/users.d/profile.xml new file mode 100644 index 000000000000..518f29708929 --- /dev/null +++ b/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/users.d/profile.xml @@ -0,0 +1,8 @@ + + + + 3 + + + + diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py new file mode 100644 index 000000000000..db6b1bb23a21 --- /dev/null +++ b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py @@ -0,0 +1,1074 @@ +import logging +import pytest +import random +import string +import time +from typing import Optional +import uuid + +from helpers.cluster import ClickHouseCluster +from helpers.network import PartitionManager + + +def wait_for_export_status( + node, + mt_table: str, + s3_table: str, + partition_id: str, + expected_status: str = "COMPLETED", + timeout: int = 30, + poll_interval: float = 0.5, +): + start_time = time.time() + last_status = None + while time.time() - start_time < timeout: + status = node.query( + f""" + SELECT status FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '{partition_id}' + """ + ).strip() + + last_status = status + + if status and status == expected_status: + return status + + time.sleep(poll_interval) + + raise TimeoutError( + f"Export status did not reach '{expected_status}' within {timeout}s. Last status: '{last_status}'") + + +def wait_for_export_to_start( + node, + mt_table: str, + s3_table: str, + partition_id: str, + timeout: int = 10, + poll_interval: float = 0.2, +): + start_time = time.time() + while time.time() - start_time < timeout: + count = node.query( + f""" + SELECT count() FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '{partition_id}' + """ + ).strip() + + if count != '0': + return True + + time.sleep(poll_interval) + + raise TimeoutError(f"Export did not start within {timeout}s. ") + + +@pytest.fixture(scope="module") +def cluster(): + try: + cluster = ClickHouseCluster(__file__) + cluster.add_instance( + "replica1", + main_configs=["configs/named_collections.xml", "configs/allow_experimental_export_partition.xml"], + user_configs=["configs/users.d/profile.xml"], + with_minio=True, + stay_alive=True, + with_zookeeper=True, + ) + cluster.add_instance( + "replica2", + main_configs=["configs/named_collections.xml", "configs/allow_experimental_export_partition.xml"], + user_configs=["configs/users.d/profile.xml"], + with_minio=True, + stay_alive=True, + with_zookeeper=True, + ) + # node that does not participate in the export, but will have visibility over the s3 table + cluster.add_instance( + "watcher_node", + main_configs=["configs/named_collections.xml"], + user_configs=[], + with_minio=True, + ) + cluster.add_instance( + "replica_with_export_disabled", + main_configs=["configs/named_collections.xml", "configs/disable_experimental_export_partition.xml"], + user_configs=["configs/users.d/profile.xml"], + with_minio=True, + stay_alive=True, + with_zookeeper=True, + ) + logging.info("Starting cluster...") + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def create_s3_table(node, s3_table): + node.query(f"CREATE TABLE {s3_table} (id UInt64, year UInt16) ENGINE = S3(s3_conn, filename='{s3_table}', format=Parquet, partition_strategy='hive') PARTITION BY year") + + +def create_tables_and_insert_data(node, mt_table, s3_table, replica_name): + node.query(f"CREATE TABLE {mt_table} (id UInt64, year UInt16) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{mt_table}', '{replica_name}') PARTITION BY year ORDER BY tuple() SETTINGS enable_block_number_column = 1, enable_block_offset_column = 1") + node.query(f"INSERT INTO {mt_table} VALUES (1, 2020), (2, 2020), (3, 2020), (4, 2021)") + + create_s3_table(node, s3_table) + + +def test_restart_nodes_during_export(cluster): + node = cluster.instances["replica1"] + node2 = cluster.instances["replica2"] + watcher_node = cluster.instances["watcher_node"] + + mt_table = "disaster_mt_table" + s3_table = "disaster_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + create_tables_and_insert_data(node2, mt_table, s3_table, "replica2") + create_s3_table(watcher_node, s3_table) + + # Block S3/MinIO requests to keep exports alive via retry mechanism + # This allows ZooKeeper operations to proceed quickly + minio_ip = cluster.minio_ip + minio_port = cluster.minio_port + + with PartitionManager() as pm: + # Block responses from MinIO (source_port matches MinIO service) + pm_rule_reject_responses_node1 = { + "instance": node, + "destination": node.ip_address, + "protocol": "tcp", + "source_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_responses_node1) + + pm_rule_reject_responses_node2 = { + "instance": node2, + "destination": node2.ip_address, + "protocol": "tcp", + "source_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_responses_node2) + + # Block requests to MinIO (destination: MinIO, destination_port: minio_port) + pm_rule_reject_requests_node1 = { + "instance": node, + "destination": minio_ip, + "protocol": "tcp", + "destination_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_requests_node1) + + pm_rule_reject_requests_node2 = { + "instance": node2, + "destination": minio_ip, + "protocol": "tcp", + "destination_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_requests_node2) + + export_queries = f""" + ALTER TABLE {mt_table} + EXPORT PARTITION ID '2020' TO TABLE {s3_table} + SETTINGS export_merge_tree_partition_max_retries = 50; + ALTER TABLE {mt_table} + EXPORT PARTITION ID '2021' TO TABLE {s3_table} + SETTINGS export_merge_tree_partition_max_retries = 50; + """ + + node.query(export_queries) + + # wait for the exports to start + wait_for_export_to_start(node, mt_table, s3_table, "2020") + wait_for_export_to_start(node, mt_table, s3_table, "2021") + + node.stop_clickhouse(kill=True) + node2.stop_clickhouse(kill=True) + + assert watcher_node.query(f"SELECT count() FROM {s3_table} where year = 2020") == '0\n', "Partition 2020 was written to S3 during network delay crash" + + assert watcher_node.query(f"SELECT count() FROM {s3_table} where year = 2021") == '0\n', "Partition 2021 was written to S3 during network delay crash" + + # start the nodes, they should finish the export + node.start_clickhouse() + node2.start_clickhouse() + + wait_for_export_status(node, mt_table, s3_table, "2020", "COMPLETED") + wait_for_export_status(node, mt_table, s3_table, "2021", "COMPLETED") + + assert node.query(f"SELECT count() FROM {s3_table} WHERE year = 2020") != f'0\n', "Export of partition 2020 did not resume after crash" + + assert node.query(f"SELECT count() FROM {s3_table} WHERE year = 2021") != f'0\n', "Export of partition 2021 did not resume after crash" + + +def test_kill_export(cluster): + node = cluster.instances["replica1"] + node2 = cluster.instances["replica2"] + watcher_node = cluster.instances["watcher_node"] + + mt_table = "kill_export_mt_table" + s3_table = "kill_export_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + create_tables_and_insert_data(node2, mt_table, s3_table, "replica2") + + # Block S3/MinIO requests to keep exports alive via retry mechanism + # This allows ZooKeeper operations (KILL) to proceed quickly + minio_ip = cluster.minio_ip + minio_port = cluster.minio_port + + with PartitionManager() as pm: + # Block responses from MinIO (source_port matches MinIO service) + pm_rule_reject_responses = { + "instance": node, + "destination": node.ip_address, + "protocol": "tcp", + "source_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_responses) + + # Block requests to MinIO (destination: MinIO, destination_port: minio_port) + pm_rule_reject_requests = { + "instance": node, + "destination": minio_ip, + "protocol": "tcp", + "destination_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_requests) + + # Block responses from MinIO for node2 + pm_rule_reject_responses_node2 = { + "instance": node2, + "destination": node2.ip_address, + "protocol": "tcp", + "source_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_responses_node2) + + # Block requests to MinIO from node2 + pm_rule_reject_requests_node2 = { + "instance": node2, + "destination": minio_ip, + "protocol": "tcp", + "destination_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_requests_node2) + + export_queries = f""" + ALTER TABLE {mt_table} + EXPORT PARTITION ID '2020' TO TABLE {s3_table} + SETTINGS export_merge_tree_partition_max_retries = 50; + ALTER TABLE {mt_table} + EXPORT PARTITION ID '2021' TO TABLE {s3_table} + SETTINGS export_merge_tree_partition_max_retries = 50; + """ + + node.query(export_queries) + + # Kill only 2020 while S3 is blocked - retry mechanism keeps exports alive + # ZooKeeper operations (KILL) proceed quickly since only S3 is blocked + node.query(f"KILL EXPORT PARTITION WHERE partition_id = '2020' and source_table = '{mt_table}' and destination_table = '{s3_table}'") + + # wait for 2021 to finish + wait_for_export_status(node, mt_table, s3_table, "2021", "COMPLETED") + + # checking for the commit file because maybe the data file was too fast? + assert node.query(f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_2020_*', format=LineAsString)") == '0\n', "Partition 2020 was written to S3, it was not killed as expected" + assert node.query(f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_2021_*', format=LineAsString)") != f'0\n', "Partition 2021 was not written to S3, but it should have been" + + # check system.replicated_partition_exports for the export, status should be KILLED + assert node.query(f"SELECT status FROM system.replicated_partition_exports WHERE partition_id = '2020' and source_table = '{mt_table}' and destination_table = '{s3_table}'") == 'KILLED\n', "Partition 2020 was not killed as expected" + assert node.query(f"SELECT status FROM system.replicated_partition_exports WHERE partition_id = '2021' and source_table = '{mt_table}' and destination_table = '{s3_table}'") == 'COMPLETED\n', "Partition 2021 was not completed, this is unexpected" + + +def test_drop_source_table_during_export(cluster): + node = cluster.instances["replica1"] + # node2 = cluster.instances["replica2"] + watcher_node = cluster.instances["watcher_node"] + + mt_table = "drop_source_table_during_export_mt_table" + s3_table = "drop_source_table_during_export_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + # create_tables_and_insert_data(node2, mt_table, s3_table, "replica2") + create_s3_table(watcher_node, s3_table) + + # Block S3/MinIO requests to keep exports alive via retry mechanism + # This allows ZooKeeper operations (KILL) to proceed quickly + minio_ip = cluster.minio_ip + minio_port = cluster.minio_port + + with PartitionManager() as pm: + # Block responses from MinIO (source_port matches MinIO service) + pm_rule_reject_responses = { + "instance": node, + "destination": node.ip_address, + "protocol": "tcp", + "source_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_responses) + + # Block requests to MinIO (destination: MinIO, destination_port: minio_port) + pm_rule_reject_requests = { + "instance": node, + "destination": minio_ip, + "protocol": "tcp", + "destination_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_requests) + + export_queries = f""" + ALTER TABLE {mt_table} + EXPORT PARTITION ID '2020' TO TABLE {s3_table}; + ALTER TABLE {mt_table} + EXPORT PARTITION ID '2021' TO TABLE {s3_table}; + """ + + node.query(export_queries) + + # This should kill the background operations and drop the table + node.query(f"DROP TABLE {mt_table}") + + # Sleep some time to let the export finish (assuming it was not properly cancelled) + time.sleep(10) + + assert node.query(f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_*', format=LineAsString)") == '0\n', "Background operations completed even with the table dropped" + + +def test_concurrent_exports_to_different_targets(cluster): + node = cluster.instances["replica1"] + + mt_table = "concurrent_diff_targets_mt_table" + s3_table_a = "concurrent_diff_targets_s3_a" + s3_table_b = "concurrent_diff_targets_s3_b" + + create_tables_and_insert_data(node, mt_table, s3_table_a, "replica1") + create_s3_table(node, s3_table_b) + + # Launch two exports of the same partition to two different S3 tables concurrently + with PartitionManager() as pm: + pm.add_network_delay(node, delay_ms=1000) + + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table_a}" + ) + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table_b}" + ) + + wait_for_export_status(node, mt_table, s3_table_a, "2020", "COMPLETED") + wait_for_export_status(node, mt_table, s3_table_b, "2020", "COMPLETED") + + # Both targets should receive the same data independently + assert node.query(f"SELECT count() FROM {s3_table_a} WHERE year = 2020") == '3\n', "First target did not receive expected rows" + assert node.query(f"SELECT count() FROM {s3_table_b} WHERE year = 2020") == '3\n', "Second target did not receive expected rows" + + # And both should have a commit marker + assert node.query( + f"SELECT count() FROM s3(s3_conn, filename='{s3_table_a}/commit_2020_*', format=LineAsString)" + ) != '0\n', "Commit file missing for first target" + assert node.query( + f"SELECT count() FROM s3(s3_conn, filename='{s3_table_b}/commit_2020_*', format=LineAsString)" + ) != '0\n', "Commit file missing for second target" + + +def test_failure_is_logged_in_system_table(cluster): + node = cluster.instances["replica1"] + + mt_table = "failure_is_logged_in_system_table_mt_table" + s3_table = "failure_is_logged_in_system_table_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + + # Block traffic to/from MinIO to force upload errors and retries, following existing S3 tests style + minio_ip = cluster.minio_ip + minio_port = cluster.minio_port + + with PartitionManager() as pm: + # Block responses from MinIO (source_port matches MinIO service) + pm_rule_reject_responses = { + "instance": node, + "destination": node.ip_address, + "protocol": "tcp", + "source_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_responses) + + # Also block requests to MinIO (destination: MinIO, destination_port: 9001) with REJECT to fail fast + pm_rule_reject_requests = { + "instance": node, + "destination": minio_ip, + "protocol": "tcp", + "destination_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_requests) + + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS export_merge_tree_partition_max_retries=1;" + ) + + # Wait so that the export fails + wait_for_export_status(node, mt_table, s3_table, "2020", "FAILED", timeout=60) + + # Network restored; verify the export is marked as FAILED in the system table + # Also verify we captured at least one exception and no commit file exists + status = node.query( + f""" + SELECT status FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '2020' + """ + ) + + assert status.strip() == "FAILED", f"Expected FAILED status, got: {status!r}" + + exception_count = node.query( + f""" + SELECT any(exception_count) FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '2020' + """ + ) + assert int(exception_count.strip()) > 0, "Expected non-zero exception_count in system.replicated_partition_exports" + + # No commit should have been produced for this partition + assert node.query( + f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_2020_*', format=LineAsString)" + ) == '0\n', "Commit file exists despite forced S3 failures" + + +def test_inject_short_living_failures(cluster): + node = cluster.instances["replica1"] + + mt_table = "inject_short_living_failures_mt_table" + s3_table = "inject_short_living_failures_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + + # Block traffic to/from MinIO to force upload errors and retries, following existing S3 tests style + minio_ip = cluster.minio_ip + minio_port = cluster.minio_port + + with PartitionManager() as pm: + # Block responses from MinIO (source_port matches MinIO service) + pm_rule_reject_responses = { + "instance": node, + "destination": node.ip_address, + "protocol": "tcp", + "source_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_responses) + + # Also block requests to MinIO (destination: MinIO, destination_port: 9001) with REJECT to fail fast + pm_rule_reject_requests = { + "instance": node, + "destination": minio_ip, + "protocol": "tcp", + "destination_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_requests) + + # set big max_retries so that the export does not fail completely + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS export_merge_tree_partition_max_retries=100;" + ) + + # wait only for a second to get at least one failure, but not enough to finish the export + time.sleep(5) + + # wait for the export to finish + wait_for_export_status(node, mt_table, s3_table, "2020", "COMPLETED") + + # Assert the export succeeded + assert node.query(f"SELECT count() FROM {s3_table} WHERE year = 2020") == '3\n', "Export did not succeed" + assert node.query(f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_2020_*', format=LineAsString)") == '1\n', "Export did not succeed" + + # check system.replicated_partition_exports for the export + assert node.query( + f""" + SELECT status FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '2020' + """ + ) == "COMPLETED\n", "Export should be marked as COMPLETED" + + exception_count = node.query( + f""" + SELECT exception_count FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '2020' + """ + ) + assert int(exception_count.strip()) >= 1, "Expected at least one exception" + + +def test_export_ttl(cluster): + node = cluster.instances["replica1"] + + mt_table = "export_ttl_mt_table" + s3_table = "export_ttl_s3_table" + + expiration_time = 3 + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + + # start export + node.query(f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS export_merge_tree_partition_manifest_ttl={expiration_time};") + + # assert that I get an error when trying to export the same partition again, query_and_get_error + error = node.query_and_get_error(f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table};") + assert "Export with key" in error, "Expected error about expired export" + + # wait for the export to finish and for the manifest to expire + wait_for_export_status(node, mt_table, s3_table, "2020", "COMPLETED") + time.sleep(expiration_time * 2) + + # assert that the export succeeded, check the commit file + assert node.query(f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_2020_*', format=LineAsString)") == '1\n', "Export did not succeed" + + # start export again + node.query(f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table}") + + # wait for the export to finish + wait_for_export_status(node, mt_table, s3_table, "2020", "COMPLETED") + + # assert that the export succeeded, check the commit file + # there should be two commit files now, one for the first export and one for the second export + assert node.query(f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_2020_*', format=LineAsString)") == '2\n', "Export did not succeed" + + +def test_export_partition_file_already_exists_policy(cluster): + node = cluster.instances["replica1"] + + mt_table = "export_partition_file_already_exists_policy_mt_table" + s3_table = "export_partition_file_already_exists_policy_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + + # stop merges so part names remain stable. it is important for the test. + node.query(f"SYSTEM STOP MERGES {mt_table}") + + # Export all parts + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table}", + ) + + # check system.replicated_partition_exports for the export + assert node.query( + f""" + SELECT status FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '2020' + """ + ) == "COMPLETED\n", "Export should be marked as COMPLETED" + + # wait for the exports to finish + wait_for_export_status(node, mt_table, s3_table, "2020", "COMPLETED") + + # try to export the partition + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS export_merge_tree_partition_force_export=1" + ) + + wait_for_export_status(node, mt_table, s3_table, "2020", "COMPLETED") + + assert node.query( + f""" + SELECT count() FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '2020' + AND status = 'COMPLETED' + """ + ) == '1\n', "Expected the export to be marked as COMPLETED" + + # overwrite policy + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS export_merge_tree_partition_force_export=1, export_merge_tree_part_file_already_exists_policy='overwrite'" + ) + + # wait for the export to finish + wait_for_export_status(node, mt_table, s3_table, "2020", "COMPLETED") + + # check system.replicated_partition_exports for the export + # ideally we would make sure the transaction id is different, but I do not have the time to do that now + assert node.query( + f""" + SELECT count() FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '2020' + AND status = 'COMPLETED' + """ + ) == '1\n', "Expected the export to be marked as COMPLETED" + + # last but not least, let's try with the error policy + # max retries = 1 so it fails fast + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS export_merge_tree_partition_force_export=1, export_merge_tree_part_file_already_exists_policy='error', export_merge_tree_partition_max_retries=1", + ) + + # wait for the export to finish + wait_for_export_status(node, mt_table, s3_table, "2020", "FAILED") + + # check system.replicated_partition_exports for the export + assert node.query( + f""" + SELECT count() FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '2020' + AND status = 'FAILED' + """ + ) == '1\n', "Expected the export to be marked as FAILED" + + +def test_export_partition_feature_is_disabled(cluster): + replica_with_export_disabled = cluster.instances["replica_with_export_disabled"] + + mt_table = "export_partition_feature_is_disabled_mt_table" + s3_table = "export_partition_feature_is_disabled_s3_table" + + create_tables_and_insert_data(replica_with_export_disabled, mt_table, s3_table, "replica1") + + error = replica_with_export_disabled.query_and_get_error(f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table};") + assert "experimental" in error, "Expected error about disabled feature" + + # make sure kill operation also throws + error = replica_with_export_disabled.query_and_get_error(f"KILL EXPORT PARTITION WHERE partition_id = '2020' and source_table = '{mt_table}' and destination_table = '{s3_table}'") + assert "experimental" in error, "Expected error about disabled feature" + + +def test_export_partition_permissions(cluster): + """Test that export partition validates permissions correctly: + - User needs ALTER permission on source table + - User needs INSERT permission on destination table + """ + node = cluster.instances["replica1"] + + mt_table = "permissions_mt_table" + s3_table = "permissions_s3_table" + + # Create tables as default user + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + + # Create test users with specific permissions + node.query("CREATE USER IF NOT EXISTS user_no_alter IDENTIFIED WITH no_password") + node.query("CREATE USER IF NOT EXISTS user_no_insert IDENTIFIED WITH no_password") + node.query("CREATE USER IF NOT EXISTS user_with_permissions IDENTIFIED WITH no_password") + + # Grant basic access to all users + node.query(f"GRANT SELECT ON {mt_table} TO user_no_alter") + node.query(f"GRANT SELECT ON {s3_table} TO user_no_alter") + + # user_no_insert has ALTER on source but no INSERT on destination + node.query(f"GRANT ALTER ON {mt_table} TO user_no_insert") + node.query(f"GRANT SELECT ON {s3_table} TO user_no_insert") + + # user_with_permissions has both ALTER and INSERT + node.query(f"GRANT ALTER ON {mt_table} TO user_with_permissions") + node.query(f"GRANT INSERT ON {s3_table} TO user_with_permissions") + + # Test 1: User without ALTER permission should fail + error = node.query_and_get_error( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table}", + user="user_no_alter" + ) + + assert "ACCESS_DENIED" in error or "Not enough privileges" in error, \ + f"Expected ACCESS_DENIED error for user without ALTER, got: {error}" + + # Test 2: User with ALTER but without INSERT permission should fail + error = node.query_and_get_error( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table}", + user="user_no_insert" + ) + + assert "ACCESS_DENIED" in error or "Not enough privileges" in error, \ + f"Expected ACCESS_DENIED error for user without INSERT, got: {error}" + + # Test 3: User with both ALTER and INSERT should succeed + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table}", + user="user_with_permissions" + ) + + # Wait for export to complete + wait_for_export_status(node, mt_table, s3_table, "2020", "COMPLETED") + + # Verify the export succeeded + result = node.query(f"SELECT count() FROM {s3_table} WHERE year = 2020") + assert result.strip() == "3", f"Expected 3 rows exported, got: {result}" + + # Verify system table shows COMPLETED status + status = node.query( + f""" + SELECT status FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '2020' + """ + ) + assert status.strip() == "COMPLETED", f"Expected COMPLETED status, got: {status}" + + +# assert multiple exports within a single query are executed. They all share the same query id +# and previously the transaction id was the query id, which would cause problems +def test_multiple_exports_within_a_single_query(cluster): + node = cluster.instances["replica1"] + + mt_table = "multiple_exports_within_a_single_query_mt_table" + s3_table = "multiple_exports_within_a_single_query_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + + node.query(f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table}, EXPORT PARTITION ID '2021' TO TABLE {s3_table};") + + wait_for_export_status(node, mt_table, s3_table, "2020", "COMPLETED") + wait_for_export_status(node, mt_table, s3_table, "2021", "COMPLETED") + + # assert the exports have been executed + assert node.query(f"SELECT count() FROM {s3_table} WHERE year = 2020") == '3\n', "Export did not succeed" + assert node.query(f"SELECT count() FROM {s3_table} WHERE year = 2021") == '1\n', "Export did not succeed" + + # check system.replicated_partition_exports for the exports + assert node.query( + f""" + SELECT status FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '2020' + """ + ) == "COMPLETED\n", "Export should be marked as COMPLETED" + + assert node.query( + f""" + SELECT status FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '2021' + """ + ) == "COMPLETED\n", "Export should be marked as COMPLETED" + + +def test_pending_mutations_throw_before_export_partition(cluster): + """Test that pending mutations before export partition throw an error.""" + node = cluster.instances["replica1"] + + mt_table = "pending_mutations_throw_partition_mt_table" + s3_table = "pending_mutations_throw_partition_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + + node.query(f"SYSTEM STOP MERGES {mt_table}") + + node.query(f"ALTER TABLE {mt_table} UPDATE id = id + 100 WHERE year = 2020") + + mutations = node.query(f"SELECT count() FROM system.mutations WHERE table = '{mt_table}' AND is_done = 0") + assert mutations.strip() != '0', "Mutation should be pending" + + error = node.query_and_get_error( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} " + f"SETTINGS export_merge_tree_part_throw_on_pending_mutations=true" + ) + + assert "PENDING_MUTATIONS_NOT_ALLOWED" in error, f"Expected error about pending mutations, got: {error}" + + +def test_pending_mutations_skip_before_export_partition(cluster): + """Test that pending mutations before export partition are skipped with throw_on_pending_mutations=false.""" + node = cluster.instances["replica1"] + + mt_table = "pending_mutations_skip_partition_mt_table" + s3_table = "pending_mutations_skip_partition_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + + node.query(f"SYSTEM STOP MERGES {mt_table}") + + node.query(f"ALTER TABLE {mt_table} UPDATE id = id + 100 WHERE year = 2020") + + mutations = node.query(f"SELECT count() FROM system.mutations WHERE table = '{mt_table}' AND is_done = 0") + assert mutations.strip() != '0', "Mutation should be pending" + + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} " + f"SETTINGS export_merge_tree_part_throw_on_pending_mutations=false" + ) + + wait_for_export_status(node, mt_table, s3_table, "2020", "COMPLETED") + + result = node.query(f"SELECT id FROM {s3_table} WHERE year = 2020 ORDER BY id") + assert "101" not in result and "102" not in result and "103" not in result, \ + "Export should contain original data before mutation" + assert "1\n2\n3" in result, "Export should contain original data" + + +def test_pending_patch_parts_throw_before_export_partition(cluster): + """Test that pending patch parts before export partition throw an error with default settings.""" + node = cluster.instances["replica1"] + + mt_table = "pending_patches_throw_partition_mt_table" + s3_table = "pending_patches_throw_partition_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + + node.query(f"SYSTEM STOP MERGES {mt_table}") + + node.query(f"UPDATE {mt_table} SET id = id + 100 WHERE year = 2020") + + error = node.query_and_get_error( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table}" + ) + + node.query(f"DROP TABLE {mt_table}") + + assert "PENDING_MUTATIONS_NOT_ALLOWED" in error or "pending patch parts" in error.lower(), \ + f"Expected error about pending patch parts, got: {error}" + + +def test_pending_patch_parts_skip_before_export_partition(cluster): + """Test that pending patch parts before export partition are skipped with throw_on_pending_patch_parts=false.""" + node = cluster.instances["replica1"] + + mt_table = "pending_patches_skip_partition_mt_table" + s3_table = "pending_patches_skip_partition_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + + node.query(f"SYSTEM STOP MERGES {mt_table}") + + node.query(f"UPDATE {mt_table} SET id = id + 100 WHERE year = 2020") + + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} " + f"SETTINGS export_merge_tree_part_throw_on_pending_patch_parts=false" + ) + + wait_for_export_status(node, mt_table, s3_table, "2020", "COMPLETED") + + result = node.query(f"SELECT id FROM {s3_table} WHERE year = 2020 ORDER BY id") + assert "1\n2\n3" in result, "Export should contain original data before patch" + + node.query(f"DROP TABLE {mt_table}") + + +def test_mutations_after_export_partition_started(cluster): + """Test that mutations applied after export partition starts don't affect the exported data.""" + node = cluster.instances["replica1"] + + mt_table = "mutations_after_export_partition_mt_table" + s3_table = "mutations_after_export_partition_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + + # Block traffic to MinIO to delay export + minio_ip = cluster.minio_ip + minio_port = cluster.minio_port + + with PartitionManager() as pm: + pm_rule_reject_responses = { + "instance": node, + "destination": node.ip_address, + "protocol": "tcp", + "source_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_responses) + + pm_rule_reject_requests = { + "instance": node, + "destination": minio_ip, + "protocol": "tcp", + "destination_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_requests) + + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} " + f"SETTINGS export_merge_tree_part_throw_on_pending_mutations=true" + ) + + # Wait for export to start + wait_for_export_to_start(node, mt_table, s3_table, "2020") + + node.query(f"ALTER TABLE {mt_table} UPDATE id = id + 100 WHERE year = 2020") + + wait_for_export_status(node, mt_table, s3_table, "2020", "COMPLETED") + + result = node.query(f"SELECT id FROM {s3_table} WHERE year = 2020 ORDER BY id") + assert "1\n2\n3" in result, "Export should contain original data before mutation" + assert "101" not in result, "Export should not contain mutated data" + + +def test_patch_parts_after_export_partition_started(cluster): + """Test that patch parts created after export partition starts don't affect the exported data.""" + node = cluster.instances["replica1"] + + mt_table = "patches_after_export_partition_mt_table" + s3_table = "patches_after_export_partition_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + + # Block traffic to MinIO to delay export + minio_ip = cluster.minio_ip + minio_port = cluster.minio_port + + with PartitionManager() as pm: + pm_rule_reject_responses = { + "instance": node, + "destination": node.ip_address, + "protocol": "tcp", + "source_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_responses) + + pm_rule_reject_requests = { + "instance": node, + "destination": minio_ip, + "protocol": "tcp", + "destination_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + } + pm.add_rule(pm_rule_reject_requests) + + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table}" + ) + + # Wait for export to start + wait_for_export_to_start(node, mt_table, s3_table, "2020") + + node.query(f"UPDATE {mt_table} SET id = id + 100 WHERE year = 2020") + + wait_for_export_status(node, mt_table, s3_table, "2020", "COMPLETED") + + result = node.query(f"SELECT id FROM {s3_table} WHERE year = 2020 ORDER BY id") + assert "1\n2\n3" in result, "Export should contain original data before patch" + assert "101" not in result, "Export should not contain patched data" + + node.query(f"DROP TABLE {mt_table}") + + +def test_mutation_in_partition_clause(cluster): + """Test that mutations limited to specific partitions using IN PARTITION clause + allow exports of unaffected partitions to succeed.""" + node = cluster.instances["replica1"] + + mt_table = "mutation_in_partition_clause_mt_table" + s3_table = "mutation_in_partition_clause_s3_table" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + + node.query(f"SYSTEM STOP MERGES {mt_table}") + + # Issue a mutation that uses IN PARTITION to limit it to partition 2020 + node.query(f"ALTER TABLE {mt_table} UPDATE id = id + 100 IN PARTITION '2020' WHERE year = 2020") + + # Verify mutation is pending for 2020 + mutations = node.query( + f"SELECT count() FROM system.mutations WHERE table = '{mt_table}' AND is_done = 0" + ) + assert mutations.strip() != '0', "Mutation should be pending" + + # Export of 2020 should fail (it has pending mutations) + error = node.query_and_get_error( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} " + f"SETTINGS export_merge_tree_part_throw_on_pending_mutations=true" + ) + assert "PENDING_MUTATIONS_NOT_ALLOWED" in error, f"Expected error about pending mutations for partition 2020, got: {error}" + + # Export of 2021 should succeed (no mutations affecting it) + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2021' TO TABLE {s3_table} " + f"SETTINGS export_merge_tree_part_throw_on_pending_mutations=true" + ) + + wait_for_export_status(node, mt_table, s3_table, "2021", "COMPLETED") + + result = node.query(f"SELECT id FROM {s3_table} WHERE year = 2021 ORDER BY id") + assert "4" in result, "Export of partition 2021 should contain original data" + + +def test_export_partition_with_mixed_computed_columns(cluster): + """Test export partition with ALIAS, MATERIALIZED, and EPHEMERAL columns.""" + node = cluster.instances["replica1"] + + mt_table = "mixed_computed_mt_table" + s3_table = "mixed_computed_s3_table" + + node.query(f""" + CREATE TABLE {mt_table} ( + id UInt32, + value UInt32, + tag_input String EPHEMERAL, + doubled UInt64 ALIAS value * 2, + tripled UInt64 MATERIALIZED value * 3, + tag String DEFAULT upper(tag_input) + ) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{mt_table}', 'replica1') + PARTITION BY id + ORDER BY id + SETTINGS index_granularity = 1 + """) + + # Create S3 destination table with regular columns (no EPHEMERAL) + node.query(f""" + CREATE TABLE {s3_table} ( + id UInt32, + value UInt32, + doubled UInt64, + tripled UInt64, + tag String + ) ENGINE = S3(s3_conn, filename='{s3_table}', format=Parquet, partition_strategy='hive') + PARTITION BY id + """) + + node.query(f"INSERT INTO {mt_table} (id, value, tag_input) VALUES (1, 5, 'test'), (1, 10, 'prod')") + + node.query(f"ALTER TABLE {mt_table} EXPORT PARTITION ID '1' TO TABLE {s3_table}") + + wait_for_export_status(node, mt_table, s3_table, "1", "COMPLETED") + + # Verify source data (ALIAS computed, EPHEMERAL not stored) + source_result = node.query(f"SELECT id, value, doubled, tripled, tag FROM {mt_table} ORDER BY value") + expected = "1\t5\t10\t15\tTEST\n1\t10\t20\t30\tPROD\n" + assert source_result == expected, f"Source table data mismatch. Expected:\n{expected}\nGot:\n{source_result}" + + dest_result = node.query(f"SELECT id, value, doubled, tripled, tag FROM {s3_table} ORDER BY value") + assert dest_result == expected, f"Exported data mismatch. Expected:\n{expected}\nGot:\n{dest_result}" + + status = node.query(f""" + SELECT status FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '1' + """) + assert status.strip() == "COMPLETED", f"Expected COMPLETED status, got: {status}" diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index 253029f72e25..02832536782f 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -44,6 +44,8 @@ ALTER MATERIALIZE TTL ['MATERIALIZE TTL'] TABLE ALTER TABLE ALTER REWRITE PARTS ['REWRITE PARTS'] TABLE ALTER TABLE ALTER SETTINGS ['ALTER SETTING','ALTER MODIFY SETTING','MODIFY SETTING','RESET SETTING'] TABLE ALTER TABLE ALTER MOVE PARTITION ['ALTER MOVE PART','MOVE PARTITION','MOVE PART'] TABLE ALTER TABLE +ALTER EXPORT PART ['ALTER EXPORT PART','EXPORT PART'] TABLE ALTER TABLE +ALTER EXPORT PARTITION ['ALTER EXPORT PARTITION','EXPORT PARTITION'] TABLE ALTER TABLE ALTER FETCH PARTITION ['ALTER FETCH PART','FETCH PARTITION'] TABLE ALTER TABLE ALTER FREEZE PARTITION ['FREEZE PARTITION','UNFREEZE'] TABLE ALTER TABLE ALTER UNLOCK SNAPSHOT ['UNLOCK SNAPSHOT'] TABLE ALTER TABLE diff --git a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.reference b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.reference index bd379621b73d..2a6bc4c98833 100644 --- a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.reference +++ b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.reference @@ -20,6 +20,8 @@ creator_info creator_info deduplication_hashes deduplication_hashes +exports +exports failed_parts failed_parts flags diff --git a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference index b1c76b04526e..db94343ec003 100644 --- a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference +++ b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference @@ -9,6 +9,7 @@ columns columns creator_info deduplication_hashes +exports failed_parts flags host @@ -51,6 +52,7 @@ columns columns creator_info deduplication_hashes +exports failed_parts flags host diff --git a/tests/queries/0_stateless/03413_experimental_settings_cannot_be_enabled_by_default.sql b/tests/queries/0_stateless/03413_experimental_settings_cannot_be_enabled_by_default.sql index 718eb63ad923..809185ed5e3e 100644 --- a/tests/queries/0_stateless/03413_experimental_settings_cannot_be_enabled_by_default.sql +++ b/tests/queries/0_stateless/03413_experimental_settings_cannot_be_enabled_by_default.sql @@ -4,5 +4,5 @@ -- However, some settings in the experimental tier are meant to control another experimental feature, and then they can be enabled as long as the feature itself is disabled. -- These are in the exceptions list inside NOT IN. -SELECT name, value FROM system.settings WHERE tier = 'Experimental' AND type = 'Bool' AND value != '0' AND name NOT IN ('throw_on_unsupported_query_inside_transaction'); +SELECT name, value FROM system.settings WHERE tier = 'Experimental' AND type = 'Bool' AND value != '0' AND name NOT IN ('throw_on_unsupported_query_inside_transaction', 'allow_experimental_export_merge_tree_part'); SELECT name, value FROM system.merge_tree_settings WHERE tier = 'Experimental' AND type = 'Bool' AND value != '0' AND name NOT IN ('remove_rolled_back_parts_immediately'); diff --git a/tests/queries/0_stateless/03572_export_merge_tree_part_basic.reference b/tests/queries/0_stateless/03572_export_merge_tree_part_basic.reference new file mode 100644 index 000000000000..8b08677b5d48 --- /dev/null +++ b/tests/queries/0_stateless/03572_export_merge_tree_part_basic.reference @@ -0,0 +1,35 @@ +---- Export 1: Export 2020_1_1_0 and 2021_2_2_0 +---- Export 2: Export 2022_3_3_0 and 2023_4_4_0 to wildcard table +---- Export 3: Export 2020_1_1_0 and 2021_2_2_0 to wildcard table with partition expression with function +---- Export 4: Export the same part again, it should be idempotent +---- Export 5: Export the same part again to wildcard, it should be idempotent +---- Verify Export 1: Both data parts should appear +1 2020 +2 2020 +3 2020 +4 2021 +---- Verify Export 4: Export the same part again, it should be idempotent +1 2020 +2 2020 +3 2020 +4 2021 +---- Verify: Data in roundtrip MergeTree table (should match s3_table) +1 2020 +2 2020 +3 2020 +4 2021 +---- Verify Export 2: Both data parts should appear (2022_3_3_0 and 2023_4_4_0) +5 2022 +6 2022 +7 2023 +8 2023 +---- Verify Export 5: Export the same part again, it should be idempotent +5 2022 +6 2022 +7 2023 +8 2023 +---- Verify Export 3: Both data parts should appear +1 2020 +2 2020 +3 2020 +4 2021 diff --git a/tests/queries/0_stateless/03572_export_merge_tree_part_basic.sh b/tests/queries/0_stateless/03572_export_merge_tree_part_basic.sh new file mode 100755 index 000000000000..fc5df9b541da --- /dev/null +++ b/tests/queries/0_stateless/03572_export_merge_tree_part_basic.sh @@ -0,0 +1,85 @@ +#!/usr/bin/env bash +# Tags: no-fasttest +# Tag no-fasttest: requires s3 storage + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +mt_table="mt_table_${RANDOM}" +mt_table_partition_expression_with_function="mt_table_partition_expression_with_function_${RANDOM}" +s3_table="s3_table_${RANDOM}" +s3_table_wildcard="s3_table_wildcard_${RANDOM}" +s3_table_wildcard_partition_expression_with_function="s3_table_wildcard_partition_expression_with_function_${RANDOM}" +mt_table_roundtrip="mt_table_roundtrip_${RANDOM}" + +query() { + $CLICKHOUSE_CLIENT --query "$1" +} + +query "DROP TABLE IF EXISTS $mt_table, $s3_table, $mt_table_roundtrip, $s3_table_wildcard, $s3_table_wildcard_partition_expression_with_function, $mt_table_partition_expression_with_function" + +# Create all tables +query "CREATE TABLE $mt_table (id UInt64, year UInt16) ENGINE = MergeTree() PARTITION BY year ORDER BY tuple()" +query "CREATE TABLE $s3_table (id UInt64, year UInt16) ENGINE = S3(s3_conn, filename='$s3_table', format=Parquet, partition_strategy='hive') PARTITION BY year" +query "CREATE TABLE $s3_table_wildcard (id UInt64, year UInt16) ENGINE = S3(s3_conn, filename='$s3_table_wildcard/{_partition_id}/{_file}.parquet', format=Parquet, partition_strategy='wildcard') PARTITION BY year" +query "CREATE TABLE $mt_table_partition_expression_with_function (id UInt64, year UInt16) ENGINE = MergeTree() PARTITION BY toString(year) ORDER BY tuple()" +query "CREATE TABLE $s3_table_wildcard_partition_expression_with_function (id UInt64, year UInt16) ENGINE = S3(s3_conn, filename='$s3_table_wildcard_partition_expression_with_function/{_partition_id}/{_file}.parquet', format=Parquet, partition_strategy='wildcard') PARTITION BY toString(year)" + +# Insert all data +query "INSERT INTO $mt_table VALUES (1, 2020), (2, 2020), (3, 2020), (4, 2021), (5, 2022), (6, 2022), (7, 2023), (8, 2023), (9, 2024), (10, 2024), (11, 2025), (12, 2025)" +query "INSERT INTO $mt_table_partition_expression_with_function VALUES (1, 2020), (2, 2020), (3, 2020), (4, 2021)" + +# ============================================================================ +# ALL EXPORTS HAPPEN HERE +# ============================================================================ + +echo "---- Export 1: Export 2020_1_1_0 and 2021_2_2_0" +query "ALTER TABLE $mt_table EXPORT PART '2020_1_1_0' TO TABLE $s3_table SETTINGS allow_experimental_export_merge_tree_part = 1" +query "ALTER TABLE $mt_table EXPORT PART '2021_2_2_0' TO TABLE $s3_table SETTINGS allow_experimental_export_merge_tree_part = 1" + +echo "---- Export 2: Export 2022_3_3_0 and 2023_4_4_0 to wildcard table" +query "ALTER TABLE $mt_table EXPORT PART '2022_3_3_0' TO TABLE $s3_table_wildcard SETTINGS allow_experimental_export_merge_tree_part = 1" +query "ALTER TABLE $mt_table EXPORT PART '2023_4_4_0' TO TABLE $s3_table_wildcard SETTINGS allow_experimental_export_merge_tree_part = 1" + +echo "---- Export 3: Export 2020_1_1_0 and 2021_2_2_0 to wildcard table with partition expression with function" +query "ALTER TABLE $mt_table_partition_expression_with_function EXPORT PART 'cb217c742dc7d143b61583011996a160_1_1_0' TO TABLE $s3_table_wildcard_partition_expression_with_function SETTINGS allow_experimental_export_merge_tree_part = 1" +query "ALTER TABLE $mt_table_partition_expression_with_function EXPORT PART '3be6d49ecf9749a383964bc6fab22d10_2_2_0' TO TABLE $s3_table_wildcard_partition_expression_with_function SETTINGS allow_experimental_export_merge_tree_part = 1" + +# below exports are using parts that were exported in export 1 and export 2, so we need to wait for them to complete +sleep 5 + +echo "---- Export 4: Export the same part again, it should be idempotent" +query "ALTER TABLE $mt_table EXPORT PART '2020_1_1_0' TO TABLE $s3_table SETTINGS allow_experimental_export_merge_tree_part = 1" + +echo "---- Export 5: Export the same part again to wildcard, it should be idempotent" +query "ALTER TABLE $mt_table EXPORT PART '2022_3_3_0' TO TABLE $s3_table_wildcard SETTINGS allow_experimental_export_merge_tree_part = 1" + +# ONE BIG SLEEP after all exports +sleep 15 + +# ============================================================================ +# ALL SELECTS/VERIFICATIONS HAPPEN HERE +# ============================================================================ + +echo "---- Verify Export 1: Both data parts should appear" +query "SELECT * FROM $s3_table ORDER BY id" + +echo "---- Verify Export 4: Export the same part again, it should be idempotent" +query "SELECT * FROM $s3_table ORDER BY id" + +query "CREATE TABLE $mt_table_roundtrip ENGINE = MergeTree() PARTITION BY year ORDER BY tuple() AS SELECT * FROM $s3_table" + +echo "---- Verify: Data in roundtrip MergeTree table (should match s3_table)" +query "SELECT * FROM $mt_table_roundtrip ORDER BY id" + +echo "---- Verify Export 2: Both data parts should appear (2022_3_3_0 and 2023_4_4_0)" +query "SELECT * FROM s3(s3_conn, filename='$s3_table_wildcard/**.parquet') ORDER BY id" + +echo "---- Verify Export 5: Export the same part again, it should be idempotent" +query "SELECT * FROM s3(s3_conn, filename='$s3_table_wildcard/**.parquet') ORDER BY id" + +echo "---- Verify Export 3: Both data parts should appear" +query "SELECT * FROM s3(s3_conn, filename='$s3_table_wildcard_partition_expression_with_function/**.parquet') ORDER BY id" + +query "DROP TABLE IF EXISTS $mt_table, $s3_table, $mt_table_roundtrip, $s3_table_wildcard, $s3_table_wildcard_partition_expression_with_function, $mt_table_partition_expression_with_function" diff --git a/tests/queries/0_stateless/03572_export_merge_tree_part_limits_and_table_functions.reference b/tests/queries/0_stateless/03572_export_merge_tree_part_limits_and_table_functions.reference new file mode 100644 index 000000000000..ce4f112ad1fa --- /dev/null +++ b/tests/queries/0_stateless/03572_export_merge_tree_part_limits_and_table_functions.reference @@ -0,0 +1,18 @@ +---- Test max_bytes and max_rows per file +---- Table function with schema inheritance (no schema specified) +---- Table function with explicit compatible schema +---- Count files in big_destination_max_bytes, should be 5 (4 parquet, 1 commit) +5 +---- Count rows in big_table and big_destination_max_bytes +4194304 +4194304 +---- Count files in big_destination_max_rows, should be 5 (4 parquet, 1 commit) +5 +---- Count rows in big_table and big_destination_max_rows +4194304 +4194304 +---- Data should be exported with inherited schema +100 test1 2022 +101 test2 2022 +---- Data should be exported with explicit schema +102 test3 2023 diff --git a/tests/queries/0_stateless/03572_export_merge_tree_part_limits_and_table_functions.sh b/tests/queries/0_stateless/03572_export_merge_tree_part_limits_and_table_functions.sh new file mode 100755 index 000000000000..449720bbf7a3 --- /dev/null +++ b/tests/queries/0_stateless/03572_export_merge_tree_part_limits_and_table_functions.sh @@ -0,0 +1,85 @@ +#!/usr/bin/env bash +# Tags: no-fasttest +# Tag no-fasttest: requires s3 storage + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +big_table="big_table_${RANDOM}" +big_destination_max_bytes="big_destination_max_bytes_${RANDOM}" +big_destination_max_rows="big_destination_max_rows_${RANDOM}" +tf_schema_inherit="tf_schema_inherit_${RANDOM}" +tf_schema_explicit="tf_schema_explicit_${RANDOM}" +mt_table_tf="mt_table_tf_${RANDOM}" + +query() { + $CLICKHOUSE_CLIENT --query "$1" +} + +query "DROP TABLE IF EXISTS $big_table, $big_destination_max_bytes, $big_destination_max_rows, $mt_table_tf" + +echo "---- Test max_bytes and max_rows per file" + +# Create all tables +query "CREATE TABLE $big_table (id UInt64, data String, year UInt16) Engine=MergeTree() order by id partition by year" +query "CREATE TABLE $big_destination_max_bytes(id UInt64, data String, year UInt16) engine=S3(s3_conn, filename='$big_destination_max_bytes', partition_strategy='hive', format=Parquet) partition by year" +query "CREATE TABLE $big_destination_max_rows(id UInt64, data String, year UInt16) engine=S3(s3_conn, filename='$big_destination_max_rows', partition_strategy='hive', format=Parquet) partition by year" +query "CREATE TABLE $mt_table_tf (id UInt64, value String, year UInt16) ENGINE = MergeTree() PARTITION BY year ORDER BY tuple()" + +# Insert all data +# 4194304 is a number that came up during multiple iterations, it does not really mean anything (aside from the fact that the below numbers depend on it) +query "INSERT INTO $big_table SELECT number AS id, repeat('x', 100) AS data, 2025 AS year FROM numbers(4194304)" +query "INSERT INTO $big_table SELECT number AS id, repeat('x', 100) AS data, 2026 AS year FROM numbers(4194304)" +query "INSERT INTO $mt_table_tf VALUES (100, 'test1', 2022), (101, 'test2', 2022), (102, 'test3', 2023)" + +# make sure we have only one part +query "OPTIMIZE TABLE $big_table FINAL" + +# Get part names +big_part_max_bytes=$(query "SELECT name FROM system.parts WHERE database = currentDatabase() AND table = '$big_table' AND partition_id = '2025' AND active = 1 ORDER BY name LIMIT 1" | tr -d '\n') +big_part_max_rows=$(query "SELECT name FROM system.parts WHERE database = currentDatabase() AND table = '$big_table' AND partition_id = '2026' AND active = 1 ORDER BY name LIMIT 1" | tr -d '\n') + +# ============================================================================ +# ALL EXPORTS HAPPEN HERE +# ============================================================================ + +# this should generate ~4 files +query "ALTER TABLE $big_table EXPORT PART '$big_part_max_bytes' TO TABLE $big_destination_max_bytes SETTINGS allow_experimental_export_merge_tree_part = 1, export_merge_tree_part_max_bytes_per_file=3500000, output_format_parquet_row_group_size_bytes=1000000" +# export_merge_tree_part_max_rows_per_file = 1048576 (which is 4194304/4) to generate 4 files +query "ALTER TABLE $big_table EXPORT PART '$big_part_max_rows' TO TABLE $big_destination_max_rows SETTINGS allow_experimental_export_merge_tree_part = 1, export_merge_tree_part_max_rows_per_file=1048576" + +echo "---- Table function with schema inheritance (no schema specified)" +query "ALTER TABLE $mt_table_tf EXPORT PART '2022_1_1_0' TO TABLE FUNCTION s3(s3_conn, filename='$tf_schema_inherit', format='Parquet', partition_strategy='hive') PARTITION BY year SETTINGS allow_experimental_export_merge_tree_part = 1" + +echo "---- Table function with explicit compatible schema" +query "ALTER TABLE $mt_table_tf EXPORT PART '2023_2_2_0' TO TABLE FUNCTION s3(s3_conn, filename='$tf_schema_explicit', format='Parquet', structure='id UInt64, value String, year UInt16', partition_strategy='hive') PARTITION BY year SETTINGS allow_experimental_export_merge_tree_part = 1" + +# ONE BIG SLEEP after all exports (longer because it writes multiple files) +sleep 20 + +# ============================================================================ +# ALL SELECTS/VERIFICATIONS HAPPEN HERE +# ============================================================================ + +echo "---- Count files in big_destination_max_bytes, should be 5 (4 parquet, 1 commit)" +query "SELECT count(_file) FROM s3(s3_conn, filename='$big_destination_max_bytes/**', format='One')" + +echo "---- Count rows in big_table and big_destination_max_bytes" +query "SELECT COUNT() from $big_table WHERE year = 2025" +query "SELECT COUNT() from $big_destination_max_bytes" + +echo "---- Count files in big_destination_max_rows, should be 5 (4 parquet, 1 commit)" +query "SELECT count(_file) FROM s3(s3_conn, filename='$big_destination_max_rows/**', format='One')" + +echo "---- Count rows in big_table and big_destination_max_rows" +query "SELECT COUNT() from $big_table WHERE year = 2026" +query "SELECT COUNT() from $big_destination_max_rows" + +echo "---- Data should be exported with inherited schema" +query "SELECT * FROM s3(s3_conn, filename='$tf_schema_inherit/**.parquet') ORDER BY id" + +echo "---- Data should be exported with explicit schema" +query "SELECT * FROM s3(s3_conn, filename='$tf_schema_explicit/**.parquet') ORDER BY id" + +query "DROP TABLE IF EXISTS $big_table, $big_destination_max_bytes, $big_destination_max_rows, $mt_table_tf" diff --git a/tests/queries/0_stateless/03572_export_merge_tree_part_special_columns.reference b/tests/queries/0_stateless/03572_export_merge_tree_part_special_columns.reference new file mode 100644 index 000000000000..14bcbb452591 --- /dev/null +++ b/tests/queries/0_stateless/03572_export_merge_tree_part_special_columns.reference @@ -0,0 +1,39 @@ +---- Test ALIAS columns export +---- Test MATERIALIZED columns export +---- Test EPHEMERAL column (not stored, ignored during export) +---- Test Mixed ALIAS, MATERIALIZED, and EPHEMERAL in same table +---- Test Complex Expressions in computed columns +---- Test Export to Table Function with mixed columns +---- Verify ALIAS column data in source table (arr_1 computed from arr[1]) +1 [1,2,3] 1 +1 [10,20,30] 10 +---- Verify ALIAS column data exported to S3 (should match source) +1 [1,2,3] 1 +1 [10,20,30] 10 +---- Verify MATERIALIZED column data in source table (arr_1 computed from arr[1]) +1 [1,2,3] 1 +1 [10,20,30] 10 +---- Verify MATERIALIZED column data exported to S3 (should match source) +1 [1,2,3] 1 +1 [10,20,30] 10 +---- Verify data in source +1 ALICE +1 BOB +---- Verify exported data +1 ALICE +1 BOB +---- Verify mixed columns in source table +1 5 10 15 TEST +1 10 20 30 PROD +2 15 30 45 DEV +---- Verify mixed columns exported to S3 +1 5 10 15 TEST +1 10 20 30 PROD +---- Verify mixed columns exported to S3 +2 15 30 45 DEV +---- Verify complex expressions in source table +1 alice ALICE alice-1 +1 bob BOB bob-1 +---- Verify complex expressions exported to S3 (should match source) +1 alice ALICE alice-1 +1 bob BOB bob-1 diff --git a/tests/queries/0_stateless/03572_export_merge_tree_part_special_columns.sh b/tests/queries/0_stateless/03572_export_merge_tree_part_special_columns.sh new file mode 100755 index 000000000000..0164dd70c4e0 --- /dev/null +++ b/tests/queries/0_stateless/03572_export_merge_tree_part_special_columns.sh @@ -0,0 +1,154 @@ +#!/usr/bin/env bash +# Tags: no-fasttest +# Tag no-fasttest: requires s3 storage + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +mt_alias="mt_alias_${RANDOM}" +mt_materialized="mt_materialized_${RANDOM}" +s3_alias_export="s3_alias_export_${RANDOM}" +s3_materialized_export="s3_materialized_export_${RANDOM}" +mt_mixed="mt_mixed_${RANDOM}" +s3_mixed_export="s3_mixed_export_${RANDOM}" +mt_complex_expr="mt_complex_expr_${RANDOM}" +s3_complex_expr_export="s3_complex_expr_export_${RANDOM}" +mt_ephemeral="mt_ephemeral_${RANDOM}" +s3_ephemeral_export="s3_ephemeral_export_${RANDOM}" +s3_mixed_export_table_function="s3_mixed_export_table_function_${RANDOM}" + +query() { + $CLICKHOUSE_CLIENT --query "$1" +} + +query "DROP TABLE IF EXISTS $mt_alias, $mt_materialized, $s3_alias_export, $s3_materialized_export, $mt_mixed, $s3_mixed_export, $mt_complex_expr, $s3_complex_expr_export, $mt_ephemeral, $s3_ephemeral_export" + +# Create all tables +echo "---- Test ALIAS columns export" +query "CREATE TABLE $mt_alias (a UInt32, arr Array(UInt64), arr_1 UInt64 ALIAS arr[1]) ENGINE = MergeTree() PARTITION BY a ORDER BY (a, arr[1]) SETTINGS index_granularity = 1" +query "CREATE TABLE $s3_alias_export (a UInt32, arr Array(UInt64), arr_1 UInt64) ENGINE = S3(s3_conn, filename='$s3_alias_export', format=Parquet, partition_strategy='hive') PARTITION BY a" + +echo "---- Test MATERIALIZED columns export" +query "CREATE TABLE $mt_materialized (a UInt32, arr Array(UInt64), arr_1 UInt64 MATERIALIZED arr[1]) ENGINE = MergeTree() PARTITION BY a ORDER BY (a, arr_1) SETTINGS index_granularity = 1" +query "CREATE TABLE $s3_materialized_export (a UInt32, arr Array(UInt64), arr_1 UInt64) ENGINE = S3(s3_conn, filename='$s3_materialized_export', format=Parquet, partition_strategy='hive') PARTITION BY a" + +echo "---- Test EPHEMERAL column (not stored, ignored during export)" +query "CREATE TABLE $mt_ephemeral ( + id UInt32, + name_input String EPHEMERAL, + name_upper String DEFAULT upper(name_input) +) ENGINE = MergeTree() PARTITION BY id ORDER BY id SETTINGS index_granularity = 1" + +query "CREATE TABLE $s3_ephemeral_export ( + id UInt32, + name_upper String +) ENGINE = S3(s3_conn, filename='$s3_ephemeral_export', format=Parquet, partition_strategy='hive') PARTITION BY id" + +echo "---- Test Mixed ALIAS, MATERIALIZED, and EPHEMERAL in same table" +query "CREATE TABLE $mt_mixed ( + id UInt32, + value UInt32, + tag_input String EPHEMERAL, + doubled UInt64 ALIAS value * 2, + tripled UInt64 MATERIALIZED value * 3, + tag String DEFAULT upper(tag_input) +) ENGINE = MergeTree() PARTITION BY id ORDER BY id SETTINGS index_granularity = 1" + +query "CREATE TABLE $s3_mixed_export ( + id UInt32, + value UInt32, + doubled UInt64, + tripled UInt64, + tag String +) ENGINE = S3(s3_conn, filename='$s3_mixed_export', format=Parquet, partition_strategy='hive') PARTITION BY id" + +echo "---- Test Complex Expressions in computed columns" +query "CREATE TABLE $mt_complex_expr ( + id UInt32, + name String, + upper_name String ALIAS upper(name), + concat_result String MATERIALIZED concat(name, '-', toString(id)) +) ENGINE = MergeTree() PARTITION BY id ORDER BY id SETTINGS index_granularity = 1" + +query "CREATE TABLE $s3_complex_expr_export ( + id UInt32, + name String, + upper_name String, + concat_result String +) ENGINE = S3(s3_conn, filename='$s3_complex_expr_export', format=Parquet, partition_strategy='hive') PARTITION BY id" + +# Insert all data +query "INSERT INTO $mt_alias VALUES (1, [1, 2, 3]), (1, [10, 20, 30])" +query "INSERT INTO $mt_materialized VALUES (1, [1, 2, 3]), (1, [10, 20, 30])" +query "INSERT INTO $mt_ephemeral (id, name_input) VALUES (1, 'alice'), (1, 'bob')" +query "INSERT INTO $mt_mixed (id, value, tag_input) VALUES (1, 5, 'test'), (1, 10, 'prod')" +query "INSERT INTO $mt_mixed (id, value, tag_input) VALUES (2, 15, 'dev')" +query "INSERT INTO $mt_complex_expr (id, name) VALUES (1, 'alice'), (1, 'bob')" + +# Get all part names +alias_part=$(query "SELECT name FROM system.parts WHERE database = currentDatabase() AND table = '$mt_alias' AND partition_id = '1' AND active = 1 ORDER BY name LIMIT 1" | tr -d '\n') +materialized_part=$(query "SELECT name FROM system.parts WHERE database = currentDatabase() AND table = '$mt_materialized' AND partition_id = '1' AND active = 1 ORDER BY name LIMIT 1" | tr -d '\n') +ephemeral_part=$(query "SELECT name FROM system.parts WHERE database = currentDatabase() AND table = '$mt_ephemeral' AND partition_id = '1' AND active = 1 ORDER BY name LIMIT 1" | tr -d '\n') +mixed_part=$(query "SELECT name FROM system.parts WHERE database = currentDatabase() AND table = '$mt_mixed' AND partition_id = '1' AND active = 1 ORDER BY name LIMIT 1" | tr -d '\n') +mixed_part_2=$(query "SELECT name FROM system.parts WHERE database = currentDatabase() AND table = '$mt_mixed' AND partition_id = '2' AND active = 1 ORDER BY name LIMIT 1" | tr -d '\n') +complex_expr_part=$(query "SELECT name FROM system.parts WHERE database = currentDatabase() AND table = '$mt_complex_expr' AND partition_id = '1' AND active = 1 ORDER BY name LIMIT 1" | tr -d '\n') + +# ============================================================================ +# ALL EXPORTS HAPPEN HERE +# ============================================================================ + +query "ALTER TABLE $mt_alias EXPORT PART '$alias_part' TO TABLE $s3_alias_export SETTINGS allow_experimental_export_merge_tree_part = 1" + +query "ALTER TABLE $mt_materialized EXPORT PART '$materialized_part' TO TABLE $s3_materialized_export SETTINGS allow_experimental_export_merge_tree_part = 1" + +query "ALTER TABLE $mt_ephemeral EXPORT PART '$ephemeral_part' TO TABLE $s3_ephemeral_export SETTINGS allow_experimental_export_merge_tree_part = 1" + +query "ALTER TABLE $mt_mixed EXPORT PART '$mixed_part' TO TABLE $s3_mixed_export SETTINGS allow_experimental_export_merge_tree_part = 1" + +echo "---- Test Export to Table Function with mixed columns" +query "ALTER TABLE $mt_mixed EXPORT PART '$mixed_part_2' TO TABLE FUNCTION s3(s3_conn, filename='$s3_mixed_export_table_function', format=Parquet, partition_strategy='hive') PARTITION BY id SETTINGS allow_experimental_export_merge_tree_part = 1" + +query "ALTER TABLE $mt_complex_expr EXPORT PART '$complex_expr_part' TO TABLE $s3_complex_expr_export SETTINGS allow_experimental_export_merge_tree_part = 1" + +# ONE BIG SLEEP after all exports +sleep 20 + +# ============================================================================ +# ALL SELECTS/VERIFICATIONS HAPPEN HERE +# ============================================================================ + +echo "---- Verify ALIAS column data in source table (arr_1 computed from arr[1])" +query "SELECT a, arr, arr_1 FROM $mt_alias ORDER BY arr" + +echo "---- Verify ALIAS column data exported to S3 (should match source)" +query "SELECT a, arr, arr_1 FROM $s3_alias_export ORDER BY arr" + +echo "---- Verify MATERIALIZED column data in source table (arr_1 computed from arr[1])" +query "SELECT a, arr, arr_1 FROM $mt_materialized ORDER BY arr" + +echo "---- Verify MATERIALIZED column data exported to S3 (should match source)" +query "SELECT a, arr, arr_1 FROM $s3_materialized_export ORDER BY arr" + +echo "---- Verify data in source" +query "SELECT id, name_upper FROM $mt_ephemeral ORDER BY name_upper" + +echo "---- Verify exported data" +query "SELECT id, name_upper FROM $s3_ephemeral_export ORDER BY name_upper" + +echo "---- Verify mixed columns in source table" +query "SELECT id, value, doubled, tripled, tag FROM $mt_mixed ORDER BY value" + +echo "---- Verify mixed columns exported to S3" +query "SELECT id, value, doubled, tripled, tag FROM $s3_mixed_export ORDER BY value" + +echo "---- Verify mixed columns exported to S3" +query "SELECT * FROM s3(s3_conn, filename='$s3_mixed_export_table_function/**.parquet', format=Parquet) ORDER BY value" + +echo "---- Verify complex expressions in source table" +query "SELECT id, name, upper_name, concat_result FROM $mt_complex_expr ORDER BY name" + +echo "---- Verify complex expressions exported to S3 (should match source)" +query "SELECT id, name, upper_name, concat_result FROM $s3_complex_expr_export ORDER BY name" + +query "DROP TABLE IF EXISTS $mt_alias, $mt_materialized, $s3_alias_export, $s3_materialized_export, $mt_ephemeral, $s3_ephemeral_export, $mt_mixed, $s3_mixed_export, $mt_complex_expr, $s3_complex_expr_export" diff --git a/tests/queries/0_stateless/03572_export_merge_tree_part_to_object_storage_simple.reference b/tests/queries/0_stateless/03572_export_merge_tree_part_to_object_storage_simple.reference new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/queries/0_stateless/03572_export_merge_tree_part_to_object_storage_simple.sql b/tests/queries/0_stateless/03572_export_merge_tree_part_to_object_storage_simple.sql new file mode 100644 index 000000000000..7cb70af024a2 --- /dev/null +++ b/tests/queries/0_stateless/03572_export_merge_tree_part_to_object_storage_simple.sql @@ -0,0 +1,39 @@ +-- Tags: no-parallel, no-fasttest + +DROP TABLE IF EXISTS 03572_mt_table, 03572_invalid_schema_table, 03572_ephemeral_mt_table, 03572_matching_ephemeral_s3_table; + +SET allow_experimental_export_merge_tree_part=1; + +CREATE TABLE 03572_mt_table (id UInt64, year UInt16) ENGINE = MergeTree() PARTITION BY year ORDER BY tuple(); + +INSERT INTO 03572_mt_table VALUES (1, 2020); + +-- Create a table with a different partition key and export a partition to it. It should throw +CREATE TABLE 03572_invalid_schema_table (id UInt64, x UInt16) ENGINE = S3(s3_conn, filename='03572_invalid_schema_table', format='Parquet', partition_strategy='hive') PARTITION BY x; + +ALTER TABLE 03572_mt_table EXPORT PART '2020_1_1_0' TO TABLE 03572_invalid_schema_table +SETTINGS allow_experimental_export_merge_tree_part = 1; -- {serverError INCOMPATIBLE_COLUMNS} + +DROP TABLE 03572_invalid_schema_table; + +-- The only partition strategy that supports exports is hive. Wildcard should throw +CREATE TABLE 03572_invalid_schema_table (id UInt64, year UInt16) ENGINE = S3(s3_conn, filename='03572_invalid_schema_table/{_partition_id}', format='Parquet', partition_strategy='wildcard') PARTITION BY (id, year); + +ALTER TABLE 03572_mt_table EXPORT PART '2020_1_1_0' TO TABLE 03572_invalid_schema_table; -- {serverError NOT_IMPLEMENTED} + +-- Not a table function, should throw +ALTER TABLE 03572_mt_table EXPORT PART '2020_1_1_0' TO TABLE FUNCTION extractKeyValuePairs('name:ronaldo'); -- {serverError UNKNOWN_FUNCTION} + +-- It is a table function, but the engine does not support exports/imports, should throw +ALTER TABLE 03572_mt_table EXPORT PART '2020_1_1_0' TO TABLE FUNCTION url('a.parquet'); -- {serverError NOT_IMPLEMENTED} + +-- Test that destination table can not have a column that matches the source ephemeral +CREATE TABLE 03572_ephemeral_mt_table (id UInt64, year UInt16, name String EPHEMERAL) ENGINE = MergeTree() PARTITION BY year ORDER BY tuple(); + +CREATE TABLE 03572_matching_ephemeral_s3_table (id UInt64, year UInt16, name String) ENGINE = S3(s3_conn, filename='03572_matching_ephemeral_s3_table', format='Parquet', partition_strategy='hive') PARTITION BY year; + +INSERT INTO 03572_ephemeral_mt_table (id, year, name) VALUES (1, 2020, 'alice'); + +ALTER TABLE 03572_ephemeral_mt_table EXPORT PART '2020_1_1_0' TO TABLE 03572_matching_ephemeral_s3_table; -- {serverError INCOMPATIBLE_COLUMNS} + +DROP TABLE IF EXISTS 03572_mt_table, 03572_invalid_schema_table, 03572_ephemeral_mt_table, 03572_matching_ephemeral_s3_table; diff --git a/tests/queries/0_stateless/03572_export_replicated_merge_tree_part_to_object_storage.reference b/tests/queries/0_stateless/03572_export_replicated_merge_tree_part_to_object_storage.reference new file mode 100644 index 000000000000..07f1ec6376a6 --- /dev/null +++ b/tests/queries/0_stateless/03572_export_replicated_merge_tree_part_to_object_storage.reference @@ -0,0 +1,16 @@ +---- Get actual part names and export them +---- Both data parts should appear +1 2020 +2 2020 +3 2020 +4 2021 +---- Export the same part again, it should be idempotent +1 2020 +2 2020 +3 2020 +4 2021 +---- Data in roundtrip ReplicatedMergeTree table (should match s3_table) +1 2020 +2 2020 +3 2020 +4 2021 diff --git a/tests/queries/0_stateless/03572_export_replicated_merge_tree_part_to_object_storage.sh b/tests/queries/0_stateless/03572_export_replicated_merge_tree_part_to_object_storage.sh new file mode 100755 index 000000000000..a691d4bdf37a --- /dev/null +++ b/tests/queries/0_stateless/03572_export_replicated_merge_tree_part_to_object_storage.sh @@ -0,0 +1,44 @@ +#!/usr/bin/env bash +# Tags: replica, no-parallel, no-replicated-database, no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +rmt_table="rmt_table_${RANDOM}" +s3_table="s3_table_${RANDOM}" +rmt_table_roundtrip="rmt_table_roundtrip_${RANDOM}" + +query() { + $CLICKHOUSE_CLIENT --query "$1" +} + +query "DROP TABLE IF EXISTS $rmt_table, $s3_table, $rmt_table_roundtrip" + +query "CREATE TABLE $rmt_table (id UInt64, year UInt16) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/$rmt_table', 'replica1') PARTITION BY year ORDER BY tuple()" +query "CREATE TABLE $s3_table (id UInt64, year UInt16) ENGINE = S3(s3_conn, filename='$s3_table', format=Parquet, partition_strategy='hive') PARTITION BY year" + +query "INSERT INTO $rmt_table VALUES (1, 2020), (2, 2020), (3, 2020), (4, 2021)" + +echo "---- Get actual part names and export them" +part_2020=$(query "SELECT name FROM system.parts WHERE database = currentDatabase() AND table = '$rmt_table' AND partition = '2020' ORDER BY name LIMIT 1" | tr -d '\n') +part_2021=$(query "SELECT name FROM system.parts WHERE database = currentDatabase() AND table = '$rmt_table' AND partition = '2021' ORDER BY name LIMIT 1" | tr -d '\n') + +query "ALTER TABLE $rmt_table EXPORT PART '$part_2020' TO TABLE $s3_table SETTINGS allow_experimental_export_merge_tree_part = 1" +query "ALTER TABLE $rmt_table EXPORT PART '$part_2021' TO TABLE $s3_table SETTINGS allow_experimental_export_merge_tree_part = 1" + +echo "---- Both data parts should appear" +query "SELECT * FROM $s3_table ORDER BY id" + +echo "---- Export the same part again, it should be idempotent" +query "ALTER TABLE $rmt_table EXPORT PART '$part_2020' TO TABLE $s3_table SETTINGS allow_experimental_export_merge_tree_part = 1" + +query "SELECT * FROM $s3_table ORDER BY id" + +query "CREATE TABLE $rmt_table_roundtrip (id UInt64, year UInt16) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/$rmt_table_roundtrip', 'replica1') PARTITION BY year ORDER BY tuple()" +query "INSERT INTO $rmt_table_roundtrip SELECT * FROM $s3_table" + +echo "---- Data in roundtrip ReplicatedMergeTree table (should match s3_table)" +query "SELECT * FROM $rmt_table_roundtrip ORDER BY id" + +query "DROP TABLE IF EXISTS $rmt_table, $s3_table, $rmt_table_roundtrip" diff --git a/tests/queries/0_stateless/03572_export_replicated_merge_tree_part_to_object_storage_simple.reference b/tests/queries/0_stateless/03572_export_replicated_merge_tree_part_to_object_storage_simple.reference new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/queries/0_stateless/03572_export_replicated_merge_tree_part_to_object_storage_simple.sql b/tests/queries/0_stateless/03572_export_replicated_merge_tree_part_to_object_storage_simple.sql new file mode 100644 index 000000000000..f8f23532f0a7 --- /dev/null +++ b/tests/queries/0_stateless/03572_export_replicated_merge_tree_part_to_object_storage_simple.sql @@ -0,0 +1,22 @@ +-- Tags: no-parallel, no-fasttest + +DROP TABLE IF EXISTS 03572_rmt_table, 03572_invalid_schema_table; + +CREATE TABLE 03572_rmt_table (id UInt64, year UInt16) ENGINE = ReplicatedMergeTree('/clickhouse/{database}/test_03572_rmt/03572_rmt_table', 'replica1') PARTITION BY year ORDER BY tuple(); + +INSERT INTO 03572_rmt_table VALUES (1, 2020); + +-- Create a table with a different partition key and export a partition to it. It should throw +CREATE TABLE 03572_invalid_schema_table (id UInt64, x UInt16) ENGINE = S3(s3_conn, filename='03572_invalid_schema_table', format='Parquet', partition_strategy='hive') PARTITION BY x; + +ALTER TABLE 03572_rmt_table EXPORT PART '2020_0_0_0' TO TABLE 03572_invalid_schema_table +SETTINGS allow_experimental_export_merge_tree_part = 1; -- {serverError INCOMPATIBLE_COLUMNS} + +DROP TABLE 03572_invalid_schema_table; + +-- The only partition strategy that supports exports is hive. Wildcard should throw +CREATE TABLE 03572_invalid_schema_table (id UInt64, year UInt16) ENGINE = S3(s3_conn, filename='03572_invalid_schema_table/{_partition_id}', format='Parquet', partition_strategy='wildcard') PARTITION BY (id, year); + +ALTER TABLE 03572_rmt_table EXPORT PART '2020_0_0_0' TO TABLE 03572_invalid_schema_table SETTINGS allow_experimental_export_merge_tree_part = 1; -- {serverError NOT_IMPLEMENTED} + +DROP TABLE IF EXISTS 03572_rmt_table, 03572_invalid_schema_table; diff --git a/tests/queries/0_stateless/03604_export_merge_tree_partition.reference b/tests/queries/0_stateless/03604_export_merge_tree_partition.reference new file mode 100644 index 000000000000..d48023362b99 --- /dev/null +++ b/tests/queries/0_stateless/03604_export_merge_tree_partition.reference @@ -0,0 +1,31 @@ +Select from source table +1 2020 +2 2020 +3 2020 +4 2021 +5 2021 +6 2022 +7 2022 +Select from destination table +1 2020 +2 2020 +3 2020 +4 2021 +5 2021 +Export partition 2022 +Select from destination table again +1 2020 +2 2020 +3 2020 +4 2021 +5 2021 +6 2022 +7 2022 +---- Data in roundtrip ReplicatedMergeTree table (should match s3_table) +1 2020 +2 2020 +3 2020 +4 2021 +5 2021 +6 2022 +7 2022 diff --git a/tests/queries/0_stateless/03604_export_merge_tree_partition.sh b/tests/queries/0_stateless/03604_export_merge_tree_partition.sh new file mode 100755 index 000000000000..87503112aadb --- /dev/null +++ b/tests/queries/0_stateless/03604_export_merge_tree_partition.sh @@ -0,0 +1,57 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, replica, no-parallel, no-replicated-database + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +rmt_table="rmt_table_${RANDOM}" +s3_table="s3_table_${RANDOM}" +rmt_table_roundtrip="rmt_table_roundtrip_${RANDOM}" + +query() { + $CLICKHOUSE_CLIENT --query "$1" +} + +query "DROP TABLE IF EXISTS $rmt_table, $s3_table, $rmt_table_roundtrip" + +query "CREATE TABLE $rmt_table (id UInt64, year UInt16) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/$rmt_table', 'replica1') PARTITION BY year ORDER BY tuple()" +query "CREATE TABLE $s3_table (id UInt64, year UInt16) ENGINE = S3(s3_conn, filename='$s3_table', format=Parquet, partition_strategy='hive') PARTITION BY year" + +query "INSERT INTO $rmt_table VALUES (1, 2020), (2, 2020), (4, 2021)" + +query "INSERT INTO $rmt_table VALUES (3, 2020), (5, 2021)" + +query "INSERT INTO $rmt_table VALUES (6, 2022), (7, 2022)" + +# sync replicas +query "SYSTEM SYNC REPLICA $rmt_table" + +query "ALTER TABLE $rmt_table EXPORT PARTITION ID '2020' TO TABLE $s3_table SETTINGS allow_experimental_export_merge_tree_part = 1" + +query "ALTER TABLE $rmt_table EXPORT PARTITION ID '2021' TO TABLE $s3_table SETTINGS allow_experimental_export_merge_tree_part = 1" + +# todo poll some kind of status +sleep 15 + +echo "Select from source table" +query "SELECT * FROM $rmt_table ORDER BY id" + +echo "Select from destination table" +query "SELECT * FROM $s3_table ORDER BY id" + +echo "Export partition 2022" +query "ALTER TABLE $rmt_table EXPORT PARTITION ID '2022' TO TABLE $s3_table SETTINGS allow_experimental_export_merge_tree_part = 1" + +# todo poll some kind of status +sleep 5 + +echo "Select from destination table again" +query "SELECT * FROM $s3_table ORDER BY id" + +query "CREATE TABLE $rmt_table_roundtrip ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/$rmt_table_roundtrip', 'replica1') PARTITION BY year ORDER BY tuple() AS SELECT * FROM $s3_table" + +echo "---- Data in roundtrip ReplicatedMergeTree table (should match s3_table)" +query "SELECT * FROM $rmt_table_roundtrip ORDER BY id" + +query "DROP TABLE IF EXISTS $rmt_table, $s3_table, $rmt_table_roundtrip" \ No newline at end of file diff --git a/tests/queries/0_stateless/03745_system_background_schedule_pool.reference b/tests/queries/0_stateless/03745_system_background_schedule_pool.reference index 91958bf0cb1e..dafcbe489b19 100644 --- a/tests/queries/0_stateless/03745_system_background_schedule_pool.reference +++ b/tests/queries/0_stateless/03745_system_background_schedule_pool.reference @@ -1,5 +1,6 @@ 1 buffer_flush default test_buffer_03745 1 StorageBuffer (default.test_buffer_03745)/Bg schedule default test_merge_tree_03745 1 BackgroundJobsAssignee:DataProcessing +schedule default test_merge_tree_03745 1 BackgroundJobsAssignee:Moving schedule default test_merge_tree_03745 1 default.test_merge_tree_03745 (CleanupThread) distributed default test_distributed_03745 1 default.test_distributed_03745.DistributedInsertQueue.default/Bg From 6cc72c1252bb2dd34faf4dd0c1a9806d1a5e30c8 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Wed, 4 Mar 2026 16:57:26 +0300 Subject: [PATCH 03/15] Merge pull request #1405 from Altinity/fp_antalya_26_1_list_objects_cache Antalya 26.1 - Forward port of list objects cache #1040 Source-PR: #1405 (https://github.com/Altinity/ClickHouse/pull/1405) --- programs/server/Server.cpp | 11 + src/Access/Common/AccessType.h | 1 + src/Common/ProfileEvents.cpp | 4 + src/Common/TTLCachePolicy.h | 4 +- src/Core/ServerSettings.cpp | 3 + src/Core/Settings.cpp | 3 + src/Core/SettingsChangesHistory.cpp | 2 + .../AzureBlobStorage/AzureObjectStorage.h | 2 + .../ObjectStorages/IObjectStorage.h | 2 + .../ObjectStorages/S3/S3ObjectStorage.h | 2 + src/Interpreters/InterpreterSystemQuery.cpp | 9 +- src/Parsers/ASTSystemQuery.cpp | 1 + src/Parsers/ASTSystemQuery.h | 1 + .../Cache/ObjectStorageListObjectsCache.cpp | 226 ++++++++++++++++ .../Cache/ObjectStorageListObjectsCache.h | 80 ++++++ ...test_object_storage_list_objects_cache.cpp | 244 ++++++++++++++++++ .../StorageObjectStorageSource.cpp | 66 ++++- .../StorageObjectStorageSource.h | 34 ++- .../01271_show_privileges.reference | 1 + ...bject_storage_list_objects_cache.reference | 103 ++++++++ ...3377_object_storage_list_objects_cache.sql | 115 +++++++++ 21 files changed, 892 insertions(+), 22 deletions(-) create mode 100644 src/Storages/Cache/ObjectStorageListObjectsCache.cpp create mode 100644 src/Storages/Cache/ObjectStorageListObjectsCache.h create mode 100644 src/Storages/Cache/tests/gtest_object_storage_list_objects_cache.cpp create mode 100644 tests/queries/0_stateless/03377_object_storage_list_objects_cache.reference create mode 100644 tests/queries/0_stateless/03377_object_storage_list_objects_cache.sql diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 1e8b2bd35918..7c36fbce1061 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -89,6 +89,7 @@ #include #include #include +#include #include #include #include @@ -428,6 +429,9 @@ namespace ServerSetting extern const ServerSettingsString hdfs_libhdfs3_conf; extern const ServerSettingsString config_file; extern const ServerSettingsString users_to_ignore_early_memory_limit_check; + extern const ServerSettingsUInt64 object_storage_list_objects_cache_ttl; + extern const ServerSettingsUInt64 object_storage_list_objects_cache_size; + extern const ServerSettingsUInt64 object_storage_list_objects_cache_max_entries; extern const ServerSettingsUInt64 input_format_parquet_metadata_cache_max_size; } @@ -439,6 +443,9 @@ namespace ErrorCodes namespace FileCacheSetting { extern const FileCacheSettingsBool load_metadata_asynchronously; + extern const ServerSettingsUInt64 object_storage_list_objects_cache_size; + extern const ServerSettingsUInt64 object_storage_list_objects_cache_max_entries; + extern const ServerSettingsUInt64 object_storage_list_objects_cache_ttl; } } @@ -2795,6 +2802,10 @@ try /// try set up encryption. There are some errors in config, error will be printed and server wouldn't start. CompressionCodecEncrypted::Configuration::instance().load(config(), "encryption_codecs"); + ObjectStorageListObjectsCache::instance().setMaxSizeInBytes(server_settings[ServerSetting::object_storage_list_objects_cache_size]); + ObjectStorageListObjectsCache::instance().setMaxCount(server_settings[ServerSetting::object_storage_list_objects_cache_max_entries]); + ObjectStorageListObjectsCache::instance().setTTL(server_settings[ServerSetting::object_storage_list_objects_cache_ttl]); + auto replicas_reconnector = ReplicasReconnector::init(global_context); #if USE_PARQUET diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index 77d8ed5f5070..3b3c1a1324ff 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -338,6 +338,7 @@ enum class AccessType : uint8_t M(SYSTEM_DROP_SCHEMA_CACHE, "SYSTEM CLEAR SCHEMA CACHE, SYSTEM DROP SCHEMA CACHE, DROP SCHEMA CACHE", GLOBAL, SYSTEM_DROP_CACHE) \ M(SYSTEM_DROP_FORMAT_SCHEMA_CACHE, "SYSTEM CLEAR FORMAT SCHEMA CACHE, SYSTEM DROP FORMAT SCHEMA CACHE, DROP FORMAT SCHEMA CACHE", GLOBAL, SYSTEM_DROP_CACHE) \ M(SYSTEM_DROP_S3_CLIENT_CACHE, "SYSTEM CLEAR S3 CLIENT CACHE, SYSTEM DROP S3 CLIENT, DROP S3 CLIENT CACHE", GLOBAL, SYSTEM_DROP_CACHE) \ + M(SYSTEM_DROP_OBJECT_STORAGE_LIST_OBJECTS_CACHE, "SYSTEM DROP OBJECT STORAGE LIST OBJECTS CACHE", GLOBAL, SYSTEM_DROP_CACHE) \ M(SYSTEM_DROP_CACHE, "DROP CACHE", GROUP, SYSTEM) \ M(SYSTEM_RELOAD_CONFIG, "RELOAD CONFIG", GLOBAL, SYSTEM_RELOAD) \ M(SYSTEM_RELOAD_USERS, "RELOAD USERS", GLOBAL, SYSTEM_RELOAD) \ diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 2ffaa114d527..f7d1ee1c57ea 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -1383,6 +1383,10 @@ The server successfully detected this situation and will download merged part fr M(RuntimeFilterRowsChecked, "Number of rows checked by JOIN Runtime Filters", ValueType::Number) \ M(RuntimeFilterRowsPassed, "Number of rows that passed (not filtered out by) JOIN Runtime Filters", ValueType::Number) \ M(RuntimeFilterRowsSkipped, "Number of rows in blocks that were skipped by JOIN Runtime Filters", ValueType::Number) \ + M(ObjectStorageListObjectsCacheHits, "Number of times object storage list objects operation hit the cache.", ValueType::Number) \ + M(ObjectStorageListObjectsCacheMisses, "Number of times object storage list objects operation miss the cache.", ValueType::Number) \ + M(ObjectStorageListObjectsCacheExactMatchHits, "Number of times object storage list objects operation hit the cache with an exact match.", ValueType::Number) \ + M(ObjectStorageListObjectsCachePrefixMatchHits, "Number of times object storage list objects operation miss the cache using prefix matching.", ValueType::Number) \ M(ParquetMetaDataCacheHits, "Number of times the read from filesystem cache hit the cache.", ValueType::Number) \ M(ParquetMetaDataCacheMisses, "Number of times the read from filesystem cache miss the cache.", ValueType::Number) \ diff --git a/src/Common/TTLCachePolicy.h b/src/Common/TTLCachePolicy.h index d481d0290330..1f894cb35bdb 100644 --- a/src/Common/TTLCachePolicy.h +++ b/src/Common/TTLCachePolicy.h @@ -278,10 +278,10 @@ class TTLCachePolicy : public ICachePolicy; Cache cache; - +private: /// TODO To speed up removal of stale entries, we could also add another container sorted on expiry times which maps keys to iterators /// into the cache. To insert an entry, add it to the cache + add the iterator to the sorted container. To remove stale entries, do a /// binary search on the sorted container and erase all left of the found key. diff --git a/src/Core/ServerSettings.cpp b/src/Core/ServerSettings.cpp index 12a02b3d4bf8..be029be90097 100644 --- a/src/Core/ServerSettings.cpp +++ b/src/Core/ServerSettings.cpp @@ -1531,6 +1531,9 @@ The policy on how to perform a scheduling of CPU slots specified by `concurrent_ 1 ``` )", 0) \ + DECLARE(UInt64, object_storage_list_objects_cache_size, 500000000, "Maximum size of ObjectStorage list objects cache in bytes. Zero means disabled.", 0) \ + DECLARE(UInt64, object_storage_list_objects_cache_max_entries, 1000, "Maximum size of ObjectStorage list objects cache in entries. Zero means disabled.", 0) \ + DECLARE(UInt64, object_storage_list_objects_cache_ttl, 3600, "Time to live of records in ObjectStorage list objects cache in seconds. Zero means unlimited", 0) \ DECLARE(Bool, enable_experimental_export_merge_tree_partition_feature, false, "Enable export replicated merge tree partition feature. It is experimental and not yet ready for production use.", 0) \ DECLARE(UInt64, input_format_parquet_metadata_cache_max_size, 500000000, "Maximum size of parquet file metadata cache", 0) diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 43a9e93e4d2c..fca7f7d1da9c 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -7682,6 +7682,9 @@ Write full paths (including s3://) into iceberg metadata files. )", EXPERIMENTAL) \ DECLARE(String, iceberg_metadata_compression_method, "", R"( Method to compress `.metadata.json` file. +)", EXPERIMENTAL) \ + DECLARE(Bool, use_object_storage_list_objects_cache, false, R"( +Cache the list of objects returned by list objects calls in object storage )", EXPERIMENTAL) \ DECLARE(Bool, make_distributed_plan, false, R"( Make distributed query plan. diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 88614a4d49d8..e9f11322f62b 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -154,6 +154,7 @@ const VersionToSettingsChangesMap & getSettingsChangesHistory() {"throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert", true, false, "It becomes obsolete."}, {"database_datalake_require_metadata_access", true, true, "New setting."}, {"automatic_parallel_replicas_min_bytes_per_replica", 0, 1_MiB, "Better default value derived from testing results"}, + {"use_object_storage_list_objects_cache", false, false, "New setting."}, }); addSettingsChanges(settings_changes_history, "25.12", { @@ -423,6 +424,7 @@ const VersionToSettingsChangesMap & getSettingsChangesHistory() {"allow_experimental_lightweight_update", false, true, "Lightweight updates were moved to Beta."}, {"s3_slow_all_threads_after_retryable_error", false, false, "Added an alias for setting `backup_slow_all_threads_after_retryable_s3_error`"}, {"serialize_string_in_memory_with_zero_byte", true, true, "New setting"}, + {"use_object_storage_list_objects_cache", false, false, "New setting."}, }); addSettingsChanges(settings_changes_history, "25.7", { diff --git a/src/Disks/DiskObjectStorage/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/DiskObjectStorage/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index bb6422697554..8ef48b8105f9 100644 --- a/src/Disks/DiskObjectStorage/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/DiskObjectStorage/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -35,6 +35,8 @@ class AzureObjectStorage : public IObjectStorage const String & description_, const String & common_key_prefix_); + bool supportsListObjectsCache() override { return true; } + void listObjects(const std::string & path, RelativePathsWithMetadata & children, size_t max_keys) const override; /// Sanitizer build may crash with max_keys=1; this looks like a false positive. diff --git a/src/Disks/DiskObjectStorage/ObjectStorages/IObjectStorage.h b/src/Disks/DiskObjectStorage/ObjectStorages/IObjectStorage.h index c56fb3877313..6c8b7d04fb8b 100644 --- a/src/Disks/DiskObjectStorage/ObjectStorages/IObjectStorage.h +++ b/src/Disks/DiskObjectStorage/ObjectStorages/IObjectStorage.h @@ -336,6 +336,8 @@ class IObjectStorage throw Exception(ErrorCodes::NOT_IMPLEMENTED, "The method 'tagObjects' is only implemented for S3 and Azure storages"); } #endif + + virtual bool supportsListObjectsCache() { return false; } }; using ObjectStoragePtr = std::shared_ptr; diff --git a/src/Disks/DiskObjectStorage/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/DiskObjectStorage/ObjectStorages/S3/S3ObjectStorage.h index 0b3bbefd2d6e..a17bdc00e905 100644 --- a/src/Disks/DiskObjectStorage/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/DiskObjectStorage/ObjectStorages/S3/S3ObjectStorage.h @@ -70,6 +70,8 @@ class S3ObjectStorage : public IObjectStorage ObjectStorageType getType() const override { return ObjectStorageType::S3; } + bool supportsListObjectsCache() override { return true; } + bool exists(const StoredObject & object) const override; std::unique_ptr readObject( /// NOLINT diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 6083afdd58a9..46b52a36521c 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -47,6 +47,7 @@ #include #include #include +#include #include #include #include @@ -509,7 +510,12 @@ BlockIO InterpreterSystemQuery::execute() #else throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "The server was compiled without the support for AWS S3"); #endif - + case Type::DROP_OBJECT_STORAGE_LIST_OBJECTS_CACHE: + { + getContext()->checkAccess(AccessType::SYSTEM_DROP_OBJECT_STORAGE_LIST_OBJECTS_CACHE); + ObjectStorageListObjectsCache::instance().clear(); + break; + } case Type::CLEAR_FILESYSTEM_CACHE: { getContext()->checkAccess(AccessType::SYSTEM_DROP_FILESYSTEM_CACHE); @@ -2193,6 +2199,7 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster() case Type::CLEAR_FORMAT_SCHEMA_CACHE: case Type::DROP_PARQUET_METADATA_CACHE: case Type::CLEAR_S3_CLIENT_CACHE: + case Type::DROP_OBJECT_STORAGE_LIST_OBJECTS_CACHE: { required_access.emplace_back(AccessType::SYSTEM_DROP_CACHE); break; diff --git a/src/Parsers/ASTSystemQuery.cpp b/src/Parsers/ASTSystemQuery.cpp index 6e97f8279598..2b9b16307d0f 100644 --- a/src/Parsers/ASTSystemQuery.cpp +++ b/src/Parsers/ASTSystemQuery.cpp @@ -600,6 +600,7 @@ void ASTSystemQuery::formatImpl(WriteBuffer & ostr, const FormatSettings & setti case Type::DROP_PARQUET_METADATA_CACHE: case Type::CLEAR_ICEBERG_METADATA_CACHE: case Type::CLEAR_PARQUET_METADATA_CACHE: + case Type::DROP_OBJECT_STORAGE_LIST_OBJECTS_CACHE: case Type::RESET_COVERAGE: case Type::RESTART_REPLICAS: case Type::JEMALLOC_PURGE: diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index 5b7682a80117..acb54c433487 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -52,6 +52,7 @@ class ASTSystemQuery : public IAST, public ASTQueryWithOnCluster CLEAR_SCHEMA_CACHE, CLEAR_FORMAT_SCHEMA_CACHE, CLEAR_S3_CLIENT_CACHE, + DROP_OBJECT_STORAGE_LIST_OBJECTS_CACHE, STOP_LISTEN, START_LISTEN, RESTART_REPLICAS, diff --git a/src/Storages/Cache/ObjectStorageListObjectsCache.cpp b/src/Storages/Cache/ObjectStorageListObjectsCache.cpp new file mode 100644 index 000000000000..78b73381c843 --- /dev/null +++ b/src/Storages/Cache/ObjectStorageListObjectsCache.cpp @@ -0,0 +1,226 @@ +#include +#include +#include +#include + +namespace ProfileEvents +{ +extern const Event ObjectStorageListObjectsCacheHits; +extern const Event ObjectStorageListObjectsCacheMisses; +extern const Event ObjectStorageListObjectsCacheExactMatchHits; +extern const Event ObjectStorageListObjectsCachePrefixMatchHits; +} + +namespace DB +{ + +template +class ObjectStorageListObjectsCachePolicy : public TTLCachePolicy +{ +public: + using BasePolicy = TTLCachePolicy; + using typename BasePolicy::MappedPtr; + using typename BasePolicy::KeyMapped; + using BasePolicy::cache; + + ObjectStorageListObjectsCachePolicy() + : BasePolicy(CurrentMetrics::end(), CurrentMetrics::end(), std::make_unique()) + { + } + + std::optional getWithKey(const Key & key) override + { + if (const auto it = cache.find(key); it != cache.end()) + { + if (!IsStaleFunction()(it->first)) + { + return std::make_optional({it->first, it->second}); + } + // found a stale entry, remove it but don't return. We still want to perform the prefix matching search + BasePolicy::remove(it->first); + } + + if (const auto it = findBestMatchingPrefixAndRemoveExpiredEntries(key); it != cache.end()) + { + return std::make_optional({it->first, it->second}); + } + + return std::nullopt; + } + +private: + auto findBestMatchingPrefixAndRemoveExpiredEntries(Key key) + { + while (!key.prefix.empty()) + { + if (const auto it = cache.find(key); it != cache.end()) + { + if (IsStaleFunction()(it->first)) + { + BasePolicy::remove(it->first); + } + else + { + return it; + } + } + + key.prefix.pop_back(); + } + + return cache.end(); + } +}; + +ObjectStorageListObjectsCache::Key::Key( + const String & storage_description_, + const String & bucket_, + const String & prefix_, + bool with_tags_, + const std::chrono::steady_clock::time_point & expires_at_, + std::optional user_id_) + : storage_description(storage_description_), bucket(bucket_), prefix(prefix_), with_tags(with_tags_), expires_at(expires_at_), user_id(user_id_) {} + +bool ObjectStorageListObjectsCache::Key::operator==(const Key & other) const +{ + return storage_description == other.storage_description && bucket == other.bucket && prefix == other.prefix && with_tags == other.with_tags; +} + +size_t ObjectStorageListObjectsCache::KeyHasher::operator()(const Key & key) const +{ + std::size_t seed = 0; + + boost::hash_combine(seed, key.storage_description); + boost::hash_combine(seed, key.bucket); + boost::hash_combine(seed, key.prefix); + boost::hash_combine(seed, key.with_tags); + + return seed; +} + +bool ObjectStorageListObjectsCache::IsStale::operator()(const Key & key) const +{ + return key.expires_at < std::chrono::steady_clock::now(); +} + +size_t ObjectStorageListObjectsCache::WeightFunction::operator()(const Value & value) const +{ + std::size_t weight = 0; + + for (const auto & object : value) + { + const auto object_metadata = object->getObjectMetadata(); + + weight += object->getPath().capacity() + sizeof(object_metadata); + + // variable size + if (object_metadata) + { + weight += object_metadata->etag.capacity(); + weight += object_metadata->attributes.size() * (sizeof(std::string) * 2); + + for (const auto & [k, v] : object_metadata->attributes) + { + weight += k.capacity() + v.capacity(); + } + + for (const auto & [k, v] : object_metadata->tags) + { + weight += k.capacity() + v.capacity(); + } + } + } + + return weight; +} + +ObjectStorageListObjectsCache::ObjectStorageListObjectsCache() + : cache(std::make_unique>()) +{ +} + +void ObjectStorageListObjectsCache::set( + const Key & key, + const std::shared_ptr & value) +{ + auto key_with_ttl = key; + + if (ttl_in_seconds == 0) + { + key_with_ttl.expires_at = std::chrono::steady_clock::time_point::max(); + } + else + { + key_with_ttl.expires_at = std::chrono::steady_clock::now() + std::chrono::seconds(ttl_in_seconds); + } + + cache.set(key_with_ttl, value); +} + +void ObjectStorageListObjectsCache::clear() +{ + cache.clear(); +} + +std::optional ObjectStorageListObjectsCache::get(const Key & key, bool filter_by_prefix) +{ + const auto pair = cache.getWithKey(key); + + if (!pair) + { + ProfileEvents::increment(ProfileEvents::ObjectStorageListObjectsCacheMisses); + return {}; + } + + ProfileEvents::increment(ProfileEvents::ObjectStorageListObjectsCacheHits); + + if (pair->key == key) + { + ProfileEvents::increment(ProfileEvents::ObjectStorageListObjectsCacheExactMatchHits); + return *pair->mapped; + } + + ProfileEvents::increment(ProfileEvents::ObjectStorageListObjectsCachePrefixMatchHits); + + if (!filter_by_prefix) + { + return *pair->mapped; + } + + Value filtered_objects; + + filtered_objects.reserve(pair->mapped->size()); + + for (const auto & object : *pair->mapped) + { + if (object->getPath().starts_with(key.prefix)) + { + filtered_objects.push_back(object); + } + } + + return filtered_objects; +} + +void ObjectStorageListObjectsCache::setMaxSizeInBytes(std::size_t size_in_bytes_) +{ + cache.setMaxSizeInBytes(size_in_bytes_); +} + +void ObjectStorageListObjectsCache::setMaxCount(std::size_t count) +{ + cache.setMaxCount(count); +} + +void ObjectStorageListObjectsCache::setTTL(std::size_t ttl_in_seconds_) +{ + ttl_in_seconds = ttl_in_seconds_; +} + +ObjectStorageListObjectsCache & ObjectStorageListObjectsCache::instance() +{ + static ObjectStorageListObjectsCache instance; + return instance; +} + +} diff --git a/src/Storages/Cache/ObjectStorageListObjectsCache.h b/src/Storages/Cache/ObjectStorageListObjectsCache.h new file mode 100644 index 000000000000..ab1eec7efb4e --- /dev/null +++ b/src/Storages/Cache/ObjectStorageListObjectsCache.h @@ -0,0 +1,80 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +class ObjectStorageListObjectsCache +{ + friend class ObjectStorageListObjectsCacheTest; +public: + ObjectStorageListObjectsCache(const ObjectStorageListObjectsCache &) = delete; + ObjectStorageListObjectsCache(ObjectStorageListObjectsCache &&) noexcept = delete; + + ObjectStorageListObjectsCache& operator=(const ObjectStorageListObjectsCache &) = delete; + ObjectStorageListObjectsCache& operator=(ObjectStorageListObjectsCache &&) noexcept = delete; + + static ObjectStorageListObjectsCache & instance(); + + struct Key + { + Key( + const String & storage_description_, + const String & bucket_, + const String & prefix_, + bool with_tags_, + const std::chrono::steady_clock::time_point & expires_at_ = std::chrono::steady_clock::now(), + std::optional user_id_ = std::nullopt); + + std::string storage_description; + std::string bucket; + std::string prefix; + bool with_tags; + std::chrono::steady_clock::time_point expires_at; + std::optional user_id; + + bool operator==(const Key & other) const; + }; + + using Value = ObjectInfos; + struct KeyHasher + { + size_t operator()(const Key & key) const; + }; + + struct IsStale + { + bool operator()(const Key & key) const; + }; + + struct WeightFunction + { + size_t operator()(const Value & value) const; + }; + + using Cache = CacheBase; + + void set( + const Key & key, + const std::shared_ptr & value); + + std::optional get(const Key & key, bool filter_by_prefix = true); + + void clear(); + + void setMaxSizeInBytes(std::size_t size_in_bytes_); + void setMaxCount(std::size_t count); + void setTTL(std::size_t ttl_in_seconds_); + +private: + ObjectStorageListObjectsCache(); + + Cache cache; + size_t ttl_in_seconds {0}; +}; + +} diff --git a/src/Storages/Cache/tests/gtest_object_storage_list_objects_cache.cpp b/src/Storages/Cache/tests/gtest_object_storage_list_objects_cache.cpp new file mode 100644 index 000000000000..8eb45d520c96 --- /dev/null +++ b/src/Storages/Cache/tests/gtest_object_storage_list_objects_cache.cpp @@ -0,0 +1,244 @@ +#include +#include +#include +#include + +namespace DB +{ + +class ObjectStorageListObjectsCacheTest : public ::testing::Test +{ +protected: + void SetUp() override + { + cache = std::unique_ptr(new ObjectStorageListObjectsCache()); + cache->setTTL(3); + cache->setMaxCount(100); + cache->setMaxSizeInBytes(1000000); + } + + std::unique_ptr cache; + static ObjectStorageListObjectsCache::Key default_key; + + static std::shared_ptr createTestValue(const std::vector& paths) + { + auto value = std::make_shared(); + for (const auto & path : paths) + { + value->push_back(std::make_shared(path)); + } + return value; + } +}; + +ObjectStorageListObjectsCache::Key ObjectStorageListObjectsCacheTest::default_key {"default", "test-bucket", "test-prefix/", false}; + +TEST_F(ObjectStorageListObjectsCacheTest, BasicSetAndGet) +{ + cache->clear(); + auto value = createTestValue({"test-prefix/file1.txt", "test-prefix/file2.txt"}); + + cache->set(default_key, value); + + auto result = cache->get(default_key).value(); + + ASSERT_EQ(result.size(), 2); + EXPECT_EQ(result[0]->getPath(), "test-prefix/file1.txt"); + EXPECT_EQ(result[1]->getPath(), "test-prefix/file2.txt"); +} + +TEST_F(ObjectStorageListObjectsCacheTest, CacheMiss) +{ + cache->clear(); + + EXPECT_FALSE(cache->get(default_key)); +} + +TEST_F(ObjectStorageListObjectsCacheTest, ClearCache) +{ + cache->clear(); + auto value = createTestValue({"test-prefix/file1.txt", "test-prefix/file2.txt"}); + + cache->set(default_key, value); + cache->clear(); + + EXPECT_FALSE(cache->get(default_key)); +} + +TEST_F(ObjectStorageListObjectsCacheTest, PrefixMatching) +{ + cache->clear(); + + auto short_prefix_key = default_key; + short_prefix_key.prefix = "parent/"; + + auto mid_prefix_key = default_key; + mid_prefix_key.prefix = "parent/child/"; + + auto long_prefix_key = default_key; + long_prefix_key.prefix = "parent/child/grandchild/"; + + auto value = createTestValue( + { + "parent/child/grandchild/file1.txt", + "parent/child/grandchild/file2.txt"}); + + cache->set(mid_prefix_key, value); + + auto result1 = cache->get(mid_prefix_key).value(); + EXPECT_EQ(result1.size(), 2); + + auto result2 = cache->get(long_prefix_key).value(); + EXPECT_EQ(result2.size(), 2); + + EXPECT_FALSE(cache->get(short_prefix_key)); +} + +TEST_F(ObjectStorageListObjectsCacheTest, PrefixFiltering) +{ + cache->clear(); + + auto key_with_short_prefix = default_key; + key_with_short_prefix.prefix = "parent/"; + + auto key_with_mid_prefix = default_key; + key_with_mid_prefix.prefix = "parent/child1/"; + + auto value = createTestValue({ + "parent/file1.txt", + "parent/child1/file2.txt", + "parent/child2/file3.txt" + }); + + cache->set(key_with_short_prefix, value); + + auto result = cache->get(key_with_mid_prefix, true).value(); + EXPECT_EQ(result.size(), 1); + EXPECT_EQ(result[0]->getPath(), "parent/child1/file2.txt"); +} + +TEST_F(ObjectStorageListObjectsCacheTest, TTLExpiration) +{ + cache->clear(); + auto value = createTestValue({"test-prefix/file1.txt"}); + + cache->set(default_key, value); + + // Verify we can get it immediately + auto result1 = cache->get(default_key).value(); + EXPECT_EQ(result1.size(), 1); + + std::this_thread::sleep_for(std::chrono::seconds(4)); + + EXPECT_FALSE(cache->get(default_key)); +} + +TEST_F(ObjectStorageListObjectsCacheTest, TTLUnlimited) +{ + cache->clear(); + cache->setTTL(0); // 0 means unlimited + auto value = createTestValue({"test-prefix/file1.txt"}); + + cache->set(default_key, value); + + // Verify we can get it immediately + auto result1 = cache->get(default_key).value(); + EXPECT_EQ(result1.size(), 1); + + // Sleep for a reasonable amount (longer than the default 3 second TTL from SetUp) + std::this_thread::sleep_for(std::chrono::seconds(5)); + + // Should still be available since TTL is unlimited + auto result2 = cache->get(default_key).value(); + EXPECT_EQ(result2.size(), 1); + EXPECT_EQ(result2[0]->getPath(), "test-prefix/file1.txt"); +} + +TEST_F(ObjectStorageListObjectsCacheTest, TTLSwitchFromUnlimitedToFinite) +{ + cache->clear(); + cache->setTTL(0); // Start with unlimited + auto value1 = createTestValue({"test-prefix/file1.txt"}); + auto key1 = default_key; + key1.prefix = "unlimited/"; + + cache->set(key1, value1); + + // Switch to finite TTL and add another entry + cache->setTTL(1); + auto value2 = createTestValue({"test-prefix/file2.txt"}); + auto key2 = default_key; + key2.prefix = "finite/"; + + cache->set(key2, value2); + + // Verify both are available immediately + EXPECT_TRUE(cache->get(key1).has_value()); + EXPECT_TRUE(cache->get(key2).has_value()); + + // Wait for finite TTL entry to expire + std::this_thread::sleep_for(std::chrono::seconds(2)); + + // Unlimited entry should still be there, finite should be gone + EXPECT_TRUE(cache->get(key1).has_value()); + EXPECT_FALSE(cache->get(key2).has_value()); +} + +TEST_F(ObjectStorageListObjectsCacheTest, BestPrefixMatch) +{ + cache->clear(); + + auto short_prefix_key = default_key; + short_prefix_key.prefix = "a/b/"; + + auto mid_prefix_key = default_key; + mid_prefix_key.prefix = "a/b/c/"; + + auto long_prefix_key = default_key; + long_prefix_key.prefix = "a/b/c/d/"; + + auto short_prefix = createTestValue({"a/b/c/d/file1.txt", "a/b/c/file1.txt", "a/b/file2.txt"}); + auto mid_prefix = createTestValue({"a/b/c/d/file1.txt", "a/b/c/file1.txt"}); + + cache->set(short_prefix_key, short_prefix); + cache->set(mid_prefix_key, mid_prefix); + + // should pick mid_prefix, which has size 2. filter_by_prefix=false so we can assert by size + auto result = cache->get(long_prefix_key, false).value(); + EXPECT_EQ(result.size(), 2u); +} + +TEST_F(ObjectStorageListObjectsCacheTest, WithTags) +{ + cache->clear(); + + auto key_with_tags = default_key; + key_with_tags.with_tags = true; + + auto value_with_tags = createTestValue({"test.txt"}); + + cache->set(key_with_tags, value_with_tags); + + /// we have set with tags, we should be able to retrieve it + auto result_with_tags = cache->get(key_with_tags).value(); + EXPECT_EQ(result_with_tags.size(), 1u); + EXPECT_EQ(result_with_tags[0]->getPath(), "test.txt"); + + /// querying by a key without tags should return nothing + auto result_without_tags = cache->get(default_key); + EXPECT_FALSE(result_without_tags.has_value()); + + cache->clear(); + + cache->set(default_key, value_with_tags); + + /// querying by a key with tags should return nothing + EXPECT_FALSE(cache->get(key_with_tags).has_value()); + + /// querying by a key without tags should return the value + auto result_without_tags_2 = cache->get(default_key).value(); + EXPECT_EQ(result_without_tags_2.size(), 1u); + EXPECT_EQ(result_without_tags_2[0]->getPath(), "test.txt"); +} + +} diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 6820d54668ef..150427a4758a 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -82,6 +82,7 @@ namespace Setting extern const SettingsUInt64 s3_path_filter_limit; extern const SettingsBool use_parquet_metadata_cache; extern const SettingsBool input_format_parquet_use_native_reader_v3; + extern const SettingsBool use_object_storage_list_objects_cache; } namespace ErrorCodes @@ -212,18 +213,52 @@ std::shared_ptr StorageObjectStorageSource::createFileIterator( // If paths contains a value, validate the extracted paths and use the key-based iterator // (even if the result is empty, indicating no scanning is required). if (!paths) + { + std::shared_ptr object_iterator = nullptr; + std::unique_ptr cache_ptr = nullptr; + + if (local_context->getSettingsRef()[Setting::use_object_storage_list_objects_cache] && object_storage->supportsListObjectsCache()) + { + auto & cache = ObjectStorageListObjectsCache::instance(); + ObjectStorageListObjectsCache::Key cache_key {object_storage->getDescription(), configuration->getNamespace(), configuration->getRawPath().cutGlobs(configuration->supportsPartialPathPrefix()), with_tags}; + + if (auto objects_info = cache.get(cache_key, /*filter_by_prefix=*/ false)) + { + /// suboptimal because of the recent upstream changes to the ObjectInfo structure + /// re-think this with more time and see if there is a more optimized approach + RelativePathsWithMetadata relative_path_with_metadata; + relative_path_with_metadata.reserve(objects_info->size()); + + for (const auto & object_info : *objects_info) + { + relative_path_with_metadata.emplace_back(std::make_shared(object_info->getPath(), object_info->getObjectMetadata())); + } + + object_iterator = std::make_shared(std::move(relative_path_with_metadata)); + } + else + { + cache_ptr = std::make_unique(cache, cache_key); + object_iterator = object_storage->iterate(configuration->getRawPath().cutGlobs(configuration->supportsPartialPathPrefix()), query_settings.list_object_keys_size, with_tags, std::nullopt); + } + } + else + { + object_iterator = object_storage->iterate(configuration->getRawPath().cutGlobs(configuration->supportsPartialPathPrefix()), query_settings.list_object_keys_size, with_tags, std::nullopt); + } + iterator = std::make_unique( - object_storage, + object_iterator, configuration, predicate, virtual_columns, hive_columns, local_context, is_archive ? nullptr : read_keys, - query_settings.list_object_keys_size, query_settings.throw_on_zero_files_match, - with_tags, - file_progress_callback); + file_progress_callback, + std::move(cache_ptr)); + } else { // Validate that extracted paths match the glob pattern to prevent scanning unallowed data @@ -1002,19 +1037,18 @@ std::unique_ptr createReadBuffer( } StorageObjectStorageSource::GlobIterator::GlobIterator( - ObjectStoragePtr object_storage_, - StorageObjectStorageConfigurationPtr configuration_, + const ObjectStorageIteratorPtr & object_storage_iterator_, + ConfigurationPtr configuration_, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns_, const NamesAndTypesList & hive_columns_, ContextPtr context_, ObjectInfos * read_keys_, - size_t list_object_keys_size, bool throw_on_zero_files_match_, - bool with_tags, - std::function file_progress_callback_) + std::function file_progress_callback_, + std::unique_ptr list_cache_) : WithContext(context_) - , object_storage(object_storage_) + , object_storage_iterator(object_storage_iterator_) , configuration(configuration_) , virtual_columns(virtual_columns_) , hive_columns(hive_columns_) @@ -1023,6 +1057,7 @@ StorageObjectStorageSource::GlobIterator::GlobIterator( , read_keys(read_keys_) , local_context(context_) , file_progress_callback(file_progress_callback_) + , list_cache(std::move(list_cache_)) { const auto & reading_path = configuration->getPathForRead(); if (reading_path.hasGlobs()) @@ -1030,8 +1065,6 @@ StorageObjectStorageSource::GlobIterator::GlobIterator( const auto & key_with_globs = reading_path; const auto key_prefix = reading_path.cutGlobs(configuration->supportsPartialPathPrefix()); - object_storage_iterator = object_storage->iterate(key_prefix, list_object_keys_size, with_tags, std::nullopt); - matcher = std::make_unique(makeRegexpPatternFromGlobs(key_with_globs.path)); if (!matcher->ok()) { @@ -1097,6 +1130,10 @@ ObjectInfoPtr StorageObjectStorageSource::GlobIterator::nextUnlocked(size_t /* p auto result = object_storage_iterator->getCurrentBatchAndScheduleNext(); if (!result.has_value()) { + if (list_cache) + { + list_cache->set(std::move(object_list)); + } is_finished = true; LOG_DEBUG(log, "Listing finished: total_listed={}, glob_filtered={}, predicate_filtered={}", total_listed, total_glob_filtered, total_predicate_filtered); @@ -1115,6 +1152,11 @@ ObjectInfoPtr StorageObjectStorageSource::GlobIterator::nextUnlocked(size_t /* p listed_in_batch = new_batch.size(); + if (list_cache) + { + object_list.insert(object_list.end(), new_batch.begin(), new_batch.end()); + } + for (auto it = new_batch.begin(); it != new_batch.end();) { if (!recursive && !re2::RE2::FullMatch((*it)->getPath(), *matcher)) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.h b/src/Storages/ObjectStorage/StorageObjectStorageSource.h index 5cbe931a047e..029598bd01bd 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.h @@ -10,6 +10,9 @@ #include #include #include +#include + + namespace DB { @@ -177,18 +180,33 @@ class StorageObjectStorageSource::ReadTaskIterator : public IObjectIterator, pri class StorageObjectStorageSource::GlobIterator : public IObjectIterator, WithContext { public: + struct ListObjectsCacheWithKey + { + ListObjectsCacheWithKey(ObjectStorageListObjectsCache & cache_, const ObjectStorageListObjectsCache::Key & key_) : cache(cache_), key(key_) {} + + void set(ObjectStorageListObjectsCache::Value && value) const + { + cache.set(key, std::make_shared(std::move(value))); + } + + private: + ObjectStorageListObjectsCache & cache; + ObjectStorageListObjectsCache::Key key; + }; + + using ConfigurationPtr = std::shared_ptr; + GlobIterator( - ObjectStoragePtr object_storage_, - StorageObjectStorageConfigurationPtr configuration_, + const ObjectStorageIteratorPtr & object_storage_iterator_, + ConfigurationPtr configuration_, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns_, const NamesAndTypesList & hive_columns_, ContextPtr context_, ObjectInfos * read_keys_, - size_t list_object_keys_size, bool throw_on_zero_files_match_, - bool with_tags, - std::function file_progress_callback_ = {}); + std::function file_progress_callback_ = {}, + std::unique_ptr list_cache_ = nullptr); ~GlobIterator() override = default; @@ -201,7 +219,7 @@ class StorageObjectStorageSource::GlobIterator : public IObjectIterator, WithCon void createFilterAST(const String & any_key); void fillBufferForKey(const std::string & uri_key); - const ObjectStoragePtr object_storage; + ObjectStorageIteratorPtr object_storage_iterator; const StorageObjectStorageConfigurationPtr configuration; const NamesAndTypesList virtual_columns; const NamesAndTypesList hive_columns; @@ -213,7 +231,6 @@ class StorageObjectStorageSource::GlobIterator : public IObjectIterator, WithCon ObjectInfos object_infos; ObjectInfos * read_keys; ExpressionActionsPtr filter_expr; - ObjectStorageIteratorPtr object_storage_iterator; bool recursive{false}; std::vector expanded_keys; std::vector::iterator expanded_keys_iter; @@ -230,6 +247,9 @@ class StorageObjectStorageSource::GlobIterator : public IObjectIterator, WithCon size_t total_listed = 0; size_t total_glob_filtered = 0; size_t total_predicate_filtered = 0; + + std::unique_ptr list_cache; + ObjectInfos object_list; }; class StorageObjectStorageSource::KeysIterator : public IObjectIterator diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index 02832536782f..7cdcc27382a2 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -148,6 +148,7 @@ SYSTEM DROP PAGE CACHE ['SYSTEM CLEAR PAGE CACHE','SYSTEM DROP PAGE CACHE','DROP SYSTEM DROP SCHEMA CACHE ['SYSTEM CLEAR SCHEMA CACHE','SYSTEM DROP SCHEMA CACHE','DROP SCHEMA CACHE'] GLOBAL SYSTEM DROP CACHE SYSTEM DROP FORMAT SCHEMA CACHE ['SYSTEM CLEAR FORMAT SCHEMA CACHE','SYSTEM DROP FORMAT SCHEMA CACHE','DROP FORMAT SCHEMA CACHE'] GLOBAL SYSTEM DROP CACHE SYSTEM DROP S3 CLIENT CACHE ['SYSTEM CLEAR S3 CLIENT CACHE','SYSTEM DROP S3 CLIENT','DROP S3 CLIENT CACHE'] GLOBAL SYSTEM DROP CACHE +SYSTEM DROP OBJECT STORAGE LIST OBJECTS CACHE ['SYSTEM DROP OBJECT STORAGE LIST OBJECTS CACHE'] GLOBAL SYSTEM DROP CACHE SYSTEM DROP CACHE ['DROP CACHE'] \N SYSTEM SYSTEM RELOAD CONFIG ['RELOAD CONFIG'] GLOBAL SYSTEM RELOAD SYSTEM RELOAD USERS ['RELOAD USERS'] GLOBAL SYSTEM RELOAD diff --git a/tests/queries/0_stateless/03377_object_storage_list_objects_cache.reference b/tests/queries/0_stateless/03377_object_storage_list_objects_cache.reference new file mode 100644 index 000000000000..76535ad25106 --- /dev/null +++ b/tests/queries/0_stateless/03377_object_storage_list_objects_cache.reference @@ -0,0 +1,103 @@ +-- { echoOn } + +-- The cached key should be `dir_`, and that includes all three files: 1, 2 and 3. Cache should return all three, but ClickHouse should filter out the third. +SELECT _path, * FROM s3(s3_conn, filename='dir_a/dir_b/t_03377_sample_{1..2}.parquet') order by id SETTINGS use_object_storage_list_objects_cache=1; +test/dir_a/dir_b/t_03377_sample_1.parquet 1 +test/dir_a/dir_b/t_03377_sample_2.parquet 2 +-- Make sure the filtering did not interfere with the cached values +SELECT _path, * FROM s3(s3_conn, filename='dir_a/dir_b/t_03377_sample_*.parquet') order by id SETTINGS use_object_storage_list_objects_cache=1; +test/dir_a/dir_b/t_03377_sample_1.parquet 1 +test/dir_a/dir_b/t_03377_sample_2.parquet 2 +test/dir_a/dir_b/t_03377_sample_3.parquet 3 +SYSTEM FLUSH LOGS; +SELECT ProfileEvents['ObjectStorageListObjectsCacheMisses'] > 0 as miss +FROM system.query_log +where log_comment = 'cold_list_cache' +AND type = 'QueryFinish' +ORDER BY event_time desc +LIMIT 1; +1 +SELECT ProfileEvents['ObjectStorageListObjectsCacheHits'] > 0 as hit +FROM system.query_log +where log_comment = 'warm_list_exact_cache' +AND type = 'QueryFinish' +ORDER BY event_time desc +LIMIT 1; +1 +SELECT ProfileEvents['ObjectStorageListObjectsCacheExactMatchHits'] > 0 as hit +FROM system.query_log +where log_comment = 'warm_list_exact_cache' +AND type = 'QueryFinish' +ORDER BY event_time desc +LIMIT 1; +1 +SELECT ProfileEvents['ObjectStorageListObjectsCachePrefixMatchHits'] > 0 as prefix_match_hit +FROM system.query_log +where log_comment = 'warm_list_exact_cache' +AND type = 'QueryFinish' +ORDER BY event_time desc +LIMIT 1; +0 +SELECT ProfileEvents['ObjectStorageListObjectsCacheHits'] > 0 as hit +FROM system.query_log +where log_comment = 'warm_list_prefix_match_cache' +AND type = 'QueryFinish' +ORDER BY event_time desc +LIMIT 1; +1 +SELECT ProfileEvents['ObjectStorageListObjectsCacheExactMatchHits'] > 0 as exact_match_hit +FROM system.query_log +where log_comment = 'warm_list_prefix_match_cache' +AND type = 'QueryFinish' +ORDER BY event_time desc +LIMIT 1; +0 +SELECT ProfileEvents['ObjectStorageListObjectsCachePrefixMatchHits'] > 0 as prefix_match_hit +FROM system.query_log +where log_comment = 'warm_list_prefix_match_cache' +AND type = 'QueryFinish' +ORDER BY event_time desc +LIMIT 1; +1 +SELECT ProfileEvents['ObjectStorageListObjectsCacheHits'] > 0 as hit +FROM system.query_log +where log_comment = 'even_shorter_prefix' +AND type = 'QueryFinish' +ORDER BY event_time desc +LIMIT 1; +0 +SELECT ProfileEvents['ObjectStorageListObjectsCacheMisses'] > 0 as miss +FROM system.query_log +where log_comment = 'even_shorter_prefix' +AND type = 'QueryFinish' +ORDER BY event_time desc +LIMIT 1; +1 +SELECT ProfileEvents['ObjectStorageListObjectsCacheHits'] > 0 as hit +FROM system.query_log +where log_comment = 'still_exact_match_after_shorter_prefix' +AND type = 'QueryFinish' +ORDER BY event_time desc +LIMIT 1; +1 +SELECT ProfileEvents['ObjectStorageListObjectsCacheExactMatchHits'] > 0 as exact_match_hit +FROM system.query_log +where log_comment = 'still_exact_match_after_shorter_prefix' +AND type = 'QueryFinish' +ORDER BY event_time desc +LIMIT 1; +1 +SELECT ProfileEvents['ObjectStorageListObjectsCacheHits'] > 0 as hit +FROM system.query_log +where log_comment = 'after_drop' +AND type = 'QueryFinish' +ORDER BY event_time desc +LIMIT 1; +0 +SELECT ProfileEvents['ObjectStorageListObjectsCacheMisses'] > 0 as miss +FROM system.query_log +where log_comment = 'after_drop' +AND type = 'QueryFinish' +ORDER BY event_time desc +LIMIT 1; +1 diff --git a/tests/queries/0_stateless/03377_object_storage_list_objects_cache.sql b/tests/queries/0_stateless/03377_object_storage_list_objects_cache.sql new file mode 100644 index 000000000000..9638faa88d23 --- /dev/null +++ b/tests/queries/0_stateless/03377_object_storage_list_objects_cache.sql @@ -0,0 +1,115 @@ +-- Tags: no-parallel, no-fasttest + +SYSTEM DROP OBJECT STORAGE LIST OBJECTS CACHE; + +INSERT INTO TABLE FUNCTION s3(s3_conn, filename='dir_a/dir_b/t_03377_sample_{_partition_id}.parquet', format='Parquet', structure='id UInt64') PARTITION BY id SETTINGS s3_truncate_on_insert=1 VALUES (1), (2), (3); + +SELECT * FROM s3(s3_conn, filename='dir_**.parquet') Format Null SETTINGS use_object_storage_list_objects_cache=1, log_comment='cold_list_cache'; +SELECT * FROM s3(s3_conn, filename='dir_**.parquet') Format Null SETTINGS use_object_storage_list_objects_cache=1, log_comment='warm_list_exact_cache'; +SELECT * FROM s3(s3_conn, filename='dir_a/dir_b**.parquet') Format Null SETTINGS use_object_storage_list_objects_cache=1, log_comment='warm_list_prefix_match_cache'; +SELECT * FROM s3(s3_conn, filename='dirr_**.parquet') Format Null SETTINGS use_object_storage_list_objects_cache=1, log_comment='warm_list_cache_miss'; -- { serverError CANNOT_EXTRACT_TABLE_STRUCTURE } +SELECT * FROM s3(s3_conn, filename='d**.parquet') Format Null SETTINGS use_object_storage_list_objects_cache=1, log_comment='even_shorter_prefix'; +SELECT * FROM s3(s3_conn, filename='dir_**.parquet') Format Null SETTINGS use_object_storage_list_objects_cache=1, log_comment='still_exact_match_after_shorter_prefix'; +SYSTEM DROP OBJECT STORAGE LIST OBJECTS CACHE; +SELECT * FROM s3(s3_conn, filename='dir_**.parquet') Format Null SETTINGS use_object_storage_list_objects_cache=1, log_comment='after_drop'; + +-- { echoOn } + +-- The cached key should be `dir_`, and that includes all three files: 1, 2 and 3. Cache should return all three, but ClickHouse should filter out the third. +SELECT _path, * FROM s3(s3_conn, filename='dir_a/dir_b/t_03377_sample_{1..2}.parquet') order by id SETTINGS use_object_storage_list_objects_cache=1; + +-- Make sure the filtering did not interfere with the cached values +SELECT _path, * FROM s3(s3_conn, filename='dir_a/dir_b/t_03377_sample_*.parquet') order by id SETTINGS use_object_storage_list_objects_cache=1; + +SYSTEM FLUSH LOGS; + +SELECT ProfileEvents['ObjectStorageListObjectsCacheMisses'] > 0 as miss +FROM system.query_log +where log_comment = 'cold_list_cache' +AND type = 'QueryFinish' +ORDER BY event_time desc +LIMIT 1; + +SELECT ProfileEvents['ObjectStorageListObjectsCacheHits'] > 0 as hit +FROM system.query_log +where log_comment = 'warm_list_exact_cache' +AND type = 'QueryFinish' +ORDER BY event_time desc +LIMIT 1; + +SELECT ProfileEvents['ObjectStorageListObjectsCacheExactMatchHits'] > 0 as hit +FROM system.query_log +where log_comment = 'warm_list_exact_cache' +AND type = 'QueryFinish' +ORDER BY event_time desc +LIMIT 1; + +SELECT ProfileEvents['ObjectStorageListObjectsCachePrefixMatchHits'] > 0 as prefix_match_hit +FROM system.query_log +where log_comment = 'warm_list_exact_cache' +AND type = 'QueryFinish' +ORDER BY event_time desc +LIMIT 1; + +SELECT ProfileEvents['ObjectStorageListObjectsCacheHits'] > 0 as hit +FROM system.query_log +where log_comment = 'warm_list_prefix_match_cache' +AND type = 'QueryFinish' +ORDER BY event_time desc +LIMIT 1; + +SELECT ProfileEvents['ObjectStorageListObjectsCacheExactMatchHits'] > 0 as exact_match_hit +FROM system.query_log +where log_comment = 'warm_list_prefix_match_cache' +AND type = 'QueryFinish' +ORDER BY event_time desc +LIMIT 1; + +SELECT ProfileEvents['ObjectStorageListObjectsCachePrefixMatchHits'] > 0 as prefix_match_hit +FROM system.query_log +where log_comment = 'warm_list_prefix_match_cache' +AND type = 'QueryFinish' +ORDER BY event_time desc +LIMIT 1; + +SELECT ProfileEvents['ObjectStorageListObjectsCacheHits'] > 0 as hit +FROM system.query_log +where log_comment = 'even_shorter_prefix' +AND type = 'QueryFinish' +ORDER BY event_time desc +LIMIT 1; + +SELECT ProfileEvents['ObjectStorageListObjectsCacheMisses'] > 0 as miss +FROM system.query_log +where log_comment = 'even_shorter_prefix' +AND type = 'QueryFinish' +ORDER BY event_time desc +LIMIT 1; + +SELECT ProfileEvents['ObjectStorageListObjectsCacheHits'] > 0 as hit +FROM system.query_log +where log_comment = 'still_exact_match_after_shorter_prefix' +AND type = 'QueryFinish' +ORDER BY event_time desc +LIMIT 1; + +SELECT ProfileEvents['ObjectStorageListObjectsCacheExactMatchHits'] > 0 as exact_match_hit +FROM system.query_log +where log_comment = 'still_exact_match_after_shorter_prefix' +AND type = 'QueryFinish' +ORDER BY event_time desc +LIMIT 1; + +SELECT ProfileEvents['ObjectStorageListObjectsCacheHits'] > 0 as hit +FROM system.query_log +where log_comment = 'after_drop' +AND type = 'QueryFinish' +ORDER BY event_time desc +LIMIT 1; + +SELECT ProfileEvents['ObjectStorageListObjectsCacheMisses'] > 0 as miss +FROM system.query_log +where log_comment = 'after_drop' +AND type = 'QueryFinish' +ORDER BY event_time desc +LIMIT 1; From 8a78664ba3e1d0d3a3d374759f7bdf8072046815 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Thu, 5 Mar 2026 22:16:28 +0300 Subject: [PATCH 04/15] Merge pull request #1478 from Altinity/skip_remote_metadata_suite_for_export_tests_that_block_minio Skip remote suite on export tests that block minio Source-PR: #1478 (https://github.com/Altinity/ClickHouse/pull/1478) --- .../test.py | 15 +++++++++++++++ .../test.py | 18 ++++++++++++++++++ 2 files changed, 33 insertions(+) diff --git a/tests/integration/test_export_merge_tree_part_to_object_storage/test.py b/tests/integration/test_export_merge_tree_part_to_object_storage/test.py index af149e682049..1539cb130598 100644 --- a/tests/integration/test_export_merge_tree_part_to_object_storage/test.py +++ b/tests/integration/test_export_merge_tree_part_to_object_storage/test.py @@ -9,6 +9,18 @@ from helpers.cluster import ClickHouseCluster from helpers.network import PartitionManager + +def skip_if_remote_database_disk_enabled(cluster): + """Skip test if any instance in the cluster has remote database disk enabled. + + Tests that block MinIO cannot run when remote database disk is enabled, + as the database metadata is stored on MinIO and blocking it would break the database. + """ + for instance in cluster.instances.values(): + if instance.with_remote_database_disk: + pytest.skip("Test cannot run with remote database disk enabled (db disk), as it blocks MinIO which stores database metadata") + + @pytest.fixture(scope="module") def cluster(): try: @@ -37,6 +49,7 @@ def create_tables_and_insert_data(node, mt_table, s3_table): def test_drop_column_during_export_snapshot(cluster): + skip_if_remote_database_disk_enabled(cluster) node = cluster.instances["node1"] mt_table = "mutations_snapshot_mt_table" @@ -88,6 +101,7 @@ def test_drop_column_during_export_snapshot(cluster): def test_add_column_during_export(cluster): + skip_if_remote_database_disk_enabled(cluster) node = cluster.instances["node1"] mt_table = "add_column_during_export_mt_table" @@ -193,6 +207,7 @@ def test_pending_mutations_skip_before_export(cluster): def test_data_mutations_after_export_started(cluster): """Test that mutations applied after export starts don't affect the exported data.""" + skip_if_remote_database_disk_enabled(cluster) node = cluster.instances["node1"] mt_table = "mutations_after_export_mt_table" diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py index db6b1bb23a21..80f65378985b 100644 --- a/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py +++ b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py @@ -69,6 +69,17 @@ def wait_for_export_to_start( raise TimeoutError(f"Export did not start within {timeout}s. ") +def skip_if_remote_database_disk_enabled(cluster): + """Skip test if any instance in the cluster has remote database disk enabled. + + Tests that block MinIO cannot run when remote database disk is enabled, + as the database metadata is stored on MinIO and blocking it would break the database. + """ + for instance in cluster.instances.values(): + if instance.with_remote_database_disk: + pytest.skip("Test cannot run with remote database disk enabled (db disk), as it blocks MinIO which stores database metadata") + + @pytest.fixture(scope="module") def cluster(): try: @@ -123,6 +134,7 @@ def create_tables_and_insert_data(node, mt_table, s3_table, replica_name): def test_restart_nodes_during_export(cluster): + skip_if_remote_database_disk_enabled(cluster) node = cluster.instances["replica1"] node2 = cluster.instances["replica2"] watcher_node = cluster.instances["watcher_node"] @@ -213,6 +225,7 @@ def test_restart_nodes_during_export(cluster): def test_kill_export(cluster): + skip_if_remote_database_disk_enabled(cluster) node = cluster.instances["replica1"] node2 = cluster.instances["replica2"] watcher_node = cluster.instances["watcher_node"] @@ -297,6 +310,7 @@ def test_kill_export(cluster): def test_drop_source_table_during_export(cluster): + skip_if_remote_database_disk_enabled(cluster) node = cluster.instances["replica1"] # node2 = cluster.instances["replica2"] watcher_node = cluster.instances["watcher_node"] @@ -390,6 +404,7 @@ def test_concurrent_exports_to_different_targets(cluster): def test_failure_is_logged_in_system_table(cluster): + skip_if_remote_database_disk_enabled(cluster) node = cluster.instances["replica1"] mt_table = "failure_is_logged_in_system_table_mt_table" @@ -459,6 +474,7 @@ def test_failure_is_logged_in_system_table(cluster): def test_inject_short_living_failures(cluster): + skip_if_remote_database_disk_enabled(cluster) node = cluster.instances["replica1"] mt_table = "inject_short_living_failures_mt_table" @@ -881,6 +897,7 @@ def test_pending_patch_parts_skip_before_export_partition(cluster): def test_mutations_after_export_partition_started(cluster): """Test that mutations applied after export partition starts don't affect the exported data.""" + skip_if_remote_database_disk_enabled(cluster) node = cluster.instances["replica1"] mt_table = "mutations_after_export_partition_mt_table" @@ -930,6 +947,7 @@ def test_mutations_after_export_partition_started(cluster): def test_patch_parts_after_export_partition_started(cluster): """Test that patch parts created after export partition starts don't affect the exported data.""" + skip_if_remote_database_disk_enabled(cluster) node = cluster.instances["replica1"] mt_table = "patches_after_export_partition_mt_table" From 0b47aec602be8c027eaebccee1c4edf80c846694 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Sat, 7 Mar 2026 20:43:06 +0300 Subject: [PATCH 05/15] Merge pull request #1402 from Altinity/26_1_export_improvements_test Improvements to partition export Source-PR: #1402 (https://github.com/Altinity/ClickHouse/pull/1402) --- ci/jobs/scripts/integration_tests_configs.py | 1 + src/Common/ProfileEvents.cpp | 12 + src/Core/Settings.cpp | 8 + src/Core/SettingsChangesHistory.cpp | 2 + ...portReplicatedMergeTreePartitionManifest.h | 5 + ...ortReplicatedMergeTreePartitionTaskEntry.h | 2 +- .../MergeTree/BackgroundJobsAssignee.cpp | 4 + .../MergeTree/BackgroundJobsAssignee.h | 2 + .../ExportPartFromPartitionExportTask.cpp | 71 ++++ .../ExportPartFromPartitionExportTask.h | 36 ++ src/Storages/MergeTree/ExportPartTask.cpp | 11 + src/Storages/MergeTree/ExportPartTask.h | 1 + .../ExportPartitionManifestUpdatingTask.cpp | 400 ++++++++++++++++++ .../ExportPartitionManifestUpdatingTask.h | 5 + .../ExportPartitionTaskScheduler.cpp | 255 ++++++++--- .../MergeTree/ExportPartitionUtils.cpp | 15 + .../MergeTree/MergeTreeBackgroundExecutor.cpp | 6 + .../MergeTree/MergeTreeBackgroundExecutor.h | 2 + src/Storages/MergeTree/MergeTreeData.cpp | 66 ++- src/Storages/MergeTree/MergeTreeData.h | 1 + .../MergeTree/MergeTreePartExportManifest.h | 6 +- src/Storages/StorageReplicatedMergeTree.cpp | 134 ++---- src/Storages/StorageReplicatedMergeTree.h | 3 +- src/Storages/System/StorageSystemExports.cpp | 1 - ...torageSystemReplicatedPartitionExports.cpp | 8 +- .../StorageSystemReplicatedPartitionExports.h | 2 +- .../test.py | 142 +++++-- 27 files changed, 945 insertions(+), 256 deletions(-) create mode 100644 src/Storages/MergeTree/ExportPartFromPartitionExportTask.cpp create mode 100644 src/Storages/MergeTree/ExportPartFromPartitionExportTask.h diff --git a/ci/jobs/scripts/integration_tests_configs.py b/ci/jobs/scripts/integration_tests_configs.py index 51cd568006b1..edfb3cc471f6 100644 --- a/ci/jobs/scripts/integration_tests_configs.py +++ b/ci/jobs/scripts/integration_tests_configs.py @@ -42,6 +42,7 @@ class TC: TC("test_storage_iceberg_no_spark/", True, "no idea why i'm sequential"), TC("test_storage_iceberg_with_spark_cache/", True, "no idea why i'm sequential"), TC("test_storage_iceberg_concurrent/", True, "no idea why i'm sequential"), + TC("test_export_replicated_mt_partition_to_object_storage/", True, "ZooKeeper can't handle too many parallel requests"), ] IMAGES_ENV = { diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index f7d1ee1c57ea..e0d73df3cffd 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -328,6 +328,18 @@ M(ZooKeeperBytesSent, "Number of bytes send over network while communicating with ZooKeeper.", ValueType::Bytes) \ M(ZooKeeperBytesReceived, "Number of bytes received over network while communicating with ZooKeeper.", ValueType::Bytes) \ \ + M(ExportPartitionZooKeeperRequests, "Total number of ZooKeeper requests made by the export partition feature.", ValueType::Number) \ + M(ExportPartitionZooKeeperGet, "Number of 'get' requests to ZooKeeper made by the export partition feature.", ValueType::Number) \ + M(ExportPartitionZooKeeperGetChildren, "Number of 'getChildren' requests to ZooKeeper made by the export partition feature.", ValueType::Number) \ + M(ExportPartitionZooKeeperGetChildrenWatch, "Number of 'getChildrenWatch' requests to ZooKeeper made by the export partition feature.", ValueType::Number) \ + M(ExportPartitionZooKeeperGetWatch, "Number of 'getWatch' requests to ZooKeeper made by the export partition feature.", ValueType::Number) \ + M(ExportPartitionZooKeeperCreate, "Number of 'create' requests to ZooKeeper made by the export partition feature.", ValueType::Number) \ + M(ExportPartitionZooKeeperSet, "Number of 'set' requests to ZooKeeper made by the export partition feature.", ValueType::Number) \ + M(ExportPartitionZooKeeperRemove, "Number of 'remove' requests to ZooKeeper made by the export partition feature.", ValueType::Number) \ + M(ExportPartitionZooKeeperRemoveRecursive, "Number of 'removeRecursive' requests to ZooKeeper made by the export partition feature.", ValueType::Number) \ + M(ExportPartitionZooKeeperMulti, "Number of 'multi' requests to ZooKeeper made by the export partition feature.", ValueType::Number) \ + M(ExportPartitionZooKeeperExists, "Number of 'exists' requests to ZooKeeper made by the export partition feature.", ValueType::Number) \ + \ M(DistributedConnectionTries, "Total count of distributed connection attempts.", ValueType::Number) \ M(DistributedConnectionUsable, "Total count of successful distributed connections to a usable server (with required table, but maybe stale).", ValueType::Number) \ M(DistributedConnectionFailTry, "Total count when distributed connection fails with retry.", ValueType::Number) \ diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index fca7f7d1da9c..cb9c33ef3c05 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -7512,6 +7512,14 @@ Throw an error if there are pending mutations when exporting a merge tree part. )", 0) \ DECLARE(Bool, export_merge_tree_part_throw_on_pending_patch_parts, true, R"( Throw an error if there are pending patch parts when exporting a merge tree part. +)", 0) \ + DECLARE(Bool, export_merge_tree_partition_lock_inside_the_task, false, R"( +Only lock a part when the task is already running. This might help with busy waiting where the scheduler locks a part, but the task ends in the pending list. +On the other hand, there is a chance once the task executes that part has already been locked by another replica and the task will simply early exit. +)", 0) \ + DECLARE(Bool, export_merge_tree_partition_system_table_prefer_remote_information, true, R"( +Controls whether the system.replicated_partition_exports will prefer to query ZooKeeper to get the most up to date information or use the local information. +Querying ZooKeeper is expensive, and only available if the ZooKeeper feature flag MULTI_READ is enabled. )", 0) \ \ /* ####################################################### */ \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index e9f11322f62b..6d6f0c0921b5 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -351,6 +351,8 @@ const VersionToSettingsChangesMap & getSettingsChangesHistory() // {"output_format_parquet_write_checksums", false, true, "New setting."}, {"export_merge_tree_part_max_bytes_per_file", 0, 0, "New setting."}, {"export_merge_tree_part_max_rows_per_file", 0, 0, "New setting."}, + {"export_merge_tree_partition_lock_inside_the_task", false, false, "New setting."}, + {"export_merge_tree_partition_system_table_prefer_remote_information", true, true, "New setting."}, // {"cluster_table_function_split_granularity", "file", "file", "New setting."}, // {"cluster_table_function_buckets_batch_size", 0, 0, "New setting."}, {"export_merge_tree_part_throw_on_pending_mutations", true, true, "New setting."}, diff --git a/src/Storages/ExportReplicatedMergeTreePartitionManifest.h b/src/Storages/ExportReplicatedMergeTreePartitionManifest.h index 7c18b8a881c2..31b8731c4e8b 100644 --- a/src/Storages/ExportReplicatedMergeTreePartitionManifest.h +++ b/src/Storages/ExportReplicatedMergeTreePartitionManifest.h @@ -116,6 +116,7 @@ struct ExportReplicatedMergeTreePartitionManifest size_t max_bytes_per_file; size_t max_rows_per_file; MergeTreePartExportManifest::FileAlreadyExistsPolicy file_already_exists_policy; + bool lock_inside_the_task; /// todo temporary std::string toJsonString() const { @@ -141,6 +142,7 @@ struct ExportReplicatedMergeTreePartitionManifest json.set("create_time", create_time); json.set("max_retries", max_retries); json.set("ttl_seconds", ttl_seconds); + json.set("lock_inside_the_task", lock_inside_the_task); std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM oss.exceptions(std::ios::failbit); Poco::JSON::Stringifier::stringify(json, oss); @@ -173,6 +175,7 @@ struct ExportReplicatedMergeTreePartitionManifest manifest.parquet_parallel_encoding = json->getValue("parquet_parallel_encoding"); manifest.max_bytes_per_file = json->getValue("max_bytes_per_file"); manifest.max_rows_per_file = json->getValue("max_rows_per_file"); + if (json->has("file_already_exists_policy")) { const auto file_already_exists_policy = magic_enum::enum_cast(json->getValue("file_already_exists_policy")); @@ -184,6 +187,8 @@ struct ExportReplicatedMergeTreePartitionManifest /// what to do if it's not a valid value? } + manifest.lock_inside_the_task = json->getValue("lock_inside_the_task"); + return manifest; } }; diff --git a/src/Storages/ExportReplicatedMergeTreePartitionTaskEntry.h b/src/Storages/ExportReplicatedMergeTreePartitionTaskEntry.h index 76674bfc4a92..e62f7de99bed 100644 --- a/src/Storages/ExportReplicatedMergeTreePartitionTaskEntry.h +++ b/src/Storages/ExportReplicatedMergeTreePartitionTaskEntry.h @@ -30,7 +30,7 @@ struct ExportReplicatedMergeTreePartitionTaskEntry /// This is used to prevent the parts from being deleted before finishing the export operation /// It does not mean this replica will export all the parts /// There is also a chance this replica does not contain a given part and it is totally ok. - std::vector part_references; + mutable std::vector part_references; std::string getCompositeKey() const { diff --git a/src/Storages/MergeTree/BackgroundJobsAssignee.cpp b/src/Storages/MergeTree/BackgroundJobsAssignee.cpp index 0b2c6a2e1282..a2b43fe5a284 100644 --- a/src/Storages/MergeTree/BackgroundJobsAssignee.cpp +++ b/src/Storages/MergeTree/BackgroundJobsAssignee.cpp @@ -97,6 +97,10 @@ bool BackgroundJobsAssignee::scheduleCommonTask(ExecutableTaskPtr common_task, b return schedule_res; } +std::size_t BackgroundJobsAssignee::getAvailableMoveExecutors() const +{ + return getContext()->getMovesExecutor()->getAvailableSlots(); +} String BackgroundJobsAssignee::toString(Type type) { diff --git a/src/Storages/MergeTree/BackgroundJobsAssignee.h b/src/Storages/MergeTree/BackgroundJobsAssignee.h index 2240f09ffc18..a333f3966f17 100644 --- a/src/Storages/MergeTree/BackgroundJobsAssignee.h +++ b/src/Storages/MergeTree/BackgroundJobsAssignee.h @@ -72,6 +72,8 @@ class BackgroundJobsAssignee : public WithContext bool scheduleMoveTask(ExecutableTaskPtr move_task); bool scheduleCommonTask(ExecutableTaskPtr common_task, bool need_trigger); + std::size_t getAvailableMoveExecutors() const; + /// Just call finish ~BackgroundJobsAssignee(); diff --git a/src/Storages/MergeTree/ExportPartFromPartitionExportTask.cpp b/src/Storages/MergeTree/ExportPartFromPartitionExportTask.cpp new file mode 100644 index 000000000000..d0dba7d6834a --- /dev/null +++ b/src/Storages/MergeTree/ExportPartFromPartitionExportTask.cpp @@ -0,0 +1,71 @@ +#include +#include + +namespace ProfileEvents +{ + extern const Event ExportPartitionZooKeeperRequests; + extern const Event ExportPartitionZooKeeperGetChildren; + extern const Event ExportPartitionZooKeeperCreate; +} +namespace DB +{ + +ExportPartFromPartitionExportTask::ExportPartFromPartitionExportTask( + StorageReplicatedMergeTree & storage_, + const std::string & key_, + const MergeTreePartExportManifest & manifest_) + : storage(storage_), + key(key_), + manifest(manifest_) +{ + export_part_task = std::make_shared(storage, manifest); +} + +bool ExportPartFromPartitionExportTask::executeStep() +{ + const auto zk = storage.getZooKeeper(); + const auto part_name = manifest.data_part->name; + + LOG_INFO(storage.log, "ExportPartFromPartitionExportTask: Attempting to lock part: {}", part_name); + + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperCreate); + if (Coordination::Error::ZOK == zk->tryCreate(fs::path(storage.zookeeper_path) / "exports" / key / "locks" / part_name, storage.replica_name, zkutil::CreateMode::Ephemeral)) + { + LOG_INFO(storage.log, "ExportPartFromPartitionExportTask: Locked part: {}", part_name); + export_part_task->executeStep(); + return false; + } + + std::lock_guard inner_lock(storage.export_manifests_mutex); + storage.export_manifests.erase(manifest); + + LOG_INFO(storage.log, "ExportPartFromPartitionExportTask: Failed to lock part {}, skipping", part_name); + return false; +} + +void ExportPartFromPartitionExportTask::cancel() noexcept +{ + export_part_task->cancel(); +} + +void ExportPartFromPartitionExportTask::onCompleted() +{ + export_part_task->onCompleted(); +} + +StorageID ExportPartFromPartitionExportTask::getStorageID() const +{ + return export_part_task->getStorageID(); +} + +Priority ExportPartFromPartitionExportTask::getPriority() const +{ + return export_part_task->getPriority(); +} + +String ExportPartFromPartitionExportTask::getQueryId() const +{ + return export_part_task->getQueryId(); +} +} diff --git a/src/Storages/MergeTree/ExportPartFromPartitionExportTask.h b/src/Storages/MergeTree/ExportPartFromPartitionExportTask.h new file mode 100644 index 000000000000..e170b22b470d --- /dev/null +++ b/src/Storages/MergeTree/ExportPartFromPartitionExportTask.h @@ -0,0 +1,36 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +/* + Decorator around the ExportPartTask to lock the part inside the task +*/ +class ExportPartFromPartitionExportTask : public IExecutableTask +{ +public: + explicit ExportPartFromPartitionExportTask( + StorageReplicatedMergeTree & storage_, + const std::string & key_, + const MergeTreePartExportManifest & manifest_); + bool executeStep() override; + void onCompleted() override; + StorageID getStorageID() const override; + Priority getPriority() const override; + String getQueryId() const override; + + void cancel() noexcept override; + +private: + StorageReplicatedMergeTree & storage; + std::string key; + MergeTreePartExportManifest manifest; + std::shared_ptr export_part_task; +}; + +} diff --git a/src/Storages/MergeTree/ExportPartTask.cpp b/src/Storages/MergeTree/ExportPartTask.cpp index f4b504a27cb3..efdbe099433b 100644 --- a/src/Storages/MergeTree/ExportPartTask.cpp +++ b/src/Storages/MergeTree/ExportPartTask.cpp @@ -101,6 +101,11 @@ ExportPartTask::ExportPartTask(MergeTreeData & storage_, const MergeTreePartExpo { } +const MergeTreePartExportManifest & ExportPartTask::getManifest() const +{ + return manifest; +} + bool ExportPartTask::executeStep() { auto local_context = Context::createCopy(storage.getContext()->getBackgroundContext()); @@ -222,6 +227,11 @@ bool ExportPartTask::executeStep() exec.setCancelCallback(is_cancelled_callback, 100); + if (isCancelled()) + { + throw Exception(ErrorCodes::QUERY_WAS_CANCELLED, "Export part was cancelled"); + } + exec.execute(); if (isCancelled()) @@ -320,6 +330,7 @@ bool ExportPartTask::executeStep() void ExportPartTask::cancel() noexcept { + LOG_INFO(getLogger("ExportPartTask"), "Export part {} task cancel() method called", manifest.data_part->name); cancel_requested.store(true); pipeline.cancel(); } diff --git a/src/Storages/MergeTree/ExportPartTask.h b/src/Storages/MergeTree/ExportPartTask.h index 9c8fa6cc01dd..a3f1635c4902 100644 --- a/src/Storages/MergeTree/ExportPartTask.h +++ b/src/Storages/MergeTree/ExportPartTask.h @@ -18,6 +18,7 @@ class ExportPartTask : public IExecutableTask StorageID getStorageID() const override; Priority getPriority() const override; String getQueryId() const override; + const MergeTreePartExportManifest & getManifest() const; void cancel() noexcept override; diff --git a/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp index 79b92663b7bf..18977902b03b 100644 --- a/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp +++ b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp @@ -5,8 +5,20 @@ #include "Common/logger_useful.h" #include #include +#include #include +namespace ProfileEvents +{ + extern const Event ExportPartitionZooKeeperRequests; + extern const Event ExportPartitionZooKeeperGet; + extern const Event ExportPartitionZooKeeperGetChildren; + extern const Event ExportPartitionZooKeeperGetChildrenWatch; + extern const Event ExportPartitionZooKeeperGetWatch; + extern const Event ExportPartitionZooKeeperRemoveRecursive; + extern const Event ExportPartitionZooKeeperMulti; +} + namespace DB { namespace @@ -35,6 +47,8 @@ namespace if (has_expired && !is_pending) { zk->tryRemoveRecursive(fs::path(entry_path)); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRemoveRecursive); auto it = entries_by_key.find(key); if (it != entries_by_key.end()) entries_by_key.erase(it); @@ -44,9 +58,12 @@ namespace } else if (is_pending) { + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperGetChildren); std::vector parts_in_processing_or_pending; if (Coordination::Error::ZOK != zk->tryGetChildren(fs::path(entry_path) / "processing", parts_in_processing_or_pending)) { + LOG_INFO(log, "ExportPartition Manifest Updating Task: Failed to get parts in processing or pending, skipping"); return false; } @@ -79,6 +96,373 @@ ExportPartitionManifestUpdatingTask::ExportPartitionManifestUpdatingTask(Storage { } +std::vector ExportPartitionManifestUpdatingTask::getPartitionExportsInfo() const +{ + std::vector infos; + const auto zk = storage.getZooKeeper(); + + const auto exports_path = fs::path(storage.zookeeper_path) / "exports"; + + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperGetChildren); + + std::vector children; + if (Coordination::Error::ZOK != zk->tryGetChildren(exports_path, children)) + { + LOG_INFO(storage.log, "Failed to get children from exports path, returning empty export info list"); + return infos; + } + + if (children.empty()) + return infos; + + /// Batch all metadata.json, status gets, and getChildren operations in a single multi request + Coordination::Requests requests; + requests.reserve(children.size() * 4); // metadata, status, processing, exceptions_per_replica + + // Track response indices for each child + struct ChildResponseIndices + { + size_t metadata_idx; + size_t status_idx; + size_t processing_idx; + size_t exceptions_per_replica_idx; + }; + std::vector response_indices; + response_indices.reserve(children.size()); + + for (const auto & child : children) + { + const auto export_partition_path = fs::path(exports_path) / child; + + ChildResponseIndices indices; + indices.metadata_idx = requests.size(); + requests.push_back(zkutil::makeGetRequest(export_partition_path / "metadata.json")); + + indices.status_idx = requests.size(); + requests.push_back(zkutil::makeGetRequest(export_partition_path / "status")); + + indices.processing_idx = requests.size(); + requests.push_back(zkutil::makeListRequest(export_partition_path / "processing")); + + indices.exceptions_per_replica_idx = requests.size(); + requests.push_back(zkutil::makeListRequest(export_partition_path / "exceptions_per_replica")); + + response_indices.push_back(indices); + } + + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperMulti); + + Coordination::Responses responses; + Coordination::Error code = zk->tryMulti(requests, responses); + + if (code != Coordination::Error::ZOK) + { + LOG_INFO(storage.log, "Failed to execute multi request for export partition info, error: {}", code); + return infos; + } + + // Helper to extract GetResponse data + auto getGetResponseData = [&responses](size_t idx) -> std::pair + { + if (idx >= responses.size()) + return {Coordination::Error::ZRUNTIMEINCONSISTENCY, ""}; + + const auto * get_response = dynamic_cast(responses[idx].get()); + if (!get_response) + return {Coordination::Error::ZRUNTIMEINCONSISTENCY, ""}; + + return {get_response->error, get_response->data}; + }; + + // Helper to extract ListResponse data + auto getListResponseData = [&responses](size_t idx) -> std::pair + { + if (idx >= responses.size()) + return {Coordination::Error::ZRUNTIMEINCONSISTENCY, Strings{}}; + + const auto * list_response = dynamic_cast(responses[idx].get()); + if (!list_response) + return {Coordination::Error::ZRUNTIMEINCONSISTENCY, Strings{}}; + + return {list_response->error, list_response->names}; + }; + + // Create response wrappers matching the MultiTryGetResponse/MultiTryGetChildrenResponse interface + struct ResponseWrapper + { + Coordination::Error error; + std::string data; + Strings names; + + ResponseWrapper(Coordination::Error err, const std::string & d, const Strings & n) + : error(err), data(d), names(n) {} + }; + + std::vector metadata_responses_wrapper; + std::vector status_responses_wrapper; + std::vector processing_responses_wrapper; + std::vector exceptions_per_replica_responses_wrapper; + + metadata_responses_wrapper.reserve(children.size()); + status_responses_wrapper.reserve(children.size()); + processing_responses_wrapper.reserve(children.size()); + exceptions_per_replica_responses_wrapper.reserve(children.size()); + + for (size_t child_idx = 0; child_idx < children.size(); ++child_idx) + { + const auto & indices = response_indices[child_idx]; + + // Extract metadata response + auto [metadata_error, metadata_data] = getGetResponseData(indices.metadata_idx); + metadata_responses_wrapper.emplace_back(metadata_error, metadata_data, Strings{}); + + // Extract status response + auto [status_error, status_data] = getGetResponseData(indices.status_idx); + status_responses_wrapper.emplace_back(status_error, status_data, Strings{}); + + // Extract processing response + auto [processing_error, processing_names] = getListResponseData(indices.processing_idx); + processing_responses_wrapper.emplace_back(processing_error, "", processing_names); + + // Extract exceptions_per_replica response + auto [exceptions_error, exceptions_names] = getListResponseData(indices.exceptions_per_replica_idx); + exceptions_per_replica_responses_wrapper.emplace_back(exceptions_error, "", exceptions_names); + } + + // Use wrapper vectors directly - they match the interface expected by the code below + auto & metadata_responses = metadata_responses_wrapper; + auto & status_responses = status_responses_wrapper; + auto & processing_responses = processing_responses_wrapper; + auto & exceptions_per_replica_responses = exceptions_per_replica_responses_wrapper; + + /// Collect all exception replica paths for batching + struct ExceptionReplicaPath + { + size_t child_idx; + std::string replica; + std::string count_path; + std::string exception_path; + std::string part_path; + }; + + std::vector exception_replica_paths; + for (size_t child_idx = 0; child_idx < children.size(); ++child_idx) + { + const auto & child = children[child_idx]; + const auto export_partition_path = fs::path(exports_path) / child; + /// Check if we got valid responses + if (metadata_responses[child_idx].error != Coordination::Error::ZOK) + { + LOG_INFO(storage.log, "Skipping {}: missing metadata.json", child); + continue; + } + if (status_responses[child_idx].error != Coordination::Error::ZOK) + { + LOG_INFO(storage.log, "Skipping {}: missing status", child); + continue; + } + if (processing_responses[child_idx].error != Coordination::Error::ZOK) + { + LOG_INFO(storage.log, "Skipping {}: missing processing parts", child); + continue; + } + if (exceptions_per_replica_responses[child_idx].error != Coordination::Error::ZOK) + { + LOG_INFO(storage.log, "Skipping {}: missing exceptions_per_replica", export_partition_path); + continue; + } + const auto exceptions_per_replica_path = export_partition_path / "exceptions_per_replica"; + const auto & exception_replicas = exceptions_per_replica_responses[child_idx].names; + for (const auto & replica : exception_replicas) + { + const auto last_exception_path = exceptions_per_replica_path / replica / "last_exception"; + exception_replica_paths.push_back({ + child_idx, + replica, + (exceptions_per_replica_path / replica / "count").string(), + (last_exception_path / "exception").string(), + (last_exception_path / "part").string() + }); + } + } + /// Batch get all exception data in a single multi request + std::map>> exception_data_by_child; + + if (!exception_replica_paths.empty()) + { + Coordination::Requests exception_requests; + exception_requests.reserve(exception_replica_paths.size() * 3); // count, exception, part for each + + // Track response indices for each exception replica path + struct ExceptionResponseIndices + { + size_t count_idx; + size_t exception_idx; + size_t part_idx; + }; + std::vector exception_response_indices; + exception_response_indices.reserve(exception_replica_paths.size()); + + for (const auto & erp : exception_replica_paths) + { + ExceptionResponseIndices indices; + indices.count_idx = exception_requests.size(); + exception_requests.push_back(zkutil::makeGetRequest(erp.count_path)); + + indices.exception_idx = exception_requests.size(); + exception_requests.push_back(zkutil::makeGetRequest(erp.exception_path)); + + indices.part_idx = exception_requests.size(); + exception_requests.push_back(zkutil::makeGetRequest(erp.part_path)); + + exception_response_indices.push_back(indices); + } + + // Execute single multi request for all exception data + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperMulti); + + Coordination::Responses exception_responses; + Coordination::Error exception_code = zk->tryMulti(exception_requests, exception_responses); + + if (exception_code != Coordination::Error::ZOK) + { + LOG_INFO(storage.log, "Failed to execute multi request for exception data, error: {}", exception_code); + } + else + { + // Parse exception responses + for (size_t exception_path_idx = 0; exception_path_idx < exception_replica_paths.size(); ++exception_path_idx) + { + const auto & erp = exception_replica_paths[exception_path_idx]; + const auto & indices = exception_response_indices[exception_path_idx]; + + std::string count_str; + std::string exception_str; + std::string part_str; + + // Extract count response + if (indices.count_idx < exception_responses.size()) + { + const auto * count_response = dynamic_cast(exception_responses[indices.count_idx].get()); + if (count_response && count_response->error == Coordination::Error::ZOK) + count_str = count_response->data; + } + + // Extract exception response + if (indices.exception_idx < exception_responses.size()) + { + const auto * exception_response = dynamic_cast(exception_responses[indices.exception_idx].get()); + if (exception_response && exception_response->error == Coordination::Error::ZOK) + exception_str = exception_response->data; + } + + // Extract part response + if (indices.part_idx < exception_responses.size()) + { + const auto * part_response = dynamic_cast(exception_responses[indices.part_idx].get()); + if (part_response && part_response->error == Coordination::Error::ZOK) + part_str = part_response->data; + } + + exception_data_by_child[erp.child_idx].emplace_back(erp.replica, count_str, exception_str, part_str); + } + } + } + + /// Build the result + for (size_t child_idx = 0; child_idx < children.size(); ++child_idx) + { + /// Skip if we already determined this child is invalid + if (metadata_responses[child_idx].error != Coordination::Error::ZOK + || status_responses[child_idx].error != Coordination::Error::ZOK + || processing_responses[child_idx].error != Coordination::Error::ZOK + || exceptions_per_replica_responses[child_idx].error != Coordination::Error::ZOK) + { + continue; + } + + ReplicatedPartitionExportInfo info; + const auto metadata_json = metadata_responses[child_idx].data; + const auto status = status_responses[child_idx].data; + const auto processing_parts = processing_responses[child_idx].names; + const auto parts_to_do = processing_parts.size(); + std::string exception_replica; + std::string last_exception; + std::string exception_part; + std::size_t exception_count = 0; + /// Process exception data + auto exception_data_it = exception_data_by_child.find(child_idx); + if (exception_data_it != exception_data_by_child.end()) + { + for (const auto & [replica, count_str, exception_str, part_str] : exception_data_it->second) + { + if (!count_str.empty()) + { + exception_count += parse(count_str); + } + if (last_exception.empty() && !exception_str.empty() && !part_str.empty()) + { + exception_replica = replica; + last_exception = exception_str; + exception_part = part_str; + } + } + } + + const auto metadata = ExportReplicatedMergeTreePartitionManifest::fromJsonString(metadata_json); + + info.destination_database = metadata.destination_database; + info.destination_table = metadata.destination_table; + info.partition_id = metadata.partition_id; + info.transaction_id = metadata.transaction_id; + info.query_id = metadata.query_id; + info.create_time = metadata.create_time; + info.source_replica = metadata.source_replica; + info.parts_count = metadata.number_of_parts; + info.parts_to_do = parts_to_do; + info.parts = metadata.parts; + info.status = status; + info.exception_replica = exception_replica; + info.last_exception = last_exception; + info.exception_part = exception_part; + info.exception_count = exception_count; + infos.emplace_back(std::move(info)); + } + + return infos; +} + +std::vector ExportPartitionManifestUpdatingTask::getPartitionExportsInfoLocal() const +{ + std::lock_guard lock(storage.export_merge_tree_partition_mutex); + + std::vector infos; + + for (const auto & entry : storage.export_merge_tree_partition_task_entries_by_key) + { + ReplicatedPartitionExportInfo info; + + info.destination_database = entry.manifest.destination_database; + info.destination_table = entry.manifest.destination_table; + info.partition_id = entry.manifest.partition_id; + info.transaction_id = entry.manifest.transaction_id; + info.query_id = entry.manifest.query_id; + info.create_time = entry.manifest.create_time; + info.source_replica = entry.manifest.source_replica; + info.parts_count = entry.manifest.number_of_parts; + info.parts_to_do = entry.manifest.parts.size(); + info.parts = entry.manifest.parts; + info.status = magic_enum::enum_name(entry.status); + + infos.emplace_back(std::move(info)); + } + + return infos; +} + void ExportPartitionManifestUpdatingTask::poll() { std::lock_guard lock(storage.export_merge_tree_partition_mutex); @@ -96,6 +480,9 @@ void ExportPartitionManifestUpdatingTask::poll() LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Cleanup lock acquired, will remove stale entries"); } + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperGetChildrenWatch); + Coordination::Stat stat; const auto children = zk->getChildrenWatch(exports_path, &stat, storage.export_merge_tree_partition_watch_callback); const std::unordered_set zk_children(children.begin(), children.end()); @@ -111,6 +498,8 @@ void ExportPartitionManifestUpdatingTask::poll() { const std::string entry_path = fs::path(exports_path) / key; + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperGet); std::string metadata_json; if (!zk->tryGet(fs::path(entry_path) / "metadata.json", metadata_json)) { @@ -144,6 +533,8 @@ void ExportPartitionManifestUpdatingTask::poll() } }); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperGetWatch); std::string status; if (!zk->tryGetWatch(fs::path(entry_path) / "status", status, nullptr, status_watch_callback)) { @@ -273,6 +664,8 @@ void ExportPartitionManifestUpdatingTask::handleStatusChanges() if (it == storage.export_merge_tree_partition_task_entries_by_key.end()) continue; + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperGet); /// get new status from zk std::string new_status_string; if (!zk->tryGet(fs::path(storage.zookeeper_path) / "exports" / key / "status", new_status_string)) @@ -295,6 +688,7 @@ void ExportPartitionManifestUpdatingTask::handleStatusChanges() { try { + LOG_INFO(storage.log, "ExportPartition Manifest Updating task: killing export partition for task {}", key); storage.killExportPart(it->manifest.transaction_id); } catch (...) @@ -304,6 +698,12 @@ void ExportPartitionManifestUpdatingTask::handleStatusChanges() } it->status = *new_status; + + if (it->status != ExportReplicatedMergeTreePartitionTaskEntry::Status::PENDING) + { + /// we no longer need to keep the data parts alive + it->part_references.clear(); + } } } diff --git a/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.h b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.h index ea52f679d654..99078e486cb3 100644 --- a/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.h +++ b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB { @@ -21,6 +22,10 @@ class ExportPartitionManifestUpdatingTask void addStatusChange(const std::string & key); + std::vector getPartitionExportsInfo() const; + + std::vector getPartitionExportsInfoLocal() const; + private: StorageReplicatedMergeTree & storage; diff --git a/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp b/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp index 925d7eafe412..89042fe694b9 100644 --- a/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp +++ b/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp @@ -4,13 +4,34 @@ #include #include #include +#include #include "Storages/MergeTree/ExportPartitionUtils.h" #include "Storages/MergeTree/MergeTreePartExportManifest.h" +#include "Storages/MergeTree/ExportPartFromPartitionExportTask.h" +#include "Formats/FormatFactory.h" +#include + +namespace ProfileEvents +{ + extern const Event ExportPartitionZooKeeperRequests; + extern const Event ExportPartitionZooKeeperGet; + extern const Event ExportPartitionZooKeeperGetChildren; + extern const Event ExportPartitionZooKeeperCreate; + extern const Event ExportPartitionZooKeeperSet; + extern const Event ExportPartitionZooKeeperRemove; + extern const Event ExportPartitionZooKeeperMulti; + extern const Event ExportPartitionZooKeeperExists; +} namespace DB { +namespace Setting +{ + extern const SettingsMergeTreePartExportFileAlreadyExistsPolicy export_merge_tree_part_file_already_exists_policy; +} + namespace ErrorCodes { extern const int QUERY_WAS_CANCELLED; @@ -46,6 +67,22 @@ ExportPartitionTaskScheduler::ExportPartitionTaskScheduler(StorageReplicatedMerg void ExportPartitionTaskScheduler::run() { + const auto available_move_executors = storage.background_moves_assignee.getAvailableMoveExecutors(); + + /// this is subject to TOCTOU - but for now we choose to live with it. + if (available_move_executors == 0) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: No available move executors, skipping"); + return; + } + + LOG_INFO(storage.log, "ExportPartition scheduler task: Available move executors: {}", available_move_executors); + + std::size_t scheduled_exports_count = 0; + + const uint32_t seed = uint32_t(std::hash{}(storage.replica_name)) ^ uint32_t(scheduled_exports_count); + pcg64_fast rng(seed); + std::lock_guard lock(storage.export_merge_tree_partition_mutex); auto zk = storage.getZooKeeper(); @@ -53,6 +90,12 @@ void ExportPartitionTaskScheduler::run() // Iterate sorted by create_time for (auto & entry : storage.export_merge_tree_partition_task_entries_by_create_time) { + if (scheduled_exports_count >= available_move_executors) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Scheduled exports count is greater than available move executors, skipping"); + break; + } + const auto & manifest = entry.manifest; const auto key = entry.getCompositeKey(); const auto database = storage.getContext()->resolveDatabase(manifest.destination_database); @@ -75,6 +118,8 @@ void ExportPartitionTaskScheduler::run() continue; } + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperGet); std::string status_in_zk_string; if (!zk->tryGet(fs::path(storage.zookeeper_path) / "exports" / key / "status", status_in_zk_string)) { @@ -93,10 +138,12 @@ void ExportPartitionTaskScheduler::run() if (status_in_zk.value() != ExportReplicatedMergeTreePartitionTaskEntry::Status::PENDING) { entry.status = status_in_zk.value(); - LOG_INFO(storage.log, "ExportPartition scheduler task: Skipping... Status from zk is {}", entry.status); + LOG_INFO(storage.log, "ExportPartition scheduler task: Skipping {}... Status from zk is {}", key, magic_enum::enum_name(entry.status).data()); continue; } + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperGetChildren); std::vector parts_in_processing_or_pending; if (Coordination::Error::ZOK != zk->tryGetChildren(fs::path(storage.zookeeper_path) / "exports" / key / "processing", parts_in_processing_or_pending)) @@ -105,12 +152,18 @@ void ExportPartitionTaskScheduler::run() continue; } + if (parts_in_processing_or_pending.empty()) { LOG_INFO(storage.log, "ExportPartition scheduler task: No parts in processing or pending, skipping"); continue; } + /// shuffle the parts to reduce the risk of lock collisions + std::shuffle(parts_in_processing_or_pending.begin(), parts_in_processing_or_pending.end(), rng); + + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperGetChildren); std::vector locked_parts; if (Coordination::Error::ZOK != zk->tryGetChildren(fs::path(storage.zookeeper_path) / "exports" / key / "locks", locked_parts)) @@ -123,6 +176,12 @@ void ExportPartitionTaskScheduler::run() for (const auto & zk_part_name : parts_in_processing_or_pending) { + if (scheduled_exports_count >= available_move_executors) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Scheduled exports count is greater than available move executors, skipping"); + break; + } + if (locked_parts_set.contains(zk_part_name)) { LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} is locked, skipping", zk_part_name); @@ -136,37 +195,91 @@ void ExportPartitionTaskScheduler::run() continue; } - if (Coordination::Error::ZOK != zk->tryCreate(fs::path(storage.zookeeper_path) / "exports" / key / "locks" / zk_part_name, storage.replica_name, zkutil::CreateMode::Ephemeral)) - { - LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to lock part {}, skipping", zk_part_name); - continue; - } + LOG_INFO(storage.log, "ExportPartition scheduler task: Scheduling part export: {}", zk_part_name); + + auto context = getContextCopyWithTaskSettings(storage.getContext(), manifest); - try + /// todo arthur this code path does not perform all the validations a simple part export does because we are not calling exportPartToTable directly. + /// the schema and everything else has been validated when the export partition task was created, but nothing prevents the destination table from being + /// recreated with a new schema before the export task is scheduled. + if (manifest.lock_inside_the_task) { - storage.exportPartToTable( - part->name, - destination_storage_id, + LOG_INFO(storage.log, "ExportPartition scheduler task: Locking part export inside the task"); + std::lock_guard part_export_lock(storage.export_manifests_mutex); + + MergeTreePartExportManifest part_export_manifest( + destination_storage, + part, manifest.transaction_id, - getContextCopyWithTaskSettings(storage.getContext(), manifest), - /*allow_outdated_parts*/ true, + manifest.query_id, + context->getSettingsRef()[Setting::export_merge_tree_part_file_already_exists_policy].value, + context->getSettingsCopy(), + storage.getInMemoryMetadataPtr(), [this, key, zk_part_name, manifest, destination_storage] (MergeTreePartExportManifest::CompletionCallbackResult result) { handlePartExportCompletion(key, zk_part_name, manifest, destination_storage, result); }); + + part_export_manifest.task = std::make_shared(storage, key, part_export_manifest); + + /// todo arthur this might conflict with the standalone export part. what to do in this case? + if (!storage.export_manifests.emplace(part_export_manifest).second) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} is already being exported, skipping", zk_part_name); + continue; + } + + if (!storage.background_moves_assignee.scheduleMoveTask(part_export_manifest.task)) + { + storage.export_manifests.erase(part_export_manifest); + LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to schedule export part task, skipping"); + return; + } } - catch (const Exception &) + else { - tryLogCurrentException(__PRETTY_FUNCTION__); - zk->tryRemove(fs::path(storage.zookeeper_path) / "exports" / key / "locks" / zk_part_name); - /// we should not increment retry_count because the node might just be full + try + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Exporting part to table"); + + LOG_INFO(storage.log, "ExportPartition scheduler task: Attempting to lock part: {}", zk_part_name); + + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperCreate); + if (Coordination::Error::ZOK != zk->tryCreate(fs::path(storage.zookeeper_path) / "exports" / key / "locks" / zk_part_name, storage.replica_name, zkutil::CreateMode::Ephemeral)) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to lock part {}, skipping", zk_part_name); + continue; + } + + LOG_INFO(storage.log, "ExportPartition scheduler task: Locked part: {}", zk_part_name); + + storage.exportPartToTable( + part->name, + destination_storage_id, + manifest.transaction_id, + getContextCopyWithTaskSettings(storage.getContext(), manifest), + /*allow_outdated_parts*/ true, + [this, key, zk_part_name, manifest, destination_storage] + (MergeTreePartExportManifest::CompletionCallbackResult result) + { + handlePartExportCompletion(key, zk_part_name, manifest, destination_storage, result); + }); + } + catch (const Exception &) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRemove); + zk->tryRemove(fs::path(storage.zookeeper_path) / "exports" / key / "locks" / zk_part_name); + /// we should not increment retry_count because the node might just be full + } } + + scheduled_exports_count++; } } - - /// maybe we failed to schedule or failed to export, need to retry eventually - storage.export_merge_tree_partition_select_task->scheduleAfter(1000 * 5); } void ExportPartitionTaskScheduler::handlePartExportCompletion( @@ -236,6 +349,8 @@ void ExportPartitionTaskScheduler::handlePartExportFailure( size_t max_retries ) { + LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} export failed, will now increment counters", part_name); + if (!exception) { throw Exception(ErrorCodes::LOGICAL_ERROR, "ExportPartition scheduler task: No exception provided for error handling. Sounds like a bug"); @@ -251,6 +366,8 @@ void ExportPartitionTaskScheduler::handlePartExportFailure( Coordination::Stat locked_by_stat; std::string locked_by; + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperGet); if (!zk->tryGet(export_path / "locks" / part_name, locked_by, &locked_by_stat)) { LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} is not locked by any replica, will not increment error counts", part_name); @@ -267,6 +384,8 @@ void ExportPartitionTaskScheduler::handlePartExportFailure( const auto processing_part_path = processing_parts_path / part_name; + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperGet); std::string processing_part_string; if (!zk->tryGet(processing_part_path, processing_part_string)) @@ -280,55 +399,72 @@ void ExportPartitionTaskScheduler::handlePartExportFailure( processing_part_entry.retry_count++; - if (processing_part_entry.retry_count) - { - ops.emplace_back(zkutil::makeRemoveRequest(export_path / "locks" / part_name, locked_by_stat.version)); - ops.emplace_back(zkutil::makeSetRequest(processing_part_path, processing_part_entry.toJsonString(), -1)); + ops.emplace_back(zkutil::makeRemoveRequest(export_path / "locks" / part_name, locked_by_stat.version)); + ops.emplace_back(zkutil::makeSetRequest(processing_part_path, processing_part_entry.toJsonString(), -1)); - if (processing_part_entry.retry_count >= max_retries) - { - /// just set status in processing_part_path and finished_by - processing_part_entry.status = ExportReplicatedMergeTreePartitionProcessingPartEntry::Status::FAILED; - processing_part_entry.finished_by = storage.replica_name; + LOG_INFO(storage.log, "ExportPartition scheduler task: Updating processing part entry for part {}, retry count: {}, max retries: {}", part_name, processing_part_entry.retry_count, max_retries); - ops.emplace_back(zkutil::makeSetRequest(export_path / "status", String(magic_enum::enum_name(ExportReplicatedMergeTreePartitionTaskEntry::Status::FAILED)).data(), -1)); - LOG_INFO(storage.log, "ExportPartition scheduler task: Retry count limit exceeded for part {}, will try to fail the entire task", part_name); - } + if (processing_part_entry.retry_count >= max_retries) + { + /// just set status in processing_part_path and finished_by + processing_part_entry.status = ExportReplicatedMergeTreePartitionProcessingPartEntry::Status::FAILED; + processing_part_entry.finished_by = storage.replica_name; - std::size_t num_exceptions = 0; - - const auto exceptions_per_replica_path = export_path / "exceptions_per_replica" / storage.replica_name; - const auto count_path = exceptions_per_replica_path / "count"; - const auto last_exception_path = exceptions_per_replica_path / "last_exception"; + ops.emplace_back(zkutil::makeSetRequest(export_path / "status", String(magic_enum::enum_name(ExportReplicatedMergeTreePartitionTaskEntry::Status::FAILED)).data(), -1)); + LOG_INFO(storage.log, "ExportPartition scheduler task: Retry count limit exceeded for part {}, will try to fail the entire task", part_name); + } + else + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Retry count limit not exceeded for part {}, will increment retry count", part_name); + } - if (zk->exists(exceptions_per_replica_path)) - { - std::string num_exceptions_string; - zk->tryGet(count_path, num_exceptions_string); - num_exceptions = std::stoull(num_exceptions_string.c_str()); + const auto exceptions_per_replica_path = export_path / "exceptions_per_replica" / storage.replica_name; + const auto count_path = exceptions_per_replica_path / "count"; + const auto last_exception_path = exceptions_per_replica_path / "last_exception"; - ops.emplace_back(zkutil::makeSetRequest(last_exception_path / "part", part_name, -1)); - ops.emplace_back(zkutil::makeSetRequest(last_exception_path / "exception", exception->message(), -1)); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperExists); + if (zk->exists(exceptions_per_replica_path)) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Exceptions per replica path exists, no need to create it"); + std::string num_exceptions_string; + if (zk->tryGet(count_path, num_exceptions_string)) + { + const auto num_exceptions = parse(num_exceptions_string) + 1; + ops.emplace_back(zkutil::makeSetRequest(count_path, std::to_string(num_exceptions), -1)); } else { - ops.emplace_back(zkutil::makeCreateRequest(exceptions_per_replica_path, "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(count_path, "0", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(last_exception_path, "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(last_exception_path / "part", part_name, zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(last_exception_path / "exception", exception->message(), zkutil::CreateMode::Persistent)); + /// TODO maybe we should find a better way to handle this case, not urgent + LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to get number of exceptions, will not increment it"); } - num_exceptions++; - ops.emplace_back(zkutil::makeSetRequest(count_path, std::to_string(num_exceptions), -1)); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperGet); - Coordination::Responses responses; - if (Coordination::Error::ZOK != zk->tryMulti(ops, responses)) - { - LOG_INFO(storage.log, "ExportPartition scheduler task: All failure mechanism failed, will not try to update it"); - return; - } + ops.emplace_back(zkutil::makeSetRequest(last_exception_path / "part", part_name, -1)); + ops.emplace_back(zkutil::makeSetRequest(last_exception_path / "exception", exception->message(), -1)); + } + else + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Exceptions per replica path does not exist, will create it"); + ops.emplace_back(zkutil::makeCreateRequest(exceptions_per_replica_path, "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(count_path, "1", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(last_exception_path, "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(last_exception_path / "part", part_name, zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(last_exception_path / "exception", exception->message(), zkutil::CreateMode::Persistent)); } + + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperMulti); + Coordination::Responses responses; + if (Coordination::Error::ZOK != zk->tryMulti(ops, responses)) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: All failure mechanism failed, will not try to update it"); + return; + } + + LOG_INFO(storage.log, "ExportPartition scheduler task: Successfully updated exception counters for part {}", part_name); } bool ExportPartitionTaskScheduler::tryToMovePartToProcessed( @@ -343,6 +479,8 @@ bool ExportPartitionTaskScheduler::tryToMovePartToProcessed( Coordination::Stat locked_by_stat; std::string locked_by; + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperGet); if (!zk->tryGet(export_path / "locks" / part_name, locked_by, &locked_by_stat)) { LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} is not locked by any replica, will not commit or set it as completed", part_name); @@ -368,9 +506,12 @@ bool ExportPartitionTaskScheduler::tryToMovePartToProcessed( requests.emplace_back(zkutil::makeCreateRequest(processed_part_path, processed_part_entry.toJsonString(), zkutil::CreateMode::Persistent)); requests.emplace_back(zkutil::makeRemoveRequest(export_path / "locks" / part_name, locked_by_stat.version)); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperMulti); Coordination::Responses responses; if (Coordination::Error::ZOK != zk->tryMulti(requests, responses)) { + /// todo arthur remember what to do here LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to update export path, skipping"); return false; @@ -383,6 +524,8 @@ bool ExportPartitionTaskScheduler::areAllPartsProcessed( const std::filesystem::path & export_path, const zkutil::ZooKeeperPtr & zk) { + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperGetChildren); Strings parts_in_processing_or_pending; if (Coordination::Error::ZOK != zk->tryGetChildren(export_path / "processing", parts_in_processing_or_pending)) { diff --git a/src/Storages/MergeTree/ExportPartitionUtils.cpp b/src/Storages/MergeTree/ExportPartitionUtils.cpp index 51ed72b7f6f6..6a0baf19f761 100644 --- a/src/Storages/MergeTree/ExportPartitionUtils.cpp +++ b/src/Storages/MergeTree/ExportPartitionUtils.cpp @@ -1,10 +1,19 @@ #include #include +#include #include #include "Storages/ExportReplicatedMergeTreePartitionManifest.h" #include "Storages/ExportReplicatedMergeTreePartitionTaskEntry.h" #include +namespace ProfileEvents +{ + extern const Event ExportPartitionZooKeeperRequests; + extern const Event ExportPartitionZooKeeperGet; + extern const Event ExportPartitionZooKeeperGetChildren; + extern const Event ExportPartitionZooKeeperSet; +} + namespace DB { @@ -23,6 +32,8 @@ namespace ExportPartitionUtils const auto processed_parts_path = fs::path(export_path) / "processed"; + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperGetChildren); std::vector processed_parts; if (Coordination::Error::ZOK != zk->tryGetChildren(processed_parts_path, processed_parts)) { @@ -39,6 +50,8 @@ namespace ExportPartitionUtils } auto responses = zk->tryGet(get_paths); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperGet, get_paths.size()); responses.waitForResponses(); @@ -90,6 +103,8 @@ namespace ExportPartitionUtils destination_storage->commitExportPartitionTransaction(manifest.transaction_id, manifest.partition_id, exported_paths, context); LOG_INFO(log, "ExportPartition: Committed export, mark as completed"); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperSet); if (Coordination::Error::ZOK == zk->trySet(fs::path(entry_path) / "status", String(magic_enum::enum_name(ExportReplicatedMergeTreePartitionTaskEntry::Status::COMPLETED)).data(), -1)) { LOG_INFO(log, "ExportPartition: Marked export as completed"); diff --git a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp index 91901e4abe8f..bb000213cb85 100644 --- a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.cpp @@ -149,6 +149,12 @@ size_t MergeTreeBackgroundExecutor::getMaxTasksCount() const return max_tasks_count.load(std::memory_order_relaxed); } +template +size_t MergeTreeBackgroundExecutor::getAvailableSlots() const +{ + return getMaxTasksCount() - CurrentMetrics::values[metric].load(std::memory_order_relaxed); +} + template bool MergeTreeBackgroundExecutor::trySchedule(ExecutableTaskPtr task) { diff --git a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h index 9f161a3cb358..b33b56d0812d 100644 --- a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h +++ b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h @@ -331,6 +331,8 @@ class MergeTreeBackgroundExecutor final : boost::noncopyable /// can lead only to some postponing, not logical error. size_t getMaxTasksCount() const; + size_t getAvailableSlots() const; + bool trySchedule(ExecutableTaskPtr task); void removeTasksCorrespondingToStorage(StorageID id); void wait(); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index cacb45e4174f..ba35d334b825 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6744,7 +6744,6 @@ void MergeTreeData::exportPartToTable( } { - const auto format_settings = getFormatSettings(query_context); MergeTreePartExportManifest manifest( dest_storage, part, @@ -6757,13 +6756,21 @@ void MergeTreeData::exportPartToTable( std::lock_guard lock(export_manifests_mutex); - if (!export_manifests.emplace(std::move(manifest)).second) + manifest.task = std::make_shared(*this, manifest); + + if (!export_manifests.emplace(manifest).second) { - throw Exception(ErrorCodes::ABORTED, "Data part '{}' is already being exported", part_name); + throw Exception(ErrorCodes::ABORTED, "Data part '{}' is already being exported", + part_name); } - } - background_moves_assignee.trigger(); + if (!background_moves_assignee.scheduleMoveTask(manifest.task)) + { + export_manifests.erase(manifest); + throw Exception(ErrorCodes::ABORTED, "Failed to schedule export part task for data part '{}'. Background executor is busy", + part_name); + } + } } void MergeTreeData::killExportPart(const String & transaction_id) @@ -9876,46 +9883,21 @@ MergeTreeData::CurrentlyMovingPartsTagger::~CurrentlyMovingPartsTagger() bool MergeTreeData::scheduleDataMovingJob(BackgroundJobsAssignee & assignee) { - if (!parts_mover.moves_blocker.isCancelled()) - { - auto moving_tagger = selectPartsForMove(); - if (!moving_tagger->parts_to_move.empty()) - { - assignee.scheduleMoveTask(std::make_shared( - [this, moving_tagger] () mutable - { - ReadSettings read_settings = Context::getGlobalContextInstance()->getReadSettings(); - WriteSettings write_settings = Context::getGlobalContextInstance()->getWriteSettings(); - return moveParts(moving_tagger, read_settings, write_settings, /* wait_for_move_if_zero_copy= */ false) == MovePartsOutcome::PartsMoved; - }, moves_assignee_trigger, getStorageID())); - return true; - } - } - - std::lock_guard lock(export_manifests_mutex); - - for (auto & manifest : export_manifests) - { - if (manifest.in_progress) - { - continue; - } - - auto task = std::make_shared(*this, manifest); + if (parts_mover.moves_blocker.isCancelled()) + return false; - manifest.in_progress = assignee.scheduleMoveTask(task); + auto moving_tagger = selectPartsForMove(); + if (moving_tagger->parts_to_move.empty()) + return false; - if (!manifest.in_progress) + assignee.scheduleMoveTask(std::make_shared( + [this, moving_tagger] () mutable { - continue; - } - - manifest.task = task; - - return true; - } - - return false; + ReadSettings read_settings = Context::getGlobalContextInstance()->getReadSettings(); + WriteSettings write_settings = Context::getGlobalContextInstance()->getWriteSettings(); + return moveParts(moving_tagger, read_settings, write_settings, /* wait_for_move_if_zero_copy= */ false) == MovePartsOutcome::PartsMoved; + }, moves_assignee_trigger, getStorageID())); + return true; } bool MergeTreeData::areBackgroundMovesNeeded() const diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 0750c2b30693..b63a78c8d83c 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -1431,6 +1431,7 @@ class MergeTreeData : public WithMutableContext, public IStorage, public IBackgr friend class IMergedBlockOutputStream; // for access to log friend struct DataPartsLock; // for access to shared_parts_list/shared_ranges_in_parts friend class ExportPartTask; + friend class ExportPartFromPartitionExportTask; bool require_part_metadata; diff --git a/src/Storages/MergeTree/MergeTreePartExportManifest.h b/src/Storages/MergeTree/MergeTreePartExportManifest.h index b9527d5140f6..e65b4172c78d 100644 --- a/src/Storages/MergeTree/MergeTreePartExportManifest.h +++ b/src/Storages/MergeTree/MergeTreePartExportManifest.h @@ -13,7 +13,7 @@ namespace DB class Exception; -class ExportPartTask; +class IExecutableTask; struct MergeTreePartExportManifest { @@ -77,8 +77,8 @@ struct MergeTreePartExportManifest std::function completion_callback; time_t create_time; - mutable bool in_progress = false; - mutable std::shared_ptr task = nullptr; + /// Required to cancel export tasks + mutable std::shared_ptr task = nullptr; bool operator<(const MergeTreePartExportManifest & rhs) const { diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index bdbe0238e76e..11a19e8322ea 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -160,6 +160,15 @@ namespace ProfileEvents extern const Event ReplicaPartialShutdown; extern const Event ReplicatedCoveredPartsInZooKeeperOnStart; extern const Event MergesRejectedByMemoryLimit; + extern const Event ExportPartitionZooKeeperRequests; + extern const Event ExportPartitionZooKeeperGet; + extern const Event ExportPartitionZooKeeperGetChildren; + extern const Event ExportPartitionZooKeeperCreate; + extern const Event ExportPartitionZooKeeperSet; + extern const Event ExportPartitionZooKeeperRemove; + extern const Event ExportPartitionZooKeeperRemoveRecursive; + extern const Event ExportPartitionZooKeeperMulti; + extern const Event ExportPartitionZooKeeperExists; } namespace CurrentMetrics @@ -208,6 +217,7 @@ namespace Setting extern const SettingsUInt64 export_merge_tree_part_max_rows_per_file; extern const SettingsBool export_merge_tree_part_throw_on_pending_mutations; extern const SettingsBool export_merge_tree_part_throw_on_pending_patch_parts; + extern const SettingsBool export_merge_tree_partition_lock_inside_the_task; } @@ -4592,120 +4602,16 @@ void StorageReplicatedMergeTree::exportMergeTreePartitionStatusHandlingTask() } } -std::vector StorageReplicatedMergeTree::getPartitionExportsInfo() const +std::vector StorageReplicatedMergeTree::getPartitionExportsInfo(bool prefer_remote_information) const { - std::vector infos; - - const auto zk = getZooKeeper(); - const auto exports_path = fs::path(zookeeper_path) / "exports"; - std::vector children; - if (Coordination::Error::ZOK != zk->tryGetChildren(exports_path, children)) + if (prefer_remote_information && getZooKeeper()->isFeatureEnabled(DB::KeeperFeatureFlag::MULTI_READ)) { - LOG_INFO(log, "Failed to get children from exports path, returning empty export info list"); - return infos; + return export_merge_tree_partition_manifest_updater->getPartitionExportsInfo(); } - for (const auto & child : children) - { - ReplicatedPartitionExportInfo info; - - const auto export_partition_path = fs::path(exports_path) / child; - std::string metadata_json; - if (!zk->tryGet(export_partition_path / "metadata.json", metadata_json)) - { - LOG_INFO(log, "Skipping {}: missing metadata.json", child); - continue; - } - - std::string status; - if (!zk->tryGet(export_partition_path / "status", status)) - { - LOG_INFO(log, "Skipping {}: missing status", child); - continue; - } - - std::vector processing_parts; - if (Coordination::Error::ZOK != zk->tryGetChildren(export_partition_path / "processing", processing_parts)) - { - LOG_INFO(log, "Skipping {}: missing processing parts", child); - continue; - } - - const auto parts_to_do = processing_parts.size(); - - std::string exception_replica; - std::string last_exception; - std::string exception_part; - std::size_t exception_count = 0; - - const auto exceptions_per_replica_path = export_partition_path / "exceptions_per_replica"; - - Strings exception_replicas; - if (Coordination::Error::ZOK != zk->tryGetChildren(exceptions_per_replica_path, exception_replicas)) - { - LOG_INFO(log, "Skipping {}: missing exceptions_per_replica", export_partition_path); - continue; - } - - for (const auto & replica : exception_replicas) - { - std::string exception_count_string; - if (!zk->tryGet(exceptions_per_replica_path / replica / "count", exception_count_string)) - { - LOG_INFO(log, "Skipping {}: missing count", replica); - continue; - } - - exception_count += std::stoull(exception_count_string.c_str()); - - if (last_exception.empty()) - { - const auto last_exception_path = exceptions_per_replica_path / replica / "last_exception"; - std::string last_exception_string; - if (!zk->tryGet(last_exception_path / "exception", last_exception_string)) - { - LOG_INFO(log, "Skipping {}: missing last_exception/exception", last_exception_path); - continue; - } - - std::string exception_part_zk; - if (!zk->tryGet(last_exception_path / "part", exception_part_zk)) - { - LOG_INFO(log, "Skipping {}: missing exception part", last_exception_path); - continue; - } - - exception_replica = replica; - last_exception = last_exception_string; - exception_part = exception_part_zk; - } - } - - const auto metadata = ExportReplicatedMergeTreePartitionManifest::fromJsonString(metadata_json); - - info.destination_database = metadata.destination_database; - info.destination_table = metadata.destination_table; - info.partition_id = metadata.partition_id; - info.transaction_id = metadata.transaction_id; - info.query_id = metadata.query_id; - info.create_time = metadata.create_time; - info.source_replica = metadata.source_replica; - info.parts_count = metadata.number_of_parts; - info.parts_to_do = parts_to_do; - info.parts = metadata.parts; - info.status = status; - info.exception_replica = exception_replica; - info.last_exception = last_exception; - info.exception_part = exception_part; - info.exception_count = exception_count; - - infos.emplace_back(std::move(info)); - } - - return infos; + return export_merge_tree_partition_manifest_updater->getPartitionExportsInfoLocal(); } - StorageReplicatedMergeTree::CreateMergeEntryResult StorageReplicatedMergeTree::createLogEntryToMergeParts( zkutil::ZooKeeperPtr & zookeeper, const DataPartsVector & parts, @@ -8378,15 +8284,21 @@ void StorageReplicatedMergeTree::exportPartitionToTable(const PartitionCommand & const auto partition_exports_path = fs::path(exports_path) / export_key; /// check if entry already exists + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperExists); if (zookeeper->exists(partition_exports_path)) { LOG_INFO(log, "Export with key {} is already exported or it is being exported. Checking if it has expired so that we can overwrite it", export_key); bool has_expired = false; + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperExists); if (zookeeper->exists(fs::path(partition_exports_path) / "metadata.json")) { std::string metadata_json; + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperGet); if (zookeeper->tryGet(fs::path(partition_exports_path) / "metadata.json", metadata_json)) { const auto manifest = ExportReplicatedMergeTreePartitionManifest::fromJsonString(metadata_json); @@ -8413,6 +8325,8 @@ void StorageReplicatedMergeTree::exportPartitionToTable(const PartitionCommand & /// Not putting in ops (same transaction) because we can't construct a "tryRemoveRecursive" request. /// It is possible that the zk being used does not support RemoveRecursive requests. /// It is ok for this to be non transactional. Worst case scenario an on-going export is going to be killed and a new task won't be scheduled. + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRemoveRecursive); zookeeper->tryRemoveRecursive(partition_exports_path); } @@ -8491,6 +8405,8 @@ void StorageReplicatedMergeTree::exportPartitionToTable(const PartitionCommand & manifest.parquet_parallel_encoding = query_context->getSettingsRef()[Setting::output_format_parquet_parallel_encoding]; manifest.max_bytes_per_file = query_context->getSettingsRef()[Setting::export_merge_tree_part_max_bytes_per_file]; manifest.max_rows_per_file = query_context->getSettingsRef()[Setting::export_merge_tree_part_max_rows_per_file]; + manifest.lock_inside_the_task = query_context->getSettingsRef()[Setting::export_merge_tree_partition_lock_inside_the_task]; + manifest.file_already_exists_policy = query_context->getSettingsRef()[Setting::export_merge_tree_part_file_already_exists_policy].value; @@ -8538,6 +8454,8 @@ void StorageReplicatedMergeTree::exportPartitionToTable(const PartitionCommand & "PENDING", zkutil::CreateMode::Persistent)); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperMulti); Coordination::Responses responses; Coordination::Error code = zookeeper->tryMulti(ops, responses); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index bc9eced6c609..2a9ef1c141b2 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -377,7 +377,7 @@ class StorageReplicatedMergeTree final : public MergeTreeData using ShutdownDeadline = std::chrono::time_point; void waitForUniquePartsToBeFetchedByOtherReplicas(ShutdownDeadline shutdown_deadline); - std::vector getPartitionExportsInfo() const; + std::vector getPartitionExportsInfo(bool prefer_remote_information) const; private: std::atomic_bool are_restoring_replica {false}; @@ -405,6 +405,7 @@ class StorageReplicatedMergeTree final : public MergeTreeData friend class ReplicatedMergeMutateTaskBase; friend class ExportPartitionManifestUpdatingTask; friend class ExportPartitionTaskScheduler; + friend class ExportPartFromPartitionExportTask; using MergeStrategyPicker = ReplicatedMergeTreeMergeStrategyPicker; using LogEntry = ReplicatedMergeTreeLogEntry; diff --git a/src/Storages/System/StorageSystemExports.cpp b/src/Storages/System/StorageSystemExports.cpp index fad8b43d0d2f..1bac86870712 100644 --- a/src/Storages/System/StorageSystemExports.cpp +++ b/src/Storages/System/StorageSystemExports.cpp @@ -8,7 +8,6 @@ #include #include - namespace DB { diff --git a/src/Storages/System/StorageSystemReplicatedPartitionExports.cpp b/src/Storages/System/StorageSystemReplicatedPartitionExports.cpp index ed4b2bc9f4c2..e088e4f77214 100644 --- a/src/Storages/System/StorageSystemReplicatedPartitionExports.cpp +++ b/src/Storages/System/StorageSystemReplicatedPartitionExports.cpp @@ -10,11 +10,17 @@ #include #include "Columns/ColumnString.h" #include "Storages/VirtualColumnUtils.h" +#include namespace DB { +namespace Setting +{ + extern const SettingsBool export_merge_tree_partition_system_table_prefer_remote_information; +} + ColumnsDescription StorageSystemReplicatedPartitionExports::getColumnsDescription() { return ColumnsDescription @@ -111,7 +117,7 @@ void StorageSystemReplicatedPartitionExports::fillData(MutableColumns & res_colu { const IStorage * storage = replicated_merge_tree_tables[database][table].get(); if (const auto * replicated_merge_tree = dynamic_cast(storage)) - partition_exports_info = replicated_merge_tree->getPartitionExportsInfo(); + partition_exports_info = replicated_merge_tree->getPartitionExportsInfo(context->getSettingsRef()[Setting::export_merge_tree_partition_system_table_prefer_remote_information]); } for (const ReplicatedPartitionExportInfo & info : partition_exports_info) diff --git a/src/Storages/System/StorageSystemReplicatedPartitionExports.h b/src/Storages/System/StorageSystemReplicatedPartitionExports.h index d57844cbb34e..15eb54f38c0e 100644 --- a/src/Storages/System/StorageSystemReplicatedPartitionExports.h +++ b/src/Storages/System/StorageSystemReplicatedPartitionExports.h @@ -23,7 +23,7 @@ struct ReplicatedPartitionExportInfo std::string exception_replica; std::string last_exception; std::string exception_part; - size_t exception_count; + size_t exception_count = 0; }; class StorageSystemReplicatedPartitionExports final : public IStorageSystemOneBlock diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py index 80f65378985b..b10349c35190 100644 --- a/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py +++ b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py @@ -91,6 +91,7 @@ def cluster(): with_minio=True, stay_alive=True, with_zookeeper=True, + keeper_required_feature_flags=["multi_read"], ) cluster.add_instance( "replica2", @@ -99,6 +100,7 @@ def cluster(): with_minio=True, stay_alive=True, with_zookeeper=True, + keeper_required_feature_flags=["multi_read"], ) # node that does not participate in the export, but will have visibility over the s3 table cluster.add_instance( @@ -114,6 +116,7 @@ def cluster(): with_minio=True, stay_alive=True, with_zookeeper=True, + keeper_required_feature_flags=["multi_read"], ) logging.info("Starting cluster...") cluster.start() @@ -122,6 +125,31 @@ def cluster(): cluster.shutdown() +@pytest.fixture(autouse=True) +def drop_tables_after_test(cluster): + """Drop all tables in the default database after every test. + + Without this, ReplicatedMergeTree tables from completed tests remain alive and keep + running ZooKeeper background threads (merge selector, queue log, cleanup, export manifest + updater). With many tables alive simultaneously the ZooKeeper session becomes overwhelmed + and subsequent tests start seeing operation-timeout / session-expired errors. + """ + yield + for instance_name, instance in cluster.instances.items(): + try: + tables_str = instance.query( + "SELECT name FROM system.tables WHERE database = 'default' FORMAT TabSeparated" + ).strip() + if not tables_str: + continue + for table in tables_str.split('\n'): + table = table.strip() + if table: + instance.query(f"DROP TABLE IF EXISTS default.`{table}` SYNC") + except Exception as e: + logging.warning(f"drop_tables_after_test: cleanup failed on {instance_name}: {e}") + + def create_s3_table(node, s3_table): node.query(f"CREATE TABLE {s3_table} (id UInt64, year UInt16) ENGINE = S3(s3_conn, filename='{s3_table}', format=Parquet, partition_strategy='hive') PARTITION BY year") @@ -139,8 +167,9 @@ def test_restart_nodes_during_export(cluster): node2 = cluster.instances["replica2"] watcher_node = cluster.instances["watcher_node"] - mt_table = "disaster_mt_table" - s3_table = "disaster_s3_table" + postfix = str(uuid.uuid4()).replace("-", "_") + mt_table = f"disaster_mt_table_{postfix}" + s3_table = f"disaster_s3_table_{postfix}" create_tables_and_insert_data(node, mt_table, s3_table, "replica1") create_tables_and_insert_data(node2, mt_table, s3_table, "replica2") @@ -224,14 +253,18 @@ def test_restart_nodes_during_export(cluster): assert node.query(f"SELECT count() FROM {s3_table} WHERE year = 2021") != f'0\n', "Export of partition 2021 did not resume after crash" -def test_kill_export(cluster): +@pytest.mark.parametrize( + "system_table_prefer_remote_information", ['0', '1'] +) +def test_kill_export(cluster, system_table_prefer_remote_information): skip_if_remote_database_disk_enabled(cluster) node = cluster.instances["replica1"] node2 = cluster.instances["replica2"] watcher_node = cluster.instances["watcher_node"] - mt_table = "kill_export_mt_table" - s3_table = "kill_export_s3_table" + postfix = str(uuid.uuid4()).replace("-", "_") + mt_table = f"kill_export_mt_table_{system_table_prefer_remote_information}_{postfix}" + s3_table = f"kill_export_s3_table_{system_table_prefer_remote_information}_{postfix}" create_tables_and_insert_data(node, mt_table, s3_table, "replica1") create_tables_and_insert_data(node2, mt_table, s3_table, "replica2") @@ -297,6 +330,9 @@ def test_kill_export(cluster): # ZooKeeper operations (KILL) proceed quickly since only S3 is blocked node.query(f"KILL EXPORT PARTITION WHERE partition_id = '2020' and source_table = '{mt_table}' and destination_table = '{s3_table}'") + # sleep for a while to let the kill to be processed + time.sleep(2) + # wait for 2021 to finish wait_for_export_status(node, mt_table, s3_table, "2021", "COMPLETED") @@ -305,8 +341,11 @@ def test_kill_export(cluster): assert node.query(f"SELECT count() FROM s3(s3_conn, filename='{s3_table}/commit_2021_*', format=LineAsString)") != f'0\n', "Partition 2021 was not written to S3, but it should have been" # check system.replicated_partition_exports for the export, status should be KILLED - assert node.query(f"SELECT status FROM system.replicated_partition_exports WHERE partition_id = '2020' and source_table = '{mt_table}' and destination_table = '{s3_table}'") == 'KILLED\n', "Partition 2020 was not killed as expected" - assert node.query(f"SELECT status FROM system.replicated_partition_exports WHERE partition_id = '2021' and source_table = '{mt_table}' and destination_table = '{s3_table}'") == 'COMPLETED\n', "Partition 2021 was not completed, this is unexpected" + assert node.query(f"SELECT status FROM system.replicated_partition_exports WHERE partition_id = '2020' and source_table = '{mt_table}' and destination_table = '{s3_table}' SETTINGS export_merge_tree_partition_system_table_prefer_remote_information = {system_table_prefer_remote_information}") == 'KILLED\n', "Partition 2020 was not killed as expected" + assert node.query(f"SELECT status FROM system.replicated_partition_exports WHERE partition_id = '2021' and source_table = '{mt_table}' and destination_table = '{s3_table}' SETTINGS export_merge_tree_partition_system_table_prefer_remote_information = {system_table_prefer_remote_information}") == 'COMPLETED\n', "Partition 2021 was not completed, this is unexpected" + + # check the data did not land on s3 + assert node.query(f"SELECT count() FROM {s3_table} WHERE year = 2020") == '0\n', "Partition 2020 was written to S3, it was not killed as expected" def test_drop_source_table_during_export(cluster): @@ -315,8 +354,9 @@ def test_drop_source_table_during_export(cluster): # node2 = cluster.instances["replica2"] watcher_node = cluster.instances["watcher_node"] - mt_table = "drop_source_table_during_export_mt_table" - s3_table = "drop_source_table_during_export_s3_table" + postfix = str(uuid.uuid4()).replace("-", "_") + mt_table = f"drop_source_table_during_export_mt_table_{postfix}" + s3_table = f"drop_source_table_during_export_s3_table_{postfix}" create_tables_and_insert_data(node, mt_table, s3_table, "replica1") # create_tables_and_insert_data(node2, mt_table, s3_table, "replica2") @@ -369,9 +409,10 @@ def test_drop_source_table_during_export(cluster): def test_concurrent_exports_to_different_targets(cluster): node = cluster.instances["replica1"] - mt_table = "concurrent_diff_targets_mt_table" - s3_table_a = "concurrent_diff_targets_s3_a" - s3_table_b = "concurrent_diff_targets_s3_b" + postfix = str(uuid.uuid4()).replace("-", "_") + mt_table = f"concurrent_diff_targets_mt_table_{postfix}" + s3_table_a = f"concurrent_diff_targets_s3_a_{postfix}" + s3_table_b = f"concurrent_diff_targets_s3_b_{postfix}" create_tables_and_insert_data(node, mt_table, s3_table_a, "replica1") create_s3_table(node, s3_table_b) @@ -407,8 +448,9 @@ def test_failure_is_logged_in_system_table(cluster): skip_if_remote_database_disk_enabled(cluster) node = cluster.instances["replica1"] - mt_table = "failure_is_logged_in_system_table_mt_table" - s3_table = "failure_is_logged_in_system_table_s3_table" + postfix = str(uuid.uuid4()).replace("-", "_") + mt_table = f"failure_is_logged_in_system_table_mt_table_{postfix}" + s3_table = f"failure_is_logged_in_system_table_s3_table_{postfix}" create_tables_and_insert_data(node, mt_table, s3_table, "replica1") @@ -452,6 +494,7 @@ def test_failure_is_logged_in_system_table(cluster): WHERE source_table = '{mt_table}' AND destination_table = '{s3_table}' AND partition_id = '2020' + SETTINGS export_merge_tree_partition_system_table_prefer_remote_information = 1 """ ) @@ -463,6 +506,7 @@ def test_failure_is_logged_in_system_table(cluster): WHERE source_table = '{mt_table}' AND destination_table = '{s3_table}' AND partition_id = '2020' + SETTINGS export_merge_tree_partition_system_table_prefer_remote_information = 1 """ ) assert int(exception_count.strip()) > 0, "Expected non-zero exception_count in system.replicated_partition_exports" @@ -477,8 +521,9 @@ def test_inject_short_living_failures(cluster): skip_if_remote_database_disk_enabled(cluster) node = cluster.instances["replica1"] - mt_table = "inject_short_living_failures_mt_table" - s3_table = "inject_short_living_failures_s3_table" + postfix = str(uuid.uuid4()).replace("-", "_") + mt_table = f"inject_short_living_failures_mt_table_{postfix}" + s3_table = f"inject_short_living_failures_s3_table_{postfix}" create_tables_and_insert_data(node, mt_table, s3_table, "replica1") @@ -546,8 +591,9 @@ def test_inject_short_living_failures(cluster): def test_export_ttl(cluster): node = cluster.instances["replica1"] - mt_table = "export_ttl_mt_table" - s3_table = "export_ttl_s3_table" + postfix = str(uuid.uuid4()).replace("-", "_") + mt_table = f"export_ttl_mt_table_{postfix}" + s3_table = f"export_ttl_s3_table_{postfix}" expiration_time = 3 @@ -581,8 +627,9 @@ def test_export_ttl(cluster): def test_export_partition_file_already_exists_policy(cluster): node = cluster.instances["replica1"] - mt_table = "export_partition_file_already_exists_policy_mt_table" - s3_table = "export_partition_file_already_exists_policy_s3_table" + postfix = str(uuid.uuid4()).replace("-", "_") + mt_table = f"export_partition_file_already_exists_policy_mt_table_{postfix}" + s3_table = f"export_partition_file_already_exists_policy_s3_table_{postfix}" create_tables_and_insert_data(node, mt_table, s3_table, "replica1") @@ -668,8 +715,9 @@ def test_export_partition_file_already_exists_policy(cluster): def test_export_partition_feature_is_disabled(cluster): replica_with_export_disabled = cluster.instances["replica_with_export_disabled"] - mt_table = "export_partition_feature_is_disabled_mt_table" - s3_table = "export_partition_feature_is_disabled_s3_table" + postfix = str(uuid.uuid4()).replace("-", "_") + mt_table = f"export_partition_feature_is_disabled_mt_table_{postfix}" + s3_table = f"export_partition_feature_is_disabled_s3_table_{postfix}" create_tables_and_insert_data(replica_with_export_disabled, mt_table, s3_table, "replica1") @@ -688,8 +736,9 @@ def test_export_partition_permissions(cluster): """ node = cluster.instances["replica1"] - mt_table = "permissions_mt_table" - s3_table = "permissions_s3_table" + postfix = str(uuid.uuid4()).replace("-", "_") + mt_table = f"permissions_mt_table_{postfix}" + s3_table = f"permissions_s3_table_{postfix}" # Create tables as default user create_tables_and_insert_data(node, mt_table, s3_table, "replica1") @@ -759,8 +808,9 @@ def test_export_partition_permissions(cluster): def test_multiple_exports_within_a_single_query(cluster): node = cluster.instances["replica1"] - mt_table = "multiple_exports_within_a_single_query_mt_table" - s3_table = "multiple_exports_within_a_single_query_s3_table" + postfix = str(uuid.uuid4()).replace("-", "_") + mt_table = f"multiple_exports_within_a_single_query_mt_table_{postfix}" + s3_table = f"multiple_exports_within_a_single_query_s3_table_{postfix}" create_tables_and_insert_data(node, mt_table, s3_table, "replica1") @@ -797,8 +847,9 @@ def test_pending_mutations_throw_before_export_partition(cluster): """Test that pending mutations before export partition throw an error.""" node = cluster.instances["replica1"] - mt_table = "pending_mutations_throw_partition_mt_table" - s3_table = "pending_mutations_throw_partition_s3_table" + postfix = str(uuid.uuid4()).replace("-", "_") + mt_table = f"pending_mutations_throw_partition_mt_table_{postfix}" + s3_table = f"pending_mutations_throw_partition_s3_table_{postfix}" create_tables_and_insert_data(node, mt_table, s3_table, "replica1") @@ -821,8 +872,9 @@ def test_pending_mutations_skip_before_export_partition(cluster): """Test that pending mutations before export partition are skipped with throw_on_pending_mutations=false.""" node = cluster.instances["replica1"] - mt_table = "pending_mutations_skip_partition_mt_table" - s3_table = "pending_mutations_skip_partition_s3_table" + postfix = str(uuid.uuid4()).replace("-", "_") + mt_table = f"pending_mutations_skip_partition_mt_table_{postfix}" + s3_table = f"pending_mutations_skip_partition_s3_table_{postfix}" create_tables_and_insert_data(node, mt_table, s3_table, "replica1") @@ -850,8 +902,9 @@ def test_pending_patch_parts_throw_before_export_partition(cluster): """Test that pending patch parts before export partition throw an error with default settings.""" node = cluster.instances["replica1"] - mt_table = "pending_patches_throw_partition_mt_table" - s3_table = "pending_patches_throw_partition_s3_table" + postfix = str(uuid.uuid4()).replace("-", "_") + mt_table = f"pending_patches_throw_partition_mt_table_{postfix}" + s3_table = f"pending_patches_throw_partition_s3_table_{postfix}" create_tables_and_insert_data(node, mt_table, s3_table, "replica1") @@ -873,8 +926,9 @@ def test_pending_patch_parts_skip_before_export_partition(cluster): """Test that pending patch parts before export partition are skipped with throw_on_pending_patch_parts=false.""" node = cluster.instances["replica1"] - mt_table = "pending_patches_skip_partition_mt_table" - s3_table = "pending_patches_skip_partition_s3_table" + postfix = str(uuid.uuid4()).replace("-", "_") + mt_table = f"pending_patches_skip_partition_mt_table_{postfix}" + s3_table = f"pending_patches_skip_partition_s3_table_{postfix}" create_tables_and_insert_data(node, mt_table, s3_table, "replica1") @@ -900,8 +954,9 @@ def test_mutations_after_export_partition_started(cluster): skip_if_remote_database_disk_enabled(cluster) node = cluster.instances["replica1"] - mt_table = "mutations_after_export_partition_mt_table" - s3_table = "mutations_after_export_partition_s3_table" + postfix = str(uuid.uuid4()).replace("-", "_") + mt_table = f"mutations_after_export_partition_mt_table_{postfix}" + s3_table = f"mutations_after_export_partition_s3_table_{postfix}" create_tables_and_insert_data(node, mt_table, s3_table, "replica1") @@ -950,8 +1005,9 @@ def test_patch_parts_after_export_partition_started(cluster): skip_if_remote_database_disk_enabled(cluster) node = cluster.instances["replica1"] - mt_table = "patches_after_export_partition_mt_table" - s3_table = "patches_after_export_partition_s3_table" + postfix = str(uuid.uuid4()).replace("-", "_") + mt_table = f"patches_after_export_partition_mt_table_{postfix}" + s3_table = f"patches_after_export_partition_s3_table_{postfix}" create_tables_and_insert_data(node, mt_table, s3_table, "replica1") @@ -1001,8 +1057,9 @@ def test_mutation_in_partition_clause(cluster): allow exports of unaffected partitions to succeed.""" node = cluster.instances["replica1"] - mt_table = "mutation_in_partition_clause_mt_table" - s3_table = "mutation_in_partition_clause_s3_table" + postfix = str(uuid.uuid4()).replace("-", "_") + mt_table = f"mutation_in_partition_clause_mt_table_{postfix}" + s3_table = f"mutation_in_partition_clause_s3_table_{postfix}" create_tables_and_insert_data(node, mt_table, s3_table, "replica1") @@ -1040,8 +1097,9 @@ def test_export_partition_with_mixed_computed_columns(cluster): """Test export partition with ALIAS, MATERIALIZED, and EPHEMERAL columns.""" node = cluster.instances["replica1"] - mt_table = "mixed_computed_mt_table" - s3_table = "mixed_computed_s3_table" + postfix = str(uuid.uuid4()).replace("-", "_") + mt_table = f"mixed_computed_mt_table_{postfix}" + s3_table = f"mixed_computed_s3_table_{postfix}" node.query(f""" CREATE TABLE {mt_table} ( From 932d91f38dc48dbf81be98dba05f07dbea7bd56d Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Sun, 8 Mar 2026 19:37:54 +0300 Subject: [PATCH 06/15] Merge pull request #1484 from Altinity/fix_parquet_metadata_cache_max_size Use serialized metadata size to calculate the cache entry cell Source-PR: #1484 (https://github.com/Altinity/ClickHouse/pull/1484) --- .../Formats/Impl/ParquetFileMetaDataCache.cpp | 12 +++++++++++- .../Formats/Impl/ParquetFileMetaDataCache.h | 7 ++++++- 2 files changed, 17 insertions(+), 2 deletions(-) diff --git a/src/Processors/Formats/Impl/ParquetFileMetaDataCache.cpp b/src/Processors/Formats/Impl/ParquetFileMetaDataCache.cpp index da8ad825f505..4aeb1d48bada 100644 --- a/src/Processors/Formats/Impl/ParquetFileMetaDataCache.cpp +++ b/src/Processors/Formats/Impl/ParquetFileMetaDataCache.cpp @@ -2,11 +2,21 @@ #if USE_PARQUET +#include + namespace DB { +size_t ParquetFileMetaDataWeightFunction::operator()(const parquet::FileMetaData & metadata) const +{ + /// TODO fix-me: using the size on disk is not ideal, but it is the simplest and best we can do for now. + /// this implementation is only used by the v1 reader, which is going to be deprecated and a new implementation for the v3 + /// reader will be added in the future. + return metadata.size(); +} + ParquetFileMetaDataCache::ParquetFileMetaDataCache() - : CacheBase(CurrentMetrics::end(), CurrentMetrics::end(), 0) + : CacheBase, ParquetFileMetaDataWeightFunction>(CurrentMetrics::end(), CurrentMetrics::end(), 0) {} ParquetFileMetaDataCache * ParquetFileMetaDataCache::instance() diff --git a/src/Processors/Formats/Impl/ParquetFileMetaDataCache.h b/src/Processors/Formats/Impl/ParquetFileMetaDataCache.h index fb5fc1bb0217..4578f2582391 100644 --- a/src/Processors/Formats/Impl/ParquetFileMetaDataCache.h +++ b/src/Processors/Formats/Impl/ParquetFileMetaDataCache.h @@ -16,7 +16,12 @@ class FileMetaData; namespace DB { -class ParquetFileMetaDataCache : public CacheBase +struct ParquetFileMetaDataWeightFunction +{ + size_t operator()(const parquet::FileMetaData & metadata) const; +}; + +class ParquetFileMetaDataCache : public CacheBase, ParquetFileMetaDataWeightFunction> { public: static ParquetFileMetaDataCache * instance(); From 750e6ce0a45182b6b83b87edefdb351b73104fc7 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Mon, 16 Mar 2026 17:16:21 +0300 Subject: [PATCH 07/15] Merge pull request #1490 from Altinity/export_filename_pattern_setting add setting to define filename pattern for part exports Source-PR: #1490 (https://github.com/Altinity/ClickHouse/pull/1490) --- .../mergetree-family/part_export.md | 7 + .../mergetree-family/partition_export.md | 6 + src/Core/Settings.cpp | 3 + src/Core/SettingsChangesHistory.cpp | 2 +- ...portReplicatedMergeTreePartitionManifest.h | 3 + src/Storages/MergeTree/ExportPartTask.cpp | 37 +++- .../ExportPartitionTaskScheduler.cpp | 4 +- src/Storages/StorageReplicatedMergeTree.cpp | 2 + .../configs/macros_shard1_replica1.xml | 6 + .../configs/macros_shard2_replica1.xml | 6 + .../test.py | 193 ++++++++++++++++++ ...merge_tree_part_filename_pattern.reference | 16 ++ ...export_merge_tree_part_filename_pattern.sh | 49 +++++ 13 files changed, 330 insertions(+), 4 deletions(-) create mode 100644 tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/macros_shard1_replica1.xml create mode 100644 tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/macros_shard2_replica1.xml create mode 100644 tests/queries/0_stateless/03608_export_merge_tree_part_filename_pattern.reference create mode 100755 tests/queries/0_stateless/03608_export_merge_tree_part_filename_pattern.sh diff --git a/docs/en/engines/table-engines/mergetree-family/part_export.md b/docs/en/engines/table-engines/mergetree-family/part_export.md index 287e0a17f3af..b7f0730596a4 100644 --- a/docs/en/engines/table-engines/mergetree-family/part_export.md +++ b/docs/en/engines/table-engines/mergetree-family/part_export.md @@ -84,6 +84,13 @@ In case a table function is used as the destination, the schema can be omitted a - **Default**: `true` - **Description**: If set to true, throws if pending patch parts exists for a given part. Note that by default mutations are applied to all parts, which means that if a mutation in practice would only affetct part/partition x, all the other parts/partition will throw upon export. The exception is when the `IN PARTITION` clause was used in the mutation command. Note the `IN PARTITION` clause is not properly implemented for plain MergeTree tables. +### export_merge_tree_part_filename_pattern + +- **Type**: `String` +- **Default**: `{part_name}_{checksum}` +- **Description**: Pattern for the filename of the exported merge tree part. The `part_name` and `checksum` are calculated and replaced on the fly. Additional macros are supported. + + ## Examples ### Basic Export to S3 diff --git a/docs/en/engines/table-engines/mergetree-family/partition_export.md b/docs/en/engines/table-engines/mergetree-family/partition_export.md index d91f226dbbf6..af503ec5180a 100644 --- a/docs/en/engines/table-engines/mergetree-family/partition_export.md +++ b/docs/en/engines/table-engines/mergetree-family/partition_export.md @@ -82,6 +82,12 @@ TO TABLE [destination_database.]destination_table - **Default**: `true` - **Description**: If set to true, throws if pending patch parts exists for a given part. Note that by default mutations are applied to all parts, which means that if a mutation in practice would only affetct part/partition x, all the other parts/partition will throw upon export. The exception is when the `IN PARTITION` clause was used in the mutation command. Note the `IN PARTITION` clause is not properly implemented for plain MergeTree tables. +### export_merge_tree_part_filename_pattern + +- **Type**: `String` +- **Default**: `{part_name}_{checksum}` +- **Description**: Pattern for the filename of the exported merge tree part. The `part_name` and `checksum` are calculated and replaced on the fly. Additional macros are supported. + ## Examples ### Basic Export to S3 diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index cb9c33ef3c05..cd10964d620d 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -7520,6 +7520,9 @@ On the other hand, there is a chance once the task executes that part has alread DECLARE(Bool, export_merge_tree_partition_system_table_prefer_remote_information, true, R"( Controls whether the system.replicated_partition_exports will prefer to query ZooKeeper to get the most up to date information or use the local information. Querying ZooKeeper is expensive, and only available if the ZooKeeper feature flag MULTI_READ is enabled. +)", 0) \ + DECLARE(String, export_merge_tree_part_filename_pattern, "{part_name}_{checksum}", R"( +Pattern for the filename of the exported merge tree part. The `part_name` and `checksum` are calculated and replaced on the fly. Additional macros are supported. )", 0) \ \ /* ####################################################### */ \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 6d6f0c0921b5..93170c7fb8d9 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -110,7 +110,7 @@ const VersionToSettingsChangesMap & getSettingsChangesHistory() { // {"iceberg_partition_timezone", "", "", "New setting."}, // {"s3_propagate_credentials_to_other_storages", false, false, "New setting"}, - // {"export_merge_tree_part_filename_pattern", "", "{part_name}_{checksum}", "New setting"}, + {"export_merge_tree_part_filename_pattern", "", "{part_name}_{checksum}", "New setting"}, // {"use_parquet_metadata_cache", false, true, "Enables cache of parquet file metadata."}, // {"input_format_parquet_use_metadata_cache", true, false, "Obsolete. No-op"}, // https://github.com/Altinity/ClickHouse/pull/586 // {"object_storage_remote_initiator_cluster", "", "", "New setting."}, diff --git a/src/Storages/ExportReplicatedMergeTreePartitionManifest.h b/src/Storages/ExportReplicatedMergeTreePartitionManifest.h index 31b8731c4e8b..a41401dee09b 100644 --- a/src/Storages/ExportReplicatedMergeTreePartitionManifest.h +++ b/src/Storages/ExportReplicatedMergeTreePartitionManifest.h @@ -116,6 +116,7 @@ struct ExportReplicatedMergeTreePartitionManifest size_t max_bytes_per_file; size_t max_rows_per_file; MergeTreePartExportManifest::FileAlreadyExistsPolicy file_already_exists_policy; + String filename_pattern; bool lock_inside_the_task; /// todo temporary std::string toJsonString() const @@ -139,6 +140,7 @@ struct ExportReplicatedMergeTreePartitionManifest json.set("max_bytes_per_file", max_bytes_per_file); json.set("max_rows_per_file", max_rows_per_file); json.set("file_already_exists_policy", String(magic_enum::enum_name(file_already_exists_policy))); + json.set("filename_pattern", filename_pattern); json.set("create_time", create_time); json.set("max_retries", max_retries); json.set("ttl_seconds", ttl_seconds); @@ -175,6 +177,7 @@ struct ExportReplicatedMergeTreePartitionManifest manifest.parquet_parallel_encoding = json->getValue("parquet_parallel_encoding"); manifest.max_bytes_per_file = json->getValue("max_bytes_per_file"); manifest.max_rows_per_file = json->getValue("max_rows_per_file"); + manifest.filename_pattern = json->getValue("filename_pattern"); if (json->has("file_already_exists_policy")) { diff --git a/src/Storages/MergeTree/ExportPartTask.cpp b/src/Storages/MergeTree/ExportPartTask.cpp index efdbe099433b..931fbb6fb9db 100644 --- a/src/Storages/MergeTree/ExportPartTask.cpp +++ b/src/Storages/MergeTree/ExportPartTask.cpp @@ -3,11 +3,13 @@ #include #include #include -#include #include #include +#include +#include #include #include +#include #include #include #include @@ -17,6 +19,7 @@ #include "Common/setThreadName.h" #include #include +#include #include #include #include @@ -47,6 +50,7 @@ namespace Setting extern const SettingsUInt64 export_merge_tree_part_max_bytes_per_file; extern const SettingsUInt64 export_merge_tree_part_max_rows_per_file; extern const SettingsBool allow_experimental_analyzer; + extern const SettingsString export_merge_tree_part_filename_pattern; } namespace @@ -93,6 +97,33 @@ namespace plan_for_part.addStep(std::move(expression_step)); } } + + String buildDestinationFilename( + const MergeTreePartExportManifest & manifest, + const StorageID & storage_id, + const ContextPtr & local_context) + { + auto filename = manifest.settings[Setting::export_merge_tree_part_filename_pattern].value; + + boost::replace_all(filename, "{part_name}", manifest.data_part->name); + boost::replace_all(filename, "{checksum}", manifest.data_part->checksums.getTotalChecksumHex()); + + Macros::MacroExpansionInfo macro_info; + macro_info.table_id = storage_id; + + if (auto database = DatabaseCatalog::instance().tryGetDatabase(storage_id.database_name)) + { + if (const auto replicated = dynamic_cast(database.get())) + { + macro_info.shard = replicated->getShardName(); + macro_info.replica = replicated->getReplicaName(); + } + } + + filename = local_context->getMacros()->expand(filename, macro_info); + + return filename; + } } ExportPartTask::ExportPartTask(MergeTreeData & storage_, const MergeTreePartExportManifest & manifest_) @@ -153,8 +184,10 @@ bool ExportPartTask::executeStep() try { + const auto filename = buildDestinationFilename(manifest, storage.getStorageID(), local_context); + sink = destination_storage->import( - manifest.data_part->name + "_" + manifest.data_part->checksums.getTotalChecksumHex(), + filename, block_with_partition_values, new_file_path_callback, manifest.file_already_exists_policy == MergeTreePartExportManifest::FileAlreadyExistsPolicy::overwrite, diff --git a/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp b/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp index 89042fe694b9..909ada39749e 100644 --- a/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp +++ b/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp @@ -56,7 +56,9 @@ namespace context_copy->setSetting("export_merge_tree_part_throw_on_pending_mutations", false); context_copy->setSetting("export_merge_tree_part_throw_on_pending_patch_parts", false); - return context_copy; + context_copy->setSetting("export_merge_tree_part_filename_pattern", manifest.filename_pattern); + + return context_copy; } } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 11a19e8322ea..d0d35c4ad89a 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -218,6 +218,7 @@ namespace Setting extern const SettingsBool export_merge_tree_part_throw_on_pending_mutations; extern const SettingsBool export_merge_tree_part_throw_on_pending_patch_parts; extern const SettingsBool export_merge_tree_partition_lock_inside_the_task; + extern const SettingsString export_merge_tree_part_filename_pattern; } @@ -8409,6 +8410,7 @@ void StorageReplicatedMergeTree::exportPartitionToTable(const PartitionCommand & manifest.file_already_exists_policy = query_context->getSettingsRef()[Setting::export_merge_tree_part_file_already_exists_policy].value; + manifest.filename_pattern = query_context->getSettingsRef()[Setting::export_merge_tree_part_filename_pattern].value; ops.emplace_back(zkutil::makeCreateRequest( fs::path(partition_exports_path) / "metadata.json", diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/macros_shard1_replica1.xml b/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/macros_shard1_replica1.xml new file mode 100644 index 000000000000..bae1ce119255 --- /dev/null +++ b/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/macros_shard1_replica1.xml @@ -0,0 +1,6 @@ + + + shard1 + replica1 + + diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/macros_shard2_replica1.xml b/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/macros_shard2_replica1.xml new file mode 100644 index 000000000000..fb9a587e736d --- /dev/null +++ b/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/macros_shard2_replica1.xml @@ -0,0 +1,6 @@ + + + shard2 + replica1 + + diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py index b10349c35190..5b96e1ad286b 100644 --- a/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py +++ b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py @@ -118,6 +118,26 @@ def cluster(): with_zookeeper=True, keeper_required_feature_flags=["multi_read"], ) + # Sharded instances for filename pattern tests + cluster.add_instance( + "shard1_replica1", + main_configs=["configs/named_collections.xml", "configs/allow_experimental_export_partition.xml", "configs/macros_shard1_replica1.xml"], + user_configs=["configs/users.d/profile.xml"], + with_minio=True, + stay_alive=True, + with_zookeeper=True, + keeper_required_feature_flags=["multi_read"], + ) + + cluster.add_instance( + "shard2_replica1", + main_configs=["configs/named_collections.xml", "configs/allow_experimental_export_partition.xml", "configs/macros_shard2_replica1.xml"], + user_configs=["configs/users.d/profile.xml"], + with_minio=True, + stay_alive=True, + with_zookeeper=True, + keeper_required_feature_flags=["multi_read"], + ) logging.info("Starting cluster...") cluster.start() yield cluster @@ -161,6 +181,14 @@ def create_tables_and_insert_data(node, mt_table, s3_table, replica_name): create_s3_table(node, s3_table) +def create_sharded_tables_and_insert_data(node, mt_table, s3_table, replica_name): + """Create sharded ReplicatedMergeTree table with {shard} macro in ZooKeeper path.""" + node.query(f"CREATE TABLE {mt_table} (id UInt64, year UInt16) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{{shard}}/{mt_table}', '{replica_name}') PARTITION BY year ORDER BY tuple()") + node.query(f"INSERT INTO {mt_table} VALUES (1, 2020), (2, 2020), (3, 2020), (4, 2021)") + + create_s3_table(node, s3_table) + + def test_restart_nodes_during_export(cluster): skip_if_remote_database_disk_enabled(cluster) node = cluster.instances["replica1"] @@ -1148,3 +1176,168 @@ def test_export_partition_with_mixed_computed_columns(cluster): AND partition_id = '1' """) assert status.strip() == "COMPLETED", f"Expected COMPLETED status, got: {status}" + + +def test_sharded_export_partition_with_filename_pattern(cluster): + """Test that export partition with filename pattern prevents collisions in sharded setup.""" + shard1_r1 = cluster.instances["shard1_replica1"] + shard2_r1 = cluster.instances["shard2_replica1"] + watcher_node = cluster.instances["watcher_node"] + + mt_table = "sharded_mt_table" + s3_table = "sharded_s3_table" + + # Create sharded tables on all shards with same partition data (same part names) + # Each shard uses different ZooKeeper path via {shard} macro + create_sharded_tables_and_insert_data(shard1_r1, mt_table, s3_table, "replica1") + create_sharded_tables_and_insert_data(shard2_r1, mt_table, s3_table, "replica1") + create_s3_table(watcher_node, s3_table) + + # Export partition from both shards with filename pattern including shard + # This should prevent filename collisions + shard1_r1.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} " + f"SETTINGS export_merge_tree_part_filename_pattern = '{{part_name}}_{{shard}}_{{replica}}_{{checksum}}'" + ) + shard2_r1.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} " + f"SETTINGS export_merge_tree_part_filename_pattern = '{{part_name}}_{{shard}}_{{replica}}_{{checksum}}'" + ) + + # Wait for exports to complete + wait_for_export_status(shard1_r1, mt_table, s3_table, "2020", "COMPLETED") + wait_for_export_status(shard2_r1, mt_table, s3_table, "2020", "COMPLETED") + + total_count = watcher_node.query(f"SELECT count() FROM {s3_table} WHERE year = 2020").strip() + assert total_count == "6", f"Expected 6 total rows (3 from each shard), got {total_count}" + + # Verify filenames contain shard information (check via S3 directly) + # Get all files from S3 - query from watcher_node since S3 is shared + files_shard1 = watcher_node.query( + f"SELECT _file FROM s3(s3_conn, filename='{s3_table}/**', format='One') WHERE _file LIKE '%shard1%' LIMIT 1" + ).strip() + files_shard2 = watcher_node.query( + f"SELECT _file FROM s3(s3_conn, filename='{s3_table}/**', format='One') WHERE _file LIKE '%shard2%' LIMIT 1" + ).strip() + + # Both shards should have files with their shard names + assert "shard1" in files_shard1 or files_shard1 == "", f"Expected shard1 in filenames, got: {files_shard1}" + assert "shard2" in files_shard2 or files_shard2 == "", f"Expected shard2 in filenames, got: {files_shard2}" + + +def test_export_partition_from_replicated_database_uses_db_shard_replica_macros(cluster): + """Test that {shard} and {replica} in the filename pattern are expanded from the + DatabaseReplicated identity, NOT from server config macros. + + replica1 has no / entries in its server config section. + Without the fix buildDestinationFilename() leaves macro_info.shard/replica unset, so + Macros::expand() falls through to the config-macros lookup and throws NO_ELEMENTS_IN_CONFIG. + With the fix the DatabaseReplicated shard_name / replica_name are injected into macro_info + before the expand call, and the pattern resolves correctly. + """ + + # The remote disk test suite sets the shard and replica macros in https://github.com/Altinity/ClickHouse/blob/bbabcaa96e8b7fe8f70ecd0bd4f76fb0f76f2166/tests/integration/helpers/cluster.py#L4356 + # When expanding the macros, the configured ones are preferred over the ones from the DatabaseReplicated definition. + # Therefore, this test fails. It is easier to skip it than to fix it. + skip_if_remote_database_disk_enabled(cluster) + + node = cluster.instances["replica1"] + watcher_node = cluster.instances["watcher_node"] + + postfix = str(uuid.uuid4()).replace("-", "_") + db_name = f"repdb_{postfix}" + table_name = "mt_table" + s3_table = f"s3_dbreplicated_{postfix}" + + # These values exist only in the DatabaseReplicated definition – they are NOT + # present anywhere in replica1's server config . + db_shard = "db_shard_x" + db_replica = "db_replica_y" + + node.query( + f"CREATE DATABASE {db_name} " + f"ENGINE = Replicated('/clickhouse/databases/{db_name}', '{db_shard}', '{db_replica}')") + + node.query(f""" + CREATE TABLE {db_name}.{table_name} + (id UInt64, year UInt16) + ENGINE = ReplicatedMergeTree() + PARTITION BY year ORDER BY tuple()""") + + node.query(f"INSERT INTO {db_name}.{table_name} VALUES (1, 2020), (2, 2020), (3, 2020)") + # Stop merges so part names stay stable during the test. + node.query(f"SYSTEM STOP MERGES {db_name}.{table_name}") + + node.query( + f"CREATE TABLE {s3_table} (id UInt64, year UInt16) " + f"ENGINE = S3(s3_conn, filename='{s3_table}', format=Parquet, partition_strategy='hive') " + f"PARTITION BY year") + + watcher_node.query( + f"CREATE TABLE {s3_table} (id UInt64, year UInt16) " + f"ENGINE = S3(s3_conn, filename='{s3_table}', format=Parquet, partition_strategy='hive') " + f"PARTITION BY year") + + # Export with {shard} and {replica} in the pattern. + # Before the fix: Macros::expand throws NO_ELEMENTS_IN_CONFIG because replica1 has + # no / server config macros. + # After the fix: DatabaseReplicated's shard_name/replica_name are wired into + # macro_info before the expand call, so this succeeds and produces the right names. + node.query( + f"ALTER TABLE {db_name}.{table_name} EXPORT PARTITION ID '2020' TO TABLE {s3_table} " + f"SETTINGS export_merge_tree_part_filename_pattern = " + f"'{{part_name}}_{{shard}}_{{replica}}_{{checksum}}'") + + # A FAILED status here almost certainly means the macro expansion threw + # NO_ELEMENTS_IN_CONFIG (i.e. the fix is missing or broken). + wait_for_export_status(node, table_name, s3_table, "2020", "COMPLETED") + + # Data should have landed in S3. + count = watcher_node.query(f"SELECT count() FROM {s3_table} WHERE year = 2020").strip() + assert count == "3", f"Expected 3 exported rows, got {count}" + + # The exported filename must contain the exact shard and replica names from the + # DatabaseReplicated definition, proving the fix injected them (not server config macros). + filename = watcher_node.query( + f"SELECT _file FROM s3(s3_conn, filename='{s3_table}/**/*.parquet', format='One') LIMIT 1" + ).strip() + + assert db_shard in filename, ( + f"Expected filename to contain DatabaseReplicated shard '{db_shard}', got: {filename!r}. " + "Suggests {shard} was not expanded from the DatabaseReplicated identity.") + + assert db_replica in filename, ( + f"Expected filename to contain DatabaseReplicated replica '{db_replica}', got: {filename!r}. " + "Suggests {replica} was not expanded from the DatabaseReplicated identity.") + + +def test_sharded_export_partition_default_pattern(cluster): + shard1_r1 = cluster.instances["shard1_replica1"] + shard2_r1 = cluster.instances["shard2_replica1"] + watcher_node = cluster.instances["watcher_node"] + + mt_table = "sharded_mt_table_default" + s3_table = "sharded_s3_table_default" + + # Create sharded tables with different ZooKeeper paths per shard + create_sharded_tables_and_insert_data(shard1_r1, mt_table, s3_table, "replica1") + create_sharded_tables_and_insert_data(shard2_r1, mt_table, s3_table, "replica1") + create_s3_table(watcher_node, s3_table) + + # Export with default pattern ({part_name}_{checksum}) - may cause collisions if parts have same name and the same checksum + shard1_r1.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table}" + ) + shard2_r1.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table}" + ) + + wait_for_export_status(shard1_r1, mt_table, s3_table, "2020", "COMPLETED") + wait_for_export_status(shard2_r1, mt_table, s3_table, "2020", "COMPLETED") + + # Both exports should complete (even if there are collisions, the overwrite policy handles it) + # S3 tables are shared, so query from watcher_node + total_count = watcher_node.query(f"SELECT count() FROM {s3_table} WHERE year = 2020").strip() + + # only one file with 3 rows should be present + assert int(total_count) == 3, f"Expected 3 rows, got {total_count}" diff --git a/tests/queries/0_stateless/03608_export_merge_tree_part_filename_pattern.reference b/tests/queries/0_stateless/03608_export_merge_tree_part_filename_pattern.reference new file mode 100644 index 000000000000..8016f5aa113e --- /dev/null +++ b/tests/queries/0_stateless/03608_export_merge_tree_part_filename_pattern.reference @@ -0,0 +1,16 @@ +---- Test: Default pattern {part_name}_{checksum} +1 2020 +2 2020 +3 2020 +---- Verify filename matches 2020_1_1_0_*.1.parquet +1 +---- Test: Custom prefix pattern +4 2021 +---- Verify filename matches myprefix_2021_2_2_0.1.parquet +1 +---- Test: Pattern with macros +1 2020 +2 2020 +3 2020 +---- Verify macros expanded (no literal braces in parquet filenames, that's the best we can do for stateless tests) +1 diff --git a/tests/queries/0_stateless/03608_export_merge_tree_part_filename_pattern.sh b/tests/queries/0_stateless/03608_export_merge_tree_part_filename_pattern.sh new file mode 100755 index 000000000000..12b47f4f2664 --- /dev/null +++ b/tests/queries/0_stateless/03608_export_merge_tree_part_filename_pattern.sh @@ -0,0 +1,49 @@ +#!/usr/bin/env bash +# Tags: no-fasttest +# Tag no-fasttest: requires s3 storage + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +R=$RANDOM +mt="mt_${R}" +dest1="fp_dest1_${R}" +dest2="fp_dest2_${R}" +dest3="fp_dest3_${R}" + +query() { + $CLICKHOUSE_CLIENT --query "$1" +} + +query "DROP TABLE IF EXISTS $mt, $dest1, $dest2, $dest3" + +query "CREATE TABLE $mt (id UInt64, year UInt16) ENGINE = MergeTree() PARTITION BY year ORDER BY tuple()" +query "INSERT INTO $mt VALUES (1, 2020), (2, 2020), (3, 2020), (4, 2021)" + +query "CREATE TABLE $dest1 (id UInt64, year UInt16) ENGINE = S3(s3_conn, filename='$dest1', format=Parquet, partition_strategy='hive') PARTITION BY year" +query "CREATE TABLE $dest2 (id UInt64, year UInt16) ENGINE = S3(s3_conn, filename='$dest2', format=Parquet, partition_strategy='hive') PARTITION BY year" +query "CREATE TABLE $dest3 (id UInt64, year UInt16) ENGINE = S3(s3_conn, filename='$dest3', format=Parquet, partition_strategy='hive') PARTITION BY year" + +echo "---- Test: Default pattern {part_name}_{checksum}" +query "ALTER TABLE $mt EXPORT PART '2020_1_1_0' TO TABLE $dest1 SETTINGS allow_experimental_export_merge_tree_part = 1, export_merge_tree_part_filename_pattern = '{part_name}_{checksum}'" +sleep 3 +query "SELECT * FROM $dest1 ORDER BY id" +echo "---- Verify filename matches 2020_1_1_0_*.1.parquet" +query "SELECT count() FROM s3(s3_conn, filename='$dest1/**/2020_1_1_0_*.1.parquet', format='One')" + +echo "---- Test: Custom prefix pattern" +query "ALTER TABLE $mt EXPORT PART '2021_2_2_0' TO TABLE $dest2 SETTINGS allow_experimental_export_merge_tree_part = 1, export_merge_tree_part_filename_pattern = 'myprefix_{part_name}'" +sleep 3 +query "SELECT * FROM $dest2 ORDER BY id" +echo "---- Verify filename matches myprefix_2021_2_2_0.1.parquet" +query "SELECT count() FROM s3(s3_conn, filename='$dest2/**/myprefix_2021_2_2_0.1.parquet', format='One')" + +echo "---- Test: Pattern with macros" +query "ALTER TABLE $mt EXPORT PART '2020_1_1_0' TO TABLE $dest3 SETTINGS allow_experimental_export_merge_tree_part = 1, export_merge_tree_part_filename_pattern = '{database}_{table}_{part_name}'" +sleep 3 +query "SELECT * FROM $dest3 ORDER BY id" +echo "---- Verify macros expanded (no literal braces in parquet filenames, that's the best we can do for stateless tests)" +query "SELECT count() = 0 FROM s3(s3_conn, filename='$dest3/**/*.1.parquet', format='One') WHERE _file LIKE '%{%'" + +query "DROP TABLE IF EXISTS $mt, $dest1, $dest2, $dest3" From 68a6b484c18608e3bf2d76f1f57c0b941fd8d20f Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Tue, 17 Mar 2026 20:12:07 +0100 Subject: [PATCH 08/15] Merge pull request #1500 from Altinity/export_partition_fix_local_table_possibly_missing_not_pending_entries Fix local replicated_partition_exports table might miss entries Source-PR: #1500 (https://github.com/Altinity/ClickHouse/pull/1500) --- .../ExportPartitionManifestUpdatingTask.cpp | 38 +++++++++++-------- .../ExportPartitionManifestUpdatingTask.h | 2 + 2 files changed, 25 insertions(+), 15 deletions(-) diff --git a/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp index 18977902b03b..e5b1c72ed4c8 100644 --- a/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp +++ b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp @@ -535,14 +535,19 @@ void ExportPartitionManifestUpdatingTask::poll() ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperGetWatch); - std::string status; - if (!zk->tryGetWatch(fs::path(entry_path) / "status", status, nullptr, status_watch_callback)) + std::string status_string; + if (!zk->tryGetWatch(fs::path(entry_path) / "status", status_string, nullptr, status_watch_callback)) { LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Skipping {}: missing status", key); continue; } - bool is_pending = status == "PENDING"; + const auto status = magic_enum::enum_cast(status_string); + if (!status) + { + LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Invalid status {} for task {}, skipping", status_string, key); + continue; + } /// if we have the cleanup lock, try to cleanup /// if we successfully cleaned it up, early exit @@ -556,25 +561,20 @@ void ExportPartitionManifestUpdatingTask::poll() key, metadata, now, - is_pending, entries_by_key); + *status == ExportReplicatedMergeTreePartitionTaskEntry::Status::PENDING, + entries_by_key); if (cleanup_successful) continue; } - if (!is_pending) - { - LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Skipping {}: status is not PENDING", key); - continue; - } - if (has_local_entry_and_is_up_to_date) { LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Skipping {}: already exists", key); continue; } - addTask(metadata, key, entries_by_key); + addTask(metadata, *status, key, entries_by_key); } /// Remove entries that were deleted by someone else @@ -587,22 +587,30 @@ void ExportPartitionManifestUpdatingTask::poll() void ExportPartitionManifestUpdatingTask::addTask( const ExportReplicatedMergeTreePartitionManifest & metadata, + ExportReplicatedMergeTreePartitionTaskEntry::Status status, const std::string & key, auto & entries_by_key ) { std::vector part_references; - for (const auto & part_name : metadata.parts) + /// If the status is PENDING, we grab references to the data parts to prevent them from being deleted from the disk + /// Otherwise, the operation has already been completed and there is no need to keep the data parts alive + /// You might also ask: why bother adding tasks that have already been completed (i.e, status != PENDING)? + /// The reason is the `replicated_partition_exports` table in the local only mode might miss entries if they are not added here. + if (status == ExportReplicatedMergeTreePartitionTaskEntry::Status::PENDING) { - if (const auto part = storage.getPartIfExists(part_name, {MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated})) + for (const auto & part_name : metadata.parts) { - part_references.push_back(part); + if (const auto part = storage.getPartIfExists(part_name, {MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated})) + { + part_references.push_back(part); + } } } /// Insert or update entry. The multi_index container automatically maintains both indexes. - auto entry = ExportReplicatedMergeTreePartitionTaskEntry {metadata, ExportReplicatedMergeTreePartitionTaskEntry::Status::PENDING, std::move(part_references)}; + auto entry = ExportReplicatedMergeTreePartitionTaskEntry {metadata, status, std::move(part_references)}; auto it = entries_by_key.find(key); if (it != entries_by_key.end()) entries_by_key.replace(it, entry); diff --git a/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.h b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.h index 99078e486cb3..855ecc334c09 100644 --- a/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.h +++ b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB { @@ -31,6 +32,7 @@ class ExportPartitionManifestUpdatingTask void addTask( const ExportReplicatedMergeTreePartitionManifest & metadata, + ExportReplicatedMergeTreePartitionTaskEntry::Status status, const std::string & key, auto & entries_by_key ); From 74ed3f7d5f91e98e9368b62e5eeb41b0bb2edfe9 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Tue, 17 Mar 2026 20:15:12 +0100 Subject: [PATCH 09/15] Merge pull request #1517 from Altinity/fix_race_condition_partition_strategy Fix IPartitionStrategy race condition Source-PR: #1517 (https://github.com/Altinity/ClickHouse/pull/1517) --- src/Storages/IPartitionStrategy.cpp | 84 ++++++++++++++----- src/Storages/IPartitionStrategy.h | 14 ++-- .../test.py | 42 ++++++---- .../test.py | 40 ++++++++- ..._part_limits_and_table_functions.reference | 2 + ...ge_tree_part_limits_and_table_functions.sh | 61 ++++++++++++-- 6 files changed, 192 insertions(+), 51 deletions(-) diff --git a/src/Storages/IPartitionStrategy.cpp b/src/Storages/IPartitionStrategy.cpp index 9f6b40f79bed..fe40adf37f79 100644 --- a/src/Storages/IPartitionStrategy.cpp +++ b/src/Storages/IPartitionStrategy.cpp @@ -22,6 +22,8 @@ extern const int BAD_ARGUMENTS; namespace { + using PartitionExpressionActionsAndColumnName = IPartitionStrategy::PartitionExpressionActionsAndColumnName; + /// Builds AST for hive partition path format /// `partition_column_1=toString(partition_value_expr_1)/ ... /partition_column_N=toString(partition_value_expr_N)/` /// for given partition columns list and a partition by AST. @@ -89,6 +91,33 @@ namespace return result; } + ASTPtr buildToStringPartitionAST(ASTPtr partition_by) + { + ASTs arguments(1, partition_by); + return makeASTFunction("toString", std::move(arguments)); + } + + template + PartitionExpressionActionsAndColumnName getCachedOrBuildActions( + const std::optional & cached_result, + const IPartitionStrategy & partition_strategy, + BuildAST && build_ast) + { + if (cached_result) + return *cached_result; + + auto expression_ast = build_ast(); + return partition_strategy.getPartitionExpressionActions(expression_ast); + } + + void cacheDeterministicActions( + std::optional & cached_result, + const PartitionExpressionActionsAndColumnName & actions_with_column) + { + if (!actions_with_column.actions->getActionsDAG().hasNonDeterministic()) + cached_result = actions_with_column; + } + std::shared_ptr createHivePartitionStrategy( ASTPtr partition_by, const Block & sample_block, @@ -197,11 +226,8 @@ const KeyDescription & IPartitionStrategy::getPartitionKeyDescription() const } IPartitionStrategy::PartitionExpressionActionsAndColumnName -IPartitionStrategy::getPartitionExpressionActions(ASTPtr & expression_ast) +IPartitionStrategy::getPartitionExpressionActions(ASTPtr & expression_ast) const { - if (cached_result) - return *cached_result; - auto syntax_result = TreeRewriter(context).analyze(expression_ast, sample_block.getNamesAndTypesList()); auto actions_dag = ExpressionAnalyzer(expression_ast, syntax_result, context).getActionsDAG(false); @@ -210,9 +236,6 @@ IPartitionStrategy::getPartitionExpressionActions(ASTPtr & expression_ast) std::move(actions_dag), ExpressionActionsSettings(context), false); result.column_name = expression_ast->getColumnName(); - if (!result.actions->getActionsDAG().hasNonDeterministic()) - cached_result = result; - return result; } @@ -266,13 +289,19 @@ std::shared_ptr PartitionStrategyFactory::get(StrategyType s WildcardPartitionStrategy::WildcardPartitionStrategy(KeyDescription partition_key_description_, const Block & sample_block_, ContextPtr context_) : IPartitionStrategy(partition_key_description_, sample_block_, context_) { + auto actions_with_column = getCachedOrBuildActions( + cached_result, + *this, + [&] { return buildToStringPartitionAST(partition_key_description.definition_ast); }); + cacheDeterministicActions(cached_result, actions_with_column); } -ColumnPtr WildcardPartitionStrategy::computePartitionKey(const Chunk & chunk) +ColumnPtr WildcardPartitionStrategy::computePartitionKey(const Chunk & chunk) const { - ASTs arguments(1, partition_key_description.definition_ast); - ASTPtr partition_by_string = makeASTFunction("toString", std::move(arguments)); - auto actions_with_column = getPartitionExpressionActions(partition_by_string); + auto actions_with_column = getCachedOrBuildActions( + cached_result, + *this, + [&] { return buildToStringPartitionAST(partition_key_description.definition_ast); }); Block block_with_partition_by_expr = sample_block.cloneWithoutColumns(); block_with_partition_by_expr.setColumns(chunk.getColumns()); @@ -281,11 +310,13 @@ ColumnPtr WildcardPartitionStrategy::computePartitionKey(const Chunk & chunk) return block_with_partition_by_expr.getByName(actions_with_column.column_name).column; } -ColumnPtr WildcardPartitionStrategy::computePartitionKey(Block & block) +ColumnPtr WildcardPartitionStrategy::computePartitionKey(Block & block) const { - ASTs arguments(1, partition_key_description.definition_ast); - ASTPtr partition_by_string = makeASTFunction("toString", std::move(arguments)); - auto actions_with_column = getPartitionExpressionActions(partition_by_string); + auto actions_with_column = getCachedOrBuildActions( + cached_result, + *this, + [&] { return buildToStringPartitionAST(partition_key_description.definition_ast); }); + actions_with_column.actions->execute(block); return block.getByName(actions_with_column.column_name).column; } @@ -307,12 +338,20 @@ HiveStylePartitionStrategy::HiveStylePartitionStrategy( } block_without_partition_columns = buildBlockWithoutPartitionColumns(sample_block, partition_columns_name_set); + + auto actions_with_column = getCachedOrBuildActions( + cached_result, + *this, + [&] { return buildHivePartitionAST(partition_key_description.definition_ast, getPartitionColumns()); }); + cacheDeterministicActions(cached_result, actions_with_column); } -ColumnPtr HiveStylePartitionStrategy::computePartitionKey(const Chunk & chunk) +ColumnPtr HiveStylePartitionStrategy::computePartitionKey(const Chunk & chunk) const { - auto hive_ast = buildHivePartitionAST(partition_key_description.definition_ast, getPartitionColumns()); - auto actions_with_column = getPartitionExpressionActions(hive_ast); + auto actions_with_column = getCachedOrBuildActions( + cached_result, + *this, + [&] { return buildHivePartitionAST(partition_key_description.definition_ast, getPartitionColumns()); }); Block block_with_partition_by_expr = sample_block.cloneWithoutColumns(); block_with_partition_by_expr.setColumns(chunk.getColumns()); @@ -321,10 +360,13 @@ ColumnPtr HiveStylePartitionStrategy::computePartitionKey(const Chunk & chunk) return block_with_partition_by_expr.getByName(actions_with_column.column_name).column; } -ColumnPtr HiveStylePartitionStrategy::computePartitionKey(Block & block) +ColumnPtr HiveStylePartitionStrategy::computePartitionKey(Block & block) const { - auto hive_ast = buildHivePartitionAST(partition_key_description.definition_ast, getPartitionColumns()); - auto actions_with_column = getPartitionExpressionActions(hive_ast); + auto actions_with_column = getCachedOrBuildActions( + cached_result, + *this, + [&] { return buildHivePartitionAST(partition_key_description.definition_ast, getPartitionColumns()); }); + actions_with_column.actions->execute(block); return block.getByName(actions_with_column.column_name).column; } diff --git a/src/Storages/IPartitionStrategy.h b/src/Storages/IPartitionStrategy.h index 91397de2362d..1378762c6911 100644 --- a/src/Storages/IPartitionStrategy.h +++ b/src/Storages/IPartitionStrategy.h @@ -27,9 +27,9 @@ struct IPartitionStrategy virtual ~IPartitionStrategy() = default; - virtual ColumnPtr computePartitionKey(const Chunk & chunk) = 0; + virtual ColumnPtr computePartitionKey(const Chunk & chunk) const = 0; - virtual ColumnPtr computePartitionKey(Block & block) = 0; + virtual ColumnPtr computePartitionKey(Block & block) const = 0; virtual ColumnRawPtrs getFormatChunkColumns(const Chunk & chunk) { @@ -48,7 +48,7 @@ struct IPartitionStrategy NamesAndTypesList getPartitionColumns() const; const KeyDescription & getPartitionKeyDescription() const; - PartitionExpressionActionsAndColumnName getPartitionExpressionActions(ASTPtr & expression_ast); + PartitionExpressionActionsAndColumnName getPartitionExpressionActions(ASTPtr & expression_ast) const; protected: const KeyDescription partition_key_description; @@ -91,9 +91,9 @@ struct WildcardPartitionStrategy : IPartitionStrategy { WildcardPartitionStrategy(KeyDescription partition_key_description_, const Block & sample_block_, ContextPtr context_); - ColumnPtr computePartitionKey(const Chunk & chunk) override; + ColumnPtr computePartitionKey(const Chunk & chunk) const override; - ColumnPtr computePartitionKey(Block & block) override; + ColumnPtr computePartitionKey(Block & block) const override; }; /* @@ -110,9 +110,9 @@ struct HiveStylePartitionStrategy : IPartitionStrategy const std::string & file_format_, bool partition_columns_in_data_file_); - ColumnPtr computePartitionKey(const Chunk & chunk) override; + ColumnPtr computePartitionKey(const Chunk & chunk) const override; - ColumnPtr computePartitionKey(Block & block) override; + ColumnPtr computePartitionKey(Block & block) const override; ColumnRawPtrs getFormatChunkColumns(const Chunk & chunk) override; Block getFormatHeader() override; diff --git a/tests/integration/test_export_merge_tree_part_to_object_storage/test.py b/tests/integration/test_export_merge_tree_part_to_object_storage/test.py index 1539cb130598..8adec2908d1c 100644 --- a/tests/integration/test_export_merge_tree_part_to_object_storage/test.py +++ b/tests/integration/test_export_merge_tree_part_to_object_storage/test.py @@ -52,8 +52,10 @@ def test_drop_column_during_export_snapshot(cluster): skip_if_remote_database_disk_enabled(cluster) node = cluster.instances["node1"] - mt_table = "mutations_snapshot_mt_table" - s3_table = "mutations_snapshot_s3_table" + postfix = str(uuid.uuid4()).replace("-", "_") + + mt_table = f"mutations_snapshot_mt_table_{postfix}" + s3_table = f"mutations_snapshot_s3_table_{postfix}" create_tables_and_insert_data(node, mt_table, s3_table) @@ -104,8 +106,10 @@ def test_add_column_during_export(cluster): skip_if_remote_database_disk_enabled(cluster) node = cluster.instances["node1"] - mt_table = "add_column_during_export_mt_table" - s3_table = "add_column_during_export_s3_table" + postfix = str(uuid.uuid4()).replace("-", "_") + + mt_table = f"add_column_during_export_mt_table_{postfix}" + s3_table = f"add_column_during_export_s3_table_{postfix}" create_tables_and_insert_data(node, mt_table, s3_table) @@ -157,8 +161,10 @@ def test_pending_mutations_throw_before_export(cluster): """Test that pending mutations before export throw an error with default settings.""" node = cluster.instances["node1"] - mt_table = "pending_mutations_throw_mt_table" - s3_table = "pending_mutations_throw_s3_table" + postfix = str(uuid.uuid4()).replace("-", "_") + + mt_table = f"pending_mutations_throw_mt_table_{postfix}" + s3_table = f"pending_mutations_throw_s3_table_{postfix}" create_tables_and_insert_data(node, mt_table, s3_table) @@ -180,8 +186,10 @@ def test_pending_mutations_skip_before_export(cluster): """Test that pending mutations before export are skipped with throw_on_pending_mutations=false.""" node = cluster.instances["node1"] - mt_table = "pending_mutations_skip_mt_table" - s3_table = "pending_mutations_skip_s3_table" + postfix = str(uuid.uuid4()).replace("-", "_") + + mt_table = f"pending_mutations_skip_mt_table_{postfix}" + s3_table = f"pending_mutations_skip_s3_table_{postfix}" create_tables_and_insert_data(node, mt_table, s3_table) @@ -210,8 +218,10 @@ def test_data_mutations_after_export_started(cluster): skip_if_remote_database_disk_enabled(cluster) node = cluster.instances["node1"] - mt_table = "mutations_after_export_mt_table" - s3_table = "mutations_after_export_s3_table" + postfix = str(uuid.uuid4()).replace("-", "_") + + mt_table = f"mutations_after_export_mt_table_{postfix}" + s3_table = f"mutations_after_export_s3_table_{postfix}" create_tables_and_insert_data(node, mt_table, s3_table) @@ -256,8 +266,10 @@ def test_pending_patch_parts_throw_before_export(cluster): """Test that pending patch parts before export throw an error with default settings.""" node = cluster.instances["node1"] - mt_table = "pending_patches_throw_mt_table" - s3_table = "pending_patches_throw_s3_table" + postfix = str(uuid.uuid4()).replace("-", "_") + + mt_table = f"pending_patches_throw_mt_table_{postfix}" + s3_table = f"pending_patches_throw_s3_table_{postfix}" create_tables_and_insert_data(node, mt_table, s3_table) @@ -279,8 +291,10 @@ def test_pending_patch_parts_skip_before_export(cluster): """Test that pending patch parts before export are skipped with throw_on_pending_patch_parts=false.""" node = cluster.instances["node1"] - mt_table = "pending_patches_skip_mt_table" - s3_table = "pending_patches_skip_s3_table" + postfix = str(uuid.uuid4()).replace("-", "_") + + mt_table = f"pending_patches_skip_mt_table_{postfix}" + s3_table = f"pending_patches_skip_s3_table_{postfix}" create_tables_and_insert_data(node, mt_table, s3_table) diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py index 5b96e1ad286b..77fe7e96a119 100644 --- a/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py +++ b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py @@ -69,6 +69,42 @@ def wait_for_export_to_start( raise TimeoutError(f"Export did not start within {timeout}s. ") +def wait_for_exception_count( + node, + mt_table: str, + s3_table: str, + partition_id: str, + min_exception_count: int = 1, + timeout: int = 30, + poll_interval: float = 0.5, +): + """Wait for exception_count to reach at least min_exception_count.""" + start_time = time.time() + last_exception_count = None + while time.time() - start_time < timeout: + exception_count_str = node.query( + f""" + SELECT exception_count FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{s3_table}' + AND partition_id = '{partition_id}' + """ + ).strip() + + if exception_count_str: + exception_count = int(exception_count_str) + last_exception_count = exception_count + if exception_count >= min_exception_count: + return exception_count + + time.sleep(poll_interval) + + raise TimeoutError( + f"Exception count did not reach {min_exception_count} within {timeout}s. " + f"Last exception_count: {last_exception_count if last_exception_count is not None else 'N/A'}" + ) + + def skip_if_remote_database_disk_enabled(cluster): """Skip test if any instance in the cluster has remote database disk enabled. @@ -585,8 +621,8 @@ def test_inject_short_living_failures(cluster): f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS export_merge_tree_partition_max_retries=100;" ) - # wait only for a second to get at least one failure, but not enough to finish the export - time.sleep(5) + # wait for at least one exception to occur, but not enough to finish the export + wait_for_exception_count(node, mt_table, s3_table, "2020", min_exception_count=1, timeout=30) # wait for the export to finish wait_for_export_status(node, mt_table, s3_table, "2020", "COMPLETED") diff --git a/tests/queries/0_stateless/03572_export_merge_tree_part_limits_and_table_functions.reference b/tests/queries/0_stateless/03572_export_merge_tree_part_limits_and_table_functions.reference index ce4f112ad1fa..b7f1f4411bf6 100644 --- a/tests/queries/0_stateless/03572_export_merge_tree_part_limits_and_table_functions.reference +++ b/tests/queries/0_stateless/03572_export_merge_tree_part_limits_and_table_functions.reference @@ -1,6 +1,8 @@ ---- Test max_bytes and max_rows per file ---- Table function with schema inheritance (no schema specified) ---- Table function with explicit compatible schema +Waiting for exports to complete (timeout: 60s)... +All exports completed. ---- Count files in big_destination_max_bytes, should be 5 (4 parquet, 1 commit) 5 ---- Count rows in big_table and big_destination_max_bytes diff --git a/tests/queries/0_stateless/03572_export_merge_tree_part_limits_and_table_functions.sh b/tests/queries/0_stateless/03572_export_merge_tree_part_limits_and_table_functions.sh index 449720bbf7a3..dff7332662d0 100755 --- a/tests/queries/0_stateless/03572_export_merge_tree_part_limits_and_table_functions.sh +++ b/tests/queries/0_stateless/03572_export_merge_tree_part_limits_and_table_functions.sh @@ -14,7 +14,14 @@ tf_schema_explicit="tf_schema_explicit_${RANDOM}" mt_table_tf="mt_table_tf_${RANDOM}" query() { - $CLICKHOUSE_CLIENT --query "$1" + local query_text="$1" + local query_id="$2" + + if [ -n "$query_id" ]; then + $CLICKHOUSE_CLIENT --query_id="$query_id" --query "$query_text" + else + $CLICKHOUSE_CLIENT --query "$query_text" + fi } query "DROP TABLE IF EXISTS $big_table, $big_destination_max_bytes, $big_destination_max_rows, $mt_table_tf" @@ -44,19 +51,59 @@ big_part_max_rows=$(query "SELECT name FROM system.parts WHERE database = curren # ALL EXPORTS HAPPEN HERE # ============================================================================ +# Generate unique query_ids for each export to track them in part_log +export_query_id_1="export_${RANDOM}_1" +export_query_id_2="export_${RANDOM}_2" +export_query_id_3="export_${RANDOM}_3" +export_query_id_4="export_${RANDOM}_4" + # this should generate ~4 files -query "ALTER TABLE $big_table EXPORT PART '$big_part_max_bytes' TO TABLE $big_destination_max_bytes SETTINGS allow_experimental_export_merge_tree_part = 1, export_merge_tree_part_max_bytes_per_file=3500000, output_format_parquet_row_group_size_bytes=1000000" +query "ALTER TABLE $big_table EXPORT PART '$big_part_max_bytes' TO TABLE $big_destination_max_bytes SETTINGS allow_experimental_export_merge_tree_part = 1, export_merge_tree_part_max_bytes_per_file=3500000, output_format_parquet_row_group_size_bytes=1000000" "$export_query_id_1" # export_merge_tree_part_max_rows_per_file = 1048576 (which is 4194304/4) to generate 4 files -query "ALTER TABLE $big_table EXPORT PART '$big_part_max_rows' TO TABLE $big_destination_max_rows SETTINGS allow_experimental_export_merge_tree_part = 1, export_merge_tree_part_max_rows_per_file=1048576" +query "ALTER TABLE $big_table EXPORT PART '$big_part_max_rows' TO TABLE $big_destination_max_rows SETTINGS allow_experimental_export_merge_tree_part = 1, export_merge_tree_part_max_rows_per_file=1048576" "$export_query_id_2" echo "---- Table function with schema inheritance (no schema specified)" -query "ALTER TABLE $mt_table_tf EXPORT PART '2022_1_1_0' TO TABLE FUNCTION s3(s3_conn, filename='$tf_schema_inherit', format='Parquet', partition_strategy='hive') PARTITION BY year SETTINGS allow_experimental_export_merge_tree_part = 1" +query "ALTER TABLE $mt_table_tf EXPORT PART '2022_1_1_0' TO TABLE FUNCTION s3(s3_conn, filename='$tf_schema_inherit', format='Parquet', partition_strategy='hive') PARTITION BY year SETTINGS allow_experimental_export_merge_tree_part = 1" "$export_query_id_3" echo "---- Table function with explicit compatible schema" -query "ALTER TABLE $mt_table_tf EXPORT PART '2023_2_2_0' TO TABLE FUNCTION s3(s3_conn, filename='$tf_schema_explicit', format='Parquet', structure='id UInt64, value String, year UInt16', partition_strategy='hive') PARTITION BY year SETTINGS allow_experimental_export_merge_tree_part = 1" +query "ALTER TABLE $mt_table_tf EXPORT PART '2023_2_2_0' TO TABLE FUNCTION s3(s3_conn, filename='$tf_schema_explicit', format='Parquet', structure='id UInt64, value String, year UInt16', partition_strategy='hive') PARTITION BY year SETTINGS allow_experimental_export_merge_tree_part = 1" "$export_query_id_4" + +# Wait for all exports to complete +wait_for_exports() { + local timeout=${1:-60} + local poll_interval=${2:-0.5} + local start_time=$(date +%s) + local elapsed=0 + + echo "Waiting for exports to complete (timeout: ${timeout}s)..." + + while [ $elapsed -lt $timeout ]; do + # Flush logs to ensure part_log entries are visible + query "SYSTEM FLUSH LOGS" > /dev/null 2>&1 || true + + # Wait for part_log entries - these are written synchronously when export completes + # Check if all expected exports have corresponding part_log entries by query_id + local completed_count=$(query "SELECT count() FROM system.part_log WHERE event_type = 'ExportPart' AND query_id IN ('$export_query_id_1', '$export_query_id_2', '$export_query_id_3', '$export_query_id_4')" | tr -d '\n') + + if [ "$completed_count" = "4" ]; then + echo "All exports completed." + return 0 + fi + + sleep $poll_interval + elapsed=$(($(date +%s) - start_time)) + done + + echo "Timeout waiting for exports to complete after ${timeout}s" + query "SYSTEM FLUSH LOGS" > /dev/null 2>&1 || true + echo "Completed exports in part_log:" + query "SELECT query_id, table, part_name, event_time FROM system.part_log WHERE event_type = 'ExportPart' AND query_id IN ('$export_query_id_1', '$export_query_id_2', '$export_query_id_3', '$export_query_id_4')" + echo "Remaining exports in system.exports:" + query "SELECT source_table, part_name, elapsed, rows_read, total_rows_to_read FROM system.exports WHERE ((source_table = '$big_table' AND part_name IN ('$big_part_max_bytes', '$big_part_max_rows')) OR (source_table = '$mt_table_tf' AND part_name IN ('2022_1_1_0', '2023_2_2_0')))" + return 1 +} -# ONE BIG SLEEP after all exports (longer because it writes multiple files) -sleep 20 +wait_for_exports 60 # ============================================================================ # ALL SELECTS/VERIFICATIONS HAPPEN HERE From 1a8a97cb6db3aa17c1119af4b08462e10ce09627 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Wed, 18 Mar 2026 14:17:21 +0100 Subject: [PATCH 10/15] Merge pull request #1499 from Altinity/export_partition_increment_scheduled_after_scheduling Bump scheduled exports count only in case it has been scheduled Source-PR: #1499 (https://github.com/Altinity/ClickHouse/pull/1499) --- src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp b/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp index 909ada39749e..23dd49fa6841 100644 --- a/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp +++ b/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp @@ -238,6 +238,8 @@ void ExportPartitionTaskScheduler::run() LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to schedule export part task, skipping"); return; } + + scheduled_exports_count++; } else { @@ -268,6 +270,8 @@ void ExportPartitionTaskScheduler::run() { handlePartExportCompletion(key, zk_part_name, manifest, destination_storage, result); }); + + scheduled_exports_count++; } catch (const Exception &) { @@ -279,7 +283,6 @@ void ExportPartitionTaskScheduler::run() } } - scheduled_exports_count++; } } } From a54dddd183eb8f7d6e5d7eb18f7c95ab9d037cf5 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Tue, 31 Mar 2026 19:27:47 +0200 Subject: [PATCH 11/15] Merge pull request #1593 from Altinity/fix_query_cancelled_not_releasing_the_lock Export Partition - release the part lock when the query is cancelled Source-PR: #1593 (https://github.com/Altinity/ClickHouse/pull/1593) --- .../ExportPartitionTaskScheduler.cpp | 43 ++++- .../test.py | 149 ++++++++++++++++++ 2 files changed, 184 insertions(+), 8 deletions(-) diff --git a/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp b/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp index 23dd49fa6841..238b7c890856 100644 --- a/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp +++ b/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp @@ -354,20 +354,13 @@ void ExportPartitionTaskScheduler::handlePartExportFailure( size_t max_retries ) { - LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} export failed, will now increment counters", part_name); + LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} export failed", part_name); if (!exception) { throw Exception(ErrorCodes::LOGICAL_ERROR, "ExportPartition scheduler task: No exception provided for error handling. Sounds like a bug"); } - /// Early exit if the query was cancelled - no need to increment error counts - if (exception->code() == ErrorCodes::QUERY_WAS_CANCELLED) - { - LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} export was cancelled, skipping error handling", part_name); - return; - } - Coordination::Stat locked_by_stat; std::string locked_by; @@ -385,6 +378,40 @@ void ExportPartitionTaskScheduler::handlePartExportFailure( return; } + /// Early exit if the query was cancelled - no need to increment error counts + if (exception->code() == ErrorCodes::QUERY_WAS_CANCELLED) + { + /// Releasing the lock is important because a query can be cancelled due to SYSTEM STOP MOVES. If this is the case, + /// other replicas should still be able to export this individual part. That's why there is a retry loop here. + /// It is very unlikely this will be a problem in practice. The lock is ephemeral, which means it is automatically released + /// if ClickHouse loses connection to ZooKeeper + std::size_t retry_count = 0; + static constexpr std::size_t max_lock_release_retries = 3; + while (retry_count < max_lock_release_retries) + { + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRemove); + + const auto removal_code = zk->tryRemove(export_path / "locks" / part_name, locked_by_stat.version); + + if (Coordination::Error::ZOK == removal_code) + { + break; + } + + if (Coordination::Error::ZBADVERSION == removal_code) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} lock version mismatch, will not increment error counts", part_name); + break; + } + + retry_count++; + } + + LOG_INFO(storage.log, "ExportPartition scheduler task: Part {} export was cancelled, skipping error handling", part_name); + return; + } + Coordination::Requests ops; const auto processing_part_path = processing_parts_path / part_name; diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py index 77fe7e96a119..327d78cfa49b 100644 --- a/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py +++ b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py @@ -1377,3 +1377,152 @@ def test_sharded_export_partition_default_pattern(cluster): # only one file with 3 rows should be present assert int(total_count) == 3, f"Expected 3 rows, got {total_count}" + + +def test_export_partition_scheduler_skipped_when_moves_stopped(cluster): + node = cluster.instances["replica1"] + + uid = str(uuid.uuid4()).replace("-", "_") + mt_table = f"sched_skip_mt_{uid}" + s3_table = f"sched_skip_s3_{uid}" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + + node.query(f"SYSTEM STOP MOVES {mt_table}") + + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table}" + ) + + wait_for_export_to_start(node, mt_table, s3_table, "2020") + + # Wait for several scheduler cycles (each fires every 5 s). + # If the guard is missing the scheduler would run and data would land in S3. + time.sleep(10) + + status = node.query( + f"SELECT status FROM system.replicated_partition_exports" + f" WHERE source_table = '{mt_table}' AND destination_table = '{s3_table}'" + f" AND partition_id = '2020'" + ).strip() + + assert status == "PENDING", ( + f"Expected PENDING while moves are stopped, got '{status}'" + ) + + row_count = int(node.query(f"SELECT count() FROM {s3_table} WHERE year = 2020").strip()) + assert row_count == 0, ( + f"Expected 0 rows in S3 while scheduler is skipped, got {row_count}" + ) + + node.query(f"SYSTEM START MOVES {mt_table}") + + wait_for_export_status(node, mt_table, s3_table, "2020", "COMPLETED", timeout=60) + + row_count = int(node.query(f"SELECT count() FROM {s3_table} WHERE year = 2020").strip()) + assert row_count == 3, f"Expected 3 rows in S3 after export completed, got {row_count}" + + +def test_export_partition_resumes_after_stop_moves(cluster): + node = cluster.instances["replica1"] + + uid = str(uuid.uuid4()).replace("-", "_") + mt_table = f"stop_moves_before_mt_{uid}" + s3_table = f"stop_moves_before_s3_{uid}" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + + node.query(f"SYSTEM STOP MOVES {mt_table}") + + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table}" + f" SETTINGS export_merge_tree_partition_max_retries = 50" + ) + + wait_for_export_to_start(node, mt_table, s3_table, "2020") + + # Give the scheduler enough time to attempt (and cancel) the part task at + # least once, exercising the lock-release code path. + time.sleep(5) + + status = node.query( + f"SELECT status FROM system.replicated_partition_exports" + f" WHERE source_table = '{mt_table}' AND destination_table = '{s3_table}'" + f" AND partition_id = '2020'" + ).strip() + assert status == "PENDING", f"Expected PENDING while moves are stopped, got '{status}'" + + row_count = int(node.query(f"SELECT count() FROM {s3_table} WHERE year = 2020").strip()) + assert row_count == 0, f"Expected 0 rows in S3 while moves are stopped, got {row_count}" + + node.query(f"SYSTEM START MOVES {mt_table}") + + wait_for_export_status(node, mt_table, s3_table, "2020", "COMPLETED", timeout=60) + + row_count = int(node.query(f"SELECT count() FROM {s3_table} WHERE year = 2020").strip()) + assert row_count == 3, f"Expected 3 rows in S3 after export completed, got {row_count}" + + +def test_export_partition_resumes_after_stop_moves_during_export(cluster): + skip_if_remote_database_disk_enabled(cluster) + + node = cluster.instances["replica1"] + + uid = str(uuid.uuid4()).replace("-", "_") + mt_table = f"stop_moves_during_mt_{uid}" + s3_table = f"stop_moves_during_s3_{uid}" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + + minio_ip = cluster.minio_ip + minio_port = cluster.minio_port + + with PartitionManager() as pm: + pm.add_rule({ + "instance": node, + "destination": node.ip_address, + "protocol": "tcp", + "source_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + }) + pm.add_rule({ + "instance": node, + "destination": minio_ip, + "protocol": "tcp", + "destination_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + }) + + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table}" + f" SETTINGS export_merge_tree_partition_max_retries = 50" + ) + + wait_for_export_to_start(node, mt_table, s3_table, "2020") + + # Let the tasks start executing and failing against the blocked S3. + time.sleep(2) + + node.query(f"SYSTEM STOP MOVES {mt_table}") + + # Give the cancel callback time to fire and the lock-release path to run. + time.sleep(3) + + status = node.query( + f"SELECT status FROM system.replicated_partition_exports" + f" WHERE source_table = '{mt_table}' AND destination_table = '{s3_table}'" + f" AND partition_id = '2020'" + ).strip() + + assert status == "PENDING", ( + f"Expected PENDING while moves are stopped and S3 is blocked, got '{status}'" + ) + + node.query(f"SYSTEM START MOVES {mt_table}") + + # MinIO is now unblocked; the next scheduler cycle should succeed. + wait_for_export_status(node, mt_table, s3_table, "2020", "COMPLETED", timeout=60) + + row_count = int(node.query(f"SELECT count() FROM {s3_table} WHERE year = 2020").strip()) + assert row_count == 3, f"Expected 3 rows in S3 after export completed, got {row_count}" + From 2ba0833ba694af4121d2c55fe68b8a6541ee899a Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Mon, 27 Apr 2026 10:17:47 +0200 Subject: [PATCH 12/15] Merge pull request #1618 from Altinity/export_partition_iceberg Export partition to apache iceberg Source-PR: #1618 (https://github.com/Altinity/ClickHouse/pull/1618) --- .../part_export.md | 25 +- .../partition_export.md | 39 +- src/Common/FailPoint.cpp | 5 + src/Core/ServerSettings.cpp | 2 +- src/Core/Settings.cpp | 11 +- src/Core/SettingsChangesHistory.cpp | 5 + .../InterpreterKillQueryQuery.cpp | 6 +- ...portReplicatedMergeTreePartitionManifest.h | 22 +- src/Storages/IStorage.h | 16 +- src/Storages/MergeTree/ExportPartTask.cpp | 19 + .../ExportPartitionManifestUpdatingTask.cpp | 247 ++++- .../ExportPartitionTaskScheduler.cpp | 96 +- .../MergeTree/ExportPartitionUtils.cpp | 393 +++++++- src/Storages/MergeTree/ExportPartitionUtils.h | 72 +- src/Storages/MergeTree/MergeTreeData.cpp | 70 +- src/Storages/MergeTree/MergeTreeData.h | 2 + .../MergeTree/MergeTreePartExportManifest.h | 4 + .../ReplicatedMergeTreeRestartingThread.cpp | 4 +- .../DataLakes/IDataLakeMetadata.h | 34 + .../DataLakes/Iceberg/AvroSchema.h | 38 + .../DataLakes/Iceberg/Constant.h | 1 + .../DataLakes/Iceberg/IcebergDataFileEntry.h | 50 + .../DataLakes/Iceberg/IcebergMetadata.cpp | 555 ++++++++++- .../DataLakes/Iceberg/IcebergMetadata.h | 59 ++ .../DataLakes/Iceberg/IcebergWrites.cpp | 441 ++++++++- .../DataLakes/Iceberg/IcebergWrites.h | 80 +- .../DataLakes/Iceberg/MetadataGenerator.cpp | 2 +- .../DataLakes/Iceberg/MultipleFileWriter.cpp | 41 +- .../DataLakes/Iceberg/MultipleFileWriter.h | 19 +- .../ObjectStorage/DataLakes/Iceberg/Utils.cpp | 5 +- .../ObjectStorage/DataLakes/Iceberg/Utils.h | 11 + .../ObjectStorage/StorageObjectStorage.cpp | 84 +- .../ObjectStorage/StorageObjectStorage.h | 5 +- .../StorageObjectStorageCluster.cpp | 6 + .../StorageObjectStorageCluster.h | 2 + src/Storages/StorageReplicatedMergeTree.cpp | 128 ++- src/Storages/System/attachSystemTables.cpp | 4 +- ...perimental_export_merge_tree_partition.xml | 3 + ...perimental_export_merge_tree_partition.xml | 3 - tests/config/install.sh | 2 +- .../helpers/export_partition_helpers.py | 201 ++++ .../helpers/iceberg_export_stats.py | 179 ++++ .../configs/config.d/metadata_log.xml | 7 + .../test.py | 481 +++++++++ .../test.py | 7 +- .../__init__.py | 0 .../allow_experimental_export_partition.xml | 3 + .../configs/config.d/metadata_log.xml | 7 + .../configs/users.d/profile.xml | 9 + .../test.py | 934 ++++++++++++++++++ .../allow_experimental_export_partition.xml | 2 +- .../disable_experimental_export_partition.xml | 2 +- .../configs/s3_retries.xml | 3 - .../test.py | 180 ++-- .../config.d/allow_export_partition.xml | 3 + .../users.d/allow_export_partition.xml | 7 + .../test_export_partition_iceberg.py | 912 +++++++++++++++++ .../test_export_partition_iceberg_catalog.py | 532 ++++++++++ 58 files changed, 5751 insertions(+), 329 deletions(-) rename docs/en/{engines/table-engines/mergetree-family => antalya}/part_export.md (91%) rename docs/en/{engines/table-engines/mergetree-family => antalya}/partition_export.md (68%) create mode 100644 src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergDataFileEntry.h create mode 100644 tests/config/config.d/allow_experimental_export_merge_tree_partition.xml delete mode 100644 tests/config/config.d/enable_experimental_export_merge_tree_partition.xml create mode 100644 tests/integration/helpers/export_partition_helpers.py create mode 100644 tests/integration/helpers/iceberg_export_stats.py create mode 100644 tests/integration/test_export_merge_tree_part_to_iceberg/configs/config.d/metadata_log.xml create mode 100644 tests/integration/test_export_merge_tree_part_to_iceberg/test.py create mode 100644 tests/integration/test_export_replicated_mt_partition_to_iceberg/__init__.py create mode 100644 tests/integration/test_export_replicated_mt_partition_to_iceberg/configs/allow_experimental_export_partition.xml create mode 100644 tests/integration/test_export_replicated_mt_partition_to_iceberg/configs/config.d/metadata_log.xml create mode 100644 tests/integration/test_export_replicated_mt_partition_to_iceberg/configs/users.d/profile.xml create mode 100644 tests/integration/test_export_replicated_mt_partition_to_iceberg/test.py delete mode 100644 tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/s3_retries.xml create mode 100644 tests/integration/test_storage_iceberg_with_spark/configs/config.d/allow_export_partition.xml create mode 100644 tests/integration/test_storage_iceberg_with_spark/configs/users.d/allow_export_partition.xml create mode 100644 tests/integration/test_storage_iceberg_with_spark/test_export_partition_iceberg.py create mode 100644 tests/integration/test_storage_iceberg_with_spark/test_export_partition_iceberg_catalog.py diff --git a/docs/en/engines/table-engines/mergetree-family/part_export.md b/docs/en/antalya/part_export.md similarity index 91% rename from docs/en/engines/table-engines/mergetree-family/part_export.md rename to docs/en/antalya/part_export.md index b7f0730596a4..03f9f479991b 100644 --- a/docs/en/engines/table-engines/mergetree-family/part_export.md +++ b/docs/en/antalya/part_export.md @@ -2,7 +2,7 @@ ## Overview -The `ALTER TABLE EXPORT PART` command exports individual MergeTree data parts to object storage (S3, Azure Blob Storage, etc.), typically in Parquet format. A commit file is shipped to the same destination directory containing all data files exported within that transaction. +The `ALTER TABLE EXPORT PART` command exports individual MergeTree data parts to object storage (S3, Azure Blob Storage, etc.) or data lakes like Apache Iceberg tables (with and without catalogs), typically in Parquet format. **Key Characteristics:** - **Experimental feature** - must be enabled via `allow_experimental_export_merge_tree_part` setting @@ -11,6 +11,14 @@ The `ALTER TABLE EXPORT PART` command exports individual MergeTree data parts to - **Idempotent** - safe to re-export the same part (skips by default if file exists) - **Preserves sort order** from the source table +### On Apache Iceberg storage exports: + +Each MergeTree part will become a separate file (or more depending on `max_bytes` and `max_rows` settings) following the engine naming convention. Once the part has been exported, new snapshots / manifest files are generated and the data is committed using the Apache Iceberg commit mechanism. + +### On plain object storage exports: + +A commit file is shipped to the same destination directory containing all data files exported within that transaction. + ## Syntax ```sql @@ -72,19 +80,28 @@ In case a table function is used as the destination, the schema can be omitted a - **Default**: `0` - **Description**: Maximum number of rows to write to a single file when exporting a merge tree part. 0 means no limit. This is not a hard limit, and it highly depends on the output format granularity and input source chunk size. Using this might break idempotency, use it with care. -### export_merge_tree_part_throw_on_pending_mutations +#### `export_merge_tree_part_file_already_exists_policy` (Optional) + +- **Type**: `MergeTreePartExportFileAlreadyExistsPolicy` +- **Default**: `skip` +- **Description**: Policy for handling files that already exist during export. Possible values: + - `skip` - Skip the file if it already exists + - `error` - Throw an error if the file already exists + - `overwrite` - Overwrite the file + +### `export_merge_tree_part_throw_on_pending_mutations` (Optional) - **Type**: `bool` - **Default**: `true` - **Description**: If set to true, throws if pending mutations exists for a given part. Note that by default mutations are applied to all parts, which means that if a mutation in practice would only affetct part/partition x, all the other parts/partition will throw upon export. The exception is when the `IN PARTITION` clause was used in the mutation command. Note the `IN PARTITION` clause is not properly implemented for plain MergeTree tables. -### export_merge_tree_part_throw_on_pending_patch_parts +### `export_merge_tree_part_throw_on_pending_patch_parts` (Optional) - **Type**: `bool` - **Default**: `true` - **Description**: If set to true, throws if pending patch parts exists for a given part. Note that by default mutations are applied to all parts, which means that if a mutation in practice would only affetct part/partition x, all the other parts/partition will throw upon export. The exception is when the `IN PARTITION` clause was used in the mutation command. Note the `IN PARTITION` clause is not properly implemented for plain MergeTree tables. -### export_merge_tree_part_filename_pattern +### `export_merge_tree_part_filename_pattern` (Optional) - **Type**: `String` - **Default**: `{part_name}_{checksum}` diff --git a/docs/en/engines/table-engines/mergetree-family/partition_export.md b/docs/en/antalya/partition_export.md similarity index 68% rename from docs/en/engines/table-engines/mergetree-family/partition_export.md rename to docs/en/antalya/partition_export.md index af503ec5180a..975915859482 100644 --- a/docs/en/engines/table-engines/mergetree-family/partition_export.md +++ b/docs/en/antalya/partition_export.md @@ -2,9 +2,7 @@ ## Overview -The `ALTER TABLE EXPORT PARTITION` command exports entire partitions from Replicated*MergeTree tables to object storage (S3, Azure Blob Storage, etc.), typically in Parquet format. This feature coordinates export part operations across all replicas using ZooKeeper. - -Each MergeTree part will become a separate file with the following name convention: `//_.`. To ensure atomicity, a commit file containing the relative paths of all exported parts is also shipped. A data file should only be considered part of the dataset if a commit file references it. The commit file will be named using the following convention: `/commit__`. +The `ALTER TABLE EXPORT PARTITION` command exports entire partitions from Replicated*MergeTree tables to object storage (S3, Azure Blob Storage, etc.) or data lakes like Apache Iceberg tables (with and without catalogs), typically in Parquet format. This feature coordinates export part operations across all replicas using ZooKeeper. The set of parts that are exported is based on the list of parts the replica that received the export command sees. The other replicas will assist in the export process if they have those parts locally. Otherwise they will ignore it. @@ -16,6 +14,20 @@ The export task can be killed by issuing the kill command: `KILL EXPORT PARTITIO The task is persistent - it should be resumed after crashes, failures and etc. +### On Apache Iceberg storage exports: + +Each MergeTree part will become a separate file (or more depending on `max_bytes` and `max_rows` settings) following the engine naming convention. Once all parts have been exported, new snapshots / manifest files are generated and the data is comitted using the Apache Iceberg commit mechanism. + +The manifest file produced by the commit contains a summary field `clickhouse.export-partition-transaction-id` that stores the transaction id. This field is used to implement idempotency and avoid data duplication. Some Apache Iceberg storage managers employ old manifests cleanup, ClickHouse does not. + +**IMPORTANT**: In case the storage is managed by a 3rd party application that cleans up old manifest files, it is important that the TTL of such files are greater than the timeout of export partition tasks. If it is not configured in such a way, it is possible to accidentally duplicate data in the extremely rare case a ClickHouse node is the only node working on a given export task, commits the data to Iceberg, crashes before marking the task as done and only boots up after the manifest cleanup has deleted the commit manifest. In such scenario, ClickHouse would attempt to commit those files again producing duplicates. The task timeout on ClickHouse side is controlled by the setting `export_merge_tree_partition_task_timeout_seconds`. + +The Iceberg manifest files contain statistics about the data. Exporting a merge tree partition is a non ephemeral long running task, in which nodes can be turned off and turned on. This means the stats of individual files need to be persisted somewhere in order to produce the final manifest. This is implemented through sidecars. Each data file exported will contain a "sibling" sidecar file named `_clickhouse_export_part_sidecar.avro`. ClickHouse does not clean up these files, and they can be safely deleted once the data is comitted. + +### On plain object storage exports: + +Each MergeTree part will become a separate file with the following name convention: `//_.`. To ensure atomicity, a commit file containing the relative paths of all exported parts is also shipped. A data file should only be considered part of the dataset if a commit file references it. The commit file will be named using the following convention: `/commit__`. + ## Syntax ```sql @@ -35,7 +47,7 @@ TO TABLE [destination_database.]destination_table ### Server Settings -#### `enable_experimental_export_merge_tree_partition_feature` (Required) +#### `allow_experimental_export_merge_tree_partition` (Required) - **Type**: `Bool` - **Default**: `false` @@ -47,7 +59,7 @@ TO TABLE [destination_database.]destination_table - **Type**: `Bool` - **Default**: `false` -- **Description**: Ignore existing partition export and overwrite the ZooKeeper entry. Allows re-exporting a partition to the same destination before the manifest expires. +- **Description**: Ignore existing partition export and overwrite the ZooKeeper entry. Allows re-exporting a partition to the same destination before the manifest expires. **IMPORTANT:** this is dangerous because it can lead to duplicated data, use it with caution. #### `export_merge_tree_partition_max_retries` (Optional) @@ -70,24 +82,35 @@ TO TABLE [destination_database.]destination_table - `error` - Throw an error if the file already exists - `overwrite` - Overwrite the file -### export_merge_tree_part_throw_on_pending_mutations +### `export_merge_tree_part_throw_on_pending_mutations` (Optional) - **Type**: `bool` - **Default**: `true` - **Description**: If set to true, throws if pending mutations exists for a given part. Note that by default mutations are applied to all parts, which means that if a mutation in practice would only affetct part/partition x, all the other parts/partition will throw upon export. The exception is when the `IN PARTITION` clause was used in the mutation command. Note the `IN PARTITION` clause is not properly implemented for plain MergeTree tables. -### export_merge_tree_part_throw_on_pending_patch_parts +### `export_merge_tree_part_throw_on_pending_patch_parts` (Optional) - **Type**: `bool` - **Default**: `true` - **Description**: If set to true, throws if pending patch parts exists for a given part. Note that by default mutations are applied to all parts, which means that if a mutation in practice would only affetct part/partition x, all the other parts/partition will throw upon export. The exception is when the `IN PARTITION` clause was used in the mutation command. Note the `IN PARTITION` clause is not properly implemented for plain MergeTree tables. -### export_merge_tree_part_filename_pattern +### `export_merge_tree_part_filename_pattern` (Optional) - **Type**: `String` - **Default**: `{part_name}_{checksum}` - **Description**: Pattern for the filename of the exported merge tree part. The `part_name` and `checksum` are calculated and replaced on the fly. Additional macros are supported. +### `export_merge_tree_partition_task_timeout_seconds` (Optional) + +- **Type**: `UInt64` +- **Default**: `3600` +- **Description**: The timeout is measured from the manifest's create_time. Set to 0 to disable the timeout. +When the timeout is exceeded the task transitions to KILLED (same terminal state as `KILL QUERY ... EXPORT PARTITION`), and `last_exception` is populated with a timeout reason. + +Notes: +- Enforcement is best-effort: actual kill latency is bounded by one manifest-updater poll cycle (~30s) plus ZooKeeper watch propagation. +- Since both this timeout and `export_merge_tree_partition_manifest_ttl` are measured from `create_time`, keep `export_merge_tree_partition_manifest_ttl` greater than `export_merge_tree_partition_task_timeout_seconds` if you want the KILLED entry to remain visible in `system.replicated_partition_exports` after the timeout fires. + ## Examples ### Basic Export to S3 diff --git a/src/Common/FailPoint.cpp b/src/Common/FailPoint.cpp index 56a4a55d07a5..6dc92963f5fb 100644 --- a/src/Common/FailPoint.cpp +++ b/src/Common/FailPoint.cpp @@ -127,6 +127,11 @@ static struct InitFiu ONCE(write_file_operation_fail_on_read) \ REGULAR(slowdown_parallel_replicas_local_plan_read) \ ONCE(iceberg_writes_cleanup) \ + ONCE(iceberg_writes_non_retry_cleanup) \ + ONCE(iceberg_writes_post_publish_throw) \ + ONCE(iceberg_export_after_commit_before_zk_completed) \ + REGULAR(export_partition_commit_always_throw) \ + ONCE(export_partition_status_change_throw) \ ONCE(backup_add_empty_memory_table) \ PAUSEABLE_ONCE(backup_pause_on_start) \ PAUSEABLE_ONCE(restore_pause_on_start) \ diff --git a/src/Core/ServerSettings.cpp b/src/Core/ServerSettings.cpp index be029be90097..957a35944fe8 100644 --- a/src/Core/ServerSettings.cpp +++ b/src/Core/ServerSettings.cpp @@ -1534,7 +1534,7 @@ The policy on how to perform a scheduling of CPU slots specified by `concurrent_ DECLARE(UInt64, object_storage_list_objects_cache_size, 500000000, "Maximum size of ObjectStorage list objects cache in bytes. Zero means disabled.", 0) \ DECLARE(UInt64, object_storage_list_objects_cache_max_entries, 1000, "Maximum size of ObjectStorage list objects cache in entries. Zero means disabled.", 0) \ DECLARE(UInt64, object_storage_list_objects_cache_ttl, 3600, "Time to live of records in ObjectStorage list objects cache in seconds. Zero means unlimited", 0) \ - DECLARE(Bool, enable_experimental_export_merge_tree_partition_feature, false, "Enable export replicated merge tree partition feature. It is experimental and not yet ready for production use.", 0) \ + DECLARE(Bool, allow_experimental_export_merge_tree_partition, false, "Enable export replicated merge tree partition feature. It is experimental and not yet ready for production use.", 0) \ DECLARE(UInt64, input_format_parquet_metadata_cache_max_size, 500000000, "Maximum size of parquet file metadata cache", 0) /// Settings with a path are server settings with at least one layer of nesting that have a fixed structure (no lists, lists, enumerations, repetitions, ...). diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index cd10964d620d..3643b9883cf2 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -7492,6 +7492,15 @@ Maximum number of retries for exporting a merge tree part in an export partition DECLARE(UInt64, export_merge_tree_partition_manifest_ttl, 180, R"( Determines how long the manifest will live in ZooKeeper. It prevents the same partition from being exported twice to the same destination. This setting does not affect / delete in progress tasks. It'll only cleanup the completed ones. +)", 0) \ + DECLARE(UInt64, export_merge_tree_partition_task_timeout_seconds, 3600, R"( +Maximum wall-clock duration (in seconds) an export partition task is allowed to remain in the PENDING state before it is auto-killed by the background cleanup loop. +The timeout is measured from the manifest's create_time. Set to 0 to disable the timeout. +When the timeout is exceeded the task transitions to KILLED (same terminal state as `KILL QUERY ... EXPORT PARTITION`), and `last_exception` is populated with a timeout reason. + +Notes: +- Enforcement is best-effort: actual kill latency is bounded by one manifest-updater poll cycle (~30s) plus ZooKeeper watch propagation. +- Since both this timeout and `export_merge_tree_partition_manifest_ttl` are measured from `create_time`, keep `export_merge_tree_partition_manifest_ttl` greater than `export_merge_tree_partition_task_timeout_seconds` if you want the KILLED entry to remain visible in `system.replicated_partition_exports` after the timeout fires. )", 0) \ DECLARE(MergeTreePartExportFileAlreadyExistsPolicy, export_merge_tree_part_file_already_exists_policy, MergeTreePartExportFileAlreadyExistsPolicy::skip, R"( Possible values: @@ -7517,7 +7526,7 @@ Throw an error if there are pending patch parts when exporting a merge tree part Only lock a part when the task is already running. This might help with busy waiting where the scheduler locks a part, but the task ends in the pending list. On the other hand, there is a chance once the task executes that part has already been locked by another replica and the task will simply early exit. )", 0) \ - DECLARE(Bool, export_merge_tree_partition_system_table_prefer_remote_information, true, R"( + DECLARE(Bool, export_merge_tree_partition_system_table_prefer_remote_information, false, R"( Controls whether the system.replicated_partition_exports will prefer to query ZooKeeper to get the most up to date information or use the local information. Querying ZooKeeper is expensive, and only available if the ZooKeeper feature flag MULTI_READ is enabled. )", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 93170c7fb8d9..bc3376dbda91 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -115,6 +115,11 @@ const VersionToSettingsChangesMap & getSettingsChangesHistory() // {"input_format_parquet_use_metadata_cache", true, false, "Obsolete. No-op"}, // https://github.com/Altinity/ClickHouse/pull/586 // {"object_storage_remote_initiator_cluster", "", "", "New setting."}, // {"iceberg_metadata_staleness_ms", 0, 0, "New setting allowing using cached metadata version at READ operations to prevent fetching from remote catalog"}, + {"use_parquet_metadata_cache", false, true, "Enables cache of parquet file metadata."}, + {"input_format_parquet_use_metadata_cache", true, false, "Obsolete. No-op"}, // https://github.com/Altinity/ClickHouse/pull/586 + {"object_storage_remote_initiator_cluster", "", "", "New setting."}, + {"iceberg_metadata_staleness_ms", 0, 0, "New setting allowing using cached metadata version at READ operations to prevent fetching from remote catalog"}, + {"export_merge_tree_partition_task_timeout_seconds", 0, 3600, "New setting to control the timeout for export partition tasks."}, }); addSettingsChanges(settings_changes_history, "26.1", { diff --git a/src/Interpreters/InterpreterKillQueryQuery.cpp b/src/Interpreters/InterpreterKillQueryQuery.cpp index a8d182f73d4e..4ab33f67cb75 100644 --- a/src/Interpreters/InterpreterKillQueryQuery.cpp +++ b/src/Interpreters/InterpreterKillQueryQuery.cpp @@ -40,7 +40,7 @@ namespace Setting namespace ServerSetting { - extern const ServerSettingsBool enable_experimental_export_merge_tree_partition_feature; + extern const ServerSettingsBool allow_experimental_export_merge_tree_partition; } namespace ErrorCodes @@ -259,10 +259,10 @@ BlockIO InterpreterKillQueryQuery::execute() } case ASTKillQueryQuery::Type::ExportPartition: { - if (!getContext()->getServerSettings()[ServerSetting::enable_experimental_export_merge_tree_partition_feature]) + if (!getContext()->getServerSettings()[ServerSetting::allow_experimental_export_merge_tree_partition]) { throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, - "Exporting merge tree partition is experimental. Set the server setting `enable_experimental_export_merge_tree_partition_feature` to enable it"); + "Exporting merge tree partition is experimental. Set the server setting `allow_experimental_export_merge_tree_partition` to enable it"); } Block exports_block = getSelectResult( diff --git a/src/Storages/ExportReplicatedMergeTreePartitionManifest.h b/src/Storages/ExportReplicatedMergeTreePartitionManifest.h index a41401dee09b..f1c96b120a28 100644 --- a/src/Storages/ExportReplicatedMergeTreePartitionManifest.h +++ b/src/Storages/ExportReplicatedMergeTreePartitionManifest.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -110,6 +111,7 @@ struct ExportReplicatedMergeTreePartitionManifest time_t create_time; size_t max_retries; size_t ttl_seconds; + size_t task_timeout_seconds; size_t max_threads; bool parallel_formatting; bool parquet_parallel_encoding; @@ -118,6 +120,8 @@ struct ExportReplicatedMergeTreePartitionManifest MergeTreePartExportManifest::FileAlreadyExistsPolicy file_already_exists_policy; String filename_pattern; bool lock_inside_the_task; /// todo temporary + bool write_full_path_in_iceberg_metadata = false; + String iceberg_metadata_json; std::string toJsonString() const { @@ -129,7 +133,12 @@ struct ExportReplicatedMergeTreePartitionManifest json.set("destination_table", destination_table); json.set("source_replica", source_replica); json.set("number_of_parts", number_of_parts); - + + if (!iceberg_metadata_json.empty()) + { + json.set("iceberg_metadata_json", iceberg_metadata_json); + } + Poco::JSON::Array::Ptr parts_array = new Poco::JSON::Array(); for (const auto & part : parts) parts_array->add(part); @@ -144,7 +153,9 @@ struct ExportReplicatedMergeTreePartitionManifest json.set("create_time", create_time); json.set("max_retries", max_retries); json.set("ttl_seconds", ttl_seconds); + json.set("task_timeout_seconds", task_timeout_seconds); json.set("lock_inside_the_task", lock_inside_the_task); + json.set("write_full_path_in_iceberg_metadata", write_full_path_in_iceberg_metadata); std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM oss.exceptions(std::ios::failbit); Poco::JSON::Stringifier::stringify(json, oss); @@ -166,12 +177,19 @@ struct ExportReplicatedMergeTreePartitionManifest manifest.source_replica = json->getValue("source_replica"); manifest.number_of_parts = json->getValue("number_of_parts"); manifest.max_retries = json->getValue("max_retries"); + + if (json->has("iceberg_metadata_json")) + { + manifest.iceberg_metadata_json = json->getValue("iceberg_metadata_json"); + } + auto parts_array = json->getArray("parts"); for (size_t i = 0; i < parts_array->size(); ++i) manifest.parts.push_back(parts_array->getElement(static_cast(i))); manifest.create_time = json->getValue("create_time"); manifest.ttl_seconds = json->getValue("ttl_seconds"); + manifest.task_timeout_seconds = json->getValue("task_timeout_seconds"); manifest.max_threads = json->getValue("max_threads"); manifest.parallel_formatting = json->getValue("parallel_formatting"); manifest.parquet_parallel_encoding = json->getValue("parquet_parallel_encoding"); @@ -192,6 +210,8 @@ struct ExportReplicatedMergeTreePartitionManifest manifest.lock_inside_the_task = json->getValue("lock_inside_the_task"); + manifest.write_full_path_in_iceberg_metadata = json->getValue("write_full_path_in_iceberg_metadata"); + return manifest; } }; diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index fa284595e994..b370342017e9 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -20,6 +21,7 @@ #include #include #include +#include #include #include @@ -475,7 +477,7 @@ class IStorage : public std::enable_shared_from_this, public TypePromo ContextPtr /*context*/, bool /*async_insert*/); - virtual bool supportsImport() const + virtual bool supportsImport(ContextPtr) const { return false; } @@ -492,16 +494,28 @@ It is currently only implemented in StorageObjectStorage. bool /* overwrite_if_exists */, std::size_t /* max_bytes_per_file */, std::size_t /* max_rows_per_file */, + const std::optional & /* iceberg_metadata_json_string */, const std::optional & /* format_settings */, ContextPtr /* context */) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Import is not implemented for storage {}", getName()); } + struct IcebergCommitExportPartitionArguments + { + std::string metadata_json_string; + /// Partition column values (after transforms). Callers are responsible for + /// populating this: the partition-export path parses them from the persisted + /// JSON string, while the direct EXPORT PART path reads them from the part's + /// partition key. + std::vector partition_values; + }; + virtual void commitExportPartitionTransaction( const String & /* transaction_id */, const String & /* partition_id */, const Strings & /* exported_paths */, + const IcebergCommitExportPartitionArguments & /* iceberg_commit_export_partition_arguments */, ContextPtr /* local_context */) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "commitExportPartitionTransaction is not implemented for storage type {}", getName()); diff --git a/src/Storages/MergeTree/ExportPartTask.cpp b/src/Storages/MergeTree/ExportPartTask.cpp index 931fbb6fb9db..9d8eee9928a5 100644 --- a/src/Storages/MergeTree/ExportPartTask.cpp +++ b/src/Storages/MergeTree/ExportPartTask.cpp @@ -21,6 +21,7 @@ #include #include #include +#include #include #include #include @@ -193,6 +194,7 @@ bool ExportPartTask::executeStep() manifest.file_already_exists_policy == MergeTreePartExportManifest::FileAlreadyExistsPolicy::overwrite, manifest.settings[Setting::export_merge_tree_part_max_bytes_per_file], manifest.settings[Setting::export_merge_tree_part_max_rows_per_file], + manifest.iceberg_metadata_json, getFormatSettings(local_context), local_context); @@ -272,6 +274,23 @@ bool ExportPartTask::executeStep() throw Exception(ErrorCodes::QUERY_WAS_CANCELLED, "Export part was cancelled"); } + /// For the direct EXPORT PART → Iceberg path there is no deferred-commit callback + /// (the partition-export path provides one that writes to ZooKeeper). + /// Commit the Iceberg metadata inline here so the rows become visible immediately. + if (destination_storage->isDataLake() && !manifest.completion_callback) + { + IStorage::IcebergCommitExportPartitionArguments iceberg_args; + iceberg_args.metadata_json_string = manifest.iceberg_metadata_json; + iceberg_args.partition_values = manifest.data_part->partition.value; + + destination_storage->commitExportPartitionTransaction( + manifest.transaction_id, + manifest.data_part->info.getPartitionId(), + (*exports_list_entry)->destination_file_paths, + iceberg_args, + local_context); + } + std::lock_guard inner_lock(storage.export_manifests_mutex); storage.writePartLog( PartLogElement::Type::EXPORT_PART, diff --git a/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp index e5b1c72ed4c8..0ed8d1033135 100644 --- a/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp +++ b/src/Storages/MergeTree/ExportPartitionManifestUpdatingTask.cpp @@ -6,7 +6,9 @@ #include #include #include +#include #include +#include namespace ProfileEvents { @@ -21,6 +23,17 @@ namespace ProfileEvents namespace DB { + +namespace ErrorCodes +{ + extern const int FAULT_INJECTED; +} + +namespace FailPoints +{ + extern const char export_partition_status_change_throw[]; +} + namespace { /* @@ -34,7 +47,8 @@ namespace const zkutil::ZooKeeperPtr & zk, const std::string & entry_path, const LoggerPtr & log, - const ContextPtr & context, + const ContextPtr & storage_context, + StorageReplicatedMergeTree & storage, const std::string & key, const ExportReplicatedMergeTreePartitionManifest & metadata, const time_t now, @@ -44,6 +58,10 @@ namespace { bool has_expired = metadata.create_time < now - static_cast(metadata.ttl_seconds); + bool task_timed_out = is_pending + && metadata.task_timeout_seconds > 0 + && metadata.create_time + static_cast(metadata.task_timeout_seconds) < now; + if (has_expired && !is_pending) { zk->tryRemoveRecursive(fs::path(entry_path)); @@ -56,8 +74,73 @@ namespace return true; } + else if (task_timed_out) + { + const std::string status_path = fs::path(entry_path) / "status"; + + Coordination::Stat status_stat; + std::string status_string; + + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperGet); + if (!zk->tryGet(status_path, status_string, &status_stat)) + { + LOG_INFO(log, "ExportPartition Manifest Updating Task: Failed to read status for {} while enforcing task timeout, skipping", entry_path); + return false; + } + + const auto current_status = magic_enum::enum_cast(status_string); + if (!current_status || *current_status != ExportReplicatedMergeTreePartitionTaskEntry::Status::PENDING) + { + LOG_INFO(log, "ExportPartition Manifest Updating Task: Task {} is not PENDING, can't set to KILLED, skipping", entry_path); + return false; + } + + const auto timeout_message = fmt::format( + "Export partition task timed out: exceeded export_merge_tree_partition_task_timeout_seconds={} (created at {}, now {})", + metadata.task_timeout_seconds, metadata.create_time, now); + + const auto killed_name = String(magic_enum::enum_name(ExportReplicatedMergeTreePartitionTaskEntry::Status::KILLED)); + + Coordination::Requests ops; + ExportPartitionUtils::appendExceptionOps( + ops, zk, fs::path(entry_path), storage.getReplicaName(), + /*part_name=*/"", timeout_message, log); + + ops.emplace_back(zkutil::makeSetRequest(status_path, killed_name, status_stat.version)); + + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperMulti); + + Coordination::Responses responses; + const auto rc = zk->tryMulti(ops, responses); + + if (rc == Coordination::Error::ZOK) + { + LOG_WARNING(log, + "ExportPartition Manifest Updating Task: task {} exceeded task_timeout_seconds={}s, " + "transitioned PENDING -> KILLED (atomic with exception record)", + entry_path, metadata.task_timeout_seconds); + } + else + { + /// ZBADVERSION (status changed), ZNODEEXISTS (lazy-create race with the scheduler), + /// counter race, or ZNONODE (entry concurrently removed). In all cases the batch + /// was rolled back atomically and the task will be re-evaluated on the next poll. + LOG_INFO(log, + "ExportPartition Manifest Updating Task: atomic kill for {} failed (rc={}); " + "status was concurrently updated or a ZK op conflicted, will retry on next poll", + entry_path, rc); + } + + /// Return false so the entry remains in entries_by_key; the status watch will drive + /// handleStatusChanges -> killExportPart on every replica, mirroring user-initiated KILL. + return false; + } else if (is_pending) { + auto context = ExportPartitionUtils::getContextCopyWithTaskSettings(storage_context, metadata); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperGetChildren); std::vector parts_in_processing_or_pending; @@ -81,7 +164,40 @@ namespace } /// it sounds like a replica exported the last part, but was not able to commit the export. Try to fix it - ExportPartitionUtils::commit(metadata, destination_storage, zk, log, entry_path, context); + try + { + ExportPartitionUtils::commit(metadata, destination_storage, zk, log, entry_path, context, storage); + } + catch (const Exception & e) + { + LOG_WARNING(log, + "ExportPartition Manifest Updating Task: " + "Caught exception while committing export for {}: {}", + entry_path, e.message()); + + /// Bump commit-attempts counter; transition to FAILED once the budget is exhausted. + /// This is the primary retry path for the commit phase — handlePartExportSuccess + /// only fires once (on the last part's completion); subsequent retries come from here. + const bool became_failed = ExportPartitionUtils::handleCommitFailure( + zk, + entry_path, + metadata.max_retries, + log); + + if (became_failed) + { + LOG_WARNING(log, + "ExportPartition Manifest Updating Task: " + "Commit for {} transitioned to FAILED after exhausting max_retries={}", + entry_path, metadata.max_retries); + } + + /// Return false so the next poll re-enters the cleanup path: + /// - if FAILED: status != PENDING on re-read, cleanup is a no-op + /// until the entry expires (handled by the first tryCleanup branch). + /// - if still PENDING: next poll increments the counter again. + return false; + } return true; } @@ -558,6 +674,7 @@ void ExportPartitionManifestUpdatingTask::poll() entry_path, storage.log.load(), storage.getContext(), + storage, key, metadata, now, @@ -656,62 +773,110 @@ void ExportPartitionManifestUpdatingTask::addStatusChange(const std::string & ke void ExportPartitionManifestUpdatingTask::handleStatusChanges() { - std::lock_guard lock(status_changes_mutex); - std::lock_guard task_entries_lock(storage.export_merge_tree_partition_mutex); - auto zk = storage.getZooKeeper(); - - LOG_INFO(storage.log, "ExportPartition Manifest Updating task: handling status changes. Number of status changes: {}", status_changes.size()); + /// copy the events to a local queue to avoid holding the status_changes_mutex while also holding export_merge_tree_partition_mutex + std::queue local_status_changes; + { + std::lock_guard lock(status_changes_mutex); + std::swap(status_changes, local_status_changes); + } - while (!status_changes.empty()) + try { - LOG_INFO(storage.log, "ExportPartition Manifest Updating task: handling status change for task {}", status_changes.front()); - const auto key = status_changes.front(); - status_changes.pop(); + std::lock_guard task_entries_lock(storage.export_merge_tree_partition_mutex); + auto zk = storage.getZooKeeper(); - auto it = storage.export_merge_tree_partition_task_entries_by_key.find(key); - if (it == storage.export_merge_tree_partition_task_entries_by_key.end()) - continue; + LOG_INFO(storage.log, "ExportPartition Manifest Updating task: handling status changes. Number of status changes: {}", local_status_changes.size()); - ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); - ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperGet); - /// get new status from zk - std::string new_status_string; - if (!zk->tryGet(fs::path(storage.zookeeper_path) / "exports" / key / "status", new_status_string)) + while (!local_status_changes.empty()) { - LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Failed to get new status for task {}, skipping", key); - continue; - } + const auto & key = local_status_changes.front(); + LOG_INFO(storage.log, "ExportPartition Manifest Updating task: handling status change for task {}", key); - const auto new_status = magic_enum::enum_cast(new_status_string); - if (!new_status) - { - LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Invalid status {} for task {}, skipping", new_status_string, key); - continue; - } + fiu_do_on(FailPoints::export_partition_status_change_throw, + { + throw Exception(ErrorCodes::FAULT_INJECTED, + "Failpoint: simulating exception during status change handling for key {}", key); + }); - LOG_INFO(storage.log, "ExportPartition Manifest Updating task: status changed for task {}. New status: {}", key, magic_enum::enum_name(*new_status).data()); + auto it = storage.export_merge_tree_partition_task_entries_by_key.find(key); + if (it == storage.export_merge_tree_partition_task_entries_by_key.end()) + { + local_status_changes.pop(); + continue; + } - /// If status changed to KILLED, cancel local export operations - if (*new_status == ExportReplicatedMergeTreePartitionTaskEntry::Status::KILLED) - { - try + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperGet); + /// get new status from zk + std::string new_status_string; + if (!zk->tryGet(fs::path(storage.zookeeper_path) / "exports" / key / "status", new_status_string)) { - LOG_INFO(storage.log, "ExportPartition Manifest Updating task: killing export partition for task {}", key); - storage.killExportPart(it->manifest.transaction_id); + LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Failed to get new status for task {}, skipping", key); + local_status_changes.pop(); + continue; } - catch (...) + + const auto new_status = magic_enum::enum_cast(new_status_string); + if (!new_status) { - tryLogCurrentException(storage.log, __PRETTY_FUNCTION__); + LOG_INFO(storage.log, "ExportPartition Manifest Updating Task: Invalid status {} for task {}, skipping", new_status_string, key); + local_status_changes.pop(); + continue; } + + LOG_INFO(storage.log, "ExportPartition Manifest Updating task: status changed for task {}. New status: {}", key, magic_enum::enum_name(*new_status).data()); + + /// If status changed to KILLED, cancel local export operations + if (*new_status == ExportReplicatedMergeTreePartitionTaskEntry::Status::KILLED) + { + try + { + LOG_INFO(storage.log, "ExportPartition Manifest Updating task: killing export partition for task {}", key); + storage.killExportPart(it->manifest.transaction_id); + } + catch (...) + { + tryLogCurrentException(storage.log, __PRETTY_FUNCTION__); + } + } + + it->status = *new_status; + + if (it->status != ExportReplicatedMergeTreePartitionTaskEntry::Status::PENDING) + { + /// we no longer need to keep the data parts alive + it->part_references.clear(); + } + + local_status_changes.pop(); } + } + catch (...) + { + tryLogCurrentException(storage.log, __PRETTY_FUNCTION__); + + LOG_INFO(storage.log, "ExportPartition Manifest Updating task: exception thrown while handling status changes, enqueuing remaining status changes back to the status_changes queue. Number of remaining status changes: {}", local_status_changes.size()); - it->status = *new_status; + std::lock_guard lock(status_changes_mutex); - if (it->status != ExportReplicatedMergeTreePartitionTaskEntry::Status::PENDING) + /// It is possible that an exception is thrown while handling the status. In this scenario + /// we need to enqueue the remaining status changes back to the status_changes queue not to lose them. + /// The other solution to this problem would be to ignore it and schedule a poll - maybe it is simpler? + if (!local_status_changes.empty()) { - /// we no longer need to keep the data parts alive - it->part_references.clear(); + // Prepend remaining items before any newly-arrived items + while (!status_changes.empty()) + { + local_status_changes.push(std::move(status_changes.front())); + status_changes.pop(); + } + + std::swap(status_changes, local_status_changes); } + + LOG_INFO(storage.log, "ExportPartition Manifest Updating task: The new number of pending status after enqueueing unprocessed ones is {}", status_changes.size()); + + throw; } } diff --git a/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp b/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp index 238b7c890856..96cc648ffbb5 100644 --- a/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp +++ b/src/Storages/MergeTree/ExportPartitionTaskScheduler.cpp @@ -20,7 +20,6 @@ namespace ProfileEvents extern const Event ExportPartitionZooKeeperSet; extern const Event ExportPartitionZooKeeperRemove; extern const Event ExportPartitionZooKeeperMulti; - extern const Event ExportPartitionZooKeeperExists; } @@ -38,30 +37,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -namespace -{ - ContextPtr getContextCopyWithTaskSettings(const ContextPtr & context, const ExportReplicatedMergeTreePartitionManifest & manifest) - { - auto context_copy = Context::createCopy(context); - context_copy->makeQueryContextForExportPart(); - context_copy->setCurrentQueryId(manifest.query_id); - context_copy->setSetting("output_format_parallel_formatting", manifest.parallel_formatting); - context_copy->setSetting("output_format_parquet_parallel_encoding", manifest.parquet_parallel_encoding); - context_copy->setSetting("max_threads", manifest.max_threads); - context_copy->setSetting("export_merge_tree_part_file_already_exists_policy", String(magic_enum::enum_name(manifest.file_already_exists_policy))); - context_copy->setSetting("export_merge_tree_part_max_bytes_per_file", manifest.max_bytes_per_file); - context_copy->setSetting("export_merge_tree_part_max_rows_per_file", manifest.max_rows_per_file); - - /// always skip pending mutations and patch parts because we already validated the parts during query processing - context_copy->setSetting("export_merge_tree_part_throw_on_pending_mutations", false); - context_copy->setSetting("export_merge_tree_part_throw_on_pending_patch_parts", false); - - context_copy->setSetting("export_merge_tree_part_filename_pattern", manifest.filename_pattern); - - return context_copy; - } -} - ExportPartitionTaskScheduler::ExportPartitionTaskScheduler(StorageReplicatedMergeTree & storage_) : storage(storage_) { @@ -199,7 +174,7 @@ void ExportPartitionTaskScheduler::run() LOG_INFO(storage.log, "ExportPartition scheduler task: Scheduling part export: {}", zk_part_name); - auto context = getContextCopyWithTaskSettings(storage.getContext(), manifest); + auto context = ExportPartitionUtils::getContextCopyWithTaskSettings(storage.getContext(), manifest); /// todo arthur this code path does not perform all the validations a simple part export does because we are not calling exportPartToTable directly. /// the schema and everything else has been validated when the export partition task was created, but nothing prevents the destination table from being @@ -217,6 +192,7 @@ void ExportPartitionTaskScheduler::run() context->getSettingsRef()[Setting::export_merge_tree_part_file_already_exists_policy].value, context->getSettingsCopy(), storage.getInMemoryMetadataPtr(), + manifest.iceberg_metadata_json, [this, key, zk_part_name, manifest, destination_storage] (MergeTreePartExportManifest::CompletionCallbackResult result) { @@ -263,7 +239,8 @@ void ExportPartitionTaskScheduler::run() part->name, destination_storage_id, manifest.transaction_id, - getContextCopyWithTaskSettings(storage.getContext(), manifest), + context, + manifest.iceberg_metadata_json, /*allow_outdated_parts*/ true, [this, key, zk_part_name, manifest, destination_storage] (MergeTreePartExportManifest::CompletionCallbackResult result) @@ -342,7 +319,31 @@ void ExportPartitionTaskScheduler::handlePartExportSuccess( LOG_INFO(storage.log, "ExportPartition scheduler task: All parts are processed, will try to commit export partition"); - ExportPartitionUtils::commit(manifest, destination_storage, zk, storage.log.load(), export_path, storage.getContext()); + try + { + auto context = ExportPartitionUtils::getContextCopyWithTaskSettings(storage.getContext(), manifest); + ExportPartitionUtils::commit(manifest, destination_storage, zk, storage.log.load(), export_path, context, storage); + } + catch (const Exception & e) + { + LOG_INFO(storage.log, "ExportPartition scheduler task: Caught exception while committing export partition, {}", e.message()); + + /// Bump commit-attempts counter; transition to FAILED once the budget is exhausted. + /// Prevents the task from remaining stuck in PENDING if commit() fails persistently + /// (e.g. schema/spec mismatch, prolonged destination outage). + const bool became_failed = ExportPartitionUtils::handleCommitFailure( + zk, + export_path, + manifest.max_retries, + storage.log.load()); + + if (became_failed) + { + LOG_WARNING(storage.log, + "ExportPartition scheduler task: Commit for {} transitioned to FAILED after exhausting max_retries={}", + export_path.string(), manifest.max_retries); + } + } } void ExportPartitionTaskScheduler::handlePartExportFailure( @@ -450,42 +451,9 @@ void ExportPartitionTaskScheduler::handlePartExportFailure( LOG_INFO(storage.log, "ExportPartition scheduler task: Retry count limit not exceeded for part {}, will increment retry count", part_name); } - const auto exceptions_per_replica_path = export_path / "exceptions_per_replica" / storage.replica_name; - const auto count_path = exceptions_per_replica_path / "count"; - const auto last_exception_path = exceptions_per_replica_path / "last_exception"; - - ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); - ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperExists); - if (zk->exists(exceptions_per_replica_path)) - { - LOG_INFO(storage.log, "ExportPartition scheduler task: Exceptions per replica path exists, no need to create it"); - std::string num_exceptions_string; - if (zk->tryGet(count_path, num_exceptions_string)) - { - const auto num_exceptions = parse(num_exceptions_string) + 1; - ops.emplace_back(zkutil::makeSetRequest(count_path, std::to_string(num_exceptions), -1)); - } - else - { - /// TODO maybe we should find a better way to handle this case, not urgent - LOG_INFO(storage.log, "ExportPartition scheduler task: Failed to get number of exceptions, will not increment it"); - } - - ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); - ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperGet); - - ops.emplace_back(zkutil::makeSetRequest(last_exception_path / "part", part_name, -1)); - ops.emplace_back(zkutil::makeSetRequest(last_exception_path / "exception", exception->message(), -1)); - } - else - { - LOG_INFO(storage.log, "ExportPartition scheduler task: Exceptions per replica path does not exist, will create it"); - ops.emplace_back(zkutil::makeCreateRequest(exceptions_per_replica_path, "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(count_path, "1", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(last_exception_path, "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(last_exception_path / "part", part_name, zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(last_exception_path / "exception", exception->message(), zkutil::CreateMode::Persistent)); - } + ExportPartitionUtils::appendExceptionOps( + ops, zk, export_path, storage.replica_name, part_name, + exception->message(), storage.log.load()); ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperMulti); diff --git a/src/Storages/MergeTree/ExportPartitionUtils.cpp b/src/Storages/MergeTree/ExportPartitionUtils.cpp index 6a0baf19f761..02069754bd3d 100644 --- a/src/Storages/MergeTree/ExportPartitionUtils.cpp +++ b/src/Storages/MergeTree/ExportPartitionUtils.cpp @@ -1,10 +1,19 @@ #include #include #include +#include #include #include "Storages/ExportReplicatedMergeTreePartitionManifest.h" #include "Storages/ExportReplicatedMergeTreePartitionTaskEntry.h" +#include #include +#include +#include + +#if USE_AVRO +#include +#include +#endif namespace ProfileEvents { @@ -12,15 +21,74 @@ namespace ProfileEvents extern const Event ExportPartitionZooKeeperGet; extern const Event ExportPartitionZooKeeperGetChildren; extern const Event ExportPartitionZooKeeperSet; + extern const Event ExportPartitionZooKeeperMulti; + extern const Event ExportPartitionZooKeeperExists; } namespace DB { +namespace ErrorCodes +{ + extern const int FAULT_INJECTED; + extern const int BAD_ARGUMENTS; + extern const int NO_SUCH_DATA_PART; + extern const int CORRUPTED_DATA; + extern const int NETWORK_ERROR; +} + +namespace FailPoints +{ + extern const char iceberg_export_after_commit_before_zk_completed[]; + extern const char export_partition_commit_always_throw[]; +} + namespace fs = std::filesystem; namespace ExportPartitionUtils { + std::vector getPartitionValuesForIcebergCommit( + MergeTreeData & storage, const String & partition_id) + { + auto lock = storage.readLockParts(); + const auto parts = storage.getDataPartsVectorInPartitionForInternalUsage( + MergeTreeDataPartState::Active, partition_id, lock); + + /// todo arthur: bad arguments for now, pick a better one + if (parts.empty()) + throw Exception(ErrorCodes::NO_SUCH_DATA_PART, + "Cannot find active part for partition_id '{}' to derive Iceberg partition " + "values. Edge case: the partition may have been dropped after export started, " + "or this replica has not yet received any part for this partition. " + "The commit will be retried.", + partition_id); + return parts.front()->partition.value; + } + + ContextPtr getContextCopyWithTaskSettings(const ContextPtr & context, const ExportReplicatedMergeTreePartitionManifest & manifest) + { + auto context_copy = Context::createCopy(context); + context_copy->makeQueryContextForExportPart(); + context_copy->setCurrentQueryId(manifest.query_id); + context_copy->setSetting("output_format_parallel_formatting", manifest.parallel_formatting); + context_copy->setSetting("output_format_parquet_parallel_encoding", manifest.parquet_parallel_encoding); + context_copy->setSetting("max_threads", manifest.max_threads); + context_copy->setSetting("export_merge_tree_part_file_already_exists_policy", String(magic_enum::enum_name(manifest.file_already_exists_policy))); + context_copy->setSetting("export_merge_tree_part_max_bytes_per_file", manifest.max_bytes_per_file); + context_copy->setSetting("export_merge_tree_part_max_rows_per_file", manifest.max_rows_per_file); + context_copy->setSetting("iceberg_insert_max_bytes_in_data_file", manifest.max_bytes_per_file); + context_copy->setSetting("iceberg_insert_max_rows_in_data_file", manifest.max_rows_per_file); + + /// always skip pending mutations and patch parts because we already validated the parts during query processing + context_copy->setSetting("export_merge_tree_part_throw_on_pending_mutations", false); + context_copy->setSetting("export_merge_tree_part_throw_on_pending_patch_parts", false); + + context_copy->setSetting("export_merge_tree_part_filename_pattern", manifest.filename_pattern); + context_copy->setSetting("write_full_path_in_iceberg_metadata", manifest.write_full_path_in_iceberg_metadata); + + return context_copy; + } + /// Collect all the exported paths from the processed parts /// If multiRead is supported by the keeper implementation, it is done in a single request /// Otherwise, multiple async requests are sent @@ -83,24 +151,54 @@ namespace ExportPartitionUtils const zkutil::ZooKeeperPtr & zk, const LoggerPtr & log, const std::string & entry_path, - const ContextPtr & context) + const ContextPtr & context_in, + MergeTreeData & source_storage) { + auto context = Context::createCopy(context_in); + context->setSetting("write_full_path_in_iceberg_metadata", manifest.write_full_path_in_iceberg_metadata); + + /// Failpoint used by integration tests to force persistent commit failure and exercise + /// the commit-attempts budget / FAILED state transition. + fiu_do_on(FailPoints::export_partition_commit_always_throw, + { + throw Exception(ErrorCodes::FAULT_INJECTED, + "Failpoint: export_partition_commit_always_throw"); + }); + const auto exported_paths = ExportPartitionUtils::getExportedPaths(log, zk, entry_path); if (exported_paths.empty()) { - LOG_WARNING(log, "ExportPartition: No exported paths found, will not commit export. This might be a bug"); - return; + throw Exception(ErrorCodes::CORRUPTED_DATA, "ExportPartition: No exported paths found, will not commit export. This might be a bug"); } //// not checking for an exact match because a single part might generate multiple files if (exported_paths.size() < manifest.parts.size()) { - LOG_WARNING(log, "ExportPartition: Reached the commit phase, but exported paths size is less than the number of parts, will not commit export. This might be a bug"); - return; + throw Exception(ErrorCodes::CORRUPTED_DATA, "ExportPartition: Reached the commit phase, but exported paths size is less than the number of parts, will not commit export. This might be a bug"); } - destination_storage->commitExportPartitionTransaction(manifest.transaction_id, manifest.partition_id, exported_paths, context); + IStorage::IcebergCommitExportPartitionArguments iceberg_args; + + if (!manifest.iceberg_metadata_json.empty()) + { + iceberg_args.metadata_json_string = manifest.iceberg_metadata_json; + if (source_storage.getInMemoryMetadataPtr()->hasPartitionKey()) + iceberg_args.partition_values = + getPartitionValuesForIcebergCommit(source_storage, manifest.partition_id); + } + + destination_storage->commitExportPartitionTransaction(manifest.transaction_id, manifest.partition_id, exported_paths, iceberg_args, context); + + /// Failpoint to simulate a crash after the Iceberg commit succeeds but before + /// ZooKeeper is updated to COMPLETED. Used by idempotency integration tests. + fiu_do_on(FailPoints::iceberg_export_after_commit_before_zk_completed, + { + LOG_INFO(log, "Failpoint: simulating crash after Iceberg commit, before ZK COMPLETED"); + std::this_thread::sleep_for(std::chrono::seconds(10)); + throw Exception(ErrorCodes::FAULT_INJECTED, + "Failpoint: simulating crash after Iceberg commit, before ZK COMPLETED"); + }); LOG_INFO(log, "ExportPartition: Committed export, mark as completed"); ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); @@ -111,9 +209,290 @@ namespace ExportPartitionUtils } else { - LOG_INFO(log, "ExportPartition: Failed to mark export as completed, will not try to fix it"); + throw Exception(ErrorCodes::NETWORK_ERROR, "ExportPartition: Failed to mark export as completed, will not try to fix it"); + } + } + + bool handleCommitFailure( + const zkutil::ZooKeeperPtr & zk, + const std::string & entry_path, + size_t max_attempts, + const LoggerPtr & log) + { + const std::string status_path = fs::path(entry_path) / "status"; + + /// Read /status together with its stat so we can (a) bail early if another + /// replica has already moved the task out of PENDING and (b) use a + /// version-checked Set later to avoid clobbering a concurrent write + /// (e.g. a racing successful commit that marked the task COMPLETED between + /// our read and our tryMulti). + Coordination::Stat status_stat; + std::string current_status; + + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperGet); + if (!zk->tryGet(status_path, current_status, &status_stat)) + { + /// Task was removed (TTL cleanup or force-overwrite). Nothing to do. + LOG_INFO(log, "ExportPartition: /status missing for {}, skipping commit-failure bookkeeping", entry_path); + return false; + } + + const auto status = magic_enum::enum_cast(current_status); + if (!status) + { + LOG_INFO(log, "ExportPartition: Invalid status {} for task {}, skipping commit-failure bookkeeping", current_status, entry_path); + return false; + } + + if (status != ExportReplicatedMergeTreePartitionTaskEntry::Status::PENDING) + { + /// Another replica already reached a terminal state (COMPLETED or FAILED). + /// Do NOT overwrite — a successful commit by a peer must win. + LOG_INFO(log, + "ExportPartition: /status for {} is {} (not PENDING), skipping commit-failure bookkeeping", + entry_path, current_status); + return false; + } + + Coordination::Requests ops; + + /// Bump the global commit_attempts counter (shared across replicas). + /// Non-atomic get+set(-1), matching exceptions_per_replica/count semantics. + /// Under a race, two replicas may see the same value and write the same +1, + /// under-counting by one. FAILED then fires one retry later than the threshold, + /// which is acceptable (we always converge to FAILED, never "never"). + const std::string commit_attempts_path = fs::path(entry_path) / "commit_attempts"; + + size_t attempts = 0; + std::string attempts_string; + + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperGet); + if (zk->tryGet(commit_attempts_path, attempts_string)) + { + try + { + attempts = parse(attempts_string); + } + catch (...) + { + LOG_WARNING(log, "ExportPartition: commit_attempts value '{}' at {} is not a valid integer, treating as 0", attempts_string, commit_attempts_path); + attempts = 0; + } + + attempts += 1; + ops.emplace_back(zkutil::makeSetRequest(commit_attempts_path, std::to_string(attempts), -1)); + } + else + { + attempts = 1; + ops.emplace_back(zkutil::makeCreateRequest(commit_attempts_path, "1", zkutil::CreateMode::Persistent)); + } + + /// Transition to FAILED if the commit budget is exhausted. + /// Uses the same setting as per-part retries (manifest.max_retries) per user decision. + /// Version-checked Set: if /status has changed since we read it (e.g. a peer's + /// commit() succeeded and wrote COMPLETED), the whole multi aborts with + /// ZBADVERSION and we safely do nothing — the winning terminal state stands. + const bool exhausted = attempts >= max_attempts; + if (exhausted) + { + ops.emplace_back(zkutil::makeSetRequest( + status_path, + String(magic_enum::enum_name(ExportReplicatedMergeTreePartitionTaskEntry::Status::FAILED)).data(), + status_stat.version)); + } + + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperMulti); + Coordination::Responses responses; + const auto rc = zk->tryMulti(ops, responses); + if (rc != Coordination::Error::ZOK) + { + /// Any error here (ZBADVERSION on /status race or counter race, ZNODEEXISTS on + /// lazy-create race, ZNONODE if someone removed the task concurrently) is + /// non-fatal: the next attempt re-reads /status and either skips (terminal + /// state won) or retries the bookkeeping. Worst case we delay FAILED by one + /// poll cycle, which matches the best-effort property of the existing counters. + LOG_INFO(log, "ExportPartition: Failed to persist commit failure bookkeeping for {}: {}", entry_path, rc); + return false; + } + + LOG_INFO(log, + "ExportPartition: Commit failure recorded for {} (attempt {}/{}){}", + entry_path, attempts, max_attempts, + exhausted ? ", task transitioned to FAILED" : ""); + + return exhausted; + } + + void appendExceptionOps( + Coordination::Requests & ops, + const zkutil::ZooKeeperPtr & zk, + const std::filesystem::path & entry_path, + const std::string & replica_name, + const std::string & part_name, + const std::string & exception_message, + const LoggerPtr & log) + { + const auto exceptions_per_replica_path = entry_path / "exceptions_per_replica" / replica_name; + const auto count_path = exceptions_per_replica_path / "count"; + const auto last_exception_path = exceptions_per_replica_path / "last_exception"; + + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperExists); + if (zk->exists(exceptions_per_replica_path)) + { + LOG_INFO(log, "ExportPartition: Exceptions per replica path exists, no need to create it"); + std::string num_exceptions_string; + + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperRequests); + ProfileEvents::increment(ProfileEvents::ExportPartitionZooKeeperGet); + if (zk->tryGet(count_path, num_exceptions_string)) + { + const auto num_exceptions = parse(num_exceptions_string) + 1; + ops.emplace_back(zkutil::makeSetRequest(count_path, std::to_string(num_exceptions), -1)); + } + else + { + /// TODO maybe we should find a better way to handle this case, not urgent + LOG_INFO(log, "ExportPartition: Failed to get number of exceptions, will not increment it"); + } + + ops.emplace_back(zkutil::makeSetRequest(last_exception_path / "part", part_name, -1)); + ops.emplace_back(zkutil::makeSetRequest(last_exception_path / "exception", exception_message, -1)); + } + else + { + LOG_INFO(log, "ExportPartition: Exceptions per replica path does not exist, will create it"); + ops.emplace_back(zkutil::makeCreateRequest(exceptions_per_replica_path, "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(count_path, "1", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(last_exception_path, "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(last_exception_path / "part", part_name, zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(last_exception_path / "exception", exception_message, zkutil::CreateMode::Persistent)); + } + } + +#if USE_AVRO + void verifyIcebergPartitionCompatibility( + const Poco::JSON::Object::Ptr & metadata_object, + const ASTPtr & partition_key_ast) + { + const auto original_schema_id = metadata_object->getValue(Iceberg::f_current_schema_id); + const auto partition_spec_id = metadata_object->getValue(Iceberg::f_default_spec_id); + + Poco::JSON::Object::Ptr current_schema_json; + { + const auto schemas = metadata_object->getArray(Iceberg::f_schemas); + for (size_t i = 0; i < schemas->size(); ++i) + { + auto s = schemas->getObject(static_cast(i)); + if (s->getValue(Iceberg::f_schema_id) == static_cast(original_schema_id)) + { + current_schema_json = s; + break; + } + } + } + + Poco::JSON::Object::Ptr partition_spec_json; + { + const auto specs = metadata_object->getArray(Iceberg::f_partition_specs); + for (size_t i = 0; i < specs->size(); ++i) + { + auto s = specs->getObject(static_cast(i)); + if (s->getValue(Iceberg::f_spec_id) == partition_spec_id) + { + partition_spec_json = s; + break; + } + } + } + + if (!current_schema_json || !partition_spec_json) + return; + + /// Build column_name → Iceberg source-id from the destination schema (and the inverse). + std::unordered_map column_name_to_source_id; + std::unordered_map source_id_to_column_name; + { + const auto schema_fields = current_schema_json->getArray(Iceberg::f_fields); + for (size_t i = 0; i < schema_fields->size(); ++i) + { + auto f = schema_fields->getObject(static_cast(i)); + const auto col_name = f->getValue(Iceberg::f_name); + const auto source_id = f->getValue(Iceberg::f_id); + column_name_to_source_id[col_name] = source_id; + source_id_to_column_name[source_id] = col_name; + } + } + + auto source_id_to_name = [&](Int32 id) -> String + { + auto it = source_id_to_column_name.find(id); + return it != source_id_to_column_name.end() ? it->second : fmt::format("", id); + }; + + /// Convert the MergeTree PARTITION BY AST into the equivalent Iceberg spec. + Poco::JSON::Array::Ptr expected_fields; + try + { + const auto expected_spec = Iceberg::getPartitionSpec( + partition_key_ast, column_name_to_source_id).first; + expected_fields = expected_spec->getArray(Iceberg::f_fields); + } + catch (const Exception & e) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Cannot export partition to Iceberg table: the source MergeTree partition " + "key cannot be represented as an Iceberg partition spec: {}", e.message()); + } + + const auto actual_fields = partition_spec_json->getArray(Iceberg::f_fields); + const size_t expected_size = expected_fields ? expected_fields->size() : 0; + const size_t actual_size = actual_fields ? actual_fields->size() : 0; + + if (expected_size != actual_size) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Cannot export partition to Iceberg table: partition scheme mismatch. " + "Source MergeTree has {} partition field(s), destination Iceberg table has {}.", + expected_size, actual_size); + + for (size_t i = 0; i < expected_size; ++i) + { + auto ef = expected_fields->getObject(static_cast(i)); + auto af = actual_fields->getObject(static_cast(i)); + + const auto expected_source_id = ef->getValue(Iceberg::f_source_id); + const auto actual_source_id = af->getValue(Iceberg::f_source_id); + const auto expected_transform = ef->getValue(Iceberg::f_transform); + const auto actual_transform = af->getValue(Iceberg::f_transform); + + /// Normalize both transform names through parseTransformAndArgument so that + /// equivalent aliases ("day"/"days", "hour"/"hours", "year"/"years", etc.) + /// produced by different writers (ClickHouse vs Spark/Trino) compare equal. + /// Comparison is on {function_name, argument}; time_zone is writer-specific + /// and not part of the partition spec identity. + const auto expected_canonical = Iceberg::parseTransformAndArgument(expected_transform); + const auto actual_canonical = Iceberg::parseTransformAndArgument(actual_transform); + const bool transforms_match = + (expected_canonical && actual_canonical) + ? (expected_canonical->transform_name == actual_canonical->transform_name + && expected_canonical->argument == actual_canonical->argument) + : (expected_transform == actual_transform); + + if (expected_source_id != actual_source_id || !transforms_match) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Cannot export partition to Iceberg table: partition field {} mismatch. " + "Source MergeTree maps to column '{}' (source_id={}) transform='{}', " + "but destination Iceberg has column '{}' (source_id={}) transform='{}'.", + i, + source_id_to_name(expected_source_id), expected_source_id, expected_transform, + source_id_to_name(actual_source_id), actual_source_id, actual_transform); } } +#endif } } diff --git a/src/Storages/MergeTree/ExportPartitionUtils.h b/src/Storages/MergeTree/ExportPartitionUtils.h index 40fe04a5bfd3..411f3b5224be 100644 --- a/src/Storages/MergeTree/ExportPartitionUtils.h +++ b/src/Storages/MergeTree/ExportPartitionUtils.h @@ -1,28 +1,98 @@ #pragma once +#include #include #include +#include #include #include #include "Storages/IStorage.h" +#include + +#if USE_AVRO +#include +#include +#endif namespace DB { +class MergeTreeData; struct ExportReplicatedMergeTreePartitionManifest; namespace ExportPartitionUtils { std::vector getExportedPaths(const LoggerPtr & log, const zkutil::ZooKeeperPtr & zk, const std::string & export_path); + ContextPtr getContextCopyWithTaskSettings(const ContextPtr & context, const ExportReplicatedMergeTreePartitionManifest & manifest); + + /// Returns the partition key values for the given partition_id by reading from + /// the first active local part. Throws LOGICAL_ERROR if no such part is found. + /// + /// Edge case: if the partition was dropped after export started, or this replica + /// has not yet received any part for this partition (extreme replication lag on a + /// recovery path), no active part will be found and the commit will fail. The task + /// will be retried on the next poll cycle or picked up by a different replica. + std::vector getPartitionValuesForIcebergCommit( + MergeTreeData & storage, const String & partition_id); + void commit( const ExportReplicatedMergeTreePartitionManifest & manifest, const StoragePtr & destination_storage, const zkutil::ZooKeeperPtr & zk, const LoggerPtr & log, const std::string & entry_path, - const ContextPtr & context + const ContextPtr & context, + MergeTreeData & source_storage ); + + /// Handles a commit-phase failure for a replicated partition export: + /// - increments /commit_attempts (lazy-created) + /// - sets /status to FAILED once attempts >= max_attempts + /// + /// The counter is a best-effort, non-atomic get+set(-1), matching + /// exceptions_per_replica/count. Concurrent failing commits may under-count by one + /// (FAILED may fire one retry later than the threshold), which is acceptable. + /// + /// `replica_name` and `exception` are currently unused and reserved for future + /// integration with per-replica diagnostics. + /// + /// Returns true if this call transitioned the task to FAILED. + bool handleCommitFailure( + const zkutil::ZooKeeperPtr & zk, + const std::string & entry_path, + size_t max_attempts, + const LoggerPtr & log); + + /// Appends ZK ops to `ops` that record a per-replica exception under + /// /exceptions_per_replica//last_exception/{exception,part} + /// and increment /exceptions_per_replica//count, + /// creating the subtree if absent. + /// + /// The count increment is non-atomic (synchronous tryGet + set with version -1). + /// Concurrent failing writers may under-count by one, which is accepted in this + /// subsystem and matches the pre-existing behaviour. + /// + /// Intended to be combined with additional ops (for example a version-guarded + /// status set) and executed as a single `tryMulti` so the exception record and + /// the accompanying state transition commit atomically. + void appendExceptionOps( + Coordination::Requests & ops, + const zkutil::ZooKeeperPtr & zk, + const std::filesystem::path & entry_path, + const std::string & replica_name, + const std::string & part_name, + const std::string & exception_message, + const LoggerPtr & log); + +#if USE_AVRO + /// Verifies that the source MergeTree partition key is compatible with the + /// destination Iceberg partition spec by comparing field source-ids and + /// transforms in order. Throws BAD_ARGUMENTS if they do not match. + void verifyIcebergPartitionCompatibility( + const Poco::JSON::Object::Ptr & metadata_object, + const ASTPtr & partition_key_ast); +#endif } } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index ba35d334b825..6f735cec54e7 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -27,7 +27,8 @@ #include #include #include -#include "Storages/MergeTree/ExportPartTask.h" +#include +#include #include #include #include @@ -88,12 +89,15 @@ #include #include #include +#include #include #include #include #include #include #include +#include +#include #include #include #include @@ -232,6 +236,7 @@ namespace Setting extern const SettingsBool output_format_parquet_parallel_encoding; extern const SettingsBool export_merge_tree_part_throw_on_pending_mutations; extern const SettingsBool export_merge_tree_part_throw_on_pending_patch_parts; + extern const SettingsBool allow_insert_into_iceberg; } namespace MergeTreeSetting @@ -6656,6 +6661,7 @@ void MergeTreeData::exportPartToTable( const StorageID & destination_storage_id, const String & transaction_id, ContextPtr query_context, + const std::optional & iceberg_metadata_json, bool allow_outdated_parts, std::function completion_callback) { @@ -6666,7 +6672,7 @@ void MergeTreeData::exportPartToTable( throw Exception(ErrorCodes::BAD_ARGUMENTS, "Exporting to the same table is not allowed"); } - exportPartToTable(part_name, dest_storage, transaction_id, query_context, allow_outdated_parts, completion_callback); + exportPartToTable(part_name, dest_storage, transaction_id, query_context, iceberg_metadata_json, allow_outdated_parts, completion_callback); } void MergeTreeData::exportPartToTable( @@ -6674,10 +6680,11 @@ void MergeTreeData::exportPartToTable( const StoragePtr & dest_storage, const String & transaction_id, ContextPtr query_context, + const std::optional & iceberg_metadata_json_, bool allow_outdated_parts, std::function completion_callback) { - if (!dest_storage->supportsImport()) + if (!dest_storage->supportsImport(query_context)) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Destination storage {} does not support MergeTree parts or uses unsupported partitioning", dest_storage->getName()); auto query_to_string = [] (const ASTPtr & ast) @@ -6688,6 +6695,53 @@ void MergeTreeData::exportPartToTable( auto source_metadata_ptr = getInMemoryMetadataPtr(); auto destination_metadata_ptr = dest_storage->getInMemoryMetadataPtr(); + std::string iceberg_metadata_json; + + if (dest_storage->isDataLake()) + { + if (!query_context->getSettingsRef()[Setting::allow_insert_into_iceberg]) + { + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "Iceberg writes are experimental. " + "To allow its usage, enable the setting allow_experimental_insert_into_iceberg"); + } + +#if USE_AVRO + if (iceberg_metadata_json_) + { + iceberg_metadata_json = *iceberg_metadata_json_; + } + else + { + /// ever since Anton introduced swarms, object storage instances are always a StorageObjectStorageCluster + auto * object_storage = dynamic_cast(dest_storage.get()); + + /// in theory this should never happen, but just in case + if (!object_storage) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Destination storage {} is not a StorageObjectStorageCluster", dest_storage->getName()); + } + + auto * iceberg_metadata = dynamic_cast(object_storage->getExternalMetadata(query_context)); + if (!iceberg_metadata) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Destination storage {} is a data lake but not an iceberg table", dest_storage->getName()); + } + + const auto metadata_object = iceberg_metadata->getMetadataJSON(query_context); + + std::ostringstream oss; + metadata_object->stringify(oss); + iceberg_metadata_json = oss.str(); + + ExportPartitionUtils::verifyIcebergPartitionCompatibility(metadata_object, source_metadata_ptr->getPartitionKeyAST()); + } +#else + (void)iceberg_metadata_json_; + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Data lake export requires Avro support"); +#endif + } + const auto & source_columns = source_metadata_ptr->getColumns(); const auto & destination_columns = destination_metadata_ptr->getColumns(); @@ -6697,8 +6751,13 @@ void MergeTreeData::exportPartToTable( if (source_columns.getReadable().sizeOfDifference(destination_columns.getInsertable())) throw Exception(ErrorCodes::INCOMPATIBLE_COLUMNS, "Tables have different structure"); - if (query_to_string(source_metadata_ptr->getPartitionKeyAST()) != query_to_string(destination_metadata_ptr->getPartitionKeyAST())) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Tables have different partition key"); + /// for data lakes this check is performed differently. It is a bit more complex as we need to convert the iceberg partition spec + /// to the MergeTree partition spec and compare the two. + if (!dest_storage->isDataLake()) + { + if (query_to_string(source_metadata_ptr->getPartitionKeyAST()) != query_to_string(destination_metadata_ptr->getPartitionKeyAST())) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Tables have different partition key"); + } auto part = getPartIfExists(part_name, {MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated}); @@ -6752,6 +6811,7 @@ void MergeTreeData::exportPartToTable( query_context->getSettingsRef()[Setting::export_merge_tree_part_file_already_exists_policy].value, query_context->getSettingsCopy(), source_metadata_ptr, + iceberg_metadata_json, completion_callback); std::lock_guard lock(export_manifests_mutex); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index b63a78c8d83c..4b923d1f26cc 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -1051,6 +1051,7 @@ class MergeTreeData : public WithMutableContext, public IStorage, public IBackgr const StoragePtr & destination_storage, const String & transaction_id, ContextPtr query_context, + const std::optional & iceberg_metadata_json = std::nullopt, bool allow_outdated_parts = false, std::function completion_callback = {}); @@ -1059,6 +1060,7 @@ class MergeTreeData : public WithMutableContext, public IStorage, public IBackgr const StorageID & destination_storage_id, const String & transaction_id, ContextPtr query_context, + const std::optional & iceberg_metadata_json = std::nullopt, bool allow_outdated_parts = false, std::function completion_callback = {}); diff --git a/src/Storages/MergeTree/MergeTreePartExportManifest.h b/src/Storages/MergeTree/MergeTreePartExportManifest.h index e65b4172c78d..08d73febf968 100644 --- a/src/Storages/MergeTree/MergeTreePartExportManifest.h +++ b/src/Storages/MergeTree/MergeTreePartExportManifest.h @@ -51,6 +51,7 @@ struct MergeTreePartExportManifest FileAlreadyExistsPolicy file_already_exists_policy_, const Settings & settings_, const StorageMetadataPtr & metadata_snapshot_, + const String & iceberg_metadata_json_, std::function completion_callback_ = {}) : destination_storage_ptr(destination_storage_ptr_), data_part(data_part_), @@ -59,6 +60,7 @@ struct MergeTreePartExportManifest file_already_exists_policy(file_already_exists_policy_), settings(settings_), metadata_snapshot(metadata_snapshot_), + iceberg_metadata_json(iceberg_metadata_json_), completion_callback(completion_callback_), create_time(time(nullptr)) {} @@ -74,6 +76,8 @@ struct MergeTreePartExportManifest /// Otherwise the export could fail if the schema changes between validation and execution StorageMetadataPtr metadata_snapshot; + String iceberg_metadata_json; + std::function completion_callback; time_t create_time; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index 8bef554cf999..c45d7ca0eb53 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -36,7 +36,7 @@ namespace MergeTreeSetting namespace ServerSetting { - extern const ServerSettingsBool enable_experimental_export_merge_tree_partition_feature; + extern const ServerSettingsBool allow_experimental_export_merge_tree_partition; } namespace ErrorCodes @@ -185,7 +185,7 @@ bool ReplicatedMergeTreeRestartingThread::runImpl() storage.mutations_finalizing_task->activateAndSchedule(); storage.merge_selecting_task->activateAndSchedule(); - if (storage.getContext()->getServerSettings()[ServerSetting::enable_experimental_export_merge_tree_partition_feature]) + if (storage.getContext()->getServerSettings()[ServerSetting::allow_experimental_export_merge_tree_partition]) { storage.export_merge_tree_partition_updating_task->activateAndSchedule(); storage.export_merge_tree_partition_select_task->activateAndSchedule(); diff --git a/src/Storages/ObjectStorage/DataLakes/IDataLakeMetadata.h b/src/Storages/ObjectStorage/DataLakes/IDataLakeMetadata.h index c7421143a7cc..be4133db1cc5 100644 --- a/src/Storages/ObjectStorage/DataLakes/IDataLakeMetadata.h +++ b/src/Storages/ObjectStorage/DataLakes/IDataLakeMetadata.h @@ -3,8 +3,11 @@ #include #include +#include #include #include +#include +#include #include #include #include @@ -131,6 +134,37 @@ class IDataLakeMetadata : boost::noncopyable throwNotImplemented("write"); } + virtual bool supportsImport(ContextPtr) const + { + return false; + } + + virtual SinkToStoragePtr import( + std::shared_ptr /* catalog */, + const std::function & /* new_file_path_callback */, + SharedHeader /* sample_block */, + const std::string & /* iceberg_metadata_json_string */, + const std::optional & /* format_settings_ */, + ContextPtr /* context */) + { + throwNotImplemented("import"); + } + + virtual void commitExportPartitionTransaction( + std::shared_ptr /* catalog */, + const StorageID & /* table_id */, + const String & /* transaction_id */, + Int64 /* original_schema_id */, + Int64 /* partition_spec_id */, + const std::vector & /* partition_values */, + SharedHeader /* sample_block */, + const std::vector & /* data_file_paths */, + StorageObjectStorageConfigurationPtr /* configuration */, + ContextPtr /* context */) + { + throwNotImplemented("commitExportPartitionTransaction"); + } + virtual bool optimize( const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr /*context*/, const std::optional & /*format_settings*/) { diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/AvroSchema.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/AvroSchema.h index ac82b8e8efb3..0c09df4d9e77 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/AvroSchema.h +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/AvroSchema.h @@ -574,4 +574,42 @@ static constexpr const char * manifest_entry_v2_schema = R"( } )"; +/// Schema for the per-data-file sidecar Avro files written alongside every data file +/// during import/export. The sidecar carries the row count and byte size that cannot +/// be cheaply inferred from the data file itself without a full scan. +static constexpr const char * data_file_sidecar_schema = R"( +{ + "type": "record", + "name": "data_file_metadata", + "fields": [ + {"name": "record_count", "type": "long"}, + {"name": "file_size_in_bytes", "type": "long"}, + { + "name": "column_sizes", + "type": {"type": "array", "items": {"type": "record", "name": "cs_entry", + "fields": [{"name": "key", "type": "int"}, {"name": "value", "type": "long"}]}}, + "default": [] + }, + { + "name": "null_value_counts", + "type": {"type": "array", "items": {"type": "record", "name": "nvc_entry", + "fields": [{"name": "key", "type": "int"}, {"name": "value", "type": "long"}]}}, + "default": [] + }, + { + "name": "lower_bounds", + "type": {"type": "array", "items": {"type": "record", "name": "lb_entry", + "fields": [{"name": "key", "type": "int"}, {"name": "value", "type": "bytes"}]}}, + "default": [] + }, + { + "name": "upper_bounds", + "type": {"type": "array", "items": {"type": "record", "name": "ub_entry", + "fields": [{"name": "key", "type": "int"}, {"name": "value", "type": "bytes"}]}}, + "default": [] + } + ] +} +)"; + } diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/Constant.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/Constant.h index 8a23dcb77d07..6f1e455e351a 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/Constant.h +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/Constant.h @@ -135,6 +135,7 @@ DEFINE_ICEBERG_FIELD_ALIAS(max_ref_age_ms, history.expire.max-ref-age-ms); DEFINE_ICEBERG_FIELD_ALIAS(ref_min_snapshots_to_keep, min-snapshots-to-keep); DEFINE_ICEBERG_FIELD_ALIAS(ref_max_snapshot_age_ms, max-snapshot-age-ms); DEFINE_ICEBERG_FIELD_ALIAS(ref_max_ref_age_ms, max-ref-age-ms); +DEFINE_ICEBERG_FIELD_ALIAS(clickhouse_export_partition_transaction_id, clickhouse.export-partition-transaction-id); /// These are compound fields like `data_file.file_path`, we use prefix 'c_' to distinguish them. DEFINE_ICEBERG_FIELD_COMPOUND(data_file, file_path); DEFINE_ICEBERG_FIELD_COMPOUND(data_file, file_format); diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergDataFileEntry.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergDataFileEntry.h new file mode 100644 index 000000000000..61fa7be9005b --- /dev/null +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergDataFileEntry.h @@ -0,0 +1,50 @@ +#pragma once + +#include "config.h" + +#if USE_AVRO + +#include +#include +#include +#include + +namespace DB +{ + +/// Column-level statistics for a single Iceberg data file stored in Iceberg wire format. +/// Bounds are pre-serialized to bytes so the struct can be persisted to sidecar Avro files +/// and used directly at manifest-commit time without requiring the original ClickHouse +/// DataFileStatistics or a live Block schema. +struct IcebergSerializedFileStats +{ + Int64 record_count = 0; + Int64 file_size_in_bytes = 0; + + /// field_id → compressed byte size of column in the file + std::vector> column_sizes; + /// field_id → number of null values in the file + std::vector> null_value_counts; + /// field_id → Iceberg-serialized lower bound (same binary format as manifest) + std::vector>> lower_bounds; + /// field_id → Iceberg-serialized upper bound (same binary format as manifest) + std::vector>> upper_bounds; +}; + +/// One entry describing a data file that will be registered in an Iceberg manifest. +/// Carries per-file statistics so that each manifest entry gets accurate metadata +/// (column sizes, null counts, min/max bounds, record count, file size). +struct IcebergDataFileEntry +{ + String path; + Int64 record_count = 0; + Int64 file_size_in_bytes = 0; + + /// Per-file column statistics (null counts, min/max bounds, column sizes). + /// Pass std::nullopt when statistics are not available or not yet computed. + std::optional statistics; +}; + +} + +#endif diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.cpp index 12379496f211..62443624d65c 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.cpp @@ -50,6 +50,7 @@ #include #include +#include #include #include #include @@ -74,8 +75,8 @@ #include #include #include +#include #include - #include #include #include @@ -130,6 +131,8 @@ extern const SettingsBool allow_experimental_expire_snapshots; extern const SettingsBool iceberg_delete_data_on_drop; } +static constexpr size_t MAX_TRANSACTION_RETRIES = 100; + namespace { String dumpMetadataObjectToString(const Poco::JSON::Object::Ptr & metadata_object) @@ -138,6 +141,51 @@ String dumpMetadataObjectToString(const Poco::JSON::Object::Ptr & metadata_objec Poco::JSON::Stringifier::stringify(metadata_object, oss); return removeEscapedSlashes(oss.str()); } + +/// Check if a previous attempt already committed this transaction the snapshot +/// (with our transaction_id embedded in its summary) is still present in the snapshots array +/// unless an external engine ran expireSnapshots in the meantime. If found, skip re-committing. +bool isExportPartitionTransactionAlreadyCommitted(const Poco::JSON::Object::Ptr & metadata, const String & transaction_id) +{ + const auto throw_error = [&](const std::string & missing_field_name) + { + throw Exception( + ErrorCodes::ICEBERG_SPECIFICATION_VIOLATION, + "No {} found in metadata for iceberg file while trying to commit export partition transaction", + missing_field_name); + }; + + const auto snapshots = metadata->getArray(Iceberg::f_snapshots); + + if (!snapshots) + { + throw_error(Iceberg::f_snapshots); + } + + for (size_t i = 0; i < snapshots->size(); ++i) + { + const auto snap = snapshots->getObject(static_cast(i)); + const auto summary = snap->getObject(Iceberg::f_summary); + + if (!summary) + { + throw_error(Iceberg::f_summary); + } + + if (summary->has(Iceberg::f_clickhouse_export_partition_transaction_id)) + { + const auto tid = summary->getValue(Iceberg::f_clickhouse_export_partition_transaction_id); + + if (tid == transaction_id) + { + return true; + } + } + } + + return false; +} + } @@ -1334,6 +1382,511 @@ KeyDescription IcebergMetadata::getSortingKey(ContextPtr local_context, TableSta return result; } +SinkToStoragePtr IcebergMetadata::import( + std::shared_ptr catalog, + const std::function & new_file_path_callback, + SharedHeader sample_block, + const std::string & iceberg_metadata_json_string, + const std::optional & format_settings, + ContextPtr context) +{ + Poco::JSON::Parser parser; /// For some reason base/base/JSON.h can not parse this json file + Poco::Dynamic::Var json = parser.parse(iceberg_metadata_json_string); + Poco::JSON::Object::Ptr metadata_json = json.extract(); + + return std::make_shared( + catalog, persistent_components, metadata_json, object_storage, + context, format_settings, write_format, sample_block, data_lake_settings, new_file_path_callback); +} + +namespace FailPoints +{ + extern const char iceberg_writes_cleanup[]; + extern const char iceberg_writes_non_retry_cleanup[]; + extern const char iceberg_writes_post_publish_throw[]; +} + +namespace +{ + +/// Find the partition spec object with the given spec-id inside a metadata JSON document. +/// Throws BAD_ARGUMENTS if the spec is not found (indicates metadata/spec-id mismatch). +Poco::JSON::Object::Ptr lookupPartitionSpec(const Poco::JSON::Object::Ptr & meta, Int64 spec_id) +{ + auto specs = meta->getArray(Iceberg::f_partition_specs); + for (size_t i = 0; i < specs->size(); ++i) + { + auto spec = specs->getObject(static_cast(i)); + if (spec->getValue(Iceberg::f_spec_id) == spec_id) + return spec; + } + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Partition spec with id {} not found in table metadata", spec_id); +} + +Poco::JSON::Object::Ptr lookupSchema(const Poco::JSON::Object::Ptr & meta, Int64 schema_id) +{ + auto schemas = meta->getArray(Iceberg::f_schemas); + for (size_t i = 0; i < schemas->size(); ++i) + { + auto schema = schemas->getObject(static_cast(i)); + if (schema->getValue(Iceberg::f_schema_id) == schema_id) + return schema; + } + + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Schema with id {} not found in table metadata", schema_id); +} + +} + +bool IcebergMetadata::commitImportPartitionTransactionImpl( + FileNamesGenerator & filename_generator, + Poco::JSON::Object::Ptr & metadata, + Poco::JSON::Object::Ptr & partition_spec, + const String & transaction_id, + Int64 original_schema_id, + Int64 partition_spec_id, + const std::vector & partition_values, + const std::vector & partition_columns, + const std::vector & partition_types, + SharedHeader sample_block, + const std::vector & data_file_paths, + const std::vector & per_file_stats, + Int64 total_data_files, + Int64 total_rows, + Int64 total_chunks_size, + std::shared_ptr catalog, + const StorageID & table_id, + const String & blob_storage_type_name, + const String & blob_storage_namespace_name, + ContextPtr context) +{ + /// this check also exists here because the metadata might have been updated upon retry attempts. + if (isExportPartitionTransactionAlreadyCommitted(metadata, transaction_id)) + { + LOG_INFO(log, + "Export transaction {} already committed, skipping re-commit", + transaction_id); + return true; + } + + CompressionMethod metadata_compression_method = persistent_components.metadata_compression_method; + + auto [metadata_name, storage_metadata_name] = filename_generator.generateMetadataName(); + + Int64 parent_snapshot = -1; + if (metadata->has(Iceberg::f_current_snapshot_id)) + parent_snapshot = metadata->getValue(Iceberg::f_current_snapshot_id); + + auto [new_snapshot, manifest_list_name, storage_manifest_list_name] = MetadataGenerator(metadata).generateNextMetadata( + filename_generator, metadata_name, parent_snapshot, total_data_files, total_rows, total_chunks_size, total_data_files, /* added_delete_files */0, /* num_deleted_rows */0); + + /// Embed the stable transaction identifier in the snapshot summary so that a retry after crash + /// can detect the commit already happened by scanning the live snapshots array, without extra S3 + /// files. The field is a ClickHouse extension; Spark/Flink readers ignore unknown summary keys. + new_snapshot->getObject(Iceberg::f_summary)->set( + Iceberg::f_clickhouse_export_partition_transaction_id, transaction_id); + + String manifest_entry_name; + String storage_manifest_entry_name; + Int32 manifest_lengths = 0; + + /// Tracks whether the snapshot has become visible to readers. + /// For the file-based layout that happens as soon as writeMetadataFileAndVersionHint + /// succeeds; for a catalog layout it happens when catalog->updateMetadata succeeds. + /// Once published, the manifest entry / manifest list are referenced by the live + /// snapshot and must NOT be deleted by the outer failure cleanup, otherwise the + /// already-published snapshot becomes unreadable. + bool published = false; + + auto cleanup = [&](bool retry_because_of_metadata_conflict) + { + /// We can't cleanup the data files upon retry even if retry_because_of_metadata_conflict == false + /// because this replica or some other replica might attempt to commit the same transaction later + /// todo arthur: in the future, we should consider failing the entire task if retry_because_of_metadata_conflict = true + + object_storage->removeObjectIfExists(StoredObject(storage_manifest_entry_name)); + object_storage->removeObjectIfExists(StoredObject(storage_manifest_list_name)); + + if (retry_because_of_metadata_conflict) + { + MetadataFileWithInfo latest_metadata_file_info; + if (catalog && catalog->isTransactional()) + { + const auto & [namespace_name, table_name] = DataLake::parseTableName(table_id.getTableName()); + DataLake::TableMetadata table_metadata = DataLake::TableMetadata().withLocation().withDataLakeSpecificProperties(); + catalog->getTableMetadata(namespace_name, table_name, table_metadata); + + auto table_specific_properties = table_metadata.getDataLakeSpecificProperties(); + if (!table_specific_properties.has_value() || table_specific_properties->iceberg_metadata_file_location.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Catalog didn't return iceberg metadata location for table {}.{}", namespace_name, table_name); + + String metadata_path = table_metadata.getMetadataLocation(table_specific_properties->iceberg_metadata_file_location); + if (!metadata_path.starts_with(persistent_components.table_path)) + metadata_path = std::filesystem::path(persistent_components.table_path) / metadata_path; + latest_metadata_file_info = Iceberg::getMetadataFileAndVersion(metadata_path); + } + else + { + latest_metadata_file_info = getLatestOrExplicitMetadataFileAndVersion( + object_storage, + persistent_components.table_path, + data_lake_settings, + persistent_components.metadata_cache, + context, + getLogger("IcebergWrites").get(), + persistent_components.table_uuid, + true); + } + + auto [last_version, metadata_path, compression_method] = latest_metadata_file_info; + + LOG_DEBUG(log, "Rereading metadata file {} with version {}", metadata_path, last_version); + + metadata_compression_method = compression_method; + filename_generator.setVersion(last_version + 1); + + metadata = getMetadataJSONObject( + metadata_path, + object_storage, + persistent_components.metadata_cache, + context, + getLogger("IcebergMetadata"), + compression_method, + persistent_components.table_uuid); + + /// For the export path the schema and partition spec are fixed at the start of the + /// operation (saved in ZooKeeper). If either changed we must fail immediately — + /// the caller has to restart the export from scratch. + const auto new_schema_id = metadata->getValue(Iceberg::f_current_schema_id); + if (new_schema_id != original_schema_id) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "Table schema changed during export (expected schema {}, got {}). Restart the export operation.", + original_schema_id, new_schema_id); + + const Int64 new_partition_spec_id = metadata->getValue(Iceberg::f_default_spec_id); + if (new_partition_spec_id != partition_spec_id) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "Partition spec changed during export (expected spec {}, got {}). Restart the export operation.", + partition_spec_id, new_partition_spec_id); + + partition_spec = lookupPartitionSpec(metadata, partition_spec_id); + + /// partition_values, partition_columns, partition_types, and + /// data_file_paths are all fixed from the saved state — no update needed. + } + }; + + try + { + { + auto result = filename_generator.generateManifestEntryName(); + manifest_entry_name = result.path_in_metadata; + storage_manifest_entry_name = result.path_in_storage; + } + + auto buffer_manifest_entry = object_storage->writeObject( + StoredObject(storage_manifest_entry_name), WriteMode::Rewrite, std::nullopt, DBMS_DEFAULT_BUFFER_SIZE, context->getWriteSettings()); + + try + { + fiu_do_on(FailPoints::iceberg_writes_non_retry_cleanup, + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Failpoint for cleanup enabled"); + }); + + generateManifestFile( + metadata, + partition_columns, + partition_values, + partition_types, + data_file_paths, + std::nullopt, /// per_file_stats is filled, no need for the generic aggregate + sample_block, + new_snapshot, + write_format, + partition_spec, + partition_spec_id, + *buffer_manifest_entry, + Iceberg::FileContentType::DATA, + per_file_stats); + buffer_manifest_entry->finalize(); + manifest_lengths += buffer_manifest_entry->count(); + } + catch (...) + { + cleanup(false); + throw; + } + + { + auto buffer_manifest_list = object_storage->writeObject( + StoredObject(storage_manifest_list_name), WriteMode::Rewrite, std::nullopt, DBMS_DEFAULT_BUFFER_SIZE, context->getWriteSettings()); + + try + { + generateManifestList( + filename_generator, metadata, object_storage, context, {manifest_entry_name}, new_snapshot, manifest_lengths, *buffer_manifest_list, Iceberg::FileContentType::DATA, true); + buffer_manifest_list->finalize(); + } + catch (...) + { + cleanup(false); + throw; + } + } + + { + std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + Poco::JSON::Stringifier::stringify(metadata, oss, 4); + std::string json_representation = removeEscapedSlashes(oss.str()); + + LOG_DEBUG(log, "Writing new metadata file {}", storage_metadata_name); + auto hint = filename_generator.generateVersionHint(); + if (!writeMetadataFileAndVersionHint( + storage_metadata_name, + json_representation, + hint.path_in_storage, + storage_metadata_name, + object_storage, + context, + metadata_compression_method, + data_lake_settings[DataLakeStorageSetting::iceberg_use_version_hint])) + { + LOG_DEBUG(log, "Failed to write metadata {}, retrying", storage_metadata_name); + cleanup(true); + return false; + } + + LOG_DEBUG(log, "Metadata file {} written", storage_metadata_name); + + if (catalog) + { + String catalog_filename = metadata_name; + if (!catalog_filename.starts_with(blob_storage_type_name)) + catalog_filename = blob_storage_type_name + "://" + blob_storage_namespace_name + "/" + metadata_name; + + const auto & [namespace_name, table_name] = DataLake::parseTableName(table_id.getTableName()); + if (!catalog->updateMetadata(namespace_name, table_name, catalog_filename, new_snapshot)) + { + cleanup(true); + return false; + } + + /// Catalog has accepted the commit - the new snapshot is now live and references + /// storage_manifest_entry_name / storage_manifest_list_name. From here on, any + /// failure must NOT delete those files. + published = true; + } + else + { + /// File-based layout: the snapshot becomes visible via the metadata file and + /// version hint that were just written above. From here on, any failure must + /// NOT delete manifest entry / manifest list. + published = true; + } + } + + /// Fault-injection hook that simulates an exception in the trailing post-publish + /// region (e.g. failure in metadata-cache invalidation). Must be placed AFTER + /// `published = true` to exercise the exception-safety guard in the outer catch. + fiu_do_on(FailPoints::iceberg_writes_post_publish_throw, + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Failpoint iceberg_writes_post_publish_throw enabled"); + }); + + if (persistent_components.metadata_cache) + { + /// If there's an active metadata cache + /// We can't just cache 'our' written version as latest, because it could've been overwritten by a concurrent catalog update + /// This is why, we are safely invalidating the cache, and the very next reader will get the most up-to-date latest version + persistent_components.metadata_cache->remove(persistent_components.table_path); + if (persistent_components.table_uuid) + persistent_components.metadata_cache->remove(*persistent_components.table_uuid); + } + } + catch (...) + { + if (published) + { + /// Commit has already become visible to readers. The failure is in trailing + /// post-publish work (e.g. metadata-cache invalidation). Running cleanup() + /// here would delete manifest files referenced by the published snapshot + /// and corrupt it. Log and swallow - any transient state (stale cache) + /// is self-healing on subsequent reads. + tryLogCurrentException(log, + "Post-publish work failed after Iceberg snapshot was committed; " + "skipping manifest cleanup to preserve published snapshot"); + return true; + } + + LOG_ERROR(log, "Failed to commit import partition transaction: {}", getCurrentExceptionMessage(false)); + cleanup(false); + throw; + } + + return true; +} + +void IcebergMetadata::commitExportPartitionTransaction( + std::shared_ptr catalog, + const StorageID & table_id, + const String & transaction_id, + Int64 original_schema_id, + Int64 partition_spec_id, + const std::vector & partition_values, + SharedHeader sample_block, + const std::vector & data_file_paths, + StorageObjectStorageConfigurationPtr configuration, + ContextPtr context) +{ + + MetadataFileWithInfo updated_metadata_file_info = getLatestOrExplicitMetadataFileAndVersion( + object_storage, + persistent_components.table_path, + data_lake_settings, + persistent_components.metadata_cache, + context, + getLogger("IcebergMetadata").get(), + persistent_components.table_uuid, + true); + + /// Latest metadata is ALWAYS necessary to commit - but we abort in case schema or partition spec changed + Poco::JSON::Object::Ptr metadata = getMetadataJSONObject( + updated_metadata_file_info.path, + object_storage, + persistent_components.metadata_cache, + context, + getLogger("IcebergMetadata"), + updated_metadata_file_info.compression_method, + persistent_components.table_uuid); + + if (isExportPartitionTransactionAlreadyCommitted(metadata, transaction_id)) + { + LOG_INFO(log, + "Export transaction {} already committed, skipping re-commit", + transaction_id); + return; + } + + /// Fail fast if the table schema or partition spec changed between export-start and commit. + /// The exported data files and partition values were produced against the original spec; + const auto latest_schema_id = metadata->getValue(Iceberg::f_current_schema_id); + if (latest_schema_id != original_schema_id) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "Table schema changed before export could commit (expected schema {}, got {}). " + "Restart the export operation.", + original_schema_id, latest_schema_id); + + const auto latest_spec_id = metadata->getValue(Iceberg::f_default_spec_id); + if (latest_spec_id != partition_spec_id) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "Partition spec changed before export could commit (expected spec {}, got {}). " + "Restart the export operation.", + partition_spec_id, latest_spec_id); + + /// Derive partition_columns and partition_types from the schema and partition spec. + /// The IDs are validated equal above so derivation from the latest metadata yields + /// the same result as from the original ZK-pinned snapshot. + + const auto schema = lookupSchema(metadata, original_schema_id); + + auto partition_spec = lookupPartitionSpec(metadata, partition_spec_id); + + ChunkPartitioner partitioner(partition_spec->getArray(Iceberg::f_fields), schema, context, sample_block); + + const auto partition_columns = partitioner.getColumns(); + const auto partition_types = partitioner.getResultTypes(); + + const auto metadata_compression_method = persistent_components.metadata_compression_method; + auto config_path = persistent_components.table_path; + if (config_path.empty() || config_path.back() != '/') + config_path += "/"; + if (!config_path.starts_with('/')) + config_path = '/' + config_path; + + FileNamesGenerator filename_generator; + if (!context->getSettingsRef()[Setting::write_full_path_in_iceberg_metadata]) + { + filename_generator = FileNamesGenerator( + config_path, config_path, (catalog != nullptr && catalog->isTransactional()), metadata_compression_method, write_format); + } + else + { + auto bucket = metadata->getValue(Iceberg::f_location); + if (bucket.empty() || bucket.back() != '/') + bucket += "/"; + filename_generator = FileNamesGenerator( + bucket, config_path, (catalog != nullptr && catalog->isTransactional()), metadata_compression_method, write_format); + } + filename_generator.setVersion(updated_metadata_file_info.version + 1); + + /// Load per-file sidecar stats, necessary to populate the manifest file stats. + std::vector per_file_stats; + const Int64 total_data_files = static_cast(data_file_paths.size()); + Int64 total_rows = 0; + Int64 total_chunks_size = 0; + per_file_stats.reserve(data_file_paths.size()); + for (const auto & path : data_file_paths) + { + const auto sidecar_path = getIcebergExportPartSidecarStoragePath(path); + auto stats = readDataFileSidecar(sidecar_path, object_storage, context); + total_rows += stats.record_count; + total_chunks_size += stats.file_size_in_bytes; + + per_file_stats.push_back(std::move(stats)); + } + + size_t attempt = 0; + while (attempt < MAX_TRANSACTION_RETRIES) + { + if (commitImportPartitionTransactionImpl( + filename_generator, + metadata, + partition_spec, + transaction_id, + original_schema_id, + partition_spec_id, + partition_values, + partition_columns, + partition_types, + sample_block, + data_file_paths, + per_file_stats, + total_data_files, + total_rows, + total_chunks_size, + catalog, + table_id, + configuration->getTypeName(), + configuration->getNamespace(), + context)) + { + return; + } + + ++attempt; + } + + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "Failed to commit export partition transaction after {} attempts due to repeated metadata conflicts.", + attempt); +} + +Poco::JSON::Object::Ptr IcebergMetadata::getMetadataJSON(ContextPtr local_context) const +{ + auto [actual_data_snapshot, actual_table_state_snapshot] = getRelevantState(local_context); + return getMetadataJSONObject( + actual_table_state_snapshot.metadata_file_path, + object_storage, + persistent_components.metadata_cache, + local_context, + log, + persistent_components.metadata_compression_method, + persistent_components.table_uuid); +} + } #endif diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.h index 3a3031ddf6e1..664473fef031 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.h +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergMetadata.h @@ -1,4 +1,6 @@ #pragma once +#include "Storages/ObjectStorage/DataLakes/Iceberg/ChunkPartitioner.h" +#include "Storages/ObjectStorage/DataLakes/Iceberg/FileNamesGenerator.h" #include "config.h" #if USE_AVRO @@ -22,6 +24,7 @@ #include #include +#include #include #include #include @@ -106,6 +109,38 @@ class IcebergMetadata : public IDataLakeMetadata ContextPtr context, std::shared_ptr catalog) override; + bool supportsImport(ContextPtr) const override { return true; } + + SinkToStoragePtr import( + std::shared_ptr catalog, + const std::function & new_file_path_callback, + SharedHeader sample_block, + const std::string & iceberg_metadata_json_string, + const std::optional & format_settings, + ContextPtr context) override; + + /// Commit an export-partition transaction. All parameters that are saved in ZooKeeper at the + /// start of the export operation (schema_id, partition_spec_id, partition_values, + /// partition_columns, partition_types) must be provided by the caller. + /// The partition spec object is derived from the metadata using partition_spec_id. + /// If the live metadata has diverged (schema or partition spec changed) the call throws + /// immediately — the caller must restart from scratch. + /// + /// data_file_paths contains the metadata-path for each exported data file (as recorded in + /// ZooKeeper). For every path a co-located sidecar Avro file (same path, ".avro" extension) + /// must exist in the object storage; it supplies record_count and file_size_in_bytes. + void commitExportPartitionTransaction( + std::shared_ptr catalog, + const StorageID & table_id, + const String & transaction_id, + Int64 original_schema_id, + Int64 partition_spec_id, + const std::vector & partition_values, + SharedHeader sample_block, + const std::vector & data_file_paths, + StorageObjectStorageConfigurationPtr configuration, + ContextPtr context) override; + CompressionMethod getCompressionMethod() const { return persistent_components.metadata_compression_method; } bool optimize(const StorageMetadataPtr & metadata_snapshot, ContextPtr context, const std::optional & format_settings) override; @@ -147,6 +182,8 @@ class IcebergMetadata : public IDataLakeMetadata std::optional partitionKey(ContextPtr) const override; std::optional sortingKey(ContextPtr) const override; + Poco::JSON::Object::Ptr getMetadataJSON(ContextPtr local_context) const; + private: Iceberg::PersistentTableComponents initializePersistentTableComponents( StorageObjectStorageConfigurationPtr configuration, IcebergMetadataFilesCachePtr cache_ptr, ContextPtr context_); @@ -166,6 +203,28 @@ class IcebergMetadata : public IDataLakeMetadata std::optional getPartitionKey(ContextPtr local_context, Iceberg::TableStateSnapshot actual_table_state_snapshot) const; KeyDescription getSortingKey(ContextPtr local_context, Iceberg::TableStateSnapshot actual_table_state_snapshot) const; + bool commitImportPartitionTransactionImpl( + FileNamesGenerator & filename_generator, + Poco::JSON::Object::Ptr & metadata, + Poco::JSON::Object::Ptr & partition_spec, + const String & transaction_id, + Int64 original_schema_id, + Int64 partition_spec_id, + const std::vector & partition_values, + const std::vector & partition_columns, + const std::vector & partition_types, + SharedHeader sample_block, + const std::vector & data_file_paths, + const std::vector & per_file_stats, + Int64 total_data_files, + Int64 total_rows, + Int64 total_chunks_size, + std::shared_ptr catalog, + const StorageID & table_id, + const String & blob_storage_type_name, + const String & blob_storage_namespace_name, + ContextPtr context); + LoggerPtr log; const ObjectStoragePtr object_storage; DB::Iceberg::PersistentTableComponents persistent_components; diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergWrites.cpp b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergWrites.cpp index bd557f8c6197..a2c15a83d6f9 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergWrites.cpp +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergWrites.cpp @@ -196,6 +196,16 @@ bool canWriteStatistics( } +String getIcebergExportPartSidecarStoragePath(const String & data_file_storage_path) +{ + static constexpr auto postfix = "_clickhouse_export_part_sidecar.avro"; + auto dot_pos = data_file_storage_path.rfind('.'); + auto slash_pos = data_file_storage_path.rfind('/'); + if (dot_pos != String::npos && (slash_pos == String::npos || dot_pos > slash_pos)) + return data_file_storage_path.substr(0, dot_pos) + postfix; + return data_file_storage_path + postfix; +} + String removeEscapedSlashes(const String & json_str) { auto result = json_str; @@ -208,6 +218,160 @@ String removeEscapedSlashes(const String & json_str) return result; } +IcebergSerializedFileStats readDataFileSidecar( + const String & sidecar_storage_path, + const ObjectStoragePtr & object_storage, + const ContextPtr & context) +{ + auto buf = object_storage->readObject(StoredObject(sidecar_storage_path), context->getReadSettings()); + auto input_stream = std::make_unique(*buf); + avro::DataFileReader reader(std::move(input_stream)); + + avro::GenericDatum datum(reader.readerSchema()); + if (!reader.read(datum)) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Data file sidecar '{}' contains no records", + sidecar_storage_path); + + const auto & record = datum.value(); + IcebergSerializedFileStats result; + result.record_count = record.field("record_count").value(); + result.file_size_in_bytes = record.field("file_size_in_bytes").value(); + + auto read_long_map = [&](const std::string & name, std::vector> & out) + { + const auto & arr = record.field(name).value().value(); + for (const auto & item : arr) + { + const auto & r = item.value(); + out.emplace_back(r.field("key").value(), r.field("value").value()); + } + }; + + auto read_bytes_map = [&](const std::string & name, std::vector>> & out) + { + const auto & arr = record.field(name).value().value(); + for (const auto & item : arr) + { + const auto & r = item.value(); + out.emplace_back(r.field("key").value(), r.field("value").value>()); + } + }; + + read_long_map("column_sizes", result.column_sizes); + read_long_map("null_value_counts", result.null_value_counts); + read_bytes_map("lower_bounds", result.lower_bounds); + read_bytes_map("upper_bounds", result.upper_bounds); + + return result; +} + +void writeDataFileSidecar( + const String & data_file_storage_path, + const IcebergSerializedFileStats & stats, + const ObjectStoragePtr & object_storage, + const ContextPtr & context) +{ + const String sidecar_path = getIcebergExportPartSidecarStoragePath(data_file_storage_path); + auto buf = object_storage->writeObject( + StoredObject(sidecar_path), WriteMode::Rewrite, std::nullopt, DBMS_DEFAULT_BUFFER_SIZE, context->getWriteSettings()); + + { + auto schema = avro::compileJsonSchemaFromString(data_file_sidecar_schema); + auto adapter = std::make_unique(*buf); + avro::DataFileWriter writer(std::move(adapter), schema); + + avro::GenericDatum datum(schema.root()); + avro::GenericRecord & rec = datum.value(); + rec.field("record_count") = avro::GenericDatum(stats.record_count); + rec.field("file_size_in_bytes") = avro::GenericDatum(stats.file_size_in_bytes); + + auto write_long_map = [&](const std::string & name, const std::vector> & entries) + { + auto & field = rec.field(name); + auto & arr = field.value(); + auto schema_element = arr.schema()->leafAt(0); + for (const auto & [k, v] : entries) + { + avro::GenericDatum item(schema_element); + auto & item_rec = item.value(); + item_rec.field("key") = avro::GenericDatum(k); + item_rec.field("value") = avro::GenericDatum(v); + arr.value().push_back(item); + } + }; + + auto write_bytes_map = [&](const std::string & name, const std::vector>> & entries) + { + auto & field = rec.field(name); + auto & arr = field.value(); + auto schema_element = arr.schema()->leafAt(0); + for (const auto & [k, v] : entries) + { + avro::GenericDatum item(schema_element); + auto & item_rec = item.value(); + item_rec.field("key") = avro::GenericDatum(k); + item_rec.field("value") = avro::GenericDatum(v); + arr.value().push_back(item); + } + }; + + write_long_map("column_sizes", stats.column_sizes); + write_long_map("null_value_counts", stats.null_value_counts); + write_bytes_map("lower_bounds", stats.lower_bounds); + write_bytes_map("upper_bounds", stats.upper_bounds); + + writer.write(datum); + writer.flush(); + // writer destructor writes the Avro end-of-file sync marker + } + + buf->finalize(); +} + +/// vibe coded - needs extra attention +IcebergSerializedFileStats serializeDataFileStats( + const DataFileStatistics & stats, + SharedHeader sample_block, + Int64 record_count, + Int64 file_size_in_bytes) +{ + IcebergSerializedFileStats result; + result.record_count = record_count; + result.file_size_in_bytes = file_size_in_bytes; + + for (const auto & [field_id, sz] : stats.getColumnSizes()) + result.column_sizes.emplace_back(static_cast(field_id), static_cast(sz)); + + for (const auto & [field_id, cnt] : stats.getNullCounts()) + result.null_value_counts.emplace_back(static_cast(field_id), static_cast(cnt)); + + std::unordered_map field_id_to_col_idx; + { + auto field_ids = stats.getFieldIds(); + for (size_t i = 0; i < field_ids.size(); ++i) + field_id_to_col_idx[field_ids[i]] = i; + } + + auto serialize_bounds = [&](const std::vector> & bounds, + std::vector>> & out) + { + if (!canWriteStatistics(bounds, field_id_to_col_idx, sample_block)) + return; + for (const auto & [field_id, value] : bounds) + { + auto bytes = dumpFieldToBytes(value, sample_block->getDataTypes()[field_id_to_col_idx.at(field_id)]); + out.emplace_back(static_cast(field_id), std::move(bytes)); + } + }; + + serialize_bounds(stats.getLowerBounds(), result.lower_bounds); + serialize_bounds(stats.getUpperBounds(), result.upper_bounds); + + return result; +} + void extendSchemaForPartitions( String & schema, const std::vector & partition_columns, @@ -249,7 +413,8 @@ void generateManifestFile( Poco::JSON::Object::Ptr partition_spec, Int64 partition_spec_id, WriteBuffer & buf, - Iceberg::FileContentType content_type) + Iceberg::FileContentType content_type, + const std::vector & per_file_stats) { Int32 version = metadata->getValue(Iceberg::f_format_version); String schema_representation; @@ -282,8 +447,9 @@ void generateManifestFile( Poco::JSON::Stringifier::stringify(partition_spec->getArray(Iceberg::f_fields), oss_partition_spec, 4); writer.setMetadata(Iceberg::f_partition_spec, oss_partition_spec.str()); writer.setMetadata(Iceberg::f_partition_spec_id, std::to_string(partition_spec_id)); - for (const auto & data_file_name : data_file_names) + for (size_t file_idx = 0; file_idx < data_file_names.size(); ++file_idx) { + const auto & data_file_name = data_file_names[file_idx]; avro::GenericDatum manifest_datum(root_schema); avro::GenericRecord & manifest = manifest_datum.value(); @@ -325,62 +491,113 @@ void generateManifestFile( data_file.field(Iceberg::f_file_path) = avro::GenericDatum(data_file_name); data_file.field(Iceberg::f_file_format) = avro::GenericDatum(format); - if (data_file_statistics) + /// vibe coded - needs extra attention + /// Export path: per-file serialized stats override everything (record count, file size, + /// and all column statistics). Existing insert/mutation paths use the aggregate path below. + if (!per_file_stats.empty() && file_idx < per_file_stats.size()) { - auto set_fields = [&]( - const std::vector> & statistics, const std::string & field_name, U && dump_function) + const auto & pf = per_file_stats[file_idx]; + + auto write_long_map = [&](const std::vector> & entries, const String & field_name) { - auto & data_file_record = data_file.field(field_name); - data_file_record.selectBranch(1); - auto & record_values = data_file_record.value(); - auto schema_element = record_values.schema()->leafAt(0); - for (const auto & [field_id, value] : statistics) + if (entries.empty()) + return; + auto & field = data_file.field(field_name); + field.selectBranch(1); + auto & arr = field.value(); + auto schema_element = arr.schema()->leafAt(0); + for (const auto & [k, v] : entries) { - avro::GenericDatum record_datum(schema_element); - auto & record = record_datum.value(); - record.field(Iceberg::f_key) = static_cast(field_id); - record.field(Iceberg::f_value) = dump_function(field_id, value); - record_values.value().push_back(record_datum); + avro::GenericDatum item(schema_element); + auto & item_rec = item.value(); + item_rec.field(Iceberg::f_key) = avro::GenericDatum(k); + item_rec.field(Iceberg::f_value) = avro::GenericDatum(v); + arr.value().push_back(item); } }; - auto statistics = data_file_statistics->getColumnSizes(); - set_fields(statistics, Iceberg::f_column_sizes, [](size_t, size_t value) { return static_cast(value); }); - - statistics = data_file_statistics->getNullCounts(); - set_fields(statistics, Iceberg::f_null_value_counts, [](size_t, size_t value) { return static_cast(value); }); - - std::unordered_map field_id_to_column_index; - auto field_ids = data_file_statistics->getFieldIds(); - for (size_t i = 0; i < field_ids.size(); ++i) - field_id_to_column_index[field_ids[i]] = i; + auto write_bytes_map = [&](const std::vector>> & entries, const String & field_name) + { + if (entries.empty()) + return; + auto & field = data_file.field(field_name); + field.selectBranch(1); + auto & arr = field.value(); + auto schema_element = arr.schema()->leafAt(0); + for (const auto & [k, v] : entries) + { + avro::GenericDatum item(schema_element); + auto & item_rec = item.value(); + item_rec.field(Iceberg::f_key) = avro::GenericDatum(k); + item_rec.field(Iceberg::f_value) = avro::GenericDatum(v); + arr.value().push_back(item); + } + }; - auto dump_fields = [&](size_t field_id, Field value) - { return dumpFieldToBytes(value, sample_block->getDataTypes()[field_id_to_column_index.at(field_id)]); }; + write_long_map(pf.column_sizes, Iceberg::f_column_sizes); + write_long_map(pf.null_value_counts, Iceberg::f_null_value_counts); + write_bytes_map(pf.lower_bounds, Iceberg::f_lower_bounds); + write_bytes_map(pf.upper_bounds, Iceberg::f_upper_bounds); - auto lower_statistics = data_file_statistics->getLowerBounds(); - if (canWriteStatistics(lower_statistics, field_id_to_column_index, sample_block)) - set_fields(lower_statistics, Iceberg::f_lower_bounds, dump_fields); - auto upper_statistics = data_file_statistics->getUpperBounds(); - if (canWriteStatistics(upper_statistics, field_id_to_column_index, sample_block)) - set_fields(upper_statistics, Iceberg::f_upper_bounds, dump_fields); - } - auto summary = new_snapshot->getObject(Iceberg::f_summary); - if (summary->has(Iceberg::f_added_records)) - { - Int64 added_records = summary->getValue(Iceberg::f_added_records); - Int64 added_files_size = summary->getValue(Iceberg::f_added_files_size); - - data_file.field(Iceberg::f_record_count) = avro::GenericDatum(added_records); - data_file.field(Iceberg::f_file_size_in_bytes) = avro::GenericDatum(added_files_size); + data_file.field(Iceberg::f_record_count) = avro::GenericDatum(pf.record_count); + data_file.field(Iceberg::f_file_size_in_bytes) = avro::GenericDatum(pf.file_size_in_bytes); } else { - Int64 added_records = summary->getValue(Iceberg::f_added_position_deletes); - Int64 added_files_size = summary->getValue(Iceberg::f_added_files_size); + /// Regular INSERT / mutation path: aggregate column statistics applied to every file. + if (data_file_statistics) + { + auto set_fields = [&]( + const std::vector> & statistics, const std::string & field_name, U && dump_function) + { + auto & data_file_record = data_file.field(field_name); + data_file_record.selectBranch(1); + auto & record_values = data_file_record.value(); + auto schema_element = record_values.schema()->leafAt(0); + for (const auto & [field_id, value] : statistics) + { + avro::GenericDatum record_datum(schema_element); + auto & record = record_datum.value(); + record.field(Iceberg::f_key) = static_cast(field_id); + record.field(Iceberg::f_value) = dump_function(field_id, value); + record_values.value().push_back(record_datum); + } + }; + + auto statistics = data_file_statistics->getColumnSizes(); + set_fields(statistics, Iceberg::f_column_sizes, [](size_t, size_t value) { return static_cast(value); }); + + statistics = data_file_statistics->getNullCounts(); + set_fields(statistics, Iceberg::f_null_value_counts, [](size_t, size_t value) { return static_cast(value); }); + + std::unordered_map field_id_to_column_index; + auto field_ids = data_file_statistics->getFieldIds(); + for (size_t i = 0; i < field_ids.size(); ++i) + field_id_to_column_index[field_ids[i]] = i; - data_file.field(Iceberg::f_record_count) = avro::GenericDatum(added_records); - data_file.field(Iceberg::f_file_size_in_bytes) = avro::GenericDatum(added_files_size); + auto dump_fields = [&](size_t field_id, Field value) + { return dumpFieldToBytes(value, sample_block->getDataTypes()[field_id_to_column_index.at(field_id)]); }; + + auto lower_statistics = data_file_statistics->getLowerBounds(); + if (canWriteStatistics(lower_statistics, field_id_to_column_index, sample_block)) + set_fields(lower_statistics, Iceberg::f_lower_bounds, dump_fields); + auto upper_statistics = data_file_statistics->getUpperBounds(); + if (canWriteStatistics(upper_statistics, field_id_to_column_index, sample_block)) + set_fields(upper_statistics, Iceberg::f_upper_bounds, dump_fields); + } + + /// Record count and file size from the snapshot summary (aggregate for all files). + auto summary = new_snapshot->getObject(Iceberg::f_summary); + if (summary->has(Iceberg::f_added_records)) + { + data_file.field(Iceberg::f_record_count) = avro::GenericDatum(summary->getValue(Iceberg::f_added_records)); + data_file.field(Iceberg::f_file_size_in_bytes) = avro::GenericDatum(summary->getValue(Iceberg::f_added_files_size)); + } + else + { + data_file.field(Iceberg::f_record_count) = avro::GenericDatum(summary->getValue(Iceberg::f_added_position_deletes)); + data_file.field(Iceberg::f_file_size_in_bytes) = avro::GenericDatum(summary->getValue(Iceberg::f_added_files_size)); + } } avro::GenericRecord & partition_record = data_file.field("partition").value(); for (size_t i = 0; i < partition_columns.size(); ++i) @@ -1074,6 +1291,138 @@ bool IcebergStorageSink::initializeMetadata() return true; } +IcebergImportSink::IcebergImportSink( + std::shared_ptr catalog_, + const Iceberg::PersistentTableComponents & persistent_table_components_, + Poco::JSON::Object::Ptr metadata_json_, + ObjectStoragePtr object_storage_, + ContextPtr context_, + std::optional format_settings_, + const String & write_format_, + SharedHeader sample_block_, + const DataLakeStorageSettings & data_lake_settings_, + std::function new_file_path_callback_) + : SinkToStorage(sample_block_) + , catalog(catalog_) + , persistent_table_components(persistent_table_components_) + , metadata_json(metadata_json_) + , object_storage(object_storage_) + , context(context_) + , format_settings(format_settings_) + , write_format(write_format_) + , sample_block(sample_block_) + , data_lake_settings(data_lake_settings_) + , new_file_path_callback(std::move(new_file_path_callback_)) +{ + const auto current_schema_id = metadata_json->getValue(Iceberg::f_current_schema_id); + const auto schemas = metadata_json->getArray(Iceberg::f_schemas); + + for (size_t i = 0; i < schemas->size(); ++i) + { + if (schemas->getObject(static_cast(i))->getValue(Iceberg::f_schema_id) == current_schema_id) + { + current_schema = schemas->getObject(static_cast(i)); + break; + } + } + + const auto metadata_compression_method = persistent_table_components.metadata_compression_method; + + auto config_path = persistent_table_components.table_path; + if (config_path.empty() || config_path.back() != '/') + config_path += "/"; + if (!config_path.starts_with('/')) + config_path = '/' + config_path; + + if (!context_->getSettingsRef()[Setting::write_full_path_in_iceberg_metadata]) + { + filename_generator = FileNamesGenerator( + config_path, config_path, (catalog != nullptr && catalog->isTransactional()), metadata_compression_method, write_format); + } + else + { + auto bucket = metadata_json->getValue(Iceberg::f_location); + if (bucket.empty() || bucket.back() != '/') + bucket += "/"; + filename_generator = FileNamesGenerator( + bucket, config_path, (catalog != nullptr && catalog->isTransactional()), metadata_compression_method, write_format); + } + + const auto [last_version, unused_meta_path, unused_compression] = getLatestOrExplicitMetadataFileAndVersion( + object_storage, + persistent_table_components.table_path, + data_lake_settings, + persistent_table_components.metadata_cache, + context_, + getLogger("IcebergWrites").get(), + persistent_table_components.table_uuid); + (void)unused_meta_path; + (void)unused_compression; + + filename_generator.setVersion(last_version + 1); + + writer = std::make_unique( + context->getSettingsRef()[Setting::iceberg_insert_max_rows_in_data_file], + context->getSettingsRef()[Setting::iceberg_insert_max_bytes_in_data_file], + current_schema->getArray(Iceberg::f_fields), + filename_generator, + object_storage, + context, + format_settings, + write_format, + sample_block, + new_file_path_callback); +} + +void IcebergImportSink::consume(Chunk & chunk) +{ + if (isCancelled()) + return; + + writer->consume(chunk); +} + +void IcebergImportSink::onFinish() +{ + if (isCancelled()) + return; + + finalizeBuffers(); + + for (const auto & entry : writer->getDataFileEntries()) + { + IcebergSerializedFileStats serialized_stats; + if (entry.statistics) + { + serialized_stats = serializeDataFileStats(*entry.statistics, sample_block, entry.record_count, entry.file_size_in_bytes); + } + else + { + serialized_stats.record_count = entry.record_count; + serialized_stats.file_size_in_bytes = entry.file_size_in_bytes; + } + + writeDataFileSidecar(entry.path, serialized_stats, object_storage, context); + } + + releaseBuffers(); +} + +void IcebergImportSink::finalizeBuffers() +{ + writer->finalize(); +} + +void IcebergImportSink::releaseBuffers() +{ + writer->release(); +} + +void IcebergImportSink::cancelBuffers() +{ + writer->cancel(); +} + } // NOLINTEND(clang-analyzer-core.uninitialized.UndefReturn) diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergWrites.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergWrites.h index b9df5b04e912..d94761098a1b 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergWrites.h +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/IcebergWrites.h @@ -25,6 +25,7 @@ #include #include #include +#include #include #include @@ -45,6 +46,37 @@ namespace DB String removeEscapedSlashes(const String & json_str); +/// Read a data-file sidecar and return its contents in Iceberg wire format. +/// The returned struct carries the row count, byte size, and per-column statistics. +IcebergSerializedFileStats readDataFileSidecar( + const String & sidecar_storage_path, + const ObjectStoragePtr & object_storage, + const ContextPtr & context); + +/// Write a sidecar Avro file alongside a data file. +/// All six fields are written; empty stat vectors are valid when statistics are unavailable. +void writeDataFileSidecar( + const String & data_file_storage_path, + const IcebergSerializedFileStats & stats, + const ObjectStoragePtr & object_storage, + const ContextPtr & context); + +/// Convert in-memory DataFileStatistics (ClickHouse-internal) to the Iceberg wire format. +/// Bounds are serialized to bytes using the same encoding used in the manifest file, +/// so the result can be stored in sidecar Avro files and used at commit time on any node. +IcebergSerializedFileStats serializeDataFileStats( + const DataFileStatistics & stats, + SharedHeader sample_block, + Int64 record_count, + Int64 file_size_in_bytes); + +/// Generate an Iceberg manifest file for a set of data files. +/// +/// \param data_file_statistics Aggregate column statistics applied to every file (regular +/// INSERT and mutation paths). Ignored when \p per_file_stats is non-empty. +/// \param per_file_stats Per-file pre-serialized statistics (export-commit path). +/// When non-empty each entry overrides both the record count / file size AND the column +/// statistics for the corresponding file. Leave empty to preserve the existing behaviour. void generateManifestFile( Poco::JSON::Object::Ptr metadata, const std::vector & partition_columns, @@ -58,7 +90,8 @@ void generateManifestFile( Poco::JSON::Object::Ptr partition_spec, Int64 partition_spec_id, WriteBuffer & buf, - Iceberg::FileContentType content_type); + Iceberg::FileContentType content_type, + const std::vector & per_file_stats = {}); void generateManifestList( const FileNamesGenerator & filename_generator, @@ -72,6 +105,8 @@ void generateManifestList( Iceberg::FileContentType content_type, bool use_previous_snapshots = true); +std::string getIcebergExportPartSidecarStoragePath(const String & data_file_storage_path); + class IcebergStorageSink : public SinkToStorage { public: @@ -132,6 +167,49 @@ class IcebergStorageSink : public SinkToStorage }; +class IcebergImportSink : public SinkToStorage +{ +public: + IcebergImportSink( + std::shared_ptr catalog_, + const Iceberg::PersistentTableComponents & persistent_table_components_, + Poco::JSON::Object::Ptr metadata_json_, + ObjectStoragePtr object_storage_, + ContextPtr context_, + std::optional format_settings_, + const String & write_format_, + SharedHeader sample_block_, + const DataLakeStorageSettings & data_lake_settings_, + std::function new_file_path_callback_ = {}); + + ~IcebergImportSink() override = default; + + String getName() const override { return "IcebergImportSink"; } + + void consume(Chunk & chunk) override; + + void onFinish() override; + +private: + void finalizeBuffers(); + void releaseBuffers(); + void cancelBuffers(); + + std::shared_ptr catalog; + const Iceberg::PersistentTableComponents & persistent_table_components; + Poco::JSON::Object::Ptr metadata_json; + Poco::JSON::Object::Ptr current_schema; + FileNamesGenerator filename_generator; + ObjectStoragePtr object_storage; + ContextPtr context; + std::optional format_settings; + const String& write_format; + SharedHeader sample_block; + std::unique_ptr writer; + const DataLakeStorageSettings & data_lake_settings; + std::function new_file_path_callback; +}; + } #endif diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/MetadataGenerator.cpp b/src/Storages/ObjectStorage/DataLakes/Iceberg/MetadataGenerator.cpp index 96cf61255ec6..bd924af181f1 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/MetadataGenerator.cpp +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/MetadataGenerator.cpp @@ -177,7 +177,7 @@ MetadataGenerator::NextMetadataResult MetadataGenerator::generateNextMetadata( metadata_object->set(Iceberg::f_current_snapshot_id, snapshot_id); if (!metadata_object->has(Iceberg::f_refs)) - metadata_object->set(Iceberg::f_refs, new Poco::JSON::Object); + metadata_object->set(Iceberg::f_refs, Poco::JSON::Object::Ptr(new Poco::JSON::Object)); if (!metadata_object->getObject(Iceberg::f_refs)->has(Iceberg::f_main)) { diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/MultipleFileWriter.cpp b/src/Storages/ObjectStorage/DataLakes/Iceberg/MultipleFileWriter.cpp index e8226dadcd76..5b4e514536d2 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/MultipleFileWriter.cpp +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/MultipleFileWriter.cpp @@ -19,29 +19,39 @@ MultipleFileWriter::MultipleFileWriter( ContextPtr context_, const std::optional & format_settings_, const String & write_format_, - SharedHeader sample_block_) + SharedHeader sample_block_, + std::function new_file_path_callback_) : max_data_file_num_rows(max_data_file_num_rows_) , max_data_file_num_bytes(max_data_file_num_bytes_) - , stats(schema) + , aggregate_stats(schema) + , current_file_stats(schema) , filename_generator(filename_generator_) , object_storage(object_storage_) , context(context_) , format_settings(format_settings_) , write_format(std::move(write_format_)) , sample_block(sample_block_) + , schema_fields_json(schema) + , new_file_path_callback(std::move(new_file_path_callback_)) { } void MultipleFileWriter::startNewFile() { if (buffer) + { finalize(); + current_file_stats = DataFileStatistics(schema_fields_json); + } current_file_num_rows = 0; current_file_num_bytes = 0; auto filename = filename_generator.generateDataFileName(); data_file_names.push_back(filename.path_in_storage); + if (new_file_path_callback) + new_file_path_callback(filename.path_in_storage); + buffer = object_storage->writeObject( StoredObject(filename.path_in_storage), WriteMode::Rewrite, std::nullopt, DBMS_DEFAULT_BUFFER_SIZE, context->getWriteSettings()); @@ -65,7 +75,8 @@ void MultipleFileWriter::consume(const Chunk & chunk) output_format->flush(); *current_file_num_rows += chunk.getNumRows(); *current_file_num_bytes += chunk.bytes(); - stats.update(chunk); + aggregate_stats.update(chunk); + current_file_stats.update(chunk); } void MultipleFileWriter::finalize() @@ -73,7 +84,29 @@ void MultipleFileWriter::finalize() output_format->flush(); output_format->finalize(); buffer->finalize(); - total_bytes += buffer->count(); + const UInt64 file_bytes = buffer->count(); + total_bytes += file_bytes; + per_file_record_counts.push_back(static_cast(*current_file_num_rows)); + per_file_byte_sizes.push_back(static_cast(file_bytes)); + per_file_stats_list.push_back(current_file_stats); +} + +std::vector MultipleFileWriter::getDataFileEntries() const +{ + chassert(data_file_names.size() == per_file_record_counts.size()); + chassert(data_file_names.size() == per_file_stats_list.size()); + + std::vector entries; + entries.reserve(data_file_names.size()); + + for (size_t i = 0; i < data_file_names.size(); ++i) + entries.emplace_back( + data_file_names[i], + per_file_record_counts[i], + per_file_byte_sizes[i], + per_file_stats_list[i]); + + return entries; } void MultipleFileWriter::release() diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/MultipleFileWriter.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/MultipleFileWriter.h index 06b8009a2184..eba5511c8d6d 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/MultipleFileWriter.h +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/MultipleFileWriter.h @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB { @@ -21,7 +22,8 @@ class MultipleFileWriter ContextPtr context_, const std::optional & format_settings_, const String & write_format_, - SharedHeader sample_block_); + SharedHeader sample_block_, + std::function new_file_path_callback_ = {}); void consume(const Chunk & chunk); void startNewFile(); @@ -39,16 +41,25 @@ class MultipleFileWriter const DataFileStatistics & getResultStatistics() const { - return stats; + return aggregate_stats; } + /// Returns one entry per written data file, with the accurate row count, byte size, + /// and per-file column statistics collected during finalization. + /// Must be called only after finalize(). + std::vector getDataFileEntries() const; + private: UInt64 max_data_file_num_rows; UInt64 max_data_file_num_bytes; - DataFileStatistics stats; + DataFileStatistics aggregate_stats; /// accumulates across all files + DataFileStatistics current_file_stats; /// accumulates for the current file only std::optional current_file_num_rows = std::nullopt; std::optional current_file_num_bytes = std::nullopt; std::vector data_file_names; + std::vector per_file_record_counts; + std::vector per_file_byte_sizes; + std::vector per_file_stats_list; std::unique_ptr buffer; OutputFormatPtr output_format; FileNamesGenerator & filename_generator; @@ -58,6 +69,8 @@ class MultipleFileWriter const String& write_format; SharedHeader sample_block; UInt64 total_bytes = 0; + Poco::JSON::Array::Ptr schema_fields_json; + std::function new_file_path_callback; }; #endif diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/Utils.cpp b/src/Storages/ObjectStorage/DataLakes/Iceberg/Utils.cpp index 5b5d557a27c3..758c4a7aede5 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/Utils.cpp +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/Utils.cpp @@ -123,7 +123,7 @@ static bool isTemporaryMetadataFile(const String & file_name) return Poco::UUID{}.tryParse(substring); } -static Iceberg::MetadataFileWithInfo getMetadataFileAndVersion(const std::string & path) +Iceberg::MetadataFileWithInfo getMetadataFileAndVersion(const std::string & path) { String file_name = std::filesystem::path(path).filename(); if (isTemporaryMetadataFile(file_name)) @@ -450,6 +450,8 @@ std::pair getIcebergType(DataTypePtr type, Int32 & ite { switch (type->getTypeId()) { + case TypeIndex::UInt16: + case TypeIndex::Int16: case TypeIndex::UInt32: case TypeIndex::Int32: return {"int", true}; @@ -537,6 +539,7 @@ Poco::Dynamic::Var getAvroType(DataTypePtr type) { switch (type->getTypeId()) { + case TypeIndex::UInt16: case TypeIndex::UInt32: case TypeIndex::Int32: case TypeIndex::Date: diff --git a/src/Storages/ObjectStorage/DataLakes/Iceberg/Utils.h b/src/Storages/ObjectStorage/DataLakes/Iceberg/Utils.h index b3720eddc793..afb2d31f1be3 100644 --- a/src/Storages/ObjectStorage/DataLakes/Iceberg/Utils.h +++ b/src/Storages/ObjectStorage/DataLakes/Iceberg/Utils.h @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include @@ -26,6 +27,8 @@ namespace DB::Iceberg { +Iceberg::MetadataFileWithInfo getMetadataFileAndVersion(const std::string & path); + void writeMessageToFile( const String & data, const String & filename, @@ -72,6 +75,14 @@ Poco::JSON::Object::Ptr getMetadataJSONObject( std::pair getIcebergType(DataTypePtr type, Int32 & iter); Poco::Dynamic::Var getAvroType(DataTypePtr type); +/// Converts a ClickHouse PARTITION BY AST into the corresponding Iceberg partition-spec JSON object. +/// column_name_to_source_id maps each column name to the Iceberg field-id from the table schema. +/// The returned Int32 is the last partition-field-id allocated (useful for tracking the id counter). +/// Throws if the AST contains expressions that cannot be represented as Iceberg transforms. +std::pair getPartitionSpec( + ASTPtr partition_by, + const std::unordered_map & column_name_to_source_id); + /// Spec: https://iceberg.apache.org/spec/?h=metadata.json#table-metadata-fields std::pair createEmptyMetadataFile( String path_location, diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 89be0d011259..ee10c0170767 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -37,6 +37,9 @@ #include #include #include +#include +#include +#include namespace DB @@ -601,8 +604,22 @@ bool StorageObjectStorage::optimize( return configuration->optimize(metadata_snapshot, context, format_settings); } -bool StorageObjectStorage::supportsImport() const +bool StorageObjectStorage::supportsImport(ContextPtr local_context) const { + if (isDataLake()) + { + /// We did configuration->update() in constructor, + /// so in case of table function there is no need to do the same here again. + if (!is_table_function) + { + configuration->update( + object_storage, + local_context); + } + + return configuration->getExternalMetadata()->supportsImport(local_context); + } + if (!configuration->partition_strategy) return false; @@ -612,7 +629,6 @@ bool StorageObjectStorage::supportsImport() const return configuration->partition_strategy_type == PartitionStrategyFactory::StrategyType::HIVE; } - SinkToStoragePtr StorageObjectStorage::import( const std::string & file_name, Block & block_with_partition_values, @@ -620,9 +636,30 @@ SinkToStoragePtr StorageObjectStorage::import( bool overwrite_if_exists, std::size_t max_bytes_per_file, std::size_t max_rows_per_file, + const std::optional & iceberg_metadata_json_string, const std::optional & format_settings_, ContextPtr local_context) { + /// We did configuration->update() in constructor, + /// so in case of table function there is no need to do the same here again. + if (!is_table_function) + { + configuration->update( + object_storage, + local_context); + } + + if (isDataLake()) + { + return configuration->getExternalMetadata()->import( + catalog, + new_file_path_callback, + std::make_shared(getInMemoryMetadataPtr()->getSampleBlock()), + *iceberg_metadata_json_string, + format_settings_ ? format_settings_ : format_settings, + local_context); + } + std::string partition_key; if (configuration->partition_strategy) @@ -651,8 +688,49 @@ SinkToStoragePtr StorageObjectStorage::import( local_context); } -void StorageObjectStorage::commitExportPartitionTransaction(const String & transaction_id, const String & partition_id, const Strings & exported_paths, ContextPtr local_context) +void StorageObjectStorage::commitExportPartitionTransaction( + const String & transaction_id, + const String & partition_id, + const Strings & exported_paths, + const IcebergCommitExportPartitionArguments & iceberg_commit_export_partition_arguments, + ContextPtr local_context) { + /// We did configuration->update() in constructor, + /// so in case of table function there is no need to do the same here again. + if (!is_table_function) + { + configuration->update( + object_storage, + local_context); + } + + if (isDataLake()) + { + /// Parse the Iceberg metadata snapshot (stored in ZooKeeper at export-start time) only to + /// extract the schema-id and partition-spec-id that were current when the export began. + /// partition_columns and partition_types are derived inside commitExportPartitionTransaction + /// from the same JSON, so only partition_values need to be carried here. + Poco::JSON::Parser iceberg_parser; + Poco::JSON::Object::Ptr iceberg_metadata = + iceberg_parser.parse(iceberg_commit_export_partition_arguments.metadata_json_string).extract(); + + const auto original_schema_id = iceberg_metadata->getValue(Iceberg::f_current_schema_id); + const auto partition_spec_id = iceberg_metadata->getValue(Iceberg::f_default_spec_id); + + configuration->getExternalMetadata()->commitExportPartitionTransaction( + catalog, + storage_id, + transaction_id, + original_schema_id, + partition_spec_id, + iceberg_commit_export_partition_arguments.partition_values, + std::make_shared(getInMemoryMetadataPtr()->getSampleBlock()), + exported_paths, + configuration, + local_context); + return; + } + const String commit_object = configuration->getRawPath().path + "/commit_" + partition_id + "_" + transaction_id; /// if file already exists, nothing to be done diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index 8a4600d6e075..2d56fbba8e75 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -74,8 +74,7 @@ class StorageObjectStorage : public IStorage ContextPtr context, bool async_insert) override; - - bool supportsImport() const override; + bool supportsImport(ContextPtr) const override; SinkToStoragePtr import( const std::string & /* file_name */, @@ -84,6 +83,7 @@ class StorageObjectStorage : public IStorage bool /* overwrite_if_exists */, std::size_t /* max_bytes_per_file */, std::size_t /* max_rows_per_file */, + const std::optional & /* iceberg_metadata_json_string */, const std::optional & /* format_settings_ */, ContextPtr /* context */) override; @@ -91,6 +91,7 @@ class StorageObjectStorage : public IStorage const String & transaction_id, const String & partition_id, const Strings & exported_paths, + const IcebergCommitExportPartitionArguments & iceberg_commit_export_partition_arguments, ContextPtr local_context) override; void truncate( diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp index 317387394e03..1d2057c97873 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp @@ -301,5 +301,11 @@ RemoteQueryExecutor::Extension StorageObjectStorageCluster::getTaskIteratorExten return RemoteQueryExecutor::Extension{ .task_iterator = std::move(callback) }; } +IDataLakeMetadata * StorageObjectStorageCluster::getExternalMetadata(ContextPtr query_context) +{ + configuration->update(object_storage, query_context); + return configuration->getExternalMetadata(); +} + } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.h b/src/Storages/ObjectStorage/StorageObjectStorageCluster.h index 012278264531..b312d2e8e88e 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.h @@ -37,6 +37,8 @@ class StorageObjectStorageCluster : public IStorageCluster void updateExternalDynamicMetadataIfExists(ContextPtr query_context) override; + IDataLakeMetadata * getExternalMetadata(ContextPtr query_context); + private: void updateQueryToSendIfNeeded( ASTPtr & query, diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index d0d35c4ad89a..5bfafde0ac67 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -68,6 +68,9 @@ #include #include #include +#include +#include +#include #include #include #include @@ -75,6 +78,7 @@ #include #include #include +#include #include #include @@ -129,6 +133,7 @@ #include "Storages/ExportReplicatedMergeTreePartitionTaskEntry.h" #include #include +#include #include #include @@ -209,6 +214,7 @@ namespace Setting extern const SettingsBool export_merge_tree_partition_force_export; extern const SettingsUInt64 export_merge_tree_partition_max_retries; extern const SettingsUInt64 export_merge_tree_partition_manifest_ttl; + extern const SettingsUInt64 export_merge_tree_partition_task_timeout_seconds; extern const SettingsBool output_format_parallel_formatting; extern const SettingsBool output_format_parquet_parallel_encoding; extern const SettingsMaxThreads max_threads; @@ -219,6 +225,10 @@ namespace Setting extern const SettingsBool export_merge_tree_part_throw_on_pending_patch_parts; extern const SettingsBool export_merge_tree_partition_lock_inside_the_task; extern const SettingsString export_merge_tree_part_filename_pattern; + extern const SettingsBool write_full_path_in_iceberg_metadata; + extern const SettingsBool allow_insert_into_iceberg; + extern const SettingsUInt64 iceberg_insert_max_bytes_in_data_file; + extern const SettingsUInt64 iceberg_insert_max_rows_in_data_file; } @@ -334,7 +344,7 @@ namespace ErrorCodes namespace ServerSetting { - extern const ServerSettingsBool enable_experimental_export_merge_tree_partition_feature; + extern const ServerSettingsBool allow_experimental_export_merge_tree_partition; } namespace ActionLocks @@ -511,7 +521,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( /// Will be activated by restarting thread. mutations_finalizing_task->deactivate(); - if (getContext()->getServerSettings()[ServerSetting::enable_experimental_export_merge_tree_partition_feature]) + if (getContext()->getServerSettings()[ServerSetting::allow_experimental_export_merge_tree_partition]) { export_merge_tree_partition_manifest_updater = std::make_shared(*this); @@ -4563,17 +4573,26 @@ void StorageReplicatedMergeTree::mutationsFinalizingTask() } void StorageReplicatedMergeTree::exportMergeTreePartitionUpdatingTask() -{ +{ try { export_merge_tree_partition_manifest_updater->poll(); } + catch (const Coordination::Exception & e) + { + tryLogCurrentException(log, __PRETTY_FUNCTION__); + if (e.code == Coordination::Error::ZSESSIONEXPIRED) + { + LOG_DEBUG(log, "Export partition updating task: ZooKeeper session expired, waking up restarting thread"); + restarting_thread.wakeup(); + return; + } + } catch (...) { tryLogCurrentException(log, __PRETTY_FUNCTION__); } - export_merge_tree_partition_updating_task->scheduleAfter(30 * 1000); } @@ -4581,7 +4600,14 @@ void StorageReplicatedMergeTree::selectPartsToExport() { try { - export_merge_tree_partition_task_scheduler->run(); + if (parts_mover.moves_blocker.isCancelled()) + { + LOG_INFO(log, "Export partition select task: Moves are blocked, skipping"); + } + else + { + export_merge_tree_partition_task_scheduler->run(); + } } catch (...) { @@ -4597,9 +4623,25 @@ void StorageReplicatedMergeTree::exportMergeTreePartitionStatusHandlingTask() { export_merge_tree_partition_manifest_updater->handleStatusChanges(); } + catch (const Coordination::Exception & e) + { + tryLogCurrentException(log, __PRETTY_FUNCTION__); + if (e.code == Coordination::Error::ZSESSIONEXPIRED) + { + restarting_thread.wakeup(); + } + else + { + /// if an exception is thrown, we might have unprocessed status changes, so we need to schedule the task again + export_merge_tree_partition_status_handling_task->scheduleAfter(5000); + } + + return; + } catch (...) { tryLogCurrentException(log, __PRETTY_FUNCTION__); + export_merge_tree_partition_status_handling_task->scheduleAfter(5000); } } @@ -6007,7 +6049,7 @@ void StorageReplicatedMergeTree::partialShutdown() mutations_updating_task->deactivate(); mutations_finalizing_task->deactivate(); - if (getContext()->getServerSettings()[ServerSetting::enable_experimental_export_merge_tree_partition_feature]) + if (getContext()->getServerSettings()[ServerSetting::allow_experimental_export_merge_tree_partition]) { export_merge_tree_partition_updating_task->deactivate(); export_merge_tree_partition_select_task->deactivate(); @@ -8239,10 +8281,11 @@ void StorageReplicatedMergeTree::fetchPartition( void StorageReplicatedMergeTree::exportPartitionToTable(const PartitionCommand & command, ContextPtr query_context) { - if (!query_context->getServerSettings()[ServerSetting::enable_experimental_export_merge_tree_partition_feature]) + if (!query_context->getServerSettings()[ServerSetting::allow_experimental_export_merge_tree_partition]) { throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, - "Exporting merge tree partition is experimental. Set the server setting `enable_experimental_export_merge_tree_partition_feature` to enable it"); + "Exporting merge tree partition is experimental. Set the server setting `allow_experimental_export_merge_tree_partition` to enable it (on all replicas).\n" + "If you are exporting to an Apache Iceberg table, you also need to enable the setting `allow_experimental_insert_into_iceberg` on all replicas. The same goes for `allow_experimental_export_merge_tree_part`"); } const auto dest_database = query_context->resolveDatabase(command.to_database); @@ -8255,7 +8298,7 @@ void StorageReplicatedMergeTree::exportPartitionToTable(const PartitionCommand & throw Exception(ErrorCodes::BAD_ARGUMENTS, "Exporting to the same table is not allowed"); } - if (!dest_storage->supportsImport()) + if (!dest_storage->supportsImport(query_context)) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Destination storage {} does not support MergeTree parts or uses unsupported partitioning", dest_storage->getName()); auto query_to_string = [] (const ASTPtr & ast) @@ -8271,8 +8314,14 @@ void StorageReplicatedMergeTree::exportPartitionToTable(const PartitionCommand & if (src_snapshot->getColumns().getReadable().sizeOfDifference(destination_snapshot->getColumns().getInsertable())) throw Exception(ErrorCodes::INCOMPATIBLE_COLUMNS, "Tables have different structure"); - if (query_to_string(src_snapshot->getPartitionKeyAST()) != query_to_string(destination_snapshot->getPartitionKeyAST())) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Tables have different partition key"); + /// for data lakes this check is performed later. It is a bit more complex as we need to convert the iceberg partition spec + /// to the MergeTree partition spec and compare the two. + if (!dest_storage->isDataLake()) + { + if (query_to_string(src_snapshot->getPartitionKeyAST()) != query_to_string(destination_snapshot->getPartitionKeyAST())) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Tables have different partition key"); + } + zkutil::ZooKeeperPtr zookeeper = getZooKeeperAndAssertNotReadonly(); @@ -8401,6 +8450,7 @@ void StorageReplicatedMergeTree::exportPartitionToTable(const PartitionCommand & manifest.create_time = time(nullptr); manifest.max_retries = query_context->getSettingsRef()[Setting::export_merge_tree_partition_max_retries]; manifest.ttl_seconds = query_context->getSettingsRef()[Setting::export_merge_tree_partition_manifest_ttl]; + manifest.task_timeout_seconds = query_context->getSettingsRef()[Setting::export_merge_tree_partition_task_timeout_seconds]; manifest.max_threads = query_context->getSettingsRef()[Setting::max_threads]; manifest.parallel_formatting = query_context->getSettingsRef()[Setting::output_format_parallel_formatting]; manifest.parquet_parallel_encoding = query_context->getSettingsRef()[Setting::output_format_parquet_parallel_encoding]; @@ -8408,9 +8458,52 @@ void StorageReplicatedMergeTree::exportPartitionToTable(const PartitionCommand & manifest.max_rows_per_file = query_context->getSettingsRef()[Setting::export_merge_tree_part_max_rows_per_file]; manifest.lock_inside_the_task = query_context->getSettingsRef()[Setting::export_merge_tree_partition_lock_inside_the_task]; - manifest.file_already_exists_policy = query_context->getSettingsRef()[Setting::export_merge_tree_part_file_already_exists_policy].value; manifest.filename_pattern = query_context->getSettingsRef()[Setting::export_merge_tree_part_filename_pattern].value; + manifest.write_full_path_in_iceberg_metadata = query_context->getSettingsRef()[Setting::write_full_path_in_iceberg_metadata]; + + if (dest_storage->isDataLake()) + { +#if USE_AVRO + /// ever since Anton introduced swarms, object storage instances are always a StorageObjectStorageCluster + auto * object_storage = dynamic_cast(dest_storage.get()); + + /// in theory this should never happen, but just in case + if (!object_storage) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Destination storage {} is not a StorageObjectStorageCluster", dest_storage->getName()); + } + + auto * iceberg_metadata = dynamic_cast(object_storage->getExternalMetadata(query_context)); + if (!iceberg_metadata) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Destination storage {} is a data lake but not an iceberg table", dest_storage->getName()); + } + + if (!query_context->getSettingsRef()[Setting::allow_insert_into_iceberg]) + { + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "Iceberg writes are experimental. " + "To allow its usage, enable the setting allow_experimental_insert_into_iceberg (on all replicas). The same goes for `allow_experimental_export_merge_tree_partition` and `allow_experimental_export_merge_tree_part`"); + } + + const auto metadata_object = iceberg_metadata->getMetadataJSON(query_context); + + ExportPartitionUtils::verifyIcebergPartitionCompatibility( + metadata_object, src_snapshot->getPartitionKeyAST()); + + std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + oss.exceptions(std::ios::failbit); + metadata_object->stringify(oss); + manifest.iceberg_metadata_json = oss.str(); + + manifest.max_bytes_per_file = query_context->getSettingsRef()[Setting::iceberg_insert_max_bytes_in_data_file]; + manifest.max_rows_per_file = query_context->getSettingsRef()[Setting::iceberg_insert_max_rows_in_data_file]; + +#else + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Data lake export requires Avro support"); +#endif + } ops.emplace_back(zkutil::makeCreateRequest( fs::path(partition_exports_path) / "metadata.json", @@ -8462,7 +8555,18 @@ void StorageReplicatedMergeTree::exportPartitionToTable(const PartitionCommand & Coordination::Error code = zookeeper->tryMulti(ops, responses); if (code != Coordination::Error::ZOK) + { + if (code == Coordination::Error::ZNODEEXISTS + && zkutil::getFailedOpIndex(code, responses) == 0) + { + /// Lost the race on the root export node. Current code already + /// validated (exists / expired / force) — so this is *always* a race. + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Export with key {} was created concurrently by another replica. Retry if needed", + export_key); + } throw zkutil::KeeperException::fromPath(code, partition_exports_path); + } } diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index fb69bdd1e154..8ccf97c492e4 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -148,7 +148,7 @@ namespace DB namespace ServerSetting { - extern const ServerSettingsBool enable_experimental_export_merge_tree_partition_feature; + extern const ServerSettingsBool allow_experimental_export_merge_tree_partition; } void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, bool has_zookeeper) @@ -239,7 +239,7 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b attach(context, system_database, "merges", "Contains a list of merges currently executing merges of MergeTree tables and their progress. Each merge operation is represented by a single row."); attach(context, system_database, "moves", "Contains information about in-progress data part moves of MergeTree tables. Each data part movement is represented by a single row."); attach(context, system_database, "exports", "Contains a list of exports currently executing exports of MergeTree tables and their progress. Each export operation is represented by a single row."); - if (context->getServerSettings()[ServerSetting::enable_experimental_export_merge_tree_partition_feature]) + if (context->getServerSettings()[ServerSetting::allow_experimental_export_merge_tree_partition]) { attach(context, system_database, "replicated_partition_exports", "Contains a list of partition exports of ReplicatedMergeTree tables and their progress. Each export operation is represented by a single row."); } diff --git a/tests/config/config.d/allow_experimental_export_merge_tree_partition.xml b/tests/config/config.d/allow_experimental_export_merge_tree_partition.xml new file mode 100644 index 000000000000..514cd710836a --- /dev/null +++ b/tests/config/config.d/allow_experimental_export_merge_tree_partition.xml @@ -0,0 +1,3 @@ + + 1 + diff --git a/tests/config/config.d/enable_experimental_export_merge_tree_partition.xml b/tests/config/config.d/enable_experimental_export_merge_tree_partition.xml deleted file mode 100644 index 72014c9de4db..000000000000 --- a/tests/config/config.d/enable_experimental_export_merge_tree_partition.xml +++ /dev/null @@ -1,3 +0,0 @@ - - 1 - diff --git a/tests/config/install.sh b/tests/config/install.sh index f3ded9587e2a..f5b5e383ed0c 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -95,7 +95,7 @@ ln -sf $SRC_PATH/config.d/blob_storage_log.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/custom_settings_prefixes.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/database_catalog_drop_table_concurrency.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/enable_access_control_improvements.xml $DEST_SERVER_PATH/config.d/ -ln -sf $SRC_PATH/config.d/enable_experimental_export_merge_tree_partition.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/allow_experimental_export_merge_tree_partition.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/macros.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/secure_ports.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/clusters.xml $DEST_SERVER_PATH/config.d/ diff --git a/tests/integration/helpers/export_partition_helpers.py b/tests/integration/helpers/export_partition_helpers.py new file mode 100644 index 000000000000..d6bf78df0998 --- /dev/null +++ b/tests/integration/helpers/export_partition_helpers.py @@ -0,0 +1,201 @@ +""" +Shared helpers for export-partition and export-part integration tests. + +Centralises wait-polling, table creation, and partition helpers that were +previously duplicated across multiple test modules. +""" + +import time +import uuid + + +MINIO_USER = "minio" +MINIO_PASS = "ClickHouse_Minio_P@ssw0rd" + + +def wait_for_export_status( + node, + source_table, + dest_table, + partition_id, + expected_status="COMPLETED", + timeout=60, + poll_interval=0.5, +): + """Poll system.replicated_partition_exports until status matches. + + *dest_table* may be ``None`` to skip filtering by destination table + (useful for catalog-based tests where the destination is a database-qualified path). + """ + start_time = time.time() + last_status = None + while time.time() - start_time < timeout: + dest_filter = ( + f" AND destination_table = '{dest_table}'" if dest_table else "" + ) + status = node.query( + f"SELECT status FROM system.replicated_partition_exports" + f" WHERE source_table = '{source_table}'" + f"{dest_filter}" + f" AND partition_id = '{partition_id}'" + ).strip() + + last_status = status + if status and status == expected_status: + return status + + time.sleep(poll_interval) + + raise TimeoutError( + f"Export status did not reach '{expected_status}' within {timeout}s. " + f"Last status: '{last_status}'" + ) + + +def wait_for_export_to_start( + node, + source_table, + dest_table, + partition_id, + timeout=10, + poll_interval=0.2, +): + """Poll until at least one row exists in system.replicated_partition_exports.""" + start_time = time.time() + while time.time() - start_time < timeout: + count = node.query( + f"SELECT count() FROM system.replicated_partition_exports" + f" WHERE source_table = '{source_table}'" + f" AND destination_table = '{dest_table}'" + f" AND partition_id = '{partition_id}'" + ).strip() + + if count != "0": + return True + + time.sleep(poll_interval) + + raise TimeoutError( + f"Export of partition {partition_id!r} did not start within {timeout}s." + ) + + +def wait_for_exception_count( + node, + source_table, + dest_table, + partition_id, + min_exception_count=1, + timeout=30, + poll_interval=0.5, +): + """Wait for exception_count to reach at least *min_exception_count*.""" + start_time = time.time() + last_exception_count = None + while time.time() - start_time < timeout: + exception_count_str = node.query( + f"SELECT exception_count FROM system.replicated_partition_exports" + f" WHERE source_table = '{source_table}'" + f" AND destination_table = '{dest_table}'" + f" AND partition_id = '{partition_id}'" + f" SETTINGS export_merge_tree_partition_system_table_prefer_remote_information = 1" + ).strip() + + if exception_count_str: + exception_count = int(exception_count_str) + last_exception_count = exception_count + if exception_count >= min_exception_count: + return exception_count + + time.sleep(poll_interval) + + raise TimeoutError( + f"Exception count did not reach {min_exception_count} within {timeout}s. " + f"Last exception_count: {last_exception_count if last_exception_count is not None else 'N/A'}" + ) + + +# -- block-number settings are needed for patch parts support +_BLOCK_SETTINGS = ( + "enable_block_number_column = 1, enable_block_offset_column = 1" +) + + +def make_rmt( + node, + name, + columns, + partition_by, + replica_name="r1", + order_by="tuple()", +): + """Create a ReplicatedMergeTree table with block-number settings.""" + node.query( + f""" + CREATE TABLE {name} ({columns}) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/{name}', '{replica_name}') + PARTITION BY {partition_by} + ORDER BY {order_by} + SETTINGS {_BLOCK_SETTINGS} + """ + ) + + +def make_mt( + node, + name, + columns, + partition_by, + order_by="tuple()", +): + """Create a MergeTree table with block-number settings.""" + node.query( + f""" + CREATE TABLE {name} ({columns}) + ENGINE = MergeTree() + PARTITION BY {partition_by} + ORDER BY {order_by} + SETTINGS {_BLOCK_SETTINGS} + """ + ) + + +def make_iceberg_s3( + node, + name, + columns, + partition_by="", + url=None, + s3_retry_attempts=3, + if_not_exists=False, +): + """Create an IcebergS3 table at a MinIO prefix. + + *url* defaults to ``http://minio1:9001/root/data/{name}/``. + """ + if url is None: + url = f"http://minio1:9001/root/data/{name}/" + ine = "IF NOT EXISTS " if if_not_exists else "" + pclause = f"PARTITION BY {partition_by}" if partition_by else "" + node.query( + f""" + CREATE TABLE {ine}{name} ({columns}) + ENGINE = IcebergS3('{url}', '{MINIO_USER}', '{MINIO_PASS}') + {pclause} + SETTINGS s3_retry_attempts = {s3_retry_attempts} + """ + ) + + +def first_partition_id(node, table): + """Return the partition_id of the first active part of *table*.""" + return node.query( + f"SELECT partition_id FROM system.parts" + f" WHERE database = currentDatabase() AND table = '{table}' AND active" + f" ORDER BY name LIMIT 1" + ).strip() + + +def unique_suffix(): + """Return a UUID with hyphens replaced by underscores, suitable for table names.""" + return str(uuid.uuid4()).replace("-", "_") diff --git a/tests/integration/helpers/iceberg_export_stats.py b/tests/integration/helpers/iceberg_export_stats.py new file mode 100644 index 000000000000..45a997922099 --- /dev/null +++ b/tests/integration/helpers/iceberg_export_stats.py @@ -0,0 +1,179 @@ +"""Shared helpers for verifying Iceberg per-file column statistics produced by +``EXPORT PART`` / ``EXPORT PARTITION``. + +Both the MergeTree and ReplicatedMergeTree export test modules drive the same +schema and expected stats shape (see ``assert_exported_stats``), so the +assertions, the manifest-entry reader, and the small byte/int decoders live +here instead of being duplicated in each test module. + +The only ClickHouse-side prerequisite is that ``system.iceberg_metadata_log`` +is enabled on the node: point the test cluster at +``configs/config.d/metadata_log.xml`` (shipped next to each test) and run the +probing SELECT with ``SETTINGS iceberg_metadata_log_level = 'manifest_file_entry'``. +""" + +import json + +from helpers.iceberg_utils import get_bound_for_column + + +# Iceberg assigns field ids positionally (starting at 1) to the non-partition +# columns in declaration order; partition-source columns share the same ids and +# partition transform outputs live in a separate 1000+ namespace. For the +# schema used by the two export-stats tests (id Int32, name String, +# tag Nullable(String), year Int32) this yields the mapping below. +STATS_FIELD_IDS = {"id": 1, "name": 2, "tag": 3} + + +def decode_int_bound(raw): + """Decode a JSON-serialized Iceberg integer bound (little-endian signed bytes). + + ClickHouse's Iceberg writer currently dumps integer bounds using the + underlying ``Field`` storage width (8 bytes for Int32/Int64/Date/...). Some + writers produce the spec-correct 4-byte Int32 encoding. Accept both. + """ + data = raw.encode("latin-1") + assert len(data) in (4, 8), f"Unexpected bound width {len(data)}: {raw!r}" + return int.from_bytes(data, "little", signed=True) + + +def get_int_for_column(m, column_id): + """Look up a value for ``column_id`` in an Iceberg integer map serialized as + either a dict ``{str(column_id): value}`` or a list of ``{key, value}`` records. + + Unlike :func:`helpers.iceberg_utils.get_bound_for_column`, this helper does + not try to unescape the value, so it works for numeric columns like + ``column_sizes`` and ``null_value_counts`` where the raw value is an int + (or a quoted int64 string). + """ + if m is None: + return None + value = None + if isinstance(m, dict): + value = m.get(str(column_id)) + elif isinstance(m, list): + for item in m: + if isinstance(item, dict) and item.get("key") == column_id: + value = item.get("value") + break + if value is None: + return None + return int(value) if isinstance(value, str) else value + + +def fetch_manifest_entries(node, query_id): + """Read JSON manifest-file entries emitted for ``query_id`` into + ``system.iceberg_metadata_log``. + + The outer ``FORMAT JSONEachRow`` is required: the default TSV format + escapes backslashes in the ``content`` string, which would double-encode + the inner ``\\uXXXX`` sequences coming from Iceberg bytes-bounds. + """ + node.query("SYSTEM FLUSH LOGS") + raw = node.query( + f""" + SELECT DISTINCT content + FROM system.iceberg_metadata_log + WHERE query_id = '{query_id}' + AND content_type = 'ManifestFileEntry' + AND content != '' + FORMAT JSONEachRow + """ + ) + entries = [] + for line in raw.strip().split("\n"): + if not line: + continue + outer = json.loads(line) + content = outer.get("content") + if content: + entries.append(json.loads(content)) + return entries + + +def assert_exported_stats(entries): + """Assert that at least one manifest entry describes the exported 2020 data file. + + Expected shape (three rows, one NULL in ``tag``): + + * ``record_count = 3`` + * ``file_size_in_bytes > 0`` + * ``column_sizes[id|name|tag] > 0`` + * ``null_value_counts = {id: 0, name: 0, tag: 1}`` + * ``lower_bounds = {id: 1, name: "aaa", tag: "x"}`` + * ``upper_bounds = {id: 3, name: "zzz", tag: "y"}`` + """ + assert entries, "No ManifestFileEntry rows recorded in system.iceberg_metadata_log" + + id_fid = STATS_FIELD_IDS["id"] + name_fid = STATS_FIELD_IDS["name"] + tag_fid = STATS_FIELD_IDS["tag"] + + matched = False + for entry in entries: + data_file = entry.get("data_file") or {} + # Skip manifest entries that explicitly mark themselves as deletes; data + # entries either omit `content` (v1 manifest) or set it to 0. + if data_file.get("content", 0) not in (0, None): + continue + + record_count = data_file.get("record_count") + if record_count != 3: + continue + + file_size = data_file.get("file_size_in_bytes") + assert file_size and file_size > 0, ( + f"Expected positive file_size_in_bytes, got {file_size!r}" + ) + + for field in ("id", "name", "tag"): + fid = STATS_FIELD_IDS[field] + size = get_int_for_column(data_file.get("column_sizes"), fid) + assert size is not None, ( + f"column_sizes missing entry for field_id={fid} ({field})" + ) + assert size > 0, f"column_sizes[{field}] expected > 0, got {size!r}" + + null_counts = data_file.get("null_value_counts") + assert get_int_for_column(null_counts, id_fid) == 0, ( + f"Expected 0 nulls in id, got null_value_counts={null_counts!r}" + ) + assert get_int_for_column(null_counts, name_fid) == 0, ( + f"Expected 0 nulls in name, got null_value_counts={null_counts!r}" + ) + assert get_int_for_column(null_counts, tag_fid) == 1, ( + f"Expected 1 null in tag (one NULL was inserted), " + f"got null_value_counts={null_counts!r}" + ) + + lower = data_file.get("lower_bounds") + upper = data_file.get("upper_bounds") + + assert decode_int_bound(get_bound_for_column(lower, id_fid)) == 1, ( + f"lower_bounds[id] expected 1, got {get_bound_for_column(lower, id_fid)!r}" + ) + assert decode_int_bound(get_bound_for_column(upper, id_fid)) == 3, ( + f"upper_bounds[id] expected 3, got {get_bound_for_column(upper, id_fid)!r}" + ) + + assert get_bound_for_column(lower, name_fid) == "aaa", ( + f"lower_bounds[name] expected 'aaa', got {get_bound_for_column(lower, name_fid)!r}" + ) + assert get_bound_for_column(upper, name_fid) == "zzz", ( + f"upper_bounds[name] expected 'zzz', got {get_bound_for_column(upper, name_fid)!r}" + ) + + assert get_bound_for_column(lower, tag_fid) == "x", ( + f"lower_bounds[tag] expected 'x' (nulls are skipped), got {get_bound_for_column(lower, tag_fid)!r}" + ) + assert get_bound_for_column(upper, tag_fid) == "y", ( + f"upper_bounds[tag] expected 'y' (nulls are skipped), got {get_bound_for_column(upper, tag_fid)!r}" + ) + + matched = True + break + + assert matched, ( + f"No data-file manifest entry with record_count=3 was found. " + f"Parsed {len(entries)} entr(y|ies) but none matched." + ) diff --git a/tests/integration/test_export_merge_tree_part_to_iceberg/configs/config.d/metadata_log.xml b/tests/integration/test_export_merge_tree_part_to_iceberg/configs/config.d/metadata_log.xml new file mode 100644 index 000000000000..c1fece21745c --- /dev/null +++ b/tests/integration/test_export_merge_tree_part_to_iceberg/configs/config.d/metadata_log.xml @@ -0,0 +1,7 @@ + + + system + iceberg_metadata_log
+ 10 +
+
diff --git a/tests/integration/test_export_merge_tree_part_to_iceberg/test.py b/tests/integration/test_export_merge_tree_part_to_iceberg/test.py new file mode 100644 index 000000000000..b371727a08cd --- /dev/null +++ b/tests/integration/test_export_merge_tree_part_to_iceberg/test.py @@ -0,0 +1,481 @@ +""" +Integration tests for EXPORT PART to an IcebergS3 destination. + +These tests cover the data-movement path from a plain MergeTree table to an +IcebergS3 table using the single-part export operation: + + ALTER TABLE EXPORT PART '' TO TABLE + +Coverage: + test_export_part_basic_to_iceberg – simple (id, year) schema; data + part_log checks + test_export_part_all_iceberg_types – schema covering all major Iceberg data types + test_export_multiple_parts_to_iceberg – two parts from different partitions land together + test_export_part_with_year_transform_partition – toYearNumSinceEpoch() partition expression + test_export_part_with_bucket_partition – icebergBucket(N, col) partition expression + test_export_part_partition_key_mismatch_is_rejected – mismatched partition spec rejected synchronously +""" + +import logging +import time + +import pytest + +from helpers.cluster import ClickHouseCluster +from helpers.export_partition_helpers import ( + first_partition_id, + make_iceberg_s3, + make_mt, + unique_suffix, +) +from helpers.iceberg_export_stats import ( + assert_exported_stats, + fetch_manifest_entries, +) + + +# --------------------------------------------------------------------------- +# Cluster fixture +# --------------------------------------------------------------------------- + + +@pytest.fixture(scope="module") +def cluster(): + try: + cluster = ClickHouseCluster(__file__) + cluster.add_instance( + "node1", + main_configs=["configs/config.d/metadata_log.xml"], + with_minio=True, + ) + logging.info("Starting cluster...") + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +# --------------------------------------------------------------------------- +# Helpers +# --------------------------------------------------------------------------- + + +def get_part(node, table: str, partition_id: str) -> str: + """Return the name of the first active part of *table* in *partition_id*.""" + return node.query( + f"SELECT name FROM system.parts " + f"WHERE database = currentDatabase() AND table = '{table}' " + f"AND partition_id = '{partition_id}' AND active " + f"ORDER BY name LIMIT 1" + ).strip() + + +def export_part(node, table: str, part: str, dest: str) -> None: + node.query( + f"ALTER TABLE {table} EXPORT PART '{part}' TO TABLE {dest} " + f"SETTINGS allow_experimental_export_merge_tree_part = 1, " + f"allow_experimental_insert_into_iceberg = 1" + ) + + +def wait_for_export_part( + node, + table: str, + part: str, + timeout: int = 60, + poll_interval: float = 0.5, +) -> None: + """Poll system.part_log until an ExportPart event appears for *part*.""" + deadline = time.time() + timeout + while time.time() < deadline: + node.query("SYSTEM FLUSH LOGS") + count = node.query( + f"SELECT count() FROM system.part_log " + f"WHERE event_type = 'ExportPart' " + f"AND database = currentDatabase() " + f"AND table = '{table}' " + f"AND part_name = '{part}'" + ).strip() + if count != "0": + return + time.sleep(poll_interval) + raise TimeoutError( + f"ExportPart event for part {part!r} in table {table!r} " + f"did not appear in system.part_log within {timeout}s" + ) + + +def assert_part_log(node, table: str, part: str) -> None: + """Assert that system.part_log contains at least one ExportPart entry.""" + log_count = int( + node.query( + f"SELECT count() FROM system.part_log " + f"WHERE event_type = 'ExportPart' " + f"AND database = currentDatabase() " + f"AND table = '{table}' " + f"AND part_name = '{part}'" + ).strip() + ) + assert log_count >= 1, ( + f"Expected at least one ExportPart entry in system.part_log " + f"for part {part!r} in table {table!r}, found {log_count}" + ) + + +# --------------------------------------------------------------------------- +# Tests +# --------------------------------------------------------------------------- + + +def test_export_part_basic_to_iceberg(cluster): + """ + Basic happy path: export a single MergeTree part to an IcebergS3 table and + verify the row count, the content, and the system.part_log ExportPart entry. + """ + node = cluster.instances["node1"] + sfx = unique_suffix() + mt = f"mt_basic_{sfx}" + iceberg = f"iceberg_basic_{sfx}" + + make_mt(node, mt, "id Int32, year Int32", "year") + make_iceberg_s3(node, iceberg, "id Int32, year Int32", "year") + + node.query(f"INSERT INTO {mt} VALUES (1, 2020), (2, 2020), (3, 2020), (4, 2021)") + + part_2020 = get_part(node, mt, "2020") + export_part(node, mt, part_2020, iceberg) + wait_for_export_part(node, mt, part_2020) + + count = int(node.query(f"SELECT count() FROM {iceberg}").strip()) + assert count == 3, f"Expected 3 rows in Iceberg table after export, got {count}" + + result = node.query(f"SELECT id, year FROM {iceberg} ORDER BY id").strip() + assert result == "1\t2020\n2\t2020\n3\t2020", f"Unexpected exported data:\n{result}" + + assert_part_log(node, mt, part_2020) + + node.query(f"DROP TABLE IF EXISTS {mt} SYNC") + node.query(f"DROP TABLE IF EXISTS {iceberg}") + + +def test_export_part_all_iceberg_types(cluster): + """ + Export a part whose schema covers every ClickHouse type that getIcebergType() + in Utils.cpp maps to an Iceberg primitive (see the switch statement): + + Iceberg type ClickHouse column + ------------- -------------------------- + int id Int32 + long big_val Int64 + float f32 Float32 + double f64 Float64 + date event_dt Date + timestamp ts DateTime64(6) (DateTime / DateTime64 both → "timestamp") + string name String + uuid uid_val UUID + + Types not in the switch (Bool/UInt8, FixedString, Decimal) are intentionally + excluded — they throw BAD_ARGUMENTS from getIcebergType(). + + Verifies that every column round-trips correctly through the Iceberg layer + and that system.part_log records the ExportPart event. + """ + node = cluster.instances["node1"] + sfx = unique_suffix() + mt = f"mt_types_{sfx}" + iceberg = f"iceberg_types_{sfx}" + + columns = ( + "id Int32, " + "big_val Int64, " + "f32 Float32, " + "f64 Float64, " + "event_dt Date, " + "ts DateTime64(6), " + "name String, " + "uid_val UUID, " + "year Int32" + ) + + make_mt(node, mt, columns, "year", order_by="id") + make_iceberg_s3(node, iceberg, columns, "year") + + node.query( + f""" + INSERT INTO {mt} (id, big_val, f32, f64, event_dt, ts, name, uid_val, year) + VALUES ( + 1, + 9999999999999, + 3.14, + 2.718281828459045, + '2024-01-15', + '2024-01-15 12:30:45.123456', + 'hello iceberg', + '550e8400-e29b-41d4-a716-446655440000', + 2024 + ) + """ + ) + + part = get_part(node, mt, "2024") + export_part(node, mt, part, iceberg) + wait_for_export_part(node, mt, part) + + count = int(node.query(f"SELECT count() FROM {iceberg}").strip()) + assert count == 1, f"Expected 1 row in Iceberg table, got {count}" + + row = node.query( + f"SELECT id, big_val, name, year FROM {iceberg}" + ).strip() + assert "1" in row, f"id column missing/wrong: {row}" + assert "9999999999999" in row, f"big_val column missing/wrong: {row}" + assert "hello iceberg" in row, f"name column missing/wrong: {row}" + assert "2024" in row, f"year column missing/wrong: {row}" + + # Verify date round-trip + date_result = node.query(f"SELECT toString(event_dt) FROM {iceberg}").strip() + assert date_result == "2024-01-15", f"Date round-trip failed: {date_result!r}" + + # Verify timestamp round-trip (date component is sufficient; exact time format varies) + ts_result = node.query(f"SELECT ts FROM {iceberg}").strip() + assert "2024-01-15" in ts_result, f"Timestamp date component missing: {ts_result!r}" + + # Verify UUID round-trip + uid_result = node.query(f"SELECT toString(uid_val) FROM {iceberg}").strip() + assert uid_result == "550e8400-e29b-41d4-a716-446655440000", ( + f"UUID round-trip failed: {uid_result!r}" + ) + + assert_part_log(node, mt, part) + + node.query(f"DROP TABLE IF EXISTS {mt} SYNC") + node.query(f"DROP TABLE IF EXISTS {iceberg}") + + +def test_export_multiple_parts_to_iceberg(cluster): + """ + Export parts from two different partitions to the same Iceberg table and + verify that both land correctly without overwriting each other. + system.part_log must contain one ExportPart entry per exported part. + """ + node = cluster.instances["node1"] + sfx = unique_suffix() + mt = f"mt_multi_{sfx}" + iceberg = f"iceberg_multi_{sfx}" + + make_mt(node, mt, "id Int32, year Int32", "year") + make_iceberg_s3(node, iceberg, "id Int32, year Int32", "year") + + # Each INSERT creates a separate part per partition + node.query(f"INSERT INTO {mt} VALUES (1, 2020), (2, 2020)") + node.query(f"INSERT INTO {mt} VALUES (10, 2021), (11, 2021), (12, 2021)") + + part_2020 = get_part(node, mt, "2020") + part_2021 = get_part(node, mt, "2021") + + export_part(node, mt, part_2020, iceberg) + export_part(node, mt, part_2021, iceberg) + + wait_for_export_part(node, mt, part_2020) + wait_for_export_part(node, mt, part_2021) + + total = int(node.query(f"SELECT count() FROM {iceberg}").strip()) + assert total == 5, f"Expected 5 rows total (2+3), got {total}" + + count_2020 = int(node.query(f"SELECT count() FROM {iceberg} WHERE year = 2020").strip()) + count_2021 = int(node.query(f"SELECT count() FROM {iceberg} WHERE year = 2021").strip()) + assert count_2020 == 2, f"Expected 2 rows for year=2020, got {count_2020}" + assert count_2021 == 3, f"Expected 3 rows for year=2021, got {count_2021}" + + result_2020 = node.query( + f"SELECT id FROM {iceberg} WHERE year = 2020 ORDER BY id" + ).strip() + assert result_2020 == "1\n2", f"Unexpected 2020 rows: {result_2020}" + + result_2021 = node.query( + f"SELECT id FROM {iceberg} WHERE year = 2021 ORDER BY id" + ).strip() + assert result_2021 == "10\n11\n12", f"Unexpected 2021 rows: {result_2021}" + + assert_part_log(node, mt, part_2020) + assert_part_log(node, mt, part_2021) + + node.query(f"DROP TABLE IF EXISTS {mt} SYNC") + node.query(f"DROP TABLE IF EXISTS {iceberg}") + + +def test_export_part_with_year_transform_partition(cluster): + """ + Export a part from a MergeTree table partitioned by toYearNumSinceEpoch(event_date) + to an Iceberg table with the matching year-transform spec. + + Verifies that the Iceberg year-transform partition expression is accepted + and that all rows survive the round-trip intact. + """ + node = cluster.instances["node1"] + sfx = unique_suffix() + mt = f"mt_year_tf_{sfx}" + iceberg = f"iceberg_year_tf_{sfx}" + + cols = "id Int64, event_date Date" + partition_by = "toYearNumSinceEpoch(event_date)" + + make_mt(node, mt, cols, partition_by, order_by="id") + make_iceberg_s3(node, iceberg, cols, partition_by) + + node.query( + f"INSERT INTO {mt} VALUES " + f"(1, '2023-03-15'), (2, '2023-11-01'), (3, '2023-06-30')" + ) + + pid = first_partition_id(node, mt) + part = get_part(node, mt, pid) + + export_part(node, mt, part, iceberg) + wait_for_export_part(node, mt, part) + + count = int(node.query(f"SELECT count() FROM {iceberg}").strip()) + assert count == 3, f"Expected 3 rows, got {count}" + + result = node.query( + f"SELECT id, toString(event_date) FROM {iceberg} ORDER BY id" + ).strip() + assert "1\t2023-03-15" in result, f"Row 1 missing or incorrect:\n{result}" + assert "2\t2023-11-01" in result, f"Row 2 missing or incorrect:\n{result}" + assert "3\t2023-06-30" in result, f"Row 3 missing or incorrect:\n{result}" + + assert_part_log(node, mt, part) + + node.query(f"DROP TABLE IF EXISTS {mt} SYNC") + node.query(f"DROP TABLE IF EXISTS {iceberg}") + + +def test_export_part_partition_key_mismatch_is_rejected(cluster): + """ + EXPORT PART must synchronously reject (BAD_ARGUMENTS) when the source + MergeTree partition key does not match the destination Iceberg partition + spec. Export does not repartition data, so the two specs must agree on + every field (same source column by Iceberg field-id and same transform, + in the same order). + + Failing case: MergeTree PARTITION BY year, Iceberg PARTITION BY id. + The part must NOT land in the Iceberg table. + """ + node = cluster.instances["node1"] + sfx = unique_suffix() + mt = f"mt_pkey_mismatch_{sfx}" + iceberg = f"iceberg_pkey_mismatch_{sfx}" + + make_mt(node, mt, "id Int32, year Int32", "year") + make_iceberg_s3(node, iceberg, "id Int32, year Int32", "id") + + node.query(f"INSERT INTO {mt} VALUES (1, 2020), (2, 2020), (3, 2020)") + + part_2020 = get_part(node, mt, "2020") + + error = node.query_and_get_error( + f"ALTER TABLE {mt} EXPORT PART '{part_2020}' TO TABLE {iceberg} " + f"SETTINGS allow_experimental_export_merge_tree_part = 1, " + f"allow_experimental_insert_into_iceberg = 1" + ) + assert "BAD_ARGUMENTS" in error, ( + f"Expected BAD_ARGUMENTS for partition key mismatch, got: {error!r}" + ) + + count = int(node.query(f"SELECT count() FROM {iceberg}").strip()) + assert count == 0, ( + f"Expected 0 rows in Iceberg table after rejected export, got {count}" + ) + + node.query(f"DROP TABLE IF EXISTS {mt} SYNC") + node.query(f"DROP TABLE IF EXISTS {iceberg}") + + +def test_export_part_with_bucket_partition(cluster): + """ + Export a part from a MergeTree table partitioned by icebergBucket(8, user_id) + to a matching Iceberg table. + + Verifies that the bucket partition expression is accepted for EXPORT PART and + that data lands correctly in the Iceberg bucket partition. + """ + node = cluster.instances["node1"] + sfx = unique_suffix() + mt = f"mt_bucket_{sfx}" + iceberg = f"iceberg_bucket_{sfx}" + + cols = "id Int64, user_id Int64, value String" + partition_by = "icebergBucket(8, user_id)" + + make_mt(node, mt, cols, partition_by) + make_iceberg_s3(node, iceberg, cols, partition_by) + + # Both rows go to the same bucket (user_id=42 → bucket 2 for N=8) + node.query(f"INSERT INTO {mt} VALUES (1, 42, 'hello'), (2, 42, 'world')") + + pid = first_partition_id(node, mt) + part = get_part(node, mt, pid) + + export_part(node, mt, part, iceberg) + wait_for_export_part(node, mt, part) + + count = int(node.query(f"SELECT count() FROM {iceberg}").strip()) + assert count == 2, f"Expected 2 rows in Iceberg table, got {count}" + + result = node.query( + f"SELECT id, user_id, value FROM {iceberg} ORDER BY id" + ).strip() + assert "1\t42\thello" in result, f"Row 1 missing or incorrect:\n{result}" + assert "2\t42\tworld" in result, f"Row 2 missing or incorrect:\n{result}" + + assert_part_log(node, mt, part) + + node.query(f"DROP TABLE IF EXISTS {mt} SYNC") + node.query(f"DROP TABLE IF EXISTS {iceberg}") + + +def test_export_part_writes_column_statistics(cluster): + """ + Export a MergeTree part that contains one NULL and verify that the resulting + Iceberg manifest entry carries accurate per-file column statistics: + record_count, file_size_in_bytes, column_sizes, null_value_counts, + and lower/upper bounds (Int32 + String + Nullable(String) mix). + """ + node = cluster.instances["node1"] + sfx = unique_suffix() + mt = f"mt_stats_{sfx}" + iceberg = f"iceberg_stats_{sfx}" + + columns = "id Int32, name String, tag Nullable(String), year Int32" + + make_mt(node, mt, columns, "year", order_by="id") + make_iceberg_s3(node, iceberg, columns, "year") + + node.query( + f""" + INSERT INTO {mt} (id, name, tag, year) VALUES + (1, 'aaa', 'x', 2020), + (2, 'mmm', NULL, 2020), + (3, 'zzz', 'y', 2020), + (4, 'kkk', 'z', 2021) + """ + ) + + part_2020 = get_part(node, mt, "2020") + export_part(node, mt, part_2020, iceberg) + wait_for_export_part(node, mt, part_2020) + + count = int(node.query(f"SELECT count() FROM {iceberg}").strip()) + assert count == 3, f"Expected 3 rows after export, got {count}" + + query_id = f"stats_part_{sfx}" + node.query( + f"SELECT * FROM {iceberg} ORDER BY id", + query_id=query_id, + settings={"iceberg_metadata_log_level": "manifest_file_entry"}, + ) + + entries = fetch_manifest_entries(node, query_id) + assert_exported_stats(entries) + + node.query(f"DROP TABLE IF EXISTS {mt} SYNC") + node.query(f"DROP TABLE IF EXISTS {iceberg}") diff --git a/tests/integration/test_export_merge_tree_part_to_object_storage/test.py b/tests/integration/test_export_merge_tree_part_to_object_storage/test.py index 8adec2908d1c..b8c15c26275f 100644 --- a/tests/integration/test_export_merge_tree_part_to_object_storage/test.py +++ b/tests/integration/test_export_merge_tree_part_to_object_storage/test.py @@ -1,11 +1,9 @@ import logging -import pytest -import random -import string import time -from typing import Optional import uuid +import pytest + from helpers.cluster import ClickHouseCluster from helpers.network import PartitionManager @@ -42,6 +40,7 @@ def create_s3_table(node, s3_table): def create_tables_and_insert_data(node, mt_table, s3_table): + # enable_block_number_column and enable_block_offset_column are needed for patch parts support node.query(f"CREATE TABLE {mt_table} (id UInt64, year UInt16) ENGINE = MergeTree() PARTITION BY year ORDER BY tuple() SETTINGS enable_block_number_column = 1, enable_block_offset_column = 1") node.query(f"INSERT INTO {mt_table} VALUES (1, 2020), (2, 2020), (3, 2020), (4, 2021)") diff --git a/tests/integration/test_export_replicated_mt_partition_to_iceberg/__init__.py b/tests/integration/test_export_replicated_mt_partition_to_iceberg/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/integration/test_export_replicated_mt_partition_to_iceberg/configs/allow_experimental_export_partition.xml b/tests/integration/test_export_replicated_mt_partition_to_iceberg/configs/allow_experimental_export_partition.xml new file mode 100644 index 000000000000..514cd710836a --- /dev/null +++ b/tests/integration/test_export_replicated_mt_partition_to_iceberg/configs/allow_experimental_export_partition.xml @@ -0,0 +1,3 @@ + + 1 + diff --git a/tests/integration/test_export_replicated_mt_partition_to_iceberg/configs/config.d/metadata_log.xml b/tests/integration/test_export_replicated_mt_partition_to_iceberg/configs/config.d/metadata_log.xml new file mode 100644 index 000000000000..c1fece21745c --- /dev/null +++ b/tests/integration/test_export_replicated_mt_partition_to_iceberg/configs/config.d/metadata_log.xml @@ -0,0 +1,7 @@ + + + system + iceberg_metadata_log
+ 10 +
+
diff --git a/tests/integration/test_export_replicated_mt_partition_to_iceberg/configs/users.d/profile.xml b/tests/integration/test_export_replicated_mt_partition_to_iceberg/configs/users.d/profile.xml new file mode 100644 index 000000000000..1b50dfbdd310 --- /dev/null +++ b/tests/integration/test_export_replicated_mt_partition_to_iceberg/configs/users.d/profile.xml @@ -0,0 +1,9 @@ + + + + 3 + 1 + + + + diff --git a/tests/integration/test_export_replicated_mt_partition_to_iceberg/test.py b/tests/integration/test_export_replicated_mt_partition_to_iceberg/test.py new file mode 100644 index 000000000000..af6ca75bafd7 --- /dev/null +++ b/tests/integration/test_export_replicated_mt_partition_to_iceberg/test.py @@ -0,0 +1,934 @@ +import io +import json +import logging +import re +import time + +import pytest +from avro.datafile import DataFileReader +from avro.io import DatumReader + +from helpers.cluster import ClickHouseCluster +from helpers.export_partition_helpers import ( + first_partition_id, + make_iceberg_s3, + make_rmt, + unique_suffix, + wait_for_export_status, + wait_for_export_to_start, +) +from helpers.iceberg_export_stats import ( + assert_exported_stats, + fetch_manifest_entries, +) +from helpers.network import PartitionManager + + +@pytest.fixture(scope="module") +def cluster(): + try: + cluster = ClickHouseCluster(__file__) + cluster.add_instance( + "replica1", + main_configs=[ + "configs/allow_experimental_export_partition.xml", + "configs/config.d/metadata_log.xml", + ], + user_configs=["configs/users.d/profile.xml"], + with_minio=True, + stay_alive=True, + with_zookeeper=True, + keeper_required_feature_flags=["multi_read"], + ) + cluster.add_instance( + "replica2", + main_configs=[ + "configs/allow_experimental_export_partition.xml", + "configs/config.d/metadata_log.xml", + ], + user_configs=["configs/users.d/profile.xml"], + with_minio=True, + stay_alive=True, + with_zookeeper=True, + keeper_required_feature_flags=["multi_read"], + ) + logging.info("Starting cluster...") + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +@pytest.fixture(autouse=True) +def drop_tables_after_test(cluster): + """Drop all tables in the default database after every test. + + Without this, ReplicatedMergeTree tables from completed tests remain alive and keep + running ZooKeeper background threads. With many tables alive simultaneously the + ZooKeeper session becomes overwhelmed and subsequent tests start seeing + operation-timeout / session-expired errors. + """ + yield + for instance_name, instance in cluster.instances.items(): + try: + tables_str = instance.query( + "SELECT name FROM system.tables WHERE database = 'default' FORMAT TabSeparated" + ).strip() + if not tables_str: + continue + for table in tables_str.split("\n"): + table = table.strip() + if table: + instance.query(f"DROP TABLE IF EXISTS default.`{table}` SYNC") + except Exception as e: + logging.warning( + f"drop_tables_after_test: cleanup failed on {instance_name}: {e}" + ) + + +# --------------------------------------------------------------------------- +# Helpers +# --------------------------------------------------------------------------- + +def create_replicated_mt(node, mt_table: str, replica_name: str): + make_rmt(node, mt_table, "id Int64, year Int32", "year", + replica_name=replica_name) + + +def create_iceberg_s3_table(node, iceberg_table: str, if_not_exists: bool = False, + s3_retry_attempts: int = 3): + """Create (or attach to an existing) IcebergS3 table at a per-test MinIO prefix.""" + make_iceberg_s3( + node, iceberg_table, "id Int64, year Int32", + partition_by="year", if_not_exists=if_not_exists, + s3_retry_attempts=s3_retry_attempts, + ) + + +def setup_tables(cluster, mt_table: str, iceberg_table: str, nodes: list | None = None, + s3_retry_attempts: int = 3): + """ + Create the ReplicatedMergeTree table on the given nodes, insert data on the first + node, wait for replication, then create the Iceberg destination table on each node. + + The Iceberg table is created on the first node (which initialises the S3 metadata). + Subsequent nodes attach to the same path with IF NOT EXISTS. + + `nodes` defaults to ["replica1", "replica2"]. + """ + if nodes is None: + nodes = ["replica1", "replica2"] + + instances = [cluster.instances[n] for n in nodes] + primary = instances[0] + + for i, instance in enumerate(instances): + create_replicated_mt(instance, mt_table, nodes[i]) + + primary.query(f"INSERT INTO {mt_table} VALUES (1, 2020), (2, 2020), (3, 2020), (4, 2021)") + for instance in instances[1:]: + instance.query(f"SYSTEM SYNC REPLICA {mt_table}") + + create_iceberg_s3_table(primary, iceberg_table, s3_retry_attempts=s3_retry_attempts) + for instance in instances[1:]: + create_iceberg_s3_table(instance, iceberg_table, if_not_exists=True, + s3_retry_attempts=s3_retry_attempts) + + +# --------------------------------------------------------------------------- +# Tests +# --------------------------------------------------------------------------- + +def test_export_partition_to_iceberg(cluster): + """ + Basic happy path: export a single partition and verify row count and content. + """ + node = cluster.instances["replica1"] + + uid = unique_suffix() + mt_table = f"mt_{uid}" + iceberg_table = f"iceberg_{uid}" + + setup_tables(cluster, mt_table, iceberg_table, nodes=["replica1"]) + + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {iceberg_table}" + ) + wait_for_export_status(node, mt_table, iceberg_table, "2020", "COMPLETED") + + count = int(node.query(f"SELECT count() FROM {iceberg_table}").strip()) + assert count == 3, f"Expected 3 rows in Iceberg table after export, got {count}" + + result = node.query(f"SELECT id, year FROM {iceberg_table} ORDER BY id").strip() + assert result == "1\t2020\n2\t2020\n3\t2020", ( + f"Unexpected data in Iceberg table:\n{result}" + ) + + +def test_export_two_partitions_to_iceberg(cluster): + """ + Export two partitions in a single ALTER TABLE statement and verify that both + land in the Iceberg table with correct row counts. + """ + node = cluster.instances["replica1"] + + uid = unique_suffix() + mt_table = f"mt_{uid}" + iceberg_table = f"iceberg_{uid}" + + setup_tables(cluster, mt_table, iceberg_table, nodes=["replica1"]) + + node.query( + f""" + ALTER TABLE {mt_table} + EXPORT PARTITION ID '2020' TO TABLE {iceberg_table}, + EXPORT PARTITION ID '2021' TO TABLE {iceberg_table} + """ + ) + + wait_for_export_status(node, mt_table, iceberg_table, "2020", "COMPLETED") + wait_for_export_status(node, mt_table, iceberg_table, "2021", "COMPLETED") + + count_2020 = int(node.query(f"SELECT count() FROM {iceberg_table} WHERE year = 2020").strip()) + count_2021 = int(node.query(f"SELECT count() FROM {iceberg_table} WHERE year = 2021").strip()) + + assert count_2020 == 3, f"Expected 3 rows for year=2020, got {count_2020}" + assert count_2021 == 1, f"Expected 1 row for year=2021, got {count_2021}" + + +def test_failure_is_logged_in_system_table(cluster): + """ + When S3 is unreachable the export must be marked FAILED in + system.replicated_partition_exports with a non-zero exception_count. + """ + node = cluster.instances["replica1"] + minio_ip = cluster.minio_ip + minio_port = cluster.minio_port + + uid = unique_suffix() + mt_table = f"mt_{uid}" + iceberg_table = f"iceberg_{uid}" + + setup_tables(cluster, mt_table, iceberg_table, nodes=["replica1"], + s3_retry_attempts=1) + + node.query(f"SYSTEM STOP MOVES {mt_table}") + + node.query(f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {iceberg_table} SETTINGS export_merge_tree_partition_max_retries = 1") + + with PartitionManager() as pm: + pm.add_rule({ + "instance": node, + "destination": node.ip_address, + "protocol": "tcp", + "source_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + }) + pm.add_rule({ + "instance": node, + "destination": minio_ip, + "protocol": "tcp", + "destination_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + }) + + node.query(f"SYSTEM START MOVES {mt_table}") + + wait_for_export_status(node, mt_table, iceberg_table, "2020", "FAILED", timeout=60) + + status = node.query( + f""" + SELECT status FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{iceberg_table}' + AND partition_id = '2020' + SETTINGS export_merge_tree_partition_system_table_prefer_remote_information = 1 + """ + ).strip() + assert status == "FAILED", f"Expected FAILED status, got: {status!r}" + + exception_count = int(node.query( + f""" + SELECT any(exception_count) FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{iceberg_table}' + AND partition_id = '2020' + SETTINGS export_merge_tree_partition_system_table_prefer_remote_information = 1 + """ + ).strip()) + assert exception_count > 0, "Expected non-zero exception_count in system.replicated_partition_exports" + + +def test_inject_short_living_failures(cluster): + """ + Transient S3 failures must not prevent the export from completing: after the + network is restored the export should retry and eventually land COMPLETED. + """ + node = cluster.instances["replica1"] + minio_ip = cluster.minio_ip + minio_port = cluster.minio_port + + uid = unique_suffix() + mt_table = f"mt_{uid}" + iceberg_table = f"iceberg_{uid}" + + setup_tables(cluster, mt_table, iceberg_table, nodes=["replica1"], + s3_retry_attempts=1) + + node.query(f"SYSTEM STOP MOVES {mt_table}") + + node.query(f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {iceberg_table} SETTINGS export_merge_tree_partition_max_retries = 100") + + with PartitionManager() as pm: + pm.add_rule({ + "instance": node, + "destination": node.ip_address, + "protocol": "tcp", + "source_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + }) + pm.add_rule({ + "instance": node, + "destination": minio_ip, + "protocol": "tcp", + "destination_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + }) + + node.query(f"SYSTEM START MOVES {mt_table}") + + # Let at least one retry happen before restoring the network. + time.sleep(15) + + wait_for_export_status(node, mt_table, iceberg_table, "2020", "COMPLETED") + + count = int(node.query(f"SELECT count() FROM {iceberg_table} WHERE year = 2020").strip()) + assert count == 3, f"Expected 3 rows after retry, got {count}" + + status = node.query( + f""" + SELECT status FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{iceberg_table}' + AND partition_id = '2020' + """ + ).strip() + assert status == "COMPLETED", f"Expected COMPLETED in system table, got: {status!r}" + + exception_count = int(node.query( + f""" + SELECT exception_count FROM system.replicated_partition_exports + WHERE source_table = '{mt_table}' + AND destination_table = '{iceberg_table}' + AND partition_id = '2020' + SETTINGS export_merge_tree_partition_system_table_prefer_remote_information = 1 + """ + ).strip()) + assert exception_count >= 1, "Expected at least one transient exception to be recorded" + + +def test_export_partition_scheduler_skipped_when_moves_stopped(cluster): + """ + Verify that selectPartsToExport() skips the scheduler entirely when moves + are stopped (moves_blocker guard at the top of the function). + + No ZK locks are acquired and no background tasks are submitted, so the + Iceberg table must remain empty across multiple scheduler cycles. Once moves + are re-enabled the export completes and rows appear in the Iceberg table. + """ + node = cluster.instances["replica1"] + + uid = unique_suffix() + mt_table = f"mt_{uid}" + iceberg_table = f"iceberg_{uid}" + + setup_tables(cluster, mt_table, iceberg_table, nodes=["replica1"]) + + node.query(f"SYSTEM STOP MOVES {mt_table}") + + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {iceberg_table}" + ) + + wait_for_export_to_start(node, mt_table, iceberg_table, "2020") + + # Wait for several scheduler cycles (each fires every 5 s). + # If the guard is absent the scheduler would run and rows would appear in the Iceberg table. + time.sleep(12) + + status = node.query( + f"SELECT status FROM system.replicated_partition_exports" + f" WHERE source_table = '{mt_table}' AND destination_table = '{iceberg_table}'" + f" AND partition_id = '2020'" + ).strip() + + assert status == "PENDING", f"Expected PENDING while moves are stopped, got '{status}'" + + count = int(node.query(f"SELECT count() FROM {iceberg_table} WHERE year = 2020").strip()) + assert count == 0, f"Expected 0 rows in Iceberg table while scheduler is skipped, got {count}" + + node.query(f"SYSTEM START MOVES {mt_table}") + + wait_for_export_status(node, mt_table, iceberg_table, "2020", "COMPLETED") + + count = int(node.query(f"SELECT count() FROM {iceberg_table} WHERE year = 2020").strip()) + assert count == 3, f"Expected 3 rows in Iceberg table after export completed, got {count}" + + +def test_export_partition_resumes_after_stop_moves(cluster): + """ + Verify that SYSTEM STOP MOVES before EXPORT PARTITION does not permanently + orphan the ZooKeeper part lock for Iceberg destinations. + + When moves are stopped the scheduler still picks parts up and submits them to + the background executor, but ExportPartTask::isCancelled() returns true (via + moves_blocker), causing QUERY_WAS_CANCELLED before any data is written. The + fix in handlePartExportFailure must release the ZK lock so the part is retried + once moves are restarted. + """ + node = cluster.instances["replica1"] + + uid = unique_suffix() + mt_table = f"mt_{uid}" + iceberg_table = f"iceberg_{uid}" + + setup_tables(cluster, mt_table, iceberg_table, nodes=["replica1"]) + + node.query(f"SYSTEM STOP MOVES {mt_table}") + + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {iceberg_table}" + f" SETTINGS export_merge_tree_partition_max_retries = 50" + ) + + wait_for_export_to_start(node, mt_table, iceberg_table, "2020") + + # Give the scheduler enough time to attempt (and cancel) the part task at least once. + time.sleep(5) + + status = node.query( + f"SELECT status FROM system.replicated_partition_exports" + f" WHERE source_table = '{mt_table}' AND destination_table = '{iceberg_table}'" + f" AND partition_id = '2020'" + ).strip() + assert status == "PENDING", f"Expected PENDING while moves are stopped, got '{status}'" + + count = int(node.query(f"SELECT count() FROM {iceberg_table} WHERE year = 2020").strip()) + assert count == 0, f"Expected 0 rows in Iceberg table while moves are stopped, got {count}" + + node.query(f"SYSTEM START MOVES {mt_table}") + + wait_for_export_status(node, mt_table, iceberg_table, "2020", "COMPLETED") + + count = int(node.query(f"SELECT count() FROM {iceberg_table} WHERE year = 2020").strip()) + assert count == 3, f"Expected 3 rows in Iceberg table after export completed, got {count}" + + +def test_export_partition_resumes_after_stop_moves_during_export(cluster): + """ + Verify that SYSTEM STOP MOVES issued while an Iceberg export is actively + retrying (S3 blocked) does not permanently orphan the ZooKeeper part lock. + """ + node = cluster.instances["replica1"] + minio_ip = cluster.minio_ip + minio_port = cluster.minio_port + + uid = unique_suffix() + mt_table = f"mt_{uid}" + iceberg_table = f"iceberg_{uid}" + + setup_tables(cluster, mt_table, iceberg_table, nodes=["replica1"]) + + node.query(f"SYSTEM STOP MOVES {mt_table}") + + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {iceberg_table}" + f" SETTINGS export_merge_tree_partition_max_retries = 50") + + wait_for_export_to_start(node, mt_table, iceberg_table, "2020") + + with PartitionManager() as pm: + pm.add_rule({ + "instance": node, + "destination": node.ip_address, + "protocol": "tcp", + "source_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + }) + pm.add_rule({ + "instance": node, + "destination": minio_ip, + "protocol": "tcp", + "destination_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + }) + + node.query(f"SYSTEM STOP MOVES {mt_table}") + + time.sleep(3) + + status = node.query( + f"SELECT status FROM system.replicated_partition_exports" + f" WHERE source_table = '{mt_table}' AND destination_table = '{iceberg_table}'" + f" AND partition_id = '2020'" + ).strip() + assert status == "PENDING", ( + f"Expected PENDING while moves are stopped and S3 is blocked, got '{status}'" + ) + + node.query(f"SYSTEM START MOVES {mt_table}") + + # MinIO is now unblocked; the next scheduler cycle should succeed. + wait_for_export_status(node, mt_table, iceberg_table, "2020", "COMPLETED") + + count = int(node.query(f"SELECT count() FROM {iceberg_table} WHERE year = 2020").strip()) + assert count == 3, f"Expected 3 rows in Iceberg table after export completed, got {count}" + + +def test_partition_transform_compatibility_accepted(cluster): + """ + Verify that EXPORT PARTITION is accepted (no BAD_ARGUMENTS) for every + supported transform when the MergeTree and Iceberg partition specs match. + + Cases covered: + 1. Compound identity (year, region) + 2. Year transform – toYearNumSinceEpoch(event_date) + 3. Month transform – toMonthNumSinceEpoch(event_date) + 4. truncate[4] – icebergTruncate(4, category) + 5. bucket[8] – icebergBucket(8, user_id) + 6. Compound mixed – (toYearNumSinceEpoch(event_date), icebergBucket(16, user_id)) + """ + node = cluster.instances["replica1"] + uid = unique_suffix() + + def check_accepted(mt, iceberg, description): + pid = first_partition_id(node, mt) + node.query( + f"ALTER TABLE {mt} EXPORT PARTITION ID '{pid}' TO TABLE {iceberg}" + ) + + # 1. Compound identity: (year, region) + cols = "id Int64, year Int32, region String" + t = f"mt_acc_1_{uid}"; i = f"iceberg_acc_1_{uid}" + make_rmt(node, t, cols, "(year, region)") + node.query(f"INSERT INTO {t} VALUES (1, 2023, 'EU')") + make_iceberg_s3(node, i, cols, "(year, region)") + check_accepted(t, i, "compound identity (year, region)") + + # 2. Year transform + cols = "id Int64, event_date Date" + t = f"mt_acc_2_{uid}"; i = f"iceberg_acc_2_{uid}" + make_rmt(node, t, cols, "toYearNumSinceEpoch(event_date)") + node.query(f"INSERT INTO {t} VALUES (1, '2020-06-15')") + make_iceberg_s3(node, i, cols, "toYearNumSinceEpoch(event_date)") + check_accepted(t, i, "year transform") + + # 3. Month transform + cols = "id Int64, event_date Date" + t = f"mt_acc_3_{uid}"; i = f"iceberg_acc_3_{uid}" + make_rmt(node, t, cols, "toMonthNumSinceEpoch(event_date)") + node.query(f"INSERT INTO {t} VALUES (1, '2020-06-15')") + make_iceberg_s3(node, i, cols, "toMonthNumSinceEpoch(event_date)") + check_accepted(t, i, "month transform") + + # 4. truncate[4] + cols = "id Int64, category String" + t = f"mt_acc_4_{uid}"; i = f"iceberg_acc_4_{uid}" + make_rmt(node, t, cols, "icebergTruncate(4, category)") + node.query(f"INSERT INTO {t} VALUES (1, 'clickhouse')") + make_iceberg_s3(node, i, cols, "icebergTruncate(4, category)") + check_accepted(t, i, "truncate[4]") + + # 5. bucket[8] + cols = "id Int64, user_id Int64" + t = f"mt_acc_5_{uid}"; i = f"iceberg_acc_5_{uid}" + make_rmt(node, t, cols, "icebergBucket(8, user_id)") + node.query(f"INSERT INTO {t} VALUES (1, 42)") + make_iceberg_s3(node, i, cols, "icebergBucket(8, user_id)") + check_accepted(t, i, "bucket[8]") + + # 6. Compound mixed: year(event_date) + bucket[16](user_id) + cols = "id Int64, event_date Date, user_id Int64" + t = f"mt_acc_6_{uid}"; i = f"iceberg_acc_6_{uid}" + make_rmt(node, t, cols, "(toYearNumSinceEpoch(event_date), icebergBucket(16, user_id))") + node.query(f"INSERT INTO {t} VALUES (1, '2021-03-01', 99)") + make_iceberg_s3(node, i, cols, "(toYearNumSinceEpoch(event_date), icebergBucket(16, user_id))") + check_accepted(t, i, "compound year+bucket[16]") + + +def test_partition_transform_compatibility_rejected(cluster): + """ + Verify that mismatched partition specs are rejected with BAD_ARGUMENTS. + + Cases covered: + 1. Compound field order reversed: MergeTree (year, region) vs Iceberg (region, year) + 2. Transform mismatch on same column: year-transform vs identity + 3. Bucket count mismatch: bucket[8] vs bucket[16] + 4. Truncate width mismatch: truncate[4] vs truncate[8] + 5. Field-count mismatch: 2-field MergeTree vs 1-field Iceberg + 6. Unsupported MergeTree expression (intDiv — not an Iceberg transform) + """ + node = cluster.instances["replica1"] + uid = unique_suffix() + + def assert_rejected(mt, iceberg, description): + # The compatibility check fires synchronously; any partition ID works here. + pid = first_partition_id(node, mt) + error = node.query_and_get_error( + f"ALTER TABLE {mt} EXPORT PARTITION ID '{pid}' TO TABLE {iceberg}" + ) + assert "BAD_ARGUMENTS" in error, ( + f"[{description}] Expected BAD_ARGUMENTS, got: {error!r}" + ) + + # 1. Compound field order reversed + cols = "id Int64, year Int32, region String" + t = f"mt_rej_1_{uid}"; i = f"iceberg_rej_1_{uid}" + make_rmt(node, t, cols, "(year, region)") + node.query(f"INSERT INTO {t} VALUES (1, 2020, 'EU')") + make_iceberg_s3(node, i, cols, "(region, year)") + assert_rejected(t, i, "compound field order reversed") + + # 2. Transform mismatch: MergeTree year-transform, Iceberg identity on same Date col + cols = "id Int64, event_date Date" + t = f"mt_rej_2_{uid}"; i = f"iceberg_rej_2_{uid}" + make_rmt(node, t, cols, "toYearNumSinceEpoch(event_date)") + node.query(f"INSERT INTO {t} VALUES (1, '2020-01-01')") + make_iceberg_s3(node, i, cols, "event_date") # identity, not year-transform + assert_rejected(t, i, "year-transform vs identity on same column") + + # 3. Bucket count mismatch: bucket[8] vs bucket[16] + cols = "id Int64, user_id Int64" + t = f"mt_rej_3_{uid}"; i = f"iceberg_rej_3_{uid}" + make_rmt(node, t, cols, "icebergBucket(8, user_id)") + node.query(f"INSERT INTO {t} VALUES (1, 42)") + make_iceberg_s3(node, i, cols, "icebergBucket(16, user_id)") + assert_rejected(t, i, "bucket[8] vs bucket[16]") + + # 4. Truncate width mismatch: truncate[4] vs truncate[8] + cols = "id Int64, category String" + t = f"mt_rej_4_{uid}"; i = f"iceberg_rej_4_{uid}" + make_rmt(node, t, cols, "icebergTruncate(4, category)") + node.query(f"INSERT INTO {t} VALUES (1, 'clickhouse')") + make_iceberg_s3(node, i, cols, "icebergTruncate(8, category)") + assert_rejected(t, i, "truncate[4] vs truncate[8]") + + # 5. Field-count mismatch: MergeTree has 2 fields, Iceberg has 1 + cols = "id Int64, year Int32, region String" + t = f"mt_rej_5_{uid}"; i = f"iceberg_rej_5_{uid}" + make_rmt(node, t, cols, "(year, region)") + node.query(f"INSERT INTO {t} VALUES (1, 2020, 'EU')") + make_iceberg_s3(node, i, cols, "year") + assert_rejected(t, i, "2-field MergeTree vs 1-field Iceberg") + + # 6. Unsupported MergeTree expression: intDiv(year, 100) is not an Iceberg transform + cols = "id Int64, year Int32" + t = f"mt_rej_6_{uid}"; i = f"iceberg_rej_6_{uid}" + make_rmt(node, t, cols, "intDiv(year, 100)") + node.query(f"INSERT INTO {t} VALUES (1, 2020)") + make_iceberg_s3(node, i, cols, "year") + assert_rejected(t, i, "unsupported MergeTree expression intDiv") + + +def test_partition_key_compatibility_check(cluster): + """ + Verify that EXPORT PARTITION throws BAD_ARGUMENTS synchronously when the + MergeTree partition key does not match the Iceberg table's partition spec, + and is accepted without error when the keys match. + + Three cases: + 1. Column mismatch – MergeTree PARTITION BY year, Iceberg PARTITION BY id + 2. Count mismatch – MergeTree PARTITION BY year, Iceberg unpartitioned + 3. Matching keys – both PARTITION BY year (must be accepted) + """ + node = cluster.instances["replica1"] + + uid = unique_suffix() + mt_table = f"mt_{uid}" + + create_replicated_mt(node, mt_table, "replica1") + node.query(f"INSERT INTO {mt_table} VALUES (1, 2020), (2, 2020), (3, 2021)") + node.query(f"SYSTEM SYNC REPLICA {mt_table}") + + # --- Case 1: Iceberg partitioned by 'id' but MergeTree by 'year' --- + iceberg_col_mismatch = f"iceberg_col_mismatch_{uid}" + node.query( + f""" + CREATE TABLE {iceberg_col_mismatch} + (id Int64, year Int32) + ENGINE = IcebergS3( + 'http://minio1:9001/root/data/{iceberg_col_mismatch}/', + 'minio', + 'ClickHouse_Minio_P@ssw0rd' + ) + PARTITION BY id SETTINGS s3_retry_attempts = 3 + """ + ) + error = node.query_and_get_error( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {iceberg_col_mismatch}" + ) + assert "BAD_ARGUMENTS" in error, ( + f"Expected BAD_ARGUMENTS for partition column mismatch, got: {error!r}" + ) + + # --- Case 2: Iceberg unpartitioned but MergeTree PARTITION BY year --- + iceberg_count_mismatch = f"iceberg_count_mismatch_{uid}" + node.query( + f""" + CREATE TABLE {iceberg_count_mismatch} + (id Int64, year Int32) + ENGINE = IcebergS3( + 'http://minio1:9001/root/data/{iceberg_count_mismatch}/', + 'minio', + 'ClickHouse_Minio_P@ssw0rd' + ) + SETTINGS s3_retry_attempts = 3 + """ + ) + error = node.query_and_get_error( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {iceberg_count_mismatch}" + ) + assert "BAD_ARGUMENTS" in error, ( + f"Expected BAD_ARGUMENTS for partition count mismatch, got: {error!r}" + ) + + # --- Case 3: Matching partition keys (both PARTITION BY year) --- + iceberg_match = f"iceberg_match_{uid}" + node.query( + f""" + CREATE TABLE {iceberg_match} + (id Int64, year Int32) + ENGINE = IcebergS3( + 'http://minio1:9001/root/data/{iceberg_match}/', + 'minio', + 'ClickHouse_Minio_P@ssw0rd' + ) + PARTITION BY year SETTINGS s3_retry_attempts = 3 + """ + ) + # Should not raise — the check passes so the export is accepted synchronously + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {iceberg_match}" + ) + + +def test_export_ttl(cluster): + """ + After a manifest TTL expires the same partition can be re-exported, and the + new data is appended to (or replaces) what is in the Iceberg table. + """ + node = cluster.instances["replica1"] + ttl_seconds = 3 + + uid = unique_suffix() + mt_table = f"mt_{uid}" + iceberg_table = f"iceberg_{uid}" + + setup_tables(cluster, mt_table, iceberg_table, nodes=["replica1"]) + + # First export. + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {iceberg_table} " + f"SETTINGS export_merge_tree_partition_manifest_ttl = {ttl_seconds}" + ) + + # A second export before the TTL expires must be rejected. + error = node.query_and_get_error( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {iceberg_table}" + ) + assert "Export with key" in error, f"Expected duplicate-export error before TTL, got: {error}" + + wait_for_export_status(node, mt_table, iceberg_table, "2020", "COMPLETED") + + count_after_first = int(node.query(f"SELECT count() FROM {iceberg_table} WHERE year = 2020").strip()) + assert count_after_first == 3, f"Expected 3 rows after first export, got {count_after_first}" + + # Wait for the manifest TTL to expire. + time.sleep(ttl_seconds * 2) + + # Second export must be accepted now. + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {iceberg_table}" + ) + wait_for_export_status(node, mt_table, iceberg_table, "2020", "COMPLETED") + + +def test_export_data_files_are_not_cleaned_up_on_commit_failure(cluster): + """ + Verify that the data files are not cleaned up on commit failure and the export is retried. + This is to avoid data loss. + + If the data files were cleaned up, a retry would commit a new snapshot that points to dangling references. + """ + node = cluster.instances["replica1"] + uid = unique_suffix() + mt_table = f"mt_{uid}" + iceberg_table = f"iceberg_{uid}" + setup_tables(cluster, mt_table, iceberg_table, nodes=["replica1"]) + + node.query("SYSTEM ENABLE FAILPOINT iceberg_writes_non_retry_cleanup") + + node.query(f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {iceberg_table}") + wait_for_export_status(node, mt_table, iceberg_table, "2020", "COMPLETED") + + count = int(node.query(f"SELECT count() FROM {iceberg_table} WHERE year = 2020").strip()) + assert count == 3, f"Expected 3 rows after first export, got {count}" + + +def test_post_publish_exception_preserves_snapshot(cluster): + """ + Regression test for the post-publish exception-safety bug in + commitImportPartitionTransactionImpl. + + Before the fix, any exception thrown after the Iceberg snapshot was published + (e.g. from metadata-cache invalidation) would fall through to the outer + `catch (...)` and invoke `cleanup(false)`, which unconditionally removed the + manifest entry and manifest list referenced by the just-published snapshot. + A subsequent read would then fail because the live snapshot points to deleted + files. + + The failpoint `iceberg_writes_post_publish_throw` is placed inside the + post-publish region (after both the metadata file is written and + `published = true` is set). With the fix in place: + - the commit stays durable (snapshot is readable, manifests are intact); + - the export is marked COMPLETED because the idempotency check on retry + detects that the transaction is already committed and returns success; + - all exported rows are visible through the Iceberg table. + """ + node = cluster.instances["replica1"] + uid = unique_suffix() + mt_table = f"mt_{uid}" + iceberg_table = f"iceberg_{uid}" + setup_tables(cluster, mt_table, iceberg_table, nodes=["replica1"]) + + node.query("SYSTEM ENABLE FAILPOINT iceberg_writes_post_publish_throw") + + node.query(f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {iceberg_table}") + wait_for_export_status(node, mt_table, iceberg_table, "2020", "COMPLETED") + + count = int(node.query(f"SELECT count() FROM {iceberg_table} WHERE year = 2020").strip()) + assert count == 3, ( + f"Snapshot must remain readable after a post-publish exception, " + f"expected 3 rows but got {count} (manifest files likely deleted by " + f"over-broad cleanup)" + ) + + result = node.query( + f"SELECT id, year FROM {iceberg_table} WHERE year = 2020 ORDER BY id" + ).strip() + assert result == "1\t2020\n2\t2020\n3\t2020", ( + f"Unexpected data after post-publish exception recovery:\n{result}" + ) + + +def test_export_task_timeout_kills_stuck_pending_task(cluster): + """ + Verify that export_merge_tree_partition_task_timeout_seconds auto-kills a task + that remains PENDING past the deadline, transitioning it to KILLED with a + descriptive last_exception. + + The export_partition_commit_always_throw failpoint wedges the task in the + commit retry loop (REGULAR failpoint, fires on every commit attempt). A very + large max_retries budget prevents the commit-attempts path from transitioning + to FAILED before the timeout fires, so the timeout branch in tryCleanup is + the actual mechanism under test. + """ + node = cluster.instances["replica1"] + uid = unique_suffix() + mt_table = f"mt_{uid}" + iceberg_table = f"iceberg_{uid}" + setup_tables(cluster, mt_table, iceberg_table, nodes=["replica1"]) + + node.query("SYSTEM ENABLE FAILPOINT export_partition_commit_always_throw") + + try: + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {iceberg_table}" + f" SETTINGS export_merge_tree_partition_task_timeout_seconds = 5," + f" export_merge_tree_partition_max_retries = 1000000," + f" export_merge_tree_partition_manifest_ttl = 3600" + ) + + # Timeout budget must cover: the 5s task timeout + one manifest-updating + # poll cycle (~30s) + watch propagation. 90s is safe. + wait_for_export_status( + node, mt_table, iceberg_table, "2020", + expected_status="KILLED", + timeout=90, + ) + + # TODO: system.replicated_partition_exports does not currently surface + # last_exception / exception_count reliably (the engine's aggregation + # from exceptions_per_replica is incomplete). Read the raw znode via + # system.zookeeper until that is fixed. + export_key = f"2020_default.{iceberg_table}" + last_exception_path = ( + f"/clickhouse/tables/{mt_table}/exports/{export_key}" + f"/exceptions_per_replica/replica1/last_exception" + ) + last_exception = node.query( + f""" + SELECT value FROM system.zookeeper + WHERE path = '{last_exception_path}' AND name = 'exception' + """ + ).strip() + assert "timed out" in last_exception, ( + f"Expected last_exception znode to mention the timeout reason, got: {last_exception!r}" + ) + finally: + node.query("SYSTEM DISABLE FAILPOINT export_partition_commit_always_throw") + + +def setup_stats_tables(node, mt_table: str, iceberg_table: str): + """Local variant of setup_tables using the wider schema with a Nullable column.""" + columns = "id Int32, name String, tag Nullable(String), year Int32" + + make_rmt( + node, mt_table, columns, "year", + order_by="id", replica_name="replica1", + ) + node.query( + f""" + INSERT INTO {mt_table} (id, name, tag, year) VALUES + (1, 'aaa', 'x', 2020), + (2, 'mmm', NULL, 2020), + (3, 'zzz', 'y', 2020), + (4, 'kkk', 'z', 2021) + """ + ) + + make_iceberg_s3(node, iceberg_table, columns, partition_by="year") + + +def test_export_partition_writes_column_statistics(cluster): + """ + Export a whole partition (EXPORT PARTITION ID '2020') that contains one NULL + and verify that the resulting Iceberg manifest entry carries accurate per-file + column statistics: record_count, file_size_in_bytes, column_sizes, + null_value_counts, and lower/upper bounds. + """ + node = cluster.instances["replica1"] + + uid = unique_suffix() + mt_table = f"mt_stats_{uid}" + iceberg_table = f"iceberg_stats_{uid}" + + setup_stats_tables(node, mt_table, iceberg_table) + + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {iceberg_table}" + ) + wait_for_export_status(node, mt_table, iceberg_table, "2020", "COMPLETED") + + count = int(node.query(f"SELECT count() FROM {iceberg_table}").strip()) + assert count == 3, f"Expected 3 rows in Iceberg table after export, got {count}" + + query_id = f"stats_partition_{uid}" + node.query( + f"SELECT * FROM {iceberg_table} ORDER BY id", + query_id=query_id, + settings={"iceberg_metadata_log_level": "manifest_file_entry"}, + ) + + entries = fetch_manifest_entries(node, query_id) + assert_exported_stats(entries) diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/allow_experimental_export_partition.xml b/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/allow_experimental_export_partition.xml index f8c5fab1a3be..d931c6fb00db 100644 --- a/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/allow_experimental_export_partition.xml +++ b/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/allow_experimental_export_partition.xml @@ -1,3 +1,3 @@ - 1 + 1 \ No newline at end of file diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/disable_experimental_export_partition.xml b/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/disable_experimental_export_partition.xml index ba6508ebd660..5379b8e892f0 100644 --- a/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/disable_experimental_export_partition.xml +++ b/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/disable_experimental_export_partition.xml @@ -1,3 +1,3 @@ - 0 + 0 \ No newline at end of file diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/s3_retries.xml b/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/s3_retries.xml deleted file mode 100644 index fb74c9bdf7de..000000000000 --- a/tests/integration/test_export_replicated_mt_partition_to_object_storage/configs/s3_retries.xml +++ /dev/null @@ -1,3 +0,0 @@ - - 3 - diff --git a/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py index 327d78cfa49b..fa7d99f1642f 100644 --- a/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py +++ b/tests/integration/test_export_replicated_mt_partition_to_object_storage/test.py @@ -1,109 +1,18 @@ import logging -import pytest -import random -import string import time -from typing import Optional import uuid +import pytest + from helpers.cluster import ClickHouseCluster +from helpers.export_partition_helpers import ( + wait_for_exception_count, + wait_for_export_status, + wait_for_export_to_start, +) from helpers.network import PartitionManager -def wait_for_export_status( - node, - mt_table: str, - s3_table: str, - partition_id: str, - expected_status: str = "COMPLETED", - timeout: int = 30, - poll_interval: float = 0.5, -): - start_time = time.time() - last_status = None - while time.time() - start_time < timeout: - status = node.query( - f""" - SELECT status FROM system.replicated_partition_exports - WHERE source_table = '{mt_table}' - AND destination_table = '{s3_table}' - AND partition_id = '{partition_id}' - """ - ).strip() - - last_status = status - - if status and status == expected_status: - return status - - time.sleep(poll_interval) - - raise TimeoutError( - f"Export status did not reach '{expected_status}' within {timeout}s. Last status: '{last_status}'") - - -def wait_for_export_to_start( - node, - mt_table: str, - s3_table: str, - partition_id: str, - timeout: int = 10, - poll_interval: float = 0.2, -): - start_time = time.time() - while time.time() - start_time < timeout: - count = node.query( - f""" - SELECT count() FROM system.replicated_partition_exports - WHERE source_table = '{mt_table}' - AND destination_table = '{s3_table}' - AND partition_id = '{partition_id}' - """ - ).strip() - - if count != '0': - return True - - time.sleep(poll_interval) - - raise TimeoutError(f"Export did not start within {timeout}s. ") - - -def wait_for_exception_count( - node, - mt_table: str, - s3_table: str, - partition_id: str, - min_exception_count: int = 1, - timeout: int = 30, - poll_interval: float = 0.5, -): - """Wait for exception_count to reach at least min_exception_count.""" - start_time = time.time() - last_exception_count = None - while time.time() - start_time < timeout: - exception_count_str = node.query( - f""" - SELECT exception_count FROM system.replicated_partition_exports - WHERE source_table = '{mt_table}' - AND destination_table = '{s3_table}' - AND partition_id = '{partition_id}' - """ - ).strip() - - if exception_count_str: - exception_count = int(exception_count_str) - last_exception_count = exception_count - if exception_count >= min_exception_count: - return exception_count - - time.sleep(poll_interval) - - raise TimeoutError( - f"Exception count did not reach {min_exception_count} within {timeout}s. " - f"Last exception_count: {last_exception_count if last_exception_count is not None else 'N/A'}" - ) - def skip_if_remote_database_disk_enabled(cluster): """Skip test if any instance in the cluster has remote database disk enabled. @@ -211,6 +120,7 @@ def create_s3_table(node, s3_table): def create_tables_and_insert_data(node, mt_table, s3_table, replica_name): + # enable_block_number_column and enable_block_offset_column are needed for patch parts support node.query(f"CREATE TABLE {mt_table} (id UInt64, year UInt16) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{mt_table}', '{replica_name}') PARTITION BY year ORDER BY tuple() SETTINGS enable_block_number_column = 1, enable_block_offset_column = 1") node.query(f"INSERT INTO {mt_table} VALUES (1, 2020), (2, 2020), (3, 2020), (4, 2021)") @@ -412,6 +322,71 @@ def test_kill_export(cluster, system_table_prefer_remote_information): assert node.query(f"SELECT count() FROM {s3_table} WHERE year = 2020") == '0\n', "Partition 2020 was written to S3, it was not killed as expected" +def test_kill_export_resilient_to_status_handling_failure(cluster): + """KILL EXPORT PARTITION must eventually take effect even when the first + attempt to handle the ZK status-change event throws (simulated via a ONCE + failpoint). The re-queue + reschedule mechanism retries after ~5 s and + the second attempt succeeds because the ONCE failpoint has already fired.""" + skip_if_remote_database_disk_enabled(cluster) + node = cluster.instances["replica1"] + + postfix = str(uuid.uuid4()).replace("-", "_") + mt_table = f"kill_resilient_mt_{postfix}" + s3_table = f"kill_resilient_s3_{postfix}" + + create_tables_and_insert_data(node, mt_table, s3_table, "replica1") + + minio_ip = cluster.minio_ip + minio_port = cluster.minio_port + + with PartitionManager() as pm: + pm.add_rule({ + "instance": node, + "destination": node.ip_address, + "protocol": "tcp", + "source_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + }) + + pm.add_rule({ + "instance": node, + "destination": minio_ip, + "protocol": "tcp", + "destination_port": minio_port, + "action": "REJECT --reject-with tcp-reset", + }) + + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {s3_table}" + f" SETTINGS export_merge_tree_partition_max_retries = 50" + ) + + node.query("SYSTEM ENABLE FAILPOINT export_partition_status_change_throw") + + node.query( + f"KILL EXPORT PARTITION WHERE partition_id = '2020'" + f" AND source_table = '{mt_table}' AND destination_table = '{s3_table}'") + + # sleep for a while to let the kill to be processed + time.sleep(5) + + # The ONCE failpoint makes the first handleStatusChanges() throw. + # The catch re-queues the key and scheduleAfter(5000) arms a retry. + # Wait up to 15 s (5 s retry delay + margin) for the kill to propagate. + wait_for_export_status(node, mt_table, s3_table, "2020", "KILLED", timeout=15) + + # query the local status export_merge_tree_partition_system_table_prefer_remote_information=0 + assert ( + node.query( + f"SELECT status FROM system.replicated_partition_exports" + f" WHERE partition_id = '2020'" + f" AND source_table = '{mt_table}'" + f" AND destination_table = '{s3_table}'" + f" SETTINGS export_merge_tree_partition_system_table_prefer_remote_information = 0" + ).strip() == "KILLED" + ), "Export was not killed — status change was lost after the injected failure" + + def test_drop_source_table_during_export(cluster): skip_if_remote_database_disk_enabled(cluster) node = cluster.instances["replica1"] @@ -454,13 +429,16 @@ def test_drop_source_table_during_export(cluster): export_queries = f""" ALTER TABLE {mt_table} - EXPORT PARTITION ID '2020' TO TABLE {s3_table}; + EXPORT PARTITION ID '2020' TO TABLE {s3_table} SETTINGS s3_retry_attempts = 500, export_merge_tree_partition_max_retries = 50; ALTER TABLE {mt_table} - EXPORT PARTITION ID '2021' TO TABLE {s3_table}; + EXPORT PARTITION ID '2021' TO TABLE {s3_table} SETTINGS s3_retry_attempts = 500, export_merge_tree_partition_max_retries = 50; """ node.query(export_queries) + wait_for_export_status(node, mt_table, s3_table, "2020", "PENDING") + wait_for_export_status(node, mt_table, s3_table, "2021", "PENDING") + # This should kill the background operations and drop the table node.query(f"DROP TABLE {mt_table}") @@ -647,6 +625,7 @@ def test_inject_short_living_failures(cluster): WHERE source_table = '{mt_table}' AND destination_table = '{s3_table}' AND partition_id = '2020' + SETTINGS export_merge_tree_partition_system_table_prefer_remote_information = 1 """ ) assert int(exception_count.strip()) >= 1, "Expected at least one exception" @@ -1525,4 +1504,3 @@ def test_export_partition_resumes_after_stop_moves_during_export(cluster): row_count = int(node.query(f"SELECT count() FROM {s3_table} WHERE year = 2020").strip()) assert row_count == 3, f"Expected 3 rows in S3 after export completed, got {row_count}" - diff --git a/tests/integration/test_storage_iceberg_with_spark/configs/config.d/allow_export_partition.xml b/tests/integration/test_storage_iceberg_with_spark/configs/config.d/allow_export_partition.xml new file mode 100644 index 000000000000..514cd710836a --- /dev/null +++ b/tests/integration/test_storage_iceberg_with_spark/configs/config.d/allow_export_partition.xml @@ -0,0 +1,3 @@ + + 1 + diff --git a/tests/integration/test_storage_iceberg_with_spark/configs/users.d/allow_export_partition.xml b/tests/integration/test_storage_iceberg_with_spark/configs/users.d/allow_export_partition.xml new file mode 100644 index 000000000000..b129913efd18 --- /dev/null +++ b/tests/integration/test_storage_iceberg_with_spark/configs/users.d/allow_export_partition.xml @@ -0,0 +1,7 @@ + + + + 1 + + + diff --git a/tests/integration/test_storage_iceberg_with_spark/test_export_partition_iceberg.py b/tests/integration/test_storage_iceberg_with_spark/test_export_partition_iceberg.py new file mode 100644 index 000000000000..d289639eac12 --- /dev/null +++ b/tests/integration/test_storage_iceberg_with_spark/test_export_partition_iceberg.py @@ -0,0 +1,912 @@ +""" +Tests for EXPORT PARTITION to an Iceberg table that was created by Apache Spark. + +The destination Iceberg metadata — including field IDs and the partition spec — +is written by Spark, not by ClickHouse, which removes any bias from tests where +both source and destination are ClickHouse-created. + +A separate module-level fixture is used because the package-level +started_cluster_iceberg_with_spark does not include ZooKeeper (which is +required for ReplicatedMergeTree / EXPORT PARTITION). + +Transform coverage (ClickHouse → Iceberg): + identity → identity + toYearNumSinceEpoch → year + toMonthNumSinceEpoch → month + toRelativeDayNum → day + toRelativeHourNum → hour + icebergBucket(N) → bucket(N) + icebergTruncate(N) → truncate(N) + compound → multiple fields +""" + +import logging +import threading +import time +from concurrent.futures import ThreadPoolExecutor + +import pytest +import pyspark + +from helpers.cluster import ClickHouseCluster +from helpers.export_partition_helpers import ( + first_partition_id, + make_iceberg_s3, + make_rmt, + unique_suffix, + wait_for_export_status, +) +from helpers.iceberg_utils import ( + create_iceberg_table, + default_upload_directory, +) +from helpers.s3_tools import S3Uploader, prepare_s3_bucket + + +# --------------------------------------------------------------------------- +# Spark session +# --------------------------------------------------------------------------- + +def get_spark(): + builder = ( + pyspark.sql.SparkSession.builder + .appName("test_export_partition_spark_iceberg") + .config( + "spark.sql.catalog.spark_catalog", + "org.apache.iceberg.spark.SparkSessionCatalog", + ) + .config("spark.sql.catalog.local", "org.apache.iceberg.spark.SparkCatalog") + .config("spark.sql.catalog.spark_catalog.type", "hadoop") + .config( + "spark.sql.catalog.spark_catalog.warehouse", + "/var/lib/clickhouse/user_files/iceberg_data", + ) + .config( + "spark.sql.extensions", + "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions", + ) + .master("local") + ) + return builder.getOrCreate() + + +# --------------------------------------------------------------------------- +# Cluster fixture +# --------------------------------------------------------------------------- + +@pytest.fixture(scope="module") +def export_cluster(): + try: + cluster = ClickHouseCluster(__file__, with_spark=True) + cluster.add_instance( + "node1", + main_configs=[ + "configs/config.d/named_collections.xml", + "configs/config.d/allow_export_partition.xml", + ], + user_configs=[ + "configs/users.d/allow_export_partition.xml", + ], + with_minio=True, + stay_alive=True, + with_zookeeper=True, + keeper_required_feature_flags=["multi_read"], + ) + for name in ["replica1", "replica2", "replica3"]: + cluster.add_instance( + name, + main_configs=[ + "configs/config.d/named_collections.xml", + "configs/config.d/allow_export_partition.xml", + ], + user_configs=[ + "configs/users.d/allow_export_partition.xml", + ], + stay_alive=True, + with_zookeeper=True, + keeper_required_feature_flags=["multi_read"], + ) + logging.info("Starting export_cluster...") + cluster.start() + prepare_s3_bucket(cluster) + cluster.spark_session = get_spark() + cluster.default_s3_uploader = S3Uploader(cluster.minio_client, cluster.minio_bucket) + yield cluster + finally: + cluster.shutdown() + + +@pytest.fixture(autouse=True) +def drop_tables(export_cluster): + yield + for node_name in ["node1", "replica1", "replica2", "replica3"]: + node = export_cluster.instances[node_name] + try: + tables = node.query( + "SELECT name FROM system.tables WHERE database = 'default' FORMAT TabSeparated" + ).strip() + for table in tables.splitlines(): + table = table.strip() + if table: + node.query(f"DROP TABLE IF EXISTS default.`{table}` SYNC") + except Exception as e: + logging.warning(f"drop_tables cleanup failed on {node_name}: {e}") + + +# --------------------------------------------------------------------------- +# Helpers +# --------------------------------------------------------------------------- + +def spark_iceberg(cluster, spark, iceberg_name: str, ddl: str): + """Execute a Spark DDL and upload the resulting Iceberg files to MinIO.""" + spark.sql(ddl) + default_upload_directory( + cluster, + "s3", + f"/iceberg_data/default/{iceberg_name}/", + f"/iceberg_data/default/{iceberg_name}/", + ) + + +def attach_ch_iceberg(node, iceberg_name: str, schema: str, cluster): + """ + Attach a ClickHouse IcebergS3 table to an existing Spark-written Iceberg path. + No PARTITION BY is specified — the spec is read from Spark's metadata. + """ + create_iceberg_table( + "s3", + node, + iceberg_name, + cluster, + schema=f"({schema})", + if_not_exists=True, + ) + + + +def run_accepted(export_cluster, label, spark_ddl, ch_schema, rmt_columns, rmt_partition_by, insert_values): + """ + Create a Spark-created Iceberg table, attach ClickHouse to it, create the + source RMT, export, wait, and return (node, source, iceberg, partition_id) + so the caller can do additional assertions. + """ + node = export_cluster.instances["node1"] + spark = export_cluster.spark_session + + uid = unique_suffix() + source = f"rmt_{label}_{uid}" + iceberg = f"spark_{label}_{uid}" + + spark_iceberg(export_cluster, spark, iceberg, spark_ddl.format(TABLE=iceberg)) + attach_ch_iceberg(node, iceberg, ch_schema, export_cluster) + make_rmt(node, source, rmt_columns, rmt_partition_by, order_by="id") + node.query(f"INSERT INTO {source} VALUES {insert_values}") + + pid = first_partition_id(node, source) + node.query(f"ALTER TABLE {source} EXPORT PARTITION ID '{pid}' TO TABLE {iceberg}") + wait_for_export_status(node, source, iceberg, pid) + + return node, source, iceberg, pid + + +def run_rejected(export_cluster, label, spark_ddl, ch_schema, rmt_columns, rmt_partition_by, insert_values): + """ + Create a mismatched pair and assert that EXPORT PARTITION fails with BAD_ARGUMENTS. + The check fires synchronously before any task is enqueued. + """ + node = export_cluster.instances["node1"] + spark = export_cluster.spark_session + + uid = unique_suffix() + source = f"rmt_{label}_{uid}" + iceberg = f"spark_{label}_{uid}" + + spark_iceberg(export_cluster, spark, iceberg, spark_ddl.format(TABLE=iceberg)) + attach_ch_iceberg(node, iceberg, ch_schema, export_cluster) + make_rmt(node, source, rmt_columns, rmt_partition_by, order_by="id") + node.query(f"INSERT INTO {source} VALUES {insert_values}") + + pid = first_partition_id(node, source) + error = node.query_and_get_error( + f"ALTER TABLE {source} EXPORT PARTITION ID '{pid}' TO TABLE {iceberg}" + ) + return error + + +# --------------------------------------------------------------------------- +# Replicated helpers +# --------------------------------------------------------------------------- + + +def create_iceberg_s3_table(node, iceberg_table: str, if_not_exists: bool = False): + """Create (or attach to an existing) IcebergS3 table at a per-test MinIO prefix.""" + make_iceberg_s3( + node, iceberg_table, "id Int64, year Int32", + partition_by="year", if_not_exists=if_not_exists, + ) + + +def setup_replicas(cluster, mt_table: str, iceberg_table: str, replica_names: list): + """ + Create RMT on each replica with a per-replica replica_name so all instances share + the same ZooKeeper path. Create IcebergS3 on the primary; attach with IF NOT EXISTS + on the rest. No data is inserted here — callers manage their own test data. + """ + instances = [cluster.instances[n] for n in replica_names] + primary = instances[0] + + for rname, instance in zip(replica_names, instances): + make_rmt(instance, mt_table, "id Int64, year Int32", "year", replica_name=rname) + + create_iceberg_s3_table(primary, iceberg_table) + for instance in instances[1:]: + create_iceberg_s3_table(instance, iceberg_table, if_not_exists=True) + + + +# --------------------------------------------------------------------------- +# Happy-path tests — one per transform +# --------------------------------------------------------------------------- + +def test_identity_transform(export_cluster): + """Spark identity(year) <-> PARTITION BY year.""" + node, _, iceberg, _ = run_accepted( + export_cluster, + "identity", + spark_ddl="CREATE TABLE {TABLE} (id BIGINT, year INT)" + " USING iceberg PARTITIONED BY (identity(year)) OPTIONS('format-version'='2')", + ch_schema="id Int64, year Int32", + rmt_columns="id Int64, year Int32", + rmt_partition_by="year", + insert_values="(1, 2024), (2, 2024), (3, 2024)", + ) + assert int(node.query(f"SELECT count() FROM {iceberg}").strip()) == 3 + + +def test_year_transform(export_cluster): + """Spark years(dt) <-> PARTITION BY toYearNumSinceEpoch(dt).""" + node, _, iceberg, _ = run_accepted( + export_cluster, + "year", + spark_ddl="CREATE TABLE {TABLE} (id BIGINT, dt DATE)" + " USING iceberg PARTITIONED BY (years(dt)) OPTIONS('format-version'='2')", + ch_schema="id Int64, dt Date", + rmt_columns="id Int64, dt Date", + rmt_partition_by="toYearNumSinceEpoch(dt)", + insert_values="(1, '2021-03-01'), (2, '2021-07-15'), (3, '2021-12-31')", + ) + assert int(node.query(f"SELECT count() FROM {iceberg}").strip()) == 3 + + +def test_month_transform(export_cluster): + """Spark months(dt) <-> PARTITION BY toMonthNumSinceEpoch(dt).""" + node, _, iceberg, _ = run_accepted( + export_cluster, + "month", + spark_ddl="CREATE TABLE {TABLE} (id BIGINT, dt DATE)" + " USING iceberg PARTITIONED BY (months(dt)) OPTIONS('format-version'='2')", + ch_schema="id Int64, dt Date", + rmt_columns="id Int64, dt Date", + rmt_partition_by="toMonthNumSinceEpoch(dt)", + insert_values="(1, '2020-06-01'), (2, '2020-06-15'), (3, '2020-06-30')", + ) + assert int(node.query(f"SELECT count() FROM {iceberg}").strip()) == 3 + + +def test_day_transform(export_cluster): + """Spark days(dt) <-> PARTITION BY toRelativeDayNum(dt).""" + node, _, iceberg, _ = run_accepted( + export_cluster, + "day", + spark_ddl="CREATE TABLE {TABLE} (id BIGINT, dt DATE)" + " USING iceberg PARTITIONED BY (days(dt)) OPTIONS('format-version'='2')", + ch_schema="id Int64, dt Date", + rmt_columns="id Int64, dt Date", + rmt_partition_by="toRelativeDayNum(dt)", + insert_values="(1, '2023-03-15'), (2, '2023-03-15'), (3, '2023-03-15')", + ) + assert int(node.query(f"SELECT count() FROM {iceberg}").strip()) == 3 + + +def test_hour_transform(export_cluster): + """Spark hours(ts) <-> PARTITION BY toRelativeHourNum(ts). + + Spark TIMESTAMP maps to Iceberg 'timestamp' which ClickHouse reads as DateTime64(6). + All three rows fall within the same hour so a single partition is exported. + """ + node, _, iceberg, _ = run_accepted( + export_cluster, + "hour", + spark_ddl="CREATE TABLE {TABLE} (id BIGINT, ts TIMESTAMP)" + " USING iceberg PARTITIONED BY (hours(ts)) OPTIONS('format-version'='2')", + ch_schema="id Int64, ts DateTime64(6)", + rmt_columns="id Int64, ts DateTime64(6)", + rmt_partition_by="toRelativeHourNum(ts)", + insert_values=( + "(1, '2023-03-15 10:00:00'), " + "(2, '2023-03-15 10:30:00'), " + "(3, '2023-03-15 10:59:00')" + ), + ) + assert int(node.query(f"SELECT count() FROM {iceberg}").strip()) == 3 + + +def test_bucket_transform(export_cluster): + """Spark bucket(8, user_id) <-> PARTITION BY icebergBucket(8, user_id).""" + node, _, iceberg, _ = run_accepted( + export_cluster, + "bucket", + spark_ddl="CREATE TABLE {TABLE} (id BIGINT, user_id BIGINT)" + " USING iceberg PARTITIONED BY (bucket(8, user_id)) OPTIONS('format-version'='2')", + ch_schema="id Int64, user_id Int64", + rmt_columns="id Int64, user_id Int64", + rmt_partition_by="icebergBucket(8, user_id)", + # All rows share the same user_id → same bucket → single partition. + insert_values="(1, 42), (2, 42), (3, 42)", + ) + assert int(node.query(f"SELECT count() FROM {iceberg}").strip()) == 3 + + +def test_truncate_transform(export_cluster): + """Spark truncate(4, category) <-> PARTITION BY icebergTruncate(4, category).""" + node, _, iceberg, _ = run_accepted( + export_cluster, + "truncate", + spark_ddl="CREATE TABLE {TABLE} (id BIGINT, category STRING)" + " USING iceberg PARTITIONED BY (truncate(4, category)) OPTIONS('format-version'='2')", + ch_schema="id Int64, category String", + rmt_columns="id Int64, category String", + rmt_partition_by="icebergTruncate(4, category)", + # All share the 4-char prefix 'clic' → same truncate bucket. + insert_values="(1, 'clickhouse'), (2, 'click'), (3, 'clickstream')", + ) + assert int(node.query(f"SELECT count() FROM {iceberg}").strip()) == 3 + + +def test_compound_transform(export_cluster): + """Spark (identity(year), identity(region)) <-> PARTITION BY (year, region).""" + node, _, iceberg, _ = run_accepted( + export_cluster, + "compound", + spark_ddl="CREATE TABLE {TABLE} (id BIGINT, year INT, region STRING)" + " USING iceberg PARTITIONED BY (identity(year), identity(region))" + " OPTIONS('format-version'='2')", + ch_schema="id Int64, year Int32, region String", + rmt_columns="id Int64, year Int32, region String", + rmt_partition_by="(year, region)", + insert_values="(1, 2022, 'EU'), (2, 2022, 'EU'), (3, 2022, 'EU')", + ) + assert int(node.query(f"SELECT count() FROM {iceberg}").strip()) == 3 + + +def test_identity_int64(export_cluster): + """Spark identity(user_id) on BIGINT <-> PARTITION BY user_id (Int64). + + Int64 → Avro 'long' is already handled by getAvroType(). This test covers + the identity transform on a 64-bit integer column, which is not covered by + the existing test_identity_transform (which uses Int32). + """ + node, _, iceberg, _ = run_accepted( + export_cluster, + "identity_int64", + spark_ddl="CREATE TABLE {TABLE} (id BIGINT, user_id BIGINT)" + " USING iceberg PARTITIONED BY (identity(user_id))" + " OPTIONS('format-version'='2')", + ch_schema="id Int64, user_id Int64", + rmt_columns="id Int64, user_id Int64", + rmt_partition_by="user_id", + insert_values="(1, 100), (2, 100), (3, 100)", + ) + assert int(node.query(f"SELECT count() FROM {iceberg}").strip()) == 3 + + +def test_identity_date(export_cluster): + """Spark identity(event_date) on DATE <-> PARTITION BY event_date (Date32). + + Date32 → Avro 'int' is already handled by getAvroType(). This test covers + the identity transform directly on a date column. Existing date-related tests + (test_year_transform, test_month_transform, etc.) use time-based transforms + such as years() and months(), not identity(). + """ + node, _, iceberg, _ = run_accepted( + export_cluster, + "identity_date", + spark_ddl="CREATE TABLE {TABLE} (id BIGINT, event_date DATE)" + " USING iceberg PARTITIONED BY (identity(event_date))" + " OPTIONS('format-version'='2')", + ch_schema="id Int64, event_date Date32", + rmt_columns="id Int64, event_date Date32", + rmt_partition_by="event_date", + insert_values="(1, '2024-03-15'), (2, '2024-03-15'), (3, '2024-03-15')", + ) + assert int(node.query(f"SELECT count() FROM {iceberg}").strip()) == 3 + + +def test_identity_string(export_cluster): + """Spark identity(region) on STRING <-> PARTITION BY region (String). + + String → Avro 'string' is already handled by getAvroType(). This test covers + identity on a string column as the sole partition field. The existing + test_compound_transform uses identity(region) only as part of a multi-field spec, + so a standalone string identity partition was not previously exercised end-to-end. + """ + node, _, iceberg, _ = run_accepted( + export_cluster, + "identity_str", + spark_ddl="CREATE TABLE {TABLE} (id BIGINT, region STRING)" + " USING iceberg PARTITIONED BY (identity(region))" + " OPTIONS('format-version'='2')", + ch_schema="id Int64, region String", + rmt_columns="id Int64, region String", + rmt_partition_by="region", + insert_values="(1, 'EU'), (2, 'EU'), (3, 'EU')", + ) + assert int(node.query(f"SELECT count() FROM {iceberg}").strip()) == 3 + + +def test_truncate_int64(export_cluster): + """Spark truncate(10, amount) on BIGINT <-> PARTITION BY icebergTruncate(10, amount) (Int64). + + Int64 truncate produces floor(v / 10) * 10, so all rows with amount=42 land in + partition value 40 (same partition). This is a distinct code path from truncate on + String (which trims a character prefix). The existing test_truncate_transform uses + String only, leaving the integer truncate path untested. + """ + node, _, iceberg, _ = run_accepted( + export_cluster, + "truncate_int64", + spark_ddl="CREATE TABLE {TABLE} (id BIGINT, amount BIGINT)" + " USING iceberg PARTITIONED BY (truncate(10, amount))" + " OPTIONS('format-version'='2')", + ch_schema="id Int64, amount Int64", + rmt_columns="id Int64, amount Int64", + rmt_partition_by="icebergTruncate(10, amount)", + # All rows have amount=42 → truncated partition value is 40. + insert_values="(1, 42), (2, 42), (3, 42)", + ) + assert int(node.query(f"SELECT count() FROM {iceberg}").strip()) == 3 + + +def test_bucket_string(export_cluster): + """Spark bucket(8, name) on STRING <-> PARTITION BY icebergBucket(8, name) (String). + + Bucket on strings uses Murmur3 hash of the UTF-8 bytes, a different hash path than + bucket on integers. All rows share the same name so they land in the same bucket. + The existing test_bucket_transform uses BIGINT only, leaving string bucketing untested. + """ + node, _, iceberg, _ = run_accepted( + export_cluster, + "bucket_str", + spark_ddl="CREATE TABLE {TABLE} (id BIGINT, name STRING)" + " USING iceberg PARTITIONED BY (bucket(8, name))" + " OPTIONS('format-version'='2')", + ch_schema="id Int64, name String", + rmt_columns="id Int64, name String", + rmt_partition_by="icebergBucket(8, name)", + # All rows share the same name → same Murmur3 bucket. + insert_values="(1, 'alice'), (2, 'alice'), (3, 'alice')", + ) + assert int(node.query(f"SELECT count() FROM {iceberg}").strip()) == 3 + + +def test_year_transform_timestamp(export_cluster): + """Spark years(ts) on TIMESTAMP <-> PARTITION BY toYearNumSinceEpoch(ts) (DateTime64(6)). + + DateTime64 → Avro 'long' is already handled by getAvroType(). The year transform on + TIMESTAMP follows a different branch than on DATE (long vs int in Avro). The existing + test_year_transform uses DATE only. All three rows fall within the same year. + """ + node, _, iceberg, _ = run_accepted( + export_cluster, + "year_ts", + spark_ddl="CREATE TABLE {TABLE} (id BIGINT, ts TIMESTAMP)" + " USING iceberg PARTITIONED BY (years(ts))" + " OPTIONS('format-version'='2')", + ch_schema="id Int64, ts DateTime64(6)", + rmt_columns="id Int64, ts DateTime64(6)", + rmt_partition_by="toYearNumSinceEpoch(ts)", + insert_values=( + "(1, '2023-01-15 08:00:00'), " + "(2, '2023-06-01 12:00:00'), " + "(3, '2023-12-31 23:59:59')" + ), + ) + assert int(node.query(f"SELECT count() FROM {iceberg}").strip()) == 3 + + +def test_month_transform_timestamp(export_cluster): + """Spark months(ts) on TIMESTAMP <-> PARTITION BY toMonthNumSinceEpoch(ts) (DateTime64(6)). + + Analogous to test_year_transform_timestamp but for the month transform. + All three rows fall within the same calendar month. + """ + node, _, iceberg, _ = run_accepted( + export_cluster, + "month_ts", + spark_ddl="CREATE TABLE {TABLE} (id BIGINT, ts TIMESTAMP)" + " USING iceberg PARTITIONED BY (months(ts))" + " OPTIONS('format-version'='2')", + ch_schema="id Int64, ts DateTime64(6)", + rmt_columns="id Int64, ts DateTime64(6)", + rmt_partition_by="toMonthNumSinceEpoch(ts)", + insert_values=( + "(1, '2023-06-01 00:00:00'), " + "(2, '2023-06-15 12:00:00'), " + "(3, '2023-06-30 23:59:59')" + ), + ) + assert int(node.query(f"SELECT count() FROM {iceberg}").strip()) == 3 + + +def test_day_transform_timestamp(export_cluster): + """Spark days(ts) on TIMESTAMP <-> PARTITION BY toRelativeDayNum(ts) (DateTime64(6)). + + Analogous to test_year_transform_timestamp but for the day transform. + All three rows fall within the same calendar day. + """ + node, _, iceberg, _ = run_accepted( + export_cluster, + "day_ts", + spark_ddl="CREATE TABLE {TABLE} (id BIGINT, ts TIMESTAMP)" + " USING iceberg PARTITIONED BY (days(ts))" + " OPTIONS('format-version'='2')", + ch_schema="id Int64, ts DateTime64(6)", + rmt_columns="id Int64, ts DateTime64(6)", + rmt_partition_by="toRelativeDayNum(ts)", + insert_values=( + "(1, '2023-06-15 00:00:00'), " + "(2, '2023-06-15 12:00:00'), " + "(3, '2023-06-15 23:59:59')" + ), + ) + assert int(node.query(f"SELECT count() FROM {iceberg}").strip()) == 3 + + +# --------------------------------------------------------------------------- +# Unhappy-path tests — BAD_ARGUMENTS must be raised synchronously +# --------------------------------------------------------------------------- + +def test_rejected_column_mismatch(export_cluster): + """Spark identity(year) — RMT PARTITION BY id: different column.""" + error = run_rejected( + export_cluster, + "rej_col_mismatch", + spark_ddl="CREATE TABLE {TABLE} (id BIGINT, year INT)" + " USING iceberg PARTITIONED BY (identity(year)) OPTIONS('format-version'='2')", + ch_schema="id Int64, year Int32", + rmt_columns="id Int64, year Int32", + rmt_partition_by="id", + insert_values="(1, 2024)", + ) + assert "BAD_ARGUMENTS" in error, f"Expected BAD_ARGUMENTS, got: {error!r}" + + +def test_rejected_transform_mismatch(export_cluster): + """Spark years(dt) — RMT PARTITION BY dt (identity, not year-transform).""" + error = run_rejected( + export_cluster, + "rej_xform_mismatch", + spark_ddl="CREATE TABLE {TABLE} (id BIGINT, dt DATE)" + " USING iceberg PARTITIONED BY (years(dt)) OPTIONS('format-version'='2')", + ch_schema="id Int64, dt Date", + rmt_columns="id Int64, dt Date", + rmt_partition_by="dt", + insert_values="(1, '2021-06-01')", + ) + assert "BAD_ARGUMENTS" in error, f"Expected BAD_ARGUMENTS, got: {error!r}" + + +def test_rejected_bucket_count_mismatch(export_cluster): + """Spark bucket(8, user_id) — RMT icebergBucket(16, user_id): wrong N.""" + error = run_rejected( + export_cluster, + "rej_bucket_n", + spark_ddl="CREATE TABLE {TABLE} (id BIGINT, user_id BIGINT)" + " USING iceberg PARTITIONED BY (bucket(8, user_id)) OPTIONS('format-version'='2')", + ch_schema="id Int64, user_id Int64", + rmt_columns="id Int64, user_id Int64", + rmt_partition_by="icebergBucket(16, user_id)", + insert_values="(1, 42)", + ) + assert "BAD_ARGUMENTS" in error, f"Expected BAD_ARGUMENTS, got: {error!r}" + + +def test_rejected_truncate_width_mismatch(export_cluster): + """Spark truncate(4, category) — RMT icebergTruncate(8, category): wrong width.""" + error = run_rejected( + export_cluster, + "rej_trunc_w", + spark_ddl="CREATE TABLE {TABLE} (id BIGINT, category STRING)" + " USING iceberg PARTITIONED BY (truncate(4, category)) OPTIONS('format-version'='2')", + ch_schema="id Int64, category String", + rmt_columns="id Int64, category String", + rmt_partition_by="icebergTruncate(8, category)", + insert_values="(1, 'clickhouse')", + ) + assert "BAD_ARGUMENTS" in error, f"Expected BAD_ARGUMENTS, got: {error!r}" + + +def test_rejected_field_count_mismatch(export_cluster): + """Spark 1-field identity(year) — RMT 2-field (year, region).""" + error = run_rejected( + export_cluster, + "rej_field_n", + spark_ddl="CREATE TABLE {TABLE} (id BIGINT, year INT, region STRING)" + " USING iceberg PARTITIONED BY (identity(year)) OPTIONS('format-version'='2')", + ch_schema="id Int64, year Int32, region String", + rmt_columns="id Int64, year Int32, region String", + rmt_partition_by="(year, region)", + insert_values="(1, 2024, 'EU')", + ) + assert "BAD_ARGUMENTS" in error, f"Expected BAD_ARGUMENTS, got: {error!r}" + + +def test_rejected_compound_order_reversed(export_cluster): + """Spark (identity(year), identity(region)) — RMT (region, year): reversed order.""" + error = run_rejected( + export_cluster, + "rej_compound_rev", + spark_ddl="CREATE TABLE {TABLE} (id BIGINT, year INT, region STRING)" + " USING iceberg PARTITIONED BY (identity(year), identity(region))" + " OPTIONS('format-version'='2')", + ch_schema="id Int64, year Int32, region String", + rmt_columns="id Int64, year Int32, region String", + rmt_partition_by="(region, year)", + insert_values="(1, 2024, 'EU')", + ) + assert "BAD_ARGUMENTS" in error, f"Expected BAD_ARGUMENTS, got: {error!r}" + + +def test_idempotency_after_commit_crash(export_cluster): + """ + Verify that an Iceberg export commit is idempotent when ClickHouse crashes (via + std::terminate() in a failpoint) after the Iceberg metadata is written but before + ZooKeeper is updated to COMPLETED. + Expected behaviour: + - The failpoint fires once: std::terminate() kills the process immediately after the + Iceberg commit; ZK task remains PENDING. + - ClickHouse is restarted. The scheduler picks up the PENDING task and retries the + commit. commitExportPartitionTransaction finds the transaction_id already present in + the Iceberg snapshot summary and skips re-committing. + - The task eventually reaches COMPLETED. + - The row count in the Iceberg table is exactly the number inserted (no duplicates). + """ + node = export_cluster.instances["node1"] + spark = export_cluster.spark_session + uid = unique_suffix() + source = f"rmt_{uid}" + iceberg = f"spark_{uid}" + spark_iceberg( + export_cluster, + spark, + iceberg, + f"CREATE TABLE {iceberg} (id BIGINT, year INT)" + f" USING iceberg PARTITIONED BY (identity(year)) OPTIONS('format-version'='2')", + ) + attach_ch_iceberg(node, iceberg, "id Int64, year Int32", export_cluster) + make_rmt(node, source, "id Int64, year Int32", "year") + node.query(f"INSERT INTO {source} VALUES (1, 2024), (2, 2024), (3, 2024)") + pid = first_partition_id(node, source) + # Enable the ONCE failpoint. When the background scheduler thread reaches the + # injection point (after a successful Iceberg commit), std::terminate() is called + # and the process exits immediately without setting ZK COMPLETED. + node.query("SYSTEM ENABLE FAILPOINT iceberg_export_after_commit_before_zk_completed") + node.query(f"ALTER TABLE {source} EXPORT PARTITION ID '{pid}' TO TABLE {iceberg}") + # the fail point will sleep for 10 seconds. Wait for 5 and then re-start clickhouse. + time.sleep(5) + # Restart ClickHouse. The ZK task is still PENDING; the scheduler will pick it up. + node.restart_clickhouse() + time.sleep(5) + # On restart the scheduler retries the commit. commitExportPartitionTransaction + # detects the transaction_id in the existing Iceberg snapshot summary and returns + # without re-writing any data, then sets ZK COMPLETED. + wait_for_export_status(node, source, iceberg, pid, timeout=60) + # Exactly 3 rows — no duplicates from the idempotent re-commit. + count = int(node.query(f"SELECT count() FROM {iceberg}").strip()) + assert count == 3, f"Expected 3 rows (no duplicates), got {count}" + + +def test_commit_attempts_budget_transitions_to_failed(export_cluster): + """ + Verify that the commit-attempts budget transitions a stuck task to FAILED + instead of leaving it in PENDING forever. + + Reproduction: + - Parts export successfully. + - A REGULAR failpoint (``export_partition_commit_always_throw``) makes every + ``ExportPartitionUtils::commit`` attempt throw before talking to Iceberg. + - ``ExportPartitionUtils::handleCommitFailure`` bumps ``/commit_attempts`` + on each failure and transitions ``/status`` to FAILED once the counter + reaches ``export_merge_tree_partition_max_retries``. + + Expected behaviour: + - The first attempt is made synchronously when the last part completes + (scheduler's ``handlePartExportSuccess``). + - Subsequent attempts come from the manifest-updating task's ``tryCleanup`` + path, polling every 30s. + - With max_retries=2, the task reaches FAILED within roughly one poll cycle. + - The ``commit_attempts`` znode reaches at least max_retries. + """ + node = export_cluster.instances["node1"] + spark = export_cluster.spark_session + + uid = unique_suffix() + source = f"rmt_commit_budget_{uid}" + iceberg = f"spark_commit_budget_{uid}" + + spark_iceberg( + export_cluster, + spark, + iceberg, + f"CREATE TABLE {iceberg} (id BIGINT, year INT)" + f" USING iceberg PARTITIONED BY (identity(year)) OPTIONS('format-version'='2')", + ) + attach_ch_iceberg(node, iceberg, "id Int64, year Int32", export_cluster) + make_rmt(node, source, "id Int64, year Int32", "year") + node.query(f"INSERT INTO {source} VALUES (1, 2024), (2, 2024), (3, 2024)") + pid = first_partition_id(node, source) + + # Force every commit attempt to throw. REGULAR failpoint fires on every hit, + # unlike ONCE which would only fire for the first call. + node.query("SYSTEM ENABLE FAILPOINT export_partition_commit_always_throw") + + # try block exists so we can add a finally that disables the failpoint + try: + # max_retries=2 bounds the test: one attempt from handlePartExportSuccess + # plus one from the manifest-updating task's next poll (~30s) is enough + # to exhaust the budget and flip the task to FAILED. + node.query( + f"ALTER TABLE {source} EXPORT PARTITION ID '{pid}' TO TABLE {iceberg}" + f" SETTINGS export_merge_tree_partition_max_retries = 2" + ) + + # Timeout must cover: at least one manifest-updating poll cycle (30s) + # plus slack for task scheduling and keeper RTT. + wait_for_export_status( + node, source, iceberg, pid, + expected_status="FAILED", + timeout=90, + ) + + # The commit_attempts znode must have reached (at least) max_retries — the + # counter is the direct mechanism that drove the FAILED transition. + # Locate the export's ZK root via the RMT's zookeeper_path and the + # partition_id_destination_db.destination_table export key convention. + export_key = f"{pid}_default.{iceberg}" + commit_attempts = int(node.query( + f"SELECT value FROM system.zookeeper" + f" WHERE path = '/clickhouse/tables/{source}/exports/{export_key}'" + f" AND name = 'commit_attempts'" + ).strip()) + assert commit_attempts >= 2, ( + f"Expected commit_attempts >= 2 (two commit attempts), got {commit_attempts}" + ) + finally: + node.query("SYSTEM DISABLE FAILPOINT export_partition_commit_always_throw") + + +# --------------------------------------------------------------------------- +# Replicated tests — IcebergS3, no catalog +# --------------------------------------------------------------------------- + + +def test_export_initiated_from_replica2(export_cluster): + """ + Export is initiated from replica2 (not the inserting replica). + Validates that any replica can start the export, not just the writer. + """ + uid = unique_suffix() + mt_table = f"rmt_from_replica2_{uid}" + iceberg_table = f"iceberg_from_replica2_{uid}" + + setup_replicas(export_cluster, mt_table, iceberg_table, ["replica1", "replica2"]) + + r1 = export_cluster.instances["replica1"] + r2 = export_cluster.instances["replica2"] + + r1.query(f"INSERT INTO {mt_table} VALUES (1, 2020), (2, 2020), (3, 2020)") + r2.query(f"SYSTEM SYNC REPLICA {mt_table}") + + r2.query(f"ALTER TABLE {mt_table} EXPORT PARTITION ID '2020' TO TABLE {iceberg_table}") + wait_for_export_status(r2, mt_table, iceberg_table, "2020") + + count_r1 = int(r1.query(f"SELECT count() FROM {iceberg_table}").strip()) + assert count_r1 == 3, f"Expected 3 rows from replica1, got {count_r1}" + count_r2 = int(r2.query(f"SELECT count() FROM {iceberg_table}").strip()) + assert count_r2 == 3, f"Expected 3 rows from replica2, got {count_r2}" + + +def test_concurrent_exports_different_partitions_across_replicas(export_cluster): + """ + Three replicas concurrently export distinct partitions (2020, 2021, 2022) to the + same IcebergS3 table. All three commits must succeed and the total row count must + equal the sum of all inserted rows. + """ + uid = unique_suffix() + mt_table = f"rmt_concurrent_diff_parts_{uid}" + iceberg_table = f"iceberg_concurrent_diff_parts_{uid}" + + setup_replicas( + export_cluster, mt_table, iceberg_table, + ["replica1", "replica2", "replica3"], + ) + + r1 = export_cluster.instances["replica1"] + r2 = export_cluster.instances["replica2"] + r3 = export_cluster.instances["replica3"] + + r1.query(f"INSERT INTO {mt_table} VALUES (1, 2020), (2, 2020), (3, 2020)") + r1.query(f"INSERT INTO {mt_table} VALUES (4, 2021), (5, 2021), (6, 2021)") + r1.query(f"INSERT INTO {mt_table} VALUES (7, 2022), (8, 2022), (9, 2022)") + r2.query(f"SYSTEM SYNC REPLICA {mt_table}") + r3.query(f"SYSTEM SYNC REPLICA {mt_table}") + + errors: list = [] + + def export_from(node, pid): + try: + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '{pid}' TO TABLE {iceberg_table}" + ) + wait_for_export_status(node, mt_table, iceberg_table, pid) + except Exception as exc: + errors.append(exc) + + threads = [ + threading.Thread(target=export_from, args=(r1, "2020")), + threading.Thread(target=export_from, args=(r2, "2021")), + threading.Thread(target=export_from, args=(r3, "2022")), + ] + for t in threads: + t.start() + for t in threads: + t.join() + + assert not errors, f"Export threads raised errors: {errors}" + + count = int(r1.query(f"SELECT count() FROM {iceberg_table}").strip()) + assert count == 9, f"Expected 9 rows total (3 per partition), got {count}" + + +def test_three_replica_concurrent_exports(export_cluster): + """ + ThreadPoolExecutor with 3 workers: each replica exports its own distinct partition. + All futures must complete successfully; total row count must be correct. + """ + uid = unique_suffix() + mt_table = f"rmt_three_replicas_concurrent_{uid}" + iceberg_table = f"iceberg_three_replicas_concurrent_{uid}" + + setup_replicas( + export_cluster, mt_table, iceberg_table, + ["replica1", "replica2", "replica3"], + ) + + r1 = export_cluster.instances["replica1"] + r2 = export_cluster.instances["replica2"] + r3 = export_cluster.instances["replica3"] + + r1.query(f"INSERT INTO {mt_table} VALUES (1, 2020), (2, 2020), (3, 2020)") + r1.query(f"INSERT INTO {mt_table} VALUES (4, 2021), (5, 2021), (6, 2021)") + r1.query(f"INSERT INTO {mt_table} VALUES (7, 2022), (8, 2022), (9, 2022)") + r2.query(f"SYSTEM SYNC REPLICA {mt_table}") + r3.query(f"SYSTEM SYNC REPLICA {mt_table}") + + def export_fn(node_pid): + node, pid = node_pid + node.query( + f"ALTER TABLE {mt_table} EXPORT PARTITION ID '{pid}' TO TABLE {iceberg_table}" + ) + wait_for_export_status(node, mt_table, iceberg_table, pid) + + with ThreadPoolExecutor(max_workers=3) as executor: + futures = [ + executor.submit(export_fn, (r1, "2020")), + executor.submit(export_fn, (r2, "2021")), + executor.submit(export_fn, (r3, "2022")), + ] + for fut in futures: + fut.result() + + count = int(r1.query(f"SELECT count() FROM {iceberg_table}").strip()) + assert count == 9, f"Expected 9 rows total (3 per partition), got {count}" diff --git a/tests/integration/test_storage_iceberg_with_spark/test_export_partition_iceberg_catalog.py b/tests/integration/test_storage_iceberg_with_spark/test_export_partition_iceberg_catalog.py new file mode 100644 index 000000000000..9d0cc557dc18 --- /dev/null +++ b/tests/integration/test_storage_iceberg_with_spark/test_export_partition_iceberg_catalog.py @@ -0,0 +1,532 @@ +""" +Tests for EXPORT PARTITION to a catalog-backed Iceberg table (Glue catalog via Moto). + +These tests verify that the catalog commit path (catalog->updateMetadata) is +exercised correctly for EXPORT PARTITION. A dedicated module-level cluster fixture +combines ZooKeeper (for ReplicatedMergeTree) with the Glue docker-compose stack +(Moto mock + MinIO warehouse bucket). + +Test coverage: + test_catalog_basic_export — single partition exported; catalog shows new snapshot + test_catalog_concurrent_export — two partitions exported in parallel; both commits succeed + test_catalog_idempotent_retry — crash after catalog commit; restart; no data duplication +""" + +import logging +import os +import threading +import time +import uuid + +import pytest +from pyiceberg.catalog import load_catalog +from pyiceberg.partitioning import PartitionField, PartitionSpec +from pyiceberg.schema import Schema +from pyiceberg.transforms import IdentityTransform +from pyiceberg.types import LongType, NestedField, StringType + +from helpers.cluster import ClickHouseCluster +from helpers.config_cluster import minio_access_key, minio_secret_key +from helpers.export_partition_helpers import ( + make_rmt, + wait_for_export_status, +) + + +GLUE_BASE_URL = "http://glue:3000" +GLUE_BASE_URL_LOCAL = "http://localhost:3000" +CH_CATALOG_DB = "glue_export_catalog" + + +# --------------------------------------------------------------------------- +# Cluster fixture +# --------------------------------------------------------------------------- + + +@pytest.fixture(scope="module") +def catalog_export_cluster(): + """ + Cluster with ZooKeeper (for ReplicatedMergeTree / EXPORT PARTITION) and the + Glue docker-compose stack (Moto mock + MinIO warehouse bucket). + Spark is not needed; pyiceberg handles table creation and catalog inspection. + replica1 and replica2 are additional nodes for replicated-export tests; they + share the same ZooKeeper, Glue, and MinIO containers as node1. + """ + try: + os.environ["AWS_ACCESS_KEY_ID"] = "testing" + os.environ["AWS_SECRET_ACCESS_KEY"] = "testing" + cluster = ClickHouseCluster(__file__) + for name in ["node1", "replica1", "replica2"]: + cluster.add_instance( + name, + main_configs=[ + "configs/config.d/allow_export_partition.xml", + ], + user_configs=[ + "configs/users.d/allow_export_partition.xml", + ], + stay_alive=True, + with_zookeeper=True, + keeper_required_feature_flags=["multi_read"], + with_glue_catalog=True, + ) + cluster.start() + + time.sleep(15) + + yield cluster + finally: + cluster.shutdown() + + +@pytest.fixture(autouse=True) +def cleanup_tables(catalog_export_cluster): + """Drop all default-DB tables on every node after each test.""" + yield + for node_name in ["node1", "replica1", "replica2"]: + node = catalog_export_cluster.instances[node_name] + try: + tables = node.query( + "SELECT name FROM system.tables WHERE database = 'default' FORMAT TabSeparated" + ).strip() + for tbl in tables.splitlines(): + tbl = tbl.strip() + if tbl: + node.query(f"DROP TABLE IF EXISTS default.`{tbl}` SYNC") + except Exception as exc: + logging.warning("cleanup_tables on %s: %s", node_name, exc) + + +# --------------------------------------------------------------------------- +# Helpers +# --------------------------------------------------------------------------- + + +def connect_catalog(cluster): + """ + Connect to the Moto Glue mock from the test host via localhost:3000. + MinIO is accessed via the container IP for S3 operations. + """ + minio_ip = cluster.get_instance_ip("minio") + return load_catalog( + "glue_test", + **{ + "type": "glue", + "glue.endpoint": GLUE_BASE_URL_LOCAL, + "glue.region": "us-east-1", + "s3.endpoint": f"http://{minio_ip}:9000", + "s3.access-key-id": minio_access_key, + "s3.secret-access-key": minio_secret_key, + }, + ) + + +def setup_ch_catalog_db(node, db_name: str = CH_CATALOG_DB) -> None: + """Drop-and-recreate the ClickHouse DataLakeCatalog database pointing at Glue (Moto).""" + node.query(f"DROP DATABASE IF EXISTS {db_name}") + node.query( + f""" + SET write_full_path_in_iceberg_metadata = 1; + SET allow_database_glue_catalog = 1; + CREATE DATABASE {db_name} + ENGINE = DataLakeCatalog('{GLUE_BASE_URL}', '{minio_access_key}', '{minio_secret_key}') + SETTINGS catalog_type = 'glue', + warehouse = 'test', + storage_endpoint = 'http://minio:9000/warehouse-glue', + region = 'us-east-1' + """ + ) + + +def create_catalog_rmt(node, name: str, replica_name: str = "r1") -> None: + """Create an identity(region)-partitioned ReplicatedMergeTree source table.""" + make_rmt(node, name, "id Int64, region String", "region", + replica_name=replica_name, order_by="id") + + +def partition_id_for(node, table: str, region: str) -> str: + return node.query( + f"SELECT DISTINCT partition_id FROM system.parts" + f" WHERE table = '{table}' AND active AND partition = '{region}'" + f" FORMAT TabSeparated" + ).strip() + + +def create_catalog_iceberg_table(catalog, ns: str, tbl: str) -> None: + """ + Create a simple identity(region)-partitioned Iceberg table in the catalog. + Using format-version 2 and uncompressed metadata for test simplicity. + """ + catalog.create_table( + identifier=f"{ns}.{tbl}", + schema=Schema( + NestedField(field_id=1, name="id", field_type=LongType(), required=True), + NestedField(field_id=2, name="region", field_type=StringType(), required=True), + ), + location=f"s3://warehouse-glue/data/{tbl}", + partition_spec=PartitionSpec( + PartitionField( + source_id=2, + field_id=1000, + transform=IdentityTransform(), + name="region", + ) + ), + properties={ + "write.metadata.compression-codec": "none", + "write.format.default": "parquet", + "format-version": "2", + }, + ) + + +# --------------------------------------------------------------------------- +# Replicated catalog helpers +# --------------------------------------------------------------------------- + + +def setup_catalog_replicas(cluster, source_table: str, replica_names: list) -> None: + """ + Create RMT on each named replica (each with its own replica_name so they share + the same ZK path) and set up the DataLakeCatalog database on every node. + No data is inserted here — callers manage their own test data. + """ + for rname in replica_names: + create_catalog_rmt(cluster.instances[rname], source_table, replica_name=rname) + setup_ch_catalog_db(cluster.instances[rname]) + + +# --------------------------------------------------------------------------- +# Tests +# --------------------------------------------------------------------------- + + +def test_catalog_basic_export(catalog_export_cluster): + """ + Create a catalog-registered Iceberg table via pyiceberg, export one partition + from a ReplicatedMergeTree, and verify: + - The catalog (Glue) shows a new snapshot after the export. + - SELECT via the DataLakeCatalog database returns the correct row count. + + This test exercises the catalog commit path: + IcebergMetadata::commitImportPartitionTransactionImpl + → catalog->updateMetadata(namespace, table, new_metadata_file, snapshot) + """ + node = catalog_export_cluster.instances["node1"] + catalog = connect_catalog(catalog_export_cluster) + + ns = f"ns_basic_{uuid.uuid4().hex[:8]}" + tbl = f"tbl_basic_{uuid.uuid4().hex[:8]}" + source = f"rmt_basic_{uuid.uuid4().hex[:8]}" + + catalog.create_namespace((ns,)) + create_catalog_iceberg_table(catalog, ns, tbl) + setup_ch_catalog_db(node) + create_catalog_rmt(node, source) + + node.query(f"INSERT INTO {source} VALUES (1, 'EU'), (2, 'EU'), (3, 'EU')") + + pid = partition_id_for(node, source, "EU") + dest_ch = f"`{CH_CATALOG_DB}`.`{ns}.{tbl}`" + + node.query( + f"ALTER TABLE {source} EXPORT PARTITION ID '{pid}' TO TABLE {dest_ch}", + settings={"write_full_path_in_iceberg_metadata": 1}, + ) + wait_for_export_status(node, source, None, pid) + + count = int(node.query(f"SELECT count() FROM {dest_ch}").strip()) + assert count == 3, f"Expected 3 rows, got {count}" + + iceberg_tbl = catalog.load_table(f"{ns}.{tbl}") + assert iceberg_tbl.current_snapshot() is not None, \ + "Expected at least one snapshot in Glue after the export" + + +def test_catalog_concurrent_export(catalog_export_cluster): + """ + Export two partitions concurrently to the same catalog-backed Iceberg table. + + Both commits go through catalog->updateMetadata (Glue). Both commits must + ultimately succeed. + + Verifies: + - Total row count equals total inserted (no rows lost). + - The catalog history contains at least two snapshots (one per partition). + """ + node = catalog_export_cluster.instances["node1"] + catalog = connect_catalog(catalog_export_cluster) + + ns = f"ns_concurrent_{uuid.uuid4().hex[:8]}" + tbl = f"tbl_concurrent_{uuid.uuid4().hex[:8]}" + source = f"rmt_concurrent_{uuid.uuid4().hex[:8]}" + + catalog.create_namespace((ns,)) + create_catalog_iceberg_table(catalog, ns, tbl) + setup_ch_catalog_db(node) + create_catalog_rmt(node, source) + + node.query(f"INSERT INTO {source} VALUES (1, 'EU'), (2, 'EU'), (3, 'EU')") + node.query(f"INSERT INTO {source} VALUES (4, 'US'), (5, 'US'), (6, 'US')") + + pid_eu = partition_id_for(node, source, "EU") + pid_us = partition_id_for(node, source, "US") + dest_ch = f"`{CH_CATALOG_DB}`.`{ns}.{tbl}`" + + errors: list = [] + + def export_partition(pid: str) -> None: + try: + node.query( + f"ALTER TABLE {source} EXPORT PARTITION ID '{pid}' TO TABLE {dest_ch}", + settings={"write_full_path_in_iceberg_metadata": 1}, + ) + wait_for_export_status(node, source, None, pid, timeout=120) + except Exception as exc: + errors.append(exc) + + t1 = threading.Thread(target=export_partition, args=(pid_eu,)) + t2 = threading.Thread(target=export_partition, args=(pid_us,)) + t1.start() + t2.start() + t1.join() + t2.join() + + assert not errors, f"Export threads raised errors: {errors}" + + count = int(node.query(f"SELECT count() FROM {dest_ch}").strip()) + assert count == 6, f"Expected 6 rows (3 EU + 3 US), got {count}" + + iceberg_tbl = catalog.load_table(f"{ns}.{tbl}") + history = iceberg_tbl.history() + assert len(history) >= 2, ( + f"Expected ≥2 snapshots (one per concurrent partition commit), got {len(history)}" + ) + + +def test_catalog_idempotent_retry(catalog_export_cluster): + """ + Simulate a crash after the catalog commit but before ZooKeeper is updated to + COMPLETED (via the iceberg_export_after_commit_before_zk_completed failpoint). + + After restart the scheduler retries the PENDING task. + IcebergMetadata::commitExportPartitionTransaction finds the transaction_id already + embedded in a snapshot summary field (clickhouse.export-partition-transaction-id) + and returns without re-committing. + + Verifies: + - Exactly 3 rows in the Iceberg table (no duplicates from the re-commit). + - Exactly 1 snapshot in the Glue catalog (the idempotent retry was a no-op). + """ + node = catalog_export_cluster.instances["node1"] + catalog = connect_catalog(catalog_export_cluster) + + ns = f"ns_idempotent_{uuid.uuid4().hex[:8]}" + tbl = f"tbl_idempotent_{uuid.uuid4().hex[:8]}" + source = f"rmt_idempotent_{uuid.uuid4().hex[:8]}" + + catalog.create_namespace((ns,)) + create_catalog_iceberg_table(catalog, ns, tbl) + setup_ch_catalog_db(node) + create_catalog_rmt(node, source) + + node.query(f"INSERT INTO {source} VALUES (1, 'EU'), (2, 'EU'), (3, 'EU')") + + pid = partition_id_for(node, source, "EU") + dest_ch = f"`{CH_CATALOG_DB}`.`{ns}.{tbl}`" + + # Enable the ONCE failpoint: after a successful catalog commit the process + # calls std::terminate() before writing ZK COMPLETED — simulating a hard crash. + node.query("SYSTEM ENABLE FAILPOINT iceberg_export_after_commit_before_zk_completed") + node.query( + f"ALTER TABLE {source} EXPORT PARTITION ID '{pid}' TO TABLE {dest_ch}", + settings={"write_full_path_in_iceberg_metadata": 1}, + ) + + # Give the background scheduler time to export the data files and reach the + # failpoint. The crash is immediate (std::terminate), so 10 s is generous. + time.sleep(10) + node.restart_clickhouse() + + # ClickHouse persists database metadata to disk so the DataLakeCatalog database + # survives the crash. Recreate it anyway to make the test self-contained. + setup_ch_catalog_db(node) + + # The scheduler picks up the PENDING task and retries. commitExportPartitionTransaction + # detects the transaction_id in the existing snapshot summary and skips the + # re-commit, then marks the task COMPLETED in ZooKeeper. + wait_for_export_status(node, source, None, pid, timeout=120) + + count = int(node.query(f"SELECT count() FROM {dest_ch}").strip()) + assert count == 3, f"Expected 3 rows (no duplicates from idempotent retry), got {count}" + + iceberg_tbl = catalog.load_table(f"{ns}.{tbl}") + history = iceberg_tbl.history() + assert len(history) == 1, ( + f"Expected exactly 1 snapshot (idempotent re-commit was a no-op), " + f"got {len(history)}" + ) + + +# --------------------------------------------------------------------------- +# Replicated catalog tests +# --------------------------------------------------------------------------- + + +def test_catalog_export_two_replicas_basic(catalog_export_cluster): + """ + End-to-end: export one partition from replica1 in a 2-replica setup. + Export is initiated on replica1; row count is verified from replica2 via + the DataLakeCatalog database to confirm the catalog commit was visible. + """ + catalog = connect_catalog(catalog_export_cluster) + + ns = f"ns_two_replicas_{uuid.uuid4().hex[:8]}" + tbl = f"tbl_two_replicas_{uuid.uuid4().hex[:8]}" + source = f"rmt_two_replicas_{uuid.uuid4().hex[:8]}" + + catalog.create_namespace((ns,)) + create_catalog_iceberg_table(catalog, ns, tbl) + + setup_catalog_replicas(catalog_export_cluster, source, ["replica1", "replica2"]) + + r1 = catalog_export_cluster.instances["replica1"] + r2 = catalog_export_cluster.instances["replica2"] + + r1.query(f"INSERT INTO {source} VALUES (1, 'EU'), (2, 'EU'), (3, 'EU')") + r2.query(f"SYSTEM SYNC REPLICA {source}") + + pid = partition_id_for(r1, source, "EU") + dest_ch = f"`{CH_CATALOG_DB}`.`{ns}.{tbl}`" + + r1.query( + f"ALTER TABLE {source} EXPORT PARTITION ID '{pid}' TO TABLE {dest_ch}", + settings={"write_full_path_in_iceberg_metadata": 1}, + ) + wait_for_export_status(r1, source, None, pid) + + iceberg_tbl = catalog.load_table(f"{ns}.{tbl}") + assert iceberg_tbl.current_snapshot() is not None, \ + "Expected at least one snapshot in Glue after export" + + count = int(r2.query(f"SELECT count() FROM {dest_ch}").strip()) + assert count == 3, f"Expected 3 rows from replica2 via catalog, got {count}" + + +def test_catalog_concurrent_export_from_different_replicas(catalog_export_cluster): + """ + Two replicas concurrently export different partitions (EU / US) to the same + catalog-backed Iceberg table. Both catalog commits must succeed; total row count + must equal 6 and Glue history must contain at least 2 snapshots. + """ + catalog = connect_catalog(catalog_export_cluster) + + ns = f"ns_conc_replicas_{uuid.uuid4().hex[:8]}" + tbl = f"tbl_conc_replicas_{uuid.uuid4().hex[:8]}" + source = f"rmt_conc_replicas_{uuid.uuid4().hex[:8]}" + + catalog.create_namespace((ns,)) + create_catalog_iceberg_table(catalog, ns, tbl) + + setup_catalog_replicas(catalog_export_cluster, source, ["replica1", "replica2"]) + + r1 = catalog_export_cluster.instances["replica1"] + r2 = catalog_export_cluster.instances["replica2"] + + r1.query(f"INSERT INTO {source} VALUES (1, 'EU'), (2, 'EU'), (3, 'EU')") + r1.query(f"INSERT INTO {source} VALUES (4, 'US'), (5, 'US'), (6, 'US')") + r2.query(f"SYSTEM SYNC REPLICA {source}") + + pid_eu = partition_id_for(r1, source, "EU") + pid_us = partition_id_for(r1, source, "US") + dest_ch = f"`{CH_CATALOG_DB}`.`{ns}.{tbl}`" + + errors: list = [] + + def export_partition(node, pid): + try: + node.query( + f"ALTER TABLE {source} EXPORT PARTITION ID '{pid}' TO TABLE {dest_ch}", + settings={"write_full_path_in_iceberg_metadata": 1}, + ) + wait_for_export_status(node, source, None, pid, timeout=120) + except Exception as exc: + errors.append(exc) + + t1 = threading.Thread(target=export_partition, args=(r1, pid_eu)) + t2 = threading.Thread(target=export_partition, args=(r2, pid_us)) + t1.start() + t2.start() + t1.join() + t2.join() + + assert not errors, f"Export threads raised errors: {errors}" + + count = int(r1.query(f"SELECT count() FROM {dest_ch}").strip()) + assert count == 6, f"Expected 6 rows (3 EU + 3 US), got {count}" + + iceberg_tbl = catalog.load_table(f"{ns}.{tbl}") + history = iceberg_tbl.history() + assert len(history) >= 2, ( + f"Expected ≥2 snapshots (one per concurrent partition commit), got {len(history)}" + ) + + +# TODO arthur fix: TOCTOU in export registration path. +# The exists() pre-check and the tryMulti() commit are not a single atomic ZK +# transaction. Depending on timing, the loser gets either KEEPER_EXCEPTION +# "Node exists" (both replicas race past exists() and collide at tryMulti) or +# BAD_ARGUMENTS "already exported" (the winner commits before the loser's +# exists() check). The test cannot reliably assert either error in isolation. +# def test_catalog_idempotent_same_partition_two_replicas(catalog_export_cluster): +# catalog = connect_catalog(catalog_export_cluster) +# +# ns = f"ns_{uuid.uuid4().hex[:8]}" +# tbl = f"tbl_{uuid.uuid4().hex[:8]}" +# source = f"rmt_{uuid.uuid4().hex[:8]}" +# +# catalog.create_namespace((ns,)) +# create_catalog_iceberg_table(catalog, ns, tbl) +# +# setup_catalog_replicas(catalog_export_cluster, source, ["replica1", "replica2"]) +# +# r1 = catalog_export_cluster.instances["replica1"] +# r2 = catalog_export_cluster.instances["replica2"] +# +# r1.query(f"INSERT INTO {source} VALUES (1, 'EU'), (2, 'EU'), (3, 'EU')") +# r2.query(f"SYSTEM SYNC REPLICA {source}") +# +# pid = partition_id_for(r1, source, "EU") +# dest_ch = f"`{CH_CATALOG_DB}`.`{ns}.{tbl}`" +# +# errors: list = [] +# +# def export_from(node): +# try: +# node.query( +# f"ALTER TABLE {source} EXPORT PARTITION ID '{pid}' TO TABLE {dest_ch}", +# settings={"write_full_path_in_iceberg_metadata": 1}, +# ) +# wait_for_export_status(node, source, None, pid, timeout=120) +# except Exception as exc: +# errors.append(exc) +# +# t1 = threading.Thread(target=export_from, args=(r1,)) +# t2 = threading.Thread(target=export_from, args=(r2,)) +# t1.start() +# t2.start() +# t1.join() +# t2.join() +# +# unexpected = [e for e in errors if "already exported" not in str(e)] +# assert not unexpected, f"Unexpected export errors: {unexpected}" +# +# count = int(r1.query(f"SELECT count() FROM {dest_ch}").strip()) +# assert count == 3, f"Expected 3 rows (no duplication), got {count}" +# +# iceberg_tbl = catalog.load_table(f"{ns}.{tbl}") +# history = iceberg_tbl.history() +# assert len(history) == 1, ( +# f"Expected exactly 1 snapshot (one winner, one rejected by export-key guard), " +# f"got {len(history)}" +# ) From 02251da432b71916ee35e8f344f696a6b08fd47e Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Mon, 27 Apr 2026 11:08:33 +0200 Subject: [PATCH 13/15] Merge pull request #1631 from Altinity/arthurpassos-patch-11 Fix condition for using parquet metadata cache Source-PR: #1631 (https://github.com/Altinity/ClickHouse/pull/1631) --- src/Storages/ObjectStorage/StorageObjectStorageSource.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 150427a4758a..6b13e2847f16 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -703,7 +703,7 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade InputFormatPtr input_format; if (context_->getSettingsRef()[Setting::use_parquet_metadata_cache] && use_native_reader_v3 - && (object_info->getFileFormat().value_or(configuration->format) == "Parquet") + && (Poco::toLower(object_info->getFileFormat().value_or(configuration->format)) == "parquet") && !object_info->getObjectMetadata()->etag.empty()) { const std::optional object_with_metadata = object_info->relative_path_with_metadata; From 0d4824b65d1182f09527eff93f47464c1cf6171d Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Tue, 28 Apr 2026 21:59:10 +0200 Subject: [PATCH 14/15] Update SettingsChangesHistory.cpp --- src/Core/SettingsChangesHistory.cpp | 46 +++++------------------------ 1 file changed, 7 insertions(+), 39 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index bc3376dbda91..3458fa899dfb 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -111,10 +111,6 @@ const VersionToSettingsChangesMap & getSettingsChangesHistory() // {"iceberg_partition_timezone", "", "", "New setting."}, // {"s3_propagate_credentials_to_other_storages", false, false, "New setting"}, {"export_merge_tree_part_filename_pattern", "", "{part_name}_{checksum}", "New setting"}, - // {"use_parquet_metadata_cache", false, true, "Enables cache of parquet file metadata."}, - // {"input_format_parquet_use_metadata_cache", true, false, "Obsolete. No-op"}, // https://github.com/Altinity/ClickHouse/pull/586 - // {"object_storage_remote_initiator_cluster", "", "", "New setting."}, - // {"iceberg_metadata_staleness_ms", 0, 0, "New setting allowing using cached metadata version at READ operations to prevent fetching from remote catalog"}, {"use_parquet_metadata_cache", false, true, "Enables cache of parquet file metadata."}, {"input_format_parquet_use_metadata_cache", true, false, "Obsolete. No-op"}, // https://github.com/Altinity/ClickHouse/pull/586 {"object_storage_remote_initiator_cluster", "", "", "New setting."}, @@ -159,7 +155,6 @@ const VersionToSettingsChangesMap & getSettingsChangesHistory() {"throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert", true, false, "It becomes obsolete."}, {"database_datalake_require_metadata_access", true, true, "New setting."}, {"automatic_parallel_replicas_min_bytes_per_replica", 0, 1_MiB, "Better default value derived from testing results"}, - {"use_object_storage_list_objects_cache", false, false, "New setting."}, }); addSettingsChanges(settings_changes_history, "25.12", { @@ -311,7 +306,6 @@ const VersionToSettingsChangesMap & getSettingsChangesHistory() // {"allow_database_iceberg", false, true, "Turned ON by default for Antalya (alias)."}, // {"allow_database_unity_catalog", false, true, "Turned ON by default for Antalya (alias)."}, // {"allow_database_glue_catalog", false, true, "Turned ON by default for Antalya (alias)."}, - // {"input_format_parquet_use_metadata_cache", true, true, "New setting, turned ON by default"}, // https://github.com/Altinity/ClickHouse/pull/586 {"input_format_parquet_use_metadata_cache", true, true, "New setting, turned ON by default"}, // https://github.com/Altinity/ClickHouse/pull/586 // {"iceberg_timezone_for_timestamptz", "UTC", "UTC", "New setting."}, // {"object_storage_remote_initiator", false, false, "New setting."}, @@ -319,49 +313,23 @@ const VersionToSettingsChangesMap & getSettingsChangesHistory() // {"object_storage_cluster_join_mode", "allow", "allow", "New setting"}, // {"lock_object_storage_task_distribution_ms", 500, 500, "New setting."}, // {"allow_retries_in_cluster_requests", false, false, "New setting"}, - // {"allow_experimental_export_merge_tree_part", false, true, "Turned ON by default for Antalya."}, - // {"export_merge_tree_part_overwrite_file_if_exists", false, false, "New setting."}, - // {"export_merge_tree_partition_force_export", false, false, "New setting."}, - // {"export_merge_tree_partition_max_retries", 3, 3, "New setting."}, - // {"export_merge_tree_partition_manifest_ttl", 180, 180, "New setting."}, - // {"export_merge_tree_part_file_already_exists_policy", "skip", "skip", "New setting."}, - // {"hybrid_table_auto_cast_columns", true, true, "New setting to automatically cast Hybrid table columns when segments disagree on types. Default enabled."}, - // {"allow_experimental_hybrid_table", false, false, "Added new setting to allow the Hybrid table engine."}, - // {"enable_alias_marker", true, true, "New setting."}, - // {"export_merge_tree_part_max_bytes_per_file", 0, 0, "New setting."}, - // {"export_merge_tree_part_max_rows_per_file", 0, 0, "New setting."}, - // {"export_merge_tree_partition_lock_inside_the_task", false, false, "New setting."}, - // {"export_merge_tree_partition_system_table_prefer_remote_information", true, true, "New setting."}, - // {"export_merge_tree_part_throw_on_pending_mutations", true, true, "New setting."}, - // {"export_merge_tree_part_throw_on_pending_patch_parts", true, true, "New setting."}, - // {"object_storage_cluster", "", "", "Antalya: New setting"}, - // {"object_storage_max_nodes", 0, 0, "Antalya: New setting"}, - // {"allow_experimental_iceberg_read_optimization", true, true, "New setting."}, - // {"object_storage_cluster_join_mode", "allow", "allow", "New setting"}, - // {"lock_object_storage_task_distribution_ms", 500, 500, "Raised the value to 500 to avoid hoping tasks between executors."}, - // {"allow_retries_in_cluster_requests", false, false, "New setting"}, - // {"object_storage_remote_initiator", false, false, "New setting."}, {"allow_experimental_export_merge_tree_part", false, true, "Turned ON by default for Antalya."}, {"export_merge_tree_part_overwrite_file_if_exists", false, false, "New setting."}, {"export_merge_tree_partition_force_export", false, false, "New setting."}, {"export_merge_tree_partition_max_retries", 3, 3, "New setting."}, {"export_merge_tree_partition_manifest_ttl", 180, 180, "New setting."}, {"export_merge_tree_part_file_already_exists_policy", "skip", "skip", "New setting."}, - // {"iceberg_timezone_for_timestamptz", "UTC", "UTC", "New setting."}, - {"hybrid_table_auto_cast_columns", true, true, "New setting to automatically cast Hybrid table columns when segments disagree on types. Default enabled."}, - {"allow_experimental_hybrid_table", false, false, "Added new setting to allow the Hybrid table engine."}, - {"enable_alias_marker", true, true, "New setting."}, - // {"input_format_parquet_use_native_reader_v3", false, true, "Seems stable"}, - // {"input_format_parquet_verify_checksums", true, true, "New setting."}, - // {"output_format_parquet_write_checksums", false, true, "New setting."}, + // {"hybrid_table_auto_cast_columns", true, true, "New setting to automatically cast Hybrid table columns when segments disagree on types. Default enabled."}, + // {"allow_experimental_hybrid_table", false, false, "Added new setting to allow the Hybrid table engine."}, + // {"enable_alias_marker", true, true, "New setting."}, {"export_merge_tree_part_max_bytes_per_file", 0, 0, "New setting."}, {"export_merge_tree_part_max_rows_per_file", 0, 0, "New setting."}, {"export_merge_tree_partition_lock_inside_the_task", false, false, "New setting."}, {"export_merge_tree_partition_system_table_prefer_remote_information", true, true, "New setting."}, - // {"cluster_table_function_split_granularity", "file", "file", "New setting."}, - // {"cluster_table_function_buckets_batch_size", 0, 0, "New setting."}, - {"export_merge_tree_part_throw_on_pending_mutations", true, true, "New setting."}, - {"export_merge_tree_part_throw_on_pending_patch_parts", true, true, "New setting."}, + {"export_merge_tree_part_throw_on_pending_mutations", true, true, "New setting."}, + {"export_merge_tree_part_throw_on_pending_patch_parts", true, true, "New setting."}, + // {"object_storage_cluster", "", "", "Antalya: New setting"}, + // {"object_storage_max_nodes", 0, 0, "Antalya: New setting"}, }); addSettingsChanges(settings_changes_history, "25.8", { From d73bc4ff8c705e72c7812bfdc404a626fbc8c3e8 Mon Sep 17 00:00:00 2001 From: Andrey Zvonov <32552679+zvonand@users.noreply.github.com> Date: Wed, 29 Apr 2026 17:10:43 +0200 Subject: [PATCH 15/15] Update SettingsChangesHistory.cpp --- src/Core/SettingsChangesHistory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 3458fa899dfb..4f12e95f4628 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -113,7 +113,7 @@ const VersionToSettingsChangesMap & getSettingsChangesHistory() {"export_merge_tree_part_filename_pattern", "", "{part_name}_{checksum}", "New setting"}, {"use_parquet_metadata_cache", false, true, "Enables cache of parquet file metadata."}, {"input_format_parquet_use_metadata_cache", true, false, "Obsolete. No-op"}, // https://github.com/Altinity/ClickHouse/pull/586 - {"object_storage_remote_initiator_cluster", "", "", "New setting."}, + // {"object_storage_remote_initiator_cluster", "", "", "New setting."}, {"iceberg_metadata_staleness_ms", 0, 0, "New setting allowing using cached metadata version at READ operations to prevent fetching from remote catalog"}, {"export_merge_tree_partition_task_timeout_seconds", 0, 3600, "New setting to control the timeout for export partition tasks."}, });