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/17 03:14:31 UTC

[GitHub] [iceberg] yyanyy commented on a change in pull request #1893: Flink : add filter push down for IcebergTableSource

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



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
##########
@@ -0,0 +1,169 @@
+/*
+ * 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.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.collect.ImmutableMap;
+
+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 Expression convert(org.apache.flink.table.expressions.Expression flinkExpression) {
+    if (!(flinkExpression instanceof CallExpression)) {
+      return null;
+    }
+
+    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 Expressions.isNull(isNullFilter.getName());
+
+        case NOT_NULL:
+          FieldReferenceExpression notNullExpression = (FieldReferenceExpression) call.getResolvedChildren().get(0);
+          return 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> expressions = values.stream().filter(

Review comment:
       Nit: I think these are input values, not expressions

##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
##########
@@ -0,0 +1,169 @@
+/*
+ * 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.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.collect.ImmutableMap;
+
+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 Expression convert(org.apache.flink.table.expressions.Expression flinkExpression) {
+    if (!(flinkExpression instanceof CallExpression)) {
+      return null;
+    }
+
+    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 Expressions.isNull(isNullFilter.getName());
+
+        case NOT_NULL:
+          FieldReferenceExpression notNullExpression = (FieldReferenceExpression) call.getResolvedChildren().get(0);
+          return 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> expressions = 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 Expressions.in(field.getName(), expressions);
+
+        case NOT:
+          Expression child = convert(call.getResolvedChildren().get(0));
+          if (child != null) {
+            return Expressions.not(child);
+          }
+
+          return null;
+
+        case AND: {
+          return convertLogicExpression(Expressions::and, call);
+        }
+
+        case OR: {
+          return convertLogicExpression(Expressions::or, call);
+        }
+      }
+    }
+
+    return null;
+  }
+
+  private static Expression convertLogicExpression(BiFunction<Expression, Expression, Expression> function,
+                                                   CallExpression call) {
+    List<ResolvedExpression> args = call.getResolvedChildren();
+    Expression left = convert(args.get(0));
+    Expression right = convert(args.get(1));
+    if (left != null && right != null) {
+      return function.apply(left, right);
+    }
+
+    return null;
+
+  }
+
+  private static 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 {
+      fieldReferenceExpression = (FieldReferenceExpression) args.get(1);
+      valueLiteralExpression = (ValueLiteralExpression) args.get(0);
+    }
+
+    String name = fieldReferenceExpression.getName();
+    Class clazz = valueLiteralExpression.getOutputDataType().getConversionClass();
+    return function.apply(name, valueLiteralExpression.getValueAs(clazz).get());
+  }
+
+  private static boolean literalOnRight(List<ResolvedExpression> args) {
+    return args.get(0) instanceof FieldReferenceExpression && args.get(1) instanceof ValueLiteralExpression ? true :

Review comment:
       Nit: no need to have `? true : false`

##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
##########
@@ -0,0 +1,169 @@
+/*
+ * 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.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.collect.ImmutableMap;
+
+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 Expression convert(org.apache.flink.table.expressions.Expression flinkExpression) {
+    if (!(flinkExpression instanceof CallExpression)) {
+      return null;

Review comment:
       I'm not familiar with flink, I wonder if it should be a valid use case here and in other places that we return null; should we throw instead? 

##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergTableSource.java
##########
@@ -112,6 +119,11 @@ public String explainSource() {
       explain += String.format(", LimitPushDown : %d", limit);
     }
 
+    if (filters != null) {
+      explain += String.format(", FilterPushDown,the filters :%s",
+          filters.stream().map(filter -> filter.toString()).collect(Collectors.joining(",")));

Review comment:
       I think returning `null` in the filters class will cause NPE here as well

##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java
##########
@@ -0,0 +1,169 @@
+/*
+ * 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.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.collect.ImmutableMap;
+
+public class FlinkFilters {
+  private FlinkFilters() {
+  }
+
+  private static final Map<BuiltInFunctionDefinition, Operation> FILTERS = ImmutableMap

Review comment:
       Seems that this is mapping flink operations to iceberg operations in the following switch statement? If that's the case we probably don't really need it, and could directly switch based on the input `flinkExpression. getFunctionDefinition()`?
   Also, there's a recent change that requires rewriting NaN in equals/notEquals to isNaN/notNaN as Iceberg's equals [no longer accepts NaN as literal](https://github.com/apache/iceberg/blob/master/api/src/main/java/org/apache/iceberg/expressions/Literals.java#L61), so we will have to rewrite here as well. [Here](https://github.com/apache/iceberg/blob/master/spark3/src/main/java/org/apache/iceberg/spark/SparkFilters.java#L205-L210) is a similar change done in spark filters. 

##########
File path: flink/src/main/java/org/apache/iceberg/flink/IcebergTableSource.java
##########
@@ -122,6 +134,21 @@ public boolean isLimitPushedDown() {
 
   @Override
   public TableSource<RowData> applyLimit(long newLimit) {
-    return new IcebergTableSource(loader, schema, properties, projectedFields, true, newLimit);
+    return new IcebergTableSource(loader, schema, properties, projectedFields, true, newLimit, filters);
+  }
+
+  public TableSource<RowData> applyPredicate(List<Expression> predicates) {

Review comment:
       Nit: `@Override`?




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