zclllyybb commented on code in PR #57569: URL: https://github.com/apache/doris/pull/57569#discussion_r2483827042
########## be/src/vec/functions/function_interval.cpp: ########## @@ -0,0 +1,144 @@ +// 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 <cstdint> +#include <limits> +#include <memory> +#include <vector> + +#include "vec/columns/column.h" +#include "vec/columns/column_nullable.h" +#include "vec/columns/column_vector.h" +#include "vec/common/assert_cast.h" +#include "vec/core/block.h" +#include "vec/core/column_numbers.h" +#include "vec/core/column_with_type_and_name.h" +#include "vec/core/types.h" +#include "vec/data_types/data_type.h" +#include "vec/data_types/data_type_number.h" +#include "vec/functions/function.h" +#include "vec/functions/simple_function_factory.h" + +namespace doris::vectorized { + +class FunctionInterval : public IFunction { +public: + static constexpr auto name = "interval"; + static FunctionPtr create() { return std::make_shared<FunctionInterval>(); } + + String get_name() const override { return name; } + bool is_variadic() const override { return true; } + size_t get_number_of_arguments() const override { return 0; } + bool use_default_implementation_for_nulls() const override { return true; } Review Comment: default value is already true ########## be/src/vec/functions/function_interval.cpp: ########## @@ -0,0 +1,144 @@ +// 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 <cstdint> +#include <limits> +#include <memory> +#include <vector> + +#include "vec/columns/column.h" +#include "vec/columns/column_nullable.h" +#include "vec/columns/column_vector.h" +#include "vec/common/assert_cast.h" +#include "vec/core/block.h" +#include "vec/core/column_numbers.h" +#include "vec/core/column_with_type_and_name.h" +#include "vec/core/types.h" +#include "vec/data_types/data_type.h" +#include "vec/data_types/data_type_number.h" +#include "vec/functions/function.h" +#include "vec/functions/simple_function_factory.h" + +namespace doris::vectorized { + +class FunctionInterval : public IFunction { +public: + static constexpr auto name = "interval"; + static FunctionPtr create() { return std::make_shared<FunctionInterval>(); } + + String get_name() const override { return name; } + bool is_variadic() const override { return true; } + size_t get_number_of_arguments() const override { return 0; } + bool use_default_implementation_for_nulls() const override { return true; } + + DataTypePtr get_return_type_impl(const DataTypes& /*arguments*/) const override { + return std::make_shared<DataTypeInt32>(); + } + + Status execute_impl(FunctionContext* /*context*/, Block& block, const ColumnNumbers& arguments, + uint32_t result, size_t input_rows_count) const override { + if (arguments.size() < 2) { + return Status::InvalidArgument("interval requires at least 2 arguments"); Review Comment: should make sure it's checked in FE. add `[[unlikely]]` and return `InternalError` ########## be/src/vec/functions/function_interval.cpp: ########## @@ -0,0 +1,144 @@ +// 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 <cstdint> +#include <limits> +#include <memory> +#include <vector> + +#include "vec/columns/column.h" +#include "vec/columns/column_nullable.h" +#include "vec/columns/column_vector.h" +#include "vec/common/assert_cast.h" +#include "vec/core/block.h" +#include "vec/core/column_numbers.h" +#include "vec/core/column_with_type_and_name.h" +#include "vec/core/types.h" +#include "vec/data_types/data_type.h" +#include "vec/data_types/data_type_number.h" +#include "vec/functions/function.h" +#include "vec/functions/simple_function_factory.h" + +namespace doris::vectorized { + +class FunctionInterval : public IFunction { +public: + static constexpr auto name = "interval"; + static FunctionPtr create() { return std::make_shared<FunctionInterval>(); } + + String get_name() const override { return name; } + bool is_variadic() const override { return true; } + size_t get_number_of_arguments() const override { return 0; } + bool use_default_implementation_for_nulls() const override { return true; } + + DataTypePtr get_return_type_impl(const DataTypes& /*arguments*/) const override { + return std::make_shared<DataTypeInt32>(); + } + + Status execute_impl(FunctionContext* /*context*/, Block& block, const ColumnNumbers& arguments, + uint32_t result, size_t input_rows_count) const override { + if (arguments.size() < 2) { + return Status::InvalidArgument("interval requires at least 2 arguments"); + } + + auto res_col = ColumnInt32::create(); + auto& res_data = res_col->get_data(); + res_data.resize(input_rows_count); + + auto compare_cwn = block.get_by_position(arguments[0]); + auto compare_col_ptr = compare_cwn.column; + bool compare_is_const = false; + std::tie(compare_col_ptr, compare_is_const) = unpack_if_const(compare_col_ptr); Review Comment: use `default_preprocess_parameter_columns` here is more proper ########## fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java: ########## @@ -1049,10 +1051,10 @@ public class BuiltinScalarFunctions implements FunctionHelper { scalar(TrimIn.class, "trim_in"), scalar(Truncate.class, "truncate"), scalar(Unhex.class, "unhex"), - scalar(UnhexNull.class, "unhex_null"), + scalar(Uncompress.class, "uncompress"), + scalar(UnhexNull.class, "unhex_null"), Review Comment: format ########## regression-test/suites/function_p0/test_interval_function.groovy: ########## @@ -0,0 +1,165 @@ +// 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_interval_function") { + + qt_interval_basic_1 """ SELECT `INTERVAL`(23, 1, 15, 17, 30, 44, 200); """ + qt_interval_basic_2 """ SELECT `INTERVAL`(10, 1, 10, 100, 1000); """ + qt_interval_basic_3 """ SELECT `INTERVAL`(22, 23, 30, 44, 200); """ + qt_interval_basic_4 """ SELECT `INTERVAL`(33, 1, 10, 32, 32, 102, 200); """ + qt_interval_basic_5 """ SELECT `INTERVAL`(33, 1, 10, 32, 33, 102, 200); """ + + qt_interval_boundary_min """ SELECT `INTERVAL`(0, 1, 10, 100); """ + qt_interval_boundary_first """ SELECT `INTERVAL`(1, 1, 10, 100); """ + qt_interval_boundary_last """ SELECT `INTERVAL`(100, 1, 10, 100); """ + qt_interval_boundary_max """ SELECT `INTERVAL`(200, 1, 10, 100); """ + qt_interval_boundary_between_1 """ SELECT `INTERVAL`(5, 1, 10, 100); """ + qt_interval_boundary_between_2 """ SELECT `INTERVAL`(50, 1, 10, 100); """ + + qt_interval_negative_1 """ SELECT `INTERVAL`(-10, -100, -50, -10, 0, 50, 100); """ + qt_interval_negative_2 """ SELECT `INTERVAL`(-5, -100, -50, -10, 0, 50, 100); """ + qt_interval_negative_3 """ SELECT `INTERVAL`(5, -100, -50, -10, 0, 50, 100); """ + qt_interval_negative_4 """ SELECT `INTERVAL`(0, -100, -50, -10, 0, 50, 100); """ + + qt_interval_duplicate_thresholds_1 """ SELECT `INTERVAL`(10, 1, 10, 10, 20, 20, 30); """ + qt_interval_duplicate_thresholds_2 """ SELECT `INTERVAL`(15, 1, 10, 10, 20, 20, 30); """ + qt_interval_duplicate_thresholds_3 """ SELECT `INTERVAL`(25, 1, 10, 10, 20, 20, 30); """ + + qt_interval_single_threshold_1 """ SELECT `INTERVAL`(0, 10); """ + qt_interval_single_threshold_2 """ SELECT `INTERVAL`(10, 10); """ + qt_interval_single_threshold_3 """ SELECT `INTERVAL`(20, 10); """ + + qt_interval_two_thresholds_1 """ SELECT `INTERVAL`(0, 10, 20); """ + qt_interval_two_thresholds_2 """ SELECT `INTERVAL`(10, 10, 20); """ + qt_interval_two_thresholds_3 """ SELECT `INTERVAL`(15, 10, 20); """ + qt_interval_two_thresholds_4 """ SELECT `INTERVAL`(20, 10, 20); """ + qt_interval_two_thresholds_5 """ SELECT `INTERVAL`(30, 10, 20); """ + + qt_interval_tinyint """ SELECT `INTERVAL`(CAST(5 AS TINYINT), CAST(1 AS TINYINT), CAST(10 AS TINYINT), CAST(20 AS TINYINT)); """ + qt_interval_smallint """ SELECT `INTERVAL`(CAST(15 AS SMALLINT), CAST(1 AS SMALLINT), CAST(10 AS SMALLINT), CAST(20 AS SMALLINT)); """ + qt_interval_int """ SELECT `INTERVAL`(CAST(15 AS INT), CAST(1 AS INT), CAST(10 AS INT), CAST(20 AS INT)); """ + qt_interval_bigint """ SELECT `INTERVAL`(CAST(15 AS BIGINT), CAST(1 AS BIGINT), CAST(10 AS BIGINT), CAST(20 AS BIGINT)); """ + qt_interval_largeint """ SELECT `INTERVAL`(CAST(15 AS LARGEINT), CAST(1 AS LARGEINT), CAST(10 AS LARGEINT), CAST(20 AS LARGEINT)); """ + + qt_interval_null_first_arg """ SELECT `INTERVAL`(NULL, 1, 10, 100); """ + qt_interval_null_threshold """ SELECT `INTERVAL`(50, NULL, 10, 100); """ + + def intervalTestTable = "interval_function_test_table" + + sql """ DROP TABLE IF EXISTS ${intervalTestTable}; """ + + sql """ + CREATE TABLE IF NOT EXISTS ${intervalTestTable} ( + id INT, + val_tinyint TINYINT, + val_smallint SMALLINT, + val_int INT, + val_bigint BIGINT, + val_largeint LARGEINT, + thresh1 INT, + thresh2 INT, + thresh3 INT, + thresh4 INT, + thresh5 INT + ) + DUPLICATE KEY(id) + DISTRIBUTED BY HASH(id) BUCKETS 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ) + """ + + sql """ + INSERT INTO ${intervalTestTable} VALUES + (1, 5, 15, 25, 35, 45, 10, 20, 30, 40, 50), + (2, 15, 25, 35, 45, 55, 10, 20, 30, 40, 50), + (3, 25, 35, 45, 55, 65, 10, 20, 30, 40, 50), + (4, 0, 5, 5, 5, 5, 10, 20, 30, 40, 50), + (5, 60, 60, 60, 60, 60, 10, 20, 30, 40, 50), + (6, 10, 20, 30, 40, 50, 10, 20, 30, 40, 50), + (7, -10, -5, 0, 5, 10, -20, -10, 0, 10, 20), + (8, NULL, NULL, NULL, NULL, NULL, 10, 20, 30, 40, 50) + """ + + qt_interval_from_table_int """ + SELECT id, `INTERVAL`(val_int, thresh1, thresh2, thresh3, thresh4, thresh5) as result + FROM ${intervalTestTable} + WHERE id <= 7 + ORDER BY id; + """ + + qt_interval_from_table_tinyint """ + SELECT id, `INTERVAL`(val_tinyint, thresh1, thresh2, thresh3, thresh4, thresh5) as result + FROM ${intervalTestTable} + WHERE id <= 7 + ORDER BY id; + """ + + qt_interval_from_table_smallint """ + SELECT id, `INTERVAL`(val_smallint, thresh1, thresh2, thresh3, thresh4, thresh5) as result + FROM ${intervalTestTable} + WHERE id <= 7 + ORDER BY id; + """ + + qt_interval_from_table_bigint """ + SELECT id, `INTERVAL`(val_bigint, thresh1, thresh2, thresh3, thresh4, thresh5) as result + FROM ${intervalTestTable} + WHERE id <= 7 + ORDER BY id; + """ + + qt_interval_from_table_largeint """ + SELECT id, `INTERVAL`(val_largeint, thresh1, thresh2, thresh3, thresh4, thresh5) as result + FROM ${intervalTestTable} + WHERE id <= 7 + ORDER BY id; + """ + + qt_interval_with_const_thresholds """ + SELECT id, `INTERVAL`(val_int, 10, 20, 30, 40, 50) as result + FROM ${intervalTestTable} + WHERE id <= 7 + ORDER BY id; + """ + + qt_interval_with_const_value """ + SELECT id, `INTERVAL`(25, thresh1, thresh2, thresh3, thresh4, thresh5) as result + FROM ${intervalTestTable} + WHERE id <= 7 + ORDER BY id; + """ + Review Comment: add test which some of thresholds are const ########## be/src/vec/functions/function_interval.cpp: ########## @@ -0,0 +1,144 @@ +// 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 <cstdint> +#include <limits> +#include <memory> +#include <vector> + +#include "vec/columns/column.h" +#include "vec/columns/column_nullable.h" +#include "vec/columns/column_vector.h" +#include "vec/common/assert_cast.h" +#include "vec/core/block.h" +#include "vec/core/column_numbers.h" +#include "vec/core/column_with_type_and_name.h" +#include "vec/core/types.h" +#include "vec/data_types/data_type.h" +#include "vec/data_types/data_type_number.h" +#include "vec/functions/function.h" +#include "vec/functions/simple_function_factory.h" + +namespace doris::vectorized { + +class FunctionInterval : public IFunction { +public: + static constexpr auto name = "interval"; + static FunctionPtr create() { return std::make_shared<FunctionInterval>(); } + + String get_name() const override { return name; } + bool is_variadic() const override { return true; } + size_t get_number_of_arguments() const override { return 0; } + bool use_default_implementation_for_nulls() const override { return true; } + + DataTypePtr get_return_type_impl(const DataTypes& /*arguments*/) const override { + return std::make_shared<DataTypeInt32>(); + } + + Status execute_impl(FunctionContext* /*context*/, Block& block, const ColumnNumbers& arguments, + uint32_t result, size_t input_rows_count) const override { + if (arguments.size() < 2) { + return Status::InvalidArgument("interval requires at least 2 arguments"); + } + + auto res_col = ColumnInt32::create(); + auto& res_data = res_col->get_data(); + res_data.resize(input_rows_count); + + auto compare_cwn = block.get_by_position(arguments[0]); + auto compare_col_ptr = compare_cwn.column; + bool compare_is_const = false; + std::tie(compare_col_ptr, compare_is_const) = unpack_if_const(compare_col_ptr); + + switch (compare_cwn.type->get_primitive_type()) { + case PrimitiveType::TYPE_TINYINT: + compute_interval<ColumnInt8>(block, arguments, *compare_col_ptr, compare_is_const, + res_data); + break; + case PrimitiveType::TYPE_SMALLINT: + compute_interval<ColumnInt16>(block, arguments, *compare_col_ptr, compare_is_const, + res_data); + break; + case PrimitiveType::TYPE_INT: + compute_interval<ColumnInt32>(block, arguments, *compare_col_ptr, compare_is_const, + res_data); + break; + case PrimitiveType::TYPE_BIGINT: + compute_interval<ColumnInt64>(block, arguments, *compare_col_ptr, compare_is_const, + res_data); + break; + case PrimitiveType::TYPE_LARGEINT: + compute_interval<ColumnInt128>(block, arguments, *compare_col_ptr, compare_is_const, + res_data); + break; + default: + return Status::InvalidArgument( + "interval only supports integer numeric types for the first argument"); + } + + block.replace_by_position(result, std::move(res_col)); + return Status::OK(); + } + +private: + template <typename ColType> + static void compute_interval(Block& block, const ColumnNumbers& arguments, + const IColumn& compare_col, bool compare_is_const, + PaddedPODArray<Int32>& res) { + const auto& compare_data = assert_cast<const ColType&>(compare_col).get_data(); + const size_t rows = res.size(); + const size_t num_thresholds = arguments.size() - 1; + + for (size_t row = 0; row < rows; ++row) { + auto compare_val = compare_data[index_check_const(row, compare_is_const)]; + + std::vector<typename ColType::value_type> thresholds; + thresholds.reserve(num_thresholds); + + for (size_t i = 1; i < arguments.size(); ++i) { + const auto& col_cwn = block.get_by_position(arguments[i]); + ColumnPtr col_ptr = col_cwn.column; + bool is_const = false; + std::tie(col_ptr, is_const) = unpack_if_const(col_ptr); + const auto& th_col = assert_cast<const ColType&>(*col_ptr); + thresholds.push_back(th_col.get_data()[index_check_const(row, is_const)]); + } + + size_t left = 0; + size_t right = num_thresholds; + size_t result_idx = num_thresholds; + + while (left < right) { Review Comment: why not use `std::upper_bound`? ########## be/src/vec/functions/function_interval.cpp: ########## @@ -0,0 +1,144 @@ +// 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 <cstdint> +#include <limits> +#include <memory> +#include <vector> + +#include "vec/columns/column.h" +#include "vec/columns/column_nullable.h" +#include "vec/columns/column_vector.h" +#include "vec/common/assert_cast.h" +#include "vec/core/block.h" +#include "vec/core/column_numbers.h" +#include "vec/core/column_with_type_and_name.h" +#include "vec/core/types.h" +#include "vec/data_types/data_type.h" +#include "vec/data_types/data_type_number.h" +#include "vec/functions/function.h" +#include "vec/functions/simple_function_factory.h" + +namespace doris::vectorized { + +class FunctionInterval : public IFunction { +public: + static constexpr auto name = "interval"; + static FunctionPtr create() { return std::make_shared<FunctionInterval>(); } + + String get_name() const override { return name; } + bool is_variadic() const override { return true; } + size_t get_number_of_arguments() const override { return 0; } + bool use_default_implementation_for_nulls() const override { return true; } + + DataTypePtr get_return_type_impl(const DataTypes& /*arguments*/) const override { + return std::make_shared<DataTypeInt32>(); + } + + Status execute_impl(FunctionContext* /*context*/, Block& block, const ColumnNumbers& arguments, + uint32_t result, size_t input_rows_count) const override { + if (arguments.size() < 2) { + return Status::InvalidArgument("interval requires at least 2 arguments"); + } + + auto res_col = ColumnInt32::create(); + auto& res_data = res_col->get_data(); + res_data.resize(input_rows_count); + + auto compare_cwn = block.get_by_position(arguments[0]); + auto compare_col_ptr = compare_cwn.column; + bool compare_is_const = false; + std::tie(compare_col_ptr, compare_is_const) = unpack_if_const(compare_col_ptr); + + switch (compare_cwn.type->get_primitive_type()) { + case PrimitiveType::TYPE_TINYINT: + compute_interval<ColumnInt8>(block, arguments, *compare_col_ptr, compare_is_const, + res_data); + break; + case PrimitiveType::TYPE_SMALLINT: + compute_interval<ColumnInt16>(block, arguments, *compare_col_ptr, compare_is_const, + res_data); + break; + case PrimitiveType::TYPE_INT: + compute_interval<ColumnInt32>(block, arguments, *compare_col_ptr, compare_is_const, + res_data); + break; + case PrimitiveType::TYPE_BIGINT: + compute_interval<ColumnInt64>(block, arguments, *compare_col_ptr, compare_is_const, + res_data); + break; + case PrimitiveType::TYPE_LARGEINT: + compute_interval<ColumnInt128>(block, arguments, *compare_col_ptr, compare_is_const, + res_data); + break; + default: + return Status::InvalidArgument( Review Comment: ditto -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
