[Feature](functions) support ignore and nullable functions (#27848)

support ignore and nullable functions
This commit is contained in:
zclllyybb
2023-12-05 14:09:32 +08:00
committed by GitHub
parent 79f6f85cf1
commit c98b80ae6a
12 changed files with 378 additions and 82 deletions

View File

@ -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>;

View File

@ -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

View File

@ -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>();
}

View File

@ -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;
}

View File

@ -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"),

View File

@ -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);
}
}

View File

@ -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);
}
}

View File

@ -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);
}

View File

@ -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']
]
}

View File

@ -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"]

View File

@ -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

View File

@ -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"
}