This is an automated email from the ASF dual-hosted git repository. kxiao pushed a commit to branch branch-2.0-beta in repository https://gitbox.apache.org/repos/asf/doris.git
commit b31747a37dc53a4157067a1361a5a727db095d52 Author: TengJianPing <[email protected]> AuthorDate: Tue Jun 6 12:04:03 2023 +0800 [fix](load) in strict mode, return error for insert if datatype convert fails (#20378) * [fix](load) in strict mode, return error for load and insert if datatype convert fails Revert "[fix](MySQL) the way Doris handles boolean type is consistent with MySQL (#19416)" This reverts commit 68eb420cabe5b26b09d6d4a2724ae12699bdee87. Since it changed other behaviours, e.g. in strict mode insert into t_int values ("a"), it will result 0 is inserted into table, but it should return error instead. * fix be ut * fix regression tests --- be/src/pipeline/pipeline_task.cpp | 4 +- be/src/runtime/runtime_state.h | 4 + be/src/vec/functions/function_cast.h | 101 ++++++---- .../org/apache/doris/analysis/BinaryPredicate.java | 4 - .../org/apache/doris/analysis/StringLiteral.java | 9 +- .../java/org/apache/doris/qe/SessionVariable.java | 2 + gensrc/thrift/PaloInternalService.thrift | 2 + regression-test/data/insert_p0/insert_invalid.out | 11 ++ .../datetime_functions/test_date_function.out | 6 +- .../datetime_functions/test_date_function.out | 6 +- .../storage/test_dup_tab_date_nullable.groovy | 2 +- .../suites/insert_p0/insert_invalid.groovy | 215 +++++++++++++++++++++ .../datetime_functions/test_date_function.groovy | 6 +- .../datetime_functions/test_date_function.groovy | 6 +- 14 files changed, 314 insertions(+), 64 deletions(-) diff --git a/be/src/pipeline/pipeline_task.cpp b/be/src/pipeline/pipeline_task.cpp index 853cd8ec0d..7c2379796a 100644 --- a/be/src/pipeline/pipeline_task.cpp +++ b/be/src/pipeline/pipeline_task.cpp @@ -219,11 +219,13 @@ Status PipelineTask::execute(bool* eos) { if (_block->rows() != 0 || *eos) { SCOPED_TIMER(_sink_timer); auto status = _sink->sink(_state, block, _data_state); + if (!status.is<ErrorCode::END_OF_FILE>()) { + RETURN_IF_ERROR(status); + } *eos = status.is<ErrorCode::END_OF_FILE>() ? true : *eos; if (*eos) { // just return, the scheduler will do finish work break; } - RETURN_IF_ERROR(status); } } diff --git a/be/src/runtime/runtime_state.h b/be/src/runtime/runtime_state.h index 6394a78a82..53382c925a 100644 --- a/be/src/runtime/runtime_state.h +++ b/be/src/runtime/runtime_state.h @@ -409,6 +409,10 @@ public: : 0; } + bool enable_insert_strict() const { + return _query_options.__isset.enable_insert_strict && _query_options.enable_insert_strict; + } + private: Status create_error_log_file(); diff --git a/be/src/vec/functions/function_cast.h b/be/src/vec/functions/function_cast.h index 90bafadf96..a716a01a75 100644 --- a/be/src/vec/functions/function_cast.h +++ b/be/src/vec/functions/function_cast.h @@ -41,6 +41,7 @@ // IWYU pragma: no_include <opentelemetry/common/threadlocal.h> #include "common/compiler_util.h" // IWYU pragma: keep #include "common/status.h" +#include "runtime/runtime_state.h" #include "udf/udf.h" #include "util/jsonb_document.h" #include "util/jsonb_stream.h" @@ -82,6 +83,7 @@ #include "vec/io/io_helper.h" #include "vec/io/reader_buffer.h" #include "vec/runtime/vdatetime_value.h" +#include "vec/utils/template_helpers.hpp" class DateLUTImpl; @@ -220,8 +222,9 @@ struct ConvertImpl { using ToFieldType = typename ToDataType::FieldType; template <typename Additions = void*> - static Status execute(Block& block, const ColumnNumbers& arguments, size_t result, - size_t /*input_rows_count*/, bool check_overflow [[maybe_unused]] = false, + static Status execute(FunctionContext* context, Block& block, const ColumnNumbers& arguments, + size_t result, size_t /*input_rows_count*/, + bool check_overflow [[maybe_unused]] = false, Additions additions [[maybe_unused]] = Additions()) { const ColumnWithTypeAndName& named_from = block.get_by_position(arguments[0]); @@ -414,8 +417,8 @@ struct ConvertImpl { */ template <typename T, typename Name> struct ConvertImpl<std::enable_if_t<!T::is_parametric, T>, T, Name> { - static Status execute(Block& block, const ColumnNumbers& arguments, size_t result, - size_t /*input_rows_count*/) { + static Status execute(FunctionContext* context, Block& block, const ColumnNumbers& arguments, + size_t result, size_t /*input_rows_count*/) { block.get_by_position(result).column = block.get_by_position(arguments[0]).column; return Status::OK(); } @@ -751,8 +754,9 @@ template <typename ToDataType, typename Name> struct ConvertImpl<DataTypeString, ToDataType, Name> { template <typename Additions = void*> - static Status execute(Block& block, const ColumnNumbers& arguments, size_t result, - size_t /*input_rows_count*/, bool check_overflow [[maybe_unused]] = false, + static Status execute(FunctionContext* context, Block& block, const ColumnNumbers& arguments, + size_t result, size_t /*input_rows_count*/, + bool check_overflow [[maybe_unused]] = false, Additions additions [[maybe_unused]] = Additions()) { return Status::RuntimeError("not support convert from string"); } @@ -1073,18 +1077,18 @@ public: UInt32 scale = extract_to_decimal_scale(scale_column); ret_status = ConvertImpl<LeftDataType, RightDataType, Name>::execute( - block, arguments, result, input_rows_count, + context, block, arguments, result, input_rows_count, context->check_overflow_for_decimal(), scale); } else if constexpr (IsDataTypeDateTimeV2<RightDataType>) { const ColumnWithTypeAndName& scale_column = block.get_by_position(result); auto type = check_and_get_data_type<DataTypeDateTimeV2>(scale_column.type.get()); ret_status = ConvertImpl<LeftDataType, RightDataType, Name>::execute( - block, arguments, result, input_rows_count, + context, block, arguments, result, input_rows_count, context->check_overflow_for_decimal(), type->get_scale()); } else { ret_status = ConvertImpl<LeftDataType, RightDataType, Name>::execute( - block, arguments, result, input_rows_count); + context, block, arguments, result, input_rows_count); } return true; }; @@ -1268,8 +1272,9 @@ struct ConvertThroughParsing { static bool is_all_read(ReadBuffer& in) { return in.eof(); } template <typename Additions = void*> - static Status execute(Block& block, const ColumnNumbers& arguments, size_t result, - size_t input_rows_count, bool check_overflow [[maybe_unused]] = false, + static Status execute(FunctionContext* context, Block& block, const ColumnNumbers& arguments, + size_t result, size_t input_rows_count, + bool check_overflow [[maybe_unused]] = false, Additions additions [[maybe_unused]] = Additions()) { using ColVecTo = std::conditional_t<IsDecimalNumber<ToFieldType>, ColumnDecimal<ToFieldType>, ColumnVector<ToFieldType>>; @@ -1311,34 +1316,54 @@ struct ConvertThroughParsing { offsets = &col_from_string->get_offsets(); } + bool is_load = (context && context->state()->query_type() == TQueryType::type::LOAD); + bool is_strict_insert = (context && context->state()->enable_insert_strict()); size_t current_offset = 0; - for (size_t i = 0; i < size; ++i) { - size_t next_offset = std::is_same_v<FromDataType, DataTypeString> - ? (*offsets)[i] - : (current_offset + fixed_string_size); - size_t string_size = std::is_same_v<FromDataType, DataTypeString> - ? next_offset - current_offset - : fixed_string_size; - - ReadBuffer read_buffer(&(*chars)[current_offset], string_size); + auto status = std::visit( + [&](auto is_load_, auto is_strict_insert_) { + for (size_t i = 0; i < size; ++i) { + size_t next_offset = std::is_same_v<FromDataType, DataTypeString> + ? (*offsets)[i] + : (current_offset + fixed_string_size); + size_t string_size = std::is_same_v<FromDataType, DataTypeString> + ? next_offset - current_offset + : fixed_string_size; + + ReadBuffer read_buffer(&(*chars)[current_offset], string_size); + + bool parsed; + if constexpr (IsDataTypeDecimal<ToDataType>) { + parsed = try_parse_impl<ToDataType>( + vec_to[i], read_buffer, local_time_zone, vec_to.get_scale()); + } else if constexpr (IsDataTypeDateTimeV2<ToDataType>) { + auto type = check_and_get_data_type<DataTypeDateTimeV2>( + block.get_by_position(result).type.get()); + parsed = try_parse_impl<ToDataType>(vec_to[i], read_buffer, + local_time_zone, type->get_scale()); + } else { + parsed = try_parse_impl<ToDataType, void*, FromDataType>( + vec_to[i], read_buffer, local_time_zone); + } + (*vec_null_map_to)[i] = !parsed || !is_all_read(read_buffer); + if constexpr (is_load_ && is_strict_insert_) { + if (string_size != 0 && (*vec_null_map_to)[i]) { + return Status::InternalError( + "Invalid value in strict mode for function {}, source " + "column {}, from " + "type " + "{} to type {}", + Name::name, col_from->get_name(), FromDataType().get_name(), + ToDataType().get_name()); + } + } - bool parsed; - if constexpr (IsDataTypeDecimal<ToDataType>) { - parsed = try_parse_impl<ToDataType>(vec_to[i], read_buffer, local_time_zone, - vec_to.get_scale()); - } else if constexpr (IsDataTypeDateTimeV2<ToDataType>) { - auto type = check_and_get_data_type<DataTypeDateTimeV2>( - block.get_by_position(result).type.get()); - parsed = try_parse_impl<ToDataType>(vec_to[i], read_buffer, local_time_zone, - type->get_scale()); - } else { - parsed = try_parse_impl<ToDataType, void*, FromDataType>(vec_to[i], read_buffer, - local_time_zone); - } - (*vec_null_map_to)[i] = !parsed || !is_all_read(read_buffer); + current_offset = next_offset; + } + return Status::OK(); + }, + make_bool_variant(is_load), make_bool_variant(is_strict_insert)); - current_offset = next_offset; - } + RETURN_IF_ERROR(status); block.get_by_position(result).column = ColumnNullable::create(std::move(col_to), std::move(col_null_map_to)); @@ -1390,7 +1415,7 @@ public: if (check_and_get_data_type<DataTypeString>(from_type)) { return ConvertThroughParsing<DataTypeString, ToDataType, Name>::execute( - block, arguments, result, input_rows_count); + context, block, arguments, result, input_rows_count); } return Status::RuntimeError( @@ -1563,7 +1588,7 @@ private: using RightDataType = typename Types::RightType; ConvertImpl<LeftDataType, RightDataType, NameCast>::execute( - block, arguments, result, input_rows_count, + context, block, arguments, result, input_rows_count, context->check_overflow_for_decimal(), scale); return true; }); diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/BinaryPredicate.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/BinaryPredicate.java index 64f802efaf..8587d801e8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/BinaryPredicate.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/BinaryPredicate.java @@ -426,10 +426,6 @@ public class BinaryPredicate extends Predicate implements Writable { && (t2 == PrimitiveType.BIGINT || t2 == PrimitiveType.LARGEINT)) { return Type.LARGEINT; } - // MySQL will try to parse string as bigint, if failed, will take string as 0. - if (t1 == PrimitiveType.BIGINT && t2.isCharFamily()) { - return Type.BIGINT; - } // Implicit conversion affects query performance. // For a common example datekey='20200825' which datekey is int type. diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/StringLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/StringLiteral.java index 3ce734d8c2..116155e0ca 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/StringLiteral.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/StringLiteral.java @@ -227,14 +227,7 @@ public class StringLiteral extends LiteralExpr { throw new AnalysisException(e.getMessage()); } } - // MySQL will try to parse string as bigint, if failed, will cast string as 0. - long longValue; - try { - longValue = Long.parseLong(value); - } catch (NumberFormatException e) { - longValue = 0L; - } - return new IntLiteral(longValue, targetType); + return new IntLiteral(value, targetType); case LARGEINT: if (VariableVarConverters.hasConverter(beConverted)) { try { 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 fe76449a1e..e1fc2b973f 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 @@ -1954,6 +1954,8 @@ public class SessionVariable implements Serializable, Writable { tResult.setEnableParquetLazyMat(enableParquetLazyMat); tResult.setEnableOrcLazyMat(enableOrcLazyMat); + tResult.setEnableInsertStrict(enableInsertStrict); + return tResult; } diff --git a/gensrc/thrift/PaloInternalService.thrift b/gensrc/thrift/PaloInternalService.thrift index 6a8ae55fd4..ca14a548b0 100644 --- a/gensrc/thrift/PaloInternalService.thrift +++ b/gensrc/thrift/PaloInternalService.thrift @@ -223,6 +223,8 @@ struct TQueryOptions { 73: optional i64 scan_queue_mem_limit 74: optional bool enable_scan_node_run_serial = false; + + 75: optional bool enable_insert_strict = false; } diff --git a/regression-test/data/insert_p0/insert_invalid.out b/regression-test/data/insert_p0/insert_invalid.out new file mode 100644 index 0000000000..7e268e6c7e --- /dev/null +++ b/regression-test/data/insert_p0/insert_invalid.out @@ -0,0 +1,11 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !select_inserted0 -- +12345678908876643 a +1234567890887664643 b + +-- !select_inserted1 -- +12345678908876643 a +1234567890887664643 b + +-- !select_inserted2 -- + diff --git a/regression-test/data/nereids_p0/sql_functions/datetime_functions/test_date_function.out b/regression-test/data/nereids_p0/sql_functions/datetime_functions/test_date_function.out index b2ed2aae64..d74dd598a1 100644 --- a/regression-test/data/nereids_p0/sql_functions/datetime_functions/test_date_function.out +++ b/regression-test/data/nereids_p0/sql_functions/datetime_functions/test_date_function.out @@ -585,21 +585,21 @@ true 2019-08-01T13:21:02.111111 -- !sql -- -\N \N \N \N 2000-02-29 2000-02-29 2000-02-29 2000-02-29 2022-01-31 2022-01-31 2022-01-31 2022-01-31 2022-02-28 2022-02-28 2022-02-28 2022-02-28 +2022-02-28 2022-02-28 2022-02-28 2022-02-28 -- !sql -- -\N \N 2000-02-29 2000-02-29 2022-01-31 2022-01-31 2022-02-28 2022-02-28 +2022-02-28 2022-02-28 -- !sql -- -\N \N \N \N 1970-01-01 1970-01-01 1970-01-01 1970-01-01 2000-01-31 2000-01-31 2000-01-31 2000-01-31 2021-12-27 2021-12-27 2021-12-27 2021-12-27 +2022-02-21 2022-02-21 2022-02-21 2022-02-21 2022-02-28 2022-02-28 2022-02-28 2022-02-28 diff --git a/regression-test/data/query_p0/sql_functions/datetime_functions/test_date_function.out b/regression-test/data/query_p0/sql_functions/datetime_functions/test_date_function.out index 3e532edaec..d179d5360f 100644 --- a/regression-test/data/query_p0/sql_functions/datetime_functions/test_date_function.out +++ b/regression-test/data/query_p0/sql_functions/datetime_functions/test_date_function.out @@ -617,22 +617,22 @@ true -1096 -- !sql -- -\N \N \N \N 2000-02-29 2000-02-29 2000-02-29 2000-02-29 2022-01-31 2022-01-31 2022-01-31 2022-01-31 2022-02-28 2022-02-28 2022-02-28 2022-02-28 +2022-02-28 2022-02-28 2022-02-28 2022-02-28 -- !sql -- -\N \N 2000-02-29 2000-02-29 2022-01-31 2022-01-31 2022-02-28 2022-02-28 +2022-02-28 2022-02-28 -- !sql -- -\N \N \N \N 1970-01-01 1970-01-01 1970-01-01 1970-01-01 2000-01-31 2000-01-31 2000-01-31 2000-01-31 2021-12-27 2021-12-27 2021-12-27 2021-12-27 +2022-02-21 2022-02-21 2022-02-21 2022-02-21 2022-02-28 2022-02-28 2022-02-28 2022-02-28 -- !sql -- diff --git a/regression-test/suites/data_model_p0/duplicate/storage/test_dup_tab_date_nullable.groovy b/regression-test/suites/data_model_p0/duplicate/storage/test_dup_tab_date_nullable.groovy index a4e43d4ce0..0f9942231f 100644 --- a/regression-test/suites/data_model_p0/duplicate/storage/test_dup_tab_date_nullable.groovy +++ b/regression-test/suites/data_model_p0/duplicate/storage/test_dup_tab_date_nullable.groovy @@ -47,7 +47,7 @@ PROPERTIES ( (1, '2021-03-01', '2021-03-02', '2021-03-03', '2021-03-01', '2021-03-02', '2021-03-03'), (1, '2021-02-01', '2021-02-02', '2021-02-03', '2021-02-01', '2021-02-02', '2021-02-03'), (1, '2021-01-01', '2021-01-02', '2021-01-03', '2021-01-01', '2021-01-02', '2021-01-03'), - (null, '2021-05-01', 'null', '2021-04-03', '2021-05-01', 'null', '2021-04-03') + (null, '2021-05-01', null, '2021-04-03', '2021-05-01', null, '2021-04-03') """ qt_sql1 "select date1 from ${table1} order by date1" diff --git a/regression-test/suites/insert_p0/insert_invalid.groovy b/regression-test/suites/insert_p0/insert_invalid.groovy new file mode 100644 index 0000000000..45703d4a50 --- /dev/null +++ b/regression-test/suites/insert_p0/insert_invalid.groovy @@ -0,0 +1,215 @@ +// 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. + +// The cases is copied from https://github.com/trinodb/trino/tree/master +// /testing/trino-product-tests/src/main/resources/sql-tests/testcases +// and modified by Doris. +suite("insert_invalid") { + sql """ DROP TABLE IF EXISTS datatype_invalid; """ + sql """ + CREATE TABLE `datatype_invalid` (`timea` bigint NOT NULL, `creatr` varchar(30) NULL) + UNIQUE KEY(`timea`) + DISTRIBUTED BY HASH(`timea`) BUCKETS 1 + PROPERTIES ("replication_num" = "1"); + """ + + // strict insert + sql """ set enable_insert_strict=true; """ + + // test insert select: out of range value + sql """ DROP TABLE IF EXISTS datatype_invalid_base; """ + sql """ + CREATE TABLE `datatype_invalid_base` ( + `timea` varchar(30) NULL, + `creatr` varchar(30) NULL + ) UNIQUE KEY(`timea`) + DISTRIBUTED BY HASH(`timea`) BUCKETS 1 + PROPERTIES ("replication_num" = "1"); + """ + + sql """ + insert into + datatype_invalid_base + values + ("12345678908876643", "a"), + ("1234567890887664643", "b"), + ("123456789088766445456", "c"); + """ + + test { + sql """ insert into datatype_invalid select * from datatype_invalid_base;""" + exception "Invalid value in strict mode" + } + + // test insert select: invalid value + sql """ DROP TABLE IF EXISTS datatype_invalid_base; """ + sql """ + CREATE TABLE `datatype_invalid_base` ( + `timea` varchar(30) NULL, + `creatr` varchar(30) NULL + ) UNIQUE KEY(`timea`) + DISTRIBUTED BY HASH(`timea`) BUCKETS 1 + PROPERTIES ("replication_num" = "1"); + """ + + sql """ + insert into + datatype_invalid_base + values + ("a", "a"); + """ + test { + sql """ insert into datatype_invalid select * from datatype_invalid_base;""" + exception "Invalid value in strict mode" + } + + // test insert select: invalid value + sql """ DROP TABLE IF EXISTS datatype_invalid_base; """ + sql """ + CREATE TABLE `datatype_invalid_base` ( + `timea` varchar(30) NULL, + `creatr` varchar(30) NULL + ) UNIQUE KEY(`timea`) + DISTRIBUTED BY HASH(`timea`) BUCKETS 1 + PROPERTIES ("replication_num" = "1"); + """ + + sql """ + insert into + datatype_invalid_base + values + (" ", "a"); + """ + test { + sql """ insert into datatype_invalid select * from datatype_invalid_base;""" + exception "Invalid value in strict mode" + } + + // test insert select: null into not nullable + sql """ DROP TABLE IF EXISTS datatype_invalid_base; """ + sql """ + CREATE TABLE `datatype_invalid_base` ( + `timea` varchar(30) NULL, + `creatr` varchar(30) NULL + ) UNIQUE KEY(`timea`) + DISTRIBUTED BY HASH(`timea`) BUCKETS 1 + PROPERTIES ("replication_num" = "1"); + """ + + sql """ + insert into + datatype_invalid_base + values + (null, "a"); + """ + test { + sql """ insert into datatype_invalid select * from datatype_invalid_base;""" + exception "Insert has filtered data in strict mode" + } + + // test insert + test { + sql """ insert into datatype_invalid values("a", "a");""" + exception "Invalid number format" + } + test { + sql """ insert into datatype_invalid values(" ", "a");""" + exception "Invalid number format" + } + test { + sql """ insert into datatype_invalid values(123456789088766445456, "a");""" + exception "Number out of range" + } + test { + sql """ insert into datatype_invalid values(null, "a");""" + exception "Insert has filtered data in strict mode" + } + + sql """ DROP TABLE IF EXISTS datatype_invalid; """ + sql """ + CREATE TABLE `datatype_invalid` (`timea` datetime NOT NULL, `creatr` varchar(30) NULL) + UNIQUE KEY(`timea`) + DISTRIBUTED BY HASH(`timea`) BUCKETS 1 + PROPERTIES ("replication_num" = "1"); + """ + test { + sql """ insert into datatype_invalid values ('2022-02-29', 'a'); """ + exception "Invalid value in strict mode" + } + + sql """ set enable_insert_strict=false; """ + sql """ DROP TABLE IF EXISTS datatype_invalid; """ + sql """ + CREATE TABLE `datatype_invalid` (`timea` bigint NOT NULL, `creatr` varchar(30) NULL) + UNIQUE KEY(`timea`) + DISTRIBUTED BY HASH(`timea`) BUCKETS 1 + PROPERTIES ("replication_num" = "1"); + """ + + // non strict insert into select + sql """ DROP TABLE IF EXISTS datatype_invalid_base; """ + sql """ + CREATE TABLE `datatype_invalid_base` ( + `timea` varchar(30) NULL, + `creatr` varchar(30) NULL + ) UNIQUE KEY(`timea`) + DISTRIBUTED BY HASH(`timea`) BUCKETS 1 + PROPERTIES ("replication_num" = "1"); + """ + sql """ + insert into + datatype_invalid_base + values + ("a", "a"), + (" ", "a"), + ("12345678908876643", "a"), + ("1234567890887664643", "b"), + ("123456789088766445456", "c"); + """ + sql """ insert into datatype_invalid select * from datatype_invalid_base;""" + + qt_select_inserted0 """ select * from datatype_invalid order by timea """ + + sql """ DROP TABLE IF EXISTS datatype_invalid_base; """ + sql """ + CREATE TABLE `datatype_invalid_base` ( + `timea` varchar(30) NULL, + `creatr` varchar(30) NULL + ) UNIQUE KEY(`timea`) + DISTRIBUTED BY HASH(`timea`) BUCKETS 1 + PROPERTIES ("replication_num" = "1"); + """ + sql """ + insert into + datatype_invalid_base + values + (null, "a"); + """ + sql """ insert into datatype_invalid select * from datatype_invalid_base;""" + + qt_select_inserted1 """ select * from datatype_invalid order by timea """ + + sql """ DROP TABLE IF EXISTS datatype_invalid; """ + sql """ + CREATE TABLE `datatype_invalid` (`timea` datetime NOT NULL, `creatr` varchar(30) NULL) + UNIQUE KEY(`timea`) + DISTRIBUTED BY HASH(`timea`) BUCKETS 1 + PROPERTIES ("replication_num" = "1"); + """ + sql """ insert into datatype_invalid values ('2022-02-29', 'a'); """ + qt_select_inserted2 """ select * from datatype_invalid order by timea """ +} diff --git a/regression-test/suites/nereids_p0/sql_functions/datetime_functions/test_date_function.groovy b/regression-test/suites/nereids_p0/sql_functions/datetime_functions/test_date_function.groovy index 787ead79ba..64c0fe2476 100644 --- a/regression-test/suites/nereids_p0/sql_functions/datetime_functions/test_date_function.groovy +++ b/regression-test/suites/nereids_p0/sql_functions/datetime_functions/test_date_function.groovy @@ -566,7 +566,7 @@ suite("test_date_function") { insert into ${tableName} values ('2022-01-01', '2022-01-01', '2022-01-01 00:00:00', '2022-01-01 00:00:00'), ('2000-02-01', '2000-02-01', '2000-02-01 00:00:00', '2000-02-01 00:00:00.123'), - ('2022-02-29', '2022-02-29', '2022-02-29 00:00:00', '2022-02-29 00:00:00'), + ('2022-02-27', '2022-02-27', '2022-02-27 00:00:00', '2022-02-27 00:00:00'), ('2022-02-28', '2022-02-28', '2022-02-28T23:59:59', '2022-02-28T23:59:59');""" qt_sql """ select last_day(birth), last_day(birth1), @@ -588,7 +588,7 @@ suite("test_date_function") { insert into ${tableName} values ('2022-01-01', '2022-01-01 00:00:00'), ('2000-02-01', '2000-02-01 00:00:00'), - ('2022-02-29', '2022-02-29 00:00:00'), + ('2022-02-27', '2022-02-27 00:00:00'), ('2022-02-28', '2022-02-28 23:59:59');""" qt_sql """ select last_day(birth), last_day(birth1) from ${tableName}; @@ -611,7 +611,7 @@ suite("test_date_function") { insert into ${tableName} values ('2022-01-01', '2022-01-01', '2022-01-01 00:00:00', '2022-01-01 00:00:00'), ('2000-02-01', '2000-02-01', '2000-02-01 00:00:00', '2000-02-01 00:00:00.123'), - ('2022-02-29', '2022-02-29', '2022-02-29 00:00:00', '2022-02-29 00:00:00'), + ('2022-02-27', '2022-02-27', '2022-02-27 00:00:00', '2022-02-27 00:00:00'), ('2022-02-28', '2022-02-28', '2022-02-28 23:59:59', '2022-02-28 23:59:59'), ('1970-01-02', '1970-01-02', '1970-01-02 01:02:03', '1970-01-02 02:03:04');""" qt_sql """ diff --git a/regression-test/suites/query_p0/sql_functions/datetime_functions/test_date_function.groovy b/regression-test/suites/query_p0/sql_functions/datetime_functions/test_date_function.groovy index ae8ff4861c..86641d242b 100644 --- a/regression-test/suites/query_p0/sql_functions/datetime_functions/test_date_function.groovy +++ b/regression-test/suites/query_p0/sql_functions/datetime_functions/test_date_function.groovy @@ -590,7 +590,7 @@ suite("test_date_function") { insert into ${tableName} values ('2022-01-01', '2022-01-01', '2022-01-01 00:00:00', '2022-01-01 00:00:00'), ('2000-02-01', '2000-02-01', '2000-02-01 00:00:00', '2000-02-01 00:00:00.123'), - ('2022-02-29', '2022-02-29', '2022-02-29 00:00:00', '2022-02-29 00:00:00'), + ('2022-02-27', '2022-02-27', '2022-02-27 00:00:00', '2022-02-27 00:00:00'), ('2022-02-28', '2022-02-28', '2022-02-28T23:59:59', '2022-02-28T23:59:59');""" qt_sql """ select last_day(birth), last_day(birth1), @@ -612,7 +612,7 @@ suite("test_date_function") { insert into ${tableName} values ('2022-01-01', '2022-01-01 00:00:00'), ('2000-02-01', '2000-02-01 00:00:00'), - ('2022-02-29', '2022-02-29 00:00:00'), + ('2022-02-27', '2022-02-27 00:00:00'), ('2022-02-28', '2022-02-28 23:59:59');""" qt_sql """ select last_day(birth), last_day(birth1) from ${tableName}; @@ -646,7 +646,7 @@ suite("test_date_function") { insert into ${tableName} values ('2022-01-01', '2022-01-01', '2022-01-01 00:00:00', '2022-01-01 00:00:00'), ('2000-02-01', '2000-02-01', '2000-02-01 00:00:00', '2000-02-01 00:00:00.123'), - ('2022-02-29', '2022-02-29', '2022-02-29 00:00:00', '2022-02-29 00:00:00'), + ('2022-02-27', '2022-02-27', '2022-02-27 00:00:00', '2022-02-27 00:00:00'), ('2022-02-28', '2022-02-28', '2022-02-28 23:59:59', '2022-02-28 23:59:59'), ('1970-01-02', '1970-01-02', '1970-01-02 01:02:03', '1970-01-02 02:03:04');""" qt_sql """ --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
