[Feature](func) Support GET_FORAMT function (#56691)
```text
mysql> SELECT * FROM get_format_test;
+------+----------+
| id | lc |
+------+----------+
| 1 | USA |
| 2 | JIS |
| 3 | ISO |
| 4 | EUR |
| 5 | INTERNAL |
| 6 | Doris |
+------+----------+
mysql> SELECT lc, GET_FORMAT(DATE, lc) FROM get_format_test;
+----------+----------------------+
| lc | GET_FORMAT(DATE, lc) |
+----------+----------------------+
| USA | %m.%d.%Y |
| JIS | %Y-%m-%d |
| ISO | %Y-%m-%d |
| EUR | %d.%m.%Y |
| INTERNAL | %Y%m%d |
| Doris | NULL |
+----------+----------------------+
mysql> SELECT lc, GET_FORMAT(DATETIME, lc) FROM get_format_test;
+----------+--------------------------+
| lc | GET_FORMAT(DATETIME, lc) |
+----------+--------------------------+
| USA | %Y-%m-%d %H.%i.%s |
| JIS | %Y-%m-%d %H:%i:%s |
| ISO | %Y-%m-%d %H:%i:%s |
| EUR | %Y-%m-%d %H.%i.%s |
| INTERNAL | %Y%m%d%H%i%s |
| Doris | NULL |
+----------+--------------------------+
mysql> SELECT lc, GET_FORMAT(TIME, lc) FROM get_format_test;
+----------+----------------------+
| lc | GET_FORMAT(TIME, lc) |
+----------+----------------------+
| USA | %h:%i:%s %p |
| JIS | %H:%i:%s |
| ISO | %H:%i:%s |
| EUR | %H.%i.%s |
| INTERNAL | %H%i%s |
| Doris | NULL |
+----------+----------------------+
mysql> SELECT GET_FORMAT(DORIS, 'USA');
ERROR 1105 (HY000): errCode = 2, detailMessage = Format type only support DATE, DATETIME and TIME, but get: DORIS
```
diff --git a/be/src/vec/functions/function_date_or_datetime_computation.cpp b/be/src/vec/functions/function_date_or_datetime_computation.cpp
index e7489d0..f9f2db9 100644
--- a/be/src/vec/functions/function_date_or_datetime_computation.cpp
+++ b/be/src/vec/functions/function_date_or_datetime_computation.cpp
@@ -93,6 +93,7 @@
factory.register_function<FunctionSecToDateTime>();
factory.register_function<FunctionMonthsBetween>();
factory.register_function<FunctionTime>();
+ factory.register_function<FunctionGetFormat>();
// alias
factory.register_alias("days_add", "date_add");
diff --git a/be/src/vec/functions/function_date_or_datetime_computation.h b/be/src/vec/functions/function_date_or_datetime_computation.h
index 65c116b..cc22393 100644
--- a/be/src/vec/functions/function_date_or_datetime_computation.h
+++ b/be/src/vec/functions/function_date_or_datetime_computation.h
@@ -1267,5 +1267,116 @@
return Status::OK();
}
};
+
+class FunctionGetFormat : public IFunction {
+public:
+ static constexpr auto name = "get_format";
+ static FunctionPtr create() { return std::make_shared<FunctionGetFormat>(); }
+ String get_name() const override { return name; }
+ size_t get_number_of_arguments() const override { return 2; }
+ DataTypePtr get_return_type_impl(const DataTypes& arguments) const override {
+ return make_nullable(std::make_shared<DataTypeString>());
+ }
+
+ Status execute_impl(FunctionContext* context, Block& block, const ColumnNumbers& arguments,
+ uint32_t result, size_t input_rows_count) const override {
+ const auto& [left_col_ptr, left_is_const] =
+ unpack_if_const(block.get_by_position(arguments[0]).column);
+ const auto& [right_col_ptr, right_is_const] =
+ unpack_if_const(block.get_by_position(arguments[1]).column);
+
+ const auto* left_col = assert_cast<const ColumnString*>(left_col_ptr.get());
+ const auto* right_col = assert_cast<const ColumnString*>(right_col_ptr.get());
+
+ auto type_ref = left_col->get_data_at(0);
+ std::string type_str(type_ref.data, type_ref.size);
+
+ auto res_col = ColumnString::create();
+ auto res_null_map = ColumnUInt8::create(input_rows_count, 0);
+ auto& res_data = res_col->get_chars();
+ auto& res_offsets = res_col->get_offsets();
+
+ if (type_str == DATE_NAME) {
+ execute_format_type<DateFormatImpl>(res_data, res_offsets, res_null_map->get_data(),
+ input_rows_count, right_col);
+ } else if (type_str == DATETIME_NAME) {
+ execute_format_type<DateTimeFormatImpl>(res_data, res_offsets, res_null_map->get_data(),
+ input_rows_count, right_col);
+ } else if (type_str == TIME_NAME) {
+ execute_format_type<TimeFormatImpl>(res_data, res_offsets, res_null_map->get_data(),
+ input_rows_count, right_col);
+ } else {
+ return Status::InvalidArgument(
+ "Function GET_FORMAT only support DATE, DATETIME or TIME");
+ }
+
+ block.replace_by_position(
+ result, ColumnNullable::create(std::move(res_col), std::move(res_null_map)));
+ return Status::OK();
+ }
+
+private:
+ template <typename Impl>
+ static void execute_format_type(ColumnString::Chars& res_data,
+ ColumnString::Offsets& res_offsets,
+ PaddedPODArray<UInt8>& res_null_map, size_t input_rows_count,
+ const ColumnString* right_col) {
+ res_data.reserve(input_rows_count * Impl::ESTIMATE_SIZE);
+ res_offsets.reserve(input_rows_count);
+
+ for (int i = 0; i < input_rows_count; ++i) {
+ StringRef format_ref = right_col->get_data_at(i);
+ std::string format_str(format_ref.data, format_ref.size);
+ std::transform(format_str.begin(), format_str.end(), format_str.begin(), ::toupper);
+
+ std::string_view format_res;
+ if (format_str == "USA") {
+ format_res = Impl::USA;
+ } else if (format_str == "JIS" || format_str == "ISO") {
+ format_res = Impl::JIS_ISO;
+ } else if (format_str == "EUR") {
+ format_res = Impl::EUR;
+ } else if (format_str == "INTERNAL") {
+ format_res = Impl::INTERNAL;
+ } else {
+ res_null_map[i] = 1;
+ res_offsets.push_back(res_data.size());
+ continue;
+ }
+
+ res_data.insert(format_res.data(), format_res.data() + format_res.size());
+ res_offsets.push_back(res_data.size());
+ }
+ }
+
+ struct DateFormatImpl {
+ static constexpr auto USA = "%m.%d.%Y";
+ static constexpr auto JIS_ISO = "%Y-%m-%d";
+ static constexpr auto EUR = "%d.%m.%Y";
+ static constexpr auto INTERNAL = "%Y%m%d";
+ static constexpr size_t ESTIMATE_SIZE = 8;
+ };
+
+ struct DateTimeFormatImpl {
+ static constexpr auto USA = "%Y-%m-%d %H.%i.%s";
+ static constexpr auto JIS_ISO = "%Y-%m-%d %H:%i:%s";
+ static constexpr auto EUR = "%Y-%m-%d %H.%i.%s";
+ static constexpr auto INTERNAL = "%Y%m%d%H%i%s";
+ static constexpr size_t ESTIMATE_SIZE = 17;
+ };
+
+ struct TimeFormatImpl {
+ static constexpr auto USA = "%h:%i:%s %p";
+ static constexpr auto JIS_ISO = "%H:%i:%s";
+ static constexpr auto EUR = "%H.%i.%s";
+ static constexpr auto INTERNAL = "%H%i%s";
+ static constexpr size_t ESTIMATE_SIZE = 11;
+ };
+
+ static constexpr auto DATE_NAME = "DATE";
+ static constexpr auto DATETIME_NAME = "DATETIME";
+ static constexpr auto TIME_NAME = "TIME";
+};
+
#include "common/compile_check_avoid_end.h"
} // namespace doris::vectorized
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 8a485a9..f4e0294 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
@@ -1605,6 +1605,8 @@
(ORDER BY sortItem (COMMA sortItem)*)?
(SEPARATOR sep=expression)? RIGHT_PAREN
(OVER windowSpec)? #groupConcat
+ | GET_FORMAT LEFT_PAREN
+ expression COMMA expression RIGHT_PAREN #getFormatFunction
| TRIM LEFT_PAREN
((BOTH | LEADING | TRAILING) expression? | expression) FROM expression RIGHT_PAREN #trim
| (SUBSTR | SUBSTRING | MID) LEFT_PAREN
@@ -1998,6 +2000,7 @@
| FRONTENDS
| FUNCTION
| GENERATED
+ | GET_FORMAT
| GENERIC
| GLOBAL
| GRAPH
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java
index c4ef1d6..f1c29e6 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinScalarFunctions.java
@@ -219,6 +219,7 @@
import org.apache.doris.nereids.trees.expressions.functions.scalar.FromUnixtime;
import org.apache.doris.nereids.trees.expressions.functions.scalar.G;
import org.apache.doris.nereids.trees.expressions.functions.scalar.Gcd;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.GetFormat;
import org.apache.doris.nereids.trees.expressions.functions.scalar.GetVariantType;
import org.apache.doris.nereids.trees.expressions.functions.scalar.Greatest;
import org.apache.doris.nereids.trees.expressions.functions.scalar.Grouping;
@@ -546,6 +547,16 @@
scalar(Abs.class, "abs"),
scalar(Acos.class, "acos"),
scalar(Acosh.class, "acosh"),
+ scalar(AITranslate.class, "ai_translate"),
+ scalar(AISentiment.class, "ai_sentiment"),
+ scalar(AIFilter.class, "ai_filter"),
+ scalar(AIFixGrammar.class, "ai_fixgrammar"),
+ scalar(AIExtract.class, "ai_extract"),
+ scalar(AIGenerate.class, "ai_generate"),
+ scalar(AIClassify.class, "ai_classify"),
+ scalar(AIMask.class, "ai_mask"),
+ scalar(AISummarize.class, "ai_summarize"),
+ scalar(AISimilarity.class, "ai_similarity"),
scalar(AesDecrypt.class, "aes_decrypt"),
scalar(AesEncrypt.class, "aes_encrypt"),
scalar(AppendTrailingCharIfAbsent.class, "append_trailing_char_if_absent"),
@@ -646,6 +657,7 @@
scalar(Char.class, "char"),
scalar(CharacterLength.class, "char_length", "character_length"),
scalar(Coalesce.class, "coalesce"),
+ scalar(Compress.class, "compress"),
scalar(Concat.class, "concat"),
scalar(ConcatWs.class, "concat_ws"),
scalar(ConnectionId.class, "connection_id"),
@@ -700,6 +712,7 @@
scalar(E.class, "e"),
scalar(ElementAt.class, "element_at"),
scalar(Elt.class, "elt"),
+ scalar(Embed.class, "embed"),
scalar(EncodeAsSmallInt.class, "encode_as_smallint"),
scalar(EncodeAsInt.class, "encode_as_int"),
scalar(EncodeAsBigInt.class, "encode_as_bigint"),
@@ -708,6 +721,7 @@
scalar(EsQuery.class, "esquery"),
scalar(Even.class, "even"),
scalar(Exp.class, "exp"),
+ scalar(ExportSet.class, "export_set"),
scalar(ExtractUrlParameter.class, "extract_url_parameter"),
scalar(Field.class, "field"),
scalar(FindInSet.class, "find_in_set"),
@@ -724,6 +738,7 @@
scalar(FromUnixtime.class, "from_unixtime"),
scalar(G.class, "g"),
scalar(Gcd.class, "gcd"),
+ scalar(GetFormat.class, "get_format"),
scalar(GetVariantType.class, "variant_type"),
scalar(Greatest.class, "greatest"),
scalar(Grouping.class, "grouping"),
@@ -802,6 +817,7 @@
scalar(L2DistanceApproximate.class, "l2_distance_approximate"),
scalar(L2Distance.class, "l2_distance"),
scalar(LastDay.class, "last_day"),
+ scalar(LastQueryId.class, "last_query_id"),
scalar(Lcm.class, "lcm"),
scalar(Least.class, "least"),
scalar(Left.class, "left", "strleft"),
@@ -819,6 +835,7 @@
scalar(Ltrim.class, "ltrim"),
scalar(LtrimIn.class, "ltrim_in"),
scalar(MakeDate.class, "makedate"),
+ scalar(MakeSet.class, "make_set"),
scalar(MapContainsEntry.class, "map_contains_entry"),
scalar(MapContainsKey.class, "map_contains_key"),
scalar(MapContainsValue.class, "map_contains_value"),
@@ -853,6 +870,7 @@
scalar(MonthsBetween.class, "months_between"),
scalar(MonthsDiff.class, "months_diff"),
scalar(MonthsSub.class, "months_sub"),
+ scalar(MultiMatch.class, "multi_match"),
scalar(MultiMatchAny.class, "multi_match_any"),
scalar(MultiSearchAllPositions.class, "multi_search_all_positions"),
scalar(MurmurHash332.class, "murmur_hash3_32"),
@@ -888,6 +906,7 @@
scalar(QuartersDiff.class, "quarters_diff"),
scalar(QuarterFloor.class, "quarter_floor"),
scalar(QuartersSub.class, "quarters_sub"),
+ scalar(Quote.class, "quote"),
scalar(Search.class, "search"),
scalar(Radians.class, "radians"),
scalar(Random.class, "rand", "random"),
@@ -925,6 +944,7 @@
scalar(MilliSecondTimestamp.class, "millisecond_timestamp"),
scalar(MicroSecondTimestamp.class, "microsecond_timestamp"),
scalar(RandomBytes.class, "random_bytes"),
+ scalar(SessionUser.class, "session_user"),
scalar(Sha1.class, "sha1", "sha"),
scalar(Sha2.class, "sha2"),
scalar(Sign.class, "sign"),
@@ -1013,7 +1033,8 @@
scalar(UnhexNull.class, "unhex_null"),
scalar(UnixTimestamp.class, "unix_timestamp"),
scalar(Upper.class, "ucase", "upper"),
- scalar(Quote.class, "quote"),
+ scalar(Uncompress.class, "uncompress"),
+ scalar(Uniform.class, "uniform"),
scalar(UrlDecode.class, "url_decode"),
scalar(UrlEncode.class, "url_encode"),
scalar(User.class, "user"),
@@ -1042,26 +1063,7 @@
scalar(YearWeek.class, "yearweek"),
scalar(YearsAdd.class, "years_add"),
scalar(YearsDiff.class, "years_diff"),
- scalar(YearsSub.class, "years_sub"),
- scalar(MultiMatch.class, "multi_match"),
- scalar(SessionUser.class, "session_user"),
- scalar(LastQueryId.class, "last_query_id"),
- scalar(Compress.class, "compress"),
- scalar(Uncompress.class, "uncompress"),
- scalar(AITranslate.class, "ai_translate"),
- scalar(AISentiment.class, "ai_sentiment"),
- scalar(AIFilter.class, "ai_filter"),
- scalar(AIFixGrammar.class, "ai_fixgrammar"),
- scalar(AIExtract.class, "ai_extract"),
- scalar(AIGenerate.class, "ai_generate"),
- scalar(AIClassify.class, "ai_classify"),
- scalar(AIMask.class, "ai_mask"),
- scalar(AISummarize.class, "ai_summarize"),
- scalar(AISimilarity.class, "ai_similarity"),
- scalar(Embed.class, "embed"),
- scalar(Uniform.class, "uniform"),
- scalar(MakeSet.class, "make_set"),
- scalar(ExportSet.class, "export_set"));
+ scalar(YearsSub.class, "years_sub"));
public static final BuiltinScalarFunctions INSTANCE = new BuiltinScalarFunctions();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ExpressionAnalyzer.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ExpressionAnalyzer.java
index 4b691e1..67a41d2 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ExpressionAnalyzer.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ExpressionAnalyzer.java
@@ -93,6 +93,7 @@
import org.apache.doris.nereids.types.BigIntType;
import org.apache.doris.nereids.types.BooleanType;
import org.apache.doris.nereids.types.DataType;
+import org.apache.doris.nereids.types.StringType;
import org.apache.doris.nereids.types.TinyIntType;
import org.apache.doris.nereids.util.ExpressionUtils;
import org.apache.doris.nereids.util.TypeCoercionUtils;
@@ -404,6 +405,19 @@
newChildrenBuilder.add(unboundFunction.child(i));
}
unboundFunction = unboundFunction.withChildren(newChildrenBuilder.build());
+ } else if (StringUtils.isEmpty(unboundFunction.getDbName())
+ && "get_format".equalsIgnoreCase(unboundFunction.getName())
+ && unboundFunction.arity() == 2
+ && unboundFunction.child(0) instanceof UnboundSlot) {
+ SlotReference slotReference = new SlotReference(new ExprId(-1),
+ ((UnboundSlot) unboundFunction.child(0)).getName(),
+ StringType.INSTANCE, false, ImmutableList.of());
+ ImmutableList.Builder<Expression> newChildrenBuilder = ImmutableList.builder();
+ newChildrenBuilder.add(slotReference);
+ for (int i = 1; i < unboundFunction.arity(); i++) {
+ newChildrenBuilder.add(unboundFunction.child(i));
+ }
+ unboundFunction = unboundFunction.withChildren(newChildrenBuilder.build());
}
unboundFunction = (UnboundFunction) super.visit(unboundFunction, context);
}
@@ -442,6 +456,14 @@
return ret;
}
}
+ if (GetFormatFunctionBinder.isGetFormatFunction(unboundFunction.getName())) {
+ Expression ret = GetFormatFunctionBinder.INSTANCE.bind(unboundFunction);
+ if (ret instanceof BoundFunction) {
+ return TypeCoercionUtils.processBoundFunction((BoundFunction) ret);
+ } else {
+ return ret;
+ }
+ }
if (DatetimeFunctionBinder.isDatetimeFunction(unboundFunction.getName())) {
Expression ret = DatetimeFunctionBinder.INSTANCE.bind(unboundFunction);
if (ret instanceof BoundFunction) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/GetFormatFunctionBinder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/GetFormatFunctionBinder.java
new file mode 100644
index 0000000..b82e0ce
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/GetFormatFunctionBinder.java
@@ -0,0 +1,78 @@
+// 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.rules.analysis;
+
+import org.apache.doris.nereids.analyzer.UnboundFunction;
+import org.apache.doris.nereids.exceptions.AnalysisException;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.trees.expressions.SlotReference;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.GetFormat;
+import org.apache.doris.nereids.trees.expressions.literal.StringLiteral;
+
+import com.google.common.collect.ImmutableSet;
+
+/**
+ * Binder for GET_FORMAT function.
+ */
+public class GetFormatFunctionBinder {
+
+ public static final GetFormatFunctionBinder INSTANCE = new GetFormatFunctionBinder();
+
+ private static final ImmutableSet<String> SUPPORTED_FORMATS = ImmutableSet.of("DATE", "DATETIME", "TIME");
+
+ public static boolean isGetFormatFunction(String functionName) {
+ return "GET_FORMAT".equalsIgnoreCase(functionName);
+ }
+
+ /**
+ * bind get_format function that have non-expression arguments.
+ *
+ * @param unboundFunction unbound get_format function
+ *
+ * @return bound function
+ */
+ public Expression bind(UnboundFunction unboundFunction) {
+ if (unboundFunction.arity() != 2) {
+ throw new AnalysisException("Can not find function 'GET_FORMAT' with "
+ + unboundFunction.arity() + " arguments");
+ }
+ StringLiteral formatLiteral = parseFormatType(unboundFunction.child(0));
+ Expression pattern = unboundFunction.child(1);
+ return new GetFormat(formatLiteral, pattern);
+ }
+
+ private StringLiteral parseFormatType(Expression formatTypeExpr) {
+ if (formatTypeExpr instanceof StringLiteral) {
+ String formatType = ((StringLiteral) formatTypeExpr).getStringValue().toUpperCase();
+ validateFormatType(formatType);
+ return new StringLiteral(formatType);
+ }
+ if (formatTypeExpr instanceof SlotReference) {
+ String formatType = ((SlotReference) formatTypeExpr).getName().toUpperCase();
+ validateFormatType(formatType);
+ return new StringLiteral(formatType);
+ }
+ throw new AnalysisException("Illegal first argument for GET_FORMAT: " + formatTypeExpr.toSql());
+ }
+
+ private void validateFormatType(String formatType) {
+ if (!SUPPORTED_FORMATS.contains(formatType)) {
+ throw new AnalysisException("Format type only support DATE, DATETIME and TIME, but get: " + formatType);
+ }
+ }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/DateTimeExtractAndTransform.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/DateTimeExtractAndTransform.java
index b43588d..087fa86 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/DateTimeExtractAndTransform.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/executable/DateTimeExtractAndTransform.java
@@ -44,6 +44,7 @@
import org.apache.doris.nereids.types.DateTimeV2Type;
import org.apache.doris.nereids.types.DateV2Type;
import org.apache.doris.nereids.types.DecimalV3Type;
+import org.apache.doris.nereids.types.StringType;
import org.apache.doris.nereids.util.DateUtils;
import org.apache.commons.lang3.StringUtils;
@@ -1342,4 +1343,83 @@
public static Expression secToTime(DoubleLiteral sec) {
return new TimeV2Literal(sec.getValue() * 1000000);
}
+
+ /**
+ * get_format function for constant folding
+ */
+ @ExecFunction(name = "get_format")
+ public static Expression getFormat(StringLikeLiteral type, StringLikeLiteral format) {
+ String typeStr = type.getValue();
+ String formatStr = format.getValue().toUpperCase();
+
+ String result = null;
+
+ switch (typeStr) {
+ case "DATE":
+ switch (formatStr) {
+ case "USA":
+ result = "%m.%d.%Y";
+ break;
+ case "JIS":
+ case "ISO":
+ result = "%Y-%m-%d";
+ break;
+ case "EUR":
+ result = "%d.%m.%Y";
+ break;
+ case "INTERNAL":
+ result = "%Y%m%d";
+ break;
+ default:
+ break;
+ }
+ break;
+ case "DATETIME":
+ switch (formatStr) {
+ case "USA":
+ result = "%Y-%m-%d %H.%i.%s";
+ break;
+ case "JIS":
+ case "ISO":
+ result = "%Y-%m-%d %H:%i:%s";
+ break;
+ case "EUR":
+ result = "%Y-%m-%d %H.%i.%s";
+ break;
+ case "INTERNAL":
+ result = "%Y%m%d%H%i%s";
+ break;
+ default:
+ break;
+ }
+ break;
+ case "TIME":
+ switch (formatStr) {
+ case "USA":
+ result = "%h:%i:%s %p";
+ break;
+ case "JIS":
+ case "ISO":
+ result = "%H:%i:%s";
+ break;
+ case "EUR":
+ result = "%H.%i.%s";
+ break;
+ case "INTERNAL":
+ result = "%H%i%s";
+ break;
+ default:
+ break;
+ }
+ break;
+ default:
+ break;
+ }
+
+ if (result == null) {
+ return new NullLiteral(StringType.INSTANCE);
+ }
+
+ return new VarcharLiteral(result);
+ }
}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/GetFormat.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/GetFormat.java
new file mode 100644
index 0000000..e5eda62
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/GetFormat.java
@@ -0,0 +1,75 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License. You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied. See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.nereids.trees.expressions.functions.scalar;
+
+import org.apache.doris.catalog.FunctionSignature;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.trees.expressions.functions.AlwaysNullable;
+import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature;
+import org.apache.doris.nereids.trees.expressions.shape.BinaryExpression;
+import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
+import org.apache.doris.nereids.types.StringType;
+import org.apache.doris.nereids.types.VarcharType;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * ScalarFunction 'get_format'.
+ */
+public class GetFormat extends ScalarFunction
+ implements AlwaysNullable, BinaryExpression, ExplicitlyCastableSignature {
+ public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
+ FunctionSignature.ret(VarcharType.SYSTEM_DEFAULT)
+ .args(VarcharType.SYSTEM_DEFAULT, VarcharType.SYSTEM_DEFAULT),
+ FunctionSignature.ret(StringType.INSTANCE)
+ .args(StringType.INSTANCE, StringType.INSTANCE)
+ );
+
+ /**
+ * constructor with 2 arguments.
+ */
+ public GetFormat(Expression arg0, Expression arg1) {
+ super("get_format", arg0, arg1);
+ }
+
+ public GetFormat(ScalarFunctionParams functionParams) {
+ super(functionParams);
+ }
+
+ /**
+ * withChildren.
+ */
+ @Override
+ public GetFormat withChildren(List<Expression> children) {
+ Preconditions.checkArgument(children.size() == 2);
+ return new GetFormat(getFunctionParams(children));
+ }
+
+ @Override
+ public List<FunctionSignature> getSignatures() {
+ return SIGNATURES;
+ }
+
+ @Override
+ public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+ return visitor.visitGetFormat(this, context);
+ }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java
index 630fcbe..655806b 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java
@@ -226,6 +226,7 @@
import org.apache.doris.nereids.trees.expressions.functions.scalar.FromUnixtime;
import org.apache.doris.nereids.trees.expressions.functions.scalar.G;
import org.apache.doris.nereids.trees.expressions.functions.scalar.Gcd;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.GetFormat;
import org.apache.doris.nereids.trees.expressions.functions.scalar.GetVariantType;
import org.apache.doris.nereids.trees.expressions.functions.scalar.Greatest;
import org.apache.doris.nereids.trees.expressions.functions.scalar.Hex;
@@ -2600,4 +2601,8 @@
default R visitExportSet(ExportSet exportSet, C context) {
return visitScalarFunction(exportSet, context);
}
+
+ default R visitGetFormat(GetFormat getFormat, C context) {
+ return visitScalarFunction(getFormat, context);
+ }
}
diff --git a/regression-test/data/nereids_p0/sql_functions/datetime_functions/test_date_function_v2.out b/regression-test/data/nereids_p0/sql_functions/datetime_functions/test_date_function_v2.out
index 3472168..11b7155 100644
--- a/regression-test/data/nereids_p0/sql_functions/datetime_functions/test_date_function_v2.out
+++ b/regression-test/data/nereids_p0/sql_functions/datetime_functions/test_date_function_v2.out
@@ -41,3 +41,84 @@
-- !sql_diff14 --
86400000000
+-- !get_format_date_1 --
+%m.%d.%Y
+%Y-%m-%d
+%Y-%m-%d
+%d.%m.%Y
+%Y%m%d
+\N
+
+-- !get_format_date_2 --
+%m.%d.%Y
+%Y-%m-%d
+%Y-%m-%d
+%d.%m.%Y
+%Y%m%d
+\N
+
+-- !get_format_date_3 --
+%m.%d.%Y
+%Y-%m-%d
+%Y-%m-%d
+%d.%m.%Y
+%Y%m%d
+\N
+
+-- !get_format_date_4 --
+\N
+
+-- !get_format_datetime_1 --
+%Y-%m-%d %H.%i.%s
+%Y-%m-%d %H:%i:%s
+%Y-%m-%d %H:%i:%s
+%Y-%m-%d %H.%i.%s
+%Y%m%d%H%i%s
+\N
+
+-- !get_format_datetime_2 --
+%Y-%m-%d %H.%i.%s
+%Y-%m-%d %H:%i:%s
+%Y-%m-%d %H:%i:%s
+%Y-%m-%d %H.%i.%s
+%Y%m%d%H%i%s
+\N
+
+-- !get_format_datetime_3 --
+%Y-%m-%d %H.%i.%s
+%Y-%m-%d %H:%i:%s
+%Y-%m-%d %H:%i:%s
+%Y-%m-%d %H.%i.%s
+%Y%m%d%H%i%s
+\N
+
+-- !get_format_datetime_4 --
+\N
+
+-- !get_format_time_1 --
+%h:%i:%s %p
+%H:%i:%s
+%H:%i:%s
+%H.%i.%s
+%H%i%s
+\N
+
+-- !get_format_time_2 --
+%h:%i:%s %p
+%H:%i:%s
+%H:%i:%s
+%H.%i.%s
+%H%i%s
+\N
+
+-- !get_format_time_3 --
+%h:%i:%s %p
+%H:%i:%s
+%H:%i:%s
+%H.%i.%s
+%H%i%s
+\N
+
+-- !get_format_time_4 --
+\N
+
diff --git a/regression-test/suites/nereids_p0/sql_functions/datetime_functions/test_date_function_v2.groovy b/regression-test/suites/nereids_p0/sql_functions/datetime_functions/test_date_function_v2.groovy
index 58a04714..8d5158e 100644
--- a/regression-test/suites/nereids_p0/sql_functions/datetime_functions/test_date_function_v2.groovy
+++ b/regression-test/suites/nereids_p0/sql_functions/datetime_functions/test_date_function_v2.groovy
@@ -86,4 +86,72 @@
testFoldConst("select microseconds_diff('2023-10-15 00:00:00', '2023-10-14 00:00:00.1');")
qt_sql_diff14 "select microseconds_diff('2023-10-15 00:00:00', '2023-10-14 00:00:00');"
testFoldConst("select microseconds_diff('2023-10-15 00:00:00', '2023-10-14 00:00:00');")
+
+ def getFormatTable = "get_format_table_test"
+ sql """ DROP TABLE IF EXISTS ${getFormatTable} """
+ sql """ CREATE TABLE ${getFormatTable} (
+ id INT,
+ lc VARCHAR(10)
+ ) DUPLICATE KEY(id)
+ DISTRIBUTED BY HASH(id) BUCKETS 1
+ PROPERTIES ( "replication_num" = "1" ); """
+ sql """ INSERT INTO ${getFormatTable} VALUES
+ (1, 'USA'), (2, 'JIS'), (3, 'ISO'), (4, 'EUR'),
+ (5, 'INTERNAL'), (6, 'Doris'); """
+
+ qt_get_format_date_1 """ SELECT GET_FORMAT(DATE, lc) FROM ${getFormatTable} ORDER BY id; """
+ qt_get_format_date_2 """SELECT GET_FORMAT(DaTe, lc) FROM ${getFormatTable} ORDER BY id; """
+ qt_get_format_date_3 """SELECT GET_FORMAT(dATe, lc) FROM ${getFormatTable} ORDER BY id; """
+ qt_get_format_date_4 """SELECT GET_FORMAT(date, '你好');"""
+ qt_get_format_datetime_1 """ SELECT GET_FORMAT(DATETIME, lc) FROM ${getFormatTable} ORDER BY id; """
+ qt_get_format_datetime_2 """SELECT GET_FORMAT(DaTETimE, lc) FROM ${getFormatTable} ORDER BY id; """
+ qt_get_format_datetime_3 """SELECT GET_FORMAT(dATetIMe, lc) FROM ${getFormatTable} ORDER BY id; """
+ qt_get_format_datetime_4 """SELECT GET_FORMAT(datetime, '你好');"""
+ qt_get_format_time_1 """ SELECT GET_FORMAT(TIME, lc) FROM ${getFormatTable} ORDER BY id; """
+ qt_get_format_time_2 """ SELECT GET_FORMAT(TiMe, lc) FROM ${getFormatTable} ORDER BY id; """
+ qt_get_format_time_3 """ SELECT GET_FORMAT(tIME, lc) FROM ${getFormatTable} ORDER BY id; """
+ qt_get_format_time_4 """ SELECT GET_FORMAT(time, '你好'); """
+ test {
+ sql """ SELECT GET_FORMAT(DATA, 'USA'); """
+ exception "Format type only support DATE, DATETIME and TIME"
+ }
+
+ sql """ DROP TABLE IF EXISTS ${getFormatTable} """
+
+ testFoldConst("SELECT GET_FORMAT(DATE, 'USA');")
+ testFoldConst("SELECT GET_FORMAT(date, 'usa');")
+ testFoldConst("SELECT GET_FORMAT(DATE, 'JIS');")
+ testFoldConst("SELECT GET_FORMAT(Date, 'JiS');")
+ testFoldConst("SELECT GET_FORMAT(DATE, 'ISO');")
+ testFoldConst("SELECT GET_FORMAT(DaTe, 'iSo');")
+ testFoldConst("SELECT GET_FORMAT(DATE, 'EUR');")
+ testFoldConst("SELECT GET_FORMAT(daTE, 'EuR');")
+ testFoldConst("SELECT GET_FORMAT(DATE, 'INTERNAL');")
+ testFoldConst("SELECT GET_FORMAT(DaTE, 'InTERnAL');")
+ testFoldConst("SELECT GET_FORMAT(DATE, 'Doris');")
+ testFoldConst("SELECT GET_FORMAT(DATE, '你好');")
+ testFoldConst("SELECT GET_FORMAT(DATETIME, 'USA');")
+ testFoldConst("SELECT GET_FORMAT(datetime, 'Usa');")
+ testFoldConst("SELECT GET_FORMAT(DATETIME, 'JIS');")
+ testFoldConst("SELECT GET_FORMAT(DateTime, 'jis');")
+ testFoldConst("SELECT GET_FORMAT(DATETIME, 'ISO');")
+ testFoldConst("SELECT GET_FORMAT(DaTeTiMe, 'IsO');")
+ testFoldConst("SELECT GET_FORMAT(DATETIME, 'EUR');")
+ testFoldConst("SELECT GET_FORMAT(dateTIME, 'EuR');")
+ testFoldConst("SELECT GET_FORMAT(DATETIME, 'INTERNAL');")
+ testFoldConst("SELECT GET_FORMAT(DaTeTimE, 'internal');")
+ testFoldConst("SELECT GET_FORMAT(DATETIME, 'Doris');")
+ testFoldConst("SELECT GET_FORMAT(DATETIME, '你好');")
+ testFoldConst("SELECT GET_FORMAT(TIME, 'USA');")
+ testFoldConst("SELECT GET_FORMAT(time, 'USa');")
+ testFoldConst("SELECT GET_FORMAT(TIME, 'JIS');")
+ testFoldConst("SELECT GET_FORMAT(TiMe, 'jiS');")
+ testFoldConst("SELECT GET_FORMAT(TIME, 'ISO');")
+ testFoldConst("SELECT GET_FORMAT(TiME, 'iso');")
+ testFoldConst("SELECT GET_FORMAT(TIME, 'EUR');")
+ testFoldConst("SELECT GET_FORMAT(TimE, 'eur');")
+ testFoldConst("SELECT GET_FORMAT(TIME, 'INTERNAL');")
+ testFoldConst("SELECT GET_FORMAT(TImE, 'INTERNAL');")
+ testFoldConst("SELECT GET_FORMAT(TIME, 'Doris');")
+ testFoldConst("SELECT GET_FORMAT(TIME, '你好');")
}