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

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


The following commit(s) were added to refs/heads/master by this push:
     new c5d2a57  [FLINK-12165][table-planner] Added resolution rule that 
checks all unresolved expressions are resolved
c5d2a57 is described below

commit c5d2a572af62799c5d4eb53a52e003de9f3d7b75
Author: Dawid Wysakowicz <[email protected]>
AuthorDate: Thu Apr 11 11:23:13 2019 +0200

    [FLINK-12165][table-planner] Added resolution rule that checks all 
unresolved expressions are resolved
    
    All resolutions should possible happen in a single place. To ensure this
    is always true we added a safety rule that verifies no unresolved
    expressions are part of the output from the ExpressionResolver.
    
    This closes #8149
---
 .../table/expressions/ExpressionResolver.java      |  3 +-
 .../table/expressions/rules/ResolverRules.java     |  6 ++
 .../rules/VerifyNoUnresolvedExpressionsRule.java   | 90 ++++++++++++++++++++
 .../VerifyNoUnresolvedExpressionsRuleTest.java     | 95 ++++++++++++++++++++++
 4 files changed, 193 insertions(+), 1 deletion(-)

diff --git 
a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/expressions/ExpressionResolver.java
 
b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/expressions/ExpressionResolver.java
index afa9360..3a37516 100644
--- 
a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/expressions/ExpressionResolver.java
+++ 
b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/expressions/ExpressionResolver.java
@@ -81,7 +81,8 @@ public class ExpressionResolver {
                        ResolverRules.OVER_WINDOWS,
                        ResolverRules.FIELD_RESOLVE,
                        ResolverRules.FLATTEN_CALL,
-                       ResolverRules.RESOLVE_CALL_BY_ARGUMENTS);
+                       ResolverRules.RESOLVE_CALL_BY_ARGUMENTS,
+                       ResolverRules.VERIFY_NO_MORE_UNRESOLVED_EXPRESSIONS);
        }
 
        private final PlannerExpressionConverter bridgeConverter = 
PlannerExpressionConverter.INSTANCE();
diff --git 
a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/expressions/rules/ResolverRules.java
 
b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/expressions/rules/ResolverRules.java
index c8e3daf..af9d63e 100644
--- 
a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/expressions/rules/ResolverRules.java
+++ 
b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/expressions/rules/ResolverRules.java
@@ -62,6 +62,12 @@ public final class ResolverRules {
         */
        public static final ResolverRule EXPAND_COLUMN_FUNCTIONS = new 
ExpandColumnFunctionsRule();
 
+       /**
+        * Checks that the output of {@link 
org.apache.flink.table.expressions.ExpressionResolver} has no more unresolved
+        * expressions. See {@link VerifyNoUnresolvedExpressionsRule} for 
details.
+        */
+       public static final ResolverRule VERIFY_NO_MORE_UNRESOLVED_EXPRESSIONS 
= new VerifyNoUnresolvedExpressionsRule();
+
        private ResolverRules() {
        }
 }
diff --git 
a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/expressions/rules/VerifyNoUnresolvedExpressionsRule.java
 
b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/expressions/rules/VerifyNoUnresolvedExpressionsRule.java
new file mode 100644
index 0000000..f184ef0
--- /dev/null
+++ 
b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/expressions/rules/VerifyNoUnresolvedExpressionsRule.java
@@ -0,0 +1,90 @@
+/*
+ * 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.expressions.rules;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.expressions.ApiExpressionDefaultVisitor;
+import org.apache.flink.table.expressions.CallExpression;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.LookupCallExpression;
+import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
+
+import java.util.List;
+
+import static 
org.apache.flink.table.expressions.BuiltInFunctionDefinitions.FLATTEN;
+import static 
org.apache.flink.table.expressions.BuiltInFunctionDefinitions.OVER;
+
+/**
+ * Verifies that there is no more unresolved expressions. Checks for 
expression like:
+ * <ul>
+ *     <li>{@link UnresolvedReferenceExpression}</li>
+ *     <li>{@link LookupCallExpression}</li>
+ *     <li>{@link 
org.apache.flink.table.expressions.BuiltInFunctionDefinitions#OVER} that still 
contains
+ *     just alias to corresponding window</li>
+ *     <li>{@link 
org.apache.flink.table.expressions.BuiltInFunctionDefinitions#FLATTEN}</li>
+ * </ul>
+ */
+@Internal
+final class VerifyNoUnresolvedExpressionsRule implements ResolverRule {
+
+       private static final NoUnresolvedCallsChecker checker = new 
NoUnresolvedCallsChecker();
+
+       @Override
+       public List<Expression> apply(List<Expression> expression, 
ResolutionContext context) {
+               expression.forEach(expr -> expr.accept(checker));
+               return expression;
+       }
+
+       private static class NoUnresolvedCallsChecker extends 
ApiExpressionDefaultVisitor<Void> {
+
+               @Override
+               public Void 
visitUnresolvedReference(UnresolvedReferenceExpression unresolvedReference) {
+                       throw getException("reference", unresolvedReference);
+               }
+
+               @Override
+               public Void visitLookupCall(LookupCallExpression lookupCall) {
+                       throw getException("lookup call", lookupCall);
+               }
+
+               @Override
+               public Void visitCall(CallExpression call) {
+                       if (call.getFunctionDefinition() == OVER && 
call.getChildren().size() <= 2) {
+                               throw getException("OVER call", call);
+                       } else if (call.getFunctionDefinition() == FLATTEN) {
+                               throw getException("FLATTEN call", call);
+                       }
+                       call.getChildren().forEach(expr -> expr.accept(this));
+
+                       return null;
+               }
+
+               private TableException getException(String expressionType, 
Expression call) {
+                       return new TableException(String.format(
+                               "Unexpected unresolved %s: %s. All expressions 
should be resolved by now",
+                               expressionType, call));
+               }
+
+               @Override
+               protected Void defaultMethod(Expression expression) {
+                       return null;
+               }
+       }
+}
diff --git 
a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/expressions/rules/VerifyNoUnresolvedExpressionsRuleTest.java
 
b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/expressions/rules/VerifyNoUnresolvedExpressionsRuleTest.java
new file mode 100644
index 0000000..8b043b7
--- /dev/null
+++ 
b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/expressions/rules/VerifyNoUnresolvedExpressionsRuleTest.java
@@ -0,0 +1,95 @@
+/*
+ * 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.expressions.rules;
+
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.FieldReferenceExpression;
+import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
+import org.apache.flink.table.expressions.rules.ResolverRule.ResolutionContext;
+
+import org.junit.Test;
+
+import java.util.List;
+
+import static java.util.Arrays.asList;
+import static java.util.Collections.singletonList;
+import static org.apache.flink.table.expressions.ApiExpressionUtils.call;
+import static org.apache.flink.table.expressions.ApiExpressionUtils.lookupCall;
+import static 
org.apache.flink.table.expressions.ApiExpressionUtils.unresolvedRef;
+import static 
org.apache.flink.table.expressions.ApiExpressionUtils.valueLiteral;
+import static org.apache.flink.table.expressions.BuiltInFunctionDefinitions.AS;
+import static 
org.apache.flink.table.expressions.BuiltInFunctionDefinitions.COUNT;
+import static 
org.apache.flink.table.expressions.BuiltInFunctionDefinitions.FLATTEN;
+import static 
org.apache.flink.table.expressions.BuiltInFunctionDefinitions.OVER;
+import static org.mockito.Mockito.mock;
+
+/**
+ * Tests for {@link VerifyNoUnresolvedExpressionsRule}.
+ */
+public class VerifyNoUnresolvedExpressionsRuleTest {
+
+       private static final ResolverRule resolverRule = 
ResolverRules.VERIFY_NO_MORE_UNRESOLVED_EXPRESSIONS;
+
+       private static final ResolutionContext resolutionContext = 
mock(ResolutionContext.class);
+
+       @Test(expected = TableException.class)
+       public void testUnresolvedReferenceIsCatched() {
+               List<Expression> expressions = asList(
+                       unresolvedRef("field"),
+                       new FieldReferenceExpression("resolvedField", 
Types.INT, 0, 0));
+               resolverRule.apply(expressions, resolutionContext);
+       }
+
+       @Test(expected = TableException.class)
+       public void testNestedUnresolvedReferenceIsCatched() {
+               List<Expression> expressions = asList(
+                       call(AS, unresolvedRef("field"), 
valueLiteral("fieldAlias")),
+                       new FieldReferenceExpression("resolvedField", 
Types.INT, 0, 0));
+               resolverRule.apply(expressions, resolutionContext);
+       }
+
+       @Test(expected = TableException.class)
+       public void testFlattenCallIsCatched() {
+               List<Expression> expressions = singletonList(
+                       call(FLATTEN, new 
FieldReferenceExpression("resolvedField", Types.INT, 0, 0))
+               );
+               resolverRule.apply(expressions, resolutionContext);
+       }
+
+       @Test(expected = TableException.class)
+       public void testUnresolvedOverWindowIsCatched() {
+               List<Expression> expressions = singletonList(
+                       call(OVER,
+                               call(COUNT, new 
FieldReferenceExpression("resolvedField", Types.INT, 0, 0)),
+                               new UnresolvedReferenceExpression("w")
+                       )
+               );
+               resolverRule.apply(expressions, resolutionContext);
+       }
+
+       @Test(expected = TableException.class)
+       public void testLookupCallIsCatched() {
+               List<Expression> expressions = singletonList(
+                       lookupCall("unresolvedCall", new 
FieldReferenceExpression("resolvedField", Types.INT, 0, 0))
+               );
+               resolverRule.apply(expressions, resolutionContext);
+       }
+}

Reply via email to