codope commented on code in PR #12455:
URL: https://github.com/apache/hudi/pull/12455#discussion_r1881328995


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/HoodieSparkExpressionIndex.java:
##########
@@ -81,40 +84,57 @@ interface SparkFunction extends Serializable {
         if (columns.size() != 1) {
           throw new IllegalArgumentException("FROM_UNIXTIME requires 1 
column");
         }
-        return functions.from_unixtime(columns.get(0), options.get("format"));
+        if (options.containsKey(FORMAT_OPTION)) {
+          return functions.from_unixtime(columns.get(0), 
options.get(FORMAT_OPTION));
+        } else {
+          return functions.from_unixtime(columns.get(0));
+        }
       }),
       Pair.of(SPARK_UNIX_TIMESTAMP, (columns, options) -> {
         if (columns.size() != 1) {
           throw new IllegalArgumentException("UNIX_TIMESTAMP requires 1 
column");
         }
-        return functions.unix_timestamp(columns.get(0), options.get("format"));
+        if (options.containsKey(FORMAT_OPTION)) {
+          return functions.unix_timestamp(columns.get(0), 
options.get(FORMAT_OPTION));
+        } else {
+          return functions.unix_timestamp(columns.get(0));
+        }
       }),
       Pair.of(SPARK_TO_DATE, (columns, options) -> {
         if (columns.size() != 1) {
           throw new IllegalArgumentException("TO_DATE requires 1 column");
         }
-        return functions.to_date(columns.get(0));
+        if (options.containsKey(FORMAT_OPTION)) {
+          return functions.to_date(columns.get(0), options.get(FORMAT_OPTION));
+        } else {
+          return functions.to_date(columns.get(0));
+        }
       }),
       Pair.of(SPARK_TO_TIMESTAMP, (columns, options) -> {
         if (columns.size() != 1) {
           throw new IllegalArgumentException("TO_TIMESTAMP requires 1 column");
         }
-        return functions.to_timestamp(columns.get(0));
+        if (options.containsKey(FORMAT_OPTION)) {
+          return functions.to_timestamp(columns.get(0), 
options.get(FORMAT_OPTION));
+        } else {
+          return functions.to_timestamp(columns.get(0));
+        }
       }),
       Pair.of(SPARK_DATE_ADD, (columns, options) -> {
         if (columns.size() != 1) {
           throw new IllegalArgumentException("DATE_ADD requires 1 column");
         }
-        return functions.date_add(columns.get(0), 
Integer.parseInt(options.get("days")));
+        return functions.date_add(columns.get(0), 
Integer.parseInt(options.get(DAYS_OPTION)));
       }),
       Pair.of(SPARK_DATE_SUB, (columns, options) -> {
         if (columns.size() != 1) {
           throw new IllegalArgumentException("DATE_SUB requires 1 column");
         }
-        return functions.date_sub(columns.get(0), 
Integer.parseInt(options.get("days")));
+        return functions.date_sub(columns.get(0), 
Integer.parseInt(options.get(DAYS_OPTION)));
       }),
 
       // String functions
+      // TODO: Concat is not yet supported fully since there is a limitation 
of only one source field within expression index

Review Comment:
   Shall we remove this?



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/RecordLevelIndexSupport.scala:
##########
@@ -351,6 +351,14 @@ object RecordLevelIndexSupport {
       case literal: Literal => expression2 match {
         case attr: AttributeReference =>
           Option.apply(attr, literal)
+        case cast: Cast if cast.child.isInstanceOf[AttributeReference] =>
+          Option.apply(cast.child.asInstanceOf[AttributeReference], literal)

Review Comment:
   At this point, is `cast.child` a resolved reference?



##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/command/index/TestExpressionIndex.scala:
##########
@@ -846,11 +852,122 @@ class TestExpressionIndex extends HoodieSparkSqlTestBase 
{
           var literal = Literal.create("rider-c")
           var dataFilter = EqualTo(lowerExpr, literal)
           verifyFilePruning(opts, dataFilter, metaClient, 
isDataSkippingExpected = true)
+          spark.sql(s"drop index idx_rider on $tableName")
 
           val fromUnixTime = resolveExpr(spark, 
unapply(functions.from_unixtime(functions.col("ts"), "yyyy-MM-dd")).get, 
tableSchema)
           literal = Literal.create("2023-11-07")
           dataFilter = EqualTo(fromUnixTime, literal)
           verifyFilePruning(opts, dataFilter, metaClient, 
isDataSkippingExpected = true)
+          spark.sql(s"drop index idx_datestr on $tableName")

Review Comment:
   Can we add a separate test? I know the test time will increase a little bit 
due to creating the table again, but it's cleaner. If needed, we can split this 
test class into two so that they can run in parallel.



##########
hudi-common/src/main/java/org/apache/hudi/index/functional/HoodieExpressionIndex.java:
##########
@@ -59,7 +59,14 @@ public interface HoodieExpressionIndex<S, T> extends 
Serializable {
   String IDENTITY_FUNCTION = "identity";
 
   String EXPRESSION_OPTION = "expr";
-
+  String FORMAT_OPTION = "format";
+  String DAYS_OPTION = "days";
+  String POSITION_OPTION = "pos";
+  String LENGTH_OPTION = "len";
+  String PATTERN_OPTION = "pattern";
+  String REPLACEMENT_OPTION = "replacement";
+  String REGEX_GROUP_INDEX_OPTION = "idx";
+  String TRIM_STRING_OPTION = "trimString";

Review Comment:
   Should we have this in a Map of function name to valid options? For example, 
`pos` option does not apply to from_unixtime but the create index command might 
still go through only to fail later.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to