You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2020/12/23 13:37:47 UTC

[GitHub] [iceberg] openinx commented on a change in pull request #1893: Flink: Support filter pushdown in IcebergTableSource

openinx commented on a change in pull request #1893:
URL: https://github.com/apache/iceberg/pull/1893#discussion_r547962504



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.iceberg.flink;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+import org.apache.flink.table.expressions.CallExpression;
+import org.apache.flink.table.expressions.FieldReferenceExpression;
+import org.apache.flink.table.expressions.ResolvedExpression;
+import org.apache.flink.table.expressions.ValueLiteralExpression;
+import org.apache.flink.table.functions.BuiltInFunctionDefinition;
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expression.Operation;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.util.NaNUtil;
+
+public class FlinkFilters {
+  private FlinkFilters() {
+  }
+
+  private static final Map<BuiltInFunctionDefinition, Operation> FILTERS = ImmutableMap
+      .<BuiltInFunctionDefinition, Operation>builder()
+      .put(BuiltInFunctionDefinitions.EQUALS, Operation.EQ)
+      .put(BuiltInFunctionDefinitions.NOT_EQUALS, Operation.NOT_EQ)
+      .put(BuiltInFunctionDefinitions.GREATER_THAN, Operation.GT)
+      .put(BuiltInFunctionDefinitions.GREATER_THAN_OR_EQUAL, Operation.GT_EQ)
+      .put(BuiltInFunctionDefinitions.LESS_THAN, Operation.LT)
+      .put(BuiltInFunctionDefinitions.LESS_THAN_OR_EQUAL, Operation.LT_EQ)
+      .put(BuiltInFunctionDefinitions.IN, Operation.IN)
+      .put(BuiltInFunctionDefinitions.IS_NULL, Operation.IS_NULL)
+      .put(BuiltInFunctionDefinitions.IS_NOT_NULL, Operation.NOT_NULL)
+      .put(BuiltInFunctionDefinitions.AND, Operation.AND)
+      .put(BuiltInFunctionDefinitions.OR, Operation.OR)
+      .put(BuiltInFunctionDefinitions.NOT, Operation.NOT)
+      .build();
+
+
+  public static Optional<Expression> convert(org.apache.flink.table.expressions.Expression flinkExpression) {
+    if (!(flinkExpression instanceof CallExpression)) {
+      return Optional.empty();
+    }
+
+    CallExpression call = (CallExpression) flinkExpression;
+    Operation op = FILTERS.get(call.getFunctionDefinition());
+    if (op != null) {
+      switch (op) {
+        case IS_NULL:
+          FieldReferenceExpression isNullFilter = (FieldReferenceExpression) call.getResolvedChildren().get(0);
+          return Optional.of(Expressions.isNull(isNullFilter.getName()));
+
+        case NOT_NULL:
+          FieldReferenceExpression notNullExpression = (FieldReferenceExpression) call.getResolvedChildren().get(0);
+          return Optional.of(Expressions.notNull(notNullExpression.getName()));
+
+        case LT:
+          return convertComparisonExpression(Expressions::lessThan, call);
+
+        case LT_EQ:
+          return convertComparisonExpression(Expressions::lessThanOrEqual, call);
+
+        case GT:
+          return convertComparisonExpression(Expressions::greaterThan, call);
+
+        case GT_EQ:
+          return convertComparisonExpression(Expressions::greaterThanOrEqual, call);
+
+        case EQ:
+          return convertComparisonExpression(Expressions::equal, call);
+
+        case NOT_EQ:
+          return convertComparisonExpression(Expressions::notEqual, call);
+
+        case IN:
+          List<ResolvedExpression> args = call.getResolvedChildren();
+          FieldReferenceExpression field = (FieldReferenceExpression) args.get(0);
+          List<ResolvedExpression> values = args.subList(1, args.size());
+
+          List<Object> inputValues = values.stream().filter(
+              expression -> {
+                if (expression instanceof ValueLiteralExpression) {
+                  return !((ValueLiteralExpression) flinkExpression).isNull();
+                }
+
+                return false;
+              }
+          ).map(expression -> {
+            ValueLiteralExpression valueLiteralExpression = (ValueLiteralExpression) expression;
+            return valueLiteralExpression.getValueAs(valueLiteralExpression.getOutputDataType().getConversionClass())
+                .get();
+          }).collect(Collectors.toList());
+          return Optional.of(Expressions.in(field.getName(), inputValues));
+
+        case NOT:
+          Optional<Expression> child = convert(call.getResolvedChildren().get(0));
+          if (child.isPresent()) {
+            return Optional.of(Expressions.not(child.get()));
+          }
+
+          return Optional.empty();
+
+        case AND: {
+          return convertLogicExpression(Expressions::and, call);
+        }
+
+        case OR: {
+          return convertLogicExpression(Expressions::or, call);
+        }
+      }
+    }
+
+    return Optional.empty();
+  }
+
+  private static Optional<Expression> convertLogicExpression(BiFunction<Expression, Expression, Expression> function,
+                                                             CallExpression call) {
+    List<ResolvedExpression> args = call.getResolvedChildren();
+    Optional<Expression> left = convert(args.get(0));
+    Optional<Expression> right = convert(args.get(1));
+    if (left.isPresent() && right.isPresent()) {
+      return Optional.of(function.apply(left.get(), right.get()));
+    }
+
+    return Optional.empty();
+  }
+
+  private static Optional<Expression> convertComparisonExpression(BiFunction<String, Object, Expression> function,
+                                                                  CallExpression call) {
+    List<ResolvedExpression> args = call.getResolvedChildren();
+    FieldReferenceExpression fieldReferenceExpression;
+    ValueLiteralExpression valueLiteralExpression;
+    if (literalOnRight(args)) {
+      fieldReferenceExpression = (FieldReferenceExpression) args.get(0);
+      valueLiteralExpression = (ValueLiteralExpression) args.get(1);
+    } else {

Review comment:
       Q:  is it possible that we have two literals in both left and right side ? For example,  someone may write the SQL: 
   `SELECT * FROM test where 1 > 0`,  then we could not think that the left MUST be a field and the right MUST be a `value` in this `else` block.   
   
   We have done the similar work in our own branch before,  you could see the PR: https://github.com/generic-datalake/iceberg-poc/pull/2/files#diff-86160616589acf1dd526b10b73418a46fe60f9e5e5ab6946a4ea3c8f019542f5R123




----------------------------------------------------------------
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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org