Skip to content

Fix Parquet reader crash when min_bytes_for_seek=0#88784

Merged
al13n321 merged 5 commits intoClickHouse:masterfrom
anibilthare:master
Nov 3, 2025
Merged

Fix Parquet reader crash when min_bytes_for_seek=0#88784
al13n321 merged 5 commits intoClickHouse:masterfrom
anibilthare:master

Conversation

@anibilthare
Copy link
Copy Markdown
Contributor

@anibilthare anibilthare commented Oct 18, 2025

Changelog category (leave one):

  • Bug Fix (user-visible misbehavior in an official stable release)

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

Fixed a segmentation fault in the Parquet reader when input_format_parquet_local_file_min_bytes_for_seek is set to 0. Resolves #78456

Details

Fixes a segmentation fault in the Parquet reader that occurs when input_format_parquet_local_file_min_bytes_for_seek is set to 0.

Root Cause

When input_format_parquet_local_file_min_bytes_for_seek = 0, the prefetch iterator's loop condition fails causing the loop to never execute, prefetched_row_groups to remain empty, nextRowGroupReader() to return nullptr, and fetchBatch() to crash on chassert(row_group_batch.record_batch_reader).

Changes

  1. Fix prefetch loop condition (Line 1009) - Ensure at least one row group is always prefetched
  2. Add defensive null check (Lines 1080-1091) - Check if record_batch_reader is null before using it

Reproduction

SET max_parsing_threads = 1;
SET input_format_parquet_local_file_min_bytes_for_seek = 0;
SELECT * FROM file('any_file.parquet', 'Parquet');

Before fix: Segmentation fault
After fix: Works correctly

Resolves #78456

anibilthare and others added 2 commits October 18, 2025 22:59
- Fix prefetch loop to always fetch at least one row group
- Add defensive null check before using record_batch_reader
- Prevents segfault when input_format_parquet_local_file_min_bytes_for_seek=0
@CLAassistant
Copy link
Copy Markdown

CLAassistant commented Oct 18, 2025

CLA assistant check
All committers have signed the CLA.

@anibilthare
Copy link
Copy Markdown
Contributor Author

I used this script to check if the bug has been fixed.

#!/bin/bash

# Comprehensive test for Parquet reader bug
# Tests both BEFORE (should crash) and AFTER (should work) the fix

set -e

CLICKHOUSE="/Users/animesh/Desktop/Clickhouse/ClickHouse/build/programs/clickhouse"
PARQUET_FILE="/Users/animesh/Desktop/Clickhouse/user_files/userdata.parquet"

echo "=========================================="
echo "Parquet Bug Reproduction Test"
echo "=========================================="
echo ""

# Check if ClickHouse binary exists
if [ ! -f "$CLICKHOUSE" ]; then
    echo "ERROR: ClickHouse binary not found at: $CLICKHOUSE"
    echo "Please build ClickHouse first: cd ClickHouse && ./build.sh"
    exit 1
fi

# Check if Parquet file exists
if [ ! -f "$PARQUET_FILE" ]; then
    echo "WARNING: Parquet file not found. Creating a sample file..."
    mkdir -p "$(dirname "$PARQUET_FILE")"
    
    # Create a sample Parquet file
    $CLICKHOUSE local --query "
    CREATE TABLE temp (c0 Int32, c1 String, c2 Float64) ENGINE = Memory;
    INSERT INTO temp VALUES (1, 'test1', 1.5), (2, 'test2', 2.5), (3, 'test3', 3.5);
    INSERT INTO FUNCTION file('$PARQUET_FILE', 'Parquet') SELECT * FROM temp;
    "
    echo "Sample Parquet file created."
fi

echo ""
echo "File: $PARQUET_FILE"
echo "File size: $(ls -lh "$PARQUET_FILE" | awk '{print $5}')"
echo ""

echo "=========================================="
echo "TEST 1: Read with DEFAULT settings (should work)"
echo "=========================================="
echo ""

$CLICKHOUSE local --query "
SELECT 'Reading Parquet with default settings...' as status;
SELECT * FROM file('$PARQUET_FILE', 'Parquet') LIMIT 5;
SELECT 'Rows:', count(*) FROM file('$PARQUET_FILE', 'Parquet');
" 2>&1

echo ""
echo "✓ Test 1 PASSED: Default settings work"
echo ""

echo "=========================================="
echo "TEST 2: Read with BUG TRIGGER (min_bytes_for_seek = 0)"
echo "=========================================="
echo ""
echo "Settings:"
echo "  - max_parsing_threads = 1"
echo "  - input_format_parquet_local_file_min_bytes_for_seek = 0"
echo ""

set +e  # Don't exit on error for this test

$CLICKHOUSE local --query "
SET max_parsing_threads = 1;
SET input_format_parquet_local_file_min_bytes_for_seek = 0;
SELECT 'Attempting to read with min_bytes_for_seek = 0...' as status;
SELECT * FROM file('$PARQUET_FILE', 'Parquet') LIMIT 5;
SELECT 'Success! Rows:', count(*) FROM file('$PARQUET_FILE', 'Parquet');
" 2>&1

EXIT_CODE=$?

echo ""
if [ $EXIT_CODE -eq 0 ]; then
    echo "✓ Test 2 PASSED: Bug is FIXED! Reading works with min_bytes_for_seek = 0"
else
    echo "✗ Test 2 FAILED: Bug still exists (exit code: $EXIT_CODE)"
    echo ""
    echo "Expected error: \"Logical error: 'row_group_batch.record_batch_reader'\""
fi

echo ""
echo "=========================================="
echo "TEST 3: Insert/Select roundtrip with bug settings"
echo "=========================================="
echo ""

set -e  # Re-enable exit on error

$CLICKHOUSE local --query "
-- Create test data
CREATE TABLE test_source (id Int32, name String, value Float64) ENGINE = Memory;
INSERT INTO test_source VALUES (1, 'Alice', 100.5), (2, 'Bob', 200.75), (3, 'Charlie', 300.25);

-- Write to Parquet
INSERT INTO FUNCTION file('/tmp/test_parquet_bug.parquet', 'Parquet') SELECT * FROM test_source;

-- Read back with bug trigger settings
SET max_parsing_threads = 1;
SET input_format_parquet_local_file_min_bytes_for_seek = 0;

CREATE TABLE test_destination (id Int32, name String, value Float64) ENGINE = Memory;
INSERT INTO test_destination SELECT * FROM file('/tmp/test_parquet_bug.parquet', 'Parquet');

-- Verify
SELECT 'Source rows:', count(*) FROM test_source;
SELECT 'Destination rows:', count(*) FROM test_destination;
SELECT 'Data matches:', if(
    (SELECT sum(id) FROM test_source) = (SELECT sum(id) FROM test_destination) AND
    (SELECT sum(value) FROM test_source) = (SELECT sum(value) FROM test_destination),
    'YES ✓',
    'NO ✗'
) as result;

-- Show data
SELECT * FROM test_destination ORDER BY id;
" 2>&1

echo ""
echo "✓ Test 3 PASSED: Roundtrip works correctly"
echo ""

# Cleanup
rm -f /tmp/test_parquet_bug.parquet

echo "=========================================="
echo "ALL TESTS COMPLETED"
echo "=========================================="
echo ""
echo "Summary:"
echo "  1. Default settings: ✓"
echo "  2. Bug trigger (min_bytes_for_seek=0): Check output above"
echo "  3. Roundtrip test: ✓"
echo ""
echo "If Test 2 passed, the bug is FIXED!"
echo "If Test 2 failed with 'Logical error', the bug is PRESENT"
echo ""


@anibilthare
Copy link
Copy Markdown
Contributor Author

@al13n321 Could you please review this. Thanks.

Copy link
Copy Markdown
Member

@alexey-milovidov alexey-milovidov left a comment

Choose a reason for hiding this comment

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

Needs a test.

@alexey-milovidov
Copy link
Copy Markdown
Member

@al13n321
Copy link
Copy Markdown
Member

Please add a test in tests/queries/0_stateless/, similar to other parquet tests there, e.g. 02181_format_from_file_extension_local.sh.

Reproduces with just

insert into function file('t.parquet') select 1 as x settings engine_file_truncate_on_insert=1;
select * from file('t.parquet') settings input_format_parquet_local_file_min_bytes_for_seek=0, input_format_parquet_use_native_reader_v3=0, max_parsing_threads=1;

(but in the test please put the file in $CLICKHOUSE_TMP directory)

@anibilthare
Copy link
Copy Markdown
Contributor Author

Without these changes the test gives this output

animesh@Animeshs-MacBook-Air ~/Desktop/Clickhouse/ClickHouse $% CLICKHOUSE_BINARY=./build/programs/clickhouse bash tests/queries/0_stateless/03668_parquet_min_bytes_for_seek_zero.sh 2>&1
2025.10.21 13:08:35.571118 [ 31876160 ] {} <Fatal> ClientBase: ########## Short fault info ############
2025.10.21 13:08:35.571140 [ 31876160 ] {} <Fatal> ClientBase: (version 25.10.1.1, build id: <unknown>, git hash: 35a699a23595508d432ab1dabd9993dc20df7e38, architecture: arm64) (from thread 31876677) Received signal 11
2025.10.21 13:08:35.571142 [ 31876160 ] {} <Fatal> ClientBase: Signal description: Segmentation fault: 11
2025.10.21 13:08:35.571144 [ 31876160 ] {} <Fatal> ClientBase: Address: NULL pointer. Access: <not available on Darwin>. Attempted access has violated the permissions assigned to the memory area.
2025.10.21 13:08:35.571146 [ 31876160 ] {} <Fatal> ClientBase: Stack trace: 0x00000001087b9624 0x00000001087b38e0 0x000000018696d6a4 0x0000000110cc0a24 0x0000000110cc0a24 0x0000000110cbfcb0 0x0000000110cc0efc 0x0000000110b90060 0x0000000110b66328 0x0000000110b65ffc 0x0000000110b8191c 0x0000000110b85acc 0x0000000110b85740 0x0000000110b8afe4 0x000000010ffe465c 0x0000000110b66328 0x0000000110b65ffc 0x0000000110b8191c 0x0000000110b85acc 0x0000000110b889d4 0x000000010882b5d0 0x0000000108835a30 0x0000000108827d30 0x0000000108833908 0x0000000186933c0c 0x000000018692eb80
2025.10.21 13:08:35.571148 [ 31876160 ] {} <Fatal> ClientBase: ########################################
2025.10.21 13:08:35.571327 [ 31876160 ] {} <Fatal> ClientBase: (version 25.10.1.1, build id: <unknown>, git hash: 35a699a23595508d432ab1dabd9993dc20df7e38) (from thread 31876677) (query_id: 387c2971-6cbe-45a9-97b4-bdfa93f483c9) (query: select * from file('./03668_test.parquet') settings input_format_parquet_local_file_min_bytes_for_seek=0, input_format_parquet_use_native_reader_v3=0, max_parsing_threads=1;) Received signal Segmentation fault: 11 (11)
2025.10.21 13:08:35.571332 [ 31876160 ] {} <Fatal> ClientBase: Address: NULL pointer. Access: <not available on Darwin>. Attempted access has violated the permissions assigned to the memory area.
2025.10.21 13:08:35.571335 [ 31876160 ] {} <Fatal> ClientBase: Stack trace: 0x00000001087b9624 0x00000001087b38e0 0x000000018696d6a4 0x0000000110cc0a24 0x0000000110cc0a24 0x0000000110cbfcb0 0x0000000110cc0efc 0x0000000110b90060 0x0000000110b66328 0x0000000110b65ffc 0x0000000110b8191c 0x0000000110b85acc 0x0000000110b85740 0x0000000110b8afe4 0x000000010ffe465c 0x0000000110b66328 0x0000000110b65ffc 0x0000000110b8191c 0x0000000110b85acc 0x0000000110b889d4 0x000000010882b5d0 0x0000000108835a30 0x0000000108827d30 0x0000000108833908 0x0000000186933c0c 0x000000018692eb80
2025.10.21 13:08:35.608466 [ 31876160 ] {} <Fatal> ClientBase: 0. StackTrace::StackTrace(__darwin_ucontext const&) @ 0x00000001087b9624
2025.10.21 13:08:35.608499 [ 31876160 ] {} <Fatal> ClientBase: 1. signalHandler(int, __siginfo*, void*) @ 0x00000001087b38e0
2025.10.21 13:08:35.608507 [ 31876160 ] {} <Fatal> ClientBase: 2. _sigtramp @ 0x000000018696d6a4
2025.10.21 13:08:35.608517 [ 31876160 ] {} <Fatal> ClientBase: 3. DB::ParquetBlockInputFormat::decodeOneChunk(unsigned long, std::unique_lock<std::mutex>&)::$_3::operator()() const @ 0x0000000110cc0a24
2025.10.21 13:08:35.608525 [ 31876160 ] {} <Fatal> ClientBase: 4. DB::ParquetBlockInputFormat::decodeOneChunk(unsigned long, std::unique_lock<std::mutex>&)::$_3::operator()() const @ 0x0000000110cc0a24
2025.10.21 13:08:35.608533 [ 31876160 ] {} <Fatal> ClientBase: 5. DB::ParquetBlockInputFormat::decodeOneChunk(unsigned long, std::unique_lock<std::mutex>&) @ 0x0000000110cbfcb0
2025.10.21 13:08:35.608540 [ 31876160 ] {} <Fatal> ClientBase: 6. DB::ParquetBlockInputFormat::read() @ 0x0000000110cc0efc
2025.10.21 13:08:35.608547 [ 31876160 ] {} <Fatal> ClientBase: 7. DB::IInputFormat::generate() @ 0x0000000110b90060
2025.10.21 13:08:35.608553 [ 31876160 ] {} <Fatal> ClientBase: 8. DB::ISource::tryGenerate() @ 0x0000000110b66328
2025.10.21 13:08:35.608559 [ 31876160 ] {} <Fatal> ClientBase: 9. DB::ISource::work() @ 0x0000000110b65ffc
2025.10.21 13:08:35.608567 [ 31876160 ] {} <Fatal> ClientBase: 10. DB::ExecutionThreadContext::executeTask() @ 0x0000000110b8191c
2025.10.21 13:08:35.608576 [ 31876160 ] {} <Fatal> ClientBase: 11. DB::PipelineExecutor::executeStepImpl(unsigned long, DB::IAcquiredSlot*, std::atomic<bool>*) @ 0x0000000110b85acc
2025.10.21 13:08:35.608583 [ 31876160 ] {} <Fatal> ClientBase: 12. DB::PipelineExecutor::executeStep(std::atomic<bool>*) @ 0x0000000110b85740
2025.10.21 13:08:35.608592 [ 31876160 ] {} <Fatal> ClientBase: 13. DB::PullingPipelineExecutor::pull(DB::Chunk&) @ 0x0000000110b8afe4
2025.10.21 13:08:35.608599 [ 31876160 ] {} <Fatal> ClientBase: 14. DB::StorageFileSource::generate() @ 0x000000010ffe465c
2025.10.21 13:08:35.608605 [ 31876160 ] {} <Fatal> ClientBase: 15. DB::ISource::tryGenerate() @ 0x0000000110b66328
2025.10.21 13:08:35.608611 [ 31876160 ] {} <Fatal> ClientBase: 16. DB::ISource::work() @ 0x0000000110b65ffc
2025.10.21 13:08:35.608617 [ 31876160 ] {} <Fatal> ClientBase: 17. DB::ExecutionThreadContext::executeTask() @ 0x0000000110b8191c
2025.10.21 13:08:35.608623 [ 31876160 ] {} <Fatal> ClientBase: 18. DB::PipelineExecutor::executeStepImpl(unsigned long, DB::IAcquiredSlot*, std::atomic<bool>*) @ 0x0000000110b85acc
2025.10.21 13:08:35.608636 [ 31876160 ] {} <Fatal> ClientBase: 19. 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*) @ 0x0000000110b889d4
2025.10.21 13:08:35.608654 [ 31876160 ] {} <Fatal> ClientBase: 20. ThreadPoolImpl<ThreadFromGlobalPoolImpl<false, true>>::ThreadFromThreadPool::worker() @ 0x000000010882b5d0
2025.10.21 13:08:35.608667 [ 31876160 ] {} <Fatal> ClientBase: 21. 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()() @ 0x0000000108835a30
2025.10.21 13:08:35.608674 [ 31876160 ] {} <Fatal> ClientBase: 22. ThreadPoolImpl<std::thread>::ThreadFromThreadPool::worker() @ 0x0000000108827d30
2025.10.21 13:08:35.608682 [ 31876160 ] {} <Fatal> ClientBase: 23. 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*) @ 0x0000000108833908
2025.10.21 13:08:35.608689 [ 31876160 ] {} <Fatal> ClientBase: 24. _pthread_start @ 0x0000000186933c0c
2025.10.21 13:08:35.608696 [ 31876160 ] {} <Fatal> ClientBase: 25. thread_start @ 0x000000018692eb80
2025.10.21 13:08:35.608896 [ 31876160 ] {} <Fatal> ClientBase: Changed settings: max_parsing_threads = 1, allow_introspection_functions = true, serialize_query_plan = false, storage_file_read_method = 'mmap', implicit_select = true, input_format_parquet_use_native_reader_v3 = false, input_format_parquet_local_file_min_bytes_for_seek = 0
[Animeshs-MacBook-Air.local] 2025.10.21 13:08:35.571148 [ 31876160 ] <Fatal> ClientBase: ########################################
[Animeshs-MacBook-Air.local] 2025.10.21 13:08:35.571327 [ 31876160 ] <Fatal> ClientBase: (version 25.10.1.1, build id: <unknown>, git hash: 35a699a23595508d432ab1dabd9993dc20df7e38) (from thread 31876677) (query_id: 387c2971-6cbe-45a9-97b4-bdfa93f483c9) (query: select * from file('./03668_test.parquet') settings input_format_parquet_local_file_min_bytes_for_seek=0, input_format_parquet_use_native_reader_v3=0, max_parsing_threads=1;) Received signal Segmentation fault: 11 (11)
[Animeshs-MacBook-Air.local] 2025.10.21 13:08:35.571332 [ 31876160 ] <Fatal> ClientBase: Address: NULL pointer. Access: <not available on Darwin>. Attempted access has violated the permissions assigned to the memory area.
[Animeshs-MacBook-Air.local] 2025.10.21 13:08:35.571335 [ 31876160 ] <Fatal> ClientBase: Stack trace: 0x00000001087b9624 0x00000001087b38e0 0x000000018696d6a4 0x0000000110cc0a24 0x0000000110cc0a24 0x0000000110cbfcb0 0x0000000110cc0efc 0x0000000110b90060 0x0000000110b66328 0x0000000110b65ffc 0x0000000110b8191c 0x0000000110b85acc 0x0000000110b85740 0x0000000110b8afe4 0x000000010ffe465c 0x0000000110b66328 0x0000000110b65ffc 0x0000000110b8191c 0x0000000110b85acc 0x0000000110b889d4 0x000000010882b5d0 0x0000000108835a30 0x0000000108827d30 0x0000000108833908 0x0000000186933c0c 0x000000018692eb80
[Animeshs-MacBook-Air.local] 2025.10.21 13:08:35.608466 [ 31876160 ] <Fatal> ClientBase: 0. StackTrace::StackTrace(__darwin_ucontext const&) @ 0x00000001087b9624
[Animeshs-MacBook-Air.local] 2025.10.21 13:08:35.608499 [ 31876160 ] <Fatal> ClientBase: 1. signalHandler(int, __siginfo*, void*) @ 0x00000001087b38e0
[Animeshs-MacBook-Air.local] 2025.10.21 13:08:35.608507 [ 31876160 ] <Fatal> ClientBase: 2. _sigtramp @ 0x000000018696d6a4
[Animeshs-MacBook-Air.local] 2025.10.21 13:08:35.608517 [ 31876160 ] <Fatal> ClientBase: 3. DB::ParquetBlockInputFormat::decodeOneChunk(unsigned long, std::unique_lock<std::mutex>&)::$_3::operator()() const @ 0x0000000110cc0a24
[Animeshs-MacBook-Air.local] 2025.10.21 13:08:35.608525 [ 31876160 ] <Fatal> ClientBase: 4. DB::ParquetBlockInputFormat::decodeOneChunk(unsigned long, std::unique_lock<std::mutex>&)::$_3::operator()() const @ 0x0000000110cc0a24
[Animeshs-MacBook-Air.local] 2025.10.21 13:08:35.608533 [ 31876160 ] <Fatal> ClientBase: 5. DB::ParquetBlockInputFormat::decodeOneChunk(unsigned long, std::unique_lock<std::mutex>&) @ 0x0000000110cbfcb0
[Animeshs-MacBook-Air.local] 2025.10.21 13:08:35.608540 [ 31876160 ] <Fatal> ClientBase: 6. DB::ParquetBlockInputFormat::read() @ 0x0000000110cc0efc
[Animeshs-MacBook-Air.local] 2025.10.21 13:08:35.608547 [ 31876160 ] <Fatal> ClientBase: 7. DB::IInputFormat::generate() @ 0x0000000110b90060
[Animeshs-MacBook-Air.local] 2025.10.21 13:08:35.608553 [ 31876160 ] <Fatal> ClientBase: 8. DB::ISource::tryGenerate() @ 0x0000000110b66328
[Animeshs-MacBook-Air.local] 2025.10.21 13:08:35.608559 [ 31876160 ] <Fatal> ClientBase: 9. DB::ISource::work() @ 0x0000000110b65ffc
[Animeshs-MacBook-Air.local] 2025.10.21 13:08:35.608567 [ 31876160 ] <Fatal> ClientBase: 10. DB::ExecutionThreadContext::executeTask() @ 0x0000000110b8191c
[Animeshs-MacBook-Air.local] 2025.10.21 13:08:35.608576 [ 31876160 ] <Fatal> ClientBase: 11. DB::PipelineExecutor::executeStepImpl(unsigned long, DB::IAcquiredSlot*, std::atomic<bool>*) @ 0x0000000110b85acc
[Animeshs-MacBook-Air.local] 2025.10.21 13:08:35.608583 [ 31876160 ] <Fatal> ClientBase: 12. DB::PipelineExecutor::executeStep(std::atomic<bool>*) @ 0x0000000110b85740
[Animeshs-MacBook-Air.local] 2025.10.21 13:08:35.608592 [ 31876160 ] <Fatal> ClientBase: 13. DB::PullingPipelineExecutor::pull(DB::Chunk&) @ 0x0000000110b8afe4
[Animeshs-MacBook-Air.local] 2025.10.21 13:08:35.608599 [ 31876160 ] <Fatal> ClientBase: 14. DB::StorageFileSource::generate() @ 0x000000010ffe465c
[Animeshs-MacBook-Air.local] 2025.10.21 13:08:35.608605 [ 31876160 ] <Fatal> ClientBase: 15. DB::ISource::tryGenerate() @ 0x0000000110b66328
[Animeshs-MacBook-Air.local] 2025.10.21 13:08:35.608611 [ 31876160 ] <Fatal> ClientBase: 16. DB::ISource::work() @ 0x0000000110b65ffc
[Animeshs-MacBook-Air.local] 2025.10.21 13:08:35.608617 [ 31876160 ] <Fatal> ClientBase: 17. DB::ExecutionThreadContext::executeTask() @ 0x0000000110b8191c
[Animeshs-MacBook-Air.local] 2025.10.21 13:08:35.608623 [ 31876160 ] <Fatal> ClientBase: 18. DB::PipelineExecutor::executeStepImpl(unsigned long, DB::IAcquiredSlot*, std::atomic<bool>*) @ 0x0000000110b85acc
[Animeshs-MacBook-Air.local] 2025.10.21 13:08:35.608636 [ 31876160 ] <Fatal> ClientBase: 19. 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*) @ 0x0000000110b889d4
[Animeshs-MacBook-Air.local] 2025.10.21 13:08:35.608654 [ 31876160 ] <Fatal> ClientBase: 20. ThreadPoolImpl<ThreadFromGlobalPoolImpl<false, true>>::ThreadFromThreadPool::worker() @ 0x000000010882b5d0
[Animeshs-MacBook-Air.local] 2025.10.21 13:08:35.608667 [ 31876160 ] <Fatal> ClientBase: 21. 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()() @ 0x0000000108835a30
[Animeshs-MacBook-Air.local] 2025.10.21 13:08:35.608674 [ 31876160 ] <Fatal> ClientBase: 22. ThreadPoolImpl<std::thread>::ThreadFromThreadPool::worker() @ 0x0000000108827d30
[Animeshs-MacBook-Air.local] 2025.10.21 13:08:35.608682 [ 31876160 ] <Fatal> ClientBase: 23. 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*) @ 0x0000000108833908
[Animeshs-MacBook-Air.local] 2025.10.21 13:08:35.608689 [ 31876160 ] <Fatal> ClientBase: 24. _pthread_start @ 0x0000000186933c0c
[Animeshs-MacBook-Air.local] 2025.10.21 13:08:35.608696 [ 31876160 ] <Fatal> ClientBase: 25. thread_start @ 0x000000018692eb80
[Animeshs-MacBook-Air.local] 2025.10.21 13:08:35.608896 [ 31876160 ] <Fatal> ClientBase: Changed settings: max_parsing_threads = 1, allow_introspection_functions = true, serialize_query_plan = false, storage_file_read_method = 'mmap', implicit_select = true, input_format_parquet_use_native_reader_v3 = false, input_format_parquet_local_file_min_bytes_for_seek = 0
tests/queries/0_stateless/03668_parquet_min_bytes_for_seek_zero.sh: line 14: 23732 Segmentation fault: 11     $CLICKHOUSE_LOCAL -q "select * from file('$CLICKHOUSE_TMP/03668_test.parquet') settings input_format_parquet_local_file_min_bytes_for_seek=0, input_format_parquet_use_native_reader_v3=0, max_parsing_threads=1;"

With these changes

animesh@Animeshs-MacBook-Air ~/Desktop/Clickhouse/ClickHouse $% CLICKHOUSE_BINARY=./build/programs/clickhouse bash tests/queries/0_stateless/03668_parquet_min_bytes_for_seek_zero.sh 2>&1
1

@anibilthare
Copy link
Copy Markdown
Contributor Author

Hey @al13n321 thanks for the comments.
Could you please take a look now and let me know if this looks okay ?

@clickhouse-gh
Copy link
Copy Markdown
Contributor

clickhouse-gh bot commented Oct 28, 2025

Workflow [PR], commit [812fa0b]

Summary:

job_name test_name status info comment
Integration tests (arm_binary, distributed plan, 4/4) failure
test_accept_invalid_certificate/test.py::test_strict_reject_with_config FAIL cidb
Stress test (amd_msan) failure
Server died FAIL cidb
Hung check failed, possible deadlock found (see hung_check.log) FAIL cidb
Killed by signal (in clickhouse-server.log) FAIL cidb
Fatal message in clickhouse-server.log (see fatal_messages.txt) FAIL cidb
Killed by signal (output files) FAIL cidb
Found signal in gdb.log FAIL cidb
AST fuzzer (arm_asan) failure
Logical error: 'FUNCTION query tree node does not have a valid source node after running L2DistanceTransposedPartialReadsPass. Before: Nullable(Float64), after: Float64'. FAIL cidb

@clickhouse-gh clickhouse-gh bot added the pr-bugfix Pull request with bugfix, not backported by default label Oct 28, 2025
@al13n321 al13n321 added the can be tested Allows running workflows for external contributors label Oct 31, 2025
@al13n321 al13n321 removed the request for review from alexey-milovidov November 1, 2025 03:31
@@ -0,0 +1,2 @@
1

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Remove trailing blank line from reference file.
@anibilthare
Copy link
Copy Markdown
Contributor Author

Fixed. Thanks.

@al13n321 al13n321 enabled auto-merge November 3, 2025 19:02
@al13n321 al13n321 added this pull request to the merge queue Nov 3, 2025
Merged via the queue into ClickHouse:master with commit 31127f2 Nov 3, 2025
120 of 124 checks passed
@robot-ch-test-poll2 robot-ch-test-poll2 added the pr-synced-to-cloud The PR is synced to the cloud repo label Nov 3, 2025
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

can be tested Allows running workflows for external contributors pr-bugfix Pull request with bugfix, not backported by default pr-synced-to-cloud The PR is synced to the cloud repo

Projects

None yet

Development

Successfully merging this pull request may close these issues.

Logical error on Parquet reader

5 participants