You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by kx...@apache.org on 2023/06/06 15:15:19 UTC

[doris] 17/36: [fix](load) in strict mode, return error for insert if datatype convert fails (#20378)

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 <18...@users.noreply.github.com>
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: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org