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]

Reply via email to