From 10b36c09e9ffd96f53d2fdada725f9c999241dd1 Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Fri, 26 Jul 2024 15:38:26 +0800 Subject: [PATCH 1/9] support set orc reader time zone name --- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.cpp | 2 +- src/Formats/FormatFactory.cpp | 2 +- src/Formats/FormatSettings.h | 2 +- .../Formats/Impl/NativeORCBlockInputFormat.cpp | 6 +----- tests/queries/0_stateless/03198_orc_read_time_zone.sh | 10 +++++----- 6 files changed, 10 insertions(+), 14 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 2a665326afc..a5220c3017c 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1073,7 +1073,7 @@ class IColumn; M(Bool, input_format_orc_allow_missing_columns, true, "Allow missing columns while reading ORC input formats", 0) \ M(Bool, input_format_orc_use_fast_decoder, true, "Use a faster ORC decoder implementation.", 0) \ M(Bool, input_format_orc_filter_push_down, true, "When reading ORC files, skip whole stripes or row groups based on the WHERE/PREWHERE expressions, min/max statistics or bloom filter in the ORC metadata.", 0) \ - M(Bool, input_format_orc_read_use_writer_time_zone, false, "Whether use the writer's time zone in ORC stripe for ORC row reader, the default ORC row reader's time zone is GMT.", 0) \ + M(String, input_format_orc_reader_time_zone_name, "GMT", "The time zone name for ORC row reader, the default ORC row reader's time zone is GMT.", 0) \ M(Bool, input_format_parquet_allow_missing_columns, true, "Allow missing columns while reading Parquet input formats", 0) \ M(UInt64, input_format_parquet_local_file_min_bytes_for_seek, 8192, "Min bytes required for local read (file) to do seek, instead of read with ignore in Parquet input format", 0) \ M(Bool, input_format_arrow_allow_missing_columns, true, "Allow missing columns while reading Arrow input formats", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index acd119c159b..457caa76bb6 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -69,7 +69,7 @@ static std::initializer_listgetWriterTimezone(); - row_reader_options.setTimezoneName(writer_time_zone); - } + row_reader_options.setTimezoneName(format_settings.orc.reader_time_zone_name); row_reader_options.range(current_stripe_info->getOffset(), current_stripe_info->getLength()); if (format_settings.orc.filter_push_down && sarg) { diff --git a/tests/queries/0_stateless/03198_orc_read_time_zone.sh b/tests/queries/0_stateless/03198_orc_read_time_zone.sh index 27530c06237..7e931e16e48 100755 --- a/tests/queries/0_stateless/03198_orc_read_time_zone.sh +++ b/tests/queries/0_stateless/03198_orc_read_time_zone.sh @@ -5,8 +5,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -q "drop table if exists test" -$CLICKHOUSE_CLIENT -q "create table test(id UInt64, t DateTime64) Engine=MergeTree order by id" -$CLICKHOUSE_CLIENT -q "insert into test from infile '$CURDIR/data_orc/test_reader_time_zone.snappy.orc' SETTINGS input_format_orc_read_use_writer_time_zone=true FORMAT ORC" -$CLICKHOUSE_CLIENT -q "select * from test SETTINGS session_timezone='Asia/Shanghai'" -$CLICKHOUSE_CLIENT -q "drop table test" \ No newline at end of file +$CLICKHOUSE_CLIENT -q "drop table if exists test_orc_read_timezone" +$CLICKHOUSE_CLIENT -q "create table test_orc_read_timezone(id UInt64, t DateTime64) Engine=MergeTree order by id" +$CLICKHOUSE_CLIENT -q "insert into test_orc_read_timezone from infile '$CURDIR/data_orc/test_reader_time_zone.snappy.orc' SETTINGS input_format_orc_reader_time_zone_name='Asia/Shanghai' FORMAT ORC" +$CLICKHOUSE_CLIENT -q "select * from test_orc_read_timezone" +$CLICKHOUSE_CLIENT -q "drop table test_orc_read_timezone" \ No newline at end of file From 72ebff825c0752e66cefa4f367ce43ff23d77703 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 26 Jul 2024 12:19:09 +0200 Subject: [PATCH 2/9] Reduce max time of 00763_long_lock_buffer_alter_destination_table --- ...ong_lock_buffer_alter_destination_table.sh | 33 +++++++++++++++---- 1 file changed, 27 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/00763_long_lock_buffer_alter_destination_table.sh b/tests/queries/0_stateless/00763_long_lock_buffer_alter_destination_table.sh index 7e2384cfc52..c12b4426740 100755 --- a/tests/queries/0_stateless/00763_long_lock_buffer_alter_destination_table.sh +++ b/tests/queries/0_stateless/00763_long_lock_buffer_alter_destination_table.sh @@ -16,18 +16,39 @@ ${CLICKHOUSE_CLIENT} --query="CREATE TABLE buffer_00763_1 (s String) ENGINE = Bu ${CLICKHOUSE_CLIENT} --query="CREATE TABLE mt_00763_1 (x UInt32, s String) ENGINE = MergeTree ORDER BY x" ${CLICKHOUSE_CLIENT} --query="INSERT INTO mt_00763_1 VALUES (1, '1'), (2, '2'), (3, '3')" -function thread1() +function thread_alter() { - seq 1 300 | sed -r -e 's/.+/ALTER TABLE mt_00763_1 MODIFY column s UInt32; ALTER TABLE mt_00763_1 MODIFY column s String;/' | ${CLICKHOUSE_CLIENT} --multiquery --ignore-error ||: + local TIMELIMIT=$((SECONDS+$1)) + local it=0 + while [ $SECONDS -lt "$TIMELIMIT" ] && [ $it -lt 300 ]; + do + it=$((it+1)) + $CLICKHOUSE_CLIENT --multiquery --ignore-error -q " + ALTER TABLE mt_00763_1 MODIFY column s UInt32; + ALTER TABLE mt_00763_1 MODIFY column s String; + " ||: + done } -function thread2() +function thread_query() { - seq 1 2000 | sed -r -e 's/.+/SELECT sum(length(s)) FROM buffer_00763_1;/' | ${CLICKHOUSE_CLIENT} --multiquery --ignore-error 2>&1 | grep -vP '(^3$|^Received exception from server|^Code: 473)' + local TIMELIMIT=$((SECONDS+$1)) + local it=0 + while [ $SECONDS -lt "$TIMELIMIT" ] && [ $it -lt 2000 ]; + do + it=$((it+1)) + $CLICKHOUSE_CLIENT --multiquery --ignore-error -q " + SELECT sum(length(s)) FROM buffer_00763_1; + " 2>&1 | grep -vP '(^3$|^Received exception from server|^Code: 473)' + done } -thread1 & -thread2 & +export -f thread_alter +export -f thread_query + +TIMEOUT=30 +thread_alter $TIMEOUT & +thread_query $TIMEOUT & wait From 343f1fa4bae219f7c287cb314ed6e04feb9a0de4 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 26 Jul 2024 17:42:06 +0000 Subject: [PATCH 3/9] Check type after optimize_rewrite_aggregate_function_with_if. --- .../RewriteAggregateFunctionWithIfPass.cpp | 32 ++++++++++++++++--- src/Analyzer/Resolve/QueryAnalyzer.cpp | 4 ++- 2 files changed, 30 insertions(+), 6 deletions(-) diff --git a/src/Analyzer/Passes/RewriteAggregateFunctionWithIfPass.cpp b/src/Analyzer/Passes/RewriteAggregateFunctionWithIfPass.cpp index c1adf05ac76..a48e88132a6 100644 --- a/src/Analyzer/Passes/RewriteAggregateFunctionWithIfPass.cpp +++ b/src/Analyzer/Passes/RewriteAggregateFunctionWithIfPass.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -42,7 +43,7 @@ public: if (lower_name.ends_with("if")) return; - auto & function_arguments_nodes = function_node->getArguments().getNodes(); + const auto & function_arguments_nodes = function_node->getArguments().getNodes(); if (function_arguments_nodes.size() != 1) return; @@ -50,6 +51,8 @@ public: if (!if_node || if_node->getFunctionName() != "if") return; + FunctionNodePtr replaced_node; + auto if_arguments_nodes = if_node->getArguments().getNodes(); auto * first_const_node = if_arguments_nodes[1]->as(); auto * second_const_node = if_arguments_nodes[2]->as(); @@ -75,8 +78,11 @@ public: new_arguments[0] = std::move(if_arguments_nodes[1]); new_arguments[1] = std::move(if_arguments_nodes[0]); - function_arguments_nodes = std::move(new_arguments); - resolveAggregateFunctionNodeByName(*function_node, function_node->getFunctionName() + "If"); + + replaced_node = std::make_shared(function_node->getFunctionName() + "If"); + replaced_node->getArguments().getNodes() = std::move(new_arguments); + replaced_node->getParameters().getNodes() = function_node->getParameters().getNodes(); + resolveAggregateFunctionNodeByName(*replaced_node, replaced_node->getFunctionName()); } } else if (first_const_node) @@ -104,10 +110,26 @@ public: FunctionFactory::instance().get("not", getContext())->build(not_function->getArgumentColumns())); new_arguments[1] = std::move(not_function); - function_arguments_nodes = std::move(new_arguments); - resolveAggregateFunctionNodeByName(*function_node, function_node->getFunctionName() + "If"); + replaced_node = std::make_shared(function_node->getFunctionName() + "If"); + replaced_node->getArguments().getNodes() = std::move(new_arguments); + replaced_node->getParameters().getNodes() = function_node->getParameters().getNodes(); + resolveAggregateFunctionNodeByName(*replaced_node, replaced_node->getFunctionName()); } } + + if (!replaced_node) + return; + + auto prev_type = function_node->getResultType(); + auto curr_type = replaced_node->getResultType(); + if (!prev_type->equals(*curr_type)) + return; + + /// Just in case, CAST compatible aggregate function states. + if (WhichDataType(prev_type).isAggregateFunction() && !DataTypeAggregateFunction::strictEquals(prev_type, curr_type)) + node = createCastFunction(std::move(replaced_node), prev_type, getContext()); + else + node = std::move(replaced_node); } }; diff --git a/src/Analyzer/Resolve/QueryAnalyzer.cpp b/src/Analyzer/Resolve/QueryAnalyzer.cpp index b1fe2554988..b1603bb18dd 100644 --- a/src/Analyzer/Resolve/QueryAnalyzer.cpp +++ b/src/Analyzer/Resolve/QueryAnalyzer.cpp @@ -3239,11 +3239,13 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi auto action = function_node_ptr->getNullsAction(); std::string aggregate_function_name = rewriteAggregateFunctionNameIfNeeded(function_name, action, scope.context); + std::cerr << "==================== " << function_name << " -> " << aggregate_function_name << std::endl; + AggregateFunctionProperties properties; auto aggregate_function = AggregateFunctionFactory::instance().get(aggregate_function_name, action, argument_types, parameters, properties); - + std::cerr << aggregate_function->getName() << ' ' << aggregate_function->getResultType()->getName() << std::endl; function_node.resolveAsAggregateFunction(std::move(aggregate_function)); return result_projection_names; From 4833b46a1a86bb1847d2520ea12ea4650c497abc Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 26 Jul 2024 17:43:30 +0000 Subject: [PATCH 4/9] Remove debug code --- src/Analyzer/Resolve/QueryAnalyzer.cpp | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Analyzer/Resolve/QueryAnalyzer.cpp b/src/Analyzer/Resolve/QueryAnalyzer.cpp index b1603bb18dd..b1fe2554988 100644 --- a/src/Analyzer/Resolve/QueryAnalyzer.cpp +++ b/src/Analyzer/Resolve/QueryAnalyzer.cpp @@ -3239,13 +3239,11 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi auto action = function_node_ptr->getNullsAction(); std::string aggregate_function_name = rewriteAggregateFunctionNameIfNeeded(function_name, action, scope.context); - std::cerr << "==================== " << function_name << " -> " << aggregate_function_name << std::endl; - AggregateFunctionProperties properties; auto aggregate_function = AggregateFunctionFactory::instance().get(aggregate_function_name, action, argument_types, parameters, properties); - std::cerr << aggregate_function->getName() << ' ' << aggregate_function->getResultType()->getName() << std::endl; + function_node.resolveAsAggregateFunction(std::move(aggregate_function)); return result_projection_names; From 327fb40178fcff7051e17ca47d8446e5671b5383 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 29 Jul 2024 13:21:22 +0000 Subject: [PATCH 5/9] Add a test. --- ...e_function_with_if_implicit_cast.reference | 32 ++++++++-------- ...function_with_if_return_type_bug.reference | 9 +++++ ...egate_function_with_if_return_type_bug.sql | 37 +++++++++++++++++++ 3 files changed, 62 insertions(+), 16 deletions(-) create mode 100644 tests/queries/0_stateless/03210_optimize_rewrite_aggregate_function_with_if_return_type_bug.reference create mode 100644 tests/queries/0_stateless/03210_optimize_rewrite_aggregate_function_with_if_return_type_bug.sql diff --git a/tests/queries/0_stateless/03132_rewrite_aggregate_function_with_if_implicit_cast.reference b/tests/queries/0_stateless/03132_rewrite_aggregate_function_with_if_implicit_cast.reference index 070a6c9e21c..8289ab68ae4 100644 --- a/tests/queries/0_stateless/03132_rewrite_aggregate_function_with_if_implicit_cast.reference +++ b/tests/queries/0_stateless/03132_rewrite_aggregate_function_with_if_implicit_cast.reference @@ -30,14 +30,14 @@ QUERY id: 0 FUNCTION id: 15, function_name: toTypeName, function_type: ordinary, result_type: String ARGUMENTS LIST id: 16, nodes: 1 - FUNCTION id: 2, function_name: anyIf, function_type: aggregate, result_type: Nullable(Int128) + FUNCTION id: 17, function_name: anyIf, function_type: aggregate, result_type: Nullable(Int128) ARGUMENTS - LIST id: 3, nodes: 2 - FUNCTION id: 4, function_name: _CAST, function_type: ordinary, result_type: Nullable(Int128) + LIST id: 18, nodes: 2 + FUNCTION id: 19, function_name: _CAST, function_type: ordinary, result_type: Nullable(Int128) ARGUMENTS - LIST id: 5, nodes: 2 + LIST id: 20, nodes: 2 COLUMN id: 6, column_name: number, result_type: UInt64, source_id: 7 - CONSTANT id: 8, constant_value: \'Nullable(Int128)\', constant_value_type: String + CONSTANT id: 21, constant_value: \'Nullable(Int128)\', constant_value_type: String FUNCTION id: 9, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 10, nodes: 2 @@ -50,8 +50,8 @@ QUERY id: 0 JOIN TREE TABLE_FUNCTION id: 7, alias: __table1, table_function_name: numbers ARGUMENTS - LIST id: 17, nodes: 1 - CONSTANT id: 18, constant_value: UInt64_100, constant_value_type: UInt8 + LIST id: 22, nodes: 1 + CONSTANT id: 23, constant_value: UInt64_100, constant_value_type: UInt8 SELECT any(if((number % 10) = 5, CAST(NULL, 'Nullable(Int128)'), number)) AS a, toTypeName(a) FROM numbers(100) AS a; 0 Nullable(Int128) EXPLAIN QUERY TREE SELECT any(if((number % 10) = 5, CAST(NULL, 'Nullable(Int128)'), number)) AS a, toTypeName(a) FROM numbers(100); @@ -84,17 +84,17 @@ QUERY id: 0 FUNCTION id: 17, function_name: toTypeName, function_type: ordinary, result_type: String ARGUMENTS LIST id: 18, nodes: 1 - FUNCTION id: 2, function_name: anyIf, function_type: aggregate, result_type: Nullable(Int128) + FUNCTION id: 19, function_name: anyIf, function_type: aggregate, result_type: Nullable(Int128) ARGUMENTS - LIST id: 3, nodes: 2 - FUNCTION id: 4, function_name: _CAST, function_type: ordinary, result_type: Nullable(Int128) + LIST id: 20, nodes: 2 + FUNCTION id: 21, function_name: _CAST, function_type: ordinary, result_type: Nullable(Int128) ARGUMENTS - LIST id: 5, nodes: 2 + LIST id: 22, nodes: 2 COLUMN id: 6, column_name: number, result_type: UInt64, source_id: 7 - CONSTANT id: 8, constant_value: \'Nullable(Int128)\', constant_value_type: String - FUNCTION id: 9, function_name: not, function_type: ordinary, result_type: UInt8 + CONSTANT id: 23, constant_value: \'Nullable(Int128)\', constant_value_type: String + FUNCTION id: 24, function_name: not, function_type: ordinary, result_type: UInt8 ARGUMENTS - LIST id: 10, nodes: 1 + LIST id: 25, nodes: 1 FUNCTION id: 11, function_name: equals, function_type: ordinary, result_type: UInt8 ARGUMENTS LIST id: 12, nodes: 2 @@ -107,5 +107,5 @@ QUERY id: 0 JOIN TREE TABLE_FUNCTION id: 7, alias: __table1, table_function_name: numbers ARGUMENTS - LIST id: 19, nodes: 1 - CONSTANT id: 20, constant_value: UInt64_100, constant_value_type: UInt8 + LIST id: 26, nodes: 1 + CONSTANT id: 27, constant_value: UInt64_100, constant_value_type: UInt8 diff --git a/tests/queries/0_stateless/03210_optimize_rewrite_aggregate_function_with_if_return_type_bug.reference b/tests/queries/0_stateless/03210_optimize_rewrite_aggregate_function_with_if_return_type_bug.reference new file mode 100644 index 00000000000..0693acb9168 --- /dev/null +++ b/tests/queries/0_stateless/03210_optimize_rewrite_aggregate_function_with_if_return_type_bug.reference @@ -0,0 +1,9 @@ +03 +AggregateFunction(count, Nullable(UInt64)) +function_name: _CAST, function_type: ordinary, result_type: AggregateFunction(count, Nullable(UInt64)) +function_name: countStateIf, function_type: aggregate, result_type: AggregateFunction(count, UInt64) +constant_value: \'AggregateFunction(count, Nullable(UInt64))\', constant_value_type: String +AggregateFunction(uniq, Nullable(UInt64)) +010003000000007518F0A8E7830665 +function_name: uniqState, function_type: aggregate, result_type: AggregateFunction(uniq, Nullable(UInt64)) +---- diff --git a/tests/queries/0_stateless/03210_optimize_rewrite_aggregate_function_with_if_return_type_bug.sql b/tests/queries/0_stateless/03210_optimize_rewrite_aggregate_function_with_if_return_type_bug.sql new file mode 100644 index 00000000000..b620a6434bf --- /dev/null +++ b/tests/queries/0_stateless/03210_optimize_rewrite_aggregate_function_with_if_return_type_bug.sql @@ -0,0 +1,37 @@ +-- For function count, rewrite countState to countStateIf changes the type from AggregateFunction(count, Nullable(UInt64)) to AggregateFunction(count, UInt64) +-- We can cast AggregateFunction(count, UInt64) back to AggregateFunction(count, Nullable(UInt64)) with additional _CAST +select hex(countState(if(toNullable(number % 2 = 0), number, null))) from numbers(5) settings optimize_rewrite_aggregate_function_with_if=1; +select toTypeName(countState(if(toNullable(number % 2 = 0), number, null))) from numbers(5) settings optimize_rewrite_aggregate_function_with_if=1; +select arrayStringConcat(arraySlice(splitByString(', ', trimLeft(explain)), 2), ', ') from (explain query tree select hex(countState(if(toNullable(number % 2 = 0), number, null))) from numbers(5) settings optimize_rewrite_aggregate_function_with_if=1) where explain like '%AggregateFunction%'; + +-- For function uniq, rewrite uniqState to uniqStateIf changes the type from AggregateFunction(uniq, Nullable(UInt64)) to AggregateFunction(uniq, UInt64) +-- We can't cast AggregateFunction(uniq, UInt64) back to AggregateFunction(uniq, Nullable(UInt64)) so rewrite is not happening. +select toTypeName(uniqState(if(toNullable(number % 2 = 0), number, null))) from numbers(5) settings optimize_rewrite_aggregate_function_with_if=1; +select hex(uniqState(if(toNullable(number % 2 = 0), number, null))) from numbers(5) settings optimize_rewrite_aggregate_function_with_if=1; +select arrayStringConcat(arraySlice(splitByString(', ', trimLeft(explain)), 2), ', ') from (explain query tree select hex(uniqState(if(toNullable(number % 2 = 0), number, null))) from numbers(5) settings optimize_rewrite_aggregate_function_with_if=1) where explain like '%AggregateFunction%'; + +select '----'; + +CREATE TABLE a +( + `a_id` String +) +ENGINE = MergeTree +PARTITION BY tuple() +ORDER BY tuple(); + + +CREATE TABLE b +( + `b_id` AggregateFunction(uniq, Nullable(String)) +) +ENGINE = AggregatingMergeTree +PARTITION BY tuple() +ORDER BY tuple(); + +CREATE MATERIALIZED VIEW mv TO b +( + `b_id` AggregateFunction(uniq, Nullable(String)) +) +AS SELECT uniqState(if(a_id != '', a_id, NULL)) AS b_id +FROM a; From cb6b6329c8e763f61f70797a95dab8ef24fd47d1 Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Tue, 30 Jul 2024 10:45:36 +0800 Subject: [PATCH 6/9] add session timezone settings --- tests/queries/0_stateless/03198_orc_read_time_zone.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03198_orc_read_time_zone.sh b/tests/queries/0_stateless/03198_orc_read_time_zone.sh index 7e931e16e48..7d1da0c1579 100755 --- a/tests/queries/0_stateless/03198_orc_read_time_zone.sh +++ b/tests/queries/0_stateless/03198_orc_read_time_zone.sh @@ -8,5 +8,5 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT -q "drop table if exists test_orc_read_timezone" $CLICKHOUSE_CLIENT -q "create table test_orc_read_timezone(id UInt64, t DateTime64) Engine=MergeTree order by id" $CLICKHOUSE_CLIENT -q "insert into test_orc_read_timezone from infile '$CURDIR/data_orc/test_reader_time_zone.snappy.orc' SETTINGS input_format_orc_reader_time_zone_name='Asia/Shanghai' FORMAT ORC" -$CLICKHOUSE_CLIENT -q "select * from test_orc_read_timezone" +$CLICKHOUSE_CLIENT -q "select * from test_orc_read_timezone SETTINGS session_timezone='Asia/Shanghai'" $CLICKHOUSE_CLIENT -q "drop table test_orc_read_timezone" \ No newline at end of file From d7534c5b40c315cdb3d52101f626dbee7d565cdb Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 30 Jul 2024 12:20:54 +0200 Subject: [PATCH 7/9] Update 03210_optimize_rewrite_aggregate_function_with_if_return_type_bug.sql --- ...imize_rewrite_aggregate_function_with_if_return_type_bug.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/03210_optimize_rewrite_aggregate_function_with_if_return_type_bug.sql b/tests/queries/0_stateless/03210_optimize_rewrite_aggregate_function_with_if_return_type_bug.sql index b620a6434bf..565a481940a 100644 --- a/tests/queries/0_stateless/03210_optimize_rewrite_aggregate_function_with_if_return_type_bug.sql +++ b/tests/queries/0_stateless/03210_optimize_rewrite_aggregate_function_with_if_return_type_bug.sql @@ -1,3 +1,5 @@ +SET allow_experimental_analyzer = 1; + -- For function count, rewrite countState to countStateIf changes the type from AggregateFunction(count, Nullable(UInt64)) to AggregateFunction(count, UInt64) -- We can cast AggregateFunction(count, UInt64) back to AggregateFunction(count, Nullable(UInt64)) with additional _CAST select hex(countState(if(toNullable(number % 2 = 0), number, null))) from numbers(5) settings optimize_rewrite_aggregate_function_with_if=1; From 7e51e9962c34320a9c60ba6abcf8b38cf517e86c Mon Sep 17 00:00:00 2001 From: Alex Katsman Date: Tue, 30 Jul 2024 17:20:07 +0000 Subject: [PATCH 8/9] Fix WriteBuffer destructor when finalize has failed for MergeTreeDeduplicationLog::shutdown --- src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp index 22ff9b7194f..a8110500f13 100644 --- a/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp +++ b/src/Storages/MergeTree/MergeTreeDeduplicationLog.cpp @@ -341,15 +341,19 @@ void MergeTreeDeduplicationLog::shutdown() stopped = true; if (current_writer) { + /// If an error has occurred during finalize, we'd like to have the exception set for reset. + /// Otherwise, we'll be in a situation when a finalization didn't happen, and we didn't get + /// any error, causing logical error (see ~MemoryBuffer()). try { current_writer->finalize(); + current_writer.reset(); } catch (...) { tryLogCurrentException(__PRETTY_FUNCTION__); + current_writer.reset(); } - current_writer.reset(); } } From 51af0d305c9959fb4870bd8a57035d48207648b2 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 30 Jul 2024 20:00:23 +0200 Subject: [PATCH 9/9] Reduce number of tested combinations --- tests/queries/0_stateless/02473_multistep_prewhere.python | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02473_multistep_prewhere.python b/tests/queries/0_stateless/02473_multistep_prewhere.python index a942568233c..11095202039 100644 --- a/tests/queries/0_stateless/02473_multistep_prewhere.python +++ b/tests/queries/0_stateless/02473_multistep_prewhere.python @@ -193,7 +193,7 @@ def main(): url = os.environ["CLICKHOUSE_URL"] + "&max_threads=1" default_index_granularity = 10 - total_rows = 8 * default_index_granularity + total_rows = 7 * default_index_granularity step = default_index_granularity session = requests.Session() for index_granularity in [