ClickHouse / ClickHouse

ClickHouse® is a real-time analytics DBMS
https://clickhouse.com
Apache License 2.0
37.49k stars 6.9k forks source link

Invalid Field get from type UInt64 to type UInt256 in `WindowTransform`/`NearestFieldTypeImpl` #66437

Closed tavplubix closed 3 months ago

tavplubix commented 3 months ago

https://s3.amazonaws.com/clickhouse-test-reports/66140/9b41fd0cf0a977d6a652527b3ea92634be5aceb5/ast_fuzzer__debug_.html

localhost.localdomain :) SELECT intDiv(number, 5), count(*) IGNORE NULLS OVER (ORDER BY toFloat32(number) ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND CURRENT ROW), leadInFrame(toNullable(toUInt256(1)), 1, 1) OVER (RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING), count(*) OVER (ORDER BY toFloat64(number) ASC NULLS FIRST RANGE BETWEEN 100 PRECEDING AND UNBOUNDED FOLLOWING) FROM numbers(7)

SELECT
    intDiv(number, 5),
    count(*) IGNORE NULLS OVER (ORDER BY toFloat32(number) ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND CURRENT ROW),
    leadInFrame(toNullable(toUInt256(1)), 1, 1) OVER (RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING),
    count(*) OVER (ORDER BY toFloat64(number) ASC NULLS FIRST RANGE BETWEEN 100 PRECEDING AND UNBOUNDED FOLLOWING)
FROM numbers(7)

Query id: 597a603e-1ea5-466b-9b9f-8eac6136407e

Connecting to localhost:9000 as user default.
Connected to ClickHouse server version 24.7.1.

[dell9510] 2024.07.12 14:35:14.827339 [ 222045 ] {597a603e-1ea5-466b-9b9f-8eac6136407e} <Fatal> : Logical error: 'Invalid Field get from type UInt64 to type UInt256'.
[dell9510] 2024.07.12 14:35:14.850146 [ 276586 ] <Fatal> BaseDaemon: ########################################
[dell9510] 2024.07.12 14:35:14.850610 [ 276586 ] <Fatal> BaseDaemon: (version 24.7.1.1, build id: BF9FF88CD91160B94D2C853AB72C7C174F2AE744, git hash: 27738747e0da63684ed1d831fd823e9966923409) (from thread 222045) (query_id: 597a603e-1ea5-466b-9b9f-8eac6136407e) (query: SELECT intDiv(number, 5), count(*) IGNORE NULLS OVER (ORDER BY toFloat32(number) ASC NULLS FIRST RANGE BETWEEN 1 PRECEDING AND CURRENT ROW), leadInFrame(toNullable(toUInt256(1)), 1, 1) OVER (RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING), count(*) OVER (ORDER BY toFloat64(number) ASC NULLS FIRST RANGE BETWEEN 100 PRECEDING AND UNBOUNDED FOLLOWING) FROM numbers(7)) Received signal Aborted (6)
[dell9510] 2024.07.12 14:35:14.850750 [ 276586 ] <Fatal> BaseDaemon: 
[dell9510] 2024.07.12 14:35:14.850823 [ 276586 ] <Fatal> BaseDaemon: Stack trace: 0x000000000c9efc88 0x000000000cc5c0ca 0x00007f08c39b8e20 0x00007f08c3a0c195 0x00007f08c39b8d70 0x00007f08c39a04c0 0x000000000c9c031c 0x000000000c9c0d2f 0x0000000006b33bbc 0x0000000006bed6bb 0x000000001295c20b 0x000000001283093e 0x0000000013a26e23 0x0000000013a15e4c 0x0000000013a161e3 0x0000000013a19392 0x0000000013716088 0x000000001370a743 0x00000000137090fa 0x0000000013708a36 0x0000000013719590 0x000000000ca6d855 0x000000000ca72479 0x00007f08c3a0a1cf 0x00007f08c3a8b6ec
[dell9510] 2024.07.12 14:35:15.001174 [ 276586 ] <Fatal> BaseDaemon: 0.0. inlined from /home/tavplubix/ch/ClickHouse/src/Common/StackTrace.cpp:349: StackTrace::tryCapture()
[dell9510] 2024.07.12 14:35:15.002015 [ 276586 ] <Fatal> BaseDaemon: 0. /home/tavplubix/ch/ClickHouse/src/Common/StackTrace.cpp:318: StackTrace::StackTrace(ucontext_t const&) @ 0x000000000c9efc88
[dell9510] 2024.07.12 14:35:15.156945 [ 276586 ] <Fatal> BaseDaemon: 1. /home/tavplubix/ch/ClickHouse/src/Common/SignalHandlers.cpp:0: signalHandler(int, siginfo_t*, void*) @ 0x000000000cc5c0ca
[dell9510] 2024.07.12 14:35:15.158114 [ 276586 ] <Fatal> BaseDaemon: 2. ? @ 0x00007f08c39b8e20
[dell9510] 2024.07.12 14:35:15.158276 [ 276586 ] <Fatal> BaseDaemon: 3. ? @ 0x00007f08c3a0c195
[dell9510] 2024.07.12 14:35:15.158406 [ 276586 ] <Fatal> BaseDaemon: 4. ? @ 0x00007f08c39b8d70
[dell9510] 2024.07.12 14:35:15.158531 [ 276586 ] <Fatal> BaseDaemon: 5. ? @ 0x00007f08c39a04c0
[dell9510] 2024.07.12 14:35:15.282082 [ 276586 ] <Fatal> BaseDaemon: 6. /home/tavplubix/ch/ClickHouse/src/Common/Exception.cpp:0: DB::abortOnFailedAssertion(String const&) @ 0x000000000c9c031c
[dell9510] 2024.07.12 14:35:15.361975 [ 276586 ] <Fatal> BaseDaemon: 7. /home/tavplubix/ch/ClickHouse/src/Common/Exception.cpp:0: DB::Exception::Exception(DB::Exception::MessageMasked&&, int, bool) @ 0x000000000c9c0d2f
[dell9510] 2024.07.12 14:35:15.412330 [ 276586 ] <Fatal> BaseDaemon: 8.0. inlined from /home/tavplubix/ch/ClickHouse/contrib/llvm-project/libcxx/include/string:1499: String::__is_long[abi:v15007]() const
[dell9510] 2024.07.12 14:35:15.413735 [ 276586 ] <Fatal> BaseDaemon: 8.1. inlined from /home/tavplubix/ch/ClickHouse/contrib/llvm-project/libcxx/include/string:2333: ~basic_string
[dell9510] 2024.07.12 14:35:15.413972 [ 276586 ] <Fatal> BaseDaemon: 8.2. inlined from /home/tavplubix/ch/ClickHouse/src/Common/Exception.h:83: ~MessageMasked
[dell9510] 2024.07.12 14:35:15.414152 [ 276586 ] <Fatal> BaseDaemon: 8.3. inlined from /home/tavplubix/ch/ClickHouse/src/Common/Exception.h:95: Exception
[dell9510] 2024.07.12 14:35:15.414319 [ 276586 ] <Fatal> BaseDaemon: 8. /home/tavplubix/ch/ClickHouse/src/Common/Exception.h:66: DB::Exception::Exception(PreformattedMessage&&, int) @ 0x0000000006b33bbc
[dell9510] 2024.07.12 14:35:15.532264 [ 276586 ] <Fatal> BaseDaemon: 9.0. inlined from /home/tavplubix/ch/ClickHouse/contrib/llvm-project/libcxx/include/vector:438: std::vector<String, std::allocator<String>>::__destroy_vector::operator()[abi:v15007]()
[dell9510] 2024.07.12 14:35:15.532648 [ 276586 ] <Fatal> BaseDaemon: 9.1. inlined from /home/tavplubix/ch/ClickHouse/contrib/llvm-project/libcxx/include/vector:449: ~vector
[dell9510] 2024.07.12 14:35:15.532840 [ 276586 ] <Fatal> BaseDaemon: 9.2. inlined from /home/tavplubix/ch/ClickHouse/src/Common/LoggingFormatStringHelpers.h:45: ~PreformattedMessage
[dell9510] 2024.07.12 14:35:15.532988 [ 276586 ] <Fatal> BaseDaemon: 9. /home/tavplubix/ch/ClickHouse/src/Common/Exception.h:113: DB::Exception::Exception<DB::Field::Types::Which&, DB::Field::Types::Which const&>(int, FormatStringHelperImpl<std::type_identity<DB::Field::Types::Which&>::type, std::type_identity<DB::Field::Types::Which const&>::type>, DB::Field::Types::Which&, DB::Field::Types::Which const&) @ 0x0000000006bed6bb
[dell9510] 2024.07.12 14:35:16.325570 [ 276586 ] <Fatal> BaseDaemon: 10.0. inlined from /home/tavplubix/ch/ClickHouse/src/Core/Field.h:886: DB::NearestFieldTypeImpl<std::decay<wide::integer<256ul, unsigned int>>::type, void>::Type& DB::Field::get<wide::integer<256ul, unsigned int>>()
[dell9510] 2024.07.12 14:35:16.325754 [ 276586 ] <Fatal> BaseDaemon: 10.1. inlined from /home/tavplubix/ch/ClickHouse/src/Core/Field.h:467: auto const& DB::Field::get<wide::integer<256ul, unsigned int>>() const
[dell9510] 2024.07.12 14:35:16.325907 [ 276586 ] <Fatal> BaseDaemon: 10. /home/tavplubix/ch/ClickHouse/src/Columns/ColumnVector.h:226: DB::ColumnVector<wide::integer<256ul, unsigned int>>::insert(DB::Field const&) @ 0x000000001295c20b
[dell9510] 2024.07.12 14:35:16.403736 [ 276586 ] <Fatal> BaseDaemon: 11.0. inlined from /home/tavplubix/ch/ClickHouse/contrib/boost/boost/smart_ptr/intrusive_ptr.hpp:201: boost::intrusive_ptr<DB::IColumn const>::operator->() const
[dell9510] 2024.07.12 14:35:16.404152 [ 276586 ] <Fatal> BaseDaemon: 11.1. inlined from /home/tavplubix/ch/ClickHouse/src/Common/COW.h:214: COW<DB::IColumn>::chameleon_ptr<DB::IColumn>::operator*()
[dell9510] 2024.07.12 14:35:16.404302 [ 276586 ] <Fatal> BaseDaemon: 11.2. inlined from /home/tavplubix/ch/ClickHouse/src/Columns/ColumnNullable.h:172: DB::ColumnNullable::getNullMapColumn()
[dell9510] 2024.07.12 14:35:16.404415 [ 276586 ] <Fatal> BaseDaemon: 11.3. inlined from /home/tavplubix/ch/ClickHouse/src/Columns/ColumnNullable.h:175: DB::ColumnNullable::getNullMapData()
[dell9510] 2024.07.12 14:35:16.404703 [ 276586 ] <Fatal> BaseDaemon: 11. /home/tavplubix/ch/ClickHouse/src/Columns/ColumnNullable.cpp:241: DB::ColumnNullable::insert(DB::Field const&) @ 0x000000001283093e
[dell9510] 2024.07.12 14:35:16.922042 [ 276586 ] <Fatal> BaseDaemon: 12. /home/tavplubix/ch/ClickHouse/src/Processors/Transforms/WindowTransform.cpp:0: DB::WindowFunctionLagLeadInFrame<true>::windowInsertResultInto(DB::WindowTransform const*, unsigned long) const @ 0x0000000013a26e23
[dell9510] 2024.07.12 14:35:17.112355 [ 276586 ] <Fatal> BaseDaemon: 13. /home/tavplubix/ch/ClickHouse/src/Processors/Transforms/WindowTransform.cpp:1066: DB::WindowTransform::writeOutCurrentRow() @ 0x0000000013a15e4c
[dell9510] 2024.07.12 14:35:17.283360 [ 276586 ] <Fatal> BaseDaemon: 14.0. inlined from /home/tavplubix/ch/ClickHouse/contrib/llvm-project/libcxx/include/atomic:958: bool std::__cxx_atomic_load[abi:v15007]<bool>(std::__cxx_atomic_base_impl<bool> const*, std::memory_order)
[dell9510] 2024.07.12 14:35:17.284760 [ 276586 ] <Fatal> BaseDaemon: 14.1. inlined from /home/tavplubix/ch/ClickHouse/contrib/llvm-project/libcxx/include/atomic:1560: std::__atomic_base<bool, false>::load[abi:v15007](std::memory_order) const
[dell9510] 2024.07.12 14:35:17.284966 [ 276586 ] <Fatal> BaseDaemon: 14.2. inlined from /home/tavplubix/ch/ClickHouse/src/Processors/IProcessor.h:240: DB::IProcessor::isCancelled() const
[dell9510] 2024.07.12 14:35:17.285121 [ 276586 ] <Fatal> BaseDaemon: 14. /home/tavplubix/ch/ClickHouse/src/Processors/Transforms/WindowTransform.cpp:1258: DB::WindowTransform::appendChunk(DB::Chunk&) @ 0x0000000013a161e3
[dell9510] 2024.07.12 14:35:17.450836 [ 276586 ] <Fatal> BaseDaemon: 15.0. inlined from /home/tavplubix/ch/ClickHouse/src/Processors/Transforms/WindowTransform.h:63: DB::RowNumber::operator<=>(DB::RowNumber const&) const
[dell9510] 2024.07.12 14:35:17.451257 [ 276586 ] <Fatal> BaseDaemon: 15. /home/tavplubix/ch/ClickHouse/src/Processors/Transforms/WindowTransform.cpp:1498: DB::WindowTransform::work() @ 0x0000000013a19392
[dell9510] 2024.07.12 14:35:17.489557 [ 276586 ] <Fatal> BaseDaemon: 16.0. inlined from /home/tavplubix/ch/ClickHouse/src/Processors/Executors/ExecutionThreadContext.cpp:0: DB::executeJob(DB::ExecutingGraph::Node*, DB::ReadProgressCallback*)
[dell9510] 2024.07.12 14:35:17.491230 [ 276586 ] <Fatal> BaseDaemon: 16. /home/tavplubix/ch/ClickHouse/src/Processors/Executors/ExecutionThreadContext.cpp:96: DB::ExecutionThreadContext::executeTask() @ 0x0000000013716088
[dell9510] 2024.07.12 14:35:17.572019 [ 276586 ] <Fatal> BaseDaemon: 17.0. inlined from /home/tavplubix/ch/ClickHouse/src/Processors/Executors/PipelineExecutor.cpp:272: DB::PipelineExecutor::executeStepImpl(unsigned long, std::atomic<bool>*)
[dell9510] 2024.07.12 14:35:17.572361 [ 276586 ] <Fatal> BaseDaemon: 17. /home/tavplubix/ch/ClickHouse/src/Processors/Executors/PipelineExecutor.cpp:238: DB::PipelineExecutor::executeSingleThread(unsigned long) @ 0x000000001370a743
[dell9510] 2024.07.12 14:35:17.625059 [ 276586 ] <Fatal> BaseDaemon: 18.0. inlined from /home/tavplubix/ch/ClickHouse/contrib/llvm-project/libcxx/include/__memory/shared_ptr.h:701: ~shared_ptr
[dell9510] 2024.07.12 14:35:17.625402 [ 276586 ] <Fatal> BaseDaemon: 18. /home/tavplubix/ch/ClickHouse/src/Processors/Executors/PipelineExecutor.cpp:411: DB::PipelineExecutor::executeImpl(unsigned long, bool) @ 0x00000000137090fa
[dell9510] 2024.07.12 14:35:17.689405 [ 276586 ] <Fatal> BaseDaemon: 19.0. inlined from /home/tavplubix/ch/ClickHouse/contrib/llvm-project/libcxx/include/__memory/unique_ptr.h:274: std::unique_ptr<DB::ExecutingGraph, std::default_delete<DB::ExecutingGraph>>::operator->[abi:v15007]() const
[dell9510] 2024.07.12 14:35:17.692891 [ 276586 ] <Fatal> BaseDaemon: 19. /home/tavplubix/ch/ClickHouse/src/Processors/Executors/PipelineExecutor.cpp:113: DB::PipelineExecutor::execute(unsigned long, bool) @ 0x0000000013708a36
[dell9510] 2024.07.12 14:35:17.760924 [ 276586 ] <Fatal> BaseDaemon: 20.0. inlined from /home/tavplubix/ch/ClickHouse/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp:0: DB::threadFunction(DB::PullingAsyncPipelineExecutor::Data&, std::shared_ptr<DB::ThreadGroup>, unsigned long, bool)
[dell9510] 2024.07.12 14:35:17.761551 [ 276586 ] <Fatal> BaseDaemon: 20.1. inlined from /home/tavplubix/ch/ClickHouse/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp:109: operator()
[dell9510] 2024.07.12 14:35:17.761698 [ 276586 ] <Fatal> BaseDaemon: 20.2. inlined from /home/tavplubix/ch/ClickHouse/contrib/llvm-project/libcxx/include/__functional/invoke.h:394: ?
[dell9510] 2024.07.12 14:35:17.761807 [ 276586 ] <Fatal> BaseDaemon: 20.3. inlined from /home/tavplubix/ch/ClickHouse/contrib/llvm-project/libcxx/include/tuple:1789: decltype(auto) std::__apply_tuple_impl[abi:v15007]<DB::PullingAsyncPipelineExecutor::pull(DB::Chunk&, unsigned long)::$_0&, std::tuple<>&>(DB::PullingAsyncPipelineExecutor::pull(DB::Chunk&, unsigned long)::$_0&, std::tuple<>&, std::__tuple_indices<>)
[dell9510] 2024.07.12 14:35:17.761922 [ 276586 ] <Fatal> BaseDaemon: 20.4. inlined from /home/tavplubix/ch/ClickHouse/contrib/llvm-project/libcxx/include/tuple:1798: decltype(auto) std::apply[abi:v15007]<DB::PullingAsyncPipelineExecutor::pull(DB::Chunk&, unsigned long)::$_0&, std::tuple<>&>(DB::PullingAsyncPipelineExecutor::pull(DB::Chunk&, unsigned long)::$_0&, std::tuple<>&)
[dell9510] 2024.07.12 14:35:17.762006 [ 276586 ] <Fatal> BaseDaemon: 20.5. inlined from /home/tavplubix/ch/ClickHouse/src/Common/ThreadPool.h:251: operator()
[dell9510] 2024.07.12 14:35:17.762107 [ 276586 ] <Fatal> BaseDaemon: 20.6. inlined from /home/tavplubix/ch/ClickHouse/contrib/llvm-project/libcxx/include/__functional/invoke.h:394: ?
[dell9510] 2024.07.12 14:35:17.762196 [ 276586 ] <Fatal> BaseDaemon: 20.7. inlined from /home/tavplubix/ch/ClickHouse/contrib/llvm-project/libcxx/include/__functional/invoke.h:479: ?
[dell9510] 2024.07.12 14:35:17.762287 [ 276586 ] <Fatal> BaseDaemon: 20.8. inlined from /home/tavplubix/ch/ClickHouse/contrib/llvm-project/libcxx/include/__functional/function.h:235: ?
[dell9510] 2024.07.12 14:35:17.762379 [ 276586 ] <Fatal> BaseDaemon: 20. /home/tavplubix/ch/ClickHouse/contrib/llvm-project/libcxx/include/__functional/function.h:716: ? @ 0x0000000013719590
[dell9510] 2024.07.12 14:35:17.810054 [ 276586 ] <Fatal> BaseDaemon: 21.0. inlined from /home/tavplubix/ch/ClickHouse/base/base/../base/strong_typedef.h:23: StrongTypedef<std::integral_constant<bool, true> >
[dell9510] 2024.07.12 14:35:17.810339 [ 276586 ] <Fatal> BaseDaemon: 21.1. inlined from /home/tavplubix/ch/ClickHouse/src/Common/OpenTelemetryTraceContext.h:65: DB::OpenTelemetry::Span::isTraceEnabled() const
[dell9510] 2024.07.12 14:35:17.810480 [ 276586 ] <Fatal> BaseDaemon: 21. /home/tavplubix/ch/ClickHouse/src/Common/ThreadPool.cpp:464: ThreadPoolImpl<std::thread>::worker(std::__list_iterator<std::thread, void*>) @ 0x000000000ca6d855
[dell9510] 2024.07.12 14:35:17.885993 [ 276586 ] <Fatal> BaseDaemon: 22.0. inlined from /home/tavplubix/ch/ClickHouse/contrib/llvm-project/libcxx/include/__memory/unique_ptr.h:302: std::unique_ptr<std::tuple<std::unique_ptr<std::__thread_struct, std::default_delete<std::__thread_struct>>, void ThreadPoolImpl<std::thread>::scheduleImpl<void>(std::function<void ()>, Priority, std::optional<unsigned long>, bool)::'lambda0'()>, std::default_delete<std::tuple<std::unique_ptr<std::__thread_struct, std::default_delete<std::__thread_struct>>, void ThreadPoolImpl<std::thread>::scheduleImpl<void>(std::function<void ()>, Priority, std::optional<unsigned long>, bool)::'lambda0'()>>>::reset[abi:v15007](std::tuple<std::unique_ptr<std::__thread_struct, std::default_delete<std::__thread_struct>>, void ThreadPoolImpl<std::thread>::scheduleImpl<void>(std::function<void ()>, Priority, std::optional<unsigned long>, bool)::'lambda0'()>*)
[dell9510] 2024.07.12 14:35:17.886371 [ 276586 ] <Fatal> BaseDaemon: 22.1. inlined from /home/tavplubix/ch/ClickHouse/contrib/llvm-project/libcxx/include/__memory/unique_ptr.h:259: ~unique_ptr
[dell9510] 2024.07.12 14:35:17.886565 [ 276586 ] <Fatal> BaseDaemon: 22. /home/tavplubix/ch/ClickHouse/contrib/llvm-project/libcxx/include/thread:297: void* std::__thread_proxy[abi:v15007]<std::tuple<std::unique_ptr<std::__thread_struct, std::default_delete<std::__thread_struct>>, void ThreadPoolImpl<std::thread>::scheduleImpl<void>(std::function<void ()>, Priority, std::optional<unsigned long>, bool)::'lambda0'()>>(void*) @ 0x000000000ca72479
[dell9510] 2024.07.12 14:35:17.886750 [ 276586 ] <Fatal> BaseDaemon: 23. ? @ 0x00007f08c3a0a1cf
[dell9510] 2024.07.12 14:35:17.886918 [ 276586 ] <Fatal> BaseDaemon: 24. ? @ 0x00007f08c3a8b6ec
[dell9510] 2024.07.12 14:35:17.887070 [ 276586 ] <Fatal> BaseDaemon: Integrity check of the executable skipped because the reference checksum could not be read.
[dell9510] 2024.07.12 14:35:19.928600 [ 276586 ] <Fatal> BaseDaemon: This ClickHouse version is not official and should be upgraded to the official build.
[dell9510] 2024.07.12 14:35:19.928953 [ 276586 ] <Fatal> BaseDaemon: Changed settings: stream_like_engine_allow_direct_select = true, log_queries = true, distributed_ddl_task_timeout = 30, query_profiler_real_time_period_ns = 1000000000, query_profiler_cpu_time_period_ns = 1000000000, show_table_uuid_in_table_create_query_if_not_nil = false, database_atomic_wait_for_drop_and_detach_synchronously = true, database_replicated_initial_query_timeout_sec = 30, database_replicated_always_detach_permanently = true, distributed_ddl_output_mode = 'none', distributed_ddl_entry_format_version = 6, allow_experimental_analyzer = true, allow_experimental_shared_merge_tree = true, allow_experimental_database_replicated = true, background_pool_size = 16, default_database_engine = 'Atomic'
← Progress: 7.00 rows, 56.00 B (0.78 rows/s., 6.21 B/s.)  99%Exception on client:
Code: 32. DB::Exception: Attempt to read after eof: while receiving packet from localhost:9000. (ATTEMPT_TO_READ_AFTER_EOF)
Algunenano commented 3 months ago

Looks really similar to https://github.com/ClickHouse/ClickHouse/issues/62449

Algunenano commented 3 months ago

Fixed by https://github.com/ClickHouse/ClickHouse/pull/67091