You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by "AMashenkov (via GitHub)" <gi...@apache.org> on 2023/05/12 08:19:34 UTC

[GitHub] [ignite-3] AMashenkov commented on a diff in pull request #2032: IGNITE-19130 Sql. Align RexImpTable implementation

AMashenkov commented on code in PR #2032:
URL: https://github.com/apache/ignite-3/pull/2032#discussion_r1192061579


##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/exp/RexToLixTranslator.java:
##########
@@ -72,96 +67,148 @@
 import org.apache.calcite.util.BuiltInMethod;
 import org.apache.calcite.util.ControlFlowException;
 import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
+
+import com.google.common.base.CaseFormat;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+
 import org.apache.ignite.internal.sql.engine.type.IgniteTypeFactory;
 import org.apache.ignite.internal.sql.engine.util.IgniteMethod;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.locationtech.jts.geom.Geometry;
+
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.lang.reflect.Type;
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.TRANSLATE3;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CASE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CHAR_LENGTH;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.OCTET_LENGTH;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.PREV;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.SEARCH;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.SUBSTRING;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.UPPER;
+
+import static java.util.Objects.requireNonNull;
 
 /**
- * Translates {@link RexNode REX expressions} to {@link Expression linq4j expressions}.
+ * Translates {@link org.apache.calcite.rex.RexNode REX expressions} to
+ * {@link Expression linq4j expressions}.
+ * Changes in comparison with original code:
+ * 1. RexToLixTranslator#visitDynamicParam() refactoring
+ * 2. RexToLixTranslator#translateCast() SqlTypeName.DECIMAL special case converter.
+ * 3. RexToLixTranslator#translateCast() BYTESTRING_TO_STRING, STRING_TO_BYTESTRING special case converters.
+ * 4. RexToLixTranslator#translateCast() case INTERVAL_SECOND -> case CHARACTER special case converters.
+ * 5. RexToLixTranslator#translateCast() case TIMESTAMP -> case CHAR  special case converters.
+ * 6. RexToLixTranslator#translateLiteral() case DECIMAL special case converters.
+ * 7. EnumUtils.convert -> ConverterUtils.convert
  */
 public class RexToLixTranslator implements RexVisitor<RexToLixTranslator.Result> {
-    final JavaTypeFactory typeFactory;
+    public static final Map<Method, SqlOperator> JAVA_TO_SQL_METHOD_MAP =
+            ImmutableMap.<Method, SqlOperator>builder()
+                    .put(findMethod(String.class, "toUpperCase"), UPPER)
+                    .put(BuiltInMethod.SUBSTRING.method, SUBSTRING)
+                    .put(BuiltInMethod.OCTET_LENGTH.method, OCTET_LENGTH)
+                    .put(BuiltInMethod.CHAR_LENGTH.method, CHAR_LENGTH)
+                    .put(BuiltInMethod.TRANSLATE3.method, TRANSLATE3)
+                    .build();
 
+    final JavaTypeFactory typeFactory;
     final RexBuilder builder;
-
-    private final RexProgram program;
-
+    private final @Nullable RexProgram program;
     final SqlConformance conformance;
-
     private final Expression root;
-
-    final RexToLixTranslator.InputGetter inputGetter;
-
+    final RexToLixTranslator.@Nullable InputGetter inputGetter;
     private final BlockBuilder list;
-
-    private final Function1<String, InputGetter> correlates;
+    private final @Nullable BlockBuilder staticList;
+    private final @Nullable Function1<String, InputGetter> correlates;
 
     /**
-     * Map from RexLiteral's variable name to its literal, which is often a ({@link ConstantExpression})).
-     * It is used in the some {@code RexCall}'s implementors, such as {@code ExtractImplementor}.
+     * Map from RexLiteral's variable name to its literal, which is often a
+     * ({@link org.apache.calcite.linq4j.tree.ConstantExpression}))
+     * It is used in the some {@code RexCall}'s implementors, such as
+     * {@code ExtractImplementor}.
      *
      * @see #getLiteral
      * @see #getLiteralValue
      */
     private final Map<Expression, Expression> literalMap = new HashMap<>();
 
-    /**
-     * For {@code RexCall}, keep the list of its operand's {@code Result}. It is useful when creating a
-     * {@code CallImplementor}.
-     */
-    private final Map<RexCall, List<Result>> callOperandResultMap = new HashMap<>();
+    /** For {@code RexCall}, keep the list of its operand's {@code Result}.

Review Comment:
   Let's revert changes in wrapping lines in javadoc.



-- 
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: notifications-unsubscribe@ignite.apache.org

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