You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by "tanclary (via GitHub)" <gi...@apache.org> on 2023/06/06 00:16:17 UTC

[GitHub] [calcite] tanclary commented on a diff in pull request #3238: [CALCITE-5744] Add STR_TO_MAP function (enabled in Spark library)

tanclary commented on code in PR #3238:
URL: https://github.com/apache/calcite/pull/3238#discussion_r1218724377


##########
core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java:
##########
@@ -3092,6 +3096,36 @@ private static class ArrayConcatImplementor extends AbstractRexCallImplementor {
     }
   }
 
+  /** Implementor for str_to_map. */
+  private static class StringToMapImplementor extends AbstractRexCallImplementor {
+    StringToMapImplementor() {
+      super("str_to_map", NullPolicy.STRICT, false);
+    }
+
+    @Override Expression implementSafe(RexToLixTranslator translator,
+        RexCall call, List<Expression> argValueList) {
+      String defaultStringDelimiter = ",";
+      String defaultKeyValueDelimiter = ":";
+      if (argValueList.size() == 1) {

Review Comment:
   i personally find switch (operand count) to look better here but it's very subjective i suppose.



##########
core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java:
##########
@@ -3092,6 +3096,36 @@ private static class ArrayConcatImplementor extends AbstractRexCallImplementor {
     }
   }
 
+  /** Implementor for str_to_map. */
+  private static class StringToMapImplementor extends AbstractRexCallImplementor {
+    StringToMapImplementor() {
+      super("str_to_map", NullPolicy.STRICT, false);
+    }
+
+    @Override Expression implementSafe(RexToLixTranslator translator,
+        RexCall call, List<Expression> argValueList) {
+      String defaultStringDelimiter = ",";
+      String defaultKeyValueDelimiter = ":";
+      if (argValueList.size() == 1) {
+        return Expressions.call(
+            BuiltInMethod.STR_TO_MAP.method,
+            argValueList.get(0),
+            Expressions.constant(defaultStringDelimiter),
+            Expressions.constant(defaultKeyValueDelimiter));
+      } else if (argValueList.size() == 2) {
+        return Expressions.call(
+          BuiltInMethod.STR_TO_MAP.method,
+              argValueList.get(0),
+              argValueList.get(1),
+              Expressions.constant(defaultKeyValueDelimiter));

Review Comment:
   if you only use the default Strings for Expressions.constant, you could just initialize them as expressions instead?



##########
core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java:
##########
@@ -3917,6 +3918,38 @@ public static List mapValues(Map map) {
     return new ArrayList<>(map.values());
   }
 
+  /** Support the MAP_FROM_ARRAYS function. */

Review Comment:
   is this different/unrelated to STR_TO_MAP? if so, the commit message etc should be updated to reflect everything the PR includes.



##########
core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java:
##########
@@ -3092,6 +3096,36 @@ private static class ArrayConcatImplementor extends AbstractRexCallImplementor {
     }
   }
 
+  /** Implementor for str_to_map. */
+  private static class StringToMapImplementor extends AbstractRexCallImplementor {
+    StringToMapImplementor() {
+      super("str_to_map", NullPolicy.STRICT, false);
+    }
+
+    @Override Expression implementSafe(RexToLixTranslator translator,
+        RexCall call, List<Expression> argValueList) {
+      String defaultStringDelimiter = ",";

Review Comment:
   is there a way to use the "SPLIT" implementation offered by Bigquery here? Some of this looks very similar.



##########
core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperators.java:
##########
@@ -944,6 +944,31 @@ private static RelDataType arrayReturnType(SqlOperatorBinding opBinding) {
           ReturnTypes.TO_MAP_VALUES_NULLABLE,
           OperandTypes.MAP);
 
+  private static RelDataType mapReturnType(SqlOperatorBinding opBinding) {
+    final RelDataType keysArrayType = opBinding.collectOperandTypes().get(0);
+    final RelDataType valuesArrayType = opBinding.collectOperandTypes().get(1);
+    final boolean nullable = keysArrayType.isNullable() || valuesArrayType.isNullable();
+    return SqlTypeUtil.createMapType(
+        opBinding.getTypeFactory(),
+        requireNonNull(keysArrayType.getComponentType(), "inferred key type"),
+        requireNonNull(valuesArrayType.getComponentType(), "inferred value type"),
+        nullable);
+  }
+
+  /** The "MAP_FROM_ARRAYS(keysArray, valuesArray)" function. */
+  @LibraryOperator(libraries = {SPARK})
+  public static final SqlFunction MAP_FROM_ARRAYS =

Review Comment:
   just reiterating that if this PR includes multiple functions, the JIRA case/commit message should reflect that



##########
core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperators.java:
##########
@@ -944,6 +944,31 @@ private static RelDataType arrayReturnType(SqlOperatorBinding opBinding) {
           ReturnTypes.TO_MAP_VALUES_NULLABLE,
           OperandTypes.MAP);
 
+  private static RelDataType mapReturnType(SqlOperatorBinding opBinding) {

Review Comment:
   I am not sure this belongs here. Maybe somewhere like ReturnTypes.java would be more appropriate.



##########
core/src/main/java/org/apache/calcite/sql/type/ReturnTypes.java:
##########
@@ -536,6 +536,18 @@ public static SqlCall stripSeparator(SqlCall call) {
   public static final SqlReturnTypeInference TO_MAP =
       ARG0.andThen(SqlTypeTransforms.TO_MAP);
 
+  /**
+   * Returns a MAP type.
+   *
+   * <p>For example, given {@code STRING}, returns
+   * {@code (STRING, STRING) MAP}.
+   */
+  public static final SqlReturnTypeInference IDENTITY_TO_MAP =
+      ARG0.andThen(SqlTypeTransforms.IDENTITY_TO_MAP);
+
+  public static final SqlReturnTypeInference IDENTITY_TO_MAP_NULLABLE =

Review Comment:
   nit: many *_NULLABLE return types have a JAVADOC



##########
core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java:
##########
@@ -3917,6 +3918,38 @@ public static List mapValues(Map map) {
     return new ArrayList<>(map.values());
   }
 
+  /** Support the MAP_FROM_ARRAYS function. */
+  public static Map mapFromArrays(List keysArray, List valuesArray) {
+    if (keysArray.size() != valuesArray.size()) {
+      throw new IllegalArgumentException(

Review Comment:
   can this exception be added as a constant to Resources like the other errors thrown from this class?



##########
core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java:
##########
@@ -3092,6 +3096,36 @@ private static class ArrayConcatImplementor extends AbstractRexCallImplementor {
     }
   }
 
+  /** Implementor for str_to_map. */
+  private static class StringToMapImplementor extends AbstractRexCallImplementor {
+    StringToMapImplementor() {
+      super("str_to_map", NullPolicy.STRICT, false);
+    }
+
+    @Override Expression implementSafe(RexToLixTranslator translator,
+        RexCall call, List<Expression> argValueList) {
+      String defaultStringDelimiter = ",";
+      String defaultKeyValueDelimiter = ":";
+      if (argValueList.size() == 1) {
+        return Expressions.call(
+            BuiltInMethod.STR_TO_MAP.method,
+            argValueList.get(0),
+            Expressions.constant(defaultStringDelimiter),
+            Expressions.constant(defaultKeyValueDelimiter));
+      } else if (argValueList.size() == 2) {
+        return Expressions.call(
+          BuiltInMethod.STR_TO_MAP.method,
+              argValueList.get(0),
+              argValueList.get(1),
+              Expressions.constant(defaultKeyValueDelimiter));

Review Comment:
   i am wondering whether the return statement could be extracted out, and the if-else-if (or switch, if you choose to use it) could just be used for assigning operands and whatnot. Right now it looks like a lot of duplicate code.



##########
core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperators.java:
##########
@@ -944,6 +944,31 @@ private static RelDataType arrayReturnType(SqlOperatorBinding opBinding) {
           ReturnTypes.TO_MAP_VALUES_NULLABLE,
           OperandTypes.MAP);
 
+  private static RelDataType mapReturnType(SqlOperatorBinding opBinding) {
+    final RelDataType keysArrayType = opBinding.collectOperandTypes().get(0);
+    final RelDataType valuesArrayType = opBinding.collectOperandTypes().get(1);
+    final boolean nullable = keysArrayType.isNullable() || valuesArrayType.isNullable();
+    return SqlTypeUtil.createMapType(
+        opBinding.getTypeFactory(),
+        requireNonNull(keysArrayType.getComponentType(), "inferred key type"),
+        requireNonNull(valuesArrayType.getComponentType(), "inferred value type"),
+        nullable);
+  }
+
+  /** The "MAP_FROM_ARRAYS(keysArray, valuesArray)" function. */
+  @LibraryOperator(libraries = {SPARK})
+  public static final SqlFunction MAP_FROM_ARRAYS =
+      SqlBasicFunction.create(SqlKind.MAP_FROM_ARRAYS,
+          SqlLibraryOperators::mapReturnType,
+          OperandTypes.ARRAY_ARRAY);
+
+  /** The "STR_TO_MAP(string[, stringDelimiter[, keyValueDelimiter]])" function. */
+  @LibraryOperator(libraries = {SPARK})
+  public static final SqlFunction STR_TO_MAP =
+      SqlBasicFunction.create(SqlKind.STR_TO_MAP,

Review Comment:
   I think there is an entry in OperandTypes called something like "STRING_OPTIONAL_STRING" or something along those lines, either use that or add an entry to fit this case, so future functions can use it easily if need be.



-- 
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: commits-unsubscribe@calcite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org