This is an automated email from the ASF dual-hosted git repository.
yiguolei 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 3787e8780f1 [feature](cast)Support try-cast (#56171)
3787e8780f1 is described below
commit 3787e8780f13c8ecc7997b19a0685f76b1b6dc12
Author: James <[email protected]>
AuthorDate: Sun Sep 28 14:33:21 2025 +0800
[feature](cast)Support try-cast (#56171)
---
be/src/vec/exprs/vcast_expr.cpp | 132 +++++++-
be/src/vec/exprs/vcast_expr.h | 37 +-
be/src/vec/exprs/vexpr.cpp | 4 +
be/test/vec/exprs/try_cast_expr_test.cpp | 247 ++++++++++++++
.../antlr4/org/apache/doris/nereids/DorisLexer.g4 | 1 +
.../antlr4/org/apache/doris/nereids/DorisParser.g4 | 1 +
.../java/org/apache/doris/analysis/CastExpr.java | 10 +-
.../main/java/org/apache/doris/analysis/Expr.java | 5 +
.../org/apache/doris/analysis/TryCastExpr.java | 137 ++++++++
.../glue/translator/ExpressionTranslator.java | 12 +
.../doris/nereids/parser/LogicalPlanBuilder.java | 24 ++
.../expression/rules/FoldConstantRuleOnFE.java | 11 +
.../doris/nereids/trees/expressions/Cast.java | 6 +-
.../doris/nereids/trees/expressions/TryCast.java | 99 ++++++
.../expressions/visitor/ExpressionVisitor.java | 6 +
.../nereids/trees/expressions/TryCastTest.java | 377 +++++++++++++++++++++
gensrc/thrift/Exprs.thrift | 4 +-
gensrc/thrift/Opcodes.thrift | 1 +
.../data/function_p0/cast/test_try_cast.out | 21 ++
.../expression/fold_constant/fe_try_cast.out | 13 +
.../suites/function_p0/cast/test_try_cast.groovy | 102 ++++++
.../expression/fold_constant/fe_try_cast.groovy | 52 +++
22 files changed, 1288 insertions(+), 14 deletions(-)
diff --git a/be/src/vec/exprs/vcast_expr.cpp b/be/src/vec/exprs/vcast_expr.cpp
index e387d6b406c..cc45fff40b9 100644
--- a/be/src/vec/exprs/vcast_expr.cpp
+++ b/be/src/vec/exprs/vcast_expr.cpp
@@ -28,9 +28,13 @@
#include "common/exception.h"
#include "common/status.h"
#include "runtime/runtime_state.h"
+#include "vec/columns/column.h"
+#include "vec/columns/column_nullable.h"
+#include "vec/common/assert_cast.h"
#include "vec/core/block.h"
#include "vec/core/column_with_type_and_name.h"
#include "vec/core/columns_with_type_and_name.h"
+#include "vec/data_types/data_type_nullable.h"
#include "vec/exprs/vexpr.h"
#include "vec/exprs/vexpr_context.h"
#include "vec/functions/simple_function_factory.h"
@@ -71,8 +75,8 @@ doris::Status VCastExpr::prepare(doris::RuntimeState* state,
const doris::RowDes
return Status::NotSupported("Function {} is not implemented",
_fn.name.function_name);
}
VExpr::register_function_context(state, context);
- _expr_name = fmt::format("(CAST {}({}) TO {})", child_name,
child->data_type()->get_name(),
- _target_data_type_name);
+ _expr_name = fmt::format("({} {}({}) TO {})", cast_name(), child_name,
+ child->data_type()->get_name(),
_target_data_type_name);
_prepare_finished = true;
return Status::OK();
}
@@ -122,13 +126,135 @@ doris::Status VCastExpr::execute(VExprContext* context,
doris::vectorized::Block
return Status::OK();
}
+bool cast_error_code(Status& st) {
+ //There may be more error codes that need to be captured by try cast in
the future.
+ if (st.is<ErrorCode::INVALID_ARGUMENT>()) {
+ return true;
+ } else {
+ return false;
+ }
+}
+
+DataTypePtr TryCastExpr::original_cast_return_type() {
+ if (_original_cast_return_is_nullable) {
+ return _data_type;
+ } else {
+ return remove_nullable(_data_type);
+ }
+}
+
+Status TryCastExpr::execute(VExprContext* context, Block* block, int*
result_column_id) {
+ DCHECK(_open_finished || _getting_const_col)
+ << _open_finished << _getting_const_col << _expr_name;
+ if (is_const_and_have_executed()) { // const have executed in open function
+ return get_result_from_const(block, _expr_name, result_column_id);
+ }
+
+ int input_column_id = 0;
+
+ // For try_cast, try to execute it in batches first.
+
+ // execute child first
+ RETURN_IF_ERROR(_children[0]->execute(context, block, &input_column_id));
+
+ // prepare block
+ int output_column_id = block->columns();
+ block->insert({nullptr, original_cast_return_type(), _expr_name});
+
+ // batch execute
+ auto batch_exec_status =
_function->execute(context->fn_context(_fn_context_index), *block,
+
{static_cast<uint32_t>(input_column_id)},
+ output_column_id,
block->rows());
+ // If batch is executed successfully,
+ // it means that there is no error and it will be returned directly.
+ if (batch_exec_status.ok()) {
+ // wrap nullable
+ block->get_by_position(output_column_id).column =
+ make_nullable(block->get_by_position(output_column_id).column);
+ block->get_by_position(output_column_id).type =
+ make_nullable(block->get_by_position(output_column_id).type);
+ *result_column_id = output_column_id;
+ return batch_exec_status;
+ }
+
+ // If there is an error that cannot be handled by try cast, it will be
returned directly.
+ if (!cast_error_code(batch_exec_status)) {
+ return batch_exec_status;
+ }
+
+ // If there is an error that can be handled by try cast,
+ // it will be converted into line execution.
+ auto& input_info = block->get_by_position(input_column_id);
+ ColumnPtr return_column;
+ // distinguish whether the return value of the original cast is nullable
+ if (_original_cast_return_is_nullable) {
+ RETURN_IF_ERROR(single_row_execute<true>(context, input_info,
return_column));
+ } else {
+ RETURN_IF_ERROR(single_row_execute<false>(context, input_info,
return_column));
+ }
+ // wrap nullable
+ block->get_by_position(output_column_id).column = return_column;
+ block->get_by_position(output_column_id).type =
+ make_nullable(block->get_by_position(output_column_id).type);
+ *result_column_id = output_column_id;
+ return Status::OK();
+}
+
+template <bool original_cast_reutrn_is_nullable>
+Status TryCastExpr::single_row_execute(VExprContext* context,
+ const ColumnWithTypeAndName& input_info,
+ ColumnPtr& return_column) {
+ auto input_column = input_info.column;
+ const auto& input_type = input_info.type;
+ const auto& input_name = input_info.name;
+ auto result_column = _data_type->create_column();
+
+ ColumnNullable& result_null_column =
assert_cast<ColumnNullable&>(*result_column);
+
+ IColumn& result_nested_column = result_null_column.get_nested_column();
+ auto& result_null_map_data = result_null_column.get_null_map_data();
+
+ auto insert_from_single_row = [&](const IColumn& single_exec_column,
size_t row) {
+ DCHECK_EQ(single_exec_column.size(), 1);
+ if constexpr (original_cast_reutrn_is_nullable) {
+ result_null_column.insert_from(single_exec_column, 0);
+ } else {
+ DCHECK(!single_exec_column.is_nullable());
+ result_nested_column.insert_from(single_exec_column, 0);
+ result_null_map_data.push_back(0);
+ }
+ };
+
+ auto insert_null = [&](size_t row) { result_null_column.insert_default();
};
+
+ const auto size = input_column->size();
+ for (size_t row = 0; row < size; ++row) {
+ Block single_row_block;
+ single_row_block.insert({input_column->cut(row, 1), input_type,
input_name});
+ single_row_block.insert({nullptr, original_cast_return_type(),
_expr_name});
+
+ auto single_exec_status =
_function->execute(context->fn_context(_fn_context_index),
+ single_row_block, {0}, 1,
1);
+ if (single_exec_status.ok()) {
+
insert_from_single_row(*single_row_block.get_by_position(1).column, row);
+ } else {
+ if (!cast_error_code(single_exec_status)) {
+ return single_exec_status;
+ }
+ insert_null(row);
+ }
+ }
+ return_column = std::move(result_column);
+ return Status::OK();
+}
+
const std::string& VCastExpr::expr_name() const {
return _expr_name;
}
std::string VCastExpr::debug_string() const {
std::stringstream out;
- out << "CastExpr(CAST " << get_child(0)->data_type()->get_name() << " to "
+ out << cast_name() << " Expr(CAST " <<
get_child(0)->data_type()->get_name() << " to "
<< _target_data_type->get_name() << "){";
bool first = true;
for (const auto& input_expr : children()) {
diff --git a/be/src/vec/exprs/vcast_expr.h b/be/src/vec/exprs/vcast_expr.h
index f553d7682a3..afcd4e687a4 100644
--- a/be/src/vec/exprs/vcast_expr.h
+++ b/be/src/vec/exprs/vcast_expr.h
@@ -23,6 +23,7 @@
#include "runtime/define_primitive_type.h"
#include "udf/udf.h"
#include "vec/aggregate_functions/aggregate_function.h"
+#include "vec/core/column_with_type_and_name.h"
#include "vec/data_types/data_type.h"
#include "vec/exprs/vexpr.h"
#include "vec/functions/function.h"
@@ -38,10 +39,13 @@ class VExprContext;
} // namespace doris
namespace doris::vectorized {
-class VCastExpr final : public VExpr {
+class VCastExpr : public VExpr {
ENABLE_FACTORY_CREATOR(VCastExpr);
public:
+#ifdef BE_TEST
+ VCastExpr() = default;
+#endif
VCastExpr(const TExprNode& node) : VExpr(node) {}
~VCastExpr() override = default;
Status execute(VExprContext* context, Block* block, int* result_column_id)
override;
@@ -53,10 +57,13 @@ public:
std::string debug_string() const override;
const DataTypePtr& get_target_type() const;
-private:
+ virtual std::string cast_name() const { return "CAST"; }
+
+protected:
FunctionBasePtr _function;
std::string _expr_name;
+private:
DataTypePtr _target_data_type;
std::string _target_data_type_name;
@@ -64,4 +71,30 @@ private:
static const constexpr char* function_name = "CAST";
};
+
+class TryCastExpr final : public VCastExpr {
+ ENABLE_FACTORY_CREATOR(TryCastExpr);
+
+public:
+#ifdef BE_TEST
+ TryCastExpr() = default;
+#endif
+
+ TryCastExpr(const TExprNode& node)
+ : VCastExpr(node),
_original_cast_return_is_nullable(node.is_cast_nullable) {}
+ Status execute(VExprContext* context, Block* block, int* result_column_id)
override;
+ ~TryCastExpr() override = default;
+ std::string cast_name() const override { return "TRY CAST"; }
+
+private:
+ DataTypePtr original_cast_return_type();
+ template <bool original_cast_reutrn_is_nullable>
+ Status single_row_execute(VExprContext* context, const
ColumnWithTypeAndName& input_info,
+ ColumnPtr& return_column);
+
+ //Try_cast always returns nullable,
+ // but we also need the information of whether the return value of the
original cast is nullable.
+ bool _original_cast_return_is_nullable = false;
+};
+
} // namespace doris::vectorized
diff --git a/be/src/vec/exprs/vexpr.cpp b/be/src/vec/exprs/vexpr.cpp
index 7cb0ac2cd0f..0a057616e50 100644
--- a/be/src/vec/exprs/vexpr.cpp
+++ b/be/src/vec/exprs/vexpr.cpp
@@ -495,6 +495,10 @@ Status VExpr::create_expr(const TExprNode& expr_node,
VExprSPtr& expr) {
expr = VCastExpr::create_shared(expr_node);
break;
}
+ case TExprNodeType::TRY_CAST_EXPR: {
+ expr = TryCastExpr::create_shared(expr_node);
+ break;
+ }
case TExprNodeType::IN_PRED: {
expr = VInPredicate::create_shared(expr_node);
break;
diff --git a/be/test/vec/exprs/try_cast_expr_test.cpp
b/be/test/vec/exprs/try_cast_expr_test.cpp
new file mode 100644
index 00000000000..2021df577c0
--- /dev/null
+++ b/be/test/vec/exprs/try_cast_expr_test.cpp
@@ -0,0 +1,247 @@
+// 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 <gmock/gmock.h>
+#include <gtest/gtest.h>
+
+#include <memory>
+
+#include "runtime/primitive_type.h"
+#include "vec/columns/column_nullable.h"
+#include "vec/core/field.h"
+#include "vec/core/types.h"
+#include "vec/exprs/vcast_expr.h"
+
+namespace doris::vectorized {
+
+template <class Impl>
+class try_cast_test_function : public IFunction {
+public:
+ static constexpr auto name = "";
+ static FunctionPtr create() { return
std::make_shared<try_cast_test_function>(); }
+ String get_name() const override { return name; }
+ bool skip_return_type_check() const override { return true; }
+ bool use_default_implementation_for_constants() const override { return
false; }
+
+ size_t get_number_of_arguments() const override { return 0; }
+
+ bool is_variadic() const override { return true; }
+
+ DataTypePtr get_return_type_impl(const DataTypes& arguments) const
override {
+ return std::make_shared<DataTypeFloat64>();
+ }
+
+ Status execute_impl(FunctionContext* context, Block& block, const
ColumnNumbers& arguments,
+ uint32_t result, size_t input_rows_count) const
override {
+ return Impl::execute_impl(context, block, arguments, result,
input_rows_count);
+ }
+};
+
+struct TryCastTestNoErrorImpl {
+ static Status execute_impl(FunctionContext* context, Block& block,
+ const ColumnNumbers& arguments, uint32_t result,
+ size_t input_rows_count) {
+ auto column =
block.get_by_position(arguments[0]).type->create_column();
+ column->insert_many_defaults(input_rows_count);
+ block.get_by_position(result).column = std::move(column);
+ return Status::OK();
+ }
+};
+
+struct TryCastTestReturnErrorImpl {
+ static Status execute_impl(FunctionContext* context, Block& block,
+ const ColumnNumbers& arguments, uint32_t result,
+ size_t input_rows_count) {
+ return Status::InternalError("try_cast test error");
+ }
+};
+
+struct TryCastTestRowExecReturnNotNullImpl {
+ static Status execute_impl(FunctionContext* context, Block& block,
+ const ColumnNumbers& arguments, uint32_t result,
+ size_t input_rows_count) {
+ auto column = block.get_by_position(arguments[0]).column;
+ if (column->size() > 1) {
+ return Status::InvalidArgument("input column size > 1");
+ }
+ auto x = column->get_int(0);
+ if (x == 1) {
+ return Status::InvalidArgument("try_cast test error");
+ }
+ auto ret_col = ColumnInt32::create();
+ ret_col->insert_value(x);
+ block.get_by_position(result).column = std::move(ret_col);
+ return Status::OK();
+ }
+};
+
+struct TryCastTestRowExecReturnNullImpl {
+ static Status execute_impl(FunctionContext* context, Block& block,
+ const ColumnNumbers& arguments, uint32_t result,
+ size_t input_rows_count) {
+ auto column = block.get_by_position(arguments[0]).column;
+ if (column->size() > 1) {
+ return Status::InvalidArgument("input column size > 1");
+ }
+ auto x = column->get_int(0);
+ if (x == 1) {
+ return Status::InvalidArgument("try_cast test error");
+ }
+ auto ret_col = ColumnInt32::create();
+ ret_col->insert_value(x);
+ auto col =
+ ColumnNullable::create(std::move(ret_col),
ColumnUInt8::create(1, x == 0 ? 0 : 1));
+ block.get_by_position(result).column = std::move(col);
+ return Status::OK();
+ }
+};
+
+struct TryCastTestRowExecReturnErrorImpl {
+ static Status execute_impl(FunctionContext* context, Block& block,
+ const ColumnNumbers& arguments, uint32_t result,
+ size_t input_rows_count) {
+ auto column = block.get_by_position(arguments[0]).column;
+ if (column->size() > 1) {
+ return Status::InvalidArgument("input column size > 1");
+ }
+ return Status::InternalError("try_cast test error");
+ }
+};
+
+class MockVExprForTryCast : public VExpr {
+public:
+ MockVExprForTryCast() = default;
+ MOCK_CONST_METHOD0(clone, VExprSPtr());
+ MOCK_CONST_METHOD0(expr_name, const std::string&());
+
+ Status execute(VExprContext* context, Block* block, int* result_column_id)
override {
+ auto int_type = std::make_shared<DataTypeInt32>();
+ auto int_column = int_type->create_column();
+ for (int i = 0; i < 3; i++) {
+ Int32 x = i;
+ int_column->insert_data((const char*)&x, sizeof(Int32));
+ }
+ block->insert({std::move(int_column), int_type, "mock_input_column"});
+ *result_column_id = 0;
+ return Status::OK();
+ }
+};
+
+struct TryCastExprTest : public ::testing::Test {
+ void SetUp() override {
+ try_cast_expr._data_type =
+
std::make_shared<DataTypeNullable>(std::make_shared<DataTypeInt32>());
+ try_cast_expr._open_finished = true;
+
+ try_cast_expr.add_child(std::make_shared<MockVExprForTryCast>());
+ try_cast_expr._fn_context_index = 0;
+
+ context =
std::make_unique<VExprContext>(std::make_shared<MockVExprForTryCast>());
+ context->_fn_contexts.push_back(nullptr);
+ }
+
+ TryCastExpr try_cast_expr;
+
+ std::unique_ptr<VExprContext> context;
+};
+
+TEST_F(TryCastExprTest, BasicTest1) {
+ try_cast_expr._function = std::make_shared<DefaultFunction>(
+ try_cast_test_function<TryCastTestNoErrorImpl>::create(),
+ DataTypes {std::make_shared<DataTypeInt32>()},
std::make_shared<DataTypeInt32>());
+
+ Block block;
+ int result_column_id = -1;
+ try_cast_expr._original_cast_return_is_nullable = true;
+ auto st = try_cast_expr.execute(context.get(), &block, &result_column_id);
+ EXPECT_TRUE(st.ok()) << st.msg();
+}
+
+TEST_F(TryCastExprTest, BasicTest2) {
+ try_cast_expr._function = std::make_shared<DefaultFunction>(
+ try_cast_test_function<TryCastTestNoErrorImpl>::create(),
+ DataTypes {std::make_shared<DataTypeInt32>()},
std::make_shared<DataTypeInt32>());
+
+ Block block;
+ int result_column_id = -1;
+ try_cast_expr._original_cast_return_is_nullable = false;
+ auto st = try_cast_expr.execute(context.get(), &block, &result_column_id);
+ EXPECT_TRUE(st.ok()) << st.msg();
+}
+
+TEST_F(TryCastExprTest, return_error) {
+ try_cast_expr._function = std::make_shared<DefaultFunction>(
+ try_cast_test_function<TryCastTestReturnErrorImpl>::create(),
+ DataTypes {std::make_shared<DataTypeInt32>()},
std::make_shared<DataTypeInt32>());
+
+ Block block;
+ int result_column_id = -1;
+ try_cast_expr._original_cast_return_is_nullable = false;
+ auto st = try_cast_expr.execute(context.get(), &block, &result_column_id);
+ EXPECT_FALSE(st.ok()) << st.msg();
+}
+
+TEST_F(TryCastExprTest, row_exec1) {
+ try_cast_expr._function = std::make_shared<DefaultFunction>(
+
try_cast_test_function<TryCastTestRowExecReturnNotNullImpl>::create(),
+ DataTypes {std::make_shared<DataTypeInt32>()},
std::make_shared<DataTypeInt32>());
+
+ Block block;
+ int result_column_id = -1;
+ try_cast_expr._original_cast_return_is_nullable = false;
+ auto st = try_cast_expr.execute(context.get(), &block, &result_column_id);
+ EXPECT_TRUE(st.ok()) << st.msg();
+
+ auto result_col = block.get_by_position(result_column_id).column;
+ EXPECT_EQ(result_col->size(), 3);
+
+ EXPECT_EQ(result_col->is_null_at(0), false);
+ EXPECT_EQ(result_col->is_null_at(1), true);
+ EXPECT_EQ(result_col->is_null_at(2), false);
+}
+
+TEST_F(TryCastExprTest, row_exec2) {
+ try_cast_expr._function = std::make_shared<DefaultFunction>(
+ try_cast_test_function<TryCastTestRowExecReturnNullImpl>::create(),
+ DataTypes {std::make_shared<DataTypeInt32>()},
std::make_shared<DataTypeInt32>());
+
+ Block block;
+ int result_column_id = -1;
+ try_cast_expr._original_cast_return_is_nullable = true;
+ auto st = try_cast_expr.execute(context.get(), &block, &result_column_id);
+ EXPECT_TRUE(st.ok()) << st.msg();
+
+ auto result_col = block.get_by_position(result_column_id).column;
+ EXPECT_EQ(result_col->size(), 3);
+
+ EXPECT_EQ(result_col->is_null_at(0), false);
+ EXPECT_EQ(result_col->is_null_at(1), true);
+ EXPECT_EQ(result_col->is_null_at(2), true);
+}
+
+TEST_F(TryCastExprTest, row_exec3) {
+ try_cast_expr._function = std::make_shared<DefaultFunction>(
+
try_cast_test_function<TryCastTestRowExecReturnErrorImpl>::create(),
+ DataTypes {std::make_shared<DataTypeInt32>()},
std::make_shared<DataTypeInt32>());
+
+ Block block;
+ int result_column_id = -1;
+ try_cast_expr._original_cast_return_is_nullable = true;
+ auto st = try_cast_expr.execute(context.get(), &block, &result_column_id);
+ EXPECT_FALSE(st.ok()) << st.msg();
+}
+
+} // namespace doris::vectorized
\ No newline at end of file
diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4
b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4
index d4b3b1b09ce..360a498f611 100644
--- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4
+++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4
@@ -544,6 +544,7 @@ TRIGGERS: 'TRIGGERS';
TRIM: 'TRIM';
TRUE: 'TRUE';
TRUNCATE: 'TRUNCATE';
+TRY_CAST: 'TRY_CAST';
TYPE: 'TYPE';
TYPECAST: 'TYPE_CAST';
TYPES: 'TYPES';
diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4
b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4
index 1441aced294..25075996fba 100644
--- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4
+++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4
@@ -1583,6 +1583,7 @@ primaryExpression
| CASE whenClause+ (ELSE elseExpression=expression)? END
#searchedCase
| CASE value=expression whenClause+ (ELSE elseExpression=expression)? END
#simpleCase
| name=CAST LEFT_PAREN expression AS castDataType RIGHT_PAREN
#cast
+ | name=TRY_CAST LEFT_PAREN expression AS castDataType RIGHT_PAREN
#tryCast
| constant
#constantDefault
| interval
#intervalLiteral
| ASTERISK (exceptOrReplace)*
#star
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/CastExpr.java
b/fe/fe-core/src/main/java/org/apache/doris/analysis/CastExpr.java
index 30f56a5650e..e2d04d352a8 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/CastExpr.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/CastExpr.java
@@ -55,18 +55,18 @@ public class CastExpr extends Expr {
// Only set for explicit casts. Null for implicit casts.
@SerializedName("ttd")
- private TypeDef targetTypeDef;
+ protected TypeDef targetTypeDef;
// True if this is a "pre-analyzed" implicit cast.
@SerializedName("ii")
- private boolean isImplicit;
+ protected boolean isImplicit;
// True if this cast does not change the type.
- private boolean noOp = false;
+ protected boolean noOp = false;
- private boolean notFold = false;
+ protected boolean notFold = false;
- private static final Map<Pair<Type, Type>, Function.NullableMode>
TYPE_NULLABLE_MODE;
+ protected static final Map<Pair<Type, Type>, Function.NullableMode>
TYPE_NULLABLE_MODE;
static {
TYPE_NULLABLE_MODE = Maps.newHashMap();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/Expr.java
b/fe/fe-core/src/main/java/org/apache/doris/analysis/Expr.java
index 2768f9527a3..702d92fb425 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/Expr.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/Expr.java
@@ -91,6 +91,7 @@ public abstract class Expr extends TreeNode<Expr> implements
Cloneable, ExprStat
public static final float FUNCTION_CALL_COST = 10;
protected Optional<Boolean> nullableFromNereids = Optional.empty();
+ protected Optional<Boolean> originCastNullable = Optional.empty();
// returns true if an Expr is a non-analytic aggregate.
private static final com.google.common.base.Predicate<Expr>
IS_AGGREGATE_PREDICATE =
@@ -1568,6 +1569,10 @@ public abstract class Expr extends TreeNode<Expr>
implements Cloneable, ExprStat
return nullableFromNereids;
}
+ public void setOriginCastNullable(boolean nullable) {
+ originCastNullable = Optional.of(nullable);
+ }
+
public void clearNullableFromNereids() {
nullableFromNereids = Optional.empty();
}
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/analysis/TryCastExpr.java
b/fe/fe-core/src/main/java/org/apache/doris/analysis/TryCastExpr.java
new file mode 100644
index 00000000000..e9f99765b57
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/TryCastExpr.java
@@ -0,0 +1,137 @@
+// 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.
+// This file is copied from
+//
https://github.com/apache/impala/blob/branch-2.9.0/fe/src/main/java/org/apache/impala/CastExpr.java
+// and modified by Doris
+
+package org.apache.doris.analysis;
+
+import org.apache.doris.catalog.TableIf;
+import org.apache.doris.catalog.TableIf.TableType;
+import org.apache.doris.catalog.Type;
+import org.apache.doris.common.AnalysisException;
+import org.apache.doris.qe.ConnectContext;
+import org.apache.doris.thrift.TExprNode;
+import org.apache.doris.thrift.TExprNodeType;
+import org.apache.doris.thrift.TExprOpcode;
+
+public class TryCastExpr extends CastExpr {
+
+ public TryCastExpr(Type targetType, Expr e) {
+ super(targetType, e);
+ opcode = TExprOpcode.TRY_CAST;
+ }
+
+ public TryCastExpr(Type targetType, Expr e, Void v) {
+ super(targetType, e, v);
+ opcode = TExprOpcode.TRY_CAST;
+ }
+
+ public TryCastExpr(TypeDef targetTypeDef, Expr e) {
+ super(targetTypeDef, e);
+ opcode = TExprOpcode.TRY_CAST;
+ }
+
+ protected TryCastExpr(TryCastExpr other) {
+ super(other);
+ opcode = TExprOpcode.TRY_CAST;
+ }
+
+ private static String getFnName(Type targetType) {
+ return "tryCastTo" + targetType.getPrimitiveType().toString();
+ }
+
+ @Override
+ public Expr clone() {
+ return new TryCastExpr(this);
+ }
+
+ @Override
+ public String toSqlImpl() {
+ if (isAnalyzed) {
+ return "TRYCAST(" + getChild(0).toSql() + " AS " + type.toSql() +
")";
+ } else {
+ return "TRYCAST(" + getChild(0).toSql() + " AS "
+ + (isImplicit ? type.toString() : targetTypeDef.toSql()) +
")";
+ }
+ }
+
+ @Override
+ public String toSqlImpl(boolean disableTableName, boolean needExternalSql,
TableType tableType, TableIf table) {
+ if (needExternalSql) {
+ return getChild(0).toSql(disableTableName, needExternalSql,
tableType, table);
+ }
+ if (isAnalyzed) {
+ return "TRYCAST(" + getChild(0).toSql(disableTableName,
needExternalSql, tableType, table) + " AS "
+ + type.toSql() + ")";
+ } else {
+ return "TRYCAST(" + getChild(0).toSql(disableTableName,
needExternalSql, tableType, table) + " AS "
+ + (isImplicit ? type.toString() : targetTypeDef.toSql()) +
")";
+ }
+ }
+
+ @Override
+ public String toDigestImpl() {
+ boolean isVerbose = ConnectContext.get() != null
+ && ConnectContext.get().getExecutor() != null
+ && ConnectContext.get().getExecutor().getParsedStmt() != null
+ &&
ConnectContext.get().getExecutor().getParsedStmt().getExplainOptions() != null
+ &&
ConnectContext.get().getExecutor().getParsedStmt().getExplainOptions().isVerbose();
+ if (isImplicit && !isVerbose) {
+ return getChild(0).toDigest();
+ }
+ if (isAnalyzed) {
+ return "TRYCAST(" + getChild(0).toDigest() + " AS " +
type.toString() + ")";
+ } else {
+ return "TRYCAST(" + getChild(0).toDigest() + " AS " +
targetTypeDef.toString() + ")";
+ }
+ }
+
+ @Override
+ protected void toThrift(TExprNode msg) {
+ msg.node_type = TExprNodeType.TRY_CAST_EXPR;
+ msg.setOpcode(opcode);
+ if (type.isNativeType() && getChild(0).getType().isNativeType()) {
+
msg.setChildType(getChild(0).getType().getPrimitiveType().toThrift());
+ }
+ originCastNullable.ifPresent(msg::setIsCastNullable);
+ }
+
+ public void analyze() throws AnalysisException {
+ super.analyze();
+ this.opcode = TExprOpcode.TRY_CAST;
+ }
+
+ @Override
+ public int hashCode() {
+ return super.hashCode();
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (!super.equals(obj)) {
+ return false;
+ }
+ TryCastExpr expr = (TryCastExpr) obj;
+ return this.opcode == expr.opcode;
+ }
+
+ @Override
+ public boolean isNullable() {
+ return true;
+ }
+}
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/ExpressionTranslator.java
b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/ExpressionTranslator.java
index 9ec0f509445..75500601361 100644
---
a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/ExpressionTranslator.java
+++
b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/ExpressionTranslator.java
@@ -38,6 +38,7 @@ import org.apache.doris.analysis.MatchPredicate;
import org.apache.doris.analysis.OrderByElement;
import org.apache.doris.analysis.SlotRef;
import org.apache.doris.analysis.TimestampArithmeticExpr;
+import org.apache.doris.analysis.TryCastExpr;
import org.apache.doris.catalog.ArrayType;
import org.apache.doris.catalog.Column;
import org.apache.doris.catalog.Function;
@@ -75,6 +76,7 @@ import org.apache.doris.nereids.trees.expressions.Or;
import org.apache.doris.nereids.trees.expressions.OrderExpression;
import org.apache.doris.nereids.trees.expressions.SlotReference;
import org.apache.doris.nereids.trees.expressions.TimestampArithmetic;
+import org.apache.doris.nereids.trees.expressions.TryCast;
import org.apache.doris.nereids.trees.expressions.UnaryArithmetic;
import org.apache.doris.nereids.trees.expressions.VirtualSlotReference;
import org.apache.doris.nereids.trees.expressions.WhenClause;
@@ -440,6 +442,16 @@ public class ExpressionTranslator extends
DefaultExpressionVisitor<Expr, PlanTra
return castExpr;
}
+ @Override
+ public Expr visitTryCast(TryCast cast, PlanTranslatorContext context) {
+ // left child of cast is expression, right child of cast is target type
+ TryCastExpr tryCastExpr = new
TryCastExpr(cast.getDataType().toCatalogDataType(),
+ cast.child().accept(this, context), null);
+ tryCastExpr.setNullableFromNereids(cast.nullable());
+ tryCastExpr.setOriginCastNullable(cast.parentNullable());
+ return tryCastExpr;
+ }
+
@Override
public Expr visitInPredicate(InPredicate inPredicate,
PlanTranslatorContext context) {
List<Expr> inList = inPredicate.getOptions().stream()
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java
b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java
index c790a0a50c0..b2665ebdd42 100644
---
a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java
+++
b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java
@@ -539,6 +539,7 @@ import org.apache.doris.nereids.trees.expressions.Slot;
import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator;
import org.apache.doris.nereids.trees.expressions.Subtract;
import org.apache.doris.nereids.trees.expressions.TimestampArithmetic;
+import org.apache.doris.nereids.trees.expressions.TryCast;
import org.apache.doris.nereids.trees.expressions.WhenClause;
import org.apache.doris.nereids.trees.expressions.WindowExpression;
import org.apache.doris.nereids.trees.expressions.WindowFrame;
@@ -3001,6 +3002,11 @@ public class LogicalPlanBuilder extends
DorisParserBaseVisitor<Object> {
return ParserUtils.withOrigin(ctx, () ->
processCast(getExpression(ctx.expression()), ctx.castDataType()));
}
+ @Override
+ public Expression visitTryCast(DorisParser.TryCastContext ctx) {
+ return ParserUtils.withOrigin(ctx, () ->
processTryCast(getExpression(ctx.expression()), ctx.castDataType()));
+ }
+
@Override
public UnboundFunction visitExtract(DorisParser.ExtractContext ctx) {
return ParserUtils.withOrigin(ctx, () -> {
@@ -3073,6 +3079,24 @@ public class LogicalPlanBuilder extends
DorisParserBaseVisitor<Object> {
}
}
+ private Expression processTryCast(Expression expression,
CastDataTypeContext castDataTypeContext) {
+ DataType dataType = visitCastDataType(castDataTypeContext);
+ Expression cast = new TryCast(expression, dataType, true);
+ if (dataType.isStringLikeType() && ((CharacterType) dataType).getLen()
>= 0) {
+ if (dataType.isVarcharType() && ((VarcharType)
dataType).isWildcardVarchar()) {
+ return cast;
+ }
+ List<Expression> args = ImmutableList.of(
+ cast,
+ new TinyIntLiteral((byte) 1),
+ Literal.of(((CharacterType) dataType).getLen())
+ );
+ return new UnboundFunction("substr", args);
+ } else {
+ return cast;
+ }
+ }
+
@Override
public Expression visitGroupConcat(GroupConcatContext ctx) {
return ParserUtils.withOrigin(ctx, () -> {
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FoldConstantRuleOnFE.java
b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FoldConstantRuleOnFE.java
index dd28ed1710b..bfc983915b3 100644
---
a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FoldConstantRuleOnFE.java
+++
b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/rules/FoldConstantRuleOnFE.java
@@ -57,6 +57,7 @@ import
org.apache.doris.nereids.trees.expressions.NullSafeEqual;
import org.apache.doris.nereids.trees.expressions.Or;
import org.apache.doris.nereids.trees.expressions.Slot;
import org.apache.doris.nereids.trees.expressions.TimestampArithmetic;
+import org.apache.doris.nereids.trees.expressions.TryCast;
import org.apache.doris.nereids.trees.expressions.Variable;
import org.apache.doris.nereids.trees.expressions.WhenClause;
import org.apache.doris.nereids.trees.expressions.functions.BoundFunction;
@@ -169,6 +170,7 @@ public class FoldConstantRuleOnFE extends
AbstractExpressionRewriteRule
matches(ConnectionId.class, this::visitConnectionId),
matches(And.class, this::visitAnd),
matches(Or.class, this::visitOr),
+ matches(TryCast.class, this::visitTryCast),
matches(Cast.class, this::visitCast),
matches(BoundFunction.class, this::visitBoundFunction),
matches(BinaryArithmetic.class, this::visitBinaryArithmetic),
@@ -538,6 +540,15 @@ public class FoldConstantRuleOnFE extends
AbstractExpressionRewriteRule
}
}
+ @Override
+ public Expression visitTryCast(TryCast cast, ExpressionRewriteContext
context) {
+ try {
+ return visitCast(cast, context);
+ } catch (CastException c) {
+ return new NullLiteral(cast.getDataType());
+ }
+ }
+
@Override
public Expression visitBoundFunction(BoundFunction boundFunction,
ExpressionRewriteContext context) {
if (!boundFunction.foldable()) {
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Cast.java
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Cast.java
index dcc90ecbe0e..edff4a5e9d0 100644
---
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Cast.java
+++
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Cast.java
@@ -45,9 +45,9 @@ import java.util.Objects;
public class Cast extends Expression implements UnaryExpression, Monotonic {
// CAST can be from SQL Query or Type Coercion. true for explicitly cast
from SQL query.
- private final boolean isExplicitType; //FIXME: now not useful
+ protected final boolean isExplicitType; //FIXME: now not useful
- private final DataType targetType;
+ protected final DataType targetType;
public Cast(Expression child, DataType targetType) {
this(child, targetType, false);
@@ -57,7 +57,7 @@ public class Cast extends Expression implements
UnaryExpression, Monotonic {
this(ImmutableList.of(child), targetType, isExplicitType);
}
- private Cast(List<Expression> child, DataType targetType, boolean
isExplicitType) {
+ protected Cast(List<Expression> child, DataType targetType, boolean
isExplicitType) {
super(child);
this.targetType = Objects.requireNonNull(targetType, "targetType can
not be null");
this.isExplicitType = isExplicitType;
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/TryCast.java
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/TryCast.java
new file mode 100644
index 00000000000..0d61f91a182
--- /dev/null
+++
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/TryCast.java
@@ -0,0 +1,99 @@
+// 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;
+
+import org.apache.doris.nereids.exceptions.UnboundException;
+import org.apache.doris.nereids.trees.expressions.functions.AlwaysNullable;
+import org.apache.doris.nereids.trees.expressions.functions.Monotonic;
+import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression;
+import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
+import org.apache.doris.nereids.types.DataType;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+import java.util.Objects;
+
+/**
+ * cast function.
+ */
+public class TryCast extends Cast implements UnaryExpression, Monotonic,
AlwaysNullable {
+
+ public TryCast(Expression child, DataType targetType) {
+ this(child, targetType, false);
+ }
+
+ public TryCast(Expression child, DataType targetType, boolean
isExplicitType) {
+ this(ImmutableList.of(child), targetType, isExplicitType);
+ }
+
+ private TryCast(List<Expression> child, DataType targetType, boolean
isExplicitType) {
+ super(child, targetType, isExplicitType);
+ }
+
+ @Override
+ public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+ return visitor.visitTryCast(this, context);
+ }
+
+ @Override
+ public boolean nullable() {
+ return true;
+ }
+
+ public boolean parentNullable() {
+ return super.nullable();
+ }
+
+ @Override
+ public TryCast withChildren(List<Expression> children) {
+ Preconditions.checkArgument(children.size() == 1);
+ return new TryCast(children, targetType, isExplicitType);
+ }
+
+ @Override
+ public String computeToSql() throws UnboundException {
+ return "tryCast(" + child().toSql() + " as " + targetType.toSql() +
")";
+ }
+
+ @Override
+ public String toString() {
+ return "tryCast(" + child() + " as " + targetType + ")";
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (!super.equals(o)) {
+ return false;
+ }
+ TryCast cast = (TryCast) o;
+ return Objects.equals(targetType, cast.targetType);
+ }
+
+ @Override
+ public int computeHashCode() {
+ return Objects.hash(super.computeHashCode() + 1, targetType);
+ }
+
+ @Override
+ public Expression withConstantArgs(Expression literal) {
+ return new TryCast(literal, targetType, isExplicitType);
+ }
+
+}
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ExpressionVisitor.java
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ExpressionVisitor.java
index 05c5243b2b4..f54ed7ca529 100644
---
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ExpressionVisitor.java
+++
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ExpressionVisitor.java
@@ -76,6 +76,7 @@ import
org.apache.doris.nereids.trees.expressions.SlotReference;
import org.apache.doris.nereids.trees.expressions.SubqueryExpr;
import org.apache.doris.nereids.trees.expressions.Subtract;
import org.apache.doris.nereids.trees.expressions.TimestampArithmetic;
+import org.apache.doris.nereids.trees.expressions.TryCast;
import org.apache.doris.nereids.trees.expressions.UnaryArithmetic;
import org.apache.doris.nereids.trees.expressions.UnaryOperator;
import org.apache.doris.nereids.trees.expressions.Variable;
@@ -363,6 +364,11 @@ public abstract class ExpressionVisitor<R, C>
return visit(cast, context);
}
+ // By default, use visitCast for TryCast.
+ public R visitTryCast(TryCast tryCast, C context) {
+ return visitCast(tryCast, context);
+ }
+
public R visitUnaryArithmetic(UnaryArithmetic unaryArithmetic, C context) {
return visitUnaryOperator(unaryArithmetic, context);
}
diff --git
a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/TryCastTest.java
b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/TryCastTest.java
new file mode 100644
index 00000000000..092e083d722
--- /dev/null
+++
b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/TryCastTest.java
@@ -0,0 +1,377 @@
+// 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;
+
+import org.apache.doris.nereids.types.ArrayType;
+import org.apache.doris.nereids.types.BigIntType;
+import org.apache.doris.nereids.types.BooleanType;
+import org.apache.doris.nereids.types.CharType;
+import org.apache.doris.nereids.types.DateTimeType;
+import org.apache.doris.nereids.types.DateType;
+import org.apache.doris.nereids.types.DecimalV2Type;
+import org.apache.doris.nereids.types.DecimalV3Type;
+import org.apache.doris.nereids.types.DoubleType;
+import org.apache.doris.nereids.types.FloatType;
+import org.apache.doris.nereids.types.IntegerType;
+import org.apache.doris.nereids.types.JsonType;
+import org.apache.doris.nereids.types.LargeIntType;
+import org.apache.doris.nereids.types.SmallIntType;
+import org.apache.doris.nereids.types.StringType;
+import org.apache.doris.nereids.types.TimeV2Type;
+import org.apache.doris.nereids.types.TinyIntType;
+import org.apache.doris.nereids.types.VarcharType;
+import org.apache.doris.qe.ConnectContext;
+import org.apache.doris.qe.SessionVariable;
+
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.mockito.MockedStatic;
+import org.mockito.Mockito;
+
+public class TryCastTest {
+
+ @Test
+ public void testTryCastFromBoolean() {
+ try (MockedStatic<SessionVariable> mockedSessionVariable =
Mockito.mockStatic(SessionVariable.class)) {
+ // When strict mode is true, return child.nullable.
+
mockedSessionVariable.when(SessionVariable::enableStrictCast).thenReturn(true);
+ SlotReference child = new SlotReference("slot",
BooleanType.INSTANCE, false);
+ TryCast cast = new TryCast(child, TinyIntType.INSTANCE);
+ Assertions.assertTrue(cast.nullable());
+ Assertions.assertFalse(cast.parentNullable());
+ child = new SlotReference("slot", BooleanType.INSTANCE, true);
+ cast = new TryCast(child, TinyIntType.INSTANCE);
+ Assertions.assertTrue(cast.nullable());
+ Assertions.assertTrue(cast.parentNullable());
+
+ // When strict mode is false, return nullable when decimal range <
1
+
mockedSessionVariable.when(SessionVariable::enableStrictCast).thenReturn(false);
+ child = new SlotReference("slot", BooleanType.INSTANCE, false);
+ cast = new TryCast(child, DecimalV2Type.createDecimalV2Type(2, 1));
+ Assertions.assertTrue(cast.nullable());
+ Assertions.assertFalse(cast.parentNullable());
+ }
+ }
+
+ @Test
+ public void testTryCastFromTinyInt() {
+ try (MockedStatic<SessionVariable> mockedSessionVariable =
Mockito.mockStatic(SessionVariable.class)) {
+ // When strict mode is true, return child.nullable.
+
mockedSessionVariable.when(SessionVariable::enableStrictCast).thenReturn(true);
+ SlotReference child = new SlotReference("slot",
TinyIntType.INSTANCE, false);
+ TryCast cast = new TryCast(child, BooleanType.INSTANCE);
+ Assertions.assertTrue(cast.nullable());
+ Assertions.assertFalse(cast.parentNullable());
+
+ // When strict mode is false, return nullable when decimal range <
1
+
mockedSessionVariable.when(SessionVariable::enableStrictCast).thenReturn(false);
+ child = new SlotReference("slot", TinyIntType.INSTANCE, false);
+ cast = new TryCast(child, DecimalV2Type.createDecimalV2Type(4, 1));
+ Assertions.assertTrue(cast.nullable());
+ Assertions.assertFalse(cast.parentNullable());
+ }
+ }
+
+ @Test
+ public void testTryCastFromSmallInt() {
+ try (MockedStatic<SessionVariable> mockedSessionVariable =
Mockito.mockStatic(SessionVariable.class)) {
+ // When strict mode is true, return child.nullable.
+
mockedSessionVariable.when(SessionVariable::enableStrictCast).thenReturn(true);
+ SlotReference child = new SlotReference("slot",
SmallIntType.INSTANCE, false);
+ TryCast cast = new TryCast(child, BooleanType.INSTANCE);
+ Assertions.assertTrue(cast.nullable());
+ Assertions.assertFalse(cast.parentNullable());
+ child = new SlotReference("slot", SmallIntType.INSTANCE, true);
+ cast = new TryCast(child, BooleanType.INSTANCE);
+ Assertions.assertTrue(cast.nullable());
+ Assertions.assertTrue(cast.parentNullable());
+
+ // When strict mode is false, return nullable when decimal range <
1
+
mockedSessionVariable.when(SessionVariable::enableStrictCast).thenReturn(false);
+ child = new SlotReference("slot", SmallIntType.INSTANCE, false);
+ cast = new TryCast(child, DecimalV2Type.createDecimalV2Type(6, 1));
+ Assertions.assertTrue(cast.nullable());
+ Assertions.assertFalse(cast.parentNullable());
+ cast = new TryCast(child, DecimalV2Type.createDecimalV2Type(6, 2));
+ Assertions.assertTrue(cast.nullable());
+ Assertions.assertTrue(cast.parentNullable());
+ }
+ }
+
+ @Test
+ public void testTryCastFromInteger() {
+ try (MockedStatic<SessionVariable> mockedSessionVariable =
Mockito.mockStatic(SessionVariable.class)) {
+ // When strict mode is true, return child.nullable.
+
mockedSessionVariable.when(SessionVariable::enableStrictCast).thenReturn(true);
+ SlotReference child = new SlotReference("slot",
IntegerType.INSTANCE, false);
+ TryCast cast = new TryCast(child, BooleanType.INSTANCE);
+ Assertions.assertTrue(cast.nullable());
+ Assertions.assertFalse(cast.parentNullable());
+ child = new SlotReference("slot", IntegerType.INSTANCE, true);
+ cast = new TryCast(child, IntegerType.INSTANCE);
+ Assertions.assertTrue(cast.nullable());
+ Assertions.assertTrue(cast.parentNullable());
+
+ // When strict mode is false, return nullable when decimal range <
1
+
mockedSessionVariable.when(SessionVariable::enableStrictCast).thenReturn(false);
+ child = new SlotReference("slot", IntegerType.INSTANCE, false);
+ cast = new TryCast(child, DecimalV2Type.createDecimalV2Type(11,
1));
+ Assertions.assertTrue(cast.nullable());
+ Assertions.assertFalse(cast.parentNullable());
+ cast = new TryCast(child, DecimalV2Type.createDecimalV2Type(11,
2));
+ Assertions.assertTrue(cast.nullable());
+ Assertions.assertTrue(cast.parentNullable());
+ }
+ }
+
+ @Test
+ public void testTryCastFromBigInt() {
+ try (MockedStatic<SessionVariable> mockedSessionVariable =
Mockito.mockStatic(SessionVariable.class)) {
+ // When strict mode is true, return child.nullable.
+
mockedSessionVariable.when(SessionVariable::enableStrictCast).thenReturn(true);
+ SlotReference child = new SlotReference("slot",
BigIntType.INSTANCE, false);
+ TryCast cast = new TryCast(child, BooleanType.INSTANCE);
+ Assertions.assertTrue(cast.nullable());
+ Assertions.assertFalse(cast.parentNullable());
+ child = new SlotReference("slot", BigIntType.INSTANCE, true);
+ cast = new TryCast(child, LargeIntType.INSTANCE);
+ Assertions.assertTrue(cast.nullable());
+ Assertions.assertTrue(cast.parentNullable());
+
+ // When strict mode is false, return nullable when decimal range <
1
+
mockedSessionVariable.when(SessionVariable::enableStrictCast).thenReturn(false);
+ child = new SlotReference("slot", BigIntType.INSTANCE, false);
+ cast = new TryCast(child, DecimalV2Type.createDecimalV2Type(20,
1));
+ Assertions.assertTrue(cast.nullable());
+ Assertions.assertFalse(cast.parentNullable());
+ cast = new TryCast(child, DecimalV2Type.createDecimalV2Type(20,
2));
+ Assertions.assertTrue(cast.nullable());
+ Assertions.assertTrue(cast.parentNullable());
+ }
+ }
+
+ @Test
+ public void testTryCastFromLargeInt() {
+ try (MockedStatic<SessionVariable> mockedSessionVariable =
Mockito.mockStatic(SessionVariable.class);
+ MockedStatic<ConnectContext> mockedContext =
Mockito.mockStatic(ConnectContext.class)) {
+ // When strict mode is true, return child.nullable.
+
mockedSessionVariable.when(SessionVariable::enableStrictCast).thenReturn(true);
+ SlotReference child = new SlotReference("slot",
LargeIntType.INSTANCE, false);
+ TryCast cast = new TryCast(child, BooleanType.INSTANCE);
+ Assertions.assertTrue(cast.nullable());
+ Assertions.assertFalse(cast.parentNullable());
+ child = new SlotReference("slot", LargeIntType.INSTANCE, true);
+ cast = new TryCast(child, DoubleType.INSTANCE);
+ Assertions.assertTrue(cast.nullable());
+ Assertions.assertTrue(cast.parentNullable());
+
+ // When strict mode is false, return nullable when decimal range <
1
+
mockedSessionVariable.when(SessionVariable::enableStrictCast).thenReturn(false);
+ ConnectContext context = new ConnectContext();
+ context.getSessionVariable().enableDecimal256 = true;
+ mockedContext.when(ConnectContext::get).thenReturn(context);
+ child = new SlotReference("slot", LargeIntType.INSTANCE, false);
+ cast = new TryCast(child, DecimalV3Type.createDecimalV3Type(40,
1));
+ Assertions.assertTrue(cast.nullable());
+ Assertions.assertFalse(cast.parentNullable());
+ cast = new TryCast(child, DecimalV3Type.createDecimalV3Type(40,
2));
+ Assertions.assertTrue(cast.nullable());
+ Assertions.assertTrue(cast.parentNullable());
+ }
+ }
+
+ @Test
+ public void testTryCastFromFloat() {
+ try (MockedStatic<SessionVariable> mockedSessionVariable =
Mockito.mockStatic(SessionVariable.class)) {
+ // When strict mode is true, return child.nullable.
+
mockedSessionVariable.when(SessionVariable::enableStrictCast).thenReturn(true);
+ SlotReference child = new SlotReference("slot",
FloatType.INSTANCE, false);
+ TryCast cast = new TryCast(child, BooleanType.INSTANCE);
+ Assertions.assertTrue(cast.nullable());
+ Assertions.assertFalse(cast.parentNullable());
+ child = new SlotReference("slot", FloatType.INSTANCE, true);
+ cast = new TryCast(child, DoubleType.INSTANCE);
+ Assertions.assertTrue(cast.nullable());
+
+ // When strict mode is false, return nullable when decimal range <
1
+
mockedSessionVariable.when(SessionVariable::enableStrictCast).thenReturn(false);
+ child = new SlotReference("slot", FloatType.INSTANCE, false);
+ // To date is always nullable
+ cast = new TryCast(child, DateType.INSTANCE);
+ Assertions.assertTrue(cast.nullable());
+ Assertions.assertTrue(cast.parentNullable());
+ }
+ }
+
+ @Test
+ public void testTryCastFromDouble() {
+ try (MockedStatic<SessionVariable> mockedSessionVariable =
Mockito.mockStatic(SessionVariable.class)) {
+ // When strict mode is true, return child.nullable.
+
mockedSessionVariable.when(SessionVariable::enableStrictCast).thenReturn(true);
+ SlotReference child = new SlotReference("slot",
DoubleType.INSTANCE, false);
+ TryCast cast = new TryCast(child, BooleanType.INSTANCE);
+ Assertions.assertTrue(cast.nullable());
+ Assertions.assertFalse(cast.parentNullable());
+ child = new SlotReference("slot", DoubleType.INSTANCE, true);
+ cast = new TryCast(child, DoubleType.INSTANCE);
+ Assertions.assertTrue(cast.nullable());
+ Assertions.assertTrue(cast.parentNullable());
+
+ // When strict mode is false, return nullable when decimal range <
1
+
mockedSessionVariable.when(SessionVariable::enableStrictCast).thenReturn(false);
+ child = new SlotReference("slot", DoubleType.INSTANCE, false);
+ // To date is always nullable
+ cast = new TryCast(child, DateType.INSTANCE);
+ Assertions.assertTrue(cast.nullable());
+ Assertions.assertTrue(cast.parentNullable());
+ }
+ }
+
+ @Test
+ public void testTryCastFromDecimal() {
+ try (MockedStatic<SessionVariable> mockedSessionVariable =
Mockito.mockStatic(SessionVariable.class);
+ MockedStatic<ConnectContext> mockedContext =
Mockito.mockStatic(ConnectContext.class)) {
+ // When strict mode is true, return child.nullable.
+
mockedSessionVariable.when(SessionVariable::enableStrictCast).thenReturn(true);
+ SlotReference child = new SlotReference("slot",
DecimalV2Type.SYSTEM_DEFAULT, false);
+ TryCast cast = new TryCast(child, BooleanType.INSTANCE);
+ Assertions.assertTrue(cast.nullable());
+ Assertions.assertFalse(cast.parentNullable());
+ child = new SlotReference("slot", DecimalV2Type.SYSTEM_DEFAULT,
true);
+ cast = new TryCast(child, DoubleType.INSTANCE);
+ Assertions.assertTrue(cast.nullable());
+ Assertions.assertTrue(cast.parentNullable());
+
+ // When strict mode is false, return nullable when decimal range <
1
+
mockedSessionVariable.when(SessionVariable::enableStrictCast).thenReturn(false);
+ ConnectContext context = new ConnectContext();
+ context.getSessionVariable().enableDecimal256 = true;
+ mockedContext.when(ConnectContext::get).thenReturn(context);
+ // To integer
+ child = new SlotReference("slot",
DecimalV2Type.createDecimalV2Type(4, 2), false);
+ cast = new TryCast(child, TinyIntType.INSTANCE);
+ Assertions.assertTrue(cast.nullable());
+ Assertions.assertFalse(cast.parentNullable());
+ }
+ }
+
+ @Test
+ public void testTryCastFromDatetime() {
+ try (MockedStatic<SessionVariable> mockedSessionVariable =
Mockito.mockStatic(SessionVariable.class)) {
+ // When strict mode is false
+
mockedSessionVariable.when(SessionVariable::enableStrictCast).thenReturn(false);
+ SlotReference child = new SlotReference("slot",
DateTimeType.INSTANCE, false);
+ TryCast cast = new TryCast(child, DateTimeType.INSTANCE);
+ Assertions.assertTrue(cast.nullable());
+ Assertions.assertTrue(cast.parentNullable());
+ }
+ }
+
+ @Test
+ public void testTryCastFromTime() {
+ try (MockedStatic<SessionVariable> mockedSessionVariable =
Mockito.mockStatic(SessionVariable.class)) {
+ // When strict mode is false
+
mockedSessionVariable.when(SessionVariable::enableStrictCast).thenReturn(false);
+ SlotReference child = new SlotReference("slot",
TimeV2Type.INSTANCE, false);
+ TryCast cast = new TryCast(child, TinyIntType.INSTANCE);
+ Assertions.assertTrue(cast.nullable());
+ Assertions.assertTrue(cast.parentNullable());
+ }
+ }
+
+ @Test
+ public void testTryCastFromString() {
+ try (MockedStatic<SessionVariable> mockedSessionVariable =
Mockito.mockStatic(SessionVariable.class)) {
+ // When strict mode is false, all nullable
+
mockedSessionVariable.when(SessionVariable::enableStrictCast).thenReturn(false);
+ SlotReference child = new SlotReference("slot",
StringType.INSTANCE, false);
+ TryCast cast = new TryCast(child, BooleanType.INSTANCE);
+ Assertions.assertTrue(cast.nullable());
+ Assertions.assertTrue(cast.parentNullable());
+ }
+ }
+
+ @Test
+ public void testTryCastFromChar() {
+ try (MockedStatic<SessionVariable> mockedSessionVariable =
Mockito.mockStatic(SessionVariable.class)) {
+ // When strict mode is false, all nullable
+
mockedSessionVariable.when(SessionVariable::enableStrictCast).thenReturn(false);
+ SlotReference child = new SlotReference("slot",
CharType.SYSTEM_DEFAULT, false);
+ TryCast cast = new TryCast(child, BooleanType.INSTANCE);
+ Assertions.assertTrue(cast.nullable());
+ Assertions.assertTrue(cast.parentNullable());
+ }
+ }
+
+ @Test
+ public void testTryCastFromVarchar() {
+ try (MockedStatic<SessionVariable> mockedSessionVariable =
Mockito.mockStatic(SessionVariable.class)) {
+ // When strict mode is false, all nullable
+
mockedSessionVariable.when(SessionVariable::enableStrictCast).thenReturn(false);
+ SlotReference child = new SlotReference("slot",
VarcharType.MAX_VARCHAR_TYPE, false);
+ TryCast cast = new TryCast(child, BooleanType.INSTANCE);
+ Assertions.assertTrue(cast.nullable());
+ Assertions.assertTrue(cast.parentNullable());
+ }
+ }
+
+ @Test
+ public void testTryCastFromJson() {
+ try (MockedStatic<SessionVariable> mockedSessionVariable =
Mockito.mockStatic(SessionVariable.class)) {
+ // When strict mode is true, always nullable. to Json is PN
+
mockedSessionVariable.when(SessionVariable::enableStrictCast).thenReturn(true);
+ SlotReference child = new SlotReference("slot", JsonType.INSTANCE,
false);
+ TryCast cast = new TryCast(child, BooleanType.INSTANCE);
+ Assertions.assertTrue(cast.nullable());
+ Assertions.assertTrue(cast.parentNullable());
+
+ // When strict mode is false, always nullable. to Json is PN
+
mockedSessionVariable.when(SessionVariable::enableStrictCast).thenReturn(false);
+ child = new SlotReference("slot", JsonType.INSTANCE, false);
+ cast = new TryCast(child, BooleanType.INSTANCE);
+ Assertions.assertTrue(cast.nullable());
+ Assertions.assertTrue(cast.parentNullable());
+ }
+ }
+
+ @Test
+ public void testTryCastFromArray() {
+ try (MockedStatic<SessionVariable> mockedSessionVariable =
Mockito.mockStatic(SessionVariable.class)) {
+
mockedSessionVariable.when(SessionVariable::enableStrictCast).thenReturn(true);
+ SlotReference child = new SlotReference("slot",
ArrayType.SYSTEM_DEFAULT, false);
+ TryCast cast = new TryCast(child, ArrayType.SYSTEM_DEFAULT);
+ Assertions.assertTrue(cast.nullable());
+ Assertions.assertFalse(cast.parentNullable());
+ child = new SlotReference("slot", ArrayType.SYSTEM_DEFAULT, true);
+ cast = new TryCast(child, ArrayType.SYSTEM_DEFAULT);
+ Assertions.assertTrue(cast.nullable());
+ Assertions.assertTrue(cast.parentNullable());
+
+ // When strict mode is false, always nullable. to Json is PN
+
mockedSessionVariable.when(SessionVariable::enableStrictCast).thenReturn(false);
+ child = new SlotReference("slot", ArrayType.SYSTEM_DEFAULT, false);
+ cast = new TryCast(child, ArrayType.SYSTEM_DEFAULT);
+ Assertions.assertTrue(cast.nullable());
+ Assertions.assertFalse(cast.parentNullable());
+ child = new SlotReference("slot", ArrayType.SYSTEM_DEFAULT, true);
+ cast = new TryCast(child, ArrayType.SYSTEM_DEFAULT);
+ Assertions.assertTrue(cast.nullable());
+ Assertions.assertTrue(cast.parentNullable());
+ }
+ }
+}
diff --git a/gensrc/thrift/Exprs.thrift b/gensrc/thrift/Exprs.thrift
index 68f7b2b6b6d..64b4fe78f57 100644
--- a/gensrc/thrift/Exprs.thrift
+++ b/gensrc/thrift/Exprs.thrift
@@ -84,7 +84,8 @@ enum TExprNodeType {
NULL_AWARE_BINARY_PRED = 37,
TIMEV2_LITERAL = 38,
VIRTUAL_SLOT_REF = 39,
- VARBINARY_LITERAL = 40
+ VARBINARY_LITERAL = 40,
+ TRY_CAST_EXPR = 41
}
//enum TAggregationOp {
@@ -283,6 +284,7 @@ struct TExprNode {
36: optional string label // alias name, a/b in `select xxx as a, count(1)
as b`
37: optional TTimeV2Literal timev2_literal
38: optional TVarBinaryLiteral varbinary_literal
+ 39: optional bool is_cast_nullable
}
// A flattened representation of a tree of Expr nodes, obtained by depth-first
diff --git a/gensrc/thrift/Opcodes.thrift b/gensrc/thrift/Opcodes.thrift
index d5910a23b5f..1e4002357e7 100644
--- a/gensrc/thrift/Opcodes.thrift
+++ b/gensrc/thrift/Opcodes.thrift
@@ -96,4 +96,5 @@ enum TExprOpcode {
MATCH_PHRASE_PREFIX = 75,
MATCH_REGEXP = 76,
MATCH_PHRASE_EDGE = 77,
+ TRY_CAST = 78,
}
diff --git a/regression-test/data/function_p0/cast/test_try_cast.out
b/regression-test/data/function_p0/cast/test_try_cast.out
new file mode 100644
index 00000000000..eba23fc04c7
--- /dev/null
+++ b/regression-test/data/function_p0/cast/test_try_cast.out
@@ -0,0 +1,21 @@
+-- This file is automatically generated. You should know what you did if you
want to edit this
+-- !sql --
+1 123 123 123
+2 abc \N \N
+3 \N \N \N
+
+-- !sql --
+\N
+
+-- !sql --
+123
+
+-- !sql --
+\N
+
+-- !sql --
+\N
+
+-- !sql --
+\N
+
diff --git
a/regression-test/data/nereids_p0/expression/fold_constant/fe_try_cast.out
b/regression-test/data/nereids_p0/expression/fold_constant/fe_try_cast.out
new file mode 100644
index 00000000000..7f0fdd7221f
--- /dev/null
+++ b/regression-test/data/nereids_p0/expression/fold_constant/fe_try_cast.out
@@ -0,0 +1,13 @@
+-- This file is automatically generated. You should know what you did if you
want to edit this
+-- !datetime1 --
+2023-07-16T19:20:30
+
+-- !datetime2 --
+\N
+
+-- !datetime3 --
+\N
+
+-- !datetime4 --
+\N
+
diff --git a/regression-test/suites/function_p0/cast/test_try_cast.groovy
b/regression-test/suites/function_p0/cast/test_try_cast.groovy
new file mode 100644
index 00000000000..d0cd1d3b9db
--- /dev/null
+++ b/regression-test/suites/function_p0/cast/test_try_cast.groovy
@@ -0,0 +1,102 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License. You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied. See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+
+suite("test_try_cast") {
+ // Test casting from integer types to boolean
+ sql "set debug_skip_fold_constant=true;"
+
+ sql "set enable_strict_cast=true;"
+
+ sql """
+ Drop table if exists test_try_cast;
+ """
+
+ sql """
+ CREATE TABLE test_try_cast (id INT, str string , not_null_str string
not null) DISTRIBUTED BY HASH(id) BUCKETS 4 PROPERTIES ("replication_num" =
"1");
+ """
+
+
+ sql """
+ insert into test_try_cast values (1, '123', '123'), (2, 'abc', 'abc'),
(3, null, 'not null');
+ """
+
+
+ qt_sql """
+ select id, str, try_cast(str as int) , try_cast(not_null_str as int)
from test_try_cast order by id;
+ """
+
+
+ qt_sql """
+ select try_cast('abc' as int);
+ """
+
+
+ qt_sql """
+ select try_cast('123' as int);
+ """
+
+
+ test {
+ sql """select cast('abc' as int);"""
+ exception "fail"
+ }
+
+
+
+ test {
+ sql """select cast('[[[]]]' as array<int>);"""
+ exception "fail"
+ }
+
+ qt_sql """
+ select try_cast('[[[]]]' as array<int>);
+ """
+
+
+ test {
+ sql """select cast('{123:456}' as json);"""
+ exception "Failed to parse json string"
+ }
+
+ qt_sql """
+ select try_cast('{123:456}' as json);
+ """
+
+
+ test {
+ sql """select cast(12345 as tinyint);"""
+ exception "Value 12345 out of range for type tinyint"
+ }
+
+ qt_sql """
+ select try_cast(12345 as tinyint);
+ """
+
+
+ test {
+ sql """select cast(array(1) as map<int,int>);"""
+ exception "can not cast from origin type ARRAY<TINYINT> to target
type=MAP<INT,INT>"
+ }
+
+
+ test {
+ sql """select try_cast(array(1) as map<int,int>);"""
+ exception "can not cast from origin type ARRAY<TINYINT> to target
type=MAP<INT,INT>"
+ }
+
+}
\ No newline at end of file
diff --git
a/regression-test/suites/nereids_p0/expression/fold_constant/fe_try_cast.groovy
b/regression-test/suites/nereids_p0/expression/fold_constant/fe_try_cast.groovy
new file mode 100644
index 00000000000..241a3ddafaf
--- /dev/null
+++
b/regression-test/suites/nereids_p0/expression/fold_constant/fe_try_cast.groovy
@@ -0,0 +1,52 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License. You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied. See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+suite("fe_try_cast") {
+ sql """set enable_fallback_to_original_planner=false"""
+ sql """set debug_skip_fold_constant=false"""
+ sql """set enable_strict_cast=true"""
+
+ qt_datetime1("""select try_cast("2023-07-16T19:20:30.123+08:00" as
datetime)""")
+ explain {
+ sql("select try_cast(\"2023-07-16T19:20:30.123+08:00\" as datetime)")
+ contains "2023-07-16 19:20:30"
+ notContains("TRYCAST")
+ }
+
+ qt_datetime2("""select try_cast("abc" as datetime)""")
+ explain {
+ sql("select try_cast(\"abc\" as datetime)")
+ contains "NULL"
+ }
+
+ qt_datetime3("""select try_cast(1000000 as tinyint);""")
+ explain {
+ sql("select try_cast(1000000 as tinyint);")
+ contains "NULL"
+ }
+
+ qt_datetime4("""select try_cast(123.456 as decimal(4, 2));""")
+ explain {
+ sql("select try_cast(123.456 as decimal(4, 2))")
+ contains "NULL"
+ }
+
+ test {
+ sql """select cast(true as date)"""
+ exception "cannot cast BOOLEAN to DATEV2"
+ }
+}
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]