From 3d6de8572fc0eaf975b5c3e6a901fa59b31c805f Mon Sep 17 00:00:00 2001 From: Mryange Date: Fri, 24 Jan 2025 05:04:20 +0800 Subject: [PATCH 01/14] [fix](exchange)fix exchange sink buffer does not update total_queue_size when EOF. (#47312) ### What problem does this PR solve? https://github.com/apache/doris/pull/41602 EOF clears _instance_to_package_queue but does not update total_queue_size, causing incorrect judgments that rely on total_queue_size. UT ``` mock transmit_blockv2 dest ins id :1 mock transmit_blockv2 dest ins id :2 mock transmit_blockv2 dest ins id :3 queue size : 6 each queue size : Instance: 2, queue size: 2 Instance: 1, queue size: 2 Instance: 3, queue size: 2 queue size : 6 // error size each queue size : Instance: 2, queue size: 0 Instance: 1, queue size: 2 Instance: 3, queue size: 2 mock transmit_blockv2 dest ins id :1 mock transmit_blockv2 dest ins id :1 mock transmit_blockv2 dest ins id :3 mock transmit_blockv2 dest ins id :3 ``` --- be/src/pipeline/exec/exchange_sink_buffer.cpp | 24 ++++++++++- be/src/pipeline/exec/exchange_sink_buffer.h | 4 ++ .../pipeline/exec/exchange_sink_operator.cpp | 13 +++--- be/test/vec/exec/exchange_sink_test.cpp | 40 +++++++++++++++++++ 4 files changed, 74 insertions(+), 7 deletions(-) diff --git a/be/src/pipeline/exec/exchange_sink_buffer.cpp b/be/src/pipeline/exec/exchange_sink_buffer.cpp index 800ef6150738d6..4acb747bbc95e7 100644 --- a/be/src/pipeline/exec/exchange_sink_buffer.cpp +++ b/be/src/pipeline/exec/exchange_sink_buffer.cpp @@ -42,6 +42,7 @@ #include "runtime/exec_env.h" #include "runtime/thread_context.h" #include "service/backend_options.h" +#include "util/defer_op.h" #include "util/proto_util.h" #include "util/time.h" #include "vec/sink/vdata_stream_sender.h" @@ -442,7 +443,8 @@ void ExchangeSinkBuffer::_set_receiver_eof(InstanceLoId id) { // When the receiving side reaches eof, it means the receiver has finished early. // The remaining data in the current rpc_channel does not need to be sent, // and the rpc_channel should be turned off immediately. - _turn_off_channel(id, lock); + Defer turn_off([&]() { _turn_off_channel(id, lock); }); + std::queue>& broadcast_q = _instance_to_broadcast_package_queue[id]; for (; !broadcast_q.empty(); broadcast_q.pop()) { @@ -458,12 +460,22 @@ void ExchangeSinkBuffer::_set_receiver_eof(InstanceLoId id) { std::queue>& q = _instance_to_package_queue[id]; for (; !q.empty(); q.pop()) { + // Must update _total_queue_size here, otherwise if _total_queue_size > _queue_capacity at EOF, + // ExchangeSinkQueueDependency will be blocked and pipeline will be deadlocked + _total_queue_size--; if (q.front().block) { COUNTER_UPDATE(q.front().channel->_parent->memory_used_counter(), -q.front().block->ByteSizeLong()); } } + // Try to wake up pipeline after clearing the queue + if (_total_queue_size <= _queue_capacity) { + for (auto& [_, dep] : _queue_deps) { + dep->set_ready(); + } + } + { std::queue> empty; swap(empty, q); @@ -575,6 +587,16 @@ void ExchangeSinkBuffer::update_profile(RuntimeProfile* profile) { } } +std::string ExchangeSinkBuffer::debug_each_instance_queue_size() { + fmt::memory_buffer debug_string_buffer; + for (auto& [id, m] : _instance_to_package_queue_mutex) { + std::unique_lock lock(*m); + fmt::format_to(debug_string_buffer, "Instance: {}, queue size: {}\n", id, + _instance_to_package_queue[id].size()); + } + return fmt::to_string(debug_string_buffer); +} + } // namespace pipeline #include "common/compile_check_end.h" } // namespace doris diff --git a/be/src/pipeline/exec/exchange_sink_buffer.h b/be/src/pipeline/exec/exchange_sink_buffer.h index 80e5dc42289e10..e6c4635aef3686 100644 --- a/be/src/pipeline/exec/exchange_sink_buffer.h +++ b/be/src/pipeline/exec/exchange_sink_buffer.h @@ -239,6 +239,8 @@ class ExchangeSinkBuffer : public HasTaskExecutionCtx { _queue_deps[sender_ins_id] = queue_dependency; _parents[sender_ins_id] = local_state; } + + std::string debug_each_instance_queue_size(); #ifdef BE_TEST public: #else @@ -306,6 +308,8 @@ class ExchangeSinkBuffer : public HasTaskExecutionCtx { void get_max_min_rpc_time(int64_t* max_time, int64_t* min_time); int64_t get_sum_rpc_time(); + // _total_queue_size is the sum of the sizes of all instance_to_package_queues. + // Any modification to instance_to_package_queue requires a corresponding modification to _total_queue_size. std::atomic _total_queue_size = 0; // _running_sink_count is used to track how many sinks have not finished yet. diff --git a/be/src/pipeline/exec/exchange_sink_operator.cpp b/be/src/pipeline/exec/exchange_sink_operator.cpp index d7422e0e8f008b..9c3e451b33328f 100644 --- a/be/src/pipeline/exec/exchange_sink_operator.cpp +++ b/be/src/pipeline/exec/exchange_sink_operator.cpp @@ -499,12 +499,13 @@ std::string ExchangeSinkLocalState::debug_string(int indentation_level) const { fmt::memory_buffer debug_string_buffer; fmt::format_to(debug_string_buffer, "{}", Base::debug_string(indentation_level)); if (_sink_buffer) { - fmt::format_to(debug_string_buffer, - ", Sink Buffer: (_is_finishing = {}, blocks in queue: {}, queue capacity: " - "{}, queue dep: {}), _reach_limit: {}, working channels: {}", - _sink_buffer->_is_failed.load(), _sink_buffer->_total_queue_size, - _sink_buffer->_queue_capacity, (void*)_queue_dependency.get(), - _reach_limit.load(), _working_channels_count.load()); + fmt::format_to( + debug_string_buffer, + ", Sink Buffer: (_is_finishing = {}, blocks in queue: {}, queue capacity: " + "{}, queue dep: {}), _reach_limit: {}, working channels: {} , each queue size: {}", + _sink_buffer->_is_failed.load(), _sink_buffer->_total_queue_size, + _sink_buffer->_queue_capacity, (void*)_queue_dependency.get(), _reach_limit.load(), + _working_channels_count.load(), _sink_buffer->debug_each_instance_queue_size()); } return fmt::to_string(debug_string_buffer); } diff --git a/be/test/vec/exec/exchange_sink_test.cpp b/be/test/vec/exec/exchange_sink_test.cpp index 9576ed71ee2ada..7dbd352bd3a23f 100644 --- a/be/test/vec/exec/exchange_sink_test.cpp +++ b/be/test/vec/exec/exchange_sink_test.cpp @@ -193,4 +193,44 @@ TEST_F(ExchangeSInkTest, test_error_end) { } } +TEST_F(ExchangeSInkTest, test_queue_size) { + { + auto state = create_runtime_state(); + auto buffer = create_buffer(state); + + auto sink1 = create_sink(state, buffer); + + EXPECT_EQ(sink1.add_block(dest_ins_id_1, false), Status::OK()); + EXPECT_EQ(sink1.add_block(dest_ins_id_1, false), Status::OK()); + EXPECT_EQ(sink1.add_block(dest_ins_id_1, false), Status::OK()); + + EXPECT_EQ(sink1.add_block(dest_ins_id_2, false), Status::OK()); + EXPECT_EQ(sink1.add_block(dest_ins_id_2, false), Status::OK()); + EXPECT_EQ(sink1.add_block(dest_ins_id_2, false), Status::OK()); + + EXPECT_EQ(sink1.add_block(dest_ins_id_3, false), Status::OK()); + EXPECT_EQ(sink1.add_block(dest_ins_id_3, false), Status::OK()); + EXPECT_EQ(sink1.add_block(dest_ins_id_3, false), Status::OK()); + + std::cout << "queue size : " << buffer->_total_queue_size << "\n"; + + EXPECT_EQ(buffer->_total_queue_size, 6); + + std::cout << "each queue size : \n" << buffer->debug_each_instance_queue_size() << "\n"; + + pop_block(dest_ins_id_2, PopState::eof); + + std::cout << "queue size : " << buffer->_total_queue_size << "\n"; + + EXPECT_EQ(buffer->_total_queue_size, 4); + + std::cout << "each queue size : \n" << buffer->debug_each_instance_queue_size() << "\n"; + + EXPECT_EQ(buffer->_rpc_channel_is_turn_off[dest_ins_id_1], false); + EXPECT_EQ(buffer->_rpc_channel_is_turn_off[dest_ins_id_2], true); + EXPECT_EQ(buffer->_rpc_channel_is_turn_off[dest_ins_id_3], false); + clear_all_done(); + } +} + } // namespace doris::vectorized From a4060c2011738c3eb1c7f0b007732d3cf4ed370c Mon Sep 17 00:00:00 2001 From: Dongyang Li Date: Fri, 24 Jan 2025 10:51:31 +0800 Subject: [PATCH 02/14] [fix](ut) prefer to use LLVM_PROFDATA from env.sh (#47351) --- cloud/script/run_all_tests.sh | 4 ++-- regression-test/pipeline/common/github-utils.sh | 1 + 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/cloud/script/run_all_tests.sh b/cloud/script/run_all_tests.sh index 330fb00449350b..f2b5f4dd9b0b06 100644 --- a/cloud/script/run_all_tests.sh +++ b/cloud/script/run_all_tests.sh @@ -120,8 +120,8 @@ function report_coverage() { for object in ${binary_objects[@]}; do binary_objects_options[${#binary_objects_options[*]}]="-object ${object}" done - llvm-profdata merge -o ${profdata} ${profraw} - llvm-cov show -output-dir=report -format=html \ + ${LLVM_PROFDATA:-llvm-profdata} merge -o ${profdata} ${profraw} + ${LLVM_COV:-llvm-cov} show -output-dir=report -format=html \ -ignore-filename-regex='(.*gensrc/.*)|(.*_test\.cpp$)' \ -instr-profile=${profdata} \ ${binary_objects_options[*]} diff --git a/regression-test/pipeline/common/github-utils.sh b/regression-test/pipeline/common/github-utils.sh index 6e2321540726fe..361eecd394cbeb 100644 --- a/regression-test/pipeline/common/github-utils.sh +++ b/regression-test/pipeline/common/github-utils.sh @@ -283,6 +283,7 @@ file_changed_cloud_ut() { if [[ -z ${all_files} ]]; then echo "return need" && return 0; fi for af in ${all_files}; do if [[ "${af}" == 'cloud/src/'* ]] || + [[ "${af}" == 'cloud/script/'* ]] || [[ "${af}" == 'gensrc'* ]] || [[ "${af}" == 'common/cpp'* ]] || [[ "${af}" == 'cloud/test/'* ]]; then From ae526f2bdceb215f57c0b8dba0b64f33302f764b Mon Sep 17 00:00:00 2001 From: Sun Chenyang Date: Fri, 24 Jan 2025 10:55:34 +0800 Subject: [PATCH 03/14] [opt](jsonb) add ut for the jsonb parser (#47181) --- be/test/util/jsonb_parser_simd_test.cpp | 406 ++++++++++++++++++++++++ 1 file changed, 406 insertions(+) create mode 100644 be/test/util/jsonb_parser_simd_test.cpp diff --git a/be/test/util/jsonb_parser_simd_test.cpp b/be/test/util/jsonb_parser_simd_test.cpp new file mode 100644 index 00000000000000..4fff867d5361ab --- /dev/null +++ b/be/test/util/jsonb_parser_simd_test.cpp @@ -0,0 +1,406 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#include "util/jsonb_parser_simd.h" + +#include "common/status.h" +#include "gtest/gtest.h" +#include "util/jsonb_error.h" +#include "util/jsonb_utils.h" +#include "util/jsonb_writer.h" + +namespace doris { +class JsonbParserTest : public testing::Test {}; + +static JsonbErrType parse_json_and_check(std::string_view json_str, std::string_view expected_str) { + JsonbParser parser; + if (!parser.parse(json_str.data(), json_str.length())) { + return parser.getErrorCode(); + } + const char* ptr = parser.getWriter().getOutput()->getBuffer(); + size_t len = (unsigned)parser.getWriter().getOutput()->getSize(); + EXPECT_EQ(JsonbToJson::jsonb_to_json_string(ptr, len), expected_str); + return JsonbErrType::E_NONE; +} + +TEST_F(JsonbParserTest, ParseSimpleJson) { + std::string_view simple_json = R"({"key":"value"})"; + std::string_view expected_simple_json = R"({"key":"value"})"; + EXPECT_EQ(parse_json_and_check(simple_json, expected_simple_json), JsonbErrType::E_NONE); +} + +TEST_F(JsonbParserTest, ParseJsonWithInt8) { + std::string_view json_with_int8 = R"({"min_int8":-128,"max_int8":127})"; + std::string_view expected_json_with_int8 = R"({"min_int8":-128,"max_int8":127})"; + EXPECT_EQ(parse_json_and_check(json_with_int8, expected_json_with_int8), JsonbErrType::E_NONE); +} + +TEST_F(JsonbParserTest, ParseJsonWithInt16) { + std::string_view json_with_int16 = R"({"min_int16":-32768,"max_int16":32767})"; + std::string_view expected_json_with_int16 = R"({"min_int16":-32768,"max_int16":32767})"; + EXPECT_EQ(parse_json_and_check(json_with_int16, expected_json_with_int16), + JsonbErrType::E_NONE); +} + +TEST_F(JsonbParserTest, ParseJsonWithInt32) { + std::string_view json_with_int32 = R"({"min_int32":-2147483648,"max_int32":2147483647})"; + std::string_view expected_json_with_int32 = + R"({"min_int32":-2147483648,"max_int32":2147483647})"; + EXPECT_EQ(parse_json_and_check(json_with_int32, expected_json_with_int32), + JsonbErrType::E_NONE); +} + +TEST_F(JsonbParserTest, ParseJsonWithInt64) { + std::string_view json_with_int64 = + R"({"min_int64":-9223372036854775808,"max_int64":9223372036854775807})"; + std::string_view expected_json_with_int64 = + R"({"min_int64":-9223372036854775808,"max_int64":9223372036854775807})"; + EXPECT_EQ(parse_json_and_check(json_with_int64, expected_json_with_int64), + JsonbErrType::E_NONE); +} + +TEST_F(JsonbParserTest, ParseJsonWithFloat) { + std::string_view json_with_float = + R"({"min_float":-3.40282e+38,"max_float":3.40282e+38,"small_float":1.17549e-38})"; + std::string_view expected_json_with_float = + R"({"min_float":-3.40282e+38,"max_float":3.40282e+38,"small_float":1.17549e-38})"; + EXPECT_EQ(parse_json_and_check(json_with_float, expected_json_with_float), + JsonbErrType::E_NONE); +} + +TEST_F(JsonbParserTest, ParseJsonWithDouble) { + std::string_view json_with_double = + R"({"min_double":-1.79769e+308,"max_double":1.79769e+308,"small_double":2.22507e-308})"; + std::string_view expected_json_with_double = + R"({"min_double":-1.79769e+308,"max_double":1.79769e+308,"small_double":2.22507e-308})"; + EXPECT_EQ(parse_json_and_check(json_with_double, expected_json_with_double), + JsonbErrType::E_NONE); +} + +TEST_F(JsonbParserTest, ParseJsonWithScientificNotation) { + std::string_view json_with_scientific_notation = + R"({"small_positive":1.23e-10,"large_positive":1.23e+10,"small_negative":-1.23e-10,"large_negative":-1.23e+10})"; + std::string_view expected_json_with_scientific_notation = + R"({"small_positive":1.23e-10,"large_positive":12300000000,"small_negative":-1.23e-10,"large_negative":-12300000000})"; + EXPECT_EQ(parse_json_and_check(json_with_scientific_notation, + expected_json_with_scientific_notation), + JsonbErrType::E_NONE); +} + +TEST_F(JsonbParserTest, ParseJsonWithMixedIntegers) { + std::string_view json_with_mixed_integers = + R"({"int8":42,"int16":32767,"int32":2147483647,"int64":9223372036854775807})"; + std::string_view expected_json_with_mixed_integers = + R"({"int8":42,"int16":32767,"int32":2147483647,"int64":9223372036854775807})"; + EXPECT_EQ(parse_json_and_check(json_with_mixed_integers, expected_json_with_mixed_integers), + JsonbErrType::E_NONE); +} + +TEST_F(JsonbParserTest, ParseJsonWithMixedFloats) { + std::string_view json_with_mixed_floats = + R"({"float_value":123.456,"double_value":123456.789012,"scientific_value":1.23e+10})"; + std::string_view expected_json_with_mixed_floats = + R"({"float_value":123.456,"double_value":123456.789012,"scientific_value":12300000000})"; + EXPECT_EQ(parse_json_and_check(json_with_mixed_floats, expected_json_with_mixed_floats), + JsonbErrType::E_NONE); +} + +TEST_F(JsonbParserTest, ParseJsonWithSimpleArray) { + std::string_view json_with_simple_array = R"({"array":[1,2,3,4,5]})"; + std::string_view expected_json_with_simple_array = R"({"array":[1,2,3,4,5]})"; + EXPECT_EQ(parse_json_and_check(json_with_simple_array, expected_json_with_simple_array), + JsonbErrType::E_NONE); +} + +TEST_F(JsonbParserTest, ParseJsonWithNestedArray) { + std::string_view json_with_nested_array = R"({"nested_array":[[1,2],[3,4],[5,6]]})"; + std::string_view expected_json_with_nested_array = R"({"nested_array":[[1,2],[3,4],[5,6]]})"; + EXPECT_EQ(parse_json_and_check(json_with_nested_array, expected_json_with_nested_array), + JsonbErrType::E_NONE); +} + +TEST_F(JsonbParserTest, ParseJsonWithMixedArray) { + std::string_view json_with_mixed_array = R"({"mixed_array":[1,"two",3.1,true,null]})"; + std::string_view expected_json_with_mixed_array = R"({"mixed_array":[1,"two",3.1,true,null]})"; + EXPECT_EQ(parse_json_and_check(json_with_mixed_array, expected_json_with_mixed_array), + JsonbErrType::E_NONE); +} + +TEST_F(JsonbParserTest, ParseJsonWithSimpleObject) { + std::string_view json_with_simple_object = R"({"object":{"key1":"value1","key2":"value2"}})"; + std::string_view expected_json_with_simple_object = + R"({"object":{"key1":"value1","key2":"value2"}})"; + EXPECT_EQ(parse_json_and_check(json_with_simple_object, expected_json_with_simple_object), + JsonbErrType::E_NONE); +} + +TEST_F(JsonbParserTest, ParseJsonWithNestedObject) { + std::string_view json_with_nested_object = + R"({"nested_object":{"level1":{"level2":{"level3":"value"}}}})"; + std::string_view expected_json_with_nested_object = + R"({"nested_object":{"level1":{"level2":{"level3":"value"}}}})"; + EXPECT_EQ(parse_json_and_check(json_with_nested_object, expected_json_with_nested_object), + JsonbErrType::E_NONE); +} + +TEST_F(JsonbParserTest, ParseJsonWithMixedObject) { + std::string_view json_with_mixed_object = + R"({"mixed_object":{"int_key":42,"string_key":"value","bool_key":true,"null_key":null}})"; + std::string_view expected_json_with_mixed_object = + R"({"mixed_object":{"int_key":42,"string_key":"value","bool_key":true,"null_key":null}})"; + EXPECT_EQ(parse_json_and_check(json_with_mixed_object, expected_json_with_mixed_object), + JsonbErrType::E_NONE); +} + +TEST_F(JsonbParserTest, ParseJsonWithArrayOfObjects) { + std::string_view json_with_array_of_objects = + R"({"array_of_objects":[{"key1":"value1"},{"key2":"value2"}]})"; + std::string_view expected_json_with_array_of_objects = + R"({"array_of_objects":[{"key1":"value1"},{"key2":"value2"}]})"; + EXPECT_EQ(parse_json_and_check(json_with_array_of_objects, expected_json_with_array_of_objects), + JsonbErrType::E_NONE); +} + +TEST_F(JsonbParserTest, ParseJsonWithObjectOfArrays) { + std::string_view json_with_object_of_arrays = + R"({"object_of_arrays":{"array1":[1,2,3],"array2":["a","b","c"]}})"; + std::string_view expected_json_with_object_of_arrays = + R"({"object_of_arrays":{"array1":[1,2,3],"array2":["a","b","c"]}})"; + EXPECT_EQ(parse_json_and_check(json_with_object_of_arrays, expected_json_with_object_of_arrays), + JsonbErrType::E_NONE); +} + +TEST_F(JsonbParserTest, ParseJsonWithComplexNestedStructure) { + std::string_view json_with_complex_nested_structure = + R"({"complex":{"array":[1,{"key":"value"},[3,4]],"object":{"nested_array":[1,2,3],"nested_object":{"key":"value"}}}})"; + std::string_view expected_json_with_complex_nested_structure = + R"({"complex":{"array":[1,{"key":"value"},[3,4]],"object":{"nested_array":[1,2,3],"nested_object":{"key":"value"}}}})"; + EXPECT_EQ(parse_json_and_check(json_with_complex_nested_structure, + expected_json_with_complex_nested_structure), + JsonbErrType::E_NONE); +} + +TEST_F(JsonbParserTest, ParseJsonWithString) { + std::string_view json_with_string = R"({"string":"hello world"})"; + std::string_view expected_json_with_string = R"({"string":"hello world"})"; + EXPECT_EQ(parse_json_and_check(json_with_string, expected_json_with_string), + JsonbErrType::E_NONE); +} + +TEST_F(JsonbParserTest, ParseJsonWithSpecialCharactersInString) { + std::string_view json_with_special_chars = + R"({"special_chars":"!@#$%^&*()_+-=[]{}|;':,./<>?"})"; + std::string_view expected_json_with_special_chars = + R"({"special_chars":"!@#$%^&*()_+-=[]{}|;':,./<>?"})"; + EXPECT_EQ(parse_json_and_check(json_with_special_chars, expected_json_with_special_chars), + JsonbErrType::E_NONE); +} + +TEST_F(JsonbParserTest, ParseJsonWithUnicodeInString) { + std::string_view json_with_unicode = R"({"unicode":"你好,世界"})"; + std::string_view expected_json_with_unicode = R"({"unicode":"你好,世界"})"; + EXPECT_EQ(parse_json_and_check(json_with_unicode, expected_json_with_unicode), + JsonbErrType::E_NONE); +} + +TEST_F(JsonbParserTest, ParseJsonWithEscapedCharactersInString) { + std::string_view json_with_escaped_chars = R"({"escaped_chars":"\"\/\b\f\n\r\t"})"; + std::string_view expected_json_with_escaped_chars = R"({"escaped_chars":"\"/\b\f\n\r\t"})"; + EXPECT_EQ(parse_json_and_check(json_with_escaped_chars, expected_json_with_escaped_chars), + JsonbErrType::E_NONE); +} + +TEST_F(JsonbParserTest, ParseJsonWithLongInt) { + std::string_view json_with_long_int = R"({"long_int":19389892839283982938923})"; + EXPECT_EQ(parse_json_and_check(json_with_long_int, json_with_long_int), + JsonbErrType::E_EXCEPTION); +} + +TEST_F(JsonbParserTest, ParseInvalidJsonFormat) { + std::string_view invalid_json = R"({"key": "value")"; + EXPECT_EQ(parse_json_and_check(invalid_json, invalid_json), JsonbErrType::E_INVALID_KEY_STRING); +} + +TEST_F(JsonbParserTest, ParseJsonWithInvalidKeyType) { + std::string_view json_with_invalid_key = R"({123: "value"})"; + EXPECT_EQ(parse_json_and_check(json_with_invalid_key, json_with_invalid_key), + JsonbErrType::E_INVALID_KEY_STRING); +} + +TEST_F(JsonbParserTest, ParseEmptyJson) { + std::string_view empty_json = ""; + EXPECT_EQ(parse_json_and_check(empty_json, empty_json), JsonbErrType::E_EMPTY_DOCUMENT); +} + +TEST_F(JsonbParserTest, ParseJsonWithDeepNesting) { + std::string_view json_with_deep_nesting = + R"({"level1":{"level2":{"level3":{"level4":{"level5":{"level6":"value"}}}}}})"; + EXPECT_EQ(parse_json_and_check(json_with_deep_nesting, json_with_deep_nesting), + JsonbErrType::E_NONE); +} + +TEST_F(JsonbParserTest, ParseJsonWithInvalidNumberFormat) { + std::string_view json_with_invalid_number = R"({"invalid_number": 1.23e})"; + EXPECT_EQ(parse_json_and_check(json_with_invalid_number, json_with_invalid_number), + JsonbErrType::E_EXCEPTION); +} + +TEST_F(JsonbParserTest, ParseJsonWithInvalidBoolean) { + std::string_view json_with_invalid_boolean = R"({"invalid_bool": True})"; + EXPECT_EQ(parse_json_and_check(json_with_invalid_boolean, json_with_invalid_boolean), + JsonbErrType::E_EXCEPTION); +} + +TEST_F(JsonbParserTest, ParseJsonWithInvalidEscapeCharacter) { + std::string_view json_with_invalid_escape = R"({"invalid_escape": "hello\xworld"})"; + EXPECT_EQ(parse_json_and_check(json_with_invalid_escape, json_with_invalid_escape), + JsonbErrType::E_EXCEPTION); +} + +TEST_F(JsonbParserTest, ParseJsonWithInvalidArrayFormat) { + std::string_view json_with_invalid_array = R"({"array": [1 2 3]})"; + EXPECT_EQ(parse_json_and_check(json_with_invalid_array, json_with_invalid_array), + JsonbErrType::E_EXCEPTION); +} + +TEST_F(JsonbParserTest, ParseJsonWithInvalidObjectFormat) { + std::string_view json_with_invalid_object = R"({"key1" "value1", "key2": "value2"})"; + EXPECT_EQ(parse_json_and_check(json_with_invalid_object, json_with_invalid_object), + JsonbErrType::E_INVALID_KEY_STRING); +} + +TEST_F(JsonbParserTest, ParseJsonWithUnsupportedSpecialValue) { + std::string_view json_with_unsupported_value = R"({"unsupported_value": NaN})"; + EXPECT_EQ(parse_json_and_check(json_with_unsupported_value, json_with_unsupported_value), + JsonbErrType::E_EXCEPTION); +} + +TEST_F(JsonbParserTest, ParseJsonWithLongString) { + std::string long_string(1000000, 'a'); + std::string json_with_long_string = R"({"long_string":")" + long_string + R"("})"; + EXPECT_EQ(parse_json_and_check(json_with_long_string, json_with_long_string), + JsonbErrType::E_NONE); +} + +TEST_F(JsonbParserTest, ParseJsonWithInvalidUnicode) { + std::string_view json_with_invalid_unicode = R"({"invalid_unicode": "\uZZZZ"})"; + EXPECT_EQ(parse_json_and_check(json_with_invalid_unicode, json_with_invalid_unicode), + JsonbErrType::E_EXCEPTION); +} + +TEST_F(JsonbParserTest, ParseJsonWithNestedKey) { + std::string_view json_with_nested_key = R"({"nested": {"": "value"}})"; + std::string_view excepted_json_with_nested_key = R"({"nested":{"":"value"}})"; + EXPECT_EQ(parse_json_and_check(json_with_nested_key, excepted_json_with_nested_key), + JsonbErrType::E_NONE); +} + +TEST_F(JsonbParserTest, ParseJsonWithNestedArrayElement) { + std::string_view json_with_nested_array = R"({"array": [1, "two", true, null, {}]})"; + std::string_view excepted_json_with_nested_array = R"({"array":[1,"two",true,null,{}]})"; + EXPECT_EQ(parse_json_and_check(json_with_nested_array, excepted_json_with_nested_array), + JsonbErrType::E_NONE); +} + +TEST_F(JsonbParserTest, ParseJsonWithNestedObjectValue) { + std::string_view json_with_nested_object = R"({"nested": {"key": [1, 2, 3]}})"; + std::string_view excepted_json_with_nested_object = R"({"nested":{"key":[1,2,3]}})"; + EXPECT_EQ(parse_json_and_check(json_with_nested_object, excepted_json_with_nested_object), + JsonbErrType::E_NONE); +} + +TEST_F(JsonbParserTest, ParseJsonWithLongDouble) { + std::string_view json_with_long_double = R"({"long_double": 3.1982938928398232132})"; + std::string_view excepted_json_with_long_double = R"({"long_double":3.19829389283982})"; + EXPECT_EQ(parse_json_and_check(json_with_long_double, excepted_json_with_long_double), + JsonbErrType::E_NONE); +} + +TEST_F(JsonbParserTest, ParseJsonWith50DigitDouble) { + std::string_view json_with_50_digit_double = + R"({"double_value": 1.2345678901234567890123456789012345678901234567890})"; + std::string_view expected_json_with_50_digit_double = R"({"double_value":1.23456789012346})"; + EXPECT_EQ(parse_json_and_check(json_with_50_digit_double, expected_json_with_50_digit_double), + JsonbErrType::E_NONE); +} + +TEST_F(JsonbParserTest, ParseJsonWithScientificNotationDouble) { + std::string_view json_with_scientific_double = + R"({"scientific_double": 1.234567890123456789e+50})"; + std::string_view expected_json_with_scientific_double = + R"({"scientific_double":1.23456789012346e+50})"; + EXPECT_EQ( + parse_json_and_check(json_with_scientific_double, expected_json_with_scientific_double), + JsonbErrType::E_NONE); +} + +TEST_F(JsonbParserTest, ParseJsonWithMaxDouble) { + std::string_view json_with_max_double = R"({"max_double": 1.7976931348623157e+308})"; + std::string_view expected_json_with_max_double = R"({"max_double":1.79769313486232e+308})"; + EXPECT_EQ(parse_json_and_check(json_with_max_double, expected_json_with_max_double), + JsonbErrType::E_NONE); +} + +TEST_F(JsonbParserTest, ParseJsonWithMinDouble) { + std::string_view json_with_min_double = R"({"min_double": 2.2250738585072014e-308})"; + std::string_view expected_json_with_min_double = R"({"min_double":2.2250738585072e-308})"; + EXPECT_EQ(parse_json_and_check(json_with_min_double, expected_json_with_min_double), + JsonbErrType::E_NONE); +} + +TEST_F(JsonbParserTest, ParseJsonWithNegativeDouble) { + std::string_view json_with_negative_double = + R"({"negative_double": -1.2345678901234567890123456789012345678901234567890})"; + std::string_view expected_json_with_negative_double = + R"({"negative_double":-1.23456789012346})"; + EXPECT_EQ(parse_json_and_check(json_with_negative_double, expected_json_with_negative_double), + JsonbErrType::E_NONE); +} + +TEST_F(JsonbParserTest, ParseJsonWithNegativeScientificDouble) { + std::string_view json_with_negative_scientific_double = + R"({"negative_scientific_double": -1.234567890123456789e-50})"; + std::string_view expected_json_with_negative_scientific_double = + R"({"negative_scientific_double":-1.23456789012346e-50})"; + EXPECT_EQ(parse_json_and_check(json_with_negative_scientific_double, + expected_json_with_negative_scientific_double), + JsonbErrType::E_NONE); +} + +TEST_F(JsonbParserTest, ParseJsonWithZeroDouble) { + std::string_view json_with_zero_double = R"({"zero_double": 0.0})"; + std::string_view expected_json_with_zero_double = R"({"zero_double":0})"; + EXPECT_EQ(parse_json_and_check(json_with_zero_double, expected_json_with_zero_double), + JsonbErrType::E_NONE); +} + +TEST_F(JsonbParserTest, ParseJsonWithZeroScientificDouble) { + std::string_view json_with_zero_scientific_double = R"({"zero_scientific_double": 0.0e+50})"; + std::string_view expected_json_with_zero_scientific_double = R"({"zero_scientific_double":0})"; + EXPECT_EQ(parse_json_and_check(json_with_zero_scientific_double, + expected_json_with_zero_scientific_double), + JsonbErrType::E_NONE); +} + +TEST_F(JsonbParserTest, ParseJsonWithNearMaxDouble) { + std::string_view json_with_near_max_double = R"({"near_max_double": 1.7976931348623156e+308})"; + std::string_view expected_json_with_near_max_double = + R"({"near_max_double":1.79769313486232e+308})"; + EXPECT_EQ(parse_json_and_check(json_with_near_max_double, expected_json_with_near_max_double), + JsonbErrType::E_NONE); +} +} // namespace doris \ No newline at end of file From 4be54d5980a6b866f536ad01f1e5422acdc2d918 Mon Sep 17 00:00:00 2001 From: minghong Date: Fri, 24 Jan 2025 10:59:33 +0800 Subject: [PATCH 04/14] [fix](regression) some join conditions are missed in tpcds 24 sql (#47303) ### What problem does this PR solve fix sql error for tpcds query 24 in regression cases --- .../suites/inverted_index_p1/tpcds_sf1_index/sql/q24_1.sql | 1 + .../suites/inverted_index_p1/tpcds_sf1_index/sql/q24_2.sql | 1 + regression-test/suites/tpcds_sf1000_p2/sql/q24_1.sql | 1 + regression-test/suites/tpcds_sf1000_p2/sql/q24_2.sql | 1 + .../suites/tpcds_sf100_dup_without_key_p2/sql/q24_2.sql | 1 + regression-test/suites/tpcds_sf100_p2/sql/q24_1.sql | 1 + regression-test/suites/tpcds_sf100_p2/sql/q24_2.sql | 1 + regression-test/suites/tpcds_sf1_p1/sql/q24_1.sql | 1 + regression-test/suites/tpcds_sf1_p1/sql/q24_2.sql | 1 + regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q24_1.groovy | 1 + regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q24_2.groovy | 1 + regression-test/suites/tpcds_sf1_unique_ck_p1/sql/q24_1.sql | 1 + regression-test/suites/tpcds_sf1_unique_ck_p1/sql/q24_2.sql | 1 + regression-test/suites/tpcds_sf1_unique_p1/spill/q24_1.groovy | 1 + regression-test/suites/tpcds_sf1_unique_p1/spill/q24_2.groovy | 1 + regression-test/suites/tpcds_sf1_unique_p1/sql/q24_1.sql | 1 + regression-test/suites/tpcds_sf1_unique_p1/sql/q24_2.sql | 1 + 17 files changed, 17 insertions(+) diff --git a/regression-test/suites/inverted_index_p1/tpcds_sf1_index/sql/q24_1.sql b/regression-test/suites/inverted_index_p1/tpcds_sf1_index/sql/q24_1.sql index 65571698e1ecc7..1f9cf6888ddc57 100644 --- a/regression-test/suites/inverted_index_p1/tpcds_sf1_index/sql/q24_1.sql +++ b/regression-test/suites/inverted_index_p1/tpcds_sf1_index/sql/q24_1.sql @@ -24,6 +24,7 @@ WITH AND (ss_customer_sk = c_customer_sk) AND (ss_item_sk = i_item_sk) AND (ss_store_sk = s_store_sk) + AND (c_current_addr_sk = ca_address_sk) AND (c_birth_country = upper(ca_country)) AND (s_zip = ca_zip) AND (s_market_id = 8) diff --git a/regression-test/suites/inverted_index_p1/tpcds_sf1_index/sql/q24_2.sql b/regression-test/suites/inverted_index_p1/tpcds_sf1_index/sql/q24_2.sql index db04507c9c98c8..b89577eac863a9 100644 --- a/regression-test/suites/inverted_index_p1/tpcds_sf1_index/sql/q24_2.sql +++ b/regression-test/suites/inverted_index_p1/tpcds_sf1_index/sql/q24_2.sql @@ -24,6 +24,7 @@ WITH AND (ss_customer_sk = c_customer_sk) AND (ss_item_sk = i_item_sk) AND (ss_store_sk = s_store_sk) + AND (c_current_addr_sk = ca_address_sk) AND (c_birth_country = upper(ca_country)) AND (s_zip = ca_zip) AND (s_market_id = 8) diff --git a/regression-test/suites/tpcds_sf1000_p2/sql/q24_1.sql b/regression-test/suites/tpcds_sf1000_p2/sql/q24_1.sql index 65571698e1ecc7..457522c6fb1b5a 100644 --- a/regression-test/suites/tpcds_sf1000_p2/sql/q24_1.sql +++ b/regression-test/suites/tpcds_sf1000_p2/sql/q24_1.sql @@ -24,6 +24,7 @@ WITH AND (ss_customer_sk = c_customer_sk) AND (ss_item_sk = i_item_sk) AND (ss_store_sk = s_store_sk) + AND (c_current_addr_sk = ca_address_sk) AND (c_birth_country = upper(ca_country)) AND (s_zip = ca_zip) AND (s_market_id = 8) diff --git a/regression-test/suites/tpcds_sf1000_p2/sql/q24_2.sql b/regression-test/suites/tpcds_sf1000_p2/sql/q24_2.sql index db04507c9c98c8..b89577eac863a9 100644 --- a/regression-test/suites/tpcds_sf1000_p2/sql/q24_2.sql +++ b/regression-test/suites/tpcds_sf1000_p2/sql/q24_2.sql @@ -24,6 +24,7 @@ WITH AND (ss_customer_sk = c_customer_sk) AND (ss_item_sk = i_item_sk) AND (ss_store_sk = s_store_sk) + AND (c_current_addr_sk = ca_address_sk) AND (c_birth_country = upper(ca_country)) AND (s_zip = ca_zip) AND (s_market_id = 8) diff --git a/regression-test/suites/tpcds_sf100_dup_without_key_p2/sql/q24_2.sql b/regression-test/suites/tpcds_sf100_dup_without_key_p2/sql/q24_2.sql index b8f7099eb47dff..6873a56b4afce4 100644 --- a/regression-test/suites/tpcds_sf100_dup_without_key_p2/sql/q24_2.sql +++ b/regression-test/suites/tpcds_sf100_dup_without_key_p2/sql/q24_2.sql @@ -24,6 +24,7 @@ WITH AND (ss_customer_sk = c_customer_sk) AND (ss_item_sk = i_item_sk) AND (ss_store_sk = s_store_sk) + AND (c_current_addr_sk = ca_address_sk) AND (c_birth_country = upper(ca_country)) AND (s_zip = ca_zip) AND (s_market_id = 8) diff --git a/regression-test/suites/tpcds_sf100_p2/sql/q24_1.sql b/regression-test/suites/tpcds_sf100_p2/sql/q24_1.sql index 71711c8579cf4c..52f3ee657da79f 100644 --- a/regression-test/suites/tpcds_sf100_p2/sql/q24_1.sql +++ b/regression-test/suites/tpcds_sf100_p2/sql/q24_1.sql @@ -24,6 +24,7 @@ WITH AND (ss_customer_sk = c_customer_sk) AND (ss_item_sk = i_item_sk) AND (ss_store_sk = s_store_sk) + AND (c_current_addr_sk = ca_address_sk) AND (c_birth_country = upper(ca_country)) AND (s_zip = ca_zip) AND (s_market_id = 8) diff --git a/regression-test/suites/tpcds_sf100_p2/sql/q24_2.sql b/regression-test/suites/tpcds_sf100_p2/sql/q24_2.sql index b8f7099eb47dff..6873a56b4afce4 100644 --- a/regression-test/suites/tpcds_sf100_p2/sql/q24_2.sql +++ b/regression-test/suites/tpcds_sf100_p2/sql/q24_2.sql @@ -24,6 +24,7 @@ WITH AND (ss_customer_sk = c_customer_sk) AND (ss_item_sk = i_item_sk) AND (ss_store_sk = s_store_sk) + AND (c_current_addr_sk = ca_address_sk) AND (c_birth_country = upper(ca_country)) AND (s_zip = ca_zip) AND (s_market_id = 8) diff --git a/regression-test/suites/tpcds_sf1_p1/sql/q24_1.sql b/regression-test/suites/tpcds_sf1_p1/sql/q24_1.sql index 65571698e1ecc7..1f9cf6888ddc57 100644 --- a/regression-test/suites/tpcds_sf1_p1/sql/q24_1.sql +++ b/regression-test/suites/tpcds_sf1_p1/sql/q24_1.sql @@ -24,6 +24,7 @@ WITH AND (ss_customer_sk = c_customer_sk) AND (ss_item_sk = i_item_sk) AND (ss_store_sk = s_store_sk) + AND (c_current_addr_sk = ca_address_sk) AND (c_birth_country = upper(ca_country)) AND (s_zip = ca_zip) AND (s_market_id = 8) diff --git a/regression-test/suites/tpcds_sf1_p1/sql/q24_2.sql b/regression-test/suites/tpcds_sf1_p1/sql/q24_2.sql index db04507c9c98c8..b89577eac863a9 100644 --- a/regression-test/suites/tpcds_sf1_p1/sql/q24_2.sql +++ b/regression-test/suites/tpcds_sf1_p1/sql/q24_2.sql @@ -24,6 +24,7 @@ WITH AND (ss_customer_sk = c_customer_sk) AND (ss_item_sk = i_item_sk) AND (ss_store_sk = s_store_sk) + AND (c_current_addr_sk = ca_address_sk) AND (c_birth_country = upper(ca_country)) AND (s_zip = ca_zip) AND (s_market_id = 8) diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q24_1.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q24_1.groovy index 1847ff8a5c65d1..80a83e3de8eed5 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q24_1.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q24_1.groovy @@ -51,6 +51,7 @@ WITH AND (ss_customer_sk = c_customer_sk) AND (ss_item_sk = i_item_sk) AND (ss_store_sk = s_store_sk) + AND (c_current_addr_sk = ca_address_sk) AND (c_birth_country = upper(ca_country)) AND (s_zip = ca_zip) AND (s_market_id = 8) diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q24_2.groovy b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q24_2.groovy index 69d40f764c5777..f2068e3203552a 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q24_2.groovy +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/spill/q24_2.groovy @@ -51,6 +51,7 @@ WITH AND (ss_customer_sk = c_customer_sk) AND (ss_item_sk = i_item_sk) AND (ss_store_sk = s_store_sk) + AND (c_current_addr_sk = ca_address_sk) AND (c_birth_country = upper(ca_country)) AND (s_zip = ca_zip) AND (s_market_id = 8) diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/sql/q24_1.sql b/regression-test/suites/tpcds_sf1_unique_ck_p1/sql/q24_1.sql index 65571698e1ecc7..1f9cf6888ddc57 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/sql/q24_1.sql +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/sql/q24_1.sql @@ -24,6 +24,7 @@ WITH AND (ss_customer_sk = c_customer_sk) AND (ss_item_sk = i_item_sk) AND (ss_store_sk = s_store_sk) + AND (c_current_addr_sk = ca_address_sk) AND (c_birth_country = upper(ca_country)) AND (s_zip = ca_zip) AND (s_market_id = 8) diff --git a/regression-test/suites/tpcds_sf1_unique_ck_p1/sql/q24_2.sql b/regression-test/suites/tpcds_sf1_unique_ck_p1/sql/q24_2.sql index db04507c9c98c8..b89577eac863a9 100644 --- a/regression-test/suites/tpcds_sf1_unique_ck_p1/sql/q24_2.sql +++ b/regression-test/suites/tpcds_sf1_unique_ck_p1/sql/q24_2.sql @@ -24,6 +24,7 @@ WITH AND (ss_customer_sk = c_customer_sk) AND (ss_item_sk = i_item_sk) AND (ss_store_sk = s_store_sk) + AND (c_current_addr_sk = ca_address_sk) AND (c_birth_country = upper(ca_country)) AND (s_zip = ca_zip) AND (s_market_id = 8) diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q24_1.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q24_1.groovy index 1c3df66e9df461..8f5982d2490fc2 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q24_1.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q24_1.groovy @@ -51,6 +51,7 @@ WITH AND (ss_customer_sk = c_customer_sk) AND (ss_item_sk = i_item_sk) AND (ss_store_sk = s_store_sk) + AND (c_current_addr_sk = ca_address_sk) AND (c_birth_country = upper(ca_country)) AND (s_zip = ca_zip) AND (s_market_id = 8) diff --git a/regression-test/suites/tpcds_sf1_unique_p1/spill/q24_2.groovy b/regression-test/suites/tpcds_sf1_unique_p1/spill/q24_2.groovy index ee4e3407e9f3ee..ddede02fd54b63 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/spill/q24_2.groovy +++ b/regression-test/suites/tpcds_sf1_unique_p1/spill/q24_2.groovy @@ -51,6 +51,7 @@ WITH AND (ss_customer_sk = c_customer_sk) AND (ss_item_sk = i_item_sk) AND (ss_store_sk = s_store_sk) + AND (c_current_addr_sk = ca_address_sk) AND (c_birth_country = upper(ca_country)) AND (s_zip = ca_zip) AND (s_market_id = 8) diff --git a/regression-test/suites/tpcds_sf1_unique_p1/sql/q24_1.sql b/regression-test/suites/tpcds_sf1_unique_p1/sql/q24_1.sql index 65571698e1ecc7..1f9cf6888ddc57 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/sql/q24_1.sql +++ b/regression-test/suites/tpcds_sf1_unique_p1/sql/q24_1.sql @@ -24,6 +24,7 @@ WITH AND (ss_customer_sk = c_customer_sk) AND (ss_item_sk = i_item_sk) AND (ss_store_sk = s_store_sk) + AND (c_current_addr_sk = ca_address_sk) AND (c_birth_country = upper(ca_country)) AND (s_zip = ca_zip) AND (s_market_id = 8) diff --git a/regression-test/suites/tpcds_sf1_unique_p1/sql/q24_2.sql b/regression-test/suites/tpcds_sf1_unique_p1/sql/q24_2.sql index db04507c9c98c8..b89577eac863a9 100644 --- a/regression-test/suites/tpcds_sf1_unique_p1/sql/q24_2.sql +++ b/regression-test/suites/tpcds_sf1_unique_p1/sql/q24_2.sql @@ -24,6 +24,7 @@ WITH AND (ss_customer_sk = c_customer_sk) AND (ss_item_sk = i_item_sk) AND (ss_store_sk = s_store_sk) + AND (c_current_addr_sk = ca_address_sk) AND (c_birth_country = upper(ca_country)) AND (s_zip = ca_zip) AND (s_market_id = 8) From 384eceeb8951c424123343f730f5ba1cfa9947f4 Mon Sep 17 00:00:00 2001 From: wuwenchi Date: Fri, 24 Jan 2025 11:10:21 +0800 Subject: [PATCH 05/14] [opt](iceberg)support iceberg in batch mode (#46398) ### What problem does this PR solve? 1. use thread pool in planFiles. 2. support batch mode. 3. When executing `doAs`, if the exception type is `UndeclaredThrowableException`, we should get the specific cause inside. Because the message of UndeclaredThrowableException is null, the message will not be obtained if the exception is captured externally 4. Modify the default configuration: fetch_splits_max_wait_time_ms: from 4000 to 1000 remote_split_source_batch_size: from 10240 to 1000 --- be/src/common/config.cpp | 2 +- .../authentication/HadoopAuthenticator.java | 7 + .../PreExecutionAuthenticator.java | 48 ++---- .../doris/common/ThreadPoolManager.java | 69 ++++++++ .../doris/datasource/ExternalCatalog.java | 6 + .../doris/datasource/FileQueryScanNode.java | 8 +- .../doris/datasource/SplitAssignment.java | 30 +++- .../doris/datasource/SplitGenerator.java | 2 +- .../iceberg/IcebergExternalCatalog.java | 15 ++ .../iceberg/IcebergExternalTable.java | 2 +- .../iceberg/IcebergHMSExternalCatalog.java | 7 - .../iceberg/IcebergMetadataOps.java | 28 +++- .../iceberg/IcebergPartitionInfo.java | 10 +- .../iceberg/source/IcebergScanNode.java | 147 +++++++++++++----- .../iceberg/source/IcebergSplit.java | 3 +- .../org/apache/doris/planner/ScanNode.java | 2 +- .../org/apache/doris/qe/SessionVariable.java | 2 +- .../iceberg/test_iceberg_filter.groovy | 2 + 18 files changed, 289 insertions(+), 101 deletions(-) diff --git a/be/src/common/config.cpp b/be/src/common/config.cpp index 39549d1898628a..8f068d11549468 100644 --- a/be/src/common/config.cpp +++ b/be/src/common/config.cpp @@ -293,7 +293,7 @@ DEFINE_Validator(doris_scanner_thread_pool_thread_num, [](const int config) -> b return true; }); DEFINE_Int32(doris_scanner_min_thread_pool_thread_num, "8"); -DEFINE_Int32(remote_split_source_batch_size, "10240"); +DEFINE_Int32(remote_split_source_batch_size, "1000"); DEFINE_Int32(doris_max_remote_scanner_thread_pool_thread_num, "-1"); // number of olap scanner thread pool queue size DEFINE_Int32(doris_scanner_thread_pool_queue_size, "102400"); diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/security/authentication/HadoopAuthenticator.java b/fe/fe-common/src/main/java/org/apache/doris/common/security/authentication/HadoopAuthenticator.java index c3cab5f410be3a..88d32a593e14dc 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/common/security/authentication/HadoopAuthenticator.java +++ b/fe/fe-common/src/main/java/org/apache/doris/common/security/authentication/HadoopAuthenticator.java @@ -20,6 +20,7 @@ import org.apache.hadoop.security.UserGroupInformation; import java.io.IOException; +import java.lang.reflect.UndeclaredThrowableException; import java.security.PrivilegedExceptionAction; public interface HadoopAuthenticator { @@ -31,6 +32,12 @@ default T doAs(PrivilegedExceptionAction action) throws IOException { return getUGI().doAs(action); } catch (InterruptedException e) { throw new IOException(e); + } catch (UndeclaredThrowableException e) { + if (e.getCause() instanceof RuntimeException) { + throw (RuntimeException) e.getCause(); + } else { + throw new RuntimeException(e.getCause()); + } } } diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/security/authentication/PreExecutionAuthenticator.java b/fe/fe-common/src/main/java/org/apache/doris/common/security/authentication/PreExecutionAuthenticator.java index 6260833b7db558..a64dd4cf717f0e 100644 --- a/fe/fe-common/src/main/java/org/apache/doris/common/security/authentication/PreExecutionAuthenticator.java +++ b/fe/fe-common/src/main/java/org/apache/doris/common/security/authentication/PreExecutionAuthenticator.java @@ -17,7 +17,6 @@ package org.apache.doris.common.security.authentication; -import java.security.PrivilegedExceptionAction; import java.util.concurrent.Callable; /** @@ -54,14 +53,26 @@ public PreExecutionAuthenticator() { public T execute(Callable task) throws Exception { if (hadoopAuthenticator != null) { // Adapts Callable to PrivilegedExceptionAction for use with Hadoop authentication - PrivilegedExceptionAction action = new CallableToPrivilegedExceptionActionAdapter<>(task); - return hadoopAuthenticator.doAs(action); + return hadoopAuthenticator.doAs(task::call); } else { // Executes the task directly if no authentication is needed return task.call(); } } + public void execute(Runnable task) throws Exception { + if (hadoopAuthenticator != null) { + // Adapts Runnable to PrivilegedExceptionAction for use with Hadoop authentication + hadoopAuthenticator.doAs(() -> { + task.run(); + return null; + }); + } else { + // Executes the task directly if no authentication is needed + task.run(); + } + } + /** * Retrieves the current HadoopAuthenticator. *

This allows checking if a HadoopAuthenticator is configured or @@ -82,35 +93,4 @@ public HadoopAuthenticator getHadoopAuthenticator() { public void setHadoopAuthenticator(HadoopAuthenticator hadoopAuthenticator) { this.hadoopAuthenticator = hadoopAuthenticator; } - - /** - * Adapter class to convert a Callable into a PrivilegedExceptionAction. - *

This is necessary to run the task within a privileged context, - * particularly for Hadoop operations with Kerberos. - * - * @param The type of result returned by the action - */ - public class CallableToPrivilegedExceptionActionAdapter implements PrivilegedExceptionAction { - private final Callable callable; - - /** - * Constructs an adapter that wraps a Callable into a PrivilegedExceptionAction. - * - * @param callable The Callable to be adapted - */ - public CallableToPrivilegedExceptionActionAdapter(Callable callable) { - this.callable = callable; - } - - /** - * Executes the wrapped Callable as a PrivilegedExceptionAction. - * - * @return The result of the callable's call method - * @throws Exception If an exception occurs during callable execution - */ - @Override - public T run() throws Exception { - return callable.call(); - } - } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/ThreadPoolManager.java b/fe/fe-core/src/main/java/org/apache/doris/common/ThreadPoolManager.java index dcb1f704271eab..0822a322dece81 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/ThreadPoolManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/ThreadPoolManager.java @@ -18,6 +18,7 @@ package org.apache.doris.common; +import org.apache.doris.common.security.authentication.PreExecutionAuthenticator; import org.apache.doris.metric.Metric; import org.apache.doris.metric.Metric.MetricUnit; import org.apache.doris.metric.MetricLabel; @@ -33,6 +34,7 @@ import java.util.Map; import java.util.concurrent.BlockingQueue; import java.util.concurrent.Callable; +import java.util.concurrent.ExecutorService; import java.util.concurrent.FutureTask; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.PriorityBlockingQueue; @@ -69,6 +71,7 @@ public class ThreadPoolManager { + private static final Logger LOG = LogManager.getLogger(ThreadPoolManager.class); private static Map nameToThreadPoolMap = Maps.newConcurrentMap(); private static String[] poolMetricTypes = {"pool_size", "active_thread_num", "task_in_queue"}; @@ -140,6 +143,17 @@ public static ThreadPoolExecutor newDaemonFixedThreadPool(int numThread, poolName, needRegisterMetric); } + public static ThreadPoolExecutor newDaemonFixedThreadPoolWithPreAuth( + int numThread, + int queueSize, + String poolName, + boolean needRegisterMetric, + PreExecutionAuthenticator preAuth) { + return newDaemonThreadPoolWithPreAuth(numThread, numThread, KEEP_ALIVE_TIME, TimeUnit.SECONDS, + new LinkedBlockingQueue<>(queueSize), new BlockedPolicy(poolName, 60), + poolName, needRegisterMetric, preAuth); + } + public static ThreadPoolExecutor newDaemonFixedThreadPool(int numThread, int queueSize, String poolName, int timeoutSeconds, boolean needRegisterMetric) { @@ -229,6 +243,40 @@ private static ThreadFactory namedThreadFactory(String poolName) { return new ThreadFactoryBuilder().setDaemon(true).setNameFormat(poolName + "-%d").build(); } + + public static ThreadPoolExecutor newDaemonThreadPoolWithPreAuth( + int corePoolSize, + int maximumPoolSize, + long keepAliveTime, + TimeUnit unit, + BlockingQueue workQueue, + RejectedExecutionHandler handler, + String poolName, + boolean needRegisterMetric, + PreExecutionAuthenticator preAuth) { + ThreadFactory threadFactory = namedThreadFactoryWithPreAuth(poolName, preAuth); + ThreadPoolExecutor threadPool = new ThreadPoolExecutor(corePoolSize, maximumPoolSize, + keepAliveTime, unit, workQueue, threadFactory, handler); + if (needRegisterMetric) { + nameToThreadPoolMap.put(poolName, threadPool); + } + return threadPool; + } + + private static ThreadFactory namedThreadFactoryWithPreAuth(String poolName, PreExecutionAuthenticator preAuth) { + return new ThreadFactoryBuilder() + .setDaemon(true) + .setNameFormat(poolName + "-%d") + .setThreadFactory(runnable -> new Thread(() -> { + try { + preAuth.execute(runnable); + } catch (Exception e) { + throw new RuntimeException(e); + } + })) + .build(); + } + private static class PriorityThreadPoolExecutor extends ThreadPoolExecutor { private final Comparator comparator; @@ -384,4 +432,25 @@ public void rejectedExecution(Runnable task, ThreadPoolExecutor executor) { } } } + + public static void shutdownExecutorService(ExecutorService executorService) { + // Disable new tasks from being submitted + executorService.shutdown(); + try { + // Wait a while for existing tasks to terminate + if (!executorService.awaitTermination(60, TimeUnit.SECONDS)) { + // Cancel currently executing tasks + executorService.shutdownNow(); + // Wait a while for tasks to respond to being cancelled + if (!executorService.awaitTermination(60, TimeUnit.SECONDS)) { + LOG.warn("ExecutorService did not terminate"); + } + } + } catch (InterruptedException e) { + // (Re-)Cancel if current thread also interrupted + executorService.shutdownNow(); + // Preserve interrupt status + Thread.currentThread().interrupt(); + } + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java index b1eb8ab6afe0f3..fc0f674d206529 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java @@ -35,6 +35,7 @@ import org.apache.doris.common.DdlException; import org.apache.doris.common.FeConstants; import org.apache.doris.common.Pair; +import org.apache.doris.common.ThreadPoolManager; import org.apache.doris.common.UserException; import org.apache.doris.common.Version; import org.apache.doris.common.io.Text; @@ -92,6 +93,7 @@ import java.util.Optional; import java.util.OptionalLong; import java.util.Set; +import java.util.concurrent.ThreadPoolExecutor; import java.util.stream.Collectors; /** @@ -158,6 +160,7 @@ public abstract class ExternalCatalog protected Optional useMetaCache = Optional.empty(); protected MetaCache> metaCache; protected PreExecutionAuthenticator preExecutionAuthenticator; + protected ThreadPoolExecutor threadPoolWithPreAuth; private volatile Configuration cachedConf = null; private byte[] confLock = new byte[0]; @@ -716,6 +719,9 @@ public void write(DataOutput out) throws IOException { @Override public void onClose() { removeAccessController(); + if (threadPoolWithPreAuth != null) { + ThreadPoolManager.shutdownExecutorService(threadPoolWithPreAuth); + } CatalogIf.super.onClose(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/FileQueryScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/FileQueryScanNode.java index 9edab0e55dee79..14c82c1606eb52 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/FileQueryScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/FileQueryScanNode.java @@ -333,17 +333,19 @@ public void createScanRangeLocations() throws UserException { if (splitAssignment.getSampleSplit() == null && !isFileStreamType()) { return; } - selectedSplitNum = numApproximateSplits(); - FileSplit fileSplit = (FileSplit) splitAssignment.getSampleSplit(); TFileType locationType = fileSplit.getLocationType(); + selectedSplitNum = numApproximateSplits(); + if (selectedSplitNum < 0) { + throw new UserException("Approximate split number should not be negative"); + } totalFileSize = fileSplit.getLength() * selectedSplitNum; long maxWaitTime = sessionVariable.getFetchSplitsMaxWaitTime(); // Not accurate, only used to estimate concurrency. // Here, we must take the max of 1, because // in the case of multiple BEs, `numApproximateSplits() / backendPolicy.numBackends()` may be 0, // and finally numSplitsPerBE is 0, resulting in no data being queried. - int numSplitsPerBE = Math.max(numApproximateSplits() / backendPolicy.numBackends(), 1); + int numSplitsPerBE = Math.max(selectedSplitNum / backendPolicy.numBackends(), 1); for (Backend backend : backendPolicy.getBackends()) { SplitSource splitSource = new SplitSource(backend, splitAssignment, maxWaitTime); splitSources.add(splitSource); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/SplitAssignment.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/SplitAssignment.java index a26abc7fc5e037..470f4092d21ff9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/SplitAssignment.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/SplitAssignment.java @@ -23,7 +23,10 @@ import org.apache.doris.thrift.TScanRangeLocations; import com.google.common.collect.Multimap; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import java.io.Closeable; import java.util.ArrayList; import java.util.Collection; import java.util.HashSet; @@ -40,6 +43,7 @@ * `SplitGenerator` provides the file splits, and `FederationBackendPolicy` assigns these splits to backends. */ public class SplitAssignment { + private static final Logger LOG = LogManager.getLogger(SplitAssignment.class); private final Set sources = new HashSet<>(); private final FederationBackendPolicy backendPolicy; private final SplitGenerator splitGenerator; @@ -50,10 +54,11 @@ public class SplitAssignment { private final List pathPartitionKeys; private final Object assignLock = new Object(); private Split sampleSplit = null; - private final AtomicBoolean isStop = new AtomicBoolean(false); + private final AtomicBoolean isStopped = new AtomicBoolean(false); private final AtomicBoolean scheduleFinished = new AtomicBoolean(false); private UserException exception = null; + private final List closeableResources = new ArrayList<>(); public SplitAssignment( FederationBackendPolicy backendPolicy, @@ -85,7 +90,7 @@ public void init() throws UserException { } private boolean waitFirstSplit() { - return !scheduleFinished.get() && !isStop.get() && exception == null; + return !scheduleFinished.get() && !isStopped.get() && exception == null; } private void appendBatch(Multimap batch) throws UserException { @@ -150,7 +155,7 @@ public BlockingQueue> getAssignedSplits(Backend } BlockingQueue> splits = assignment.computeIfAbsent(backend, be -> new LinkedBlockingQueue<>()); - if (scheduleFinished.get() && splits.isEmpty() || isStop.get()) { + if (scheduleFinished.get() && splits.isEmpty() || isStopped.get()) { return null; } return splits; @@ -167,11 +172,26 @@ public void finishSchedule() { } public void stop() { - isStop.set(true); + if (isStop()) { + return; + } + isStopped.set(true); + closeableResources.forEach((closeable) -> { + try { + closeable.close(); + } catch (Exception e) { + LOG.warn("close resource error:{}", e.getMessage(), e); + // ignore + } + }); notifyAssignment(); } public boolean isStop() { - return isStop.get(); + return isStopped.get(); + } + + public void addCloseable(Closeable resource) { + closeableResources.add(resource); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/SplitGenerator.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/SplitGenerator.java index 34ff3911445bfe..391552a5106a83 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/SplitGenerator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/SplitGenerator.java @@ -52,7 +52,7 @@ default int numApproximateSplits() { return -1; } - default void startSplit(int numBackends) { + default void startSplit(int numBackends) throws UserException { } /** diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalCatalog.java index 0fa69825a01ef4..6a3265388f3345 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalCatalog.java @@ -17,6 +17,9 @@ package org.apache.doris.datasource.iceberg; +import org.apache.doris.common.ThreadPoolManager; +import org.apache.doris.common.security.authentication.AuthenticationConfig; +import org.apache.doris.common.security.authentication.HadoopAuthenticator; import org.apache.doris.common.security.authentication.PreExecutionAuthenticator; import org.apache.doris.datasource.ExternalCatalog; import org.apache.doris.datasource.InitCatalogLog; @@ -43,6 +46,7 @@ public abstract class IcebergExternalCatalog extends ExternalCatalog { public static final String EXTERNAL_CATALOG_NAME = "external_catalog.name"; protected String icebergCatalogType; protected Catalog catalog; + private static final int ICEBERG_CATALOG_EXECUTOR_THREAD_NUM = 16; public IcebergExternalCatalog(long catalogId, String name, String comment) { super(catalogId, name, InitCatalogLog.Type.ICEBERG, comment); @@ -54,9 +58,20 @@ public IcebergExternalCatalog(long catalogId, String name, String comment) { @Override protected void initLocalObjectsImpl() { preExecutionAuthenticator = new PreExecutionAuthenticator(); + // TODO If the storage environment does not support Kerberos (such as s3), + // there is no need to generate a simple authentication information anymore. + AuthenticationConfig config = AuthenticationConfig.getKerberosConfig(getConfiguration()); + HadoopAuthenticator authenticator = HadoopAuthenticator.getHadoopAuthenticator(config); + preExecutionAuthenticator.setHadoopAuthenticator(authenticator); initCatalog(); IcebergMetadataOps ops = ExternalMetadataOperations.newIcebergMetadataOps(this, catalog); transactionManager = TransactionManagerFactory.createIcebergTransactionManager(ops); + threadPoolWithPreAuth = ThreadPoolManager.newDaemonFixedThreadPoolWithPreAuth( + ICEBERG_CATALOG_EXECUTOR_THREAD_NUM, + Integer.MAX_VALUE, + String.format("iceberg_catalog_%s_executor_pool", name), + true, + preExecutionAuthenticator); metadataOps = ops; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalTable.java index 2feab480d7edb8..23b0b0cbba34b1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergExternalTable.java @@ -323,7 +323,7 @@ public IcebergSchemaCacheValue getIcebergSchemaCacheValue(long schemaId) { public IcebergPartitionInfo loadPartitionInfo(long snapshotId) throws AnalysisException { // snapshotId == UNKNOWN_SNAPSHOT_ID means this is an empty table, haven't contained any snapshot yet. if (!isValidRelatedTable() || snapshotId == IcebergUtils.UNKNOWN_SNAPSHOT_ID) { - return new IcebergPartitionInfo(); + return IcebergPartitionInfo.empty(); } List icebergPartitions = loadIcebergPartition(snapshotId); Map nameToPartition = Maps.newHashMap(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergHMSExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergHMSExternalCatalog.java index c5a99c157ce8e6..51d39357b816fa 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergHMSExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergHMSExternalCatalog.java @@ -17,8 +17,6 @@ package org.apache.doris.datasource.iceberg; -import org.apache.doris.common.security.authentication.AuthenticationConfig; -import org.apache.doris.common.security.authentication.HadoopAuthenticator; import org.apache.doris.datasource.CatalogProperty; import org.apache.doris.datasource.property.PropertyConverter; @@ -37,11 +35,6 @@ public IcebergHMSExternalCatalog(long catalogId, String name, String resource, M protected void initCatalog() { icebergCatalogType = ICEBERG_HMS; catalog = IcebergUtils.createIcebergHiveCatalog(this, getName()); - if (preExecutionAuthenticator.getHadoopAuthenticator() == null) { - AuthenticationConfig config = AuthenticationConfig.getKerberosConfig(getConfiguration()); - HadoopAuthenticator authenticator = HadoopAuthenticator.getHadoopAuthenticator(config); - preExecutionAuthenticator.setHadoopAuthenticator(authenticator); - } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataOps.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataOps.java index f8e7181ea929fa..8c3277316d7477 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataOps.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergMetadataOps.java @@ -91,11 +91,19 @@ public void close() { @Override public boolean tableExist(String dbName, String tblName) { - return catalog.tableExists(getTableIdentifier(dbName, tblName)); + try { + return preExecutionAuthenticator.execute(() -> catalog.tableExists(getTableIdentifier(dbName, tblName))); + } catch (Exception e) { + throw new RuntimeException("Failed to check table exist, error message is:" + e.getMessage(), e); + } } public boolean databaseExist(String dbName) { - return nsCatalog.namespaceExists(getNamespace(dbName)); + try { + return preExecutionAuthenticator.execute(() -> nsCatalog.namespaceExists(getNamespace(dbName))); + } catch (Exception e) { + throw new RuntimeException("Failed to check database exist, error message is:" + e.getMessage(), e); + } } public List listDatabaseNames() { @@ -112,8 +120,14 @@ public List listDatabaseNames() { @Override public List listTableNames(String dbName) { - List tableIdentifiers = catalog.listTables(getNamespace(dbName)); - return tableIdentifiers.stream().map(TableIdentifier::name).collect(Collectors.toList()); + try { + return preExecutionAuthenticator.execute(() -> { + List tableIdentifiers = catalog.listTables(getNamespace(dbName)); + return tableIdentifiers.stream().map(TableIdentifier::name).collect(Collectors.toList()); + }); + } catch (Exception e) { + throw new RuntimeException("Failed to list table names, error message is:" + e.getMessage(), e); + } } @Override @@ -275,7 +289,11 @@ public PreExecutionAuthenticator getPreExecutionAuthenticator() { @Override public Table loadTable(String dbName, String tblName) { - return catalog.loadTable(getTableIdentifier(dbName, tblName)); + try { + return preExecutionAuthenticator.execute(() -> catalog.loadTable(getTableIdentifier(dbName, tblName))); + } catch (Exception e) { + throw new RuntimeException("Failed to load table, error message is:" + e.getMessage(), e); + } } private TableIdentifier getTableIdentifier(String dbName, String tblName) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergPartitionInfo.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergPartitionInfo.java index 9edb2137f4f389..04f3b22a026098 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergPartitionInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergPartitionInfo.java @@ -29,20 +29,26 @@ public class IcebergPartitionInfo { private final Map nameToIcebergPartition; private final Map> nameToIcebergPartitionNames; - public IcebergPartitionInfo() { + private static final IcebergPartitionInfo EMPTY = new IcebergPartitionInfo(); + + private IcebergPartitionInfo() { this.nameToPartitionItem = Maps.newHashMap(); this.nameToIcebergPartition = Maps.newHashMap(); this.nameToIcebergPartitionNames = Maps.newHashMap(); } public IcebergPartitionInfo(Map nameToPartitionItem, - Map nameToIcebergPartition, + Map nameToIcebergPartition, Map> nameToIcebergPartitionNames) { this.nameToPartitionItem = nameToPartitionItem; this.nameToIcebergPartition = nameToIcebergPartition; this.nameToIcebergPartitionNames = nameToIcebergPartitionNames; } + static IcebergPartitionInfo empty() { + return EMPTY; + } + public Map getNameToPartitionItem() { return nameToPartitionItem; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/IcebergScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/IcebergScanNode.java index 29c07be81929fb..756c9024cdcdd1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/IcebergScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/IcebergScanNode.java @@ -25,6 +25,7 @@ import org.apache.doris.catalog.TableIf; import org.apache.doris.common.DdlException; import org.apache.doris.common.UserException; +import org.apache.doris.common.security.authentication.PreExecutionAuthenticator; import org.apache.doris.common.util.LocationPath; import org.apache.doris.common.util.TimeUtils; import org.apache.doris.datasource.ExternalTable; @@ -52,7 +53,6 @@ import com.google.common.collect.Lists; import org.apache.commons.lang3.exception.ExceptionUtils; import org.apache.iceberg.BaseTable; -import org.apache.iceberg.CombinedScanTask; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileContent; import org.apache.iceberg.FileScanTask; @@ -72,11 +72,12 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collections; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.OptionalLong; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; public class IcebergScanNode extends FileQueryScanNode { @@ -94,6 +95,11 @@ public class IcebergScanNode extends FileQueryScanNode { // And for split level count push down opt, the flag is set in each split. private boolean tableLevelPushDownCount = false; private static final long COUNT_WITH_PARALLEL_SPLITS = 10000; + private long targetSplitSize; + private ConcurrentHashMap.KeySetView partitionPathSet; + private boolean isPartitionedTable; + private int formatVersion; + private PreExecutionAuthenticator preExecutionAuthenticator; /** * External file scan node for Query iceberg table @@ -128,6 +134,11 @@ public IcebergScanNode(PlanNodeId id, TupleDescriptor desc, boolean needCheckCol @Override protected void doInitialize() throws UserException { icebergTable = source.getIcebergTable(); + targetSplitSize = getRealFileSplitSize(0); + partitionPathSet = ConcurrentHashMap.newKeySet(); + isPartitionedTable = icebergTable.spec().isPartitioned(); + formatVersion = ((BaseTable) icebergTable).operations().current().formatVersion(); + preExecutionAuthenticator = source.getCatalog().getPreExecutionAuthenticator(); super.doInitialize(); } @@ -142,7 +153,6 @@ private void setIcebergParams(TFileRangeDesc rangeDesc, IcebergSplit icebergSpli TTableFormatFileDesc tableFormatFileDesc = new TTableFormatFileDesc(); tableFormatFileDesc.setTableFormatType(icebergSplit.getTableFormatType().value()); TIcebergFileDesc fileDesc = new TIcebergFileDesc(); - int formatVersion = icebergSplit.getFormatVersion(); fileDesc.setFormatVersion(formatVersion); fileDesc.setOriginalFilePath(icebergSplit.getOriginalPath()); if (tableLevelPushDownCount) { @@ -184,14 +194,50 @@ private void setIcebergParams(TFileRangeDesc rangeDesc, IcebergSplit icebergSpli @Override public List getSplits(int numBackends) throws UserException { try { - return source.getCatalog().getPreExecutionAuthenticator().execute(() -> doGetSplits(numBackends)); + return preExecutionAuthenticator.execute(() -> doGetSplits(numBackends)); } catch (Exception e) { throw new RuntimeException(ExceptionUtils.getRootCauseMessage(e), e); } + } + @Override + public void startSplit(int numBackends) throws UserException { + try { + preExecutionAuthenticator.execute(() -> { + doStartSplit(); + return null; + }); + } catch (Exception e) { + throw new UserException(e.getMessage(), e); + } } - private List doGetSplits(int numBackends) throws UserException { + public void doStartSplit() throws UserException { + TableScan scan = createTableScan(); + CompletableFuture.runAsync(() -> { + try { + preExecutionAuthenticator.execute( + () -> { + CloseableIterable fileScanTasks = planFileScanTask(scan); + + // 1. this task should stop when all splits are assigned + // 2. if we want to stop this plan, we can close the fileScanTasks to stop + splitAssignment.addCloseable(fileScanTasks); + + fileScanTasks.forEach(fileScanTask -> + splitAssignment.addToQueue(Lists.newArrayList(createIcebergSplit(fileScanTask)))); + + return null; + } + ); + splitAssignment.finishSchedule(); + } catch (Exception e) { + splitAssignment.setException(new UserException(e.getMessage(), e)); + } + }); + } + + private TableScan createTableScan() throws UserException { TableScan scan = icebergTable.newScan(); // set snapshot @@ -213,16 +259,16 @@ private List doGetSplits(int numBackends) throws UserException { this.pushdownIcebergPredicates.add(predicate.toString()); } - // get splits - List splits = new ArrayList<>(); - int formatVersion = ((BaseTable) icebergTable).operations().current().formatVersion(); - HashSet partitionPathSet = new HashSet<>(); - boolean isPartitionedTable = icebergTable.spec().isPartitioned(); + scan = scan.planWith(source.getCatalog().getThreadPoolWithPreAuth()); + + return scan; + } - long realFileSplitSize = getRealFileSplitSize(0); - CloseableIterable fileScanTasks = null; + private CloseableIterable planFileScanTask(TableScan scan) { + long targetSplitSize = getRealFileSplitSize(0); + CloseableIterable splitFiles; try { - fileScanTasks = TableScanUtil.splitFiles(scan.planFiles(), realFileSplitSize); + splitFiles = TableScanUtil.splitFiles(scan.planFiles(), targetSplitSize); } catch (NullPointerException e) { /* Caused by: java.lang.NullPointerException: Type cannot be null @@ -253,33 +299,45 @@ private List doGetSplits(int numBackends) throws UserException { LOG.warn("Iceberg TableScanUtil.splitFiles throw NullPointerException. Cause : ", e); throw new NotSupportedException("Unable to read Iceberg table with dropped old partition column."); } - try (CloseableIterable combinedScanTasks = - TableScanUtil.planTasks(fileScanTasks, realFileSplitSize, 1, 0)) { - combinedScanTasks.forEach(taskGrp -> taskGrp.files().forEach(splitTask -> { - if (isPartitionedTable) { - StructLike structLike = splitTask.file().partition(); - // Counts the number of partitions read - partitionPathSet.add(structLike.toString()); - } - String originalPath = splitTask.file().path().toString(); - LocationPath locationPath = new LocationPath(originalPath, source.getCatalog().getProperties()); - IcebergSplit split = new IcebergSplit( - locationPath, - splitTask.start(), - splitTask.length(), - splitTask.file().fileSizeInBytes(), - new String[0], - formatVersion, - source.getCatalog().getProperties(), - new ArrayList<>(), - originalPath); - split.setTargetSplitSize(realFileSplitSize); - if (formatVersion >= MIN_DELETE_FILE_SUPPORT_VERSION) { - split.setDeleteFileFilters(getDeleteFileFilters(splitTask)); - } - split.setTableFormatType(TableFormatType.ICEBERG); + return splitFiles; + } + + private Split createIcebergSplit(FileScanTask fileScanTask) { + if (isPartitionedTable) { + StructLike structLike = fileScanTask.file().partition(); + // Counts the number of partitions read + partitionPathSet.add(structLike.toString()); + } + String originalPath = fileScanTask.file().path().toString(); + LocationPath locationPath = new LocationPath(originalPath, source.getCatalog().getProperties()); + IcebergSplit split = new IcebergSplit( + locationPath, + fileScanTask.start(), + fileScanTask.length(), + fileScanTask.file().fileSizeInBytes(), + new String[0], + formatVersion, + source.getCatalog().getProperties(), + new ArrayList<>(), + originalPath); + if (!fileScanTask.deletes().isEmpty()) { + split.setDeleteFileFilters(getDeleteFileFilters(fileScanTask)); + } + split.setTableFormatType(TableFormatType.ICEBERG); + split.setTargetSplitSize(targetSplitSize); + return split; + } + + private List doGetSplits(int numBackends) throws UserException { + + TableScan scan = createTableScan(); + List splits = new ArrayList<>(); + + try (CloseableIterable fileScanTasks = planFileScanTask(scan)) { + fileScanTasks.forEach(taskGrp -> { + Split split = createIcebergSplit(taskGrp); splits.add(split); - })); + }); } catch (IOException e) { throw new UserException(e.getMessage(), e.getCause()); } @@ -308,6 +366,12 @@ private List doGetSplits(int numBackends) throws UserException { return splits; } + @Override + public boolean isBatchMode() { + // TODO Use a better judgment method to decide whether to use batch mode. + return sessionVariable.getNumPartitionsInBatchMode() > 1024; + } + public Long getSpecifiedSnapshot() throws UserException { TableSnapshot tableSnapshot = getQueryTableSnapshot(); if (tableSnapshot != null) { @@ -460,4 +524,9 @@ private void assignCountToSplits(List splits, long totalCount) { } ((IcebergSplit) splits.get(size - 1)).setTableLevelRowCount(countPerSplit + totalCount % size); } + + @Override + public int numApproximateSplits() { + return NUM_SPLITS_PER_PARTITION * partitionPathSet.size() > 0 ? partitionPathSet.size() : 1; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/IcebergSplit.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/IcebergSplit.java index 0520612935a778..e31ec5c3fadb79 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/IcebergSplit.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/source/IcebergSplit.java @@ -22,6 +22,7 @@ import lombok.Data; +import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -35,7 +36,7 @@ public class IcebergSplit extends FileSplit { // but the original datafile path must be used. private final String originalPath; private Integer formatVersion; - private List deleteFileFilters; + private List deleteFileFilters = new ArrayList<>(); private Map config; // tableLevelRowCount will be set only table-level count push down opt is available. private long tableLevelRowCount = -1; diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/ScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/ScanNode.java index 2b10520e67d317..37488a001f69e2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/ScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/ScanNode.java @@ -101,7 +101,7 @@ public abstract class ScanNode extends PlanNode implements SplitGenerator { protected SplitAssignment splitAssignment = null; protected long selectedPartitionNum = 0; - protected long selectedSplitNum = 0; + protected int selectedSplitNum = 0; // create a mapping between output slot's id and project expr Map outputSlotToProjectExpr = new HashMap<>(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java index 29b8df43c944c0..a694b70250f18a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java @@ -1763,7 +1763,7 @@ public void setEnableLeftZigZag(boolean enableLeftZigZag) { description = {"batch方式中BE获取splits的最大等待时间", "The max wait time of getting splits in batch mode."}, needForward = true) - public long fetchSplitsMaxWaitTime = 4000; + public long fetchSplitsMaxWaitTime = 1000; @VariableMgr.VarAttr( name = ENABLE_PARQUET_LAZY_MAT, diff --git a/regression-test/suites/external_table_p0/iceberg/test_iceberg_filter.groovy b/regression-test/suites/external_table_p0/iceberg/test_iceberg_filter.groovy index 7e654175f9cbe3..e74d3b07f93f9a 100644 --- a/regression-test/suites/external_table_p0/iceberg/test_iceberg_filter.groovy +++ b/regression-test/suites/external_table_p0/iceberg/test_iceberg_filter.groovy @@ -19,6 +19,7 @@ suite("test_iceberg_filter", "p0,external,doris,external_docker,external_docker_ String enabled = context.config.otherConfigs.get("enableIcebergTest") if (enabled != null && enabled.equalsIgnoreCase("true")) { try { + sql """set num_partitions_in_batch_mode=0""" String rest_port = context.config.otherConfigs.get("iceberg_rest_uri_port") String minio_port = context.config.otherConfigs.get("iceberg_minio_port") String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") @@ -91,6 +92,7 @@ suite("test_iceberg_filter", "p0,external,doris,external_docker,external_docker_ } } finally { + sql """set num_partitions_in_batch_mode=1024""" } } } From 12d84c1b73311f35d2c8e54991c728afe35654a4 Mon Sep 17 00:00:00 2001 From: Jiwen liu <61498169+liujiwen-up@users.noreply.github.com> Date: Fri, 24 Jan 2025 11:35:40 +0800 Subject: [PATCH 06/14] [feature](sql-dialect)support convert hive view and presto view use sql convertor service (#46308) ### What problem does this PR solve? Problem Summary: 1. Supports presto view query 2. Supports hive view and presto view rewriting based on dialect service 3. Migrated dialect_compatible case to external_table_p0 --- .../create_preinstalled_scripts/run73.hql | 21 ++ .../create_view_scripts/create_view.hql | 2 + .../datasource/hive/HMSExternalTable.java | 50 +++- .../nereids/parser/SqlDialectHelper.java | 66 +++++ .../nereids/rules/analysis/BindRelation.java | 5 +- .../org/apache/doris/qe/ConnectProcessor.java | 29 +-- .../datasource/hive/HMSExternalTableTest.java | 126 ++++++++++ .../dialect_compatible/sql/clickhouse.out | 0 .../scalar/AbstractTestRegexpFunctions.out | 0 .../scalar/TestArrayCombinationsFunction.out | 0 .../scalar/TestArrayContainsSequence.out | 0 .../presto/scalar/TestArrayExceptFunction.out | 0 .../presto/scalar/TestArrayFilterFunction.out | 0 .../sql/presto/scalar/TestArrayFunctions.out | 0 .../scalar/TestArrayHistogramFunction.out | 0 .../presto/scalar/TestArrayMatchFunctions.out | 0 .../presto/scalar/TestArrayNgramsFunction.out | 0 .../presto/scalar/TestArrayReduceFunction.out | 0 .../scalar/TestArrayTransformFunction.out | 0 .../presto/scalar/TestArrayTrimFunction.out | 0 .../presto/scalar/TestBitwiseFunctions.out | 0 .../TestBlockAndPositionNullConvention.out | 0 .../presto/scalar/TestConcatWsFunction.out | 0 .../sql/presto/scalar/TestCustomFunctions.out | 0 .../presto/scalar/TestDataSizeFunctions.out | 0 .../presto/scalar/TestDateTimeFunctions.out | 0 .../presto/scalar/TestIpAddressFunctions.out | 0 .../presto/scalar/TestIsNullAnnotation.out | 0 .../sql/presto/scalar/TestJsonExtract.out | 0 .../sql/presto/scalar/TestJsonFunctions.out | 0 .../presto/scalar/TestLuhnCheckFunction.out | 0 .../presto/scalar/TestMapFilterFunction.out | 0 .../scalar/TestMapTransformKeysFunction.out | 0 .../scalar/TestMapTransformValuesFunction.out | 0 .../sql/presto/scalar/TestMathFunctions.out | 0 .../sql/presto/scalar/TestScalarParser.out | 0 .../sql/presto/scalar/TestStringFunctions.out | Bin .../sql/presto/scalar/TestTryFunction.out | 0 .../sql/presto/scalar/TestTypeOfFunction.out | 0 .../sql/presto/scalar/TestUrlFunctions.out | 0 .../presto/scalar/TestVarbinaryFunctions.out | 0 .../presto/scalar/TestWordStemFunction.out | 0 .../sql/presto/scalar/TestZipFunction.out | 0 .../sql/presto/scalar/TestZipWithFunction.out | 0 .../sql/presto/scalar/date/TestExtract.out | 0 .../scalar/interval/TestIntervalDayTime.out | 0 .../scalar/interval/TestIntervalYearMonth.out | 0 .../sql/presto/scalar/time/TestExtract.out | 0 .../sql/presto/scalar/time/TestOperators.out | 0 .../sql/presto/scalar/time/TestTime.out | 0 .../presto/scalar/timestamp/TestDateTrunc.out | 0 .../presto/scalar/timestamp/TestExtract.out | 0 .../timestamp/TestHumanReadableSeconds.out | 0 .../presto/scalar/timestamp/TestOperators.out | 0 .../presto/scalar/timestamp/TestTimestamp.out | 0 .../scalar/timestamptz/TestAtTimeZone.out | 0 .../scalar/timestamptz/TestDateTrunc.out | 0 .../presto/scalar/timestamptz/TestExtract.out | 0 .../scalar/timestamptz/TestOperators.out | 0 .../timestamptz/TestTimestampWithTimeZone.out | 0 .../presto/scalar/timetz/TestAtTimeZone.out | 0 .../presto/scalar/timetz/TestDateTrunc.out | 0 .../sql/presto/scalar/timetz/TestExtract.out | 0 .../presto/scalar/timetz/TestOperators.out | 0 .../scalar/timetz/TestTimeWithTimeZone.out | 0 .../timetz/TestTimestampWithTimeZone.out | 0 .../sql/test_hive_view_rewrite.out | 15 ++ .../string_functions/test_trim.out | 230 ++++++++++++++++++ .../dialect_compatible/sql/clickhouse.sql | 0 .../scalar/AbstractTestRegexpFunctions.sql | 0 .../scalar/TestArrayCombinationsFunction.sql | 0 .../scalar/TestArrayContainsSequence.sql | 0 .../presto/scalar/TestArrayExceptFunction.sql | 0 .../presto/scalar/TestArrayFilterFunction.sql | 0 .../sql/presto/scalar/TestArrayFunctions.sql | 0 .../scalar/TestArrayHistogramFunction.sql | 0 .../presto/scalar/TestArrayMatchFunctions.sql | 0 .../presto/scalar/TestArrayNgramsFunction.sql | 0 .../presto/scalar/TestArrayReduceFunction.sql | 0 .../scalar/TestArrayTransformFunction.sql | 0 .../presto/scalar/TestArrayTrimFunction.sql | 0 .../presto/scalar/TestBitwiseFunctions.sql | 0 .../TestBlockAndPositionNullConvention.sql | 0 .../presto/scalar/TestConcatWsFunction.sql | 0 .../sql/presto/scalar/TestCustomFunctions.sql | 0 .../presto/scalar/TestDataSizeFunctions.sql | 0 .../presto/scalar/TestDateTimeFunctions.sql | 0 .../presto/scalar/TestIpAddressFunctions.sql | 0 .../presto/scalar/TestIsNullAnnotation.sql | 0 .../sql/presto/scalar/TestJsonExtract.sql | 0 .../sql/presto/scalar/TestJsonFunctions.sql | 0 .../presto/scalar/TestLuhnCheckFunction.sql | 0 .../presto/scalar/TestMapFilterFunction.sql | 0 .../scalar/TestMapTransformKeysFunction.sql | 0 .../scalar/TestMapTransformValuesFunction.sql | 0 .../sql/presto/scalar/TestMathFunctions.sql | 0 .../sql/presto/scalar/TestScalarParser.sql | 0 .../sql/presto/scalar/TestStringFunctions.sql | 0 .../sql/presto/scalar/TestTryFunction.sql | 0 .../sql/presto/scalar/TestTypeOfFunction.sql | 0 .../sql/presto/scalar/TestUrlFunctions.sql | 0 .../presto/scalar/TestVarbinaryFunctions.sql | 0 .../presto/scalar/TestWordStemFunction.sql | 0 .../sql/presto/scalar/TestZipFunction.sql | 0 .../sql/presto/scalar/TestZipWithFunction.sql | 0 .../sql/presto/scalar/date/TestExtract.sql | 0 .../scalar/interval/TestIntervalDayTime.sql | 0 .../scalar/interval/TestIntervalYearMonth.sql | 0 .../sql/presto/scalar/time/TestExtract.sql | 0 .../sql/presto/scalar/time/TestOperators.sql | 0 .../sql/presto/scalar/time/TestTime.sql | 0 .../presto/scalar/timestamp/TestDateTrunc.sql | 0 .../presto/scalar/timestamp/TestExtract.sql | 0 .../timestamp/TestHumanReadableSeconds.sql | 0 .../presto/scalar/timestamp/TestOperators.sql | 0 .../presto/scalar/timestamp/TestTimestamp.sql | 0 .../scalar/timestamptz/TestAtTimeZone.sql | 0 .../scalar/timestamptz/TestDateTrunc.sql | 0 .../presto/scalar/timestamptz/TestExtract.sql | 0 .../scalar/timestamptz/TestOperators.sql | 0 .../timestamptz/TestTimestampWithTimeZone.sql | 0 .../sql/presto/scalar/timetz/TestExtract.sql | 0 .../presto/scalar/timetz/TestOperators.sql | 0 .../scalar/timetz/TestTimeWithTimeZone.sql | 0 .../sql/test_hive_view_rewrite.groovy | 48 ++++ 125 files changed, 562 insertions(+), 30 deletions(-) create mode 100644 docker/thirdparties/docker-compose/hive/scripts/create_preinstalled_scripts/run73.hql create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/parser/SqlDialectHelper.java create mode 100644 fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HMSExternalTableTest.java rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/clickhouse.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/AbstractTestRegexpFunctions.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestArrayCombinationsFunction.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestArrayContainsSequence.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestArrayExceptFunction.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestArrayFilterFunction.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestArrayFunctions.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestArrayHistogramFunction.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestArrayMatchFunctions.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestArrayNgramsFunction.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestArrayReduceFunction.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestArrayTransformFunction.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestArrayTrimFunction.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestBitwiseFunctions.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestBlockAndPositionNullConvention.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestConcatWsFunction.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestCustomFunctions.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestDataSizeFunctions.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestDateTimeFunctions.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestIpAddressFunctions.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestIsNullAnnotation.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestJsonExtract.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestJsonFunctions.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestLuhnCheckFunction.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestMapFilterFunction.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestMapTransformKeysFunction.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestMapTransformValuesFunction.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestMathFunctions.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestScalarParser.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestStringFunctions.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestTryFunction.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestTypeOfFunction.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestUrlFunctions.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestVarbinaryFunctions.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestWordStemFunction.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestZipFunction.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestZipWithFunction.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/date/TestExtract.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/interval/TestIntervalDayTime.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/interval/TestIntervalYearMonth.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/time/TestExtract.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/time/TestOperators.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/time/TestTime.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/timestamp/TestDateTrunc.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/timestamp/TestExtract.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/timestamp/TestHumanReadableSeconds.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/timestamp/TestOperators.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/timestamp/TestTimestamp.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/timestamptz/TestAtTimeZone.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/timestamptz/TestDateTrunc.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/timestamptz/TestExtract.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/timestamptz/TestOperators.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/timestamptz/TestTimestampWithTimeZone.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/timetz/TestAtTimeZone.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/timetz/TestDateTrunc.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/timetz/TestExtract.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/timetz/TestOperators.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/timetz/TestTimeWithTimeZone.out (100%) rename regression-test/data/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/timetz/TestTimestampWithTimeZone.out (100%) create mode 100644 regression-test/data/external_table_p0/dialect_compatible/sql/test_hive_view_rewrite.out create mode 100644 regression-test/data/query_p0/sql_functions/string_functions/test_trim.out rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/clickhouse.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/AbstractTestRegexpFunctions.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestArrayCombinationsFunction.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestArrayContainsSequence.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestArrayExceptFunction.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestArrayFilterFunction.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestArrayFunctions.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestArrayHistogramFunction.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestArrayMatchFunctions.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestArrayNgramsFunction.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestArrayReduceFunction.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestArrayTransformFunction.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestArrayTrimFunction.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestBitwiseFunctions.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestBlockAndPositionNullConvention.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestConcatWsFunction.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestCustomFunctions.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestDataSizeFunctions.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestDateTimeFunctions.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestIpAddressFunctions.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestIsNullAnnotation.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestJsonExtract.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestJsonFunctions.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestLuhnCheckFunction.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestMapFilterFunction.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestMapTransformKeysFunction.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestMapTransformValuesFunction.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestMathFunctions.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestScalarParser.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestStringFunctions.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestTryFunction.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestTypeOfFunction.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestUrlFunctions.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestVarbinaryFunctions.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestWordStemFunction.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestZipFunction.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/TestZipWithFunction.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/date/TestExtract.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/interval/TestIntervalDayTime.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/interval/TestIntervalYearMonth.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/time/TestExtract.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/time/TestOperators.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/time/TestTime.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/timestamp/TestDateTrunc.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/timestamp/TestExtract.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/timestamp/TestHumanReadableSeconds.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/timestamp/TestOperators.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/timestamp/TestTimestamp.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/timestamptz/TestAtTimeZone.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/timestamptz/TestDateTrunc.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/timestamptz/TestExtract.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/timestamptz/TestOperators.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/timestamptz/TestTimestampWithTimeZone.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/timetz/TestExtract.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/timetz/TestOperators.sql (100%) rename regression-test/suites/{ => external_table_p0}/dialect_compatible/sql/presto/scalar/timetz/TestTimeWithTimeZone.sql (100%) create mode 100644 regression-test/suites/external_table_p0/dialect_compatible/sql/test_hive_view_rewrite.groovy diff --git a/docker/thirdparties/docker-compose/hive/scripts/create_preinstalled_scripts/run73.hql b/docker/thirdparties/docker-compose/hive/scripts/create_preinstalled_scripts/run73.hql new file mode 100644 index 00000000000000..8fb0e3728355a5 --- /dev/null +++ b/docker/thirdparties/docker-compose/hive/scripts/create_preinstalled_scripts/run73.hql @@ -0,0 +1,21 @@ +CREATE TABLE employees ( + id INT, + name VARCHAR(100), + department VARCHAR(100), + salary DECIMAL(10,2), + hire_date DATE +); + + +INSERT INTO employees VALUES + (1, 'John Doe', 'IT', 75000.00, '2020-01-15'), + (2, 'Jane Smith', 'HR', 65000.00, '2019-03-20'), + (3, 'Bob Johnson', 'IT', 80000.00, '2021-05-10'), + (4, 'Alice Brown', 'Finance', 70000.00, '2020-11-30'), + (5, 'Charlie Wilson', 'HR', 62000.00, '2022-01-05'); + + + +msck repair table employees; + + diff --git a/docker/thirdparties/docker-compose/hive/scripts/create_view_scripts/create_view.hql b/docker/thirdparties/docker-compose/hive/scripts/create_view_scripts/create_view.hql index 221b536e263fe7..5f9d5499d15ad6 100644 --- a/docker/thirdparties/docker-compose/hive/scripts/create_view_scripts/create_view.hql +++ b/docker/thirdparties/docker-compose/hive/scripts/create_view_scripts/create_view.hql @@ -4,3 +4,5 @@ create view test_view2 as select * from default.sale_table; create view test_view3 as select * from sale_table where bill_code="bill_code1"; create view test_view4 as select parquet_zstd_all_types.t_int, parquet_zstd_all_types.t_varchar from parquet_zstd_all_types join multi_catalog.parquet_all_types on parquet_zstd_all_types.t_varchar = parquet_all_types.t_varchar order by t_int limit 10; create view unsupported_view as select bill_code from sale_table union all select t_varchar from multi_catalog.parquet_all_types order by bill_code limit 10; +create view department_view as select department,length(department) as department_length,trunc(hire_date,'YEAR') as year from default.employees; +create view department_nesting_view as select department,trunc(to_date(year),'YEAR') as year,count(*) as emp_count,avg(department_length) as avg_dept_name_length from department_view group by department, year order by year, department; diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java index b554f508103992..dccefc8b7432fa 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSExternalTable.java @@ -63,6 +63,8 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; +import com.google.gson.Gson; +import com.google.gson.JsonObject; import org.apache.commons.collections.CollectionUtils; import org.apache.commons.collections.MapUtils; import org.apache.commons.lang3.StringUtils; @@ -81,7 +83,9 @@ import org.apache.logging.log4j.Logger; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.Base64; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -454,9 +458,51 @@ public BaseAnalysisTask createAnalysisTask(AnalysisInfo info) { public String getViewText() { String viewText = getViewExpandedText(); if (StringUtils.isNotEmpty(viewText)) { - return viewText; + if (!viewText.equals("/* Presto View */")) { + return viewText; + } + } + + String originalText = getViewOriginalText(); + return parseTrinoViewDefinition(originalText); + } + + /** + * Parse Trino/Presto view definition from the original text. + * The definition is stored in the format: /* Presto View: * / + * + * The base64 encoded JSON contains the following fields: + * { + * "originalSql": "SELECT * FROM employees", // The original SQL statement + * "catalog": "hive", // The data catalog name + * "schema": "mmc_hive", // The schema name + * ... + * } + * + * @param originalText The original view definition text + * @return The parsed SQL statement, or original text if parsing fails + */ + private String parseTrinoViewDefinition(String originalText) { + if (originalText == null || !originalText.contains("/* Presto View: ")) { + return originalText; + } + + try { + String base64String = originalText.substring( + originalText.indexOf("/* Presto View: ") + "/* Presto View: ".length(), + originalText.lastIndexOf(" */") + ).trim(); + byte[] decodedBytes = Base64.getDecoder().decode(base64String); + String decodedString = new String(decodedBytes, StandardCharsets.UTF_8); + JsonObject jsonObject = new Gson().fromJson(decodedString, JsonObject.class); + + if (jsonObject.has("originalSql")) { + return jsonObject.get("originalSql").getAsString(); + } + } catch (Exception e) { + LOG.warn("Decoding Presto view definition failed", e); } - return getViewOriginalText(); + return originalText; } public String getViewExpandedText() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/SqlDialectHelper.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/SqlDialectHelper.java new file mode 100644 index 00000000000000..c959982e25449c --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/SqlDialectHelper.java @@ -0,0 +1,66 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.parser; + +import org.apache.doris.catalog.Env; +import org.apache.doris.plugin.DialectConverterPlugin; +import org.apache.doris.plugin.PluginMgr; +import org.apache.doris.qe.SessionVariable; + +import org.apache.commons.lang3.StringUtils; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.List; +import javax.annotation.Nullable; + +/** + * Helper class for SQL dialect conversion. + */ +public class SqlDialectHelper { + private static final Logger LOG = LogManager.getLogger(SqlDialectHelper.class); + + /** + * Convert SQL statement based on current SQL dialect + * + * @param originStmt original SQL statement + * @param sessionVariable session variable containing dialect settings + * @return converted SQL statement, or original statement if conversion fails + */ + public static String convertSqlByDialect(String originStmt, SessionVariable sessionVariable) { + String convertedStmt = originStmt; + @Nullable Dialect sqlDialect = Dialect.getByName(sessionVariable.getSqlDialect()); + if (sqlDialect != null && sqlDialect != Dialect.DORIS) { + PluginMgr pluginMgr = Env.getCurrentEnv().getPluginMgr(); + List plugins = pluginMgr.getActiveDialectPluginList(sqlDialect); + for (DialectConverterPlugin plugin : plugins) { + try { + String convertedSql = plugin.convertSql(originStmt, sessionVariable); + if (StringUtils.isNotEmpty(convertedSql)) { + convertedStmt = convertedSql; + break; + } + } catch (Throwable throwable) { + LOG.warn("Convert sql with dialect {} failed, plugin: {}, sql: {}, use origin sql.", + sqlDialect, plugin.getClass().getSimpleName(), originStmt, throwable); + } + } + } + return convertedStmt; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java index 40702011ab0ab4..54edc592ddda42 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java @@ -46,6 +46,7 @@ import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.hint.LeadingHint; import org.apache.doris.nereids.parser.NereidsParser; +import org.apache.doris.nereids.parser.SqlDialectHelper; import org.apache.doris.nereids.pattern.MatchingContext; import org.apache.doris.nereids.properties.LogicalProperties; import org.apache.doris.nereids.properties.PhysicalProperties; @@ -452,11 +453,12 @@ private Plan parseAndAnalyzeHiveView( ConnectContext ctx = cascadesContext.getConnectContext(); String previousCatalog = ctx.getCurrentCatalog().getName(); String previousDb = ctx.getDatabase(); + String convertedSql = SqlDialectHelper.convertSqlByDialect(ddlSql, ctx.getSessionVariable()); // change catalog and db to hive catalog and db, so that we can parse and analyze the view sql in hive context. ctx.changeDefaultCatalog(hiveCatalog); ctx.setDatabase(hiveDb); try { - return parseAndAnalyzeView(table, ddlSql, cascadesContext); + return parseAndAnalyzeView(table, convertedSql, cascadesContext); } finally { // restore catalog and db in connect context ctx.changeDefaultCatalog(previousCatalog); @@ -522,4 +524,5 @@ private List getPartitionIds(TableIf t, UnboundRelation unboundRelation, L return part.getId(); }).collect(ImmutableList.toImmutableList()); } + } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java index 0c633186abf8ae..687d69eb709083 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java @@ -56,14 +56,12 @@ import org.apache.doris.nereids.exceptions.NotSupportedException; import org.apache.doris.nereids.glue.LogicalPlanAdapter; import org.apache.doris.nereids.minidump.MinidumpUtils; -import org.apache.doris.nereids.parser.Dialect; import org.apache.doris.nereids.parser.NereidsParser; +import org.apache.doris.nereids.parser.SqlDialectHelper; import org.apache.doris.nereids.stats.StatsErrorEstimator; import org.apache.doris.nereids.trees.plans.commands.ExplainCommand; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; import org.apache.doris.nereids.trees.plans.logical.LogicalSqlCache; -import org.apache.doris.plugin.DialectConverterPlugin; -import org.apache.doris.plugin.PluginMgr; import org.apache.doris.proto.Data; import org.apache.doris.qe.QueryState.MysqlStateType; import org.apache.doris.qe.cache.CacheAnalyzer; @@ -79,7 +77,6 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import org.apache.commons.codec.digest.DigestUtils; -import org.apache.commons.lang3.StringUtils; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.thrift.TException; @@ -91,7 +88,6 @@ import java.util.Map; import java.util.Optional; import java.util.UUID; -import javax.annotation.Nullable; /** * Process one connection, the life cycle is the same as connection @@ -269,7 +265,7 @@ public void executeQuery(String originStmt) throws Exception { } } - String convertedStmt = convertOriginStmt(originStmt); + String convertedStmt = SqlDialectHelper.convertSqlByDialect(originStmt, ctx.getSessionVariable()); String sqlHash = DigestUtils.md5Hex(convertedStmt); ctx.setSqlHash(sqlHash); @@ -435,27 +431,6 @@ private List parseFromSqlCache(String originStmt) { return null; } - private String convertOriginStmt(String originStmt) { - String convertedStmt = originStmt; - @Nullable Dialect sqlDialect = Dialect.getByName(ctx.getSessionVariable().getSqlDialect()); - if (sqlDialect != null && sqlDialect != Dialect.DORIS) { - PluginMgr pluginMgr = Env.getCurrentEnv().getPluginMgr(); - List plugins = pluginMgr.getActiveDialectPluginList(sqlDialect); - for (DialectConverterPlugin plugin : plugins) { - try { - String convertedSql = plugin.convertSql(originStmt, ctx.getSessionVariable()); - if (StringUtils.isNotEmpty(convertedSql)) { - convertedStmt = convertedSql; - break; - } - } catch (Throwable throwable) { - LOG.warn("Convert sql with dialect {} failed, plugin: {}, sql: {}, use origin sql.", - sqlDialect, plugin.getClass().getSimpleName(), originStmt, throwable); - } - } - } - return convertedStmt; - } // Use a handler for exception to avoid big try catch block which is a little hard to understand protected void handleQueryException(Throwable throwable, String origStmt, diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HMSExternalTableTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HMSExternalTableTest.java new file mode 100644 index 00000000000000..bd29766f39ed80 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HMSExternalTableTest.java @@ -0,0 +1,126 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.datasource.hive; + +import mockit.Injectable; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + + +/** + * Test class for HMSExternalTable, focusing on view-related functionality + */ +public class HMSExternalTableTest { + private TestHMSExternalTable table; + private static final String TEST_VIEW_TEXT = "SELECT * FROM test_table"; + private static final String TEST_EXPANDED_VIEW = "/* Presto View */"; + + // Real example of a Presto View definition + private static final String PRESTO_VIEW_ORIGINAL = "/* Presto View: eyJvcmlnaW5hbFNxbCI6IlNFTEVDVFxuICBkZXBhcnRtZW50XG4sIGxlbmd0aChkZXBhcnRtZW50KSBkZXBhcnRtZW50X2xlbmd0aFxuLCBkYXRlX3RydW5jKCd5ZWFyJywgaGlyZV9kYXRlKSB5ZWFyXG5GUk9NXG4gIGVtcGxveWVlc1xuIiwiY2F0YWxvZyI6ImhpdmUiLCJzY2hlbWEiOiJtbWNfaGl2ZSIsImNvbHVtbnMiOlt7Im5hbWUiOiJkZXBhcnRtZW50IiwidHlwZSI6InZhcmNoYXIifSx7Im5hbWUiOiJkZXBhcnRtZW50X2xlbmd0aCIsInR5cGUiOiJiaWdpbnQifSx7Im5hbWUiOiJ5ZWFyIiwidHlwZSI6ImRhdGUifV0sIm93bmVyIjoidHJpbm8vbWFzdGVyLTEtMS5jLTA1OTYxNzY2OThiZDRkMTcuY24tYmVpamluZy5lbXIuYWxpeXVuY3MuY29tIiwicnVuQXNJbnZva2VyIjpmYWxzZX0= */"; + + // Expected SQL query after decoding and parsing + private static final String EXPECTED_SQL = "SELECT\n department\n, length(department) department_length\n, date_trunc('year', hire_date) year\nFROM\n employees\n"; + + @Injectable + private HMSExternalCatalog mockCatalog; + + private HMSExternalDatabase mockDb; + + @BeforeEach + public void setUp() { + // Create a mock database with minimal required functionality + mockDb = new HMSExternalDatabase(mockCatalog, 1L, "test_db", "remote_test_db") { + @Override + public String getFullName() { + return "test_catalog.test_db"; + } + }; + + table = new TestHMSExternalTable(mockCatalog, mockDb); + } + + @Test + public void testGetViewText_Normal() { + // Test regular view text retrieval + table.setViewOriginalText(TEST_VIEW_TEXT); + table.setViewExpandedText(TEST_VIEW_TEXT); + Assertions.assertEquals(TEST_VIEW_TEXT, table.getViewText()); + } + + @Test + public void testGetViewText_PrestoView() { + // Test Presto view parsing including base64 decode and JSON extraction + table.setViewOriginalText(PRESTO_VIEW_ORIGINAL); + table.setViewExpandedText(TEST_EXPANDED_VIEW); + Assertions.assertEquals(EXPECTED_SQL, table.getViewText()); + } + + @Test + public void testGetViewText_InvalidPrestoView() { + // Test handling of invalid Presto view definition + String invalidPrestoView = "/* Presto View: invalid_base64_content */"; + table.setViewOriginalText(invalidPrestoView); + table.setViewExpandedText(TEST_EXPANDED_VIEW); + Assertions.assertEquals(invalidPrestoView, table.getViewText()); + } + + @Test + public void testGetViewText_EmptyExpandedView() { + // Test handling of empty expanded view text + table.setViewOriginalText(TEST_VIEW_TEXT); + table.setViewExpandedText(""); + Assertions.assertEquals(TEST_VIEW_TEXT, table.getViewText()); + } + + /** + * Test implementation of HMSExternalTable that allows setting view texts + * Uses parent's getViewText() implementation for actual testing + */ + private static class TestHMSExternalTable extends HMSExternalTable { + private String viewExpandedText; + private String viewOriginalText; + + public TestHMSExternalTable(HMSExternalCatalog catalog, HMSExternalDatabase db) { + super(1L, "test_table", "test_table", catalog, db); + } + + @Override + public String getViewExpandedText() { + return viewExpandedText; + } + + @Override + public String getViewOriginalText() { + return viewOriginalText; + } + + public void setViewExpandedText(String viewExpandedText) { + this.viewExpandedText = viewExpandedText; + } + + public void setViewOriginalText(String viewOriginalText) { + this.viewOriginalText = viewOriginalText; + } + + @Override + protected synchronized void makeSureInitialized() { + this.objectCreated = true; + } + } +} diff --git a/regression-test/data/dialect_compatible/sql/clickhouse.out b/regression-test/data/external_table_p0/dialect_compatible/sql/clickhouse.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/clickhouse.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/clickhouse.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/AbstractTestRegexpFunctions.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/AbstractTestRegexpFunctions.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/AbstractTestRegexpFunctions.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/AbstractTestRegexpFunctions.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/TestArrayCombinationsFunction.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestArrayCombinationsFunction.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/TestArrayCombinationsFunction.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestArrayCombinationsFunction.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/TestArrayContainsSequence.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestArrayContainsSequence.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/TestArrayContainsSequence.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestArrayContainsSequence.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/TestArrayExceptFunction.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestArrayExceptFunction.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/TestArrayExceptFunction.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestArrayExceptFunction.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/TestArrayFilterFunction.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestArrayFilterFunction.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/TestArrayFilterFunction.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestArrayFilterFunction.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/TestArrayFunctions.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestArrayFunctions.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/TestArrayFunctions.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestArrayFunctions.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/TestArrayHistogramFunction.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestArrayHistogramFunction.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/TestArrayHistogramFunction.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestArrayHistogramFunction.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/TestArrayMatchFunctions.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestArrayMatchFunctions.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/TestArrayMatchFunctions.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestArrayMatchFunctions.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/TestArrayNgramsFunction.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestArrayNgramsFunction.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/TestArrayNgramsFunction.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestArrayNgramsFunction.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/TestArrayReduceFunction.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestArrayReduceFunction.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/TestArrayReduceFunction.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestArrayReduceFunction.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/TestArrayTransformFunction.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestArrayTransformFunction.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/TestArrayTransformFunction.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestArrayTransformFunction.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/TestArrayTrimFunction.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestArrayTrimFunction.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/TestArrayTrimFunction.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestArrayTrimFunction.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/TestBitwiseFunctions.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestBitwiseFunctions.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/TestBitwiseFunctions.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestBitwiseFunctions.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/TestBlockAndPositionNullConvention.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestBlockAndPositionNullConvention.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/TestBlockAndPositionNullConvention.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestBlockAndPositionNullConvention.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/TestConcatWsFunction.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestConcatWsFunction.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/TestConcatWsFunction.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestConcatWsFunction.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/TestCustomFunctions.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestCustomFunctions.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/TestCustomFunctions.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestCustomFunctions.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/TestDataSizeFunctions.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestDataSizeFunctions.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/TestDataSizeFunctions.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestDataSizeFunctions.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/TestDateTimeFunctions.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestDateTimeFunctions.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/TestDateTimeFunctions.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestDateTimeFunctions.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/TestIpAddressFunctions.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestIpAddressFunctions.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/TestIpAddressFunctions.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestIpAddressFunctions.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/TestIsNullAnnotation.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestIsNullAnnotation.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/TestIsNullAnnotation.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestIsNullAnnotation.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/TestJsonExtract.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestJsonExtract.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/TestJsonExtract.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestJsonExtract.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/TestJsonFunctions.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestJsonFunctions.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/TestJsonFunctions.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestJsonFunctions.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/TestLuhnCheckFunction.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestLuhnCheckFunction.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/TestLuhnCheckFunction.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestLuhnCheckFunction.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/TestMapFilterFunction.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestMapFilterFunction.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/TestMapFilterFunction.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestMapFilterFunction.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/TestMapTransformKeysFunction.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestMapTransformKeysFunction.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/TestMapTransformKeysFunction.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestMapTransformKeysFunction.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/TestMapTransformValuesFunction.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestMapTransformValuesFunction.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/TestMapTransformValuesFunction.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestMapTransformValuesFunction.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/TestMathFunctions.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestMathFunctions.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/TestMathFunctions.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestMathFunctions.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/TestScalarParser.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestScalarParser.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/TestScalarParser.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestScalarParser.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/TestStringFunctions.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestStringFunctions.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/TestStringFunctions.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestStringFunctions.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/TestTryFunction.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestTryFunction.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/TestTryFunction.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestTryFunction.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/TestTypeOfFunction.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestTypeOfFunction.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/TestTypeOfFunction.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestTypeOfFunction.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/TestUrlFunctions.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestUrlFunctions.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/TestUrlFunctions.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestUrlFunctions.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/TestVarbinaryFunctions.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestVarbinaryFunctions.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/TestVarbinaryFunctions.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestVarbinaryFunctions.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/TestWordStemFunction.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestWordStemFunction.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/TestWordStemFunction.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestWordStemFunction.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/TestZipFunction.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestZipFunction.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/TestZipFunction.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestZipFunction.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/TestZipWithFunction.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestZipWithFunction.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/TestZipWithFunction.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/TestZipWithFunction.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/date/TestExtract.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/date/TestExtract.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/date/TestExtract.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/date/TestExtract.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/interval/TestIntervalDayTime.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/interval/TestIntervalDayTime.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/interval/TestIntervalDayTime.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/interval/TestIntervalDayTime.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/interval/TestIntervalYearMonth.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/interval/TestIntervalYearMonth.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/interval/TestIntervalYearMonth.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/interval/TestIntervalYearMonth.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/time/TestExtract.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/time/TestExtract.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/time/TestExtract.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/time/TestExtract.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/time/TestOperators.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/time/TestOperators.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/time/TestOperators.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/time/TestOperators.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/time/TestTime.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/time/TestTime.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/time/TestTime.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/time/TestTime.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/timestamp/TestDateTrunc.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/timestamp/TestDateTrunc.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/timestamp/TestDateTrunc.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/timestamp/TestDateTrunc.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/timestamp/TestExtract.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/timestamp/TestExtract.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/timestamp/TestExtract.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/timestamp/TestExtract.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/timestamp/TestHumanReadableSeconds.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/timestamp/TestHumanReadableSeconds.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/timestamp/TestHumanReadableSeconds.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/timestamp/TestHumanReadableSeconds.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/timestamp/TestOperators.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/timestamp/TestOperators.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/timestamp/TestOperators.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/timestamp/TestOperators.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/timestamp/TestTimestamp.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/timestamp/TestTimestamp.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/timestamp/TestTimestamp.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/timestamp/TestTimestamp.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/timestamptz/TestAtTimeZone.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/timestamptz/TestAtTimeZone.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/timestamptz/TestAtTimeZone.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/timestamptz/TestAtTimeZone.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/timestamptz/TestDateTrunc.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/timestamptz/TestDateTrunc.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/timestamptz/TestDateTrunc.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/timestamptz/TestDateTrunc.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/timestamptz/TestExtract.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/timestamptz/TestExtract.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/timestamptz/TestExtract.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/timestamptz/TestExtract.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/timestamptz/TestOperators.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/timestamptz/TestOperators.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/timestamptz/TestOperators.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/timestamptz/TestOperators.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/timestamptz/TestTimestampWithTimeZone.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/timestamptz/TestTimestampWithTimeZone.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/timestamptz/TestTimestampWithTimeZone.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/timestamptz/TestTimestampWithTimeZone.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/timetz/TestAtTimeZone.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/timetz/TestAtTimeZone.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/timetz/TestAtTimeZone.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/timetz/TestAtTimeZone.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/timetz/TestDateTrunc.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/timetz/TestDateTrunc.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/timetz/TestDateTrunc.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/timetz/TestDateTrunc.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/timetz/TestExtract.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/timetz/TestExtract.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/timetz/TestExtract.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/timetz/TestExtract.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/timetz/TestOperators.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/timetz/TestOperators.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/timetz/TestOperators.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/timetz/TestOperators.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/timetz/TestTimeWithTimeZone.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/timetz/TestTimeWithTimeZone.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/timetz/TestTimeWithTimeZone.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/timetz/TestTimeWithTimeZone.out diff --git a/regression-test/data/dialect_compatible/sql/presto/scalar/timetz/TestTimestampWithTimeZone.out b/regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/timetz/TestTimestampWithTimeZone.out similarity index 100% rename from regression-test/data/dialect_compatible/sql/presto/scalar/timetz/TestTimestampWithTimeZone.out rename to regression-test/data/external_table_p0/dialect_compatible/sql/presto/scalar/timetz/TestTimestampWithTimeZone.out diff --git a/regression-test/data/external_table_p0/dialect_compatible/sql/test_hive_view_rewrite.out b/regression-test/data/external_table_p0/dialect_compatible/sql/test_hive_view_rewrite.out new file mode 100644 index 00000000000000..35eb52d576ffae --- /dev/null +++ b/regression-test/data/external_table_p0/dialect_compatible/sql/test_hive_view_rewrite.out @@ -0,0 +1,15 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql1 -- +Finance 7 2020-01-01 +HR 2 2019-01-01 +HR 2 2022-01-01 +IT 2 2020-01-01 +IT 2 2021-01-01 + +-- !sql2 -- +Finance 2020-01-01 1 7.0 +HR 2019-01-01 1 2.0 +HR 2022-01-01 1 2.0 +IT 2020-01-01 1 2.0 +IT 2021-01-01 1 2.0 + diff --git a/regression-test/data/query_p0/sql_functions/string_functions/test_trim.out b/regression-test/data/query_p0/sql_functions/string_functions/test_trim.out new file mode 100644 index 00000000000000..24b3deca22ae17 --- /dev/null +++ b/regression-test/data/query_p0/sql_functions/string_functions/test_trim.out @@ -0,0 +1,230 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !const_nullable -- + +-- !partial_const_nullable -- + +-- !const_not_nullable -- + +-- !const_other_nullable -- + +-- !const_other_not_nullable -- + +-- !const_partial_nullable_no_null -- +abc + +-- !1 -- + + +-- !2 -- + + +-- !3 -- +hello + +-- !4 -- +hello + +-- !5 -- +hello + +-- !6 -- +hello world + +-- !7 -- + + +-- !8 -- +hello + +-- !9 -- +hello + +-- !10 -- +hello + +-- !11 -- +hello world + +-- !12 -- + + +-- !13 -- + + +-- !14 -- + hello + +-- !15 -- + hello + +-- !16 -- +hello + +-- !17 -- + hello world + +-- !18 -- + + +-- !19 -- + hello + +-- !20 -- + hello + +-- !21 -- +hello + +-- !22 -- + hello world + +-- !23 -- + + +-- !24 -- + + +-- !25 -- + hello + +-- !26 -- +hello + +-- !27 -- +hello + +-- !28 -- +llo + +-- !29 -- +hello + +-- !30 -- +llo + +-- !31 -- +hello + +-- !32 -- +hello world + +-- !33 -- +llo world + +-- !1 -- + + +-- !1 -- +hello world + +-- !1 -- + + +-- !1 -- + + +-- !1 -- + hello + +-- !1 -- +hello + +-- !1 -- +llo + +-- !1 -- +hello + +-- !1 -- +llo + +-- !1 -- +hello + +-- !1 -- +hello world + +-- !1 -- +llo world + +-- !1 -- + + +-- !1 -- +hello world + +-- !1 -- + + +-- !1 -- + + +-- !1 -- + hello + +-- !1 -- + hello + +-- !1 -- + he + +-- !1 -- +hello + +-- !1 -- +he + +-- !1 -- +hello + +-- !1 -- + hello world + +-- !1 -- + hello wor + +-- !1 -- + + +-- !1 -- + hello world + +-- !1 -- +abc + +-- !1 -- + + +-- !1 -- + + +-- !1 -- + hello + +-- !1 -- + hello + +-- !1 -- + hello + +-- !1 -- + hello + +-- !1 -- + hello + +-- !1 -- +hello + +-- !1 -- + hello world + +-- !1 -- + hello world + +-- !1 -- + + +-- !1 -- + hello world + diff --git a/regression-test/suites/dialect_compatible/sql/clickhouse.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/clickhouse.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/clickhouse.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/clickhouse.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/AbstractTestRegexpFunctions.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/AbstractTestRegexpFunctions.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/AbstractTestRegexpFunctions.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/AbstractTestRegexpFunctions.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/TestArrayCombinationsFunction.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestArrayCombinationsFunction.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/TestArrayCombinationsFunction.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestArrayCombinationsFunction.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/TestArrayContainsSequence.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestArrayContainsSequence.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/TestArrayContainsSequence.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestArrayContainsSequence.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/TestArrayExceptFunction.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestArrayExceptFunction.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/TestArrayExceptFunction.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestArrayExceptFunction.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/TestArrayFilterFunction.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestArrayFilterFunction.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/TestArrayFilterFunction.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestArrayFilterFunction.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/TestArrayFunctions.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestArrayFunctions.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/TestArrayFunctions.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestArrayFunctions.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/TestArrayHistogramFunction.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestArrayHistogramFunction.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/TestArrayHistogramFunction.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestArrayHistogramFunction.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/TestArrayMatchFunctions.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestArrayMatchFunctions.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/TestArrayMatchFunctions.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestArrayMatchFunctions.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/TestArrayNgramsFunction.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestArrayNgramsFunction.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/TestArrayNgramsFunction.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestArrayNgramsFunction.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/TestArrayReduceFunction.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestArrayReduceFunction.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/TestArrayReduceFunction.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestArrayReduceFunction.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/TestArrayTransformFunction.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestArrayTransformFunction.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/TestArrayTransformFunction.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestArrayTransformFunction.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/TestArrayTrimFunction.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestArrayTrimFunction.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/TestArrayTrimFunction.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestArrayTrimFunction.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/TestBitwiseFunctions.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestBitwiseFunctions.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/TestBitwiseFunctions.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestBitwiseFunctions.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/TestBlockAndPositionNullConvention.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestBlockAndPositionNullConvention.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/TestBlockAndPositionNullConvention.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestBlockAndPositionNullConvention.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/TestConcatWsFunction.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestConcatWsFunction.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/TestConcatWsFunction.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestConcatWsFunction.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/TestCustomFunctions.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestCustomFunctions.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/TestCustomFunctions.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestCustomFunctions.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/TestDataSizeFunctions.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestDataSizeFunctions.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/TestDataSizeFunctions.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestDataSizeFunctions.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/TestDateTimeFunctions.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestDateTimeFunctions.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/TestDateTimeFunctions.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestDateTimeFunctions.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/TestIpAddressFunctions.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestIpAddressFunctions.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/TestIpAddressFunctions.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestIpAddressFunctions.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/TestIsNullAnnotation.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestIsNullAnnotation.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/TestIsNullAnnotation.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestIsNullAnnotation.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/TestJsonExtract.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestJsonExtract.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/TestJsonExtract.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestJsonExtract.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/TestJsonFunctions.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestJsonFunctions.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/TestJsonFunctions.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestJsonFunctions.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/TestLuhnCheckFunction.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestLuhnCheckFunction.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/TestLuhnCheckFunction.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestLuhnCheckFunction.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/TestMapFilterFunction.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestMapFilterFunction.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/TestMapFilterFunction.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestMapFilterFunction.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/TestMapTransformKeysFunction.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestMapTransformKeysFunction.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/TestMapTransformKeysFunction.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestMapTransformKeysFunction.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/TestMapTransformValuesFunction.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestMapTransformValuesFunction.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/TestMapTransformValuesFunction.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestMapTransformValuesFunction.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/TestMathFunctions.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestMathFunctions.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/TestMathFunctions.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestMathFunctions.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/TestScalarParser.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestScalarParser.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/TestScalarParser.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestScalarParser.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/TestStringFunctions.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestStringFunctions.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/TestStringFunctions.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestStringFunctions.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/TestTryFunction.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestTryFunction.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/TestTryFunction.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestTryFunction.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/TestTypeOfFunction.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestTypeOfFunction.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/TestTypeOfFunction.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestTypeOfFunction.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/TestUrlFunctions.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestUrlFunctions.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/TestUrlFunctions.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestUrlFunctions.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/TestVarbinaryFunctions.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestVarbinaryFunctions.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/TestVarbinaryFunctions.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestVarbinaryFunctions.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/TestWordStemFunction.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestWordStemFunction.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/TestWordStemFunction.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestWordStemFunction.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/TestZipFunction.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestZipFunction.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/TestZipFunction.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestZipFunction.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/TestZipWithFunction.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestZipWithFunction.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/TestZipWithFunction.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/TestZipWithFunction.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/date/TestExtract.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/date/TestExtract.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/date/TestExtract.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/date/TestExtract.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/interval/TestIntervalDayTime.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/interval/TestIntervalDayTime.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/interval/TestIntervalDayTime.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/interval/TestIntervalDayTime.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/interval/TestIntervalYearMonth.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/interval/TestIntervalYearMonth.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/interval/TestIntervalYearMonth.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/interval/TestIntervalYearMonth.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/time/TestExtract.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/time/TestExtract.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/time/TestExtract.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/time/TestExtract.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/time/TestOperators.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/time/TestOperators.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/time/TestOperators.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/time/TestOperators.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/time/TestTime.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/time/TestTime.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/time/TestTime.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/time/TestTime.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/timestamp/TestDateTrunc.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/timestamp/TestDateTrunc.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/timestamp/TestDateTrunc.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/timestamp/TestDateTrunc.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/timestamp/TestExtract.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/timestamp/TestExtract.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/timestamp/TestExtract.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/timestamp/TestExtract.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/timestamp/TestHumanReadableSeconds.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/timestamp/TestHumanReadableSeconds.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/timestamp/TestHumanReadableSeconds.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/timestamp/TestHumanReadableSeconds.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/timestamp/TestOperators.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/timestamp/TestOperators.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/timestamp/TestOperators.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/timestamp/TestOperators.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/timestamp/TestTimestamp.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/timestamp/TestTimestamp.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/timestamp/TestTimestamp.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/timestamp/TestTimestamp.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/timestamptz/TestAtTimeZone.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/timestamptz/TestAtTimeZone.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/timestamptz/TestAtTimeZone.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/timestamptz/TestAtTimeZone.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/timestamptz/TestDateTrunc.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/timestamptz/TestDateTrunc.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/timestamptz/TestDateTrunc.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/timestamptz/TestDateTrunc.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/timestamptz/TestExtract.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/timestamptz/TestExtract.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/timestamptz/TestExtract.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/timestamptz/TestExtract.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/timestamptz/TestOperators.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/timestamptz/TestOperators.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/timestamptz/TestOperators.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/timestamptz/TestOperators.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/timestamptz/TestTimestampWithTimeZone.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/timestamptz/TestTimestampWithTimeZone.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/timestamptz/TestTimestampWithTimeZone.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/timestamptz/TestTimestampWithTimeZone.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/timetz/TestExtract.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/timetz/TestExtract.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/timetz/TestExtract.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/timetz/TestExtract.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/timetz/TestOperators.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/timetz/TestOperators.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/timetz/TestOperators.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/timetz/TestOperators.sql diff --git a/regression-test/suites/dialect_compatible/sql/presto/scalar/timetz/TestTimeWithTimeZone.sql b/regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/timetz/TestTimeWithTimeZone.sql similarity index 100% rename from regression-test/suites/dialect_compatible/sql/presto/scalar/timetz/TestTimeWithTimeZone.sql rename to regression-test/suites/external_table_p0/dialect_compatible/sql/presto/scalar/timetz/TestTimeWithTimeZone.sql diff --git a/regression-test/suites/external_table_p0/dialect_compatible/sql/test_hive_view_rewrite.groovy b/regression-test/suites/external_table_p0/dialect_compatible/sql/test_hive_view_rewrite.groovy new file mode 100644 index 00000000000000..4793786909f274 --- /dev/null +++ b/regression-test/suites/external_table_p0/dialect_compatible/sql/test_hive_view_rewrite.groovy @@ -0,0 +1,48 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_hive_view_rewrite", "external_docker,hive,external_docker_hive,p0,external") { + String enabled = context.config.otherConfigs.get("enableHiveTest") + if (enabled == null || !enabled.equalsIgnoreCase("true")) { + logger.info("diable Hive test.") + return; + } + + for (String hivePrefix : ["hive2"]) { + setHivePrefix(hivePrefix) + String catalog_name = "test_${hivePrefix}_view_rewrite" + String ex_db_name = "`default`" + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + String hms_port = context.config.otherConfigs.get(hivePrefix + "HmsPort") + String hdfs_port = context.config.otherConfigs.get(hivePrefix + "HdfsPort") + + sql """drop catalog if exists ${catalog_name} """ + + sql """CREATE CATALOG ${catalog_name} PROPERTIES ( + 'type'='hms', + 'hive.metastore.uris' = 'thrift://${externalEnvIp}:${hms_port}', + 'hadoop.username' = 'hive' + );""" + + sql """set sql_dialect='hive';""" + + order_qt_sql1 """ select * from ${catalog_name}.${ex_db_name}.department_view """ + + order_qt_sql2 """ select * from ${catalog_name}.${ex_db_name}.department_nesting_view """ + } +} + From 4f3575ab80df32d59524ca9b55e17ecde0c11ae3 Mon Sep 17 00:00:00 2001 From: Pxl Date: Fri, 24 Jan 2025 11:38:52 +0800 Subject: [PATCH 07/14] =?UTF-8?q?[Bug](function)=20fix=20wrong=20result=20?= =?UTF-8?q?when=20case=20when=20have=20more=20than=20256=20conditions=20an?= =?UTF-8?q?d=20the=E2=80=A6=20(#47179)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What problem does this PR solve? fix wrong result when case when have more than 256 conditions and then column is not nullable ### Check List (For Author) - Test - [x] Regression test - [ ] Unit Test - [ ] Manual test (add detailed scripts or steps below) - [ ] No need to test or manual test. Explain why: - [ ] This is a refactor/code format and no logic has been changed. - [ ] Previous test can cover this change. - [ ] No code files have been changed. - [ ] Other reason - Behavior changed: - [x] No. - [ ] Yes. - Does this need documentation? - [x] No. - [ ] Yes. ### Check List (For Reviewer who merge this PR) - [ ] Confirm the release note - [ ] Confirm test cases - [ ] Confirm document - [ ] Add branch pick label --- be/src/vec/functions/function_case.h | 13 +- .../case_function/test_case_function.out | 10 + .../case_function/test_case_function.groovy | 548 ++++++++++++++++++ 3 files changed, 561 insertions(+), 10 deletions(-) diff --git a/be/src/vec/functions/function_case.h b/be/src/vec/functions/function_case.h index 81f08f682ef0ef..83fa1a5fe58e7b 100644 --- a/be/src/vec/functions/function_case.h +++ b/be/src/vec/functions/function_case.h @@ -174,16 +174,9 @@ class FunctionCase : public IFunction { break; } } else { - if constexpr (when_null) { - if (!then_idx_ptr[row_idx] && when_column_ptr->get_bool(row_idx)) { - then_idx_ptr[row_idx] = i; - break; - } - } else { - if (!then_idx_ptr[row_idx]) { - then_idx_ptr[row_idx] = i; - break; - } + if (!then_idx_ptr[row_idx] && when_column_ptr->get_bool(row_idx)) { + then_idx_ptr[row_idx] = i; + break; } } } diff --git a/regression-test/data/query_p0/sql_functions/case_function/test_case_function.out b/regression-test/data/query_p0/sql_functions/case_function/test_case_function.out index f670d4336d6fa2..db01dac929a0ca 100644 --- a/regression-test/data/query_p0/sql_functions/case_function/test_case_function.out +++ b/regression-test/data/query_p0/sql_functions/case_function/test_case_function.out @@ -39,3 +39,13 @@ number 1 number 5 number 9 +-- !test_not_null_then -- +A259 A259 +A300 OTHER +A5 A5 + +-- !test_null_then -- +A259 A259 +A300 \N +A5 A5 + diff --git a/regression-test/suites/query_p0/sql_functions/case_function/test_case_function.groovy b/regression-test/suites/query_p0/sql_functions/case_function/test_case_function.groovy index 9fe635c4590352..ad61348266ed1f 100644 --- a/regression-test/suites/query_p0/sql_functions/case_function/test_case_function.groovy +++ b/regression-test/suites/query_p0/sql_functions/case_function/test_case_function.groovy @@ -1277,4 +1277,552 @@ suite("test_case_function", "query,p0") { when k1>10 then '+' when k1>10 then '+' when k1<10 then '-' end order by 1, 2""" + + sql "drop table if exists d_table;" + sql """ + create table d_table (k1 varchar(100) not null) + duplicate key (k1) + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + sql "insert into d_table values ('A259'),('A300'),('A5');" + qt_test_not_null_then """ + +SELECT k1, + CASE k1 + WHEN 'A1' THEN 'A1' + WHEN 'A2' THEN 'A2' + WHEN 'A3' THEN 'A3' + WHEN 'A4' THEN 'A4' + WHEN 'A5' THEN 'A5' + WHEN 'A6' THEN 'A6' + WHEN 'A7' THEN 'A7' + WHEN 'A8' THEN 'A8' + WHEN 'A9' THEN 'A9' + WHEN 'A10' THEN 'A10' + WHEN 'A11' THEN 'A11' + WHEN 'A12' THEN 'A12' + WHEN 'A13' THEN 'A13' + WHEN 'A14' THEN 'A14' + WHEN 'A15' THEN 'A15' + WHEN 'A16' THEN 'A16' + WHEN 'A17' THEN 'A17' + WHEN 'A18' THEN 'A18' + WHEN 'A19' THEN 'A19' + WHEN 'A20' THEN 'A20' + WHEN 'A21' THEN 'A21' + WHEN 'A22' THEN 'A22' + WHEN 'A23' THEN 'A23' + WHEN 'A24' THEN 'A24' + WHEN 'A25' THEN 'A25' + WHEN 'A26' THEN 'A26' + WHEN 'A27' THEN 'A27' + WHEN 'A28' THEN 'A28' + WHEN 'A29' THEN 'A29' + WHEN 'A30' THEN 'A30' + WHEN 'A31' THEN 'A31' + WHEN 'A32' THEN 'A32' + WHEN 'A33' THEN 'A33' + WHEN 'A34' THEN 'A34' + WHEN 'A35' THEN 'A35' + WHEN 'A36' THEN 'A36' + WHEN 'A37' THEN 'A37' + WHEN 'A38' THEN 'A38' + WHEN 'A39' THEN 'A39' + WHEN 'A40' THEN 'A40' + WHEN 'A41' THEN 'A41' + WHEN 'A42' THEN 'A42' + WHEN 'A43' THEN 'A43' + WHEN 'A44' THEN 'A44' + WHEN 'A45' THEN 'A45' + WHEN 'A46' THEN 'A46' + WHEN 'A47' THEN 'A47' + WHEN 'A48' THEN 'A48' + WHEN 'A49' THEN 'A49' + WHEN 'A50' THEN 'A50' + WHEN 'A51' THEN 'A51' + WHEN 'A52' THEN 'A52' + WHEN 'A53' THEN 'A53' + WHEN 'A54' THEN 'A54' + WHEN 'A55' THEN 'A55' + WHEN 'A56' THEN 'A56' + WHEN 'A57' THEN 'A57' + WHEN 'A58' THEN 'A58' + WHEN 'A59' THEN 'A59' + WHEN 'A60' THEN 'A60' + WHEN 'A61' THEN 'A61' + WHEN 'A62' THEN 'A62' + WHEN 'A63' THEN 'A63' + WHEN 'A64' THEN 'A64' + WHEN 'A65' THEN 'A65' + WHEN 'A66' THEN 'A66' + WHEN 'A67' THEN 'A67' + WHEN 'A68' THEN 'A68' + WHEN 'A69' THEN 'A69' + WHEN 'A70' THEN 'A70' + WHEN 'A71' THEN 'A71' + WHEN 'A72' THEN 'A72' + WHEN 'A73' THEN 'A73' + WHEN 'A74' THEN 'A74' + WHEN 'A75' THEN 'A75' + WHEN 'A76' THEN 'A76' + WHEN 'A77' THEN 'A77' + WHEN 'A78' THEN 'A78' + WHEN 'A79' THEN 'A79' + WHEN 'A80' THEN 'A80' + WHEN 'A81' THEN 'A81' + WHEN 'A82' THEN 'A82' + WHEN 'A83' THEN 'A83' + WHEN 'A84' THEN 'A84' + WHEN 'A85' THEN 'A85' + WHEN 'A86' THEN 'A86' + WHEN 'A87' THEN 'A87' + WHEN 'A88' THEN 'A88' + WHEN 'A89' THEN 'A89' + WHEN 'A90' THEN 'A90' + WHEN 'A91' THEN 'A91' + WHEN 'A92' THEN 'A92' + WHEN 'A93' THEN 'A93' + WHEN 'A94' THEN 'A94' + WHEN 'A95' THEN 'A95' + WHEN 'A96' THEN 'A96' + WHEN 'A97' THEN 'A97' + WHEN 'A98' THEN 'A98' + WHEN 'A99' THEN 'A99' + WHEN 'A100' THEN 'A100' + WHEN 'A101' THEN 'A101' + WHEN 'A102' THEN 'A102' + WHEN 'A103' THEN 'A103' + WHEN 'A104' THEN 'A104' + WHEN 'A105' THEN 'A105' + WHEN 'A106' THEN 'A106' + WHEN 'A107' THEN 'A107' + WHEN 'A108' THEN 'A108' + WHEN 'A109' THEN 'A109' + WHEN 'A110' THEN 'A110' + WHEN 'A111' THEN 'A111' + WHEN 'A112' THEN 'A112' + WHEN 'A113' THEN 'A113' + WHEN 'A114' THEN 'A114' + WHEN 'A115' THEN 'A115' + WHEN 'A116' THEN 'A116' + WHEN 'A117' THEN 'A117' + WHEN 'A118' THEN 'A118' + WHEN 'A119' THEN 'A119' + WHEN 'A120' THEN 'A120' + WHEN 'A121' THEN 'A121' + WHEN 'A122' THEN 'A122' + WHEN 'A123' THEN 'A123' + WHEN 'A124' THEN 'A124' + WHEN 'A125' THEN 'A125' + WHEN 'A126' THEN 'A126' + WHEN 'A127' THEN 'A127' + WHEN 'A128' THEN 'A128' + WHEN 'A129' THEN 'A129' + WHEN 'A130' THEN 'A130' + WHEN 'A131' THEN 'A131' + WHEN 'A132' THEN 'A132' + WHEN 'A133' THEN 'A133' + WHEN 'A134' THEN 'A134' + WHEN 'A135' THEN 'A135' + WHEN 'A136' THEN 'A136' + WHEN 'A137' THEN 'A137' + WHEN 'A138' THEN 'A138' + WHEN 'A139' THEN 'A139' + WHEN 'A140' THEN 'A140' + WHEN 'A141' THEN 'A141' + WHEN 'A142' THEN 'A142' + WHEN 'A143' THEN 'A143' + WHEN 'A144' THEN 'A144' + WHEN 'A145' THEN 'A145' + WHEN 'A146' THEN 'A146' + WHEN 'A147' THEN 'A147' + WHEN 'A148' THEN 'A148' + WHEN 'A149' THEN 'A149' + WHEN 'A150' THEN 'A150' + WHEN 'A151' THEN 'A151' + WHEN 'A152' THEN 'A152' + WHEN 'A153' THEN 'A153' + WHEN 'A154' THEN 'A154' + WHEN 'A155' THEN 'A155' + WHEN 'A156' THEN 'A156' + WHEN 'A157' THEN 'A157' + WHEN 'A158' THEN 'A158' + WHEN 'A159' THEN 'A159' + WHEN 'A160' THEN 'A160' + WHEN 'A161' THEN 'A161' + WHEN 'A162' THEN 'A162' + WHEN 'A163' THEN 'A163' + WHEN 'A164' THEN 'A164' + WHEN 'A165' THEN 'A165' + WHEN 'A166' THEN 'A166' + WHEN 'A167' THEN 'A167' + WHEN 'A168' THEN 'A168' + WHEN 'A169' THEN 'A169' + WHEN 'A170' THEN 'A170' + WHEN 'A171' THEN 'A171' + WHEN 'A172' THEN 'A172' + WHEN 'A173' THEN 'A173' + WHEN 'A174' THEN 'A174' + WHEN 'A175' THEN 'A175' + WHEN 'A176' THEN 'A176' + WHEN 'A177' THEN 'A177' + WHEN 'A178' THEN 'A178' + WHEN 'A179' THEN 'A179' + WHEN 'A180' THEN 'A180' + WHEN 'A181' THEN 'A181' + WHEN 'A182' THEN 'A182' + WHEN 'A183' THEN 'A183' + WHEN 'A184' THEN 'A184' + WHEN 'A185' THEN 'A185' + WHEN 'A186' THEN 'A186' + WHEN 'A187' THEN 'A187' + WHEN 'A188' THEN 'A188' + WHEN 'A189' THEN 'A189' + WHEN 'A190' THEN 'A190' + WHEN 'A191' THEN 'A191' + WHEN 'A192' THEN 'A192' + WHEN 'A193' THEN 'A193' + WHEN 'A194' THEN 'A194' + WHEN 'A195' THEN 'A195' + WHEN 'A196' THEN 'A196' + WHEN 'A197' THEN 'A197' + WHEN 'A198' THEN 'A198' + WHEN 'A199' THEN 'A199' + WHEN 'A200' THEN 'A200' + WHEN 'A201' THEN 'A201' + WHEN 'A202' THEN 'A202' + WHEN 'A203' THEN 'A203' + WHEN 'A204' THEN 'A204' + WHEN 'A205' THEN 'A205' + WHEN 'A206' THEN 'A206' + WHEN 'A207' THEN 'A207' + WHEN 'A208' THEN 'A208' + WHEN 'A209' THEN 'A209' + WHEN 'A210' THEN 'A210' + WHEN 'A211' THEN 'A211' + WHEN 'A212' THEN 'A212' + WHEN 'A213' THEN 'A213' + WHEN 'A214' THEN 'A214' + WHEN 'A215' THEN 'A215' + WHEN 'A216' THEN 'A216' + WHEN 'A217' THEN 'A217' + WHEN 'A218' THEN 'A218' + WHEN 'A219' THEN 'A219' + WHEN 'A220' THEN 'A220' + WHEN 'A221' THEN 'A221' + WHEN 'A222' THEN 'A222' + WHEN 'A223' THEN 'A223' + WHEN 'A224' THEN 'A224' + WHEN 'A225' THEN 'A225' + WHEN 'A226' THEN 'A226' + WHEN 'A227' THEN 'A227' + WHEN 'A228' THEN 'A228' + WHEN 'A229' THEN 'A229' + WHEN 'A230' THEN 'A230' + WHEN 'A231' THEN 'A231' + WHEN 'A232' THEN 'A232' + WHEN 'A233' THEN 'A233' + WHEN 'A234' THEN 'A234' + WHEN 'A235' THEN 'A235' + WHEN 'A236' THEN 'A236' + WHEN 'A237' THEN 'A237' + WHEN 'A238' THEN 'A238' + WHEN 'A239' THEN 'A239' + WHEN 'A240' THEN 'A240' + WHEN 'A241' THEN 'A241' + WHEN 'A242' THEN 'A242' + WHEN 'A243' THEN 'A243' + WHEN 'A244' THEN 'A244' + WHEN 'A245' THEN 'A245' + WHEN 'A246' THEN 'A246' + WHEN 'A247' THEN 'A247' + WHEN 'A248' THEN 'A248' + WHEN 'A249' THEN 'A249' + WHEN 'A250' THEN 'A250' + WHEN 'A251' THEN 'A251' + WHEN 'A252' THEN 'A252' + WHEN 'A253' THEN 'A253' + WHEN 'A254' THEN 'A254' + WHEN 'A255' THEN 'A255' + WHEN 'A256' THEN 'A256' + WHEN 'A257' THEN 'A257' + WHEN 'A258' THEN 'A258' + WHEN 'A259' THEN 'A259' + WHEN 'A260' THEN 'A260' + ELSE 'OTHER' + END AS k2 +FROM `d_table` order by k1; + """ + + qt_test_null_then """ + +SELECT k1, + CASE k1 + WHEN 'A1' THEN 'A1' + WHEN 'A2' THEN 'A2' + WHEN 'A3' THEN 'A3' + WHEN 'A4' THEN 'A4' + WHEN 'A5' THEN 'A5' + WHEN 'A6' THEN 'A6' + WHEN 'A7' THEN 'A7' + WHEN 'A8' THEN 'A8' + WHEN 'A9' THEN 'A9' + WHEN 'A10' THEN 'A10' + WHEN 'A11' THEN 'A11' + WHEN 'A12' THEN 'A12' + WHEN 'A13' THEN 'A13' + WHEN 'A14' THEN 'A14' + WHEN 'A15' THEN 'A15' + WHEN 'A16' THEN 'A16' + WHEN 'A17' THEN 'A17' + WHEN 'A18' THEN 'A18' + WHEN 'A19' THEN 'A19' + WHEN 'A20' THEN 'A20' + WHEN 'A21' THEN 'A21' + WHEN 'A22' THEN 'A22' + WHEN 'A23' THEN 'A23' + WHEN 'A24' THEN 'A24' + WHEN 'A25' THEN 'A25' + WHEN 'A26' THEN 'A26' + WHEN 'A27' THEN 'A27' + WHEN 'A28' THEN 'A28' + WHEN 'A29' THEN 'A29' + WHEN 'A30' THEN 'A30' + WHEN 'A31' THEN 'A31' + WHEN 'A32' THEN 'A32' + WHEN 'A33' THEN 'A33' + WHEN 'A34' THEN 'A34' + WHEN 'A35' THEN 'A35' + WHEN 'A36' THEN 'A36' + WHEN 'A37' THEN 'A37' + WHEN 'A38' THEN 'A38' + WHEN 'A39' THEN 'A39' + WHEN 'A40' THEN 'A40' + WHEN 'A41' THEN 'A41' + WHEN 'A42' THEN 'A42' + WHEN 'A43' THEN 'A43' + WHEN 'A44' THEN 'A44' + WHEN 'A45' THEN 'A45' + WHEN 'A46' THEN 'A46' + WHEN 'A47' THEN 'A47' + WHEN 'A48' THEN 'A48' + WHEN 'A49' THEN 'A49' + WHEN 'A50' THEN 'A50' + WHEN 'A51' THEN 'A51' + WHEN 'A52' THEN 'A52' + WHEN 'A53' THEN 'A53' + WHEN 'A54' THEN 'A54' + WHEN 'A55' THEN 'A55' + WHEN 'A56' THEN 'A56' + WHEN 'A57' THEN 'A57' + WHEN 'A58' THEN 'A58' + WHEN 'A59' THEN 'A59' + WHEN 'A60' THEN 'A60' + WHEN 'A61' THEN 'A61' + WHEN 'A62' THEN 'A62' + WHEN 'A63' THEN 'A63' + WHEN 'A64' THEN 'A64' + WHEN 'A65' THEN 'A65' + WHEN 'A66' THEN 'A66' + WHEN 'A67' THEN 'A67' + WHEN 'A68' THEN 'A68' + WHEN 'A69' THEN 'A69' + WHEN 'A70' THEN 'A70' + WHEN 'A71' THEN 'A71' + WHEN 'A72' THEN 'A72' + WHEN 'A73' THEN 'A73' + WHEN 'A74' THEN 'A74' + WHEN 'A75' THEN 'A75' + WHEN 'A76' THEN 'A76' + WHEN 'A77' THEN 'A77' + WHEN 'A78' THEN 'A78' + WHEN 'A79' THEN 'A79' + WHEN 'A80' THEN 'A80' + WHEN 'A81' THEN 'A81' + WHEN 'A82' THEN 'A82' + WHEN 'A83' THEN 'A83' + WHEN 'A84' THEN 'A84' + WHEN 'A85' THEN 'A85' + WHEN 'A86' THEN 'A86' + WHEN 'A87' THEN 'A87' + WHEN 'A88' THEN 'A88' + WHEN 'A89' THEN 'A89' + WHEN 'A90' THEN 'A90' + WHEN 'A91' THEN 'A91' + WHEN 'A92' THEN 'A92' + WHEN 'A93' THEN 'A93' + WHEN 'A94' THEN 'A94' + WHEN 'A95' THEN 'A95' + WHEN 'A96' THEN 'A96' + WHEN 'A97' THEN 'A97' + WHEN 'A98' THEN 'A98' + WHEN 'A99' THEN 'A99' + WHEN 'A100' THEN 'A100' + WHEN 'A101' THEN 'A101' + WHEN 'A102' THEN 'A102' + WHEN 'A103' THEN 'A103' + WHEN 'A104' THEN 'A104' + WHEN 'A105' THEN 'A105' + WHEN 'A106' THEN 'A106' + WHEN 'A107' THEN 'A107' + WHEN 'A108' THEN 'A108' + WHEN 'A109' THEN 'A109' + WHEN 'A110' THEN 'A110' + WHEN 'A111' THEN 'A111' + WHEN 'A112' THEN 'A112' + WHEN 'A113' THEN 'A113' + WHEN 'A114' THEN 'A114' + WHEN 'A115' THEN 'A115' + WHEN 'A116' THEN 'A116' + WHEN 'A117' THEN 'A117' + WHEN 'A118' THEN 'A118' + WHEN 'A119' THEN 'A119' + WHEN 'A120' THEN 'A120' + WHEN 'A121' THEN 'A121' + WHEN 'A122' THEN 'A122' + WHEN 'A123' THEN 'A123' + WHEN 'A124' THEN 'A124' + WHEN 'A125' THEN 'A125' + WHEN 'A126' THEN 'A126' + WHEN 'A127' THEN 'A127' + WHEN 'A128' THEN 'A128' + WHEN 'A129' THEN 'A129' + WHEN 'A130' THEN 'A130' + WHEN 'A131' THEN 'A131' + WHEN 'A132' THEN 'A132' + WHEN 'A133' THEN 'A133' + WHEN 'A134' THEN 'A134' + WHEN 'A135' THEN 'A135' + WHEN 'A136' THEN 'A136' + WHEN 'A137' THEN 'A137' + WHEN 'A138' THEN 'A138' + WHEN 'A139' THEN 'A139' + WHEN 'A140' THEN 'A140' + WHEN 'A141' THEN 'A141' + WHEN 'A142' THEN 'A142' + WHEN 'A143' THEN 'A143' + WHEN 'A144' THEN 'A144' + WHEN 'A145' THEN 'A145' + WHEN 'A146' THEN 'A146' + WHEN 'A147' THEN 'A147' + WHEN 'A148' THEN 'A148' + WHEN 'A149' THEN 'A149' + WHEN 'A150' THEN 'A150' + WHEN 'A151' THEN 'A151' + WHEN 'A152' THEN 'A152' + WHEN 'A153' THEN 'A153' + WHEN 'A154' THEN 'A154' + WHEN 'A155' THEN 'A155' + WHEN 'A156' THEN 'A156' + WHEN 'A157' THEN 'A157' + WHEN 'A158' THEN 'A158' + WHEN 'A159' THEN 'A159' + WHEN 'A160' THEN 'A160' + WHEN 'A161' THEN 'A161' + WHEN 'A162' THEN 'A162' + WHEN 'A163' THEN 'A163' + WHEN 'A164' THEN 'A164' + WHEN 'A165' THEN 'A165' + WHEN 'A166' THEN 'A166' + WHEN 'A167' THEN 'A167' + WHEN 'A168' THEN 'A168' + WHEN 'A169' THEN 'A169' + WHEN 'A170' THEN 'A170' + WHEN 'A171' THEN 'A171' + WHEN 'A172' THEN 'A172' + WHEN 'A173' THEN 'A173' + WHEN 'A174' THEN 'A174' + WHEN 'A175' THEN 'A175' + WHEN 'A176' THEN 'A176' + WHEN 'A177' THEN 'A177' + WHEN 'A178' THEN 'A178' + WHEN 'A179' THEN 'A179' + WHEN 'A180' THEN 'A180' + WHEN 'A181' THEN 'A181' + WHEN 'A182' THEN 'A182' + WHEN 'A183' THEN 'A183' + WHEN 'A184' THEN 'A184' + WHEN 'A185' THEN 'A185' + WHEN 'A186' THEN 'A186' + WHEN 'A187' THEN 'A187' + WHEN 'A188' THEN 'A188' + WHEN 'A189' THEN 'A189' + WHEN 'A190' THEN 'A190' + WHEN 'A191' THEN 'A191' + WHEN 'A192' THEN 'A192' + WHEN 'A193' THEN 'A193' + WHEN 'A194' THEN 'A194' + WHEN 'A195' THEN 'A195' + WHEN 'A196' THEN 'A196' + WHEN 'A197' THEN 'A197' + WHEN 'A198' THEN 'A198' + WHEN 'A199' THEN 'A199' + WHEN 'A200' THEN 'A200' + WHEN 'A201' THEN 'A201' + WHEN 'A202' THEN 'A202' + WHEN 'A203' THEN 'A203' + WHEN 'A204' THEN 'A204' + WHEN 'A205' THEN 'A205' + WHEN 'A206' THEN 'A206' + WHEN 'A207' THEN 'A207' + WHEN 'A208' THEN 'A208' + WHEN 'A209' THEN 'A209' + WHEN 'A210' THEN 'A210' + WHEN 'A211' THEN 'A211' + WHEN 'A212' THEN 'A212' + WHEN 'A213' THEN 'A213' + WHEN 'A214' THEN 'A214' + WHEN 'A215' THEN 'A215' + WHEN 'A216' THEN 'A216' + WHEN 'A217' THEN 'A217' + WHEN 'A218' THEN 'A218' + WHEN 'A219' THEN 'A219' + WHEN 'A220' THEN 'A220' + WHEN 'A221' THEN 'A221' + WHEN 'A222' THEN 'A222' + WHEN 'A223' THEN 'A223' + WHEN 'A224' THEN 'A224' + WHEN 'A225' THEN 'A225' + WHEN 'A226' THEN 'A226' + WHEN 'A227' THEN 'A227' + WHEN 'A228' THEN 'A228' + WHEN 'A229' THEN 'A229' + WHEN 'A230' THEN 'A230' + WHEN 'A231' THEN 'A231' + WHEN 'A232' THEN 'A232' + WHEN 'A233' THEN 'A233' + WHEN 'A234' THEN 'A234' + WHEN 'A235' THEN 'A235' + WHEN 'A236' THEN 'A236' + WHEN 'A237' THEN 'A237' + WHEN 'A238' THEN 'A238' + WHEN 'A239' THEN 'A239' + WHEN 'A240' THEN 'A240' + WHEN 'A241' THEN 'A241' + WHEN 'A242' THEN 'A242' + WHEN 'A243' THEN 'A243' + WHEN 'A244' THEN 'A244' + WHEN 'A245' THEN 'A245' + WHEN 'A246' THEN 'A246' + WHEN 'A247' THEN 'A247' + WHEN 'A248' THEN 'A248' + WHEN 'A249' THEN 'A249' + WHEN 'A250' THEN 'A250' + WHEN 'A251' THEN 'A251' + WHEN 'A252' THEN 'A252' + WHEN 'A253' THEN 'A253' + WHEN 'A254' THEN 'A254' + WHEN 'A255' THEN 'A255' + WHEN 'A256' THEN 'A256' + WHEN 'A257' THEN 'A257' + WHEN 'A258' THEN 'A258' + WHEN 'A259' THEN 'A259' + WHEN 'A260' THEN 'A260' + ELSE null + END AS k2 +FROM `d_table` order by k1; + + """ } From 7dd0fc134d7fbe2ee0ccd9f16f9a507f1a61240a Mon Sep 17 00:00:00 2001 From: shuke Date: Fri, 24 Jan 2025 11:45:26 +0800 Subject: [PATCH 08/14] [regression-test](case) fix percentile case (#47359) --- .../aggregate_functions/test_aggregate_all_functions2.out | 3 --- .../aggregate_functions/test_aggregate_all_functions2.groovy | 5 ++++- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/regression-test/data/query_p0/sql_functions/aggregate_functions/test_aggregate_all_functions2.out b/regression-test/data/query_p0/sql_functions/aggregate_functions/test_aggregate_all_functions2.out index 454d1b49353150..27f36371b01533 100644 --- a/regression-test/data/query_p0/sql_functions/aggregate_functions/test_aggregate_all_functions2.out +++ b/regression-test/data/query_p0/sql_functions/aggregate_functions/test_aggregate_all_functions2.out @@ -44,9 +44,6 @@ -- !select_intersect_count_2 -- 5 --- !select_percentile_approx1 -- -0.0 - -- !select_percentile_array -- [255, 1989, 1991] diff --git a/regression-test/suites/query_p0/sql_functions/aggregate_functions/test_aggregate_all_functions2.groovy b/regression-test/suites/query_p0/sql_functions/aggregate_functions/test_aggregate_all_functions2.groovy index 126c2b63f46371..615c3bf98b13fa 100644 --- a/regression-test/suites/query_p0/sql_functions/aggregate_functions/test_aggregate_all_functions2.groovy +++ b/regression-test/suites/query_p0/sql_functions/aggregate_functions/test_aggregate_all_functions2.groovy @@ -66,7 +66,10 @@ suite("test_aggregate_all_functions2") { qt_select_min_by3 """ select min_by(k1,cast(k1 as string)) from baseall; """ qt_select_intersect_count_1 """ select intersect_count(bitmap_from_array(array(1,2,3,4,5)),cast(k1 as string),1,2) from baseall; """ qt_select_intersect_count_2 """ select intersect_count(bitmap_from_array(array(1,2,3,4,5)),k1,1,2) from baseall; """ - qt_select_percentile_approx1 """ select percentile_approx(k2,10001) from baseall; """ + test { + sql """ select percentile_approx(k2,10001) from baseall; """ + exception "INVALID_ARGUMENT" + } qt_select_percentile_array """ select percentile_array(k2,[0.2,0.5,0.7]) from baseall; """ qt_select_array_product """ select array_product(array(cast(k5 as decimalv3(30,10)))) from baseall order by k1; """ qt_select_quantile_percent """ select QUANTILE_PERCENT(QUANTILE_UNION(TO_QUANTILE_STATE(k1,2048)),0.5) from baseall; """ From d1ff16676184a1dcfb7f64c0e67d004dcffc4180 Mon Sep 17 00:00:00 2001 From: James Date: Fri, 24 Jan 2025 12:15:28 +0800 Subject: [PATCH 09/14] [feature](view)Support modify view comment (#46910) ### What problem does this PR solve? Support modify view comment: ALTER VIEW [db_name.]view_name MODIFY COMMENT "new comment" ### Release note Support modify view comment --- .../org/apache/doris/nereids/DorisParser.g4 | 4 ++-- .../java/org/apache/doris/alter/Alter.java | 24 +++++++++++++------ .../apache/doris/analysis/AlterViewStmt.java | 15 ++++++++++++ .../java/org/apache/doris/catalog/Env.java | 4 +++- .../nereids/parser/LogicalPlanBuilder.java | 12 +++++++--- .../plans/commands/AlterViewCommand.java | 5 +++- .../plans/commands/info/AlterViewInfo.java | 20 ++++++++++++++++ .../apache/doris/persist/AlterViewInfo.java | 12 ++++++++-- .../doris/persist/AlterViewInfoTest.java | 2 +- regression-test/data/view_p0/view_p0.out | 15 ++++++++++++ regression-test/suites/view_p0/view_p0.groovy | 23 +++++++++++++++++- 11 files changed, 118 insertions(+), 18 deletions(-) diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 index d24a1f74f2c7ce..9315a67e8510b3 100644 --- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 +++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 @@ -202,8 +202,8 @@ supportedCreateStatement ; supportedAlterStatement - : ALTER VIEW name=multipartIdentifier (LEFT_PAREN cols=simpleColumnDefs RIGHT_PAREN)? - AS query #alterView + : ALTER VIEW name=multipartIdentifier + ((MODIFY commentSpec) | ((LEFT_PAREN cols=simpleColumnDefs RIGHT_PAREN)? AS query)) #alterView | ALTER CATALOG name=identifier RENAME newName=identifier #alterCatalogRename | ALTER ROLE role=identifier commentSpec #alterRole | ALTER STORAGE VAULT name=multipartIdentifier properties=propertyClause #alterStorageVault diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java index 3b59c3bac9b056..3741600a1b7c2e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java @@ -817,23 +817,28 @@ public void processAlterView(AlterViewStmt stmt, ConnectContext ctx) throws User String tableName = dbTableName.getTbl(); View view = (View) db.getTableOrMetaException(tableName, TableType.VIEW); - modifyViewDef(db, view, stmt.getInlineViewDef(), ctx.getSessionVariable().getSqlMode(), stmt.getColumns()); + modifyViewDef(db, view, stmt.getInlineViewDef(), ctx.getSessionVariable().getSqlMode(), stmt.getColumns(), + stmt.getComment()); } private void modifyViewDef(Database db, View view, String inlineViewDef, long sqlMode, - List newFullSchema) throws DdlException { + List newFullSchema, String comment) throws DdlException { db.writeLockOrDdlException(); try { view.writeLockOrDdlException(); try { - view.setInlineViewDefWithSqlMode(inlineViewDef, sqlMode); - view.setNewFullSchema(newFullSchema); + if (comment != null) { + view.setComment(comment); + } else { + view.setInlineViewDefWithSqlMode(inlineViewDef, sqlMode); + view.setNewFullSchema(newFullSchema); + } String viewName = view.getName(); db.unregisterTable(viewName); db.registerTable(view); AlterViewInfo alterViewInfo = new AlterViewInfo(db.getId(), view.getId(), - inlineViewDef, newFullSchema, sqlMode); + inlineViewDef, newFullSchema, sqlMode, comment); Env.getCurrentEnv().getEditLog().logModifyViewDef(alterViewInfo); LOG.info("modify view[{}] definition to {}", viewName, inlineViewDef); } finally { @@ -849,6 +854,7 @@ public void replayModifyViewDef(AlterViewInfo alterViewInfo) throws MetaNotFound long tableId = alterViewInfo.getTableId(); String inlineViewDef = alterViewInfo.getInlineViewDef(); List newFullSchema = alterViewInfo.getNewFullSchema(); + String comment = alterViewInfo.getComment(); Database db = Env.getCurrentInternalCatalog().getDbOrMetaException(dbId); View view = (View) db.getTableOrMetaException(tableId, TableType.VIEW); @@ -857,8 +863,12 @@ public void replayModifyViewDef(AlterViewInfo alterViewInfo) throws MetaNotFound view.writeLock(); try { String viewName = view.getName(); - view.setInlineViewDefWithSqlMode(inlineViewDef, alterViewInfo.getSqlMode()); - view.setNewFullSchema(newFullSchema); + if (comment != null) { + view.setComment(comment); + } else { + view.setInlineViewDefWithSqlMode(inlineViewDef, alterViewInfo.getSqlMode()); + view.setNewFullSchema(newFullSchema); + } // We do not need to init view here. // During the `init` phase, some `Alter-View` statements will access the remote file system, diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterViewStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterViewStmt.java index 5a2d184e20d24c..24efbd909052b6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterViewStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterViewStmt.java @@ -36,14 +36,29 @@ @Deprecated public class AlterViewStmt extends BaseViewStmt implements NotFallbackInParser { + private final String comment; + + public AlterViewStmt(TableName tbl, String comment) { + this(tbl, null, null, comment); + } + public AlterViewStmt(TableName tbl, List cols, QueryStmt queryStmt) { + this(tbl, cols, queryStmt, null); + } + + public AlterViewStmt(TableName tbl, List cols, QueryStmt queryStmt, String comment) { super(tbl, cols, queryStmt); + this.comment = comment; } public TableName getTbl() { return tableName; } + public String getComment() { + return comment; + } + @Override public void analyze(Analyzer analyzer) throws UserException { super.analyze(analyzer); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java index a4e2778cb5abfc..fdfd1e3e9e2e24 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java @@ -5681,8 +5681,10 @@ public void createView(CreateViewStmt stmt) throws DdlException { } if (replace) { + String comment = stmt.getComment(); + comment = comment == null || comment.isEmpty() ? null : comment; AlterViewStmt alterViewStmt = new AlterViewStmt(stmt.getTableName(), stmt.getColWithComments(), - stmt.getViewDefStmt()); + stmt.getViewDefStmt(), comment); alterViewStmt.setInlineViewDef(stmt.getInlineViewDef()); try { alterView(alterViewStmt); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index a3d215441374f5..49c568c58428b4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -1292,9 +1292,15 @@ public AlterMTMVCommand visitAlterMTMV(AlterMTMVContext ctx) { @Override public LogicalPlan visitAlterView(AlterViewContext ctx) { List nameParts = visitMultipartIdentifier(ctx.name); - String querySql = getOriginSql(ctx.query()); - AlterViewInfo info = new AlterViewInfo(new TableNameInfo(nameParts), querySql, - ctx.cols == null ? Lists.newArrayList() : visitSimpleColumnDefs(ctx.cols)); + String comment = ctx.commentSpec() == null ? null : visitCommentSpec(ctx.commentSpec()); + AlterViewInfo info; + if (comment != null) { + info = new AlterViewInfo(new TableNameInfo(nameParts), comment); + } else { + String querySql = getOriginSql(ctx.query()); + info = new AlterViewInfo(new TableNameInfo(nameParts), querySql, + ctx.cols == null ? Lists.newArrayList() : visitSimpleColumnDefs(ctx.cols)); + } return new AlterViewCommand(info); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterViewCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterViewCommand.java index 900986874b8fbf..80e6b4c864007f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterViewCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterViewCommand.java @@ -39,7 +39,10 @@ public AlterViewCommand(AlterViewInfo alterViewInfo) { public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { executor.checkBlockRules(); alterViewInfo.init(ctx); - alterViewInfo.validate(ctx); + // For modify comment command, doesn't need to do validation. + if (alterViewInfo.getComment() == null) { + alterViewInfo.validate(ctx); + } AlterViewStmt alterViewStmt = alterViewInfo.translateToLegacyStmt(ctx); Env.getCurrentEnv().alterView(alterViewStmt); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AlterViewInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AlterViewInfo.java index eb8af115ac0c14..483c37a8388637 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AlterViewInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AlterViewInfo.java @@ -40,10 +40,19 @@ /** AlterViewInfo */ public class AlterViewInfo extends BaseViewInfo { + + private final String comment; + /** constructor*/ public AlterViewInfo(TableNameInfo viewName, String querySql, List simpleColumnDefinitions) { super(viewName, querySql, simpleColumnDefinitions); + this.comment = null; + } + + public AlterViewInfo(TableNameInfo viewName, String comment) { + super(viewName, null, null); + this.comment = comment; } /** init */ @@ -69,6 +78,10 @@ public void init(ConnectContext ctx) throws UserException { ErrorReport.reportAnalysisException(ErrorCode.ERR_TABLE_ACCESS_DENIED_ERROR, PrivPredicate.ALTER.getPrivs().toString(), viewName.getTbl()); } + if (comment != null) { + // Modify comment only. + return; + } analyzeAndFillRewriteSqlMap(querySql, ctx); PlanUtils.OutermostPlanFinderContext outermostPlanFinderContext = new PlanUtils.OutermostPlanFinderContext(); analyzedPlan.accept(PlanUtils.OutermostPlanFinder.INSTANCE, outermostPlanFinderContext); @@ -78,6 +91,9 @@ public void init(ConnectContext ctx) throws UserException { /**translateToLegacyStmt*/ public AlterViewStmt translateToLegacyStmt(ConnectContext ctx) { + if (comment != null) { + return new AlterViewStmt(viewName.transferToTableName(), comment); + } // expand star(*) in project list and replace table name with qualifier String rewrittenSql = rewriteSql(ctx.getStatementContext().getIndexInSqlToString(), querySql); // rewrite project alias @@ -92,4 +108,8 @@ public AlterViewStmt translateToLegacyStmt(ConnectContext ctx) { alterViewStmt.setFinalColumns(finalCols); return alterViewStmt; } + + public String getComment() { + return comment; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/AlterViewInfo.java b/fe/fe-core/src/main/java/org/apache/doris/persist/AlterViewInfo.java index c54dfcf7c3100f..141a35d6834a16 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/AlterViewInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/AlterViewInfo.java @@ -42,18 +42,22 @@ public class AlterViewInfo implements Writable { private long sqlMode; @SerializedName(value = "newFullSchema") private List newFullSchema; + @SerializedName(value = "comment") + private String comment; public AlterViewInfo() { // for persist newFullSchema = Lists.newArrayList(); } - public AlterViewInfo(long dbId, long tableId, String inlineViewDef, List newFullSchema, long sqlMode) { + public AlterViewInfo(long dbId, long tableId, String inlineViewDef, List newFullSchema, long sqlMode, + String comment) { this.dbId = dbId; this.tableId = tableId; this.inlineViewDef = inlineViewDef; this.newFullSchema = newFullSchema; this.sqlMode = sqlMode; + this.comment = comment; } public long getDbId() { @@ -76,6 +80,10 @@ public long getSqlMode() { return sqlMode; } + public String getComment() { + return comment; + } + @Override public int hashCode() { return Objects.hash(dbId, tableId, inlineViewDef, sqlMode, newFullSchema); @@ -92,7 +100,7 @@ public boolean equals(Object other) { AlterViewInfo otherInfo = (AlterViewInfo) other; return dbId == otherInfo.getDbId() && tableId == otherInfo.getTableId() && inlineViewDef.equalsIgnoreCase(otherInfo.getInlineViewDef()) && sqlMode == otherInfo.getSqlMode() - && newFullSchema.equals(otherInfo.getNewFullSchema()); + && newFullSchema.equals(otherInfo.getNewFullSchema()) && Objects.equals(comment, otherInfo.comment); } @Override diff --git a/fe/fe-core/src/test/java/org/apache/doris/persist/AlterViewInfoTest.java b/fe/fe-core/src/test/java/org/apache/doris/persist/AlterViewInfoTest.java index 8b179c2889eef8..b8e8044d22e95f 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/persist/AlterViewInfoTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/persist/AlterViewInfoTest.java @@ -57,7 +57,7 @@ public void testSerializeAlterViewInfo() throws IOException, AnalysisException { Column column1 = new Column("col1", PrimitiveType.BIGINT); Column column2 = new Column("col2", PrimitiveType.DOUBLE); - AlterViewInfo alterViewInfo = new AlterViewInfo(dbId, tableId, inlineViewDef, Lists.newArrayList(column1, column2), sqlMode); + AlterViewInfo alterViewInfo = new AlterViewInfo(dbId, tableId, inlineViewDef, Lists.newArrayList(column1, column2), sqlMode, null); alterViewInfo.write(out); out.flush(); out.close(); diff --git a/regression-test/data/view_p0/view_p0.out b/regression-test/data/view_p0/view_p0.out index 928c01f1aa6b9a..2fd94f5b553d6a 100644 --- a/regression-test/data/view_p0/view_p0.out +++ b/regression-test/data/view_p0/view_p0.out @@ -30,3 +30,18 @@ internal regression_test_view_p0 test_view select 1,to_base64(AES_ENCRYPT('doris -- !show_aes -- test_view_aes CREATE VIEW `test_view_aes` AS SELECT aes_decrypt(from_base64("EXp7k7M9Zv1mIwPpno28Hg=="), '17IMZrGdwWf2Piy8', 'II2HLtihr5TQpQgR', 'AES_128_CBC'); utf8mb4 utf8mb4_0900_bin +-- !comment0 -- +test_view_table2_view CREATE VIEW `test_view_table2_view` AS SELECT \n date_format(`internal`.`regression_test_view_p0`.`test_view_table2`.`c_date`,'%Y-%m-%d') AS `CREATE_DATE`\n FROM \n `internal`.`regression_test_view_p0`.`test_view_table2`\n GROUP BY \n date_format(`internal`.`regression_test_view_p0`.`test_view_table2`.`c_date`, '%Y-%m-%d'); utf8mb4 utf8mb4_0900_bin + +-- !comment1 -- +test_view_table2_view CREATE VIEW `test_view_table2_view` COMMENT 'comment1' AS SELECT \n date_format(`internal`.`regression_test_view_p0`.`test_view_table2`.`c_date`,'%Y-%m-%d') AS `CREATE_DATE`\n FROM \n `internal`.`regression_test_view_p0`.`test_view_table2`\n GROUP BY \n date_format(`internal`.`regression_test_view_p0`.`test_view_table2`.`c_date`, '%Y-%m-%d'); utf8mb4 utf8mb4_0900_bin + +-- !comment2 -- +test_view_table2_view CREATE VIEW `test_view_table2_view` COMMENT 'comment1' AS SELECT\n date_format(`internal`.`regression_test_view_p0`.`test_view_table2`.`c_date`,'%Y-%m-%d') AS `CREATE_DATE`\n FROM\n `internal`.`regression_test_view_p0`.`test_view_table2`\n GROUP BY\n date_format(`internal`.`regression_test_view_p0`.`test_view_table2`.`c_date`, '%Y-%m-%d'); utf8mb4 utf8mb4_0900_bin + +-- !comment4 -- +test_view_table2_view CREATE VIEW `test_view_table2_view` COMMENT 'comment4' AS SELECT\n date_format(`internal`.`regression_test_view_p0`.`test_view_table2`.`c_date`,'%Y-%m-%d') AS `CREATE_DATE`\n FROM\n `internal`.`regression_test_view_p0`.`test_view_table2`\n GROUP BY\n date_format(`internal`.`regression_test_view_p0`.`test_view_table2`.`c_date`, '%Y-%m-%d'); utf8mb4 utf8mb4_0900_bin + +-- !comment_empty -- +test_view_table2_view CREATE VIEW `test_view_table2_view` AS SELECT\n date_format(`internal`.`regression_test_view_p0`.`test_view_table2`.`c_date`,'%Y-%m-%d') AS `CREATE_DATE`\n FROM\n `internal`.`regression_test_view_p0`.`test_view_table2`\n GROUP BY\n date_format(`internal`.`regression_test_view_p0`.`test_view_table2`.`c_date`, '%Y-%m-%d'); utf8mb4 utf8mb4_0900_bin + diff --git a/regression-test/suites/view_p0/view_p0.groovy b/regression-test/suites/view_p0/view_p0.groovy index f69bd808dc6f72..2a7c467c5a7718 100644 --- a/regression-test/suites/view_p0/view_p0.groovy +++ b/regression-test/suites/view_p0/view_p0.groovy @@ -172,7 +172,28 @@ suite("view_p0") { """ sql """select * from test_view_table2_view;""" + + qt_comment0 """show create table test_view_table2_view""" + sql """ALTER VIEW `test_view_table2_view` MODIFY COMMENT "comment1";""" + qt_comment1 """show create table test_view_table2_view""" + + sql """Alter VIEW `test_view_table2_view` + AS + SELECT + date_format(c_date,'%Y-%m-%d') AS `CREATE_DATE` + FROM + test_view_table2 + GROUP BY + date_format(c_date, '%Y-%m-%d'); + """ + qt_comment2 """show create table test_view_table2_view""" + + sql """Alter VIEW `test_view_table2_view` MODIFY COMMENT "comment4";""" + qt_comment4 """show create table test_view_table2_view""" + + sql """Alter VIEW `test_view_table2_view` MODIFY COMMENT "";""" + qt_comment_empty """show create table test_view_table2_view""" + sql """ drop view if exists test_view_table2_view;""" sql """DROP TABLE IF EXISTS test_view_table2""" } - From 88ceb24d08e17abf1f386350af0449c8e617656f Mon Sep 17 00:00:00 2001 From: walter Date: Fri, 24 Jan 2025 14:13:10 +0800 Subject: [PATCH 10/14] [chore](binlog) Log download binlog cost (#47360) --- be/src/service/backend_service.cpp | 21 +++++++++++++++++++-- 1 file changed, 19 insertions(+), 2 deletions(-) diff --git a/be/src/service/backend_service.cpp b/be/src/service/backend_service.cpp index 29504fef1873c2..d80d407a4a1c5d 100644 --- a/be/src/service/backend_service.cpp +++ b/be/src/service/backend_service.cpp @@ -113,10 +113,23 @@ void _ingest_binlog(StorageEngine& engine, IngestBinlogArg* arg) { auto& request = arg->request; + MonotonicStopWatch watch; + watch.start(); + int64_t total_download_bytes = 0; + int64_t total_download_files = 0; TStatus tstatus; std::vector download_success_files; - Defer defer {[=, &engine, &tstatus, ingest_binlog_tstatus = arg->tstatus]() { - LOG(INFO) << "ingest binlog. result: " << apache::thrift::ThriftDebugString(tstatus); + Defer defer {[=, &engine, &tstatus, ingest_binlog_tstatus = arg->tstatus, &watch, + &total_download_bytes, &total_download_files]() { + auto elapsed_time_ms = static_cast(watch.elapsed_time() / 1000000); + double copy_rate = 0.0; + if (elapsed_time_ms > 0) { + copy_rate = total_download_bytes / ((double)elapsed_time_ms) / 1000; + } + LOG(INFO) << "ingest binlog elapsed " << elapsed_time_ms << " ms, download " + << total_download_files << " files, total " << total_download_bytes + << " bytes, avg rate " << copy_rate + << " MB/s. result: " << apache::thrift::ThriftDebugString(tstatus); if (tstatus.status_code != TStatusCode::OK) { // abort txn engine.txn_manager()->abort_txn(partition_id, txn_id, local_tablet_id, @@ -269,6 +282,8 @@ void _ingest_binlog(StorageEngine& engine, IngestBinlogArg* arg) { status.to_thrift(&tstatus); return; } + total_download_bytes = total_size; + total_download_files = num_segments; // Step 5.3: get all segment files for (int64_t segment_index = 0; segment_index < num_segments; ++segment_index) { @@ -442,6 +457,8 @@ void _ingest_binlog(StorageEngine& engine, IngestBinlogArg* arg) { status.to_thrift(&tstatus); return; } + total_download_bytes += total_index_size; + total_download_files += segment_index_file_urls.size(); // Step 6.3: get all segment index files DCHECK(segment_index_file_sizes.size() == segment_index_file_names.size()); From eb4056a2ae342f15cbcc5e4e1e9c6a536082854c Mon Sep 17 00:00:00 2001 From: Thearas Date: Fri, 24 Jan 2025 15:13:56 +0900 Subject: [PATCH 11/14] [fix](hive docker)Sysctl reserve ports error (#47373) ### What problem does this PR solve? Problem Summary: ```bash sysctl: setting key "net.ipv4.ip_local_reserved_ports": Invalid argument ``` --- docker/thirdparties/run-thirdparties-docker.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/thirdparties/run-thirdparties-docker.sh b/docker/thirdparties/run-thirdparties-docker.sh index 07195adf4a6f91..061eef669d4ca9 100755 --- a/docker/thirdparties/run-thirdparties-docker.sh +++ b/docker/thirdparties/run-thirdparties-docker.sh @@ -147,7 +147,7 @@ RUN_LAKESOUL=0 RUN_KERBEROS=0 RUN_MINIO=0 -RESERVED_PORTS="" +RESERVED_PORTS="65535" for element in "${COMPONENTS_ARR[@]}"; do if [[ "${element}"x == "mysql"x ]]; then From 61fba9268b229e6e9f2bf6e213e95abd9a6e630c Mon Sep 17 00:00:00 2001 From: zzzxl Date: Fri, 24 Jan 2025 14:15:25 +0800 Subject: [PATCH 12/14] [feature](inverted index) Add ICU library to support ICU tokenization. (#47323) ### What problem does this PR solve? Problem Summary: 1. To enhance the inverted index tokenization functionality, it is necessary to incorporate the ICU tokenizer. --- dist/LICENSE-dist.txt | 1 + dist/licenses/LICENSE-icu.txt | 542 +++++++++++++++++++++++++++++++++ thirdparty/CHANGELOG.md | 4 + thirdparty/build-thirdparty.sh | 20 ++ thirdparty/vars.sh | 7 + 5 files changed, 574 insertions(+) create mode 100644 dist/licenses/LICENSE-icu.txt diff --git a/dist/LICENSE-dist.txt b/dist/LICENSE-dist.txt index 25cfe636d88d49..512897abb8f0f0 100644 --- a/dist/LICENSE-dist.txt +++ b/dist/LICENSE-dist.txt @@ -1536,3 +1536,4 @@ Other dependencies: * xxhash: 0.8.1 -- licenses/LICENSE-xxhash.txt * concurrentqueue: 1.0.3 -- licenses/LICENSE-concurrentqueue.txt * FlameGraph -- licenses/LICENSE-CDDL-1.0.txt + * icu 75.1 -- licenses/LICENSE-icu.txt diff --git a/dist/licenses/LICENSE-icu.txt b/dist/licenses/LICENSE-icu.txt new file mode 100644 index 00000000000000..56cf04a96b1938 --- /dev/null +++ b/dist/licenses/LICENSE-icu.txt @@ -0,0 +1,542 @@ +UNICODE LICENSE V3 + +COPYRIGHT AND PERMISSION NOTICE + +Copyright © 2016-2024 Unicode, Inc. + +NOTICE TO USER: Carefully read the following legal agreement. BY +DOWNLOADING, INSTALLING, COPYING OR OTHERWISE USING DATA FILES, AND/OR +SOFTWARE, YOU UNEQUIVOCALLY ACCEPT, AND AGREE TO BE BOUND BY, ALL OF THE +TERMS AND CONDITIONS OF THIS AGREEMENT. IF YOU DO NOT AGREE, DO NOT +DOWNLOAD, INSTALL, COPY, DISTRIBUTE OR USE THE DATA FILES OR SOFTWARE. + +Permission is hereby granted, free of charge, to any person obtaining a +copy of data files and any associated documentation (the "Data Files") or +software and any associated documentation (the "Software") to deal in the +Data Files or Software without restriction, including without limitation +the rights to use, copy, modify, merge, publish, distribute, and/or sell +copies of the Data Files or Software, and to permit persons to whom the +Data Files or Software are furnished to do so, provided that either (a) +this copyright and permission notice appear with all copies of the Data +Files or Software, or (b) this copyright and permission notice appear in +associated Documentation. + +THE DATA FILES AND SOFTWARE ARE PROVIDED "AS IS", WITHOUT WARRANTY OF ANY +KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF +MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT OF +THIRD PARTY RIGHTS. + +IN NO EVENT SHALL THE COPYRIGHT HOLDER OR HOLDERS INCLUDED IN THIS NOTICE +BE LIABLE FOR ANY CLAIM, OR ANY SPECIAL INDIRECT OR CONSEQUENTIAL DAMAGES, +OR ANY DAMAGES WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, +WHETHER IN AN ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, +ARISING OUT OF OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THE DATA +FILES OR SOFTWARE. + +Except as contained in this notice, the name of a copyright holder shall +not be used in advertising or otherwise to promote the sale, use or other +dealings in these Data Files or Software without prior written +authorization of the copyright holder. + +SPDX-License-Identifier: Unicode-3.0 + +---------------------------------------------------------------------- + +Third-Party Software Licenses + +This section contains third-party software notices and/or additional +terms for licensed third-party software components included within ICU +libraries. + +---------------------------------------------------------------------- + +ICU License - ICU 1.8.1 to ICU 57.1 + +COPYRIGHT AND PERMISSION NOTICE + +Copyright (c) 1995-2016 International Business Machines Corporation and others +All rights reserved. + +Permission is hereby granted, free of charge, to any person obtaining +a copy of this software and associated documentation files (the +"Software"), to deal in the Software without restriction, including +without limitation the rights to use, copy, modify, merge, publish, +distribute, and/or sell copies of the Software, and to permit persons +to whom the Software is furnished to do so, provided that the above +copyright notice(s) and this permission notice appear in all copies of +the Software and that both the above copyright notice(s) and this +permission notice appear in supporting documentation. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF +MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT +OF THIRD PARTY RIGHTS. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR +HOLDERS INCLUDED IN THIS NOTICE BE LIABLE FOR ANY CLAIM, OR ANY +SPECIAL INDIRECT OR CONSEQUENTIAL DAMAGES, OR ANY DAMAGES WHATSOEVER +RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN ACTION OF +CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF OR IN +CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE. + +Except as contained in this notice, the name of a copyright holder +shall not be used in advertising or otherwise to promote the sale, use +or other dealings in this Software without prior written authorization +of the copyright holder. + +All trademarks and registered trademarks mentioned herein are the +property of their respective owners. + +---------------------------------------------------------------------- + +Chinese/Japanese Word Break Dictionary Data (cjdict.txt) + + # The Google Chrome software developed by Google is licensed under + # the BSD license. Other software included in this distribution is + # provided under other licenses, as set forth below. + # + # The BSD License + # http://opensource.org/licenses/bsd-license.php + # Copyright (C) 2006-2008, Google Inc. + # + # All rights reserved. + # + # Redistribution and use in source and binary forms, with or without + # modification, are permitted provided that the following conditions are met: + # + # Redistributions of source code must retain the above copyright notice, + # this list of conditions and the following disclaimer. + # Redistributions in binary form must reproduce the above + # copyright notice, this list of conditions and the following + # disclaimer in the documentation and/or other materials provided with + # the distribution. + # Neither the name of Google Inc. nor the names of its + # contributors may be used to endorse or promote products derived from + # this software without specific prior written permission. + # + # + # THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND + # CONTRIBUTORS "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, + # INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF + # MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + # DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE + # LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR + # CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF + # SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR + # BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF + # LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING + # NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS + # SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + # + # + # The word list in cjdict.txt are generated by combining three word lists + # listed below with further processing for compound word breaking. The + # frequency is generated with an iterative training against Google web + # corpora. + # + # * Libtabe (Chinese) + # - https://sourceforge.net/project/?group_id=1519 + # - Its license terms and conditions are shown below. + # + # * IPADIC (Japanese) + # - http://chasen.aist-nara.ac.jp/chasen/distribution.html + # - Its license terms and conditions are shown below. + # + # ---------COPYING.libtabe ---- BEGIN-------------------- + # + # /* + # * Copyright (c) 1999 TaBE Project. + # * Copyright (c) 1999 Pai-Hsiang Hsiao. + # * All rights reserved. + # * + # * Redistribution and use in source and binary forms, with or without + # * modification, are permitted provided that the following conditions + # * are met: + # * + # * . Redistributions of source code must retain the above copyright + # * notice, this list of conditions and the following disclaimer. + # * . Redistributions in binary form must reproduce the above copyright + # * notice, this list of conditions and the following disclaimer in + # * the documentation and/or other materials provided with the + # * distribution. + # * . Neither the name of the TaBE Project nor the names of its + # * contributors may be used to endorse or promote products derived + # * from this software without specific prior written permission. + # * + # * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + # * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + # * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS + # * FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE + # * REGENTS OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, + # * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES + # * (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR + # * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) + # * HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, + # * STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) + # * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED + # * OF THE POSSIBILITY OF SUCH DAMAGE. + # */ + # + # /* + # * Copyright (c) 1999 Computer Systems and Communication Lab, + # * Institute of Information Science, Academia + # * Sinica. All rights reserved. + # * + # * Redistribution and use in source and binary forms, with or without + # * modification, are permitted provided that the following conditions + # * are met: + # * + # * . Redistributions of source code must retain the above copyright + # * notice, this list of conditions and the following disclaimer. + # * . Redistributions in binary form must reproduce the above copyright + # * notice, this list of conditions and the following disclaimer in + # * the documentation and/or other materials provided with the + # * distribution. + # * . Neither the name of the Computer Systems and Communication Lab + # * nor the names of its contributors may be used to endorse or + # * promote products derived from this software without specific + # * prior written permission. + # * + # * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + # * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + # * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS + # * FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE + # * REGENTS OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, + # * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES + # * (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR + # * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) + # * HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, + # * STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) + # * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED + # * OF THE POSSIBILITY OF SUCH DAMAGE. + # */ + # + # Copyright 1996 Chih-Hao Tsai @ Beckman Institute, + # University of Illinois + # c-tsai4@uiuc.edu http://casper.beckman.uiuc.edu/~c-tsai4 + # + # ---------------COPYING.libtabe-----END-------------------------------- + # + # + # ---------------COPYING.ipadic-----BEGIN------------------------------- + # + # Copyright 2000, 2001, 2002, 2003 Nara Institute of Science + # and Technology. All Rights Reserved. + # + # Use, reproduction, and distribution of this software is permitted. + # Any copy of this software, whether in its original form or modified, + # must include both the above copyright notice and the following + # paragraphs. + # + # Nara Institute of Science and Technology (NAIST), + # the copyright holders, disclaims all warranties with regard to this + # software, including all implied warranties of merchantability and + # fitness, in no event shall NAIST be liable for + # any special, indirect or consequential damages or any damages + # whatsoever resulting from loss of use, data or profits, whether in an + # action of contract, negligence or other tortuous action, arising out + # of or in connection with the use or performance of this software. + # + # A large portion of the dictionary entries + # originate from ICOT Free Software. The following conditions for ICOT + # Free Software applies to the current dictionary as well. + # + # Each User may also freely distribute the Program, whether in its + # original form or modified, to any third party or parties, PROVIDED + # that the provisions of Section 3 ("NO WARRANTY") will ALWAYS appear + # on, or be attached to, the Program, which is distributed substantially + # in the same form as set out herein and that such intended + # distribution, if actually made, will neither violate or otherwise + # contravene any of the laws and regulations of the countries having + # jurisdiction over the User or the intended distribution itself. + # + # NO WARRANTY + # + # The program was produced on an experimental basis in the course of the + # research and development conducted during the project and is provided + # to users as so produced on an experimental basis. Accordingly, the + # program is provided without any warranty whatsoever, whether express, + # implied, statutory or otherwise. The term "warranty" used herein + # includes, but is not limited to, any warranty of the quality, + # performance, merchantability and fitness for a particular purpose of + # the program and the nonexistence of any infringement or violation of + # any right of any third party. + # + # Each user of the program will agree and understand, and be deemed to + # have agreed and understood, that there is no warranty whatsoever for + # the program and, accordingly, the entire risk arising from or + # otherwise connected with the program is assumed by the user. + # + # Therefore, neither ICOT, the copyright holder, or any other + # organization that participated in or was otherwise related to the + # development of the program and their respective officials, directors, + # officers and other employees shall be held liable for any and all + # damages, including, without limitation, general, special, incidental + # and consequential damages, arising out of or otherwise in connection + # with the use or inability to use the program or any product, material + # or result produced or otherwise obtained by using the program, + # regardless of whether they have been advised of, or otherwise had + # knowledge of, the possibility of such damages at any time during the + # project or thereafter. Each user will be deemed to have agreed to the + # foregoing by his or her commencement of use of the program. The term + # "use" as used herein includes, but is not limited to, the use, + # modification, copying and distribution of the program and the + # production of secondary products from the program. + # + # In the case where the program, whether in its original form or + # modified, was distributed or delivered to or received by a user from + # any person, organization or entity other than ICOT, unless it makes or + # grants independently of ICOT any specific warranty to the user in + # writing, such person, organization or entity, will also be exempted + # from and not be held liable to the user for any such damages as noted + # above as far as the program is concerned. + # + # ---------------COPYING.ipadic-----END---------------------------------- + +---------------------------------------------------------------------- + +Lao Word Break Dictionary Data (laodict.txt) + + # Copyright (C) 2016 and later: Unicode, Inc. and others. + # License & terms of use: http://www.unicode.org/copyright.html + # Copyright (c) 2015 International Business Machines Corporation + # and others. All Rights Reserved. + # + # Project: https://github.com/rober42539/lao-dictionary + # Dictionary: https://github.com/rober42539/lao-dictionary/laodict.txt + # License: https://github.com/rober42539/lao-dictionary/LICENSE.txt + # (copied below) + # + # This file is derived from the above dictionary version of Nov 22, 2020 + # ---------------------------------------------------------------------- + # Copyright (C) 2013 Brian Eugene Wilson, Robert Martin Campbell. + # All rights reserved. + # + # Redistribution and use in source and binary forms, with or without + # modification, are permitted provided that the following conditions are met: + # + # Redistributions of source code must retain the above copyright notice, this + # list of conditions and the following disclaimer. Redistributions in binary + # form must reproduce the above copyright notice, this list of conditions and + # the following disclaimer in the documentation and/or other materials + # provided with the distribution. + # + # THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + # "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + # LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS + # FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE + # COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, + # INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES + # (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR + # SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) + # HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, + # STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) + # ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED + # OF THE POSSIBILITY OF SUCH DAMAGE. + # -------------------------------------------------------------------------- + +---------------------------------------------------------------------- + +Burmese Word Break Dictionary Data (burmesedict.txt) + + # Copyright (c) 2014 International Business Machines Corporation + # and others. All Rights Reserved. + # + # This list is part of a project hosted at: + # github.com/kanyawtech/myanmar-karen-word-lists + # + # -------------------------------------------------------------------------- + # Copyright (c) 2013, LeRoy Benjamin Sharon + # All rights reserved. + # + # Redistribution and use in source and binary forms, with or without + # modification, are permitted provided that the following conditions + # are met: Redistributions of source code must retain the above + # copyright notice, this list of conditions and the following + # disclaimer. Redistributions in binary form must reproduce the + # above copyright notice, this list of conditions and the following + # disclaimer in the documentation and/or other materials provided + # with the distribution. + # + # Neither the name Myanmar Karen Word Lists, nor the names of its + # contributors may be used to endorse or promote products derived + # from this software without specific prior written permission. + # + # THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND + # CONTRIBUTORS "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, + # INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF + # MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + # DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS + # BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, + # EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED + # TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + # DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON + # ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR + # TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF + # THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF + # SUCH DAMAGE. + # -------------------------------------------------------------------------- + +---------------------------------------------------------------------- + +Time Zone Database + + ICU uses the public domain data and code derived from Time Zone +Database for its time zone support. The ownership of the TZ database +is explained in BCP 175: Procedure for Maintaining the Time Zone +Database section 7. + + # 7. Database Ownership + # + # The TZ database itself is not an IETF Contribution or an IETF + # document. Rather it is a pre-existing and regularly updated work + # that is in the public domain, and is intended to remain in the + # public domain. Therefore, BCPs 78 [RFC5378] and 79 [RFC3979] do + # not apply to the TZ Database or contributions that individuals make + # to it. Should any claims be made and substantiated against the TZ + # Database, the organization that is providing the IANA + # Considerations defined in this RFC, under the memorandum of + # understanding with the IETF, currently ICANN, may act in accordance + # with all competent court orders. No ownership claims will be made + # by ICANN or the IETF Trust on the database or the code. Any person + # making a contribution to the database or code waives all rights to + # future claims in that contribution or in the TZ Database. + +---------------------------------------------------------------------- + +Google double-conversion + +Copyright 2006-2011, the V8 project authors. All rights reserved. +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following + disclaimer in the documentation and/or other materials provided + with the distribution. + * Neither the name of Google Inc. nor the names of its + contributors may be used to endorse or promote products derived + from this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +---------------------------------------------------------------------- + +JSON parsing library (nlohmann/json) + +File: vendor/json/upstream/single_include/nlohmann/json.hpp (only for ICU4C) + +MIT License + +Copyright (c) 2013-2022 Niels Lohmann + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. + +---------------------------------------------------------------------- + +File: aclocal.m4 (only for ICU4C) +Section: pkg.m4 - Macros to locate and utilise pkg-config. + + +Copyright © 2004 Scott James Remnant . +Copyright © 2012-2015 Dan Nicholson + +This program is free software; you can redistribute it and/or modify +it under the terms of the GNU General Public License as published by +the Free Software Foundation; either version 2 of the License, or +(at your option) any later version. + +This program is distributed in the hope that it will be useful, but +WITHOUT ANY WARRANTY; without even the implied warranty of +MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU +General Public License for more details. + +You should have received a copy of the GNU General Public License +along with this program; if not, write to the Free Software +Foundation, Inc., 59 Temple Place - Suite 330, Boston, MA +02111-1307, USA. + +As a special exception to the GNU General Public License, if you +distribute this file as part of a program that contains a +configuration script generated by Autoconf, you may include it under +the same distribution terms that you use for the rest of that +program. + + +(The condition for the exception is fulfilled because +ICU4C includes a configuration script generated by Autoconf, +namely the `configure` script.) + +---------------------------------------------------------------------- + +File: config.guess (only for ICU4C) + + +This file is free software; you can redistribute it and/or modify it +under the terms of the GNU General Public License as published by +the Free Software Foundation, either version 3 of the License, or +(at your option) any later version. + +This program is distributed in the hope that it will be useful, but +WITHOUT ANY WARRANTY; without even the implied warranty of +MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU +General Public License for more details. + +You should have received a copy of the GNU General Public License +along with this program; if not, see . + +As a special exception to the GNU General Public License, if you +distribute this file as part of a program that contains a +configuration script generated by Autoconf, you may include it under +the same distribution terms that you use for the rest of that +program. This Exception is an additional permission under section 7 +of the GNU General Public License, version 3 ("GPLv3"). + + +(The condition for the exception is fulfilled because +ICU4C includes a configuration script generated by Autoconf, +namely the `configure` script.) + +---------------------------------------------------------------------- + +File: install-sh (only for ICU4C) + + +Copyright 1991 by the Massachusetts Institute of Technology + +Permission to use, copy, modify, distribute, and sell this software and its +documentation for any purpose is hereby granted without fee, provided that +the above copyright notice appear in all copies and that both that +copyright notice and this permission notice appear in supporting +documentation, and that the name of M.I.T. not be used in advertising or +publicity pertaining to distribution of the software without specific, +written prior permission. M.I.T. makes no representations about the +suitability of this software for any purpose. It is provided "as is" +without express or implied warranty. \ No newline at end of file diff --git a/thirdparty/CHANGELOG.md b/thirdparty/CHANGELOG.md index 7c74c7a0997430..1f86f45c116a94 100644 --- a/thirdparty/CHANGELOG.md +++ b/thirdparty/CHANGELOG.md @@ -2,6 +2,10 @@ This file contains version of the third-party dependency libraries in the build-env image. The docker build-env image is apache/doris, and the tag is `build-env-${version}` +## 20250123 + +- Added: icu 75-1, develop ICU tokenizer based on ICU library. + ## 20241031 - Modified: hadoop-libs 3.3.6.3 -> 3.3.6.4 diff --git a/thirdparty/build-thirdparty.sh b/thirdparty/build-thirdparty.sh index 333787cb64d808..3c72137c10cf47 100755 --- a/thirdparty/build-thirdparty.sh +++ b/thirdparty/build-thirdparty.sh @@ -1836,6 +1836,25 @@ build_dragonbox() { "${BUILD_SYSTEM}" install } +# icu +build_icu() { + check_if_source_exist "${ICU_SOURCE}" + cd "${TP_SOURCE_DIR}/${ICU_SOURCE}/icu4c/source" + + rm -rf "${BUILD_DIR}" + mkdir -p "${BUILD_DIR}" + cd "${BUILD_DIR}" + + ../configure --prefix="${TP_INSTALL_DIR}" \ + --disable-shared \ + --enable-static \ + --disable-samples \ + --disable-tests + + make -j "${PARALLEL}" + make install +} + if [[ "${#packages[@]}" -eq 0 ]]; then packages=( odbc @@ -1905,6 +1924,7 @@ if [[ "${#packages[@]}" -eq 0 ]]; then azure dragonbox brotli + icu ) if [[ "$(uname -s)" == 'Darwin' ]]; then read -r -a packages <<<"binutils gettext ${packages[*]}" diff --git a/thirdparty/vars.sh b/thirdparty/vars.sh index 5e442a2b80805a..a09831cead560e 100644 --- a/thirdparty/vars.sh +++ b/thirdparty/vars.sh @@ -520,6 +520,12 @@ DRAGONBOX_NAME=dragonbox-1.1.3.tar.gz DRAGONBOX_SOURCE=dragonbox-1.1.3 DRAGONBOX_MD5SUM="889dc00db9612c6949a4ccf8115e0e6a" +# icu +ICU_DOWNLOAD="https://github.com/unicode-org/icu/archive/refs/tags/release-75-1.tar.gz" +ICU_NAME=release-75-1.tar.gz +ICU_SOURCE=icu-release-75-1 +ICU_MD5SUM="4003649b8731f938c852748ffa393847" + # all thirdparties which need to be downloaded is set in array TP_ARCHIVES export TP_ARCHIVES=( 'LIBEVENT' @@ -597,6 +603,7 @@ export TP_ARCHIVES=( 'BASE64' 'AZURE' 'DRAGONBOX' + 'ICU' ) if [[ "$(uname -s)" == 'Darwin' ]]; then From 6b10ee7f778bc66e71c31e1dc0efab84d1a97a00 Mon Sep 17 00:00:00 2001 From: Pxl Date: Fri, 24 Jan 2025 14:43:28 +0800 Subject: [PATCH 13/14] [Chore](fuzzy) remove some spill variable fuzzy (#47246) ### What problem does this PR solve? remove some spill variable fuzzy those spill variables will be deprecated soon ### Check List (For Author) - Test - [ ] Regression test - [ ] Unit Test - [ ] Manual test (add detailed scripts or steps below) - [x] No need to test or manual test. Explain why: - [ ] This is a refactor/code format and no logic has been changed. - [x] Previous test can cover this change. - [ ] No code files have been changed. - [ ] Other reason - Behavior changed: - [x] No. - [ ] Yes. - Does this need documentation? - [x] No. - [ ] Yes. ### Check List (For Reviewer who merge this PR) - [ ] Confirm the release note - [ ] Confirm test cases - [ ] Confirm document - [ ] Add branch pick label --- .../org/apache/doris/qe/SessionVariable.java | 56 +++++-------------- 1 file changed, 14 insertions(+), 42 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java index a694b70250f18a..3296396520ad92 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java @@ -1106,7 +1106,7 @@ public enum IgnoreSplitType { @VariableMgr.VarAttr(name = ENABLE_VECTORIZED_ENGINE, varType = VariableAnnotation.REMOVED) public boolean enableVectorizedEngine = true; - @VariableMgr.VarAttr(name = ENABLE_PIPELINE_ENGINE, fuzzy = true, needForward = true, + @VariableMgr.VarAttr(name = ENABLE_PIPELINE_ENGINE, fuzzy = false, needForward = true, varType = VariableAnnotation.REMOVED) private boolean enablePipelineEngine = true; @@ -1219,7 +1219,7 @@ public enum IgnoreSplitType { @VariableMgr.VarAttr(name = RUNTIME_FILTER_WAIT_TIME_MS, needForward = true) private int runtimeFilterWaitTimeMs = 1000; - @VariableMgr.VarAttr(name = runtime_filter_wait_infinitely, needForward = true) + @VariableMgr.VarAttr(name = runtime_filter_wait_infinitely, fuzzy = true, needForward = true) private boolean runtimeFilterWaitInfinitely = false; @VariableMgr.VarAttr(name = RUNTIME_FILTERS_MAX_NUM, needForward = true) @@ -1528,7 +1528,7 @@ public void setEnableLeftZigZag(boolean enableLeftZigZag) { @VariableMgr.VarAttr(name = ENABLE_COMMON_EXPR_PUSHDOWN, fuzzy = true) public boolean enableCommonExprPushdown = true; - @VariableMgr.VarAttr(name = ENABLE_LOCAL_EXCHANGE, fuzzy = true, flag = VariableMgr.INVISIBLE, + @VariableMgr.VarAttr(name = ENABLE_LOCAL_EXCHANGE, fuzzy = false, flag = VariableMgr.INVISIBLE, varType = VariableAnnotation.DEPRECATED) public boolean enableLocalExchange = true; @@ -2220,7 +2220,7 @@ public void setIgnoreShapePlanNodes(String ignoreShapePlanNodes) { description = {"控制是否启用join算子落盘。默认为 false。", "Controls whether to enable spill to disk of join operation. " + "The default value is false."}, - needForward = true, fuzzy = true) + needForward = true, fuzzy = false) public boolean enableJoinSpill = false; @VariableMgr.VarAttr( @@ -2228,7 +2228,7 @@ public void setIgnoreShapePlanNodes(String ignoreShapePlanNodes) { description = {"控制是否启用排序算子落盘。默认为 false。", "Controls whether to enable spill to disk of sort operation. " + "The default value is false."}, - needForward = true, fuzzy = true) + needForward = true, fuzzy = false) public boolean enableSortSpill = false; @VariableMgr.VarAttr( @@ -2245,7 +2245,7 @@ public void setIgnoreShapePlanNodes(String ignoreShapePlanNodes) { description = {"控制是否启用聚合算子落盘。默认为 false。", "Controls whether to enable spill to disk of aggregation operation. " + "The default value is false."}, - needForward = true, fuzzy = true) + needForward = true, fuzzy = false) public boolean enableAggSpill = false; @VariableMgr.VarAttr( @@ -2253,7 +2253,7 @@ public void setIgnoreShapePlanNodes(String ignoreShapePlanNodes) { description = {"控制是否开启强制落盘(即使在内存足够的情况),默认为 false。", "Controls whether enable force spill." }, - needForward = true, fuzzy = true + needForward = true, fuzzy = false ) public boolean enableForceSpill = false; @@ -2261,14 +2261,14 @@ public void setIgnoreShapePlanNodes(String ignoreShapePlanNodes) { name = DATA_QUEUE_MAX_BLOCKS, description = {"DataQueue 中每个子队列允许最大的 block 个数", "Max blocks in DataQueue."}, - needForward = true, fuzzy = true) + needForward = true, fuzzy = false) public long dataQueueMaxBlocks = 1; @VariableMgr.VarAttr( name = FUZZY_DISABLE_RUNTIME_FILTER_IN_BE, description = {"在 BE 上开启禁用 runtime filter 的随机开关,用于测试", "Disable the runtime filter on the BE for testing purposes."}, - needForward = true, fuzzy = false) + needForward = true, fuzzy = true) public boolean fuzzyDisableRuntimeFilterInBE = false; // If the memory consumption of sort node exceed this limit, will trigger spill to disk; @@ -2280,13 +2280,13 @@ public void setIgnoreShapePlanNodes(String ignoreShapePlanNodes) { // The memory limit of streaming agg when spilling is enabled // NOTE: streaming agg operator will not spill to disk. - @VariableMgr.VarAttr(name = SPILL_STREAMING_AGG_MEM_LIMIT, fuzzy = true) + @VariableMgr.VarAttr(name = SPILL_STREAMING_AGG_MEM_LIMIT, fuzzy = false) public long spillStreamingAggMemLimit = 268435456; //256MB public static final int MIN_EXTERNAL_AGG_PARTITION_BITS = 4; public static final int MAX_EXTERNAL_AGG_PARTITION_BITS = 20; @VariableMgr.VarAttr(name = EXTERNAL_AGG_PARTITION_BITS, - checker = "checkExternalAggPartitionBits", fuzzy = true) + checker = "checkExternalAggPartitionBits", fuzzy = false) public int externalAggPartitionBits = 5; // means that the hash table will be partitioned into 32 blocks. @VariableMgr.VarAttr(name = USE_MAX_LENGTH_OF_VARCHAR_IN_CTAS, needForward = true, description = { @@ -2502,7 +2502,7 @@ public void initFuzzyModeVariables() { this.runtimeFilterType = 1 << randomInt; this.enableParallelScan = random.nextInt(2) == 0; - this.enableRuntimeFilterPrune = (randomInt % 2) == 0; + this.enableRuntimeFilterPrune = (randomInt % 10) == 0; switch (randomInt) { case 0: @@ -2533,40 +2533,12 @@ public void initFuzzyModeVariables() { this.enableFoldConstantByBe = false; } - this.fuzzyDisableRuntimeFilterInBE = true; } + this.fuzzyDisableRuntimeFilterInBE = random.nextBoolean(); + this.runtimeFilterWaitInfinitely = random.nextBoolean(); // set random 1, 10, 100, 1000, 10000 this.topnOptLimitThreshold = (int) Math.pow(10, random.nextInt(5)); - - // for spill to disk - if (Config.pull_request_id > 10000) { - if (Config.pull_request_id % 2 == 0) { - this.enableJoinSpill = true; - this.enableSortSpill = true; - this.enableAggSpill = true; - - randomInt = random.nextInt(4); - switch (randomInt) { - case 0: - this.minRevocableMem = 0; - break; - case 1: - this.minRevocableMem = 1; - break; - case 2: - this.minRevocableMem = 1024 * 1024; - break; - default: - this.minRevocableMem = 100L * 1024 * 1024 * 1024; - break; - } - } else { - this.enableJoinSpill = false; - this.enableSortSpill = false; - this.enableAggSpill = false; - } - } } public String printFuzzyVariables() { From 4aaa65744b2dc5dba22da2d0e511d3e87f342147 Mon Sep 17 00:00:00 2001 From: morrySnow Date: Fri, 24 Jan 2025 14:56:28 +0800 Subject: [PATCH 14/14] [fix](Nereids) Use the schema saved during planning as the schema of the original target table (#47337) ### What problem does this PR solve? Related PR: #47033 #45045 Problem Summary: because schema change does not involve recreating the table object, but rather rebuilding the full schema. So, we should use the schema saved during planning as the schema of the original target table. --- .../apache/doris/nereids/StatementContext.java | 7 +++++++ .../generator/PlanPatternGeneratorAnalyzer.java | 4 +--- .../nereids/rules/analysis/CollectRelation.java | 14 +++++++++++--- .../commands/insert/InsertIntoTableCommand.java | 15 ++++++--------- 4 files changed, 25 insertions(+), 15 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java index 597cef2d47e8c1..75353f446a4433 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java @@ -18,6 +18,7 @@ package org.apache.doris.nereids; import org.apache.doris.analysis.StatementBase; +import org.apache.doris.catalog.Column; import org.apache.doris.catalog.TableIf; import org.apache.doris.catalog.View; import org.apache.doris.catalog.constraint.TableIdentifier; @@ -178,6 +179,8 @@ public enum TableFrom { private final Map, TableIf> insertTargetTables = Maps.newHashMap(); // save view's def and sql mode to avoid them change before lock private final Map, Pair> viewInfos = Maps.newHashMap(); + // save insert into schema to avoid schema changed between two read locks + private final List insertTargetSchema = new ArrayList<>(); // for create view support in nereids // key is the start and end position of the sql substring that needs to be replaced, @@ -281,6 +284,10 @@ public Map, TableIf> getTables() { return tables; } + public List getInsertTargetSchema() { + return insertTargetSchema; + } + public void setTables(Map, TableIf> tables) { this.tables.clear(); this.tables.putAll(tables); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/PlanPatternGeneratorAnalyzer.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/PlanPatternGeneratorAnalyzer.java index 99d7c308dacf0d..23e7b5eca762ac 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/PlanPatternGeneratorAnalyzer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/generator/PlanPatternGeneratorAnalyzer.java @@ -19,7 +19,6 @@ import org.apache.doris.nereids.pattern.generator.javaast.ClassDeclaration; -import java.lang.reflect.Modifier; import java.util.List; import java.util.Map; import java.util.Optional; @@ -45,8 +44,7 @@ public String generatePatterns(String className, String parentClassName, boolean Map> planClassMap = analyzer.getParentClassMap().entrySet().stream() .filter(kv -> kv.getValue().contains("org.apache.doris.nereids.trees.plans.Plan")) .filter(kv -> !kv.getKey().name.equals("GroupPlan")) - .filter(kv -> !Modifier.isAbstract(kv.getKey().modifiers.mod) - && kv.getKey() instanceof ClassDeclaration) + .filter(kv -> kv.getKey() instanceof ClassDeclaration) .collect(Collectors.toMap(kv -> (ClassDeclaration) kv.getKey(), kv -> kv.getValue())); List generators = planClassMap.entrySet() diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CollectRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CollectRelation.java index 9426ab4d382b77..92a4fb76d49aaa 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CollectRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CollectRelation.java @@ -41,6 +41,7 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalCTE; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; import org.apache.doris.nereids.trees.plans.logical.LogicalSubQueryAlias; +import org.apache.doris.nereids.trees.plans.logical.UnboundLogicalSink; import org.apache.doris.nereids.util.RelationUtil; import com.google.common.collect.ImmutableList; @@ -75,8 +76,8 @@ public List buildRules() { unboundRelation() .thenApply(this::collectFromUnboundRelation) .toRule(RuleType.COLLECT_TABLE_FROM_RELATION), - unboundTableSink() - .thenApply(this::collectFromUnboundTableSink) + unboundLogicalSink() + .thenApply(this::collectFromUnboundSink) .toRule(RuleType.COLLECT_TABLE_FROM_SINK), any().whenNot(UnboundRelation.class::isInstance) .whenNot(UnboundTableSink.class::isInstance) @@ -124,7 +125,7 @@ private Plan collectFromAny(MatchingContext ctx) { return null; } - private Plan collectFromUnboundTableSink(MatchingContext> ctx) { + private Plan collectFromUnboundSink(MatchingContext> ctx) { List nameParts = ctx.root.getNameParts(); switch (nameParts.size()) { case 1: @@ -182,6 +183,13 @@ private void collectFromUnboundRelation(CascadesContext cascadesContext, if (tableFrom == TableFrom.QUERY) { collectMTMVCandidates(table, cascadesContext); } + if (tableFrom == TableFrom.INSERT_TARGET) { + if (!cascadesContext.getStatementContext().getInsertTargetSchema().isEmpty()) { + LOG.warn("collect insert target table '{}' more than once.", tableQualifier); + } + cascadesContext.getStatementContext().getInsertTargetSchema().clear(); + cascadesContext.getStatementContext().getInsertTargetSchema().addAll(table.getFullSchema()); + } if (table instanceof View) { parseAndCollectFromView(tableQualifier, (View) table, cascadesContext); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java index 76c72f82f90552..39c5909d4f553d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertIntoTableCommand.java @@ -28,7 +28,6 @@ import org.apache.doris.common.UserException; import org.apache.doris.common.profile.ProfileManager.ProfileType; import org.apache.doris.common.util.DebugUtil; -import org.apache.doris.common.util.MetaLockUtils; import org.apache.doris.datasource.hive.HMSExternalTable; import org.apache.doris.datasource.iceberg.IcebergExternalTable; import org.apache.doris.datasource.jdbc.JdbcExternalTable; @@ -73,7 +72,6 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import java.util.Comparator; import java.util.List; import java.util.Objects; import java.util.Optional; @@ -186,9 +184,7 @@ public AbstractInsertExecutor initPlan(ConnectContext ctx, StmtExecutor stmtExec // lock after plan and check does table's schema changed to ensure we lock table order by id. TableIf newestTargetTableIf = RelationUtil.getTable(qualifiedTargetTableName, ctx.getEnv()); - List targetTables = Lists.newArrayList(targetTableIf, newestTargetTableIf); - targetTables.sort(Comparator.comparing(TableIf::getId)); - MetaLockUtils.readLockTables(targetTables); + newestTargetTableIf.readLock(); try { if (targetTableIf.getId() != newestTargetTableIf.getId()) { LOG.warn("insert plan failed {} times. query id is {}. table id changed from {} to {}", @@ -196,10 +192,11 @@ public AbstractInsertExecutor initPlan(ConnectContext ctx, StmtExecutor stmtExec targetTableIf.getId(), newestTargetTableIf.getId()); continue; } - if (!targetTableIf.getFullSchema().equals(newestTargetTableIf.getFullSchema())) { + // Use the schema saved during planning as the schema of the original target table. + if (!ctx.getStatementContext().getInsertTargetSchema().equals(newestTargetTableIf.getFullSchema())) { LOG.warn("insert plan failed {} times. query id is {}. table schema changed from {} to {}", retryTimes, DebugUtil.printId(ctx.queryId()), - targetTableIf.getFullSchema(), newestTargetTableIf.getFullSchema()); + ctx.getStatementContext().getInsertTargetSchema(), newestTargetTableIf.getFullSchema()); continue; } if (!insertExecutor.isEmptyInsert()) { @@ -209,9 +206,9 @@ public AbstractInsertExecutor initPlan(ConnectContext ctx, StmtExecutor stmtExec buildResult.physicalSink ); } - MetaLockUtils.readUnlockTables(targetTables); + newestTargetTableIf.readUnlock(); } catch (Throwable e) { - MetaLockUtils.readUnlockTables(targetTables); + newestTargetTableIf.readUnlock(); // the abortTxn in onFail need to acquire table write lock if (insertExecutor != null) { insertExecutor.onFail(e);