This is an automated email from the ASF dual-hosted git repository.

panxiaolei pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/doris.git


The following commit(s) were added to refs/heads/master by this push:
     new c98b80ae6ae [Feature](functions) support ignore and nullable functions 
(#27848)
c98b80ae6ae is described below

commit c98b80ae6ae5fb9cdad084fcfb60a3d8bfec5d91
Author: zclllyybb <zhaochan...@selectdb.com>
AuthorDate: Tue Dec 5 14:09:32 2023 +0800

    [Feature](functions) support ignore and nullable functions (#27848)
    
    support ignore and nullable functions
---
 be/src/vec/columns/columns_number.h                |  2 +
 be/src/vec/functions/function_ignore.cpp           | 50 +++++++++++++
 ...tion_nonnullable.cpp => function_nullables.cpp} | 66 ++++++++++++-----
 be/src/vec/functions/simple_function_factory.h     |  6 +-
 .../doris/catalog/BuiltinScalarFunctions.java      |  4 ++
 .../trees/expressions/functions/scalar/Ignore.java | 69 ++++++++++++++++++
 .../expressions/functions/scalar/Nullable.java     | 42 +++++++++++
 .../expressions/visitor/ScalarFunctionVisitor.java |  5 ++
 gensrc/script/doris_builtins_functions.py          | 82 ++++++++++++++++++++++
 .../cast_function/test_non_nullable_function.out   | 43 ------------
 .../cast_function/test_nullable_functions.out      | 43 ++++++++++++
 ...ction.groovy => test_nullable_functions.groovy} | 48 ++++++++-----
 12 files changed, 378 insertions(+), 82 deletions(-)

diff --git a/be/src/vec/columns/columns_number.h 
b/be/src/vec/columns/columns_number.h
index 78311fe218a..926db2d47cf 100644
--- a/be/src/vec/columns/columns_number.h
+++ b/be/src/vec/columns/columns_number.h
@@ -40,6 +40,8 @@ using ColumnInt32 = ColumnVector<Int32>;
 using ColumnInt64 = ColumnVector<Int64>;
 using ColumnInt128 = ColumnVector<Int128>;
 
+using ColumnBool = ColumnUInt8;
+
 using ColumnDate = ColumnVector<Int64>;
 using ColumnDateTime = ColumnVector<Int64>;
 
diff --git a/be/src/vec/functions/function_ignore.cpp 
b/be/src/vec/functions/function_ignore.cpp
new file mode 100644
index 00000000000..1769dd94fc7
--- /dev/null
+++ b/be/src/vec/functions/function_ignore.cpp
@@ -0,0 +1,50 @@
+// 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 "vec/columns/columns_number.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 FunctionIgnore : public IFunction {
+public:
+    static constexpr auto name = "ignore";
+    String get_name() const override { return name; }
+
+    static FunctionPtr create() { return std::make_shared<FunctionIgnore>(); }
+    DataTypePtr get_return_type_impl(const DataTypes& arguments) const 
override {
+        return std::make_shared<DataTypeUInt8>();
+    }
+
+    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 false; 
}
+
+    Status execute_impl(FunctionContext* context, Block& block, const 
ColumnNumbers& arguments,
+                        size_t result, size_t input_rows_count) const override 
{
+        ColumnPtr col = ColumnBool::create(1, false);
+        block.replace_by_position(result, ColumnConst::create(col, 
input_rows_count));
+        return Status::OK();
+    }
+};
+
+void register_function_ignore(SimpleFunctionFactory& factory) {
+    factory.register_function<FunctionIgnore>();
+}
+} // namespace doris::vectorized
diff --git a/be/src/vec/functions/function_nonnullable.cpp 
b/be/src/vec/functions/function_nullables.cpp
similarity index 52%
rename from be/src/vec/functions/function_nonnullable.cpp
rename to be/src/vec/functions/function_nullables.cpp
index fa7b5623b76..0fdcfbc2175 100644
--- a/be/src/vec/functions/function_nonnullable.cpp
+++ b/be/src/vec/functions/function_nullables.cpp
@@ -14,18 +14,11 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
-// This file is copied from
-// https://github.com/ClickHouse/ClickHouse/blob/master/src/Functions/Ifnull.h
-// and modified by Doris
-
-#include <stddef.h>
-
-#include <memory>
 
 #include "common/status.h"
-#include "vec/aggregate_functions/aggregate_function.h"
 #include "vec/columns/column.h"
 #include "vec/columns/column_nullable.h"
+#include "vec/columns/columns_number.h"
 #include "vec/core/block.h"
 #include "vec/core/column_numbers.h"
 #include "vec/core/column_with_type_and_name.h"
@@ -40,6 +33,38 @@ class FunctionContext;
 } // namespace doris
 
 namespace doris::vectorized {
+
+class FunctionNullable : public IFunction {
+public:
+    static constexpr auto name = "nullable";
+
+    static FunctionPtr create() { return std::make_shared<FunctionNullable>(); 
}
+
+    String get_name() const override { return name; }
+
+    size_t get_number_of_arguments() const override { return 1; }
+
+    DataTypePtr get_return_type_impl(const DataTypes& arguments) const 
override {
+        return make_nullable(arguments[0]);
+    }
+
+    bool use_default_implementation_for_nulls() const override { return false; 
}
+
+    // trans nullable column to non-nullable column. If argument is already 
non-nullable, raise error.
+    Status execute_impl(FunctionContext* context, Block& block, const 
ColumnNumbers& arguments,
+                        size_t result, size_t input_rows_count) const override 
{
+        ColumnPtr& col = block.get_by_position(arguments[0]).column;
+        if (const auto* col_null = check_and_get_column<ColumnNullable>(col); 
col_null == nullptr) {
+            // not null
+            block.replace_by_position(
+                    result, ColumnNullable::create(col, 
ColumnBool::create(input_rows_count, 0)));
+        } else { // column is ColumnNullable
+            block.replace_by_position(result, 
col->clone_resized(input_rows_count));
+        }
+        return Status::OK();
+    }
+};
+
 class FunctionNonNullable : public IFunction {
 public:
     static constexpr auto name = "non_nullable";
@@ -60,23 +85,28 @@ public:
     Status execute_impl(FunctionContext* context, Block& block, const 
ColumnNumbers& arguments,
                         size_t result, size_t input_rows_count) const override 
{
         auto& data = block.get_by_position(arguments[0]);
-        const ColumnNullable* column = 
check_and_get_column<ColumnNullable>(data.column);
-
-        if (column == nullptr) // raise error if input is not nullable.
+        if (const auto* col_null = 
check_and_get_column<ColumnNullable>(data.column);
+            col_null == nullptr) // raise error if input is not nullable.
         {
-            return Status::RuntimeError(
-                    "Try to use originally non-nullable column {} in 
nullable's non-nullable \
-                    convertion.",
-                    data.column->get_name(), get_name());
+            return Status::InvalidArgument(
+                    "Try to use originally non-nullable column {} in 
nullable's non-nullable "
+                    "convertion.",
+                    data.column->get_name());
         } else { // column is ColumnNullable
-            const ColumnPtr& type_ptr = column->get_nested_column_ptr();
-            block.replace_by_position(result, 
type_ptr->clone_resized(type_ptr->size()));
+            if (col_null->has_null()) [[unlikely]] {
+                return Status::InvalidArgument(
+                        "There's NULL value in column {} which is illegal for 
non_nullable",
+                        data.column->get_name());
+            }
+            const ColumnPtr& nest_col = col_null->get_nested_column_ptr();
+            block.replace_by_position(result, 
nest_col->clone_resized(nest_col->size()));
         }
         return Status::OK();
     }
 };
 
-void register_function_non_nullable(SimpleFunctionFactory& factory) {
+void register_function_nullables(SimpleFunctionFactory& factory) {
+    factory.register_function<FunctionNullable>();
     factory.register_function<FunctionNonNullable>();
 }
 
diff --git a/be/src/vec/functions/simple_function_factory.h 
b/be/src/vec/functions/simple_function_factory.h
index 8d64e9da34b..66cd5c67bac 100644
--- a/be/src/vec/functions/simple_function_factory.h
+++ b/be/src/vec/functions/simple_function_factory.h
@@ -55,7 +55,7 @@ void register_function_bitmap_variadic(SimpleFunctionFactory& 
factory);
 void register_function_quantile_state(SimpleFunctionFactory& factory);
 void register_function_is_null(SimpleFunctionFactory& factory);
 void register_function_is_not_null(SimpleFunctionFactory& factory);
-void register_function_non_nullable(SimpleFunctionFactory& factory);
+void register_function_nullables(SimpleFunctionFactory& factory);
 void register_function_to_time_function(SimpleFunctionFactory& factory);
 void register_function_time_of_function(SimpleFunctionFactory& factory);
 void register_function_string(SimpleFunctionFactory& factory);
@@ -92,6 +92,7 @@ void register_function_geo(SimpleFunctionFactory& factory);
 void register_function_multi_string_position(SimpleFunctionFactory& factory);
 void register_function_multi_string_search(SimpleFunctionFactory& factory);
 void register_function_width_bucket(SimpleFunctionFactory& factory);
+void register_function_ignore(SimpleFunctionFactory& factory);
 
 void register_function_encryption(SimpleFunctionFactory& factory);
 void register_function_regexp_extract(SimpleFunctionFactory& factory);
@@ -230,7 +231,7 @@ public:
             register_function_bit(instance);
             register_function_is_null(instance);
             register_function_is_not_null(instance);
-            register_function_non_nullable(instance);
+            register_function_nullables(instance);
             register_function_to_time_function(instance);
             register_function_time_of_function(instance);
             register_function_string(instance);
@@ -274,6 +275,7 @@ public:
             register_function_match(instance);
             register_function_ip(instance);
             register_function_tokenize(instance);
+            register_function_ignore(instance);
         });
         return instance;
     }
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java 
b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java
index ee646e58ed3..726cef054eb 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java
@@ -186,6 +186,7 @@ import 
org.apache.doris.nereids.trees.expressions.functions.scalar.HoursAdd;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.HoursDiff;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.HoursSub;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.If;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.Ignore;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Initcap;
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.InnerProduct;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Instr;
@@ -284,6 +285,7 @@ import 
org.apache.doris.nereids.trees.expressions.functions.scalar.NotNullOrEmpt
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Now;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.NullIf;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.NullOrEmpty;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.Nullable;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Nvl;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.ParseUrl;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Password;
@@ -582,6 +584,7 @@ public class BuiltinScalarFunctions implements 
FunctionHelper {
             scalar(HoursDiff.class, "hours_diff"),
             scalar(HoursSub.class, "hours_sub"),
             scalar(If.class, "if"),
+            scalar(Ignore.class, "ignore"),
             scalar(Initcap.class, "initcap"),
             scalar(InnerProduct.class, "inner_product"),
             scalar(Instr.class, "instr"),
@@ -698,6 +701,7 @@ public class BuiltinScalarFunctions implements 
FunctionHelper {
             scalar(NonNullable.class, "non_nullable"),
             scalar(NotNullOrEmpty.class, "not_null_or_empty"),
             scalar(Now.class, "now", "current_timestamp", "localtime", 
"localtimestamp"),
+            scalar(Nullable.class, "nullable"),
             scalar(NullIf.class, "nullif"),
             scalar(NullOrEmpty.class, "null_or_empty"),
             scalar(Nvl.class, "ifnull", "nvl"),
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Ignore.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Ignore.java
new file mode 100644
index 00000000000..029e9fea3e3
--- /dev/null
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Ignore.java
@@ -0,0 +1,69 @@
+// 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.trees.expressions.functions.scalar;
+
+import org.apache.doris.catalog.FunctionSignature;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable;
+import org.apache.doris.nereids.trees.expressions.functions.CustomSignature;
+import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
+import org.apache.doris.nereids.types.BooleanType;
+import org.apache.doris.nereids.util.ExpressionUtils;
+
+import com.google.common.base.Preconditions;
+
+import java.util.List;
+
+/**
+ * accept any arguments, return false.
+ */
+public class Ignore extends ScalarFunction implements CustomSignature, 
AlwaysNotNullable {
+    /**
+     * constructor with 0 argument.
+     */
+    public Ignore() {
+        super("ignore");
+    }
+
+    /**
+     * constructor with 1 or more arguments.
+     */
+    public Ignore(Expression arg, Expression... varArgs) {
+        super("ignore", ExpressionUtils.mergeArguments(arg, varArgs));
+    }
+
+    @Override
+    public FunctionSignature customSignature() {
+        return FunctionSignature.of(BooleanType.INSTANCE, true, 
getArgumentsTypes());
+    }
+
+    /**
+     * withChildren.
+     */
+    @Override
+    public Ignore withChildren(List<Expression> children) {
+        Preconditions.checkArgument(children.size() >= 1);
+        return new Ignore(children.get(0),
+                children.subList(1, children.size()).toArray(new 
Expression[0]));
+    }
+
+    @Override
+    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+        return visitor.visitIgnore(this, context);
+    }
+}
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Nullable.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Nullable.java
new file mode 100644
index 00000000000..49db085a5f9
--- /dev/null
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/Nullable.java
@@ -0,0 +1,42 @@
+// 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.trees.expressions.functions.scalar;
+
+import org.apache.doris.catalog.FunctionSignature;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.trees.expressions.functions.AlwaysNullable;
+import org.apache.doris.nereids.trees.expressions.functions.CustomSignature;
+import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression;
+import org.apache.doris.nereids.types.DataType;
+
+/**
+ * change non_nullable input col to nullable col
+ */
+public class Nullable extends ScalarFunction implements UnaryExpression, 
CustomSignature, AlwaysNullable {
+
+    public Nullable(Expression expr) {
+        super("nullable", expr);
+    }
+
+    @Override
+    public FunctionSignature customSignature() {
+        DataType dataType = getArgument(0).getDataType();
+        return FunctionSignature.ret(dataType).args(dataType);
+    }
+
+}
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java
index aab82b48bcb..ae147b633bd 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java
@@ -182,6 +182,7 @@ import 
org.apache.doris.nereids.trees.expressions.functions.scalar.HoursAdd;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.HoursDiff;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.HoursSub;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.If;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.Ignore;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Initcap;
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.InnerProduct;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Instr;
@@ -1084,6 +1085,10 @@ public interface ScalarFunctionVisitor<R, C> {
         return visitScalarFunction(function, context);
     }
 
+    default R visitIgnore(Ignore function, C context) {
+        return visitScalarFunction(function, context);
+    }
+
     default R visitInitcap(Initcap initcap, C context) {
         return visitScalarFunction(initcap, context);
     }
diff --git a/gensrc/script/doris_builtins_functions.py 
b/gensrc/script/doris_builtins_functions.py
index ea72795d085..0bcb5e080ab 100644
--- a/gensrc/script/doris_builtins_functions.py
+++ b/gensrc/script/doris_builtins_functions.py
@@ -2054,6 +2054,88 @@ visible_functions = {
         [['non_nullable'], 'ARRAY_DECIMAL128', ['ARRAY_DECIMAL128'], 
'ALWAYS_NOT_NULLABLE'],
         [['non_nullable'], 'ARRAY_VARCHAR', ['ARRAY_VARCHAR'], 
'ALWAYS_NOT_NULLABLE'],
         [['non_nullable'], 'ARRAY_STRING', ['ARRAY_STRING'], 
'ALWAYS_NOT_NULLABLE']
+    ],
+
+    "Nullalbe": [
+        [['nullable'], 'BOOLEAN', ['BOOLEAN'], 'ALWAYS_NULLABLE'],
+        [['nullable'], 'TINYINT', ['TINYINT'], 'ALWAYS_NULLABLE'],
+        [['nullable'], 'SMALLINT', ['SMALLINT'], 'ALWAYS_NULLABLE'],
+        [['nullable'], 'INT', ['INT'], 'ALWAYS_NULLABLE'],
+        [['nullable'], 'BIGINT', ['BIGINT'], 'ALWAYS_NULLABLE'],
+        [['nullable'], 'LARGEINT', ['LARGEINT'], 'ALWAYS_NULLABLE'],
+        [['nullable'], 'FLOAT', ['FLOAT'], 'ALWAYS_NULLABLE'],
+        [['nullable'], 'DOUBLE', ['DOUBLE'], 'ALWAYS_NULLABLE'],
+        [['nullable'], 'DATE', ['DATE'], 'ALWAYS_NULLABLE'],
+        [['nullable'], 'DATEV2', ['DATEV2'], 'ALWAYS_NULLABLE'],
+        [['nullable'], 'DATETIME', ['DATETIME'], 'ALWAYS_NULLABLE'],
+        [['nullable'], 'DATETIMEV2', ['DATETIMEV2'], 'ALWAYS_NULLABLE'],
+        [['nullable'], 'DECIMALV2', ['DECIMALV2'], 'ALWAYS_NULLABLE'],
+        [['nullable'], 'DECIMAL32', ['DECIMAL32'], 'ALWAYS_NULLABLE'],
+        [['nullable'], 'DECIMAL64', ['DECIMAL64'], 'ALWAYS_NULLABLE'],
+        [['nullable'], 'DECIMAL128', ['DECIMAL128'], 'ALWAYS_NULLABLE'],
+        [['nullable'], 'VARCHAR', ['VARCHAR'], 'ALWAYS_NULLABLE'],
+        [['nullable'], 'STRING', ['STRING'], 'ALWAYS_NULLABLE'],
+        [['nullable'], 'BITMAP', ['BITMAP'], 'ALWAYS_NULLABLE'],
+        [['nullable'], 'JSONB', ['JSONB'], 'ALWAYS_NULLABLE'],
+        [['nullable'], 'ARRAY_BOOLEAN', ['ARRAY_BOOLEAN'], 'ALWAYS_NULLABLE'],
+        [['nullable'], 'ARRAY_TINYINT', ['ARRAY_TINYINT'], 'ALWAYS_NULLABLE'],
+        [['nullable'], 'ARRAY_SMALLINT', ['ARRAY_SMALLINT'], 
'ALWAYS_NULLABLE'],
+        [['nullable'], 'ARRAY_INT', ['ARRAY_INT'], 'ALWAYS_NULLABLE'],
+        [['nullable'], 'ARRAY_BIGINT', ['ARRAY_BIGINT'], 'ALWAYS_NULLABLE'],
+        [['nullable'], 'ARRAY_LARGEINT', ['ARRAY_LARGEINT'], 
'ALWAYS_NULLABLE'],
+        [['nullable'], 'ARRAY_DATETIME', ['ARRAY_DATETIME'], 
'ALWAYS_NULLABLE'],
+        [['nullable'], 'ARRAY_DATE', ['ARRAY_DATE'], 'ALWAYS_NULLABLE'],
+        [['nullable'], 'ARRAY_DATETIMEV2', ['ARRAY_DATETIMEV2'], 
'ALWAYS_NULLABLE'],
+        [['nullable'], 'ARRAY_DATEV2', ['ARRAY_DATEV2'], 'ALWAYS_NULLABLE'],
+        [['nullable'], 'ARRAY_FLOAT', ['ARRAY_FLOAT'], 'ALWAYS_NULLABLE'],
+        [['nullable'], 'ARRAY_DOUBLE', ['ARRAY_DOUBLE'], 'ALWAYS_NULLABLE'],
+        [['nullable'], 'ARRAY_DECIMALV2', ['ARRAY_DECIMALV2'], 
'ALWAYS_NULLABLE'],
+        [['nullable'], 'ARRAY_DECIMAL32', ['ARRAY_DECIMAL32'], 
'ALWAYS_NULLABLE'],
+        [['nullable'], 'ARRAY_DECIMAL64', ['ARRAY_DECIMAL64'], 
'ALWAYS_NULLABLE'],
+        [['nullable'], 'ARRAY_DECIMAL128', ['ARRAY_DECIMAL128'], 
'ALWAYS_NULLABLE'],
+        [['nullable'], 'ARRAY_VARCHAR', ['ARRAY_VARCHAR'], 'ALWAYS_NULLABLE'],
+        [['nullable'], 'ARRAY_STRING', ['ARRAY_STRING'], 'ALWAYS_NULLABLE']
+    ],
+
+    "Ignore": [
+        [['ignore'], 'BOOLEAN', ['BOOLEAN', '...'], 'ALWAYS_NOT_NULLABLE'],
+        [['ignore'], 'BOOLEAN', ['TINYINT', '...'], 'ALWAYS_NOT_NULLABLE'],
+        [['ignore'], 'BOOLEAN', ['SMALLINT', '...'], 'ALWAYS_NOT_NULLABLE'],
+        [['ignore'], 'BOOLEAN', ['INT', '...'], 'ALWAYS_NOT_NULLABLE'],
+        [['ignore'], 'BOOLEAN', ['BIGINT', '...'], 'ALWAYS_NOT_NULLABLE'],
+        [['ignore'], 'BOOLEAN', ['LARGEINT', '...'], 'ALWAYS_NOT_NULLABLE'],
+        [['ignore'], 'BOOLEAN', ['FLOAT', '...'], 'ALWAYS_NOT_NULLABLE'],
+        [['ignore'], 'BOOLEAN', ['DOUBLE', '...'], 'ALWAYS_NOT_NULLABLE'],
+        [['ignore'], 'BOOLEAN', ['DATE', '...'], 'ALWAYS_NOT_NULLABLE'],
+        [['ignore'], 'BOOLEAN', ['DATEV2', '...'], 'ALWAYS_NOT_NULLABLE'],
+        [['ignore'], 'BOOLEAN', ['DATETIME', '...'], 'ALWAYS_NOT_NULLABLE'],
+        [['ignore'], 'BOOLEAN', ['DATETIMEV2', '...'], 'ALWAYS_NOT_NULLABLE'],
+        [['ignore'], 'BOOLEAN', ['DECIMALV2', '...'], 'ALWAYS_NOT_NULLABLE'],
+        [['ignore'], 'BOOLEAN', ['DECIMAL32', '...'], 'ALWAYS_NOT_NULLABLE'],
+        [['ignore'], 'BOOLEAN', ['DECIMAL64', '...'], 'ALWAYS_NOT_NULLABLE'],
+        [['ignore'], 'BOOLEAN', ['DECIMAL128', '...'], 'ALWAYS_NOT_NULLABLE'],
+        [['ignore'], 'BOOLEAN', ['VARCHAR', '...'], 'ALWAYS_NOT_NULLABLE'],
+        [['ignore'], 'BOOLEAN', ['STRING', '...'], 'ALWAYS_NOT_NULLABLE'],
+        [['ignore'], 'BOOLEAN', ['BITMAP', '...'], 'ALWAYS_NOT_NULLABLE'],
+        [['ignore'], 'BOOLEAN', ['JSONB', '...'], 'ALWAYS_NOT_NULLABLE'],
+        [['ignore'], 'BOOLEAN', ['ARRAY_BOOLEAN', '...'], 
'ALWAYS_NOT_NULLABLE'],
+        [['ignore'], 'BOOLEAN', ['ARRAY_TINYINT', '...'], 
'ALWAYS_NOT_NULLABLE'],
+        [['ignore'], 'BOOLEAN', ['ARRAY_SMALLINT', '...'], 
'ALWAYS_NOT_NULLABLE'],
+        [['ignore'], 'BOOLEAN', ['ARRAY_INT', '...'], 'ALWAYS_NOT_NULLABLE'],
+        [['ignore'], 'BOOLEAN', ['ARRAY_BIGINT', '...'], 
'ALWAYS_NOT_NULLABLE'],
+        [['ignore'], 'BOOLEAN', ['ARRAY_LARGEINT', '...'], 
'ALWAYS_NOT_NULLABLE'],
+        [['ignore'], 'BOOLEAN', ['ARRAY_DATETIME', '...'], 
'ALWAYS_NOT_NULLABLE'],
+        [['ignore'], 'BOOLEAN', ['ARRAY_DATE', '...'], 'ALWAYS_NOT_NULLABLE'],
+        [['ignore'], 'BOOLEAN', ['ARRAY_DATETIMEV2', '...'], 
'ALWAYS_NOT_NULLABLE'],
+        [['ignore'], 'BOOLEAN', ['ARRAY_DATEV2', '...'], 
'ALWAYS_NOT_NULLABLE'],
+        [['ignore'], 'BOOLEAN', ['ARRAY_FLOAT', '...'], 'ALWAYS_NOT_NULLABLE'],
+        [['ignore'], 'BOOLEAN', ['ARRAY_DOUBLE', '...'], 
'ALWAYS_NOT_NULLABLE'],
+        [['ignore'], 'BOOLEAN', ['ARRAY_DECIMALV2', '...'], 
'ALWAYS_NOT_NULLABLE'],
+        [['ignore'], 'BOOLEAN', ['ARRAY_DECIMAL32', '...'], 
'ALWAYS_NOT_NULLABLE'],
+        [['ignore'], 'BOOLEAN', ['ARRAY_DECIMAL64', '...'], 
'ALWAYS_NOT_NULLABLE'],
+        [['ignore'], 'BOOLEAN', ['ARRAY_DECIMAL128', '...'], 
'ALWAYS_NOT_NULLABLE'],
+        [['ignore'], 'BOOLEAN', ['ARRAY_VARCHAR', '...'], 
'ALWAYS_NOT_NULLABLE'],
+        [['ignore'], 'BOOLEAN', ['ARRAY_STRING', '...'], 'ALWAYS_NOT_NULLABLE']
     ]
 }
 
diff --git 
a/regression-test/data/query_p0/sql_functions/cast_function/test_non_nullable_function.out
 
b/regression-test/data/query_p0/sql_functions/cast_function/test_non_nullable_function.out
deleted file mode 100644
index 0f52eee2dbf..00000000000
--- 
a/regression-test/data/query_p0/sql_functions/cast_function/test_non_nullable_function.out
+++ /dev/null
@@ -1,43 +0,0 @@
--- This file is automatically generated. You should know what you did if you 
want to edit this
--- !nullable_1 --
-\N     0
-1      1
-2      2
-3      3
-4      4
-
--- !nullable_2 --
-\N     []
-1      [1, 2, 3]
-2      []
-3      [1, 2, 3]
-4      []
-
--- !nullable_3 --
-\N     []
-1      ["a", "b", "c"]
-2      []
-3      []
-4      ["a", "b", "c"]
-
--- !nullable_4 --
-\N     0
-1      1
-2      2
-3      3
-4      4
-
--- !nullable_5 --
-\N     []
-1      [1, 2, 3]
-2      []
-3      [1, 2, 3]
-4      []
-
--- !nullable_6 --
-\N     []
-1      ["a", "b", "c"]
-2      []
-3      []
-4      ["a", "b", "c"]
-
diff --git 
a/regression-test/data/query_p0/sql_functions/cast_function/test_nullable_functions.out
 
b/regression-test/data/query_p0/sql_functions/cast_function/test_nullable_functions.out
new file mode 100644
index 00000000000..3f9d5501b21
--- /dev/null
+++ 
b/regression-test/data/query_p0/sql_functions/cast_function/test_nullable_functions.out
@@ -0,0 +1,43 @@
+-- This file is automatically generated. You should know what you did if you 
want to edit this
+-- !nullable_1 --
+1      1       1
+2      2       2
+3      3       3
+4      4       4
+
+-- !nullable_2 --
+1      [1, 2, 3]       [1300000000, 2140000000]
+2      []      [1300000000, 2140000000]
+3      [1, 2, 3]       [1300000000, 2140000000]
+4      []      [1300000000, 2140000000]
+
+-- !nullable_3 --
+1      ["a", "b", "c"] 1
+2      []      2
+3      []      3
+4      ["a", "b", "c"] 4
+
+-- !nullable_4 --
+1      1       1
+2      2       2
+3      3       3
+4      4       4
+
+-- !nullable_5 --
+1      [1, 2, 3]       [1.300000000, 2.140000000]
+2      []      [1.300000000, 2.140000000]
+3      [1, 2, 3]       [1.300000000, 2.140000000]
+4      []      [1.300000000, 2.140000000]
+
+-- !nullable_6 --
+1      ["a", "b", "c"] 1
+2      []      2
+3      []      3
+4      ["a", "b", "c"] 4
+
+-- !ignore --
+false
+false
+false
+false
+
diff --git 
a/regression-test/suites/query_p0/sql_functions/cast_function/test_non_nullable_function.groovy
 
b/regression-test/suites/query_p0/sql_functions/cast_function/test_nullable_functions.groovy
similarity index 57%
rename from 
regression-test/suites/query_p0/sql_functions/cast_function/test_non_nullable_function.groovy
rename to 
regression-test/suites/query_p0/sql_functions/cast_function/test_nullable_functions.groovy
index ccd893813b2..e51582d256e 100644
--- 
a/regression-test/suites/query_p0/sql_functions/cast_function/test_non_nullable_function.groovy
+++ 
b/regression-test/suites/query_p0/sql_functions/cast_function/test_nullable_functions.groovy
@@ -15,8 +15,8 @@
 // specific language governing permissions and limitations
 // under the License.
 
-suite("test_non_nullable_function", "query") {
-    def tableName = "tbl_test_non_nullable_function"
+suite("test_nullable_functions", "query") {
+    def tableName = "test_nullable_functions"
 
     sql """DROP TABLE IF EXISTS ${tableName}"""
     sql """
@@ -24,40 +24,50 @@ suite("test_non_nullable_function", "query") {
               `k1` int(11) NULL COMMENT "",
               `k2` ARRAY<int(11)> NULL COMMENT "",
               `k3` ARRAY<VARCHAR(11)> NULL COMMENT "",
-              `k4` ARRAY<decimal(27,9)> NOT NULL COMMENT ""
+              `k4` ARRAY<decimal(27,9)> NOT NULL COMMENT "",
+              `k5` int(11) NOT NULL,
+              `k6` int(11) NULL
             ) ENGINE=OLAP
             DUPLICATE KEY(`k1`)
             DISTRIBUTED BY HASH(`k1`) BUCKETS 1
             PROPERTIES (
-            "replication_allocation" = "tag.location.default: 1",
-            "storage_format" = "V2"
+                "replication_allocation" = "tag.location.default: 1",
+                "storage_format" = "V2"
             )
         """
-    sql """ INSERT INTO ${tableName} VALUES(1, [1, 2, 3], ["a", "b", "c"], 
[1.3, 2.14]) """
-    sql """ INSERT INTO ${tableName} VALUES(2, [], [], [1.3, 2.14]) """
-    sql """ INSERT INTO ${tableName} VALUES(3, [1, 2, 3], [], [1.3, 2.14]) """
-    sql """ INSERT INTO ${tableName} VALUES(4, [], ["a", "b", "c"], [1.3, 
2.14]) """
-    sql """ INSERT INTO ${tableName} VALUES(null, null, null, [1.1,2.2,3.3]) 
"""
-    
+    sql """ INSERT INTO ${tableName} VALUES(1, [1, 2, 3], ["a", "b", "c"], 
[1.3, 2.14], 1, 1) """
+    sql """ INSERT INTO ${tableName} VALUES(2, [], [], [1.3, 2.14], 2, 2) """
+    sql """ INSERT INTO ${tableName} VALUES(3, [1, 2, 3], [], [1.3, 2.14], 3, 
3) """
+    sql """ INSERT INTO ${tableName} VALUES(4, [], ["a", "b", "c"], [1.3, 
2.14], 4, null) """
+
     sql "set enable_nereids_planner=false"
-    qt_nullable_1 "SELECT k1, non_nullable(k1) FROM ${tableName} ORDER BY k1"
-    qt_nullable_2 "SELECT k1, non_nullable(k2) FROM ${tableName} ORDER BY k1"
-    qt_nullable_3 "SELECT k1, non_nullable(k3) FROM ${tableName} ORDER BY k1"
+    qt_nullable_1 "SELECT k1, non_nullable(k1), nullable(k1) FROM ${tableName} 
ORDER BY k1"
+    qt_nullable_2 "SELECT k1, non_nullable(k2), nullable(k4) FROM ${tableName} 
ORDER BY k1"
+    qt_nullable_3 "SELECT k1, non_nullable(k3), nullable(k5) FROM ${tableName} 
ORDER BY k1"
     try {
-        def result = "SELECT k1, non_nullable(k4) FROM ${tableName} ORDER BY 
k1"
+        def result = sql "SELECT k1, non_nullable(k4) FROM ${tableName} ORDER 
BY k1"
+        fail()
     } catch (Exception e) {
         assertTrue(e.getMessage().contains("Try to use originally non-nullable 
column"))
     }
+    try {
+        result = sql "SELECT k1, non_nullable(k6) FROM ${tableName} ORDER BY 
k1"
+        fail()
+    } catch (Exception e) {
+        assertTrue(e.getMessage().contains("There's NULL value"))
+    }
 
     sql "set enable_nereids_planner=true"
     sql "set forbid_unknown_col_stats=false"
     sql "set enable_fallback_to_original_planner=false"
-    qt_nullable_4 "SELECT k1, non_nullable(k1) FROM ${tableName} ORDER BY k1"
-    qt_nullable_5 "SELECT k1, non_nullable(k2) FROM ${tableName} ORDER BY k1"
-    qt_nullable_6 "SELECT k1, non_nullable(k3) FROM ${tableName} ORDER BY k1"
+    qt_nullable_4 "SELECT k1, non_nullable(k1), nullable(k1) FROM ${tableName} 
ORDER BY k1"
+    qt_nullable_5 "SELECT k1, non_nullable(k2), nullable(k4) FROM ${tableName} 
ORDER BY k1"
+    qt_nullable_6 "SELECT k1, non_nullable(k3), nullable(k5) FROM ${tableName} 
ORDER BY k1"
     try {
-        def result = "SELECT k1, non_nullable(k4) FROM ${tableName} ORDER BY 
k1"
+        result = sql "SELECT k1, non_nullable(k4) FROM ${tableName} ORDER BY 
k1"
+        fail()
     } catch (Exception e) {
         assertTrue(e.getMessage().contains("Try to use originally non-nullable 
column"))
     }
+    qt_ignore "SELECT ignore(k1*k5, k2, k3, k4, k5) FROM ${tableName} ORDER BY 
k1"
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org
For additional commands, e-mail: commits-h...@doris.apache.org

Reply via email to