You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by tw...@apache.org on 2021/11/12 14:43:35 UTC

[flink] 01/04: [hotfix][table-planner] Add ExpressionCodeGeneratorCastRule extracting generateExpression from AbstractExpressionCodeGeneratorCastRule

This is an automated email from the ASF dual-hosted git repository.

twalthr pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 5c914e15e2016be12ce3ea2b2127a2607df24a75
Author: slinkydeveloper <fr...@gmail.com>
AuthorDate: Mon Nov 8 17:31:51 2021 +0100

    [hotfix][table-planner] Add ExpressionCodeGeneratorCastRule extracting generateExpression from AbstractExpressionCodeGeneratorCastRule
    
    Signed-off-by: slinkydeveloper <fr...@gmail.com>
---
 .../casting/ExpressionCodeGeneratorCastRule.java   | 44 ++++++++++++++++++++++
 .../rules/AbstractCharacterFamilyTargetRule.java   |  2 +-
 .../AbstractExpressionCodeGeneratorCastRule.java   | 12 ++----
 .../functions/casting/rules/IdentityCastRule.java  | 13 ++++++-
 4 files changed, 61 insertions(+), 10 deletions(-)

diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/ExpressionCodeGeneratorCastRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/ExpressionCodeGeneratorCastRule.java
new file mode 100644
index 0000000..a41db96
--- /dev/null
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/ExpressionCodeGeneratorCastRule.java
@@ -0,0 +1,44 @@
+/*
+ * 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.flink.table.planner.functions.casting;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.types.logical.LogicalType;
+
+/**
+ * Cast rule that is able to generate a single expression containing all the casting logic.
+ *
+ * @param <IN> Input internal type
+ * @param <OUT> Output internal type
+ */
+@Internal
+public interface ExpressionCodeGeneratorCastRule<IN, OUT> extends CodeGeneratorCastRule<IN, OUT> {
+
+    /**
+     * Generate a Java expression performing the casting. This expression can be wrapped in another
+     * expression, or assigned to a variable or returned from a function.
+     *
+     * <p>NOTE: the {@code inputTerm} is always either a primitive or a non-null object.
+     */
+    String generateExpression(
+            CodeGeneratorCastRule.Context context,
+            String inputTerm,
+            LogicalType inputLogicalType,
+            LogicalType targetLogicalType);
+}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/rules/AbstractCharacterFamilyTargetRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/rules/AbstractCharacterFamilyTargetRule.java
index 25cbdf1..0309566 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/rules/AbstractCharacterFamilyTargetRule.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/rules/AbstractCharacterFamilyTargetRule.java
@@ -46,7 +46,7 @@ public abstract class AbstractCharacterFamilyTargetRule<IN>
             LogicalType targetLogicalType);
 
     @Override
-    String generateExpression(
+    public String generateExpression(
             CodeGeneratorCastRule.Context context,
             String inputTerm,
             LogicalType inputLogicalType,
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/rules/AbstractExpressionCodeGeneratorCastRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/rules/AbstractExpressionCodeGeneratorCastRule.java
index a4027c5..f3ed23f 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/rules/AbstractExpressionCodeGeneratorCastRule.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/rules/AbstractExpressionCodeGeneratorCastRule.java
@@ -23,6 +23,7 @@ import org.apache.flink.table.data.utils.CastExecutor;
 import org.apache.flink.table.planner.functions.casting.CastRule;
 import org.apache.flink.table.planner.functions.casting.CastRulePredicate;
 import org.apache.flink.table.planner.functions.casting.CodeGeneratorCastRule;
+import org.apache.flink.table.planner.functions.casting.ExpressionCodeGeneratorCastRule;
 import org.apache.flink.table.runtime.generated.CompileUtils;
 import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.table.types.logical.utils.LogicalTypeUtils;
@@ -36,18 +37,13 @@ import java.util.Collections;
  */
 @Internal
 public abstract class AbstractExpressionCodeGeneratorCastRule<IN, OUT>
-        extends AbstractNullAwareCodeGeneratorCastRule<IN, OUT> {
+        extends AbstractNullAwareCodeGeneratorCastRule<IN, OUT>
+        implements ExpressionCodeGeneratorCastRule<IN, OUT> {
 
     protected AbstractExpressionCodeGeneratorCastRule(CastRulePredicate predicate) {
         super(predicate);
     }
 
-    abstract String generateExpression(
-            CodeGeneratorCastRule.Context context,
-            String inputTerm,
-            LogicalType inputLogicalType,
-            LogicalType targetLogicalType);
-
     @Override
     protected String generateCodeBlockInternal(
             CodeGeneratorCastRule.Context context,
@@ -58,7 +54,7 @@ public abstract class AbstractExpressionCodeGeneratorCastRule<IN, OUT>
         return returnVariable
                 + " = "
                 + generateExpression(context, inputTerm, inputLogicalType, targetLogicalType)
-                + ";";
+                + ";\n";
     }
 
     @Override
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/rules/IdentityCastRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/rules/IdentityCastRule.java
index e96c4d4..184520f 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/rules/IdentityCastRule.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/rules/IdentityCastRule.java
@@ -22,6 +22,7 @@ import org.apache.flink.annotation.Internal;
 import org.apache.flink.table.planner.functions.casting.CastCodeBlock;
 import org.apache.flink.table.planner.functions.casting.CastRulePredicate;
 import org.apache.flink.table.planner.functions.casting.CodeGeneratorCastRule;
+import org.apache.flink.table.planner.functions.casting.ExpressionCodeGeneratorCastRule;
 import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.table.types.logical.LogicalTypeFamily;
 import org.apache.flink.table.types.logical.utils.LogicalTypeCasts;
@@ -31,7 +32,8 @@ import org.apache.flink.table.types.logical.utils.LogicalTypeCasts;
  * #isIdentityCast(LogicalType, LogicalType)}
  */
 @Internal
-public class IdentityCastRule extends AbstractCodeGeneratorCastRule<Object, Object> {
+public class IdentityCastRule extends AbstractCodeGeneratorCastRule<Object, Object>
+        implements ExpressionCodeGeneratorCastRule<Object, Object> {
 
     public static final IdentityCastRule INSTANCE = new IdentityCastRule();
 
@@ -51,6 +53,15 @@ public class IdentityCastRule extends AbstractCodeGeneratorCastRule<Object, Obje
     }
 
     @Override
+    public String generateExpression(
+            CodeGeneratorCastRule.Context context,
+            String inputTerm,
+            LogicalType inputLogicalType,
+            LogicalType targetLogicalType) {
+        return inputTerm;
+    }
+
+    @Override
     public CastCodeBlock generateCodeBlock(
             CodeGeneratorCastRule.Context context,
             String inputTerm,