Skip to content

Conversation

@arthurpassos
Copy link
Collaborator

@arthurpassos arthurpassos commented Jan 19, 2026

Closes #1321

Changelog category (leave one):

  • Improvement

Changelog entry (a user-readable short description of the changes that goes to CHANGELOG.md):

Accept table function as destination for part export, inherit schema if not explicitly provided. Example:

ALTER TABLE src_merge_tree EXPORT PART '2020_1_1_0' TO TABLE FUNCTION s3(s3_conn, filename='...', partition_strategy...) PARTITION BY ..

Uniquely identifies part exports by part name, it is no longer possible to export the same part simultaneously to different locations.

Documentation entry for user-facing changes

...

CI/CD Options

Exclude tests:

  • Fast test
  • Integration Tests
  • Stateless tests
  • Stateful tests
  • Performance tests
  • All with ASAN
  • All with TSAN
  • All with MSAN
  • All with UBSAN
  • All with Coverage
  • All with Aarch64
  • All Regression
  • Disable CI Cache

Regression jobs to run:

  • Fast suites (mostly <1h)
  • Aggregate Functions (2h)
  • Alter (1.5h)
  • Benchmark (30m)
  • ClickHouse Keeper (1h)
  • Iceberg (2h)
  • LDAP (1h)
  • Parquet (1.5h)
  • RBAC (1.5h)
  • SSL Server (1h)
  • S3 (2h)
  • Tiered Storage (2h)

@github-actions
Copy link

github-actions bot commented Jan 19, 2026

Workflow [PR], commit [57c1fb7]

Copy link

@chatgpt-codex-connector chatgpt-codex-connector bot left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

💡 Codex Review

Here are some automated review suggestions for this pull request.

Reviewed commit: 0c7f7c7c90

ℹ️ About Codex in GitHub

Your team has set up Codex to review pull requests in this repo. Reviews are triggered when you

  • Open a pull request for review
  • Mark a draft as ready
  • Comment "@codex review".

If Codex has suggestions, it will comment; otherwise it will react with 👍.

Codex can also answer questions or update the PR. Try commenting "@codex address that feedback".

Comment on lines +72 to 74
const auto & destination_storage = manifest.destination_storage_ptr;
const auto destination_storage_id = destination_storage->getStorageID();

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

P2 Badge Revalidate destination table before export task runs

The export task now uses the captured destination_storage_ptr directly without rechecking whether the destination table still exists or is dropped/detached. If the destination table is dropped after the ALTER EXPORT is queued but before this background task executes, destination_storage->import(...) will still run and can write data for a table that no longer exists (or has been recreated with a different definition). Previously the task re-resolved the destination from the catalog and aborted on UNKNOWN_TABLE; consider re-looking it up or at least checking is_dropped/locking before proceeding.

Useful? React with 👍 / 👎.

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is an interesting one. This is kind of an existing problem, not exclusive to this PR.

Plain object storage tables don't have state, and there is no trigger / event for when these tables are dropped. At the same time, it is safe (in terms of thread and memory safety) to write to S3 even after the table has been dropped.

Being that said, I think we have a few options:

  1. Document it and leave it as is, meaning that if a table gets deleted after the export process has started, it will continue writing to the S3 bucket.
  2. Somehow trigger an event when the table gets deleted, and cancel exports. Might leak abstractions
  3. Constantly check in the pipeline if the table remains alive

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In any case, I don't think this is a major issue. We probably have bigger fish to fry

@arthurpassos
Copy link
Collaborator Author

arthurpassos commented Jan 19, 2026

There is one interesting thing to validate on this one: exports are considered duplicate based on source table + part name + destination table name. I need to check what is the name used for table functions.

Edit1:

arthur :) alter table replicated_source export part '2020_0_0_0' to table function s3(s3_conn, filename='function', structure='id UInt32, year UInt16', format=Parquet, partition_strategy='hive') PARTITION BY year;

ALTER TABLE replicated_source
    (EXPORT PART '2020_0_0_0' TO TABLE FUNCTION s3(s3_conn, filename = 'function', structure = 'id UInt32, year UInt16', format = Parquet, partition_strategy = 'hive') PARTITION BY year)

Query id: 5ca93a65-5ab3-45b3-97c0-b0f87d10e060

Ok.

0 rows in set. Elapsed: 50.131 sec. 

arthur :) alter table replicated_source export part '2020_0_0_0' to table function s3(s3_conn, filename='function2', structure='id UInt32, year UInt16', format=Parquet, partition_strategy='hive') PARTITION BY year;

ALTER TABLE replicated_source
    (EXPORT PART '2020_0_0_0' TO TABLE FUNCTION s3(s3_conn, filename = 'function2', structure = 'id UInt32, year UInt16', format = Parquet, partition_strategy = 'hive') PARTITION BY year)

Query id: 89710dcd-0e8c-48c7-9c3a-f003a82b1dcb


Elapsed: 2.795 sec. 

Received exception from server (version 25.8.12):
Code: 236. DB::Exception: Received from localhost:9000. DB::Exception: Data part '2020_0_0_0' is already being exported to table '_table_function.s3'. (ABORTED)


Edit2:

Table function names, by default, are all the same as observed above.

As stated earlier, the list of manifests is unique based on part name + destination table name. But why? IIRC, the only reason is that we wanted to avoid colliding exports (i.e, exporting the same part to the same file path concurrently). According to the S3 docs (chatgpt, actually), the last upload will win in non-versioned buckets. So whoever sends the last PutObject or CompleteMultiPartUpload, wins.

This isn't a big problem I suppose, but it sounds like a cool protection to have. At the same time, it is not a very good protection as it is based on the table name, not the destination storage. For instance: if you have two s3 tables with different names pointing to the same location, it will export it.

The options we have from the top of my head:

  1. Drop this protection. Easiest, up to the user.
  2. Generate a unique id (query id) for each table function, this will allow multiple exports to table functions, but does not prevent the user from exporting the same part to the same location.
  3. Use a key based on the s3 path.

@arthurpassos
Copy link
Collaborator Author

arthurpassos commented Jan 19, 2026

There is one interesting thing to validate on this one: exports are considered duplicate based on source table + part name + destination table name. I need to check what is the name used for table functions.

Edit1:

arthur :) alter table replicated_source export part '2020_0_0_0' to table function s3(s3_conn, filename='function', structure='id UInt32, year UInt16', format=Parquet, partition_strategy='hive') PARTITION BY year;

ALTER TABLE replicated_source
    (EXPORT PART '2020_0_0_0' TO TABLE FUNCTION s3(s3_conn, filename = 'function', structure = 'id UInt32, year UInt16', format = Parquet, partition_strategy = 'hive') PARTITION BY year)

Query id: 5ca93a65-5ab3-45b3-97c0-b0f87d10e060

Ok.

0 rows in set. Elapsed: 50.131 sec. 

arthur :) alter table replicated_source export part '2020_0_0_0' to table function s3(s3_conn, filename='function2', structure='id UInt32, year UInt16', format=Parquet, partition_strategy='hive') PARTITION BY year;

ALTER TABLE replicated_source
    (EXPORT PART '2020_0_0_0' TO TABLE FUNCTION s3(s3_conn, filename = 'function2', structure = 'id UInt32, year UInt16', format = Parquet, partition_strategy = 'hive') PARTITION BY year)

Query id: 89710dcd-0e8c-48c7-9c3a-f003a82b1dcb


Elapsed: 2.795 sec. 

Received exception from server (version 25.8.12):
Code: 236. DB::Exception: Received from localhost:9000. DB::Exception: Data part '2020_0_0_0' is already being exported to table '_table_function.s3'. (ABORTED)

Edit2:

Table function names, by default, are all the same as observed above.

As stated earlier, the list of manifests is unique based on part name + destination table name. But why? IIRC, the only reason is that we wanted to avoid colliding exports (i.e, exporting the same part to the same file path concurrently). According to the S3 docs (chatgpt, actually), the last upload will win in non-versioned buckets. So whoever sends the last PutObject or CompleteMultiPartUpload, wins.

This isn't a big problem I suppose, but it sounds like a cool protection to have. At the same time, it is not a very good protection as it is based on the table name, not the destination storage. For instance: if you have two s3 tables with different names pointing to the same location, it will export it.

The options we have from the top of my head:

  1. Drop this protection. Easiest, up to the user.
  2. Generate a unique id (query id) for each table function, this will allow multiple exports to table functions, but does not prevent the user from exporting the same part to the same location.
  3. Use a key based on the s3 path.

Discussed with Misha and Dima, two options:

  1. Just use part name as the key, meaning a given part can't be exported to different locations at the same time.
  2. Use a key based on the s3 path (n3 above).

For now, I implemented n1.

@arthurpassos
Copy link
Collaborator Author

@codex review

Copy link

@chatgpt-codex-connector chatgpt-codex-connector bot left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

💡 Codex Review

Here are some automated review suggestions for this pull request.

Reviewed commit: d613c808e3

ℹ️ About Codex in GitHub

Your team has set up Codex to review pull requests in this repo. Reviews are triggered when you

  • Open a pull request for review
  • Mark a draft as ready
  • Comment "@codex review".

If Codex has suggestions, it will comment; otherwise it will react with 👍.

Codex can also answer questions or update the PR. Try commenting "@codex address that feedback".

Comment on lines 80 to 83
bool operator<(const MergeTreePartExportManifest & 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;
}

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

P2 Badge Include destination in export manifest ordering

Because MergeTreeData::exportPartToTable stores manifests in a std::set and relies on the comparator for uniqueness, comparing only data_part->name makes all exports of the same part equivalent even if the destination differs. This means concurrent exports of the same part to two different tables or table functions will now collide and the second will fail with “already being exported,” which is a regression from the previous per-destination behavior. Consider including the destination (e.g., storage ID or pointer) in operator</operator== so the set only deduplicates identical part+destination pairs.

Useful? React with 👍 / 👎.

@arthurpassos arthurpassos added antalya-25.8 port-antalya PRs to be ported to all new Antalya releases antalya labels Jan 20, 2026
@ilejn
Copy link
Collaborator

ilejn commented Jan 20, 2026

Hello @arthurpassos ,
am I right that not all table functions are suitable as a destination, and if such an incompatible function is used, NOT_IMPLEMENTED would be thrown?
If yes, it probably should be backed by a test and documented more verbosely.

Examples are executable(), url() and many others.

@arthurpassos
Copy link
Collaborator Author

arthurpassos commented Jan 20, 2026

Hello @arthurpassos , am I right that not all table functions are suitable as a destination, and if such an incompatible function is used, NOT_IMPLEMENTED would be thrown? If yes, it probably should be backed by a test and documented more verbosely.

That is true. The export code path will call IStorage::supportsImport on the destination storage. This method is only implemented for object storage, and the instance must match a few conditions.

I do have a one or two tests for table engines supported / not supported, but not for table functions like you asked. They should behave the same, that's why I did not add those. Regardless of that, it won't harm to add it :).

Edit:

Examples are executable(), url() and many others.

That's a fairly good point. I have no idea how it behaves when the destination table function is not a table engine.

Edit2:

Destination table function does not support:

arthur :) alter table mt_alias export part '2020_1_1_0' to table function executable('generate_random.py', TabSeparated, 'id UInt32, random String', (SELECT 10));

ALTER TABLE mt_alias
    (EXPORT PART '2020_1_1_0' TO TABLE FUNCTION executable('generate_random.py', TabSeparated, 'id UInt32, random String', (
        SELECT 10
    )))

Query id: ab0e41ba-d9dc-4ca3-b180-24908f64481d


Elapsed: 0.235 sec. 

Received exception from server (version 25.8.12):
Code: 48. DB::Exception: Received from localhost:9000. DB::Exception: Destination storage Executable does not support MergeTree parts or uses unsupported partitioning. (NOT_IMPLEMENTED)

Destination function is not a table function:

arthur :) alter table mt_alias export part '2020_1_1_0' to table function extractKeyValuePairs('name:neymar, age:31 team:psg,nationality:brazil');

ALTER TABLE mt_alias
    (EXPORT PART '2020_1_1_0' TO TABLE FUNCTION extractKeyValuePairs('name:neymar, age:31 team:psg,nationality:brazil'))

Query id: 3390f802-361d-48f6-b19f-ab069eebda63


Elapsed: 83.329 sec. 

Received exception from server (version 25.8.12):
Code: 46. DB::Exception: Received from localhost:9000. DB::Exception: Unknown table function extractKeyValuePairs. (UNKNOWN_FUNCTION)

I will add tests

@DimensionWieldr
Copy link
Collaborator

DimensionWieldr commented Jan 23, 2026

Verified using a TABLE FUNCTION as the destination in ALTER EXPORT PART. All tests pass.

Tests: https://github.com/Altinity/clickhouse-regression/blob/main/s3/tests/export_part/table_functions.py

  • explicit_schema: Verifies exporting parts to a table function with an explicit structure parameter. Confirms the export uses the provided schema and exported data matches the source.
  • schema_inheritance: Verifies exporting parts to a table function without a structure parameter, ensuring the schema is inherited from the source table. Confirms column names match and exported data matches the source.
  • same_part_simultaneous_export_error: Verifies that the same part cannot be exported simultaneously to different table function destinations. Confirms the first export succeeds and the second fails with an error.
  • multiple_parts: Verifies exporting multiple parts from a source table to a table function destination. Confirms all parts are exported correctly and both data and row counts match between source and destination.

@DimensionWieldr DimensionWieldr added the verified Verified by QA label Jan 23, 2026
mkmkme
mkmkme previously approved these changes Jan 23, 2026
Copy link
Collaborator

@mkmkme mkmkme left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Added a couple of nits (optional, cosmetics only), otherwise LGTM

mkmkme
mkmkme previously approved these changes Jan 23, 2026
mkmkme
mkmkme previously approved these changes Jan 26, 2026
@DimensionWieldr DimensionWieldr removed the verified Verified by QA label Jan 27, 2026
@DimensionWieldr
Copy link
Collaborator

@arthurpassos Build checks are failing from these errors:

/home/ubuntu/_work/ClickHouse/ClickHouse/src/Storages/MergeTree/MergeTreeData.cpp:6265:10: error: unused parameter 'allow_outdated_parts' [-Werror,-Wunused-parameter]
 6265 |     bool allow_outdated_parts,
      |          ^
/home/ubuntu/_work/ClickHouse/ClickHouse/src/Storages/MergeTree/MergeTreeData.cpp:6314:66: error: use of undeclared identifier 'allow_outdated_parts'
 6314 |     if (part->getState() == MergeTreeDataPartState::Outdated && !allow_outdated_parts)
      |                                                                  ^
2 errors generated.

This looks like a simple fix, probably.

@DimensionWieldr
Copy link
Collaborator

DimensionWieldr commented Jan 28, 2026

From the CI Workflow Run Report, new fails in this PR are from the following upstream tests:

Stateless tests (amd_msan, sequential, 2/2): 03271_s3_table_function_asterisk_glob

Stateless tests (arm_binary, sequential): 03604_export_merge_tree_partition

Stress test (amd_msan): various failures

For regression fails, swarms are flaky, not caused by this PR.
Export part fails are due to an outdated commit hash (the failing tests have been updated or no longer exist, so this PR is not the cause).

Install packages (amd_debug) fails might be a problem. This test in consistently failing on this PR (as well as 1328).

@arthurpassos
Copy link
Collaborator Author

arthurpassos commented Jan 28, 2026

Stateless tests (arm_binary, sequential): 03604_export_merge_tree_partition

I can not see that list due to an auth error, but just because it happened in the past it does not mean this PR is not causing it. I believe we are observing a crash in https://altinity-build-artifacts.s3.amazonaws.com/json.html?PR=1320&sha=8a0ed9e57ff863976ba5a26f4a515bc8a088fb34&name_0=PR&name_1=Stateless+tests+%28arm_binary%2C+sequential%29&name_2=Tests. I am not sure yet if this is related, but we need to look into it

Edit Unless the very same crash has happened in the past

@DimensionWieldr
Copy link
Collaborator

DimensionWieldr commented Jan 28, 2026

Details for fail Stateless tests (arm_binary, sequential): 03604_export_merge_tree_partition:
Server error crash log

2026.01.28 13:41:34.176182 [ 2606 ] {b9b09379-bc4d-405d-8b38-414c01ad8d9c} <Fatal> : Logical error: 'Deadlock in Parquet::ReadManager (single-threaded)'.
2026.01.28 13:41:34.198073 [ 2606 ] {b9b09379-bc4d-405d-8b38-414c01ad8d9c} <Fatal> : Stack trace (when copying this message, always include the lines below):

0. ./ci/tmp/build/./base/poco/Foundation/src/Exception.cpp:28: Poco::Exception::Exception(String const&, int) @ 0x000000001c867af8
1. ./ci/tmp/build/./src/Common/Exception.cpp:128: DB::Exception::Exception(DB::Exception::MessageMasked&&, int, bool) @ 0x000000000e67c06c
2. DB::Exception::Exception(String&&, int, String, bool) @ 0x0000000009121b18
3. DB::Exception::Exception(PreformattedMessage&&, int) @ 0x0000000009121728
4. DB::Exception::Exception<>(int, FormatStringHelperImpl<>) @ 0x000000000912b434
5. ./ci/tmp/build/./src/Processors/Formats/Impl/Parquet/ReadManager.cpp:908: DB::Parquet::ReadManager::read() @ 0x00000000179ac8cc
6. ./ci/tmp/build/./src/Processors/Formats/Impl/ParquetV3BlockInputFormat.cpp:119: DB::ParquetV3BlockInputFormat::read() @ 0x0000000017591b3c
7. ./ci/tmp/build/./src/Processors/Formats/IInputFormat.cpp:33: DB::IInputFormat::generate() @ 0x0000000017413320
8. ./ci/tmp/build/./src/Processors/ISource.cpp:144: DB::ISource::tryGenerate() @ 0x00000000173ef468
9. ./ci/tmp/build/./src/Processors/ISource.cpp:110: DB::ISource::work() @ 0x00000000173ef13c
10. ./ci/tmp/build/./src/Processors/Executors/ExecutionThreadContext.cpp:53: DB::ExecutionThreadContext::executeTask() @ 0x000000001740625c
11. ./ci/tmp/build/./src/Processors/Executors/PipelineExecutor.cpp:351: DB::PipelineExecutor::executeStepImpl(unsigned long, DB::IAcquiredSlot*, std::atomic<bool>*) @ 0x00000000173fad88
12. ./ci/tmp/build/./src/Processors/Executors/PipelineExecutor.cpp:179: DB::PipelineExecutor::executeStep(std::atomic<bool>*) @ 0x00000000173faa3c
13. ./ci/tmp/build/./src/Processors/Executors/PullingPipelineExecutor.cpp:59: DB::PullingPipelineExecutor::pull(DB::Chunk&) @ 0x000000001740e520
14. ./ci/tmp/build/./src/Storages/ObjectStorage/StorageObjectStorageSource.cpp:331: DB::StorageObjectStorageSource::generate() @ 0x00000000127dbf34
15. ./ci/tmp/build/./src/Processors/ISource.cpp:144: DB::ISource::tryGenerate() @ 0x00000000173ef468
16. ./ci/tmp/build/./src/Processors/ISource.cpp:110: DB::ISource::work() @ 0x00000000173ef13c
17. ./ci/tmp/build/./src/Processors/Executors/ExecutionThreadContext.cpp:53: DB::ExecutionThreadContext::executeTask() @ 0x000000001740625c
18. ./ci/tmp/build/./src/Processors/Executors/PipelineExecutor.cpp:351: DB::PipelineExecutor::executeStepImpl(unsigned long, DB::IAcquiredSlot*, std::atomic<bool>*) @ 0x00000000173fad88
19. ./ci/tmp/build/./src/Processors/Executors/PipelineExecutor.cpp:279: void std::__function::__policy_invoker<void ()>::__call_impl[abi:ne190107]<std::__function::__default_alloc_func<DB::PipelineExecutor::spawnThreads(std::shared_ptr<DB::IAcquiredSlot>)::$_0, void ()>>(std::__function::__policy_storage const*) @ 0x00000000173fc0d4
20. ./contrib/llvm-project/libcxx/include/__functional/function.h:716: ? @ 0x000000000e7a6c28
21. ./contrib/llvm-project/libcxx/include/__type_traits/invoke.h:117: ThreadFromGlobalPoolImpl<false, true>::ThreadFromGlobalPoolImpl<void (ThreadPoolImpl<ThreadFromGlobalPoolImpl<false, true>>::ThreadFromThreadPool::*)(), ThreadPoolImpl<ThreadFromGlobalPoolImpl<false, true>>::ThreadFromThreadPool*>(void (ThreadPoolImpl<ThreadFromGlobalPoolImpl<false, true>>::ThreadFromThreadPool::*&&)(), ThreadPoolImpl<ThreadFromGlobalPoolImpl<false, true>>::ThreadFromThreadPool*&&)::'lambda'()::operator()() @ 0x000000000e7ab3a0
22. ./contrib/llvm-project/libcxx/include/__functional/function.h:716: ? @ 0x000000000e7a4b6c
23. ./contrib/llvm-project/libcxx/include/__type_traits/invoke.h:117: void* std::__thread_proxy[abi:ne190107]<std::tuple<std::unique_ptr<std::__thread_struct, std::default_delete<std::__thread_struct>>, void (ThreadPoolImpl<std::thread>::ThreadFromThreadPool::*)(), ThreadPoolImpl<std::thread>::ThreadFromThreadPool*>>(void*) @ 0x000000000e7a9d48
24. ? @ 0x000000000007d5b8
25. ? @ 0x00000000000e5edc

Tests report

2026-01-28 21:42:00 Reason: server died 
2026-01-28 21:42:00 ConnectionRefusedError
2026-01-28 21:42:00 [Errno 111] Connection refused
2026-01-28 21:42:00   File "/home/ubuntu/_work/ClickHouse/ClickHouse/tests/clickhouse-test", line 2357, in run
2026-01-28 21:42:00     self._cleanup(result.status == TestStatus.OK)
2026-01-28 21:42:00 
2026-01-28 21:42:00   File "/home/ubuntu/_work/ClickHouse/ClickHouse/tests/clickhouse-test", line 2420, in _cleanup
2026-01-28 21:42:00     self._cleanup_database(args, timeout)
2026-01-28 21:42:00 
2026-01-28 21:42:00   File "/home/ubuntu/_work/ClickHouse/ClickHouse/tests/clickhouse-test", line 2455, in _cleanup_database
2026-01-28 21:42:00     clickhouse_execute(
2026-01-28 21:42:00 
2026-01-28 21:42:00   File "/home/ubuntu/_work/ClickHouse/ClickHouse/tests/clickhouse-test", line 620, in clickhouse_execute
2026-01-28 21:42:00     return clickhouse_execute_http(
2026-01-28 21:42:00 
2026-01-28 21:42:00   File "/home/ubuntu/_work/ClickHouse/ClickHouse/tests/clickhouse-test", line 601, in clickhouse_execute_http
2026-01-28 21:42:00     raise ex
2026-01-28 21:42:00 
2026-01-28 21:42:00   File "/home/ubuntu/_work/ClickHouse/ClickHouse/tests/clickhouse-test", line 590, in clickhouse_execute_http
2026-01-28 21:42:00     client.request(
2026-01-28 21:42:00 
2026-01-28 21:42:00   File "/usr/lib/python3.10/http/client.py", line 1283, in request
2026-01-28 21:42:00     self._send_request(method, url, body, headers, encode_chunked)
2026-01-28 21:42:00 
2026-01-28 21:42:00   File "/usr/lib/python3.10/http/client.py", line 1329, in _send_request
2026-01-28 21:42:00     self.endheaders(body, encode_chunked=encode_chunked)
2026-01-28 21:42:00 
2026-01-28 21:42:00   File "/usr/lib/python3.10/http/client.py", line 1278, in endheaders
2026-01-28 21:42:00     self._send_output(message_body, encode_chunked=encode_chunked)
2026-01-28 21:42:00 
2026-01-28 21:42:00   File "/usr/lib/python3.10/http/client.py", line 1038, in _send_output
2026-01-28 21:42:00     self.send(msg)
2026-01-28 21:42:00 
2026-01-28 21:42:00 Settings used in the test: --max_insert_threads 1 --group_by_two_level_threshold 839809 --group_by_two_level_threshold_bytes 6350302 --distributed_aggregation_memory_efficient 1 --fsync_metadata 0 --output_format_parallel_formatting 1 --input_format_parallel_parsing 0 --min_chunk_bytes_for_parallel_parsing 11235272 --max_read_buffer_size 596666 --prefer_localhost_replica 1 --max_block_size 75824 --max_joined_block_size_rows 38017 --max_threads 3 --optimize_append_index 1 --use_hedged_requests 1 --optimize_if_chain_to_multiif 1 --optimize_if_transform_strings_to_enum 1 --optimize_read_in_order 0 --optimize_or_like_chain 0 --optimize_substitute_columns 1 --enable_multiple_prewhere_read_steps 1 --read_in_order_two_level_merge_threshold 69 --optimize_aggregation_in_order 0 --aggregation_in_order_max_block_bytes 16057707 --use_uncompressed_cache 0 --min_bytes_to_use_direct_io 10737418240 --min_bytes_to_use_mmap_io 10737418240 --local_filesystem_read_method read --remote_filesystem_read_method threadpool --local_filesystem_read_prefetch 1 --filesystem_cache_segments_batch_size 100 --read_from_filesystem_cache_if_exists_otherwise_bypass_cache 1 --throw_on_error_from_cache_on_write_operations 1 --remote_filesystem_read_prefetch 0 --allow_prefetched_read_pool_for_remote_filesystem 1 --filesystem_prefetch_max_memory_usage 64Mi --filesystem_prefetches_limit 0 --filesystem_prefetch_min_bytes_for_single_read_task 1Mi --filesystem_prefetch_step_marks 50 --filesystem_prefetch_step_bytes 0 --compile_expressions 0 --compile_aggregate_expressions 1 --compile_sort_description 0 --merge_tree_coarse_index_granularity 14 --optimize_distinct_in_order 1 --max_bytes_before_remerge_sort 2855095770 --min_compress_block_size 1742417 --max_compress_block_size 257092 --merge_tree_compact_parts_min_granules_to_multibuffer_read 84 --optimize_sorting_by_input_stream_properties 0 --http_response_buffer_size 3157953 --http_wait_end_of_query False --enable_memory_bound_merging_of_aggregation_results 1 --min_count_to_compile_expression 0 --min_count_to_compile_aggregate_expression 0 --min_count_to_compile_sort_description 3 --session_timezone Europe/Amsterdam --use_page_cache_for_disks_without_file_cache False --page_cache_inject_eviction True --merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability 0.29 --prefer_external_sort_block_bytes 1 --cross_join_min_rows_to_compress 100000000 --cross_join_min_bytes_to_compress 1 --min_external_table_block_size_bytes 0 --max_parsing_threads 1 --optimize_functions_to_subcolumns 0 --parallel_replicas_local_plan 1 --query_plan_join_swap_table auto --enable_vertical_final 0 --optimize_extract_common_expressions 1 --use_async_executor_for_materialized_views 1 --use_query_condition_cache 1 --secondary_indices_enable_bulk_filtering 1 --use_skip_indexes_if_final 0 --input_format_parquet_use_native_reader_v3 1 --max_bytes_before_external_sort 0 --max_bytes_before_external_group_by 0 --max_bytes_ratio_before_external_sort 0.0 --max_bytes_ratio_before_external_group_by 0.2 --use_skip_indexes_if_final_exact_mode 0
2026-01-28 21:42:00 
2026-01-28 21:42:00 MergeTree settings used in test: --ratio_of_defaults_for_sparse_serialization 1.0 --prefer_fetch_merged_part_size_threshold 1 --vertical_merge_algorithm_min_rows_to_activate 556201 --vertical_merge_algorithm_min_columns_to_activate 2 --allow_vertical_merges_from_compact_to_wide_parts 0 --min_merge_bytes_to_use_direct_io 10737418240 --index_granularity_bytes 11625568 --merge_max_block_size 24561 --index_granularity 63231 --min_bytes_for_wide_part 0 --marks_compress_block_size 69843 --primary_key_compress_block_size 73717 --replace_long_file_name_to_hash 0 --max_file_name_length 96 --min_bytes_for_full_part_storage 536870912 --compact_parts_max_bytes_to_buffer 240139521 --compact_parts_max_granules_to_buffer 1 --compact_parts_merge_max_bytes_to_prefetch_part 26860086 --cache_populated_by_fetch 1 --concurrent_part_removal_threshold 12 --old_parts_lifetime 480 --prewarm_mark_cache 1 --use_const_adaptive_granularity 1 --enable_index_granularity_compression 0 --enable_block_number_column 0 --enable_block_offset_column 0 --use_primary_key_cache 0 --prewarm_primary_key_cache 0 --object_serialization_version v2 --object_shared_data_serialization_version advanced --object_shared_data_serialization_version_for_zero_level_parts advanced --object_shared_data_buckets_for_compact_part 13 --object_shared_data_buckets_for_wide_part 17 --dynamic_serialization_version v2
2026-01-28 21:42:00 
2026-01-28 21:42:00 Database: test_gsxbq5zu

We are unable to find an identical crash in the past because the list of fails is older than 30 days and old reports have been wiped, notably PR 1124.

Arthur is testing a backport to see if it fixes the issue.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

Accept table function as the destination for part export

7 participants